config.go 13 KB

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