config.go 21 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330331332333334335336337338339340341342343344345346347348349350351352353354355356357358359360361362363364365366367368369370371372373374375376377378379380381382383384385386387388389390391392393394395396397398399400401402403404405406407408409410411412413414415416417418419420421422423424425426427428429430431432433434435436437438439440441442443444445446447448449450451452453454455456457458459460461462463464465466467468469470471472473474475476477478479480481482483484485486487488489490491492493494495496
  1. package sarama
  2. import (
  3. "compress/gzip"
  4. "crypto/tls"
  5. "fmt"
  6. "io/ioutil"
  7. "net"
  8. "regexp"
  9. "time"
  10. "github.com/rcrowley/go-metrics"
  11. )
  12. const defaultClientID = "sarama"
  13. var validID = regexp.MustCompile(`\A[A-Za-z0-9._-]+\z`)
  14. // Config is used to pass multiple configuration options to Sarama's constructors.
  15. type Config struct {
  16. // Admin is the namespace for ClusterAdmin properties used by the administrative Kafka client.
  17. Admin struct {
  18. // The maximum duration the administrative Kafka client will wait for ClusterAdmin operations,
  19. // including topics, brokers, configurations and ACLs (defaults to 3 seconds).
  20. Timeout time.Duration
  21. }
  22. // Net is the namespace for network-level properties used by the Broker, and
  23. // shared by the Client/Producer/Consumer.
  24. Net struct {
  25. // How many outstanding requests a connection is allowed to have before
  26. // sending on it blocks (default 5).
  27. MaxOpenRequests int
  28. // All three of the below configurations are similar to the
  29. // `socket.timeout.ms` setting in JVM kafka. All of them default
  30. // to 30 seconds.
  31. DialTimeout time.Duration // How long to wait for the initial connection.
  32. ReadTimeout time.Duration // How long to wait for a response.
  33. WriteTimeout time.Duration // How long to wait for a transmit.
  34. TLS struct {
  35. // Whether or not to use TLS when connecting to the broker
  36. // (defaults to false).
  37. Enable bool
  38. // The TLS configuration to use for secure connections if
  39. // enabled (defaults to nil).
  40. Config *tls.Config
  41. }
  42. // SASL based authentication with broker. While there are multiple SASL authentication methods
  43. // the current implementation is limited to plaintext (SASL/PLAIN) authentication
  44. SASL struct {
  45. // Whether or not to use SASL authentication when connecting to the broker
  46. // (defaults to false).
  47. Enable bool
  48. // Whether or not to send the Kafka SASL handshake first if enabled
  49. // (defaults to true). You should only set this to false if you're using
  50. // a non-Kafka SASL proxy.
  51. Handshake bool
  52. //username and password for SASL/PLAIN authentication
  53. User string
  54. Password string
  55. }
  56. // KeepAlive specifies the keep-alive period for an active network connection.
  57. // If zero, keep-alives are disabled. (default is 0: disabled).
  58. KeepAlive time.Duration
  59. // LocalAddr is the local address to use when dialing an
  60. // address. The address must be of a compatible type for the
  61. // network being dialed.
  62. // If nil, a local address is automatically chosen.
  63. LocalAddr net.Addr
  64. }
  65. // Metadata is the namespace for metadata management properties used by the
  66. // Client, and shared by the Producer/Consumer.
  67. Metadata struct {
  68. Retry struct {
  69. // The total number of times to retry a metadata request when the
  70. // cluster is in the middle of a leader election (default 3).
  71. Max int
  72. // How long to wait for leader election to occur before retrying
  73. // (default 250ms). Similar to the JVM's `retry.backoff.ms`.
  74. Backoff time.Duration
  75. }
  76. // How frequently to refresh the cluster metadata in the background.
  77. // Defaults to 10 minutes. Set to 0 to disable. Similar to
  78. // `topic.metadata.refresh.interval.ms` in the JVM version.
  79. RefreshFrequency time.Duration
  80. // Whether to maintain a full set of metadata for all topics, or just
  81. // the minimal set that has been necessary so far. The full set is simpler
  82. // and usually more convenient, but can take up a substantial amount of
  83. // memory if you have many topics and partitions. Defaults to true.
  84. Full bool
  85. }
  86. // Producer is the namespace for configuration related to producing messages,
  87. // used by the Producer.
  88. Producer struct {
  89. // The maximum permitted size of a message (defaults to 1000000). Should be
  90. // set equal to or smaller than the broker's `message.max.bytes`.
  91. MaxMessageBytes int
  92. // The level of acknowledgement reliability needed from the broker (defaults
  93. // to WaitForLocal). Equivalent to the `request.required.acks` setting of the
  94. // JVM producer.
  95. RequiredAcks RequiredAcks
  96. // The maximum duration the broker will wait the receipt of the number of
  97. // RequiredAcks (defaults to 10 seconds). This is only relevant when
  98. // RequiredAcks is set to WaitForAll or a number > 1. Only supports
  99. // millisecond resolution, nanoseconds will be truncated. Equivalent to
  100. // the JVM producer's `request.timeout.ms` setting.
  101. Timeout time.Duration
  102. // The type of compression to use on messages (defaults to no compression).
  103. // Similar to `compression.codec` setting of the JVM producer.
  104. Compression CompressionCodec
  105. // The level of compression to use on messages. The meaning depends
  106. // on the actual compression type used and defaults to default compression
  107. // level for the codec.
  108. CompressionLevel int
  109. // Generates partitioners for choosing the partition to send messages to
  110. // (defaults to hashing the message key). Similar to the `partitioner.class`
  111. // setting for the JVM producer.
  112. Partitioner PartitionerConstructor
  113. // Return specifies what channels will be populated. If they are set to true,
  114. // you must read from the respective channels to prevent deadlock. If,
  115. // however, this config is used to create a `SyncProducer`, both must be set
  116. // to true and you shall not read from the channels since the producer does
  117. // this internally.
  118. Return struct {
  119. // If enabled, successfully delivered messages will be returned on the
  120. // Successes channel (default disabled).
  121. Successes bool
  122. // If enabled, messages that failed to deliver will be returned on the
  123. // Errors channel, including error (default enabled).
  124. Errors bool
  125. }
  126. // The following config options control how often messages are batched up and
  127. // sent to the broker. By default, messages are sent as fast as possible, and
  128. // all messages received while the current batch is in-flight are placed
  129. // into the subsequent batch.
  130. Flush struct {
  131. // The best-effort number of bytes needed to trigger a flush. Use the
  132. // global sarama.MaxRequestSize to set a hard upper limit.
  133. Bytes int
  134. // The best-effort number of messages needed to trigger a flush. Use
  135. // `MaxMessages` to set a hard upper limit.
  136. Messages int
  137. // The best-effort frequency of flushes. Equivalent to
  138. // `queue.buffering.max.ms` setting of JVM producer.
  139. Frequency time.Duration
  140. // The maximum number of messages the producer will send in a single
  141. // broker request. Defaults to 0 for unlimited. Similar to
  142. // `queue.buffering.max.messages` in the JVM producer.
  143. MaxMessages int
  144. }
  145. Retry struct {
  146. // The total number of times to retry sending a message (default 3).
  147. // Similar to the `message.send.max.retries` setting of the JVM producer.
  148. Max int
  149. // How long to wait for the cluster to settle between retries
  150. // (default 100ms). Similar to the `retry.backoff.ms` setting of the
  151. // JVM producer.
  152. Backoff time.Duration
  153. // Called to compute backoff time dynamically. Useful for implementing
  154. // more sophisticated backoff strategies. This takes precedence over
  155. // `Backoff` if set.
  156. BackoffFunc func(retries, maxRetries int) time.Duration
  157. }
  158. }
  159. // Consumer is the namespace for configuration related to consuming messages,
  160. // used by the Consumer.
  161. //
  162. // Note that Sarama's Consumer type does not currently support automatic
  163. // consumer-group rebalancing and offset tracking. For Zookeeper-based
  164. // tracking (Kafka 0.8.2 and earlier), the https://github.com/wvanbergen/kafka
  165. // library builds on Sarama to add this support. For Kafka-based tracking
  166. // (Kafka 0.9 and later), the https://github.com/bsm/sarama-cluster library
  167. // builds on Sarama to add this support.
  168. Consumer struct {
  169. Retry struct {
  170. // How long to wait after a failing to read from a partition before
  171. // trying again (default 2s).
  172. Backoff time.Duration
  173. // Called to compute backoff time dynamically. Useful for implementing
  174. // more sophisticated backoff strategies. This takes precedence over
  175. // `Backoff` if set.
  176. BackoffFunc func(retries int) time.Duration
  177. }
  178. // Fetch is the namespace for controlling how many bytes are retrieved by any
  179. // given request.
  180. Fetch struct {
  181. // The minimum number of message bytes to fetch in a request - the broker
  182. // will wait until at least this many are available. The default is 1,
  183. // as 0 causes the consumer to spin when no messages are available.
  184. // Equivalent to the JVM's `fetch.min.bytes`.
  185. Min int32
  186. // The default number of message bytes to fetch from the broker in each
  187. // request (default 1MB). This should be larger than the majority of
  188. // your messages, or else the consumer will spend a lot of time
  189. // negotiating sizes and not actually consuming. Similar to the JVM's
  190. // `fetch.message.max.bytes`.
  191. Default int32
  192. // The maximum number of message bytes to fetch from the broker in a
  193. // single request. Messages larger than this will return
  194. // ErrMessageTooLarge and will not be consumable, so you must be sure
  195. // this is at least as large as your largest message. Defaults to 0
  196. // (no limit). Similar to the JVM's `fetch.message.max.bytes`. The
  197. // global `sarama.MaxResponseSize` still applies.
  198. Max int32
  199. }
  200. // The maximum amount of time the broker will wait for Consumer.Fetch.Min
  201. // bytes to become available before it returns fewer than that anyways. The
  202. // default is 250ms, since 0 causes the consumer to spin when no events are
  203. // available. 100-500ms is a reasonable range for most cases. Kafka only
  204. // supports precision up to milliseconds; nanoseconds will be truncated.
  205. // Equivalent to the JVM's `fetch.wait.max.ms`.
  206. MaxWaitTime time.Duration
  207. // The maximum amount of time the consumer expects a message takes to
  208. // process for the user. If writing to the Messages channel takes longer
  209. // than this, that partition will stop fetching more messages until it
  210. // can proceed again.
  211. // Note that, since the Messages channel is buffered, the actual grace time is
  212. // (MaxProcessingTime * ChanneBufferSize). Defaults to 100ms.
  213. // If a message is not written to the Messages channel between two ticks
  214. // of the expiryTicker then a timeout is detected.
  215. // Using a ticker instead of a timer to detect timeouts should typically
  216. // result in many fewer calls to Timer functions which may result in a
  217. // significant performance improvement if many messages are being sent
  218. // and timeouts are infrequent.
  219. // The disadvantage of using a ticker instead of a timer is that
  220. // timeouts will be less accurate. That is, the effective timeout could
  221. // be between `MaxProcessingTime` and `2 * MaxProcessingTime`. For
  222. // example, if `MaxProcessingTime` is 100ms then a delay of 180ms
  223. // between two messages being sent may not be recognized as a timeout.
  224. MaxProcessingTime time.Duration
  225. // Return specifies what channels will be populated. If they are set to true,
  226. // you must read from them to prevent deadlock.
  227. Return struct {
  228. // If enabled, any errors that occurred while consuming are returned on
  229. // the Errors channel (default disabled).
  230. Errors bool
  231. }
  232. // Offsets specifies configuration for how and when to commit consumed
  233. // offsets. This currently requires the manual use of an OffsetManager
  234. // but will eventually be automated.
  235. Offsets struct {
  236. // How frequently to commit updated offsets. Defaults to 1s.
  237. CommitInterval time.Duration
  238. // The initial offset to use if no offset was previously committed.
  239. // Should be OffsetNewest or OffsetOldest. Defaults to OffsetNewest.
  240. Initial int64
  241. // The retention duration for committed offsets. If zero, disabled
  242. // (in which case the `offsets.retention.minutes` option on the
  243. // broker will be used). Kafka only supports precision up to
  244. // milliseconds; nanoseconds will be truncated. Requires Kafka
  245. // broker version 0.9.0 or later.
  246. // (default is 0: disabled).
  247. Retention time.Duration
  248. Retry struct {
  249. // The total number of times to retry failing commit
  250. // requests during OffsetManager shutdown (default 3).
  251. Max int
  252. }
  253. }
  254. }
  255. // A user-provided string sent with every request to the brokers for logging,
  256. // debugging, and auditing purposes. Defaults to "sarama", but you should
  257. // probably set it to something specific to your application.
  258. ClientID string
  259. // The number of events to buffer in internal and external channels. This
  260. // permits the producer and consumer to continue processing some messages
  261. // in the background while user code is working, greatly improving throughput.
  262. // Defaults to 256.
  263. ChannelBufferSize int
  264. // The version of Kafka that Sarama will assume it is running against.
  265. // Defaults to the oldest supported stable version. Since Kafka provides
  266. // backwards-compatibility, setting it to a version older than you have
  267. // will not break anything, although it may prevent you from using the
  268. // latest features. Setting it to a version greater than you are actually
  269. // running may lead to random breakage.
  270. Version KafkaVersion
  271. // The registry to define metrics into.
  272. // Defaults to a local registry.
  273. // If you want to disable metrics gathering, set "metrics.UseNilMetrics" to "true"
  274. // prior to starting Sarama.
  275. // See Examples on how to use the metrics registry
  276. MetricRegistry metrics.Registry
  277. }
  278. // NewConfig returns a new configuration instance with sane defaults.
  279. func NewConfig() *Config {
  280. c := &Config{}
  281. c.Admin.Timeout = 3 * time.Second
  282. c.Net.MaxOpenRequests = 5
  283. c.Net.DialTimeout = 30 * time.Second
  284. c.Net.ReadTimeout = 30 * time.Second
  285. c.Net.WriteTimeout = 30 * time.Second
  286. c.Net.SASL.Handshake = true
  287. c.Metadata.Retry.Max = 3
  288. c.Metadata.Retry.Backoff = 250 * time.Millisecond
  289. c.Metadata.RefreshFrequency = 10 * time.Minute
  290. c.Metadata.Full = true
  291. c.Producer.MaxMessageBytes = 1000000
  292. c.Producer.RequiredAcks = WaitForLocal
  293. c.Producer.Timeout = 10 * time.Second
  294. c.Producer.Partitioner = NewHashPartitioner
  295. c.Producer.Retry.Max = 3
  296. c.Producer.Retry.Backoff = 100 * time.Millisecond
  297. c.Producer.Return.Errors = true
  298. c.Producer.CompressionLevel = CompressionLevelDefault
  299. c.Consumer.Fetch.Min = 1
  300. c.Consumer.Fetch.Default = 1024 * 1024
  301. c.Consumer.Retry.Backoff = 2 * time.Second
  302. c.Consumer.MaxWaitTime = 250 * time.Millisecond
  303. c.Consumer.MaxProcessingTime = 100 * time.Millisecond
  304. c.Consumer.Return.Errors = false
  305. c.Consumer.Offsets.CommitInterval = 1 * time.Second
  306. c.Consumer.Offsets.Initial = OffsetNewest
  307. c.Consumer.Offsets.Retry.Max = 3
  308. c.ClientID = defaultClientID
  309. c.ChannelBufferSize = 256
  310. c.Version = MinVersion
  311. c.MetricRegistry = metrics.NewRegistry()
  312. return c
  313. }
  314. // Validate checks a Config instance. It will return a
  315. // ConfigurationError if the specified values don't make sense.
  316. func (c *Config) Validate() error {
  317. // some configuration values should be warned on but not fail completely, do those first
  318. if c.Net.TLS.Enable == false && c.Net.TLS.Config != nil {
  319. Logger.Println("Net.TLS is disabled but a non-nil configuration was provided.")
  320. }
  321. if c.Net.SASL.Enable == false {
  322. if c.Net.SASL.User != "" {
  323. Logger.Println("Net.SASL is disabled but a non-empty username was provided.")
  324. }
  325. if c.Net.SASL.Password != "" {
  326. Logger.Println("Net.SASL is disabled but a non-empty password was provided.")
  327. }
  328. }
  329. if c.Producer.RequiredAcks > 1 {
  330. Logger.Println("Producer.RequiredAcks > 1 is deprecated and will raise an exception with kafka >= 0.8.2.0.")
  331. }
  332. if c.Producer.MaxMessageBytes >= int(MaxRequestSize) {
  333. Logger.Println("Producer.MaxMessageBytes must be smaller than MaxRequestSize; it will be ignored.")
  334. }
  335. if c.Producer.Flush.Bytes >= int(MaxRequestSize) {
  336. Logger.Println("Producer.Flush.Bytes must be smaller than MaxRequestSize; it will be ignored.")
  337. }
  338. if (c.Producer.Flush.Bytes > 0 || c.Producer.Flush.Messages > 0) && c.Producer.Flush.Frequency == 0 {
  339. Logger.Println("Producer.Flush: Bytes or Messages are set, but Frequency is not; messages may not get flushed.")
  340. }
  341. if c.Producer.Timeout%time.Millisecond != 0 {
  342. Logger.Println("Producer.Timeout only supports millisecond resolution; nanoseconds will be truncated.")
  343. }
  344. if c.Consumer.MaxWaitTime < 100*time.Millisecond {
  345. Logger.Println("Consumer.MaxWaitTime is very low, which can cause high CPU and network usage. See documentation for details.")
  346. }
  347. if c.Consumer.MaxWaitTime%time.Millisecond != 0 {
  348. Logger.Println("Consumer.MaxWaitTime only supports millisecond precision; nanoseconds will be truncated.")
  349. }
  350. if c.Consumer.Offsets.Retention%time.Millisecond != 0 {
  351. Logger.Println("Consumer.Offsets.Retention only supports millisecond precision; nanoseconds will be truncated.")
  352. }
  353. if c.ClientID == defaultClientID {
  354. Logger.Println("ClientID is the default of 'sarama', you should consider setting it to something application-specific.")
  355. }
  356. // validate Net values
  357. switch {
  358. case c.Net.MaxOpenRequests <= 0:
  359. return ConfigurationError("Net.MaxOpenRequests must be > 0")
  360. case c.Net.DialTimeout <= 0:
  361. return ConfigurationError("Net.DialTimeout must be > 0")
  362. case c.Net.ReadTimeout <= 0:
  363. return ConfigurationError("Net.ReadTimeout must be > 0")
  364. case c.Net.WriteTimeout <= 0:
  365. return ConfigurationError("Net.WriteTimeout must be > 0")
  366. case c.Net.KeepAlive < 0:
  367. return ConfigurationError("Net.KeepAlive must be >= 0")
  368. case c.Net.SASL.Enable == true && c.Net.SASL.User == "":
  369. return ConfigurationError("Net.SASL.User must not be empty when SASL is enabled")
  370. case c.Net.SASL.Enable == true && c.Net.SASL.Password == "":
  371. return ConfigurationError("Net.SASL.Password must not be empty when SASL is enabled")
  372. }
  373. // validate the Admin values
  374. switch {
  375. case c.Admin.Timeout <= 0:
  376. return ConfigurationError("Admin.Timeout must be > 0")
  377. }
  378. // validate the Metadata values
  379. switch {
  380. case c.Metadata.Retry.Max < 0:
  381. return ConfigurationError("Metadata.Retry.Max must be >= 0")
  382. case c.Metadata.Retry.Backoff < 0:
  383. return ConfigurationError("Metadata.Retry.Backoff must be >= 0")
  384. case c.Metadata.RefreshFrequency < 0:
  385. return ConfigurationError("Metadata.RefreshFrequency must be >= 0")
  386. }
  387. // validate the Producer values
  388. switch {
  389. case c.Producer.MaxMessageBytes <= 0:
  390. return ConfigurationError("Producer.MaxMessageBytes must be > 0")
  391. case c.Producer.RequiredAcks < -1:
  392. return ConfigurationError("Producer.RequiredAcks must be >= -1")
  393. case c.Producer.Timeout <= 0:
  394. return ConfigurationError("Producer.Timeout must be > 0")
  395. case c.Producer.Partitioner == nil:
  396. return ConfigurationError("Producer.Partitioner must not be nil")
  397. case c.Producer.Flush.Bytes < 0:
  398. return ConfigurationError("Producer.Flush.Bytes must be >= 0")
  399. case c.Producer.Flush.Messages < 0:
  400. return ConfigurationError("Producer.Flush.Messages must be >= 0")
  401. case c.Producer.Flush.Frequency < 0:
  402. return ConfigurationError("Producer.Flush.Frequency must be >= 0")
  403. case c.Producer.Flush.MaxMessages < 0:
  404. return ConfigurationError("Producer.Flush.MaxMessages must be >= 0")
  405. case c.Producer.Flush.MaxMessages > 0 && c.Producer.Flush.MaxMessages < c.Producer.Flush.Messages:
  406. return ConfigurationError("Producer.Flush.MaxMessages must be >= Producer.Flush.Messages when set")
  407. case c.Producer.Retry.Max < 0:
  408. return ConfigurationError("Producer.Retry.Max must be >= 0")
  409. case c.Producer.Retry.Backoff < 0:
  410. return ConfigurationError("Producer.Retry.Backoff must be >= 0")
  411. }
  412. if c.Producer.Compression == CompressionLZ4 && !c.Version.IsAtLeast(V0_10_0_0) {
  413. return ConfigurationError("lz4 compression requires Version >= V0_10_0_0")
  414. }
  415. if c.Producer.Compression == CompressionGZIP {
  416. if c.Producer.CompressionLevel != CompressionLevelDefault {
  417. if _, err := gzip.NewWriterLevel(ioutil.Discard, c.Producer.CompressionLevel); err != nil {
  418. return ConfigurationError(fmt.Sprintf("gzip compression does not work with level %d: %v", c.Producer.CompressionLevel, err))
  419. }
  420. }
  421. }
  422. // validate the Consumer values
  423. switch {
  424. case c.Consumer.Fetch.Min <= 0:
  425. return ConfigurationError("Consumer.Fetch.Min must be > 0")
  426. case c.Consumer.Fetch.Default <= 0:
  427. return ConfigurationError("Consumer.Fetch.Default must be > 0")
  428. case c.Consumer.Fetch.Max < 0:
  429. return ConfigurationError("Consumer.Fetch.Max must be >= 0")
  430. case c.Consumer.MaxWaitTime < 1*time.Millisecond:
  431. return ConfigurationError("Consumer.MaxWaitTime must be >= 1ms")
  432. case c.Consumer.MaxProcessingTime <= 0:
  433. return ConfigurationError("Consumer.MaxProcessingTime must be > 0")
  434. case c.Consumer.Retry.Backoff < 0:
  435. return ConfigurationError("Consumer.Retry.Backoff must be >= 0")
  436. case c.Consumer.Offsets.CommitInterval <= 0:
  437. return ConfigurationError("Consumer.Offsets.CommitInterval must be > 0")
  438. case c.Consumer.Offsets.Initial != OffsetOldest && c.Consumer.Offsets.Initial != OffsetNewest:
  439. return ConfigurationError("Consumer.Offsets.Initial must be OffsetOldest or OffsetNewest")
  440. case c.Consumer.Offsets.Retry.Max < 0:
  441. return ConfigurationError("Consumer.Offsets.Retry.Max must be >= 0")
  442. }
  443. // validate misc shared values
  444. switch {
  445. case c.ChannelBufferSize < 0:
  446. return ConfigurationError("ChannelBufferSize must be >= 0")
  447. case !validID.MatchString(c.ClientID):
  448. return ConfigurationError("ClientID is invalid")
  449. }
  450. return nil
  451. }