ruby-kafka-custom 0.7.7.26

Sign up to get free protection for your applications and to get access to all the features.
Files changed (105) hide show
  1. checksums.yaml +7 -0
  2. data/lib/kafka/async_producer.rb +279 -0
  3. data/lib/kafka/broker.rb +205 -0
  4. data/lib/kafka/broker_info.rb +16 -0
  5. data/lib/kafka/broker_pool.rb +41 -0
  6. data/lib/kafka/broker_uri.rb +43 -0
  7. data/lib/kafka/client.rb +754 -0
  8. data/lib/kafka/cluster.rb +455 -0
  9. data/lib/kafka/compression.rb +43 -0
  10. data/lib/kafka/compressor.rb +85 -0
  11. data/lib/kafka/connection.rb +220 -0
  12. data/lib/kafka/connection_builder.rb +33 -0
  13. data/lib/kafka/consumer.rb +592 -0
  14. data/lib/kafka/consumer_group.rb +208 -0
  15. data/lib/kafka/datadog.rb +413 -0
  16. data/lib/kafka/fetch_operation.rb +115 -0
  17. data/lib/kafka/fetched_batch.rb +54 -0
  18. data/lib/kafka/fetched_batch_generator.rb +117 -0
  19. data/lib/kafka/fetched_message.rb +47 -0
  20. data/lib/kafka/fetched_offset_resolver.rb +48 -0
  21. data/lib/kafka/fetcher.rb +221 -0
  22. data/lib/kafka/gzip_codec.rb +30 -0
  23. data/lib/kafka/heartbeat.rb +25 -0
  24. data/lib/kafka/instrumenter.rb +38 -0
  25. data/lib/kafka/lz4_codec.rb +23 -0
  26. data/lib/kafka/message_buffer.rb +87 -0
  27. data/lib/kafka/offset_manager.rb +248 -0
  28. data/lib/kafka/partitioner.rb +35 -0
  29. data/lib/kafka/pause.rb +92 -0
  30. data/lib/kafka/pending_message.rb +29 -0
  31. data/lib/kafka/pending_message_queue.rb +41 -0
  32. data/lib/kafka/produce_operation.rb +205 -0
  33. data/lib/kafka/producer.rb +504 -0
  34. data/lib/kafka/protocol.rb +217 -0
  35. data/lib/kafka/protocol/add_partitions_to_txn_request.rb +34 -0
  36. data/lib/kafka/protocol/add_partitions_to_txn_response.rb +47 -0
  37. data/lib/kafka/protocol/alter_configs_request.rb +44 -0
  38. data/lib/kafka/protocol/alter_configs_response.rb +49 -0
  39. data/lib/kafka/protocol/api_versions_request.rb +21 -0
  40. data/lib/kafka/protocol/api_versions_response.rb +53 -0
  41. data/lib/kafka/protocol/consumer_group_protocol.rb +19 -0
  42. data/lib/kafka/protocol/create_partitions_request.rb +42 -0
  43. data/lib/kafka/protocol/create_partitions_response.rb +28 -0
  44. data/lib/kafka/protocol/create_topics_request.rb +45 -0
  45. data/lib/kafka/protocol/create_topics_response.rb +26 -0
  46. data/lib/kafka/protocol/decoder.rb +175 -0
  47. data/lib/kafka/protocol/delete_topics_request.rb +33 -0
  48. data/lib/kafka/protocol/delete_topics_response.rb +26 -0
  49. data/lib/kafka/protocol/describe_configs_request.rb +35 -0
  50. data/lib/kafka/protocol/describe_configs_response.rb +73 -0
  51. data/lib/kafka/protocol/describe_groups_request.rb +27 -0
  52. data/lib/kafka/protocol/describe_groups_response.rb +73 -0
  53. data/lib/kafka/protocol/encoder.rb +184 -0
  54. data/lib/kafka/protocol/end_txn_request.rb +29 -0
  55. data/lib/kafka/protocol/end_txn_response.rb +19 -0
  56. data/lib/kafka/protocol/fetch_request.rb +70 -0
  57. data/lib/kafka/protocol/fetch_response.rb +136 -0
  58. data/lib/kafka/protocol/find_coordinator_request.rb +29 -0
  59. data/lib/kafka/protocol/find_coordinator_response.rb +29 -0
  60. data/lib/kafka/protocol/heartbeat_request.rb +27 -0
  61. data/lib/kafka/protocol/heartbeat_response.rb +17 -0
  62. data/lib/kafka/protocol/init_producer_id_request.rb +26 -0
  63. data/lib/kafka/protocol/init_producer_id_response.rb +27 -0
  64. data/lib/kafka/protocol/join_group_request.rb +41 -0
  65. data/lib/kafka/protocol/join_group_response.rb +33 -0
  66. data/lib/kafka/protocol/leave_group_request.rb +25 -0
  67. data/lib/kafka/protocol/leave_group_response.rb +17 -0
  68. data/lib/kafka/protocol/list_groups_request.rb +23 -0
  69. data/lib/kafka/protocol/list_groups_response.rb +35 -0
  70. data/lib/kafka/protocol/list_offset_request.rb +53 -0
  71. data/lib/kafka/protocol/list_offset_response.rb +89 -0
  72. data/lib/kafka/protocol/member_assignment.rb +42 -0
  73. data/lib/kafka/protocol/message.rb +172 -0
  74. data/lib/kafka/protocol/message_set.rb +55 -0
  75. data/lib/kafka/protocol/metadata_request.rb +31 -0
  76. data/lib/kafka/protocol/metadata_response.rb +185 -0
  77. data/lib/kafka/protocol/offset_commit_request.rb +47 -0
  78. data/lib/kafka/protocol/offset_commit_response.rb +29 -0
  79. data/lib/kafka/protocol/offset_fetch_request.rb +36 -0
  80. data/lib/kafka/protocol/offset_fetch_response.rb +56 -0
  81. data/lib/kafka/protocol/produce_request.rb +92 -0
  82. data/lib/kafka/protocol/produce_response.rb +63 -0
  83. data/lib/kafka/protocol/record.rb +88 -0
  84. data/lib/kafka/protocol/record_batch.rb +222 -0
  85. data/lib/kafka/protocol/request_message.rb +26 -0
  86. data/lib/kafka/protocol/sasl_handshake_request.rb +33 -0
  87. data/lib/kafka/protocol/sasl_handshake_response.rb +28 -0
  88. data/lib/kafka/protocol/sync_group_request.rb +33 -0
  89. data/lib/kafka/protocol/sync_group_response.rb +23 -0
  90. data/lib/kafka/round_robin_assignment_strategy.rb +54 -0
  91. data/lib/kafka/sasl/gssapi.rb +76 -0
  92. data/lib/kafka/sasl/oauth.rb +64 -0
  93. data/lib/kafka/sasl/plain.rb +39 -0
  94. data/lib/kafka/sasl/scram.rb +177 -0
  95. data/lib/kafka/sasl_authenticator.rb +61 -0
  96. data/lib/kafka/snappy_codec.rb +25 -0
  97. data/lib/kafka/socket_with_timeout.rb +96 -0
  98. data/lib/kafka/ssl_context.rb +66 -0
  99. data/lib/kafka/ssl_socket_with_timeout.rb +187 -0
  100. data/lib/kafka/statsd.rb +296 -0
  101. data/lib/kafka/tagged_logger.rb +72 -0
  102. data/lib/kafka/transaction_manager.rb +261 -0
  103. data/lib/kafka/transaction_state_machine.rb +72 -0
  104. data/lib/kafka/version.rb +5 -0
  105. metadata +461 -0
