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