123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330331332333334335336337338339340341342343344345346347348349350351352353354355356357358359360361362363364365366367368369370371372373374375376377378379380381382383384385386387388389390391392393394395396397398399400401402403404405406407408409410411412413414415416417418419420421422423424425426427428429430431432433434435436437438439440441442443444445446447448449450451452453454455456457458459460461462463464465466467468469470471472473474475476477478479480481482483484485486487488489490491492493494495496497498499500501502503504505506507508509510511512513514515516517518519520521522523524525526527528529530531532533534535536537538539540541542543544545546547548549550551552553554555556557558559560561562563564565566567568569570571572573574575576577578579580581582583584585586587588589590591592593594595596597598599600601602603604605606607608609610611612613614615616617618619620621622623624625626627628629630631632633634635636637638639640641642643644645646647648649650651652653654655656657658659660661662663664665666667668669670671672673674675676677678679680681682683684685686687688689690691692693694695696697698699700701702703704705706707708709710711712713714715716717718719720721722723724725726727728729730731732733734735736737738739740741742743744745746747748749750751752753754755756757758759760761762763764765 |
- package sarama
- import (
- "compress/gzip"
- "crypto/tls"
- "fmt"
- "io/ioutil"
- "net"
- "regexp"
- "time"
- "github.com/rcrowley/go-metrics"
- "golang.org/x/net/proxy"
- )
- const defaultClientID = "sarama"
- var validID = regexp.MustCompile(`\A[A-Za-z0-9._-]+\z`)
- type Config struct {
-
- Admin struct {
- Retry struct {
-
-
- Max int
-
- Backoff time.Duration
- }
-
-
- Timeout time.Duration
- }
-
-
- Net struct {
-
-
- MaxOpenRequests int
-
-
-
- DialTimeout time.Duration
- ReadTimeout time.Duration
- WriteTimeout time.Duration
- TLS struct {
-
-
- Enable bool
-
-
- Config *tls.Config
- }
-
-
- SASL struct {
-
-
- Enable bool
-
-
- Mechanism SASLMechanism
-
-
- Version int16
-
-
-
- Handshake bool
-
-
-
-
- AuthIdentity string
-
-
- User string
-
- Password string
-
- SCRAMAuthzID string
-
-
- SCRAMClientGeneratorFunc func() SCRAMClient
-
-
-
-
- TokenProvider AccessTokenProvider
- GSSAPI GSSAPIConfig
- }
-
-
-
- KeepAlive time.Duration
-
-
-
-
- LocalAddr net.Addr
- Proxy struct {
-
-
- Enable bool
-
- Dialer proxy.Dialer
- }
- }
-
-
- Metadata struct {
- Retry struct {
-
-
- Max int
-
-
- Backoff time.Duration
-
-
-
- BackoffFunc func(retries, maxRetries int) time.Duration
- }
-
-
-
- RefreshFrequency time.Duration
-
-
-
-
- Full bool
-
-
-
-
-
- Timeout time.Duration
- }
-
-
- Producer struct {
-
-
- MaxMessageBytes int
-
-
-
- RequiredAcks RequiredAcks
-
-
-
-
-
- Timeout time.Duration
-
-
- Compression CompressionCodec
-
-
-
- CompressionLevel int
-
-
-
- Partitioner PartitionerConstructor
-
-
- Idempotent bool
-
-
-
-
-
- Return struct {
-
-
- Successes bool
-
-
- Errors bool
- }
-
-
-
-
- Flush struct {
-
-
- Bytes int
-
-
- Messages int
-
-
- Frequency time.Duration
-
-
-
- MaxMessages int
- }
- Retry struct {
-
-
- Max int
-
-
-
- Backoff time.Duration
-
-
-
- BackoffFunc func(retries, maxRetries int) time.Duration
- }
-
-
-
-
-
-
- Interceptors []ProducerInterceptor
- }
-
-
- Consumer struct {
-
- Group struct {
- Session struct {
-
-
-
-
-
-
- Timeout time.Duration
- }
- Heartbeat struct {
-
-
-
-
-
-
- Interval time.Duration
- }
- Rebalance struct {
-
- Strategy BalanceStrategy
-
-
-
-
- Timeout time.Duration
- Retry struct {
-
-
-
-
- Max int
-
- Backoff time.Duration
- }
- }
- Member struct {
-
-
-
- UserData []byte
- }
- }
- Retry struct {
-
-
- Backoff time.Duration
-
-
-
- BackoffFunc func(retries int) time.Duration
- }
-
-
- Fetch struct {
-
-
-
-
- Min int32
-
-
-
-
-
- Default int32
-
-
-
-
-
-
- Max int32
- }
-
-
-
-
-
-
- MaxWaitTime time.Duration
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
- MaxProcessingTime time.Duration
-
-
- Return struct {
-
-
- Errors bool
- }
-
-
-
- Offsets struct {
-
-
- CommitInterval time.Duration
-
- AutoCommit struct {
-
-
- Enable bool
-
-
- Interval time.Duration
- }
-
-
- Initial int64
-
-
-
-
-
-
- Retention time.Duration
- Retry struct {
-
-
- Max int
- }
- }
-
-
-
- IsolationLevel IsolationLevel
-
-
-
-
-
-
- Interceptors []ConsumerInterceptor
- }
-
-
-
- ClientID string
-
-
-
- RackID string
-
-
-
-
- ChannelBufferSize int
-
-
-
-
-
-
- Version KafkaVersion
-
-
-
-
-
- MetricRegistry metrics.Registry
- }
- func NewConfig() *Config {
- c := &Config{}
- c.Admin.Retry.Max = 5
- c.Admin.Retry.Backoff = 100 * time.Millisecond
- c.Admin.Timeout = 3 * time.Second
- c.Net.MaxOpenRequests = 5
- c.Net.DialTimeout = 30 * time.Second
- c.Net.ReadTimeout = 30 * time.Second
- c.Net.WriteTimeout = 30 * time.Second
- c.Net.SASL.Handshake = true
- c.Net.SASL.Version = SASLHandshakeV0
- c.Metadata.Retry.Max = 3
- c.Metadata.Retry.Backoff = 250 * time.Millisecond
- c.Metadata.RefreshFrequency = 10 * time.Minute
- c.Metadata.Full = true
- 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.Producer.CompressionLevel = CompressionLevelDefault
- c.Consumer.Fetch.Min = 1
- c.Consumer.Fetch.Default = 1024 * 1024
- 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.AutoCommit.Enable = true
- c.Consumer.Offsets.AutoCommit.Interval = 1 * time.Second
- c.Consumer.Offsets.Initial = OffsetNewest
- c.Consumer.Offsets.Retry.Max = 3
- c.Consumer.Group.Session.Timeout = 10 * time.Second
- c.Consumer.Group.Heartbeat.Interval = 3 * time.Second
- c.Consumer.Group.Rebalance.Strategy = BalanceStrategyRange
- c.Consumer.Group.Rebalance.Timeout = 60 * time.Second
- c.Consumer.Group.Rebalance.Retry.Max = 4
- c.Consumer.Group.Rebalance.Retry.Backoff = 2 * time.Second
- c.ClientID = defaultClientID
- c.ChannelBufferSize = 256
- c.Version = MinVersion
- c.MetricRegistry = metrics.NewRegistry()
- return c
- }
- func (c *Config) Validate() error {
-
- if !c.Net.TLS.Enable && c.Net.TLS.Config != nil {
- Logger.Println("Net.TLS is disabled but a non-nil configuration was provided.")
- }
- if !c.Net.SASL.Enable {
- if c.Net.SASL.User != "" {
- Logger.Println("Net.SASL is disabled but a non-empty username was provided.")
- }
- if c.Net.SASL.Password != "" {
- Logger.Println("Net.SASL is disabled but a non-empty password 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 must be smaller than MaxRequestSize; it will be ignored.")
- }
- if c.Producer.Flush.Bytes >= int(MaxRequestSize) {
- Logger.Println("Producer.Flush.Bytes must be smaller than MaxRequestSize; it will be ignored.")
- }
- if (c.Producer.Flush.Bytes > 0 || c.Producer.Flush.Messages > 0) && c.Producer.Flush.Frequency == 0 {
- Logger.Println("Producer.Flush: Bytes or Messages are set, but Frequency is not; messages may not get flushed.")
- }
- 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.Consumer.Offsets.Retention%time.Millisecond != 0 {
- Logger.Println("Consumer.Offsets.Retention only supports millisecond precision; nanoseconds will be truncated.")
- }
- if c.Consumer.Group.Session.Timeout%time.Millisecond != 0 {
- Logger.Println("Consumer.Group.Session.Timeout only supports millisecond precision; nanoseconds will be truncated.")
- }
- if c.Consumer.Group.Heartbeat.Interval%time.Millisecond != 0 {
- Logger.Println("Consumer.Group.Heartbeat.Interval only supports millisecond precision; nanoseconds will be truncated.")
- }
- if c.Consumer.Group.Rebalance.Timeout%time.Millisecond != 0 {
- Logger.Println("Consumer.Group.Rebalance.Timeout only supports millisecond precision; nanoseconds will be truncated.")
- }
- if c.ClientID == defaultClientID {
- Logger.Println("ClientID is the default of 'sarama', you should consider setting it to something application-specific.")
- }
-
- 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.SASL.Enable:
- if c.Net.SASL.Mechanism == "" {
- c.Net.SASL.Mechanism = SASLTypePlaintext
- }
- switch c.Net.SASL.Mechanism {
- case SASLTypePlaintext:
- if c.Net.SASL.User == "" {
- return ConfigurationError("Net.SASL.User must not be empty when SASL is enabled")
- }
- if c.Net.SASL.Password == "" {
- return ConfigurationError("Net.SASL.Password must not be empty when SASL is enabled")
- }
- case SASLTypeOAuth:
- if c.Net.SASL.TokenProvider == nil {
- return ConfigurationError("An AccessTokenProvider instance must be provided to Net.SASL.TokenProvider")
- }
- case SASLTypeSCRAMSHA256, SASLTypeSCRAMSHA512:
- if c.Net.SASL.User == "" {
- return ConfigurationError("Net.SASL.User must not be empty when SASL is enabled")
- }
- if c.Net.SASL.Password == "" {
- return ConfigurationError("Net.SASL.Password must not be empty when SASL is enabled")
- }
- if c.Net.SASL.SCRAMClientGeneratorFunc == nil {
- return ConfigurationError("A SCRAMClientGeneratorFunc function must be provided to Net.SASL.SCRAMClientGeneratorFunc")
- }
- case SASLTypeGSSAPI:
- if c.Net.SASL.GSSAPI.ServiceName == "" {
- return ConfigurationError("Net.SASL.GSSAPI.ServiceName must not be empty when GSS-API mechanism is used")
- }
- if c.Net.SASL.GSSAPI.AuthType == KRB5_USER_AUTH {
- if c.Net.SASL.GSSAPI.Password == "" {
- return ConfigurationError("Net.SASL.GSSAPI.Password must not be empty when GSS-API " +
- "mechanism is used and Net.SASL.GSSAPI.AuthType = KRB5_USER_AUTH")
- }
- } else if c.Net.SASL.GSSAPI.AuthType == KRB5_KEYTAB_AUTH {
- if c.Net.SASL.GSSAPI.KeyTabPath == "" {
- return ConfigurationError("Net.SASL.GSSAPI.KeyTabPath must not be empty when GSS-API mechanism is used" +
- " and Net.SASL.GSSAPI.AuthType = KRB5_KEYTAB_AUTH")
- }
- } else {
- return ConfigurationError("Net.SASL.GSSAPI.AuthType is invalid. Possible values are KRB5_USER_AUTH and KRB5_KEYTAB_AUTH")
- }
- if c.Net.SASL.GSSAPI.KerberosConfigPath == "" {
- return ConfigurationError("Net.SASL.GSSAPI.KerberosConfigPath must not be empty when GSS-API mechanism is used")
- }
- if c.Net.SASL.GSSAPI.Username == "" {
- return ConfigurationError("Net.SASL.GSSAPI.Username must not be empty when GSS-API mechanism is used")
- }
- if c.Net.SASL.GSSAPI.Realm == "" {
- return ConfigurationError("Net.SASL.GSSAPI.Realm must not be empty when GSS-API mechanism is used")
- }
- default:
- msg := fmt.Sprintf("The SASL mechanism configuration is invalid. Possible values are `%s`, `%s`, `%s`, `%s` and `%s`",
- SASLTypeOAuth, SASLTypePlaintext, SASLTypeSCRAMSHA256, SASLTypeSCRAMSHA512, SASLTypeGSSAPI)
- return ConfigurationError(msg)
- }
- }
-
- switch {
- case c.Admin.Timeout <= 0:
- return ConfigurationError("Admin.Timeout must be > 0")
- }
-
- 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")
- }
-
- 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")
- }
- if c.Producer.Compression == CompressionLZ4 && !c.Version.IsAtLeast(V0_10_0_0) {
- return ConfigurationError("lz4 compression requires Version >= V0_10_0_0")
- }
- if c.Producer.Compression == CompressionGZIP {
- if c.Producer.CompressionLevel != CompressionLevelDefault {
- if _, err := gzip.NewWriterLevel(ioutil.Discard, c.Producer.CompressionLevel); err != nil {
- return ConfigurationError(fmt.Sprintf("gzip compression does not work with level %d: %v", c.Producer.CompressionLevel, err))
- }
- }
- }
- if c.Producer.Compression == CompressionZSTD && !c.Version.IsAtLeast(V2_1_0_0) {
- return ConfigurationError("zstd compression requires Version >= V2_1_0_0")
- }
- if c.Producer.Idempotent {
- if !c.Version.IsAtLeast(V0_11_0_0) {
- return ConfigurationError("Idempotent producer requires Version >= V0_11_0_0")
- }
- if c.Producer.Retry.Max == 0 {
- return ConfigurationError("Idempotent producer requires Producer.Retry.Max >= 1")
- }
- if c.Producer.RequiredAcks != WaitForAll {
- return ConfigurationError("Idempotent producer requires Producer.RequiredAcks to be WaitForAll")
- }
- if c.Net.MaxOpenRequests > 1 {
- return ConfigurationError("Idempotent producer requires Net.MaxOpenRequests to be 1")
- }
- }
-
- 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.AutoCommit.Interval <= 0:
- return ConfigurationError("Consumer.Offsets.AutoCommit.Interval must be > 0")
- case c.Consumer.Offsets.Initial != OffsetOldest && c.Consumer.Offsets.Initial != OffsetNewest:
- return ConfigurationError("Consumer.Offsets.Initial must be OffsetOldest or OffsetNewest")
- case c.Consumer.Offsets.Retry.Max < 0:
- return ConfigurationError("Consumer.Offsets.Retry.Max must be >= 0")
- case c.Consumer.IsolationLevel != ReadUncommitted && c.Consumer.IsolationLevel != ReadCommitted:
- return ConfigurationError("Consumer.IsolationLevel must be ReadUncommitted or ReadCommitted")
- }
- if c.Consumer.Offsets.CommitInterval != 0 {
- Logger.Println("Deprecation warning: Consumer.Offsets.CommitInterval exists for historical compatibility" +
- " and should not be used. Please use Consumer.Offsets.AutoCommit, the current value will be ignored")
- }
-
- if c.Consumer.IsolationLevel == ReadCommitted && !c.Version.IsAtLeast(V0_11_0_0) {
- return ConfigurationError("ReadCommitted requires Version >= V0_11_0_0")
- }
-
- switch {
- case c.Consumer.Group.Session.Timeout <= 2*time.Millisecond:
- return ConfigurationError("Consumer.Group.Session.Timeout must be >= 2ms")
- case c.Consumer.Group.Heartbeat.Interval < 1*time.Millisecond:
- return ConfigurationError("Consumer.Group.Heartbeat.Interval must be >= 1ms")
- case c.Consumer.Group.Heartbeat.Interval >= c.Consumer.Group.Session.Timeout:
- return ConfigurationError("Consumer.Group.Heartbeat.Interval must be < Consumer.Group.Session.Timeout")
- case c.Consumer.Group.Rebalance.Strategy == nil:
- return ConfigurationError("Consumer.Group.Rebalance.Strategy must not be empty")
- case c.Consumer.Group.Rebalance.Timeout <= time.Millisecond:
- return ConfigurationError("Consumer.Group.Rebalance.Timeout must be >= 1ms")
- case c.Consumer.Group.Rebalance.Retry.Max < 0:
- return ConfigurationError("Consumer.Group.Rebalance.Retry.Max must be >= 0")
- case c.Consumer.Group.Rebalance.Retry.Backoff < 0:
- return ConfigurationError("Consumer.Group.Rebalance.Retry.Backoff must be >= 0")
- }
-
- switch {
- case c.ChannelBufferSize < 0:
- return ConfigurationError("ChannelBufferSize must be >= 0")
- case !validID.MatchString(c.ClientID):
- return ConfigurationError("ClientID is invalid")
- }
- return nil
- }
- func (c *Config) getDialer() proxy.Dialer {
- if c.Net.Proxy.Enable {
- Logger.Printf("using proxy %s", c.Net.Proxy.Dialer)
- return c.Net.Proxy.Dialer
- } else {
- return &net.Dialer{
- Timeout: c.Net.DialTimeout,
- KeepAlive: c.Net.KeepAlive,
- LocalAddr: c.Net.LocalAddr,
- }
- }
- }
|