config.go 32 KB

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