config.go 14 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330331332333334335336337338339340341342343344345346347348349350351352
  1. package sarama
  2. import (
  3. "crypto/tls"
  4. "time"
  5. )
  6. // Config is used to pass multiple configuration options to Sarama's constructors.
  7. type Config struct {
  8. // Net is the namespace for network-level properties used by the Broker, and
  9. // shared by the Client/Producer/Consumer.
  10. Net struct {
  11. // How many outstanding requests a connection is allowed to have before
  12. // sending on it blocks (default 5).
  13. MaxOpenRequests int
  14. // All three of the below configurations are similar to the
  15. // `socket.timeout.ms` setting in JVM kafka. All of them default
  16. // to 30 seconds.
  17. DialTimeout time.Duration // How long to wait for the initial connection.
  18. ReadTimeout time.Duration // How long to wait for a response.
  19. WriteTimeout time.Duration // How long to wait for a transmit.
  20. TLS struct {
  21. // Whether or not to use TLS when connecting to the broker
  22. // (defaults to false).
  23. Enable bool
  24. // The TLS configuration to use for secure connections if
  25. // enabled (defaults to nil).
  26. Config *tls.Config
  27. }
  28. // KeepAlive specifies the keep-alive period for an active network connection.
  29. // If zero, keep-alives are disabled. (default is 0: disabled).
  30. KeepAlive time.Duration
  31. }
  32. // Metadata is the namespace for metadata management properties used by the
  33. // Client, and shared by the Producer/Consumer.
  34. Metadata struct {
  35. Retry struct {
  36. // The total number of times to retry a metadata request when the
  37. // cluster is in the middle of a leader election (default 3).
  38. Max int
  39. // How long to wait for leader election to occur before retrying
  40. // (default 250ms). Similar to the JVM's `retry.backoff.ms`.
  41. Backoff time.Duration
  42. }
  43. // How frequently to refresh the cluster metadata in the background.
  44. // Defaults to 10 minutes. Set to 0 to disable. Similar to
  45. // `topic.metadata.refresh.interval.ms` in the JVM version.
  46. RefreshFrequency time.Duration
  47. }
  48. // Producer is the namespace for configuration related to producing messages,
  49. // used by the Producer.
  50. Producer struct {
  51. // The maximum permitted size of a message (defaults to 1000000). Should be
  52. // set equal to or smaller than the broker's `message.max.bytes`.
  53. MaxMessageBytes int
  54. // The level of acknowledgement reliability needed from the broker (defaults
  55. // to WaitForLocal). Equivalent to the `request.required.acks` setting of the
  56. // JVM producer.
  57. RequiredAcks RequiredAcks
  58. // The maximum duration the broker will wait the receipt of the number of
  59. // RequiredAcks (defaults to 10 seconds). This is only relevant when
  60. // RequiredAcks is set to WaitForAll or a number > 1. Only supports
  61. // millisecond resolution, nanoseconds will be truncated. Equivalent to
  62. // the JVM producer's `request.timeout.ms` setting.
  63. Timeout time.Duration
  64. // The type of compression to use on messages (defaults to no compression).
  65. // Similar to `compression.codec` setting of the JVM producer.
  66. Compression CompressionCodec
  67. // Generates partitioners for choosing the partition to send messages to
  68. // (defaults to hashing the message key). Similar to the `partitioner.class`
  69. // setting for the JVM producer.
  70. Partitioner PartitionerConstructor
  71. // Return specifies what channels will be populated. If they are set to true,
  72. // you must read from the respective channels to prevent deadlock.
  73. Return struct {
  74. // If enabled, successfully delivered messages will be returned on the
  75. // Successes channel (default disabled).
  76. Successes bool
  77. // If enabled, messages that failed to deliver will be returned on the
  78. // Errors channel, including error (default enabled).
  79. Errors bool
  80. }
  81. // The following config options control how often messages are batched up and
  82. // sent to the broker. By default, messages are sent as fast as possible, and
  83. // all messages received while the current batch is in-flight are placed
  84. // into the subsequent batch.
  85. Flush struct {
  86. // The best-effort number of bytes needed to trigger a flush. Use the
  87. // global sarama.MaxRequestSize to set a hard upper limit.
  88. Bytes int
  89. // The best-effort number of messages needed to trigger a flush. Use
  90. // `MaxMessages` to set a hard upper limit.
  91. Messages int
  92. // The best-effort frequency of flushes. Equivalent to
  93. // `queue.buffering.max.ms` setting of JVM producer.
  94. Frequency time.Duration
  95. // The maximum number of messages the producer will send in a single
  96. // broker request. Defaults to 0 for unlimited. Similar to
  97. // `queue.buffering.max.messages` in the JVM producer.
  98. MaxMessages int
  99. }
  100. Retry struct {
  101. // The total number of times to retry sending a message (default 3).
  102. // Similar to the `message.send.max.retries` setting of the JVM producer.
  103. Max int
  104. // How long to wait for the cluster to settle between retries
  105. // (default 100ms). Similar to the `retry.backoff.ms` setting of the
  106. // JVM producer.
  107. Backoff time.Duration
  108. }
  109. }
  110. // Consumer is the namespace for configuration related to consuming messages,
  111. // used by the Consumer.
  112. Consumer struct {
  113. Retry struct {
  114. // How long to wait after a failing to read from a partition before
  115. // trying again (default 2s).
  116. Backoff time.Duration
  117. }
  118. // Fetch is the namespace for controlling how many bytes are retrieved by any
  119. // given request.
  120. Fetch struct {
  121. // The minimum number of message bytes to fetch in a request - the broker
  122. // will wait until at least this many are available. The default is 1,
  123. // as 0 causes the consumer to spin when no messages are available.
  124. // Equivalent to the JVM's `fetch.min.bytes`.
  125. Min int32
  126. // The default number of message bytes to fetch from the broker in each
  127. // request (default 32768). This should be larger than the majority of
  128. // your messages, or else the consumer will spend a lot of time
  129. // negotiating sizes and not actually consuming. Similar to the JVM's
  130. // `fetch.message.max.bytes`.
  131. Default int32
  132. // The maximum number of message bytes to fetch from the broker in a
  133. // single request. Messages larger than this will return
  134. // ErrMessageTooLarge and will not be consumable, so you must be sure
  135. // this is at least as large as your largest message. Defaults to 0
  136. // (no limit). Similar to the JVM's `fetch.message.max.bytes`. The
  137. // global `sarama.MaxResponseSize` still applies.
  138. Max int32
  139. }
  140. // The maximum amount of time the broker will wait for Consumer.Fetch.Min
  141. // bytes to become available before it returns fewer than that anyways. The
  142. // default is 250ms, since 0 causes the consumer to spin when no events are
  143. // available. 100-500ms is a reasonable range for most cases. Kafka only
  144. // supports precision up to milliseconds; nanoseconds will be truncated.
  145. // Equivalent to the JVM's `fetch.wait.max.ms`.
  146. MaxWaitTime time.Duration
  147. // The maximum amount of time the consumer expects a message takes to process
  148. // for the user. If writing to the Messages channel takes longer than this,
  149. // that partition will stop fetching more messages until it can proceed again.
  150. // Note that, since the Messages channel is buffered, the actual grace time is
  151. // (MaxProcessingTime * ChanneBufferSize). Defaults to 100ms.
  152. MaxProcessingTime time.Duration
  153. // Return specifies what channels will be populated. If they are set to true,
  154. // you must read from them to prevent deadlock.
  155. Return struct {
  156. // If enabled, any errors that occured while consuming are returned on
  157. // the Errors channel (default disabled).
  158. Errors bool
  159. }
  160. // Offsets specifies configuration for how and when to commit consumed
  161. // offsets. This currently requires the manual use of an OffsetManager
  162. // but will eventually be automated.
  163. Offsets struct {
  164. // How frequently to commit updated offsets. Defaults to 1s.
  165. CommitInterval time.Duration
  166. // The initial offset to use if no offset was previously committed.
  167. // Should be OffsetNewest or OffsetOldest. Defaults to OffsetNewest.
  168. Initial int64
  169. // The retention duration for committed offsets. If zero, disabled
  170. // (in which case the `offsets.retention.minutes` option on the
  171. // broker will be used). Kafka only supports precision up to
  172. // milliseconds; nanoseconds will be truncated.
  173. // (default is 0: disabled).
  174. Retention time.Duration
  175. }
  176. }
  177. // A user-provided string sent with every request to the brokers for logging,
  178. // debugging, and auditing purposes. Defaults to "sarama", but you should
  179. // probably set it to something specific to your application.
  180. ClientID string
  181. // The number of events to buffer in internal and external channels. This
  182. // permits the producer and consumer to continue processing some messages
  183. // in the background while user code is working, greatly improving throughput.
  184. // Defaults to 256.
  185. ChannelBufferSize int
  186. }
  187. // NewConfig returns a new configuration instance with sane defaults.
  188. func NewConfig() *Config {
  189. c := &Config{}
  190. c.Net.MaxOpenRequests = 5
  191. c.Net.DialTimeout = 30 * time.Second
  192. c.Net.ReadTimeout = 30 * time.Second
  193. c.Net.WriteTimeout = 30 * time.Second
  194. c.Metadata.Retry.Max = 3
  195. c.Metadata.Retry.Backoff = 250 * time.Millisecond
  196. c.Metadata.RefreshFrequency = 10 * time.Minute
  197. c.Producer.MaxMessageBytes = 1000000
  198. c.Producer.RequiredAcks = WaitForLocal
  199. c.Producer.Timeout = 10 * time.Second
  200. c.Producer.Partitioner = NewHashPartitioner
  201. c.Producer.Retry.Max = 3
  202. c.Producer.Retry.Backoff = 100 * time.Millisecond
  203. c.Producer.Return.Errors = true
  204. c.Consumer.Fetch.Min = 1
  205. c.Consumer.Fetch.Default = 32768
  206. c.Consumer.Retry.Backoff = 2 * time.Second
  207. c.Consumer.MaxWaitTime = 250 * time.Millisecond
  208. c.Consumer.MaxProcessingTime = 100 * time.Millisecond
  209. c.Consumer.Return.Errors = false
  210. c.Consumer.Offsets.CommitInterval = 1 * time.Second
  211. c.Consumer.Offsets.Initial = OffsetNewest
  212. c.ChannelBufferSize = 256
  213. return c
  214. }
  215. // Validate checks a Config instance. It will return a
  216. // ConfigurationError if the specified values don't make sense.
  217. func (c *Config) Validate() error {
  218. // some configuration values should be warned on but not fail completely, do those first
  219. if c.Net.TLS.Enable == false && c.Net.TLS.Config != nil {
  220. Logger.Println("Net.TLS is disabled but a non-nil configuration was provided.")
  221. }
  222. if c.Producer.RequiredAcks > 1 {
  223. Logger.Println("Producer.RequiredAcks > 1 is deprecated and will raise an exception with kafka >= 0.8.2.0.")
  224. }
  225. if c.Producer.MaxMessageBytes >= int(MaxRequestSize) {
  226. Logger.Println("Producer.MaxMessageBytes is larger than MaxRequestSize; it will be ignored.")
  227. }
  228. if c.Producer.Flush.Bytes >= int(MaxRequestSize) {
  229. Logger.Println("Producer.Flush.Bytes is larger than MaxRequestSize; it will be ignored.")
  230. }
  231. if c.Producer.Timeout%time.Millisecond != 0 {
  232. Logger.Println("Producer.Timeout only supports millisecond resolution; nanoseconds will be truncated.")
  233. }
  234. if c.Consumer.MaxWaitTime < 100*time.Millisecond {
  235. Logger.Println("Consumer.MaxWaitTime is very low, which can cause high CPU and network usage. See documentation for details.")
  236. }
  237. if c.Consumer.MaxWaitTime%time.Millisecond != 0 {
  238. Logger.Println("Consumer.MaxWaitTime only supports millisecond precision; nanoseconds will be truncated.")
  239. }
  240. if c.Consumer.Offsets.Retention%time.Millisecond != 0 {
  241. Logger.Println("Consumer.Offsets.Retention only supports millisecond precision; nanoseconds will be truncated.")
  242. }
  243. if c.ClientID == "sarama" {
  244. Logger.Println("ClientID is the default of 'sarama', you should consider setting it to something application-specific.")
  245. }
  246. // validate Net values
  247. switch {
  248. case c.Net.MaxOpenRequests <= 0:
  249. return ConfigurationError("Net.MaxOpenRequests must be > 0")
  250. case c.Net.DialTimeout <= 0:
  251. return ConfigurationError("Net.DialTimeout must be > 0")
  252. case c.Net.ReadTimeout <= 0:
  253. return ConfigurationError("Net.ReadTimeout must be > 0")
  254. case c.Net.WriteTimeout <= 0:
  255. return ConfigurationError("Net.WriteTimeout must be > 0")
  256. case c.Net.KeepAlive < 0:
  257. return ConfigurationError("Net.KeepAlive must be >= 0")
  258. }
  259. // validate the Metadata values
  260. switch {
  261. case c.Metadata.Retry.Max < 0:
  262. return ConfigurationError("Metadata.Retry.Max must be >= 0")
  263. case c.Metadata.Retry.Backoff < 0:
  264. return ConfigurationError("Metadata.Retry.Backoff must be >= 0")
  265. case c.Metadata.RefreshFrequency < 0:
  266. return ConfigurationError("Metadata.RefreshFrequency must be >= 0")
  267. }
  268. // validate the Producer values
  269. switch {
  270. case c.Producer.MaxMessageBytes <= 0:
  271. return ConfigurationError("Producer.MaxMessageBytes must be > 0")
  272. case c.Producer.RequiredAcks < -1:
  273. return ConfigurationError("Producer.RequiredAcks must be >= -1")
  274. case c.Producer.Timeout <= 0:
  275. return ConfigurationError("Producer.Timeout must be > 0")
  276. case c.Producer.Partitioner == nil:
  277. return ConfigurationError("Producer.Partitioner must not be nil")
  278. case c.Producer.Flush.Bytes < 0:
  279. return ConfigurationError("Producer.Flush.Bytes must be >= 0")
  280. case c.Producer.Flush.Messages < 0:
  281. return ConfigurationError("Producer.Flush.Messages must be >= 0")
  282. case c.Producer.Flush.Frequency < 0:
  283. return ConfigurationError("Producer.Flush.Frequency must be >= 0")
  284. case c.Producer.Flush.MaxMessages < 0:
  285. return ConfigurationError("Producer.Flush.MaxMessages must be >= 0")
  286. case c.Producer.Flush.MaxMessages > 0 && c.Producer.Flush.MaxMessages < c.Producer.Flush.Messages:
  287. return ConfigurationError("Producer.Flush.MaxMessages must be >= Producer.Flush.Messages when set")
  288. case c.Producer.Retry.Max < 0:
  289. return ConfigurationError("Producer.Retry.Max must be >= 0")
  290. case c.Producer.Retry.Backoff < 0:
  291. return ConfigurationError("Producer.Retry.Backoff must be >= 0")
  292. }
  293. // validate the Consumer values
  294. switch {
  295. case c.Consumer.Fetch.Min <= 0:
  296. return ConfigurationError("Consumer.Fetch.Min must be > 0")
  297. case c.Consumer.Fetch.Default <= 0:
  298. return ConfigurationError("Consumer.Fetch.Default must be > 0")
  299. case c.Consumer.Fetch.Max < 0:
  300. return ConfigurationError("Consumer.Fetch.Max must be >= 0")
  301. case c.Consumer.MaxWaitTime < 1*time.Millisecond:
  302. return ConfigurationError("Consumer.MaxWaitTime must be >= 1ms")
  303. case c.Consumer.MaxProcessingTime <= 0:
  304. return ConfigurationError("Consumer.MaxProcessingTime must be > 0")
  305. case c.Consumer.Retry.Backoff < 0:
  306. return ConfigurationError("Consumer.Retry.Backoff must be >= 0")
  307. case c.Consumer.Offsets.CommitInterval <= 0:
  308. return ConfigurationError("Consumer.Offsets.CommitInterval must be > 0")
  309. case c.Consumer.Offsets.Initial != OffsetOldest && c.Consumer.Offsets.Initial != OffsetNewest:
  310. return ConfigurationError("Consumer.Offsets.Initial must be OffsetOldest or OffsetNewest")
  311. }
  312. // validate misc shared values
  313. switch {
  314. case c.ChannelBufferSize < 0:
  315. return ConfigurationError("ChannelBufferSize must be >= 0")
  316. }
  317. return nil
  318. }