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