@@ -0,0 +1,16 @@
1
+ # Represents a broker in a Kafka cluster.
2
+ module Kafka
3
+ class BrokerInfo
4
+ attr_reader :node_id, :host, :port
5
+
6
+ def initialize(node_id:, host:, port:)
7
+ @node_id = node_id
8
+ @host = host
9
+ @port = port
10
+ end
11
+
12
+ def to_s
13
+ "#{host}:#{port} (node_id=#{node_id})"
14
+ end
15
+ end
16
+ end
@@ -0,0 +1,41 @@
1
+ # frozen_string_literal: true
2
+
3
+ require "kafka/broker"
4
+
5
+ module Kafka
6
+ class BrokerPool
7
+ def initialize(connection_builder:, logger:)
8
+ @logger = TaggedLogger.new(logger)
9
+ @connection_builder = connection_builder
10
+ @brokers = {}
11
+ end
12
+
13
+ def connect(host, port, node_id: nil)
14
+ if @brokers.key?(node_id)
15
+ broker = @brokers.fetch(node_id)
16
+ return broker if broker.address_match?(host, port)
17
+ broker.disconnect
18
+ @brokers[node_id] = nil
19
+ end
20
+
21
+ broker = Broker.new(
22
+ connection_builder: @connection_builder,
23
+ host: host,
24
+ port: port,
25
+ node_id: node_id,
26
+ logger: @logger,
27
+ )
28
+
29
+ @brokers[node_id] = broker unless node_id.nil?
30
+
31
+ broker
32
+ end
33
+
34
+ def close
35
+ @brokers.each do |id, broker|
36
+ @logger.info "Disconnecting broker #{id}"
37
+ broker.disconnect
38
+ end
39
+ end
40
+ end
41
+ end
@@ -0,0 +1,43 @@
1
+ # frozen_string_literal: true
2
+
3
+ require "uri"
4
+
5
+ module Kafka
6
+ module BrokerUri
7
+ DEFAULT_PORT = 9092
8
+ URI_SCHEMES = ["kafka", "kafka+ssl", "plaintext", "ssl"]
9
+
10
+ # Parses a Kafka broker URI string.
11
+ #
12
+ # Examples of valid strings:
13
+ # * `kafka1.something`
14
+ # * `kafka1.something:1234`
15
+ # * `kafka://kafka1.something:1234`
16
+ # * `kafka+ssl://kafka1.something:1234`
17
+ # * `plaintext://kafka1.something:1234`
18
+ #
19
+ # @param str [String] a Kafka broker URI string.
20
+ # @return [URI]
21
+ def self.parse(str)
22
+ # Make sure there's a scheme part if it's missing.
23
+ str = "kafka://" + str unless str.include?("://")
24
+
25
+ uri = URI.parse(str)
26
+ uri.port ||= DEFAULT_PORT
27
+
28
+ # Map some schemes to others.
29
+ case uri.scheme
30
+ when 'plaintext'
31
+ uri.scheme = 'kafka'
32
+ when 'ssl'
33
+ uri.scheme = 'kafka+ssl'
34
+ end
35
+
36
+ unless URI_SCHEMES.include?(uri.scheme)
37
+ raise Kafka::Error, "invalid protocol `#{uri.scheme}` in `#{str}`"
38
+ end
39
+
40
+ uri
41
+ end
42
+ end
43
+ end
@@ -0,0 +1,754 @@
1
+ # frozen_string_literal: true
2
+
3
+ require "kafka/ssl_context"
4
+ require "kafka/cluster"
5
+ require "kafka/transaction_manager"
6
+ require "kafka/broker_info"
7
+ require "kafka/producer"
8
+ require "kafka/consumer"
9
+ require "kafka/heartbeat"
10
+ require "kafka/broker_uri"
11
+ require "kafka/async_producer"
12
+ require "kafka/fetched_message"
13
+ require "kafka/fetch_operation"
14
+ require "kafka/connection_builder"
15
+ require "kafka/instrumenter"
16
+ require "kafka/sasl_authenticator"
17
+ require "kafka/tagged_logger"
18
+
19
+ module Kafka
20
+ class Client
21
+ # Initializes a new Kafka client.
22
+ #
23
+ # @param seed_brokers [Array<String>, String] the list of brokers used to initialize
24
+ # the client. Either an Array of connections, or a comma separated string of connections.
25
+ # A connection can either be a string of "host:port" or a full URI with a scheme.
26
+ # If there's a scheme it's ignored and only host/port are used.
27
+ #
28
+ # @param client_id [String] the identifier for this application.
29
+ #
30
+ # @param logger [Logger] the logger that should be used by the client.
31
+ #
32
+ # @param connect_timeout [Integer, nil] the timeout setting for connecting
33
+ # to brokers. See {BrokerPool#initialize}.
34
+ #
35
+ # @param socket_timeout [Integer, nil] the timeout setting for socket
36
+ # connections. See {BrokerPool#initialize}.
37
+ #
38
+ # @param ssl_ca_cert [String, Array<String>, nil] a PEM encoded CA cert, or an Array of
39
+ # PEM encoded CA certs, to use with an SSL connection.
40
+ #
41
+ # @param ssl_ca_cert_file_path [String, nil] a path on the filesystem to a PEM encoded CA cert
42
+ # to use with an SSL connection.
43
+ #
44
+ # @param ssl_client_cert [String, nil] a PEM encoded client cert to use with an
45
+ # SSL connection. Must be used in combination with ssl_client_cert_key.
46
+ #
47
+ # @param ssl_client_cert_key [String, nil] a PEM encoded client cert key to use with an
48
+ # SSL connection. Must be used in combination with ssl_client_cert.
49
+ #
50
+ # @param ssl_client_cert_key_password [String, nil] the password required to read the
51
+ # ssl_client_cert_key. Must be used in combination with ssl_client_cert_key.
52
+ #
53
+ # @param sasl_gssapi_principal [String, nil] a KRB5 principal
54
+ #
55
+ # @param sasl_gssapi_keytab [String, nil] a KRB5 keytab filepath
56
+ #
57
+ # @param sasl_scram_username [String, nil] SCRAM username
58
+ #
59
+ # @param sasl_scram_password [String, nil] SCRAM password
60
+ #
61
+ # @param sasl_scram_mechanism [String, nil] Scram mechanism, either "sha256" or "sha512"
62
+ #
63
+ # @param sasl_over_ssl [Boolean] whether to enforce SSL with SASL
64
+ #
65
+ # @param sasl_oauth_token_provider [Object, nil] OAuthBearer Token Provider instance that
66
+ # implements method token. See {Sasl::OAuth#initialize}
67
+ #
68
+ # @return [Client]
69
+ def initialize(seed_brokers:, client_id: "ruby-kafka", logger: nil, connect_timeout: nil, socket_timeout: nil,
70
+ ssl_ca_cert_file_path: nil, ssl_ca_cert: nil, ssl_client_cert: nil, ssl_client_cert_key: nil,
71
+ ssl_client_cert_key_password: nil, ssl_client_cert_chain: nil, sasl_gssapi_principal: nil,
72
+ sasl_gssapi_keytab: nil, sasl_plain_authzid: '', sasl_plain_username: nil, sasl_plain_password: nil,
73
+ sasl_scram_username: nil, sasl_scram_password: nil, sasl_scram_mechanism: nil,
74
+ sasl_over_ssl: true, ssl_ca_certs_from_system: false, sasl_oauth_token_provider: nil)
75
+ @logger = TaggedLogger.new(logger)
76
+ @instrumenter = Instrumenter.new(client_id: client_id)
77
+ @seed_brokers = normalize_seed_brokers(seed_brokers)
78
+
79
+ ssl_context = SslContext.build(
80
+ ca_cert_file_path: ssl_ca_cert_file_path,
81
+ ca_cert: ssl_ca_cert,
82
+ client_cert: ssl_client_cert,
83
+ client_cert_key: ssl_client_cert_key,
84
+ client_cert_key_password: ssl_client_cert_key_password,
85
+ client_cert_chain: ssl_client_cert_chain,
86
+ ca_certs_from_system: ssl_ca_certs_from_system,
87
+ )
88
+
89
+ sasl_authenticator = SaslAuthenticator.new(
90
+ sasl_gssapi_principal: sasl_gssapi_principal,
91
+ sasl_gssapi_keytab: sasl_gssapi_keytab,
92
+ sasl_plain_authzid: sasl_plain_authzid,
93
+ sasl_plain_username: sasl_plain_username,
94
+ sasl_plain_password: sasl_plain_password,
95
+ sasl_scram_username: sasl_scram_username,
96
+ sasl_scram_password: sasl_scram_password,
97
+ sasl_scram_mechanism: sasl_scram_mechanism,
98
+ sasl_oauth_token_provider: sasl_oauth_token_provider,
99
+ logger: @logger
100
+ )
101
+
102
+ if sasl_authenticator.enabled? && sasl_over_ssl && ssl_context.nil?
103
+ raise ArgumentError, "SASL authentication requires that SSL is configured"
104
+ end
105
+
106
+ @connection_builder = ConnectionBuilder.new(
107
+ client_id: client_id,
108
+ connect_timeout: connect_timeout,
109
+ socket_timeout: socket_timeout,
110
+ ssl_context: ssl_context,
111
+ logger: @logger,
112
+ instrumenter: @instrumenter,
113
+ sasl_authenticator: sasl_authenticator
114
+ )
115
+
116
+ @cluster = initialize_cluster
117
+ end
118
+
119
+ # Delivers a single message to the Kafka cluster.
120
+ #
121
+ # **Note:** Only use this API for low-throughput scenarios. If you want to deliver
122
+ # many messages at a high rate, or if you want to configure the way messages are
123
+ # sent, use the {#producer} or {#async_producer} APIs instead.
124
+ #
125
+ # @param value [String, nil] the message value.
126
+ # @param key [String, nil] the message key.
127
+ # @param headers [Hash<String, String>] the headers for the message.
128
+ # @param topic [String] the topic that the message should be written to.
129
+ # @param partition [Integer, nil] the partition that the message should be written
130
+ # to, or `nil` if either `partition_key` is passed or the partition should be
131
+ # chosen at random.
132
+ # @param partition_key [String] a value used to deterministically choose a
133
+ # partition to write to.
134
+ # @param retries [Integer] the number of times to retry the delivery before giving
135
+ # up.
136
+ # @return [nil]
137
+ def deliver_message(value, key: nil, headers: {}, topic:, partition: nil, partition_key: nil, retries: 1)
138
+ create_time = Time.now
139
+
140
+ message = PendingMessage.new(
141
+ value: value,
142
+ key: key,
143
+ headers: headers,
144
+ topic: topic,
145
+ partition: partition,
146
+ partition_key: partition_key,
147
+ create_time: create_time
148
+ )
149
+
150
+ if partition.nil?
151
+ partition_count = @cluster.partitions_for(topic).count
152
+ partition = Partitioner.partition_for_key(partition_count, message)
153
+ end
154
+
155
+ buffer = MessageBuffer.new
156
+
157
+ buffer.write(
158
+ value: message.value,
159
+ key: message.key,
160
+ headers: message.headers,
161
+ topic: message.topic,
162
+ partition: partition,
163
+ create_time: message.create_time,
164
+ )
165
+
166
+ @cluster.add_target_topics([topic])
167
+
168
+ compressor = Compressor.new(
169
+ instrumenter: @instrumenter,
170
+ )
171
+
172
+ transaction_manager = TransactionManager.new(
173
+ cluster: @cluster,
174
+ logger: @logger,
175
+ idempotent: false,
176
+ transactional: false
177
+ )
178
+
179
+ operation = ProduceOperation.new(
180
+ cluster: @cluster,
181
+ transaction_manager: transaction_manager,
182
+ buffer: buffer,
183
+ required_acks: 1,
184
+ ack_timeout: 10,
185
+ compressor: compressor,
186
+ logger: @logger,
187
+ instrumenter: @instrumenter,
188
+ )
189
+
190
+ attempt = 1
191
+
192
+ begin
193
+ operation.execute
194
+
195
+ unless buffer.empty?
196
+ raise DeliveryFailed.new(nil, [message])
197
+ end
198
+ rescue Kafka::Error => e
199
+ @cluster.mark_as_stale!
200
+
201
+ if attempt >= (retries + 1)
202
+ raise
203
+ else
204
+ attempt += 1
205
+ @logger.warn "Error while delivering message, #{e.class}: #{e.message}; retrying after 1s..."
206
+
207
+ sleep 1
208
+
209
+ retry
210
+ end
211
+ end
212
+ end
213
+
214
+ # Initializes a new Kafka producer.
215
+ #
216
+ # @param ack_timeout [Integer] The number of seconds a broker can wait for
217
+ # replicas to acknowledge a write before responding with a timeout.
218
+ #
219
+ # @param required_acks [Integer, Symbol] The number of replicas that must acknowledge
220
+ # a write, or `:all` if all in-sync replicas must acknowledge.
221
+ #
222
+ # @param max_retries [Integer] the number of retries that should be attempted
223
+ # before giving up sending messages to the cluster. Does not include the
224
+ # original attempt.
225
+ #
226
+ # @param retry_backoff [Integer] the number of seconds to wait between retries.
227
+ #
228
+ # @param max_buffer_size [Integer] the number of messages allowed in the buffer
229
+ # before new writes will raise {BufferOverflow} exceptions.
230
+ #
231
+ # @param max_buffer_bytesize [Integer] the maximum size of the buffer in bytes.
232
+ # attempting to produce messages when the buffer reaches this size will
233
+ # result in {BufferOverflow} being raised.
234
+ #
235
+ # @param compression_codec [Symbol, nil] the name of the compression codec to
236
+ # use, or nil if no compression should be performed. Valid codecs: `:snappy`
237
+ # and `:gzip`.
238
+ #
239
+ # @param compression_threshold [Integer] the number of messages that needs to
240
+ # be in a message set before it should be compressed. Note that message sets
241
+ # are per-partition rather than per-topic or per-producer.
242
+ #
243
+ # @return [Kafka::Producer] the Kafka producer.
244
+ def producer(
245
+ compression_codec: nil,
246
+ compression_threshold: 1,
247
+ ack_timeout: 5,
248
+ required_acks: :all,
249
+ max_retries: 2,
250
+ retry_backoff: 1,
251
+ max_buffer_size: 1000,
252
+ max_buffer_bytesize: 10_000_000,
253
+ idempotent: false,
254
+ transactional: false,
255
+ transactional_id: nil,
256
+ transactional_timeout: 60
257
+ )
258
+ cluster = initialize_cluster
259
+ compressor = Compressor.new(
260
+ codec_name: compression_codec,
261
+ threshold: compression_threshold,
262
+ instrumenter: @instrumenter,
263
+ )
264
+
265
+ transaction_manager = TransactionManager.new(
266
+ cluster: cluster,
267
+ logger: @logger,
268
+ idempotent: idempotent,
269
+ transactional: transactional,
270
+ transactional_id: transactional_id,
271
+ transactional_timeout: transactional_timeout,
272
+ )
273
+
274
+ Producer.new(
275
+ cluster: cluster,
276
+ transaction_manager: transaction_manager,
277
+ logger: @logger,
278
+ instrumenter: @instrumenter,
279
+ compressor: compressor,
280
+ ack_timeout: ack_timeout,
281
+ required_acks: required_acks,
282
+ max_retries: max_retries,
283
+ retry_backoff: retry_backoff,
284
+ max_buffer_size: max_buffer_size,
285
+ max_buffer_bytesize: max_buffer_bytesize,
286
+ )
287
+ end
288
+
289
+ # Creates a new AsyncProducer instance.
290
+ #
291
+ # All parameters allowed by {#producer} can be passed. In addition to this,
292
+ # a few extra parameters can be passed when creating an async producer.
293
+ #
294
+ # @param max_queue_size [Integer] the maximum number of messages allowed in
295
+ # the queue.
296
+ # @param delivery_threshold [Integer] if greater than zero, the number of
297
+ # buffered messages that will automatically trigger a delivery.
298
+ # @param delivery_interval [Integer] if greater than zero, the number of
299
+ # seconds between automatic message deliveries.
300
+ #
301
+ # @see AsyncProducer
302
+ # @return [AsyncProducer]
303
+ def async_producer(delivery_interval: 0, delivery_threshold: 0, max_queue_size: 1000, max_retries: -1, retry_backoff: 0, **options)
304
+ sync_producer = producer(**options)
305
+
306
+ AsyncProducer.new(
307
+ sync_producer: sync_producer,
308
+ delivery_interval: delivery_interval,
309
+ delivery_threshold: delivery_threshold,
310
+ max_queue_size: max_queue_size,
311
+ max_retries: max_retries,
312
+ retry_backoff: retry_backoff,
313
+ instrumenter: @instrumenter,
314
+ logger: @logger,
315
+ )
316
+ end
317
+
318
+ # Creates a new Kafka consumer.
319
+ #
320
+ # @param group_id [String] the id of the group that the consumer should join.
321
+ # @param session_timeout [Integer] the number of seconds after which, if a client
322
+ # hasn't contacted the Kafka cluster, it will be kicked out of the group.
323
+ # @param offset_commit_interval [Integer] the interval between offset commits,
324
+ # in seconds.
325
+ # @param offset_commit_threshold [Integer] the number of messages that can be
326
+ # processed before their offsets are committed. If zero, offset commits are
327
+ # not triggered by message processing.
328
+ # @param heartbeat_interval [Integer] the interval between heartbeats; must be less
329
+ # than the session window.
330
+ # @param offset_retention_time [Integer] the time period that committed
331
+ # offsets will be retained, in seconds. Defaults to the broker setting.
332
+ # @param fetcher_max_queue_size [Integer] max number of items in the fetch queue that
333
+ # are stored for further processing. Note, that each item in the queue represents a
334
+ # response from a single broker.
335
+ # @return [Consumer]
336
+ def consumer(
337
+ group_id:,
338
+ session_timeout: 30,
339
+ offset_commit_interval: 10,
340
+ offset_commit_threshold: 0,
341
+ heartbeat_interval: 10,
342
+ offset_retention_time: nil,
343
+ fetcher_max_queue_size: 100
344
+ )
345
+ cluster = initialize_cluster
346
+
347
+ instrumenter = DecoratingInstrumenter.new(@instrumenter, {
348
+ group_id: group_id,
349
+ })
350
+
351
+ # The Kafka protocol expects the retention time to be in ms.
352
+ retention_time = (offset_retention_time && offset_retention_time * 1_000) || -1
353
+
354
+ group = ConsumerGroup.new(
355
+ cluster: cluster,
356
+ logger: @logger,
357
+ group_id: group_id,
358
+ session_timeout: session_timeout,
359
+ retention_time: retention_time,
360
+ instrumenter: instrumenter,
361
+ )
362
+
363
+ fetcher = Fetcher.new(
364
+ cluster: initialize_cluster,
365
+ group: group,
366
+ logger: @logger,
367
+ instrumenter: instrumenter,
368
+ max_queue_size: fetcher_max_queue_size
369
+ )
370
+
371
+ offset_manager = OffsetManager.new(
372
+ cluster: cluster,
373
+ group: group,
374
+ fetcher: fetcher,
375
+ logger: @logger,
376
+ commit_interval: offset_commit_interval,
377
+ commit_threshold: offset_commit_threshold,
378
+ offset_retention_time: offset_retention_time
379
+ )
380
+
381
+ heartbeat = Heartbeat.new(
382
+ group: group,
383
+ interval: heartbeat_interval,
384
+ instrumenter: instrumenter
385
+ )
386
+
387
+ Consumer.new(
388
+ cluster: cluster,
389
+ logger: @logger,
390
+ instrumenter: instrumenter,
391
+ group: group,
392
+ offset_manager: offset_manager,
393
+ fetcher: fetcher,
394
+ session_timeout: session_timeout,
395
+ heartbeat: heartbeat,
396
+ )
397
+ end
398
+
399
+ # Fetches a batch of messages from a single partition. Note that it's possible
400
+ # to get back empty batches.
401
+ #
402
+ # The starting point for the fetch can be configured with the `:offset` argument.
403
+ # If you pass a number, the fetch will start at that offset. However, there are
404
+ # two special Symbol values that can be passed instead:
405
+ #
406
+ # * `:earliest` — the first offset in the partition.
407
+ # * `:latest` — the next offset that will be written to, effectively making the
408
+ # call block until there is a new message in the partition.
409
+ #
410
+ # The Kafka protocol specifies the numeric values of these two options: -2 and -1,
411
+ # respectively. You can also pass in these numbers directly.
412
+ #
413
+ # ## Example
414
+ #
415
+ # When enumerating the messages in a partition, you typically fetch batches
416
+ # sequentially.
417
+ #
418
+ # offset = :earliest
419
+ #
420
+ # loop do
421
+ # messages = kafka.fetch_messages(
422
+ # topic: "my-topic",
423
+ # partition: 42,
424
+ # offset: offset,
425
+ # )
426
+ #
427
+ # messages.each do |message|
428
+ # puts message.offset, message.key, message.value
429
+ #
430
+ # # Set the next offset that should be read to be the subsequent
431
+ # # offset.
432
+ # offset = message.offset + 1
433
+ # end
434
+ # end
435
+ #
436
+ # See a working example in `examples/simple-consumer.rb`.
437
+ #
438
+ # @param topic [String] the topic that messages should be fetched from.
439
+ #
440
+ # @param partition [Integer] the partition that messages should be fetched from.
441
+ #
442
+ # @param offset [Integer, Symbol] the offset to start reading from. Default is
443
+ # the latest offset.
444
+ #
445
+ # @param max_wait_time [Integer] the maximum amount of time to wait before
446
+ # the server responds, in seconds.
447
+ #
448
+ # @param min_bytes [Integer] the minimum number of bytes to wait for. If set to
449
+ # zero, the broker will respond immediately, but the response may be empty.
450
+ # The default is 1 byte, which means that the broker will respond as soon as
451
+ # a message is written to the partition.
452
+ #
453
+ # @param max_bytes [Integer] the maximum number of bytes to include in the
454
+ # response message set. Default is 1 MB. You need to set this higher if you
455
+ # expect messages to be larger than this.
456
+ #
457
+ # @return [Array<Kafka::FetchedMessage>] the messages returned from the broker.
458
+ def fetch_messages(topic:, partition:, offset: :latest, max_wait_time: 5, min_bytes: 1, max_bytes: 1048576, retries: 1)
459
+ operation = FetchOperation.new(
460
+ cluster: @cluster,
461
+ logger: @logger,
462
+ min_bytes: min_bytes,
463
+ max_bytes: max_bytes,
464
+ max_wait_time: max_wait_time,
465
+ )
466
+
467
+ operation.fetch_from_partition(topic, partition, offset: offset, max_bytes: max_bytes)
468
+
469
+ attempt = 1
470
+
471
+ begin
472
+ operation.execute.flat_map {|batch| batch.messages }
473
+ rescue Kafka::Error => e
474
+ @cluster.mark_as_stale!
475
+
476
+ if attempt >= (retries + 1)
477
+ raise
478
+ else
479
+ attempt += 1
480
+ @logger.warn "Error while fetching messages, #{e.class}: #{e.message}; retrying..."
481
+ retry
482
+ end
483
+ end
484
+ end
485
+
486
+ # Enumerate all messages in a topic.
487
+ #
488
+ # @param topic [String] the topic to consume messages from.
489
+ #
490
+ # @param start_from_beginning [Boolean] whether to start from the beginning
491
+ # of the topic or just subscribe to new messages being produced.
492
+ #
493
+ # @param max_wait_time [Integer] the maximum amount of time to wait before
494
+ # the server responds, in seconds.
495
+ #
496
+ # @param min_bytes [Integer] the minimum number of bytes to wait for. If set to
497
+ # zero, the broker will respond immediately, but the response may be empty.
498
+ # The default is 1 byte, which means that the broker will respond as soon as
499
+ # a message is written to the partition.
500
+ #
501
+ # @param max_bytes [Integer] the maximum number of bytes to include in the
502
+ # response message set. Default is 1 MB. You need to set this higher if you
503
+ # expect messages to be larger than this.
504
+ #
505
+ # @return [nil]
506
+ def each_message(topic:, start_from_beginning: true, max_wait_time: 5, min_bytes: 1, max_bytes: 1048576, &block)
507
+ default_offset ||= start_from_beginning ? :earliest : :latest
508
+ offsets = Hash.new { default_offset }
509
+
510
+ loop do
511
+ operation = FetchOperation.new(
512
+ cluster: @cluster,
513
+ logger: @logger,
514
+ min_bytes: min_bytes,
515
+ max_wait_time: max_wait_time,
516
+ )
517
+
518
+ @cluster.partitions_for(topic).map(&:partition_id).each do |partition|
519
+ partition_offset = offsets[partition]
520
+ operation.fetch_from_partition(topic, partition, offset: partition_offset, max_bytes: max_bytes)
521
+ end
522
+
523
+ batches = operation.execute
524
+
525
+ batches.each do |batch|
526
+ batch.messages.each(&block)
527
+ offsets[batch.partition] = batch.last_offset + 1 unless batch.unknown_last_offset?
528
+ end
529
+ end
530
+ end
531
+
532
+ # Creates a topic in the cluster.
533
+ #
534
+ # @example Creating a topic with log compaction
535
+ # # Enable log compaction:
536
+ # config = { "cleanup.policy" => "compact" }
537
+ #
538
+ # # Create the topic:
539
+ # kafka.create_topic("dns-mappings", config: config)
540
+ #
541
+ # @param name [String] the name of the topic.
542
+ # @param num_partitions [Integer] the number of partitions that should be created
543
+ # in the topic.
544
+ # @param replication_factor [Integer] the replication factor of the topic.
545
+ # @param timeout [Integer] a duration of time to wait for the topic to be
546
+ # completely created.
547
+ # @param config [Hash] topic configuration entries. See
548
+ # [the Kafka documentation](https://kafka.apache.org/documentation/#topicconfigs)
549
+ # for more information.
550
+ # @raise [Kafka::TopicAlreadyExists] if the topic already exists.
551
+ # @return [nil]
552
+ def create_topic(name, num_partitions: 1, replication_factor: 1, timeout: 30, config: {})
553
+ @cluster.create_topic(
554
+ name,
555
+ num_partitions: num_partitions,
556
+ replication_factor: replication_factor,
557
+ timeout: timeout,
558
+ config: config,
559
+ )
560
+ end
561
+
562
+ # Delete a topic in the cluster.
563
+ #
564
+ # @param name [String] the name of the topic.
565
+ # @param timeout [Integer] a duration of time to wait for the topic to be
566
+ # completely marked deleted.
567
+ # @return [nil]
568
+ def delete_topic(name, timeout: 30)
569
+ @cluster.delete_topic(name, timeout: timeout)
570
+ end
571
+
572
+ # Describe the configuration of a topic.
573
+ #
574
+ # Retrieves the topic configuration from the Kafka brokers. Configuration names
575
+ # refer to [Kafka's topic-level configs](https://kafka.apache.org/documentation/#topicconfigs).
576
+ #
577
+ # @note This is an alpha level API and is subject to change.
578
+ #
579
+ # @example Describing the cleanup policy config of a topic
580
+ # kafka = Kafka.new(["kafka1:9092"])
581
+ # kafka.describe_topic("my-topic", ["cleanup.policy"])
582
+ # #=> { "cleanup.policy" => "delete" }
583
+ #
584
+ # @param name [String] the name of the topic.
585
+ # @param configs [Array<String>] array of desired config names.
586
+ # @return [Hash<String, String>]
587
+ def describe_topic(name, configs = [])
588
+ @cluster.describe_topic(name, configs)
589
+ end
590
+
591
+ # Alter the configuration of a topic.
592
+ #
593
+ # Configuration keys must match
594
+ # [Kafka's topic-level configs](https://kafka.apache.org/documentation/#topicconfigs).
595
+ #
596
+ # @note This is an alpha level API and is subject to change.
597
+ #
598
+ # @example Describing the cleanup policy config of a topic
599
+ # kafka = Kafka.new(["kafka1:9092"])
600
+ # kafka.alter_topic("my-topic", "cleanup.policy" => "delete", "max.message.byte" => "100000")
601
+ #
602
+ # @param name [String] the name of the topic.
603
+ # @param configs [Hash<String, String>] hash of desired config keys and values.
604
+ # @return [nil]
605
+ def alter_topic(name, configs = {})
606
+ @cluster.alter_topic(name, configs)
607
+ end
608
+
609
+ # Describe a consumer group
610
+ #
611
+ # @param group_id [String] the id of the consumer group
612
+ # @return [Kafka::Protocol::DescribeGroupsResponse::Group]
613
+ def describe_group(group_id)
614
+ @cluster.describe_group(group_id)
615
+ end
616
+
617
+ # Create partitions for a topic.
618
+ #
619
+ # @param name [String] the name of the topic.
620
+ # @param num_partitions [Integer] the number of desired partitions for
621
+ # the topic
622
+ # @param timeout [Integer] a duration of time to wait for the new
623
+ # partitions to be added.
624
+ # @return [nil]
625
+ def create_partitions_for(name, num_partitions: 1, timeout: 30)
626
+ @cluster.create_partitions_for(name, num_partitions: num_partitions, timeout: timeout)
627
+ end
628
+
629
+ # Lists all topics in the cluster.
630
+ #
631
+ # @return [Array<String>] the list of topic names.
632
+ def topics
633
+ attempts = 0
634
+ begin
635
+ attempts += 1
636
+ @cluster.list_topics
637
+ rescue Kafka::ConnectionError
638
+ @cluster.mark_as_stale!
639
+ retry unless attempts > 1
640
+ raise
641
+ end
642
+ end
643
+
644
+ # Lists all consumer groups in the cluster
645
+ #
646
+ # @return [Array<String>] the list of group ids
647
+ def groups
648
+ @cluster.list_groups
649
+ end
650
+
651
+ def has_topic?(topic)
652
+ @cluster.clear_target_topics
653
+ @cluster.add_target_topics([topic])
654
+ @cluster.topics.include?(topic)
655
+ end
656
+
657
+ # Counts the number of partitions in a topic.
658
+ #
659
+ # @param topic [String]
660
+ # @return [Integer] the number of partitions in the topic.
661
+ def partitions_for(topic)
662
+ @cluster.partitions_for(topic).count
663
+ end
664
+
665
+ # Retrieve the offset of the last message in a partition. If there are no
666
+ # messages in the partition -1 is returned.
667
+ #
668
+ # @param topic [String]
669
+ # @param partition [Integer]
670
+ # @return [Integer] the offset of the last message in the partition, or -1 if
671
+ # there are no messages in the partition.
672
+ def last_offset_for(topic, partition)
673
+ # The offset resolution API will return the offset of the "next" message to
674
+ # be written when resolving the "latest" offset, so we subtract one.
675
+ @cluster.resolve_offset(topic, partition, :latest) - 1
676
+ end
677
+
678
+ # Retrieve the offset of the last message in each partition of the specified topics.
679
+ #
680
+ # @param topics [Array<String>] topic names.
681
+ # @return [Hash<String, Hash<Integer, Integer>>]
682
+ # @example
683
+ # last_offsets_for('topic-1', 'topic-2') # =>
684
+ # # {
685
+ # # 'topic-1' => { 0 => 100, 1 => 100 },
686
+ # # 'topic-2' => { 0 => 100, 1 => 100 }
687
+ # # }
688
+ def last_offsets_for(*topics)
689
+ @cluster.add_target_topics(topics)
690
+ topics.map {|topic|
691
+ partition_ids = @cluster.partitions_for(topic).collect(&:partition_id)
692
+ partition_offsets = @cluster.resolve_offsets(topic, partition_ids, :latest)
693
+ [topic, partition_offsets.collect { |k, v| [k, v - 1] }.to_h]
694
+ }.to_h
695
+ end
696
+
697
+ # Check whether current cluster supports a specific version or not
698
+ #
699
+ # @param api_key [Integer] API key.
700
+ # @param version [Integer] API version.
701
+ # @return [Boolean]
702
+ def supports_api?(api_key, version = nil)
703
+ @cluster.supports_api?(api_key, version)
704
+ end
705
+
706
+ def apis
707
+ @cluster.apis
708
+ end
709
+
710
+ # List all brokers in the cluster.
711
+ #
712
+ # @return [Array<Kafka::BrokerInfo>] the list of brokers.
713
+ def brokers
714
+ @cluster.cluster_info.brokers
715
+ end
716
+
717
+ # The current controller broker in the cluster.
718
+ #
719
+ # @return [Kafka::BrokerInfo] information on the controller broker.
720
+ def controller_broker
721
+ brokers.find {|broker| broker.node_id == @cluster.cluster_info.controller_id }
722
+ end
723
+
724
+ # Closes all connections to the Kafka brokers and frees up used resources.
725
+ #
726
+ # @return [nil]
727
+ def close
728
+ @cluster.disconnect
729
+ end
730
+
731
+ private
732
+
733
+ def initialize_cluster
734
+ broker_pool = BrokerPool.new(
735
+ connection_builder: @connection_builder,
736
+ logger: @logger,
737
+ )
738
+
739
+ Cluster.new(
740
+ seed_brokers: @seed_brokers,
741
+ broker_pool: broker_pool,
742
+ logger: @logger,
743
+ )
744
+ end
745
+
746
+ def normalize_seed_brokers(seed_brokers)
747
+ if seed_brokers.is_a?(String)
748
+ seed_brokers = seed_brokers.split(",")
749
+ end
750
+
751
+ seed_brokers.map {|str| BrokerUri.parse(str) }
752
+ end
753
+ end
754
+ end