config.go 38 KB

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