config.go 32 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330331332333334335336337338339340341342343344345346347348349350351352353354355356357358359360361362363364365366367368369370371372373374375376377378379380381382383384385386387388389390391392393394395396397398399400401402403404405406407408409410411412413414415416417418419420421422423424425426427428429430431432433434435436437438439440441442443444445446447448449450451452453454455456457458459460461462463464465466467468469470471472473474475476477478479480481482483484485486487488489490491492493494495496497498499500501502503504505506507508509510511512513514515516517518519520521522523524525526527528529530531532533534535536537538539540541542543544545546547548549550551552553554555556557558559560561562563564565566567568569570571572573574575576577578579580581582583584585586587588589590591592593594595596597598599600601602603604605606607608609610611612613614615616617618619620621622623624625626627628629630631632633634635636637638639640641642643644645646647648649650651652653654655656657658659660661662663664665666667668669670671672673674675676677678679680681682683684685686687688689690691692693694695696697698699700701702703704705706707708709710711712713714715716717718719720721722723724725726727728729730731732733734735736737
  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. "golang.org/x/net/proxy"
  12. )
  13. const defaultClientID = "sarama"
  14. var validID = regexp.MustCompile(`\A[A-Za-z0-9._-]+\z`)
  15. // Config is used to pass multiple configuration options to Sarama's constructors.
  16. type Config struct {
  17. // Admin is the namespace for ClusterAdmin properties used by the administrative Kafka client.
  18. Admin struct {
  19. Retry struct {
  20. // The total number of times to retry sending (retriable) admin requests (default 5).
  21. // Similar to the `retries` setting of the JVM AdminClientConfig.
  22. Max int
  23. // Backoff time between retries of a failed request (default 100ms)
  24. Backoff time.Duration
  25. }
  26. // The maximum duration the administrative Kafka client will wait for ClusterAdmin operations,
  27. // including topics, brokers, configurations and ACLs (defaults to 3 seconds).
  28. Timeout time.Duration
  29. }
  30. // Net is the namespace for network-level properties used by the Broker, and
  31. // shared by the Client/Producer/Consumer.
  32. Net struct {
  33. // How many outstanding requests a connection is allowed to have before
  34. // sending on it blocks (default 5).
  35. MaxOpenRequests int
  36. // All three of the below configurations are similar to the
  37. // `socket.timeout.ms` setting in JVM kafka. All of them default
  38. // to 30 seconds.
  39. DialTimeout time.Duration // How long to wait for the initial connection.
  40. ReadTimeout time.Duration // How long to wait for a response.
  41. WriteTimeout time.Duration // How long to wait for a transmit.
  42. TLS struct {
  43. // Whether or not to use TLS when connecting to the broker
  44. // (defaults to false).
  45. Enable bool
  46. // The TLS configuration to use for secure connections if
  47. // enabled (defaults to nil).
  48. Config *tls.Config
  49. }
  50. // SASL based authentication with broker. While there are multiple SASL authentication methods
  51. // the current implementation is limited to plaintext (SASL/PLAIN) authentication
  52. SASL struct {
  53. // Whether or not to use SASL authentication when connecting to the broker
  54. // (defaults to false).
  55. Enable bool
  56. // SASLMechanism is the name of the enabled SASL mechanism.
  57. // Possible values: OAUTHBEARER, PLAIN (defaults to PLAIN).
  58. Mechanism SASLMechanism
  59. // Version is the SASL Protocol Version to use
  60. // Kafka > 1.x should use V1, except on Azure EventHub which use V0
  61. Version int16
  62. // Whether or not to send the Kafka SASL handshake first if enabled
  63. // (defaults to true). You should only set this to false if you're using
  64. // a non-Kafka SASL proxy.
  65. Handshake bool
  66. // AuthIdentity is an (optional) authorization identity (authzid) to
  67. // use for SASL/PLAIN authentication (if different from User) when
  68. // an authenticated user is permitted to act as the presented
  69. // alternative user. See RFC4616 for details.
  70. AuthIdentity string
  71. // User is the authentication identity (authcid) to present for
  72. // SASL/PLAIN or SASL/SCRAM authentication
  73. User string
  74. // Password for SASL/PLAIN authentication
  75. Password string
  76. // authz id used for SASL/SCRAM authentication
  77. SCRAMAuthzID string
  78. // SCRAMClientGeneratorFunc is a generator of a user provided implementation of a SCRAM
  79. // client used to perform the SCRAM exchange with the server.
  80. SCRAMClientGeneratorFunc func() SCRAMClient
  81. // TokenProvider is a user-defined callback for generating
  82. // access tokens for SASL/OAUTHBEARER auth. See the
  83. // AccessTokenProvider interface docs for proper implementation
  84. // guidelines.
  85. TokenProvider AccessTokenProvider
  86. GSSAPI GSSAPIConfig
  87. }
  88. // KeepAlive specifies the keep-alive period for an active network connection.
  89. // If zero, keep-alives are disabled. (default is 0: disabled).
  90. KeepAlive time.Duration
  91. // LocalAddr is the local address to use when dialing an
  92. // address. The address must be of a compatible type for the
  93. // network being dialed.
  94. // If nil, a local address is automatically chosen.
  95. LocalAddr net.Addr
  96. Proxy struct {
  97. // Whether or not to use proxy when connecting to the broker
  98. // (defaults to false).
  99. Enable bool
  100. // The proxy dialer to use enabled (defaults to nil).
  101. Dialer proxy.Dialer
  102. }
  103. }
  104. // Metadata is the namespace for metadata management properties used by the
  105. // Client, and shared by the Producer/Consumer.
  106. Metadata struct {
  107. Retry struct {
  108. // The total number of times to retry a metadata request when the
  109. // cluster is in the middle of a leader election (default 3).
  110. Max int
  111. // How long to wait for leader election to occur before retrying
  112. // (default 250ms). Similar to the JVM's `retry.backoff.ms`.
  113. Backoff time.Duration
  114. // Called to compute backoff time dynamically. Useful for implementing
  115. // more sophisticated backoff strategies. This takes precedence over
  116. // `Backoff` if set.
  117. BackoffFunc func(retries, maxRetries int) time.Duration
  118. }
  119. // How frequently to refresh the cluster metadata in the background.
  120. // Defaults to 10 minutes. Set to 0 to disable. Similar to
  121. // `topic.metadata.refresh.interval.ms` in the JVM version.
  122. RefreshFrequency time.Duration
  123. // Whether to maintain a full set of metadata for all topics, or just
  124. // the minimal set that has been necessary so far. The full set is simpler
  125. // and usually more convenient, but can take up a substantial amount of
  126. // memory if you have many topics and partitions. Defaults to true.
  127. Full bool
  128. // How long to wait for a successful metadata response.
  129. // Disabled by default which means a metadata request against an unreachable
  130. // cluster (all brokers are unreachable or unresponsive) can take up to
  131. // `Net.[Dial|Read]Timeout * BrokerCount * (Metadata.Retry.Max + 1) + Metadata.Retry.Backoff * Metadata.Retry.Max`
  132. // to fail.
  133. Timeout time.Duration
  134. }
  135. // Producer is the namespace for configuration related to producing messages,
  136. // used by the Producer.
  137. Producer struct {
  138. // The maximum permitted size of a message (defaults to 1000000). Should be
  139. // set equal to or smaller than the broker's `message.max.bytes`.
  140. MaxMessageBytes int
  141. // The level of acknowledgement reliability needed from the broker (defaults
  142. // to WaitForLocal). Equivalent to the `request.required.acks` setting of the
  143. // JVM producer.
  144. RequiredAcks RequiredAcks
  145. // The maximum duration the broker will wait the receipt of the number of
  146. // RequiredAcks (defaults to 10 seconds). This is only relevant when
  147. // RequiredAcks is set to WaitForAll or a number > 1. Only supports
  148. // millisecond resolution, nanoseconds will be truncated. Equivalent to
  149. // the JVM producer's `request.timeout.ms` setting.
  150. Timeout time.Duration
  151. // The type of compression to use on messages (defaults to no compression).
  152. // Similar to `compression.codec` setting of the JVM producer.
  153. Compression CompressionCodec
  154. // The level of compression to use on messages. The meaning depends
  155. // on the actual compression type used and defaults to default compression
  156. // level for the codec.
  157. CompressionLevel int
  158. // Generates partitioners for choosing the partition to send messages to
  159. // (defaults to hashing the message key). Similar to the `partitioner.class`
  160. // setting for the JVM producer.
  161. Partitioner PartitionerConstructor
  162. // If enabled, the producer will ensure that exactly one copy of each message is
  163. // written.
  164. Idempotent bool
  165. // Return specifies what channels will be populated. If they are set to true,
  166. // you must read from the respective channels to prevent deadlock. If,
  167. // however, this config is used to create a `SyncProducer`, both must be set
  168. // to true and you shall not read from the channels since the producer does
  169. // this internally.
  170. Return struct {
  171. // If enabled, successfully delivered messages will be returned on the
  172. // Successes channel (default disabled).
  173. Successes bool
  174. // If enabled, messages that failed to deliver will be returned on the
  175. // Errors channel, including error (default enabled).
  176. Errors bool
  177. }
  178. // The following config options control how often messages are batched up and
  179. // sent to the broker. By default, messages are sent as fast as possible, and
  180. // all messages received while the current batch is in-flight are placed
  181. // into the subsequent batch.
  182. Flush struct {
  183. // The best-effort number of bytes needed to trigger a flush. Use the
  184. // global sarama.MaxRequestSize to set a hard upper limit.
  185. Bytes int
  186. // The best-effort number of messages needed to trigger a flush. Use
  187. // `MaxMessages` to set a hard upper limit.
  188. Messages int
  189. // The best-effort frequency of flushes. Equivalent to
  190. // `queue.buffering.max.ms` setting of JVM producer.
  191. Frequency time.Duration
  192. // The maximum number of messages the producer will send in a single
  193. // broker request. Defaults to 0 for unlimited. Similar to
  194. // `queue.buffering.max.messages` in the JVM producer.
  195. MaxMessages int
  196. }
  197. Retry struct {
  198. // The total number of times to retry sending a message (default 3).
  199. // Similar to the `message.send.max.retries` setting of the JVM producer.
  200. Max int
  201. // How long to wait for the cluster to settle between retries
  202. // (default 100ms). Similar to the `retry.backoff.ms` setting of the
  203. // JVM producer.
  204. Backoff time.Duration
  205. // Called to compute backoff time dynamically. Useful for implementing
  206. // more sophisticated backoff strategies. This takes precedence over
  207. // `Backoff` if set.
  208. BackoffFunc func(retries, maxRetries int) time.Duration
  209. }
  210. }
  211. // Consumer is the namespace for configuration related to consuming messages,
  212. // used by the Consumer.
  213. Consumer struct {
  214. // Group is the namespace for configuring consumer group.
  215. Group struct {
  216. Session struct {
  217. // The timeout used to detect consumer failures when using Kafka's group management facility.
  218. // The consumer sends periodic heartbeats to indicate its liveness to the broker.
  219. // If no heartbeats are received by the broker before the expiration of this session timeout,
  220. // then the broker will remove this consumer from the group and initiate a rebalance.
  221. // Note that the value must be in the allowable range as configured in the broker configuration
  222. // by `group.min.session.timeout.ms` and `group.max.session.timeout.ms` (default 10s)
  223. Timeout time.Duration
  224. }
  225. Heartbeat struct {
  226. // The expected time between heartbeats to the consumer coordinator when using Kafka's group
  227. // management facilities. Heartbeats are used to ensure that the consumer's session stays active and
  228. // to facilitate rebalancing when new consumers join or leave the group.
  229. // The value must be set lower than Consumer.Group.Session.Timeout, but typically should be set no
  230. // higher than 1/3 of that value.
  231. // It can be adjusted even lower to control the expected time for normal rebalances (default 3s)
  232. Interval time.Duration
  233. }
  234. Rebalance struct {
  235. // Strategy for allocating topic partitions to members (default BalanceStrategyRange)
  236. Strategy BalanceStrategy
  237. // The maximum allowed time for each worker to join the group once a rebalance has begun.
  238. // This is basically a limit on the amount of time needed for all tasks to flush any pending
  239. // data and commit offsets. If the timeout is exceeded, then the worker will be removed from
  240. // the group, which will cause offset commit failures (default 60s).
  241. Timeout time.Duration
  242. Retry struct {
  243. // When a new consumer joins a consumer group the set of consumers attempt to "rebalance"
  244. // the load to assign partitions to each consumer. If the set of consumers changes while
  245. // this assignment is taking place the rebalance will fail and retry. This setting controls
  246. // the maximum number of attempts before giving up (default 4).
  247. Max int
  248. // Backoff time between retries during rebalance (default 2s)
  249. Backoff time.Duration
  250. }
  251. }
  252. Member struct {
  253. // Custom metadata to include when joining the group. The user data for all joined members
  254. // can be retrieved by sending a DescribeGroupRequest to the broker that is the
  255. // coordinator for the group.
  256. UserData []byte
  257. }
  258. }
  259. Retry struct {
  260. // How long to wait after a failing to read from a partition before
  261. // trying again (default 2s).
  262. Backoff time.Duration
  263. // Called to compute backoff time dynamically. Useful for implementing
  264. // more sophisticated backoff strategies. This takes precedence over
  265. // `Backoff` if set.
  266. BackoffFunc func(retries int) time.Duration
  267. }
  268. // Fetch is the namespace for controlling how many bytes are retrieved by any
  269. // given request.
  270. Fetch struct {
  271. // The minimum number of message bytes to fetch in a request - the broker
  272. // will wait until at least this many are available. The default is 1,
  273. // as 0 causes the consumer to spin when no messages are available.
  274. // Equivalent to the JVM's `fetch.min.bytes`.
  275. Min int32
  276. // The default number of message bytes to fetch from the broker in each
  277. // request (default 1MB). This should be larger than the majority of
  278. // your messages, or else the consumer will spend a lot of time
  279. // negotiating sizes and not actually consuming. Similar to the JVM's
  280. // `fetch.message.max.bytes`.
  281. Default int32
  282. // The maximum number of message bytes to fetch from the broker in a
  283. // single request. Messages larger than this will return
  284. // ErrMessageTooLarge and will not be consumable, so you must be sure
  285. // this is at least as large as your largest message. Defaults to 0
  286. // (no limit). Similar to the JVM's `fetch.message.max.bytes`. The
  287. // global `sarama.MaxResponseSize` still applies.
  288. Max int32
  289. }
  290. // The maximum amount of time the broker will wait for Consumer.Fetch.Min
  291. // bytes to become available before it returns fewer than that anyways. The
  292. // default is 250ms, since 0 causes the consumer to spin when no events are
  293. // available. 100-500ms is a reasonable range for most cases. Kafka only
  294. // supports precision up to milliseconds; nanoseconds will be truncated.
  295. // Equivalent to the JVM's `fetch.wait.max.ms`.
  296. MaxWaitTime time.Duration
  297. // The maximum amount of time the consumer expects a message takes to
  298. // process for the user. If writing to the Messages channel takes longer
  299. // than this, that partition will stop fetching more messages until it
  300. // can proceed again.
  301. // Note that, since the Messages channel is buffered, the actual grace time is
  302. // (MaxProcessingTime * ChannelBufferSize). Defaults to 100ms.
  303. // If a message is not written to the Messages channel between two ticks
  304. // of the expiryTicker then a timeout is detected.
  305. // Using a ticker instead of a timer to detect timeouts should typically
  306. // result in many fewer calls to Timer functions which may result in a
  307. // significant performance improvement if many messages are being sent
  308. // and timeouts are infrequent.
  309. // The disadvantage of using a ticker instead of a timer is that
  310. // timeouts will be less accurate. That is, the effective timeout could
  311. // be between `MaxProcessingTime` and `2 * MaxProcessingTime`. For
  312. // example, if `MaxProcessingTime` is 100ms then a delay of 180ms
  313. // between two messages being sent may not be recognized as a timeout.
  314. MaxProcessingTime time.Duration
  315. // Return specifies what channels will be populated. If they are set to true,
  316. // you must read from them to prevent deadlock.
  317. Return struct {
  318. // If enabled, any errors that occurred while consuming are returned on
  319. // the Errors channel (default disabled).
  320. Errors bool
  321. }
  322. // Offsets specifies configuration for how and when to commit consumed
  323. // offsets. This currently requires the manual use of an OffsetManager
  324. // but will eventually be automated.
  325. Offsets struct {
  326. // Deprecated: CommitInterval exists for historical compatibility
  327. // and should not be used. Please use Consumer.Offsets.AutoCommit
  328. CommitInterval time.Duration
  329. // AutoCommit specifies configuration for commit messages automatically.
  330. AutoCommit struct {
  331. // Whether or not to auto-commit updated offsets back to the broker.
  332. // (default enabled).
  333. Enable bool
  334. // How frequently to commit updated offsets. Ineffective unless
  335. // auto-commit is enabled (default 1s)
  336. Interval time.Duration
  337. }
  338. // The initial offset to use if no offset was previously committed.
  339. // Should be OffsetNewest or OffsetOldest. Defaults to OffsetNewest.
  340. Initial int64
  341. // The retention duration for committed offsets. If zero, disabled
  342. // (in which case the `offsets.retention.minutes` option on the
  343. // broker will be used). Kafka only supports precision up to
  344. // milliseconds; nanoseconds will be truncated. Requires Kafka
  345. // broker version 0.9.0 or later.
  346. // (default is 0: disabled).
  347. Retention time.Duration
  348. Retry struct {
  349. // The total number of times to retry failing commit
  350. // requests during OffsetManager shutdown (default 3).
  351. Max int
  352. }
  353. }
  354. // IsolationLevel support 2 mode:
  355. // - use `ReadUncommitted` (default) to consume and return all messages in message channel
  356. // - use `ReadCommitted` to hide messages that are part of an aborted transaction
  357. IsolationLevel IsolationLevel
  358. }
  359. // A user-provided string sent with every request to the brokers for logging,
  360. // debugging, and auditing purposes. Defaults to "sarama", but you should
  361. // probably set it to something specific to your application.
  362. ClientID string
  363. // A rack identifier for this client. This can be any string value which
  364. // indicates where this client is physically located.
  365. // It corresponds with the broker config 'broker.rack'
  366. RackID string
  367. // The number of events to buffer in internal and external channels. This
  368. // permits the producer and consumer to continue processing some messages
  369. // in the background while user code is working, greatly improving throughput.
  370. // Defaults to 256.
  371. ChannelBufferSize int
  372. // The version of Kafka that Sarama will assume it is running against.
  373. // Defaults to the oldest supported stable version. Since Kafka provides
  374. // backwards-compatibility, setting it to a version older than you have
  375. // will not break anything, although it may prevent you from using the
  376. // latest features. Setting it to a version greater than you are actually
  377. // running may lead to random breakage.
  378. Version KafkaVersion
  379. // The registry to define metrics into.
  380. // Defaults to a local registry.
  381. // If you want to disable metrics gathering, set "metrics.UseNilMetrics" to "true"
  382. // prior to starting Sarama.
  383. // See Examples on how to use the metrics registry
  384. MetricRegistry metrics.Registry
  385. }
  386. // NewConfig returns a new configuration instance with sane defaults.
  387. func NewConfig() *Config {
  388. c := &Config{}
  389. c.Admin.Retry.Max = 5
  390. c.Admin.Retry.Backoff = 100 * time.Millisecond
  391. c.Admin.Timeout = 3 * time.Second
  392. c.Net.MaxOpenRequests = 5
  393. c.Net.DialTimeout = 30 * time.Second
  394. c.Net.ReadTimeout = 30 * time.Second
  395. c.Net.WriteTimeout = 30 * time.Second
  396. c.Net.SASL.Handshake = true
  397. c.Net.SASL.Version = SASLHandshakeV0
  398. c.Metadata.Retry.Max = 3
  399. c.Metadata.Retry.Backoff = 250 * time.Millisecond
  400. c.Metadata.RefreshFrequency = 10 * time.Minute
  401. c.Metadata.Full = true
  402. c.Producer.MaxMessageBytes = 1000000
  403. c.Producer.RequiredAcks = WaitForLocal
  404. c.Producer.Timeout = 10 * time.Second
  405. c.Producer.Partitioner = NewHashPartitioner
  406. c.Producer.Retry.Max = 3
  407. c.Producer.Retry.Backoff = 100 * time.Millisecond
  408. c.Producer.Return.Errors = true
  409. c.Producer.CompressionLevel = CompressionLevelDefault
  410. c.Consumer.Fetch.Min = 1
  411. c.Consumer.Fetch.Default = 1024 * 1024
  412. c.Consumer.Retry.Backoff = 2 * time.Second
  413. c.Consumer.MaxWaitTime = 250 * time.Millisecond
  414. c.Consumer.MaxProcessingTime = 100 * time.Millisecond
  415. c.Consumer.Return.Errors = false
  416. c.Consumer.Offsets.AutoCommit.Enable = true
  417. c.Consumer.Offsets.AutoCommit.Interval = 1 * time.Second
  418. c.Consumer.Offsets.Initial = OffsetNewest
  419. c.Consumer.Offsets.Retry.Max = 3
  420. c.Consumer.Group.Session.Timeout = 10 * time.Second
  421. c.Consumer.Group.Heartbeat.Interval = 3 * time.Second
  422. c.Consumer.Group.Rebalance.Strategy = BalanceStrategyRange
  423. c.Consumer.Group.Rebalance.Timeout = 60 * time.Second
  424. c.Consumer.Group.Rebalance.Retry.Max = 4
  425. c.Consumer.Group.Rebalance.Retry.Backoff = 2 * time.Second
  426. c.ClientID = defaultClientID
  427. c.ChannelBufferSize = 256
  428. c.Version = MinVersion
  429. c.MetricRegistry = metrics.NewRegistry()
  430. return c
  431. }
  432. // Validate checks a Config instance. It will return a
  433. // ConfigurationError if the specified values don't make sense.
  434. func (c *Config) Validate() error {
  435. // some configuration values should be warned on but not fail completely, do those first
  436. if !c.Net.TLS.Enable && c.Net.TLS.Config != nil {
  437. Logger.Println("Net.TLS is disabled but a non-nil configuration was provided.")
  438. }
  439. if !c.Net.SASL.Enable {
  440. if c.Net.SASL.User != "" {
  441. Logger.Println("Net.SASL is disabled but a non-empty username was provided.")
  442. }
  443. if c.Net.SASL.Password != "" {
  444. Logger.Println("Net.SASL is disabled but a non-empty password was provided.")
  445. }
  446. }
  447. if c.Producer.RequiredAcks > 1 {
  448. Logger.Println("Producer.RequiredAcks > 1 is deprecated and will raise an exception with kafka >= 0.8.2.0.")
  449. }
  450. if c.Producer.MaxMessageBytes >= int(MaxRequestSize) {
  451. Logger.Println("Producer.MaxMessageBytes must be smaller than MaxRequestSize; it will be ignored.")
  452. }
  453. if c.Producer.Flush.Bytes >= int(MaxRequestSize) {
  454. Logger.Println("Producer.Flush.Bytes must be smaller than MaxRequestSize; it will be ignored.")
  455. }
  456. if (c.Producer.Flush.Bytes > 0 || c.Producer.Flush.Messages > 0) && c.Producer.Flush.Frequency == 0 {
  457. Logger.Println("Producer.Flush: Bytes or Messages are set, but Frequency is not; messages may not get flushed.")
  458. }
  459. if c.Producer.Timeout%time.Millisecond != 0 {
  460. Logger.Println("Producer.Timeout only supports millisecond resolution; nanoseconds will be truncated.")
  461. }
  462. if c.Consumer.MaxWaitTime < 100*time.Millisecond {
  463. Logger.Println("Consumer.MaxWaitTime is very low, which can cause high CPU and network usage. See documentation for details.")
  464. }
  465. if c.Consumer.MaxWaitTime%time.Millisecond != 0 {
  466. Logger.Println("Consumer.MaxWaitTime only supports millisecond precision; nanoseconds will be truncated.")
  467. }
  468. if c.Consumer.Offsets.Retention%time.Millisecond != 0 {
  469. Logger.Println("Consumer.Offsets.Retention only supports millisecond precision; nanoseconds will be truncated.")
  470. }
  471. if c.Consumer.Group.Session.Timeout%time.Millisecond != 0 {
  472. Logger.Println("Consumer.Group.Session.Timeout only supports millisecond precision; nanoseconds will be truncated.")
  473. }
  474. if c.Consumer.Group.Heartbeat.Interval%time.Millisecond != 0 {
  475. Logger.Println("Consumer.Group.Heartbeat.Interval only supports millisecond precision; nanoseconds will be truncated.")
  476. }
  477. if c.Consumer.Group.Rebalance.Timeout%time.Millisecond != 0 {
  478. Logger.Println("Consumer.Group.Rebalance.Timeout only supports millisecond precision; nanoseconds will be truncated.")
  479. }
  480. if c.ClientID == defaultClientID {
  481. Logger.Println("ClientID is the default of 'sarama', you should consider setting it to something application-specific.")
  482. }
  483. // validate Net values
  484. switch {
  485. case c.Net.MaxOpenRequests <= 0:
  486. return ConfigurationError("Net.MaxOpenRequests must be > 0")
  487. case c.Net.DialTimeout <= 0:
  488. return ConfigurationError("Net.DialTimeout must be > 0")
  489. case c.Net.ReadTimeout <= 0:
  490. return ConfigurationError("Net.ReadTimeout must be > 0")
  491. case c.Net.WriteTimeout <= 0:
  492. return ConfigurationError("Net.WriteTimeout must be > 0")
  493. case c.Net.KeepAlive < 0:
  494. return ConfigurationError("Net.KeepAlive must be >= 0")
  495. case c.Net.SASL.Enable:
  496. if c.Net.SASL.Mechanism == "" {
  497. c.Net.SASL.Mechanism = SASLTypePlaintext
  498. }
  499. switch c.Net.SASL.Mechanism {
  500. case SASLTypePlaintext:
  501. if c.Net.SASL.User == "" {
  502. return ConfigurationError("Net.SASL.User must not be empty when SASL is enabled")
  503. }
  504. if c.Net.SASL.Password == "" {
  505. return ConfigurationError("Net.SASL.Password must not be empty when SASL is enabled")
  506. }
  507. case SASLTypeOAuth:
  508. if c.Net.SASL.TokenProvider == nil {
  509. return ConfigurationError("An AccessTokenProvider instance must be provided to Net.SASL.TokenProvider")
  510. }
  511. case SASLTypeSCRAMSHA256, SASLTypeSCRAMSHA512:
  512. if c.Net.SASL.User == "" {
  513. return ConfigurationError("Net.SASL.User must not be empty when SASL is enabled")
  514. }
  515. if c.Net.SASL.Password == "" {
  516. return ConfigurationError("Net.SASL.Password must not be empty when SASL is enabled")
  517. }
  518. if c.Net.SASL.SCRAMClientGeneratorFunc == nil {
  519. return ConfigurationError("A SCRAMClientGeneratorFunc function must be provided to Net.SASL.SCRAMClientGeneratorFunc")
  520. }
  521. case SASLTypeGSSAPI:
  522. if c.Net.SASL.GSSAPI.ServiceName == "" {
  523. return ConfigurationError("Net.SASL.GSSAPI.ServiceName must not be empty when GSS-API mechanism is used")
  524. }
  525. if c.Net.SASL.GSSAPI.AuthType == KRB5_USER_AUTH {
  526. if c.Net.SASL.GSSAPI.Password == "" {
  527. return ConfigurationError("Net.SASL.GSSAPI.Password must not be empty when GSS-API " +
  528. "mechanism is used and Net.SASL.GSSAPI.AuthType = KRB5_USER_AUTH")
  529. }
  530. } else if c.Net.SASL.GSSAPI.AuthType == KRB5_KEYTAB_AUTH {
  531. if c.Net.SASL.GSSAPI.KeyTabPath == "" {
  532. return ConfigurationError("Net.SASL.GSSAPI.KeyTabPath must not be empty when GSS-API mechanism is used" +
  533. " and Net.SASL.GSSAPI.AuthType = KRB5_KEYTAB_AUTH")
  534. }
  535. } else {
  536. return ConfigurationError("Net.SASL.GSSAPI.AuthType is invalid. Possible values are KRB5_USER_AUTH and KRB5_KEYTAB_AUTH")
  537. }
  538. if c.Net.SASL.GSSAPI.KerberosConfigPath == "" {
  539. return ConfigurationError("Net.SASL.GSSAPI.KerberosConfigPath must not be empty when GSS-API mechanism is used")
  540. }
  541. if c.Net.SASL.GSSAPI.Username == "" {
  542. return ConfigurationError("Net.SASL.GSSAPI.Username must not be empty when GSS-API mechanism is used")
  543. }
  544. if c.Net.SASL.GSSAPI.Realm == "" {
  545. return ConfigurationError("Net.SASL.GSSAPI.Realm must not be empty when GSS-API mechanism is used")
  546. }
  547. default:
  548. msg := fmt.Sprintf("The SASL mechanism configuration is invalid. Possible values are `%s`, `%s`, `%s`, `%s` and `%s`",
  549. SASLTypeOAuth, SASLTypePlaintext, SASLTypeSCRAMSHA256, SASLTypeSCRAMSHA512, SASLTypeGSSAPI)
  550. return ConfigurationError(msg)
  551. }
  552. }
  553. // validate the Admin values
  554. switch {
  555. case c.Admin.Timeout <= 0:
  556. return ConfigurationError("Admin.Timeout must be > 0")
  557. }
  558. // validate the Metadata values
  559. switch {
  560. case c.Metadata.Retry.Max < 0:
  561. return ConfigurationError("Metadata.Retry.Max must be >= 0")
  562. case c.Metadata.Retry.Backoff < 0:
  563. return ConfigurationError("Metadata.Retry.Backoff must be >= 0")
  564. case c.Metadata.RefreshFrequency < 0:
  565. return ConfigurationError("Metadata.RefreshFrequency must be >= 0")
  566. }
  567. // validate the Producer values
  568. switch {
  569. case c.Producer.MaxMessageBytes <= 0:
  570. return ConfigurationError("Producer.MaxMessageBytes must be > 0")
  571. case c.Producer.RequiredAcks < -1:
  572. return ConfigurationError("Producer.RequiredAcks must be >= -1")
  573. case c.Producer.Timeout <= 0:
  574. return ConfigurationError("Producer.Timeout must be > 0")
  575. case c.Producer.Partitioner == nil:
  576. return ConfigurationError("Producer.Partitioner must not be nil")
  577. case c.Producer.Flush.Bytes < 0:
  578. return ConfigurationError("Producer.Flush.Bytes must be >= 0")
  579. case c.Producer.Flush.Messages < 0:
  580. return ConfigurationError("Producer.Flush.Messages must be >= 0")
  581. case c.Producer.Flush.Frequency < 0:
  582. return ConfigurationError("Producer.Flush.Frequency must be >= 0")
  583. case c.Producer.Flush.MaxMessages < 0:
  584. return ConfigurationError("Producer.Flush.MaxMessages must be >= 0")
  585. case c.Producer.Flush.MaxMessages > 0 && c.Producer.Flush.MaxMessages < c.Producer.Flush.Messages:
  586. return ConfigurationError("Producer.Flush.MaxMessages must be >= Producer.Flush.Messages when set")
  587. case c.Producer.Retry.Max < 0:
  588. return ConfigurationError("Producer.Retry.Max must be >= 0")
  589. case c.Producer.Retry.Backoff < 0:
  590. return ConfigurationError("Producer.Retry.Backoff must be >= 0")
  591. }
  592. if c.Producer.Compression == CompressionLZ4 && !c.Version.IsAtLeast(V0_10_0_0) {
  593. return ConfigurationError("lz4 compression requires Version >= V0_10_0_0")
  594. }
  595. if c.Producer.Compression == CompressionGZIP {
  596. if c.Producer.CompressionLevel != CompressionLevelDefault {
  597. if _, err := gzip.NewWriterLevel(ioutil.Discard, c.Producer.CompressionLevel); err != nil {
  598. return ConfigurationError(fmt.Sprintf("gzip compression does not work with level %d: %v", c.Producer.CompressionLevel, err))
  599. }
  600. }
  601. }
  602. if c.Producer.Compression == CompressionZSTD && !c.Version.IsAtLeast(V2_1_0_0) {
  603. return ConfigurationError("zstd compression requires Version >= V2_1_0_0")
  604. }
  605. if c.Producer.Idempotent {
  606. if !c.Version.IsAtLeast(V0_11_0_0) {
  607. return ConfigurationError("Idempotent producer requires Version >= V0_11_0_0")
  608. }
  609. if c.Producer.Retry.Max == 0 {
  610. return ConfigurationError("Idempotent producer requires Producer.Retry.Max >= 1")
  611. }
  612. if c.Producer.RequiredAcks != WaitForAll {
  613. return ConfigurationError("Idempotent producer requires Producer.RequiredAcks to be WaitForAll")
  614. }
  615. if c.Net.MaxOpenRequests > 1 {
  616. return ConfigurationError("Idempotent producer requires Net.MaxOpenRequests to be 1")
  617. }
  618. }
  619. // validate the Consumer values
  620. switch {
  621. case c.Consumer.Fetch.Min <= 0:
  622. return ConfigurationError("Consumer.Fetch.Min must be > 0")
  623. case c.Consumer.Fetch.Default <= 0:
  624. return ConfigurationError("Consumer.Fetch.Default must be > 0")
  625. case c.Consumer.Fetch.Max < 0:
  626. return ConfigurationError("Consumer.Fetch.Max must be >= 0")
  627. case c.Consumer.MaxWaitTime < 1*time.Millisecond:
  628. return ConfigurationError("Consumer.MaxWaitTime must be >= 1ms")
  629. case c.Consumer.MaxProcessingTime <= 0:
  630. return ConfigurationError("Consumer.MaxProcessingTime must be > 0")
  631. case c.Consumer.Retry.Backoff < 0:
  632. return ConfigurationError("Consumer.Retry.Backoff must be >= 0")
  633. case c.Consumer.Offsets.AutoCommit.Interval <= 0:
  634. return ConfigurationError("Consumer.Offsets.AutoCommit.Interval must be > 0")
  635. case c.Consumer.Offsets.Initial != OffsetOldest && c.Consumer.Offsets.Initial != OffsetNewest:
  636. return ConfigurationError("Consumer.Offsets.Initial must be OffsetOldest or OffsetNewest")
  637. case c.Consumer.Offsets.Retry.Max < 0:
  638. return ConfigurationError("Consumer.Offsets.Retry.Max must be >= 0")
  639. case c.Consumer.IsolationLevel != ReadUncommitted && c.Consumer.IsolationLevel != ReadCommitted:
  640. return ConfigurationError("Consumer.IsolationLevel must be ReadUncommitted or ReadCommitted")
  641. }
  642. if c.Consumer.Offsets.CommitInterval != 0 {
  643. Logger.Println("Deprecation warning: Consumer.Offsets.CommitInterval exists for historical compatibility" +
  644. " and should not be used. Please use Consumer.Offsets.AutoCommit, the current value will be ignored")
  645. }
  646. // validate IsolationLevel
  647. if c.Consumer.IsolationLevel == ReadCommitted && !c.Version.IsAtLeast(V0_11_0_0) {
  648. return ConfigurationError("ReadCommitted requires Version >= V0_11_0_0")
  649. }
  650. // validate the Consumer Group values
  651. switch {
  652. case c.Consumer.Group.Session.Timeout <= 2*time.Millisecond:
  653. return ConfigurationError("Consumer.Group.Session.Timeout must be >= 2ms")
  654. case c.Consumer.Group.Heartbeat.Interval < 1*time.Millisecond:
  655. return ConfigurationError("Consumer.Group.Heartbeat.Interval must be >= 1ms")
  656. case c.Consumer.Group.Heartbeat.Interval >= c.Consumer.Group.Session.Timeout:
  657. return ConfigurationError("Consumer.Group.Heartbeat.Interval must be < Consumer.Group.Session.Timeout")
  658. case c.Consumer.Group.Rebalance.Strategy == nil:
  659. return ConfigurationError("Consumer.Group.Rebalance.Strategy must not be empty")
  660. case c.Consumer.Group.Rebalance.Timeout <= time.Millisecond:
  661. return ConfigurationError("Consumer.Group.Rebalance.Timeout must be >= 1ms")
  662. case c.Consumer.Group.Rebalance.Retry.Max < 0:
  663. return ConfigurationError("Consumer.Group.Rebalance.Retry.Max must be >= 0")
  664. case c.Consumer.Group.Rebalance.Retry.Backoff < 0:
  665. return ConfigurationError("Consumer.Group.Rebalance.Retry.Backoff must be >= 0")
  666. }
  667. // validate misc shared values
  668. switch {
  669. case c.ChannelBufferSize < 0:
  670. return ConfigurationError("ChannelBufferSize must be >= 0")
  671. case !validID.MatchString(c.ClientID):
  672. return ConfigurationError("ClientID is invalid")
  673. }
  674. return nil
  675. }