diff --git a/CHANGELOG.asciidoc b/CHANGELOG.asciidoc index de0fb83d4179..5e41b85a806c 100644 --- a/CHANGELOG.asciidoc +++ b/CHANGELOG.asciidoc @@ -61,6 +61,7 @@ https://github.com/elastic/beats/compare/v1.1.0...master[Check the HEAD diff] - Add ability to create Elasticsearch mapping on startup {pull}639[639] - Add option to elasticsearch output to pass http parameters in index operations {issue}805[805] - Improve logstash and elasticsearch backoff behavior. {pull}927[927] +- Add experimental Kafka output. {pull}942[942] *Packetbeat* - Change the DNS library used throughout the dns package to github.com/miekg/dns. {pull}803[803] diff --git a/glide.yaml b/glide.yaml index 7da1ee058209..9ec96b14af7c 100644 --- a/glide.yaml +++ b/glide.yaml @@ -38,3 +38,15 @@ import: version: v1.2.0 - package: github.com/miekg/dns version: 85b661b2a6fc95a5a83e66d7730c4bc0b6e9c99e +- package: github.com/Shopify/sarama + version: v1.8.0 +- package: github.com/klauspost/crc32 + version: 6973dcf6594efa905c08260fe9120cae92ab4305 +- package: github.com/golang/snappy + version: 894fd4616c897c201d223c3c0c128e8c648c96a2 +- package: github.com/eapache/queue + version: ded5959c0d4e360646dc9e9908cff48666781367 +- package: github.com/eapache/go-resiliency + version: b86b1ec0dd4209a588dc1285cdd471e73525c0b3 + subpackages: + - breaker diff --git a/libbeat/docs/outputconfig.asciidoc b/libbeat/docs/outputconfig.asciidoc index 545462c2fa1f..cb7e420f10ae 100644 --- a/libbeat/docs/outputconfig.asciidoc +++ b/libbeat/docs/outputconfig.asciidoc @@ -18,6 +18,7 @@ You can configure multiple outputs for exporting the correlated transactions. Cu * <> * <> * <> +* <> * <> * <> @@ -523,6 +524,80 @@ setting does not affect how events are published. Setting `bulk_max_size` to values less than or equal to 0 disables buffering in libbeat. +[[kafka-output]] +==== Kafka Output + +===== hosts + +List of Kafka broker addressed to connect to. + +===== topic + +The kafka topic used for produced events. If use_type is set to true, topic will not be used. + +===== use_type + +Set kafka topic by event type. If use_type is false, topic must be configured. The deault is false. + +===== client_id + +Configurable ClientID used for logging, debugging and auditing purposes. The default is "beats". + +===== worker + +Number of concurrent load-balanced kafka output workers. + +===== max_retries + +The number of times to retry publishing an event after a publishing failure. +After the specified number of retries, the events are typically dropped. +Some Beats, such as Filebeat, ignore the `max_retries` setting and retry until all +events are published. + +Set `max_retries` to a value less than 0 to retry until all events are published. + +The default is 3. + +===== bulk_max_size + +The maximum number of events to bulk in a single Logstash request. The default is 2048. + +===== timeout + +The number of seconds to wait for responses from the Kafka brokers before timing +out. The default is 30 (seconds). + +===== broker_timeout + +Maximum duration a broker will wait for number of required ACKs. The default is 10s. + +===== keep_alive + +Keep-alive period for an active network connection. If 0s, keep-alives are disabled. The default is 0 seconds. + +===== compression + +Select output compression codec. Must be one of `none`, `snappy` and `gzip`. The default is `snappy`. + +===== max_message_bytes + +Max permitted size of json-encoded messages. Bigger messages will be dropped. The default value is 1000000 (bytes). Should be equal or less to the brokers `message.max.bytes`. + +===== required_acks + +ACK reliability level required from broker. 0=no response, 1=wait for local commit, -1=wait for all replicas to commit. The default is 1. + +Note: If set to 0, no ACKs are returned by kafka. Messages might be lost silently on error. + +===== flush_interval + +The number of seconds to wait for new events between two producer API calls. + +===== tls + +Configuration options for TLS parameters like the root CA for Kibana connections. See +<> for more information. + [[file-output]] ==== File Output diff --git a/libbeat/outputs/kafka/client.go b/libbeat/outputs/kafka/client.go new file mode 100644 index 000000000000..123a7491a6bb --- /dev/null +++ b/libbeat/outputs/kafka/client.go @@ -0,0 +1,188 @@ +package kafka + +import ( + "encoding/json" + "expvar" + "sync" + "sync/atomic" + "time" + + "github.com/Shopify/sarama" + + "github.com/elastic/beats/libbeat/common" + "github.com/elastic/beats/libbeat/logp" +) + +type client struct { + hosts []string + topic string + useType bool + config sarama.Config + + producer sarama.AsyncProducer + + wg sync.WaitGroup + + isConnected int32 +} + +type msgRef struct { + count int32 + err atomic.Value + batch []common.MapStr + cb func([]common.MapStr, error) +} + +var ( + ackedEvents = expvar.NewInt("libbeatKafkaPublishedAndAckedEvents") + eventsNotAcked = expvar.NewInt("libbeatKafkaPublishedButNotAckedEvents") + publishEventsCallCount = expvar.NewInt("libbeatKafkaPublishEventsCallCount") +) + +func newKafkaClient(hosts []string, topic string, useType bool, cfg *sarama.Config) (*client, error) { + c := &client{ + hosts: hosts, + useType: useType, + topic: topic, + config: *cfg, + } + return c, nil +} + +func (c *client) Connect(timeout time.Duration) error { + debugf("connect: %v", c.hosts) + + c.config.Net.DialTimeout = timeout + + // try to connect + producer, err := sarama.NewAsyncProducer(c.hosts, &c.config) + if err != nil { + logp.Err("Kafka connect fails with: %v", err) + return err + } + + c.producer = producer + + c.wg.Add(2) + go c.successWorker(producer.Successes()) + go c.errorWorker(producer.Errors()) + atomic.StoreInt32(&c.isConnected, 1) + + return nil +} + +func (c *client) Close() error { + if c.IsConnected() { + debugf("closed kafka client") + + c.producer.AsyncClose() + c.wg.Wait() + atomic.StoreInt32(&c.isConnected, 0) + c.producer = nil + } + return nil +} + +func (c *client) IsConnected() bool { + return atomic.LoadInt32(&c.isConnected) != 0 +} + +func (c *client) AsyncPublishEvent( + cb func(error), + event common.MapStr, +) error { + return c.AsyncPublishEvents(func(_ []common.MapStr, err error) { + cb(err) + }, []common.MapStr{event}) +} + +func (c *client) AsyncPublishEvents( + cb func([]common.MapStr, error), + events []common.MapStr, +) error { + publishEventsCallCount.Add(1) + debugf("publish events") + + ref := &msgRef{ + count: int32(len(events)), + batch: events, + cb: cb, + } + + ch := c.producer.Input() + + for _, event := range events { + topic := c.topic + if c.useType { + topic = event["type"].(string) + } + + jsonEvent, err := json.Marshal(event) + if err != nil { + ref.done() + continue + } + + msg := &sarama.ProducerMessage{ + Metadata: ref, + Topic: topic, + Value: sarama.ByteEncoder(jsonEvent), + } + + ch <- msg + } + + return nil +} + +func (c *client) successWorker(ch <-chan *sarama.ProducerMessage) { + defer c.wg.Done() + defer debugf("Stop kafka ack worker") + + for msg := range ch { + ref := msg.Metadata.(*msgRef) + ref.done() + } +} + +func (c *client) errorWorker(ch <-chan *sarama.ProducerError) { + defer c.wg.Done() + defer debugf("Stop kafka error handler") + + for errMsg := range ch { + msg := errMsg.Msg + ref := msg.Metadata.(*msgRef) + ref.fail(errMsg.Err) + } +} + +func (r *msgRef) done() { + r.dec() +} + +func (r *msgRef) fail(err error) { + debugf("Kafka publish failed with: %v", err) + + r.err.Store(err) + r.dec() +} + +func (r *msgRef) dec() { + i := atomic.AddInt32(&r.count, -1) + if i > 0 { + return + } + + debugf("finished kafka batch") + + var err error + v := r.err.Load() + if v != nil { + err = v.(error) + eventsNotAcked.Add(int64(len(r.batch))) + r.cb(r.batch, err) + } else { + ackedEvents.Add(int64(len(r.batch))) + r.cb(nil, nil) + } +} diff --git a/libbeat/outputs/kafka/kafka.go b/libbeat/outputs/kafka/kafka.go new file mode 100644 index 000000000000..1f11effd52fb --- /dev/null +++ b/libbeat/outputs/kafka/kafka.go @@ -0,0 +1,221 @@ +package kafka + +import ( + "errors" + "fmt" + "strings" + "time" + + "github.com/Shopify/sarama" + + "github.com/elastic/beats/libbeat/common" + "github.com/elastic/beats/libbeat/logp" + "github.com/elastic/beats/libbeat/outputs" + "github.com/elastic/beats/libbeat/outputs/mode" +) + +var debugf = logp.MakeDebug("kafka") + +func init() { + outputs.RegisterOutputPlugin("kafka", kafkaOutputPlugin{}) +} + +type kafkaOutputPlugin struct{} + +type kafka struct { + mode mode.ConnectionMode +} + +var ( + kafkaDefaultTimeout = 30 * time.Second + kafkaDefaultKeepAlive = 0 * time.Second + kafkaDefaultCompression = sarama.CompressionSnappy + kafkaDefaultClientID = "beats" + kafkaDefaultTopicsFromType = false +) + +var ( + compressionModes = map[string]sarama.CompressionCodec{ + "none": sarama.CompressionNone, + "no": sarama.CompressionNone, + "off": sarama.CompressionNone, + "gzip": sarama.CompressionGZIP, + "snappy": sarama.CompressionSnappy, + } +) + +var ( + errNoTopicSet = errors.New("No topic configured") + errNoHosts = errors.New("No hosts configured") +) + +func (p kafkaOutputPlugin) NewOutput( + config *outputs.MothershipConfig, + topologyExpire int, +) (outputs.Outputer, error) { + output := &kafka{} + err := output.init(config) + if err != nil { + return nil, err + } + return output, nil +} + +func (k *kafka) init(config *outputs.MothershipConfig) error { + debugf("initialize kafka output") + + cfg, retries, err := newKafkaConfig(config) + if err != nil { + return err + } + + hosts := config.Hosts + if len(hosts) < 1 { + logp.Err("Kafka configuration failed with: %v", errNoHosts) + return errNoHosts + } + debugf("hosts: %v", hosts) + + useType := kafkaDefaultTopicsFromType + if config.UseType != nil { + useType = *config.UseType + } + + topic := config.Topic + if topic == "" && !useType { + logp.Err("Kafka configuration failed with: %v", errNoTopicSet) + return errNoTopicSet + } + + var clients []mode.AsyncProtocolClient + worker := 1 + if config.Worker > 1 { + worker = config.Worker + } + for i := 0; i < worker; i++ { + client, err := newKafkaClient(hosts, topic, useType, cfg) + if err != nil { + logp.Err("Failed to create kafka client: %v", err) + return err + } + + clients = append(clients, client) + } + + mode, err := mode.NewAsyncConnectionMode( + clients, + false, + retries, // retry implemented by kafka client + cfg.Producer.Retry.Backoff, + cfg.Net.WriteTimeout, + 10*time.Second) + if err != nil { + logp.Err("Failed to configure kafka connection: %v", err) + return err + } + + k.mode = mode + return nil +} + +func (k *kafka) PublishEvent( + signal outputs.Signaler, + opts outputs.Options, + event common.MapStr, +) error { + return k.mode.PublishEvent(signal, opts, event) +} + +func (k *kafka) BulkPublish( + signal outputs.Signaler, + opts outputs.Options, + event []common.MapStr, +) error { + return k.mode.PublishEvents(signal, opts, event) +} + +func newKafkaConfig(config *outputs.MothershipConfig) (*sarama.Config, int, error) { + k := sarama.NewConfig() + modeRetries := 1 + + // configure network level properties + timeout := kafkaDefaultTimeout + if config.Timeout > 0 { + timeout = time.Duration(config.Timeout) * time.Second + } + + k.Net.DialTimeout = timeout + k.Net.ReadTimeout = timeout + k.Net.WriteTimeout = timeout + + if config.TLS != nil { + tls, err := outputs.LoadTLSConfig(config.TLS) + if err != nil { + return nil, modeRetries, err + } + k.Net.TLS.Enable = true + k.Net.TLS.Config = tls + } + + keepAlive := kafkaDefaultKeepAlive + if config.KeepAlive != "" { + var err error + keepAlive, err = time.ParseDuration(config.KeepAlive) + if err != nil { + return nil, modeRetries, err + } + } + k.Net.KeepAlive = keepAlive + + // TODO: configure metadata level properties + // use lib defaults + + // configure producer API properties + if config.MaxMessageBytes != nil { + k.Producer.MaxMessageBytes = *config.MaxMessageBytes + } + if config.RequiredACKs != nil { + k.Producer.RequiredAcks = sarama.RequiredAcks(*config.RequiredACKs) + } + if config.BrokerTimeout != "" { + var err error + k.Producer.Timeout, err = time.ParseDuration(config.BrokerTimeout) + if err != nil { + return nil, modeRetries, err + } + } + compressionMode := kafkaDefaultCompression + if config.Compression != "" { + mode, ok := compressionModes[strings.ToLower(config.Compression)] + if !ok { + return nil, modeRetries, fmt.Errorf("Unknown compression mode: %v", config.Compression) + } + compressionMode = mode + } + k.Producer.Compression = compressionMode + + k.Producer.Return.Successes = true // enable return channel for signaling + k.Producer.Return.Errors = true + + if config.MaxRetries != nil { + retries := *config.MaxRetries + if retries < 0 { + retries = 10 + modeRetries = -1 + } + k.Producer.Retry.Max = retries + } + + // configure client ID + clientID := kafkaDefaultClientID + if config.ClientID != "" { + clientID = config.ClientID + } + k.ClientID = clientID + + if err := k.Validate(); err != nil { + logp.Err("Invalid kafka configuration: %v", err) + return nil, modeRetries, err + } + return k, modeRetries, nil +} diff --git a/libbeat/outputs/mode/balance_async.go b/libbeat/outputs/mode/balance_async.go index 83aecd437076..a19945245965 100644 --- a/libbeat/outputs/mode/balance_async.go +++ b/libbeat/outputs/mode/balance_async.go @@ -64,6 +64,8 @@ func NewAsyncLoadBalancerMode( waitRetry, timeout, maxWaitRetry time.Duration, ) (*AsyncLoadBalancerMode, error) { + debug("configure maxattempts: %v", maxAttempts) + // maxAttempts signals infinite retry. Convert to -1, so attempts left and // and infinite retry can be more easily distinguished by load balancer if maxAttempts == 0 { @@ -119,9 +121,13 @@ func (m *AsyncLoadBalancerMode) publishEventsMessage( ) error { maxAttempts := m.maxAttempts if opts.Guaranteed { + debug("guaranteed flag is set") maxAttempts = -1 + } else { + debug("guaranteed flag is not set") } msg.attemptsLeft = maxAttempts + debug("publish events with attempts=%v", msg.attemptsLeft) if ok := m.forwardEvent(m.work, msg); !ok { dropping(msg) @@ -224,7 +230,11 @@ func handlePublishEventsResult( ) func([]common.MapStr, error) { total := len(msg.events) return func(events []common.MapStr, err error) { + debug("handlePublishEventsResult") + if err != nil { + debug("handle publish error: %v", err) + if msg.attemptsLeft > 0 { msg.attemptsLeft-- } @@ -255,6 +265,7 @@ func handlePublishEventsResult( // re-insert non-published events into pipeline if len(events) != 0 { + debug("add non-published events back into pipeline: %v", len(events)) msg.events = events if ok := m.forwardEvent(m.retries, msg); !ok { dropping(msg) @@ -263,6 +274,7 @@ func handlePublishEventsResult( } // all events published -> signal success + debug("async bulk publish success") outputs.SignalCompleted(msg.signaler) } } @@ -287,21 +299,28 @@ func (m *AsyncLoadBalancerMode) forwardEvent( ch chan eventsMessage, msg eventsMessage, ) bool { + debug("forwards msg with attempts=%v", msg.attemptsLeft) + if msg.attemptsLeft < 0 { select { case ch <- msg: + debug("message forwarded") return true case <-m.done: // shutdown + debug("shutting down") return false } } else { for ; msg.attemptsLeft > 0; msg.attemptsLeft-- { select { case ch <- msg: + debug("message forwarded") return true case <-m.done: // shutdown + debug("shutting down") return false case <-time.After(m.timeout): + debug("forward timed out") } } } diff --git a/libbeat/outputs/outputs.go b/libbeat/outputs/outputs.go index ef3a0ea80671..2cec66580685 100644 --- a/libbeat/outputs/outputs.go +++ b/libbeat/outputs/outputs.go @@ -33,7 +33,15 @@ type MothershipConfig struct { Pretty *bool `yaml:"pretty"` TLS *TLSConfig Worker int - CompressionLevel *int `yaml:"compression_level"` + CompressionLevel *int `yaml:"compression_level"` + KeepAlive string `yaml:"keep_alive"` + MaxMessageBytes *int `yaml:"max_message_bytes"` + RequiredACKs *int `yaml:"required_acks"` + BrokerTimeout string `yaml:"broker_timeout"` + Compression string `yaml:"compression"` + ClientID string `yaml:"client_id"` + Topic string `yaml:"topic"` + UseType *bool `yaml:"use_type"` } type Template struct { diff --git a/libbeat/publisher/publish.go b/libbeat/publisher/publish.go index 3d7a6d475dc2..e3e3692b51c1 100644 --- a/libbeat/publisher/publish.go +++ b/libbeat/publisher/publish.go @@ -16,6 +16,7 @@ import ( _ "github.com/elastic/beats/libbeat/outputs/console" _ "github.com/elastic/beats/libbeat/outputs/elasticsearch" _ "github.com/elastic/beats/libbeat/outputs/fileout" + _ "github.com/elastic/beats/libbeat/outputs/kafka" _ "github.com/elastic/beats/libbeat/outputs/logstash" _ "github.com/elastic/beats/libbeat/outputs/redis" ) diff --git a/vendor/github.com/Shopify/sarama/.gitignore b/vendor/github.com/Shopify/sarama/.gitignore new file mode 100644 index 000000000000..3591f9ff3053 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/.gitignore @@ -0,0 +1,24 @@ +# Compiled Object files, Static and Dynamic libs (Shared Objects) +*.o +*.a +*.so +*.test + +# Folders +_obj +_test +.vagrant + +# Architecture specific extensions/prefixes +*.[568vq] +[568vq].out + +*.cgo1.go +*.cgo2.c +_cgo_defun.c +_cgo_gotypes.go +_cgo_export.* + +_testmain.go + +*.exe diff --git a/vendor/github.com/Shopify/sarama/.travis.yml b/vendor/github.com/Shopify/sarama/.travis.yml new file mode 100644 index 000000000000..10e8dbd1a922 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/.travis.yml @@ -0,0 +1,32 @@ +language: go +go: +- 1.4.3 +- 1.5.3 + +env: + global: + - KAFKA_PEERS=localhost:9091,localhost:9092,localhost:9093,localhost:9094,localhost:9095 + - TOXIPROXY_ADDR=http://localhost:8474 + - KAFKA_INSTALL_ROOT=/home/travis/kafka + - KAFKA_HOSTNAME=localhost + - DEBUG=true + matrix: + - KAFKA_VERSION=0.8.2.2 + - KAFKA_VERSION=0.9.0.0 + +before_install: +- export REPOSITORY_ROOT=${TRAVIS_BUILD_DIR} +- vagrant/install_cluster.sh +- vagrant/boot_cluster.sh +- vagrant/create_topics.sh + +install: +- make install_dependencies + +script: +- make test +- make vet +- make errcheck +- make fmt + +sudo: false diff --git a/vendor/github.com/Shopify/sarama/CHANGELOG.md b/vendor/github.com/Shopify/sarama/CHANGELOG.md new file mode 100644 index 000000000000..ec6577196e6b --- /dev/null +++ b/vendor/github.com/Shopify/sarama/CHANGELOG.md @@ -0,0 +1,235 @@ +# Changelog + +#### Version 1.8.0 (2016-02-01) + +New Features: + - Full support for Kafka 0.9: + - All protocol messages and fields + ([#586](https://github.com/Shopify/sarama/pull/586), + [#588](https://github.com/Shopify/sarama/pull/588), + [#590](https://github.com/Shopify/sarama/pull/590)). + - Verified that TLS support works + ([#581](https://github.com/Shopify/sarama/pull/581)). + - Fixed the OffsetManager compatibility + ([#585](https://github.com/Shopify/sarama/pull/585)). + +Improvements: + - Optimize for fewer system calls when reading from the network + ([#584](https://github.com/Shopify/sarama/pull/584)). + - Automatically retry `InvalidMessage` errors to match upstream behaviour + ([#589](https://github.com/Shopify/sarama/pull/589)). + +#### Version 1.7.0 (2015-12-11) + +New Features: + - Preliminary support for Kafka 0.9 + ([#572](https://github.com/Shopify/sarama/pull/572)). This comes with several + caveats: + - Protocol-layer support is mostly in place + ([#577](https://github.com/Shopify/sarama/pull/577)), however Kafka 0.9 + renamed some messages and fields, which we did not in order to preserve API + compatibility. + - The producer and consumer work against 0.9, but the offset manager does + not ([#573](https://github.com/Shopify/sarama/pull/573)). + - TLS support may or may not work + ([#581](https://github.com/Shopify/sarama/pull/581)). + +Improvements: + - Don't wait for request timeouts on dead brokers, greatly speeding recovery + when the TCP connection is left hanging + ([#548](https://github.com/Shopify/sarama/pull/548)). + - Refactored part of the producer. The new version provides a much more elegant + solution to [#449](https://github.com/Shopify/sarama/pull/449). It is also + slightly more efficient, and much more precise in calculating batch sizes + when compression is used + ([#549](https://github.com/Shopify/sarama/pull/549), + [#550](https://github.com/Shopify/sarama/pull/550), + [#551](https://github.com/Shopify/sarama/pull/551)). + +Bug Fixes: + - Fix race condition in consumer test mock + ([#553](https://github.com/Shopify/sarama/pull/553)). + +#### Version 1.6.1 (2015-09-25) + +Bug Fixes: + - Fix panic that could occur if a user-supplied message value failed to encode + ([#449](https://github.com/Shopify/sarama/pull/449)). + +#### Version 1.6.0 (2015-09-04) + +New Features: + - Implementation of a consumer offset manager using the APIs introduced in + Kafka 0.8.2. The API is designed mainly for integration into a future + high-level consumer, not for direct use, although it is *possible* to use it + directly. + ([#461](https://github.com/Shopify/sarama/pull/461)). + +Improvements: + - CRC32 calculation is much faster on machines with SSE4.2 instructions, + removing a major hotspot from most profiles + ([#255](https://github.com/Shopify/sarama/pull/255)). + +Bug Fixes: + - Make protocol decoding more robust against some malformed packets generated + by go-fuzz ([#523](https://github.com/Shopify/sarama/pull/523), + [#525](https://github.com/Shopify/sarama/pull/525)) or found in other ways + ([#528](https://github.com/Shopify/sarama/pull/528)). + - Fix a potential race condition panic in the consumer on shutdown + ([#529](https://github.com/Shopify/sarama/pull/529)). + +#### Version 1.5.0 (2015-08-17) + +New Features: + - TLS-encrypted network connections are now supported. This feature is subject + to change when Kafka releases built-in TLS support, but for now this is + enough to work with TLS-terminating proxies + ([#154](https://github.com/Shopify/sarama/pull/154)). + +Improvements: + - The consumer will not block if a single partition is not drained by the user; + all other partitions will continue to consume normally + ([#485](https://github.com/Shopify/sarama/pull/485)). + - Formatting of error strings has been much improved + ([#495](https://github.com/Shopify/sarama/pull/495)). + - Internal refactoring of the producer for code cleanliness and to enable + future work ([#300](https://github.com/Shopify/sarama/pull/300)). + +Bug Fixes: + - Fix a potential deadlock in the consumer on shutdown + ([#475](https://github.com/Shopify/sarama/pull/475)). + +#### Version 1.4.3 (2015-07-21) + +Bug Fixes: + - Don't include the partitioner in the producer's "fetch partitions" + circuit-breaker ([#466](https://github.com/Shopify/sarama/pull/466)). + - Don't retry messages until the broker is closed when abandoning a broker in + the producer ([#468](https://github.com/Shopify/sarama/pull/468)). + - Update the import path for snappy-go, it has moved again and the API has + changed slightly ([#486](https://github.com/Shopify/sarama/pull/486)). + +#### Version 1.4.2 (2015-05-27) + +Bug Fixes: + - Update the import path for snappy-go, it has moved from google code to github + ([#456](https://github.com/Shopify/sarama/pull/456)). + +#### Version 1.4.1 (2015-05-25) + +Improvements: + - Optimizations when decoding snappy messages, thanks to John Potocny + ([#446](https://github.com/Shopify/sarama/pull/446)). + +Bug Fixes: + - Fix hypothetical race conditions on producer shutdown + ([#450](https://github.com/Shopify/sarama/pull/450), + [#451](https://github.com/Shopify/sarama/pull/451)). + +#### Version 1.4.0 (2015-05-01) + +New Features: + - The consumer now implements `Topics()` and `Partitions()` methods to enable + users to dynamically choose what topics/partitions to consume without + instantiating a full client + ([#431](https://github.com/Shopify/sarama/pull/431)). + - The partition-consumer now exposes the high water mark offset value returned + by the broker via the `HighWaterMarkOffset()` method ([#339](https://github.com/Shopify/sarama/pull/339)). + - Added a `kafka-console-consumer` tool capable of handling multiple + partitions, and deprecated the now-obsolete `kafka-console-partitionConsumer` + ([#439](https://github.com/Shopify/sarama/pull/439), + [#442](https://github.com/Shopify/sarama/pull/442)). + +Improvements: + - The producer's logging during retry scenarios is more consistent, more + useful, and slightly less verbose + ([#429](https://github.com/Shopify/sarama/pull/429)). + - The client now shuffles its initial list of seed brokers in order to prevent + thundering herd on the first broker in the list + ([#441](https://github.com/Shopify/sarama/pull/441)). + +Bug Fixes: + - The producer now correctly manages its state if retries occur when it is + shutting down, fixing several instances of confusing behaviour and at least + one potential deadlock ([#419](https://github.com/Shopify/sarama/pull/419)). + - The consumer now handles messages for different partitions asynchronously, + making it much more resilient to specific user code ordering + ([#325](https://github.com/Shopify/sarama/pull/325)). + +#### Version 1.3.0 (2015-04-16) + +New Features: + - The client now tracks consumer group coordinators using + ConsumerMetadataRequests similar to how it tracks partition leadership using + regular MetadataRequests ([#411](https://github.com/Shopify/sarama/pull/411)). + This adds two methods to the client API: + - `Coordinator(consumerGroup string) (*Broker, error)` + - `RefreshCoordinator(consumerGroup string) error` + +Improvements: + - ConsumerMetadataResponses now automatically create a Broker object out of the + ID/address/port combination for the Coordinator; accessing the fields + individually has been deprecated + ([#413](https://github.com/Shopify/sarama/pull/413)). + - Much improved handling of `OffsetOutOfRange` errors in the consumer. + Consumers will fail to start if the provided offset is out of range + ([#418](https://github.com/Shopify/sarama/pull/418)) + and they will automatically shut down if the offset falls out of range + ([#424](https://github.com/Shopify/sarama/pull/424)). + - Small performance improvement in encoding and decoding protocol messages + ([#427](https://github.com/Shopify/sarama/pull/427)). + +Bug Fixes: + - Fix a rare race condition in the client's background metadata refresher if + it happens to be activated while the client is being closed + ([#422](https://github.com/Shopify/sarama/pull/422)). + +#### Version 1.2.0 (2015-04-07) + +Improvements: + - The producer's behaviour when `Flush.Frequency` is set is now more intuitive + ([#389](https://github.com/Shopify/sarama/pull/389)). + - The producer is now somewhat more memory-efficient during and after retrying + messages due to an improved queue implementation + ([#396](https://github.com/Shopify/sarama/pull/396)). + - The consumer produces much more useful logging output when leadership + changes ([#385](https://github.com/Shopify/sarama/pull/385)). + - The client's `GetOffset` method will now automatically refresh metadata and + retry once in the event of stale information or similar + ([#394](https://github.com/Shopify/sarama/pull/394)). + - Broker connections now have support for using TCP keepalives + ([#407](https://github.com/Shopify/sarama/issues/407)). + +Bug Fixes: + - The OffsetCommitRequest message now correctly implements all three possible + API versions ([#390](https://github.com/Shopify/sarama/pull/390), + [#400](https://github.com/Shopify/sarama/pull/400)). + +#### Version 1.1.0 (2015-03-20) + +Improvements: + - Wrap the producer's partitioner call in a circuit-breaker so that repeatedly + broken topics don't choke throughput + ([#373](https://github.com/Shopify/sarama/pull/373)). + +Bug Fixes: + - Fix the producer's internal reference counting in certain unusual scenarios + ([#367](https://github.com/Shopify/sarama/pull/367)). + - Fix the consumer's internal reference counting in certain unusual scenarios + ([#369](https://github.com/Shopify/sarama/pull/369)). + - Fix a condition where the producer's internal control messages could have + gotten stuck ([#368](https://github.com/Shopify/sarama/pull/368)). + - Fix an issue where invalid partition lists would be cached when asking for + metadata for a non-existant topic ([#372](https://github.com/Shopify/sarama/pull/372)). + + +#### Version 1.0.0 (2015-03-17) + +Version 1.0.0 is the first tagged version, and is almost a complete rewrite. The primary differences with previous untagged versions are: + +- The producer has been rewritten; there is now a `SyncProducer` with a blocking API, and an `AsyncProducer` that is non-blocking. +- The consumer has been rewritten to only open one connection per broker instead of one connection per partition. +- The main types of Sarama are now interfaces to make depedency injection easy; mock implementations for `Consumer`, `SyncProducer` and `AsyncProducer` are provided in the `github.com/Shopify/sarama/mocks` package. +- For most uses cases, it is no longer necessary to open a `Client`; this will be done for you. +- All the configuration values have been unified in the `Config` struct. +- Much improved test suite. diff --git a/vendor/github.com/Shopify/sarama/CONTRIBUTING.md b/vendor/github.com/Shopify/sarama/CONTRIBUTING.md new file mode 100644 index 000000000000..b0f107cbc76b --- /dev/null +++ b/vendor/github.com/Shopify/sarama/CONTRIBUTING.md @@ -0,0 +1,31 @@ +# Contributing + +Contributions are always welcome, both reporting issues and submitting pull requests! + +### Reporting issues + +Please make sure to include any potentially useful information in the issue, so we can pinpoint the issue faster without going back and forth. + +- What SHA of Sarama are you running? If this is not the latest SHA on the master branch, please try if the problem persists with the latest version. +- You can set `sarama.Logger` to a [log.Logger](http://golang.org/pkg/log/#Logger) instance to capture debug output. Please include it in your issue description. +- Also look at the logs of the Kafka broker you are connected to. If you see anything out of the ordinary, please include it. + +Also, please include the following information about your environment, so we can help you faster: + +- What version of Kafka are you using? +- What version of Go are you using? +- What are the values of your Producer/Consumer/Client configuration? + + +### Submitting pull requests + +We will gladly accept bug fixes, or additions to this library. Please fork this library, commit & push your changes, and open a pull request. Because this library is in production use by many people and applications, we code review all additions. To make the review process go as smooth as possible, please consider the following. + +- If you plan to work on something major, please open an issue to discuss the design first. +- Don't break backwards compatibility. If you really have to, open an issue to discuss this first. +- Make sure to use the `go fmt` command to format your code according to the standards. Even better, set up your editor to do this for you when saving. +- Run [go vet](https://godoc.org/golang.org/x/tools/cmd/vet) to detect any suspicious constructs in your code that could be bugs. +- Explicitly handle all error return values. If you really want to ignore an error value, you can assign it to `_`.You can use [errcheck](https://github.com/kisielk/errcheck) to verify whether you have handled all errors. +- You may also want to run [golint](https://github.com/golang/lint) as well to detect style problems. +- Add tests that cover the changes you made. Make sure to run `go test` with the `-race` argument to test for race conditions. +- Make sure your code is supported by all the Go versions we support. You can rely on [Travis CI](https://travis-ci.org/Shopify/sarama) for testing older Go versions diff --git a/vendor/github.com/Shopify/sarama/MIT-LICENSE b/vendor/github.com/Shopify/sarama/MIT-LICENSE new file mode 100644 index 000000000000..8121b63b1c4a --- /dev/null +++ b/vendor/github.com/Shopify/sarama/MIT-LICENSE @@ -0,0 +1,20 @@ +Copyright (c) 2013 Evan Huus + +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/Shopify/sarama/Makefile b/vendor/github.com/Shopify/sarama/Makefile new file mode 100644 index 000000000000..01c6d3598c46 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/Makefile @@ -0,0 +1,24 @@ +default: fmt vet errcheck test + +test: + go test -v -timeout 60s -race ./... + +vet: + go vet ./... + +errcheck: + @if go version | grep -q go1.5; then errcheck github.com/Shopify/sarama/...; fi + +fmt: + @if [ -n "$$(go fmt ./...)" ]; then echo 'Please run go fmt on your code.' && exit 1; fi + +install_dependencies: install_errcheck install_go_vet get + +install_errcheck: + @if go version | grep -q go1.5; then go get github.com/kisielk/errcheck; fi + +install_go_vet: + go get golang.org/x/tools/cmd/vet + +get: + go get -t diff --git a/vendor/github.com/Shopify/sarama/README.md b/vendor/github.com/Shopify/sarama/README.md new file mode 100644 index 000000000000..26582f95a9e6 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/README.md @@ -0,0 +1,36 @@ +sarama +====== + +[![GoDoc](https://godoc.org/github.com/Shopify/sarama?status.png)](https://godoc.org/github.com/Shopify/sarama) +[![Build Status](https://travis-ci.org/Shopify/sarama.svg?branch=master)](https://travis-ci.org/Shopify/sarama) + +Sarama is an MIT-licensed Go client library for [Apache Kafka](https://kafka.apache.org/) version 0.8 (and later). + +### Getting started + +- API documentation and examples are available via [godoc](https://godoc.org/github.com/Shopify/sarama). +- Mocks for testing are available in the [mocks](./mocks) subpackage. +- The [examples](./examples) directory contains more elaborate example applications. +- The [tools](./tools) directory contains command line tools that can be useful for testing, diagnostics, and instrumentation. + +### Compatibility and API stability + +Sarama provides a "2 releases + 2 months" compatibility guarantee: we support +the two latest stable releases of Kafka and Go, and we provide a two month +grace period for older releases. This means we currently officially support +Go 1.5 and 1.4, and Kafka 0.9.0 and 0.8.2, although older releases are still +likely to work. + +Sarama follows semantic versioning and provides API stability via the gopkg.in service. +You can import a version with a guaranteed stable API via http://gopkg.in/Shopify/sarama.v1. +A changelog is available [here](CHANGELOG.md). + +### Contributing + +* Get started by checking our [contribution guidelines](https://github.com/Shopify/sarama/blob/master/CONTRIBUTING.md). +* Read the [Sarama wiki](https://github.com/Shopify/sarama/wiki) for more + technical and design details. +* The [Kafka Protocol Specification](https://cwiki.apache.org/confluence/display/KAFKA/A+Guide+To+The+Kafka+Protocol) + contains a wealth of useful information. +* For more general issues, there is [a google group](https://groups.google.com/forum/#!forum/kafka-clients) for Kafka client developers. +* If you have any questions, just ask! diff --git a/vendor/github.com/Shopify/sarama/Vagrantfile b/vendor/github.com/Shopify/sarama/Vagrantfile new file mode 100644 index 000000000000..4586d9ae8cca --- /dev/null +++ b/vendor/github.com/Shopify/sarama/Vagrantfile @@ -0,0 +1,19 @@ +# -*- mode: ruby -*- +# vi: set ft=ruby : + +# Vagrantfile API/syntax version. Don't touch unless you know what you're doing! +VAGRANTFILE_API_VERSION = "2" + +MEMORY = 3072 + +Vagrant.configure(VAGRANTFILE_API_VERSION) do |config| + config.vm.box = "ubuntu/trusty64" + + config.vm.provision :shell, path: "vagrant/provision.sh" + + config.vm.network "private_network", ip: "192.168.100.67" + + config.vm.provider "virtualbox" do |v| + v.memory = MEMORY + end +end diff --git a/vendor/github.com/Shopify/sarama/async_producer.go b/vendor/github.com/Shopify/sarama/async_producer.go new file mode 100644 index 000000000000..e7ae8c2e5d19 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/async_producer.go @@ -0,0 +1,893 @@ +package sarama + +import ( + "fmt" + "sync" + "time" + + "github.com/eapache/go-resiliency/breaker" + "github.com/eapache/queue" +) + +// AsyncProducer publishes Kafka messages using a non-blocking API. It routes messages +// to the correct broker for the provided topic-partition, refreshing metadata as appropriate, +// and parses responses for errors. You must read from the Errors() channel or the +// producer will deadlock. You must call Close() or AsyncClose() on a producer to avoid +// leaks: it will not be garbage-collected automatically when it passes out of +// scope. +type AsyncProducer interface { + + // AsyncClose triggers a shutdown of the producer, flushing any messages it may + // have buffered. The shutdown has completed when both the Errors and Successes + // channels have been closed. When calling AsyncClose, you *must* continue to + // read from those channels in order to drain the results of any messages in + // flight. + AsyncClose() + + // Close shuts down the producer and flushes any messages it may have buffered. + // You must call this function before a producer object passes out of scope, as + // it may otherwise leak memory. You must call this before calling Close on the + // underlying client. + Close() error + + // Input is the input channel for the user to write messages to that they + // wish to send. + Input() chan<- *ProducerMessage + + // Successes is the success output channel back to the user when AckSuccesses is + // enabled. If Return.Successes is true, you MUST read from this channel or the + // Producer will deadlock. It is suggested that you send and read messages + // together in a single select statement. + Successes() <-chan *ProducerMessage + + // Errors is the error output channel back to the user. You MUST read from this + // channel or the Producer will deadlock when the channel is full. Alternatively, + // you can set Producer.Return.Errors in your config to false, which prevents + // errors to be returned. + Errors() <-chan *ProducerError +} + +type asyncProducer struct { + client Client + conf *Config + ownClient bool + + errors chan *ProducerError + input, successes, retries chan *ProducerMessage + inFlight sync.WaitGroup + + brokers map[*Broker]chan<- *ProducerMessage + brokerRefs map[chan<- *ProducerMessage]int + brokerLock sync.Mutex +} + +// NewAsyncProducer creates a new AsyncProducer using the given broker addresses and configuration. +func NewAsyncProducer(addrs []string, conf *Config) (AsyncProducer, error) { + client, err := NewClient(addrs, conf) + if err != nil { + return nil, err + } + + p, err := NewAsyncProducerFromClient(client) + if err != nil { + return nil, err + } + p.(*asyncProducer).ownClient = true + return p, nil +} + +// NewAsyncProducerFromClient creates a new Producer using the given client. It is still +// necessary to call Close() on the underlying client when shutting down this producer. +func NewAsyncProducerFromClient(client Client) (AsyncProducer, error) { + // Check that we are not dealing with a closed Client before processing any other arguments + if client.Closed() { + return nil, ErrClosedClient + } + + p := &asyncProducer{ + client: client, + conf: client.Config(), + errors: make(chan *ProducerError), + input: make(chan *ProducerMessage), + successes: make(chan *ProducerMessage), + retries: make(chan *ProducerMessage), + brokers: make(map[*Broker]chan<- *ProducerMessage), + brokerRefs: make(map[chan<- *ProducerMessage]int), + } + + // launch our singleton dispatchers + go withRecover(p.dispatcher) + go withRecover(p.retryHandler) + + return p, nil +} + +type flagSet int8 + +const ( + syn flagSet = 1 << iota // first message from partitionProducer to brokerProducer + fin // final message from partitionProducer to brokerProducer and back + shutdown // start the shutdown process +) + +// ProducerMessage is the collection of elements passed to the Producer in order to send a message. +type ProducerMessage struct { + Topic string // The Kafka topic for this message. + // The partitioning key for this message. Pre-existing Encoders include + // StringEncoder and ByteEncoder. + Key Encoder + // The actual message to store in Kafka. Pre-existing Encoders include + // StringEncoder and ByteEncoder. + Value Encoder + + // This field is used to hold arbitrary data you wish to include so it + // will be available when receiving on the Successes and Errors channels. + // Sarama completely ignores this field and is only to be used for + // pass-through data. + Metadata interface{} + + // Below this point are filled in by the producer as the message is processed + + // Offset is the offset of the message stored on the broker. This is only + // guaranteed to be defined if the message was successfully delivered and + // RequiredAcks is not NoResponse. + Offset int64 + // Partition is the partition that the message was sent to. This is only + // guaranteed to be defined if the message was successfully delivered. + Partition int32 + + retries int + flags flagSet +} + +const producerMessageOverhead = 26 // the metadata overhead of CRC, flags, etc. + +func (m *ProducerMessage) byteSize() int { + size := producerMessageOverhead + if m.Key != nil { + size += m.Key.Length() + } + if m.Value != nil { + size += m.Value.Length() + } + return size +} + +func (m *ProducerMessage) clear() { + m.flags = 0 + m.retries = 0 +} + +// ProducerError is the type of error generated when the producer fails to deliver a message. +// It contains the original ProducerMessage as well as the actual error value. +type ProducerError struct { + Msg *ProducerMessage + Err error +} + +func (pe ProducerError) Error() string { + return fmt.Sprintf("kafka: Failed to produce message to topic %s: %s", pe.Msg.Topic, pe.Err) +} + +// ProducerErrors is a type that wraps a batch of "ProducerError"s and implements the Error interface. +// It can be returned from the Producer's Close method to avoid the need to manually drain the Errors channel +// when closing a producer. +type ProducerErrors []*ProducerError + +func (pe ProducerErrors) Error() string { + return fmt.Sprintf("kafka: Failed to deliver %d messages.", len(pe)) +} + +func (p *asyncProducer) Errors() <-chan *ProducerError { + return p.errors +} + +func (p *asyncProducer) Successes() <-chan *ProducerMessage { + return p.successes +} + +func (p *asyncProducer) Input() chan<- *ProducerMessage { + return p.input +} + +func (p *asyncProducer) Close() error { + p.AsyncClose() + + if p.conf.Producer.Return.Successes { + go withRecover(func() { + for _ = range p.successes { + } + }) + } + + var errors ProducerErrors + if p.conf.Producer.Return.Errors { + for event := range p.errors { + errors = append(errors, event) + } + } + + if len(errors) > 0 { + return errors + } + return nil +} + +func (p *asyncProducer) AsyncClose() { + go withRecover(p.shutdown) +} + +// singleton +// dispatches messages by topic +func (p *asyncProducer) dispatcher() { + handlers := make(map[string]chan<- *ProducerMessage) + shuttingDown := false + + for msg := range p.input { + if msg == nil { + Logger.Println("Something tried to send a nil message, it was ignored.") + continue + } + + if msg.flags&shutdown != 0 { + shuttingDown = true + p.inFlight.Done() + continue + } else if msg.retries == 0 { + if shuttingDown { + // we can't just call returnError here because that decrements the wait group, + // which hasn't been incremented yet for this message, and shouldn't be + pErr := &ProducerError{Msg: msg, Err: ErrShuttingDown} + if p.conf.Producer.Return.Errors { + p.errors <- pErr + } else { + Logger.Println(pErr) + } + continue + } + p.inFlight.Add(1) + } + + if msg.byteSize() > p.conf.Producer.MaxMessageBytes { + p.returnError(msg, ErrMessageSizeTooLarge) + continue + } + + handler := handlers[msg.Topic] + if handler == nil { + handler = p.newTopicProducer(msg.Topic) + handlers[msg.Topic] = handler + } + + handler <- msg + } + + for _, handler := range handlers { + close(handler) + } +} + +// one per topic +// partitions messages, then dispatches them by partition +type topicProducer struct { + parent *asyncProducer + topic string + input <-chan *ProducerMessage + + breaker *breaker.Breaker + handlers map[int32]chan<- *ProducerMessage + partitioner Partitioner +} + +func (p *asyncProducer) newTopicProducer(topic string) chan<- *ProducerMessage { + input := make(chan *ProducerMessage, p.conf.ChannelBufferSize) + tp := &topicProducer{ + parent: p, + topic: topic, + input: input, + breaker: breaker.New(3, 1, 10*time.Second), + handlers: make(map[int32]chan<- *ProducerMessage), + partitioner: p.conf.Producer.Partitioner(topic), + } + go withRecover(tp.dispatch) + return input +} + +func (tp *topicProducer) dispatch() { + for msg := range tp.input { + if msg.retries == 0 { + if err := tp.partitionMessage(msg); err != nil { + tp.parent.returnError(msg, err) + continue + } + } + + handler := tp.handlers[msg.Partition] + if handler == nil { + handler = tp.parent.newPartitionProducer(msg.Topic, msg.Partition) + tp.handlers[msg.Partition] = handler + } + + handler <- msg + } + + for _, handler := range tp.handlers { + close(handler) + } +} + +func (tp *topicProducer) partitionMessage(msg *ProducerMessage) error { + var partitions []int32 + + err := tp.breaker.Run(func() (err error) { + if tp.partitioner.RequiresConsistency() { + partitions, err = tp.parent.client.Partitions(msg.Topic) + } else { + partitions, err = tp.parent.client.WritablePartitions(msg.Topic) + } + return + }) + + if err != nil { + return err + } + + numPartitions := int32(len(partitions)) + + if numPartitions == 0 { + return ErrLeaderNotAvailable + } + + choice, err := tp.partitioner.Partition(msg, numPartitions) + + if err != nil { + return err + } else if choice < 0 || choice >= numPartitions { + return ErrInvalidPartition + } + + msg.Partition = partitions[choice] + + return nil +} + +// one per partition per topic +// dispatches messages to the appropriate broker +// also responsible for maintaining message order during retries +type partitionProducer struct { + parent *asyncProducer + topic string + partition int32 + input <-chan *ProducerMessage + + leader *Broker + breaker *breaker.Breaker + output chan<- *ProducerMessage + + // highWatermark tracks the "current" retry level, which is the only one where we actually let messages through, + // all other messages get buffered in retryState[msg.retries].buf to preserve ordering + // retryState[msg.retries].expectChaser simply tracks whether we've seen a fin message for a given level (and + // therefore whether our buffer is complete and safe to flush) + highWatermark int + retryState []partitionRetryState +} + +type partitionRetryState struct { + buf []*ProducerMessage + expectChaser bool +} + +func (p *asyncProducer) newPartitionProducer(topic string, partition int32) chan<- *ProducerMessage { + input := make(chan *ProducerMessage, p.conf.ChannelBufferSize) + pp := &partitionProducer{ + parent: p, + topic: topic, + partition: partition, + input: input, + + breaker: breaker.New(3, 1, 10*time.Second), + retryState: make([]partitionRetryState, p.conf.Producer.Retry.Max+1), + } + go withRecover(pp.dispatch) + return input +} + +func (pp *partitionProducer) dispatch() { + // try to prefetch the leader; if this doesn't work, we'll do a proper call to `updateLeader` + // on the first message + pp.leader, _ = pp.parent.client.Leader(pp.topic, pp.partition) + if pp.leader != nil { + pp.output = pp.parent.getBrokerProducer(pp.leader) + pp.parent.inFlight.Add(1) // we're generating a syn message; track it so we don't shut down while it's still inflight + pp.output <- &ProducerMessage{Topic: pp.topic, Partition: pp.partition, flags: syn} + } + + for msg := range pp.input { + if msg.retries > pp.highWatermark { + // a new, higher, retry level; handle it and then back off + pp.newHighWatermark(msg.retries) + time.Sleep(pp.parent.conf.Producer.Retry.Backoff) + } else if pp.highWatermark > 0 { + // we are retrying something (else highWatermark would be 0) but this message is not a *new* retry level + if msg.retries < pp.highWatermark { + // in fact this message is not even the current retry level, so buffer it for now (unless it's a just a fin) + if msg.flags&fin == fin { + pp.retryState[msg.retries].expectChaser = false + pp.parent.inFlight.Done() // this fin is now handled and will be garbage collected + } else { + pp.retryState[msg.retries].buf = append(pp.retryState[msg.retries].buf, msg) + } + continue + } else if msg.flags&fin == fin { + // this message is of the current retry level (msg.retries == highWatermark) and the fin flag is set, + // meaning this retry level is done and we can go down (at least) one level and flush that + pp.retryState[pp.highWatermark].expectChaser = false + pp.flushRetryBuffers() + pp.parent.inFlight.Done() // this fin is now handled and will be garbage collected + continue + } + } + + // if we made it this far then the current msg contains real data, and can be sent to the next goroutine + // without breaking any of our ordering guarantees + + if pp.output == nil { + if err := pp.updateLeader(); err != nil { + pp.parent.returnError(msg, err) + time.Sleep(pp.parent.conf.Producer.Retry.Backoff) + continue + } + Logger.Printf("producer/leader/%s/%d selected broker %d\n", pp.topic, pp.partition, pp.leader.ID()) + } + + pp.output <- msg + } + + if pp.output != nil { + pp.parent.unrefBrokerProducer(pp.leader, pp.output) + } +} + +func (pp *partitionProducer) newHighWatermark(hwm int) { + Logger.Printf("producer/leader/%s/%d state change to [retrying-%d]\n", pp.topic, pp.partition, hwm) + pp.highWatermark = hwm + + // send off a fin so that we know when everything "in between" has made it + // back to us and we can safely flush the backlog (otherwise we risk re-ordering messages) + pp.retryState[pp.highWatermark].expectChaser = true + pp.parent.inFlight.Add(1) // we're generating a fin message; track it so we don't shut down while it's still inflight + pp.output <- &ProducerMessage{Topic: pp.topic, Partition: pp.partition, flags: fin, retries: pp.highWatermark - 1} + + // a new HWM means that our current broker selection is out of date + Logger.Printf("producer/leader/%s/%d abandoning broker %d\n", pp.topic, pp.partition, pp.leader.ID()) + pp.parent.unrefBrokerProducer(pp.leader, pp.output) + pp.output = nil +} + +func (pp *partitionProducer) flushRetryBuffers() { + Logger.Printf("producer/leader/%s/%d state change to [flushing-%d]\n", pp.topic, pp.partition, pp.highWatermark) + for { + pp.highWatermark-- + + if pp.output == nil { + if err := pp.updateLeader(); err != nil { + pp.parent.returnErrors(pp.retryState[pp.highWatermark].buf, err) + goto flushDone + } + Logger.Printf("producer/leader/%s/%d selected broker %d\n", pp.topic, pp.partition, pp.leader.ID()) + } + + for _, msg := range pp.retryState[pp.highWatermark].buf { + pp.output <- msg + } + + flushDone: + pp.retryState[pp.highWatermark].buf = nil + if pp.retryState[pp.highWatermark].expectChaser { + Logger.Printf("producer/leader/%s/%d state change to [retrying-%d]\n", pp.topic, pp.partition, pp.highWatermark) + break + } else if pp.highWatermark == 0 { + Logger.Printf("producer/leader/%s/%d state change to [normal]\n", pp.topic, pp.partition) + break + } + } +} + +func (pp *partitionProducer) updateLeader() error { + return pp.breaker.Run(func() (err error) { + if err = pp.parent.client.RefreshMetadata(pp.topic); err != nil { + return err + } + + if pp.leader, err = pp.parent.client.Leader(pp.topic, pp.partition); err != nil { + return err + } + + pp.output = pp.parent.getBrokerProducer(pp.leader) + pp.parent.inFlight.Add(1) // we're generating a syn message; track it so we don't shut down while it's still inflight + pp.output <- &ProducerMessage{Topic: pp.topic, Partition: pp.partition, flags: syn} + + return nil + }) +} + +// one per broker; also constructs an associated flusher +func (p *asyncProducer) newBrokerProducer(broker *Broker) chan<- *ProducerMessage { + var ( + input = make(chan *ProducerMessage) + bridge = make(chan *produceSet) + responses = make(chan *brokerProducerResponse) + ) + + bp := &brokerProducer{ + parent: p, + broker: broker, + input: input, + output: bridge, + responses: responses, + buffer: newProduceSet(p), + currentRetries: make(map[string]map[int32]error), + } + go withRecover(bp.run) + + // minimal bridge to make the network response `select`able + go withRecover(func() { + for set := range bridge { + request := set.buildRequest() + + response, err := broker.Produce(request) + + responses <- &brokerProducerResponse{ + set: set, + err: err, + res: response, + } + } + close(responses) + }) + + return input +} + +type brokerProducerResponse struct { + set *produceSet + err error + res *ProduceResponse +} + +// groups messages together into appropriately-sized batches for sending to the broker +// handles state related to retries etc +type brokerProducer struct { + parent *asyncProducer + broker *Broker + + input <-chan *ProducerMessage + output chan<- *produceSet + responses <-chan *brokerProducerResponse + + buffer *produceSet + timer <-chan time.Time + timerFired bool + + closing error + currentRetries map[string]map[int32]error +} + +func (bp *brokerProducer) run() { + var output chan<- *produceSet + Logger.Printf("producer/broker/%d starting up\n", bp.broker.ID()) + + for { + select { + case msg := <-bp.input: + if msg == nil { + bp.shutdown() + return + } + + if msg.flags&syn == syn { + Logger.Printf("producer/broker/%d state change to [open] on %s/%d\n", + bp.broker.ID(), msg.Topic, msg.Partition) + if bp.currentRetries[msg.Topic] == nil { + bp.currentRetries[msg.Topic] = make(map[int32]error) + } + bp.currentRetries[msg.Topic][msg.Partition] = nil + bp.parent.inFlight.Done() + continue + } + + if reason := bp.needsRetry(msg); reason != nil { + bp.parent.retryMessage(msg, reason) + + if bp.closing == nil && msg.flags&fin == fin { + // we were retrying this partition but we can start processing again + delete(bp.currentRetries[msg.Topic], msg.Partition) + Logger.Printf("producer/broker/%d state change to [closed] on %s/%d\n", + bp.broker.ID(), msg.Topic, msg.Partition) + } + + continue + } + + if bp.buffer.wouldOverflow(msg) { + if err := bp.waitForSpace(msg); err != nil { + bp.parent.retryMessage(msg, err) + continue + } + } + + if err := bp.buffer.add(msg); err != nil { + bp.parent.returnError(msg, err) + continue + } + + if bp.parent.conf.Producer.Flush.Frequency > 0 && bp.timer == nil { + bp.timer = time.After(bp.parent.conf.Producer.Flush.Frequency) + } + case <-bp.timer: + bp.timerFired = true + case output <- bp.buffer: + bp.rollOver() + case response := <-bp.responses: + bp.handleResponse(response) + } + + if bp.timerFired || bp.buffer.readyToFlush() { + output = bp.output + } else { + output = nil + } + } +} + +func (bp *brokerProducer) shutdown() { + for !bp.buffer.empty() { + select { + case response := <-bp.responses: + bp.handleResponse(response) + case bp.output <- bp.buffer: + bp.rollOver() + } + } + close(bp.output) + for response := range bp.responses { + bp.handleResponse(response) + } + + Logger.Printf("producer/broker/%d shut down\n", bp.broker.ID()) +} + +func (bp *brokerProducer) needsRetry(msg *ProducerMessage) error { + if bp.closing != nil { + return bp.closing + } + + return bp.currentRetries[msg.Topic][msg.Partition] +} + +func (bp *brokerProducer) waitForSpace(msg *ProducerMessage) error { + Logger.Printf("producer/broker/%d maximum request accumulated, waiting for space\n", bp.broker.ID()) + + for { + select { + case response := <-bp.responses: + bp.handleResponse(response) + // handling a response can change our state, so re-check some things + if reason := bp.needsRetry(msg); reason != nil { + return reason + } else if !bp.buffer.wouldOverflow(msg) { + return nil + } + case bp.output <- bp.buffer: + bp.rollOver() + return nil + } + } +} + +func (bp *brokerProducer) rollOver() { + bp.timer = nil + bp.timerFired = false + bp.buffer = newProduceSet(bp.parent) +} + +func (bp *brokerProducer) handleResponse(response *brokerProducerResponse) { + if response.err != nil { + bp.handleError(response.set, response.err) + } else { + bp.handleSuccess(response.set, response.res) + } + + if bp.buffer.empty() { + bp.rollOver() // this can happen if the response invalidated our buffer + } +} + +func (bp *brokerProducer) handleSuccess(sent *produceSet, response *ProduceResponse) { + // we iterate through the blocks in the request set, not the response, so that we notice + // if the response is missing a block completely + sent.eachPartition(func(topic string, partition int32, msgs []*ProducerMessage) { + if response == nil { + // this only happens when RequiredAcks is NoResponse, so we have to assume success + bp.parent.returnSuccesses(msgs) + return + } + + block := response.GetBlock(topic, partition) + if block == nil { + bp.parent.returnErrors(msgs, ErrIncompleteResponse) + return + } + + switch block.Err { + // Success + case ErrNoError: + for i, msg := range msgs { + msg.Offset = block.Offset + int64(i) + } + bp.parent.returnSuccesses(msgs) + // Retriable errors + case ErrInvalidMessage, ErrUnknownTopicOrPartition, ErrLeaderNotAvailable, ErrNotLeaderForPartition, + ErrRequestTimedOut, ErrNotEnoughReplicas, ErrNotEnoughReplicasAfterAppend: + Logger.Printf("producer/broker/%d state change to [retrying] on %s/%d because %v\n", + bp.broker.ID(), topic, partition, block.Err) + bp.currentRetries[topic][partition] = block.Err + bp.parent.retryMessages(msgs, block.Err) + bp.parent.retryMessages(bp.buffer.dropPartition(topic, partition), block.Err) + // Other non-retriable errors + default: + bp.parent.returnErrors(msgs, block.Err) + } + }) +} + +func (bp *brokerProducer) handleError(sent *produceSet, err error) { + switch err.(type) { + case PacketEncodingError: + sent.eachPartition(func(topic string, partition int32, msgs []*ProducerMessage) { + bp.parent.returnErrors(msgs, err) + }) + default: + Logger.Printf("producer/broker/%d state change to [closing] because %s\n", bp.broker.ID(), err) + bp.parent.abandonBrokerConnection(bp.broker) + _ = bp.broker.Close() + bp.closing = err + sent.eachPartition(func(topic string, partition int32, msgs []*ProducerMessage) { + bp.parent.retryMessages(msgs, err) + }) + bp.buffer.eachPartition(func(topic string, partition int32, msgs []*ProducerMessage) { + bp.parent.retryMessages(msgs, err) + }) + bp.rollOver() + } +} + +// singleton +// effectively a "bridge" between the flushers and the dispatcher in order to avoid deadlock +// based on https://godoc.org/github.com/eapache/channels#InfiniteChannel +func (p *asyncProducer) retryHandler() { + var msg *ProducerMessage + buf := queue.New() + + for { + if buf.Length() == 0 { + msg = <-p.retries + } else { + select { + case msg = <-p.retries: + case p.input <- buf.Peek().(*ProducerMessage): + buf.Remove() + continue + } + } + + if msg == nil { + return + } + + buf.Add(msg) + } +} + +// utility functions + +func (p *asyncProducer) shutdown() { + Logger.Println("Producer shutting down.") + p.inFlight.Add(1) + p.input <- &ProducerMessage{flags: shutdown} + + p.inFlight.Wait() + + if p.ownClient { + err := p.client.Close() + if err != nil { + Logger.Println("producer/shutdown failed to close the embedded client:", err) + } + } + + close(p.input) + close(p.retries) + close(p.errors) + close(p.successes) +} + +func (p *asyncProducer) returnError(msg *ProducerMessage, err error) { + msg.clear() + pErr := &ProducerError{Msg: msg, Err: err} + if p.conf.Producer.Return.Errors { + p.errors <- pErr + } else { + Logger.Println(pErr) + } + p.inFlight.Done() +} + +func (p *asyncProducer) returnErrors(batch []*ProducerMessage, err error) { + for _, msg := range batch { + p.returnError(msg, err) + } +} + +func (p *asyncProducer) returnSuccesses(batch []*ProducerMessage) { + for _, msg := range batch { + if p.conf.Producer.Return.Successes { + msg.clear() + p.successes <- msg + } + p.inFlight.Done() + } +} + +func (p *asyncProducer) retryMessage(msg *ProducerMessage, err error) { + if msg.retries >= p.conf.Producer.Retry.Max { + p.returnError(msg, err) + } else { + msg.retries++ + p.retries <- msg + } +} + +func (p *asyncProducer) retryMessages(batch []*ProducerMessage, err error) { + for _, msg := range batch { + p.retryMessage(msg, err) + } +} + +func (p *asyncProducer) getBrokerProducer(broker *Broker) chan<- *ProducerMessage { + p.brokerLock.Lock() + defer p.brokerLock.Unlock() + + bp := p.brokers[broker] + + if bp == nil { + bp = p.newBrokerProducer(broker) + p.brokers[broker] = bp + p.brokerRefs[bp] = 0 + } + + p.brokerRefs[bp]++ + + return bp +} + +func (p *asyncProducer) unrefBrokerProducer(broker *Broker, bp chan<- *ProducerMessage) { + p.brokerLock.Lock() + defer p.brokerLock.Unlock() + + p.brokerRefs[bp]-- + if p.brokerRefs[bp] == 0 { + close(bp) + delete(p.brokerRefs, bp) + + if p.brokers[broker] == bp { + delete(p.brokers, broker) + } + } +} + +func (p *asyncProducer) abandonBrokerConnection(broker *Broker) { + p.brokerLock.Lock() + defer p.brokerLock.Unlock() + + delete(p.brokers, broker) +} diff --git a/vendor/github.com/Shopify/sarama/async_producer_test.go b/vendor/github.com/Shopify/sarama/async_producer_test.go new file mode 100644 index 000000000000..9aa13da53381 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/async_producer_test.go @@ -0,0 +1,801 @@ +package sarama + +import ( + "errors" + "log" + "os" + "os/signal" + "sync" + "testing" + "time" +) + +const TestMessage = "ABC THE MESSAGE" + +func closeProducer(t *testing.T, p AsyncProducer) { + var wg sync.WaitGroup + p.AsyncClose() + + wg.Add(2) + go func() { + for _ = range p.Successes() { + t.Error("Unexpected message on Successes()") + } + wg.Done() + }() + go func() { + for msg := range p.Errors() { + t.Error(msg.Err) + } + wg.Done() + }() + wg.Wait() +} + +func expectResults(t *testing.T, p AsyncProducer, successes, errors int) { + expect := successes + errors + for expect > 0 { + select { + case msg := <-p.Errors(): + if msg.Msg.flags != 0 { + t.Error("Message had flags set") + } + errors-- + expect-- + if errors < 0 { + t.Error(msg.Err) + } + case msg := <-p.Successes(): + if msg.flags != 0 { + t.Error("Message had flags set") + } + successes-- + expect-- + if successes < 0 { + t.Error("Too many successes") + } + } + } + if successes != 0 || errors != 0 { + t.Error("Unexpected successes", successes, "or errors", errors) + } +} + +type testPartitioner chan *int32 + +func (p testPartitioner) Partition(msg *ProducerMessage, numPartitions int32) (int32, error) { + part := <-p + if part == nil { + return 0, errors.New("BOOM") + } + + return *part, nil +} + +func (p testPartitioner) RequiresConsistency() bool { + return true +} + +func (p testPartitioner) feed(partition int32) { + p <- &partition +} + +type flakyEncoder bool + +func (f flakyEncoder) Length() int { + return len(TestMessage) +} + +func (f flakyEncoder) Encode() ([]byte, error) { + if !bool(f) { + return nil, errors.New("flaky encoding error") + } + return []byte(TestMessage), nil +} + +func TestAsyncProducer(t *testing.T) { + seedBroker := newMockBroker(t, 1) + leader := newMockBroker(t, 2) + + metadataResponse := new(MetadataResponse) + metadataResponse.AddBroker(leader.Addr(), leader.BrokerID()) + metadataResponse.AddTopicPartition("my_topic", 0, leader.BrokerID(), nil, nil, ErrNoError) + seedBroker.Returns(metadataResponse) + + prodSuccess := new(ProduceResponse) + prodSuccess.AddTopicPartition("my_topic", 0, ErrNoError) + leader.Returns(prodSuccess) + + config := NewConfig() + config.Producer.Flush.Messages = 10 + config.Producer.Return.Successes = true + producer, err := NewAsyncProducer([]string{seedBroker.Addr()}, config) + if err != nil { + t.Fatal(err) + } + + for i := 0; i < 10; i++ { + producer.Input() <- &ProducerMessage{Topic: "my_topic", Key: nil, Value: StringEncoder(TestMessage), Metadata: i} + } + for i := 0; i < 10; i++ { + select { + case msg := <-producer.Errors(): + t.Error(msg.Err) + if msg.Msg.flags != 0 { + t.Error("Message had flags set") + } + case msg := <-producer.Successes(): + if msg.flags != 0 { + t.Error("Message had flags set") + } + if msg.Metadata.(int) != i { + t.Error("Message metadata did not match") + } + } + } + + closeProducer(t, producer) + leader.Close() + seedBroker.Close() +} + +func TestAsyncProducerMultipleFlushes(t *testing.T) { + seedBroker := newMockBroker(t, 1) + leader := newMockBroker(t, 2) + + metadataResponse := new(MetadataResponse) + metadataResponse.AddBroker(leader.Addr(), leader.BrokerID()) + metadataResponse.AddTopicPartition("my_topic", 0, leader.BrokerID(), nil, nil, ErrNoError) + seedBroker.Returns(metadataResponse) + + prodSuccess := new(ProduceResponse) + prodSuccess.AddTopicPartition("my_topic", 0, ErrNoError) + leader.Returns(prodSuccess) + leader.Returns(prodSuccess) + leader.Returns(prodSuccess) + + config := NewConfig() + config.Producer.Flush.Messages = 5 + config.Producer.Return.Successes = true + producer, err := NewAsyncProducer([]string{seedBroker.Addr()}, config) + if err != nil { + t.Fatal(err) + } + + for flush := 0; flush < 3; flush++ { + for i := 0; i < 5; i++ { + producer.Input() <- &ProducerMessage{Topic: "my_topic", Key: nil, Value: StringEncoder(TestMessage)} + } + expectResults(t, producer, 5, 0) + } + + closeProducer(t, producer) + leader.Close() + seedBroker.Close() +} + +func TestAsyncProducerMultipleBrokers(t *testing.T) { + seedBroker := newMockBroker(t, 1) + leader0 := newMockBroker(t, 2) + leader1 := newMockBroker(t, 3) + + metadataResponse := new(MetadataResponse) + metadataResponse.AddBroker(leader0.Addr(), leader0.BrokerID()) + metadataResponse.AddBroker(leader1.Addr(), leader1.BrokerID()) + metadataResponse.AddTopicPartition("my_topic", 0, leader0.BrokerID(), nil, nil, ErrNoError) + metadataResponse.AddTopicPartition("my_topic", 1, leader1.BrokerID(), nil, nil, ErrNoError) + seedBroker.Returns(metadataResponse) + + prodResponse0 := new(ProduceResponse) + prodResponse0.AddTopicPartition("my_topic", 0, ErrNoError) + leader0.Returns(prodResponse0) + + prodResponse1 := new(ProduceResponse) + prodResponse1.AddTopicPartition("my_topic", 1, ErrNoError) + leader1.Returns(prodResponse1) + + config := NewConfig() + config.Producer.Flush.Messages = 5 + config.Producer.Return.Successes = true + config.Producer.Partitioner = NewRoundRobinPartitioner + producer, err := NewAsyncProducer([]string{seedBroker.Addr()}, config) + if err != nil { + t.Fatal(err) + } + + for i := 0; i < 10; i++ { + producer.Input() <- &ProducerMessage{Topic: "my_topic", Key: nil, Value: StringEncoder(TestMessage)} + } + expectResults(t, producer, 10, 0) + + closeProducer(t, producer) + leader1.Close() + leader0.Close() + seedBroker.Close() +} + +func TestAsyncProducerCustomPartitioner(t *testing.T) { + seedBroker := newMockBroker(t, 1) + leader := newMockBroker(t, 2) + + metadataResponse := new(MetadataResponse) + metadataResponse.AddBroker(leader.Addr(), leader.BrokerID()) + metadataResponse.AddTopicPartition("my_topic", 0, leader.BrokerID(), nil, nil, ErrNoError) + seedBroker.Returns(metadataResponse) + + prodResponse := new(ProduceResponse) + prodResponse.AddTopicPartition("my_topic", 0, ErrNoError) + leader.Returns(prodResponse) + + config := NewConfig() + config.Producer.Flush.Messages = 2 + config.Producer.Return.Successes = true + config.Producer.Partitioner = func(topic string) Partitioner { + p := make(testPartitioner) + go func() { + p.feed(0) + p <- nil + p <- nil + p <- nil + p.feed(0) + }() + return p + } + producer, err := NewAsyncProducer([]string{seedBroker.Addr()}, config) + if err != nil { + t.Fatal(err) + } + + for i := 0; i < 5; i++ { + producer.Input() <- &ProducerMessage{Topic: "my_topic", Key: nil, Value: StringEncoder(TestMessage)} + } + expectResults(t, producer, 2, 3) + + closeProducer(t, producer) + leader.Close() + seedBroker.Close() +} + +func TestAsyncProducerFailureRetry(t *testing.T) { + seedBroker := newMockBroker(t, 1) + leader1 := newMockBroker(t, 2) + leader2 := newMockBroker(t, 3) + + metadataLeader1 := new(MetadataResponse) + metadataLeader1.AddBroker(leader1.Addr(), leader1.BrokerID()) + metadataLeader1.AddTopicPartition("my_topic", 0, leader1.BrokerID(), nil, nil, ErrNoError) + seedBroker.Returns(metadataLeader1) + + config := NewConfig() + config.Producer.Flush.Messages = 10 + config.Producer.Return.Successes = true + config.Producer.Retry.Backoff = 0 + producer, err := NewAsyncProducer([]string{seedBroker.Addr()}, config) + if err != nil { + t.Fatal(err) + } + seedBroker.Close() + + for i := 0; i < 10; i++ { + producer.Input() <- &ProducerMessage{Topic: "my_topic", Key: nil, Value: StringEncoder(TestMessage)} + } + prodNotLeader := new(ProduceResponse) + prodNotLeader.AddTopicPartition("my_topic", 0, ErrNotLeaderForPartition) + leader1.Returns(prodNotLeader) + + metadataLeader2 := new(MetadataResponse) + metadataLeader2.AddBroker(leader2.Addr(), leader2.BrokerID()) + metadataLeader2.AddTopicPartition("my_topic", 0, leader2.BrokerID(), nil, nil, ErrNoError) + leader1.Returns(metadataLeader2) + + prodSuccess := new(ProduceResponse) + prodSuccess.AddTopicPartition("my_topic", 0, ErrNoError) + leader2.Returns(prodSuccess) + expectResults(t, producer, 10, 0) + leader1.Close() + + for i := 0; i < 10; i++ { + producer.Input() <- &ProducerMessage{Topic: "my_topic", Key: nil, Value: StringEncoder(TestMessage)} + } + leader2.Returns(prodSuccess) + expectResults(t, producer, 10, 0) + + leader2.Close() + closeProducer(t, producer) +} + +func TestAsyncProducerEncoderFailures(t *testing.T) { + seedBroker := newMockBroker(t, 1) + leader := newMockBroker(t, 2) + + metadataResponse := new(MetadataResponse) + metadataResponse.AddBroker(leader.Addr(), leader.BrokerID()) + metadataResponse.AddTopicPartition("my_topic", 0, leader.BrokerID(), nil, nil, ErrNoError) + seedBroker.Returns(metadataResponse) + + prodSuccess := new(ProduceResponse) + prodSuccess.AddTopicPartition("my_topic", 0, ErrNoError) + leader.Returns(prodSuccess) + leader.Returns(prodSuccess) + leader.Returns(prodSuccess) + + config := NewConfig() + config.Producer.Flush.Messages = 1 + config.Producer.Return.Successes = true + config.Producer.Partitioner = NewManualPartitioner + producer, err := NewAsyncProducer([]string{seedBroker.Addr()}, config) + if err != nil { + t.Fatal(err) + } + + for flush := 0; flush < 3; flush++ { + producer.Input() <- &ProducerMessage{Topic: "my_topic", Key: flakyEncoder(true), Value: flakyEncoder(false)} + producer.Input() <- &ProducerMessage{Topic: "my_topic", Key: flakyEncoder(false), Value: flakyEncoder(true)} + producer.Input() <- &ProducerMessage{Topic: "my_topic", Key: flakyEncoder(true), Value: flakyEncoder(true)} + expectResults(t, producer, 1, 2) + } + + closeProducer(t, producer) + leader.Close() + seedBroker.Close() +} + +// If a Kafka broker becomes unavailable and then returns back in service, then +// producer reconnects to it and continues sending messages. +func TestAsyncProducerBrokerBounce(t *testing.T) { + // Given + seedBroker := newMockBroker(t, 1) + leader := newMockBroker(t, 2) + leaderAddr := leader.Addr() + + metadataResponse := new(MetadataResponse) + metadataResponse.AddBroker(leaderAddr, leader.BrokerID()) + metadataResponse.AddTopicPartition("my_topic", 0, leader.BrokerID(), nil, nil, ErrNoError) + seedBroker.Returns(metadataResponse) + + prodSuccess := new(ProduceResponse) + prodSuccess.AddTopicPartition("my_topic", 0, ErrNoError) + + config := NewConfig() + config.Producer.Flush.Messages = 1 + config.Producer.Return.Successes = true + config.Producer.Retry.Backoff = 0 + producer, err := NewAsyncProducer([]string{seedBroker.Addr()}, config) + if err != nil { + t.Fatal(err) + } + producer.Input() <- &ProducerMessage{Topic: "my_topic", Key: nil, Value: StringEncoder(TestMessage)} + leader.Returns(prodSuccess) + expectResults(t, producer, 1, 0) + + // When: a broker connection gets reset by a broker (network glitch, restart, you name it). + leader.Close() // producer should get EOF + leader = newMockBrokerAddr(t, 2, leaderAddr) // start it up again right away for giggles + seedBroker.Returns(metadataResponse) // tell it to go to broker 2 again + + // Then: a produced message goes through the new broker connection. + producer.Input() <- &ProducerMessage{Topic: "my_topic", Key: nil, Value: StringEncoder(TestMessage)} + leader.Returns(prodSuccess) + expectResults(t, producer, 1, 0) + + closeProducer(t, producer) + seedBroker.Close() + leader.Close() +} + +func TestAsyncProducerBrokerBounceWithStaleMetadata(t *testing.T) { + seedBroker := newMockBroker(t, 1) + leader1 := newMockBroker(t, 2) + leader2 := newMockBroker(t, 3) + + metadataLeader1 := new(MetadataResponse) + metadataLeader1.AddBroker(leader1.Addr(), leader1.BrokerID()) + metadataLeader1.AddTopicPartition("my_topic", 0, leader1.BrokerID(), nil, nil, ErrNoError) + seedBroker.Returns(metadataLeader1) + + config := NewConfig() + config.Producer.Flush.Messages = 10 + config.Producer.Return.Successes = true + config.Producer.Retry.Max = 3 + config.Producer.Retry.Backoff = 0 + producer, err := NewAsyncProducer([]string{seedBroker.Addr()}, config) + if err != nil { + t.Fatal(err) + } + + for i := 0; i < 10; i++ { + producer.Input() <- &ProducerMessage{Topic: "my_topic", Key: nil, Value: StringEncoder(TestMessage)} + } + leader1.Close() // producer should get EOF + seedBroker.Returns(metadataLeader1) // tell it to go to leader1 again even though it's still down + seedBroker.Returns(metadataLeader1) // tell it to go to leader1 again even though it's still down + + // ok fine, tell it to go to leader2 finally + metadataLeader2 := new(MetadataResponse) + metadataLeader2.AddBroker(leader2.Addr(), leader2.BrokerID()) + metadataLeader2.AddTopicPartition("my_topic", 0, leader2.BrokerID(), nil, nil, ErrNoError) + seedBroker.Returns(metadataLeader2) + + prodSuccess := new(ProduceResponse) + prodSuccess.AddTopicPartition("my_topic", 0, ErrNoError) + leader2.Returns(prodSuccess) + expectResults(t, producer, 10, 0) + seedBroker.Close() + leader2.Close() + + closeProducer(t, producer) +} + +func TestAsyncProducerMultipleRetries(t *testing.T) { + seedBroker := newMockBroker(t, 1) + leader1 := newMockBroker(t, 2) + leader2 := newMockBroker(t, 3) + + metadataLeader1 := new(MetadataResponse) + metadataLeader1.AddBroker(leader1.Addr(), leader1.BrokerID()) + metadataLeader1.AddTopicPartition("my_topic", 0, leader1.BrokerID(), nil, nil, ErrNoError) + seedBroker.Returns(metadataLeader1) + + config := NewConfig() + config.Producer.Flush.Messages = 10 + config.Producer.Return.Successes = true + config.Producer.Retry.Max = 4 + config.Producer.Retry.Backoff = 0 + producer, err := NewAsyncProducer([]string{seedBroker.Addr()}, config) + if err != nil { + t.Fatal(err) + } + + for i := 0; i < 10; i++ { + producer.Input() <- &ProducerMessage{Topic: "my_topic", Key: nil, Value: StringEncoder(TestMessage)} + } + prodNotLeader := new(ProduceResponse) + prodNotLeader.AddTopicPartition("my_topic", 0, ErrNotLeaderForPartition) + leader1.Returns(prodNotLeader) + + metadataLeader2 := new(MetadataResponse) + metadataLeader2.AddBroker(leader2.Addr(), leader2.BrokerID()) + metadataLeader2.AddTopicPartition("my_topic", 0, leader2.BrokerID(), nil, nil, ErrNoError) + seedBroker.Returns(metadataLeader2) + leader2.Returns(prodNotLeader) + seedBroker.Returns(metadataLeader1) + leader1.Returns(prodNotLeader) + seedBroker.Returns(metadataLeader1) + leader1.Returns(prodNotLeader) + seedBroker.Returns(metadataLeader2) + + prodSuccess := new(ProduceResponse) + prodSuccess.AddTopicPartition("my_topic", 0, ErrNoError) + leader2.Returns(prodSuccess) + expectResults(t, producer, 10, 0) + + for i := 0; i < 10; i++ { + producer.Input() <- &ProducerMessage{Topic: "my_topic", Key: nil, Value: StringEncoder(TestMessage)} + } + leader2.Returns(prodSuccess) + expectResults(t, producer, 10, 0) + + seedBroker.Close() + leader1.Close() + leader2.Close() + closeProducer(t, producer) +} + +func TestAsyncProducerOutOfRetries(t *testing.T) { + t.Skip("Enable once bug #294 is fixed.") + + seedBroker := newMockBroker(t, 1) + leader := newMockBroker(t, 2) + + metadataResponse := new(MetadataResponse) + metadataResponse.AddBroker(leader.Addr(), leader.BrokerID()) + metadataResponse.AddTopicPartition("my_topic", 0, leader.BrokerID(), nil, nil, ErrNoError) + seedBroker.Returns(metadataResponse) + + config := NewConfig() + config.Producer.Flush.Messages = 10 + config.Producer.Return.Successes = true + config.Producer.Retry.Backoff = 0 + config.Producer.Retry.Max = 0 + producer, err := NewAsyncProducer([]string{seedBroker.Addr()}, config) + if err != nil { + t.Fatal(err) + } + + for i := 0; i < 10; i++ { + producer.Input() <- &ProducerMessage{Topic: "my_topic", Key: nil, Value: StringEncoder(TestMessage)} + } + + prodNotLeader := new(ProduceResponse) + prodNotLeader.AddTopicPartition("my_topic", 0, ErrNotLeaderForPartition) + leader.Returns(prodNotLeader) + + for i := 0; i < 10; i++ { + select { + case msg := <-producer.Errors(): + if msg.Err != ErrNotLeaderForPartition { + t.Error(msg.Err) + } + case <-producer.Successes(): + t.Error("Unexpected success") + } + } + + seedBroker.Returns(metadataResponse) + + for i := 0; i < 10; i++ { + producer.Input() <- &ProducerMessage{Topic: "my_topic", Key: nil, Value: StringEncoder(TestMessage)} + } + + prodSuccess := new(ProduceResponse) + prodSuccess.AddTopicPartition("my_topic", 0, ErrNoError) + leader.Returns(prodSuccess) + + expectResults(t, producer, 10, 0) + + leader.Close() + seedBroker.Close() + safeClose(t, producer) +} + +func TestAsyncProducerRetryWithReferenceOpen(t *testing.T) { + seedBroker := newMockBroker(t, 1) + leader := newMockBroker(t, 2) + leaderAddr := leader.Addr() + + metadataResponse := new(MetadataResponse) + metadataResponse.AddBroker(leaderAddr, leader.BrokerID()) + metadataResponse.AddTopicPartition("my_topic", 0, leader.BrokerID(), nil, nil, ErrNoError) + metadataResponse.AddTopicPartition("my_topic", 1, leader.BrokerID(), nil, nil, ErrNoError) + seedBroker.Returns(metadataResponse) + + config := NewConfig() + config.Producer.Return.Successes = true + config.Producer.Retry.Backoff = 0 + config.Producer.Retry.Max = 1 + config.Producer.Partitioner = NewRoundRobinPartitioner + producer, err := NewAsyncProducer([]string{seedBroker.Addr()}, config) + if err != nil { + t.Fatal(err) + } + + // prime partition 0 + producer.Input() <- &ProducerMessage{Topic: "my_topic", Key: nil, Value: StringEncoder(TestMessage)} + prodSuccess := new(ProduceResponse) + prodSuccess.AddTopicPartition("my_topic", 0, ErrNoError) + leader.Returns(prodSuccess) + expectResults(t, producer, 1, 0) + + // prime partition 1 + producer.Input() <- &ProducerMessage{Topic: "my_topic", Key: nil, Value: StringEncoder(TestMessage)} + prodSuccess = new(ProduceResponse) + prodSuccess.AddTopicPartition("my_topic", 1, ErrNoError) + leader.Returns(prodSuccess) + expectResults(t, producer, 1, 0) + + // reboot the broker (the producer will get EOF on its existing connection) + leader.Close() + leader = newMockBrokerAddr(t, 2, leaderAddr) + + // send another message on partition 0 to trigger the EOF and retry + producer.Input() <- &ProducerMessage{Topic: "my_topic", Key: nil, Value: StringEncoder(TestMessage)} + + // tell partition 0 to go to that broker again + seedBroker.Returns(metadataResponse) + + // succeed this time + prodSuccess = new(ProduceResponse) + prodSuccess.AddTopicPartition("my_topic", 0, ErrNoError) + leader.Returns(prodSuccess) + expectResults(t, producer, 1, 0) + + // shutdown + closeProducer(t, producer) + seedBroker.Close() + leader.Close() +} + +func TestAsyncProducerFlusherRetryCondition(t *testing.T) { + seedBroker := newMockBroker(t, 1) + leader := newMockBroker(t, 2) + + metadataResponse := new(MetadataResponse) + metadataResponse.AddBroker(leader.Addr(), leader.BrokerID()) + metadataResponse.AddTopicPartition("my_topic", 0, leader.BrokerID(), nil, nil, ErrNoError) + metadataResponse.AddTopicPartition("my_topic", 1, leader.BrokerID(), nil, nil, ErrNoError) + seedBroker.Returns(metadataResponse) + + config := NewConfig() + config.Producer.Flush.Messages = 5 + config.Producer.Return.Successes = true + config.Producer.Retry.Backoff = 0 + config.Producer.Retry.Max = 1 + config.Producer.Partitioner = NewManualPartitioner + producer, err := NewAsyncProducer([]string{seedBroker.Addr()}, config) + if err != nil { + t.Fatal(err) + } + + // prime partitions + for p := int32(0); p < 2; p++ { + for i := 0; i < 5; i++ { + producer.Input() <- &ProducerMessage{Topic: "my_topic", Key: nil, Value: StringEncoder(TestMessage), Partition: p} + } + prodSuccess := new(ProduceResponse) + prodSuccess.AddTopicPartition("my_topic", p, ErrNoError) + leader.Returns(prodSuccess) + expectResults(t, producer, 5, 0) + } + + // send more messages on partition 0 + for i := 0; i < 5; i++ { + producer.Input() <- &ProducerMessage{Topic: "my_topic", Key: nil, Value: StringEncoder(TestMessage), Partition: 0} + } + prodNotLeader := new(ProduceResponse) + prodNotLeader.AddTopicPartition("my_topic", 0, ErrNotLeaderForPartition) + leader.Returns(prodNotLeader) + + time.Sleep(50 * time.Millisecond) + + leader.SetHandlerByMap(map[string]MockResponse{ + "ProduceRequest": newMockProduceResponse(t). + SetError("my_topic", 0, ErrNoError), + }) + + // tell partition 0 to go to that broker again + seedBroker.Returns(metadataResponse) + + // succeed this time + expectResults(t, producer, 5, 0) + + // put five more through + for i := 0; i < 5; i++ { + producer.Input() <- &ProducerMessage{Topic: "my_topic", Key: nil, Value: StringEncoder(TestMessage), Partition: 0} + } + expectResults(t, producer, 5, 0) + + // shutdown + closeProducer(t, producer) + seedBroker.Close() + leader.Close() +} + +func TestAsyncProducerRetryShutdown(t *testing.T) { + seedBroker := newMockBroker(t, 1) + leader := newMockBroker(t, 2) + + metadataLeader := new(MetadataResponse) + metadataLeader.AddBroker(leader.Addr(), leader.BrokerID()) + metadataLeader.AddTopicPartition("my_topic", 0, leader.BrokerID(), nil, nil, ErrNoError) + seedBroker.Returns(metadataLeader) + + config := NewConfig() + config.Producer.Flush.Messages = 10 + config.Producer.Return.Successes = true + config.Producer.Retry.Backoff = 0 + producer, err := NewAsyncProducer([]string{seedBroker.Addr()}, config) + if err != nil { + t.Fatal(err) + } + + for i := 0; i < 10; i++ { + producer.Input() <- &ProducerMessage{Topic: "my_topic", Key: nil, Value: StringEncoder(TestMessage)} + } + producer.AsyncClose() + time.Sleep(5 * time.Millisecond) // let the shutdown goroutine kick in + + producer.Input() <- &ProducerMessage{Topic: "FOO"} + if err := <-producer.Errors(); err.Err != ErrShuttingDown { + t.Error(err) + } + + prodNotLeader := new(ProduceResponse) + prodNotLeader.AddTopicPartition("my_topic", 0, ErrNotLeaderForPartition) + leader.Returns(prodNotLeader) + + seedBroker.Returns(metadataLeader) + + prodSuccess := new(ProduceResponse) + prodSuccess.AddTopicPartition("my_topic", 0, ErrNoError) + leader.Returns(prodSuccess) + expectResults(t, producer, 10, 0) + + seedBroker.Close() + leader.Close() + + // wait for the async-closed producer to shut down fully + for err := range producer.Errors() { + t.Error(err) + } +} + +// This example shows how to use the producer while simultaneously +// reading the Errors channel to know about any failures. +func ExampleAsyncProducer_select() { + producer, err := NewAsyncProducer([]string{"localhost:9092"}, nil) + if err != nil { + panic(err) + } + + defer func() { + if err := producer.Close(); err != nil { + log.Fatalln(err) + } + }() + + // Trap SIGINT to trigger a shutdown. + signals := make(chan os.Signal, 1) + signal.Notify(signals, os.Interrupt) + + var enqueued, errors int +ProducerLoop: + for { + select { + case producer.Input() <- &ProducerMessage{Topic: "my_topic", Key: nil, Value: StringEncoder("testing 123")}: + enqueued++ + case err := <-producer.Errors(): + log.Println("Failed to produce message", err) + errors++ + case <-signals: + break ProducerLoop + } + } + + log.Printf("Enqueued: %d; errors: %d\n", enqueued, errors) +} + +// This example shows how to use the producer with separate goroutines +// reading from the Successes and Errors channels. Note that in order +// for the Successes channel to be populated, you have to set +// config.Producer.Return.Successes to true. +func ExampleAsyncProducer_goroutines() { + config := NewConfig() + config.Producer.Return.Successes = true + producer, err := NewAsyncProducer([]string{"localhost:9092"}, config) + if err != nil { + panic(err) + } + + // Trap SIGINT to trigger a graceful shutdown. + signals := make(chan os.Signal, 1) + signal.Notify(signals, os.Interrupt) + + var ( + wg sync.WaitGroup + enqueued, successes, errors int + ) + + wg.Add(1) + go func() { + defer wg.Done() + for _ = range producer.Successes() { + successes++ + } + }() + + wg.Add(1) + go func() { + defer wg.Done() + for err := range producer.Errors() { + log.Println(err) + errors++ + } + }() + +ProducerLoop: + for { + message := &ProducerMessage{Topic: "my_topic", Value: StringEncoder("testing 123")} + select { + case producer.Input() <- message: + enqueued++ + + case <-signals: + producer.AsyncClose() // Trigger a shutdown of the producer. + break ProducerLoop + } + } + + wg.Wait() + + log.Printf("Successfully produced: %d; errors: %d\n", successes, errors) +} diff --git a/vendor/github.com/Shopify/sarama/broker.go b/vendor/github.com/Shopify/sarama/broker.go new file mode 100644 index 000000000000..5523fd4e92cc --- /dev/null +++ b/vendor/github.com/Shopify/sarama/broker.go @@ -0,0 +1,462 @@ +package sarama + +import ( + "crypto/tls" + "fmt" + "io" + "net" + "strconv" + "sync" + "sync/atomic" + "time" +) + +// Broker represents a single Kafka broker connection. All operations on this object are entirely concurrency-safe. +type Broker struct { + id int32 + addr string + + conf *Config + correlationID int32 + conn net.Conn + connErr error + lock sync.Mutex + opened int32 + + responses chan responsePromise + done chan bool +} + +type responsePromise struct { + correlationID int32 + packets chan []byte + errors chan error +} + +// NewBroker creates and returns a Broker targetting the given host:port address. +// This does not attempt to actually connect, you have to call Open() for that. +func NewBroker(addr string) *Broker { + return &Broker{id: -1, addr: addr} +} + +// Open tries to connect to the Broker if it is not already connected or connecting, but does not block +// waiting for the connection to complete. This means that any subsequent operations on the broker will +// block waiting for the connection to succeed or fail. To get the effect of a fully synchronous Open call, +// follow it by a call to Connected(). The only errors Open will return directly are ConfigurationError or +// AlreadyConnected. If conf is nil, the result of NewConfig() is used. +func (b *Broker) Open(conf *Config) error { + if conf == nil { + conf = NewConfig() + } + + err := conf.Validate() + if err != nil { + return err + } + + if !atomic.CompareAndSwapInt32(&b.opened, 0, 1) { + return ErrAlreadyConnected + } + + b.lock.Lock() + + if b.conn != nil { + b.lock.Unlock() + Logger.Printf("Failed to connect to broker %s: %s\n", b.addr, ErrAlreadyConnected) + return ErrAlreadyConnected + } + + go withRecover(func() { + defer b.lock.Unlock() + + dialer := net.Dialer{ + Timeout: conf.Net.DialTimeout, + KeepAlive: conf.Net.KeepAlive, + } + + if conf.Net.TLS.Enable { + b.conn, b.connErr = tls.DialWithDialer(&dialer, "tcp", b.addr, conf.Net.TLS.Config) + } else { + b.conn, b.connErr = dialer.Dial("tcp", b.addr) + } + if b.connErr != nil { + b.conn = nil + atomic.StoreInt32(&b.opened, 0) + Logger.Printf("Failed to connect to broker %s: %s\n", b.addr, b.connErr) + return + } + b.conn = newBufConn(b.conn) + + b.conf = conf + b.done = make(chan bool) + b.responses = make(chan responsePromise, b.conf.Net.MaxOpenRequests-1) + + if b.id >= 0 { + Logger.Printf("Connected to broker at %s (registered as #%d)\n", b.addr, b.id) + } else { + Logger.Printf("Connected to broker at %s (unregistered)\n", b.addr) + } + go withRecover(b.responseReceiver) + }) + + return nil +} + +// Connected returns true if the broker is connected and false otherwise. If the broker is not +// connected but it had tried to connect, the error from that connection attempt is also returned. +func (b *Broker) Connected() (bool, error) { + b.lock.Lock() + defer b.lock.Unlock() + + return b.conn != nil, b.connErr +} + +func (b *Broker) Close() error { + b.lock.Lock() + defer b.lock.Unlock() + + if b.conn == nil { + return ErrNotConnected + } + + close(b.responses) + <-b.done + + err := b.conn.Close() + + b.conn = nil + b.connErr = nil + b.done = nil + b.responses = nil + + atomic.StoreInt32(&b.opened, 0) + + if err == nil { + Logger.Printf("Closed connection to broker %s\n", b.addr) + } else { + Logger.Printf("Error while closing connection to broker %s: %s\n", b.addr, err) + } + + return err +} + +// ID returns the broker ID retrieved from Kafka's metadata, or -1 if that is not known. +func (b *Broker) ID() int32 { + return b.id +} + +// Addr returns the broker address as either retrieved from Kafka's metadata or passed to NewBroker. +func (b *Broker) Addr() string { + return b.addr +} + +func (b *Broker) GetMetadata(request *MetadataRequest) (*MetadataResponse, error) { + response := new(MetadataResponse) + + err := b.sendAndReceive(request, response) + + if err != nil { + return nil, err + } + + return response, nil +} + +func (b *Broker) GetConsumerMetadata(request *ConsumerMetadataRequest) (*ConsumerMetadataResponse, error) { + response := new(ConsumerMetadataResponse) + + err := b.sendAndReceive(request, response) + + if err != nil { + return nil, err + } + + return response, nil +} + +func (b *Broker) GetAvailableOffsets(request *OffsetRequest) (*OffsetResponse, error) { + response := new(OffsetResponse) + + err := b.sendAndReceive(request, response) + + if err != nil { + return nil, err + } + + return response, nil +} + +func (b *Broker) Produce(request *ProduceRequest) (*ProduceResponse, error) { + var response *ProduceResponse + var err error + + if request.RequiredAcks == NoResponse { + err = b.sendAndReceive(request, nil) + } else { + response = new(ProduceResponse) + err = b.sendAndReceive(request, response) + } + + if err != nil { + return nil, err + } + + return response, nil +} + +func (b *Broker) Fetch(request *FetchRequest) (*FetchResponse, error) { + response := new(FetchResponse) + + err := b.sendAndReceive(request, response) + + if err != nil { + return nil, err + } + + return response, nil +} + +func (b *Broker) CommitOffset(request *OffsetCommitRequest) (*OffsetCommitResponse, error) { + response := new(OffsetCommitResponse) + + err := b.sendAndReceive(request, response) + + if err != nil { + return nil, err + } + + return response, nil +} + +func (b *Broker) FetchOffset(request *OffsetFetchRequest) (*OffsetFetchResponse, error) { + response := new(OffsetFetchResponse) + + err := b.sendAndReceive(request, response) + + if err != nil { + return nil, err + } + + return response, nil +} + +func (b *Broker) JoinGroup(request *JoinGroupRequest) (*JoinGroupResponse, error) { + response := new(JoinGroupResponse) + + err := b.sendAndReceive(request, response) + if err != nil { + return nil, err + } + + return response, nil +} + +func (b *Broker) SyncGroup(request *SyncGroupRequest) (*SyncGroupResponse, error) { + response := new(SyncGroupResponse) + + err := b.sendAndReceive(request, response) + if err != nil { + return nil, err + } + + return response, nil +} + +func (b *Broker) LeaveGroup(request *LeaveGroupRequest) (*LeaveGroupResponse, error) { + response := new(LeaveGroupResponse) + + err := b.sendAndReceive(request, response) + if err != nil { + return nil, err + } + + return response, nil +} + +func (b *Broker) Heartbeat(request *HeartbeatRequest) (*HeartbeatResponse, error) { + response := new(HeartbeatResponse) + + err := b.sendAndReceive(request, response) + if err != nil { + return nil, err + } + + return response, nil +} + +func (b *Broker) ListGroups(request *ListGroupsRequest) (*ListGroupsResponse, error) { + response := new(ListGroupsResponse) + + err := b.sendAndReceive(request, response) + if err != nil { + return nil, err + } + + return response, nil +} + +func (b *Broker) DescribeGroups(request *DescribeGroupsRequest) (*DescribeGroupsResponse, error) { + response := new(DescribeGroupsResponse) + + err := b.sendAndReceive(request, response) + if err != nil { + return nil, err + } + + return response, nil +} + +func (b *Broker) send(rb requestBody, promiseResponse bool) (*responsePromise, error) { + b.lock.Lock() + defer b.lock.Unlock() + + if b.conn == nil { + if b.connErr != nil { + return nil, b.connErr + } + return nil, ErrNotConnected + } + + req := &request{correlationID: b.correlationID, clientID: b.conf.ClientID, body: rb} + buf, err := encode(req) + if err != nil { + return nil, err + } + + err = b.conn.SetWriteDeadline(time.Now().Add(b.conf.Net.WriteTimeout)) + if err != nil { + return nil, err + } + + _, err = b.conn.Write(buf) + if err != nil { + return nil, err + } + b.correlationID++ + + if !promiseResponse { + return nil, nil + } + + promise := responsePromise{req.correlationID, make(chan []byte), make(chan error)} + b.responses <- promise + + return &promise, nil +} + +func (b *Broker) sendAndReceive(req requestBody, res decoder) error { + promise, err := b.send(req, res != nil) + + if err != nil { + return err + } + + if promise == nil { + return nil + } + + select { + case buf := <-promise.packets: + return decode(buf, res) + case err = <-promise.errors: + return err + } +} + +func (b *Broker) decode(pd packetDecoder) (err error) { + b.id, err = pd.getInt32() + if err != nil { + return err + } + + host, err := pd.getString() + if err != nil { + return err + } + + port, err := pd.getInt32() + if err != nil { + return err + } + + b.addr = net.JoinHostPort(host, fmt.Sprint(port)) + if _, _, err := net.SplitHostPort(b.addr); err != nil { + return err + } + + return nil +} + +func (b *Broker) encode(pe packetEncoder) (err error) { + + host, portstr, err := net.SplitHostPort(b.addr) + if err != nil { + return err + } + port, err := strconv.Atoi(portstr) + if err != nil { + return err + } + + pe.putInt32(b.id) + + err = pe.putString(host) + if err != nil { + return err + } + + pe.putInt32(int32(port)) + + return nil +} + +func (b *Broker) responseReceiver() { + var dead error + header := make([]byte, 8) + for response := range b.responses { + if dead != nil { + response.errors <- dead + continue + } + + err := b.conn.SetReadDeadline(time.Now().Add(b.conf.Net.ReadTimeout)) + if err != nil { + dead = err + response.errors <- err + continue + } + + _, err = io.ReadFull(b.conn, header) + if err != nil { + dead = err + response.errors <- err + continue + } + + decodedHeader := responseHeader{} + err = decode(header, &decodedHeader) + if err != nil { + dead = err + response.errors <- err + continue + } + if decodedHeader.correlationID != response.correlationID { + // TODO if decoded ID < cur ID, discard until we catch up + // TODO if decoded ID > cur ID, save it so when cur ID catches up we have a response + dead = PacketDecodingError{fmt.Sprintf("correlation ID didn't match, wanted %d, got %d", response.correlationID, decodedHeader.correlationID)} + response.errors <- dead + continue + } + + buf := make([]byte, decodedHeader.length-4) + _, err = io.ReadFull(b.conn, buf) + if err != nil { + dead = err + response.errors <- err + continue + } + + response.packets <- buf + } + close(b.done) +} diff --git a/vendor/github.com/Shopify/sarama/broker_test.go b/vendor/github.com/Shopify/sarama/broker_test.go new file mode 100644 index 000000000000..22731bfa82b7 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/broker_test.go @@ -0,0 +1,251 @@ +package sarama + +import ( + "fmt" + "testing" +) + +func ExampleBroker() { + broker := NewBroker("localhost:9092") + err := broker.Open(nil) + if err != nil { + panic(err) + } + + request := MetadataRequest{Topics: []string{"myTopic"}} + response, err := broker.GetMetadata(&request) + if err != nil { + _ = broker.Close() + panic(err) + } + + fmt.Println("There are", len(response.Topics), "topics active in the cluster.") + + if err = broker.Close(); err != nil { + panic(err) + } +} + +type mockEncoder struct { + bytes []byte +} + +func (m mockEncoder) encode(pe packetEncoder) error { + return pe.putRawBytes(m.bytes) +} + +func TestBrokerAccessors(t *testing.T) { + broker := NewBroker("abc:123") + + if broker.ID() != -1 { + t.Error("New broker didn't have an ID of -1.") + } + + if broker.Addr() != "abc:123" { + t.Error("New broker didn't have the correct address") + } + + broker.id = 34 + if broker.ID() != 34 { + t.Error("Manually setting broker ID did not take effect.") + } +} + +func TestSimpleBrokerCommunication(t *testing.T) { + mb := newMockBroker(t, 0) + defer mb.Close() + + broker := NewBroker(mb.Addr()) + err := broker.Open(nil) + if err != nil { + t.Fatal(err) + } + + for _, tt := range brokerTestTable { + mb.Returns(&mockEncoder{tt.response}) + } + for _, tt := range brokerTestTable { + tt.runner(t, broker) + } + + err = broker.Close() + if err != nil { + t.Error(err) + } +} + +// We're not testing encoding/decoding here, so most of the requests/responses will be empty for simplicity's sake +var brokerTestTable = []struct { + response []byte + runner func(*testing.T, *Broker) +}{ + {[]byte{0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00}, + func(t *testing.T, broker *Broker) { + request := MetadataRequest{} + response, err := broker.GetMetadata(&request) + if err != nil { + t.Error(err) + } + if response == nil { + t.Error("Metadata request got no response!") + } + }}, + + {[]byte{0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x01, 't', 0x00, 0x00, 0x00, 0x00}, + func(t *testing.T, broker *Broker) { + request := ConsumerMetadataRequest{} + response, err := broker.GetConsumerMetadata(&request) + if err != nil { + t.Error(err) + } + if response == nil { + t.Error("Consumer Metadata request got no response!") + } + }}, + + {[]byte{}, + func(t *testing.T, broker *Broker) { + request := ProduceRequest{} + request.RequiredAcks = NoResponse + response, err := broker.Produce(&request) + if err != nil { + t.Error(err) + } + if response != nil { + t.Error("Produce request with NoResponse got a response!") + } + }}, + + {[]byte{0x00, 0x00, 0x00, 0x00}, + func(t *testing.T, broker *Broker) { + request := ProduceRequest{} + request.RequiredAcks = WaitForLocal + response, err := broker.Produce(&request) + if err != nil { + t.Error(err) + } + if response == nil { + t.Error("Produce request without NoResponse got no response!") + } + }}, + + {[]byte{0x00, 0x00, 0x00, 0x00}, + func(t *testing.T, broker *Broker) { + request := FetchRequest{} + response, err := broker.Fetch(&request) + if err != nil { + t.Error(err) + } + if response == nil { + t.Error("Fetch request got no response!") + } + }}, + + {[]byte{0x00, 0x00, 0x00, 0x00}, + func(t *testing.T, broker *Broker) { + request := OffsetFetchRequest{} + response, err := broker.FetchOffset(&request) + if err != nil { + t.Error(err) + } + if response == nil { + t.Error("OffsetFetch request got no response!") + } + }}, + + {[]byte{0x00, 0x00, 0x00, 0x00}, + func(t *testing.T, broker *Broker) { + request := OffsetCommitRequest{} + response, err := broker.CommitOffset(&request) + if err != nil { + t.Error(err) + } + if response == nil { + t.Error("OffsetCommit request got no response!") + } + }}, + + {[]byte{0x00, 0x00, 0x00, 0x00}, + func(t *testing.T, broker *Broker) { + request := OffsetRequest{} + response, err := broker.GetAvailableOffsets(&request) + if err != nil { + t.Error(err) + } + if response == nil { + t.Error("Offset request got no response!") + } + }}, + + {[]byte{0x00, 0x17, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00}, + func(t *testing.T, broker *Broker) { + request := JoinGroupRequest{} + response, err := broker.JoinGroup(&request) + if err != nil { + t.Error(err) + } + if response == nil { + t.Error("JoinGroup request got no response!") + } + }}, + + {[]byte{0x00, 0x00, 0x00, 0x00, 0x00, 0x00}, + func(t *testing.T, broker *Broker) { + request := SyncGroupRequest{} + response, err := broker.SyncGroup(&request) + if err != nil { + t.Error(err) + } + if response == nil { + t.Error("SyncGroup request got no response!") + } + }}, + + {[]byte{0x00, 0x00}, + func(t *testing.T, broker *Broker) { + request := LeaveGroupRequest{} + response, err := broker.LeaveGroup(&request) + if err != nil { + t.Error(err) + } + if response == nil { + t.Error("LeaveGroup request got no response!") + } + }}, + + {[]byte{0x00, 0x00}, + func(t *testing.T, broker *Broker) { + request := HeartbeatRequest{} + response, err := broker.Heartbeat(&request) + if err != nil { + t.Error(err) + } + if response == nil { + t.Error("Heartbeat request got no response!") + } + }}, + + {[]byte{0x00, 0x00, 0x00, 0x00, 0x00, 0x00}, + func(t *testing.T, broker *Broker) { + request := ListGroupsRequest{} + response, err := broker.ListGroups(&request) + if err != nil { + t.Error(err) + } + if response == nil { + t.Error("ListGroups request got no response!") + } + }}, + + {[]byte{0x00, 0x00, 0x00, 0x00}, + func(t *testing.T, broker *Broker) { + request := DescribeGroupsRequest{} + response, err := broker.DescribeGroups(&request) + if err != nil { + t.Error(err) + } + if response == nil { + t.Error("DescribeGroups request got no response!") + } + }}, +} diff --git a/vendor/github.com/Shopify/sarama/client.go b/vendor/github.com/Shopify/sarama/client.go new file mode 100644 index 000000000000..c7aecc3e0861 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/client.go @@ -0,0 +1,732 @@ +package sarama + +import ( + "math/rand" + "sort" + "sync" + "time" +) + +// Client is a generic Kafka client. It manages connections to one or more Kafka brokers. +// You MUST call Close() on a client to avoid leaks, it will not be garbage-collected +// automatically when it passes out of scope. A single client can be safely shared by +// multiple concurrent Producers and Consumers. +type Client interface { + // Config returns the Config struct of the client. This struct should not be + // altered after it has been created. + Config() *Config + + // Topics returns the set of available topics as retrieved from cluster metadata. + Topics() ([]string, error) + + // Partitions returns the sorted list of all partition IDs for the given topic. + Partitions(topic string) ([]int32, error) + + // WritablePartitions returns the sorted list of all writable partition IDs for + // the given topic, where "writable" means "having a valid leader accepting + // writes". + WritablePartitions(topic string) ([]int32, error) + + // Leader returns the broker object that is the leader of the current + // topic/partition, as determined by querying the cluster metadata. + Leader(topic string, partitionID int32) (*Broker, error) + + // Replicas returns the set of all replica IDs for the given partition. + Replicas(topic string, partitionID int32) ([]int32, error) + + // RefreshMetadata takes a list of topics and queries the cluster to refresh the + // available metadata for those topics. If no topics are provided, it will refresh + // metadata for all topics. + RefreshMetadata(topics ...string) error + + // GetOffset queries the cluster to get the most recent available offset at the + // given time on the topic/partition combination. Time should be OffsetOldest for + // the earliest available offset, OffsetNewest for the offset of the message that + // will be produced next, or a time. + GetOffset(topic string, partitionID int32, time int64) (int64, error) + + // Coordinator returns the coordinating broker for a consumer group. It will + // return a locally cached value if it's available. You can call + // RefreshCoordinator to update the cached value. This function only works on + // Kafka 0.8.2 and higher. + Coordinator(consumerGroup string) (*Broker, error) + + // RefreshCoordinator retrieves the coordinator for a consumer group and stores it + // in local cache. This function only works on Kafka 0.8.2 and higher. + RefreshCoordinator(consumerGroup string) error + + // Close shuts down all broker connections managed by this client. It is required + // to call this function before a client object passes out of scope, as it will + // otherwise leak memory. You must close any Producers or Consumers using a client + // before you close the client. + Close() error + + // Closed returns true if the client has already had Close called on it + Closed() bool +} + +const ( + // OffsetNewest stands for the log head offset, i.e. the offset that will be + // assigned to the next message that will be produced to the partition. You + // can send this to a client's GetOffset method to get this offset, or when + // calling ConsumePartition to start consuming new messages. + OffsetNewest int64 = -1 + // OffsetOldest stands for the oldest offset available on the broker for a + // partition. You can send this to a client's GetOffset method to get this + // offset, or when calling ConsumePartition to start consuming from the + // oldest offset that is still available on the broker. + OffsetOldest int64 = -2 +) + +type client struct { + conf *Config + closer, closed chan none // for shutting down background metadata updater + + // the broker addresses given to us through the constructor are not guaranteed to be returned in + // the cluster metadata (I *think* it only returns brokers who are currently leading partitions?) + // so we store them separately + seedBrokers []*Broker + deadSeeds []*Broker + + brokers map[int32]*Broker // maps broker ids to brokers + metadata map[string]map[int32]*PartitionMetadata // maps topics to partition ids to metadata + coordinators map[string]int32 // Maps consumer group names to coordinating broker IDs + + // If the number of partitions is large, we can get some churn calling cachedPartitions, + // so the result is cached. It is important to update this value whenever metadata is changed + cachedPartitionsResults map[string][maxPartitionIndex][]int32 + + lock sync.RWMutex // protects access to the maps that hold cluster state. +} + +// NewClient creates a new Client. It connects to one of the given broker addresses +// and uses that broker to automatically fetch metadata on the rest of the kafka cluster. If metadata cannot +// be retrieved from any of the given broker addresses, the client is not created. +func NewClient(addrs []string, conf *Config) (Client, error) { + Logger.Println("Initializing new client") + + if conf == nil { + conf = NewConfig() + } + + if err := conf.Validate(); err != nil { + return nil, err + } + + if len(addrs) < 1 { + return nil, ConfigurationError("You must provide at least one broker address") + } + + client := &client{ + conf: conf, + closer: make(chan none), + closed: make(chan none), + brokers: make(map[int32]*Broker), + metadata: make(map[string]map[int32]*PartitionMetadata), + cachedPartitionsResults: make(map[string][maxPartitionIndex][]int32), + coordinators: make(map[string]int32), + } + + random := rand.New(rand.NewSource(time.Now().UnixNano())) + for _, index := range random.Perm(len(addrs)) { + client.seedBrokers = append(client.seedBrokers, NewBroker(addrs[index])) + } + + // do an initial fetch of all cluster metadata by specifing an empty list of topics + err := client.RefreshMetadata() + switch err { + case nil: + break + case ErrLeaderNotAvailable, ErrReplicaNotAvailable: + // indicates that maybe part of the cluster is down, but is not fatal to creating the client + Logger.Println(err) + default: + close(client.closed) // we haven't started the background updater yet, so we have to do this manually + _ = client.Close() + return nil, err + } + go withRecover(client.backgroundMetadataUpdater) + + Logger.Println("Successfully initialized new client") + + return client, nil +} + +func (client *client) Config() *Config { + return client.conf +} + +func (client *client) Close() error { + if client.Closed() { + // Chances are this is being called from a defer() and the error will go unobserved + // so we go ahead and log the event in this case. + Logger.Printf("Close() called on already closed client") + return ErrClosedClient + } + + // shutdown and wait for the background thread before we take the lock, to avoid races + close(client.closer) + <-client.closed + + client.lock.Lock() + defer client.lock.Unlock() + Logger.Println("Closing Client") + + for _, broker := range client.brokers { + safeAsyncClose(broker) + } + + for _, broker := range client.seedBrokers { + safeAsyncClose(broker) + } + + client.brokers = nil + client.metadata = nil + + return nil +} + +func (client *client) Closed() bool { + return client.brokers == nil +} + +func (client *client) Topics() ([]string, error) { + if client.Closed() { + return nil, ErrClosedClient + } + + client.lock.RLock() + defer client.lock.RUnlock() + + ret := make([]string, 0, len(client.metadata)) + for topic := range client.metadata { + ret = append(ret, topic) + } + + return ret, nil +} + +func (client *client) Partitions(topic string) ([]int32, error) { + if client.Closed() { + return nil, ErrClosedClient + } + + partitions := client.cachedPartitions(topic, allPartitions) + + if len(partitions) == 0 { + err := client.RefreshMetadata(topic) + if err != nil { + return nil, err + } + partitions = client.cachedPartitions(topic, allPartitions) + } + + if partitions == nil { + return nil, ErrUnknownTopicOrPartition + } + + return partitions, nil +} + +func (client *client) WritablePartitions(topic string) ([]int32, error) { + if client.Closed() { + return nil, ErrClosedClient + } + + partitions := client.cachedPartitions(topic, writablePartitions) + + // len==0 catches when it's nil (no such topic) and the odd case when every single + // partition is undergoing leader election simultaneously. Callers have to be able to handle + // this function returning an empty slice (which is a valid return value) but catching it + // here the first time (note we *don't* catch it below where we return ErrUnknownTopicOrPartition) triggers + // a metadata refresh as a nicety so callers can just try again and don't have to manually + // trigger a refresh (otherwise they'd just keep getting a stale cached copy). + if len(partitions) == 0 { + err := client.RefreshMetadata(topic) + if err != nil { + return nil, err + } + partitions = client.cachedPartitions(topic, writablePartitions) + } + + if partitions == nil { + return nil, ErrUnknownTopicOrPartition + } + + return partitions, nil +} + +func (client *client) Replicas(topic string, partitionID int32) ([]int32, error) { + if client.Closed() { + return nil, ErrClosedClient + } + + metadata := client.cachedMetadata(topic, partitionID) + + if metadata == nil { + err := client.RefreshMetadata(topic) + if err != nil { + return nil, err + } + metadata = client.cachedMetadata(topic, partitionID) + } + + if metadata == nil { + return nil, ErrUnknownTopicOrPartition + } + + if metadata.Err == ErrReplicaNotAvailable { + return nil, metadata.Err + } + return dupeAndSort(metadata.Replicas), nil +} + +func (client *client) Leader(topic string, partitionID int32) (*Broker, error) { + if client.Closed() { + return nil, ErrClosedClient + } + + leader, err := client.cachedLeader(topic, partitionID) + + if leader == nil { + err := client.RefreshMetadata(topic) + if err != nil { + return nil, err + } + leader, err = client.cachedLeader(topic, partitionID) + } + + return leader, err +} + +func (client *client) RefreshMetadata(topics ...string) error { + if client.Closed() { + return ErrClosedClient + } + + // Prior to 0.8.2, Kafka will throw exceptions on an empty topic and not return a proper + // error. This handles the case by returning an error instead of sending it + // off to Kafka. See: https://github.com/Shopify/sarama/pull/38#issuecomment-26362310 + for _, topic := range topics { + if len(topic) == 0 { + return ErrInvalidTopic // this is the error that 0.8.2 and later correctly return + } + } + + return client.tryRefreshMetadata(topics, client.conf.Metadata.Retry.Max) +} + +func (client *client) GetOffset(topic string, partitionID int32, time int64) (int64, error) { + if client.Closed() { + return -1, ErrClosedClient + } + + offset, err := client.getOffset(topic, partitionID, time) + + if err != nil { + if err := client.RefreshMetadata(topic); err != nil { + return -1, err + } + return client.getOffset(topic, partitionID, time) + } + + return offset, err +} + +func (client *client) Coordinator(consumerGroup string) (*Broker, error) { + if client.Closed() { + return nil, ErrClosedClient + } + + coordinator := client.cachedCoordinator(consumerGroup) + + if coordinator == nil { + if err := client.RefreshCoordinator(consumerGroup); err != nil { + return nil, err + } + coordinator = client.cachedCoordinator(consumerGroup) + } + + if coordinator == nil { + return nil, ErrConsumerCoordinatorNotAvailable + } + + _ = coordinator.Open(client.conf) + return coordinator, nil +} + +func (client *client) RefreshCoordinator(consumerGroup string) error { + if client.Closed() { + return ErrClosedClient + } + + response, err := client.getConsumerMetadata(consumerGroup, client.conf.Metadata.Retry.Max) + if err != nil { + return err + } + + client.lock.Lock() + defer client.lock.Unlock() + client.registerBroker(response.Coordinator) + client.coordinators[consumerGroup] = response.Coordinator.ID() + return nil +} + +// private broker management helpers + +// registerBroker makes sure a broker received by a Metadata or Coordinator request is registered +// in the brokers map. It returns the broker that is registered, which may be the provided broker, +// or a previously registered Broker instance. You must hold the write lock before calling this function. +func (client *client) registerBroker(broker *Broker) { + if client.brokers[broker.ID()] == nil { + client.brokers[broker.ID()] = broker + Logger.Printf("client/brokers registered new broker #%d at %s", broker.ID(), broker.Addr()) + } else if broker.Addr() != client.brokers[broker.ID()].Addr() { + safeAsyncClose(client.brokers[broker.ID()]) + client.brokers[broker.ID()] = broker + Logger.Printf("client/brokers replaced registered broker #%d with %s", broker.ID(), broker.Addr()) + } +} + +// deregisterBroker removes a broker from the seedsBroker list, and if it's +// not the seedbroker, removes it from brokers map completely. +func (client *client) deregisterBroker(broker *Broker) { + client.lock.Lock() + defer client.lock.Unlock() + + if len(client.seedBrokers) > 0 && broker == client.seedBrokers[0] { + client.deadSeeds = append(client.deadSeeds, broker) + client.seedBrokers = client.seedBrokers[1:] + } else { + // we do this so that our loop in `tryRefreshMetadata` doesn't go on forever, + // but we really shouldn't have to; once that loop is made better this case can be + // removed, and the function generally can be renamed from `deregisterBroker` to + // `nextSeedBroker` or something + Logger.Printf("client/brokers deregistered broker #%d at %s", broker.ID(), broker.Addr()) + delete(client.brokers, broker.ID()) + } +} + +func (client *client) resurrectDeadBrokers() { + client.lock.Lock() + defer client.lock.Unlock() + + Logger.Printf("client/brokers resurrecting %d dead seed brokers", len(client.deadSeeds)) + client.seedBrokers = append(client.seedBrokers, client.deadSeeds...) + client.deadSeeds = nil +} + +func (client *client) any() *Broker { + client.lock.RLock() + defer client.lock.RUnlock() + + if len(client.seedBrokers) > 0 { + _ = client.seedBrokers[0].Open(client.conf) + return client.seedBrokers[0] + } + + // not guaranteed to be random *or* deterministic + for _, broker := range client.brokers { + _ = broker.Open(client.conf) + return broker + } + + return nil +} + +// private caching/lazy metadata helpers + +type partitionType int + +const ( + allPartitions partitionType = iota + writablePartitions + // If you add any more types, update the partition cache in update() + + // Ensure this is the last partition type value + maxPartitionIndex +) + +func (client *client) cachedMetadata(topic string, partitionID int32) *PartitionMetadata { + client.lock.RLock() + defer client.lock.RUnlock() + + partitions := client.metadata[topic] + if partitions != nil { + return partitions[partitionID] + } + + return nil +} + +func (client *client) cachedPartitions(topic string, partitionSet partitionType) []int32 { + client.lock.RLock() + defer client.lock.RUnlock() + + partitions, exists := client.cachedPartitionsResults[topic] + + if !exists { + return nil + } + return partitions[partitionSet] +} + +func (client *client) setPartitionCache(topic string, partitionSet partitionType) []int32 { + partitions := client.metadata[topic] + + if partitions == nil { + return nil + } + + ret := make([]int32, 0, len(partitions)) + for _, partition := range partitions { + if partitionSet == writablePartitions && partition.Err == ErrLeaderNotAvailable { + continue + } + ret = append(ret, partition.ID) + } + + sort.Sort(int32Slice(ret)) + return ret +} + +func (client *client) cachedLeader(topic string, partitionID int32) (*Broker, error) { + client.lock.RLock() + defer client.lock.RUnlock() + + partitions := client.metadata[topic] + if partitions != nil { + metadata, ok := partitions[partitionID] + if ok { + if metadata.Err == ErrLeaderNotAvailable { + return nil, ErrLeaderNotAvailable + } + b := client.brokers[metadata.Leader] + if b == nil { + return nil, ErrLeaderNotAvailable + } + _ = b.Open(client.conf) + return b, nil + } + } + + return nil, ErrUnknownTopicOrPartition +} + +func (client *client) getOffset(topic string, partitionID int32, time int64) (int64, error) { + broker, err := client.Leader(topic, partitionID) + if err != nil { + return -1, err + } + + request := &OffsetRequest{} + request.AddBlock(topic, partitionID, time, 1) + + response, err := broker.GetAvailableOffsets(request) + if err != nil { + _ = broker.Close() + return -1, err + } + + block := response.GetBlock(topic, partitionID) + if block == nil { + _ = broker.Close() + return -1, ErrIncompleteResponse + } + if block.Err != ErrNoError { + return -1, block.Err + } + if len(block.Offsets) != 1 { + return -1, ErrOffsetOutOfRange + } + + return block.Offsets[0], nil +} + +// core metadata update logic + +func (client *client) backgroundMetadataUpdater() { + defer close(client.closed) + + if client.conf.Metadata.RefreshFrequency == time.Duration(0) { + return + } + + ticker := time.NewTicker(client.conf.Metadata.RefreshFrequency) + defer ticker.Stop() + + for { + select { + case <-ticker.C: + if err := client.RefreshMetadata(); err != nil { + Logger.Println("Client background metadata update:", err) + } + case <-client.closer: + return + } + } +} + +func (client *client) tryRefreshMetadata(topics []string, attemptsRemaining int) error { + retry := func(err error) error { + if attemptsRemaining > 0 { + Logger.Printf("client/metadata retrying after %dms... (%d attempts remaining)\n", client.conf.Metadata.Retry.Backoff/time.Millisecond, attemptsRemaining) + time.Sleep(client.conf.Metadata.Retry.Backoff) + return client.tryRefreshMetadata(topics, attemptsRemaining-1) + } + return err + } + + for broker := client.any(); broker != nil; broker = client.any() { + if len(topics) > 0 { + Logger.Printf("client/metadata fetching metadata for %v from broker %s\n", topics, broker.addr) + } else { + Logger.Printf("client/metadata fetching metadata for all topics from broker %s\n", broker.addr) + } + response, err := broker.GetMetadata(&MetadataRequest{Topics: topics}) + + switch err.(type) { + case nil: + // valid response, use it + if shouldRetry, err := client.updateMetadata(response); shouldRetry { + Logger.Println("client/metadata found some partitions to be leaderless") + return retry(err) // note: err can be nil + } else { + return err + } + + case PacketEncodingError: + // didn't even send, return the error + return err + default: + // some other error, remove that broker and try again + Logger.Println("client/metadata got error from broker while fetching metadata:", err) + _ = broker.Close() + client.deregisterBroker(broker) + } + } + + Logger.Println("client/metadata no available broker to send metadata request to") + client.resurrectDeadBrokers() + return retry(ErrOutOfBrokers) +} + +// if no fatal error, returns a list of topics that need retrying due to ErrLeaderNotAvailable +func (client *client) updateMetadata(data *MetadataResponse) (retry bool, err error) { + client.lock.Lock() + defer client.lock.Unlock() + + // For all the brokers we received: + // - if it is a new ID, save it + // - if it is an existing ID, but the address we have is stale, discard the old one and save it + // - otherwise ignore it, replacing our existing one would just bounce the connection + for _, broker := range data.Brokers { + client.registerBroker(broker) + } + + for _, topic := range data.Topics { + delete(client.metadata, topic.Name) + delete(client.cachedPartitionsResults, topic.Name) + + switch topic.Err { + case ErrNoError: + break + case ErrInvalidTopic: // don't retry, don't store partial results + err = topic.Err + continue + case ErrUnknownTopicOrPartition: // retry, do not store partial partition results + err = topic.Err + retry = true + continue + case ErrLeaderNotAvailable: // retry, but store partial partition results + retry = true + break + default: // don't retry, don't store partial results + Logger.Printf("Unexpected topic-level metadata error: %s", topic.Err) + err = topic.Err + continue + } + + client.metadata[topic.Name] = make(map[int32]*PartitionMetadata, len(topic.Partitions)) + for _, partition := range topic.Partitions { + client.metadata[topic.Name][partition.ID] = partition + if partition.Err == ErrLeaderNotAvailable { + retry = true + } + } + + var partitionCache [maxPartitionIndex][]int32 + partitionCache[allPartitions] = client.setPartitionCache(topic.Name, allPartitions) + partitionCache[writablePartitions] = client.setPartitionCache(topic.Name, writablePartitions) + client.cachedPartitionsResults[topic.Name] = partitionCache + } + + return +} + +func (client *client) cachedCoordinator(consumerGroup string) *Broker { + client.lock.RLock() + defer client.lock.RUnlock() + if coordinatorID, ok := client.coordinators[consumerGroup]; ok { + return client.brokers[coordinatorID] + } + return nil +} + +func (client *client) getConsumerMetadata(consumerGroup string, attemptsRemaining int) (*ConsumerMetadataResponse, error) { + retry := func(err error) (*ConsumerMetadataResponse, error) { + if attemptsRemaining > 0 { + Logger.Printf("client/coordinator retrying after %dms... (%d attempts remaining)\n", client.conf.Metadata.Retry.Backoff/time.Millisecond, attemptsRemaining) + time.Sleep(client.conf.Metadata.Retry.Backoff) + return client.getConsumerMetadata(consumerGroup, attemptsRemaining-1) + } + return nil, err + } + + for broker := client.any(); broker != nil; broker = client.any() { + Logger.Printf("client/coordinator requesting coordinator for consumergoup %s from %s\n", consumerGroup, broker.Addr()) + + request := new(ConsumerMetadataRequest) + request.ConsumerGroup = consumerGroup + + response, err := broker.GetConsumerMetadata(request) + + if err != nil { + Logger.Printf("client/coordinator request to broker %s failed: %s\n", broker.Addr(), err) + + switch err.(type) { + case PacketEncodingError: + return nil, err + default: + _ = broker.Close() + client.deregisterBroker(broker) + continue + } + } + + switch response.Err { + case ErrNoError: + Logger.Printf("client/coordinator coordinator for consumergoup %s is #%d (%s)\n", consumerGroup, response.Coordinator.ID(), response.Coordinator.Addr()) + return response, nil + + case ErrConsumerCoordinatorNotAvailable: + Logger.Printf("client/coordinator coordinator for consumer group %s is not available\n", consumerGroup) + + // This is very ugly, but this scenario will only happen once per cluster. + // The __consumer_offsets topic only has to be created one time. + // The number of partitions not configurable, but partition 0 should always exist. + if _, err := client.Leader("__consumer_offsets", 0); err != nil { + Logger.Printf("client/coordinator the __consumer_offsets topic is not initialized completely yet. Waiting 2 seconds...\n") + time.Sleep(2 * time.Second) + } + + return retry(ErrConsumerCoordinatorNotAvailable) + default: + return nil, response.Err + } + } + + Logger.Println("client/coordinator no available broker to send consumer metadata request to") + client.resurrectDeadBrokers() + return retry(ErrOutOfBrokers) +} diff --git a/vendor/github.com/Shopify/sarama/client_test.go b/vendor/github.com/Shopify/sarama/client_test.go new file mode 100644 index 000000000000..f84b9af31a68 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/client_test.go @@ -0,0 +1,608 @@ +package sarama + +import ( + "io" + "sync" + "testing" + "time" +) + +func safeClose(t testing.TB, c io.Closer) { + err := c.Close() + if err != nil { + t.Error(err) + } +} + +func TestSimpleClient(t *testing.T) { + seedBroker := newMockBroker(t, 1) + + seedBroker.Returns(new(MetadataResponse)) + + client, err := NewClient([]string{seedBroker.Addr()}, nil) + if err != nil { + t.Fatal(err) + } + + seedBroker.Close() + safeClose(t, client) +} + +func TestCachedPartitions(t *testing.T) { + seedBroker := newMockBroker(t, 1) + + replicas := []int32{3, 1, 5} + isr := []int32{5, 1} + + metadataResponse := new(MetadataResponse) + metadataResponse.AddBroker("localhost:12345", 2) + metadataResponse.AddTopicPartition("my_topic", 0, 2, replicas, isr, ErrNoError) + metadataResponse.AddTopicPartition("my_topic", 1, 2, replicas, isr, ErrLeaderNotAvailable) + seedBroker.Returns(metadataResponse) + + config := NewConfig() + config.Metadata.Retry.Max = 0 + c, err := NewClient([]string{seedBroker.Addr()}, config) + if err != nil { + t.Fatal(err) + } + client := c.(*client) + + // Verify they aren't cached the same + allP := client.cachedPartitionsResults["my_topic"][allPartitions] + writeP := client.cachedPartitionsResults["my_topic"][writablePartitions] + if len(allP) == len(writeP) { + t.Fatal("Invalid lengths!") + } + + tmp := client.cachedPartitionsResults["my_topic"] + // Verify we actually use the cache at all! + tmp[allPartitions] = []int32{1, 2, 3, 4} + client.cachedPartitionsResults["my_topic"] = tmp + if 4 != len(client.cachedPartitions("my_topic", allPartitions)) { + t.Fatal("Not using the cache!") + } + + seedBroker.Close() + safeClose(t, client) +} + +func TestClientDoesntCachePartitionsForTopicsWithErrors(t *testing.T) { + seedBroker := newMockBroker(t, 1) + + replicas := []int32{seedBroker.BrokerID()} + + metadataResponse := new(MetadataResponse) + metadataResponse.AddBroker(seedBroker.Addr(), seedBroker.BrokerID()) + metadataResponse.AddTopicPartition("my_topic", 1, replicas[0], replicas, replicas, ErrNoError) + metadataResponse.AddTopicPartition("my_topic", 2, replicas[0], replicas, replicas, ErrNoError) + seedBroker.Returns(metadataResponse) + + config := NewConfig() + config.Metadata.Retry.Max = 0 + client, err := NewClient([]string{seedBroker.Addr()}, config) + if err != nil { + t.Fatal(err) + } + + metadataResponse = new(MetadataResponse) + metadataResponse.AddTopic("unknown", ErrUnknownTopicOrPartition) + seedBroker.Returns(metadataResponse) + + partitions, err := client.Partitions("unknown") + + if err != ErrUnknownTopicOrPartition { + t.Error("Expected ErrUnknownTopicOrPartition, found", err) + } + if partitions != nil { + t.Errorf("Should return nil as partition list, found %v", partitions) + } + + // Should still use the cache of a known topic + partitions, err = client.Partitions("my_topic") + if err != nil { + t.Errorf("Expected no error, found %v", err) + } + + metadataResponse = new(MetadataResponse) + metadataResponse.AddTopic("unknown", ErrUnknownTopicOrPartition) + seedBroker.Returns(metadataResponse) + + // Should not use cache for unknown topic + partitions, err = client.Partitions("unknown") + if err != ErrUnknownTopicOrPartition { + t.Error("Expected ErrUnknownTopicOrPartition, found", err) + } + if partitions != nil { + t.Errorf("Should return nil as partition list, found %v", partitions) + } + + seedBroker.Close() + safeClose(t, client) +} + +func TestClientSeedBrokers(t *testing.T) { + seedBroker := newMockBroker(t, 1) + + metadataResponse := new(MetadataResponse) + metadataResponse.AddBroker("localhost:12345", 2) + seedBroker.Returns(metadataResponse) + + client, err := NewClient([]string{seedBroker.Addr()}, nil) + if err != nil { + t.Fatal(err) + } + + seedBroker.Close() + safeClose(t, client) +} + +func TestClientMetadata(t *testing.T) { + seedBroker := newMockBroker(t, 1) + leader := newMockBroker(t, 5) + + replicas := []int32{3, 1, 5} + isr := []int32{5, 1} + + metadataResponse := new(MetadataResponse) + metadataResponse.AddBroker(leader.Addr(), leader.BrokerID()) + metadataResponse.AddTopicPartition("my_topic", 0, leader.BrokerID(), replicas, isr, ErrNoError) + metadataResponse.AddTopicPartition("my_topic", 1, leader.BrokerID(), replicas, isr, ErrLeaderNotAvailable) + seedBroker.Returns(metadataResponse) + + config := NewConfig() + config.Metadata.Retry.Max = 0 + client, err := NewClient([]string{seedBroker.Addr()}, config) + if err != nil { + t.Fatal(err) + } + + topics, err := client.Topics() + if err != nil { + t.Error(err) + } else if len(topics) != 1 || topics[0] != "my_topic" { + t.Error("Client returned incorrect topics:", topics) + } + + parts, err := client.Partitions("my_topic") + if err != nil { + t.Error(err) + } else if len(parts) != 2 || parts[0] != 0 || parts[1] != 1 { + t.Error("Client returned incorrect partitions for my_topic:", parts) + } + + parts, err = client.WritablePartitions("my_topic") + if err != nil { + t.Error(err) + } else if len(parts) != 1 || parts[0] != 0 { + t.Error("Client returned incorrect writable partitions for my_topic:", parts) + } + + tst, err := client.Leader("my_topic", 0) + if err != nil { + t.Error(err) + } else if tst.ID() != 5 { + t.Error("Leader for my_topic had incorrect ID.") + } + + replicas, err = client.Replicas("my_topic", 0) + if err != nil { + t.Error(err) + } else if replicas[0] != 1 { + t.Error("Incorrect (or unsorted) replica") + } else if replicas[1] != 3 { + t.Error("Incorrect (or unsorted) replica") + } else if replicas[2] != 5 { + t.Error("Incorrect (or unsorted) replica") + } + + leader.Close() + seedBroker.Close() + safeClose(t, client) +} + +func TestClientGetOffset(t *testing.T) { + seedBroker := newMockBroker(t, 1) + leader := newMockBroker(t, 2) + leaderAddr := leader.Addr() + + metadata := new(MetadataResponse) + metadata.AddTopicPartition("foo", 0, leader.BrokerID(), nil, nil, ErrNoError) + metadata.AddBroker(leaderAddr, leader.BrokerID()) + seedBroker.Returns(metadata) + + client, err := NewClient([]string{seedBroker.Addr()}, nil) + if err != nil { + t.Fatal(err) + } + + offsetResponse := new(OffsetResponse) + offsetResponse.AddTopicPartition("foo", 0, 123) + leader.Returns(offsetResponse) + + offset, err := client.GetOffset("foo", 0, OffsetNewest) + if err != nil { + t.Error(err) + } + if offset != 123 { + t.Error("Unexpected offset, got ", offset) + } + + leader.Close() + seedBroker.Returns(metadata) + + leader = newMockBrokerAddr(t, 2, leaderAddr) + offsetResponse = new(OffsetResponse) + offsetResponse.AddTopicPartition("foo", 0, 456) + leader.Returns(offsetResponse) + + offset, err = client.GetOffset("foo", 0, OffsetNewest) + if err != nil { + t.Error(err) + } + if offset != 456 { + t.Error("Unexpected offset, got ", offset) + } + + seedBroker.Close() + leader.Close() + safeClose(t, client) +} + +func TestClientReceivingUnknownTopic(t *testing.T) { + seedBroker := newMockBroker(t, 1) + + metadataResponse1 := new(MetadataResponse) + seedBroker.Returns(metadataResponse1) + + config := NewConfig() + config.Metadata.Retry.Max = 1 + config.Metadata.Retry.Backoff = 0 + client, err := NewClient([]string{seedBroker.Addr()}, config) + if err != nil { + t.Fatal(err) + } + + metadataUnknownTopic := new(MetadataResponse) + metadataUnknownTopic.AddTopic("new_topic", ErrUnknownTopicOrPartition) + seedBroker.Returns(metadataUnknownTopic) + seedBroker.Returns(metadataUnknownTopic) + + if err := client.RefreshMetadata("new_topic"); err != ErrUnknownTopicOrPartition { + t.Error("ErrUnknownTopicOrPartition expected, got", err) + } + + // If we are asking for the leader of a partition of the non-existing topic. + // we will request metadata again. + seedBroker.Returns(metadataUnknownTopic) + seedBroker.Returns(metadataUnknownTopic) + + if _, err = client.Leader("new_topic", 1); err != ErrUnknownTopicOrPartition { + t.Error("Expected ErrUnknownTopicOrPartition, got", err) + } + + safeClose(t, client) + seedBroker.Close() +} + +func TestClientReceivingPartialMetadata(t *testing.T) { + seedBroker := newMockBroker(t, 1) + leader := newMockBroker(t, 5) + + metadataResponse1 := new(MetadataResponse) + metadataResponse1.AddBroker(leader.Addr(), leader.BrokerID()) + seedBroker.Returns(metadataResponse1) + + config := NewConfig() + config.Metadata.Retry.Max = 0 + client, err := NewClient([]string{seedBroker.Addr()}, config) + if err != nil { + t.Fatal(err) + } + + replicas := []int32{leader.BrokerID(), seedBroker.BrokerID()} + + metadataPartial := new(MetadataResponse) + metadataPartial.AddTopic("new_topic", ErrLeaderNotAvailable) + metadataPartial.AddTopicPartition("new_topic", 0, leader.BrokerID(), replicas, replicas, ErrNoError) + metadataPartial.AddTopicPartition("new_topic", 1, -1, replicas, []int32{}, ErrLeaderNotAvailable) + seedBroker.Returns(metadataPartial) + + if err := client.RefreshMetadata("new_topic"); err != nil { + t.Error("ErrLeaderNotAvailable should not make RefreshMetadata respond with an error") + } + + // Even though the metadata was incomplete, we should be able to get the leader of a partition + // for which we did get a useful response, without doing additional requests. + + partition0Leader, err := client.Leader("new_topic", 0) + if err != nil { + t.Error(err) + } else if partition0Leader.Addr() != leader.Addr() { + t.Error("Unexpected leader returned", partition0Leader.Addr()) + } + + // If we are asking for the leader of a partition that didn't have a leader before, + // we will do another metadata request. + + seedBroker.Returns(metadataPartial) + + // Still no leader for the partition, so asking for it should return an error. + _, err = client.Leader("new_topic", 1) + if err != ErrLeaderNotAvailable { + t.Error("Expected ErrLeaderNotAvailable, got", err) + } + + safeClose(t, client) + seedBroker.Close() + leader.Close() +} + +func TestClientRefreshBehaviour(t *testing.T) { + seedBroker := newMockBroker(t, 1) + leader := newMockBroker(t, 5) + + metadataResponse1 := new(MetadataResponse) + metadataResponse1.AddBroker(leader.Addr(), leader.BrokerID()) + seedBroker.Returns(metadataResponse1) + + metadataResponse2 := new(MetadataResponse) + metadataResponse2.AddTopicPartition("my_topic", 0xb, leader.BrokerID(), nil, nil, ErrNoError) + seedBroker.Returns(metadataResponse2) + + client, err := NewClient([]string{seedBroker.Addr()}, nil) + if err != nil { + t.Fatal(err) + } + + parts, err := client.Partitions("my_topic") + if err != nil { + t.Error(err) + } else if len(parts) != 1 || parts[0] != 0xb { + t.Error("Client returned incorrect partitions for my_topic:", parts) + } + + tst, err := client.Leader("my_topic", 0xb) + if err != nil { + t.Error(err) + } else if tst.ID() != 5 { + t.Error("Leader for my_topic had incorrect ID.") + } + + leader.Close() + seedBroker.Close() + safeClose(t, client) +} + +func TestClientResurrectDeadSeeds(t *testing.T) { + initialSeed := newMockBroker(t, 0) + emptyMetadata := new(MetadataResponse) + initialSeed.Returns(emptyMetadata) + + conf := NewConfig() + conf.Metadata.Retry.Backoff = 0 + conf.Metadata.RefreshFrequency = 0 + c, err := NewClient([]string{initialSeed.Addr()}, conf) + if err != nil { + t.Fatal(err) + } + initialSeed.Close() + + client := c.(*client) + + seed1 := newMockBroker(t, 1) + seed2 := newMockBroker(t, 2) + seed3 := newMockBroker(t, 3) + addr1 := seed1.Addr() + addr2 := seed2.Addr() + addr3 := seed3.Addr() + + // Overwrite the seed brokers with a fixed ordering to make this test deterministic. + safeClose(t, client.seedBrokers[0]) + client.seedBrokers = []*Broker{NewBroker(addr1), NewBroker(addr2), NewBroker(addr3)} + client.deadSeeds = []*Broker{} + + wg := sync.WaitGroup{} + wg.Add(1) + go func() { + if err := client.RefreshMetadata(); err != nil { + t.Error(err) + } + wg.Done() + }() + seed1.Close() + seed2.Close() + + seed1 = newMockBrokerAddr(t, 1, addr1) + seed2 = newMockBrokerAddr(t, 2, addr2) + + seed3.Close() + + seed1.Close() + seed2.Returns(emptyMetadata) + + wg.Wait() + + if len(client.seedBrokers) != 2 { + t.Error("incorrect number of live seeds") + } + if len(client.deadSeeds) != 1 { + t.Error("incorrect number of dead seeds") + } + + safeClose(t, c) +} + +func TestClientCoordinatorWithConsumerOffsetsTopic(t *testing.T) { + seedBroker := newMockBroker(t, 1) + staleCoordinator := newMockBroker(t, 2) + freshCoordinator := newMockBroker(t, 3) + + replicas := []int32{staleCoordinator.BrokerID(), freshCoordinator.BrokerID()} + metadataResponse1 := new(MetadataResponse) + metadataResponse1.AddBroker(staleCoordinator.Addr(), staleCoordinator.BrokerID()) + metadataResponse1.AddBroker(freshCoordinator.Addr(), freshCoordinator.BrokerID()) + metadataResponse1.AddTopicPartition("__consumer_offsets", 0, replicas[0], replicas, replicas, ErrNoError) + seedBroker.Returns(metadataResponse1) + + client, err := NewClient([]string{seedBroker.Addr()}, nil) + if err != nil { + t.Fatal(err) + } + + coordinatorResponse1 := new(ConsumerMetadataResponse) + coordinatorResponse1.Err = ErrConsumerCoordinatorNotAvailable + seedBroker.Returns(coordinatorResponse1) + + coordinatorResponse2 := new(ConsumerMetadataResponse) + coordinatorResponse2.CoordinatorID = staleCoordinator.BrokerID() + coordinatorResponse2.CoordinatorHost = "127.0.0.1" + coordinatorResponse2.CoordinatorPort = staleCoordinator.Port() + + seedBroker.Returns(coordinatorResponse2) + + broker, err := client.Coordinator("my_group") + if err != nil { + t.Error(err) + } + + if staleCoordinator.Addr() != broker.Addr() { + t.Errorf("Expected coordinator to have address %s, found %s", staleCoordinator.Addr(), broker.Addr()) + } + + if staleCoordinator.BrokerID() != broker.ID() { + t.Errorf("Expected coordinator to have ID %d, found %d", staleCoordinator.BrokerID(), broker.ID()) + } + + // Grab the cached value + broker2, err := client.Coordinator("my_group") + if err != nil { + t.Error(err) + } + + if broker2.Addr() != broker.Addr() { + t.Errorf("Expected the coordinator to be the same, but found %s vs. %s", broker2.Addr(), broker.Addr()) + } + + coordinatorResponse3 := new(ConsumerMetadataResponse) + coordinatorResponse3.CoordinatorID = freshCoordinator.BrokerID() + coordinatorResponse3.CoordinatorHost = "127.0.0.1" + coordinatorResponse3.CoordinatorPort = freshCoordinator.Port() + + seedBroker.Returns(coordinatorResponse3) + + // Refresh the locally cahced value because it's stale + if err := client.RefreshCoordinator("my_group"); err != nil { + t.Error(err) + } + + // Grab the fresh value + broker3, err := client.Coordinator("my_group") + if err != nil { + t.Error(err) + } + + if broker3.Addr() != freshCoordinator.Addr() { + t.Errorf("Expected the freshCoordinator to be returned, but found %s.", broker3.Addr()) + } + + freshCoordinator.Close() + staleCoordinator.Close() + seedBroker.Close() + safeClose(t, client) +} + +func TestClientCoordinatorWithoutConsumerOffsetsTopic(t *testing.T) { + seedBroker := newMockBroker(t, 1) + coordinator := newMockBroker(t, 2) + + metadataResponse1 := new(MetadataResponse) + seedBroker.Returns(metadataResponse1) + + config := NewConfig() + config.Metadata.Retry.Max = 1 + config.Metadata.Retry.Backoff = 0 + client, err := NewClient([]string{seedBroker.Addr()}, config) + if err != nil { + t.Fatal(err) + } + + coordinatorResponse1 := new(ConsumerMetadataResponse) + coordinatorResponse1.Err = ErrConsumerCoordinatorNotAvailable + seedBroker.Returns(coordinatorResponse1) + + metadataResponse2 := new(MetadataResponse) + metadataResponse2.AddTopic("__consumer_offsets", ErrUnknownTopicOrPartition) + seedBroker.Returns(metadataResponse2) + + replicas := []int32{coordinator.BrokerID()} + metadataResponse3 := new(MetadataResponse) + metadataResponse3.AddTopicPartition("__consumer_offsets", 0, replicas[0], replicas, replicas, ErrNoError) + seedBroker.Returns(metadataResponse3) + + coordinatorResponse2 := new(ConsumerMetadataResponse) + coordinatorResponse2.CoordinatorID = coordinator.BrokerID() + coordinatorResponse2.CoordinatorHost = "127.0.0.1" + coordinatorResponse2.CoordinatorPort = coordinator.Port() + + seedBroker.Returns(coordinatorResponse2) + + broker, err := client.Coordinator("my_group") + if err != nil { + t.Error(err) + } + + if coordinator.Addr() != broker.Addr() { + t.Errorf("Expected coordinator to have address %s, found %s", coordinator.Addr(), broker.Addr()) + } + + if coordinator.BrokerID() != broker.ID() { + t.Errorf("Expected coordinator to have ID %d, found %d", coordinator.BrokerID(), broker.ID()) + } + + coordinator.Close() + seedBroker.Close() + safeClose(t, client) +} + +func TestClientAutorefreshShutdownRace(t *testing.T) { + seedBroker := newMockBroker(t, 1) + + metadataResponse := new(MetadataResponse) + seedBroker.Returns(metadataResponse) + + conf := NewConfig() + conf.Metadata.RefreshFrequency = 100 * time.Millisecond + client, err := NewClient([]string{seedBroker.Addr()}, conf) + if err != nil { + t.Fatal(err) + } + + // Wait for the background refresh to kick in + time.Sleep(110 * time.Millisecond) + + done := make(chan none) + go func() { + // Close the client + if err := client.Close(); err != nil { + t.Fatal(err) + } + close(done) + }() + + // Wait for the Close to kick in + time.Sleep(10 * time.Millisecond) + + // Then return some metadata to the still-running background thread + leader := newMockBroker(t, 2) + metadataResponse.AddBroker(leader.Addr(), leader.BrokerID()) + metadataResponse.AddTopicPartition("foo", 0, leader.BrokerID(), []int32{2}, []int32{2}, ErrNoError) + seedBroker.Returns(metadataResponse) + + <-done + + seedBroker.Close() + + // give the update time to happen so we get a panic if it's still running (which it shouldn't) + time.Sleep(10 * time.Millisecond) +} diff --git a/vendor/github.com/Shopify/sarama/config.go b/vendor/github.com/Shopify/sarama/config.go new file mode 100644 index 000000000000..7cc91e76176c --- /dev/null +++ b/vendor/github.com/Shopify/sarama/config.go @@ -0,0 +1,342 @@ +package sarama + +import ( + "crypto/tls" + "time" +) + +// Config is used to pass multiple configuration options to Sarama's constructors. +type Config struct { + // Net is the namespace for network-level properties used by the Broker, and + // shared by the Client/Producer/Consumer. + Net struct { + // How many outstanding requests a connection is allowed to have before + // sending on it blocks (default 5). + MaxOpenRequests int + + // All three of the below configurations are similar to the + // `socket.timeout.ms` setting in JVM kafka. All of them default + // to 30 seconds. + DialTimeout time.Duration // How long to wait for the initial connection. + ReadTimeout time.Duration // How long to wait for a response. + WriteTimeout time.Duration // How long to wait for a transmit. + + TLS struct { + // Whether or not to use TLS when connecting to the broker + // (defaults to false). + Enable bool + // The TLS configuration to use for secure connections if + // enabled (defaults to nil). + Config *tls.Config + } + + // KeepAlive specifies the keep-alive period for an active network connection. + // If zero, keep-alives are disabled. (default is 0: disabled). + KeepAlive time.Duration + } + + // Metadata is the namespace for metadata management properties used by the + // Client, and shared by the Producer/Consumer. + Metadata struct { + Retry struct { + // The total number of times to retry a metadata request when the + // cluster is in the middle of a leader election (default 3). + Max int + // How long to wait for leader election to occur before retrying + // (default 250ms). Similar to the JVM's `retry.backoff.ms`. + Backoff time.Duration + } + // How frequently to refresh the cluster metadata in the background. + // Defaults to 10 minutes. Set to 0 to disable. Similar to + // `topic.metadata.refresh.interval.ms` in the JVM version. + RefreshFrequency time.Duration + } + + // Producer is the namespace for configuration related to producing messages, + // used by the Producer. + Producer struct { + // The maximum permitted size of a message (defaults to 1000000). Should be + // set equal to or smaller than the broker's `message.max.bytes`. + MaxMessageBytes int + // The level of acknowledgement reliability needed from the broker (defaults + // to WaitForLocal). Equivalent to the `request.required.acks` setting of the + // JVM producer. + RequiredAcks RequiredAcks + // The maximum duration the broker will wait the receipt of the number of + // RequiredAcks (defaults to 10 seconds). This is only relevant when + // RequiredAcks is set to WaitForAll or a number > 1. Only supports + // millisecond resolution, nanoseconds will be truncated. Equivalent to + // the JVM producer's `request.timeout.ms` setting. + Timeout time.Duration + // The type of compression to use on messages (defaults to no compression). + // Similar to `compression.codec` setting of the JVM producer. + Compression CompressionCodec + // Generates partitioners for choosing the partition to send messages to + // (defaults to hashing the message key). Similar to the `partitioner.class` + // setting for the JVM producer. + Partitioner PartitionerConstructor + + // Return specifies what channels will be populated. If they are set to true, + // you must read from the respective channels to prevent deadlock. + Return struct { + // If enabled, successfully delivered messages will be returned on the + // Successes channel (default disabled). + Successes bool + + // If enabled, messages that failed to deliver will be returned on the + // Errors channel, including error (default enabled). + Errors bool + } + + // The following config options control how often messages are batched up and + // sent to the broker. By default, messages are sent as fast as possible, and + // all messages received while the current batch is in-flight are placed + // into the subsequent batch. + Flush struct { + // The best-effort number of bytes needed to trigger a flush. Use the + // global sarama.MaxRequestSize to set a hard upper limit. + Bytes int + // The best-effort number of messages needed to trigger a flush. Use + // `MaxMessages` to set a hard upper limit. + Messages int + // The best-effort frequency of flushes. Equivalent to + // `queue.buffering.max.ms` setting of JVM producer. + Frequency time.Duration + // The maximum number of messages the producer will send in a single + // broker request. Defaults to 0 for unlimited. Similar to + // `queue.buffering.max.messages` in the JVM producer. + MaxMessages int + } + + Retry struct { + // The total number of times to retry sending a message (default 3). + // Similar to the `message.send.max.retries` setting of the JVM producer. + Max int + // How long to wait for the cluster to settle between retries + // (default 100ms). Similar to the `retry.backoff.ms` setting of the + // JVM producer. + Backoff time.Duration + } + } + + // Consumer is the namespace for configuration related to consuming messages, + // used by the Consumer. + Consumer struct { + Retry struct { + // How long to wait after a failing to read from a partition before + // trying again (default 2s). + Backoff time.Duration + } + + // Fetch is the namespace for controlling how many bytes are retrieved by any + // given request. + Fetch struct { + // The minimum number of message bytes to fetch in a request - the broker + // will wait until at least this many are available. The default is 1, + // as 0 causes the consumer to spin when no messages are available. + // Equivalent to the JVM's `fetch.min.bytes`. + Min int32 + // The default number of message bytes to fetch from the broker in each + // request (default 32768). This should be larger than the majority of + // your messages, or else the consumer will spend a lot of time + // negotiating sizes and not actually consuming. Similar to the JVM's + // `fetch.message.max.bytes`. + Default int32 + // The maximum number of message bytes to fetch from the broker in a + // single request. Messages larger than this will return + // ErrMessageTooLarge and will not be consumable, so you must be sure + // this is at least as large as your largest message. Defaults to 0 + // (no limit). Similar to the JVM's `fetch.message.max.bytes`. The + // global `sarama.MaxResponseSize` still applies. + Max int32 + } + // The maximum amount of time the broker will wait for Consumer.Fetch.Min + // bytes to become available before it returns fewer than that anyways. The + // default is 250ms, since 0 causes the consumer to spin when no events are + // available. 100-500ms is a reasonable range for most cases. Kafka only + // supports precision up to milliseconds; nanoseconds will be truncated. + // Equivalent to the JVM's `fetch.wait.max.ms`. + MaxWaitTime time.Duration + + // The maximum amount of time the consumer expects a message takes to process + // for the user. If writing to the Messages channel takes longer than this, + // that partition will stop fetching more messages until it can proceed again. + // Note that, since the Messages channel is buffered, the actual grace time is + // (MaxProcessingTime * ChanneBufferSize). Defaults to 100ms. + MaxProcessingTime time.Duration + + // Return specifies what channels will be populated. If they are set to true, + // you must read from them to prevent deadlock. + Return struct { + // If enabled, any errors that occured while consuming are returned on + // the Errors channel (default disabled). + Errors bool + } + + // Offsets specifies configuration for how and when to commit consumed + // offsets. This currently requires the manual use of an OffsetManager + // but will eventually be automated. + Offsets struct { + // How frequently to commit updated offsets. Defaults to 1s. + CommitInterval time.Duration + + // The initial offset to use if no offset was previously committed. + // Should be OffsetNewest or OffsetOldest. Defaults to OffsetNewest. + Initial int64 + } + } + + // A user-provided string sent with every request to the brokers for logging, + // debugging, and auditing purposes. Defaults to "sarama", but you should + // probably set it to something specific to your application. + ClientID string + // The number of events to buffer in internal and external channels. This + // permits the producer and consumer to continue processing some messages + // in the background while user code is working, greatly improving throughput. + // Defaults to 256. + ChannelBufferSize int +} + +// NewConfig returns a new configuration instance with sane defaults. +func NewConfig() *Config { + c := &Config{} + + c.Net.MaxOpenRequests = 5 + c.Net.DialTimeout = 30 * time.Second + c.Net.ReadTimeout = 30 * time.Second + c.Net.WriteTimeout = 30 * time.Second + + c.Metadata.Retry.Max = 3 + c.Metadata.Retry.Backoff = 250 * time.Millisecond + c.Metadata.RefreshFrequency = 10 * time.Minute + + c.Producer.MaxMessageBytes = 1000000 + c.Producer.RequiredAcks = WaitForLocal + c.Producer.Timeout = 10 * time.Second + c.Producer.Partitioner = NewHashPartitioner + c.Producer.Retry.Max = 3 + c.Producer.Retry.Backoff = 100 * time.Millisecond + c.Producer.Return.Errors = true + + c.Consumer.Fetch.Min = 1 + c.Consumer.Fetch.Default = 32768 + c.Consumer.Retry.Backoff = 2 * time.Second + c.Consumer.MaxWaitTime = 250 * time.Millisecond + c.Consumer.MaxProcessingTime = 100 * time.Millisecond + c.Consumer.Return.Errors = false + c.Consumer.Offsets.CommitInterval = 1 * time.Second + c.Consumer.Offsets.Initial = OffsetNewest + + c.ChannelBufferSize = 256 + + return c +} + +// Validate checks a Config instance. It will return a +// ConfigurationError if the specified values don't make sense. +func (c *Config) Validate() error { + // some configuration values should be warned on but not fail completely, do those first + if c.Net.TLS.Enable == false && c.Net.TLS.Config != nil { + Logger.Println("Net.TLS is disabled but a non-nil configuration was provided.") + } + if c.Producer.RequiredAcks > 1 { + Logger.Println("Producer.RequiredAcks > 1 is deprecated and will raise an exception with kafka >= 0.8.2.0.") + } + if c.Producer.MaxMessageBytes >= int(MaxRequestSize) { + Logger.Println("Producer.MaxMessageBytes is larger than MaxRequestSize; it will be ignored.") + } + if c.Producer.Flush.Bytes >= int(MaxRequestSize) { + Logger.Println("Producer.Flush.Bytes is larger than MaxRequestSize; it will be ignored.") + } + if c.Producer.Timeout%time.Millisecond != 0 { + Logger.Println("Producer.Timeout only supports millisecond resolution; nanoseconds will be truncated.") + } + if c.Consumer.MaxWaitTime < 100*time.Millisecond { + Logger.Println("Consumer.MaxWaitTime is very low, which can cause high CPU and network usage. See documentation for details.") + } + if c.Consumer.MaxWaitTime%time.Millisecond != 0 { + Logger.Println("Consumer.MaxWaitTime only supports millisecond precision; nanoseconds will be truncated.") + } + if c.ClientID == "sarama" { + Logger.Println("ClientID is the default of 'sarama', you should consider setting it to something application-specific.") + } + + // validate Net values + switch { + case c.Net.MaxOpenRequests <= 0: + return ConfigurationError("Net.MaxOpenRequests must be > 0") + case c.Net.DialTimeout <= 0: + return ConfigurationError("Net.DialTimeout must be > 0") + case c.Net.ReadTimeout <= 0: + return ConfigurationError("Net.ReadTimeout must be > 0") + case c.Net.WriteTimeout <= 0: + return ConfigurationError("Net.WriteTimeout must be > 0") + case c.Net.KeepAlive < 0: + return ConfigurationError("Net.KeepAlive must be >= 0") + } + + // validate the Metadata values + switch { + case c.Metadata.Retry.Max < 0: + return ConfigurationError("Metadata.Retry.Max must be >= 0") + case c.Metadata.Retry.Backoff < 0: + return ConfigurationError("Metadata.Retry.Backoff must be >= 0") + case c.Metadata.RefreshFrequency < 0: + return ConfigurationError("Metadata.RefreshFrequency must be >= 0") + } + + // validate the Producer values + switch { + case c.Producer.MaxMessageBytes <= 0: + return ConfigurationError("Producer.MaxMessageBytes must be > 0") + case c.Producer.RequiredAcks < -1: + return ConfigurationError("Producer.RequiredAcks must be >= -1") + case c.Producer.Timeout <= 0: + return ConfigurationError("Producer.Timeout must be > 0") + case c.Producer.Partitioner == nil: + return ConfigurationError("Producer.Partitioner must not be nil") + case c.Producer.Flush.Bytes < 0: + return ConfigurationError("Producer.Flush.Bytes must be >= 0") + case c.Producer.Flush.Messages < 0: + return ConfigurationError("Producer.Flush.Messages must be >= 0") + case c.Producer.Flush.Frequency < 0: + return ConfigurationError("Producer.Flush.Frequency must be >= 0") + case c.Producer.Flush.MaxMessages < 0: + return ConfigurationError("Producer.Flush.MaxMessages must be >= 0") + case c.Producer.Flush.MaxMessages > 0 && c.Producer.Flush.MaxMessages < c.Producer.Flush.Messages: + return ConfigurationError("Producer.Flush.MaxMessages must be >= Producer.Flush.Messages when set") + case c.Producer.Retry.Max < 0: + return ConfigurationError("Producer.Retry.Max must be >= 0") + case c.Producer.Retry.Backoff < 0: + return ConfigurationError("Producer.Retry.Backoff must be >= 0") + } + + // validate the Consumer values + switch { + case c.Consumer.Fetch.Min <= 0: + return ConfigurationError("Consumer.Fetch.Min must be > 0") + case c.Consumer.Fetch.Default <= 0: + return ConfigurationError("Consumer.Fetch.Default must be > 0") + case c.Consumer.Fetch.Max < 0: + return ConfigurationError("Consumer.Fetch.Max must be >= 0") + case c.Consumer.MaxWaitTime < 1*time.Millisecond: + return ConfigurationError("Consumer.MaxWaitTime must be >= 1ms") + case c.Consumer.MaxProcessingTime <= 0: + return ConfigurationError("Consumer.MaxProcessingTime must be > 0") + case c.Consumer.Retry.Backoff < 0: + return ConfigurationError("Consumer.Retry.Backoff must be >= 0") + case c.Consumer.Offsets.CommitInterval <= 0: + return ConfigurationError("Consumer.Offsets.CommitInterval must be > 0") + case c.Consumer.Offsets.Initial != OffsetOldest && c.Consumer.Offsets.Initial != OffsetNewest: + return ConfigurationError("Consumer.Offsets.Initial must be OffsetOldest or OffsetNewest") + + } + + // validate misc shared values + switch { + case c.ChannelBufferSize < 0: + return ConfigurationError("ChannelBufferSize must be >= 0") + } + + return nil +} diff --git a/vendor/github.com/Shopify/sarama/config_test.go b/vendor/github.com/Shopify/sarama/config_test.go new file mode 100644 index 000000000000..255281a6554c --- /dev/null +++ b/vendor/github.com/Shopify/sarama/config_test.go @@ -0,0 +1,10 @@ +package sarama + +import "testing" + +func TestDefaultConfigValidates(t *testing.T) { + config := NewConfig() + if err := config.Validate(); err != nil { + t.Error(err) + } +} diff --git a/vendor/github.com/Shopify/sarama/consumer.go b/vendor/github.com/Shopify/sarama/consumer.go new file mode 100644 index 000000000000..877fb041fc14 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/consumer.go @@ -0,0 +1,690 @@ +package sarama + +import ( + "errors" + "fmt" + "sync" + "sync/atomic" + "time" +) + +// ConsumerMessage encapsulates a Kafka message returned by the consumer. +type ConsumerMessage struct { + Key, Value []byte + Topic string + Partition int32 + Offset int64 +} + +// ConsumerError is what is provided to the user when an error occurs. +// It wraps an error and includes the topic and partition. +type ConsumerError struct { + Topic string + Partition int32 + Err error +} + +func (ce ConsumerError) Error() string { + return fmt.Sprintf("kafka: error while consuming %s/%d: %s", ce.Topic, ce.Partition, ce.Err) +} + +// ConsumerErrors is a type that wraps a batch of errors and implements the Error interface. +// It can be returned from the PartitionConsumer's Close methods to avoid the need to manually drain errors +// when stopping. +type ConsumerErrors []*ConsumerError + +func (ce ConsumerErrors) Error() string { + return fmt.Sprintf("kafka: %d errors while consuming", len(ce)) +} + +// Consumer manages PartitionConsumers which process Kafka messages from brokers. You MUST call Close() +// on a consumer to avoid leaks, it will not be garbage-collected automatically when it passes out of +// scope. +// +// Sarama's Consumer type does not currently support automatic consumer group rebalancing and offset tracking, +// however the https://github.com/wvanbergen/kafka library builds on Sarama to add this support. We plan +// to properly integrate this functionality at a later date. +type Consumer interface { + + // Topics returns the set of available topics as retrieved from the cluster + // metadata. This method is the same as Client.Topics(), and is provided for + // convenience. + Topics() ([]string, error) + + // Partitions returns the sorted list of all partition IDs for the given topic. + // This method is the same as Client.Partitions(), and is provided for convenience. + Partitions(topic string) ([]int32, error) + + // ConsumePartition creates a PartitionConsumer on the given topic/partition with + // the given offset. It will return an error if this Consumer is already consuming + // on the given topic/partition. Offset can be a literal offset, or OffsetNewest + // or OffsetOldest + ConsumePartition(topic string, partition int32, offset int64) (PartitionConsumer, error) + + // Close shuts down the consumer. It must be called after all child + // PartitionConsumers have already been closed. + Close() error +} + +type consumer struct { + client Client + conf *Config + ownClient bool + + lock sync.Mutex + children map[string]map[int32]*partitionConsumer + brokerConsumers map[*Broker]*brokerConsumer +} + +// NewConsumer creates a new consumer using the given broker addresses and configuration. +func NewConsumer(addrs []string, config *Config) (Consumer, error) { + client, err := NewClient(addrs, config) + if err != nil { + return nil, err + } + + c, err := NewConsumerFromClient(client) + if err != nil { + return nil, err + } + c.(*consumer).ownClient = true + return c, nil +} + +// NewConsumerFromClient creates a new consumer using the given client. It is still +// necessary to call Close() on the underlying client when shutting down this consumer. +func NewConsumerFromClient(client Client) (Consumer, error) { + // Check that we are not dealing with a closed Client before processing any other arguments + if client.Closed() { + return nil, ErrClosedClient + } + + c := &consumer{ + client: client, + conf: client.Config(), + children: make(map[string]map[int32]*partitionConsumer), + brokerConsumers: make(map[*Broker]*brokerConsumer), + } + + return c, nil +} + +func (c *consumer) Close() error { + if c.ownClient { + return c.client.Close() + } + return nil +} + +func (c *consumer) Topics() ([]string, error) { + return c.client.Topics() +} + +func (c *consumer) Partitions(topic string) ([]int32, error) { + return c.client.Partitions(topic) +} + +func (c *consumer) ConsumePartition(topic string, partition int32, offset int64) (PartitionConsumer, error) { + child := &partitionConsumer{ + consumer: c, + conf: c.conf, + topic: topic, + partition: partition, + messages: make(chan *ConsumerMessage, c.conf.ChannelBufferSize), + errors: make(chan *ConsumerError, c.conf.ChannelBufferSize), + feeder: make(chan *FetchResponse, 1), + trigger: make(chan none, 1), + dying: make(chan none), + fetchSize: c.conf.Consumer.Fetch.Default, + } + + if err := child.chooseStartingOffset(offset); err != nil { + return nil, err + } + + var leader *Broker + var err error + if leader, err = c.client.Leader(child.topic, child.partition); err != nil { + return nil, err + } + + if err := c.addChild(child); err != nil { + return nil, err + } + + go withRecover(child.dispatcher) + go withRecover(child.responseFeeder) + + child.broker = c.refBrokerConsumer(leader) + child.broker.input <- child + + return child, nil +} + +func (c *consumer) addChild(child *partitionConsumer) error { + c.lock.Lock() + defer c.lock.Unlock() + + topicChildren := c.children[child.topic] + if topicChildren == nil { + topicChildren = make(map[int32]*partitionConsumer) + c.children[child.topic] = topicChildren + } + + if topicChildren[child.partition] != nil { + return ConfigurationError("That topic/partition is already being consumed") + } + + topicChildren[child.partition] = child + return nil +} + +func (c *consumer) removeChild(child *partitionConsumer) { + c.lock.Lock() + defer c.lock.Unlock() + + delete(c.children[child.topic], child.partition) +} + +func (c *consumer) refBrokerConsumer(broker *Broker) *brokerConsumer { + c.lock.Lock() + defer c.lock.Unlock() + + bc := c.brokerConsumers[broker] + if bc == nil { + bc = c.newBrokerConsumer(broker) + c.brokerConsumers[broker] = bc + } + + bc.refs++ + + return bc +} + +func (c *consumer) unrefBrokerConsumer(brokerWorker *brokerConsumer) { + c.lock.Lock() + defer c.lock.Unlock() + + brokerWorker.refs-- + + if brokerWorker.refs == 0 { + close(brokerWorker.input) + if c.brokerConsumers[brokerWorker.broker] == brokerWorker { + delete(c.brokerConsumers, brokerWorker.broker) + } + } +} + +func (c *consumer) abandonBrokerConsumer(brokerWorker *brokerConsumer) { + c.lock.Lock() + defer c.lock.Unlock() + + delete(c.brokerConsumers, brokerWorker.broker) +} + +// PartitionConsumer + +// PartitionConsumer processes Kafka messages from a given topic and partition. You MUST call Close() +// or AsyncClose() on a PartitionConsumer to avoid leaks, it will not be garbage-collected automatically +// when it passes out of scope. +// +// The simplest way of using a PartitionConsumer is to loop over its Messages channel using a for/range +// loop. The PartitionConsumer will only stop itself in one case: when the offset being consumed is reported +// as out of range by the brokers. In this case you should decide what you want to do (try a different offset, +// notify a human, etc) and handle it appropriately. For all other error cases, it will just keep retrying. +// By default, it logs these errors to sarama.Logger; if you want to be notified directly of all errors, set +// your config's Consumer.Return.Errors to true and read from the Errors channel, using a select statement +// or a separate goroutine. Check out the Consumer examples to see implementations of these different approaches. +type PartitionConsumer interface { + + // AsyncClose initiates a shutdown of the PartitionConsumer. This method will + // return immediately, after which you should wait until the 'messages' and + // 'errors' channel are drained. It is required to call this function, or + // Close before a consumer object passes out of scope, as it will otherwise + // leak memory. You must call this before calling Close on the underlying client. + AsyncClose() + + // Close stops the PartitionConsumer from fetching messages. It is required to + // call this function (or AsyncClose) before a consumer object passes out of + // scope, as it will otherwise leak memory. You must call this before calling + // Close on the underlying client. + Close() error + + // Messages returns the read channel for the messages that are returned by + // the broker. + Messages() <-chan *ConsumerMessage + + // Errors returns a read channel of errors that occured during consuming, if + // enabled. By default, errors are logged and not returned over this channel. + // If you want to implement any custom error handling, set your config's + // Consumer.Return.Errors setting to true, and read from this channel. + Errors() <-chan *ConsumerError + + // HighWaterMarkOffset returns the high water mark offset of the partition, + // i.e. the offset that will be used for the next message that will be produced. + // You can use this to determine how far behind the processing is. + HighWaterMarkOffset() int64 +} + +type partitionConsumer struct { + consumer *consumer + conf *Config + topic string + partition int32 + + broker *brokerConsumer + messages chan *ConsumerMessage + errors chan *ConsumerError + feeder chan *FetchResponse + + trigger, dying chan none + responseResult error + + fetchSize int32 + offset int64 + highWaterMarkOffset int64 +} + +var errTimedOut = errors.New("timed out feeding messages to the user") // not user-facing + +func (child *partitionConsumer) sendError(err error) { + cErr := &ConsumerError{ + Topic: child.topic, + Partition: child.partition, + Err: err, + } + + if child.conf.Consumer.Return.Errors { + child.errors <- cErr + } else { + Logger.Println(cErr) + } +} + +func (child *partitionConsumer) dispatcher() { + for _ = range child.trigger { + select { + case <-child.dying: + close(child.trigger) + case <-time.After(child.conf.Consumer.Retry.Backoff): + if child.broker != nil { + child.consumer.unrefBrokerConsumer(child.broker) + child.broker = nil + } + + Logger.Printf("consumer/%s/%d finding new broker\n", child.topic, child.partition) + if err := child.dispatch(); err != nil { + child.sendError(err) + child.trigger <- none{} + } + } + } + + if child.broker != nil { + child.consumer.unrefBrokerConsumer(child.broker) + } + child.consumer.removeChild(child) + close(child.feeder) +} + +func (child *partitionConsumer) dispatch() error { + if err := child.consumer.client.RefreshMetadata(child.topic); err != nil { + return err + } + + var leader *Broker + var err error + if leader, err = child.consumer.client.Leader(child.topic, child.partition); err != nil { + return err + } + + child.broker = child.consumer.refBrokerConsumer(leader) + + child.broker.input <- child + + return nil +} + +func (child *partitionConsumer) chooseStartingOffset(offset int64) error { + newestOffset, err := child.consumer.client.GetOffset(child.topic, child.partition, OffsetNewest) + if err != nil { + return err + } + oldestOffset, err := child.consumer.client.GetOffset(child.topic, child.partition, OffsetOldest) + if err != nil { + return err + } + + switch { + case offset == OffsetNewest: + child.offset = newestOffset + case offset == OffsetOldest: + child.offset = oldestOffset + case offset >= oldestOffset && offset <= newestOffset: + child.offset = offset + default: + return ErrOffsetOutOfRange + } + + return nil +} + +func (child *partitionConsumer) Messages() <-chan *ConsumerMessage { + return child.messages +} + +func (child *partitionConsumer) Errors() <-chan *ConsumerError { + return child.errors +} + +func (child *partitionConsumer) AsyncClose() { + // this triggers whatever broker owns this child to abandon it and close its trigger channel, which causes + // the dispatcher to exit its loop, which removes it from the consumer then closes its 'messages' and + // 'errors' channel (alternatively, if the child is already at the dispatcher for some reason, that will + // also just close itself) + close(child.dying) +} + +func (child *partitionConsumer) Close() error { + child.AsyncClose() + + go withRecover(func() { + for _ = range child.messages { + // drain + } + }) + + var errors ConsumerErrors + for err := range child.errors { + errors = append(errors, err) + } + + if len(errors) > 0 { + return errors + } + return nil +} + +func (child *partitionConsumer) HighWaterMarkOffset() int64 { + return atomic.LoadInt64(&child.highWaterMarkOffset) +} + +func (child *partitionConsumer) responseFeeder() { + var msgs []*ConsumerMessage + +feederLoop: + for response := range child.feeder { + msgs, child.responseResult = child.parseResponse(response) + + for i, msg := range msgs { + select { + case child.messages <- msg: + case <-time.After(child.conf.Consumer.MaxProcessingTime): + child.responseResult = errTimedOut + child.broker.acks.Done() + for _, msg = range msgs[i:] { + child.messages <- msg + } + child.broker.input <- child + continue feederLoop + } + } + + child.broker.acks.Done() + } + + close(child.messages) + close(child.errors) +} + +func (child *partitionConsumer) parseResponse(response *FetchResponse) ([]*ConsumerMessage, error) { + block := response.GetBlock(child.topic, child.partition) + if block == nil { + return nil, ErrIncompleteResponse + } + + if block.Err != ErrNoError { + return nil, block.Err + } + + if len(block.MsgSet.Messages) == 0 { + // We got no messages. If we got a trailing one then we need to ask for more data. + // Otherwise we just poll again and wait for one to be produced... + if block.MsgSet.PartialTrailingMessage { + if child.conf.Consumer.Fetch.Max > 0 && child.fetchSize == child.conf.Consumer.Fetch.Max { + // we can't ask for more data, we've hit the configured limit + child.sendError(ErrMessageTooLarge) + child.offset++ // skip this one so we can keep processing future messages + } else { + child.fetchSize *= 2 + if child.conf.Consumer.Fetch.Max > 0 && child.fetchSize > child.conf.Consumer.Fetch.Max { + child.fetchSize = child.conf.Consumer.Fetch.Max + } + } + } + + return nil, nil + } + + // we got messages, reset our fetch size in case it was increased for a previous request + child.fetchSize = child.conf.Consumer.Fetch.Default + atomic.StoreInt64(&child.highWaterMarkOffset, block.HighWaterMarkOffset) + + incomplete := false + prelude := true + var messages []*ConsumerMessage + for _, msgBlock := range block.MsgSet.Messages { + + for _, msg := range msgBlock.Messages() { + if prelude && msg.Offset < child.offset { + continue + } + prelude = false + + if msg.Offset >= child.offset { + messages = append(messages, &ConsumerMessage{ + Topic: child.topic, + Partition: child.partition, + Key: msg.Msg.Key, + Value: msg.Msg.Value, + Offset: msg.Offset, + }) + child.offset = msg.Offset + 1 + } else { + incomplete = true + } + } + + } + + if incomplete || len(messages) == 0 { + return nil, ErrIncompleteResponse + } + return messages, nil +} + +// brokerConsumer + +type brokerConsumer struct { + consumer *consumer + broker *Broker + input chan *partitionConsumer + newSubscriptions chan []*partitionConsumer + wait chan none + subscriptions map[*partitionConsumer]none + acks sync.WaitGroup + refs int +} + +func (c *consumer) newBrokerConsumer(broker *Broker) *brokerConsumer { + bc := &brokerConsumer{ + consumer: c, + broker: broker, + input: make(chan *partitionConsumer), + newSubscriptions: make(chan []*partitionConsumer), + wait: make(chan none), + subscriptions: make(map[*partitionConsumer]none), + refs: 0, + } + + go withRecover(bc.subscriptionManager) + go withRecover(bc.subscriptionConsumer) + + return bc +} + +func (bc *brokerConsumer) subscriptionManager() { + var buffer []*partitionConsumer + + // The subscriptionManager constantly accepts new subscriptions on `input` (even when the main subscriptionConsumer + // goroutine is in the middle of a network request) and batches it up. The main worker goroutine picks + // up a batch of new subscriptions between every network request by reading from `newSubscriptions`, so we give + // it nil if no new subscriptions are available. We also write to `wait` only when new subscriptions is available, + // so the main goroutine can block waiting for work if it has none. + for { + if len(buffer) > 0 { + select { + case event, ok := <-bc.input: + if !ok { + goto done + } + buffer = append(buffer, event) + case bc.newSubscriptions <- buffer: + buffer = nil + case bc.wait <- none{}: + } + } else { + select { + case event, ok := <-bc.input: + if !ok { + goto done + } + buffer = append(buffer, event) + case bc.newSubscriptions <- nil: + } + } + } + +done: + close(bc.wait) + if len(buffer) > 0 { + bc.newSubscriptions <- buffer + } + close(bc.newSubscriptions) +} + +func (bc *brokerConsumer) subscriptionConsumer() { + <-bc.wait // wait for our first piece of work + + // the subscriptionConsumer ensures we will get nil right away if no new subscriptions is available + for newSubscriptions := range bc.newSubscriptions { + bc.updateSubscriptions(newSubscriptions) + + if len(bc.subscriptions) == 0 { + // We're about to be shut down or we're about to receive more subscriptions. + // Either way, the signal just hasn't propagated to our goroutine yet. + <-bc.wait + continue + } + + response, err := bc.fetchNewMessages() + + if err != nil { + Logger.Printf("consumer/broker/%d disconnecting due to error processing FetchRequest: %s\n", bc.broker.ID(), err) + bc.abort(err) + return + } + + bc.acks.Add(len(bc.subscriptions)) + for child := range bc.subscriptions { + child.feeder <- response + } + bc.acks.Wait() + bc.handleResponses() + } +} + +func (bc *brokerConsumer) updateSubscriptions(newSubscriptions []*partitionConsumer) { + for _, child := range newSubscriptions { + bc.subscriptions[child] = none{} + Logger.Printf("consumer/broker/%d added subscription to %s/%d\n", bc.broker.ID(), child.topic, child.partition) + } + + for child := range bc.subscriptions { + select { + case <-child.dying: + Logger.Printf("consumer/broker/%d closed dead subscription to %s/%d\n", bc.broker.ID(), child.topic, child.partition) + close(child.trigger) + delete(bc.subscriptions, child) + default: + break + } + } +} + +func (bc *brokerConsumer) handleResponses() { + // handles the response codes left for us by our subscriptions, and abandons ones that have been closed + for child := range bc.subscriptions { + result := child.responseResult + child.responseResult = nil + + switch result { + case nil: + break + case errTimedOut: + Logger.Printf("consumer/broker/%d abandoned subscription to %s/%d because consuming was taking too long\n", + bc.broker.ID(), child.topic, child.partition) + delete(bc.subscriptions, child) + case ErrOffsetOutOfRange: + // there's no point in retrying this it will just fail the same way again + // shut it down and force the user to choose what to do + child.sendError(result) + Logger.Printf("consumer/%s/%d shutting down because %s\n", child.topic, child.partition, result) + close(child.trigger) + delete(bc.subscriptions, child) + case ErrUnknownTopicOrPartition, ErrNotLeaderForPartition, ErrLeaderNotAvailable: + // not an error, but does need redispatching + Logger.Printf("consumer/broker/%d abandoned subscription to %s/%d because %s\n", + bc.broker.ID(), child.topic, child.partition, result) + child.trigger <- none{} + delete(bc.subscriptions, child) + default: + // dunno, tell the user and try redispatching + child.sendError(result) + Logger.Printf("consumer/broker/%d abandoned subscription to %s/%d because %s\n", + bc.broker.ID(), child.topic, child.partition, result) + child.trigger <- none{} + delete(bc.subscriptions, child) + } + } +} + +func (bc *brokerConsumer) abort(err error) { + bc.consumer.abandonBrokerConsumer(bc) + _ = bc.broker.Close() // we don't care about the error this might return, we already have one + + for child := range bc.subscriptions { + child.sendError(err) + child.trigger <- none{} + } + + for newSubscription := range bc.newSubscriptions { + for _, child := range newSubscription { + child.sendError(err) + child.trigger <- none{} + } + } +} + +func (bc *brokerConsumer) fetchNewMessages() (*FetchResponse, error) { + request := &FetchRequest{ + MinBytes: bc.consumer.conf.Consumer.Fetch.Min, + MaxWaitTime: int32(bc.consumer.conf.Consumer.MaxWaitTime / time.Millisecond), + } + + for child := range bc.subscriptions { + request.AddBlock(child.topic, child.partition, child.offset, child.fetchSize) + } + + return bc.broker.Fetch(request) +} diff --git a/vendor/github.com/Shopify/sarama/consumer_group_members.go b/vendor/github.com/Shopify/sarama/consumer_group_members.go new file mode 100644 index 000000000000..9d92d350a5dc --- /dev/null +++ b/vendor/github.com/Shopify/sarama/consumer_group_members.go @@ -0,0 +1,94 @@ +package sarama + +type ConsumerGroupMemberMetadata struct { + Version int16 + Topics []string + UserData []byte +} + +func (m *ConsumerGroupMemberMetadata) encode(pe packetEncoder) error { + pe.putInt16(m.Version) + + if err := pe.putStringArray(m.Topics); err != nil { + return err + } + + if err := pe.putBytes(m.UserData); err != nil { + return err + } + + return nil +} + +func (m *ConsumerGroupMemberMetadata) decode(pd packetDecoder) (err error) { + if m.Version, err = pd.getInt16(); err != nil { + return + } + + if m.Topics, err = pd.getStringArray(); err != nil { + return + } + + if m.UserData, err = pd.getBytes(); err != nil { + return + } + + return nil +} + +type ConsumerGroupMemberAssignment struct { + Version int16 + Topics map[string][]int32 + UserData []byte +} + +func (m *ConsumerGroupMemberAssignment) encode(pe packetEncoder) error { + pe.putInt16(m.Version) + + if err := pe.putArrayLength(len(m.Topics)); err != nil { + return err + } + + for topic, partitions := range m.Topics { + if err := pe.putString(topic); err != nil { + return err + } + if err := pe.putInt32Array(partitions); err != nil { + return err + } + } + + if err := pe.putBytes(m.UserData); err != nil { + return err + } + + return nil +} + +func (m *ConsumerGroupMemberAssignment) decode(pd packetDecoder) (err error) { + if m.Version, err = pd.getInt16(); err != nil { + return + } + + var topicLen int + if topicLen, err = pd.getArrayLength(); err != nil { + return + } + + m.Topics = make(map[string][]int32, topicLen) + for i := 0; i < topicLen; i++ { + var topic string + if topic, err = pd.getString(); err != nil { + return + } + if m.Topics[topic], err = pd.getInt32Array(); err != nil { + return + } + } + + if m.UserData, err = pd.getBytes(); err != nil { + return + } + + return nil +} diff --git a/vendor/github.com/Shopify/sarama/consumer_group_members_test.go b/vendor/github.com/Shopify/sarama/consumer_group_members_test.go new file mode 100644 index 000000000000..1c1d154abbc8 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/consumer_group_members_test.go @@ -0,0 +1,73 @@ +package sarama + +import ( + "bytes" + "reflect" + "testing" +) + +var ( + groupMemberMetadata = []byte{ + 0, 1, // Version + 0, 0, 0, 2, // Topic array length + 0, 3, 'o', 'n', 'e', // Topic one + 0, 3, 't', 'w', 'o', // Topic two + 0, 0, 0, 3, 0x01, 0x02, 0x03, // Userdata + } + groupMemberAssignment = []byte{ + 0, 1, // Version + 0, 0, 0, 1, // Topic array length + 0, 3, 'o', 'n', 'e', // Topic one + 0, 0, 0, 3, // Topic one, partition array length + 0, 0, 0, 0, 0, 0, 0, 2, 0, 0, 0, 4, // 0, 2, 4 + 0, 0, 0, 3, 0x01, 0x02, 0x03, // Userdata + } +) + +func TestConsumerGroupMemberMetadata(t *testing.T) { + meta := &ConsumerGroupMemberMetadata{ + Version: 1, + Topics: []string{"one", "two"}, + UserData: []byte{0x01, 0x02, 0x03}, + } + + buf, err := encode(meta) + if err != nil { + t.Error("Failed to encode data", err) + } else if !bytes.Equal(groupMemberMetadata, buf) { + t.Errorf("Encoded data does not match expectation\nexpected: %v\nactual: %v", groupMemberMetadata, buf) + } + + meta2 := new(ConsumerGroupMemberMetadata) + err = decode(buf, meta2) + if err != nil { + t.Error("Failed to decode data", err) + } else if !reflect.DeepEqual(meta, meta2) { + t.Errorf("Encoded data does not match expectation\nexpected: %v\nactual: %v", meta, meta2) + } +} + +func TestConsumerGroupMemberAssignment(t *testing.T) { + amt := &ConsumerGroupMemberAssignment{ + Version: 1, + Topics: map[string][]int32{ + "one": []int32{0, 2, 4}, + }, + UserData: []byte{0x01, 0x02, 0x03}, + } + + buf, err := encode(amt) + if err != nil { + t.Error("Failed to encode data", err) + } else if !bytes.Equal(groupMemberAssignment, buf) { + t.Errorf("Encoded data does not match expectation\nexpected: %v\nactual: %v", groupMemberAssignment, buf) + } + + amt2 := new(ConsumerGroupMemberAssignment) + err = decode(buf, amt2) + if err != nil { + t.Error("Failed to decode data", err) + } else if !reflect.DeepEqual(amt, amt2) { + t.Errorf("Encoded data does not match expectation\nexpected: %v\nactual: %v", amt, amt2) + } +} diff --git a/vendor/github.com/Shopify/sarama/consumer_metadata_request.go b/vendor/github.com/Shopify/sarama/consumer_metadata_request.go new file mode 100644 index 000000000000..9b8fcd74e864 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/consumer_metadata_request.go @@ -0,0 +1,22 @@ +package sarama + +type ConsumerMetadataRequest struct { + ConsumerGroup string +} + +func (r *ConsumerMetadataRequest) encode(pe packetEncoder) error { + return pe.putString(r.ConsumerGroup) +} + +func (r *ConsumerMetadataRequest) decode(pd packetDecoder) (err error) { + r.ConsumerGroup, err = pd.getString() + return err +} + +func (r *ConsumerMetadataRequest) key() int16 { + return 10 +} + +func (r *ConsumerMetadataRequest) version() int16 { + return 0 +} diff --git a/vendor/github.com/Shopify/sarama/consumer_metadata_request_test.go b/vendor/github.com/Shopify/sarama/consumer_metadata_request_test.go new file mode 100644 index 000000000000..4509631a0402 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/consumer_metadata_request_test.go @@ -0,0 +1,19 @@ +package sarama + +import "testing" + +var ( + consumerMetadataRequestEmpty = []byte{ + 0x00, 0x00} + + consumerMetadataRequestString = []byte{ + 0x00, 0x06, 'f', 'o', 'o', 'b', 'a', 'r'} +) + +func TestConsumerMetadataRequest(t *testing.T) { + request := new(ConsumerMetadataRequest) + testRequest(t, "empty string", request, consumerMetadataRequestEmpty) + + request.ConsumerGroup = "foobar" + testRequest(t, "with string", request, consumerMetadataRequestString) +} diff --git a/vendor/github.com/Shopify/sarama/consumer_metadata_response.go b/vendor/github.com/Shopify/sarama/consumer_metadata_response.go new file mode 100644 index 000000000000..d6b5614b432f --- /dev/null +++ b/vendor/github.com/Shopify/sarama/consumer_metadata_response.go @@ -0,0 +1,73 @@ +package sarama + +import ( + "net" + "strconv" +) + +type ConsumerMetadataResponse struct { + Err KError + Coordinator *Broker + CoordinatorID int32 // deprecated: use Coordinator.ID() + CoordinatorHost string // deprecated: use Coordinator.Addr() + CoordinatorPort int32 // deprecated: use Coordinator.Addr() +} + +func (r *ConsumerMetadataResponse) decode(pd packetDecoder) (err error) { + tmp, err := pd.getInt16() + if err != nil { + return err + } + r.Err = KError(tmp) + + coordinator := new(Broker) + if err := coordinator.decode(pd); err != nil { + return err + } + if coordinator.addr == ":0" { + return nil + } + r.Coordinator = coordinator + + // this can all go away in 2.0, but we have to fill in deprecated fields to maintain + // backwards compatibility + host, portstr, err := net.SplitHostPort(r.Coordinator.Addr()) + if err != nil { + return err + } + port, err := strconv.ParseInt(portstr, 10, 32) + if err != nil { + return err + } + r.CoordinatorID = r.Coordinator.ID() + r.CoordinatorHost = host + r.CoordinatorPort = int32(port) + + return nil +} + +func (r *ConsumerMetadataResponse) encode(pe packetEncoder) error { + pe.putInt16(int16(r.Err)) + if r.Coordinator != nil { + host, portstr, err := net.SplitHostPort(r.Coordinator.Addr()) + if err != nil { + return err + } + port, err := strconv.ParseInt(portstr, 10, 32) + if err != nil { + return err + } + pe.putInt32(r.Coordinator.ID()) + if err := pe.putString(host); err != nil { + return err + } + pe.putInt32(int32(port)) + return nil + } + pe.putInt32(r.CoordinatorID) + if err := pe.putString(r.CoordinatorHost); err != nil { + return err + } + pe.putInt32(r.CoordinatorPort) + return nil +} diff --git a/vendor/github.com/Shopify/sarama/consumer_metadata_response_test.go b/vendor/github.com/Shopify/sarama/consumer_metadata_response_test.go new file mode 100644 index 000000000000..b748784d7efa --- /dev/null +++ b/vendor/github.com/Shopify/sarama/consumer_metadata_response_test.go @@ -0,0 +1,35 @@ +package sarama + +import "testing" + +var ( + consumerMetadataResponseError = []byte{ + 0x00, 0x0E, + 0x00, 0x00, 0x00, 0x00, + 0x00, 0x00, + 0x00, 0x00, 0x00, 0x00} + + consumerMetadataResponseSuccess = []byte{ + 0x00, 0x00, + 0x00, 0x00, 0x00, 0xAB, + 0x00, 0x03, 'f', 'o', 'o', + 0x00, 0x00, 0xCC, 0xDD} +) + +func TestConsumerMetadataResponseError(t *testing.T) { + response := ConsumerMetadataResponse{Err: ErrOffsetsLoadInProgress} + testResponse(t, "error", &response, consumerMetadataResponseError) +} + +func TestConsumerMetadataResponseSuccess(t *testing.T) { + broker := NewBroker("foo:52445") + broker.id = 0xAB + response := ConsumerMetadataResponse{ + Coordinator: broker, + CoordinatorID: 0xAB, + CoordinatorHost: "foo", + CoordinatorPort: 0xCCDD, + Err: ErrNoError, + } + testResponse(t, "success", &response, consumerMetadataResponseSuccess) +} diff --git a/vendor/github.com/Shopify/sarama/consumer_test.go b/vendor/github.com/Shopify/sarama/consumer_test.go new file mode 100644 index 000000000000..5286ef21209a --- /dev/null +++ b/vendor/github.com/Shopify/sarama/consumer_test.go @@ -0,0 +1,854 @@ +package sarama + +import ( + "log" + "os" + "os/signal" + "sync" + "testing" + "time" +) + +var testMsg = StringEncoder("Foo") + +// If a particular offset is provided then messages are consumed starting from +// that offset. +func TestConsumerOffsetManual(t *testing.T) { + // Given + broker0 := newMockBroker(t, 0) + + mockFetchResponse := newMockFetchResponse(t, 1) + for i := 0; i < 10; i++ { + mockFetchResponse.SetMessage("my_topic", 0, int64(i+1234), testMsg) + } + + broker0.SetHandlerByMap(map[string]MockResponse{ + "MetadataRequest": newMockMetadataResponse(t). + SetBroker(broker0.Addr(), broker0.BrokerID()). + SetLeader("my_topic", 0, broker0.BrokerID()), + "OffsetRequest": newMockOffsetResponse(t). + SetOffset("my_topic", 0, OffsetOldest, 0). + SetOffset("my_topic", 0, OffsetNewest, 2345), + "FetchRequest": mockFetchResponse, + }) + + // When + master, err := NewConsumer([]string{broker0.Addr()}, nil) + if err != nil { + t.Fatal(err) + } + + consumer, err := master.ConsumePartition("my_topic", 0, 1234) + if err != nil { + t.Fatal(err) + } + + // Then: messages starting from offset 1234 are consumed. + for i := 0; i < 10; i++ { + select { + case message := <-consumer.Messages(): + assertMessageOffset(t, message, int64(i+1234)) + case err := <-consumer.Errors(): + t.Error(err) + } + } + + safeClose(t, consumer) + safeClose(t, master) + broker0.Close() +} + +// If `OffsetNewest` is passed as the initial offset then the first consumed +// message is indeed corresponds to the offset that broker claims to be the +// newest in its metadata response. +func TestConsumerOffsetNewest(t *testing.T) { + // Given + broker0 := newMockBroker(t, 0) + broker0.SetHandlerByMap(map[string]MockResponse{ + "MetadataRequest": newMockMetadataResponse(t). + SetBroker(broker0.Addr(), broker0.BrokerID()). + SetLeader("my_topic", 0, broker0.BrokerID()), + "OffsetRequest": newMockOffsetResponse(t). + SetOffset("my_topic", 0, OffsetNewest, 10). + SetOffset("my_topic", 0, OffsetOldest, 7), + "FetchRequest": newMockFetchResponse(t, 1). + SetMessage("my_topic", 0, 9, testMsg). + SetMessage("my_topic", 0, 10, testMsg). + SetMessage("my_topic", 0, 11, testMsg). + SetHighWaterMark("my_topic", 0, 14), + }) + + master, err := NewConsumer([]string{broker0.Addr()}, nil) + if err != nil { + t.Fatal(err) + } + + // When + consumer, err := master.ConsumePartition("my_topic", 0, OffsetNewest) + if err != nil { + t.Fatal(err) + } + + // Then + assertMessageOffset(t, <-consumer.Messages(), 10) + if hwmo := consumer.HighWaterMarkOffset(); hwmo != 14 { + t.Errorf("Expected high water mark offset 14, found %d", hwmo) + } + + safeClose(t, consumer) + safeClose(t, master) + broker0.Close() +} + +// It is possible to close a partition consumer and create the same anew. +func TestConsumerRecreate(t *testing.T) { + // Given + broker0 := newMockBroker(t, 0) + broker0.SetHandlerByMap(map[string]MockResponse{ + "MetadataRequest": newMockMetadataResponse(t). + SetBroker(broker0.Addr(), broker0.BrokerID()). + SetLeader("my_topic", 0, broker0.BrokerID()), + "OffsetRequest": newMockOffsetResponse(t). + SetOffset("my_topic", 0, OffsetOldest, 0). + SetOffset("my_topic", 0, OffsetNewest, 1000), + "FetchRequest": newMockFetchResponse(t, 1). + SetMessage("my_topic", 0, 10, testMsg), + }) + + c, err := NewConsumer([]string{broker0.Addr()}, nil) + if err != nil { + t.Fatal(err) + } + + pc, err := c.ConsumePartition("my_topic", 0, 10) + if err != nil { + t.Fatal(err) + } + assertMessageOffset(t, <-pc.Messages(), 10) + + // When + safeClose(t, pc) + pc, err = c.ConsumePartition("my_topic", 0, 10) + if err != nil { + t.Fatal(err) + } + + // Then + assertMessageOffset(t, <-pc.Messages(), 10) + + safeClose(t, pc) + safeClose(t, c) + broker0.Close() +} + +// An attempt to consume the same partition twice should fail. +func TestConsumerDuplicate(t *testing.T) { + // Given + broker0 := newMockBroker(t, 0) + broker0.SetHandlerByMap(map[string]MockResponse{ + "MetadataRequest": newMockMetadataResponse(t). + SetBroker(broker0.Addr(), broker0.BrokerID()). + SetLeader("my_topic", 0, broker0.BrokerID()), + "OffsetRequest": newMockOffsetResponse(t). + SetOffset("my_topic", 0, OffsetOldest, 0). + SetOffset("my_topic", 0, OffsetNewest, 1000), + "FetchRequest": newMockFetchResponse(t, 1), + }) + + config := NewConfig() + config.ChannelBufferSize = 0 + c, err := NewConsumer([]string{broker0.Addr()}, config) + if err != nil { + t.Fatal(err) + } + + pc1, err := c.ConsumePartition("my_topic", 0, 0) + if err != nil { + t.Fatal(err) + } + + // When + pc2, err := c.ConsumePartition("my_topic", 0, 0) + + // Then + if pc2 != nil || err != ConfigurationError("That topic/partition is already being consumed") { + t.Fatal("A partition cannot be consumed twice at the same time") + } + + safeClose(t, pc1) + safeClose(t, c) + broker0.Close() +} + +// If consumer fails to refresh metadata it keeps retrying with frequency +// specified by `Config.Consumer.Retry.Backoff`. +func TestConsumerLeaderRefreshError(t *testing.T) { + // Given + broker0 := newMockBroker(t, 100) + + // Stage 1: my_topic/0 served by broker0 + Logger.Printf(" STAGE 1") + + broker0.SetHandlerByMap(map[string]MockResponse{ + "MetadataRequest": newMockMetadataResponse(t). + SetBroker(broker0.Addr(), broker0.BrokerID()). + SetLeader("my_topic", 0, broker0.BrokerID()), + "OffsetRequest": newMockOffsetResponse(t). + SetOffset("my_topic", 0, OffsetOldest, 123). + SetOffset("my_topic", 0, OffsetNewest, 1000), + "FetchRequest": newMockFetchResponse(t, 1). + SetMessage("my_topic", 0, 123, testMsg), + }) + + config := NewConfig() + config.Net.ReadTimeout = 100 * time.Millisecond + config.Consumer.Retry.Backoff = 200 * time.Millisecond + config.Consumer.Return.Errors = true + config.Metadata.Retry.Max = 0 + c, err := NewConsumer([]string{broker0.Addr()}, config) + if err != nil { + t.Fatal(err) + } + + pc, err := c.ConsumePartition("my_topic", 0, OffsetOldest) + if err != nil { + t.Fatal(err) + } + + assertMessageOffset(t, <-pc.Messages(), 123) + + // Stage 2: broker0 says that it is no longer the leader for my_topic/0, + // but the requests to retrieve metadata fail with network timeout. + Logger.Printf(" STAGE 2") + + fetchResponse2 := &FetchResponse{} + fetchResponse2.AddError("my_topic", 0, ErrNotLeaderForPartition) + + broker0.SetHandlerByMap(map[string]MockResponse{ + "FetchRequest": newMockWrapper(fetchResponse2), + }) + + if consErr := <-pc.Errors(); consErr.Err != ErrOutOfBrokers { + t.Errorf("Unexpected error: %v", consErr.Err) + } + + // Stage 3: finally the metadata returned by broker0 tells that broker1 is + // a new leader for my_topic/0. Consumption resumes. + + Logger.Printf(" STAGE 3") + + broker1 := newMockBroker(t, 101) + + broker1.SetHandlerByMap(map[string]MockResponse{ + "FetchRequest": newMockFetchResponse(t, 1). + SetMessage("my_topic", 0, 124, testMsg), + }) + broker0.SetHandlerByMap(map[string]MockResponse{ + "MetadataRequest": newMockMetadataResponse(t). + SetBroker(broker0.Addr(), broker0.BrokerID()). + SetBroker(broker1.Addr(), broker1.BrokerID()). + SetLeader("my_topic", 0, broker1.BrokerID()), + }) + + assertMessageOffset(t, <-pc.Messages(), 124) + + safeClose(t, pc) + safeClose(t, c) + broker1.Close() + broker0.Close() +} + +func TestConsumerInvalidTopic(t *testing.T) { + // Given + broker0 := newMockBroker(t, 100) + broker0.SetHandlerByMap(map[string]MockResponse{ + "MetadataRequest": newMockMetadataResponse(t). + SetBroker(broker0.Addr(), broker0.BrokerID()), + }) + + c, err := NewConsumer([]string{broker0.Addr()}, nil) + if err != nil { + t.Fatal(err) + } + + // When + pc, err := c.ConsumePartition("my_topic", 0, OffsetOldest) + + // Then + if pc != nil || err != ErrUnknownTopicOrPartition { + t.Errorf("Should fail with, err=%v", err) + } + + safeClose(t, c) + broker0.Close() +} + +// Nothing bad happens if a partition consumer that has no leader assigned at +// the moment is closed. +func TestConsumerClosePartitionWithoutLeader(t *testing.T) { + // Given + broker0 := newMockBroker(t, 100) + broker0.SetHandlerByMap(map[string]MockResponse{ + "MetadataRequest": newMockMetadataResponse(t). + SetBroker(broker0.Addr(), broker0.BrokerID()). + SetLeader("my_topic", 0, broker0.BrokerID()), + "OffsetRequest": newMockOffsetResponse(t). + SetOffset("my_topic", 0, OffsetOldest, 123). + SetOffset("my_topic", 0, OffsetNewest, 1000), + "FetchRequest": newMockFetchResponse(t, 1). + SetMessage("my_topic", 0, 123, testMsg), + }) + + config := NewConfig() + config.Net.ReadTimeout = 100 * time.Millisecond + config.Consumer.Retry.Backoff = 100 * time.Millisecond + config.Consumer.Return.Errors = true + config.Metadata.Retry.Max = 0 + c, err := NewConsumer([]string{broker0.Addr()}, config) + if err != nil { + t.Fatal(err) + } + + pc, err := c.ConsumePartition("my_topic", 0, OffsetOldest) + if err != nil { + t.Fatal(err) + } + + assertMessageOffset(t, <-pc.Messages(), 123) + + // broker0 says that it is no longer the leader for my_topic/0, but the + // requests to retrieve metadata fail with network timeout. + fetchResponse2 := &FetchResponse{} + fetchResponse2.AddError("my_topic", 0, ErrNotLeaderForPartition) + + broker0.SetHandlerByMap(map[string]MockResponse{ + "FetchRequest": newMockWrapper(fetchResponse2), + }) + + // When + if consErr := <-pc.Errors(); consErr.Err != ErrOutOfBrokers { + t.Errorf("Unexpected error: %v", consErr.Err) + } + + // Then: the partition consumer can be closed without any problem. + safeClose(t, pc) + safeClose(t, c) + broker0.Close() +} + +// If the initial offset passed on partition consumer creation is out of the +// actual offset range for the partition, then the partition consumer stops +// immediately closing its output channels. +func TestConsumerShutsDownOutOfRange(t *testing.T) { + // Given + broker0 := newMockBroker(t, 0) + fetchResponse := new(FetchResponse) + fetchResponse.AddError("my_topic", 0, ErrOffsetOutOfRange) + broker0.SetHandlerByMap(map[string]MockResponse{ + "MetadataRequest": newMockMetadataResponse(t). + SetBroker(broker0.Addr(), broker0.BrokerID()). + SetLeader("my_topic", 0, broker0.BrokerID()), + "OffsetRequest": newMockOffsetResponse(t). + SetOffset("my_topic", 0, OffsetNewest, 1234). + SetOffset("my_topic", 0, OffsetOldest, 7), + "FetchRequest": newMockWrapper(fetchResponse), + }) + + master, err := NewConsumer([]string{broker0.Addr()}, nil) + if err != nil { + t.Fatal(err) + } + + // When + consumer, err := master.ConsumePartition("my_topic", 0, 101) + if err != nil { + t.Fatal(err) + } + + // Then: consumer should shut down closing its messages and errors channels. + if _, ok := <-consumer.Messages(); ok { + t.Error("Expected the consumer to shut down") + } + safeClose(t, consumer) + + safeClose(t, master) + broker0.Close() +} + +// If a fetch response contains messages with offsets that are smaller then +// requested, then such messages are ignored. +func TestConsumerExtraOffsets(t *testing.T) { + // Given + broker0 := newMockBroker(t, 0) + fetchResponse1 := &FetchResponse{} + fetchResponse1.AddMessage("my_topic", 0, nil, testMsg, 1) + fetchResponse1.AddMessage("my_topic", 0, nil, testMsg, 2) + fetchResponse1.AddMessage("my_topic", 0, nil, testMsg, 3) + fetchResponse1.AddMessage("my_topic", 0, nil, testMsg, 4) + fetchResponse2 := &FetchResponse{} + fetchResponse2.AddError("my_topic", 0, ErrNoError) + broker0.SetHandlerByMap(map[string]MockResponse{ + "MetadataRequest": newMockMetadataResponse(t). + SetBroker(broker0.Addr(), broker0.BrokerID()). + SetLeader("my_topic", 0, broker0.BrokerID()), + "OffsetRequest": newMockOffsetResponse(t). + SetOffset("my_topic", 0, OffsetNewest, 1234). + SetOffset("my_topic", 0, OffsetOldest, 0), + "FetchRequest": newMockSequence(fetchResponse1, fetchResponse2), + }) + + master, err := NewConsumer([]string{broker0.Addr()}, nil) + if err != nil { + t.Fatal(err) + } + + // When + consumer, err := master.ConsumePartition("my_topic", 0, 3) + if err != nil { + t.Fatal(err) + } + + // Then: messages with offsets 1 and 2 are not returned even though they + // are present in the response. + assertMessageOffset(t, <-consumer.Messages(), 3) + assertMessageOffset(t, <-consumer.Messages(), 4) + + safeClose(t, consumer) + safeClose(t, master) + broker0.Close() +} + +// It is fine if offsets of fetched messages are not sequential (although +// strictly increasing!). +func TestConsumerNonSequentialOffsets(t *testing.T) { + // Given + broker0 := newMockBroker(t, 0) + fetchResponse1 := &FetchResponse{} + fetchResponse1.AddMessage("my_topic", 0, nil, testMsg, 5) + fetchResponse1.AddMessage("my_topic", 0, nil, testMsg, 7) + fetchResponse1.AddMessage("my_topic", 0, nil, testMsg, 11) + fetchResponse2 := &FetchResponse{} + fetchResponse2.AddError("my_topic", 0, ErrNoError) + broker0.SetHandlerByMap(map[string]MockResponse{ + "MetadataRequest": newMockMetadataResponse(t). + SetBroker(broker0.Addr(), broker0.BrokerID()). + SetLeader("my_topic", 0, broker0.BrokerID()), + "OffsetRequest": newMockOffsetResponse(t). + SetOffset("my_topic", 0, OffsetNewest, 1234). + SetOffset("my_topic", 0, OffsetOldest, 0), + "FetchRequest": newMockSequence(fetchResponse1, fetchResponse2), + }) + + master, err := NewConsumer([]string{broker0.Addr()}, nil) + if err != nil { + t.Fatal(err) + } + + // When + consumer, err := master.ConsumePartition("my_topic", 0, 3) + if err != nil { + t.Fatal(err) + } + + // Then: messages with offsets 1 and 2 are not returned even though they + // are present in the response. + assertMessageOffset(t, <-consumer.Messages(), 5) + assertMessageOffset(t, <-consumer.Messages(), 7) + assertMessageOffset(t, <-consumer.Messages(), 11) + + safeClose(t, consumer) + safeClose(t, master) + broker0.Close() +} + +// If leadership for a partition is changing then consumer resolves the new +// leader and switches to it. +func TestConsumerRebalancingMultiplePartitions(t *testing.T) { + // initial setup + seedBroker := newMockBroker(t, 10) + leader0 := newMockBroker(t, 0) + leader1 := newMockBroker(t, 1) + + seedBroker.SetHandlerByMap(map[string]MockResponse{ + "MetadataRequest": newMockMetadataResponse(t). + SetBroker(leader0.Addr(), leader0.BrokerID()). + SetBroker(leader1.Addr(), leader1.BrokerID()). + SetLeader("my_topic", 0, leader0.BrokerID()). + SetLeader("my_topic", 1, leader1.BrokerID()), + }) + + mockOffsetResponse1 := newMockOffsetResponse(t). + SetOffset("my_topic", 0, OffsetOldest, 0). + SetOffset("my_topic", 0, OffsetNewest, 1000). + SetOffset("my_topic", 1, OffsetOldest, 0). + SetOffset("my_topic", 1, OffsetNewest, 1000) + leader0.SetHandlerByMap(map[string]MockResponse{ + "OffsetRequest": mockOffsetResponse1, + "FetchRequest": newMockFetchResponse(t, 1), + }) + leader1.SetHandlerByMap(map[string]MockResponse{ + "OffsetRequest": mockOffsetResponse1, + "FetchRequest": newMockFetchResponse(t, 1), + }) + + // launch test goroutines + config := NewConfig() + config.Consumer.Retry.Backoff = 50 + master, err := NewConsumer([]string{seedBroker.Addr()}, config) + if err != nil { + t.Fatal(err) + } + + // we expect to end up (eventually) consuming exactly ten messages on each partition + var wg sync.WaitGroup + for i := int32(0); i < 2; i++ { + consumer, err := master.ConsumePartition("my_topic", i, 0) + if err != nil { + t.Error(err) + } + + go func(c PartitionConsumer) { + for err := range c.Errors() { + t.Error(err) + } + }(consumer) + + wg.Add(1) + go func(partition int32, c PartitionConsumer) { + for i := 0; i < 10; i++ { + message := <-consumer.Messages() + if message.Offset != int64(i) { + t.Error("Incorrect message offset!", i, partition, message.Offset) + } + if message.Partition != partition { + t.Error("Incorrect message partition!") + } + } + safeClose(t, consumer) + wg.Done() + }(i, consumer) + } + + time.Sleep(50 * time.Millisecond) + Logger.Printf(" STAGE 1") + // Stage 1: + // * my_topic/0 -> leader0 serves 4 messages + // * my_topic/1 -> leader1 serves 0 messages + + mockFetchResponse := newMockFetchResponse(t, 1) + for i := 0; i < 4; i++ { + mockFetchResponse.SetMessage("my_topic", 0, int64(i), testMsg) + } + leader0.SetHandlerByMap(map[string]MockResponse{ + "FetchRequest": mockFetchResponse, + }) + + time.Sleep(50 * time.Millisecond) + Logger.Printf(" STAGE 2") + // Stage 2: + // * leader0 says that it is no longer serving my_topic/0 + // * seedBroker tells that leader1 is serving my_topic/0 now + + // seed broker tells that the new partition 0 leader is leader1 + seedBroker.SetHandlerByMap(map[string]MockResponse{ + "MetadataRequest": newMockMetadataResponse(t). + SetLeader("my_topic", 0, leader1.BrokerID()). + SetLeader("my_topic", 1, leader1.BrokerID()), + }) + + // leader0 says no longer leader of partition 0 + fetchResponse := new(FetchResponse) + fetchResponse.AddError("my_topic", 0, ErrNotLeaderForPartition) + leader0.SetHandlerByMap(map[string]MockResponse{ + "FetchRequest": newMockWrapper(fetchResponse), + }) + + time.Sleep(50 * time.Millisecond) + Logger.Printf(" STAGE 3") + // Stage 3: + // * my_topic/0 -> leader1 serves 3 messages + // * my_topic/1 -> leader1 server 8 messages + + // leader1 provides 3 message on partition 0, and 8 messages on partition 1 + mockFetchResponse2 := newMockFetchResponse(t, 2) + for i := 4; i < 7; i++ { + mockFetchResponse2.SetMessage("my_topic", 0, int64(i), testMsg) + } + for i := 0; i < 8; i++ { + mockFetchResponse2.SetMessage("my_topic", 1, int64(i), testMsg) + } + leader1.SetHandlerByMap(map[string]MockResponse{ + "FetchRequest": mockFetchResponse2, + }) + + time.Sleep(50 * time.Millisecond) + Logger.Printf(" STAGE 4") + // Stage 4: + // * my_topic/0 -> leader1 serves 3 messages + // * my_topic/1 -> leader1 tells that it is no longer the leader + // * seedBroker tells that leader0 is a new leader for my_topic/1 + + // metadata assigns 0 to leader1 and 1 to leader0 + seedBroker.SetHandlerByMap(map[string]MockResponse{ + "MetadataRequest": newMockMetadataResponse(t). + SetLeader("my_topic", 0, leader1.BrokerID()). + SetLeader("my_topic", 1, leader0.BrokerID()), + }) + + // leader1 provides three more messages on partition0, says no longer leader of partition1 + mockFetchResponse3 := newMockFetchResponse(t, 3). + SetMessage("my_topic", 0, int64(7), testMsg). + SetMessage("my_topic", 0, int64(8), testMsg). + SetMessage("my_topic", 0, int64(9), testMsg) + fetchResponse4 := new(FetchResponse) + fetchResponse4.AddError("my_topic", 1, ErrNotLeaderForPartition) + leader1.SetHandlerByMap(map[string]MockResponse{ + "FetchRequest": newMockSequence(mockFetchResponse3, fetchResponse4), + }) + + // leader0 provides two messages on partition 1 + mockFetchResponse4 := newMockFetchResponse(t, 2) + for i := 8; i < 10; i++ { + mockFetchResponse4.SetMessage("my_topic", 1, int64(i), testMsg) + } + leader0.SetHandlerByMap(map[string]MockResponse{ + "FetchRequest": mockFetchResponse4, + }) + + wg.Wait() + safeClose(t, master) + leader1.Close() + leader0.Close() + seedBroker.Close() +} + +// When two partitions have the same broker as the leader, if one partition +// consumer channel buffer is full then that does not affect the ability to +// read messages by the other consumer. +func TestConsumerInterleavedClose(t *testing.T) { + // Given + broker0 := newMockBroker(t, 0) + broker0.SetHandlerByMap(map[string]MockResponse{ + "MetadataRequest": newMockMetadataResponse(t). + SetBroker(broker0.Addr(), broker0.BrokerID()). + SetLeader("my_topic", 0, broker0.BrokerID()). + SetLeader("my_topic", 1, broker0.BrokerID()), + "OffsetRequest": newMockOffsetResponse(t). + SetOffset("my_topic", 0, OffsetOldest, 1000). + SetOffset("my_topic", 0, OffsetNewest, 1100). + SetOffset("my_topic", 1, OffsetOldest, 2000). + SetOffset("my_topic", 1, OffsetNewest, 2100), + "FetchRequest": newMockFetchResponse(t, 1). + SetMessage("my_topic", 0, 1000, testMsg). + SetMessage("my_topic", 0, 1001, testMsg). + SetMessage("my_topic", 0, 1002, testMsg). + SetMessage("my_topic", 1, 2000, testMsg), + }) + + config := NewConfig() + config.ChannelBufferSize = 0 + master, err := NewConsumer([]string{broker0.Addr()}, config) + if err != nil { + t.Fatal(err) + } + + c0, err := master.ConsumePartition("my_topic", 0, 1000) + if err != nil { + t.Fatal(err) + } + + c1, err := master.ConsumePartition("my_topic", 1, 2000) + if err != nil { + t.Fatal(err) + } + + // When/Then: we can read from partition 0 even if nobody reads from partition 1 + assertMessageOffset(t, <-c0.Messages(), 1000) + assertMessageOffset(t, <-c0.Messages(), 1001) + assertMessageOffset(t, <-c0.Messages(), 1002) + + safeClose(t, c1) + safeClose(t, c0) + safeClose(t, master) + broker0.Close() +} + +func TestConsumerBounceWithReferenceOpen(t *testing.T) { + broker0 := newMockBroker(t, 0) + broker0Addr := broker0.Addr() + broker1 := newMockBroker(t, 1) + + mockMetadataResponse := newMockMetadataResponse(t). + SetBroker(broker0.Addr(), broker0.BrokerID()). + SetBroker(broker1.Addr(), broker1.BrokerID()). + SetLeader("my_topic", 0, broker0.BrokerID()). + SetLeader("my_topic", 1, broker1.BrokerID()) + + mockOffsetResponse := newMockOffsetResponse(t). + SetOffset("my_topic", 0, OffsetOldest, 1000). + SetOffset("my_topic", 0, OffsetNewest, 1100). + SetOffset("my_topic", 1, OffsetOldest, 2000). + SetOffset("my_topic", 1, OffsetNewest, 2100) + + mockFetchResponse := newMockFetchResponse(t, 1) + for i := 0; i < 10; i++ { + mockFetchResponse.SetMessage("my_topic", 0, int64(1000+i), testMsg) + mockFetchResponse.SetMessage("my_topic", 1, int64(2000+i), testMsg) + } + + broker0.SetHandlerByMap(map[string]MockResponse{ + "OffsetRequest": mockOffsetResponse, + "FetchRequest": mockFetchResponse, + }) + broker1.SetHandlerByMap(map[string]MockResponse{ + "MetadataRequest": mockMetadataResponse, + "OffsetRequest": mockOffsetResponse, + "FetchRequest": mockFetchResponse, + }) + + config := NewConfig() + config.Consumer.Return.Errors = true + config.Consumer.Retry.Backoff = 100 * time.Millisecond + config.ChannelBufferSize = 1 + master, err := NewConsumer([]string{broker1.Addr()}, config) + if err != nil { + t.Fatal(err) + } + + c0, err := master.ConsumePartition("my_topic", 0, 1000) + if err != nil { + t.Fatal(err) + } + + c1, err := master.ConsumePartition("my_topic", 1, 2000) + if err != nil { + t.Fatal(err) + } + + // read messages from both partition to make sure that both brokers operate + // normally. + assertMessageOffset(t, <-c0.Messages(), 1000) + assertMessageOffset(t, <-c1.Messages(), 2000) + + // Simulate broker shutdown. Note that metadata response does not change, + // that is the leadership does not move to another broker. So partition + // consumer will keep retrying to restore the connection with the broker. + broker0.Close() + + // Make sure that while the partition/0 leader is down, consumer/partition/1 + // is capable of pulling messages from broker1. + for i := 1; i < 7; i++ { + offset := (<-c1.Messages()).Offset + if offset != int64(2000+i) { + t.Errorf("Expected offset %d from consumer/partition/1", int64(2000+i)) + } + } + + // Bring broker0 back to service. + broker0 = newMockBrokerAddr(t, 0, broker0Addr) + broker0.SetHandlerByMap(map[string]MockResponse{ + "FetchRequest": mockFetchResponse, + }) + + // Read the rest of messages from both partitions. + for i := 7; i < 10; i++ { + assertMessageOffset(t, <-c1.Messages(), int64(2000+i)) + } + for i := 1; i < 10; i++ { + assertMessageOffset(t, <-c0.Messages(), int64(1000+i)) + } + + select { + case <-c0.Errors(): + default: + t.Errorf("Partition consumer should have detected broker restart") + } + + safeClose(t, c1) + safeClose(t, c0) + safeClose(t, master) + broker0.Close() + broker1.Close() +} + +func TestConsumerOffsetOutOfRange(t *testing.T) { + // Given + broker0 := newMockBroker(t, 2) + broker0.SetHandlerByMap(map[string]MockResponse{ + "MetadataRequest": newMockMetadataResponse(t). + SetBroker(broker0.Addr(), broker0.BrokerID()). + SetLeader("my_topic", 0, broker0.BrokerID()), + "OffsetRequest": newMockOffsetResponse(t). + SetOffset("my_topic", 0, OffsetNewest, 1234). + SetOffset("my_topic", 0, OffsetOldest, 2345), + }) + + master, err := NewConsumer([]string{broker0.Addr()}, nil) + if err != nil { + t.Fatal(err) + } + + // When/Then + if _, err := master.ConsumePartition("my_topic", 0, 0); err != ErrOffsetOutOfRange { + t.Fatal("Should return ErrOffsetOutOfRange, got:", err) + } + if _, err := master.ConsumePartition("my_topic", 0, 3456); err != ErrOffsetOutOfRange { + t.Fatal("Should return ErrOffsetOutOfRange, got:", err) + } + if _, err := master.ConsumePartition("my_topic", 0, -3); err != ErrOffsetOutOfRange { + t.Fatal("Should return ErrOffsetOutOfRange, got:", err) + } + + safeClose(t, master) + broker0.Close() +} + +func assertMessageOffset(t *testing.T, msg *ConsumerMessage, expectedOffset int64) { + if msg.Offset != expectedOffset { + t.Errorf("Incorrect message offset: expected=%d, actual=%d", expectedOffset, msg.Offset) + } +} + +// This example shows how to use the consumer to read messages +// from a single partition. +func ExampleConsumer() { + consumer, err := NewConsumer([]string{"localhost:9092"}, nil) + if err != nil { + panic(err) + } + + defer func() { + if err := consumer.Close(); err != nil { + log.Fatalln(err) + } + }() + + partitionConsumer, err := consumer.ConsumePartition("my_topic", 0, OffsetNewest) + if err != nil { + panic(err) + } + + defer func() { + if err := partitionConsumer.Close(); err != nil { + log.Fatalln(err) + } + }() + + // Trap SIGINT to trigger a shutdown. + signals := make(chan os.Signal, 1) + signal.Notify(signals, os.Interrupt) + + consumed := 0 +ConsumerLoop: + for { + select { + case msg := <-partitionConsumer.Messages(): + log.Printf("Consumed message offset %d\n", msg.Offset) + consumed++ + case <-signals: + break ConsumerLoop + } + } + + log.Printf("Consumed: %d\n", consumed) +} diff --git a/vendor/github.com/Shopify/sarama/crc32_field.go b/vendor/github.com/Shopify/sarama/crc32_field.go new file mode 100644 index 000000000000..5c286079056c --- /dev/null +++ b/vendor/github.com/Shopify/sarama/crc32_field.go @@ -0,0 +1,36 @@ +package sarama + +import ( + "encoding/binary" + + "github.com/klauspost/crc32" +) + +// crc32Field implements the pushEncoder and pushDecoder interfaces for calculating CRC32s. +type crc32Field struct { + startOffset int +} + +func (c *crc32Field) saveOffset(in int) { + c.startOffset = in +} + +func (c *crc32Field) reserveLength() int { + return 4 +} + +func (c *crc32Field) run(curOffset int, buf []byte) error { + crc := crc32.ChecksumIEEE(buf[c.startOffset+4 : curOffset]) + binary.BigEndian.PutUint32(buf[c.startOffset:], crc) + return nil +} + +func (c *crc32Field) check(curOffset int, buf []byte) error { + crc := crc32.ChecksumIEEE(buf[c.startOffset+4 : curOffset]) + + if crc != binary.BigEndian.Uint32(buf[c.startOffset:]) { + return PacketDecodingError{"CRC didn't match"} + } + + return nil +} diff --git a/vendor/github.com/Shopify/sarama/describe_groups_request.go b/vendor/github.com/Shopify/sarama/describe_groups_request.go new file mode 100644 index 000000000000..c9426a6b7b94 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/describe_groups_request.go @@ -0,0 +1,26 @@ +package sarama + +type DescribeGroupsRequest struct { + Groups []string +} + +func (r *DescribeGroupsRequest) encode(pe packetEncoder) error { + return pe.putStringArray(r.Groups) +} + +func (r *DescribeGroupsRequest) decode(pd packetDecoder) (err error) { + r.Groups, err = pd.getStringArray() + return +} + +func (r *DescribeGroupsRequest) key() int16 { + return 15 +} + +func (r *DescribeGroupsRequest) version() int16 { + return 0 +} + +func (r *DescribeGroupsRequest) AddGroup(group string) { + r.Groups = append(r.Groups, group) +} diff --git a/vendor/github.com/Shopify/sarama/describe_groups_request_test.go b/vendor/github.com/Shopify/sarama/describe_groups_request_test.go new file mode 100644 index 000000000000..7d45f3fee444 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/describe_groups_request_test.go @@ -0,0 +1,34 @@ +package sarama + +import "testing" + +var ( + emptyDescribeGroupsRequest = []byte{0, 0, 0, 0} + + singleDescribeGroupsRequest = []byte{ + 0, 0, 0, 1, // 1 group + 0, 3, 'f', 'o', 'o', // group name: foo + } + + doubleDescribeGroupsRequest = []byte{ + 0, 0, 0, 2, // 2 groups + 0, 3, 'f', 'o', 'o', // group name: foo + 0, 3, 'b', 'a', 'r', // group name: foo + } +) + +func TestDescribeGroupsRequest(t *testing.T) { + var request *DescribeGroupsRequest + + request = new(DescribeGroupsRequest) + testRequest(t, "no groups", request, emptyDescribeGroupsRequest) + + request = new(DescribeGroupsRequest) + request.AddGroup("foo") + testRequest(t, "one group", request, singleDescribeGroupsRequest) + + request = new(DescribeGroupsRequest) + request.AddGroup("foo") + request.AddGroup("bar") + testRequest(t, "two groups", request, doubleDescribeGroupsRequest) +} diff --git a/vendor/github.com/Shopify/sarama/describe_groups_response.go b/vendor/github.com/Shopify/sarama/describe_groups_response.go new file mode 100644 index 000000000000..b4b32dd8b2aa --- /dev/null +++ b/vendor/github.com/Shopify/sarama/describe_groups_response.go @@ -0,0 +1,162 @@ +package sarama + +type DescribeGroupsResponse struct { + Groups []*GroupDescription +} + +func (r *DescribeGroupsResponse) encode(pe packetEncoder) error { + if err := pe.putArrayLength(len(r.Groups)); err != nil { + return err + } + + for _, groupDescription := range r.Groups { + if err := groupDescription.encode(pe); err != nil { + return err + } + } + + return nil +} + +func (r *DescribeGroupsResponse) decode(pd packetDecoder) (err error) { + n, err := pd.getArrayLength() + if err != nil { + return err + } + + r.Groups = make([]*GroupDescription, n) + for i := 0; i < n; i++ { + r.Groups[i] = new(GroupDescription) + if err := r.Groups[i].decode(pd); err != nil { + return err + } + } + + return nil +} + +type GroupDescription struct { + Err KError + GroupId string + State string + ProtocolType string + Protocol string + Members map[string]*GroupMemberDescription +} + +func (gd *GroupDescription) encode(pe packetEncoder) error { + pe.putInt16(int16(gd.Err)) + + if err := pe.putString(gd.GroupId); err != nil { + return err + } + if err := pe.putString(gd.State); err != nil { + return err + } + if err := pe.putString(gd.ProtocolType); err != nil { + return err + } + if err := pe.putString(gd.Protocol); err != nil { + return err + } + + if err := pe.putArrayLength(len(gd.Members)); err != nil { + return err + } + + for memberId, groupMemberDescription := range gd.Members { + if err := pe.putString(memberId); err != nil { + return err + } + if err := groupMemberDescription.encode(pe); err != nil { + return err + } + } + + return nil +} + +func (gd *GroupDescription) decode(pd packetDecoder) (err error) { + if kerr, err := pd.getInt16(); err != nil { + return err + } else { + gd.Err = KError(kerr) + } + + if gd.GroupId, err = pd.getString(); err != nil { + return + } + if gd.State, err = pd.getString(); err != nil { + return + } + if gd.ProtocolType, err = pd.getString(); err != nil { + return + } + if gd.Protocol, err = pd.getString(); err != nil { + return + } + + n, err := pd.getArrayLength() + if err != nil { + return err + } + if n == 0 { + return nil + } + + gd.Members = make(map[string]*GroupMemberDescription) + for i := 0; i < n; i++ { + memberId, err := pd.getString() + if err != nil { + return err + } + + gd.Members[memberId] = new(GroupMemberDescription) + if err := gd.Members[memberId].decode(pd); err != nil { + return err + } + } + + return nil +} + +type GroupMemberDescription struct { + ClientId string + ClientHost string + MemberMetadata []byte + MemberAssignment []byte +} + +func (gmd *GroupMemberDescription) encode(pe packetEncoder) error { + if err := pe.putString(gmd.ClientId); err != nil { + return err + } + if err := pe.putString(gmd.ClientHost); err != nil { + return err + } + if err := pe.putBytes(gmd.MemberMetadata); err != nil { + return err + } + if err := pe.putBytes(gmd.MemberAssignment); err != nil { + return err + } + + return nil +} + +func (gmd *GroupMemberDescription) decode(pd packetDecoder) (err error) { + if gmd.ClientId, err = pd.getString(); err != nil { + return + } + if gmd.ClientHost, err = pd.getString(); err != nil { + return + } + if gmd.MemberMetadata, err = pd.getBytes(); err != nil { + return + } + if gmd.MemberAssignment, err = pd.getBytes(); err != nil { + return + } + + return nil +} diff --git a/vendor/github.com/Shopify/sarama/describe_groups_response_test.go b/vendor/github.com/Shopify/sarama/describe_groups_response_test.go new file mode 100644 index 000000000000..6f85f067e1c4 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/describe_groups_response_test.go @@ -0,0 +1,91 @@ +package sarama + +import ( + "reflect" + "testing" +) + +var ( + describeGroupsResponseEmpty = []byte{ + 0, 0, 0, 0, // no groups + } + + describeGroupsResponsePopulated = []byte{ + 0, 0, 0, 2, // 2 groups + + 0, 0, // no error + 0, 3, 'f', 'o', 'o', // Group ID + 0, 3, 'b', 'a', 'r', // State + 0, 8, 'c', 'o', 'n', 's', 'u', 'm', 'e', 'r', // ConsumerProtocol type + 0, 3, 'b', 'a', 'z', // Protocol name + 0, 0, 0, 1, // 1 member + 0, 2, 'i', 'd', // Member ID + 0, 6, 's', 'a', 'r', 'a', 'm', 'a', // Client ID + 0, 9, 'l', 'o', 'c', 'a', 'l', 'h', 'o', 's', 't', // Client Host + 0, 0, 0, 3, 0x01, 0x02, 0x03, // MemberMetadata + 0, 0, 0, 3, 0x04, 0x05, 0x06, // MemberAssignment + + 0, 30, // ErrGroupAuthorizationFailed + 0, 0, + 0, 0, + 0, 0, + 0, 0, + 0, 0, 0, 0, + } +) + +func TestDescribeGroupsResponse(t *testing.T) { + var response *DescribeGroupsResponse + + response = new(DescribeGroupsResponse) + testDecodable(t, "empty", response, describeGroupsResponseEmpty) + if len(response.Groups) != 0 { + t.Error("Expected no groups") + } + + response = new(DescribeGroupsResponse) + testDecodable(t, "populated", response, describeGroupsResponsePopulated) + if len(response.Groups) != 2 { + t.Error("Expected two groups") + } + + group0 := response.Groups[0] + if group0.Err != ErrNoError { + t.Error("Unxpected groups[0].Err, found", group0.Err) + } + if group0.GroupId != "foo" { + t.Error("Unxpected groups[0].GroupId, found", group0.GroupId) + } + if group0.State != "bar" { + t.Error("Unxpected groups[0].State, found", group0.State) + } + if group0.ProtocolType != "consumer" { + t.Error("Unxpected groups[0].ProtocolType, found", group0.ProtocolType) + } + if group0.Protocol != "baz" { + t.Error("Unxpected groups[0].Protocol, found", group0.Protocol) + } + if len(group0.Members) != 1 { + t.Error("Unxpected groups[0].Members, found", group0.Members) + } + if group0.Members["id"].ClientId != "sarama" { + t.Error("Unxpected groups[0].Members[id].ClientId, found", group0.Members["id"].ClientId) + } + if group0.Members["id"].ClientHost != "localhost" { + t.Error("Unxpected groups[0].Members[id].ClientHost, found", group0.Members["id"].ClientHost) + } + if !reflect.DeepEqual(group0.Members["id"].MemberMetadata, []byte{0x01, 0x02, 0x03}) { + t.Error("Unxpected groups[0].Members[id].MemberMetadata, found", group0.Members["id"].MemberMetadata) + } + if !reflect.DeepEqual(group0.Members["id"].MemberAssignment, []byte{0x04, 0x05, 0x06}) { + t.Error("Unxpected groups[0].Members[id].MemberAssignment, found", group0.Members["id"].MemberAssignment) + } + + group1 := response.Groups[1] + if group1.Err != ErrGroupAuthorizationFailed { + t.Error("Unxpected groups[1].Err, found", group0.Err) + } + if len(group1.Members) != 0 { + t.Error("Unxpected groups[1].Members, found", group0.Members) + } +} diff --git a/vendor/github.com/Shopify/sarama/encoder_decoder.go b/vendor/github.com/Shopify/sarama/encoder_decoder.go new file mode 100644 index 000000000000..b91efaa0ee60 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/encoder_decoder.go @@ -0,0 +1,62 @@ +package sarama + +import "fmt" + +// Encoder is the interface that wraps the basic Encode method. +// Anything implementing Encoder can be turned into bytes using Kafka's encoding rules. +type encoder interface { + encode(pe packetEncoder) error +} + +// Encode takes an Encoder and turns it into bytes. +func encode(e encoder) ([]byte, error) { + if e == nil { + return nil, nil + } + + var prepEnc prepEncoder + var realEnc realEncoder + + err := e.encode(&prepEnc) + if err != nil { + return nil, err + } + + if prepEnc.length < 0 || prepEnc.length > int(MaxRequestSize) { + return nil, PacketEncodingError{fmt.Sprintf("invalid request size (%d)", prepEnc.length)} + } + + realEnc.raw = make([]byte, prepEnc.length) + err = e.encode(&realEnc) + if err != nil { + return nil, err + } + + return realEnc.raw, nil +} + +// Decoder is the interface that wraps the basic Decode method. +// Anything implementing Decoder can be extracted from bytes using Kafka's encoding rules. +type decoder interface { + decode(pd packetDecoder) error +} + +// Decode takes bytes and a Decoder and fills the fields of the decoder from the bytes, +// interpreted using Kafka's encoding rules. +func decode(buf []byte, in decoder) error { + if buf == nil { + return nil + } + + helper := realDecoder{raw: buf} + err := in.decode(&helper) + if err != nil { + return err + } + + if helper.off != len(buf) { + return PacketDecodingError{"invalid length"} + } + + return nil +} diff --git a/vendor/github.com/Shopify/sarama/errors.go b/vendor/github.com/Shopify/sarama/errors.go new file mode 100644 index 000000000000..a837087f1d76 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/errors.go @@ -0,0 +1,179 @@ +package sarama + +import ( + "errors" + "fmt" +) + +// ErrOutOfBrokers is the error returned when the client has run out of brokers to talk to because all of them errored +// or otherwise failed to respond. +var ErrOutOfBrokers = errors.New("kafka: client has run out of available brokers to talk to (Is your cluster reachable?)") + +// ErrClosedClient is the error returned when a method is called on a client that has been closed. +var ErrClosedClient = errors.New("kafka: tried to use a client that was closed") + +// ErrIncompleteResponse is the error returned when the server returns a syntactically valid response, but it does +// not contain the expected information. +var ErrIncompleteResponse = errors.New("kafka: response did not contain all the expected topic/partition blocks") + +// ErrInvalidPartition is the error returned when a partitioner returns an invalid partition index +// (meaning one outside of the range [0...numPartitions-1]). +var ErrInvalidPartition = errors.New("kafka: partitioner returned an invalid partition index") + +// ErrAlreadyConnected is the error returned when calling Open() on a Broker that is already connected or connecting. +var ErrAlreadyConnected = errors.New("kafka: broker connection already initiated") + +// ErrNotConnected is the error returned when trying to send or call Close() on a Broker that is not connected. +var ErrNotConnected = errors.New("kafka: broker not connected") + +// ErrInsufficientData is returned when decoding and the packet is truncated. This can be expected +// when requesting messages, since as an optimization the server is allowed to return a partial message at the end +// of the message set. +var ErrInsufficientData = errors.New("kafka: insufficient data to decode packet, more bytes expected") + +// ErrShuttingDown is returned when a producer receives a message during shutdown. +var ErrShuttingDown = errors.New("kafka: message received by producer in process of shutting down") + +// ErrMessageTooLarge is returned when the next message to consume is larger than the configured Consumer.Fetch.Max +var ErrMessageTooLarge = errors.New("kafka: message is larger than Consumer.Fetch.Max") + +// PacketEncodingError is returned from a failure while encoding a Kafka packet. This can happen, for example, +// if you try to encode a string over 2^15 characters in length, since Kafka's encoding rules do not permit that. +type PacketEncodingError struct { + Info string +} + +func (err PacketEncodingError) Error() string { + return fmt.Sprintf("kafka: error encoding packet: %s", err.Info) +} + +// PacketDecodingError is returned when there was an error (other than truncated data) decoding the Kafka broker's response. +// This can be a bad CRC or length field, or any other invalid value. +type PacketDecodingError struct { + Info string +} + +func (err PacketDecodingError) Error() string { + return fmt.Sprintf("kafka: error decoding packet: %s", err.Info) +} + +// ConfigurationError is the type of error returned from a constructor (e.g. NewClient, or NewConsumer) +// when the specified configuration is invalid. +type ConfigurationError string + +func (err ConfigurationError) Error() string { + return "kafka: invalid configuration (" + string(err) + ")" +} + +// KError is the type of error that can be returned directly by the Kafka broker. +// See https://cwiki.apache.org/confluence/display/KAFKA/A+Guide+To+The+Kafka+Protocol#AGuideToTheKafkaProtocol-ErrorCodes +type KError int16 + +// Numeric error codes returned by the Kafka server. +const ( + ErrNoError KError = 0 + ErrUnknown KError = -1 + ErrOffsetOutOfRange KError = 1 + ErrInvalidMessage KError = 2 + ErrUnknownTopicOrPartition KError = 3 + ErrInvalidMessageSize KError = 4 + ErrLeaderNotAvailable KError = 5 + ErrNotLeaderForPartition KError = 6 + ErrRequestTimedOut KError = 7 + ErrBrokerNotAvailable KError = 8 + ErrReplicaNotAvailable KError = 9 + ErrMessageSizeTooLarge KError = 10 + ErrStaleControllerEpochCode KError = 11 + ErrOffsetMetadataTooLarge KError = 12 + ErrOffsetsLoadInProgress KError = 14 + ErrConsumerCoordinatorNotAvailable KError = 15 + ErrNotCoordinatorForConsumer KError = 16 + ErrInvalidTopic KError = 17 + ErrMessageSetSizeTooLarge KError = 18 + ErrNotEnoughReplicas KError = 19 + ErrNotEnoughReplicasAfterAppend KError = 20 + ErrInvalidRequiredAcks KError = 21 + ErrIllegalGeneration KError = 22 + ErrInconsistentGroupProtocol KError = 23 + ErrInvalidGroupId KError = 24 + ErrUnknownMemberId KError = 25 + ErrInvalidSessionTimeout KError = 26 + ErrRebalanceInProgress KError = 27 + ErrInvalidCommitOffsetSize KError = 28 + ErrTopicAuthorizationFailed KError = 29 + ErrGroupAuthorizationFailed KError = 30 + ErrClusterAuthorizationFailed KError = 31 +) + +func (err KError) Error() string { + // Error messages stolen/adapted from + // https://cwiki.apache.org/confluence/display/KAFKA/A+Guide+To+The+Kafka+Protocol + switch err { + case ErrNoError: + return "kafka server: Not an error, why are you printing me?" + case ErrUnknown: + return "kafka server: Unexpected (unknown?) server error." + case ErrOffsetOutOfRange: + return "kafka server: The requested offset is outside the range of offsets maintained by the server for the given topic/partition." + case ErrInvalidMessage: + return "kafka server: Message contents does not match its CRC." + case ErrUnknownTopicOrPartition: + return "kafka server: Request was for a topic or partition that does not exist on this broker." + case ErrInvalidMessageSize: + return "kafka server: The message has a negative size." + case ErrLeaderNotAvailable: + return "kafka server: In the middle of a leadership election, there is currently no leader for this partition and hence it is unavailable for writes." + case ErrNotLeaderForPartition: + return "kafka server: Tried to send a message to a replica that is not the leader for some partition. Your metadata is out of date." + case ErrRequestTimedOut: + return "kafka server: Request exceeded the user-specified time limit in the request." + case ErrBrokerNotAvailable: + return "kafka server: Broker not available. Not a client facing error, we should never receive this!!!" + case ErrReplicaNotAvailable: + return "kafka server: Replica infomation not available, one or more brokers are down." + case ErrMessageSizeTooLarge: + return "kafka server: Message was too large, server rejected it to avoid allocation error." + case ErrStaleControllerEpochCode: + return "kafka server: StaleControllerEpochCode (internal error code for broker-to-broker communication)." + case ErrOffsetMetadataTooLarge: + return "kafka server: Specified a string larger than the configured maximum for offset metadata." + case ErrOffsetsLoadInProgress: + return "kafka server: The broker is still loading offsets after a leader change for that offset's topic partition." + case ErrConsumerCoordinatorNotAvailable: + return "kafka server: Offset's topic has not yet been created." + case ErrNotCoordinatorForConsumer: + return "kafka server: Request was for a consumer group that is not coordinated by this broker." + case ErrInvalidTopic: + return "kafka server: The request attempted to perform an operation on an invalid topic." + case ErrMessageSetSizeTooLarge: + return "kafka server: The request included message batch larger than the configured segment size on the server." + case ErrNotEnoughReplicas: + return "kafka server: Messages are rejected since there are fewer in-sync replicas than required." + case ErrNotEnoughReplicasAfterAppend: + return "kafka server: Messages are written to the log, but to fewer in-sync replicas than required." + case ErrInvalidRequiredAcks: + return "kafka server: The number of required acks is invalid (should be either -1, 0, or 1)." + case ErrIllegalGeneration: + return "kafka server: The provided generation id is not the current generation." + case ErrInconsistentGroupProtocol: + return "kafka server: The provider group protocol type is incompatible with the other members." + case ErrInvalidGroupId: + return "kafka server: The provided group id was empty." + case ErrUnknownMemberId: + return "kafka server: The provided member is not known in the current generation." + case ErrInvalidSessionTimeout: + return "kafka server: The provided session timeout is outside the allowed range." + case ErrRebalanceInProgress: + return "kafka server: A rebalance for the group is in progress. Please re-join the group." + case ErrInvalidCommitOffsetSize: + return "kafka server: The provided commit metadata was too large." + case ErrTopicAuthorizationFailed: + return "kafka server: The client is not authorized to access this topic." + case ErrGroupAuthorizationFailed: + return "kafka server: The client is not authorized to access this group." + case ErrClusterAuthorizationFailed: + return "kafka server: The client is not authorized to send this request type." + } + + return fmt.Sprintf("Unknown error, how did this happen? Error code = %d", err) +} diff --git a/vendor/github.com/Shopify/sarama/examples/README.md b/vendor/github.com/Shopify/sarama/examples/README.md new file mode 100644 index 000000000000..b6588051eb3e --- /dev/null +++ b/vendor/github.com/Shopify/sarama/examples/README.md @@ -0,0 +1,9 @@ +# Sarama examples + +This folder contains example applications to demonstrate the use of Sarama. For code snippet examples on how to use the different types in Sarama, see [Sarams's API documentation on godoc.org](https://godoc.org/github.com/Shopify/sarama) + +In these examples, we use `github.com/Shopify/sarama` as import path. We do this to ensure all the examples are up to date with the latest changes in Sarama. For your own applications, you may want to use `gopkg.in/Shopify/sarama.v1` to lock into a stable API version. + +#### HTTP server + +[http_server](./http_server) is a simple HTTP server uses both the sync producer to produce data as part of the request handling cycle, as well as the async producer to maintain an access log. It also uses the [mocks subpackage](https://godoc.org/github.com/Shopify/sarama/mocks) to test both. diff --git a/vendor/github.com/Shopify/sarama/examples/http_server/.gitignore b/vendor/github.com/Shopify/sarama/examples/http_server/.gitignore new file mode 100644 index 000000000000..9f6ed425f9c7 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/examples/http_server/.gitignore @@ -0,0 +1,2 @@ +http_server +http_server.test diff --git a/vendor/github.com/Shopify/sarama/examples/http_server/README.md b/vendor/github.com/Shopify/sarama/examples/http_server/README.md new file mode 100644 index 000000000000..5ff2bc2533b8 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/examples/http_server/README.md @@ -0,0 +1,7 @@ +# HTTP server example + +This HTTP server example shows you how to use the AsyncProducer and SyncProducer, and how to test them using mocks. The server simply sends the data of the HTTP request's query string to Kafka, and send a 200 result if that succeeds. For every request, it will send an access log entry to Kafka as well in the background. + +If you need to know whether a message was successfully sent to the Kafka cluster before you can send your HTTP response, using the `SyncProducer` is probably the simplest way to achieve this. If you don't care, e.g. for the access log, using the `AsyncProducer` will let you fire and forget. You can send the HTTP response, while the message is being produced in the background. + +One important thing to note is that both the `SyncProducer` and `AsyncProducer` are **thread-safe**. Go's `http.Server` handles requests concurrently in different goroutines, but you can use a single producer safely. This will actually achieve efficiency gains as the producer will be able to batch messages from concurrent requests together. diff --git a/vendor/github.com/Shopify/sarama/examples/http_server/http_server.go b/vendor/github.com/Shopify/sarama/examples/http_server/http_server.go new file mode 100644 index 000000000000..03e47b6b2367 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/examples/http_server/http_server.go @@ -0,0 +1,246 @@ +package main + +import ( + "github.com/Shopify/sarama" + + "crypto/tls" + "crypto/x509" + "encoding/json" + "flag" + "fmt" + "io/ioutil" + "log" + "net/http" + "os" + "strings" + "time" +) + +var ( + addr = flag.String("addr", ":8080", "The address to bind to") + brokers = flag.String("brokers", os.Getenv("KAFKA_PEERS"), "The Kafka brokers to connect to, as a comma separated list") + verbose = flag.Bool("verbose", false, "Turn on Sarama logging") + certFile = flag.String("certificate", "", "The optional certificate file for client authentication") + keyFile = flag.String("key", "", "The optional key file for client authentication") + caFile = flag.String("ca", "", "The optional certificate authority file for TLS client authentication") + verifySsl = flag.Bool("verify", false, "Optional verify ssl certificates chain") +) + +func main() { + flag.Parse() + + if *verbose { + sarama.Logger = log.New(os.Stdout, "[sarama] ", log.LstdFlags) + } + + if *brokers == "" { + flag.PrintDefaults() + os.Exit(1) + } + + brokerList := strings.Split(*brokers, ",") + log.Printf("Kafka brokers: %s", strings.Join(brokerList, ", ")) + + server := &Server{ + DataCollector: newDataCollector(brokerList), + AccessLogProducer: newAccessLogProducer(brokerList), + } + defer func() { + if err := server.Close(); err != nil { + log.Println("Failed to close server", err) + } + }() + + log.Fatal(server.Run(*addr)) +} + +func createTlsConfiguration() (t *tls.Config) { + if *certFile != "" && *keyFile != "" && *caFile != "" { + cert, err := tls.LoadX509KeyPair(*certFile, *keyFile) + if err != nil { + log.Fatal(err) + } + + caCert, err := ioutil.ReadFile(*caFile) + if err != nil { + log.Fatal(err) + } + + caCertPool := x509.NewCertPool() + caCertPool.AppendCertsFromPEM(caCert) + + t = &tls.Config{ + Certificates: []tls.Certificate{cert}, + RootCAs: caCertPool, + InsecureSkipVerify: *verifySsl, + } + } + // will be nil by default if nothing is provided + return t +} + +type Server struct { + DataCollector sarama.SyncProducer + AccessLogProducer sarama.AsyncProducer +} + +func (s *Server) Close() error { + if err := s.DataCollector.Close(); err != nil { + log.Println("Failed to shut down data collector cleanly", err) + } + + if err := s.AccessLogProducer.Close(); err != nil { + log.Println("Failed to shut down access log producer cleanly", err) + } + + return nil +} + +func (s *Server) Handler() http.Handler { + return s.withAccessLog(s.collectQueryStringData()) +} + +func (s *Server) Run(addr string) error { + httpServer := &http.Server{ + Addr: addr, + Handler: s.Handler(), + } + + log.Printf("Listening for requests on %s...\n", addr) + return httpServer.ListenAndServe() +} + +func (s *Server) collectQueryStringData() http.Handler { + return http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) { + if r.URL.Path != "/" { + http.NotFound(w, r) + return + } + + // We are not setting a message key, which means that all messages will + // be distributed randomly over the different partitions. + partition, offset, err := s.DataCollector.SendMessage(&sarama.ProducerMessage{ + Topic: "important", + Value: sarama.StringEncoder(r.URL.RawQuery), + }) + + if err != nil { + w.WriteHeader(http.StatusInternalServerError) + fmt.Fprintf(w, "Failed to store your data:, %s", err) + } else { + // The tuple (topic, partition, offset) can be used as a unique identifier + // for a message in a Kafka cluster. + fmt.Fprintf(w, "Your data is stored with unique identifier important/%d/%d", partition, offset) + } + }) +} + +type accessLogEntry struct { + Method string `json:"method"` + Host string `json:"host"` + Path string `json:"path"` + IP string `json:"ip"` + ResponseTime float64 `json:"response_time"` + + encoded []byte + err error +} + +func (ale *accessLogEntry) ensureEncoded() { + if ale.encoded == nil && ale.err == nil { + ale.encoded, ale.err = json.Marshal(ale) + } +} + +func (ale *accessLogEntry) Length() int { + ale.ensureEncoded() + return len(ale.encoded) +} + +func (ale *accessLogEntry) Encode() ([]byte, error) { + ale.ensureEncoded() + return ale.encoded, ale.err +} + +func (s *Server) withAccessLog(next http.Handler) http.Handler { + + return http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) { + started := time.Now() + + next.ServeHTTP(w, r) + + entry := &accessLogEntry{ + Method: r.Method, + Host: r.Host, + Path: r.RequestURI, + IP: r.RemoteAddr, + ResponseTime: float64(time.Since(started)) / float64(time.Second), + } + + // We will use the client's IP address as key. This will cause + // all the access log entries of the same IP address to end up + // on the same partition. + s.AccessLogProducer.Input() <- &sarama.ProducerMessage{ + Topic: "access_log", + Key: sarama.StringEncoder(r.RemoteAddr), + Value: entry, + } + }) +} + +func newDataCollector(brokerList []string) sarama.SyncProducer { + + // For the data collector, we are looking for strong consistency semantics. + // Because we don't change the flush settings, sarama will try to produce messages + // as fast as possible to keep latency low. + config := sarama.NewConfig() + config.Producer.RequiredAcks = sarama.WaitForAll // Wait for all in-sync replicas to ack the message + config.Producer.Retry.Max = 10 // Retry up to 10 times to produce the message + tlsConfig := createTlsConfiguration() + if tlsConfig != nil { + config.Net.TLS.Config = tlsConfig + config.Net.TLS.Enable = true + } + + // On the broker side, you may want to change the following settings to get + // stronger consistency guarantees: + // - For your broker, set `unclean.leader.election.enable` to false + // - For the topic, you could increase `min.insync.replicas`. + + producer, err := sarama.NewSyncProducer(brokerList, config) + if err != nil { + log.Fatalln("Failed to start Sarama producer:", err) + } + + return producer +} + +func newAccessLogProducer(brokerList []string) sarama.AsyncProducer { + + // For the access log, we are looking for AP semantics, with high throughput. + // By creating batches of compressed messages, we reduce network I/O at a cost of more latency. + config := sarama.NewConfig() + tlsConfig := createTlsConfiguration() + if tlsConfig != nil { + config.Net.TLS.Enable = true + config.Net.TLS.Config = tlsConfig + } + config.Producer.RequiredAcks = sarama.WaitForLocal // Only wait for the leader to ack + config.Producer.Compression = sarama.CompressionSnappy // Compress messages + config.Producer.Flush.Frequency = 500 * time.Millisecond // Flush batches every 500ms + + producer, err := sarama.NewAsyncProducer(brokerList, config) + if err != nil { + log.Fatalln("Failed to start Sarama producer:", err) + } + + // We will just log to STDOUT if we're not able to produce messages. + // Note: messages will only be returned here after all retry attempts are exhausted. + go func() { + for err := range producer.Errors() { + log.Println("Failed to write access log entry:", err) + } + }() + + return producer +} diff --git a/vendor/github.com/Shopify/sarama/examples/http_server/http_server_test.go b/vendor/github.com/Shopify/sarama/examples/http_server/http_server_test.go new file mode 100644 index 000000000000..7b2451e282cc --- /dev/null +++ b/vendor/github.com/Shopify/sarama/examples/http_server/http_server_test.go @@ -0,0 +1,109 @@ +package main + +import ( + "io" + "net/http" + "net/http/httptest" + "testing" + + "github.com/Shopify/sarama" + "github.com/Shopify/sarama/mocks" +) + +// In normal operation, we expect one access log entry, +// and one data collector entry. Let's assume both will succeed. +// We should return a HTTP 200 status. +func TestCollectSuccessfully(t *testing.T) { + dataCollectorMock := mocks.NewSyncProducer(t, nil) + dataCollectorMock.ExpectSendMessageAndSucceed() + + accessLogProducerMock := mocks.NewAsyncProducer(t, nil) + accessLogProducerMock.ExpectInputAndSucceed() + + // Now, use dependency injection to use the mocks. + s := &Server{ + DataCollector: dataCollectorMock, + AccessLogProducer: accessLogProducerMock, + } + + // The Server's Close call is important; it will call Close on + // the two mock producers, which will then validate whether all + // expectations are resolved. + defer safeClose(t, s) + + req, err := http.NewRequest("GET", "http://example.com/?data", nil) + if err != nil { + t.Fatal(err) + } + res := httptest.NewRecorder() + s.Handler().ServeHTTP(res, req) + + if res.Code != 200 { + t.Errorf("Expected HTTP status 200, found %d", res.Code) + } + + if string(res.Body.Bytes()) != "Your data is stored with unique identifier important/0/1" { + t.Error("Unexpected response body", res.Body) + } +} + +// Now, let's see if we handle the case of not being able to produce +// to the data collector properly. In this case we should return a 500 status. +func TestCollectionFailure(t *testing.T) { + dataCollectorMock := mocks.NewSyncProducer(t, nil) + dataCollectorMock.ExpectSendMessageAndFail(sarama.ErrRequestTimedOut) + + accessLogProducerMock := mocks.NewAsyncProducer(t, nil) + accessLogProducerMock.ExpectInputAndSucceed() + + s := &Server{ + DataCollector: dataCollectorMock, + AccessLogProducer: accessLogProducerMock, + } + defer safeClose(t, s) + + req, err := http.NewRequest("GET", "http://example.com/?data", nil) + if err != nil { + t.Fatal(err) + } + res := httptest.NewRecorder() + s.Handler().ServeHTTP(res, req) + + if res.Code != 500 { + t.Errorf("Expected HTTP status 500, found %d", res.Code) + } +} + +// We don't expect any data collector calls because the path is wrong, +// so we are not setting any expectations on the dataCollectorMock. It +// will still generate an access log entry though. +func TestWrongPath(t *testing.T) { + dataCollectorMock := mocks.NewSyncProducer(t, nil) + + accessLogProducerMock := mocks.NewAsyncProducer(t, nil) + accessLogProducerMock.ExpectInputAndSucceed() + + s := &Server{ + DataCollector: dataCollectorMock, + AccessLogProducer: accessLogProducerMock, + } + defer safeClose(t, s) + + req, err := http.NewRequest("GET", "http://example.com/wrong?data", nil) + if err != nil { + t.Fatal(err) + } + res := httptest.NewRecorder() + + s.Handler().ServeHTTP(res, req) + + if res.Code != 404 { + t.Errorf("Expected HTTP status 404, found %d", res.Code) + } +} + +func safeClose(t *testing.T, o io.Closer) { + if err := o.Close(); err != nil { + t.Error(err) + } +} diff --git a/vendor/github.com/Shopify/sarama/fetch_request.go b/vendor/github.com/Shopify/sarama/fetch_request.go new file mode 100644 index 000000000000..3c00fad65302 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/fetch_request.go @@ -0,0 +1,123 @@ +package sarama + +type fetchRequestBlock struct { + fetchOffset int64 + maxBytes int32 +} + +func (f *fetchRequestBlock) encode(pe packetEncoder) error { + pe.putInt64(f.fetchOffset) + pe.putInt32(f.maxBytes) + return nil +} + +func (f *fetchRequestBlock) decode(pd packetDecoder) (err error) { + if f.fetchOffset, err = pd.getInt64(); err != nil { + return err + } + if f.maxBytes, err = pd.getInt32(); err != nil { + return err + } + return nil +} + +type FetchRequest struct { + MaxWaitTime int32 + MinBytes int32 + blocks map[string]map[int32]*fetchRequestBlock +} + +func (f *FetchRequest) encode(pe packetEncoder) (err error) { + pe.putInt32(-1) // replica ID is always -1 for clients + pe.putInt32(f.MaxWaitTime) + pe.putInt32(f.MinBytes) + err = pe.putArrayLength(len(f.blocks)) + if err != nil { + return err + } + for topic, blocks := range f.blocks { + err = pe.putString(topic) + if err != nil { + return err + } + err = pe.putArrayLength(len(blocks)) + if err != nil { + return err + } + for partition, block := range blocks { + pe.putInt32(partition) + err = block.encode(pe) + if err != nil { + return err + } + } + } + return nil +} + +func (f *FetchRequest) decode(pd packetDecoder) (err error) { + if _, err = pd.getInt32(); err != nil { + return err + } + if f.MaxWaitTime, err = pd.getInt32(); err != nil { + return err + } + if f.MinBytes, err = pd.getInt32(); err != nil { + return err + } + topicCount, err := pd.getArrayLength() + if err != nil { + return err + } + if topicCount == 0 { + return nil + } + f.blocks = make(map[string]map[int32]*fetchRequestBlock) + for i := 0; i < topicCount; i++ { + topic, err := pd.getString() + if err != nil { + return err + } + partitionCount, err := pd.getArrayLength() + if err != nil { + return err + } + f.blocks[topic] = make(map[int32]*fetchRequestBlock) + for j := 0; j < partitionCount; j++ { + partition, err := pd.getInt32() + if err != nil { + return err + } + fetchBlock := &fetchRequestBlock{} + if err = fetchBlock.decode(pd); err != nil { + return nil + } + f.blocks[topic][partition] = fetchBlock + } + } + return nil +} + +func (f *FetchRequest) key() int16 { + return 1 +} + +func (f *FetchRequest) version() int16 { + return 0 +} + +func (f *FetchRequest) AddBlock(topic string, partitionID int32, fetchOffset int64, maxBytes int32) { + if f.blocks == nil { + f.blocks = make(map[string]map[int32]*fetchRequestBlock) + } + + if f.blocks[topic] == nil { + f.blocks[topic] = make(map[int32]*fetchRequestBlock) + } + + tmp := new(fetchRequestBlock) + tmp.maxBytes = maxBytes + tmp.fetchOffset = fetchOffset + + f.blocks[topic][partitionID] = tmp +} diff --git a/vendor/github.com/Shopify/sarama/fetch_request_test.go b/vendor/github.com/Shopify/sarama/fetch_request_test.go new file mode 100644 index 000000000000..32c083c7d3ef --- /dev/null +++ b/vendor/github.com/Shopify/sarama/fetch_request_test.go @@ -0,0 +1,34 @@ +package sarama + +import "testing" + +var ( + fetchRequestNoBlocks = []byte{ + 0xFF, 0xFF, 0xFF, 0xFF, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, + 0x00, 0x00, 0x00, 0x00} + + fetchRequestWithProperties = []byte{ + 0xFF, 0xFF, 0xFF, 0xFF, 0x00, 0x00, 0x00, 0x20, 0x00, 0x00, 0x00, 0xEF, + 0x00, 0x00, 0x00, 0x00} + + fetchRequestOneBlock = []byte{ + 0xFF, 0xFF, 0xFF, 0xFF, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, + 0x00, 0x00, 0x00, 0x01, + 0x00, 0x05, 't', 'o', 'p', 'i', 'c', + 0x00, 0x00, 0x00, 0x01, + 0x00, 0x00, 0x00, 0x12, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x34, 0x00, 0x00, 0x00, 0x56} +) + +func TestFetchRequest(t *testing.T) { + request := new(FetchRequest) + testRequest(t, "no blocks", request, fetchRequestNoBlocks) + + request.MaxWaitTime = 0x20 + request.MinBytes = 0xEF + testRequest(t, "with properties", request, fetchRequestWithProperties) + + request.MaxWaitTime = 0 + request.MinBytes = 0 + request.AddBlock("topic", 0x12, 0x34, 0x56) + testRequest(t, "one block", request, fetchRequestOneBlock) +} diff --git a/vendor/github.com/Shopify/sarama/fetch_response.go b/vendor/github.com/Shopify/sarama/fetch_response.go new file mode 100644 index 000000000000..1ac543921423 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/fetch_response.go @@ -0,0 +1,173 @@ +package sarama + +type FetchResponseBlock struct { + Err KError + HighWaterMarkOffset int64 + MsgSet MessageSet +} + +func (pr *FetchResponseBlock) decode(pd packetDecoder) (err error) { + tmp, err := pd.getInt16() + if err != nil { + return err + } + pr.Err = KError(tmp) + + pr.HighWaterMarkOffset, err = pd.getInt64() + if err != nil { + return err + } + + msgSetSize, err := pd.getInt32() + if err != nil { + return err + } + + msgSetDecoder, err := pd.getSubset(int(msgSetSize)) + if err != nil { + return err + } + err = (&pr.MsgSet).decode(msgSetDecoder) + + return err +} + +type FetchResponse struct { + Blocks map[string]map[int32]*FetchResponseBlock +} + +func (pr *FetchResponseBlock) encode(pe packetEncoder) (err error) { + pe.putInt16(int16(pr.Err)) + + pe.putInt64(pr.HighWaterMarkOffset) + + pe.push(&lengthField{}) + err = pr.MsgSet.encode(pe) + if err != nil { + return err + } + return pe.pop() +} + +func (fr *FetchResponse) decode(pd packetDecoder) (err error) { + numTopics, err := pd.getArrayLength() + if err != nil { + return err + } + + fr.Blocks = make(map[string]map[int32]*FetchResponseBlock, numTopics) + for i := 0; i < numTopics; i++ { + name, err := pd.getString() + if err != nil { + return err + } + + numBlocks, err := pd.getArrayLength() + if err != nil { + return err + } + + fr.Blocks[name] = make(map[int32]*FetchResponseBlock, numBlocks) + + for j := 0; j < numBlocks; j++ { + id, err := pd.getInt32() + if err != nil { + return err + } + + block := new(FetchResponseBlock) + err = block.decode(pd) + if err != nil { + return err + } + fr.Blocks[name][id] = block + } + } + + return nil +} + +func (fr *FetchResponse) encode(pe packetEncoder) (err error) { + err = pe.putArrayLength(len(fr.Blocks)) + if err != nil { + return err + } + + for topic, partitions := range fr.Blocks { + err = pe.putString(topic) + if err != nil { + return err + } + + err = pe.putArrayLength(len(partitions)) + if err != nil { + return err + } + + for id, block := range partitions { + pe.putInt32(id) + err = block.encode(pe) + if err != nil { + return err + } + } + + } + return nil +} + +func (fr *FetchResponse) GetBlock(topic string, partition int32) *FetchResponseBlock { + if fr.Blocks == nil { + return nil + } + + if fr.Blocks[topic] == nil { + return nil + } + + return fr.Blocks[topic][partition] +} + +func (fr *FetchResponse) AddError(topic string, partition int32, err KError) { + if fr.Blocks == nil { + fr.Blocks = make(map[string]map[int32]*FetchResponseBlock) + } + partitions, ok := fr.Blocks[topic] + if !ok { + partitions = make(map[int32]*FetchResponseBlock) + fr.Blocks[topic] = partitions + } + frb, ok := partitions[partition] + if !ok { + frb = new(FetchResponseBlock) + partitions[partition] = frb + } + frb.Err = err +} + +func (fr *FetchResponse) AddMessage(topic string, partition int32, key, value Encoder, offset int64) { + if fr.Blocks == nil { + fr.Blocks = make(map[string]map[int32]*FetchResponseBlock) + } + partitions, ok := fr.Blocks[topic] + if !ok { + partitions = make(map[int32]*FetchResponseBlock) + fr.Blocks[topic] = partitions + } + frb, ok := partitions[partition] + if !ok { + frb = new(FetchResponseBlock) + partitions[partition] = frb + } + var kb []byte + var vb []byte + if key != nil { + kb, _ = key.Encode() + } + if value != nil { + vb, _ = value.Encode() + } + msg := &Message{Key: kb, Value: vb} + msgBlock := &MessageBlock{Msg: msg, Offset: offset} + frb.MsgSet.Messages = append(frb.MsgSet.Messages, msgBlock) +} diff --git a/vendor/github.com/Shopify/sarama/fetch_response_test.go b/vendor/github.com/Shopify/sarama/fetch_response_test.go new file mode 100644 index 000000000000..a23a05340f81 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/fetch_response_test.go @@ -0,0 +1,84 @@ +package sarama + +import ( + "bytes" + "testing" +) + +var ( + emptyFetchResponse = []byte{ + 0x00, 0x00, 0x00, 0x00} + + oneMessageFetchResponse = []byte{ + 0x00, 0x00, 0x00, 0x01, + 0x00, 0x05, 't', 'o', 'p', 'i', 'c', + 0x00, 0x00, 0x00, 0x01, + 0x00, 0x00, 0x00, 0x05, + 0x00, 0x01, + 0x00, 0x00, 0x00, 0x00, 0x10, 0x10, 0x10, 0x10, + 0x00, 0x00, 0x00, 0x1C, + // messageSet + 0x00, 0x00, 0x00, 0x00, 0x00, 0x55, 0x00, 0x00, + 0x00, 0x00, 0x00, 0x10, + // message + 0x23, 0x96, 0x4a, 0xf7, // CRC + 0x00, + 0x00, + 0xFF, 0xFF, 0xFF, 0xFF, + 0x00, 0x00, 0x00, 0x02, 0x00, 0xEE} +) + +func TestEmptyFetchResponse(t *testing.T) { + response := FetchResponse{} + testDecodable(t, "empty", &response, emptyFetchResponse) + + if len(response.Blocks) != 0 { + t.Error("Decoding produced topic blocks where there were none.") + } + +} + +func TestOneMessageFetchResponse(t *testing.T) { + response := FetchResponse{} + testDecodable(t, "one message", &response, oneMessageFetchResponse) + + if len(response.Blocks) != 1 { + t.Fatal("Decoding produced incorrect number of topic blocks.") + } + + if len(response.Blocks["topic"]) != 1 { + t.Fatal("Decoding produced incorrect number of partition blocks for topic.") + } + + block := response.GetBlock("topic", 5) + if block == nil { + t.Fatal("GetBlock didn't return block.") + } + if block.Err != ErrOffsetOutOfRange { + t.Error("Decoding didn't produce correct error code.") + } + if block.HighWaterMarkOffset != 0x10101010 { + t.Error("Decoding didn't produce correct high water mark offset.") + } + if block.MsgSet.PartialTrailingMessage { + t.Error("Decoding detected a partial trailing message where there wasn't one.") + } + + if len(block.MsgSet.Messages) != 1 { + t.Fatal("Decoding produced incorrect number of messages.") + } + msgBlock := block.MsgSet.Messages[0] + if msgBlock.Offset != 0x550000 { + t.Error("Decoding produced incorrect message offset.") + } + msg := msgBlock.Msg + if msg.Codec != CompressionNone { + t.Error("Decoding produced incorrect message compression.") + } + if msg.Key != nil { + t.Error("Decoding produced message key where there was none.") + } + if !bytes.Equal(msg.Value, []byte{0x00, 0xEE}) { + t.Error("Decoding produced incorrect message value.") + } +} diff --git a/vendor/github.com/Shopify/sarama/functional_client_test.go b/vendor/github.com/Shopify/sarama/functional_client_test.go new file mode 100644 index 000000000000..9e8e32968d2b --- /dev/null +++ b/vendor/github.com/Shopify/sarama/functional_client_test.go @@ -0,0 +1,90 @@ +package sarama + +import ( + "fmt" + "testing" + "time" +) + +func TestFuncConnectionFailure(t *testing.T) { + setupFunctionalTest(t) + defer teardownFunctionalTest(t) + + Proxies["kafka1"].Enabled = false + SaveProxy(t, "kafka1") + + config := NewConfig() + config.Metadata.Retry.Max = 1 + + _, err := NewClient([]string{kafkaBrokers[0]}, config) + if err != ErrOutOfBrokers { + t.Fatal("Expected returned error to be ErrOutOfBrokers, but was: ", err) + } +} + +func TestFuncClientMetadata(t *testing.T) { + setupFunctionalTest(t) + defer teardownFunctionalTest(t) + + config := NewConfig() + config.Metadata.Retry.Max = 1 + config.Metadata.Retry.Backoff = 10 * time.Millisecond + client, err := NewClient(kafkaBrokers, config) + if err != nil { + t.Fatal(err) + } + + if err := client.RefreshMetadata("unknown_topic"); err != ErrUnknownTopicOrPartition { + t.Error("Expected ErrUnknownTopicOrPartition, got", err) + } + + if _, err := client.Leader("unknown_topic", 0); err != ErrUnknownTopicOrPartition { + t.Error("Expected ErrUnknownTopicOrPartition, got", err) + } + + if _, err := client.Replicas("invalid/topic", 0); err != ErrUnknownTopicOrPartition { + t.Error("Expected ErrUnknownTopicOrPartition, got", err) + } + + partitions, err := client.Partitions("test.4") + if err != nil { + t.Error(err) + } + if len(partitions) != 4 { + t.Errorf("Expected test.4 topic to have 4 partitions, found %v", partitions) + } + + partitions, err = client.Partitions("test.1") + if err != nil { + t.Error(err) + } + if len(partitions) != 1 { + t.Errorf("Expected test.1 topic to have 1 partitions, found %v", partitions) + } + + safeClose(t, client) +} + +func TestFuncClientCoordinator(t *testing.T) { + checkKafkaVersion(t, "0.8.2") + setupFunctionalTest(t) + defer teardownFunctionalTest(t) + + client, err := NewClient(kafkaBrokers, nil) + if err != nil { + t.Fatal(err) + } + + for i := 0; i < 10; i++ { + broker, err := client.Coordinator(fmt.Sprintf("another_new_consumer_group_%d", i)) + if err != nil { + t.Error(err) + } + + if connected, err := broker.Connected(); !connected || err != nil { + t.Errorf("Expected to coordinator %s broker to be properly connected.", broker.Addr()) + } + } + + safeClose(t, client) +} diff --git a/vendor/github.com/Shopify/sarama/functional_consumer_test.go b/vendor/github.com/Shopify/sarama/functional_consumer_test.go new file mode 100644 index 000000000000..ab8433109116 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/functional_consumer_test.go @@ -0,0 +1,61 @@ +package sarama + +import ( + "math" + "testing" +) + +func TestFuncConsumerOffsetOutOfRange(t *testing.T) { + setupFunctionalTest(t) + defer teardownFunctionalTest(t) + + consumer, err := NewConsumer(kafkaBrokers, nil) + if err != nil { + t.Fatal(err) + } + + if _, err := consumer.ConsumePartition("test.1", 0, -10); err != ErrOffsetOutOfRange { + t.Error("Expected ErrOffsetOutOfRange, got:", err) + } + + if _, err := consumer.ConsumePartition("test.1", 0, math.MaxInt64); err != ErrOffsetOutOfRange { + t.Error("Expected ErrOffsetOutOfRange, got:", err) + } + + safeClose(t, consumer) +} + +func TestConsumerHighWaterMarkOffset(t *testing.T) { + setupFunctionalTest(t) + defer teardownFunctionalTest(t) + + p, err := NewSyncProducer(kafkaBrokers, nil) + if err != nil { + t.Fatal(err) + } + defer safeClose(t, p) + + _, offset, err := p.SendMessage(&ProducerMessage{Topic: "test.1", Value: StringEncoder("Test")}) + if err != nil { + t.Fatal(err) + } + + c, err := NewConsumer(kafkaBrokers, nil) + if err != nil { + t.Fatal(err) + } + defer safeClose(t, c) + + pc, err := c.ConsumePartition("test.1", 0, OffsetOldest) + if err != nil { + t.Fatal(err) + } + + <-pc.Messages() + + if hwmo := pc.HighWaterMarkOffset(); hwmo != offset+1 { + t.Logf("Last produced offset %d; high water mark should be one higher but found %d.", offset, hwmo) + } + + safeClose(t, pc) +} diff --git a/vendor/github.com/Shopify/sarama/functional_offset_manager_test.go b/vendor/github.com/Shopify/sarama/functional_offset_manager_test.go new file mode 100644 index 000000000000..b759669c0679 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/functional_offset_manager_test.go @@ -0,0 +1,47 @@ +package sarama + +import ( + "testing" +) + +func TestFuncOffsetManager(t *testing.T) { + checkKafkaVersion(t, "0.8.2") + setupFunctionalTest(t) + defer teardownFunctionalTest(t) + + client, err := NewClient(kafkaBrokers, nil) + if err != nil { + t.Fatal(err) + } + + offsetManager, err := NewOffsetManagerFromClient("sarama.TestFuncOffsetManager", client) + if err != nil { + t.Fatal(err) + } + + pom1, err := offsetManager.ManagePartition("test.1", 0) + if err != nil { + t.Fatal(err) + } + + pom1.MarkOffset(10, "test metadata") + safeClose(t, pom1) + + pom2, err := offsetManager.ManagePartition("test.1", 0) + if err != nil { + t.Fatal(err) + } + + offset, metadata := pom2.NextOffset() + + if offset != 10+1 { + t.Errorf("Expected the next offset to be 11, found %d.", offset) + } + if metadata != "test metadata" { + t.Errorf("Expected metadata to be 'test metadata', found %s.", metadata) + } + + safeClose(t, pom2) + safeClose(t, offsetManager) + safeClose(t, client) +} diff --git a/vendor/github.com/Shopify/sarama/functional_producer_test.go b/vendor/github.com/Shopify/sarama/functional_producer_test.go new file mode 100644 index 000000000000..1504e76005bc --- /dev/null +++ b/vendor/github.com/Shopify/sarama/functional_producer_test.go @@ -0,0 +1,203 @@ +package sarama + +import ( + "fmt" + "sync" + "testing" + "time" +) + +const TestBatchSize = 1000 + +func TestFuncProducing(t *testing.T) { + config := NewConfig() + testProducingMessages(t, config) +} + +func TestFuncProducingGzip(t *testing.T) { + config := NewConfig() + config.Producer.Compression = CompressionGZIP + testProducingMessages(t, config) +} + +func TestFuncProducingSnappy(t *testing.T) { + config := NewConfig() + config.Producer.Compression = CompressionSnappy + testProducingMessages(t, config) +} + +func TestFuncProducingNoResponse(t *testing.T) { + config := NewConfig() + config.Producer.RequiredAcks = NoResponse + testProducingMessages(t, config) +} + +func TestFuncProducingFlushing(t *testing.T) { + config := NewConfig() + config.Producer.Flush.Messages = TestBatchSize / 8 + config.Producer.Flush.Frequency = 250 * time.Millisecond + testProducingMessages(t, config) +} + +func TestFuncMultiPartitionProduce(t *testing.T) { + setupFunctionalTest(t) + defer teardownFunctionalTest(t) + + config := NewConfig() + config.ChannelBufferSize = 20 + config.Producer.Flush.Frequency = 50 * time.Millisecond + config.Producer.Flush.Messages = 200 + config.Producer.Return.Successes = true + producer, err := NewSyncProducer(kafkaBrokers, config) + if err != nil { + t.Fatal(err) + } + + var wg sync.WaitGroup + wg.Add(TestBatchSize) + + for i := 1; i <= TestBatchSize; i++ { + go func(i int) { + defer wg.Done() + msg := &ProducerMessage{Topic: "test.64", Key: nil, Value: StringEncoder(fmt.Sprintf("hur %d", i))} + if _, _, err := producer.SendMessage(msg); err != nil { + t.Error(i, err) + } + }(i) + } + + wg.Wait() + if err := producer.Close(); err != nil { + t.Error(err) + } +} + +func TestFuncProducingToInvalidTopic(t *testing.T) { + setupFunctionalTest(t) + defer teardownFunctionalTest(t) + + producer, err := NewSyncProducer(kafkaBrokers, nil) + if err != nil { + t.Fatal(err) + } + + if _, _, err := producer.SendMessage(&ProducerMessage{Topic: "in/valid"}); err != ErrUnknownTopicOrPartition { + t.Error("Expected ErrUnknownTopicOrPartition, found", err) + } + + if _, _, err := producer.SendMessage(&ProducerMessage{Topic: "in/valid"}); err != ErrUnknownTopicOrPartition { + t.Error("Expected ErrUnknownTopicOrPartition, found", err) + } + + safeClose(t, producer) +} + +func testProducingMessages(t *testing.T, config *Config) { + setupFunctionalTest(t) + defer teardownFunctionalTest(t) + + config.Producer.Return.Successes = true + config.Consumer.Return.Errors = true + + client, err := NewClient(kafkaBrokers, config) + if err != nil { + t.Fatal(err) + } + + master, err := NewConsumerFromClient(client) + if err != nil { + t.Fatal(err) + } + consumer, err := master.ConsumePartition("test.1", 0, OffsetNewest) + if err != nil { + t.Fatal(err) + } + + producer, err := NewAsyncProducerFromClient(client) + if err != nil { + t.Fatal(err) + } + + expectedResponses := TestBatchSize + for i := 1; i <= TestBatchSize; { + msg := &ProducerMessage{Topic: "test.1", Key: nil, Value: StringEncoder(fmt.Sprintf("testing %d", i))} + select { + case producer.Input() <- msg: + i++ + case ret := <-producer.Errors(): + t.Fatal(ret.Err) + case <-producer.Successes(): + expectedResponses-- + } + } + for expectedResponses > 0 { + select { + case ret := <-producer.Errors(): + t.Fatal(ret.Err) + case <-producer.Successes(): + expectedResponses-- + } + } + safeClose(t, producer) + + for i := 1; i <= TestBatchSize; i++ { + select { + case <-time.After(10 * time.Second): + t.Fatal("Not received any more events in the last 10 seconds.") + + case err := <-consumer.Errors(): + t.Error(err) + + case message := <-consumer.Messages(): + if string(message.Value) != fmt.Sprintf("testing %d", i) { + t.Fatalf("Unexpected message with index %d: %s", i, message.Value) + } + } + + } + safeClose(t, consumer) + safeClose(t, client) +} + +// Benchmarks + +func BenchmarkProducerSmall(b *testing.B) { + benchmarkProducer(b, nil, "test.64", ByteEncoder(make([]byte, 128))) +} +func BenchmarkProducerMedium(b *testing.B) { + benchmarkProducer(b, nil, "test.64", ByteEncoder(make([]byte, 1024))) +} +func BenchmarkProducerLarge(b *testing.B) { + benchmarkProducer(b, nil, "test.64", ByteEncoder(make([]byte, 8192))) +} +func BenchmarkProducerSmallSinglePartition(b *testing.B) { + benchmarkProducer(b, nil, "test.1", ByteEncoder(make([]byte, 128))) +} +func BenchmarkProducerMediumSnappy(b *testing.B) { + conf := NewConfig() + conf.Producer.Compression = CompressionSnappy + benchmarkProducer(b, conf, "test.1", ByteEncoder(make([]byte, 1024))) +} + +func benchmarkProducer(b *testing.B, conf *Config, topic string, value Encoder) { + setupFunctionalTest(b) + defer teardownFunctionalTest(b) + + producer, err := NewAsyncProducer(kafkaBrokers, conf) + if err != nil { + b.Fatal(err) + } + + b.ResetTimer() + + for i := 1; i <= b.N; { + msg := &ProducerMessage{Topic: topic, Key: StringEncoder(fmt.Sprintf("%d", i)), Value: value} + select { + case producer.Input() <- msg: + i++ + case ret := <-producer.Errors(): + b.Fatal(ret.Err) + } + } + safeClose(b, producer) +} diff --git a/vendor/github.com/Shopify/sarama/functional_test.go b/vendor/github.com/Shopify/sarama/functional_test.go new file mode 100644 index 000000000000..846eb29f9fd6 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/functional_test.go @@ -0,0 +1,148 @@ +package sarama + +import ( + "log" + "math/rand" + "net" + "os" + "strconv" + "strings" + "testing" + "time" + + toxiproxy "github.com/Shopify/toxiproxy/client" +) + +const ( + VagrantToxiproxy = "http://192.168.100.67:8474" + VagrantKafkaPeers = "192.168.100.67:9091,192.168.100.67:9092,192.168.100.67:9093,192.168.100.67:9094,192.168.100.67:9095" + VagrantZookeeperPeers = "192.168.100.67:2181,192.168.100.67:2182,192.168.100.67:2183,192.168.100.67:2184,192.168.100.67:2185" +) + +var ( + kafkaAvailable, kafkaRequired bool + kafkaBrokers []string + + proxyClient *toxiproxy.Client + Proxies map[string]*toxiproxy.Proxy + ZKProxies = []string{"zk1", "zk2", "zk3", "zk4", "zk5"} + KafkaProxies = []string{"kafka1", "kafka2", "kafka3", "kafka4", "kafka5"} +) + +func init() { + if os.Getenv("DEBUG") == "true" { + Logger = log.New(os.Stdout, "[sarama] ", log.LstdFlags) + } + + seed := time.Now().UTC().UnixNano() + if tmp := os.Getenv("TEST_SEED"); tmp != "" { + seed, _ = strconv.ParseInt(tmp, 0, 64) + } + Logger.Println("Using random seed:", seed) + rand.Seed(seed) + + proxyAddr := os.Getenv("TOXIPROXY_ADDR") + if proxyAddr == "" { + proxyAddr = VagrantToxiproxy + } + proxyClient = toxiproxy.NewClient(proxyAddr) + + kafkaPeers := os.Getenv("KAFKA_PEERS") + if kafkaPeers == "" { + kafkaPeers = VagrantKafkaPeers + } + kafkaBrokers = strings.Split(kafkaPeers, ",") + + if c, err := net.DialTimeout("tcp", kafkaBrokers[0], 5*time.Second); err == nil { + if err = c.Close(); err == nil { + kafkaAvailable = true + } + } + + kafkaRequired = os.Getenv("CI") != "" +} + +func checkKafkaAvailability(t testing.TB) { + if !kafkaAvailable { + if kafkaRequired { + t.Fatalf("Kafka broker is not available on %s. Set KAFKA_PEERS to connect to Kafka on a different location.", kafkaBrokers[0]) + } else { + t.Skipf("Kafka broker is not available on %s. Set KAFKA_PEERS to connect to Kafka on a different location.", kafkaBrokers[0]) + } + } +} + +func checkKafkaVersion(t testing.TB, requiredVersion string) { + kafkaVersion := os.Getenv("KAFKA_VERSION") + if kafkaVersion == "" { + t.Logf("No KAFKA_VERSION set. This test requires Kafka version %s or higher. Continuing...", requiredVersion) + } else { + available := parseKafkaVersion(kafkaVersion) + required := parseKafkaVersion(requiredVersion) + if !available.satisfies(required) { + t.Skipf("Kafka version %s is required for this test; you have %s. Skipping...", requiredVersion, kafkaVersion) + } + } +} + +func resetProxies(t testing.TB) { + if err := proxyClient.ResetState(); err != nil { + t.Error(err) + } + Proxies = nil +} + +func fetchProxies(t testing.TB) { + var err error + Proxies, err = proxyClient.Proxies() + if err != nil { + t.Fatal(err) + } +} + +func SaveProxy(t *testing.T, px string) { + if err := Proxies[px].Save(); err != nil { + t.Fatal(err) + } +} + +func setupFunctionalTest(t testing.TB) { + checkKafkaAvailability(t) + resetProxies(t) + fetchProxies(t) +} + +func teardownFunctionalTest(t testing.TB) { + resetProxies(t) +} + +type kafkaVersion []int + +func (kv kafkaVersion) satisfies(other kafkaVersion) bool { + var ov int + for index, v := range kv { + if len(other) <= index { + ov = 0 + } else { + ov = other[index] + } + + if v < ov { + return false + } else if v > ov { + return true + } + } + return true +} + +func parseKafkaVersion(version string) kafkaVersion { + numbers := strings.Split(version, ".") + result := make(kafkaVersion, 0, len(numbers)) + for _, number := range numbers { + nr, _ := strconv.Atoi(number) + result = append(result, nr) + } + + return result +} diff --git a/vendor/github.com/Shopify/sarama/heartbeat_request.go b/vendor/github.com/Shopify/sarama/heartbeat_request.go new file mode 100644 index 000000000000..b89d290f12bb --- /dev/null +++ b/vendor/github.com/Shopify/sarama/heartbeat_request.go @@ -0,0 +1,43 @@ +package sarama + +type HeartbeatRequest struct { + GroupId string + GenerationId int32 + MemberId string +} + +func (r *HeartbeatRequest) encode(pe packetEncoder) error { + if err := pe.putString(r.GroupId); err != nil { + return err + } + + pe.putInt32(r.GenerationId) + + if err := pe.putString(r.MemberId); err != nil { + return err + } + + return nil +} + +func (r *HeartbeatRequest) decode(pd packetDecoder) (err error) { + if r.GroupId, err = pd.getString(); err != nil { + return + } + if r.GenerationId, err = pd.getInt32(); err != nil { + return + } + if r.MemberId, err = pd.getString(); err != nil { + return + } + + return nil +} + +func (r *HeartbeatRequest) key() int16 { + return 12 +} + +func (r *HeartbeatRequest) version() int16 { + return 0 +} diff --git a/vendor/github.com/Shopify/sarama/heartbeat_request_test.go b/vendor/github.com/Shopify/sarama/heartbeat_request_test.go new file mode 100644 index 000000000000..da6cd18f5a3d --- /dev/null +++ b/vendor/github.com/Shopify/sarama/heartbeat_request_test.go @@ -0,0 +1,21 @@ +package sarama + +import "testing" + +var ( + basicHeartbeatRequest = []byte{ + 0, 3, 'f', 'o', 'o', // Group ID + 0x00, 0x01, 0x02, 0x03, // Generatiuon ID + 0, 3, 'b', 'a', 'z', // Member ID + } +) + +func TestHeartbeatRequest(t *testing.T) { + var request *HeartbeatRequest + + request = new(HeartbeatRequest) + request.GroupId = "foo" + request.GenerationId = 66051 + request.MemberId = "baz" + testRequest(t, "basic", request, basicHeartbeatRequest) +} diff --git a/vendor/github.com/Shopify/sarama/heartbeat_response.go b/vendor/github.com/Shopify/sarama/heartbeat_response.go new file mode 100644 index 000000000000..b48b8c1f3523 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/heartbeat_response.go @@ -0,0 +1,20 @@ +package sarama + +type HeartbeatResponse struct { + Err KError +} + +func (r *HeartbeatResponse) encode(pe packetEncoder) error { + pe.putInt16(int16(r.Err)) + return nil +} + +func (r *HeartbeatResponse) decode(pd packetDecoder) error { + if kerr, err := pd.getInt16(); err != nil { + return err + } else { + r.Err = KError(kerr) + } + + return nil +} diff --git a/vendor/github.com/Shopify/sarama/heartbeat_response_test.go b/vendor/github.com/Shopify/sarama/heartbeat_response_test.go new file mode 100644 index 000000000000..f8545781a441 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/heartbeat_response_test.go @@ -0,0 +1,18 @@ +package sarama + +import "testing" + +var ( + heartbeatResponseNoError = []byte{ + 0x00, 0x00} +) + +func TestHeartbeatResponse(t *testing.T) { + var response *HeartbeatResponse + + response = new(HeartbeatResponse) + testDecodable(t, "no error", response, heartbeatResponseNoError) + if response.Err != ErrNoError { + t.Error("Decoding error failed: no error expected but found", response.Err) + } +} diff --git a/vendor/github.com/Shopify/sarama/join_group_request.go b/vendor/github.com/Shopify/sarama/join_group_request.go new file mode 100644 index 000000000000..5884d79d4490 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/join_group_request.go @@ -0,0 +1,104 @@ +package sarama + +type JoinGroupRequest struct { + GroupId string + SessionTimeout int32 + MemberId string + ProtocolType string + GroupProtocols map[string][]byte +} + +func (r *JoinGroupRequest) encode(pe packetEncoder) error { + if err := pe.putString(r.GroupId); err != nil { + return err + } + pe.putInt32(r.SessionTimeout) + if err := pe.putString(r.MemberId); err != nil { + return err + } + if err := pe.putString(r.ProtocolType); err != nil { + return err + } + + if err := pe.putArrayLength(len(r.GroupProtocols)); err != nil { + return err + } + for name, metadata := range r.GroupProtocols { + if err := pe.putString(name); err != nil { + return err + } + if err := pe.putBytes(metadata); err != nil { + return err + } + } + + return nil +} + +func (r *JoinGroupRequest) decode(pd packetDecoder) (err error) { + if r.GroupId, err = pd.getString(); err != nil { + return + } + + if r.SessionTimeout, err = pd.getInt32(); err != nil { + return + } + + if r.MemberId, err = pd.getString(); err != nil { + return + } + + if r.ProtocolType, err = pd.getString(); err != nil { + return + } + + n, err := pd.getArrayLength() + if err != nil { + return err + } + if n == 0 { + return nil + } + + r.GroupProtocols = make(map[string][]byte) + for i := 0; i < n; i++ { + name, err := pd.getString() + if err != nil { + return err + } + metadata, err := pd.getBytes() + if err != nil { + return err + } + + r.GroupProtocols[name] = metadata + } + + return nil +} + +func (r *JoinGroupRequest) key() int16 { + return 11 +} + +func (r *JoinGroupRequest) version() int16 { + return 0 +} + +func (r *JoinGroupRequest) AddGroupProtocol(name string, metadata []byte) { + if r.GroupProtocols == nil { + r.GroupProtocols = make(map[string][]byte) + } + + r.GroupProtocols[name] = metadata +} + +func (r *JoinGroupRequest) AddGroupProtocolMetadata(name string, metadata *ConsumerGroupMemberMetadata) error { + bin, err := encode(metadata) + if err != nil { + return err + } + + r.AddGroupProtocol(name, bin) + return nil +} diff --git a/vendor/github.com/Shopify/sarama/join_group_request_test.go b/vendor/github.com/Shopify/sarama/join_group_request_test.go new file mode 100644 index 000000000000..8a6448c0e190 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/join_group_request_test.go @@ -0,0 +1,41 @@ +package sarama + +import "testing" + +var ( + joinGroupRequestNoProtocols = []byte{ + 0, 9, 'T', 'e', 's', 't', 'G', 'r', 'o', 'u', 'p', // Group ID + 0, 0, 0, 100, // Session timeout + 0, 0, // Member ID + 0, 8, 'c', 'o', 'n', 's', 'u', 'm', 'e', 'r', // Protocol Type + 0, 0, 0, 0, // 0 protocol groups + } + + joinGroupRequestOneProtocol = []byte{ + 0, 9, 'T', 'e', 's', 't', 'G', 'r', 'o', 'u', 'p', // Group ID + 0, 0, 0, 100, // Session timeout + 0, 11, 'O', 'n', 'e', 'P', 'r', 'o', 't', 'o', 'c', 'o', 'l', // Member ID + 0, 8, 'c', 'o', 'n', 's', 'u', 'm', 'e', 'r', // Protocol Type + 0, 0, 0, 1, // 1 group protocol + 0, 3, 'o', 'n', 'e', // Protocol name + 0, 0, 0, 3, 0x01, 0x02, 0x03, // protocol metadata + } +) + +func TestJoinGroupRequest(t *testing.T) { + var request *JoinGroupRequest + + request = new(JoinGroupRequest) + request.GroupId = "TestGroup" + request.SessionTimeout = 100 + request.ProtocolType = "consumer" + testRequest(t, "no protocols", request, joinGroupRequestNoProtocols) + + request = new(JoinGroupRequest) + request.GroupId = "TestGroup" + request.SessionTimeout = 100 + request.MemberId = "OneProtocol" + request.ProtocolType = "consumer" + request.AddGroupProtocol("one", []byte{0x01, 0x02, 0x03}) + testRequest(t, "one protocol", request, joinGroupRequestOneProtocol) +} diff --git a/vendor/github.com/Shopify/sarama/join_group_response.go b/vendor/github.com/Shopify/sarama/join_group_response.go new file mode 100644 index 000000000000..16f6b9b40615 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/join_group_response.go @@ -0,0 +1,102 @@ +package sarama + +type JoinGroupResponse struct { + Err KError + GenerationId int32 + GroupProtocol string + LeaderId string + MemberId string + Members map[string][]byte +} + +func (r *JoinGroupResponse) GetMembers() (map[string]ConsumerGroupMemberMetadata, error) { + members := make(map[string]ConsumerGroupMemberMetadata, len(r.Members)) + for id, bin := range r.Members { + meta := new(ConsumerGroupMemberMetadata) + if err := decode(bin, meta); err != nil { + return nil, err + } + members[id] = *meta + } + return members, nil +} + +func (r *JoinGroupResponse) encode(pe packetEncoder) error { + pe.putInt16(int16(r.Err)) + pe.putInt32(r.GenerationId) + + if err := pe.putString(r.GroupProtocol); err != nil { + return err + } + if err := pe.putString(r.LeaderId); err != nil { + return err + } + if err := pe.putString(r.MemberId); err != nil { + return err + } + + if err := pe.putArrayLength(len(r.Members)); err != nil { + return err + } + + for memberId, memberMetadata := range r.Members { + if err := pe.putString(memberId); err != nil { + return err + } + + if err := pe.putBytes(memberMetadata); err != nil { + return err + } + } + + return nil +} + +func (r *JoinGroupResponse) decode(pd packetDecoder) (err error) { + if kerr, err := pd.getInt16(); err != nil { + return err + } else { + r.Err = KError(kerr) + } + + if r.GenerationId, err = pd.getInt32(); err != nil { + return + } + + if r.GroupProtocol, err = pd.getString(); err != nil { + return + } + + if r.LeaderId, err = pd.getString(); err != nil { + return + } + + if r.MemberId, err = pd.getString(); err != nil { + return + } + + n, err := pd.getArrayLength() + if err != nil { + return err + } + if n == 0 { + return nil + } + + r.Members = make(map[string][]byte) + for i := 0; i < n; i++ { + memberId, err := pd.getString() + if err != nil { + return err + } + + memberMetadata, err := pd.getBytes() + if err != nil { + return err + } + + r.Members[memberId] = memberMetadata + } + + return nil +} diff --git a/vendor/github.com/Shopify/sarama/join_group_response_test.go b/vendor/github.com/Shopify/sarama/join_group_response_test.go new file mode 100644 index 000000000000..560a726ff001 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/join_group_response_test.go @@ -0,0 +1,98 @@ +package sarama + +import ( + "reflect" + "testing" +) + +var ( + joinGroupResponseNoError = []byte{ + 0x00, 0x00, // No error + 0x00, 0x01, 0x02, 0x03, // Generation ID + 0, 8, 'p', 'r', 'o', 't', 'o', 'c', 'o', 'l', // Protocol name chosen + 0, 3, 'f', 'o', 'o', // Leader ID + 0, 3, 'b', 'a', 'r', // Member ID + 0, 0, 0, 0, // No member info + } + + joinGroupResponseWithError = []byte{ + 0, 23, // Error: inconsistent group protocol + 0x00, 0x00, 0x00, 0x00, // Generation ID + 0, 0, // Protocol name chosen + 0, 0, // Leader ID + 0, 0, // Member ID + 0, 0, 0, 0, // No member info + } + + joinGroupResponseLeader = []byte{ + 0x00, 0x00, // No error + 0x00, 0x01, 0x02, 0x03, // Generation ID + 0, 8, 'p', 'r', 'o', 't', 'o', 'c', 'o', 'l', // Protocol name chosen + 0, 3, 'f', 'o', 'o', // Leader ID + 0, 3, 'f', 'o', 'o', // Member ID == Leader ID + 0, 0, 0, 1, // 1 member + 0, 3, 'f', 'o', 'o', // Member ID + 0, 0, 0, 3, 0x01, 0x02, 0x03, // Member metadata + } +) + +func TestJoinGroupResponse(t *testing.T) { + var response *JoinGroupResponse + + response = new(JoinGroupResponse) + testDecodable(t, "no error", response, joinGroupResponseNoError) + if response.Err != ErrNoError { + t.Error("Decoding Err failed: no error expected but found", response.Err) + } + if response.GenerationId != 66051 { + t.Error("Decoding GenerationId failed, found:", response.GenerationId) + } + if response.LeaderId != "foo" { + t.Error("Decoding LeaderId failed, found:", response.LeaderId) + } + if response.MemberId != "bar" { + t.Error("Decoding MemberId failed, found:", response.MemberId) + } + if len(response.Members) != 0 { + t.Error("Decoding Members failed, found:", response.Members) + } + + response = new(JoinGroupResponse) + testDecodable(t, "with error", response, joinGroupResponseWithError) + if response.Err != ErrInconsistentGroupProtocol { + t.Error("Decoding Err failed: ErrInconsistentGroupProtocol expected but found", response.Err) + } + if response.GenerationId != 0 { + t.Error("Decoding GenerationId failed, found:", response.GenerationId) + } + if response.LeaderId != "" { + t.Error("Decoding LeaderId failed, found:", response.LeaderId) + } + if response.MemberId != "" { + t.Error("Decoding MemberId failed, found:", response.MemberId) + } + if len(response.Members) != 0 { + t.Error("Decoding Members failed, found:", response.Members) + } + + response = new(JoinGroupResponse) + testDecodable(t, "with error", response, joinGroupResponseLeader) + if response.Err != ErrNoError { + t.Error("Decoding Err failed: ErrNoError expected but found", response.Err) + } + if response.GenerationId != 66051 { + t.Error("Decoding GenerationId failed, found:", response.GenerationId) + } + if response.LeaderId != "foo" { + t.Error("Decoding LeaderId failed, found:", response.LeaderId) + } + if response.MemberId != "foo" { + t.Error("Decoding MemberId failed, found:", response.MemberId) + } + if len(response.Members) != 1 { + t.Error("Decoding Members failed, found:", response.Members) + } + if !reflect.DeepEqual(response.Members["foo"], []byte{0x01, 0x02, 0x03}) { + t.Error("Decoding foo member failed, found:", response.Members["foo"]) + } +} diff --git a/vendor/github.com/Shopify/sarama/leave_group_request.go b/vendor/github.com/Shopify/sarama/leave_group_request.go new file mode 100644 index 000000000000..cdb4d14fd60a --- /dev/null +++ b/vendor/github.com/Shopify/sarama/leave_group_request.go @@ -0,0 +1,36 @@ +package sarama + +type LeaveGroupRequest struct { + GroupId string + MemberId string +} + +func (r *LeaveGroupRequest) encode(pe packetEncoder) error { + if err := pe.putString(r.GroupId); err != nil { + return err + } + if err := pe.putString(r.MemberId); err != nil { + return err + } + + return nil +} + +func (r *LeaveGroupRequest) decode(pd packetDecoder) (err error) { + if r.GroupId, err = pd.getString(); err != nil { + return + } + if r.MemberId, err = pd.getString(); err != nil { + return + } + + return nil +} + +func (r *LeaveGroupRequest) key() int16 { + return 13 +} + +func (r *LeaveGroupRequest) version() int16 { + return 0 +} diff --git a/vendor/github.com/Shopify/sarama/leave_group_request_test.go b/vendor/github.com/Shopify/sarama/leave_group_request_test.go new file mode 100644 index 000000000000..c1fed6d25f1b --- /dev/null +++ b/vendor/github.com/Shopify/sarama/leave_group_request_test.go @@ -0,0 +1,19 @@ +package sarama + +import "testing" + +var ( + basicLeaveGroupRequest = []byte{ + 0, 3, 'f', 'o', 'o', + 0, 3, 'b', 'a', 'r', + } +) + +func TestLeaveGroupRequest(t *testing.T) { + var request *LeaveGroupRequest + + request = new(LeaveGroupRequest) + request.GroupId = "foo" + request.MemberId = "bar" + testRequest(t, "basic", request, basicLeaveGroupRequest) +} diff --git a/vendor/github.com/Shopify/sarama/leave_group_response.go b/vendor/github.com/Shopify/sarama/leave_group_response.go new file mode 100644 index 000000000000..bad1dba2f2b4 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/leave_group_response.go @@ -0,0 +1,20 @@ +package sarama + +type LeaveGroupResponse struct { + Err KError +} + +func (r *LeaveGroupResponse) encode(pe packetEncoder) error { + pe.putInt16(int16(r.Err)) + return nil +} + +func (r *LeaveGroupResponse) decode(pd packetDecoder) (err error) { + if kerr, err := pd.getInt16(); err != nil { + return err + } else { + r.Err = KError(kerr) + } + + return nil +} diff --git a/vendor/github.com/Shopify/sarama/leave_group_response_test.go b/vendor/github.com/Shopify/sarama/leave_group_response_test.go new file mode 100644 index 000000000000..0e9fb74b4fab --- /dev/null +++ b/vendor/github.com/Shopify/sarama/leave_group_response_test.go @@ -0,0 +1,24 @@ +package sarama + +import "testing" + +var ( + leaveGroupResponseNoError = []byte{0x00, 0x00} + leaveGroupResponseWithError = []byte{0, 25} +) + +func TestLeaveGroupResponse(t *testing.T) { + var response *LeaveGroupResponse + + response = new(LeaveGroupResponse) + testDecodable(t, "no error", response, leaveGroupResponseNoError) + if response.Err != ErrNoError { + t.Error("Decoding error failed: no error expected but found", response.Err) + } + + response = new(LeaveGroupResponse) + testDecodable(t, "with error", response, leaveGroupResponseWithError) + if response.Err != ErrUnknownMemberId { + t.Error("Decoding error failed: ErrUnknownMemberId expected but found", response.Err) + } +} diff --git a/vendor/github.com/Shopify/sarama/length_field.go b/vendor/github.com/Shopify/sarama/length_field.go new file mode 100644 index 000000000000..70078be5d9f2 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/length_field.go @@ -0,0 +1,29 @@ +package sarama + +import "encoding/binary" + +// LengthField implements the PushEncoder and PushDecoder interfaces for calculating 4-byte lengths. +type lengthField struct { + startOffset int +} + +func (l *lengthField) saveOffset(in int) { + l.startOffset = in +} + +func (l *lengthField) reserveLength() int { + return 4 +} + +func (l *lengthField) run(curOffset int, buf []byte) error { + binary.BigEndian.PutUint32(buf[l.startOffset:], uint32(curOffset-l.startOffset-4)) + return nil +} + +func (l *lengthField) check(curOffset int, buf []byte) error { + if uint32(curOffset-l.startOffset-4) != binary.BigEndian.Uint32(buf[l.startOffset:]) { + return PacketDecodingError{"length field invalid"} + } + + return nil +} diff --git a/vendor/github.com/Shopify/sarama/list_groups_request.go b/vendor/github.com/Shopify/sarama/list_groups_request.go new file mode 100644 index 000000000000..4d74c2665ae2 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/list_groups_request.go @@ -0,0 +1,20 @@ +package sarama + +type ListGroupsRequest struct { +} + +func (r *ListGroupsRequest) encode(pe packetEncoder) error { + return nil +} + +func (r *ListGroupsRequest) decode(pd packetDecoder) (err error) { + return nil +} + +func (r *ListGroupsRequest) key() int16 { + return 16 +} + +func (r *ListGroupsRequest) version() int16 { + return 0 +} diff --git a/vendor/github.com/Shopify/sarama/list_groups_request_test.go b/vendor/github.com/Shopify/sarama/list_groups_request_test.go new file mode 100644 index 000000000000..2e977d9a581f --- /dev/null +++ b/vendor/github.com/Shopify/sarama/list_groups_request_test.go @@ -0,0 +1,7 @@ +package sarama + +import "testing" + +func TestListGroupsRequest(t *testing.T) { + testRequest(t, "ListGroupsRequest", &ListGroupsRequest{}, []byte{}) +} diff --git a/vendor/github.com/Shopify/sarama/list_groups_response.go b/vendor/github.com/Shopify/sarama/list_groups_response.go new file mode 100644 index 000000000000..2f5314902185 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/list_groups_response.go @@ -0,0 +1,56 @@ +package sarama + +type ListGroupsResponse struct { + Err KError + Groups map[string]string +} + +func (r *ListGroupsResponse) encode(pe packetEncoder) error { + pe.putInt16(int16(r.Err)) + + if err := pe.putArrayLength(len(r.Groups)); err != nil { + return err + } + for groupId, protocolType := range r.Groups { + if err := pe.putString(groupId); err != nil { + return err + } + if err := pe.putString(protocolType); err != nil { + return err + } + } + + return nil +} + +func (r *ListGroupsResponse) decode(pd packetDecoder) error { + if kerr, err := pd.getInt16(); err != nil { + return err + } else { + r.Err = KError(kerr) + } + + n, err := pd.getArrayLength() + if err != nil { + return err + } + if n == 0 { + return nil + } + + r.Groups = make(map[string]string) + for i := 0; i < n; i++ { + groupId, err := pd.getString() + if err != nil { + return err + } + protocolType, err := pd.getString() + if err != nil { + return err + } + + r.Groups[groupId] = protocolType + } + + return nil +} diff --git a/vendor/github.com/Shopify/sarama/list_groups_response_test.go b/vendor/github.com/Shopify/sarama/list_groups_response_test.go new file mode 100644 index 000000000000..626e7f9abe59 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/list_groups_response_test.go @@ -0,0 +1,58 @@ +package sarama + +import ( + "testing" +) + +var ( + listGroupsResponseEmpty = []byte{ + 0, 0, // no error + 0, 0, 0, 0, // no groups + } + + listGroupsResponseError = []byte{ + 0, 31, // no error + 0, 0, 0, 0, // ErrClusterAuthorizationFailed + } + + listGroupsResponseWithConsumer = []byte{ + 0, 0, // no error + 0, 0, 0, 1, // 1 group + 0, 3, 'f', 'o', 'o', // group name + 0, 8, 'c', 'o', 'n', 's', 'u', 'm', 'e', 'r', // protocol type + } +) + +func TestListGroupsResponse(t *testing.T) { + var response *ListGroupsResponse + + response = new(ListGroupsResponse) + testDecodable(t, "no error", response, listGroupsResponseEmpty) + if response.Err != ErrNoError { + t.Error("Expected no gerror, found:", response.Err) + } + if len(response.Groups) != 0 { + t.Error("Expected no groups") + } + + response = new(ListGroupsResponse) + testDecodable(t, "no error", response, listGroupsResponseError) + if response.Err != ErrClusterAuthorizationFailed { + t.Error("Expected no gerror, found:", response.Err) + } + if len(response.Groups) != 0 { + t.Error("Expected no groups") + } + + response = new(ListGroupsResponse) + testDecodable(t, "no error", response, listGroupsResponseWithConsumer) + if response.Err != ErrNoError { + t.Error("Expected no gerror, found:", response.Err) + } + if len(response.Groups) != 1 { + t.Error("Expected one group") + } + if response.Groups["foo"] != "consumer" { + t.Error("Expected foo group to use consumer protocol") + } +} diff --git a/vendor/github.com/Shopify/sarama/message.go b/vendor/github.com/Shopify/sarama/message.go new file mode 100644 index 000000000000..c4bdb9e9d77f --- /dev/null +++ b/vendor/github.com/Shopify/sarama/message.go @@ -0,0 +1,153 @@ +package sarama + +import ( + "bytes" + "compress/gzip" + "fmt" + "io/ioutil" +) + +// CompressionCodec represents the various compression codecs recognized by Kafka in messages. +type CompressionCodec int8 + +// only the last two bits are really used +const compressionCodecMask int8 = 0x03 + +const ( + CompressionNone CompressionCodec = 0 + CompressionGZIP CompressionCodec = 1 + CompressionSnappy CompressionCodec = 2 +) + +// The spec just says: "This is a version id used to allow backwards compatible evolution of the message +// binary format." but it doesn't say what the current value is, so presumably 0... +const messageFormat int8 = 0 + +type Message struct { + Codec CompressionCodec // codec used to compress the message contents + Key []byte // the message key, may be nil + Value []byte // the message contents + Set *MessageSet // the message set a message might wrap + + compressedCache []byte +} + +func (m *Message) encode(pe packetEncoder) error { + pe.push(&crc32Field{}) + + pe.putInt8(messageFormat) + + attributes := int8(m.Codec) & compressionCodecMask + pe.putInt8(attributes) + + err := pe.putBytes(m.Key) + if err != nil { + return err + } + + var payload []byte + + if m.compressedCache != nil { + payload = m.compressedCache + m.compressedCache = nil + } else { + switch m.Codec { + case CompressionNone: + payload = m.Value + case CompressionGZIP: + var buf bytes.Buffer + writer := gzip.NewWriter(&buf) + if _, err = writer.Write(m.Value); err != nil { + return err + } + if err = writer.Close(); err != nil { + return err + } + m.compressedCache = buf.Bytes() + payload = m.compressedCache + case CompressionSnappy: + tmp := snappyEncode(m.Value) + m.compressedCache = tmp + payload = m.compressedCache + default: + return PacketEncodingError{fmt.Sprintf("unsupported compression codec (%d)", m.Codec)} + } + } + + if err = pe.putBytes(payload); err != nil { + return err + } + + return pe.pop() +} + +func (m *Message) decode(pd packetDecoder) (err error) { + err = pd.push(&crc32Field{}) + if err != nil { + return err + } + + format, err := pd.getInt8() + if err != nil { + return err + } + if format != messageFormat { + return PacketDecodingError{"unexpected messageFormat"} + } + + attribute, err := pd.getInt8() + if err != nil { + return err + } + m.Codec = CompressionCodec(attribute & compressionCodecMask) + + m.Key, err = pd.getBytes() + if err != nil { + return err + } + + m.Value, err = pd.getBytes() + if err != nil { + return err + } + + switch m.Codec { + case CompressionNone: + // nothing to do + case CompressionGZIP: + if m.Value == nil { + return PacketDecodingError{"GZIP compression specified, but no data to uncompress"} + } + reader, err := gzip.NewReader(bytes.NewReader(m.Value)) + if err != nil { + return err + } + if m.Value, err = ioutil.ReadAll(reader); err != nil { + return err + } + if err := m.decodeSet(); err != nil { + return err + } + case CompressionSnappy: + if m.Value == nil { + return PacketDecodingError{"Snappy compression specified, but no data to uncompress"} + } + if m.Value, err = snappyDecode(m.Value); err != nil { + return err + } + if err := m.decodeSet(); err != nil { + return err + } + default: + return PacketDecodingError{fmt.Sprintf("invalid compression specified (%d)", m.Codec)} + } + + return pd.pop() +} + +// decodes a message set from a previousy encoded bulk-message +func (m *Message) decodeSet() (err error) { + pd := realDecoder{raw: m.Value} + m.Set = &MessageSet{} + return m.Set.decode(&pd) +} diff --git a/vendor/github.com/Shopify/sarama/message_set.go b/vendor/github.com/Shopify/sarama/message_set.go new file mode 100644 index 000000000000..f028784e51a0 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/message_set.go @@ -0,0 +1,89 @@ +package sarama + +type MessageBlock struct { + Offset int64 + Msg *Message +} + +// Messages convenience helper which returns either all the +// messages that are wrapped in this block +func (msb *MessageBlock) Messages() []*MessageBlock { + if msb.Msg.Set != nil { + return msb.Msg.Set.Messages + } + return []*MessageBlock{msb} +} + +func (msb *MessageBlock) encode(pe packetEncoder) error { + pe.putInt64(msb.Offset) + pe.push(&lengthField{}) + err := msb.Msg.encode(pe) + if err != nil { + return err + } + return pe.pop() +} + +func (msb *MessageBlock) decode(pd packetDecoder) (err error) { + if msb.Offset, err = pd.getInt64(); err != nil { + return err + } + + if err = pd.push(&lengthField{}); err != nil { + return err + } + + msb.Msg = new(Message) + if err = msb.Msg.decode(pd); err != nil { + return err + } + + if err = pd.pop(); err != nil { + return err + } + + return nil +} + +type MessageSet struct { + PartialTrailingMessage bool // whether the set on the wire contained an incomplete trailing MessageBlock + Messages []*MessageBlock +} + +func (ms *MessageSet) encode(pe packetEncoder) error { + for i := range ms.Messages { + err := ms.Messages[i].encode(pe) + if err != nil { + return err + } + } + return nil +} + +func (ms *MessageSet) decode(pd packetDecoder) (err error) { + ms.Messages = nil + + for pd.remaining() > 0 { + msb := new(MessageBlock) + err = msb.decode(pd) + switch err { + case nil: + ms.Messages = append(ms.Messages, msb) + case ErrInsufficientData: + // As an optimization the server is allowed to return a partial message at the + // end of the message set. Clients should handle this case. So we just ignore such things. + ms.PartialTrailingMessage = true + return nil + default: + return err + } + } + + return nil +} + +func (ms *MessageSet) addMessage(msg *Message) { + block := new(MessageBlock) + block.Msg = msg + ms.Messages = append(ms.Messages, block) +} diff --git a/vendor/github.com/Shopify/sarama/message_test.go b/vendor/github.com/Shopify/sarama/message_test.go new file mode 100644 index 000000000000..1dae896fee24 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/message_test.go @@ -0,0 +1,113 @@ +package sarama + +import "testing" + +var ( + emptyMessage = []byte{ + 167, 236, 104, 3, // CRC + 0x00, // magic version byte + 0x00, // attribute flags + 0xFF, 0xFF, 0xFF, 0xFF, // key + 0xFF, 0xFF, 0xFF, 0xFF} // value + + emptyGzipMessage = []byte{ + 97, 79, 149, 90, //CRC + 0x00, // magic version byte + 0x01, // attribute flags + 0xFF, 0xFF, 0xFF, 0xFF, // key + // value + 0x00, 0x00, 0x00, 0x17, + 0x1f, 0x8b, + 0x08, + 0, 0, 9, 110, 136, 0, 255, 1, 0, 0, 255, 255, 0, 0, 0, 0, 0, 0, 0, 0} + + emptyBulkSnappyMessage = []byte{ + 180, 47, 53, 209, //CRC + 0x00, // magic version byte + 0x02, // attribute flags + 0xFF, 0xFF, 0xFF, 0xFF, // key + 0, 0, 0, 42, + 130, 83, 78, 65, 80, 80, 89, 0, // SNAPPY magic + 0, 0, 0, 1, // min version + 0, 0, 0, 1, // default version + 0, 0, 0, 22, 52, 0, 0, 25, 1, 16, 14, 227, 138, 104, 118, 25, 15, 13, 1, 8, 1, 0, 0, 62, 26, 0} + + emptyBulkGzipMessage = []byte{ + 139, 160, 63, 141, //CRC + 0x00, // magic version byte + 0x01, // attribute flags + 0xFF, 0xFF, 0xFF, 0xFF, // key + 0x00, 0x00, 0x00, 0x27, // len + 0x1f, 0x8b, // Gzip Magic + 0x08, // deflate compressed + 0, 0, 0, 0, 0, 0, 0, 99, 96, 128, 3, 190, 202, 112, 143, 7, 12, 12, 255, 129, 0, 33, 200, 192, 136, 41, 3, 0, 199, 226, 155, 70, 52, 0, 0, 0} +) + +func TestMessageEncoding(t *testing.T) { + message := Message{} + testEncodable(t, "empty", &message, emptyMessage) + + message.Value = []byte{} + message.Codec = CompressionGZIP + testEncodable(t, "empty gzip", &message, emptyGzipMessage) +} + +func TestMessageDecoding(t *testing.T) { + message := Message{} + testDecodable(t, "empty", &message, emptyMessage) + if message.Codec != CompressionNone { + t.Error("Decoding produced compression codec where there was none.") + } + if message.Key != nil { + t.Error("Decoding produced key where there was none.") + } + if message.Value != nil { + t.Error("Decoding produced value where there was none.") + } + if message.Set != nil { + t.Error("Decoding produced set where there was none.") + } + + testDecodable(t, "empty gzip", &message, emptyGzipMessage) + if message.Codec != CompressionGZIP { + t.Error("Decoding produced incorrect compression codec (was gzip).") + } + if message.Key != nil { + t.Error("Decoding produced key where there was none.") + } + if message.Value == nil || len(message.Value) != 0 { + t.Error("Decoding produced nil or content-ful value where there was an empty array.") + } +} + +func TestMessageDecodingBulkSnappy(t *testing.T) { + message := Message{} + testDecodable(t, "bulk snappy", &message, emptyBulkSnappyMessage) + if message.Codec != CompressionSnappy { + t.Errorf("Decoding produced codec %d, but expected %d.", message.Codec, CompressionSnappy) + } + if message.Key != nil { + t.Errorf("Decoding produced key %+v, but none was expected.", message.Key) + } + if message.Set == nil { + t.Error("Decoding produced no set, but one was expected.") + } else if len(message.Set.Messages) != 2 { + t.Errorf("Decoding produced a set with %d messages, but 2 were expected.", len(message.Set.Messages)) + } +} + +func TestMessageDecodingBulkGzip(t *testing.T) { + message := Message{} + testDecodable(t, "bulk gzip", &message, emptyBulkGzipMessage) + if message.Codec != CompressionGZIP { + t.Errorf("Decoding produced codec %d, but expected %d.", message.Codec, CompressionGZIP) + } + if message.Key != nil { + t.Errorf("Decoding produced key %+v, but none was expected.", message.Key) + } + if message.Set == nil { + t.Error("Decoding produced no set, but one was expected.") + } else if len(message.Set.Messages) != 2 { + t.Errorf("Decoding produced a set with %d messages, but 2 were expected.", len(message.Set.Messages)) + } +} diff --git a/vendor/github.com/Shopify/sarama/metadata_request.go b/vendor/github.com/Shopify/sarama/metadata_request.go new file mode 100644 index 000000000000..130cfd4f1196 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/metadata_request.go @@ -0,0 +1,48 @@ +package sarama + +type MetadataRequest struct { + Topics []string +} + +func (mr *MetadataRequest) encode(pe packetEncoder) error { + err := pe.putArrayLength(len(mr.Topics)) + if err != nil { + return err + } + + for i := range mr.Topics { + err = pe.putString(mr.Topics[i]) + if err != nil { + return err + } + } + return nil +} + +func (mr *MetadataRequest) decode(pd packetDecoder) error { + topicCount, err := pd.getArrayLength() + if err != nil { + return err + } + if topicCount == 0 { + return nil + } + + mr.Topics = make([]string, topicCount) + for i := range mr.Topics { + topic, err := pd.getString() + if err != nil { + return err + } + mr.Topics[i] = topic + } + return nil +} + +func (mr *MetadataRequest) key() int16 { + return 3 +} + +func (mr *MetadataRequest) version() int16 { + return 0 +} diff --git a/vendor/github.com/Shopify/sarama/metadata_request_test.go b/vendor/github.com/Shopify/sarama/metadata_request_test.go new file mode 100644 index 000000000000..44f3146e4ad3 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/metadata_request_test.go @@ -0,0 +1,29 @@ +package sarama + +import "testing" + +var ( + metadataRequestNoTopics = []byte{ + 0x00, 0x00, 0x00, 0x00} + + metadataRequestOneTopic = []byte{ + 0x00, 0x00, 0x00, 0x01, + 0x00, 0x06, 't', 'o', 'p', 'i', 'c', '1'} + + metadataRequestThreeTopics = []byte{ + 0x00, 0x00, 0x00, 0x03, + 0x00, 0x03, 'f', 'o', 'o', + 0x00, 0x03, 'b', 'a', 'r', + 0x00, 0x03, 'b', 'a', 'z'} +) + +func TestMetadataRequest(t *testing.T) { + request := new(MetadataRequest) + testRequest(t, "no topics", request, metadataRequestNoTopics) + + request.Topics = []string{"topic1"} + testRequest(t, "one topic", request, metadataRequestOneTopic) + + request.Topics = []string{"foo", "bar", "baz"} + testRequest(t, "three topics", request, metadataRequestThreeTopics) +} diff --git a/vendor/github.com/Shopify/sarama/metadata_response.go b/vendor/github.com/Shopify/sarama/metadata_response.go new file mode 100644 index 000000000000..b82221f7ed3e --- /dev/null +++ b/vendor/github.com/Shopify/sarama/metadata_response.go @@ -0,0 +1,227 @@ +package sarama + +type PartitionMetadata struct { + Err KError + ID int32 + Leader int32 + Replicas []int32 + Isr []int32 +} + +func (pm *PartitionMetadata) decode(pd packetDecoder) (err error) { + tmp, err := pd.getInt16() + if err != nil { + return err + } + pm.Err = KError(tmp) + + pm.ID, err = pd.getInt32() + if err != nil { + return err + } + + pm.Leader, err = pd.getInt32() + if err != nil { + return err + } + + pm.Replicas, err = pd.getInt32Array() + if err != nil { + return err + } + + pm.Isr, err = pd.getInt32Array() + if err != nil { + return err + } + + return nil +} + +func (pm *PartitionMetadata) encode(pe packetEncoder) (err error) { + pe.putInt16(int16(pm.Err)) + pe.putInt32(pm.ID) + pe.putInt32(pm.Leader) + + err = pe.putInt32Array(pm.Replicas) + if err != nil { + return err + } + + err = pe.putInt32Array(pm.Isr) + if err != nil { + return err + } + + return nil +} + +type TopicMetadata struct { + Err KError + Name string + Partitions []*PartitionMetadata +} + +func (tm *TopicMetadata) decode(pd packetDecoder) (err error) { + tmp, err := pd.getInt16() + if err != nil { + return err + } + tm.Err = KError(tmp) + + tm.Name, err = pd.getString() + if err != nil { + return err + } + + n, err := pd.getArrayLength() + if err != nil { + return err + } + tm.Partitions = make([]*PartitionMetadata, n) + for i := 0; i < n; i++ { + tm.Partitions[i] = new(PartitionMetadata) + err = tm.Partitions[i].decode(pd) + if err != nil { + return err + } + } + + return nil +} + +func (tm *TopicMetadata) encode(pe packetEncoder) (err error) { + pe.putInt16(int16(tm.Err)) + + err = pe.putString(tm.Name) + if err != nil { + return err + } + + err = pe.putArrayLength(len(tm.Partitions)) + if err != nil { + return err + } + + for _, pm := range tm.Partitions { + err = pm.encode(pe) + if err != nil { + return err + } + } + + return nil +} + +type MetadataResponse struct { + Brokers []*Broker + Topics []*TopicMetadata +} + +func (m *MetadataResponse) decode(pd packetDecoder) (err error) { + n, err := pd.getArrayLength() + if err != nil { + return err + } + + m.Brokers = make([]*Broker, n) + for i := 0; i < n; i++ { + m.Brokers[i] = new(Broker) + err = m.Brokers[i].decode(pd) + if err != nil { + return err + } + } + + n, err = pd.getArrayLength() + if err != nil { + return err + } + + m.Topics = make([]*TopicMetadata, n) + for i := 0; i < n; i++ { + m.Topics[i] = new(TopicMetadata) + err = m.Topics[i].decode(pd) + if err != nil { + return err + } + } + + return nil +} + +func (m *MetadataResponse) encode(pe packetEncoder) error { + err := pe.putArrayLength(len(m.Brokers)) + if err != nil { + return err + } + for _, broker := range m.Brokers { + err = broker.encode(pe) + if err != nil { + return err + } + } + + err = pe.putArrayLength(len(m.Topics)) + if err != nil { + return err + } + for _, tm := range m.Topics { + err = tm.encode(pe) + if err != nil { + return err + } + } + + return nil +} + +// testing API + +func (m *MetadataResponse) AddBroker(addr string, id int32) { + m.Brokers = append(m.Brokers, &Broker{id: id, addr: addr}) +} + +func (m *MetadataResponse) AddTopic(topic string, err KError) *TopicMetadata { + var tmatch *TopicMetadata + + for _, tm := range m.Topics { + if tm.Name == topic { + tmatch = tm + goto foundTopic + } + } + + tmatch = new(TopicMetadata) + tmatch.Name = topic + m.Topics = append(m.Topics, tmatch) + +foundTopic: + + tmatch.Err = err + return tmatch +} + +func (m *MetadataResponse) AddTopicPartition(topic string, partition, brokerID int32, replicas, isr []int32, err KError) { + tmatch := m.AddTopic(topic, ErrNoError) + var pmatch *PartitionMetadata + + for _, pm := range tmatch.Partitions { + if pm.ID == partition { + pmatch = pm + goto foundPartition + } + } + + pmatch = new(PartitionMetadata) + pmatch.ID = partition + tmatch.Partitions = append(tmatch.Partitions, pmatch) + +foundPartition: + + pmatch.Leader = brokerID + pmatch.Replicas = replicas + pmatch.Isr = isr + pmatch.Err = err + +} diff --git a/vendor/github.com/Shopify/sarama/metadata_response_test.go b/vendor/github.com/Shopify/sarama/metadata_response_test.go new file mode 100644 index 000000000000..1f1a51549e61 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/metadata_response_test.go @@ -0,0 +1,139 @@ +package sarama + +import "testing" + +var ( + emptyMetadataResponse = []byte{ + 0x00, 0x00, 0x00, 0x00, + 0x00, 0x00, 0x00, 0x00} + + brokersNoTopicsMetadataResponse = []byte{ + 0x00, 0x00, 0x00, 0x02, + + 0x00, 0x00, 0xab, 0xff, + 0x00, 0x09, 'l', 'o', 'c', 'a', 'l', 'h', 'o', 's', 't', + 0x00, 0x00, 0x00, 0x33, + + 0x00, 0x01, 0x02, 0x03, + 0x00, 0x0a, 'g', 'o', 'o', 'g', 'l', 'e', '.', 'c', 'o', 'm', + 0x00, 0x00, 0x01, 0x11, + + 0x00, 0x00, 0x00, 0x00} + + topicsNoBrokersMetadataResponse = []byte{ + 0x00, 0x00, 0x00, 0x00, + 0x00, 0x00, 0x00, 0x02, + + 0x00, 0x00, + 0x00, 0x03, 'f', 'o', 'o', + 0x00, 0x00, 0x00, 0x01, + 0x00, 0x04, + 0x00, 0x00, 0x00, 0x01, + 0x00, 0x00, 0x00, 0x07, + 0x00, 0x00, 0x00, 0x03, 0x00, 0x00, 0x00, 0x01, 0x00, 0x00, 0x00, 0x02, 0x00, 0x00, 0x00, 0x03, + 0x00, 0x00, 0x00, 0x00, + + 0x00, 0x00, + 0x00, 0x03, 'b', 'a', 'r', + 0x00, 0x00, 0x00, 0x00} +) + +func TestEmptyMetadataResponse(t *testing.T) { + response := MetadataResponse{} + + testDecodable(t, "empty", &response, emptyMetadataResponse) + if len(response.Brokers) != 0 { + t.Error("Decoding produced", len(response.Brokers), "brokers where there were none!") + } + if len(response.Topics) != 0 { + t.Error("Decoding produced", len(response.Topics), "topics where there were none!") + } +} + +func TestMetadataResponseWithBrokers(t *testing.T) { + response := MetadataResponse{} + + testDecodable(t, "brokers, no topics", &response, brokersNoTopicsMetadataResponse) + if len(response.Brokers) != 2 { + t.Fatal("Decoding produced", len(response.Brokers), "brokers where there were two!") + } + + if response.Brokers[0].id != 0xabff { + t.Error("Decoding produced invalid broker 0 id.") + } + if response.Brokers[0].addr != "localhost:51" { + t.Error("Decoding produced invalid broker 0 address.") + } + if response.Brokers[1].id != 0x010203 { + t.Error("Decoding produced invalid broker 1 id.") + } + if response.Brokers[1].addr != "google.com:273" { + t.Error("Decoding produced invalid broker 1 address.") + } + + if len(response.Topics) != 0 { + t.Error("Decoding produced", len(response.Topics), "topics where there were none!") + } +} + +func TestMetadataResponseWithTopics(t *testing.T) { + response := MetadataResponse{} + + testDecodable(t, "topics, no brokers", &response, topicsNoBrokersMetadataResponse) + if len(response.Brokers) != 0 { + t.Error("Decoding produced", len(response.Brokers), "brokers where there were none!") + } + + if len(response.Topics) != 2 { + t.Fatal("Decoding produced", len(response.Topics), "topics where there were two!") + } + + if response.Topics[0].Err != ErrNoError { + t.Error("Decoding produced invalid topic 0 error.") + } + + if response.Topics[0].Name != "foo" { + t.Error("Decoding produced invalid topic 0 name.") + } + + if len(response.Topics[0].Partitions) != 1 { + t.Fatal("Decoding produced invalid partition count for topic 0.") + } + + if response.Topics[0].Partitions[0].Err != ErrInvalidMessageSize { + t.Error("Decoding produced invalid topic 0 partition 0 error.") + } + + if response.Topics[0].Partitions[0].ID != 0x01 { + t.Error("Decoding produced invalid topic 0 partition 0 id.") + } + + if response.Topics[0].Partitions[0].Leader != 0x07 { + t.Error("Decoding produced invalid topic 0 partition 0 leader.") + } + + if len(response.Topics[0].Partitions[0].Replicas) != 3 { + t.Fatal("Decoding produced invalid topic 0 partition 0 replicas.") + } + for i := 0; i < 3; i++ { + if response.Topics[0].Partitions[0].Replicas[i] != int32(i+1) { + t.Error("Decoding produced invalid topic 0 partition 0 replica", i) + } + } + + if len(response.Topics[0].Partitions[0].Isr) != 0 { + t.Error("Decoding produced invalid topic 0 partition 0 isr length.") + } + + if response.Topics[1].Err != ErrNoError { + t.Error("Decoding produced invalid topic 1 error.") + } + + if response.Topics[1].Name != "bar" { + t.Error("Decoding produced invalid topic 0 name.") + } + + if len(response.Topics[1].Partitions) != 0 { + t.Error("Decoding produced invalid partition count for topic 1.") + } +} diff --git a/vendor/github.com/Shopify/sarama/mockbroker_test.go b/vendor/github.com/Shopify/sarama/mockbroker_test.go new file mode 100644 index 000000000000..28f85ba35da6 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/mockbroker_test.go @@ -0,0 +1,273 @@ +package sarama + +import ( + "bytes" + "encoding/binary" + "fmt" + "io" + "net" + "reflect" + "strconv" + "sync" + "testing" + "time" + + "github.com/davecgh/go-spew/spew" +) + +const ( + expectationTimeout = 500 * time.Millisecond +) + +type requestHandlerFunc func(req *request) (res encoder) + +// mockBroker is a mock Kafka broker. It consists of a TCP server on a +// kernel-selected localhost port that can accept many connections. It reads +// Kafka requests from that connection and passes them to the user specified +// handler function (see SetHandler) that generates respective responses. If +// the handler has not been explicitly specified then the broker returns +// responses set by the Returns function in the exact order they were provided. +// (if a response has a len of 0, nothing is sent, and the client request will +// timeout in this case). +// +// When running tests with one of these, it is strongly recommended to specify +// a timeout to `go test` so that if the broker hangs waiting for a response, +// the test panics. +// +// It is not necessary to prefix message length or correlation ID to your +// response bytes, the server does that automatically as a convenience. +type mockBroker struct { + brokerID int32 + port int32 + closing chan none + stopper chan none + expectations chan encoder + listener net.Listener + t *testing.T + latency time.Duration + handler requestHandlerFunc + history []RequestResponse + lock sync.Mutex +} + +type RequestResponse struct { + Request requestBody + Response encoder +} + +func (b *mockBroker) SetLatency(latency time.Duration) { + b.latency = latency +} + +func (b *mockBroker) SetHandlerByMap(handlerMap map[string]MockResponse) { + b.setHandler(func(req *request) (res encoder) { + reqTypeName := reflect.TypeOf(req.body).Elem().Name() + mockResponse := handlerMap[reqTypeName] + if mockResponse == nil { + return nil + } + return mockResponse.For(req.body) + }) +} + +func (b *mockBroker) BrokerID() int32 { + return b.brokerID +} + +func (b *mockBroker) History() []RequestResponse { + b.lock.Lock() + history := make([]RequestResponse, len(b.history)) + copy(history, b.history) + b.lock.Unlock() + return history +} + +func (b *mockBroker) Port() int32 { + return b.port +} + +func (b *mockBroker) Addr() string { + return b.listener.Addr().String() +} + +func (b *mockBroker) Close() { + close(b.expectations) + if len(b.expectations) > 0 { + buf := bytes.NewBufferString(fmt.Sprintf("mockbroker/%d: not all expectations were satisfied! Still waiting on:\n", b.BrokerID())) + for e := range b.expectations { + _, _ = buf.WriteString(spew.Sdump(e)) + } + b.t.Error(buf.String()) + } + close(b.closing) + <-b.stopper +} + +// setHandler sets the specified function as the request handler. Whenever +// a mock broker reads a request from the wire it passes the request to the +// function and sends back whatever the handler function returns. +func (b *mockBroker) setHandler(handler requestHandlerFunc) { + b.lock.Lock() + b.handler = handler + b.lock.Unlock() +} + +func (b *mockBroker) serverLoop() { + defer close(b.stopper) + var err error + var conn net.Conn + + go func() { + <-b.closing + safeClose(b.t, b.listener) + }() + + wg := &sync.WaitGroup{} + i := 0 + for conn, err = b.listener.Accept(); err == nil; conn, err = b.listener.Accept() { + wg.Add(1) + go b.handleRequests(conn, i, wg) + i++ + } + wg.Wait() + Logger.Printf("*** mockbroker/%d: listener closed, err=%v", b.BrokerID(), err) +} + +func (b *mockBroker) handleRequests(conn net.Conn, idx int, wg *sync.WaitGroup) { + defer wg.Done() + defer func() { + _ = conn.Close() + }() + Logger.Printf("*** mockbroker/%d/%d: connection opened", b.BrokerID(), idx) + var err error + + abort := make(chan none) + defer close(abort) + go func() { + select { + case <-b.closing: + _ = conn.Close() + case <-abort: + } + }() + + resHeader := make([]byte, 8) + for { + req, err := decodeRequest(conn) + if err != nil { + Logger.Printf("*** mockbroker/%d/%d: invalid request: err=%+v, %+v", b.brokerID, idx, err, spew.Sdump(req)) + b.serverError(err) + break + } + + if b.latency > 0 { + time.Sleep(b.latency) + } + + b.lock.Lock() + res := b.handler(req) + b.history = append(b.history, RequestResponse{req.body, res}) + b.lock.Unlock() + + if res == nil { + Logger.Printf("*** mockbroker/%d/%d: ignored %v", b.brokerID, idx, spew.Sdump(req)) + continue + } + Logger.Printf("*** mockbroker/%d/%d: served %v -> %v", b.brokerID, idx, req, res) + + encodedRes, err := encode(res) + if err != nil { + b.serverError(err) + break + } + if len(encodedRes) == 0 { + continue + } + + binary.BigEndian.PutUint32(resHeader, uint32(len(encodedRes)+4)) + binary.BigEndian.PutUint32(resHeader[4:], uint32(req.correlationID)) + if _, err = conn.Write(resHeader); err != nil { + b.serverError(err) + break + } + if _, err = conn.Write(encodedRes); err != nil { + b.serverError(err) + break + } + } + Logger.Printf("*** mockbroker/%d/%d: connection closed, err=%v", b.BrokerID(), idx, err) +} + +func (b *mockBroker) defaultRequestHandler(req *request) (res encoder) { + select { + case res, ok := <-b.expectations: + if !ok { + return nil + } + return res + case <-time.After(expectationTimeout): + return nil + } +} + +func (b *mockBroker) serverError(err error) { + isConnectionClosedError := false + if _, ok := err.(*net.OpError); ok { + isConnectionClosedError = true + } else if err == io.EOF { + isConnectionClosedError = true + } else if err.Error() == "use of closed network connection" { + isConnectionClosedError = true + } + + if isConnectionClosedError { + return + } + + b.t.Errorf(err.Error()) +} + +// newMockBroker launches a fake Kafka broker. It takes a *testing.T as provided by the +// test framework and a channel of responses to use. If an error occurs it is +// simply logged to the *testing.T and the broker exits. +func newMockBroker(t *testing.T, brokerID int32) *mockBroker { + return newMockBrokerAddr(t, brokerID, "localhost:0") +} + +// newMockBrokerAddr behaves like newMockBroker but listens on the address you give +// it rather than just some ephemeral port. +func newMockBrokerAddr(t *testing.T, brokerID int32, addr string) *mockBroker { + var err error + + broker := &mockBroker{ + closing: make(chan none), + stopper: make(chan none), + t: t, + brokerID: brokerID, + expectations: make(chan encoder, 512), + } + broker.handler = broker.defaultRequestHandler + + broker.listener, err = net.Listen("tcp", addr) + if err != nil { + t.Fatal(err) + } + Logger.Printf("*** mockbroker/%d listening on %s\n", brokerID, broker.listener.Addr().String()) + _, portStr, err := net.SplitHostPort(broker.listener.Addr().String()) + if err != nil { + t.Fatal(err) + } + tmp, err := strconv.ParseInt(portStr, 10, 32) + if err != nil { + t.Fatal(err) + } + broker.port = int32(tmp) + + go broker.serverLoop() + + return broker +} + +func (b *mockBroker) Returns(e encoder) { + b.expectations <- e +} diff --git a/vendor/github.com/Shopify/sarama/mockresponses_test.go b/vendor/github.com/Shopify/sarama/mockresponses_test.go new file mode 100644 index 000000000000..55b648d6f64d --- /dev/null +++ b/vendor/github.com/Shopify/sarama/mockresponses_test.go @@ -0,0 +1,446 @@ +package sarama + +import ( + "fmt" + "testing" +) + +// MockResponse is a response builder interface it defines one method that +// allows generating a response based on a request body. +type MockResponse interface { + For(reqBody decoder) (res encoder) +} + +// mockWrapper is a mock response builder that returns a particular concrete +// response regardless of the actual request passed to the `For` method. +type mockWrapper struct { + res encoder +} + +func (mw *mockWrapper) For(reqBody decoder) (res encoder) { + return mw.res +} + +func newMockWrapper(res encoder) *mockWrapper { + return &mockWrapper{res: res} +} + +// mockSequence is a mock response builder that is created from a sequence of +// concrete responses. Every time when a `MockBroker` calls its `For` method +// the next response from the sequence is returned. When the end of the +// sequence is reached the last element from the sequence is returned. +type mockSequence struct { + responses []MockResponse +} + +func newMockSequence(responses ...interface{}) *mockSequence { + ms := &mockSequence{} + ms.responses = make([]MockResponse, len(responses)) + for i, res := range responses { + switch res := res.(type) { + case MockResponse: + ms.responses[i] = res + case encoder: + ms.responses[i] = newMockWrapper(res) + default: + panic(fmt.Sprintf("Unexpected response type: %T", res)) + } + } + return ms +} + +func (mc *mockSequence) For(reqBody decoder) (res encoder) { + res = mc.responses[0].For(reqBody) + if len(mc.responses) > 1 { + mc.responses = mc.responses[1:] + } + return res +} + +// mockMetadataResponse is a `MetadataResponse` builder. +type mockMetadataResponse struct { + leaders map[string]map[int32]int32 + brokers map[string]int32 + t *testing.T +} + +func newMockMetadataResponse(t *testing.T) *mockMetadataResponse { + return &mockMetadataResponse{ + leaders: make(map[string]map[int32]int32), + brokers: make(map[string]int32), + t: t, + } +} + +func (mmr *mockMetadataResponse) SetLeader(topic string, partition, brokerID int32) *mockMetadataResponse { + partitions := mmr.leaders[topic] + if partitions == nil { + partitions = make(map[int32]int32) + mmr.leaders[topic] = partitions + } + partitions[partition] = brokerID + return mmr +} + +func (mmr *mockMetadataResponse) SetBroker(addr string, brokerID int32) *mockMetadataResponse { + mmr.brokers[addr] = brokerID + return mmr +} + +func (mmr *mockMetadataResponse) For(reqBody decoder) encoder { + metadataRequest := reqBody.(*MetadataRequest) + metadataResponse := &MetadataResponse{} + for addr, brokerID := range mmr.brokers { + metadataResponse.AddBroker(addr, brokerID) + } + if len(metadataRequest.Topics) == 0 { + for topic, partitions := range mmr.leaders { + for partition, brokerID := range partitions { + metadataResponse.AddTopicPartition(topic, partition, brokerID, nil, nil, ErrNoError) + } + } + return metadataResponse + } + for _, topic := range metadataRequest.Topics { + for partition, brokerID := range mmr.leaders[topic] { + metadataResponse.AddTopicPartition(topic, partition, brokerID, nil, nil, ErrNoError) + } + } + return metadataResponse +} + +// mockOffsetResponse is an `OffsetResponse` builder. +type mockOffsetResponse struct { + offsets map[string]map[int32]map[int64]int64 + t *testing.T +} + +func newMockOffsetResponse(t *testing.T) *mockOffsetResponse { + return &mockOffsetResponse{ + offsets: make(map[string]map[int32]map[int64]int64), + t: t, + } +} + +func (mor *mockOffsetResponse) SetOffset(topic string, partition int32, time, offset int64) *mockOffsetResponse { + partitions := mor.offsets[topic] + if partitions == nil { + partitions = make(map[int32]map[int64]int64) + mor.offsets[topic] = partitions + } + times := partitions[partition] + if times == nil { + times = make(map[int64]int64) + partitions[partition] = times + } + times[time] = offset + return mor +} + +func (mor *mockOffsetResponse) For(reqBody decoder) encoder { + offsetRequest := reqBody.(*OffsetRequest) + offsetResponse := &OffsetResponse{} + for topic, partitions := range offsetRequest.blocks { + for partition, block := range partitions { + offset := mor.getOffset(topic, partition, block.time) + offsetResponse.AddTopicPartition(topic, partition, offset) + } + } + return offsetResponse +} + +func (mor *mockOffsetResponse) getOffset(topic string, partition int32, time int64) int64 { + partitions := mor.offsets[topic] + if partitions == nil { + mor.t.Errorf("missing topic: %s", topic) + } + times := partitions[partition] + if times == nil { + mor.t.Errorf("missing partition: %d", partition) + } + offset, ok := times[time] + if !ok { + mor.t.Errorf("missing time: %d", time) + } + return offset +} + +// mockFetchResponse is a `FetchResponse` builder. +type mockFetchResponse struct { + messages map[string]map[int32]map[int64]Encoder + highWaterMarks map[string]map[int32]int64 + t *testing.T + batchSize int +} + +func newMockFetchResponse(t *testing.T, batchSize int) *mockFetchResponse { + return &mockFetchResponse{ + messages: make(map[string]map[int32]map[int64]Encoder), + highWaterMarks: make(map[string]map[int32]int64), + t: t, + batchSize: batchSize, + } +} + +func (mfr *mockFetchResponse) SetMessage(topic string, partition int32, offset int64, msg Encoder) *mockFetchResponse { + partitions := mfr.messages[topic] + if partitions == nil { + partitions = make(map[int32]map[int64]Encoder) + mfr.messages[topic] = partitions + } + messages := partitions[partition] + if messages == nil { + messages = make(map[int64]Encoder) + partitions[partition] = messages + } + messages[offset] = msg + return mfr +} + +func (mfr *mockFetchResponse) SetHighWaterMark(topic string, partition int32, offset int64) *mockFetchResponse { + partitions := mfr.highWaterMarks[topic] + if partitions == nil { + partitions = make(map[int32]int64) + mfr.highWaterMarks[topic] = partitions + } + partitions[partition] = offset + return mfr +} + +func (mfr *mockFetchResponse) For(reqBody decoder) encoder { + fetchRequest := reqBody.(*FetchRequest) + res := &FetchResponse{} + for topic, partitions := range fetchRequest.blocks { + for partition, block := range partitions { + initialOffset := block.fetchOffset + offset := initialOffset + maxOffset := initialOffset + int64(mfr.getMessageCount(topic, partition)) + for i := 0; i < mfr.batchSize && offset < maxOffset; { + msg := mfr.getMessage(topic, partition, offset) + if msg != nil { + res.AddMessage(topic, partition, nil, msg, offset) + i++ + } + offset++ + } + fb := res.GetBlock(topic, partition) + if fb == nil { + res.AddError(topic, partition, ErrNoError) + fb = res.GetBlock(topic, partition) + } + fb.HighWaterMarkOffset = mfr.getHighWaterMark(topic, partition) + } + } + return res +} + +func (mfr *mockFetchResponse) getMessage(topic string, partition int32, offset int64) Encoder { + partitions := mfr.messages[topic] + if partitions == nil { + return nil + } + messages := partitions[partition] + if messages == nil { + return nil + } + return messages[offset] +} + +func (mfr *mockFetchResponse) getMessageCount(topic string, partition int32) int { + partitions := mfr.messages[topic] + if partitions == nil { + return 0 + } + messages := partitions[partition] + if messages == nil { + return 0 + } + return len(messages) +} + +func (mfr *mockFetchResponse) getHighWaterMark(topic string, partition int32) int64 { + partitions := mfr.highWaterMarks[topic] + if partitions == nil { + return 0 + } + return partitions[partition] +} + +// mockConsumerMetadataResponse is a `ConsumerMetadataResponse` builder. +type mockConsumerMetadataResponse struct { + coordinators map[string]interface{} + t *testing.T +} + +func newMockConsumerMetadataResponse(t *testing.T) *mockConsumerMetadataResponse { + return &mockConsumerMetadataResponse{ + coordinators: make(map[string]interface{}), + t: t, + } +} + +func (mr *mockConsumerMetadataResponse) SetCoordinator(group string, broker *mockBroker) *mockConsumerMetadataResponse { + mr.coordinators[group] = broker + return mr +} + +func (mr *mockConsumerMetadataResponse) SetError(group string, kerror KError) *mockConsumerMetadataResponse { + mr.coordinators[group] = kerror + return mr +} + +func (mr *mockConsumerMetadataResponse) For(reqBody decoder) encoder { + req := reqBody.(*ConsumerMetadataRequest) + group := req.ConsumerGroup + res := &ConsumerMetadataResponse{} + v := mr.coordinators[group] + switch v := v.(type) { + case *mockBroker: + res.Coordinator = &Broker{id: v.BrokerID(), addr: v.Addr()} + case KError: + res.Err = v + } + return res +} + +// mockOffsetCommitResponse is a `OffsetCommitResponse` builder. +type mockOffsetCommitResponse struct { + errors map[string]map[string]map[int32]KError + t *testing.T +} + +func newMockOffsetCommitResponse(t *testing.T) *mockOffsetCommitResponse { + return &mockOffsetCommitResponse{t: t} +} + +func (mr *mockOffsetCommitResponse) SetError(group, topic string, partition int32, kerror KError) *mockOffsetCommitResponse { + if mr.errors == nil { + mr.errors = make(map[string]map[string]map[int32]KError) + } + topics := mr.errors[group] + if topics == nil { + topics = make(map[string]map[int32]KError) + mr.errors[group] = topics + } + partitions := topics[topic] + if partitions == nil { + partitions = make(map[int32]KError) + topics[topic] = partitions + } + partitions[partition] = kerror + return mr +} + +func (mr *mockOffsetCommitResponse) For(reqBody decoder) encoder { + req := reqBody.(*OffsetCommitRequest) + group := req.ConsumerGroup + res := &OffsetCommitResponse{} + for topic, partitions := range req.blocks { + for partition := range partitions { + res.AddError(topic, partition, mr.getError(group, topic, partition)) + } + } + return res +} + +func (mr *mockOffsetCommitResponse) getError(group, topic string, partition int32) KError { + topics := mr.errors[group] + if topics == nil { + return ErrNoError + } + partitions := topics[topic] + if partitions == nil { + return ErrNoError + } + kerror, ok := partitions[partition] + if !ok { + return ErrNoError + } + return kerror +} + +// mockProduceResponse is a `ProduceResponse` builder. +type mockProduceResponse struct { + errors map[string]map[int32]KError + t *testing.T +} + +func newMockProduceResponse(t *testing.T) *mockProduceResponse { + return &mockProduceResponse{t: t} +} + +func (mr *mockProduceResponse) SetError(topic string, partition int32, kerror KError) *mockProduceResponse { + if mr.errors == nil { + mr.errors = make(map[string]map[int32]KError) + } + partitions := mr.errors[topic] + if partitions == nil { + partitions = make(map[int32]KError) + mr.errors[topic] = partitions + } + partitions[partition] = kerror + return mr +} + +func (mr *mockProduceResponse) For(reqBody decoder) encoder { + req := reqBody.(*ProduceRequest) + res := &ProduceResponse{} + for topic, partitions := range req.msgSets { + for partition := range partitions { + res.AddTopicPartition(topic, partition, mr.getError(topic, partition)) + } + } + return res +} + +func (mr *mockProduceResponse) getError(topic string, partition int32) KError { + partitions := mr.errors[topic] + if partitions == nil { + return ErrNoError + } + kerror, ok := partitions[partition] + if !ok { + return ErrNoError + } + return kerror +} + +// mockOffsetFetchResponse is a `OffsetFetchResponse` builder. +type mockOffsetFetchResponse struct { + offsets map[string]map[string]map[int32]*OffsetFetchResponseBlock + t *testing.T +} + +func newMockOffsetFetchResponse(t *testing.T) *mockOffsetFetchResponse { + return &mockOffsetFetchResponse{t: t} +} + +func (mr *mockOffsetFetchResponse) SetOffset(group, topic string, partition int32, offset int64, metadata string, kerror KError) *mockOffsetFetchResponse { + if mr.offsets == nil { + mr.offsets = make(map[string]map[string]map[int32]*OffsetFetchResponseBlock) + } + topics := mr.offsets[group] + if topics == nil { + topics = make(map[string]map[int32]*OffsetFetchResponseBlock) + mr.offsets[group] = topics + } + partitions := topics[topic] + if partitions == nil { + partitions = make(map[int32]*OffsetFetchResponseBlock) + topics[topic] = partitions + } + partitions[partition] = &OffsetFetchResponseBlock{offset, metadata, kerror} + return mr +} + +func (mr *mockOffsetFetchResponse) For(reqBody decoder) encoder { + req := reqBody.(*OffsetFetchRequest) + group := req.ConsumerGroup + res := &OffsetFetchResponse{} + for topic, partitions := range mr.offsets[group] { + for partition, block := range partitions { + res.AddBlock(topic, partition, block) + } + } + return res +} diff --git a/vendor/github.com/Shopify/sarama/mocks/README.md b/vendor/github.com/Shopify/sarama/mocks/README.md new file mode 100644 index 000000000000..55a6c2e61cd3 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/mocks/README.md @@ -0,0 +1,13 @@ +# sarama/mocks + +The `mocks` subpackage includes mock implementations that implement the interfaces of the major sarama types. +You can use them to test your sarama applications using dependency injection. + +The following mock objects are available: + +- [Consumer](https://godoc.org/github.com/Shopify/sarama/mocks#Consumer), which will create [PartitionConsumer](https://godoc.org/github.com/Shopify/sarama/mocks#PartitionConsumer) mocks. +- [AsyncProducer](https://godoc.org/github.com/Shopify/sarama/mocks#AsyncProducer) +- [SyncProducer](https://godoc.org/github.com/Shopify/sarama/mocks#SyncProducer) + +The mocks allow you to set expectations on them. When you close the mocks, the expectations will be verified, +and the results will be reported to the `*testing.T` object you provided when creating the mock. diff --git a/vendor/github.com/Shopify/sarama/mocks/async_producer.go b/vendor/github.com/Shopify/sarama/mocks/async_producer.go new file mode 100644 index 000000000000..6ccf1f145547 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/mocks/async_producer.go @@ -0,0 +1,142 @@ +package mocks + +import ( + "sync" + + "github.com/Shopify/sarama" +) + +// AsyncProducer implements sarama's Producer interface for testing purposes. +// Before you can send messages to it's Input channel, you have to set expectations +// so it knows how to handle the input. This way you can easily test success and +// failure scenarios. +type AsyncProducer struct { + l sync.Mutex + t ErrorReporter + expectations []*producerExpectation + closed chan struct{} + input chan *sarama.ProducerMessage + successes chan *sarama.ProducerMessage + errors chan *sarama.ProducerError + lastOffset int64 +} + +// NewAsyncProducer instantiates a new Producer mock. The t argument should +// be the *testing.T instance of your test method. An error will be written to it if +// an expectation is violated. The config argument is used to determine whether it +// should ack successes on the Successes channel. +func NewAsyncProducer(t ErrorReporter, config *sarama.Config) *AsyncProducer { + if config == nil { + config = sarama.NewConfig() + } + mp := &AsyncProducer{ + t: t, + closed: make(chan struct{}, 0), + expectations: make([]*producerExpectation, 0), + input: make(chan *sarama.ProducerMessage, config.ChannelBufferSize), + successes: make(chan *sarama.ProducerMessage, config.ChannelBufferSize), + errors: make(chan *sarama.ProducerError, config.ChannelBufferSize), + } + + go func() { + defer func() { + close(mp.successes) + close(mp.errors) + }() + + for msg := range mp.input { + mp.l.Lock() + if mp.expectations == nil || len(mp.expectations) == 0 { + mp.expectations = nil + mp.t.Errorf("No more expectation set on this mock producer to handle the input message.") + } else { + expectation := mp.expectations[0] + mp.expectations = mp.expectations[1:] + if expectation.Result == errProduceSuccess { + mp.lastOffset++ + if config.Producer.Return.Successes { + msg.Offset = mp.lastOffset + mp.successes <- msg + } + } else { + if config.Producer.Return.Errors { + mp.errors <- &sarama.ProducerError{Err: expectation.Result, Msg: msg} + } + } + } + mp.l.Unlock() + } + + mp.l.Lock() + if len(mp.expectations) > 0 { + mp.t.Errorf("Expected to exhaust all expectations, but %d are left.", len(mp.expectations)) + } + mp.l.Unlock() + + close(mp.closed) + }() + + return mp +} + +//////////////////////////////////////////////// +// Implement Producer interface +//////////////////////////////////////////////// + +// AsyncClose corresponds with the AsyncClose method of sarama's Producer implementation. +// By closing a mock producer, you also tell it that no more input will be provided, so it will +// write an error to the test state if there's any remaining expectations. +func (mp *AsyncProducer) AsyncClose() { + close(mp.input) +} + +// Close corresponds with the Close method of sarama's Producer implementation. +// By closing a mock producer, you also tell it that no more input will be provided, so it will +// write an error to the test state if there's any remaining expectations. +func (mp *AsyncProducer) Close() error { + mp.AsyncClose() + <-mp.closed + return nil +} + +// Input corresponds with the Input method of sarama's Producer implementation. +// You have to set expectations on the mock producer before writing messages to the Input +// channel, so it knows how to handle them. If there is no more remaining expectations and +// a messages is written to the Input channel, the mock producer will write an error to the test +// state object. +func (mp *AsyncProducer) Input() chan<- *sarama.ProducerMessage { + return mp.input +} + +// Successes corresponds with the Successes method of sarama's Producer implementation. +func (mp *AsyncProducer) Successes() <-chan *sarama.ProducerMessage { + return mp.successes +} + +// Errors corresponds with the Errors method of sarama's Producer implementation. +func (mp *AsyncProducer) Errors() <-chan *sarama.ProducerError { + return mp.errors +} + +//////////////////////////////////////////////// +// Setting expectations +//////////////////////////////////////////////// + +// ExpectInputAndSucceed sets an expectation on the mock producer that a message will be provided +// on the input channel. The mock producer will handle the message as if it is produced successfully, +// i.e. it will make it available on the Successes channel if the Producer.Return.Successes setting +// is set to true. +func (mp *AsyncProducer) ExpectInputAndSucceed() { + mp.l.Lock() + defer mp.l.Unlock() + mp.expectations = append(mp.expectations, &producerExpectation{Result: errProduceSuccess}) +} + +// ExpectInputAndFail sets an expectation on the mock producer that a message will be provided +// on the input channel. The mock producer will handle the message as if it failed to produce +// successfully. This means it will make a ProducerError available on the Errors channel. +func (mp *AsyncProducer) ExpectInputAndFail(err error) { + mp.l.Lock() + defer mp.l.Unlock() + mp.expectations = append(mp.expectations, &producerExpectation{Result: err}) +} diff --git a/vendor/github.com/Shopify/sarama/mocks/async_producer_test.go b/vendor/github.com/Shopify/sarama/mocks/async_producer_test.go new file mode 100644 index 000000000000..520bf58b9267 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/mocks/async_producer_test.go @@ -0,0 +1,94 @@ +package mocks + +import ( + "fmt" + "testing" + + "github.com/Shopify/sarama" +) + +type testReporterMock struct { + errors []string +} + +func newTestReporterMock() *testReporterMock { + return &testReporterMock{errors: make([]string, 0)} +} + +func (trm *testReporterMock) Errorf(format string, args ...interface{}) { + trm.errors = append(trm.errors, fmt.Sprintf(format, args...)) +} + +func TestMockAsyncProducerImplementsAsyncProducerInterface(t *testing.T) { + var mp interface{} = &AsyncProducer{} + if _, ok := mp.(sarama.AsyncProducer); !ok { + t.Error("The mock producer should implement the sarama.Producer interface.") + } +} + +func TestProducerReturnsExpectationsToChannels(t *testing.T) { + config := sarama.NewConfig() + config.Producer.Return.Successes = true + mp := NewAsyncProducer(t, config) + + mp.ExpectInputAndSucceed() + mp.ExpectInputAndSucceed() + mp.ExpectInputAndFail(sarama.ErrOutOfBrokers) + + mp.Input() <- &sarama.ProducerMessage{Topic: "test 1"} + mp.Input() <- &sarama.ProducerMessage{Topic: "test 2"} + mp.Input() <- &sarama.ProducerMessage{Topic: "test 3"} + + msg1 := <-mp.Successes() + msg2 := <-mp.Successes() + err1 := <-mp.Errors() + + if msg1.Topic != "test 1" { + t.Error("Expected message 1 to be returned first") + } + + if msg2.Topic != "test 2" { + t.Error("Expected message 2 to be returned second") + } + + if err1.Msg.Topic != "test 3" || err1.Err != sarama.ErrOutOfBrokers { + t.Error("Expected message 3 to be returned as error") + } + + if err := mp.Close(); err != nil { + t.Error(err) + } +} + +func TestProducerWithTooFewExpectations(t *testing.T) { + trm := newTestReporterMock() + mp := NewAsyncProducer(trm, nil) + mp.ExpectInputAndSucceed() + + mp.Input() <- &sarama.ProducerMessage{Topic: "test"} + mp.Input() <- &sarama.ProducerMessage{Topic: "test"} + + if err := mp.Close(); err != nil { + t.Error(err) + } + + if len(trm.errors) != 1 { + t.Error("Expected to report an error") + } +} + +func TestProducerWithTooManyExpectations(t *testing.T) { + trm := newTestReporterMock() + mp := NewAsyncProducer(trm, nil) + mp.ExpectInputAndSucceed() + mp.ExpectInputAndFail(sarama.ErrOutOfBrokers) + + mp.Input() <- &sarama.ProducerMessage{Topic: "test"} + if err := mp.Close(); err != nil { + t.Error(err) + } + + if len(trm.errors) != 1 { + t.Error("Expected to report an error") + } +} diff --git a/vendor/github.com/Shopify/sarama/mocks/consumer.go b/vendor/github.com/Shopify/sarama/mocks/consumer.go new file mode 100644 index 000000000000..a2c394e34912 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/mocks/consumer.go @@ -0,0 +1,299 @@ +package mocks + +import ( + "sync" + "sync/atomic" + + "github.com/Shopify/sarama" +) + +// Consumer implements sarama's Consumer interface for testing purposes. +// Before you can start consuming from this consumer, you have to register +// topic/partitions using ExpectConsumePartition, and set expectations on them. +type Consumer struct { + l sync.Mutex + t ErrorReporter + config *sarama.Config + partitionConsumers map[string]map[int32]*PartitionConsumer + metadata map[string][]int32 +} + +// NewConsumer returns a new mock Consumer instance. The t argument should +// be the *testing.T instance of your test method. An error will be written to it if +// an expectation is violated. The config argument is currently unused and can be set to nil. +func NewConsumer(t ErrorReporter, config *sarama.Config) *Consumer { + if config == nil { + config = sarama.NewConfig() + } + + c := &Consumer{ + t: t, + config: config, + partitionConsumers: make(map[string]map[int32]*PartitionConsumer), + } + return c +} + +/////////////////////////////////////////////////// +// Consumer interface implementation +/////////////////////////////////////////////////// + +// ConsumePartition implements the ConsumePartition method from the sarama.Consumer interface. +// Before you can start consuming a partition, you have to set expectations on it using +// ExpectConsumePartition. You can only consume a partition once per consumer. +func (c *Consumer) ConsumePartition(topic string, partition int32, offset int64) (sarama.PartitionConsumer, error) { + c.l.Lock() + defer c.l.Unlock() + + if c.partitionConsumers[topic] == nil || c.partitionConsumers[topic][partition] == nil { + c.t.Errorf("No expectations set for %s/%d", topic, partition) + return nil, errOutOfExpectations + } + + pc := c.partitionConsumers[topic][partition] + if pc.consumed { + return nil, sarama.ConfigurationError("The topic/partition is already being consumed") + } + + if pc.offset != AnyOffset && pc.offset != offset { + c.t.Errorf("Unexpected offset when calling ConsumePartition for %s/%d. Expected %d, got %d.", topic, partition, pc.offset, offset) + } + + pc.consumed = true + return pc, nil +} + +// Topics returns a list of topics, as registered with SetMetadata +func (c *Consumer) Topics() ([]string, error) { + c.l.Lock() + defer c.l.Unlock() + + if c.metadata == nil { + c.t.Errorf("Unexpected call to Topics. Initialize the mock's topic metadata with SetMetadata.") + return nil, sarama.ErrOutOfBrokers + } + + var result []string + for topic := range c.metadata { + result = append(result, topic) + } + return result, nil +} + +// Partitions returns the list of parititons for the given topic, as registered with SetMetadata +func (c *Consumer) Partitions(topic string) ([]int32, error) { + c.l.Lock() + defer c.l.Unlock() + + if c.metadata == nil { + c.t.Errorf("Unexpected call to Partitions. Initialize the mock's topic metadata with SetMetadata.") + return nil, sarama.ErrOutOfBrokers + } + if c.metadata[topic] == nil { + return nil, sarama.ErrUnknownTopicOrPartition + } + + return c.metadata[topic], nil +} + +// Close implements the Close method from the sarama.Consumer interface. It will close +// all registered PartitionConsumer instances. +func (c *Consumer) Close() error { + c.l.Lock() + defer c.l.Unlock() + + for _, partitions := range c.partitionConsumers { + for _, partitionConsumer := range partitions { + _ = partitionConsumer.Close() + } + } + + return nil +} + +/////////////////////////////////////////////////// +// Expectation API +/////////////////////////////////////////////////// + +// SetTopicMetadata sets the clusters topic/partition metadata, +// which will be returned by Topics() and Partitions(). +func (c *Consumer) SetTopicMetadata(metadata map[string][]int32) { + c.l.Lock() + defer c.l.Unlock() + + c.metadata = metadata +} + +// ExpectConsumePartition will register a topic/partition, so you can set expectations on it. +// The registered PartitionConsumer will be returned, so you can set expectations +// on it using method chanining. Once a topic/partition is registered, you are +// expected to start consuming it using ConsumePartition. If that doesn't happen, +// an error will be written to the error reporter once the mock consumer is closed. It will +// also expect that the +func (c *Consumer) ExpectConsumePartition(topic string, partition int32, offset int64) *PartitionConsumer { + c.l.Lock() + defer c.l.Unlock() + + if c.partitionConsumers[topic] == nil { + c.partitionConsumers[topic] = make(map[int32]*PartitionConsumer) + } + + if c.partitionConsumers[topic][partition] == nil { + c.partitionConsumers[topic][partition] = &PartitionConsumer{ + t: c.t, + topic: topic, + partition: partition, + offset: offset, + messages: make(chan *sarama.ConsumerMessage, c.config.ChannelBufferSize), + errors: make(chan *sarama.ConsumerError, c.config.ChannelBufferSize), + } + } + + return c.partitionConsumers[topic][partition] +} + +/////////////////////////////////////////////////// +// PartitionConsumer mock type +/////////////////////////////////////////////////// + +// PartitionConsumer implements sarama's PartitionConsumer interface for testing purposes. +// It is returned by the mock Consumers ConsumePartitionMethod, but only if it is +// registered first using the Consumer's ExpectConsumePartition method. Before consuming the +// Errors and Messages channel, you should specify what values will be provided on these +// channels using YieldMessage and YieldError. +type PartitionConsumer struct { + l sync.Mutex + t ErrorReporter + topic string + partition int32 + offset int64 + messages chan *sarama.ConsumerMessage + errors chan *sarama.ConsumerError + singleClose sync.Once + consumed bool + errorsShouldBeDrained bool + messagesShouldBeDrained bool + highWaterMarkOffset int64 +} + +/////////////////////////////////////////////////// +// PartitionConsumer interface implementation +/////////////////////////////////////////////////// + +// AsyncClose implements the AsyncClose method from the sarama.PartitionConsumer interface. +func (pc *PartitionConsumer) AsyncClose() { + pc.singleClose.Do(func() { + close(pc.messages) + close(pc.errors) + }) +} + +// Close implements the Close method from the sarama.PartitionConsumer interface. It will +// verify whether the partition consumer was actually started. +func (pc *PartitionConsumer) Close() error { + if !pc.consumed { + pc.t.Errorf("Expectations set on %s/%d, but no partition consumer was started.", pc.topic, pc.partition) + return errPartitionConsumerNotStarted + } + + if pc.errorsShouldBeDrained && len(pc.errors) > 0 { + pc.t.Errorf("Expected the errors channel for %s/%d to be drained on close, but found %d errors.", pc.topic, pc.partition, len(pc.errors)) + } + + if pc.messagesShouldBeDrained && len(pc.messages) > 0 { + pc.t.Errorf("Expected the messages channel for %s/%d to be drained on close, but found %d messages.", pc.topic, pc.partition, len(pc.messages)) + } + + pc.AsyncClose() + + var ( + closeErr error + wg sync.WaitGroup + ) + + wg.Add(1) + go func() { + defer wg.Done() + + var errs = make(sarama.ConsumerErrors, 0) + for err := range pc.errors { + errs = append(errs, err) + } + + if len(errs) > 0 { + closeErr = errs + } + }() + + wg.Add(1) + go func() { + defer wg.Done() + for _ = range pc.messages { + // drain + } + }() + + wg.Wait() + return closeErr +} + +// Errors implements the Errors method from the sarama.PartitionConsumer interface. +func (pc *PartitionConsumer) Errors() <-chan *sarama.ConsumerError { + return pc.errors +} + +// Messages implements the Messages method from the sarama.PartitionConsumer interface. +func (pc *PartitionConsumer) Messages() <-chan *sarama.ConsumerMessage { + return pc.messages +} + +func (pc *PartitionConsumer) HighWaterMarkOffset() int64 { + return atomic.LoadInt64(&pc.highWaterMarkOffset) + 1 +} + +/////////////////////////////////////////////////// +// Expectation API +/////////////////////////////////////////////////// + +// YieldMessage will yield a messages Messages channel of this partition consumer +// when it is consumed. By default, the mock consumer will not verify whether this +// message was consumed from the Messages channel, because there are legitimate +// reasons forthis not to happen. ou can call ExpectMessagesDrainedOnClose so it will +// verify that the channel is empty on close. +func (pc *PartitionConsumer) YieldMessage(msg *sarama.ConsumerMessage) { + pc.l.Lock() + defer pc.l.Unlock() + + msg.Topic = pc.topic + msg.Partition = pc.partition + msg.Offset = atomic.AddInt64(&pc.highWaterMarkOffset, 1) + + pc.messages <- msg +} + +// YieldError will yield an error on the Errors channel of this partition consumer +// when it is consumed. By default, the mock consumer will not verify whether this error was +// consumed from the Errors channel, because there are legitimate reasons for this +// not to happen. You can call ExpectErrorsDrainedOnClose so it will verify that +// the channel is empty on close. +func (pc *PartitionConsumer) YieldError(err error) { + pc.errors <- &sarama.ConsumerError{ + Topic: pc.topic, + Partition: pc.partition, + Err: err, + } +} + +// ExpectMessagesDrainedOnClose sets an expectation on the partition consumer +// that the messages channel will be fully drained when Close is called. If this +// expectation is not met, an error is reported to the error reporter. +func (pc *PartitionConsumer) ExpectMessagesDrainedOnClose() { + pc.messagesShouldBeDrained = true +} + +// ExpectErrorsDrainedOnClose sets an expectation on the partition consumer +// that the errors channel will be fully drained when Close is called. If this +// expectation is not met, an error is reported to the error reporter. +func (pc *PartitionConsumer) ExpectErrorsDrainedOnClose() { + pc.errorsShouldBeDrained = true +} diff --git a/vendor/github.com/Shopify/sarama/mocks/consumer_test.go b/vendor/github.com/Shopify/sarama/mocks/consumer_test.go new file mode 100644 index 000000000000..50dad3a69414 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/mocks/consumer_test.go @@ -0,0 +1,249 @@ +package mocks + +import ( + "sort" + "testing" + + "github.com/Shopify/sarama" +) + +func TestMockConsumerImplementsConsumerInterface(t *testing.T) { + var c interface{} = &Consumer{} + if _, ok := c.(sarama.Consumer); !ok { + t.Error("The mock consumer should implement the sarama.Consumer interface.") + } + + var pc interface{} = &PartitionConsumer{} + if _, ok := pc.(sarama.PartitionConsumer); !ok { + t.Error("The mock partitionconsumer should implement the sarama.PartitionConsumer interface.") + } +} + +func TestConsumerHandlesExpectations(t *testing.T) { + consumer := NewConsumer(t, nil) + defer func() { + if err := consumer.Close(); err != nil { + t.Error(err) + } + }() + + consumer.ExpectConsumePartition("test", 0, sarama.OffsetOldest).YieldMessage(&sarama.ConsumerMessage{Value: []byte("hello world")}) + consumer.ExpectConsumePartition("test", 0, sarama.OffsetOldest).YieldError(sarama.ErrOutOfBrokers) + consumer.ExpectConsumePartition("test", 1, sarama.OffsetOldest).YieldMessage(&sarama.ConsumerMessage{Value: []byte("hello world again")}) + consumer.ExpectConsumePartition("other", 0, AnyOffset).YieldMessage(&sarama.ConsumerMessage{Value: []byte("hello other")}) + + pc_test0, err := consumer.ConsumePartition("test", 0, sarama.OffsetOldest) + if err != nil { + t.Fatal(err) + } + test0_msg := <-pc_test0.Messages() + if test0_msg.Topic != "test" || test0_msg.Partition != 0 || string(test0_msg.Value) != "hello world" { + t.Error("Message was not as expected:", test0_msg) + } + test0_err := <-pc_test0.Errors() + if test0_err.Err != sarama.ErrOutOfBrokers { + t.Error("Expected sarama.ErrOutOfBrokers, found:", test0_err.Err) + } + + pc_test1, err := consumer.ConsumePartition("test", 1, sarama.OffsetOldest) + if err != nil { + t.Fatal(err) + } + test1_msg := <-pc_test1.Messages() + if test1_msg.Topic != "test" || test1_msg.Partition != 1 || string(test1_msg.Value) != "hello world again" { + t.Error("Message was not as expected:", test1_msg) + } + + pc_other0, err := consumer.ConsumePartition("other", 0, sarama.OffsetNewest) + if err != nil { + t.Fatal(err) + } + other0_msg := <-pc_other0.Messages() + if other0_msg.Topic != "other" || other0_msg.Partition != 0 || string(other0_msg.Value) != "hello other" { + t.Error("Message was not as expected:", other0_msg) + } +} + +func TestConsumerReturnsNonconsumedErrorsOnClose(t *testing.T) { + consumer := NewConsumer(t, nil) + consumer.ExpectConsumePartition("test", 0, sarama.OffsetOldest).YieldError(sarama.ErrOutOfBrokers) + consumer.ExpectConsumePartition("test", 0, sarama.OffsetOldest).YieldError(sarama.ErrOutOfBrokers) + + pc, err := consumer.ConsumePartition("test", 0, sarama.OffsetOldest) + if err != nil { + t.Fatal(err) + } + + select { + case <-pc.Messages(): + t.Error("Did not epxect a message on the messages channel.") + case err := <-pc.Errors(): + if err.Err != sarama.ErrOutOfBrokers { + t.Error("Expected sarama.ErrOutOfBrokers, found", err) + } + } + + errs := pc.Close().(sarama.ConsumerErrors) + if len(errs) != 1 && errs[0].Err != sarama.ErrOutOfBrokers { + t.Error("Expected Close to return the remaining sarama.ErrOutOfBrokers") + } +} + +func TestConsumerWithoutExpectationsOnPartition(t *testing.T) { + trm := newTestReporterMock() + consumer := NewConsumer(trm, nil) + + _, err := consumer.ConsumePartition("test", 1, sarama.OffsetOldest) + if err != errOutOfExpectations { + t.Error("Expected ConsumePartition to return errOutOfExpectations") + } + + if err := consumer.Close(); err != nil { + t.Error("No error expected on close, but found:", err) + } + + if len(trm.errors) != 1 { + t.Errorf("Expected an expectation failure to be set on the error reporter.") + } +} + +func TestConsumerWithExpectationsOnUnconsumedPartition(t *testing.T) { + trm := newTestReporterMock() + consumer := NewConsumer(trm, nil) + consumer.ExpectConsumePartition("test", 0, sarama.OffsetOldest).YieldMessage(&sarama.ConsumerMessage{Value: []byte("hello world")}) + + if err := consumer.Close(); err != nil { + t.Error("No error expected on close, but found:", err) + } + + if len(trm.errors) != 1 { + t.Errorf("Expected an expectation failure to be set on the error reporter.") + } +} + +func TestConsumerWithWrongOffsetExpectation(t *testing.T) { + trm := newTestReporterMock() + consumer := NewConsumer(trm, nil) + consumer.ExpectConsumePartition("test", 0, sarama.OffsetOldest) + + _, err := consumer.ConsumePartition("test", 0, sarama.OffsetNewest) + if err != nil { + t.Error("Did not expect error, found:", err) + } + + if len(trm.errors) != 1 { + t.Errorf("Expected an expectation failure to be set on the error reporter.") + } + + if err := consumer.Close(); err != nil { + t.Error(err) + } +} + +func TestConsumerViolatesMessagesDrainedExpectation(t *testing.T) { + trm := newTestReporterMock() + consumer := NewConsumer(trm, nil) + pcmock := consumer.ExpectConsumePartition("test", 0, sarama.OffsetOldest) + pcmock.YieldMessage(&sarama.ConsumerMessage{Value: []byte("hello")}) + pcmock.YieldMessage(&sarama.ConsumerMessage{Value: []byte("hello")}) + pcmock.ExpectMessagesDrainedOnClose() + + pc, err := consumer.ConsumePartition("test", 0, sarama.OffsetOldest) + if err != nil { + t.Error(err) + } + + // consume first message, not second one + <-pc.Messages() + + if err := consumer.Close(); err != nil { + t.Error(err) + } + + if len(trm.errors) != 1 { + t.Errorf("Expected an expectation failure to be set on the error reporter.") + } +} + +func TestConsumerMeetsErrorsDrainedExpectation(t *testing.T) { + trm := newTestReporterMock() + consumer := NewConsumer(trm, nil) + + pcmock := consumer.ExpectConsumePartition("test", 0, sarama.OffsetOldest) + pcmock.YieldError(sarama.ErrInvalidMessage) + pcmock.YieldError(sarama.ErrInvalidMessage) + pcmock.ExpectErrorsDrainedOnClose() + + pc, err := consumer.ConsumePartition("test", 0, sarama.OffsetOldest) + if err != nil { + t.Error(err) + } + + // consume first and second error, + <-pc.Errors() + <-pc.Errors() + + if err := consumer.Close(); err != nil { + t.Error(err) + } + + if len(trm.errors) != 0 { + t.Errorf("Expected no expectation failures to be set on the error reporter.") + } +} + +func TestConsumerTopicMetadata(t *testing.T) { + trm := newTestReporterMock() + consumer := NewConsumer(trm, nil) + + consumer.SetTopicMetadata(map[string][]int32{ + "test1": []int32{0, 1, 2, 3}, + "test2": []int32{0, 1, 2, 3, 4, 5, 6, 7}, + }) + + topics, err := consumer.Topics() + if err != nil { + t.Error(t) + } + + sortedTopics := sort.StringSlice(topics) + sortedTopics.Sort() + if len(sortedTopics) != 2 || sortedTopics[0] != "test1" || sortedTopics[1] != "test2" { + t.Error("Unexpected topics returned:", sortedTopics) + } + + partitions1, err := consumer.Partitions("test1") + if err != nil { + t.Error(t) + } + + if len(partitions1) != 4 { + t.Error("Unexpected partitions returned:", len(partitions1)) + } + + partitions2, err := consumer.Partitions("test2") + if err != nil { + t.Error(t) + } + + if len(partitions2) != 8 { + t.Error("Unexpected partitions returned:", len(partitions2)) + } + + if len(trm.errors) != 0 { + t.Errorf("Expected no expectation failures to be set on the error reporter.") + } +} + +func TestConsumerUnexpectedTopicMetadata(t *testing.T) { + trm := newTestReporterMock() + consumer := NewConsumer(trm, nil) + + if _, err := consumer.Topics(); err != sarama.ErrOutOfBrokers { + t.Error("Expected sarama.ErrOutOfBrokers, found", err) + } + + if len(trm.errors) != 1 { + t.Errorf("Expected an expectation failure to be set on the error reporter.") + } +} diff --git a/vendor/github.com/Shopify/sarama/mocks/mocks.go b/vendor/github.com/Shopify/sarama/mocks/mocks.go new file mode 100644 index 000000000000..96b79bc06265 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/mocks/mocks.go @@ -0,0 +1,43 @@ +/* +Package mocks provides mocks that can be used for testing applications +that use Sarama. The mock types provided by this package implement the +interfaces Sarama exports, so you can use them for dependency injection +in your tests. + +All mock instances require you to set expectations on them before you +can use them. It will determine how the mock will behave. If an +expectation is not met, it will make your test fail. + +NOTE: this package currently does not fall under the API stability +guarantee of Sarama as it is still considered experimental. +*/ +package mocks + +import ( + "errors" + + "github.com/Shopify/sarama" +) + +// ErrorReporter is a simple interface that includes the testing.T methods we use to report +// expectation violations when using the mock objects. +type ErrorReporter interface { + Errorf(string, ...interface{}) +} + +var ( + errProduceSuccess error = nil + errOutOfExpectations = errors.New("No more expectations set on mock") + errPartitionConsumerNotStarted = errors.New("The partition consumer was never started") +) + +const AnyOffset int64 = -1000 + +type producerExpectation struct { + Result error +} + +type consumerExpectation struct { + Err error + Msg *sarama.ConsumerMessage +} diff --git a/vendor/github.com/Shopify/sarama/mocks/sync_producer.go b/vendor/github.com/Shopify/sarama/mocks/sync_producer.go new file mode 100644 index 000000000000..fa86b245cdf3 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/mocks/sync_producer.go @@ -0,0 +1,94 @@ +package mocks + +import ( + "sync" + + "github.com/Shopify/sarama" +) + +// SyncProducer implements sarama's SyncProducer interface for testing purposes. +// Before you can use it, you have to set expectations on the mock SyncProducer +// to tell it how to handle calls to SendMessage, so you can easily test success +// and failure scenarios. +type SyncProducer struct { + l sync.Mutex + t ErrorReporter + expectations []*producerExpectation + lastOffset int64 +} + +// NewSyncProducer instantiates a new SyncProducer mock. The t argument should +// be the *testing.T instance of your test method. An error will be written to it if +// an expectation is violated. The config argument is currently unused, but is +// maintained to be compatible with the async Producer. +func NewSyncProducer(t ErrorReporter, config *sarama.Config) *SyncProducer { + return &SyncProducer{ + t: t, + expectations: make([]*producerExpectation, 0), + } +} + +//////////////////////////////////////////////// +// Implement SyncProducer interface +//////////////////////////////////////////////// + +// SendMessage corresponds with the SendMessage method of sarama's SyncProducer implementation. +// You have to set expectations on the mock producer before calling SendMessage, so it knows +// how to handle them. If there is no more remaining expectations when SendMessage is called, +// the mock producer will write an error to the test state object. +func (sp *SyncProducer) SendMessage(msg *sarama.ProducerMessage) (partition int32, offset int64, err error) { + sp.l.Lock() + defer sp.l.Unlock() + + if len(sp.expectations) > 0 { + expectation := sp.expectations[0] + sp.expectations = sp.expectations[1:] + + if expectation.Result == errProduceSuccess { + sp.lastOffset++ + msg.Offset = sp.lastOffset + return 0, msg.Offset, nil + } else { + return -1, -1, expectation.Result + } + } else { + sp.t.Errorf("No more expectation set on this mock producer to handle the input message.") + return -1, -1, errOutOfExpectations + } +} + +// Close corresponds with the Close method of sarama's SyncProducer implementation. +// By closing a mock syncproducer, you also tell it that no more SendMessage calls will follow, +// so it will write an error to the test state if there's any remaining expectations. +func (sp *SyncProducer) Close() error { + sp.l.Lock() + defer sp.l.Unlock() + + if len(sp.expectations) > 0 { + sp.t.Errorf("Expected to exhaust all expectations, but %d are left.", len(sp.expectations)) + } + + return nil +} + +//////////////////////////////////////////////// +// Setting expectations +//////////////////////////////////////////////// + +// ExpectSendMessageAndSucceed sets an expectation on the mock producer that SendMessage will be +// called. The mock producer will handle the message as if it produced successfully, i.e. by +// returning a valid partition, and offset, and a nil error. +func (sp *SyncProducer) ExpectSendMessageAndSucceed() { + sp.l.Lock() + defer sp.l.Unlock() + sp.expectations = append(sp.expectations, &producerExpectation{Result: errProduceSuccess}) +} + +// ExpectSendMessageAndFail sets an expectation on the mock producer that SendMessage will be +// called. The mock producer will handle the message as if it failed to produce +// successfully, i.e. by returning the provided error. +func (sp *SyncProducer) ExpectSendMessageAndFail(err error) { + sp.l.Lock() + defer sp.l.Unlock() + sp.expectations = append(sp.expectations, &producerExpectation{Result: err}) +} diff --git a/vendor/github.com/Shopify/sarama/mocks/sync_producer_test.go b/vendor/github.com/Shopify/sarama/mocks/sync_producer_test.go new file mode 100644 index 000000000000..a674138e90f1 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/mocks/sync_producer_test.go @@ -0,0 +1,98 @@ +package mocks + +import ( + "testing" + + "github.com/Shopify/sarama" +) + +func TestMockSyncProducerImplementsSyncProducerInterface(t *testing.T) { + var mp interface{} = &SyncProducer{} + if _, ok := mp.(sarama.SyncProducer); !ok { + t.Error("The mock async producer should implement the sarama.SyncProducer interface.") + } +} + +func TestSyncProducerReturnsExpectationsToSendMessage(t *testing.T) { + sp := NewSyncProducer(t, nil) + defer func() { + if err := sp.Close(); err != nil { + t.Error(err) + } + }() + + sp.ExpectSendMessageAndSucceed() + sp.ExpectSendMessageAndSucceed() + sp.ExpectSendMessageAndFail(sarama.ErrOutOfBrokers) + + msg := &sarama.ProducerMessage{Topic: "test", Value: sarama.StringEncoder("test")} + + _, offset, err := sp.SendMessage(msg) + if err != nil { + t.Errorf("The first message should have been produced successfully, but got %s", err) + } + if offset != 1 || offset != msg.Offset { + t.Errorf("The first message should have been assigned offset 1, but got %d", msg.Offset) + } + + _, offset, err = sp.SendMessage(msg) + if err != nil { + t.Errorf("The second message should have been produced successfully, but got %s", err) + } + if offset != 2 || offset != msg.Offset { + t.Errorf("The second message should have been assigned offset 2, but got %d", offset) + } + + _, _, err = sp.SendMessage(msg) + if err != sarama.ErrOutOfBrokers { + t.Errorf("The third message should not have been produced successfully") + } + + if err := sp.Close(); err != nil { + t.Error(err) + } +} + +func TestSyncProducerWithTooManyExpectations(t *testing.T) { + trm := newTestReporterMock() + + sp := NewSyncProducer(trm, nil) + sp.ExpectSendMessageAndSucceed() + sp.ExpectSendMessageAndFail(sarama.ErrOutOfBrokers) + + msg := &sarama.ProducerMessage{Topic: "test", Value: sarama.StringEncoder("test")} + if _, _, err := sp.SendMessage(msg); err != nil { + t.Error("No error expected on first SendMessage call", err) + } + + if err := sp.Close(); err != nil { + t.Error(err) + } + + if len(trm.errors) != 1 { + t.Error("Expected to report an error") + } +} + +func TestSyncProducerWithTooFewExpectations(t *testing.T) { + trm := newTestReporterMock() + + sp := NewSyncProducer(trm, nil) + sp.ExpectSendMessageAndSucceed() + + msg := &sarama.ProducerMessage{Topic: "test", Value: sarama.StringEncoder("test")} + if _, _, err := sp.SendMessage(msg); err != nil { + t.Error("No error expected on first SendMessage call", err) + } + if _, _, err := sp.SendMessage(msg); err != errOutOfExpectations { + t.Error("errOutOfExpectations expected on second SendMessage call, found:", err) + } + + if err := sp.Close(); err != nil { + t.Error(err) + } + + if len(trm.errors) != 1 { + t.Error("Expected to report an error") + } +} diff --git a/vendor/github.com/Shopify/sarama/offset_commit_request.go b/vendor/github.com/Shopify/sarama/offset_commit_request.go new file mode 100644 index 000000000000..f51815260317 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/offset_commit_request.go @@ -0,0 +1,177 @@ +package sarama + +// ReceiveTime is a special value for the timestamp field of Offset Commit Requests which +// tells the broker to set the timestamp to the time at which the request was received. +// The timestamp is only used if message version 1 is used, which requires kafka 0.8.2. +const ReceiveTime int64 = -1 + +// GroupGenerationUndefined is a special value for the group generation field of +// Offset Commit Requests that should be used when a consumer group does not rely +// on Kafka for partition management. +const GroupGenerationUndefined = -1 + +type offsetCommitRequestBlock struct { + offset int64 + timestamp int64 + metadata string +} + +func (r *offsetCommitRequestBlock) encode(pe packetEncoder, version int16) error { + pe.putInt64(r.offset) + if version == 1 { + pe.putInt64(r.timestamp) + } else if r.timestamp != 0 { + Logger.Println("Non-zero timestamp specified for OffsetCommitRequest not v1, it will be ignored") + } + + return pe.putString(r.metadata) +} + +func (r *offsetCommitRequestBlock) decode(pd packetDecoder, version int16) (err error) { + if r.offset, err = pd.getInt64(); err != nil { + return err + } + if version == 1 { + if r.timestamp, err = pd.getInt64(); err != nil { + return err + } + } + r.metadata, err = pd.getString() + return err +} + +type OffsetCommitRequest struct { + ConsumerGroup string + ConsumerGroupGeneration int32 // v1 or later + ConsumerID string // v1 or later + RetentionTime int64 // v2 or later + + // Version can be: + // - 0 (kafka 0.8.1 and later) + // - 1 (kafka 0.8.2 and later) + // - 2 (kafka 0.8.3 and later) + Version int16 + blocks map[string]map[int32]*offsetCommitRequestBlock +} + +func (r *OffsetCommitRequest) encode(pe packetEncoder) error { + if r.Version < 0 || r.Version > 2 { + return PacketEncodingError{"invalid or unsupported OffsetCommitRequest version field"} + } + + if err := pe.putString(r.ConsumerGroup); err != nil { + return err + } + + if r.Version >= 1 { + pe.putInt32(r.ConsumerGroupGeneration) + if err := pe.putString(r.ConsumerID); err != nil { + return err + } + } else { + if r.ConsumerGroupGeneration != 0 { + Logger.Println("Non-zero ConsumerGroupGeneration specified for OffsetCommitRequest v0, it will be ignored") + } + if r.ConsumerID != "" { + Logger.Println("Non-empty ConsumerID specified for OffsetCommitRequest v0, it will be ignored") + } + } + + if r.Version >= 2 { + pe.putInt64(r.RetentionTime) + } else if r.RetentionTime != 0 { + Logger.Println("Non-zero RetentionTime specified for OffsetCommitRequest version <2, it will be ignored") + } + + if err := pe.putArrayLength(len(r.blocks)); err != nil { + return err + } + for topic, partitions := range r.blocks { + if err := pe.putString(topic); err != nil { + return err + } + if err := pe.putArrayLength(len(partitions)); err != nil { + return err + } + for partition, block := range partitions { + pe.putInt32(partition) + if err := block.encode(pe, r.Version); err != nil { + return err + } + } + } + return nil +} + +func (r *OffsetCommitRequest) decode(pd packetDecoder) (err error) { + if r.ConsumerGroup, err = pd.getString(); err != nil { + return err + } + + if r.Version >= 1 { + if r.ConsumerGroupGeneration, err = pd.getInt32(); err != nil { + return err + } + if r.ConsumerID, err = pd.getString(); err != nil { + return err + } + } + + if r.Version >= 2 { + if r.RetentionTime, err = pd.getInt64(); err != nil { + return err + } + } + + topicCount, err := pd.getArrayLength() + if err != nil { + return err + } + if topicCount == 0 { + return nil + } + r.blocks = make(map[string]map[int32]*offsetCommitRequestBlock) + for i := 0; i < topicCount; i++ { + topic, err := pd.getString() + if err != nil { + return err + } + partitionCount, err := pd.getArrayLength() + if err != nil { + return err + } + r.blocks[topic] = make(map[int32]*offsetCommitRequestBlock) + for j := 0; j < partitionCount; j++ { + partition, err := pd.getInt32() + if err != nil { + return err + } + block := &offsetCommitRequestBlock{} + if err := block.decode(pd, r.Version); err != nil { + return err + } + r.blocks[topic][partition] = block + } + } + return nil +} + +func (r *OffsetCommitRequest) key() int16 { + return 8 +} + +func (r *OffsetCommitRequest) version() int16 { + return r.Version +} + +func (r *OffsetCommitRequest) AddBlock(topic string, partitionID int32, offset int64, timestamp int64, metadata string) { + if r.blocks == nil { + r.blocks = make(map[string]map[int32]*offsetCommitRequestBlock) + } + + if r.blocks[topic] == nil { + r.blocks[topic] = make(map[int32]*offsetCommitRequestBlock) + } + + r.blocks[topic][partitionID] = &offsetCommitRequestBlock{offset, timestamp, metadata} +} diff --git a/vendor/github.com/Shopify/sarama/offset_commit_request_test.go b/vendor/github.com/Shopify/sarama/offset_commit_request_test.go new file mode 100644 index 000000000000..afc25b7b380c --- /dev/null +++ b/vendor/github.com/Shopify/sarama/offset_commit_request_test.go @@ -0,0 +1,90 @@ +package sarama + +import "testing" + +var ( + offsetCommitRequestNoBlocksV0 = []byte{ + 0x00, 0x06, 'f', 'o', 'o', 'b', 'a', 'r', + 0x00, 0x00, 0x00, 0x00} + + offsetCommitRequestNoBlocksV1 = []byte{ + 0x00, 0x06, 'f', 'o', 'o', 'b', 'a', 'r', + 0x00, 0x00, 0x11, 0x22, + 0x00, 0x04, 'c', 'o', 'n', 's', + 0x00, 0x00, 0x00, 0x00} + + offsetCommitRequestNoBlocksV2 = []byte{ + 0x00, 0x06, 'f', 'o', 'o', 'b', 'a', 'r', + 0x00, 0x00, 0x11, 0x22, + 0x00, 0x04, 'c', 'o', 'n', 's', + 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x44, 0x33, + 0x00, 0x00, 0x00, 0x00} + + offsetCommitRequestOneBlockV0 = []byte{ + 0x00, 0x06, 'f', 'o', 'o', 'b', 'a', 'r', + 0x00, 0x00, 0x00, 0x01, + 0x00, 0x05, 't', 'o', 'p', 'i', 'c', + 0x00, 0x00, 0x00, 0x01, + 0x00, 0x00, 0x52, 0x21, + 0x00, 0x00, 0x00, 0x00, 0xDE, 0xAD, 0xBE, 0xEF, + 0x00, 0x08, 'm', 'e', 't', 'a', 'd', 'a', 't', 'a'} + + offsetCommitRequestOneBlockV1 = []byte{ + 0x00, 0x06, 'f', 'o', 'o', 'b', 'a', 'r', + 0x00, 0x00, 0x11, 0x22, + 0x00, 0x04, 'c', 'o', 'n', 's', + 0x00, 0x00, 0x00, 0x01, + 0x00, 0x05, 't', 'o', 'p', 'i', 'c', + 0x00, 0x00, 0x00, 0x01, + 0x00, 0x00, 0x52, 0x21, + 0x00, 0x00, 0x00, 0x00, 0xDE, 0xAD, 0xBE, 0xEF, + 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, + 0x00, 0x08, 'm', 'e', 't', 'a', 'd', 'a', 't', 'a'} + + offsetCommitRequestOneBlockV2 = []byte{ + 0x00, 0x06, 'f', 'o', 'o', 'b', 'a', 'r', + 0x00, 0x00, 0x11, 0x22, + 0x00, 0x04, 'c', 'o', 'n', 's', + 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x44, 0x33, + 0x00, 0x00, 0x00, 0x01, + 0x00, 0x05, 't', 'o', 'p', 'i', 'c', + 0x00, 0x00, 0x00, 0x01, + 0x00, 0x00, 0x52, 0x21, + 0x00, 0x00, 0x00, 0x00, 0xDE, 0xAD, 0xBE, 0xEF, + 0x00, 0x08, 'm', 'e', 't', 'a', 'd', 'a', 't', 'a'} +) + +func TestOffsetCommitRequestV0(t *testing.T) { + request := new(OffsetCommitRequest) + request.Version = 0 + request.ConsumerGroup = "foobar" + testRequest(t, "no blocks v0", request, offsetCommitRequestNoBlocksV0) + + request.AddBlock("topic", 0x5221, 0xDEADBEEF, 0, "metadata") + testRequest(t, "one block v0", request, offsetCommitRequestOneBlockV0) +} + +func TestOffsetCommitRequestV1(t *testing.T) { + request := new(OffsetCommitRequest) + request.ConsumerGroup = "foobar" + request.ConsumerID = "cons" + request.ConsumerGroupGeneration = 0x1122 + request.Version = 1 + testRequest(t, "no blocks v1", request, offsetCommitRequestNoBlocksV1) + + request.AddBlock("topic", 0x5221, 0xDEADBEEF, ReceiveTime, "metadata") + testRequest(t, "one block v1", request, offsetCommitRequestOneBlockV1) +} + +func TestOffsetCommitRequestV2(t *testing.T) { + request := new(OffsetCommitRequest) + request.ConsumerGroup = "foobar" + request.ConsumerID = "cons" + request.ConsumerGroupGeneration = 0x1122 + request.RetentionTime = 0x4433 + request.Version = 2 + testRequest(t, "no blocks v2", request, offsetCommitRequestNoBlocksV2) + + request.AddBlock("topic", 0x5221, 0xDEADBEEF, 0, "metadata") + testRequest(t, "one block v2", request, offsetCommitRequestOneBlockV2) +} diff --git a/vendor/github.com/Shopify/sarama/offset_commit_response.go b/vendor/github.com/Shopify/sarama/offset_commit_response.go new file mode 100644 index 000000000000..573a3b6a100f --- /dev/null +++ b/vendor/github.com/Shopify/sarama/offset_commit_response.go @@ -0,0 +1,73 @@ +package sarama + +type OffsetCommitResponse struct { + Errors map[string]map[int32]KError +} + +func (r *OffsetCommitResponse) AddError(topic string, partition int32, kerror KError) { + if r.Errors == nil { + r.Errors = make(map[string]map[int32]KError) + } + partitions := r.Errors[topic] + if partitions == nil { + partitions = make(map[int32]KError) + r.Errors[topic] = partitions + } + partitions[partition] = kerror +} + +func (r *OffsetCommitResponse) encode(pe packetEncoder) error { + if err := pe.putArrayLength(len(r.Errors)); err != nil { + return err + } + for topic, partitions := range r.Errors { + if err := pe.putString(topic); err != nil { + return err + } + if err := pe.putArrayLength(len(partitions)); err != nil { + return err + } + for partition, kerror := range partitions { + pe.putInt32(partition) + pe.putInt16(int16(kerror)) + } + } + return nil +} + +func (r *OffsetCommitResponse) decode(pd packetDecoder) (err error) { + numTopics, err := pd.getArrayLength() + if err != nil || numTopics == 0 { + return err + } + + r.Errors = make(map[string]map[int32]KError, numTopics) + for i := 0; i < numTopics; i++ { + name, err := pd.getString() + if err != nil { + return err + } + + numErrors, err := pd.getArrayLength() + if err != nil { + return err + } + + r.Errors[name] = make(map[int32]KError, numErrors) + + for j := 0; j < numErrors; j++ { + id, err := pd.getInt32() + if err != nil { + return err + } + + tmp, err := pd.getInt16() + if err != nil { + return err + } + r.Errors[name][id] = KError(tmp) + } + } + + return nil +} diff --git a/vendor/github.com/Shopify/sarama/offset_commit_response_test.go b/vendor/github.com/Shopify/sarama/offset_commit_response_test.go new file mode 100644 index 000000000000..074ec923220a --- /dev/null +++ b/vendor/github.com/Shopify/sarama/offset_commit_response_test.go @@ -0,0 +1,24 @@ +package sarama + +import ( + "testing" +) + +var ( + emptyOffsetCommitResponse = []byte{ + 0x00, 0x00, 0x00, 0x00} +) + +func TestEmptyOffsetCommitResponse(t *testing.T) { + response := OffsetCommitResponse{} + testResponse(t, "empty", &response, emptyOffsetCommitResponse) +} + +func TestNormalOffsetCommitResponse(t *testing.T) { + response := OffsetCommitResponse{} + response.AddError("t", 0, ErrNotLeaderForPartition) + response.Errors["m"] = make(map[int32]KError) + // The response encoded form cannot be checked for it varies due to + // unpredictable map traversal order. + testResponse(t, "normal", &response, nil) +} diff --git a/vendor/github.com/Shopify/sarama/offset_fetch_request.go b/vendor/github.com/Shopify/sarama/offset_fetch_request.go new file mode 100644 index 000000000000..30bbbbbd0d21 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/offset_fetch_request.go @@ -0,0 +1,71 @@ +package sarama + +type OffsetFetchRequest struct { + ConsumerGroup string + Version int16 + partitions map[string][]int32 +} + +func (r *OffsetFetchRequest) encode(pe packetEncoder) (err error) { + if r.Version < 0 || r.Version > 1 { + return PacketEncodingError{"invalid or unsupported OffsetFetchRequest version field"} + } + + if err = pe.putString(r.ConsumerGroup); err != nil { + return err + } + if err = pe.putArrayLength(len(r.partitions)); err != nil { + return err + } + for topic, partitions := range r.partitions { + if err = pe.putString(topic); err != nil { + return err + } + if err = pe.putInt32Array(partitions); err != nil { + return err + } + } + return nil +} + +func (r *OffsetFetchRequest) decode(pd packetDecoder) (err error) { + if r.ConsumerGroup, err = pd.getString(); err != nil { + return err + } + partitionCount, err := pd.getArrayLength() + if err != nil { + return err + } + if partitionCount == 0 { + return nil + } + r.partitions = make(map[string][]int32) + for i := 0; i < partitionCount; i++ { + topic, err := pd.getString() + if err != nil { + return err + } + partitions, err := pd.getInt32Array() + if err != nil { + return err + } + r.partitions[topic] = partitions + } + return nil +} + +func (r *OffsetFetchRequest) key() int16 { + return 9 +} + +func (r *OffsetFetchRequest) version() int16 { + return r.Version +} + +func (r *OffsetFetchRequest) AddPartition(topic string, partitionID int32) { + if r.partitions == nil { + r.partitions = make(map[string][]int32) + } + + r.partitions[topic] = append(r.partitions[topic], partitionID) +} diff --git a/vendor/github.com/Shopify/sarama/offset_fetch_request_test.go b/vendor/github.com/Shopify/sarama/offset_fetch_request_test.go new file mode 100644 index 000000000000..025d725c986b --- /dev/null +++ b/vendor/github.com/Shopify/sarama/offset_fetch_request_test.go @@ -0,0 +1,31 @@ +package sarama + +import "testing" + +var ( + offsetFetchRequestNoGroupNoPartitions = []byte{ + 0x00, 0x00, + 0x00, 0x00, 0x00, 0x00} + + offsetFetchRequestNoPartitions = []byte{ + 0x00, 0x04, 'b', 'l', 'a', 'h', + 0x00, 0x00, 0x00, 0x00} + + offsetFetchRequestOnePartition = []byte{ + 0x00, 0x04, 'b', 'l', 'a', 'h', + 0x00, 0x00, 0x00, 0x01, + 0x00, 0x0D, 't', 'o', 'p', 'i', 'c', 'T', 'h', 'e', 'F', 'i', 'r', 's', 't', + 0x00, 0x00, 0x00, 0x01, + 0x4F, 0x4F, 0x4F, 0x4F} +) + +func TestOffsetFetchRequest(t *testing.T) { + request := new(OffsetFetchRequest) + testRequest(t, "no group, no partitions", request, offsetFetchRequestNoGroupNoPartitions) + + request.ConsumerGroup = "blah" + testRequest(t, "no partitions", request, offsetFetchRequestNoPartitions) + + request.AddPartition("topicTheFirst", 0x4F4F4F4F) + testRequest(t, "one partition", request, offsetFetchRequestOnePartition) +} diff --git a/vendor/github.com/Shopify/sarama/offset_fetch_response.go b/vendor/github.com/Shopify/sarama/offset_fetch_response.go new file mode 100644 index 000000000000..93078c350f8d --- /dev/null +++ b/vendor/github.com/Shopify/sarama/offset_fetch_response.go @@ -0,0 +1,131 @@ +package sarama + +type OffsetFetchResponseBlock struct { + Offset int64 + Metadata string + Err KError +} + +func (r *OffsetFetchResponseBlock) decode(pd packetDecoder) (err error) { + r.Offset, err = pd.getInt64() + if err != nil { + return err + } + + r.Metadata, err = pd.getString() + if err != nil { + return err + } + + tmp, err := pd.getInt16() + if err != nil { + return err + } + r.Err = KError(tmp) + + return nil +} + +func (r *OffsetFetchResponseBlock) encode(pe packetEncoder) (err error) { + pe.putInt64(r.Offset) + + err = pe.putString(r.Metadata) + if err != nil { + return err + } + + pe.putInt16(int16(r.Err)) + + return nil +} + +type OffsetFetchResponse struct { + Blocks map[string]map[int32]*OffsetFetchResponseBlock +} + +func (r *OffsetFetchResponse) encode(pe packetEncoder) error { + if err := pe.putArrayLength(len(r.Blocks)); err != nil { + return err + } + for topic, partitions := range r.Blocks { + if err := pe.putString(topic); err != nil { + return err + } + if err := pe.putArrayLength(len(partitions)); err != nil { + return err + } + for partition, block := range partitions { + pe.putInt32(partition) + if err := block.encode(pe); err != nil { + return err + } + } + } + return nil +} + +func (r *OffsetFetchResponse) decode(pd packetDecoder) (err error) { + numTopics, err := pd.getArrayLength() + if err != nil || numTopics == 0 { + return err + } + + r.Blocks = make(map[string]map[int32]*OffsetFetchResponseBlock, numTopics) + for i := 0; i < numTopics; i++ { + name, err := pd.getString() + if err != nil { + return err + } + + numBlocks, err := pd.getArrayLength() + if err != nil { + return err + } + + if numBlocks == 0 { + r.Blocks[name] = nil + continue + } + r.Blocks[name] = make(map[int32]*OffsetFetchResponseBlock, numBlocks) + + for j := 0; j < numBlocks; j++ { + id, err := pd.getInt32() + if err != nil { + return err + } + + block := new(OffsetFetchResponseBlock) + err = block.decode(pd) + if err != nil { + return err + } + r.Blocks[name][id] = block + } + } + + return nil +} + +func (r *OffsetFetchResponse) GetBlock(topic string, partition int32) *OffsetFetchResponseBlock { + if r.Blocks == nil { + return nil + } + + if r.Blocks[topic] == nil { + return nil + } + + return r.Blocks[topic][partition] +} + +func (r *OffsetFetchResponse) AddBlock(topic string, partition int32, block *OffsetFetchResponseBlock) { + if r.Blocks == nil { + r.Blocks = make(map[string]map[int32]*OffsetFetchResponseBlock) + } + partitions := r.Blocks[topic] + if partitions == nil { + partitions = make(map[int32]*OffsetFetchResponseBlock) + r.Blocks[topic] = partitions + } + partitions[partition] = block +} diff --git a/vendor/github.com/Shopify/sarama/offset_fetch_response_test.go b/vendor/github.com/Shopify/sarama/offset_fetch_response_test.go new file mode 100644 index 000000000000..7614ae424930 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/offset_fetch_response_test.go @@ -0,0 +1,22 @@ +package sarama + +import "testing" + +var ( + emptyOffsetFetchResponse = []byte{ + 0x00, 0x00, 0x00, 0x00} +) + +func TestEmptyOffsetFetchResponse(t *testing.T) { + response := OffsetFetchResponse{} + testResponse(t, "empty", &response, emptyOffsetFetchResponse) +} + +func TestNormalOffsetFetchResponse(t *testing.T) { + response := OffsetFetchResponse{} + response.AddBlock("t", 0, &OffsetFetchResponseBlock{0, "md", ErrRequestTimedOut}) + response.Blocks["m"] = nil + // The response encoded form cannot be checked for it varies due to + // unpredictable map traversal order. + testResponse(t, "normal", &response, nil) +} diff --git a/vendor/github.com/Shopify/sarama/offset_manager.go b/vendor/github.com/Shopify/sarama/offset_manager.go new file mode 100644 index 000000000000..0c90f7c21782 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/offset_manager.go @@ -0,0 +1,516 @@ +package sarama + +import ( + "sync" + "time" +) + +// Offset Manager + +// OffsetManager uses Kafka to store and fetch consumed partition offsets. +type OffsetManager interface { + // ManagePartition creates a PartitionOffsetManager on the given topic/partition. + // It will return an error if this OffsetManager is already managing the given + // topic/partition. + ManagePartition(topic string, partition int32) (PartitionOffsetManager, error) + + // Close stops the OffsetManager from managing offsets. It is required to call + // this function before an OffsetManager object passes out of scope, as it + // will otherwise leak memory. You must call this after all the + // PartitionOffsetManagers are closed. + Close() error +} + +type offsetManager struct { + client Client + conf *Config + group string + + lock sync.Mutex + poms map[string]map[int32]*partitionOffsetManager + boms map[*Broker]*brokerOffsetManager +} + +// NewOffsetManagerFromClient creates a new OffsetManager from the given client. +// It is still necessary to call Close() on the underlying client when finished with the partition manager. +func NewOffsetManagerFromClient(group string, client Client) (OffsetManager, error) { + // Check that we are not dealing with a closed Client before processing any other arguments + if client.Closed() { + return nil, ErrClosedClient + } + + om := &offsetManager{ + client: client, + conf: client.Config(), + group: group, + poms: make(map[string]map[int32]*partitionOffsetManager), + boms: make(map[*Broker]*brokerOffsetManager), + } + + return om, nil +} + +func (om *offsetManager) ManagePartition(topic string, partition int32) (PartitionOffsetManager, error) { + pom, err := om.newPartitionOffsetManager(topic, partition) + if err != nil { + return nil, err + } + + om.lock.Lock() + defer om.lock.Unlock() + + topicManagers := om.poms[topic] + if topicManagers == nil { + topicManagers = make(map[int32]*partitionOffsetManager) + om.poms[topic] = topicManagers + } + + if topicManagers[partition] != nil { + return nil, ConfigurationError("That topic/partition is already being managed") + } + + topicManagers[partition] = pom + return pom, nil +} + +func (om *offsetManager) Close() error { + return nil +} + +func (om *offsetManager) refBrokerOffsetManager(broker *Broker) *brokerOffsetManager { + om.lock.Lock() + defer om.lock.Unlock() + + bom := om.boms[broker] + if bom == nil { + bom = om.newBrokerOffsetManager(broker) + om.boms[broker] = bom + } + + bom.refs++ + + return bom +} + +func (om *offsetManager) unrefBrokerOffsetManager(bom *brokerOffsetManager) { + om.lock.Lock() + defer om.lock.Unlock() + + bom.refs-- + + if bom.refs == 0 { + close(bom.updateSubscriptions) + if om.boms[bom.broker] == bom { + delete(om.boms, bom.broker) + } + } +} + +func (om *offsetManager) abandonBroker(bom *brokerOffsetManager) { + om.lock.Lock() + defer om.lock.Unlock() + + delete(om.boms, bom.broker) +} + +func (om *offsetManager) abandonPartitionOffsetManager(pom *partitionOffsetManager) { + om.lock.Lock() + defer om.lock.Unlock() + + delete(om.poms[pom.topic], pom.partition) + if len(om.poms[pom.topic]) == 0 { + delete(om.poms, pom.topic) + } +} + +// Partition Offset Manager + +// PartitionOffsetManager uses Kafka to store and fetch consumed partition offsets. You MUST call Close() +// on a partition offset manager to avoid leaks, it will not be garbage-collected automatically when it passes +// out of scope. +type PartitionOffsetManager interface { + // NextOffset returns the next offset that should be consumed for the managed + // partition, accompanied by metadata which can be used to reconstruct the state + // of the partition consumer when it resumes. NextOffset() will return + // `config.Consumer.Offsets.Initial` and an empty metadata string if no offset + // was committed for this partition yet. + NextOffset() (int64, string) + + // MarkOffset marks the provided offset as processed, alongside a metadata string + // that represents the state of the partition consumer at that point in time. The + // metadata string can be used by another consumer to restore that state, so it + // can resume consumption. + // + // Note: calling MarkOffset does not necessarily commit the offset to the backend + // store immediately for efficiency reasons, and it may never be committed if + // your application crashes. This means that you may end up processing the same + // message twice, and your processing should ideally be idempotent. + MarkOffset(offset int64, metadata string) + + // Errors returns a read channel of errors that occur during offset management, if + // enabled. By default, errors are logged and not returned over this channel. If + // you want to implement any custom error handling, set your config's + // Consumer.Return.Errors setting to true, and read from this channel. + Errors() <-chan *ConsumerError + + // AsyncClose initiates a shutdown of the PartitionOffsetManager. This method will + // return immediately, after which you should wait until the 'errors' channel has + // been drained and closed. It is required to call this function, or Close before + // a consumer object passes out of scope, as it will otherwise leak memory. You + // must call this before calling Close on the underlying client. + AsyncClose() + + // Close stops the PartitionOffsetManager from managing offsets. It is required to + // call this function (or AsyncClose) before a PartitionOffsetManager object + // passes out of scope, as it will otherwise leak memory. You must call this + // before calling Close on the underlying client. + Close() error +} + +type partitionOffsetManager struct { + parent *offsetManager + topic string + partition int32 + + lock sync.Mutex + offset int64 + metadata string + dirty bool + clean chan none + broker *brokerOffsetManager + + errors chan *ConsumerError + rebalance chan none + dying chan none +} + +func (om *offsetManager) newPartitionOffsetManager(topic string, partition int32) (*partitionOffsetManager, error) { + pom := &partitionOffsetManager{ + parent: om, + topic: topic, + partition: partition, + clean: make(chan none), + errors: make(chan *ConsumerError, om.conf.ChannelBufferSize), + rebalance: make(chan none, 1), + dying: make(chan none), + } + + if err := pom.selectBroker(); err != nil { + return nil, err + } + + if err := pom.fetchInitialOffset(om.conf.Metadata.Retry.Max); err != nil { + return nil, err + } + + pom.broker.updateSubscriptions <- pom + + go withRecover(pom.mainLoop) + + return pom, nil +} + +func (pom *partitionOffsetManager) mainLoop() { + for { + select { + case <-pom.rebalance: + if err := pom.selectBroker(); err != nil { + pom.handleError(err) + pom.rebalance <- none{} + } else { + pom.broker.updateSubscriptions <- pom + } + case <-pom.dying: + if pom.broker != nil { + select { + case <-pom.rebalance: + case pom.broker.updateSubscriptions <- pom: + } + pom.parent.unrefBrokerOffsetManager(pom.broker) + } + pom.parent.abandonPartitionOffsetManager(pom) + close(pom.errors) + return + } + } +} + +func (pom *partitionOffsetManager) selectBroker() error { + if pom.broker != nil { + pom.parent.unrefBrokerOffsetManager(pom.broker) + pom.broker = nil + } + + var broker *Broker + var err error + + if err = pom.parent.client.RefreshCoordinator(pom.parent.group); err != nil { + return err + } + + if broker, err = pom.parent.client.Coordinator(pom.parent.group); err != nil { + return err + } + + pom.broker = pom.parent.refBrokerOffsetManager(broker) + return nil +} + +func (pom *partitionOffsetManager) fetchInitialOffset(retries int) error { + request := new(OffsetFetchRequest) + request.Version = 1 + request.ConsumerGroup = pom.parent.group + request.AddPartition(pom.topic, pom.partition) + + response, err := pom.broker.broker.FetchOffset(request) + if err != nil { + return err + } + + block := response.GetBlock(pom.topic, pom.partition) + if block == nil { + return ErrIncompleteResponse + } + + switch block.Err { + case ErrNoError: + pom.offset = block.Offset + pom.metadata = block.Metadata + return nil + case ErrNotCoordinatorForConsumer: + if retries <= 0 { + return block.Err + } + if err := pom.selectBroker(); err != nil { + return err + } + return pom.fetchInitialOffset(retries - 1) + case ErrOffsetsLoadInProgress: + if retries <= 0 { + return block.Err + } + time.Sleep(pom.parent.conf.Metadata.Retry.Backoff) + return pom.fetchInitialOffset(retries - 1) + default: + return block.Err + } +} + +func (pom *partitionOffsetManager) handleError(err error) { + cErr := &ConsumerError{ + Topic: pom.topic, + Partition: pom.partition, + Err: err, + } + + if pom.parent.conf.Consumer.Return.Errors { + pom.errors <- cErr + } else { + Logger.Println(cErr) + } +} + +func (pom *partitionOffsetManager) Errors() <-chan *ConsumerError { + return pom.errors +} + +func (pom *partitionOffsetManager) MarkOffset(offset int64, metadata string) { + pom.lock.Lock() + defer pom.lock.Unlock() + + if offset > pom.offset { + pom.offset = offset + pom.metadata = metadata + pom.dirty = true + } +} + +func (pom *partitionOffsetManager) updateCommitted(offset int64, metadata string) { + pom.lock.Lock() + defer pom.lock.Unlock() + + if pom.offset == offset && pom.metadata == metadata { + pom.dirty = false + + select { + case pom.clean <- none{}: + default: + } + } +} + +func (pom *partitionOffsetManager) NextOffset() (int64, string) { + pom.lock.Lock() + defer pom.lock.Unlock() + + if pom.offset >= 0 { + return pom.offset + 1, pom.metadata + } + + return pom.parent.conf.Consumer.Offsets.Initial, "" +} + +func (pom *partitionOffsetManager) AsyncClose() { + go func() { + pom.lock.Lock() + dirty := pom.dirty + pom.lock.Unlock() + + if dirty { + <-pom.clean + } + + close(pom.dying) + }() +} + +func (pom *partitionOffsetManager) Close() error { + pom.AsyncClose() + + var errors ConsumerErrors + for err := range pom.errors { + errors = append(errors, err) + } + + if len(errors) > 0 { + return errors + } + return nil +} + +// Broker Offset Manager + +type brokerOffsetManager struct { + parent *offsetManager + broker *Broker + timer *time.Ticker + updateSubscriptions chan *partitionOffsetManager + subscriptions map[*partitionOffsetManager]none + refs int +} + +func (om *offsetManager) newBrokerOffsetManager(broker *Broker) *brokerOffsetManager { + bom := &brokerOffsetManager{ + parent: om, + broker: broker, + timer: time.NewTicker(om.conf.Consumer.Offsets.CommitInterval), + updateSubscriptions: make(chan *partitionOffsetManager), + subscriptions: make(map[*partitionOffsetManager]none), + } + + go withRecover(bom.mainLoop) + + return bom +} + +func (bom *brokerOffsetManager) mainLoop() { + for { + select { + case <-bom.timer.C: + if len(bom.subscriptions) > 0 { + bom.flushToBroker() + } + case s, ok := <-bom.updateSubscriptions: + if !ok { + bom.timer.Stop() + return + } + if _, ok := bom.subscriptions[s]; ok { + delete(bom.subscriptions, s) + } else { + bom.subscriptions[s] = none{} + } + } + } +} + +func (bom *brokerOffsetManager) flushToBroker() { + request := bom.constructRequest() + if request == nil { + return + } + + response, err := bom.broker.CommitOffset(request) + + if err != nil { + bom.abort(err) + return + } + + for s := range bom.subscriptions { + if request.blocks[s.topic] == nil || request.blocks[s.topic][s.partition] == nil { + continue + } + + var err KError + var ok bool + + if response.Errors[s.topic] == nil { + s.handleError(ErrIncompleteResponse) + delete(bom.subscriptions, s) + s.rebalance <- none{} + continue + } + if err, ok = response.Errors[s.topic][s.partition]; !ok { + s.handleError(ErrIncompleteResponse) + delete(bom.subscriptions, s) + s.rebalance <- none{} + continue + } + + switch err { + case ErrNoError: + block := request.blocks[s.topic][s.partition] + s.updateCommitted(block.offset, block.metadata) + break + case ErrUnknownTopicOrPartition, ErrNotLeaderForPartition, ErrLeaderNotAvailable: + delete(bom.subscriptions, s) + s.rebalance <- none{} + default: + s.handleError(err) + delete(bom.subscriptions, s) + s.rebalance <- none{} + } + } +} + +func (bom *brokerOffsetManager) constructRequest() *OffsetCommitRequest { + r := &OffsetCommitRequest{ + Version: 1, + ConsumerGroup: bom.parent.group, + ConsumerGroupGeneration: GroupGenerationUndefined, + } + + for s := range bom.subscriptions { + s.lock.Lock() + if s.dirty { + r.AddBlock(s.topic, s.partition, s.offset, ReceiveTime, s.metadata) + } + s.lock.Unlock() + } + + if len(r.blocks) > 0 { + return r + } + + return nil +} + +func (bom *brokerOffsetManager) abort(err error) { + _ = bom.broker.Close() // we don't care about the error this might return, we already have one + bom.parent.abandonBroker(bom) + + for pom := range bom.subscriptions { + pom.handleError(err) + pom.rebalance <- none{} + } + + for s := range bom.updateSubscriptions { + if _, ok := bom.subscriptions[s]; !ok { + s.handleError(err) + s.rebalance <- none{} + } + } + + bom.subscriptions = make(map[*partitionOffsetManager]none) +} diff --git a/vendor/github.com/Shopify/sarama/offset_manager_test.go b/vendor/github.com/Shopify/sarama/offset_manager_test.go new file mode 100644 index 000000000000..00d5fba9ed1d --- /dev/null +++ b/vendor/github.com/Shopify/sarama/offset_manager_test.go @@ -0,0 +1,331 @@ +package sarama + +import ( + "testing" + "time" +) + +func initOffsetManager(t *testing.T) (om OffsetManager, + testClient Client, broker, coordinator *mockBroker) { + + config := NewConfig() + config.Metadata.Retry.Max = 1 + config.Consumer.Offsets.CommitInterval = 1 * time.Millisecond + + broker = newMockBroker(t, 1) + coordinator = newMockBroker(t, 2) + + seedMeta := new(MetadataResponse) + seedMeta.AddBroker(coordinator.Addr(), coordinator.BrokerID()) + seedMeta.AddTopicPartition("my_topic", 0, 1, []int32{}, []int32{}, ErrNoError) + seedMeta.AddTopicPartition("my_topic", 1, 1, []int32{}, []int32{}, ErrNoError) + broker.Returns(seedMeta) + + var err error + testClient, err = NewClient([]string{broker.Addr()}, config) + if err != nil { + t.Fatal(err) + } + + broker.Returns(&ConsumerMetadataResponse{ + CoordinatorID: coordinator.BrokerID(), + CoordinatorHost: "127.0.0.1", + CoordinatorPort: coordinator.Port(), + }) + + om, err = NewOffsetManagerFromClient("group", testClient) + if err != nil { + t.Fatal(err) + } + + return om, testClient, broker, coordinator +} + +func initPartitionOffsetManager(t *testing.T, om OffsetManager, + coordinator *mockBroker, initialOffset int64, metadata string) PartitionOffsetManager { + + fetchResponse := new(OffsetFetchResponse) + fetchResponse.AddBlock("my_topic", 0, &OffsetFetchResponseBlock{ + Err: ErrNoError, + Offset: initialOffset, + Metadata: metadata, + }) + coordinator.Returns(fetchResponse) + + pom, err := om.ManagePartition("my_topic", 0) + if err != nil { + t.Fatal(err) + } + + return pom +} + +func TestNewOffsetManager(t *testing.T) { + seedBroker := newMockBroker(t, 1) + seedBroker.Returns(new(MetadataResponse)) + + testClient, err := NewClient([]string{seedBroker.Addr()}, nil) + if err != nil { + t.Fatal(err) + } + + _, err = NewOffsetManagerFromClient("group", testClient) + if err != nil { + t.Error(err) + } + + safeClose(t, testClient) + + _, err = NewOffsetManagerFromClient("group", testClient) + if err != ErrClosedClient { + t.Errorf("Error expected for closed client; actual value: %v", err) + } + + seedBroker.Close() +} + +// Test recovery from ErrNotCoordinatorForConsumer +// on first fetchInitialOffset call +func TestOffsetManagerFetchInitialFail(t *testing.T) { + om, testClient, broker, coordinator := initOffsetManager(t) + + // Error on first fetchInitialOffset call + responseBlock := OffsetFetchResponseBlock{ + Err: ErrNotCoordinatorForConsumer, + Offset: 5, + Metadata: "test_meta", + } + + fetchResponse := new(OffsetFetchResponse) + fetchResponse.AddBlock("my_topic", 0, &responseBlock) + coordinator.Returns(fetchResponse) + + // Refresh coordinator + newCoordinator := newMockBroker(t, 3) + broker.Returns(&ConsumerMetadataResponse{ + CoordinatorID: newCoordinator.BrokerID(), + CoordinatorHost: "127.0.0.1", + CoordinatorPort: newCoordinator.Port(), + }) + + // Second fetchInitialOffset call is fine + fetchResponse2 := new(OffsetFetchResponse) + responseBlock2 := responseBlock + responseBlock2.Err = ErrNoError + fetchResponse2.AddBlock("my_topic", 0, &responseBlock2) + newCoordinator.Returns(fetchResponse2) + + pom, err := om.ManagePartition("my_topic", 0) + if err != nil { + t.Error(err) + } + + broker.Close() + coordinator.Close() + newCoordinator.Close() + safeClose(t, pom) + safeClose(t, om) + safeClose(t, testClient) +} + +// Test fetchInitialOffset retry on ErrOffsetsLoadInProgress +func TestOffsetManagerFetchInitialLoadInProgress(t *testing.T) { + om, testClient, broker, coordinator := initOffsetManager(t) + + // Error on first fetchInitialOffset call + responseBlock := OffsetFetchResponseBlock{ + Err: ErrOffsetsLoadInProgress, + Offset: 5, + Metadata: "test_meta", + } + + fetchResponse := new(OffsetFetchResponse) + fetchResponse.AddBlock("my_topic", 0, &responseBlock) + coordinator.Returns(fetchResponse) + + // Second fetchInitialOffset call is fine + fetchResponse2 := new(OffsetFetchResponse) + responseBlock2 := responseBlock + responseBlock2.Err = ErrNoError + fetchResponse2.AddBlock("my_topic", 0, &responseBlock2) + coordinator.Returns(fetchResponse2) + + pom, err := om.ManagePartition("my_topic", 0) + if err != nil { + t.Error(err) + } + + broker.Close() + coordinator.Close() + safeClose(t, pom) + safeClose(t, om) + safeClose(t, testClient) +} + +func TestPartitionOffsetManagerInitialOffset(t *testing.T) { + om, testClient, broker, coordinator := initOffsetManager(t) + testClient.Config().Consumer.Offsets.Initial = OffsetOldest + + // Kafka returns -1 if no offset has been stored for this partition yet. + pom := initPartitionOffsetManager(t, om, coordinator, -1, "") + + offset, meta := pom.NextOffset() + if offset != OffsetOldest { + t.Errorf("Expected offset 5. Actual: %v", offset) + } + if meta != "" { + t.Errorf("Expected metadata to be empty. Actual: %q", meta) + } + + safeClose(t, pom) + safeClose(t, om) + broker.Close() + coordinator.Close() + safeClose(t, testClient) +} + +func TestPartitionOffsetManagerNextOffset(t *testing.T) { + om, testClient, broker, coordinator := initOffsetManager(t) + pom := initPartitionOffsetManager(t, om, coordinator, 5, "test_meta") + + offset, meta := pom.NextOffset() + if offset != 6 { + t.Errorf("Expected offset 5. Actual: %v", offset) + } + if meta != "test_meta" { + t.Errorf("Expected metadata \"test_meta\". Actual: %q", meta) + } + + safeClose(t, pom) + safeClose(t, om) + broker.Close() + coordinator.Close() + safeClose(t, testClient) +} + +func TestPartitionOffsetManagerMarkOffset(t *testing.T) { + om, testClient, broker, coordinator := initOffsetManager(t) + pom := initPartitionOffsetManager(t, om, coordinator, 5, "original_meta") + + ocResponse := new(OffsetCommitResponse) + ocResponse.AddError("my_topic", 0, ErrNoError) + coordinator.Returns(ocResponse) + + pom.MarkOffset(100, "modified_meta") + offset, meta := pom.NextOffset() + + if offset != 101 { + t.Errorf("Expected offset 100. Actual: %v", offset) + } + if meta != "modified_meta" { + t.Errorf("Expected metadata \"modified_meta\". Actual: %q", meta) + } + + safeClose(t, pom) + safeClose(t, om) + safeClose(t, testClient) + broker.Close() + coordinator.Close() +} + +func TestPartitionOffsetManagerCommitErr(t *testing.T) { + om, testClient, broker, coordinator := initOffsetManager(t) + pom := initPartitionOffsetManager(t, om, coordinator, 5, "meta") + + // Error on one partition + ocResponse := new(OffsetCommitResponse) + ocResponse.AddError("my_topic", 0, ErrOffsetOutOfRange) + ocResponse.AddError("my_topic", 1, ErrNoError) + coordinator.Returns(ocResponse) + + newCoordinator := newMockBroker(t, 3) + + // For RefreshCoordinator() + broker.Returns(&ConsumerMetadataResponse{ + CoordinatorID: newCoordinator.BrokerID(), + CoordinatorHost: "127.0.0.1", + CoordinatorPort: newCoordinator.Port(), + }) + + // Nothing in response.Errors at all + ocResponse2 := new(OffsetCommitResponse) + newCoordinator.Returns(ocResponse2) + + // For RefreshCoordinator() + broker.Returns(&ConsumerMetadataResponse{ + CoordinatorID: newCoordinator.BrokerID(), + CoordinatorHost: "127.0.0.1", + CoordinatorPort: newCoordinator.Port(), + }) + + // Error on the wrong partition for this pom + ocResponse3 := new(OffsetCommitResponse) + ocResponse3.AddError("my_topic", 1, ErrNoError) + newCoordinator.Returns(ocResponse3) + + // For RefreshCoordinator() + broker.Returns(&ConsumerMetadataResponse{ + CoordinatorID: newCoordinator.BrokerID(), + CoordinatorHost: "127.0.0.1", + CoordinatorPort: newCoordinator.Port(), + }) + + // ErrUnknownTopicOrPartition/ErrNotLeaderForPartition/ErrLeaderNotAvailable block + ocResponse4 := new(OffsetCommitResponse) + ocResponse4.AddError("my_topic", 0, ErrUnknownTopicOrPartition) + newCoordinator.Returns(ocResponse4) + + // For RefreshCoordinator() + broker.Returns(&ConsumerMetadataResponse{ + CoordinatorID: newCoordinator.BrokerID(), + CoordinatorHost: "127.0.0.1", + CoordinatorPort: newCoordinator.Port(), + }) + + // Normal error response + ocResponse5 := new(OffsetCommitResponse) + ocResponse5.AddError("my_topic", 0, ErrNoError) + newCoordinator.Returns(ocResponse5) + + pom.MarkOffset(100, "modified_meta") + + err := pom.Close() + if err != nil { + t.Error(err) + } + + broker.Close() + coordinator.Close() + newCoordinator.Close() + safeClose(t, om) + safeClose(t, testClient) +} + +// Test of recovery from abort +func TestAbortPartitionOffsetManager(t *testing.T) { + om, testClient, broker, coordinator := initOffsetManager(t) + pom := initPartitionOffsetManager(t, om, coordinator, 5, "meta") + + // this triggers an error in the CommitOffset request, + // which leads to the abort call + coordinator.Close() + + // Response to refresh coordinator request + newCoordinator := newMockBroker(t, 3) + broker.Returns(&ConsumerMetadataResponse{ + CoordinatorID: newCoordinator.BrokerID(), + CoordinatorHost: "127.0.0.1", + CoordinatorPort: newCoordinator.Port(), + }) + + ocResponse := new(OffsetCommitResponse) + ocResponse.AddError("my_topic", 0, ErrNoError) + newCoordinator.Returns(ocResponse) + + pom.MarkOffset(100, "modified_meta") + + safeClose(t, pom) + safeClose(t, om) + broker.Close() + safeClose(t, testClient) +} diff --git a/vendor/github.com/Shopify/sarama/offset_request.go b/vendor/github.com/Shopify/sarama/offset_request.go new file mode 100644 index 000000000000..842d5c0f8360 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/offset_request.go @@ -0,0 +1,113 @@ +package sarama + +type offsetRequestBlock struct { + time int64 + maxOffsets int32 +} + +func (r *offsetRequestBlock) encode(pe packetEncoder) error { + pe.putInt64(int64(r.time)) + pe.putInt32(r.maxOffsets) + return nil +} + +func (r *offsetRequestBlock) decode(pd packetDecoder) (err error) { + if r.time, err = pd.getInt64(); err != nil { + return err + } + if r.maxOffsets, err = pd.getInt32(); err != nil { + return err + } + return nil +} + +type OffsetRequest struct { + blocks map[string]map[int32]*offsetRequestBlock +} + +func (r *OffsetRequest) encode(pe packetEncoder) error { + pe.putInt32(-1) // replica ID is always -1 for clients + err := pe.putArrayLength(len(r.blocks)) + if err != nil { + return err + } + for topic, partitions := range r.blocks { + err = pe.putString(topic) + if err != nil { + return err + } + err = pe.putArrayLength(len(partitions)) + if err != nil { + return err + } + for partition, block := range partitions { + pe.putInt32(partition) + if err = block.encode(pe); err != nil { + return err + } + } + } + return nil +} + +func (r *OffsetRequest) decode(pd packetDecoder) error { + // Ignore replica ID + if _, err := pd.getInt32(); err != nil { + return err + } + blockCount, err := pd.getArrayLength() + if err != nil { + return err + } + if blockCount == 0 { + return nil + } + r.blocks = make(map[string]map[int32]*offsetRequestBlock) + for i := 0; i < blockCount; i++ { + topic, err := pd.getString() + if err != nil { + return err + } + partitionCount, err := pd.getArrayLength() + if err != nil { + return err + } + r.blocks[topic] = make(map[int32]*offsetRequestBlock) + for j := 0; j < partitionCount; j++ { + partition, err := pd.getInt32() + if err != nil { + return err + } + block := &offsetRequestBlock{} + if err := block.decode(pd); err != nil { + return err + } + r.blocks[topic][partition] = block + } + } + return nil +} + +func (r *OffsetRequest) key() int16 { + return 2 +} + +func (r *OffsetRequest) version() int16 { + return 0 +} + +func (r *OffsetRequest) AddBlock(topic string, partitionID int32, time int64, maxOffsets int32) { + if r.blocks == nil { + r.blocks = make(map[string]map[int32]*offsetRequestBlock) + } + + if r.blocks[topic] == nil { + r.blocks[topic] = make(map[int32]*offsetRequestBlock) + } + + tmp := new(offsetRequestBlock) + tmp.time = time + tmp.maxOffsets = maxOffsets + + r.blocks[topic][partitionID] = tmp +} diff --git a/vendor/github.com/Shopify/sarama/offset_request_test.go b/vendor/github.com/Shopify/sarama/offset_request_test.go new file mode 100644 index 000000000000..f3b3046bbb51 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/offset_request_test.go @@ -0,0 +1,26 @@ +package sarama + +import "testing" + +var ( + offsetRequestNoBlocks = []byte{ + 0xFF, 0xFF, 0xFF, 0xFF, + 0x00, 0x00, 0x00, 0x00} + + offsetRequestOneBlock = []byte{ + 0xFF, 0xFF, 0xFF, 0xFF, + 0x00, 0x00, 0x00, 0x01, + 0x00, 0x03, 'f', 'o', 'o', + 0x00, 0x00, 0x00, 0x01, + 0x00, 0x00, 0x00, 0x04, + 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x01, + 0x00, 0x00, 0x00, 0x02} +) + +func TestOffsetRequest(t *testing.T) { + request := new(OffsetRequest) + testRequest(t, "no blocks", request, offsetRequestNoBlocks) + + request.AddBlock("foo", 4, 1, 2) + testRequest(t, "one block", request, offsetRequestOneBlock) +} diff --git a/vendor/github.com/Shopify/sarama/offset_response.go b/vendor/github.com/Shopify/sarama/offset_response.go new file mode 100644 index 000000000000..07d71ca7218b --- /dev/null +++ b/vendor/github.com/Shopify/sarama/offset_response.go @@ -0,0 +1,130 @@ +package sarama + +type OffsetResponseBlock struct { + Err KError + Offsets []int64 +} + +func (r *OffsetResponseBlock) decode(pd packetDecoder) (err error) { + tmp, err := pd.getInt16() + if err != nil { + return err + } + r.Err = KError(tmp) + + r.Offsets, err = pd.getInt64Array() + + return err +} + +func (r *OffsetResponseBlock) encode(pe packetEncoder) (err error) { + pe.putInt16(int16(r.Err)) + + return pe.putInt64Array(r.Offsets) +} + +type OffsetResponse struct { + Blocks map[string]map[int32]*OffsetResponseBlock +} + +func (r *OffsetResponse) decode(pd packetDecoder) (err error) { + numTopics, err := pd.getArrayLength() + if err != nil { + return err + } + + r.Blocks = make(map[string]map[int32]*OffsetResponseBlock, numTopics) + for i := 0; i < numTopics; i++ { + name, err := pd.getString() + if err != nil { + return err + } + + numBlocks, err := pd.getArrayLength() + if err != nil { + return err + } + + r.Blocks[name] = make(map[int32]*OffsetResponseBlock, numBlocks) + + for j := 0; j < numBlocks; j++ { + id, err := pd.getInt32() + if err != nil { + return err + } + + block := new(OffsetResponseBlock) + err = block.decode(pd) + if err != nil { + return err + } + r.Blocks[name][id] = block + } + } + + return nil +} + +func (r *OffsetResponse) GetBlock(topic string, partition int32) *OffsetResponseBlock { + if r.Blocks == nil { + return nil + } + + if r.Blocks[topic] == nil { + return nil + } + + return r.Blocks[topic][partition] +} + +/* +// [0 0 0 1 ntopics +0 8 109 121 95 116 111 112 105 99 topic +0 0 0 1 npartitions +0 0 0 0 id +0 0 + +0 0 0 1 0 0 0 0 +0 1 1 1 0 0 0 1 +0 8 109 121 95 116 111 112 +105 99 0 0 0 1 0 0 +0 0 0 0 0 0 0 1 +0 0 0 0 0 1 1 1] + +*/ +func (r *OffsetResponse) encode(pe packetEncoder) (err error) { + if err = pe.putArrayLength(len(r.Blocks)); err != nil { + return err + } + + for topic, partitions := range r.Blocks { + if err = pe.putString(topic); err != nil { + return err + } + if err = pe.putArrayLength(len(partitions)); err != nil { + return err + } + for partition, block := range partitions { + pe.putInt32(partition) + if err = block.encode(pe); err != nil { + return err + } + } + } + + return nil +} + +// testing API + +func (r *OffsetResponse) AddTopicPartition(topic string, partition int32, offset int64) { + if r.Blocks == nil { + r.Blocks = make(map[string]map[int32]*OffsetResponseBlock) + } + byTopic, ok := r.Blocks[topic] + if !ok { + byTopic = make(map[int32]*OffsetResponseBlock) + r.Blocks[topic] = byTopic + } + byTopic[partition] = &OffsetResponseBlock{Offsets: []int64{offset}} +} diff --git a/vendor/github.com/Shopify/sarama/offset_response_test.go b/vendor/github.com/Shopify/sarama/offset_response_test.go new file mode 100644 index 000000000000..a427cbd20c62 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/offset_response_test.go @@ -0,0 +1,62 @@ +package sarama + +import "testing" + +var ( + emptyOffsetResponse = []byte{ + 0x00, 0x00, 0x00, 0x00} + + normalOffsetResponse = []byte{ + 0x00, 0x00, 0x00, 0x02, + + 0x00, 0x01, 'a', + 0x00, 0x00, 0x00, 0x00, + + 0x00, 0x01, 'z', + 0x00, 0x00, 0x00, 0x01, + 0x00, 0x00, 0x00, 0x02, + 0x00, 0x00, + 0x00, 0x00, 0x00, 0x02, + 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x05, + 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x06} +) + +func TestEmptyOffsetResponse(t *testing.T) { + response := OffsetResponse{} + + testDecodable(t, "empty", &response, emptyOffsetResponse) + if len(response.Blocks) != 0 { + t.Error("Decoding produced", len(response.Blocks), "topics where there were none.") + } +} + +func TestNormalOffsetResponse(t *testing.T) { + response := OffsetResponse{} + + testDecodable(t, "normal", &response, normalOffsetResponse) + + if len(response.Blocks) != 2 { + t.Fatal("Decoding produced", len(response.Blocks), "topics where there were two.") + } + + if len(response.Blocks["a"]) != 0 { + t.Fatal("Decoding produced", len(response.Blocks["a"]), "partitions for topic 'a' where there were none.") + } + + if len(response.Blocks["z"]) != 1 { + t.Fatal("Decoding produced", len(response.Blocks["z"]), "partitions for topic 'z' where there was one.") + } + + if response.Blocks["z"][2].Err != ErrNoError { + t.Fatal("Decoding produced invalid error for topic z partition 2.") + } + + if len(response.Blocks["z"][2].Offsets) != 2 { + t.Fatal("Decoding produced invalid number of offsets for topic z partition 2.") + } + + if response.Blocks["z"][2].Offsets[0] != 5 || response.Blocks["z"][2].Offsets[1] != 6 { + t.Fatal("Decoding produced invalid offsets for topic z partition 2.") + } + +} diff --git a/vendor/github.com/Shopify/sarama/packet_decoder.go b/vendor/github.com/Shopify/sarama/packet_decoder.go new file mode 100644 index 000000000000..28670c0e625a --- /dev/null +++ b/vendor/github.com/Shopify/sarama/packet_decoder.go @@ -0,0 +1,45 @@ +package sarama + +// PacketDecoder is the interface providing helpers for reading with Kafka's encoding rules. +// Types implementing Decoder only need to worry about calling methods like GetString, +// not about how a string is represented in Kafka. +type packetDecoder interface { + // Primitives + getInt8() (int8, error) + getInt16() (int16, error) + getInt32() (int32, error) + getInt64() (int64, error) + getArrayLength() (int, error) + + // Collections + getBytes() ([]byte, error) + getString() (string, error) + getInt32Array() ([]int32, error) + getInt64Array() ([]int64, error) + getStringArray() ([]string, error) + + // Subsets + remaining() int + getSubset(length int) (packetDecoder, error) + + // Stacks, see PushDecoder + push(in pushDecoder) error + pop() error +} + +// PushDecoder is the interface for decoding fields like CRCs and lengths where the validity +// of the field depends on what is after it in the packet. Start them with PacketDecoder.Push() where +// the actual value is located in the packet, then PacketDecoder.Pop() them when all the bytes they +// depend upon have been decoded. +type pushDecoder interface { + // Saves the offset into the input buffer as the location to actually read the calculated value when able. + saveOffset(in int) + + // Returns the length of data to reserve for the input of this encoder (eg 4 bytes for a CRC32). + reserveLength() int + + // Indicates that all required data is now available to calculate and check the field. + // SaveOffset is guaranteed to have been called first. The implementation should read ReserveLength() bytes + // of data from the saved offset, and verify it based on the data between the saved offset and curOffset. + check(curOffset int, buf []byte) error +} diff --git a/vendor/github.com/Shopify/sarama/packet_encoder.go b/vendor/github.com/Shopify/sarama/packet_encoder.go new file mode 100644 index 000000000000..0df6e24aa6d9 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/packet_encoder.go @@ -0,0 +1,42 @@ +package sarama + +// PacketEncoder is the interface providing helpers for writing with Kafka's encoding rules. +// Types implementing Encoder only need to worry about calling methods like PutString, +// not about how a string is represented in Kafka. +type packetEncoder interface { + // Primitives + putInt8(in int8) + putInt16(in int16) + putInt32(in int32) + putInt64(in int64) + putArrayLength(in int) error + + // Collections + putBytes(in []byte) error + putRawBytes(in []byte) error + putString(in string) error + putStringArray(in []string) error + putInt32Array(in []int32) error + putInt64Array(in []int64) error + + // Stacks, see PushEncoder + push(in pushEncoder) + pop() error +} + +// PushEncoder is the interface for encoding fields like CRCs and lengths where the value +// of the field depends on what is encoded after it in the packet. Start them with PacketEncoder.Push() where +// the actual value is located in the packet, then PacketEncoder.Pop() them when all the bytes they +// depend upon have been written. +type pushEncoder interface { + // Saves the offset into the input buffer as the location to actually write the calculated value when able. + saveOffset(in int) + + // Returns the length of data to reserve for the output of this encoder (eg 4 bytes for a CRC32). + reserveLength() int + + // Indicates that all required data is now available to calculate and write the field. + // SaveOffset is guaranteed to have been called first. The implementation should write ReserveLength() bytes + // of data to the saved offset, based on the data between the saved offset and curOffset. + run(curOffset int, buf []byte) error +} diff --git a/vendor/github.com/Shopify/sarama/partitioner.go b/vendor/github.com/Shopify/sarama/partitioner.go new file mode 100644 index 000000000000..8c2bca4138d1 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/partitioner.go @@ -0,0 +1,123 @@ +package sarama + +import ( + "hash" + "hash/fnv" + "math/rand" + "time" +) + +// Partitioner is anything that, given a Kafka message and a number of partitions indexed [0...numPartitions-1], +// decides to which partition to send the message. RandomPartitioner, RoundRobinPartitioner and HashPartitioner are provided +// as simple default implementations. +type Partitioner interface { + // Partition takes a message and partition count and chooses a partition + Partition(message *ProducerMessage, numPartitions int32) (int32, error) + + // RequiresConsistency indicates to the user of the partitioner whether the + // mapping of key->partition is consistent or not. Specifically, if a + // partitioner requires consistency then it must be allowed to choose from all + // partitions (even ones known to be unavailable), and its choice must be + // respected by the caller. The obvious example is the HashPartitioner. + RequiresConsistency() bool +} + +// PartitionerConstructor is the type for a function capable of constructing new Partitioners. +type PartitionerConstructor func(topic string) Partitioner + +type manualPartitioner struct{} + +// NewManualPartitioner returns a Partitioner which uses the partition manually set in the provided +// ProducerMessage's Partition field as the partition to produce to. +func NewManualPartitioner(topic string) Partitioner { + return new(manualPartitioner) +} + +func (p *manualPartitioner) Partition(message *ProducerMessage, numPartitions int32) (int32, error) { + return message.Partition, nil +} + +func (p *manualPartitioner) RequiresConsistency() bool { + return true +} + +type randomPartitioner struct { + generator *rand.Rand +} + +// NewRandomPartitioner returns a Partitioner which chooses a random partition each time. +func NewRandomPartitioner(topic string) Partitioner { + p := new(randomPartitioner) + p.generator = rand.New(rand.NewSource(time.Now().UTC().UnixNano())) + return p +} + +func (p *randomPartitioner) Partition(message *ProducerMessage, numPartitions int32) (int32, error) { + return int32(p.generator.Intn(int(numPartitions))), nil +} + +func (p *randomPartitioner) RequiresConsistency() bool { + return false +} + +type roundRobinPartitioner struct { + partition int32 +} + +// NewRoundRobinPartitioner returns a Partitioner which walks through the available partitions one at a time. +func NewRoundRobinPartitioner(topic string) Partitioner { + return &roundRobinPartitioner{} +} + +func (p *roundRobinPartitioner) Partition(message *ProducerMessage, numPartitions int32) (int32, error) { + if p.partition >= numPartitions { + p.partition = 0 + } + ret := p.partition + p.partition++ + return ret, nil +} + +func (p *roundRobinPartitioner) RequiresConsistency() bool { + return false +} + +type hashPartitioner struct { + random Partitioner + hasher hash.Hash32 +} + +// NewHashPartitioner returns a Partitioner which behaves as follows. If the message's key is nil, or fails to +// encode, then a random partition is chosen. Otherwise the FNV-1a hash of the encoded bytes of the message key +// is used, modulus the number of partitions. This ensures that messages with the same key always end up on the +// same partition. +func NewHashPartitioner(topic string) Partitioner { + p := new(hashPartitioner) + p.random = NewRandomPartitioner(topic) + p.hasher = fnv.New32a() + return p +} + +func (p *hashPartitioner) Partition(message *ProducerMessage, numPartitions int32) (int32, error) { + if message.Key == nil { + return p.random.Partition(message, numPartitions) + } + bytes, err := message.Key.Encode() + if err != nil { + return -1, err + } + p.hasher.Reset() + _, err = p.hasher.Write(bytes) + if err != nil { + return -1, err + } + hash := int32(p.hasher.Sum32()) + if hash < 0 { + hash = -hash + } + return hash % numPartitions, nil +} + +func (p *hashPartitioner) RequiresConsistency() bool { + return true +} diff --git a/vendor/github.com/Shopify/sarama/partitioner_test.go b/vendor/github.com/Shopify/sarama/partitioner_test.go new file mode 100644 index 000000000000..f44c509d6522 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/partitioner_test.go @@ -0,0 +1,198 @@ +package sarama + +import ( + "crypto/rand" + "log" + "testing" +) + +func assertPartitioningConsistent(t *testing.T, partitioner Partitioner, message *ProducerMessage, numPartitions int32) { + choice, err := partitioner.Partition(message, numPartitions) + if err != nil { + t.Error(partitioner, err) + } + if choice < 0 || choice >= numPartitions { + t.Error(partitioner, "returned partition", choice, "outside of range for", message) + } + for i := 1; i < 50; i++ { + newChoice, err := partitioner.Partition(message, numPartitions) + if err != nil { + t.Error(partitioner, err) + } + if newChoice != choice { + t.Error(partitioner, "returned partition", newChoice, "inconsistent with", choice, ".") + } + } +} + +func TestRandomPartitioner(t *testing.T) { + partitioner := NewRandomPartitioner("mytopic") + + choice, err := partitioner.Partition(nil, 1) + if err != nil { + t.Error(partitioner, err) + } + if choice != 0 { + t.Error("Returned non-zero partition when only one available.") + } + + for i := 1; i < 50; i++ { + choice, err := partitioner.Partition(nil, 50) + if err != nil { + t.Error(partitioner, err) + } + if choice < 0 || choice >= 50 { + t.Error("Returned partition", choice, "outside of range.") + } + } +} + +func TestRoundRobinPartitioner(t *testing.T) { + partitioner := NewRoundRobinPartitioner("mytopic") + + choice, err := partitioner.Partition(nil, 1) + if err != nil { + t.Error(partitioner, err) + } + if choice != 0 { + t.Error("Returned non-zero partition when only one available.") + } + + var i int32 + for i = 1; i < 50; i++ { + choice, err := partitioner.Partition(nil, 7) + if err != nil { + t.Error(partitioner, err) + } + if choice != i%7 { + t.Error("Returned partition", choice, "expecting", i%7) + } + } +} + +func TestHashPartitioner(t *testing.T) { + partitioner := NewHashPartitioner("mytopic") + + choice, err := partitioner.Partition(&ProducerMessage{}, 1) + if err != nil { + t.Error(partitioner, err) + } + if choice != 0 { + t.Error("Returned non-zero partition when only one available.") + } + + for i := 1; i < 50; i++ { + choice, err := partitioner.Partition(&ProducerMessage{}, 50) + if err != nil { + t.Error(partitioner, err) + } + if choice < 0 || choice >= 50 { + t.Error("Returned partition", choice, "outside of range for nil key.") + } + } + + buf := make([]byte, 256) + for i := 1; i < 50; i++ { + if _, err := rand.Read(buf); err != nil { + t.Error(err) + } + assertPartitioningConsistent(t, partitioner, &ProducerMessage{Key: ByteEncoder(buf)}, 50) + } +} + +func TestManualPartitioner(t *testing.T) { + partitioner := NewManualPartitioner("mytopic") + + choice, err := partitioner.Partition(&ProducerMessage{}, 1) + if err != nil { + t.Error(partitioner, err) + } + if choice != 0 { + t.Error("Returned non-zero partition when only one available.") + } + + for i := int32(1); i < 50; i++ { + choice, err := partitioner.Partition(&ProducerMessage{Partition: i}, 50) + if err != nil { + t.Error(partitioner, err) + } + if choice != i { + t.Error("Returned partition not the same as the input partition") + } + } +} + +// By default, Sarama uses the message's key to consistently assign a partition to +// a message using hashing. If no key is set, a random partition will be chosen. +// This example shows how you can partition messages randomly, even when a key is set, +// by overriding Config.Producer.Partitioner. +func ExamplePartitioner_random() { + config := NewConfig() + config.Producer.Partitioner = NewRandomPartitioner + + producer, err := NewSyncProducer([]string{"localhost:9092"}, config) + if err != nil { + log.Fatal(err) + } + defer func() { + if err := producer.Close(); err != nil { + log.Println("Failed to close producer:", err) + } + }() + + msg := &ProducerMessage{Topic: "test", Key: StringEncoder("key is set"), Value: StringEncoder("test")} + partition, offset, err := producer.SendMessage(msg) + if err != nil { + log.Fatalln("Failed to produce message to kafka cluster.") + } + + log.Printf("Produced message to partition %d with offset %d", partition, offset) +} + +// This example shows how to assign partitions to your messages manually. +func ExamplePartitioner_manual() { + config := NewConfig() + + // First, we tell the producer that we are going to partition ourselves. + config.Producer.Partitioner = NewManualPartitioner + + producer, err := NewSyncProducer([]string{"localhost:9092"}, config) + if err != nil { + log.Fatal(err) + } + defer func() { + if err := producer.Close(); err != nil { + log.Println("Failed to close producer:", err) + } + }() + + // Now, we set the Partition field of the ProducerMessage struct. + msg := &ProducerMessage{Topic: "test", Partition: 6, Value: StringEncoder("test")} + + partition, offset, err := producer.SendMessage(msg) + if err != nil { + log.Fatalln("Failed to produce message to kafka cluster.") + } + + if partition != 6 { + log.Fatal("Message should have been produced to partition 6!") + } + + log.Printf("Produced message to partition %d with offset %d", partition, offset) +} + +// This example shows how to set a different partitioner depending on the topic. +func ExamplePartitioner_per_topic() { + config := NewConfig() + config.Producer.Partitioner = func(topic string) Partitioner { + switch topic { + case "access_log", "error_log": + return NewRandomPartitioner(topic) + + default: + return NewHashPartitioner(topic) + } + } + + // ... +} diff --git a/vendor/github.com/Shopify/sarama/prep_encoder.go b/vendor/github.com/Shopify/sarama/prep_encoder.go new file mode 100644 index 000000000000..8c6ba8502c43 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/prep_encoder.go @@ -0,0 +1,110 @@ +package sarama + +import ( + "fmt" + "math" +) + +type prepEncoder struct { + length int +} + +// primitives + +func (pe *prepEncoder) putInt8(in int8) { + pe.length++ +} + +func (pe *prepEncoder) putInt16(in int16) { + pe.length += 2 +} + +func (pe *prepEncoder) putInt32(in int32) { + pe.length += 4 +} + +func (pe *prepEncoder) putInt64(in int64) { + pe.length += 8 +} + +func (pe *prepEncoder) putArrayLength(in int) error { + if in > math.MaxInt32 { + return PacketEncodingError{fmt.Sprintf("array too long (%d)", in)} + } + pe.length += 4 + return nil +} + +// arrays + +func (pe *prepEncoder) putBytes(in []byte) error { + pe.length += 4 + if in == nil { + return nil + } + if len(in) > math.MaxInt32 { + return PacketEncodingError{fmt.Sprintf("byteslice too long (%d)", len(in))} + } + pe.length += len(in) + return nil +} + +func (pe *prepEncoder) putRawBytes(in []byte) error { + if len(in) > math.MaxInt32 { + return PacketEncodingError{fmt.Sprintf("byteslice too long (%d)", len(in))} + } + pe.length += len(in) + return nil +} + +func (pe *prepEncoder) putString(in string) error { + pe.length += 2 + if len(in) > math.MaxInt16 { + return PacketEncodingError{fmt.Sprintf("string too long (%d)", len(in))} + } + pe.length += len(in) + return nil +} + +func (pe *prepEncoder) putStringArray(in []string) error { + err := pe.putArrayLength(len(in)) + if err != nil { + return err + } + + for _, str := range in { + if err := pe.putString(str); err != nil { + return err + } + } + + return nil +} + +func (pe *prepEncoder) putInt32Array(in []int32) error { + err := pe.putArrayLength(len(in)) + if err != nil { + return err + } + pe.length += 4 * len(in) + return nil +} + +func (pe *prepEncoder) putInt64Array(in []int64) error { + err := pe.putArrayLength(len(in)) + if err != nil { + return err + } + pe.length += 8 * len(in) + return nil +} + +// stackable + +func (pe *prepEncoder) push(in pushEncoder) { + pe.length += in.reserveLength() +} + +func (pe *prepEncoder) pop() error { + return nil +} diff --git a/vendor/github.com/Shopify/sarama/produce_request.go b/vendor/github.com/Shopify/sarama/produce_request.go new file mode 100644 index 000000000000..473513c6a2ef --- /dev/null +++ b/vendor/github.com/Shopify/sarama/produce_request.go @@ -0,0 +1,145 @@ +package sarama + +// RequiredAcks is used in Produce Requests to tell the broker how many replica acknowledgements +// it must see before responding. Any of the constants defined here are valid. On broker versions +// prior to 0.8.2.0 any other positive int16 is also valid (the broker will wait for that many +// acknowledgements) but in 0.8.2.0 and later this will raise an exception (it has been replaced +// by setting the `min.isr` value in the brokers configuration). +type RequiredAcks int16 + +const ( + // NoResponse doesn't send any response, the TCP ACK is all you get. + NoResponse RequiredAcks = 0 + // WaitForLocal waits for only the local commit to succeed before responding. + WaitForLocal RequiredAcks = 1 + // WaitForAll waits for all replicas to commit before responding. + WaitForAll RequiredAcks = -1 +) + +type ProduceRequest struct { + RequiredAcks RequiredAcks + Timeout int32 + msgSets map[string]map[int32]*MessageSet +} + +func (p *ProduceRequest) encode(pe packetEncoder) error { + pe.putInt16(int16(p.RequiredAcks)) + pe.putInt32(p.Timeout) + err := pe.putArrayLength(len(p.msgSets)) + if err != nil { + return err + } + for topic, partitions := range p.msgSets { + err = pe.putString(topic) + if err != nil { + return err + } + err = pe.putArrayLength(len(partitions)) + if err != nil { + return err + } + for id, msgSet := range partitions { + pe.putInt32(id) + pe.push(&lengthField{}) + err = msgSet.encode(pe) + if err != nil { + return err + } + err = pe.pop() + if err != nil { + return err + } + } + } + return nil +} + +func (p *ProduceRequest) decode(pd packetDecoder) error { + requiredAcks, err := pd.getInt16() + if err != nil { + return err + } + p.RequiredAcks = RequiredAcks(requiredAcks) + if p.Timeout, err = pd.getInt32(); err != nil { + return err + } + topicCount, err := pd.getArrayLength() + if err != nil { + return err + } + if topicCount == 0 { + return nil + } + p.msgSets = make(map[string]map[int32]*MessageSet) + for i := 0; i < topicCount; i++ { + topic, err := pd.getString() + if err != nil { + return err + } + partitionCount, err := pd.getArrayLength() + if err != nil { + return err + } + p.msgSets[topic] = make(map[int32]*MessageSet) + for j := 0; j < partitionCount; j++ { + partition, err := pd.getInt32() + if err != nil { + return err + } + messageSetSize, err := pd.getInt32() + if err != nil { + return err + } + msgSetDecoder, err := pd.getSubset(int(messageSetSize)) + if err != nil { + return err + } + msgSet := &MessageSet{} + err = msgSet.decode(msgSetDecoder) + if err != nil { + return err + } + p.msgSets[topic][partition] = msgSet + } + } + return nil +} + +func (p *ProduceRequest) key() int16 { + return 0 +} + +func (p *ProduceRequest) version() int16 { + return 0 +} + +func (p *ProduceRequest) AddMessage(topic string, partition int32, msg *Message) { + if p.msgSets == nil { + p.msgSets = make(map[string]map[int32]*MessageSet) + } + + if p.msgSets[topic] == nil { + p.msgSets[topic] = make(map[int32]*MessageSet) + } + + set := p.msgSets[topic][partition] + + if set == nil { + set = new(MessageSet) + p.msgSets[topic][partition] = set + } + + set.addMessage(msg) +} + +func (p *ProduceRequest) AddSet(topic string, partition int32, set *MessageSet) { + if p.msgSets == nil { + p.msgSets = make(map[string]map[int32]*MessageSet) + } + + if p.msgSets[topic] == nil { + p.msgSets[topic] = make(map[int32]*MessageSet) + } + + p.msgSets[topic][partition] = set +} diff --git a/vendor/github.com/Shopify/sarama/produce_request_test.go b/vendor/github.com/Shopify/sarama/produce_request_test.go new file mode 100644 index 000000000000..21f4ba5b14b3 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/produce_request_test.go @@ -0,0 +1,47 @@ +package sarama + +import ( + "testing" +) + +var ( + produceRequestEmpty = []byte{ + 0x00, 0x00, + 0x00, 0x00, 0x00, 0x00, + 0x00, 0x00, 0x00, 0x00} + + produceRequestHeader = []byte{ + 0x01, 0x23, + 0x00, 0x00, 0x04, 0x44, + 0x00, 0x00, 0x00, 0x00} + + produceRequestOneMessage = []byte{ + 0x01, 0x23, + 0x00, 0x00, 0x04, 0x44, + 0x00, 0x00, 0x00, 0x01, + 0x00, 0x05, 't', 'o', 'p', 'i', 'c', + 0x00, 0x00, 0x00, 0x01, + 0x00, 0x00, 0x00, 0xAD, + 0x00, 0x00, 0x00, 0x1C, + // messageSet + 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, + 0x00, 0x00, 0x00, 0x10, + // message + 0x23, 0x96, 0x4a, 0xf7, // CRC + 0x00, + 0x00, + 0xFF, 0xFF, 0xFF, 0xFF, + 0x00, 0x00, 0x00, 0x02, 0x00, 0xEE} +) + +func TestProduceRequest(t *testing.T) { + request := new(ProduceRequest) + testRequest(t, "empty", request, produceRequestEmpty) + + request.RequiredAcks = 0x123 + request.Timeout = 0x444 + testRequest(t, "header", request, produceRequestHeader) + + request.AddMessage("topic", 0xAD, &Message{Codec: CompressionNone, Key: nil, Value: []byte{0x00, 0xEE}}) + testRequest(t, "one message", request, produceRequestOneMessage) +} diff --git a/vendor/github.com/Shopify/sarama/produce_response.go b/vendor/github.com/Shopify/sarama/produce_response.go new file mode 100644 index 000000000000..1f49a85600fe --- /dev/null +++ b/vendor/github.com/Shopify/sarama/produce_response.go @@ -0,0 +1,112 @@ +package sarama + +type ProduceResponseBlock struct { + Err KError + Offset int64 +} + +func (pr *ProduceResponseBlock) decode(pd packetDecoder) (err error) { + tmp, err := pd.getInt16() + if err != nil { + return err + } + pr.Err = KError(tmp) + + pr.Offset, err = pd.getInt64() + if err != nil { + return err + } + + return nil +} + +type ProduceResponse struct { + Blocks map[string]map[int32]*ProduceResponseBlock +} + +func (pr *ProduceResponse) decode(pd packetDecoder) (err error) { + numTopics, err := pd.getArrayLength() + if err != nil { + return err + } + + pr.Blocks = make(map[string]map[int32]*ProduceResponseBlock, numTopics) + for i := 0; i < numTopics; i++ { + name, err := pd.getString() + if err != nil { + return err + } + + numBlocks, err := pd.getArrayLength() + if err != nil { + return err + } + + pr.Blocks[name] = make(map[int32]*ProduceResponseBlock, numBlocks) + + for j := 0; j < numBlocks; j++ { + id, err := pd.getInt32() + if err != nil { + return err + } + + block := new(ProduceResponseBlock) + err = block.decode(pd) + if err != nil { + return err + } + pr.Blocks[name][id] = block + } + } + + return nil +} + +func (pr *ProduceResponse) encode(pe packetEncoder) error { + err := pe.putArrayLength(len(pr.Blocks)) + if err != nil { + return err + } + for topic, partitions := range pr.Blocks { + err = pe.putString(topic) + if err != nil { + return err + } + err = pe.putArrayLength(len(partitions)) + if err != nil { + return err + } + for id, prb := range partitions { + pe.putInt32(id) + pe.putInt16(int16(prb.Err)) + pe.putInt64(prb.Offset) + } + } + return nil +} + +func (pr *ProduceResponse) GetBlock(topic string, partition int32) *ProduceResponseBlock { + if pr.Blocks == nil { + return nil + } + + if pr.Blocks[topic] == nil { + return nil + } + + return pr.Blocks[topic][partition] +} + +// Testing API + +func (pr *ProduceResponse) AddTopicPartition(topic string, partition int32, err KError) { + if pr.Blocks == nil { + pr.Blocks = make(map[string]map[int32]*ProduceResponseBlock) + } + byTopic, ok := pr.Blocks[topic] + if !ok { + byTopic = make(map[int32]*ProduceResponseBlock) + pr.Blocks[topic] = byTopic + } + byTopic[partition] = &ProduceResponseBlock{Err: err} +} diff --git a/vendor/github.com/Shopify/sarama/produce_response_test.go b/vendor/github.com/Shopify/sarama/produce_response_test.go new file mode 100644 index 000000000000..5c3131af4a3a --- /dev/null +++ b/vendor/github.com/Shopify/sarama/produce_response_test.go @@ -0,0 +1,67 @@ +package sarama + +import "testing" + +var ( + produceResponseNoBlocks = []byte{ + 0x00, 0x00, 0x00, 0x00} + + produceResponseManyBlocks = []byte{ + 0x00, 0x00, 0x00, 0x02, + + 0x00, 0x03, 'f', 'o', 'o', + 0x00, 0x00, 0x00, 0x00, + + 0x00, 0x03, 'b', 'a', 'r', + 0x00, 0x00, 0x00, 0x02, + + 0x00, 0x00, 0x00, 0x01, + 0x00, 0x00, + 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0xFF, + + 0x00, 0x00, 0x00, 0x02, + 0x00, 0x02, + 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00} +) + +func TestProduceResponse(t *testing.T) { + response := ProduceResponse{} + + testDecodable(t, "no blocks", &response, produceResponseNoBlocks) + if len(response.Blocks) != 0 { + t.Error("Decoding produced", len(response.Blocks), "topics where there were none") + } + + testDecodable(t, "many blocks", &response, produceResponseManyBlocks) + if len(response.Blocks) != 2 { + t.Error("Decoding produced", len(response.Blocks), "topics where there were 2") + } + if len(response.Blocks["foo"]) != 0 { + t.Error("Decoding produced", len(response.Blocks["foo"]), "partitions for 'foo' where there were none") + } + if len(response.Blocks["bar"]) != 2 { + t.Error("Decoding produced", len(response.Blocks["bar"]), "partitions for 'bar' where there were two") + } + block := response.GetBlock("bar", 1) + if block == nil { + t.Error("Decoding did not produce a block for bar/1") + } else { + if block.Err != ErrNoError { + t.Error("Decoding failed for bar/1/Err, got:", int16(block.Err)) + } + if block.Offset != 0xFF { + t.Error("Decoding failed for bar/1/Offset, got:", block.Offset) + } + } + block = response.GetBlock("bar", 2) + if block == nil { + t.Error("Decoding did not produce a block for bar/2") + } else { + if block.Err != ErrInvalidMessage { + t.Error("Decoding failed for bar/2/Err, got:", int16(block.Err)) + } + if block.Offset != 0 { + t.Error("Decoding failed for bar/2/Offset, got:", block.Offset) + } + } +} diff --git a/vendor/github.com/Shopify/sarama/produce_set.go b/vendor/github.com/Shopify/sarama/produce_set.go new file mode 100644 index 000000000000..9fe5f79d4c50 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/produce_set.go @@ -0,0 +1,158 @@ +package sarama + +import "time" + +type partitionSet struct { + msgs []*ProducerMessage + setToSend *MessageSet + bufferBytes int +} + +type produceSet struct { + parent *asyncProducer + msgs map[string]map[int32]*partitionSet + + bufferBytes int + bufferCount int +} + +func newProduceSet(parent *asyncProducer) *produceSet { + return &produceSet{ + msgs: make(map[string]map[int32]*partitionSet), + parent: parent, + } +} + +func (ps *produceSet) add(msg *ProducerMessage) error { + var err error + var key, val []byte + + if msg.Key != nil { + if key, err = msg.Key.Encode(); err != nil { + return err + } + } + + if msg.Value != nil { + if val, err = msg.Value.Encode(); err != nil { + return err + } + } + + partitions := ps.msgs[msg.Topic] + if partitions == nil { + partitions = make(map[int32]*partitionSet) + ps.msgs[msg.Topic] = partitions + } + + set := partitions[msg.Partition] + if set == nil { + set = &partitionSet{setToSend: new(MessageSet)} + partitions[msg.Partition] = set + } + + set.msgs = append(set.msgs, msg) + set.setToSend.addMessage(&Message{Codec: CompressionNone, Key: key, Value: val}) + + size := producerMessageOverhead + len(key) + len(val) + set.bufferBytes += size + ps.bufferBytes += size + ps.bufferCount++ + + return nil +} + +func (ps *produceSet) buildRequest() *ProduceRequest { + req := &ProduceRequest{ + RequiredAcks: ps.parent.conf.Producer.RequiredAcks, + Timeout: int32(ps.parent.conf.Producer.Timeout / time.Millisecond), + } + + for topic, partitionSet := range ps.msgs { + for partition, set := range partitionSet { + if ps.parent.conf.Producer.Compression == CompressionNone { + req.AddSet(topic, partition, set.setToSend) + } else { + // When compression is enabled, the entire set for each partition is compressed + // and sent as the payload of a single fake "message" with the appropriate codec + // set and no key. When the server sees a message with a compression codec, it + // decompresses the payload and treats the result as its message set. + payload, err := encode(set.setToSend) + if err != nil { + Logger.Println(err) // if this happens, it's basically our fault. + panic(err) + } + req.AddMessage(topic, partition, &Message{ + Codec: ps.parent.conf.Producer.Compression, + Key: nil, + Value: payload, + }) + } + } + } + + return req +} + +func (ps *produceSet) eachPartition(cb func(topic string, partition int32, msgs []*ProducerMessage)) { + for topic, partitionSet := range ps.msgs { + for partition, set := range partitionSet { + cb(topic, partition, set.msgs) + } + } +} + +func (ps *produceSet) dropPartition(topic string, partition int32) []*ProducerMessage { + if ps.msgs[topic] == nil { + return nil + } + set := ps.msgs[topic][partition] + if set == nil { + return nil + } + ps.bufferBytes -= set.bufferBytes + ps.bufferCount -= len(set.msgs) + delete(ps.msgs[topic], partition) + return set.msgs +} + +func (ps *produceSet) wouldOverflow(msg *ProducerMessage) bool { + switch { + // Would we overflow our maximum possible size-on-the-wire? 10KiB is arbitrary overhead for safety. + case ps.bufferBytes+msg.byteSize() >= int(MaxRequestSize-(10*1024)): + return true + // Would we overflow the size-limit of a compressed message-batch for this partition? + case ps.parent.conf.Producer.Compression != CompressionNone && + ps.msgs[msg.Topic] != nil && ps.msgs[msg.Topic][msg.Partition] != nil && + ps.msgs[msg.Topic][msg.Partition].bufferBytes+msg.byteSize() >= ps.parent.conf.Producer.MaxMessageBytes: + return true + // Would we overflow simply in number of messages? + case ps.parent.conf.Producer.Flush.MaxMessages > 0 && ps.bufferCount >= ps.parent.conf.Producer.Flush.MaxMessages: + return true + default: + return false + } +} + +func (ps *produceSet) readyToFlush() bool { + switch { + // If we don't have any messages, nothing else matters + case ps.empty(): + return false + // If all three config values are 0, we always flush as-fast-as-possible + case ps.parent.conf.Producer.Flush.Frequency == 0 && ps.parent.conf.Producer.Flush.Bytes == 0 && ps.parent.conf.Producer.Flush.Messages == 0: + return true + // If we've passed the message trigger-point + case ps.parent.conf.Producer.Flush.Messages > 0 && ps.bufferCount >= ps.parent.conf.Producer.Flush.Messages: + return true + // If we've passed the byte trigger-point + case ps.parent.conf.Producer.Flush.Bytes > 0 && ps.bufferBytes >= ps.parent.conf.Producer.Flush.Bytes: + return true + default: + return false + } +} + +func (ps *produceSet) empty() bool { + return ps.bufferCount == 0 +} diff --git a/vendor/github.com/Shopify/sarama/produce_set_test.go b/vendor/github.com/Shopify/sarama/produce_set_test.go new file mode 100644 index 000000000000..da62da914896 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/produce_set_test.go @@ -0,0 +1,143 @@ +package sarama + +import ( + "testing" + "time" +) + +func makeProduceSet() (*asyncProducer, *produceSet) { + parent := &asyncProducer{ + conf: NewConfig(), + } + return parent, newProduceSet(parent) +} + +func safeAddMessage(t *testing.T, ps *produceSet, msg *ProducerMessage) { + if err := ps.add(msg); err != nil { + t.Error(err) + } +} + +func TestProduceSetInitial(t *testing.T) { + _, ps := makeProduceSet() + + if !ps.empty() { + t.Error("New produceSet should be empty") + } + + if ps.readyToFlush() { + t.Error("Empty produceSet must never be ready to flush") + } +} + +func TestProduceSetAddingMessages(t *testing.T) { + parent, ps := makeProduceSet() + parent.conf.Producer.Flush.MaxMessages = 1000 + + msg := &ProducerMessage{Key: StringEncoder(TestMessage), Value: StringEncoder(TestMessage)} + safeAddMessage(t, ps, msg) + + if ps.empty() { + t.Error("set shouldn't be empty when a message is added") + } + + if !ps.readyToFlush() { + t.Error("by default set should be ready to flush when any message is in place") + } + + for i := 0; i < 999; i++ { + if ps.wouldOverflow(msg) { + t.Error("set shouldn't fill up after only", i+1, "messages") + } + safeAddMessage(t, ps, msg) + } + + if !ps.wouldOverflow(msg) { + t.Error("set should be full after 1000 messages") + } +} + +func TestProduceSetPartitionTracking(t *testing.T) { + _, ps := makeProduceSet() + + m1 := &ProducerMessage{Topic: "t1", Partition: 0} + m2 := &ProducerMessage{Topic: "t1", Partition: 1} + m3 := &ProducerMessage{Topic: "t2", Partition: 0} + safeAddMessage(t, ps, m1) + safeAddMessage(t, ps, m2) + safeAddMessage(t, ps, m3) + + seenT1P0 := false + seenT1P1 := false + seenT2P0 := false + + ps.eachPartition(func(topic string, partition int32, msgs []*ProducerMessage) { + if len(msgs) != 1 { + t.Error("Wrong message count") + } + + if topic == "t1" && partition == 0 { + seenT1P0 = true + } else if topic == "t1" && partition == 1 { + seenT1P1 = true + } else if topic == "t2" && partition == 0 { + seenT2P0 = true + } + }) + + if !seenT1P0 { + t.Error("Didn't see t1p0") + } + if !seenT1P1 { + t.Error("Didn't see t1p1") + } + if !seenT2P0 { + t.Error("Didn't see t2p0") + } + + if len(ps.dropPartition("t1", 1)) != 1 { + t.Error("Got wrong messages back from dropping partition") + } + + if ps.bufferCount != 2 { + t.Error("Incorrect buffer count after dropping partition") + } +} + +func TestProduceSetRequestBuilding(t *testing.T) { + parent, ps := makeProduceSet() + parent.conf.Producer.RequiredAcks = WaitForAll + parent.conf.Producer.Timeout = 10 * time.Second + + msg := &ProducerMessage{ + Topic: "t1", + Partition: 0, + Key: StringEncoder(TestMessage), + Value: StringEncoder(TestMessage), + } + for i := 0; i < 10; i++ { + safeAddMessage(t, ps, msg) + } + msg.Partition = 1 + for i := 0; i < 10; i++ { + safeAddMessage(t, ps, msg) + } + msg.Topic = "t2" + for i := 0; i < 10; i++ { + safeAddMessage(t, ps, msg) + } + + req := ps.buildRequest() + + if req.RequiredAcks != WaitForAll { + t.Error("RequiredAcks not set properly") + } + + if req.Timeout != 10000 { + t.Error("Timeout not set properly") + } + + if len(req.msgSets) != 2 { + t.Error("Wrong number of topics in request") + } +} diff --git a/vendor/github.com/Shopify/sarama/real_decoder.go b/vendor/github.com/Shopify/sarama/real_decoder.go new file mode 100644 index 000000000000..e3ea33104839 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/real_decoder.go @@ -0,0 +1,254 @@ +package sarama + +import ( + "encoding/binary" + "math" +) + +type realDecoder struct { + raw []byte + off int + stack []pushDecoder +} + +// primitives + +func (rd *realDecoder) getInt8() (int8, error) { + if rd.remaining() < 1 { + rd.off = len(rd.raw) + return -1, ErrInsufficientData + } + tmp := int8(rd.raw[rd.off]) + rd.off++ + return tmp, nil +} + +func (rd *realDecoder) getInt16() (int16, error) { + if rd.remaining() < 2 { + rd.off = len(rd.raw) + return -1, ErrInsufficientData + } + tmp := int16(binary.BigEndian.Uint16(rd.raw[rd.off:])) + rd.off += 2 + return tmp, nil +} + +func (rd *realDecoder) getInt32() (int32, error) { + if rd.remaining() < 4 { + rd.off = len(rd.raw) + return -1, ErrInsufficientData + } + tmp := int32(binary.BigEndian.Uint32(rd.raw[rd.off:])) + rd.off += 4 + return tmp, nil +} + +func (rd *realDecoder) getInt64() (int64, error) { + if rd.remaining() < 8 { + rd.off = len(rd.raw) + return -1, ErrInsufficientData + } + tmp := int64(binary.BigEndian.Uint64(rd.raw[rd.off:])) + rd.off += 8 + return tmp, nil +} + +func (rd *realDecoder) getArrayLength() (int, error) { + if rd.remaining() < 4 { + rd.off = len(rd.raw) + return -1, ErrInsufficientData + } + tmp := int(binary.BigEndian.Uint32(rd.raw[rd.off:])) + rd.off += 4 + if tmp > rd.remaining() { + rd.off = len(rd.raw) + return -1, ErrInsufficientData + } else if tmp > 2*math.MaxUint16 { + return -1, PacketDecodingError{"invalid array length"} + } + return tmp, nil +} + +// collections + +func (rd *realDecoder) getBytes() ([]byte, error) { + tmp, err := rd.getInt32() + + if err != nil { + return nil, err + } + + n := int(tmp) + + switch { + case n < -1: + return nil, PacketDecodingError{"invalid byteslice length"} + case n == -1: + return nil, nil + case n == 0: + return make([]byte, 0), nil + case n > rd.remaining(): + rd.off = len(rd.raw) + return nil, ErrInsufficientData + } + + tmpStr := rd.raw[rd.off : rd.off+n] + rd.off += n + return tmpStr, nil +} + +func (rd *realDecoder) getString() (string, error) { + tmp, err := rd.getInt16() + + if err != nil { + return "", err + } + + n := int(tmp) + + switch { + case n < -1: + return "", PacketDecodingError{"invalid string length"} + case n == -1: + return "", nil + case n == 0: + return "", nil + case n > rd.remaining(): + rd.off = len(rd.raw) + return "", ErrInsufficientData + } + + tmpStr := string(rd.raw[rd.off : rd.off+n]) + rd.off += n + return tmpStr, nil +} + +func (rd *realDecoder) getInt32Array() ([]int32, error) { + if rd.remaining() < 4 { + rd.off = len(rd.raw) + return nil, ErrInsufficientData + } + n := int(binary.BigEndian.Uint32(rd.raw[rd.off:])) + rd.off += 4 + + if rd.remaining() < 4*n { + rd.off = len(rd.raw) + return nil, ErrInsufficientData + } + + if n == 0 { + return nil, nil + } + + if n < 0 { + return nil, PacketDecodingError{"invalid array length"} + } + + ret := make([]int32, n) + for i := range ret { + ret[i] = int32(binary.BigEndian.Uint32(rd.raw[rd.off:])) + rd.off += 4 + } + return ret, nil +} + +func (rd *realDecoder) getInt64Array() ([]int64, error) { + if rd.remaining() < 4 { + rd.off = len(rd.raw) + return nil, ErrInsufficientData + } + n := int(binary.BigEndian.Uint32(rd.raw[rd.off:])) + rd.off += 4 + + if rd.remaining() < 8*n { + rd.off = len(rd.raw) + return nil, ErrInsufficientData + } + + if n == 0 { + return nil, nil + } + + if n < 0 { + return nil, PacketDecodingError{"invalid array length"} + } + + ret := make([]int64, n) + for i := range ret { + ret[i] = int64(binary.BigEndian.Uint64(rd.raw[rd.off:])) + rd.off += 8 + } + return ret, nil +} + +func (rd *realDecoder) getStringArray() ([]string, error) { + if rd.remaining() < 4 { + rd.off = len(rd.raw) + return nil, ErrInsufficientData + } + n := int(binary.BigEndian.Uint32(rd.raw[rd.off:])) + rd.off += 4 + + if n == 0 { + return nil, nil + } + + if n < 0 { + return nil, PacketDecodingError{"invalid array length"} + } + + ret := make([]string, n) + for i := range ret { + if str, err := rd.getString(); err != nil { + return nil, err + } else { + ret[i] = str + } + } + return ret, nil +} + +// subsets + +func (rd *realDecoder) remaining() int { + return len(rd.raw) - rd.off +} + +func (rd *realDecoder) getSubset(length int) (packetDecoder, error) { + if length < 0 { + return nil, PacketDecodingError{"invalid subset size"} + } else if length > rd.remaining() { + rd.off = len(rd.raw) + return nil, ErrInsufficientData + } + + start := rd.off + rd.off += length + return &realDecoder{raw: rd.raw[start:rd.off]}, nil +} + +// stacks + +func (rd *realDecoder) push(in pushDecoder) error { + in.saveOffset(rd.off) + + reserve := in.reserveLength() + if rd.remaining() < reserve { + rd.off = len(rd.raw) + return ErrInsufficientData + } + + rd.stack = append(rd.stack, in) + + rd.off += reserve + + return nil +} + +func (rd *realDecoder) pop() error { + // this is go's ugly pop pattern (the inverse of append) + in := rd.stack[len(rd.stack)-1] + rd.stack = rd.stack[:len(rd.stack)-1] + + return in.check(rd.off, rd.raw) +} diff --git a/vendor/github.com/Shopify/sarama/real_encoder.go b/vendor/github.com/Shopify/sarama/real_encoder.go new file mode 100644 index 000000000000..076fdd0ca1db --- /dev/null +++ b/vendor/github.com/Shopify/sarama/real_encoder.go @@ -0,0 +1,115 @@ +package sarama + +import "encoding/binary" + +type realEncoder struct { + raw []byte + off int + stack []pushEncoder +} + +// primitives + +func (re *realEncoder) putInt8(in int8) { + re.raw[re.off] = byte(in) + re.off++ +} + +func (re *realEncoder) putInt16(in int16) { + binary.BigEndian.PutUint16(re.raw[re.off:], uint16(in)) + re.off += 2 +} + +func (re *realEncoder) putInt32(in int32) { + binary.BigEndian.PutUint32(re.raw[re.off:], uint32(in)) + re.off += 4 +} + +func (re *realEncoder) putInt64(in int64) { + binary.BigEndian.PutUint64(re.raw[re.off:], uint64(in)) + re.off += 8 +} + +func (re *realEncoder) putArrayLength(in int) error { + re.putInt32(int32(in)) + return nil +} + +// collection + +func (re *realEncoder) putRawBytes(in []byte) error { + copy(re.raw[re.off:], in) + re.off += len(in) + return nil +} + +func (re *realEncoder) putBytes(in []byte) error { + if in == nil { + re.putInt32(-1) + return nil + } + re.putInt32(int32(len(in))) + copy(re.raw[re.off:], in) + re.off += len(in) + return nil +} + +func (re *realEncoder) putString(in string) error { + re.putInt16(int16(len(in))) + copy(re.raw[re.off:], in) + re.off += len(in) + return nil +} + +func (re *realEncoder) putStringArray(in []string) error { + err := re.putArrayLength(len(in)) + if err != nil { + return err + } + + for _, val := range in { + if err := re.putString(val); err != nil { + return err + } + } + + return nil +} + +func (re *realEncoder) putInt32Array(in []int32) error { + err := re.putArrayLength(len(in)) + if err != nil { + return err + } + for _, val := range in { + re.putInt32(val) + } + return nil +} + +func (re *realEncoder) putInt64Array(in []int64) error { + err := re.putArrayLength(len(in)) + if err != nil { + return err + } + for _, val := range in { + re.putInt64(val) + } + return nil +} + +// stacks + +func (re *realEncoder) push(in pushEncoder) { + in.saveOffset(re.off) + re.off += in.reserveLength() + re.stack = append(re.stack, in) +} + +func (re *realEncoder) pop() error { + // this is go's ugly pop pattern (the inverse of append) + in := re.stack[len(re.stack)-1] + re.stack = re.stack[:len(re.stack)-1] + + return in.run(re.off, re.raw) +} diff --git a/vendor/github.com/Shopify/sarama/request.go b/vendor/github.com/Shopify/sarama/request.go new file mode 100644 index 000000000000..b9f654ba21e7 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/request.go @@ -0,0 +1,112 @@ +package sarama + +import ( + "encoding/binary" + "fmt" + "io" +) + +type requestBody interface { + encoder + decoder + key() int16 + version() int16 +} + +type request struct { + correlationID int32 + clientID string + body requestBody +} + +func (r *request) encode(pe packetEncoder) (err error) { + pe.push(&lengthField{}) + pe.putInt16(r.body.key()) + pe.putInt16(r.body.version()) + pe.putInt32(r.correlationID) + err = pe.putString(r.clientID) + if err != nil { + return err + } + err = r.body.encode(pe) + if err != nil { + return err + } + return pe.pop() +} + +func (r *request) decode(pd packetDecoder) (err error) { + var key int16 + if key, err = pd.getInt16(); err != nil { + return err + } + var version int16 + if version, err = pd.getInt16(); err != nil { + return err + } + if r.correlationID, err = pd.getInt32(); err != nil { + return err + } + r.clientID, err = pd.getString() + + r.body = allocateBody(key, version) + if r.body == nil { + return PacketDecodingError{fmt.Sprintf("unknown request key (%d)", key)} + } + return r.body.decode(pd) +} + +func decodeRequest(r io.Reader) (req *request, err error) { + lengthBytes := make([]byte, 4) + if _, err := io.ReadFull(r, lengthBytes); err != nil { + return nil, err + } + + length := int32(binary.BigEndian.Uint32(lengthBytes)) + if length <= 4 || length > MaxRequestSize { + return nil, PacketDecodingError{fmt.Sprintf("message of length %d too large or too small", length)} + } + + encodedReq := make([]byte, length) + if _, err := io.ReadFull(r, encodedReq); err != nil { + return nil, err + } + + req = &request{} + if err := decode(encodedReq, req); err != nil { + return nil, err + } + return req, nil +} + +func allocateBody(key, version int16) requestBody { + switch key { + case 0: + return &ProduceRequest{} + case 1: + return &FetchRequest{} + case 2: + return &OffsetRequest{} + case 3: + return &MetadataRequest{} + case 8: + return &OffsetCommitRequest{Version: version} + case 9: + return &OffsetFetchRequest{} + case 10: + return &ConsumerMetadataRequest{} + case 11: + return &JoinGroupRequest{} + case 12: + return &HeartbeatRequest{} + case 13: + return &LeaveGroupRequest{} + case 14: + return &SyncGroupRequest{} + case 15: + return &DescribeGroupsRequest{} + case 16: + return &ListGroupsRequest{} + } + return nil +} diff --git a/vendor/github.com/Shopify/sarama/request_test.go b/vendor/github.com/Shopify/sarama/request_test.go new file mode 100644 index 000000000000..1e47292c1059 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/request_test.go @@ -0,0 +1,80 @@ +package sarama + +import ( + "bytes" + "reflect" + "testing" +) + +type testRequestBody struct { +} + +func (s *testRequestBody) key() int16 { + return 0x666 +} + +func (s *testRequestBody) version() int16 { + return 0xD2 +} + +func (s *testRequestBody) encode(pe packetEncoder) error { + return pe.putString("abc") +} + +// not specific to request tests, just helper functions for testing structures that +// implement the encoder or decoder interfaces that needed somewhere to live + +func testEncodable(t *testing.T, name string, in encoder, expect []byte) { + packet, err := encode(in) + if err != nil { + t.Error(err) + } else if !bytes.Equal(packet, expect) { + t.Error("Encoding", name, "failed\ngot ", packet, "\nwant", expect) + } +} + +func testDecodable(t *testing.T, name string, out decoder, in []byte) { + err := decode(in, out) + if err != nil { + t.Error("Decoding", name, "failed:", err) + } +} + +func testRequest(t *testing.T, name string, rb requestBody, expected []byte) { + // Encoder request + req := &request{correlationID: 123, clientID: "foo", body: rb} + packet, err := encode(req) + headerSize := 14 + len("foo") + if err != nil { + t.Error(err) + } else if !bytes.Equal(packet[headerSize:], expected) { + t.Error("Encoding", name, "failed\ngot ", packet[headerSize:], "\nwant", expected) + } + // Decoder request + decoded, err := decodeRequest(bytes.NewReader(packet)) + if err != nil { + t.Error("Failed to decode request", err) + } else if decoded.correlationID != 123 || decoded.clientID != "foo" { + t.Errorf("Decoded header is not valid: %v", decoded) + } else if !reflect.DeepEqual(rb, decoded.body) { + t.Errorf("Decoded request does not match the encoded one\nencoded: %v\ndecoded: %v", rb, decoded.body) + } +} + +func testResponse(t *testing.T, name string, res encoder, expected []byte) { + encoded, err := encode(res) + if err != nil { + t.Error(err) + } else if expected != nil && !bytes.Equal(encoded, expected) { + t.Error("Encoding", name, "failed\ngot ", encoded, "\nwant", expected) + } + + decoded := reflect.New(reflect.TypeOf(res).Elem()).Interface().(decoder) + if err := decode(encoded, decoded); err != nil { + t.Error("Decoding", name, "failed:", err) + } + + if !reflect.DeepEqual(decoded, res) { + t.Errorf("Decoded response does not match the encoded one\nencoded: %#v\ndecoded: %#v", res, decoded) + } +} diff --git a/vendor/github.com/Shopify/sarama/response_header.go b/vendor/github.com/Shopify/sarama/response_header.go new file mode 100644 index 000000000000..f3f4d27d6c42 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/response_header.go @@ -0,0 +1,21 @@ +package sarama + +import "fmt" + +type responseHeader struct { + length int32 + correlationID int32 +} + +func (r *responseHeader) decode(pd packetDecoder) (err error) { + r.length, err = pd.getInt32() + if err != nil { + return err + } + if r.length <= 4 || r.length > MaxResponseSize { + return PacketDecodingError{fmt.Sprintf("message of length %d too large or too small", r.length)} + } + + r.correlationID, err = pd.getInt32() + return err +} diff --git a/vendor/github.com/Shopify/sarama/response_header_test.go b/vendor/github.com/Shopify/sarama/response_header_test.go new file mode 100644 index 000000000000..8f9fdb80c5b9 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/response_header_test.go @@ -0,0 +1,21 @@ +package sarama + +import "testing" + +var ( + responseHeaderBytes = []byte{ + 0x00, 0x00, 0x0f, 0x00, + 0x0a, 0xbb, 0xcc, 0xff} +) + +func TestResponseHeader(t *testing.T) { + header := responseHeader{} + + testDecodable(t, "response header", &header, responseHeaderBytes) + if header.length != 0xf00 { + t.Error("Decoding header length failed, got", header.length) + } + if header.correlationID != 0x0abbccff { + t.Error("Decoding header correlation id failed, got", header.correlationID) + } +} diff --git a/vendor/github.com/Shopify/sarama/sarama.go b/vendor/github.com/Shopify/sarama/sarama.go new file mode 100644 index 000000000000..d5982175030f --- /dev/null +++ b/vendor/github.com/Shopify/sarama/sarama.go @@ -0,0 +1,47 @@ +/* +Package sarama provides client libraries for the Kafka 0.8 protocol. The AsyncProducer object is the high-level +API for producing messages asynchronously; the SyncProducer provides a blocking API for the same purpose. +The Consumer object is the high-level API for consuming messages. The Client object provides metadata +management functionality that is shared between the higher-level objects. + +For lower-level needs, the Broker and Request/Response objects permit precise control over each connection +and message sent on the wire. + +The Request/Response objects and properties are mostly undocumented, as they line up exactly with the +protocol fields documented by Kafka at https://cwiki.apache.org/confluence/display/KAFKA/A+Guide+To+The+Kafka+Protocol +*/ +package sarama + +import ( + "io/ioutil" + "log" +) + +// Logger is the instance of a StdLogger interface that Sarama writes connection +// management events to. By default it is set to discard all log messages via ioutil.Discard, +// but you can set it to redirect wherever you want. +var Logger StdLogger = log.New(ioutil.Discard, "[Sarama] ", log.LstdFlags) + +// StdLogger is used to log error messages. +type StdLogger interface { + Print(v ...interface{}) + Printf(format string, v ...interface{}) + Println(v ...interface{}) +} + +// PanicHandler is called for recovering from panics spawned internally to the library (and thus +// not recoverable by the caller's goroutine). Defaults to nil, which means panics are not recovered. +var PanicHandler func(interface{}) + +// MaxRequestSize is the maximum size (in bytes) of any request that Sarama will attempt to send. Trying +// to send a request larger than this will result in an PacketEncodingError. The default of 100 MiB is aligned +// with Kafka's default `socket.request.max.bytes`, which is the largest request the broker will attempt +// to process. +var MaxRequestSize int32 = 100 * 1024 * 1024 + +// MaxResponseSize is the maximum size (in bytes) of any response that Sarama will attempt to parse. If +// a broker returns a response message larger than this value, Sarama will return a PacketDecodingError to +// protect the client from running out of memory. Please note that brokers do not have any natural limit on +// the size of responses they send. In particular, they can send arbitrarily large fetch responses to consumers +// (see https://issues.apache.org/jira/browse/KAFKA-2063). +var MaxResponseSize int32 = 100 * 1024 * 1024 diff --git a/vendor/github.com/Shopify/sarama/snappy.go b/vendor/github.com/Shopify/sarama/snappy.go new file mode 100644 index 000000000000..e86cb7039715 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/snappy.go @@ -0,0 +1,41 @@ +package sarama + +import ( + "bytes" + "encoding/binary" + + "github.com/golang/snappy" +) + +var snappyMagic = []byte{130, 83, 78, 65, 80, 80, 89, 0} + +// SnappyEncode encodes binary data +func snappyEncode(src []byte) []byte { + return snappy.Encode(nil, src) +} + +// SnappyDecode decodes snappy data +func snappyDecode(src []byte) ([]byte, error) { + if bytes.Equal(src[:8], snappyMagic) { + var ( + pos = uint32(16) + max = uint32(len(src)) + dst = make([]byte, 0, len(src)) + chunk []byte + err error + ) + for pos < max { + size := binary.BigEndian.Uint32(src[pos : pos+4]) + pos += 4 + + chunk, err = snappy.Decode(chunk, src[pos:pos+size]) + if err != nil { + return nil, err + } + pos += size + dst = append(dst, chunk...) + } + return dst, nil + } + return snappy.Decode(nil, src) +} diff --git a/vendor/github.com/Shopify/sarama/snappy_test.go b/vendor/github.com/Shopify/sarama/snappy_test.go new file mode 100644 index 000000000000..71e67b892932 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/snappy_test.go @@ -0,0 +1,49 @@ +package sarama + +import ( + "bytes" + "testing" +) + +var snappyTestCases = map[string][]byte{ + "REPEATREPEATREPEATREPEATREPEATREPEAT": {36, 20, 82, 69, 80, 69, 65, 84, 118, 6, 0}, + "REALLY SHORT": {12, 44, 82, 69, 65, 76, 76, 89, 32, 83, 72, 79, 82, 84}, + "AXBXCXDXEXFX": {12, 44, 65, 88, 66, 88, 67, 88, 68, 88, 69, 88, 70, 88}, +} + +var snappyStreamTestCases = map[string][]byte{ + "PLAINDATA": {130, 83, 78, 65, 80, 80, 89, 0, 0, 0, 0, 1, 0, 0, 0, 1, 0, 0, 0, 11, 9, 32, 80, 76, 65, 73, 78, 68, 65, 84, 65}, + `{"a":"UtaitILHMDAAAAfU","b":"日本"}`: {130, 83, 78, 65, 80, 80, 89, 0, 0, 0, 0, 1, 0, 0, 0, 1, 0, 0, 0, 39, 37, 144, 123, 34, 97, 34, 58, 34, 85, 116, 97, 105, 116, 73, 76, 72, 77, 68, 65, 65, 65, 65, 102, 85, 34, 44, 34, 98, 34, 58, 34, 230, 151, 165, 230, 156, 172, 34, 125}, + `Sed ut perspiciatis unde omnis iste natus error sit voluptatem accusantium doloremque laudantium, totam rem aperiam, eaque ipsa quae ab illo inventore veritatis et quasi architecto beatae vitae dicta sunt explicabo. Nemo enim ipsam voluptatem quia voluptas sit aspernatur aut odit aut fugit, sed quia consequuntur magni dolores eos qui ratione voluptatem sequi nesciunt. Neque porro quisquam est, qui dolorem ipsum quia dolor sit amet, consectetur, adipisci velit, sed quia non numquam eius modi tempora incidunt ut labore et dolore magnam aliquam quaerat voluptatem. Ut enim ad minima veniam, quis nostrum exercitationem ullam corporis suscipit laboriosam, nisi ut aliquid ex ea commodi consequatur? Quis autem vel eum iure reprehenderit qui in ea voluptate velit esse quam nihil molestiae consequatur, vel illum qui dolorem eum fugiat quo voluptas nulla pariatur? At vero eos et accusamus et iusto odio dignissimos ducimus qui blanditiis praesentium voluptatum deleniti atque corrupti quos dolores et quas molestias except`: {130, 83, 78, 65, 80, 80, 89, 0, 0, 0, 0, 1, 0, 0, 0, 1, 0, 0, 3, 89, 128, 8, 240, 90, 83, 101, 100, 32, 117, 116, 32, 112, 101, 114, 115, 112, 105, 99, 105, 97, 116, 105, 115, 32, 117, 110, 100, 101, 32, 111, 109, 110, 105, 115, 32, 105, 115, 116, 101, 32, 110, 97, 116, 117, 115, 32, 101, 114, 114, 111, 114, 32, 115, 105, 116, 32, 118, 111, 108, 117, 112, 116, 97, 116, 101, 109, 32, 97, 99, 99, 117, 115, 97, 110, 116, 105, 117, 109, 32, 100, 111, 108, 111, 114, 101, 109, 113, 117, 101, 32, 108, 97, 117, 100, 97, 5, 22, 240, 60, 44, 32, 116, 111, 116, 97, 109, 32, 114, 101, 109, 32, 97, 112, 101, 114, 105, 97, 109, 44, 32, 101, 97, 113, 117, 101, 32, 105, 112, 115, 97, 32, 113, 117, 97, 101, 32, 97, 98, 32, 105, 108, 108, 111, 32, 105, 110, 118, 101, 110, 116, 111, 114, 101, 32, 118, 101, 114, 105, 116, 97, 1, 141, 4, 101, 116, 1, 36, 88, 115, 105, 32, 97, 114, 99, 104, 105, 116, 101, 99, 116, 111, 32, 98, 101, 97, 116, 97, 101, 32, 118, 105, 1, 6, 120, 100, 105, 99, 116, 97, 32, 115, 117, 110, 116, 32, 101, 120, 112, 108, 105, 99, 97, 98, 111, 46, 32, 78, 101, 109, 111, 32, 101, 110, 105, 109, 5, 103, 0, 109, 46, 180, 0, 12, 113, 117, 105, 97, 17, 16, 0, 115, 5, 209, 72, 97, 115, 112, 101, 114, 110, 97, 116, 117, 114, 32, 97, 117, 116, 32, 111, 100, 105, 116, 5, 9, 36, 102, 117, 103, 105, 116, 44, 32, 115, 101, 100, 9, 53, 32, 99, 111, 110, 115, 101, 113, 117, 117, 110, 1, 42, 20, 109, 97, 103, 110, 105, 32, 9, 245, 16, 115, 32, 101, 111, 115, 1, 36, 28, 32, 114, 97, 116, 105, 111, 110, 101, 17, 96, 33, 36, 1, 51, 36, 105, 32, 110, 101, 115, 99, 105, 117, 110, 116, 1, 155, 1, 254, 16, 112, 111, 114, 114, 111, 1, 51, 36, 115, 113, 117, 97, 109, 32, 101, 115, 116, 44, 1, 14, 13, 81, 5, 183, 4, 117, 109, 1, 18, 0, 97, 9, 19, 4, 32, 115, 1, 149, 12, 109, 101, 116, 44, 9, 135, 76, 99, 116, 101, 116, 117, 114, 44, 32, 97, 100, 105, 112, 105, 115, 99, 105, 32, 118, 101, 108, 50, 173, 0, 24, 110, 111, 110, 32, 110, 117, 109, 9, 94, 84, 105, 117, 115, 32, 109, 111, 100, 105, 32, 116, 101, 109, 112, 111, 114, 97, 32, 105, 110, 99, 105, 100, 33, 52, 20, 117, 116, 32, 108, 97, 98, 33, 116, 4, 101, 116, 9, 106, 0, 101, 5, 219, 20, 97, 109, 32, 97, 108, 105, 5, 62, 33, 164, 8, 114, 97, 116, 29, 212, 12, 46, 32, 85, 116, 41, 94, 52, 97, 100, 32, 109, 105, 110, 105, 109, 97, 32, 118, 101, 110, 105, 33, 221, 72, 113, 117, 105, 115, 32, 110, 111, 115, 116, 114, 117, 109, 32, 101, 120, 101, 114, 99, 105, 33, 202, 104, 111, 110, 101, 109, 32, 117, 108, 108, 97, 109, 32, 99, 111, 114, 112, 111, 114, 105, 115, 32, 115, 117, 115, 99, 105, 112, 105, 13, 130, 8, 105, 111, 115, 1, 64, 12, 110, 105, 115, 105, 1, 150, 5, 126, 44, 105, 100, 32, 101, 120, 32, 101, 97, 32, 99, 111, 109, 5, 192, 0, 99, 41, 131, 33, 172, 8, 63, 32, 81, 1, 107, 4, 97, 117, 33, 101, 96, 118, 101, 108, 32, 101, 117, 109, 32, 105, 117, 114, 101, 32, 114, 101, 112, 114, 101, 104, 101, 110, 100, 101, 114, 105, 65, 63, 12, 105, 32, 105, 110, 1, 69, 16, 118, 111, 108, 117, 112, 65, 185, 1, 47, 24, 105, 116, 32, 101, 115, 115, 101, 1, 222, 64, 109, 32, 110, 105, 104, 105, 108, 32, 109, 111, 108, 101, 115, 116, 105, 97, 101, 46, 103, 0, 0, 44, 1, 45, 16, 32, 105, 108, 108, 117, 37, 143, 45, 36, 0, 109, 5, 110, 65, 33, 20, 97, 116, 32, 113, 117, 111, 17, 92, 44, 115, 32, 110, 117, 108, 108, 97, 32, 112, 97, 114, 105, 9, 165, 24, 65, 116, 32, 118, 101, 114, 111, 69, 34, 44, 101, 116, 32, 97, 99, 99, 117, 115, 97, 109, 117, 115, 1, 13, 104, 105, 117, 115, 116, 111, 32, 111, 100, 105, 111, 32, 100, 105, 103, 110, 105, 115, 115, 105, 109, 111, 115, 32, 100, 117, 99, 105, 1, 34, 80, 113, 117, 105, 32, 98, 108, 97, 110, 100, 105, 116, 105, 105, 115, 32, 112, 114, 97, 101, 115, 101, 101, 87, 17, 111, 56, 116, 117, 109, 32, 100, 101, 108, 101, 110, 105, 116, 105, 32, 97, 116, 65, 89, 28, 99, 111, 114, 114, 117, 112, 116, 105, 1, 150, 0, 115, 13, 174, 5, 109, 8, 113, 117, 97, 65, 5, 52, 108, 101, 115, 116, 105, 97, 115, 32, 101, 120, 99, 101, 112, 116, 0, 0, 0, 1, 0}, +} + +func TestSnappyEncode(t *testing.T) { + for src, exp := range snappyTestCases { + dst := snappyEncode([]byte(src)) + if !bytes.Equal(dst, exp) { + t.Errorf("Expected %s to generate %v, but was %v", src, exp, dst) + } + } +} + +func TestSnappyDecode(t *testing.T) { + for exp, src := range snappyTestCases { + dst, err := snappyDecode(src) + if err != nil { + t.Error("Encoding error: ", err) + } else if !bytes.Equal(dst, []byte(exp)) { + t.Errorf("Expected %s to be generated from %v, but was %s", exp, src, string(dst)) + } + } +} + +func TestSnappyDecodeStreams(t *testing.T) { + for exp, src := range snappyStreamTestCases { + dst, err := snappyDecode(src) + if err != nil { + t.Error("Encoding error: ", err) + } else if !bytes.Equal(dst, []byte(exp)) { + t.Errorf("Expected %s to be generated from [%d]byte, but was %s", exp, len(src), string(dst)) + } + } +} diff --git a/vendor/github.com/Shopify/sarama/sync_group_request.go b/vendor/github.com/Shopify/sarama/sync_group_request.go new file mode 100644 index 000000000000..031cf0f2f67a --- /dev/null +++ b/vendor/github.com/Shopify/sarama/sync_group_request.go @@ -0,0 +1,96 @@ +package sarama + +type SyncGroupRequest struct { + GroupId string + GenerationId int32 + MemberId string + GroupAssignments map[string][]byte +} + +func (r *SyncGroupRequest) encode(pe packetEncoder) error { + if err := pe.putString(r.GroupId); err != nil { + return err + } + + pe.putInt32(r.GenerationId) + + if err := pe.putString(r.MemberId); err != nil { + return err + } + + if err := pe.putArrayLength(len(r.GroupAssignments)); err != nil { + return err + } + for memberId, memberAssignment := range r.GroupAssignments { + if err := pe.putString(memberId); err != nil { + return err + } + if err := pe.putBytes(memberAssignment); err != nil { + return err + } + } + + return nil +} + +func (r *SyncGroupRequest) decode(pd packetDecoder) (err error) { + if r.GroupId, err = pd.getString(); err != nil { + return + } + if r.GenerationId, err = pd.getInt32(); err != nil { + return + } + if r.MemberId, err = pd.getString(); err != nil { + return + } + + n, err := pd.getArrayLength() + if err != nil { + return err + } + if n == 0 { + return nil + } + + r.GroupAssignments = make(map[string][]byte) + for i := 0; i < n; i++ { + memberId, err := pd.getString() + if err != nil { + return err + } + memberAssignment, err := pd.getBytes() + if err != nil { + return err + } + + r.GroupAssignments[memberId] = memberAssignment + } + + return nil +} + +func (r *SyncGroupRequest) key() int16 { + return 14 +} + +func (r *SyncGroupRequest) version() int16 { + return 0 +} + +func (r *SyncGroupRequest) AddGroupAssignment(memberId string, memberAssignment []byte) { + if r.GroupAssignments == nil { + r.GroupAssignments = make(map[string][]byte) + } + + r.GroupAssignments[memberId] = memberAssignment +} + +func (r *SyncGroupRequest) AddGroupAssignmentMember(memberId string, memberAssignment *ConsumerGroupMemberAssignment) error { + bin, err := encode(memberAssignment) + if err != nil { + return err + } + + r.AddGroupAssignment(memberId, bin) + return nil +} diff --git a/vendor/github.com/Shopify/sarama/sync_group_request_test.go b/vendor/github.com/Shopify/sarama/sync_group_request_test.go new file mode 100644 index 000000000000..3f537ef9fbf4 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/sync_group_request_test.go @@ -0,0 +1,38 @@ +package sarama + +import "testing" + +var ( + emptySyncGroupRequest = []byte{ + 0, 3, 'f', 'o', 'o', // Group ID + 0x00, 0x01, 0x02, 0x03, // Generation ID + 0, 3, 'b', 'a', 'z', // Member ID + 0, 0, 0, 0, // no assignments + } + + populatedSyncGroupRequest = []byte{ + 0, 3, 'f', 'o', 'o', // Group ID + 0x00, 0x01, 0x02, 0x03, // Generation ID + 0, 3, 'b', 'a', 'z', // Member ID + 0, 0, 0, 1, // one assignment + 0, 3, 'b', 'a', 'z', // Member ID + 0, 0, 0, 3, 'f', 'o', 'o', // Member assignment + } +) + +func TestSyncGroupRequest(t *testing.T) { + var request *SyncGroupRequest + + request = new(SyncGroupRequest) + request.GroupId = "foo" + request.GenerationId = 66051 + request.MemberId = "baz" + testRequest(t, "empty", request, emptySyncGroupRequest) + + request = new(SyncGroupRequest) + request.GroupId = "foo" + request.GenerationId = 66051 + request.MemberId = "baz" + request.AddGroupAssignment("baz", []byte("foo")) + testRequest(t, "populated", request, populatedSyncGroupRequest) +} diff --git a/vendor/github.com/Shopify/sarama/sync_group_response.go b/vendor/github.com/Shopify/sarama/sync_group_response.go new file mode 100644 index 000000000000..49c86922dc59 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/sync_group_response.go @@ -0,0 +1,28 @@ +package sarama + +type SyncGroupResponse struct { + Err KError + MemberAssignment []byte +} + +func (r *SyncGroupResponse) GetMemberAssignment() (*ConsumerGroupMemberAssignment, error) { + assignment := new(ConsumerGroupMemberAssignment) + err := decode(r.MemberAssignment, assignment) + return assignment, err +} + +func (r *SyncGroupResponse) encode(pe packetEncoder) error { + pe.putInt16(int16(r.Err)) + return pe.putBytes(r.MemberAssignment) +} + +func (r *SyncGroupResponse) decode(pd packetDecoder) (err error) { + if kerr, err := pd.getInt16(); err != nil { + return err + } else { + r.Err = KError(kerr) + } + + r.MemberAssignment, err = pd.getBytes() + return +} diff --git a/vendor/github.com/Shopify/sarama/sync_group_response_test.go b/vendor/github.com/Shopify/sarama/sync_group_response_test.go new file mode 100644 index 000000000000..164908f37f97 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/sync_group_response_test.go @@ -0,0 +1,40 @@ +package sarama + +import ( + "reflect" + "testing" +) + +var ( + syncGroupResponseNoError = []byte{ + 0x00, 0x00, // No error + 0, 0, 0, 3, 0x01, 0x02, 0x03, // Member assignment data + } + + syncGroupResponseWithError = []byte{ + 0, 27, // ErrRebalanceInProgress + 0, 0, 0, 0, // No member assignment data + } +) + +func TestSyncGroupResponse(t *testing.T) { + var response *SyncGroupResponse + + response = new(SyncGroupResponse) + testDecodable(t, "no error", response, syncGroupResponseNoError) + if response.Err != ErrNoError { + t.Error("Decoding Err failed: no error expected but found", response.Err) + } + if !reflect.DeepEqual(response.MemberAssignment, []byte{0x01, 0x02, 0x03}) { + t.Error("Decoding MemberAssignment failed, found:", response.MemberAssignment) + } + + response = new(SyncGroupResponse) + testDecodable(t, "no error", response, syncGroupResponseWithError) + if response.Err != ErrRebalanceInProgress { + t.Error("Decoding Err failed: ErrRebalanceInProgress expected but found", response.Err) + } + if !reflect.DeepEqual(response.MemberAssignment, []byte{}) { + t.Error("Decoding MemberAssignment failed, found:", response.MemberAssignment) + } +} diff --git a/vendor/github.com/Shopify/sarama/sync_producer.go b/vendor/github.com/Shopify/sarama/sync_producer.go new file mode 100644 index 000000000000..69a26d1337be --- /dev/null +++ b/vendor/github.com/Shopify/sarama/sync_producer.go @@ -0,0 +1,95 @@ +package sarama + +import "sync" + +// SyncProducer publishes Kafka messages. It routes messages to the correct broker, refreshing metadata as appropriate, +// and parses responses for errors. You must call Close() on a producer to avoid leaks, it may not be garbage-collected automatically when +// it passes out of scope. +type SyncProducer interface { + + // SendMessage produces a given message, and returns only when it either has + // succeeded or failed to produce. It will return the partition and the offset + // of the produced message, or an error if the message failed to produce. + SendMessage(msg *ProducerMessage) (partition int32, offset int64, err error) + + // Close shuts down the producer and flushes any messages it may have buffered. + // You must call this function before a producer object passes out of scope, as + // it may otherwise leak memory. You must call this before calling Close on the + // underlying client. + Close() error +} + +type syncProducer struct { + producer *asyncProducer + wg sync.WaitGroup +} + +// NewSyncProducer creates a new SyncProducer using the given broker addresses and configuration. +func NewSyncProducer(addrs []string, config *Config) (SyncProducer, error) { + p, err := NewAsyncProducer(addrs, config) + if err != nil { + return nil, err + } + return newSyncProducerFromAsyncProducer(p.(*asyncProducer)), nil +} + +// NewSyncProducerFromClient creates a new SyncProducer using the given client. It is still +// necessary to call Close() on the underlying client when shutting down this producer. +func NewSyncProducerFromClient(client Client) (SyncProducer, error) { + p, err := NewAsyncProducerFromClient(client) + if err != nil { + return nil, err + } + return newSyncProducerFromAsyncProducer(p.(*asyncProducer)), nil +} + +func newSyncProducerFromAsyncProducer(p *asyncProducer) *syncProducer { + p.conf.Producer.Return.Successes = true + p.conf.Producer.Return.Errors = true + sp := &syncProducer{producer: p} + + sp.wg.Add(2) + go withRecover(sp.handleSuccesses) + go withRecover(sp.handleErrors) + + return sp +} + +func (sp *syncProducer) SendMessage(msg *ProducerMessage) (partition int32, offset int64, err error) { + oldMetadata := msg.Metadata + defer func() { + msg.Metadata = oldMetadata + }() + + expectation := make(chan error, 1) + msg.Metadata = expectation + sp.producer.Input() <- msg + + if err := <-expectation; err != nil { + return -1, -1, err + } + + return msg.Partition, msg.Offset, nil +} + +func (sp *syncProducer) handleSuccesses() { + defer sp.wg.Done() + for msg := range sp.producer.Successes() { + expectation := msg.Metadata.(chan error) + expectation <- nil + } +} + +func (sp *syncProducer) handleErrors() { + defer sp.wg.Done() + for err := range sp.producer.Errors() { + expectation := err.Msg.Metadata.(chan error) + expectation <- err.Err + } +} + +func (sp *syncProducer) Close() error { + sp.producer.AsyncClose() + sp.wg.Wait() + return nil +} diff --git a/vendor/github.com/Shopify/sarama/sync_producer_test.go b/vendor/github.com/Shopify/sarama/sync_producer_test.go new file mode 100644 index 000000000000..d378949b18d3 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/sync_producer_test.go @@ -0,0 +1,149 @@ +package sarama + +import ( + "log" + "sync" + "testing" +) + +func TestSyncProducer(t *testing.T) { + seedBroker := newMockBroker(t, 1) + leader := newMockBroker(t, 2) + + metadataResponse := new(MetadataResponse) + metadataResponse.AddBroker(leader.Addr(), leader.BrokerID()) + metadataResponse.AddTopicPartition("my_topic", 0, leader.BrokerID(), nil, nil, ErrNoError) + seedBroker.Returns(metadataResponse) + + prodSuccess := new(ProduceResponse) + prodSuccess.AddTopicPartition("my_topic", 0, ErrNoError) + for i := 0; i < 10; i++ { + leader.Returns(prodSuccess) + } + + producer, err := NewSyncProducer([]string{seedBroker.Addr()}, nil) + if err != nil { + t.Fatal(err) + } + + for i := 0; i < 10; i++ { + msg := &ProducerMessage{ + Topic: "my_topic", + Value: StringEncoder(TestMessage), + Metadata: "test", + } + + partition, offset, err := producer.SendMessage(msg) + + if partition != 0 || msg.Partition != partition { + t.Error("Unexpected partition") + } + if offset != 0 || msg.Offset != offset { + t.Error("Unexpected offset") + } + if str, ok := msg.Metadata.(string); !ok || str != "test" { + t.Error("Unexpected metadata") + } + if err != nil { + t.Error(err) + } + } + + safeClose(t, producer) + leader.Close() + seedBroker.Close() +} + +func TestConcurrentSyncProducer(t *testing.T) { + seedBroker := newMockBroker(t, 1) + leader := newMockBroker(t, 2) + + metadataResponse := new(MetadataResponse) + metadataResponse.AddBroker(leader.Addr(), leader.BrokerID()) + metadataResponse.AddTopicPartition("my_topic", 0, leader.BrokerID(), nil, nil, ErrNoError) + seedBroker.Returns(metadataResponse) + + prodSuccess := new(ProduceResponse) + prodSuccess.AddTopicPartition("my_topic", 0, ErrNoError) + leader.Returns(prodSuccess) + + config := NewConfig() + config.Producer.Flush.Messages = 100 + producer, err := NewSyncProducer([]string{seedBroker.Addr()}, config) + if err != nil { + t.Fatal(err) + } + + wg := sync.WaitGroup{} + + for i := 0; i < 100; i++ { + wg.Add(1) + go func() { + msg := &ProducerMessage{Topic: "my_topic", Value: StringEncoder(TestMessage)} + partition, _, err := producer.SendMessage(msg) + if partition != 0 { + t.Error("Unexpected partition") + } + if err != nil { + t.Error(err) + } + wg.Done() + }() + } + wg.Wait() + + safeClose(t, producer) + leader.Close() + seedBroker.Close() +} + +func TestSyncProducerToNonExistingTopic(t *testing.T) { + broker := newMockBroker(t, 1) + + metadataResponse := new(MetadataResponse) + metadataResponse.AddBroker(broker.Addr(), broker.BrokerID()) + metadataResponse.AddTopicPartition("my_topic", 0, broker.BrokerID(), nil, nil, ErrNoError) + broker.Returns(metadataResponse) + + config := NewConfig() + config.Metadata.Retry.Max = 0 + config.Producer.Retry.Max = 0 + + producer, err := NewSyncProducer([]string{broker.Addr()}, config) + if err != nil { + t.Fatal(err) + } + + metadataResponse = new(MetadataResponse) + metadataResponse.AddTopic("unknown", ErrUnknownTopicOrPartition) + broker.Returns(metadataResponse) + + _, _, err = producer.SendMessage(&ProducerMessage{Topic: "unknown"}) + if err != ErrUnknownTopicOrPartition { + t.Error("Uxpected ErrUnknownTopicOrPartition, found:", err) + } + + safeClose(t, producer) + broker.Close() +} + +// This example shows the basic usage pattern of the SyncProducer. +func ExampleSyncProducer() { + producer, err := NewSyncProducer([]string{"localhost:9092"}, nil) + if err != nil { + log.Fatalln(err) + } + defer func() { + if err := producer.Close(); err != nil { + log.Fatalln(err) + } + }() + + msg := &ProducerMessage{Topic: "my_topic", Value: StringEncoder("testing 123")} + partition, offset, err := producer.SendMessage(msg) + if err != nil { + log.Printf("FAILED to send message: %s\n", err) + } else { + log.Printf("> message sent to partition %d at offset %d\n", partition, offset) + } +} diff --git a/vendor/github.com/Shopify/sarama/tools/README.md b/vendor/github.com/Shopify/sarama/tools/README.md new file mode 100644 index 000000000000..3464c4ad8040 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/tools/README.md @@ -0,0 +1,10 @@ +# Sarama tools + +This folder contains applications that are useful for exploration of your Kafka cluster, or instrumentation. +Some of these tools mirror tools that ship with Kafka, but these tools won't require installing the JVM to function. + +- [kafka-console-producer](./kafka-console-producer): a command line tool to produce a single message to your Kafka custer. +- [kafka-console-partitionconsumer](./kafka-console-partitionconsumer): (deprecated) a command line tool to consume a single partition of a topic on your Kafka cluster. +- [kafka-console-consumer](./kafka-console-consumer): a command line tool to consume arbitrary partitions of a topic on your Kafka cluster. + +To install all tools, run `go get github.com/Shopify/sarama/tools/...` diff --git a/vendor/github.com/Shopify/sarama/tools/kafka-console-consumer/.gitignore b/vendor/github.com/Shopify/sarama/tools/kafka-console-consumer/.gitignore new file mode 100644 index 000000000000..67da9dfa9f29 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/tools/kafka-console-consumer/.gitignore @@ -0,0 +1,2 @@ +kafka-console-consumer +kafka-console-consumer.test diff --git a/vendor/github.com/Shopify/sarama/tools/kafka-console-consumer/README.md b/vendor/github.com/Shopify/sarama/tools/kafka-console-consumer/README.md new file mode 100644 index 000000000000..4e77f0b70508 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/tools/kafka-console-consumer/README.md @@ -0,0 +1,29 @@ +# kafka-console-consumer + +A simple command line tool to consume partitions of a topic and print the +messages on the standard output. + +### Installation + + go get github.com/Shopify/sarama/tools/kafka-console-consumer + +### Usage + + # Minimum invocation + kafka-console-consumer -topic=test -brokers=kafka1:9092 + + # It will pick up a KAFKA_PEERS environment variable + export KAFKA_PEERS=kafka1:9092,kafka2:9092,kafka3:9092 + kafka-console-consumer -topic=test + + # You can specify the offset you want to start at. It can be either + # `oldest`, `newest`. The default is `newest`. + kafka-console-consumer -topic=test -offset=oldest + kafka-console-consumer -topic=test -offset=newest + + # You can specify the partition(s) you want to consume as a comma-separated + # list. The default is `all`. + kafka-console-consumer -topic=test -partitions=1,2,3 + + # Display all command line options + kafka-console-consumer -help diff --git a/vendor/github.com/Shopify/sarama/tools/kafka-console-consumer/kafka-console-consumer.go b/vendor/github.com/Shopify/sarama/tools/kafka-console-consumer/kafka-console-consumer.go new file mode 100644 index 000000000000..0f1eb89a906b --- /dev/null +++ b/vendor/github.com/Shopify/sarama/tools/kafka-console-consumer/kafka-console-consumer.go @@ -0,0 +1,145 @@ +package main + +import ( + "flag" + "fmt" + "log" + "os" + "os/signal" + "strconv" + "strings" + "sync" + + "github.com/Shopify/sarama" +) + +var ( + brokerList = flag.String("brokers", os.Getenv("KAFKA_PEERS"), "The comma separated list of brokers in the Kafka cluster") + topic = flag.String("topic", "", "REQUIRED: the topic to consume") + partitions = flag.String("partitions", "all", "The partitions to consume, can be 'all' or comma-separated numbers") + offset = flag.String("offset", "newest", "The offset to start with. Can be `oldest`, `newest`") + verbose = flag.Bool("verbose", false, "Whether to turn on sarama logging") + bufferSize = flag.Int("buffer-size", 256, "The buffer size of the message channel.") + + logger = log.New(os.Stderr, "", log.LstdFlags) +) + +func main() { + flag.Parse() + + if *brokerList == "" { + printUsageErrorAndExit("You have to provide -brokers as a comma-separated list, or set the KAFKA_PEERS environment variable.") + } + + if *topic == "" { + printUsageErrorAndExit("-topic is required") + } + + if *verbose { + sarama.Logger = logger + } + + var initialOffset int64 + switch *offset { + case "oldest": + initialOffset = sarama.OffsetOldest + case "newest": + initialOffset = sarama.OffsetNewest + default: + printUsageErrorAndExit("-offset should be `oldest` or `newest`") + } + + c, err := sarama.NewConsumer(strings.Split(*brokerList, ","), nil) + if err != nil { + printErrorAndExit(69, "Failed to start consumer: %s", err) + } + + partitionList, err := getPartitions(c) + if err != nil { + printErrorAndExit(69, "Failed to get the list of partitions: %s", err) + } + + var ( + messages = make(chan *sarama.ConsumerMessage, *bufferSize) + closing = make(chan struct{}) + wg sync.WaitGroup + ) + + go func() { + signals := make(chan os.Signal, 1) + signal.Notify(signals, os.Kill, os.Interrupt) + <-signals + logger.Println("Initiating shutdown of consumer...") + close(closing) + }() + + for _, partition := range partitionList { + pc, err := c.ConsumePartition(*topic, partition, initialOffset) + if err != nil { + printErrorAndExit(69, "Failed to start consumer for partition %d: %s", partition, err) + } + + go func(pc sarama.PartitionConsumer) { + <-closing + pc.AsyncClose() + }(pc) + + wg.Add(1) + go func(pc sarama.PartitionConsumer) { + defer wg.Done() + for message := range pc.Messages() { + messages <- message + } + }(pc) + } + + go func() { + for msg := range messages { + fmt.Printf("Partition:\t%d\n", msg.Partition) + fmt.Printf("Offset:\t%d\n", msg.Offset) + fmt.Printf("Key:\t%s\n", string(msg.Key)) + fmt.Printf("Value:\t%s\n", string(msg.Value)) + fmt.Println() + } + }() + + wg.Wait() + logger.Println("Done consuming topic", *topic) + close(messages) + + if err := c.Close(); err != nil { + logger.Println("Failed to close consumer: ", err) + } +} + +func getPartitions(c sarama.Consumer) ([]int32, error) { + if *partitions == "all" { + return c.Partitions(*topic) + } + + tmp := strings.Split(*partitions, ",") + var pList []int32 + for i := range tmp { + val, err := strconv.ParseInt(tmp[i], 10, 32) + if err != nil { + return nil, err + } + pList = append(pList, int32(val)) + } + + return pList, nil +} + +func printErrorAndExit(code int, format string, values ...interface{}) { + fmt.Fprintf(os.Stderr, "ERROR: %s\n", fmt.Sprintf(format, values...)) + fmt.Fprintln(os.Stderr) + os.Exit(code) +} + +func printUsageErrorAndExit(format string, values ...interface{}) { + fmt.Fprintf(os.Stderr, "ERROR: %s\n", fmt.Sprintf(format, values...)) + fmt.Fprintln(os.Stderr) + fmt.Fprintln(os.Stderr, "Available command line options:") + flag.PrintDefaults() + os.Exit(64) +} diff --git a/vendor/github.com/Shopify/sarama/tools/kafka-console-partitionconsumer/.gitignore b/vendor/github.com/Shopify/sarama/tools/kafka-console-partitionconsumer/.gitignore new file mode 100644 index 000000000000..5837fe8ca69f --- /dev/null +++ b/vendor/github.com/Shopify/sarama/tools/kafka-console-partitionconsumer/.gitignore @@ -0,0 +1,2 @@ +kafka-console-partitionconsumer +kafka-console-partitionconsumer.test diff --git a/vendor/github.com/Shopify/sarama/tools/kafka-console-partitionconsumer/README.md b/vendor/github.com/Shopify/sarama/tools/kafka-console-partitionconsumer/README.md new file mode 100644 index 000000000000..646dd5f5c296 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/tools/kafka-console-partitionconsumer/README.md @@ -0,0 +1,28 @@ +# kafka-console-partitionconsumer + +NOTE: this tool is deprecated in favour of the more general and more powerful +`kafka-console-consumer`. + +A simple command line tool to consume a partition of a topic and print the messages +on the standard output. + +### Installation + + go get github.com/Shopify/sarama/tools/kafka-console-partitionconsumer + +### Usage + + # Minimum invocation + kafka-console-partitionconsumer -topic=test -partition=4 -brokers=kafka1:9092 + + # It will pick up a KAFKA_PEERS environment variable + export KAFKA_PEERS=kafka1:9092,kafka2:9092,kafka3:9092 + kafka-console-partitionconsumer -topic=test -partition=4 + + # You can specify the offset you want to start at. It can be either + # `oldest`, `newest`, or a specific offset number + kafka-console-partitionconsumer -topic=test -partition=3 -offset=oldest + kafka-console-partitionconsumer -topic=test -partition=2 -offset=1337 + + # Display all command line options + kafka-console-partitionconsumer -help diff --git a/vendor/github.com/Shopify/sarama/tools/kafka-console-partitionconsumer/kafka-console-partitionconsumer.go b/vendor/github.com/Shopify/sarama/tools/kafka-console-partitionconsumer/kafka-console-partitionconsumer.go new file mode 100644 index 000000000000..d5e4464de184 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/tools/kafka-console-partitionconsumer/kafka-console-partitionconsumer.go @@ -0,0 +1,102 @@ +package main + +import ( + "flag" + "fmt" + "log" + "os" + "os/signal" + "strconv" + "strings" + + "github.com/Shopify/sarama" +) + +var ( + brokerList = flag.String("brokers", os.Getenv("KAFKA_PEERS"), "The comma separated list of brokers in the Kafka cluster") + topic = flag.String("topic", "", "REQUIRED: the topic to consume") + partition = flag.Int("partition", -1, "REQUIRED: the partition to consume") + offset = flag.String("offset", "newest", "The offset to start with. Can be `oldest`, `newest`, or an actual offset") + verbose = flag.Bool("verbose", false, "Whether to turn on sarama logging") + + logger = log.New(os.Stderr, "", log.LstdFlags) +) + +func main() { + flag.Parse() + + if *brokerList == "" { + printUsageErrorAndExit("You have to provide -brokers as a comma-separated list, or set the KAFKA_PEERS environment variable.") + } + + if *topic == "" { + printUsageErrorAndExit("-topic is required") + } + + if *partition == -1 { + printUsageErrorAndExit("-partition is required") + } + + if *verbose { + sarama.Logger = logger + } + + var ( + initialOffset int64 + offsetError error + ) + switch *offset { + case "oldest": + initialOffset = sarama.OffsetOldest + case "newest": + initialOffset = sarama.OffsetNewest + default: + initialOffset, offsetError = strconv.ParseInt(*offset, 10, 64) + } + + if offsetError != nil { + printUsageErrorAndExit("Invalid initial offset: %s", *offset) + } + + c, err := sarama.NewConsumer(strings.Split(*brokerList, ","), nil) + if err != nil { + printErrorAndExit(69, "Failed to start consumer: %s", err) + } + + pc, err := c.ConsumePartition(*topic, int32(*partition), initialOffset) + if err != nil { + printErrorAndExit(69, "Failed to start partition consumer: %s", err) + } + + go func() { + signals := make(chan os.Signal, 1) + signal.Notify(signals, os.Kill, os.Interrupt) + <-signals + pc.AsyncClose() + }() + + for msg := range pc.Messages() { + fmt.Printf("Offset:\t%d\n", msg.Offset) + fmt.Printf("Key:\t%s\n", string(msg.Key)) + fmt.Printf("Value:\t%s\n", string(msg.Value)) + fmt.Println() + } + + if err := c.Close(); err != nil { + logger.Println("Failed to close consumer: ", err) + } +} + +func printErrorAndExit(code int, format string, values ...interface{}) { + fmt.Fprintf(os.Stderr, "ERROR: %s\n", fmt.Sprintf(format, values...)) + fmt.Fprintln(os.Stderr) + os.Exit(code) +} + +func printUsageErrorAndExit(format string, values ...interface{}) { + fmt.Fprintf(os.Stderr, "ERROR: %s\n", fmt.Sprintf(format, values...)) + fmt.Fprintln(os.Stderr) + fmt.Fprintln(os.Stderr, "Available command line options:") + flag.PrintDefaults() + os.Exit(64) +} diff --git a/vendor/github.com/Shopify/sarama/tools/kafka-console-producer/.gitignore b/vendor/github.com/Shopify/sarama/tools/kafka-console-producer/.gitignore new file mode 100644 index 000000000000..2b9e563a1c62 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/tools/kafka-console-producer/.gitignore @@ -0,0 +1,2 @@ +kafka-console-producer +kafka-console-producer.test diff --git a/vendor/github.com/Shopify/sarama/tools/kafka-console-producer/README.md b/vendor/github.com/Shopify/sarama/tools/kafka-console-producer/README.md new file mode 100644 index 000000000000..6b3a65f211ab --- /dev/null +++ b/vendor/github.com/Shopify/sarama/tools/kafka-console-producer/README.md @@ -0,0 +1,34 @@ +# kafka-console-producer + +A simple command line tool to produce a single message to Kafka. + +### Installation + + go get github.com/Shopify/sarama/tools/kafka-console-producer + + +### Usage + + # Minimum invocation + kafka-console-producer -topic=test -value=value -brokers=kafka1:9092 + + # It will pick up a KAFKA_PEERS environment variable + export KAFKA_PEERS=kafka1:9092,kafka2:9092,kafka3:9092 + kafka-console-producer -topic=test -value=value + + # It will read the value from stdin by using pipes + echo "hello world" | kafka-console-producer -topic=test + + # Specify a key: + echo "hello world" | kafka-console-producer -topic=test -key=key + + # Partitioning: by default, kafka-console-producer will partition as follows: + # - manual partitioning if a -partition is provided + # - hash partitioning by key if a -key is provided + # - random partioning otherwise. + # + # You can override this using the -partitioner argument: + echo "hello world" | kafka-console-producer -topic=test -key=key -partitioner=random + + # Display all command line options + kafka-console-producer -help diff --git a/vendor/github.com/Shopify/sarama/tools/kafka-console-producer/kafka-console-producer.go b/vendor/github.com/Shopify/sarama/tools/kafka-console-producer/kafka-console-producer.go new file mode 100644 index 000000000000..6a1765d7c9f1 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/tools/kafka-console-producer/kafka-console-producer.go @@ -0,0 +1,118 @@ +package main + +import ( + "flag" + "fmt" + "io/ioutil" + "log" + "os" + "strings" + + "github.com/Shopify/sarama" +) + +var ( + brokerList = flag.String("brokers", os.Getenv("KAFKA_PEERS"), "The comma separated list of brokers in the Kafka cluster. You can also set the KAFKA_PEERS environment variable") + topic = flag.String("topic", "", "REQUIRED: the topic to produce to") + key = flag.String("key", "", "The key of the message to produce. Can be empty.") + value = flag.String("value", "", "REQUIRED: the value of the message to produce. You can also provide the value on stdin.") + partitioner = flag.String("partitioner", "", "The partitioning scheme to use. Can be `hash`, `manual`, or `random`") + partition = flag.Int("partition", -1, "The partition to produce to.") + verbose = flag.Bool("verbose", false, "Turn on sarama logging to stderr") + silent = flag.Bool("silent", false, "Turn off printing the message's topic, partition, and offset to stdout") + + logger = log.New(os.Stderr, "", log.LstdFlags) +) + +func main() { + flag.Parse() + + if *brokerList == "" { + printUsageErrorAndExit("no -brokers specified. Alternatively, set the KAFKA_PEERS environment variable") + } + + if *topic == "" { + printUsageErrorAndExit("no -topic specified") + } + + if *verbose { + sarama.Logger = logger + } + + config := sarama.NewConfig() + config.Producer.RequiredAcks = sarama.WaitForAll + + switch *partitioner { + case "": + if *partition >= 0 { + config.Producer.Partitioner = sarama.NewManualPartitioner + } else { + config.Producer.Partitioner = sarama.NewHashPartitioner + } + case "hash": + config.Producer.Partitioner = sarama.NewHashPartitioner + case "random": + config.Producer.Partitioner = sarama.NewRandomPartitioner + case "manual": + config.Producer.Partitioner = sarama.NewManualPartitioner + if *partition == -1 { + printUsageErrorAndExit("-partition is required when partitioning manually") + } + default: + printUsageErrorAndExit(fmt.Sprintf("Partitioner %s not supported.", *partitioner)) + } + + message := &sarama.ProducerMessage{Topic: *topic, Partition: int32(*partition)} + + if *key != "" { + message.Key = sarama.StringEncoder(*key) + } + + if *value != "" { + message.Value = sarama.StringEncoder(*value) + } else if stdinAvailable() { + bytes, err := ioutil.ReadAll(os.Stdin) + if err != nil { + printErrorAndExit(66, "Failed to read data from the standard input: %s", err) + } + message.Value = sarama.ByteEncoder(bytes) + } else { + printUsageErrorAndExit("-value is required, or you have to provide the value on stdin") + } + + producer, err := sarama.NewSyncProducer(strings.Split(*brokerList, ","), config) + if err != nil { + printErrorAndExit(69, "Failed to open Kafka producer: %s", err) + } + defer func() { + if err := producer.Close(); err != nil { + logger.Println("Failed to close Kafka producer cleanly:", err) + } + }() + + partition, offset, err := producer.SendMessage(message) + if err != nil { + printErrorAndExit(69, "Failed to produce message: %s", err) + } else if !*silent { + fmt.Printf("topic=%s\tpartition=%d\toffset=%d\n", *topic, partition, offset) + } +} + +func printErrorAndExit(code int, format string, values ...interface{}) { + fmt.Fprintf(os.Stderr, "ERROR: %s\n", fmt.Sprintf(format, values...)) + fmt.Fprintln(os.Stderr) + os.Exit(code) +} + +func printUsageErrorAndExit(message string) { + fmt.Fprintln(os.Stderr, "ERROR:", message) + fmt.Fprintln(os.Stderr) + fmt.Fprintln(os.Stderr, "Available command line options:") + flag.PrintDefaults() + os.Exit(64) +} + +func stdinAvailable() bool { + stat, _ := os.Stdin.Stat() + return (stat.Mode() & os.ModeCharDevice) == 0 +} diff --git a/vendor/github.com/Shopify/sarama/utils.go b/vendor/github.com/Shopify/sarama/utils.go new file mode 100644 index 000000000000..04ca88750de1 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/utils.go @@ -0,0 +1,111 @@ +package sarama + +import ( + "bufio" + "net" + "sort" +) + +type none struct{} + +// make []int32 sortable so we can sort partition numbers +type int32Slice []int32 + +func (slice int32Slice) Len() int { + return len(slice) +} + +func (slice int32Slice) Less(i, j int) bool { + return slice[i] < slice[j] +} + +func (slice int32Slice) Swap(i, j int) { + slice[i], slice[j] = slice[j], slice[i] +} + +func dupeAndSort(input []int32) []int32 { + ret := make([]int32, 0, len(input)) + for _, val := range input { + ret = append(ret, val) + } + + sort.Sort(int32Slice(ret)) + return ret +} + +func withRecover(fn func()) { + defer func() { + handler := PanicHandler + if handler != nil { + if err := recover(); err != nil { + handler(err) + } + } + }() + + fn() +} + +func safeAsyncClose(b *Broker) { + tmp := b // local var prevents clobbering in goroutine + go withRecover(func() { + if connected, _ := tmp.Connected(); connected { + if err := tmp.Close(); err != nil { + Logger.Println("Error closing broker", tmp.ID(), ":", err) + } + } + }) +} + +// Encoder is a simple interface for any type that can be encoded as an array of bytes +// in order to be sent as the key or value of a Kafka message. Length() is provided as an +// optimization, and must return the same as len() on the result of Encode(). +type Encoder interface { + Encode() ([]byte, error) + Length() int +} + +// make strings and byte slices encodable for convenience so they can be used as keys +// and/or values in kafka messages + +// StringEncoder implements the Encoder interface for Go strings so that they can be used +// as the Key or Value in a ProducerMessage. +type StringEncoder string + +func (s StringEncoder) Encode() ([]byte, error) { + return []byte(s), nil +} + +func (s StringEncoder) Length() int { + return len(s) +} + +// ByteEncoder implements the Encoder interface for Go byte slices so that they can be used +// as the Key or Value in a ProducerMessage. +type ByteEncoder []byte + +func (b ByteEncoder) Encode() ([]byte, error) { + return b, nil +} + +func (b ByteEncoder) Length() int { + return len(b) +} + +// bufConn wraps a net.Conn with a buffer for reads to reduce the number of +// reads that trigger syscalls. +type bufConn struct { + net.Conn + buf *bufio.Reader +} + +func newBufConn(conn net.Conn) *bufConn { + return &bufConn{ + Conn: conn, + buf: bufio.NewReader(conn), + } +} + +func (bc *bufConn) Read(b []byte) (n int, err error) { + return bc.buf.Read(b) +} diff --git a/vendor/github.com/Shopify/sarama/vagrant/boot_cluster.sh b/vendor/github.com/Shopify/sarama/vagrant/boot_cluster.sh new file mode 100755 index 000000000000..95e47dde4344 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/vagrant/boot_cluster.sh @@ -0,0 +1,22 @@ +#!/bin/sh + +set -ex + +# Launch and wait for toxiproxy +${REPOSITORY_ROOT}/vagrant/run_toxiproxy.sh & +while ! nc -q 1 localhost 2181 ${KAFKA_INSTALL_ROOT}/zookeeper-${ZK_PORT}/myid +done diff --git a/vendor/github.com/Shopify/sarama/vagrant/kafka.conf b/vendor/github.com/Shopify/sarama/vagrant/kafka.conf new file mode 100644 index 000000000000..d975de4387d4 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/vagrant/kafka.conf @@ -0,0 +1,5 @@ +start on started zookeeper-ZK_PORT +stop on stopping zookeeper-ZK_PORT + +pre-start exec sleep 2 +exec /opt/kafka-KAFKAID/bin/kafka-server-start.sh /opt/kafka-KAFKAID/config/server.properties diff --git a/vendor/github.com/Shopify/sarama/vagrant/provision.sh b/vendor/github.com/Shopify/sarama/vagrant/provision.sh new file mode 100755 index 000000000000..b1e429fb31b5 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/vagrant/provision.sh @@ -0,0 +1,15 @@ +#!/bin/sh + +set -ex + +apt-get update +yes | apt-get install default-jre + +export KAFKA_INSTALL_ROOT=/opt +export KAFKA_HOSTNAME=192.168.100.67 +export KAFKA_VERSION=0.9.0.0 +export REPOSITORY_ROOT=/vagrant + +sh /vagrant/vagrant/install_cluster.sh +sh /vagrant/vagrant/setup_services.sh +sh /vagrant/vagrant/create_topics.sh diff --git a/vendor/github.com/Shopify/sarama/vagrant/run_toxiproxy.sh b/vendor/github.com/Shopify/sarama/vagrant/run_toxiproxy.sh new file mode 100755 index 000000000000..e52c00e7b5a8 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/vagrant/run_toxiproxy.sh @@ -0,0 +1,22 @@ +#!/bin/sh + +set -ex + +${KAFKA_INSTALL_ROOT}/toxiproxy -port 8474 -host 0.0.0.0 & +PID=$! + +while ! nc -q 1 localhost 8474 + +# The number of threads handling network requests +num.network.threads=2 + +# The number of threads doing disk I/O +num.io.threads=8 + +# The send buffer (SO_SNDBUF) used by the socket server +socket.send.buffer.bytes=1048576 + +# The receive buffer (SO_RCVBUF) used by the socket server +socket.receive.buffer.bytes=1048576 + +# The maximum size of a request that the socket server will accept (protection against OOM) +socket.request.max.bytes=104857600 + + +############################# Log Basics ############################# + +# A comma seperated list of directories under which to store log files +log.dirs=KAFKA_DATADIR + +# The default number of log partitions per topic. More partitions allow greater +# parallelism for consumption, but this will also result in more files across +# the brokers. +num.partitions=2 + +# Create new topics with a replication factor of 2 so failover can be tested +# more easily. +default.replication.factor=2 + +auto.create.topics.enable=false +delete.topic.enable=true + +############################# Log Flush Policy ############################# + +# Messages are immediately written to the filesystem but by default we only fsync() to sync +# the OS cache lazily. The following configurations control the flush of data to disk. +# There are a few important trade-offs here: +# 1. Durability: Unflushed data may be lost if you are not using replication. +# 2. Latency: Very large flush intervals may lead to latency spikes when the flush does occur as there will be a lot of data to flush. +# 3. Throughput: The flush is generally the most expensive operation, and a small flush interval may lead to exceessive seeks. +# The settings below allow one to configure the flush policy to flush data after a period of time or +# every N messages (or both). This can be done globally and overridden on a per-topic basis. + +# The number of messages to accept before forcing a flush of data to disk +#log.flush.interval.messages=10000 + +# The maximum amount of time a message can sit in a log before we force a flush +#log.flush.interval.ms=1000 + +############################# Log Retention Policy ############################# + +# The following configurations control the disposal of log segments. The policy can +# be set to delete segments after a period of time, or after a given size has accumulated. +# A segment will be deleted whenever *either* of these criteria are met. Deletion always happens +# from the end of the log. + +# The minimum age of a log file to be eligible for deletion +log.retention.hours=168 + +# A size-based retention policy for logs. Segments are pruned from the log as long as the remaining +# segments don't drop below log.retention.bytes. +log.retention.bytes=268435456 + +# The maximum size of a log segment file. When this size is reached a new log segment will be created. +log.segment.bytes=268435456 + +# The interval at which log segments are checked to see if they can be deleted according +# to the retention policies +log.retention.check.interval.ms=60000 + +# By default the log cleaner is disabled and the log retention policy will default to just delete segments after their retention expires. +# If log.cleaner.enable=true is set the cleaner will be enabled and individual logs can then be marked for log compaction. +log.cleaner.enable=false + +############################# Zookeeper ############################# + +# Zookeeper connection string (see zookeeper docs for details). +# This is a comma separated host:port pairs, each corresponding to a zk +# server. e.g. "127.0.0.1:3000,127.0.0.1:3001,127.0.0.1:3002". +# You can also append an optional chroot string to the urls to specify the +# root directory for all kafka znodes. +zookeeper.connect=localhost:ZK_PORT + +# Timeout in ms for connecting to zookeeper +zookeeper.session.timeout.ms=3000 +zookeeper.connection.timeout.ms=3000 diff --git a/vendor/github.com/Shopify/sarama/vagrant/setup_services.sh b/vendor/github.com/Shopify/sarama/vagrant/setup_services.sh new file mode 100755 index 000000000000..81d8ea05d3b1 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/vagrant/setup_services.sh @@ -0,0 +1,29 @@ +#!/bin/sh + +set -ex + +stop toxiproxy || true +cp ${REPOSITORY_ROOT}/vagrant/toxiproxy.conf /etc/init/toxiproxy.conf +cp ${REPOSITORY_ROOT}/vagrant/run_toxiproxy.sh ${KAFKA_INSTALL_ROOT}/ +start toxiproxy + +for i in 1 2 3 4 5; do + ZK_PORT=`expr $i + 2180` + KAFKA_PORT=`expr $i + 9090` + + stop zookeeper-${ZK_PORT} || true + + # set up zk service + cp ${REPOSITORY_ROOT}/vagrant/zookeeper.conf /etc/init/zookeeper-${ZK_PORT}.conf + sed -i s/KAFKAID/${KAFKA_PORT}/g /etc/init/zookeeper-${ZK_PORT}.conf + + # set up kafka service + cp ${REPOSITORY_ROOT}/vagrant/kafka.conf /etc/init/kafka-${KAFKA_PORT}.conf + sed -i s/KAFKAID/${KAFKA_PORT}/g /etc/init/kafka-${KAFKA_PORT}.conf + sed -i s/ZK_PORT/${ZK_PORT}/g /etc/init/kafka-${KAFKA_PORT}.conf + + start zookeeper-${ZK_PORT} +done + +# Wait for the last kafka node to finish booting +while ! nc -q 1 localhost 29095 0 { + expiry := b.lastError.Add(b.timeout) + if time.Now().After(expiry) { + b.errors = 0 + } + } + + switch b.state { + case closed: + b.errors++ + if b.errors == b.errorThreshold { + b.openBreaker() + } else { + b.lastError = time.Now() + } + case halfOpen: + b.openBreaker() + } + } +} + +func (b *Breaker) openBreaker() { + b.changeState(open) + go b.timer() +} + +func (b *Breaker) closeBreaker() { + b.changeState(closed) +} + +func (b *Breaker) timer() { + time.Sleep(b.timeout) + + b.lock.Lock() + defer b.lock.Unlock() + + b.changeState(halfOpen) +} + +func (b *Breaker) changeState(newState uint32) { + b.errors = 0 + b.successes = 0 + atomic.StoreUint32(&b.state, newState) +} diff --git a/vendor/github.com/eapache/go-resiliency/breaker/breaker_test.go b/vendor/github.com/eapache/go-resiliency/breaker/breaker_test.go new file mode 100644 index 000000000000..b41308db60b4 --- /dev/null +++ b/vendor/github.com/eapache/go-resiliency/breaker/breaker_test.go @@ -0,0 +1,196 @@ +package breaker + +import ( + "errors" + "testing" + "time" +) + +var errSomeError = errors.New("errSomeError") + +func alwaysPanics() error { + panic("foo") +} + +func returnsError() error { + return errSomeError +} + +func returnsSuccess() error { + return nil +} + +func TestBreakerErrorExpiry(t *testing.T) { + breaker := New(2, 1, 1*time.Second) + + for i := 0; i < 3; i++ { + if err := breaker.Run(returnsError); err != errSomeError { + t.Error(err) + } + time.Sleep(1 * time.Second) + } + + for i := 0; i < 3; i++ { + if err := breaker.Go(returnsError); err != nil { + t.Error(err) + } + time.Sleep(1 * time.Second) + } +} + +func TestBreakerPanicsCountAsErrors(t *testing.T) { + breaker := New(3, 2, 1*time.Second) + + // three errors opens the breaker + for i := 0; i < 3; i++ { + func() { + defer func() { + val := recover() + if val.(string) != "foo" { + t.Error("incorrect panic") + } + }() + if err := breaker.Run(alwaysPanics); err != nil { + t.Error(err) + } + t.Error("shouldn't get here") + }() + } + + // breaker is open + for i := 0; i < 5; i++ { + if err := breaker.Run(returnsError); err != ErrBreakerOpen { + t.Error(err) + } + } +} + +func TestBreakerStateTransitions(t *testing.T) { + breaker := New(3, 2, 1*time.Second) + + // three errors opens the breaker + for i := 0; i < 3; i++ { + if err := breaker.Run(returnsError); err != errSomeError { + t.Error(err) + } + } + + // breaker is open + for i := 0; i < 5; i++ { + if err := breaker.Run(returnsError); err != ErrBreakerOpen { + t.Error(err) + } + } + + // wait for it to half-close + time.Sleep(2 * time.Second) + // one success works, but is not enough to fully close + if err := breaker.Run(returnsSuccess); err != nil { + t.Error(err) + } + // error works, but re-opens immediately + if err := breaker.Run(returnsError); err != errSomeError { + t.Error(err) + } + // breaker is open + if err := breaker.Run(returnsError); err != ErrBreakerOpen { + t.Error(err) + } + + // wait for it to half-close + time.Sleep(2 * time.Second) + // two successes is enough to close it for good + for i := 0; i < 2; i++ { + if err := breaker.Run(returnsSuccess); err != nil { + t.Error(err) + } + } + // error works + if err := breaker.Run(returnsError); err != errSomeError { + t.Error(err) + } + // breaker is still closed + if err := breaker.Run(returnsSuccess); err != nil { + t.Error(err) + } +} + +func TestBreakerAsyncStateTransitions(t *testing.T) { + breaker := New(3, 2, 1*time.Second) + + // three errors opens the breaker + for i := 0; i < 3; i++ { + if err := breaker.Go(returnsError); err != nil { + t.Error(err) + } + } + + // just enough to yield the scheduler and let the goroutines work off + time.Sleep(1 * time.Millisecond) + + // breaker is open + for i := 0; i < 5; i++ { + if err := breaker.Go(returnsError); err != ErrBreakerOpen { + t.Error(err) + } + } + + // wait for it to half-close + time.Sleep(2 * time.Second) + // one success works, but is not enough to fully close + if err := breaker.Go(returnsSuccess); err != nil { + t.Error(err) + } + // error works, but re-opens immediately + if err := breaker.Go(returnsError); err != nil { + t.Error(err) + } + // just enough to yield the scheduler and let the goroutines work off + time.Sleep(1 * time.Millisecond) + // breaker is open + if err := breaker.Go(returnsError); err != ErrBreakerOpen { + t.Error(err) + } + + // wait for it to half-close + time.Sleep(2 * time.Second) + // two successes is enough to close it for good + for i := 0; i < 2; i++ { + if err := breaker.Go(returnsSuccess); err != nil { + t.Error(err) + } + } + // just enough to yield the scheduler and let the goroutines work off + time.Sleep(1 * time.Millisecond) + // error works + if err := breaker.Go(returnsError); err != nil { + t.Error(err) + } + // just enough to yield the scheduler and let the goroutines work off + time.Sleep(1 * time.Millisecond) + // breaker is still closed + if err := breaker.Go(returnsSuccess); err != nil { + t.Error(err) + } +} + +func ExampleBreaker() { + breaker := New(3, 1, 5*time.Second) + + for { + result := breaker.Run(func() error { + // communicate with some external service and + // return an error if the communication failed + return nil + }) + + switch result { + case nil: + // success! + case ErrBreakerOpen: + // our function wasn't run because the breaker was open + default: + // some other error + } + } +} diff --git a/vendor/github.com/eapache/go-resiliency/deadline/README.md b/vendor/github.com/eapache/go-resiliency/deadline/README.md new file mode 100644 index 000000000000..ac97b460f481 --- /dev/null +++ b/vendor/github.com/eapache/go-resiliency/deadline/README.md @@ -0,0 +1,27 @@ +deadline +======== + +[![Build Status](https://travis-ci.org/eapache/go-resiliency.svg?branch=master)](https://travis-ci.org/eapache/go-resiliency) +[![GoDoc](https://godoc.org/github.com/eapache/go-resiliency/deadline?status.svg)](https://godoc.org/github.com/eapache/go-resiliency/deadline) +[![Code of Conduct](https://img.shields.io/badge/code%20of%20conduct-active-blue.svg)](https://eapache.github.io/conduct.html) + +The deadline/timeout resiliency pattern for golang. + +Creating a deadline takes one parameter: how long to wait. + +```go +dl := deadline.New(1 * time.Second) + +err := dl.Run(func(stopper <-chan struct{}) error { + // do something potentially slow + // give up when the `stopper` channel is closed (indicating a time-out) + return nil +}) + +switch err { +case deadline.ErrTimedOut: + // execution took too long, oops +default: + // some other error +} +``` diff --git a/vendor/github.com/eapache/go-resiliency/deadline/deadline.go b/vendor/github.com/eapache/go-resiliency/deadline/deadline.go new file mode 100644 index 000000000000..3a6dfb0eec14 --- /dev/null +++ b/vendor/github.com/eapache/go-resiliency/deadline/deadline.go @@ -0,0 +1,45 @@ +// Package deadline implements the deadline (also known as "timeout") resiliency pattern for Go. +package deadline + +import ( + "errors" + "time" +) + +// ErrTimedOut is the error returned from Run when the deadline expires. +var ErrTimedOut = errors.New("timed out waiting for function to finish") + +// Deadline implements the deadline/timeout resiliency pattern. +type Deadline struct { + timeout time.Duration +} + +// New constructs a new Deadline with the given timeout. +func New(timeout time.Duration) *Deadline { + return &Deadline{ + timeout: timeout, + } +} + +// Run runs the given function, passing it a stopper channel. If the deadline passes before +// the function finishes executing, Run returns ErrTimeOut to the caller and closes the stopper +// channel so that the work function can attempt to exit gracefully. It does not (and cannot) +// simply kill the running function, so if it doesn't respect the stopper channel then it may +// keep running after the deadline passes. If the function finishes before the deadline, then +// the return value of the function is returned from Run. +func (d *Deadline) Run(work func(<-chan struct{}) error) error { + result := make(chan error) + stopper := make(chan struct{}) + + go func() { + result <- work(stopper) + }() + + select { + case ret := <-result: + return ret + case <-time.After(d.timeout): + close(stopper) + return ErrTimedOut + } +} diff --git a/vendor/github.com/eapache/go-resiliency/deadline/deadline_test.go b/vendor/github.com/eapache/go-resiliency/deadline/deadline_test.go new file mode 100644 index 000000000000..6939f52e8ba9 --- /dev/null +++ b/vendor/github.com/eapache/go-resiliency/deadline/deadline_test.go @@ -0,0 +1,65 @@ +package deadline + +import ( + "errors" + "testing" + "time" +) + +func takesFiveMillis(stopper <-chan struct{}) error { + time.Sleep(5 * time.Millisecond) + return nil +} + +func takesTwentyMillis(stopper <-chan struct{}) error { + time.Sleep(20 * time.Millisecond) + return nil +} + +func returnsError(stopper <-chan struct{}) error { + return errors.New("foo") +} + +func TestDeadline(t *testing.T) { + dl := New(10 * time.Millisecond) + + if err := dl.Run(takesFiveMillis); err != nil { + t.Error(err) + } + + if err := dl.Run(takesTwentyMillis); err != ErrTimedOut { + t.Error(err) + } + + if err := dl.Run(returnsError); err.Error() != "foo" { + t.Error(err) + } + + done := make(chan struct{}) + err := dl.Run(func(stopper <-chan struct{}) error { + <-stopper + close(done) + return nil + }) + if err != ErrTimedOut { + t.Error(err) + } + <-done +} + +func ExampleDeadline() { + dl := New(1 * time.Second) + + err := dl.Run(func(stopper <-chan struct{}) error { + // do something possibly slow + // check stopper function and give up if timed out + return nil + }) + + switch err { + case ErrTimedOut: + // execution took too long, oops + default: + // some other error + } +} diff --git a/vendor/github.com/eapache/go-resiliency/retrier/README.md b/vendor/github.com/eapache/go-resiliency/retrier/README.md new file mode 100644 index 000000000000..dd30af7a0428 --- /dev/null +++ b/vendor/github.com/eapache/go-resiliency/retrier/README.md @@ -0,0 +1,26 @@ +retrier +======= + +[![Build Status](https://travis-ci.org/eapache/go-resiliency.svg?branch=master)](https://travis-ci.org/eapache/go-resiliency) +[![GoDoc](https://godoc.org/github.com/eapache/go-resiliency/retrier?status.svg)](https://godoc.org/github.com/eapache/go-resiliency/retrier) +[![Code of Conduct](https://img.shields.io/badge/code%20of%20conduct-active-blue.svg)](https://eapache.github.io/conduct.html) + +The retriable resiliency pattern for golang. + +Creating a retrier takes two parameters: +- the times to back-off between retries (and implicitly the number of times to + retry) +- the classifier that determines which errors to retry + +```go +r := retrier.New(retrier.ConstantBackoff(3, 100*time.Millisecond), nil) + +err := r.Run(func() error { + // do some work + return nil +}) + +if err != nil { + // handle the case where the work failed three times +} +``` diff --git a/vendor/github.com/eapache/go-resiliency/retrier/backoffs.go b/vendor/github.com/eapache/go-resiliency/retrier/backoffs.go new file mode 100644 index 000000000000..faf6f8cf9853 --- /dev/null +++ b/vendor/github.com/eapache/go-resiliency/retrier/backoffs.go @@ -0,0 +1,24 @@ +package retrier + +import "time" + +// ConstantBackoff generates a simple back-off strategy of retrying 'n' times, and waiting 'amount' time after each one. +func ConstantBackoff(n int, amount time.Duration) []time.Duration { + ret := make([]time.Duration, n) + for i := range ret { + ret[i] = amount + } + return ret +} + +// ExponentialBackoff generates a simple back-off strategy of retrying 'n' times, and doubling the amount of +// time waited after each one. +func ExponentialBackoff(n int, initialAmount time.Duration) []time.Duration { + ret := make([]time.Duration, n) + next := initialAmount + for i := range ret { + ret[i] = next + next *= 2 + } + return ret +} diff --git a/vendor/github.com/eapache/go-resiliency/retrier/backoffs_test.go b/vendor/github.com/eapache/go-resiliency/retrier/backoffs_test.go new file mode 100644 index 000000000000..1168adfeb010 --- /dev/null +++ b/vendor/github.com/eapache/go-resiliency/retrier/backoffs_test.go @@ -0,0 +1,55 @@ +package retrier + +import ( + "testing" + "time" +) + +func TestConstantBackoff(t *testing.T) { + b := ConstantBackoff(1, 10*time.Millisecond) + if len(b) != 1 { + t.Error("incorrect length") + } + for i := range b { + if b[i] != 10*time.Millisecond { + t.Error("incorrect value at", i) + } + } + + b = ConstantBackoff(10, 250*time.Hour) + if len(b) != 10 { + t.Error("incorrect length") + } + for i := range b { + if b[i] != 250*time.Hour { + t.Error("incorrect value at", i) + } + } +} + +func TestExponentialBackoff(t *testing.T) { + b := ExponentialBackoff(1, 10*time.Millisecond) + if len(b) != 1 { + t.Error("incorrect length") + } + if b[0] != 10*time.Millisecond { + t.Error("incorrect value") + } + + b = ExponentialBackoff(4, 1*time.Minute) + if len(b) != 4 { + t.Error("incorrect length") + } + if b[0] != 1*time.Minute { + t.Error("incorrect value") + } + if b[1] != 2*time.Minute { + t.Error("incorrect value") + } + if b[2] != 4*time.Minute { + t.Error("incorrect value") + } + if b[3] != 8*time.Minute { + t.Error("incorrect value") + } +} diff --git a/vendor/github.com/eapache/go-resiliency/retrier/classifier.go b/vendor/github.com/eapache/go-resiliency/retrier/classifier.go new file mode 100644 index 000000000000..7dd71c7984d1 --- /dev/null +++ b/vendor/github.com/eapache/go-resiliency/retrier/classifier.go @@ -0,0 +1,66 @@ +package retrier + +// Action is the type returned by a Classifier to indicate how the Retrier should proceed. +type Action int + +const ( + Succeed Action = iota // Succeed indicates the Retrier should treat this value as a success. + Fail // Fail indicates the Retrier should treat this value as a hard failure and not retry. + Retry // Retry indicates the Retrier should treat this value as a soft failure and retry. +) + +// Classifier is the interface implemented by anything that can classify Errors for a Retrier. +type Classifier interface { + Classify(error) Action +} + +// DefaultClassifier classifies errors in the simplest way possible. If +// the error is nil, it returns Succeed, otherwise it returns Retry. +type DefaultClassifier struct{} + +// Classify implements the Classifier interface. +func (c DefaultClassifier) Classify(err error) Action { + if err == nil { + return Succeed + } + + return Retry +} + +// WhitelistClassifier classifies errors based on a whitelist. If the error is nil, it +// returns Succeed; if the error is in the whitelist, it returns Retry; otherwise, it returns Fail. +type WhitelistClassifier []error + +// Classify implements the Classifier interface. +func (list WhitelistClassifier) Classify(err error) Action { + if err == nil { + return Succeed + } + + for _, pass := range list { + if err == pass { + return Retry + } + } + + return Fail +} + +// BlacklistClassifier classifies errors based on a blacklist. If the error is nil, it +// returns Succeed; if the error is in the blacklist, it returns Fail; otherwise, it returns Retry. +type BlacklistClassifier []error + +// Classify implements the Classifier interface. +func (list BlacklistClassifier) Classify(err error) Action { + if err == nil { + return Succeed + } + + for _, pass := range list { + if err == pass { + return Fail + } + } + + return Retry +} diff --git a/vendor/github.com/eapache/go-resiliency/retrier/classifier_test.go b/vendor/github.com/eapache/go-resiliency/retrier/classifier_test.go new file mode 100644 index 000000000000..953102fbba38 --- /dev/null +++ b/vendor/github.com/eapache/go-resiliency/retrier/classifier_test.go @@ -0,0 +1,66 @@ +package retrier + +import ( + "errors" + "testing" +) + +var ( + errFoo = errors.New("FOO") + errBar = errors.New("BAR") + errBaz = errors.New("BAZ") +) + +func TestDefaultClassifier(t *testing.T) { + c := DefaultClassifier{} + + if c.Classify(nil) != Succeed { + t.Error("default misclassified nil") + } + + if c.Classify(errFoo) != Retry { + t.Error("default misclassified foo") + } + if c.Classify(errBar) != Retry { + t.Error("default misclassified bar") + } + if c.Classify(errBaz) != Retry { + t.Error("default misclassified baz") + } +} + +func TestWhitelistClassifier(t *testing.T) { + c := WhitelistClassifier{errFoo, errBar} + + if c.Classify(nil) != Succeed { + t.Error("whitelist misclassified nil") + } + + if c.Classify(errFoo) != Retry { + t.Error("whitelist misclassified foo") + } + if c.Classify(errBar) != Retry { + t.Error("whitelist misclassified bar") + } + if c.Classify(errBaz) != Fail { + t.Error("whitelist misclassified baz") + } +} + +func TestBlacklistClassifier(t *testing.T) { + c := BlacklistClassifier{errBar} + + if c.Classify(nil) != Succeed { + t.Error("blacklist misclassified nil") + } + + if c.Classify(errFoo) != Retry { + t.Error("blacklist misclassified foo") + } + if c.Classify(errBar) != Fail { + t.Error("blacklist misclassified bar") + } + if c.Classify(errBaz) != Retry { + t.Error("blacklist misclassified baz") + } +} diff --git a/vendor/github.com/eapache/go-resiliency/retrier/retrier.go b/vendor/github.com/eapache/go-resiliency/retrier/retrier.go new file mode 100644 index 000000000000..ff328742b26e --- /dev/null +++ b/vendor/github.com/eapache/go-resiliency/retrier/retrier.go @@ -0,0 +1,69 @@ +// Package retrier implements the "retriable" resiliency pattern for Go. +package retrier + +import ( + "math/rand" + "time" +) + +// Retrier implements the "retriable" resiliency pattern, abstracting out the process of retrying a failed action +// a certain number of times with an optional back-off between each retry. +type Retrier struct { + backoff []time.Duration + class Classifier + jitter float64 + rand *rand.Rand +} + +// New constructs a Retrier with the given backoff pattern and classifier. The length of the backoff pattern +// indicates how many times an action will be retried, and the value at each index indicates the amount of time +// waited before each subsequent retry. The classifier is used to determine which errors should be retried and +// which should cause the retrier to fail fast. The DefaultClassifier is used if nil is passed. +func New(backoff []time.Duration, class Classifier) *Retrier { + if class == nil { + class = DefaultClassifier{} + } + + return &Retrier{ + backoff: backoff, + class: class, + rand: rand.New(rand.NewSource(time.Now().UnixNano())), + } +} + +// Run executes the given work function, then classifies its return value based on the classifier used +// to construct the Retrier. If the result is Succeed or Fail, the return value of the work function is +// returned to the caller. If the result is Retry, then Run sleeps according to the its backoff policy +// before retrying. If the total number of retries is exceeded then the return value of the work function +// is returned to the caller regardless. +func (r *Retrier) Run(work func() error) error { + retries := 0 + for { + ret := work() + + switch r.class.Classify(ret) { + case Succeed, Fail: + return ret + case Retry: + if retries >= len(r.backoff) { + return ret + } + time.Sleep(r.calcSleep(retries)) + retries++ + } + } +} + +func (r *Retrier) calcSleep(i int) time.Duration { + // take a random float in the range (-r.jitter, +r.jitter) and multiply it by the base amount + return r.backoff[i] + time.Duration(((r.rand.Float64()*2)-1)*r.jitter*float64(r.backoff[i])) +} + +// SetJitter sets the amount of jitter on each back-off to a factor between 0.0 and 1.0 (values outside this range +// are silently ignored). When a retry occurs, the back-off is adjusted by a random amount up to this value. +func (r *Retrier) SetJitter(jit float64) { + if jit < 0 || jit > 1 { + return + } + r.jitter = jit +} diff --git a/vendor/github.com/eapache/go-resiliency/retrier/retrier_test.go b/vendor/github.com/eapache/go-resiliency/retrier/retrier_test.go new file mode 100644 index 000000000000..2d061d9b24f0 --- /dev/null +++ b/vendor/github.com/eapache/go-resiliency/retrier/retrier_test.go @@ -0,0 +1,129 @@ +package retrier + +import ( + "testing" + "time" +) + +var i int + +func genWork(returns []error) func() error { + i = 0 + return func() error { + i++ + if i > len(returns) { + return nil + } + return returns[i-1] + } +} + +func TestRetrier(t *testing.T) { + r := New([]time.Duration{0, 10 * time.Millisecond}, WhitelistClassifier{errFoo}) + + err := r.Run(genWork([]error{errFoo, errFoo})) + if err != nil { + t.Error(err) + } + if i != 3 { + t.Error("run wrong number of times") + } + + err = r.Run(genWork([]error{errFoo, errBar})) + if err != errBar { + t.Error(err) + } + if i != 2 { + t.Error("run wrong number of times") + } + + err = r.Run(genWork([]error{errBar, errBaz})) + if err != errBar { + t.Error(err) + } + if i != 1 { + t.Error("run wrong number of times") + } +} + +func TestRetrierNone(t *testing.T) { + r := New(nil, nil) + + i = 0 + err := r.Run(func() error { + i++ + return errFoo + }) + if err != errFoo { + t.Error(err) + } + if i != 1 { + t.Error("run wrong number of times") + } + + i = 0 + err = r.Run(func() error { + i++ + return nil + }) + if err != nil { + t.Error(err) + } + if i != 1 { + t.Error("run wrong number of times") + } +} + +func TestRetrierJitter(t *testing.T) { + r := New([]time.Duration{0, 10 * time.Millisecond, 4 * time.Hour}, nil) + + if r.calcSleep(0) != 0 { + t.Error("Incorrect sleep calculated") + } + if r.calcSleep(1) != 10*time.Millisecond { + t.Error("Incorrect sleep calculated") + } + if r.calcSleep(2) != 4*time.Hour { + t.Error("Incorrect sleep calculated") + } + + r.SetJitter(0.25) + for i := 0; i < 20; i++ { + if r.calcSleep(0) != 0 { + t.Error("Incorrect sleep calculated") + } + + slp := r.calcSleep(1) + if slp < 7500*time.Microsecond || slp > 12500*time.Microsecond { + t.Error("Incorrect sleep calculated") + } + + slp = r.calcSleep(2) + if slp < 3*time.Hour || slp > 5*time.Hour { + t.Error("Incorrect sleep calculated") + } + } + + r.SetJitter(-1) + if r.jitter != 0.25 { + t.Error("Invalid jitter value accepted") + } + + r.SetJitter(2) + if r.jitter != 0.25 { + t.Error("Invalid jitter value accepted") + } +} + +func ExampleRetrier() { + r := New(ConstantBackoff(3, 100*time.Millisecond), nil) + + err := r.Run(func() error { + // do some work + return nil + }) + + if err != nil { + // handle the case where the work failed three times + } +} diff --git a/vendor/github.com/eapache/go-resiliency/semaphore/README.md b/vendor/github.com/eapache/go-resiliency/semaphore/README.md new file mode 100644 index 000000000000..a4a73ea07e0a --- /dev/null +++ b/vendor/github.com/eapache/go-resiliency/semaphore/README.md @@ -0,0 +1,22 @@ +semaphore +========= + +[![Build Status](https://travis-ci.org/eapache/go-resiliency.svg?branch=master)](https://travis-ci.org/eapache/go-resiliency) +[![GoDoc](https://godoc.org/github.com/eapache/go-resiliency/semaphore?status.svg)](https://godoc.org/github.com/eapache/go-resiliency/semaphore) +[![Code of Conduct](https://img.shields.io/badge/code%20of%20conduct-active-blue.svg)](https://eapache.github.io/conduct.html) + +The semaphore resiliency pattern for golang. + +Creating a semaphore takes two parameters: +- ticket count (how many tickets to give out at once) +- timeout (how long to wait for a ticket if none are currently available) + +```go +sem := semaphore.New(3, 1*time.Second) + +if err := sem.Acquire(); err != nil { + // could not acquire semaphore + return err +} +defer sem.Release() +``` diff --git a/vendor/github.com/eapache/go-resiliency/semaphore/semaphore.go b/vendor/github.com/eapache/go-resiliency/semaphore/semaphore.go new file mode 100644 index 000000000000..90cb1fd15091 --- /dev/null +++ b/vendor/github.com/eapache/go-resiliency/semaphore/semaphore.go @@ -0,0 +1,45 @@ +// Package semaphore implements the semaphore resiliency pattern for Go. +package semaphore + +import ( + "errors" + "time" +) + +// ErrNoTickets is the error returned by Acquire when it could not acquire +// a ticket from the semaphore within the configured timeout. +var ErrNoTickets = errors.New("could not aquire semaphore ticket") + +// Semaphore implements the semaphore resiliency pattern +type Semaphore struct { + sem chan struct{} + timeout time.Duration +} + +// New constructs a new Semaphore with the given ticket-count +// and timeout. +func New(tickets int, timeout time.Duration) *Semaphore { + return &Semaphore{ + sem: make(chan struct{}, tickets), + timeout: timeout, + } +} + +// Acquire tries to acquire a ticket from the semaphore. If it can, it returns nil. +// If it cannot after "timeout" amount of time, it returns ErrNoTickets. It is +// safe to call Acquire concurrently on a single Semaphore. +func (s *Semaphore) Acquire() error { + select { + case s.sem <- struct{}{}: + return nil + case <-time.After(s.timeout): + return ErrNoTickets + } +} + +// Release releases an acquired ticket back to the semaphore. It is safe to call +// Release concurrently on a single Semaphore. It is an error to call Release on +// a Semaphore from which you have not first acquired a ticket. +func (s *Semaphore) Release() { + <-s.sem +} diff --git a/vendor/github.com/eapache/go-resiliency/semaphore/semaphore_test.go b/vendor/github.com/eapache/go-resiliency/semaphore/semaphore_test.go new file mode 100644 index 000000000000..3eb85f1a8230 --- /dev/null +++ b/vendor/github.com/eapache/go-resiliency/semaphore/semaphore_test.go @@ -0,0 +1,61 @@ +package semaphore + +import ( + "testing" + "time" +) + +func TestSemaphoreAcquireRelease(t *testing.T) { + sem := New(3, 1*time.Second) + + for i := 0; i < 10; i++ { + if err := sem.Acquire(); err != nil { + t.Error(err) + } + if err := sem.Acquire(); err != nil { + t.Error(err) + } + if err := sem.Acquire(); err != nil { + t.Error(err) + } + sem.Release() + sem.Release() + sem.Release() + } +} + +func TestSemaphoreBlockTimeout(t *testing.T) { + sem := New(1, 200*time.Millisecond) + + if err := sem.Acquire(); err != nil { + t.Error(err) + } + + start := time.Now() + if err := sem.Acquire(); err != ErrNoTickets { + t.Error(err) + } + if start.Add(200 * time.Millisecond).After(time.Now()) { + t.Error("semaphore did not wait long enough") + } + + sem.Release() + if err := sem.Acquire(); err != nil { + t.Error(err) + } +} + +func ExampleSemaphore() { + sem := New(3, 1*time.Second) + + for i := 0; i < 10; i++ { + go func() { + if err := sem.Acquire(); err != nil { + return //could not acquire semaphore + } + defer sem.Release() + + // do something semaphore-guarded + }() + } +} diff --git a/vendor/github.com/eapache/queue/.gitignore b/vendor/github.com/eapache/queue/.gitignore new file mode 100644 index 000000000000..836562412fe8 --- /dev/null +++ b/vendor/github.com/eapache/queue/.gitignore @@ -0,0 +1,23 @@ +# Compiled Object files, Static and Dynamic libs (Shared Objects) +*.o +*.a +*.so + +# Folders +_obj +_test + +# Architecture specific extensions/prefixes +*.[568vq] +[568vq].out + +*.cgo1.go +*.cgo2.c +_cgo_defun.c +_cgo_gotypes.go +_cgo_export.* + +_testmain.go + +*.exe +*.test diff --git a/vendor/github.com/eapache/queue/.travis.yml b/vendor/github.com/eapache/queue/.travis.yml new file mode 100644 index 000000000000..235a40a493ff --- /dev/null +++ b/vendor/github.com/eapache/queue/.travis.yml @@ -0,0 +1,7 @@ +language: go +sudo: false + +go: + - 1.2 + - 1.3 + - 1.4 diff --git a/vendor/github.com/eapache/queue/LICENSE b/vendor/github.com/eapache/queue/LICENSE new file mode 100644 index 000000000000..d5f36dbcaaf6 --- /dev/null +++ b/vendor/github.com/eapache/queue/LICENSE @@ -0,0 +1,21 @@ +The MIT License (MIT) + +Copyright (c) 2014 Evan Huus + +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. \ No newline at end of file diff --git a/vendor/github.com/eapache/queue/README.md b/vendor/github.com/eapache/queue/README.md new file mode 100644 index 000000000000..8e782335cd77 --- /dev/null +++ b/vendor/github.com/eapache/queue/README.md @@ -0,0 +1,16 @@ +Queue +===== + +[![Build Status](https://travis-ci.org/eapache/queue.svg)](https://travis-ci.org/eapache/queue) +[![GoDoc](https://godoc.org/github.com/eapache/queue?status.png)](https://godoc.org/github.com/eapache/queue) +[![Code of Conduct](https://img.shields.io/badge/code%20of%20conduct-active-blue.svg)](https://eapache.github.io/conduct.html) + +A fast Golang queue using a ring-buffer, based on the version suggested by Dariusz Górecki. +Using this instead of other, simpler, queue implementations (slice+append or linked list) provides +substantial memory and time benefits, and fewer GC pauses. + +The queue implemented here is as fast as it is in part because it is *not* thread-safe. + +Follows semantic versioning using https://gopkg.in/ - import from +[`gopkg.in/eapache/queue.v1`](https://gopkg.in/eapache/queue.v1) +for guaranteed API stability. diff --git a/vendor/github.com/eapache/queue/queue.go b/vendor/github.com/eapache/queue/queue.go new file mode 100644 index 000000000000..2dc8d9395660 --- /dev/null +++ b/vendor/github.com/eapache/queue/queue.go @@ -0,0 +1,88 @@ +/* +Package queue provides a fast, ring-buffer queue based on the version suggested by Dariusz Górecki. +Using this instead of other, simpler, queue implementations (slice+append or linked list) provides +substantial memory and time benefits, and fewer GC pauses. + +The queue implemented here is as fast as it is for an additional reason: it is *not* thread-safe. +*/ +package queue + +const minQueueLen = 16 + +// Queue represents a single instance of the queue data structure. +type Queue struct { + buf []interface{} + head, tail, count int +} + +// New constructs and returns a new Queue. +func New() *Queue { + return &Queue{ + buf: make([]interface{}, minQueueLen), + } +} + +// Length returns the number of elements currently stored in the queue. +func (q *Queue) Length() int { + return q.count +} + +// resizes the queue to fit exactly twice its current contents +// this can result in shrinking if the queue is less than half-full +func (q *Queue) resize() { + newBuf := make([]interface{}, q.count*2) + + if q.tail > q.head { + copy(newBuf, q.buf[q.head:q.tail]) + } else { + n := copy(newBuf, q.buf[q.head:]) + copy(newBuf[n:], q.buf[:q.tail]) + } + + q.head = 0 + q.tail = q.count + q.buf = newBuf +} + +// Add puts an element on the end of the queue. +func (q *Queue) Add(elem interface{}) { + if q.count == len(q.buf) { + q.resize() + } + + q.buf[q.tail] = elem + q.tail = (q.tail + 1) % len(q.buf) + q.count++ +} + +// Peek returns the element at the head of the queue. This call panics +// if the queue is empty. +func (q *Queue) Peek() interface{} { + if q.count <= 0 { + panic("queue: Peek() called on empty queue") + } + return q.buf[q.head] +} + +// Get returns the element at index i in the queue. If the index is +// invalid, the call will panic. +func (q *Queue) Get(i int) interface{} { + if i < 0 || i >= q.count { + panic("queue: Get() called with index out of range") + } + return q.buf[(q.head+i)%len(q.buf)] +} + +// Remove removes the element from the front of the queue. If you actually +// want the element, call Peek first. This call panics if the queue is empty. +func (q *Queue) Remove() { + if q.count <= 0 { + panic("queue: Remove() called on empty queue") + } + q.buf[q.head] = nil + q.head = (q.head + 1) % len(q.buf) + q.count-- + if len(q.buf) > minQueueLen && q.count*4 == len(q.buf) { + q.resize() + } +} diff --git a/vendor/github.com/eapache/queue/queue_test.go b/vendor/github.com/eapache/queue/queue_test.go new file mode 100644 index 000000000000..f2765c14d66b --- /dev/null +++ b/vendor/github.com/eapache/queue/queue_test.go @@ -0,0 +1,162 @@ +package queue + +import "testing" + +func TestQueueSimple(t *testing.T) { + q := New() + + for i := 0; i < minQueueLen; i++ { + q.Add(i) + } + for i := 0; i < minQueueLen; i++ { + if q.Peek().(int) != i { + t.Error("peek", i, "had value", q.Peek()) + } + q.Remove() + } +} + +func TestQueueWrapping(t *testing.T) { + q := New() + + for i := 0; i < minQueueLen; i++ { + q.Add(i) + } + for i := 0; i < 3; i++ { + q.Remove() + q.Add(minQueueLen + i) + } + + for i := 0; i < minQueueLen; i++ { + if q.Peek().(int) != i+3 { + t.Error("peek", i, "had value", q.Peek()) + } + q.Remove() + } +} + +func TestQueueLength(t *testing.T) { + q := New() + + if q.Length() != 0 { + t.Error("empty queue length not 0") + } + + for i := 0; i < 1000; i++ { + q.Add(i) + if q.Length() != i+1 { + t.Error("adding: queue with", i, "elements has length", q.Length()) + } + } + for i := 0; i < 1000; i++ { + q.Remove() + if q.Length() != 1000-i-1 { + t.Error("removing: queue with", 1000-i-i, "elements has length", q.Length()) + } + } +} + +func TestQueueGet(t *testing.T) { + q := New() + + for i := 0; i < 1000; i++ { + q.Add(i) + for j := 0; j < q.Length(); j++ { + if q.Get(j).(int) != j { + t.Errorf("index %d doesn't contain %d", j, j) + } + } + } +} + +func TestQueueGetOutOfRangePanics(t *testing.T) { + q := New() + + q.Add(1) + q.Add(2) + q.Add(3) + + assertPanics(t, "should panic when negative index", func() { + q.Get(-1) + }) + + assertPanics(t, "should panic when index greater than length", func() { + q.Get(4) + }) +} + +func TestQueuePeekOutOfRangePanics(t *testing.T) { + q := New() + + assertPanics(t, "should panic when peeking empty queue", func() { + q.Peek() + }) + + q.Add(1) + q.Remove() + + assertPanics(t, "should panic when peeking emptied queue", func() { + q.Peek() + }) +} + +func TestQueueRemoveOutOfRangePanics(t *testing.T) { + q := New() + + assertPanics(t, "should panic when removing empty queue", func() { + q.Remove() + }) + + q.Add(1) + q.Remove() + + assertPanics(t, "should panic when removing emptied queue", func() { + q.Remove() + }) +} + +func assertPanics(t *testing.T, name string, f func()) { + defer func() { + if r := recover(); r == nil { + t.Errorf("%s: didn't panic as expected", name) + } + }() + + f() +} + +// General warning: Go's benchmark utility (go test -bench .) increases the number of +// iterations until the benchmarks take a reasonable amount of time to run; memory usage +// is *NOT* considered. On my machine, these benchmarks hit around ~1GB before they've had +// enough, but if you have less than that available and start swapping, then all bets are off. + +func BenchmarkQueueSerial(b *testing.B) { + q := New() + for i := 0; i < b.N; i++ { + q.Add(nil) + } + for i := 0; i < b.N; i++ { + q.Peek() + q.Remove() + } +} + +func BenchmarkQueueGet(b *testing.B) { + q := New() + for i := 0; i < b.N; i++ { + q.Add(i) + } + b.ResetTimer() + for i := 0; i < b.N; i++ { + q.Get(i) + } +} + +func BenchmarkQueueTickTock(b *testing.B) { + q := New() + for i := 0; i < b.N; i++ { + q.Add(nil) + q.Peek() + q.Remove() + } +} diff --git a/vendor/github.com/golang/snappy/AUTHORS b/vendor/github.com/golang/snappy/AUTHORS new file mode 100644 index 000000000000..bcfa19520af9 --- /dev/null +++ b/vendor/github.com/golang/snappy/AUTHORS @@ -0,0 +1,15 @@ +# This is the official list of Snappy-Go authors for copyright purposes. +# This file is distinct from the CONTRIBUTORS files. +# See the latter for an explanation. + +# Names should be added to this file as +# Name or Organization +# The email address is not required for organizations. + +# Please keep the list sorted. + +Damian Gryski +Google Inc. +Jan Mercl <0xjnml@gmail.com> +Rodolfo Carvalho +Sebastien Binet diff --git a/vendor/github.com/golang/snappy/CONTRIBUTORS b/vendor/github.com/golang/snappy/CONTRIBUTORS new file mode 100644 index 000000000000..931ae31606f8 --- /dev/null +++ b/vendor/github.com/golang/snappy/CONTRIBUTORS @@ -0,0 +1,37 @@ +# This is the official list of people who can contribute +# (and typically have contributed) code to the Snappy-Go repository. +# The AUTHORS file lists the copyright holders; this file +# lists people. For example, Google employees are listed here +# but not in AUTHORS, because Google holds the copyright. +# +# The submission process automatically checks to make sure +# that people submitting code are listed in this file (by email address). +# +# Names should be added to this file only after verifying that +# the individual or the individual's organization has agreed to +# the appropriate Contributor License Agreement, found here: +# +# http://code.google.com/legal/individual-cla-v1.0.html +# http://code.google.com/legal/corporate-cla-v1.0.html +# +# The agreement for individuals can be filled out on the web. +# +# When adding J Random Contributor's name to this file, +# either J's name or J's organization's name should be +# added to the AUTHORS file, depending on whether the +# individual or corporate CLA was used. + +# Names should be added to this file like so: +# Name + +# Please keep the list sorted. + +Damian Gryski +Jan Mercl <0xjnml@gmail.com> +Kai Backman +Marc-Antoine Ruel +Nigel Tao +Rob Pike +Rodolfo Carvalho +Russ Cox +Sebastien Binet diff --git a/vendor/github.com/golang/snappy/LICENSE b/vendor/github.com/golang/snappy/LICENSE new file mode 100644 index 000000000000..6050c10f4c8b --- /dev/null +++ b/vendor/github.com/golang/snappy/LICENSE @@ -0,0 +1,27 @@ +Copyright (c) 2011 The Snappy-Go Authors. All rights reserved. + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions are +met: + + * Redistributions of source code must retain the above copyright +notice, this list of conditions and the following disclaimer. + * Redistributions in binary form must reproduce the above +copyright notice, this list of conditions and the following disclaimer +in the documentation and/or other materials provided with the +distribution. + * Neither the name of Google Inc. nor the names of its +contributors may be used to endorse or promote products derived from +this software without specific prior written permission. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS +"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT +LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR +A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT +OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, +SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT +LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, +DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY +THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE +OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. diff --git a/vendor/github.com/golang/snappy/README b/vendor/github.com/golang/snappy/README new file mode 100644 index 000000000000..5074bbab8dee --- /dev/null +++ b/vendor/github.com/golang/snappy/README @@ -0,0 +1,7 @@ +The Snappy compression format in the Go programming language. + +To download and install from source: +$ go get github.com/golang/snappy + +Unless otherwise noted, the Snappy-Go source files are distributed +under the BSD-style license found in the LICENSE file. diff --git a/vendor/github.com/golang/snappy/decode.go b/vendor/github.com/golang/snappy/decode.go new file mode 100644 index 000000000000..8bab5bdd02d1 --- /dev/null +++ b/vendor/github.com/golang/snappy/decode.go @@ -0,0 +1,294 @@ +// Copyright 2011 The Snappy-Go Authors. All rights reserved. +// Use of this source code is governed by a BSD-style +// license that can be found in the LICENSE file. + +package snappy + +import ( + "encoding/binary" + "errors" + "io" +) + +var ( + // ErrCorrupt reports that the input is invalid. + ErrCorrupt = errors.New("snappy: corrupt input") + // ErrTooLarge reports that the uncompressed length is too large. + ErrTooLarge = errors.New("snappy: decoded block is too large") + // ErrUnsupported reports that the input isn't supported. + ErrUnsupported = errors.New("snappy: unsupported input") +) + +// DecodedLen returns the length of the decoded block. +func DecodedLen(src []byte) (int, error) { + v, _, err := decodedLen(src) + return v, err +} + +// decodedLen returns the length of the decoded block and the number of bytes +// that the length header occupied. +func decodedLen(src []byte) (blockLen, headerLen int, err error) { + v, n := binary.Uvarint(src) + if n <= 0 || v > 0xffffffff { + return 0, 0, ErrCorrupt + } + + const wordSize = 32 << (^uint(0) >> 32 & 1) + if wordSize == 32 && v > 0x7fffffff { + return 0, 0, ErrTooLarge + } + return int(v), n, nil +} + +// Decode returns the decoded form of src. The returned slice may be a sub- +// slice of dst if dst was large enough to hold the entire decoded block. +// Otherwise, a newly allocated slice will be returned. +// It is valid to pass a nil dst. +func Decode(dst, src []byte) ([]byte, error) { + dLen, s, err := decodedLen(src) + if err != nil { + return nil, err + } + if len(dst) < dLen { + dst = make([]byte, dLen) + } + + var d, offset, length int + for s < len(src) { + switch src[s] & 0x03 { + case tagLiteral: + x := uint(src[s] >> 2) + switch { + case x < 60: + s++ + case x == 60: + s += 2 + if s > len(src) { + return nil, ErrCorrupt + } + x = uint(src[s-1]) + case x == 61: + s += 3 + if s > len(src) { + return nil, ErrCorrupt + } + x = uint(src[s-2]) | uint(src[s-1])<<8 + case x == 62: + s += 4 + if s > len(src) { + return nil, ErrCorrupt + } + x = uint(src[s-3]) | uint(src[s-2])<<8 | uint(src[s-1])<<16 + case x == 63: + s += 5 + if s > len(src) { + return nil, ErrCorrupt + } + x = uint(src[s-4]) | uint(src[s-3])<<8 | uint(src[s-2])<<16 | uint(src[s-1])<<24 + } + length = int(x + 1) + if length <= 0 { + return nil, errors.New("snappy: unsupported literal length") + } + if length > len(dst)-d || length > len(src)-s { + return nil, ErrCorrupt + } + copy(dst[d:], src[s:s+length]) + d += length + s += length + continue + + case tagCopy1: + s += 2 + if s > len(src) { + return nil, ErrCorrupt + } + length = 4 + int(src[s-2])>>2&0x7 + offset = int(src[s-2])&0xe0<<3 | int(src[s-1]) + + case tagCopy2: + s += 3 + if s > len(src) { + return nil, ErrCorrupt + } + length = 1 + int(src[s-3])>>2 + offset = int(src[s-2]) | int(src[s-1])<<8 + + case tagCopy4: + return nil, errors.New("snappy: unsupported COPY_4 tag") + } + + end := d + length + if offset > d || end > len(dst) { + return nil, ErrCorrupt + } + for ; d < end; d++ { + dst[d] = dst[d-offset] + } + } + if d != dLen { + return nil, ErrCorrupt + } + return dst[:d], nil +} + +// NewReader returns a new Reader that decompresses from r, using the framing +// format described at +// https://github.com/google/snappy/blob/master/framing_format.txt +func NewReader(r io.Reader) *Reader { + return &Reader{ + r: r, + decoded: make([]byte, maxUncompressedChunkLen), + buf: make([]byte, MaxEncodedLen(maxUncompressedChunkLen)+checksumSize), + } +} + +// Reader is an io.Reader that can read Snappy-compressed bytes. +type Reader struct { + r io.Reader + err error + decoded []byte + buf []byte + // decoded[i:j] contains decoded bytes that have not yet been passed on. + i, j int + readHeader bool +} + +// Reset discards any buffered data, resets all state, and switches the Snappy +// reader to read from r. This permits reusing a Reader rather than allocating +// a new one. +func (r *Reader) Reset(reader io.Reader) { + r.r = reader + r.err = nil + r.i = 0 + r.j = 0 + r.readHeader = false +} + +func (r *Reader) readFull(p []byte) (ok bool) { + if _, r.err = io.ReadFull(r.r, p); r.err != nil { + if r.err == io.ErrUnexpectedEOF { + r.err = ErrCorrupt + } + return false + } + return true +} + +// Read satisfies the io.Reader interface. +func (r *Reader) Read(p []byte) (int, error) { + if r.err != nil { + return 0, r.err + } + for { + if r.i < r.j { + n := copy(p, r.decoded[r.i:r.j]) + r.i += n + return n, nil + } + if !r.readFull(r.buf[:4]) { + return 0, r.err + } + chunkType := r.buf[0] + if !r.readHeader { + if chunkType != chunkTypeStreamIdentifier { + r.err = ErrCorrupt + return 0, r.err + } + r.readHeader = true + } + chunkLen := int(r.buf[1]) | int(r.buf[2])<<8 | int(r.buf[3])<<16 + if chunkLen > len(r.buf) { + r.err = ErrUnsupported + return 0, r.err + } + + // The chunk types are specified at + // https://github.com/google/snappy/blob/master/framing_format.txt + switch chunkType { + case chunkTypeCompressedData: + // Section 4.2. Compressed data (chunk type 0x00). + if chunkLen < checksumSize { + r.err = ErrCorrupt + return 0, r.err + } + buf := r.buf[:chunkLen] + if !r.readFull(buf) { + return 0, r.err + } + checksum := uint32(buf[0]) | uint32(buf[1])<<8 | uint32(buf[2])<<16 | uint32(buf[3])<<24 + buf = buf[checksumSize:] + + n, err := DecodedLen(buf) + if err != nil { + r.err = err + return 0, r.err + } + if n > len(r.decoded) { + r.err = ErrCorrupt + return 0, r.err + } + if _, err := Decode(r.decoded, buf); err != nil { + r.err = err + return 0, r.err + } + if crc(r.decoded[:n]) != checksum { + r.err = ErrCorrupt + return 0, r.err + } + r.i, r.j = 0, n + continue + + case chunkTypeUncompressedData: + // Section 4.3. Uncompressed data (chunk type 0x01). + if chunkLen < checksumSize { + r.err = ErrCorrupt + return 0, r.err + } + buf := r.buf[:checksumSize] + if !r.readFull(buf) { + return 0, r.err + } + checksum := uint32(buf[0]) | uint32(buf[1])<<8 | uint32(buf[2])<<16 | uint32(buf[3])<<24 + // Read directly into r.decoded instead of via r.buf. + n := chunkLen - checksumSize + if !r.readFull(r.decoded[:n]) { + return 0, r.err + } + if crc(r.decoded[:n]) != checksum { + r.err = ErrCorrupt + return 0, r.err + } + r.i, r.j = 0, n + continue + + case chunkTypeStreamIdentifier: + // Section 4.1. Stream identifier (chunk type 0xff). + if chunkLen != len(magicBody) { + r.err = ErrCorrupt + return 0, r.err + } + if !r.readFull(r.buf[:len(magicBody)]) { + return 0, r.err + } + for i := 0; i < len(magicBody); i++ { + if r.buf[i] != magicBody[i] { + r.err = ErrCorrupt + return 0, r.err + } + } + continue + } + + if chunkType <= 0x7f { + // Section 4.5. Reserved unskippable chunks (chunk types 0x02-0x7f). + r.err = ErrUnsupported + return 0, r.err + } + // Section 4.4 Padding (chunk type 0xfe). + // Section 4.6. Reserved skippable chunks (chunk types 0x80-0xfd). + if !r.readFull(r.buf[:chunkLen]) { + return 0, r.err + } + } +} diff --git a/vendor/github.com/golang/snappy/encode.go b/vendor/github.com/golang/snappy/encode.go new file mode 100644 index 000000000000..f3b5484bc7e0 --- /dev/null +++ b/vendor/github.com/golang/snappy/encode.go @@ -0,0 +1,254 @@ +// Copyright 2011 The Snappy-Go Authors. All rights reserved. +// Use of this source code is governed by a BSD-style +// license that can be found in the LICENSE file. + +package snappy + +import ( + "encoding/binary" + "io" +) + +// We limit how far copy back-references can go, the same as the C++ code. +const maxOffset = 1 << 15 + +// emitLiteral writes a literal chunk and returns the number of bytes written. +func emitLiteral(dst, lit []byte) int { + i, n := 0, uint(len(lit)-1) + switch { + case n < 60: + dst[0] = uint8(n)<<2 | tagLiteral + i = 1 + case n < 1<<8: + dst[0] = 60<<2 | tagLiteral + dst[1] = uint8(n) + i = 2 + case n < 1<<16: + dst[0] = 61<<2 | tagLiteral + dst[1] = uint8(n) + dst[2] = uint8(n >> 8) + i = 3 + case n < 1<<24: + dst[0] = 62<<2 | tagLiteral + dst[1] = uint8(n) + dst[2] = uint8(n >> 8) + dst[3] = uint8(n >> 16) + i = 4 + case int64(n) < 1<<32: + dst[0] = 63<<2 | tagLiteral + dst[1] = uint8(n) + dst[2] = uint8(n >> 8) + dst[3] = uint8(n >> 16) + dst[4] = uint8(n >> 24) + i = 5 + default: + panic("snappy: source buffer is too long") + } + if copy(dst[i:], lit) != len(lit) { + panic("snappy: destination buffer is too short") + } + return i + len(lit) +} + +// emitCopy writes a copy chunk and returns the number of bytes written. +func emitCopy(dst []byte, offset, length int) int { + i := 0 + for length > 0 { + x := length - 4 + if 0 <= x && x < 1<<3 && offset < 1<<11 { + dst[i+0] = uint8(offset>>8)&0x07<<5 | uint8(x)<<2 | tagCopy1 + dst[i+1] = uint8(offset) + i += 2 + break + } + + x = length + if x > 1<<6 { + x = 1 << 6 + } + dst[i+0] = uint8(x-1)<<2 | tagCopy2 + dst[i+1] = uint8(offset) + dst[i+2] = uint8(offset >> 8) + i += 3 + length -= x + } + return i +} + +// Encode returns the encoded form of src. The returned slice may be a sub- +// slice of dst if dst was large enough to hold the entire encoded block. +// Otherwise, a newly allocated slice will be returned. +// It is valid to pass a nil dst. +func Encode(dst, src []byte) []byte { + if n := MaxEncodedLen(len(src)); len(dst) < n { + dst = make([]byte, n) + } + + // The block starts with the varint-encoded length of the decompressed bytes. + d := binary.PutUvarint(dst, uint64(len(src))) + + // Return early if src is short. + if len(src) <= 4 { + if len(src) != 0 { + d += emitLiteral(dst[d:], src) + } + return dst[:d] + } + + // Initialize the hash table. Its size ranges from 1<<8 to 1<<14 inclusive. + const maxTableSize = 1 << 14 + shift, tableSize := uint(32-8), 1<<8 + for tableSize < maxTableSize && tableSize < len(src) { + shift-- + tableSize *= 2 + } + var table [maxTableSize]int + + // Iterate over the source bytes. + var ( + s int // The iterator position. + t int // The last position with the same hash as s. + lit int // The start position of any pending literal bytes. + ) + for s+3 < len(src) { + // Update the hash table. + b0, b1, b2, b3 := src[s], src[s+1], src[s+2], src[s+3] + h := uint32(b0) | uint32(b1)<<8 | uint32(b2)<<16 | uint32(b3)<<24 + p := &table[(h*0x1e35a7bd)>>shift] + // We need to to store values in [-1, inf) in table. To save + // some initialization time, (re)use the table's zero value + // and shift the values against this zero: add 1 on writes, + // subtract 1 on reads. + t, *p = *p-1, s+1 + // If t is invalid or src[s:s+4] differs from src[t:t+4], accumulate a literal byte. + if t < 0 || s-t >= maxOffset || b0 != src[t] || b1 != src[t+1] || b2 != src[t+2] || b3 != src[t+3] { + s++ + continue + } + // Otherwise, we have a match. First, emit any pending literal bytes. + if lit != s { + d += emitLiteral(dst[d:], src[lit:s]) + } + // Extend the match to be as long as possible. + s0 := s + s, t = s+4, t+4 + for s < len(src) && src[s] == src[t] { + s++ + t++ + } + // Emit the copied bytes. + d += emitCopy(dst[d:], s-t, s-s0) + lit = s + } + + // Emit any final pending literal bytes and return. + if lit != len(src) { + d += emitLiteral(dst[d:], src[lit:]) + } + return dst[:d] +} + +// MaxEncodedLen returns the maximum length of a snappy block, given its +// uncompressed length. +func MaxEncodedLen(srcLen int) int { + // Compressed data can be defined as: + // compressed := item* literal* + // item := literal* copy + // + // The trailing literal sequence has a space blowup of at most 62/60 + // since a literal of length 60 needs one tag byte + one extra byte + // for length information. + // + // Item blowup is trickier to measure. Suppose the "copy" op copies + // 4 bytes of data. Because of a special check in the encoding code, + // we produce a 4-byte copy only if the offset is < 65536. Therefore + // the copy op takes 3 bytes to encode, and this type of item leads + // to at most the 62/60 blowup for representing literals. + // + // Suppose the "copy" op copies 5 bytes of data. If the offset is big + // enough, it will take 5 bytes to encode the copy op. Therefore the + // worst case here is a one-byte literal followed by a five-byte copy. + // That is, 6 bytes of input turn into 7 bytes of "compressed" data. + // + // This last factor dominates the blowup, so the final estimate is: + return 32 + srcLen + srcLen/6 +} + +// NewWriter returns a new Writer that compresses to w, using the framing +// format described at +// https://github.com/google/snappy/blob/master/framing_format.txt +func NewWriter(w io.Writer) *Writer { + return &Writer{ + w: w, + enc: make([]byte, MaxEncodedLen(maxUncompressedChunkLen)), + } +} + +// Writer is an io.Writer than can write Snappy-compressed bytes. +type Writer struct { + w io.Writer + err error + enc []byte + buf [checksumSize + chunkHeaderSize]byte + wroteHeader bool +} + +// Reset discards the writer's state and switches the Snappy writer to write to +// w. This permits reusing a Writer rather than allocating a new one. +func (w *Writer) Reset(writer io.Writer) { + w.w = writer + w.err = nil + w.wroteHeader = false +} + +// Write satisfies the io.Writer interface. +func (w *Writer) Write(p []byte) (n int, errRet error) { + if w.err != nil { + return 0, w.err + } + if !w.wroteHeader { + copy(w.enc, magicChunk) + if _, err := w.w.Write(w.enc[:len(magicChunk)]); err != nil { + w.err = err + return n, err + } + w.wroteHeader = true + } + for len(p) > 0 { + var uncompressed []byte + if len(p) > maxUncompressedChunkLen { + uncompressed, p = p[:maxUncompressedChunkLen], p[maxUncompressedChunkLen:] + } else { + uncompressed, p = p, nil + } + checksum := crc(uncompressed) + + // Compress the buffer, discarding the result if the improvement + // isn't at least 12.5%. + chunkType := uint8(chunkTypeCompressedData) + chunkBody := Encode(w.enc, uncompressed) + if len(chunkBody) >= len(uncompressed)-len(uncompressed)/8 { + chunkType, chunkBody = chunkTypeUncompressedData, uncompressed + } + + chunkLen := 4 + len(chunkBody) + w.buf[0] = chunkType + w.buf[1] = uint8(chunkLen >> 0) + w.buf[2] = uint8(chunkLen >> 8) + w.buf[3] = uint8(chunkLen >> 16) + w.buf[4] = uint8(checksum >> 0) + w.buf[5] = uint8(checksum >> 8) + w.buf[6] = uint8(checksum >> 16) + w.buf[7] = uint8(checksum >> 24) + if _, err := w.w.Write(w.buf[:]); err != nil { + w.err = err + return n, err + } + if _, err := w.w.Write(chunkBody); err != nil { + w.err = err + return n, err + } + n += len(uncompressed) + } + return n, nil +} diff --git a/vendor/github.com/golang/snappy/snappy.go b/vendor/github.com/golang/snappy/snappy.go new file mode 100644 index 000000000000..15af18d0a52f --- /dev/null +++ b/vendor/github.com/golang/snappy/snappy.go @@ -0,0 +1,68 @@ +// Copyright 2011 The Snappy-Go Authors. All rights reserved. +// Use of this source code is governed by a BSD-style +// license that can be found in the LICENSE file. + +// Package snappy implements the snappy block-based compression format. +// It aims for very high speeds and reasonable compression. +// +// The C++ snappy implementation is at https://github.com/google/snappy +package snappy // import "github.com/golang/snappy" + +import ( + "hash/crc32" +) + +/* +Each encoded block begins with the varint-encoded length of the decoded data, +followed by a sequence of chunks. Chunks begin and end on byte boundaries. The +first byte of each chunk is broken into its 2 least and 6 most significant bits +called l and m: l ranges in [0, 4) and m ranges in [0, 64). l is the chunk tag. +Zero means a literal tag. All other values mean a copy tag. + +For literal tags: + - If m < 60, the next 1 + m bytes are literal bytes. + - Otherwise, let n be the little-endian unsigned integer denoted by the next + m - 59 bytes. The next 1 + n bytes after that are literal bytes. + +For copy tags, length bytes are copied from offset bytes ago, in the style of +Lempel-Ziv compression algorithms. In particular: + - For l == 1, the offset ranges in [0, 1<<11) and the length in [4, 12). + The length is 4 + the low 3 bits of m. The high 3 bits of m form bits 8-10 + of the offset. The next byte is bits 0-7 of the offset. + - For l == 2, the offset ranges in [0, 1<<16) and the length in [1, 65). + The length is 1 + m. The offset is the little-endian unsigned integer + denoted by the next 2 bytes. + - For l == 3, this tag is a legacy format that is no longer supported. +*/ +const ( + tagLiteral = 0x00 + tagCopy1 = 0x01 + tagCopy2 = 0x02 + tagCopy4 = 0x03 +) + +const ( + checksumSize = 4 + chunkHeaderSize = 4 + magicChunk = "\xff\x06\x00\x00" + magicBody + magicBody = "sNaPpY" + // https://github.com/google/snappy/blob/master/framing_format.txt says + // that "the uncompressed data in a chunk must be no longer than 65536 bytes". + maxUncompressedChunkLen = 65536 +) + +const ( + chunkTypeCompressedData = 0x00 + chunkTypeUncompressedData = 0x01 + chunkTypePadding = 0xfe + chunkTypeStreamIdentifier = 0xff +) + +var crcTable = crc32.MakeTable(crc32.Castagnoli) + +// crc implements the checksum specified in section 3 of +// https://github.com/google/snappy/blob/master/framing_format.txt +func crc(b []byte) uint32 { + c := crc32.Update(0, crcTable, b) + return uint32(c>>15|c<<17) + 0xa282ead8 +} diff --git a/vendor/github.com/golang/snappy/snappy_test.go b/vendor/github.com/golang/snappy/snappy_test.go new file mode 100644 index 000000000000..efa046eedbea --- /dev/null +++ b/vendor/github.com/golang/snappy/snappy_test.go @@ -0,0 +1,386 @@ +// Copyright 2011 The Snappy-Go Authors. All rights reserved. +// Use of this source code is governed by a BSD-style +// license that can be found in the LICENSE file. + +package snappy + +import ( + "bytes" + "flag" + "fmt" + "io" + "io/ioutil" + "math/rand" + "net/http" + "os" + "path/filepath" + "strings" + "testing" +) + +var ( + download = flag.Bool("download", false, "If true, download any missing files before running benchmarks") + testdata = flag.String("testdata", "testdata", "Directory containing the test data") +) + +func roundtrip(b, ebuf, dbuf []byte) error { + d, err := Decode(dbuf, Encode(ebuf, b)) + if err != nil { + return fmt.Errorf("decoding error: %v", err) + } + if !bytes.Equal(b, d) { + return fmt.Errorf("roundtrip mismatch:\n\twant %v\n\tgot %v", b, d) + } + return nil +} + +func TestEmpty(t *testing.T) { + if err := roundtrip(nil, nil, nil); err != nil { + t.Fatal(err) + } +} + +func TestSmallCopy(t *testing.T) { + for _, ebuf := range [][]byte{nil, make([]byte, 20), make([]byte, 64)} { + for _, dbuf := range [][]byte{nil, make([]byte, 20), make([]byte, 64)} { + for i := 0; i < 32; i++ { + s := "aaaa" + strings.Repeat("b", i) + "aaaabbbb" + if err := roundtrip([]byte(s), ebuf, dbuf); err != nil { + t.Errorf("len(ebuf)=%d, len(dbuf)=%d, i=%d: %v", len(ebuf), len(dbuf), i, err) + } + } + } + } +} + +func TestSmallRand(t *testing.T) { + rng := rand.New(rand.NewSource(1)) + for n := 1; n < 20000; n += 23 { + b := make([]byte, n) + for i := range b { + b[i] = uint8(rng.Uint32()) + } + if err := roundtrip(b, nil, nil); err != nil { + t.Fatal(err) + } + } +} + +func TestSmallRegular(t *testing.T) { + for n := 1; n < 20000; n += 23 { + b := make([]byte, n) + for i := range b { + b[i] = uint8(i%10 + 'a') + } + if err := roundtrip(b, nil, nil); err != nil { + t.Fatal(err) + } + } +} + +func TestInvalidVarint(t *testing.T) { + data := []byte("\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\x00") + if _, err := DecodedLen(data); err != ErrCorrupt { + t.Errorf("DecodedLen: got %v, want ErrCorrupt", err) + } + if _, err := Decode(nil, data); err != ErrCorrupt { + t.Errorf("Decode: got %v, want ErrCorrupt", err) + } + + // The encoded varint overflows 32 bits + data = []byte("\xff\xff\xff\xff\xff\x00") + + if _, err := DecodedLen(data); err != ErrCorrupt { + t.Errorf("DecodedLen: got %v, want ErrCorrupt", err) + } + if _, err := Decode(nil, data); err != ErrCorrupt { + t.Errorf("Decode: got %v, want ErrCorrupt", err) + } +} + +func cmp(a, b []byte) error { + if len(a) != len(b) { + return fmt.Errorf("got %d bytes, want %d", len(a), len(b)) + } + for i := range a { + if a[i] != b[i] { + return fmt.Errorf("byte #%d: got 0x%02x, want 0x%02x", i, a[i], b[i]) + } + } + return nil +} + +func TestFramingFormat(t *testing.T) { + // src is comprised of alternating 1e5-sized sequences of random + // (incompressible) bytes and repeated (compressible) bytes. 1e5 was chosen + // because it is larger than maxUncompressedChunkLen (64k). + src := make([]byte, 1e6) + rng := rand.New(rand.NewSource(1)) + for i := 0; i < 10; i++ { + if i%2 == 0 { + for j := 0; j < 1e5; j++ { + src[1e5*i+j] = uint8(rng.Intn(256)) + } + } else { + for j := 0; j < 1e5; j++ { + src[1e5*i+j] = uint8(i) + } + } + } + + buf := new(bytes.Buffer) + if _, err := NewWriter(buf).Write(src); err != nil { + t.Fatalf("Write: encoding: %v", err) + } + dst, err := ioutil.ReadAll(NewReader(buf)) + if err != nil { + t.Fatalf("ReadAll: decoding: %v", err) + } + if err := cmp(dst, src); err != nil { + t.Fatal(err) + } +} + +func TestReaderReset(t *testing.T) { + gold := bytes.Repeat([]byte("All that is gold does not glitter,\n"), 10000) + buf := new(bytes.Buffer) + if _, err := NewWriter(buf).Write(gold); err != nil { + t.Fatalf("Write: %v", err) + } + encoded, invalid, partial := buf.String(), "invalid", "partial" + r := NewReader(nil) + for i, s := range []string{encoded, invalid, partial, encoded, partial, invalid, encoded, encoded} { + if s == partial { + r.Reset(strings.NewReader(encoded)) + if _, err := r.Read(make([]byte, 101)); err != nil { + t.Errorf("#%d: %v", i, err) + continue + } + continue + } + r.Reset(strings.NewReader(s)) + got, err := ioutil.ReadAll(r) + switch s { + case encoded: + if err != nil { + t.Errorf("#%d: %v", i, err) + continue + } + if err := cmp(got, gold); err != nil { + t.Errorf("#%d: %v", i, err) + continue + } + case invalid: + if err == nil { + t.Errorf("#%d: got nil error, want non-nil", i) + continue + } + } + } +} + +func TestWriterReset(t *testing.T) { + gold := bytes.Repeat([]byte("Not all those who wander are lost;\n"), 10000) + var gots, wants [][]byte + const n = 20 + w, failed := NewWriter(nil), false + for i := 0; i <= n; i++ { + buf := new(bytes.Buffer) + w.Reset(buf) + want := gold[:len(gold)*i/n] + if _, err := w.Write(want); err != nil { + t.Errorf("#%d: Write: %v", i, err) + failed = true + continue + } + got, err := ioutil.ReadAll(NewReader(buf)) + if err != nil { + t.Errorf("#%d: ReadAll: %v", i, err) + failed = true + continue + } + gots = append(gots, got) + wants = append(wants, want) + } + if failed { + return + } + for i := range gots { + if err := cmp(gots[i], wants[i]); err != nil { + t.Errorf("#%d: %v", i, err) + } + } +} + +func benchDecode(b *testing.B, src []byte) { + encoded := Encode(nil, src) + // Bandwidth is in amount of uncompressed data. + b.SetBytes(int64(len(src))) + b.ResetTimer() + for i := 0; i < b.N; i++ { + Decode(src, encoded) + } +} + +func benchEncode(b *testing.B, src []byte) { + // Bandwidth is in amount of uncompressed data. + b.SetBytes(int64(len(src))) + dst := make([]byte, MaxEncodedLen(len(src))) + b.ResetTimer() + for i := 0; i < b.N; i++ { + Encode(dst, src) + } +} + +func readFile(b testing.TB, filename string) []byte { + src, err := ioutil.ReadFile(filename) + if err != nil { + b.Skipf("skipping benchmark: %v", err) + } + if len(src) == 0 { + b.Fatalf("%s has zero length", filename) + } + return src +} + +// expand returns a slice of length n containing repeated copies of src. +func expand(src []byte, n int) []byte { + dst := make([]byte, n) + for x := dst; len(x) > 0; { + i := copy(x, src) + x = x[i:] + } + return dst +} + +func benchWords(b *testing.B, n int, decode bool) { + // Note: the file is OS-language dependent so the resulting values are not + // directly comparable for non-US-English OS installations. + data := expand(readFile(b, "/usr/share/dict/words"), n) + if decode { + benchDecode(b, data) + } else { + benchEncode(b, data) + } +} + +func BenchmarkWordsDecode1e3(b *testing.B) { benchWords(b, 1e3, true) } +func BenchmarkWordsDecode1e4(b *testing.B) { benchWords(b, 1e4, true) } +func BenchmarkWordsDecode1e5(b *testing.B) { benchWords(b, 1e5, true) } +func BenchmarkWordsDecode1e6(b *testing.B) { benchWords(b, 1e6, true) } +func BenchmarkWordsEncode1e3(b *testing.B) { benchWords(b, 1e3, false) } +func BenchmarkWordsEncode1e4(b *testing.B) { benchWords(b, 1e4, false) } +func BenchmarkWordsEncode1e5(b *testing.B) { benchWords(b, 1e5, false) } +func BenchmarkWordsEncode1e6(b *testing.B) { benchWords(b, 1e6, false) } + +func BenchmarkRandomEncode(b *testing.B) { + rng := rand.New(rand.NewSource(1)) + data := make([]byte, 1<<20) + for i := range data { + data[i] = uint8(rng.Intn(256)) + } + benchEncode(b, data) +} + +// testFiles' values are copied directly from +// https://raw.githubusercontent.com/google/snappy/master/snappy_unittest.cc +// The label field is unused in snappy-go. +var testFiles = []struct { + label string + filename string +}{ + {"html", "html"}, + {"urls", "urls.10K"}, + {"jpg", "fireworks.jpeg"}, + {"jpg_200", "fireworks.jpeg"}, + {"pdf", "paper-100k.pdf"}, + {"html4", "html_x_4"}, + {"txt1", "alice29.txt"}, + {"txt2", "asyoulik.txt"}, + {"txt3", "lcet10.txt"}, + {"txt4", "plrabn12.txt"}, + {"pb", "geo.protodata"}, + {"gaviota", "kppkn.gtb"}, +} + +// The test data files are present at this canonical URL. +const baseURL = "https://raw.githubusercontent.com/google/snappy/master/testdata/" + +func downloadTestdata(b *testing.B, basename string) (errRet error) { + filename := filepath.Join(*testdata, basename) + if stat, err := os.Stat(filename); err == nil && stat.Size() != 0 { + return nil + } + + if !*download { + b.Skipf("test data not found; skipping benchmark without the -download flag") + } + // Download the official snappy C++ implementation reference test data + // files for benchmarking. + if err := os.Mkdir(*testdata, 0777); err != nil && !os.IsExist(err) { + return fmt.Errorf("failed to create testdata: %s", err) + } + + f, err := os.Create(filename) + if err != nil { + return fmt.Errorf("failed to create %s: %s", filename, err) + } + defer f.Close() + defer func() { + if errRet != nil { + os.Remove(filename) + } + }() + url := baseURL + basename + resp, err := http.Get(url) + if err != nil { + return fmt.Errorf("failed to download %s: %s", url, err) + } + defer resp.Body.Close() + if s := resp.StatusCode; s != http.StatusOK { + return fmt.Errorf("downloading %s: HTTP status code %d (%s)", url, s, http.StatusText(s)) + } + _, err = io.Copy(f, resp.Body) + if err != nil { + return fmt.Errorf("failed to download %s to %s: %s", url, filename, err) + } + return nil +} + +func benchFile(b *testing.B, n int, decode bool) { + if err := downloadTestdata(b, testFiles[n].filename); err != nil { + b.Fatalf("failed to download testdata: %s", err) + } + data := readFile(b, filepath.Join(*testdata, testFiles[n].filename)) + if decode { + benchDecode(b, data) + } else { + benchEncode(b, data) + } +} + +// Naming convention is kept similar to what snappy's C++ implementation uses. +func Benchmark_UFlat0(b *testing.B) { benchFile(b, 0, true) } +func Benchmark_UFlat1(b *testing.B) { benchFile(b, 1, true) } +func Benchmark_UFlat2(b *testing.B) { benchFile(b, 2, true) } +func Benchmark_UFlat3(b *testing.B) { benchFile(b, 3, true) } +func Benchmark_UFlat4(b *testing.B) { benchFile(b, 4, true) } +func Benchmark_UFlat5(b *testing.B) { benchFile(b, 5, true) } +func Benchmark_UFlat6(b *testing.B) { benchFile(b, 6, true) } +func Benchmark_UFlat7(b *testing.B) { benchFile(b, 7, true) } +func Benchmark_UFlat8(b *testing.B) { benchFile(b, 8, true) } +func Benchmark_UFlat9(b *testing.B) { benchFile(b, 9, true) } +func Benchmark_UFlat10(b *testing.B) { benchFile(b, 10, true) } +func Benchmark_UFlat11(b *testing.B) { benchFile(b, 11, true) } +func Benchmark_ZFlat0(b *testing.B) { benchFile(b, 0, false) } +func Benchmark_ZFlat1(b *testing.B) { benchFile(b, 1, false) } +func Benchmark_ZFlat2(b *testing.B) { benchFile(b, 2, false) } +func Benchmark_ZFlat3(b *testing.B) { benchFile(b, 3, false) } +func Benchmark_ZFlat4(b *testing.B) { benchFile(b, 4, false) } +func Benchmark_ZFlat5(b *testing.B) { benchFile(b, 5, false) } +func Benchmark_ZFlat6(b *testing.B) { benchFile(b, 6, false) } +func Benchmark_ZFlat7(b *testing.B) { benchFile(b, 7, false) } +func Benchmark_ZFlat8(b *testing.B) { benchFile(b, 8, false) } +func Benchmark_ZFlat9(b *testing.B) { benchFile(b, 9, false) } +func Benchmark_ZFlat10(b *testing.B) { benchFile(b, 10, false) } +func Benchmark_ZFlat11(b *testing.B) { benchFile(b, 11, false) } diff --git a/vendor/github.com/klauspost/crc32/.gitignore b/vendor/github.com/klauspost/crc32/.gitignore new file mode 100644 index 000000000000..daf913b1b347 --- /dev/null +++ b/vendor/github.com/klauspost/crc32/.gitignore @@ -0,0 +1,24 @@ +# Compiled Object files, Static and Dynamic libs (Shared Objects) +*.o +*.a +*.so + +# Folders +_obj +_test + +# Architecture specific extensions/prefixes +*.[568vq] +[568vq].out + +*.cgo1.go +*.cgo2.c +_cgo_defun.c +_cgo_gotypes.go +_cgo_export.* + +_testmain.go + +*.exe +*.test +*.prof diff --git a/vendor/github.com/klauspost/crc32/.travis.yml b/vendor/github.com/klauspost/crc32/.travis.yml new file mode 100644 index 000000000000..c62e25f5a5b4 --- /dev/null +++ b/vendor/github.com/klauspost/crc32/.travis.yml @@ -0,0 +1,11 @@ +language: go + +go: + - 1.3 + - 1.4 + - 1.5 + - tip + +script: + - go test -v . + - go test -v -race . diff --git a/vendor/github.com/klauspost/crc32/LICENSE b/vendor/github.com/klauspost/crc32/LICENSE new file mode 100644 index 000000000000..4fd5963e39c1 --- /dev/null +++ b/vendor/github.com/klauspost/crc32/LICENSE @@ -0,0 +1,28 @@ +Copyright (c) 2012 The Go Authors. All rights reserved. +Copyright (c) 2015 Klaus Post + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions are +met: + + * Redistributions of source code must retain the above copyright +notice, this list of conditions and the following disclaimer. + * Redistributions in binary form must reproduce the above +copyright notice, this list of conditions and the following disclaimer +in the documentation and/or other materials provided with the +distribution. + * Neither the name of Google Inc. nor the names of its +contributors may be used to endorse or promote products derived from +this software without specific prior written permission. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS +"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT +LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR +A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT +OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, +SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT +LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, +DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY +THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE +OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. diff --git a/vendor/github.com/klauspost/crc32/README.md b/vendor/github.com/klauspost/crc32/README.md new file mode 100644 index 000000000000..440541c7ff31 --- /dev/null +++ b/vendor/github.com/klauspost/crc32/README.md @@ -0,0 +1,84 @@ +# crc32 +CRC32 hash with x64 optimizations + +This package is a drop-in replacement for the standard library `hash/crc32` package, that features SSE 4.2 optimizations on x64 platforms, for a 10x speedup. + +[![Build Status](https://travis-ci.org/klauspost/crc32.svg?branch=master)](https://travis-ci.org/klauspost/crc32) + +# usage + +Install using `go get github.com/klauspost/crc32`. This library is based on Go 1.5 code and requires Go 1.3 or newer. + +Replace `import "hash/crc32"` with `import "github.com/klauspost/crc32"` and you are good to go. + +# changes + +* Dec 4, 2015: Uses the "slice-by-8" trick more extensively, which gives a 1.5 to 2.5x speedup if assembler is unavailable. + + +# performance + +For IEEE tables (the most common), there is approximately a factor 10 speedup with "CLMUL" (Carryless multiplication) instruction: +``` +benchmark old ns/op new ns/op delta +BenchmarkCrc32KB 99955 10258 -89.74% + +benchmark old MB/s new MB/s speedup +BenchmarkCrc32KB 327.83 3194.20 9.74x +``` + +For other tables and "CLMUL" capable machines the performance is the same as the standard library. + +Here are some detailed benchmarks, comparing to go 1.5 standard library with and without assembler enabled. + +``` +Std: Standard Go 1.5 library +Crc: Indicates IEEE type CRC. +40B: Size of each slice encoded. +NoAsm: Assembler was disabled (ie. not an AMD64 or SSE 4.2+ capable machine). +Castagnoli: Castagnoli CRC type. + +BenchmarkStdCrc40B-4 10000000 158 ns/op 252.88 MB/s +BenchmarkCrc40BNoAsm-4 20000000 105 ns/op 377.38 MB/s (slice8) +BenchmarkCrc40B-4 20000000 105 ns/op 378.77 MB/s (slice8) + +BenchmarkStdCrc1KB-4 500000 3604 ns/op 284.10 MB/s +BenchmarkCrc1KBNoAsm-4 1000000 1463 ns/op 699.79 MB/s (slice8) +BenchmarkCrc1KB-4 3000000 396 ns/op 2583.69 MB/s (asm) + +BenchmarkStdCrc8KB-4 200000 11417 ns/op 717.48 MB/s (slice8) +BenchmarkCrc8KBNoAsm-4 200000 11317 ns/op 723.85 MB/s (slice8) +BenchmarkCrc8KB-4 500000 2919 ns/op 2805.73 MB/s (asm) + +BenchmarkStdCrc32KB-4 30000 45749 ns/op 716.24 MB/s (slice8) +BenchmarkCrc32KBNoAsm-4 30000 45109 ns/op 726.42 MB/s (slice8) +BenchmarkCrc32KB-4 100000 11497 ns/op 2850.09 MB/s (asm) + +BenchmarkStdNoAsmCastagnol40B-4 10000000 161 ns/op 246.94 MB/s +BenchmarkStdCastagnoli40B-4 50000000 28.4 ns/op 1410.69 MB/s (asm) +BenchmarkCastagnoli40BNoAsm-4 20000000 100 ns/op 398.01 MB/s (slice8) +BenchmarkCastagnoli40B-4 50000000 28.2 ns/op 1419.54 MB/s (asm) + +BenchmarkStdNoAsmCastagnoli1KB-4 500000 3622 ns/op 282.67 MB/s +BenchmarkStdCastagnoli1KB-4 10000000 144 ns/op 7099.78 MB/s (asm) +BenchmarkCastagnoli1KBNoAsm-4 1000000 1475 ns/op 694.14 MB/s (slice8) +BenchmarkCastagnoli1KB-4 10000000 146 ns/op 6993.35 MB/s (asm) + +BenchmarkStdNoAsmCastagnoli8KB-4 50000 28781 ns/op 284.63 MB/s +BenchmarkStdCastagnoli8KB-4 1000000 1029 ns/op 7957.89 MB/s (asm) +BenchmarkCastagnoli8KBNoAsm-4 200000 11410 ns/op 717.94 MB/s (slice8) +BenchmarkCastagnoli8KB-4 1000000 1000 ns/op 8188.71 MB/s (asm) + +BenchmarkStdNoAsmCastagnoli32KB-4 10000 115426 ns/op 283.89 MB/s +BenchmarkStdCastagnoli32KB-4 300000 4065 ns/op 8059.13 MB/s (asm) +BenchmarkCastagnoli32KBNoAsm-4 30000 45171 ns/op 725.41 MB/s (slice8) +BenchmarkCastagnoli32KB-4 500000 4077 ns/op 8035.89 MB/s (asm) +``` + +The IEEE assembler optimizations has been submitted and will be part of the Go 1.6 standard library. + +However, the improved use of slice-by-8 has not, but will probably be submitted for Go 1.7. + +# license + +Standard Go license. Changes are Copyright (c) 2015 Klaus Post under same conditions. diff --git a/vendor/github.com/klauspost/crc32/crc32.go b/vendor/github.com/klauspost/crc32/crc32.go new file mode 100644 index 000000000000..8d6ba5d3dc57 --- /dev/null +++ b/vendor/github.com/klauspost/crc32/crc32.go @@ -0,0 +1,186 @@ +// Copyright 2009 The Go Authors. All rights reserved. +// Use of this source code is governed by a BSD-style +// license that can be found in the LICENSE file. + +// Package crc32 implements the 32-bit cyclic redundancy check, or CRC-32, +// checksum. See http://en.wikipedia.org/wiki/Cyclic_redundancy_check for +// information. +// +// Polynomials are represented in LSB-first form also known as reversed representation. +// +// See http://en.wikipedia.org/wiki/Mathematics_of_cyclic_redundancy_checks#Reversed_representations_and_reciprocal_polynomials +// for information. +package crc32 + +import ( + "hash" + "sync" +) + +// The size of a CRC-32 checksum in bytes. +const Size = 4 + +// Predefined polynomials. +const ( + // IEEE is by far and away the most common CRC-32 polynomial. + // Used by ethernet (IEEE 802.3), v.42, fddi, gzip, zip, png, ... + IEEE = 0xedb88320 + + // Castagnoli's polynomial, used in iSCSI. + // Has better error detection characteristics than IEEE. + // http://dx.doi.org/10.1109/26.231911 + Castagnoli = 0x82f63b78 + + // Koopman's polynomial. + // Also has better error detection characteristics than IEEE. + // http://dx.doi.org/10.1109/DSN.2002.1028931 + Koopman = 0xeb31d82e +) + +// Table is a 256-word table representing the polynomial for efficient processing. +type Table [256]uint32 + +// castagnoliTable points to a lazily initialized Table for the Castagnoli +// polynomial. MakeTable will always return this value when asked to make a +// Castagnoli table so we can compare against it to find when the caller is +// using this polynomial. +var castagnoliTable *Table +var castagnoliTable8 *slicing8Table +var castagnoliOnce sync.Once + +func castagnoliInit() { + castagnoliTable = makeTable(Castagnoli) + castagnoliTable8 = makeTable8(Castagnoli) +} + +// IEEETable is the table for the IEEE polynomial. +var IEEETable = makeTable(IEEE) + +// slicing8Table is array of 8 Tables +type slicing8Table [8]Table + +// ieeeTable8 is the slicing8Table for IEEE +var ieeeTable8 *slicing8Table +var ieeeTable8Once sync.Once + +// MakeTable returns a Table constructed from the specified polynomial. +// The contents of this Table must not be modified. +func MakeTable(poly uint32) *Table { + switch poly { + case IEEE: + return IEEETable + case Castagnoli: + castagnoliOnce.Do(castagnoliInit) + return castagnoliTable + } + return makeTable(poly) +} + +// makeTable returns the Table constructed from the specified polynomial. +func makeTable(poly uint32) *Table { + t := new(Table) + for i := 0; i < 256; i++ { + crc := uint32(i) + for j := 0; j < 8; j++ { + if crc&1 == 1 { + crc = (crc >> 1) ^ poly + } else { + crc >>= 1 + } + } + t[i] = crc + } + return t +} + +// makeTable8 returns slicing8Table constructed from the specified polynomial. +func makeTable8(poly uint32) *slicing8Table { + t := new(slicing8Table) + t[0] = *makeTable(poly) + for i := 0; i < 256; i++ { + crc := t[0][i] + for j := 1; j < 8; j++ { + crc = t[0][crc&0xFF] ^ (crc >> 8) + t[j][i] = crc + } + } + return t +} + +// digest represents the partial evaluation of a checksum. +type digest struct { + crc uint32 + tab *Table +} + +// New creates a new hash.Hash32 computing the CRC-32 checksum +// using the polynomial represented by the Table. +// Its Sum method will lay the value out in big-endian byte order. +func New(tab *Table) hash.Hash32 { return &digest{0, tab} } + +// NewIEEE creates a new hash.Hash32 computing the CRC-32 checksum +// using the IEEE polynomial. +// Its Sum method will lay the value out in big-endian byte order. +func NewIEEE() hash.Hash32 { return New(IEEETable) } + +func (d *digest) Size() int { return Size } + +func (d *digest) BlockSize() int { return 1 } + +func (d *digest) Reset() { d.crc = 0 } + +func update(crc uint32, tab *Table, p []byte) uint32 { + crc = ^crc + for _, v := range p { + crc = tab[byte(crc)^v] ^ (crc >> 8) + } + return ^crc +} + +// updateSlicingBy8 updates CRC using Slicing-by-8 +func updateSlicingBy8(crc uint32, tab *slicing8Table, p []byte) uint32 { + crc = ^crc + for len(p) > 8 { + crc ^= uint32(p[0]) | uint32(p[1])<<8 | uint32(p[2])<<16 | uint32(p[3])<<24 + crc = tab[0][p[7]] ^ tab[1][p[6]] ^ tab[2][p[5]] ^ tab[3][p[4]] ^ + tab[4][crc>>24] ^ tab[5][(crc>>16)&0xFF] ^ + tab[6][(crc>>8)&0xFF] ^ tab[7][crc&0xFF] + p = p[8:] + } + crc = ^crc + if len(p) == 0 { + return crc + } + return update(crc, &tab[0], p) +} + +// Update returns the result of adding the bytes in p to the crc. +func Update(crc uint32, tab *Table, p []byte) uint32 { + if tab == castagnoliTable { + return updateCastagnoli(crc, p) + } + if tab == IEEETable { + return updateIEEE(crc, p) + } + return update(crc, tab, p) +} + +func (d *digest) Write(p []byte) (n int, err error) { + d.crc = Update(d.crc, d.tab, p) + return len(p), nil +} + +func (d *digest) Sum32() uint32 { return d.crc } + +func (d *digest) Sum(in []byte) []byte { + s := d.Sum32() + return append(in, byte(s>>24), byte(s>>16), byte(s>>8), byte(s)) +} + +// Checksum returns the CRC-32 checksum of data +// using the polynomial represented by the Table. +func Checksum(data []byte, tab *Table) uint32 { return Update(0, tab, data) } + +// ChecksumIEEE returns the CRC-32 checksum of data +// using the IEEE polynomial. +func ChecksumIEEE(data []byte) uint32 { return updateIEEE(0, data) } diff --git a/vendor/github.com/klauspost/crc32/crc32_amd64.go b/vendor/github.com/klauspost/crc32/crc32_amd64.go new file mode 100644 index 000000000000..4827128ea04a --- /dev/null +++ b/vendor/github.com/klauspost/crc32/crc32_amd64.go @@ -0,0 +1,62 @@ +// Copyright 2011 The Go Authors. All rights reserved. +// Use of this source code is governed by a BSD-style +// license that can be found in the LICENSE file. + +// +build !appengine,!gccgo + +package crc32 + +// This file contains the code to call the SSE 4.2 version of the Castagnoli +// and IEEE CRC. + +// haveSSE41/haveSSE42/haveCLMUL are defined in crc_amd64.s and use +// CPUID to test for SSE 4.1, 4.2 and CLMUL support. +func haveSSE41() bool +func haveSSE42() bool +func haveCLMUL() bool + +// castagnoliSSE42 is defined in crc_amd64.s and uses the SSE4.2 CRC32 +// instruction. +//go:noescape +func castagnoliSSE42(crc uint32, p []byte) uint32 + +// ieeeCLMUL is defined in crc_amd64.s and uses the PCLMULQDQ +// instruction as well as SSE 4.1. +//go:noescape +func ieeeCLMUL(crc uint32, p []byte) uint32 + +var sse42 = haveSSE42() +var useFastIEEE = haveCLMUL() && haveSSE41() + +func updateCastagnoli(crc uint32, p []byte) uint32 { + if sse42 { + return castagnoliSSE42(crc, p) + } + // only use slicing-by-8 when input is >= 16 Bytes + if len(p) >= 16 { + return updateSlicingBy8(crc, castagnoliTable8, p) + } + return update(crc, castagnoliTable, p) +} + +func updateIEEE(crc uint32, p []byte) uint32 { + if useFastIEEE && len(p) >= 64 { + left := len(p) & 15 + do := len(p) - left + crc = ^ieeeCLMUL(^crc, p[:do]) + if left > 0 { + crc = update(crc, IEEETable, p[do:]) + } + return crc + } + + // only use slicing-by-8 when input is >= 16 Bytes + if len(p) >= 16 { + ieeeTable8Once.Do(func() { + ieeeTable8 = makeTable8(IEEE) + }) + return updateSlicingBy8(crc, ieeeTable8, p) + } + + return update(crc, IEEETable, p) +} diff --git a/vendor/github.com/klauspost/crc32/crc32_amd64.s b/vendor/github.com/klauspost/crc32/crc32_amd64.s new file mode 100644 index 000000000000..ca73c73256df --- /dev/null +++ b/vendor/github.com/klauspost/crc32/crc32_amd64.s @@ -0,0 +1,237 @@ +// Copyright 2011 The Go Authors. All rights reserved. +// Use of this source code is governed by a BSD-style +// license that can be found in the LICENSE file. + +// +build gc + +#define NOSPLIT 4 +#define RODATA 8 + +// func castagnoliSSE42(crc uint32, p []byte) uint32 +TEXT ·castagnoliSSE42(SB), NOSPLIT, $0 + MOVL crc+0(FP), AX // CRC value + MOVQ p+8(FP), SI // data pointer + MOVQ p_len+16(FP), CX // len(p) + + NOTL AX + + // If there's less than 8 bytes to process, we do it byte-by-byte. + CMPQ CX, $8 + JL cleanup + + // Process individual bytes until the input is 8-byte aligned. +startup: + MOVQ SI, BX + ANDQ $7, BX + JZ aligned + + CRC32B (SI), AX + DECQ CX + INCQ SI + JMP startup + +aligned: + // The input is now 8-byte aligned and we can process 8-byte chunks. + CMPQ CX, $8 + JL cleanup + + CRC32Q (SI), AX + ADDQ $8, SI + SUBQ $8, CX + JMP aligned + +cleanup: + // We may have some bytes left over that we process one at a time. + CMPQ CX, $0 + JE done + + CRC32B (SI), AX + INCQ SI + DECQ CX + JMP cleanup + +done: + NOTL AX + MOVL AX, ret+32(FP) + RET + +// func haveSSE42() bool +TEXT ·haveSSE42(SB), NOSPLIT, $0 + XORQ AX, AX + INCL AX + CPUID + SHRQ $20, CX + ANDQ $1, CX + MOVB CX, ret+0(FP) + RET + +// func haveCLMUL() bool +TEXT ·haveCLMUL(SB), NOSPLIT, $0 + XORQ AX, AX + INCL AX + CPUID + SHRQ $1, CX + ANDQ $1, CX + MOVB CX, ret+0(FP) + RET + +// func haveSSE41() bool +TEXT ·haveSSE41(SB), NOSPLIT, $0 + XORQ AX, AX + INCL AX + CPUID + SHRQ $19, CX + ANDQ $1, CX + MOVB CX, ret+0(FP) + RET + +// CRC32 polynomial data +// +// These constants are lifted from the +// Linux kernel, since they avoid the costly +// PSHUFB 16 byte reversal proposed in the +// original Intel paper. +DATA r2r1kp<>+0(SB)/8, $0x154442bd4 +DATA r2r1kp<>+8(SB)/8, $0x1c6e41596 +DATA r4r3kp<>+0(SB)/8, $0x1751997d0 +DATA r4r3kp<>+8(SB)/8, $0x0ccaa009e +DATA rupolykp<>+0(SB)/8, $0x1db710641 +DATA rupolykp<>+8(SB)/8, $0x1f7011641 +DATA r5kp<>+0(SB)/8, $0x163cd6124 + +GLOBL r2r1kp<>(SB), RODATA, $16 +GLOBL r4r3kp<>(SB), RODATA, $16 +GLOBL rupolykp<>(SB), RODATA, $16 +GLOBL r5kp<>(SB), RODATA, $8 + +// Based on http://www.intel.com/content/dam/www/public/us/en/documents/white-papers/fast-crc-computation-generic-polynomials-pclmulqdq-paper.pdf +// len(p) must be at least 64, and must be a multiple of 16. + +// func ieeeCLMUL(crc uint32, p []byte) uint32 +TEXT ·ieeeCLMUL(SB), NOSPLIT, $0 + MOVL crc+0(FP), X0 // Initial CRC value + MOVQ p+8(FP), SI // data pointer + MOVQ p_len+16(FP), CX // len(p) + + MOVOU (SI), X1 + MOVOU 16(SI), X2 + MOVOU 32(SI), X3 + MOVOU 48(SI), X4 + PXOR X0, X1 + ADDQ $64, SI // buf+=64 + SUBQ $64, CX // len-=64 + CMPQ CX, $64 // Less than 64 bytes left + JB remain64 + + MOVOU r2r1kp<>+0(SB), X0 + +loopback64: + MOVOA X1, X5 + MOVOA X2, X6 + MOVOA X3, X7 + MOVOA X4, X8 + + PCLMULQDQ $0, X0, X1 + PCLMULQDQ $0, X0, X2 + PCLMULQDQ $0, X0, X3 + PCLMULQDQ $0, X0, X4 + + // Load next early + MOVOU (SI), X11 + MOVOU 16(SI), X12 + MOVOU 32(SI), X13 + MOVOU 48(SI), X14 + + PCLMULQDQ $0x11, X0, X5 + PCLMULQDQ $0x11, X0, X6 + PCLMULQDQ $0x11, X0, X7 + PCLMULQDQ $0x11, X0, X8 + + PXOR X5, X1 + PXOR X6, X2 + PXOR X7, X3 + PXOR X8, X4 + + PXOR X11, X1 + PXOR X12, X2 + PXOR X13, X3 + PXOR X14, X4 + + ADDQ $0x40, DI + ADDQ $64, SI // buf+=64 + SUBQ $64, CX // len-=64 + CMPQ CX, $64 // Less than 64 bytes left? + JGE loopback64 + + // Fold result into a single register (X1) +remain64: + MOVOU r4r3kp<>+0(SB), X0 + + MOVOA X1, X5 + PCLMULQDQ $0, X0, X1 + PCLMULQDQ $0x11, X0, X5 + PXOR X5, X1 + PXOR X2, X1 + + MOVOA X1, X5 + PCLMULQDQ $0, X0, X1 + PCLMULQDQ $0x11, X0, X5 + PXOR X5, X1 + PXOR X3, X1 + + MOVOA X1, X5 + PCLMULQDQ $0, X0, X1 + PCLMULQDQ $0x11, X0, X5 + PXOR X5, X1 + PXOR X4, X1 + + // More than 16 bytes left? + CMPQ CX, $16 + JB finish + + // Encode 16 bytes +remain16: + MOVOU (SI), X10 + MOVOA X1, X5 + PCLMULQDQ $0, X0, X1 + PCLMULQDQ $0x11, X0, X5 + PXOR X5, X1 + PXOR X10, X1 + SUBQ $16, CX + ADDQ $16, SI + CMPQ CX, $16 + JGE remain16 + +finish: + // Fold final result into 32 bits and return it + PCMPEQB X3, X3 + PCLMULQDQ $1, X1, X0 + PSRLDQ $8, X1 + PXOR X0, X1 + + MOVOA X1, X2 + MOVQ r5kp<>+0(SB), X0 + + // Creates 32 bit mask. Note that we don't care about upper half. + PSRLQ $32, X3 + + PSRLDQ $4, X2 + PAND X3, X1 + PCLMULQDQ $0, X0, X1 + PXOR X2, X1 + + MOVOU rupolykp<>+0(SB), X0 + + MOVOA X1, X2 + PAND X3, X1 + PCLMULQDQ $0x10, X0, X1 + PAND X3, X1 + PCLMULQDQ $0, X0, X1 + PXOR X2, X1 + + // PEXTRD $1, X1, AX (SSE 4.1) + BYTE $0x66; BYTE $0x0f; BYTE $0x3a + BYTE $0x16; BYTE $0xc8; BYTE $0x01 + MOVL AX, ret+32(FP) + + RET diff --git a/vendor/github.com/klauspost/crc32/crc32_amd64p32.go b/vendor/github.com/klauspost/crc32/crc32_amd64p32.go new file mode 100644 index 000000000000..926473e7c0e3 --- /dev/null +++ b/vendor/github.com/klauspost/crc32/crc32_amd64p32.go @@ -0,0 +1,40 @@ +// Copyright 2011 The Go Authors. All rights reserved. +// Use of this source code is governed by a BSD-style +// license that can be found in the LICENSE file. + +// +build !appengine,!gccgo + +package crc32 + +// This file contains the code to call the SSE 4.2 version of the Castagnoli +// CRC. + +// haveSSE42 is defined in crc_amd64p32.s and uses CPUID to test for SSE 4.2 +// support. +func haveSSE42() bool + +// castagnoliSSE42 is defined in crc_amd64.s and uses the SSE4.2 CRC32 +// instruction. +//go:noescape +func castagnoliSSE42(crc uint32, p []byte) uint32 + +var sse42 = haveSSE42() + +func updateCastagnoli(crc uint32, p []byte) uint32 { + if sse42 { + return castagnoliSSE42(crc, p) + } + return update(crc, castagnoliTable, p) +} + +func updateIEEE(crc uint32, p []byte) uint32 { + // only use slicing-by-8 when input is >= 4KB + if len(p) >= 4096 { + ieeeTable8Once.Do(func() { + ieeeTable8 = makeTable8(IEEE) + }) + return updateSlicingBy8(crc, ieeeTable8, p) + } + + return update(crc, IEEETable, p) +} diff --git a/vendor/github.com/klauspost/crc32/crc32_amd64p32.s b/vendor/github.com/klauspost/crc32/crc32_amd64p32.s new file mode 100644 index 000000000000..a578d685cc52 --- /dev/null +++ b/vendor/github.com/klauspost/crc32/crc32_amd64p32.s @@ -0,0 +1,67 @@ +// Copyright 2011 The Go Authors. All rights reserved. +// Use of this source code is governed by a BSD-style +// license that can be found in the LICENSE file. + +// +build gc + +#define NOSPLIT 4 +#define RODATA 8 + +// func castagnoliSSE42(crc uint32, p []byte) uint32 +TEXT ·castagnoliSSE42(SB), NOSPLIT, $0 + MOVL crc+0(FP), AX // CRC value + MOVL p+4(FP), SI // data pointer + MOVL p_len+8(FP), CX // len(p) + + NOTL AX + + // If there's less than 8 bytes to process, we do it byte-by-byte. + CMPQ CX, $8 + JL cleanup + + // Process individual bytes until the input is 8-byte aligned. +startup: + MOVQ SI, BX + ANDQ $7, BX + JZ aligned + + CRC32B (SI), AX + DECQ CX + INCQ SI + JMP startup + +aligned: + // The input is now 8-byte aligned and we can process 8-byte chunks. + CMPQ CX, $8 + JL cleanup + + CRC32Q (SI), AX + ADDQ $8, SI + SUBQ $8, CX + JMP aligned + +cleanup: + // We may have some bytes left over that we process one at a time. + CMPQ CX, $0 + JE done + + CRC32B (SI), AX + INCQ SI + DECQ CX + JMP cleanup + +done: + NOTL AX + MOVL AX, ret+16(FP) + RET + +// func haveSSE42() bool +TEXT ·haveSSE42(SB), NOSPLIT, $0 + XORQ AX, AX + INCL AX + CPUID + SHRQ $20, CX + ANDQ $1, CX + MOVB CX, ret+0(FP) + RET + diff --git a/vendor/github.com/klauspost/crc32/crc32_generic.go b/vendor/github.com/klauspost/crc32/crc32_generic.go new file mode 100644 index 000000000000..a53cf96a00b2 --- /dev/null +++ b/vendor/github.com/klauspost/crc32/crc32_generic.go @@ -0,0 +1,29 @@ +// Copyright 2011 The Go Authors. All rights reserved. +// Use of this source code is governed by a BSD-style +// license that can be found in the LICENSE file. + +// +build !amd64,!amd64p32 appengine gccgo + +package crc32 + +// This file contains the generic version of updateCastagnoli which does +// slicing-by-8, or uses the fallback for very small sizes. + +func updateCastagnoli(crc uint32, p []byte) uint32 { + // only use slicing-by-8 when input is >= 16 Bytes + if len(p) >= 16 { + return updateSlicingBy8(crc, castagnoliTable8, p) + } + return update(crc, castagnoliTable, p) +} + +func updateIEEE(crc uint32, p []byte) uint32 { + // only use slicing-by-8 when input is >= 16 Bytes + if len(p) >= 16 { + ieeeTable8Once.Do(func() { + ieeeTable8 = makeTable8(IEEE) + }) + return updateSlicingBy8(crc, ieeeTable8, p) + } + return update(crc, IEEETable, p) +} diff --git a/vendor/github.com/klauspost/crc32/crc32_test.go b/vendor/github.com/klauspost/crc32/crc32_test.go new file mode 100644 index 000000000000..d9dd3099f9d9 --- /dev/null +++ b/vendor/github.com/klauspost/crc32/crc32_test.go @@ -0,0 +1,170 @@ +// Copyright 2009 The Go Authors. All rights reserved. +// Use of this source code is governed by a BSD-style +// license that can be found in the LICENSE file. + +package crc32 + +import ( + "hash" + "hash/crc32" + "io" + "testing" +) + +type test struct { + ieee, castagnoli uint32 + in string +} + +var golden = []test{ + {0x0, 0x0, ""}, + {0xe8b7be43, 0xc1d04330, "a"}, + {0x9e83486d, 0xe2a22936, "ab"}, + {0x352441c2, 0x364b3fb7, "abc"}, + {0xed82cd11, 0x92c80a31, "abcd"}, + {0x8587d865, 0xc450d697, "abcde"}, + {0x4b8e39ef, 0x53bceff1, "abcdef"}, + {0x312a6aa6, 0xe627f441, "abcdefg"}, + {0xaeef2a50, 0xa9421b7, "abcdefgh"}, + {0x8da988af, 0x2ddc99fc, "abcdefghi"}, + {0x3981703a, 0xe6599437, "abcdefghij"}, + {0x6b9cdfe7, 0xb2cc01fe, "Discard medicine more than two years old."}, + {0xc90ef73f, 0xe28207f, "He who has a shady past knows that nice guys finish last."}, + {0xb902341f, 0xbe93f964, "I wouldn't marry him with a ten foot pole."}, + {0x42080e8, 0x9e3be0c3, "Free! Free!/A trip/to Mars/for 900/empty jars/Burma Shave"}, + {0x154c6d11, 0xf505ef04, "The days of the digital watch are numbered. -Tom Stoppard"}, + {0x4c418325, 0x85d3dc82, "Nepal premier won't resign."}, + {0x33955150, 0xc5142380, "For every action there is an equal and opposite government program."}, + {0x26216a4b, 0x75eb77dd, "His money is twice tainted: 'taint yours and 'taint mine."}, + {0x1abbe45e, 0x91ebe9f7, "There is no reason for any individual to have a computer in their home. -Ken Olsen, 1977"}, + {0xc89a94f7, 0xf0b1168e, "It's a tiny change to the code and not completely disgusting. - Bob Manchek"}, + {0xab3abe14, 0x572b74e2, "size: a.out: bad magic"}, + {0xbab102b6, 0x8a58a6d5, "The major problem is with sendmail. -Mark Horton"}, + {0x999149d7, 0x9c426c50, "Give me a rock, paper and scissors and I will move the world. CCFestoon"}, + {0x6d52a33c, 0x735400a4, "If the enemy is within range, then so are you."}, + {0x90631e8d, 0xbec49c95, "It's well we cannot hear the screams/That we create in others' dreams."}, + {0x78309130, 0xa95a2079, "You remind me of a TV show, but that's all right: I watch it anyway."}, + {0x7d0a377f, 0xde2e65c5, "C is as portable as Stonehedge!!"}, + {0x8c79fd79, 0x297a88ed, "Even if I could be Shakespeare, I think I should still choose to be Faraday. - A. Huxley"}, + {0xa20b7167, 0x66ed1d8b, "The fugacity of a constituent in a mixture of gases at a given temperature is proportional to its mole fraction. Lewis-Randall Rule"}, + {0x8e0bb443, 0xdcded527, "How can you write a big system without C++? -Paul Glick"}, +} + +func TestGolden(t *testing.T) { + castagnoliTab := MakeTable(Castagnoli) + + for _, g := range golden { + ieee := NewIEEE() + io.WriteString(ieee, g.in) + s := ieee.Sum32() + if s != g.ieee { + t.Errorf("IEEE(%s) = 0x%x want 0x%x", g.in, s, g.ieee) + } + + castagnoli := New(castagnoliTab) + io.WriteString(castagnoli, g.in) + s = castagnoli.Sum32() + if s != g.castagnoli { + t.Errorf("Castagnoli(%s) = 0x%x want 0x%x", g.in, s, g.castagnoli) + } + + if len(g.in) > 0 { + // The SSE4.2 implementation of this has code to deal + // with misaligned data so we ensure that we test that + // too. + castagnoli = New(castagnoliTab) + io.WriteString(castagnoli, g.in[:1]) + io.WriteString(castagnoli, g.in[1:]) + s = castagnoli.Sum32() + if s != g.castagnoli { + t.Errorf("Castagnoli[misaligned](%s) = 0x%x want 0x%x", g.in, s, g.castagnoli) + } + } + } +} + +func BenchmarkCrc40B(b *testing.B) { + benchmark(b, NewIEEE(), 40) +} + +func BenchmarkStdCrc40B(b *testing.B) { + benchmark(b, crc32.NewIEEE(), 40) +} + +func BenchmarkCrc1KB(b *testing.B) { + benchmark(b, NewIEEE(), 1024) +} + +func BenchmarkStdCrc1KB(b *testing.B) { + benchmark(b, crc32.NewIEEE(), 1024) +} + +func BenchmarkCrc8KB(b *testing.B) { + benchmark(b, NewIEEE(), 8*1024) +} + +func BenchmarkStdCrc8KB(b *testing.B) { + benchmark(b, crc32.NewIEEE(), 8*1024) +} + +func BenchmarkCrc32KB(b *testing.B) { + benchmark(b, NewIEEE(), 32*1024) +} + +func BenchmarkStdCrc32KB(b *testing.B) { + benchmark(b, crc32.NewIEEE(), 32*1024) +} + +func BenchmarkCastagnoli40B(b *testing.B) { + benchmark(b, New(MakeTable(Castagnoli)), 40) +} + +func BenchmarkStdCastagnoli40B(b *testing.B) { + benchmark(b, crc32.New(crc32.MakeTable(Castagnoli)), 40) +} + +func BenchmarkCastagnoli1KB(b *testing.B) { + benchmark(b, New(MakeTable(Castagnoli)), 1024) +} + +func BenchmarkStdCastagnoli1KB(b *testing.B) { + benchmark(b, crc32.New(crc32.MakeTable(Castagnoli)), 1024) +} + +func BenchmarkCastagnoli8KB(b *testing.B) { + benchmark(b, New(MakeTable(Castagnoli)), 8*1024) +} + +func BenchmarkStdCastagnoli8KB(b *testing.B) { + benchmark(b, crc32.New(crc32.MakeTable(Castagnoli)), 8*1024) +} + +func BenchmarkCastagnoli32KB(b *testing.B) { + benchmark(b, New(MakeTable(Castagnoli)), 32*1024) +} + +func BenchmarkStdCastagnoli32KB(b *testing.B) { + benchmark(b, crc32.New(crc32.MakeTable(Castagnoli)), 32*1024) +} + +func benchmark(b *testing.B, h hash.Hash32, n int64) { + b.SetBytes(n) + data := make([]byte, n) + for i := range data { + data[i] = byte(i) + } + in := make([]byte, 0, h.Size()) + + // Warm up + h.Reset() + h.Write(data) + h.Sum(in) + + b.ResetTimer() + + for i := 0; i < b.N; i++ { + h.Reset() + h.Write(data) + h.Sum(in) + } +} diff --git a/vendor/github.com/klauspost/crc32/example_test.go b/vendor/github.com/klauspost/crc32/example_test.go new file mode 100644 index 000000000000..621bf838309e --- /dev/null +++ b/vendor/github.com/klauspost/crc32/example_test.go @@ -0,0 +1,28 @@ +// Copyright 2015 The Go Authors. All rights reserved. +// Use of this source code is governed by a BSD-style +// license that can be found in the LICENSE file. + +package crc32_test + +import ( + "fmt" + "hash/crc32" +) + +func ExampleMakeTable() { + // In this package, the CRC polynomial is represented in reversed notation, + // or LSB-first representation. + // + // LSB-first representation is a hexadecimal number with n bits, in which the + // most significant bit represents the coefficient of x⁰ and the least significant + // bit represents the coefficient of xⁿ⁻¹ (the coefficient for xⁿ is implicit). + // + // For example, CRC32-Q, as defined by the following polynomial, + // x³²+ x³¹+ x²⁴+ x²²+ x¹⁶+ x¹⁴+ x⁸+ x⁷+ x⁵+ x³+ x¹+ x⁰ + // has the reversed notation 0b11010101100000101000001010000001, so the value + // that should be passed to MakeTable is 0xD5828281. + crc32q := crc32.MakeTable(0xD5828281) + fmt.Printf("%08x\n", crc32.Checksum([]byte("Hello world"), crc32q)) + // Output: + // 2964d064 +}