ruby-kafka 0.1.7 → 0.2.0
Sign up to get free protection for your applications and to get access to all the features.
- checksums.yaml +4 -4
- data/CHANGELOG.md +10 -0
- data/README.md +12 -1
- data/lib/kafka.rb +18 -0
- data/lib/kafka/broker.rb +42 -0
- data/lib/kafka/client.rb +35 -5
- data/lib/kafka/cluster.rb +30 -0
- data/lib/kafka/compressor.rb +59 -0
- data/lib/kafka/connection.rb +1 -0
- data/lib/kafka/consumer.rb +211 -0
- data/lib/kafka/consumer_group.rb +172 -0
- data/lib/kafka/fetch_operation.rb +2 -2
- data/lib/kafka/produce_operation.rb +4 -8
- data/lib/kafka/producer.rb +7 -5
- data/lib/kafka/protocol.rb +27 -0
- data/lib/kafka/protocol/consumer_group_protocol.rb +17 -0
- data/lib/kafka/protocol/group_coordinator_request.rb +21 -0
- data/lib/kafka/protocol/group_coordinator_response.rb +25 -0
- data/lib/kafka/protocol/heartbeat_request.rb +25 -0
- data/lib/kafka/protocol/heartbeat_response.rb +15 -0
- data/lib/kafka/protocol/join_group_request.rb +39 -0
- data/lib/kafka/protocol/join_group_response.rb +31 -0
- data/lib/kafka/protocol/leave_group_request.rb +23 -0
- data/lib/kafka/protocol/leave_group_response.rb +15 -0
- data/lib/kafka/protocol/member_assignment.rb +40 -0
- data/lib/kafka/protocol/message_set.rb +5 -37
- data/lib/kafka/protocol/metadata_response.rb +5 -1
- data/lib/kafka/protocol/offset_commit_request.rb +42 -0
- data/lib/kafka/protocol/offset_commit_response.rb +27 -0
- data/lib/kafka/protocol/offset_fetch_request.rb +34 -0
- data/lib/kafka/protocol/offset_fetch_response.rb +51 -0
- data/lib/kafka/protocol/sync_group_request.rb +31 -0
- data/lib/kafka/protocol/sync_group_response.rb +21 -0
- data/lib/kafka/round_robin_assignment_strategy.rb +40 -0
- data/lib/kafka/version.rb +1 -1
- metadata +23 -2
checksums.yaml
CHANGED
@@ -1,7 +1,7 @@
|
|
1
1
|
---
|
2
2
|
SHA1:
|
3
|
-
metadata.gz:
|
4
|
-
data.tar.gz:
|
3
|
+
metadata.gz: e58cc6609bed1f99291323a15a90e1a8433370a9
|
4
|
+
data.tar.gz: eca9c8f1efc71812b4d4d7580f0917a7921864f1
|
5
5
|
SHA512:
|
6
|
-
metadata.gz:
|
7
|
-
data.tar.gz:
|
6
|
+
metadata.gz: 7be411f4c72bd6de0154bbee286df4a0d67052af54f371fc643248cc6f6a8cb4760a2f4c18c47921a6031722be8c280502617fb27aa38423cf6add20667db992
|
7
|
+
data.tar.gz: cb721b17f832fc9c4485e6db40902e5ebdd2fbbcf62d24b27879ffa697455f38eb2ebc05355b37a413264a2452169052a35c3f0910312752a328f35d787371a2
|
data/CHANGELOG.md
ADDED
data/README.md
CHANGED
@@ -4,7 +4,7 @@
|
|
4
4
|
|
5
5
|
A Ruby client library for [Apache Kafka](http://kafka.apache.org/), a distributed log and message bus. The focus of this library will be operational simplicity, with good logging and metrics that can make debugging issues easier.
|
6
6
|
|
7
|
-
|
7
|
+
The Producer API is currently beta level and used in production. There's an alpha level Consumer Group API that has not yet been used in production and that may change without warning. Feel free to try it out but don't expect it to be stable or correct quite yet.
|
8
8
|
|
9
9
|
## Installation
|
10
10
|
|
@@ -208,6 +208,17 @@ Note that there's a maximum buffer size; pass in a different value for `max_buff
|
|
208
208
|
|
209
209
|
A final note on buffers: local buffers give resilience against broker and network failures, and allow higher throughput due to message batching, but they also trade off consistency guarantees for higher availibility and resilience. If your local process dies while messages are buffered, those messages will be lost. If you require high levels of consistency, you should call `#deliver_messages` immediately after `#produce`.
|
210
210
|
|
211
|
+
### Logging
|
212
|
+
|
213
|
+
It's a very good idea to configure the Kafka client with a logger. All important operations and errors are logged. When instantiating your client, simply pass in a valid logger:
|
214
|
+
|
215
|
+
```ruby
|
216
|
+
logger = Logger.new("log/kafka.log")
|
217
|
+
kafka = Kafka.new(logger: logger, ...)
|
218
|
+
```
|
219
|
+
|
220
|
+
By default, nothing is logged.
|
221
|
+
|
211
222
|
### Understanding Timeouts
|
212
223
|
|
213
224
|
It's important to understand how timeouts work if you have a latency sensitive application. This library allows configuring timeouts on different levels:
|
data/lib/kafka.rb
CHANGED
@@ -59,6 +59,12 @@ module Kafka
|
|
59
59
|
class OffsetMetadataTooLarge < ProtocolError
|
60
60
|
end
|
61
61
|
|
62
|
+
class GroupCoordinatorNotAvailable < ProtocolError
|
63
|
+
end
|
64
|
+
|
65
|
+
class NotCoordinatorForGroup < ProtocolError
|
66
|
+
end
|
67
|
+
|
62
68
|
# For a request which attempts to access an invalid topic (e.g. one which has
|
63
69
|
# an illegal name), or if an attempt is made to write to an internal topic
|
64
70
|
# (such as the consumer offsets topic).
|
@@ -89,6 +95,18 @@ module Kafka
|
|
89
95
|
class ReplicaNotAvailable < ProtocolError
|
90
96
|
end
|
91
97
|
|
98
|
+
class UnknownMemberId < ProtocolError
|
99
|
+
end
|
100
|
+
|
101
|
+
class RebalanceInProgress < ProtocolError
|
102
|
+
end
|
103
|
+
|
104
|
+
class IllegalGeneration < ProtocolError
|
105
|
+
end
|
106
|
+
|
107
|
+
class InvalidSessionTimeout < ProtocolError
|
108
|
+
end
|
109
|
+
|
92
110
|
# Raised when there's a network connection error.
|
93
111
|
class ConnectionError < Error
|
94
112
|
end
|
data/lib/kafka/broker.rb
CHANGED
@@ -64,5 +64,47 @@ module Kafka
|
|
64
64
|
|
65
65
|
@connection.send_request(request)
|
66
66
|
end
|
67
|
+
|
68
|
+
def fetch_offsets(**options)
|
69
|
+
request = Protocol::OffsetFetchRequest.new(**options)
|
70
|
+
|
71
|
+
@connection.send_request(request)
|
72
|
+
end
|
73
|
+
|
74
|
+
def commit_offsets(**options)
|
75
|
+
request = Protocol::OffsetCommitRequest.new(**options)
|
76
|
+
|
77
|
+
@connection.send_request(request)
|
78
|
+
end
|
79
|
+
|
80
|
+
def join_group(**options)
|
81
|
+
request = Protocol::JoinGroupRequest.new(**options)
|
82
|
+
|
83
|
+
@connection.send_request(request)
|
84
|
+
end
|
85
|
+
|
86
|
+
def sync_group(**options)
|
87
|
+
request = Protocol::SyncGroupRequest.new(**options)
|
88
|
+
|
89
|
+
@connection.send_request(request)
|
90
|
+
end
|
91
|
+
|
92
|
+
def leave_group(**options)
|
93
|
+
request = Protocol::LeaveGroupRequest.new(**options)
|
94
|
+
|
95
|
+
@connection.send_request(request)
|
96
|
+
end
|
97
|
+
|
98
|
+
def find_group_coordinator(**options)
|
99
|
+
request = Protocol::GroupCoordinatorRequest.new(**options)
|
100
|
+
|
101
|
+
@connection.send_request(request)
|
102
|
+
end
|
103
|
+
|
104
|
+
def heartbeat(**options)
|
105
|
+
request = Protocol::HeartbeatRequest.new(**options)
|
106
|
+
|
107
|
+
@connection.send_request(request)
|
108
|
+
end
|
67
109
|
end
|
68
110
|
end
|
data/lib/kafka/client.rb
CHANGED
@@ -1,13 +1,12 @@
|
|
1
1
|
require "kafka/cluster"
|
2
2
|
require "kafka/producer"
|
3
|
+
require "kafka/consumer"
|
3
4
|
require "kafka/async_producer"
|
4
5
|
require "kafka/fetched_message"
|
5
6
|
require "kafka/fetch_operation"
|
6
7
|
|
7
8
|
module Kafka
|
8
9
|
class Client
|
9
|
-
DEFAULT_CLIENT_ID = "ruby-kafka"
|
10
|
-
DEFAULT_LOGGER = Logger.new("/dev/null")
|
11
10
|
|
12
11
|
# Initializes a new Kafka client.
|
13
12
|
#
|
@@ -16,7 +15,7 @@ module Kafka
|
|
16
15
|
#
|
17
16
|
# @param client_id [String] the identifier for this application.
|
18
17
|
#
|
19
|
-
# @param logger [Logger]
|
18
|
+
# @param logger [Logger] the logger that should be used by the client.
|
20
19
|
#
|
21
20
|
# @param connect_timeout [Integer, nil] the timeout setting for connecting
|
22
21
|
# to brokers. See {BrokerPool#initialize}.
|
@@ -25,8 +24,8 @@ module Kafka
|
|
25
24
|
# connections. See {BrokerPool#initialize}.
|
26
25
|
#
|
27
26
|
# @return [Client]
|
28
|
-
def initialize(seed_brokers:, client_id:
|
29
|
-
@logger = logger
|
27
|
+
def initialize(seed_brokers:, client_id: "ruby-kafka", logger: nil, connect_timeout: nil, socket_timeout: nil)
|
28
|
+
@logger = logger || Logger.new("/dev/null")
|
30
29
|
|
31
30
|
broker_pool = BrokerPool.new(
|
32
31
|
client_id: client_id,
|
@@ -52,6 +51,20 @@ module Kafka
|
|
52
51
|
Producer.new(cluster: @cluster, logger: @logger, **options)
|
53
52
|
end
|
54
53
|
|
54
|
+
# Creates a new AsyncProducer instance.
|
55
|
+
#
|
56
|
+
# All parameters allowed by {#producer} can be passed. In addition to this,
|
57
|
+
# a few extra parameters can be passed when creating an async producer.
|
58
|
+
#
|
59
|
+
# @param max_queue_size [Integer] the maximum number of messages allowed in
|
60
|
+
# the queue.
|
61
|
+
# @param delivery_threshold [Integer] if greater than zero, the number of
|
62
|
+
# buffered messages that will automatically trigger a delivery.
|
63
|
+
# @param delivery_interval [Integer] if greater than zero, the number of
|
64
|
+
# seconds between automatic message deliveries.
|
65
|
+
#
|
66
|
+
# @see AsyncProducer
|
67
|
+
# @return [AsyncProducer]
|
55
68
|
def async_producer(delivery_interval: 0, delivery_threshold: 0, max_queue_size: 1000, **options)
|
56
69
|
sync_producer = producer(**options)
|
57
70
|
|
@@ -63,6 +76,20 @@ module Kafka
|
|
63
76
|
)
|
64
77
|
end
|
65
78
|
|
79
|
+
# Creates a new Consumer instance.
|
80
|
+
#
|
81
|
+
# `options` are passed to {Consumer#initialize}.
|
82
|
+
#
|
83
|
+
# @see Consumer
|
84
|
+
# @return [Consumer]
|
85
|
+
def consumer(**options)
|
86
|
+
Consumer.new(
|
87
|
+
cluster: @cluster,
|
88
|
+
logger: @logger,
|
89
|
+
**options,
|
90
|
+
)
|
91
|
+
end
|
92
|
+
|
66
93
|
# Fetches a batch of messages from a single partition. Note that it's possible
|
67
94
|
# to get back empty batches.
|
68
95
|
#
|
@@ -152,6 +179,9 @@ module Kafka
|
|
152
179
|
@cluster.partitions_for(topic).count
|
153
180
|
end
|
154
181
|
|
182
|
+
# Closes all connections to the Kafka brokers and frees up used resources.
|
183
|
+
#
|
184
|
+
# @return [nil]
|
155
185
|
def close
|
156
186
|
@cluster.disconnect
|
157
187
|
end
|
data/lib/kafka/cluster.rb
CHANGED
@@ -66,6 +66,36 @@ module Kafka
|
|
66
66
|
connect_to_broker(get_leader_id(topic, partition))
|
67
67
|
end
|
68
68
|
|
69
|
+
def get_group_coordinator(group_id:)
|
70
|
+
@logger.debug "Getting group coordinator for `#{group_id}`"
|
71
|
+
|
72
|
+
refresh_metadata_if_necessary!
|
73
|
+
|
74
|
+
cluster_info.brokers.each do |broker_info|
|
75
|
+
begin
|
76
|
+
broker = connect_to_broker(broker_info.node_id)
|
77
|
+
response = broker.find_group_coordinator(group_id: group_id)
|
78
|
+
|
79
|
+
Protocol.handle_error(response.error_code)
|
80
|
+
|
81
|
+
coordinator_id = response.coordinator_id
|
82
|
+
coordinator = connect_to_broker(coordinator_id)
|
83
|
+
|
84
|
+
@logger.debug "Coordinator for group `#{group_id}` is #{coordinator}"
|
85
|
+
|
86
|
+
return coordinator
|
87
|
+
rescue GroupCoordinatorNotAvailable
|
88
|
+
@logger.debug "Coordinator not available; retrying in 1s"
|
89
|
+
sleep 1
|
90
|
+
retry
|
91
|
+
rescue ConnectionError => e
|
92
|
+
@logger.error "Failed to get group coordinator info from #{broker}: #{e}"
|
93
|
+
end
|
94
|
+
end
|
95
|
+
|
96
|
+
raise Kafka::Error, "Failed to find group coordinator"
|
97
|
+
end
|
98
|
+
|
69
99
|
def partitions_for(topic)
|
70
100
|
add_target_topics([topic])
|
71
101
|
cluster_info.partitions_for(topic)
|
@@ -0,0 +1,59 @@
|
|
1
|
+
require "kafka/compression"
|
2
|
+
|
3
|
+
module Kafka
|
4
|
+
|
5
|
+
# Compresses message sets using a specified codec.
|
6
|
+
#
|
7
|
+
# A message set is only compressed if its size meets the defined threshold.
|
8
|
+
#
|
9
|
+
# ## Instrumentation
|
10
|
+
#
|
11
|
+
# Whenever a message set is compressed, the notification
|
12
|
+
# `compress.compressor.kafka` will be emitted with the following payload:
|
13
|
+
#
|
14
|
+
# * `message_count` – the number of messages in the message set.
|
15
|
+
# * `uncompressed_bytesize` – the byte size of the original data.
|
16
|
+
# * `compressed_bytesize` – the byte size of the compressed data.
|
17
|
+
#
|
18
|
+
class Compressor
|
19
|
+
|
20
|
+
# @param codec_name [Symbol, nil]
|
21
|
+
# @param threshold [Integer] the minimum number of messages in a message set
|
22
|
+
# that will trigger compression.
|
23
|
+
def initialize(codec_name:, threshold:)
|
24
|
+
@codec = Compression.find_codec(codec_name)
|
25
|
+
@threshold = threshold
|
26
|
+
end
|
27
|
+
|
28
|
+
# @param message_set [Protocol::MessageSet]
|
29
|
+
# @return [Protocol::MessageSet]
|
30
|
+
def compress(message_set)
|
31
|
+
return message_set if @codec.nil? || message_set.size < @threshold
|
32
|
+
|
33
|
+
compressed_data = compress_data(message_set)
|
34
|
+
|
35
|
+
wrapper_message = Protocol::Message.new(
|
36
|
+
value: compressed_data,
|
37
|
+
attributes: @codec.codec_id,
|
38
|
+
)
|
39
|
+
|
40
|
+
Protocol::MessageSet.new(messages: [wrapper_message])
|
41
|
+
end
|
42
|
+
|
43
|
+
private
|
44
|
+
|
45
|
+
def compress_data(message_set)
|
46
|
+
data = Protocol::Encoder.encode_with(message_set)
|
47
|
+
|
48
|
+
Instrumentation.instrument("compress.compressor.kafka") do |notification|
|
49
|
+
compressed_data = @codec.compress(data)
|
50
|
+
|
51
|
+
notification[:message_count] = message_set.size
|
52
|
+
notification[:uncompressed_bytesize] = data.bytesize
|
53
|
+
notification[:compressed_bytesize] = compressed_data.bytesize
|
54
|
+
|
55
|
+
compressed_data
|
56
|
+
end
|
57
|
+
end
|
58
|
+
end
|
59
|
+
end
|
data/lib/kafka/connection.rb
CHANGED
@@ -126,6 +126,7 @@ module Kafka
|
|
126
126
|
|
127
127
|
message = Kafka::Protocol::RequestMessage.new(
|
128
128
|
api_key: request.api_key,
|
129
|
+
api_version: request.respond_to?(:api_version) ? request.api_version : 0,
|
129
130
|
correlation_id: @correlation_id,
|
130
131
|
client_id: @client_id,
|
131
132
|
request: request,
|
@@ -0,0 +1,211 @@
|
|
1
|
+
require "kafka/consumer_group"
|
2
|
+
require "kafka/fetch_operation"
|
3
|
+
|
4
|
+
module Kafka
|
5
|
+
|
6
|
+
# @note This code is still alpha level. Don't use this for anything important.
|
7
|
+
# The API may also change without warning.
|
8
|
+
#
|
9
|
+
# A client that consumes messages from a Kafka cluster in coordination with
|
10
|
+
# other clients.
|
11
|
+
#
|
12
|
+
# A Consumer subscribes to one or more Kafka topics; all consumers with the
|
13
|
+
# same *group id* then agree on who should read from the individual topic
|
14
|
+
# partitions. When group members join or leave, the group synchronizes,
|
15
|
+
# making sure that all partitions are assigned to a single member, and that
|
16
|
+
# all members have some partitions to read from.
|
17
|
+
#
|
18
|
+
# ## Example
|
19
|
+
#
|
20
|
+
# A simple producer that simply writes the messages it consumes to the
|
21
|
+
# console.
|
22
|
+
#
|
23
|
+
# require "kafka"
|
24
|
+
#
|
25
|
+
# kafka = Kafka.new(seed_brokers: ["kafka1:9092", "kafka2:9092"])
|
26
|
+
#
|
27
|
+
# # Create a new Consumer instance in the group `my-group`:
|
28
|
+
# consumer = kafka.consumer(group_id: "my-group")
|
29
|
+
#
|
30
|
+
# # Subscribe to a Kafka topic:
|
31
|
+
# consumer.subscribe("messages")
|
32
|
+
#
|
33
|
+
# begin
|
34
|
+
# # Loop forever, reading in messages from all topics that have been
|
35
|
+
# # subscribed to.
|
36
|
+
# consumer.each_message do |message|
|
37
|
+
# puts message.topic
|
38
|
+
# puts message.partition
|
39
|
+
# puts message.key
|
40
|
+
# puts message.value
|
41
|
+
# puts message.offset
|
42
|
+
# end
|
43
|
+
# ensure
|
44
|
+
# # Make sure to shut down the consumer after use. This lets
|
45
|
+
# # the consumer notify the Kafka cluster that it's leaving
|
46
|
+
# # the group, causing a synchronization and re-balancing of
|
47
|
+
# # the group.
|
48
|
+
# consumer.shutdown
|
49
|
+
# end
|
50
|
+
#
|
51
|
+
class Consumer
|
52
|
+
|
53
|
+
# Creates a new Consumer.
|
54
|
+
#
|
55
|
+
# @param cluster [Kafka::Cluster]
|
56
|
+
# @param logger [Logger]
|
57
|
+
# @param group_id [String] the id of the group that the consumer should join.
|
58
|
+
# @param session_timeout [Integer] the interval between consumer heartbeats,
|
59
|
+
# in seconds.
|
60
|
+
def initialize(cluster:, logger:, group_id:, session_timeout: 30)
|
61
|
+
@cluster = cluster
|
62
|
+
@logger = logger
|
63
|
+
@group_id = group_id
|
64
|
+
@session_timeout = session_timeout
|
65
|
+
|
66
|
+
@group = ConsumerGroup.new(
|
67
|
+
cluster: cluster,
|
68
|
+
logger: logger,
|
69
|
+
group_id: group_id,
|
70
|
+
session_timeout: @session_timeout,
|
71
|
+
)
|
72
|
+
|
73
|
+
@offsets = {}
|
74
|
+
@default_offsets = {}
|
75
|
+
end
|
76
|
+
|
77
|
+
# Subscribes the consumer to a topic.
|
78
|
+
#
|
79
|
+
# Typically you either want to start reading messages from the very
|
80
|
+
# beginning of the topic's partitions or you simply want to wait for new
|
81
|
+
# messages to be written. In the former case, set `default_offsets` to
|
82
|
+
# `:earliest` (the default); in the latter, set it to `:latest`.
|
83
|
+
#
|
84
|
+
# @param topic [String] the name of the topic to subscribe to.
|
85
|
+
# @param default_offset [Symbol] whether to start from the beginning or the
|
86
|
+
# end of the topic's partitions.
|
87
|
+
# @return [nil]
|
88
|
+
def subscribe(topic, default_offset: :earliest)
|
89
|
+
@group.subscribe(topic)
|
90
|
+
@default_offsets[topic] = default_offset
|
91
|
+
|
92
|
+
nil
|
93
|
+
end
|
94
|
+
|
95
|
+
# Fetches and enumerates the messages in the topics that the consumer group
|
96
|
+
# subscribes to.
|
97
|
+
#
|
98
|
+
# Each message is yielded to the provided block. If the block returns
|
99
|
+
# without raising an exception, the message will be considered successfully
|
100
|
+
# processed. At regular intervals the offset of the most recent successfully
|
101
|
+
# processed message in each partition will be committed to the Kafka
|
102
|
+
# offset store. If the consumer crashes or leaves the group, the group member
|
103
|
+
# that is tasked with taking over processing of these partitions will resume
|
104
|
+
# at the last committed offsets.
|
105
|
+
#
|
106
|
+
# @yieldparam message [Kafka::FetchedMessage] a message fetched from Kafka.
|
107
|
+
# @return [nil]
|
108
|
+
def each_message
|
109
|
+
loop do
|
110
|
+
begin
|
111
|
+
batch = fetch_batch
|
112
|
+
|
113
|
+
batch.each do |message|
|
114
|
+
yield message
|
115
|
+
|
116
|
+
send_heartbeat_if_necessary
|
117
|
+
mark_message_as_processed(message)
|
118
|
+
end
|
119
|
+
rescue ConnectionError => e
|
120
|
+
@logger.error "Connection error while fetching messages: #{e}"
|
121
|
+
else
|
122
|
+
commit_offsets unless batch.nil? || batch.empty?
|
123
|
+
end
|
124
|
+
end
|
125
|
+
end
|
126
|
+
|
127
|
+
# Shuts down the consumer.
|
128
|
+
#
|
129
|
+
# In order to quickly have the consumer group re-balance itself, it's
|
130
|
+
# important that members explicitly tell Kafka when they're leaving.
|
131
|
+
# Therefore it's a good idea to call this method whenever your consumer
|
132
|
+
# is about to quit. If this method is not called, it may take up to
|
133
|
+
# the amount of time defined by the `session_timeout` parameter for
|
134
|
+
# Kafka to realize that this consumer is no longer present and trigger
|
135
|
+
# a group re-balance. In that period of time, the partitions that used
|
136
|
+
# to be assigned to this consumer won't be processed.
|
137
|
+
#
|
138
|
+
# @return [nil]
|
139
|
+
def shutdown
|
140
|
+
@group.leave
|
141
|
+
end
|
142
|
+
|
143
|
+
private
|
144
|
+
|
145
|
+
def fetch_batch
|
146
|
+
@group.join unless @group.member?
|
147
|
+
|
148
|
+
@logger.debug "Fetching a batch of messages"
|
149
|
+
|
150
|
+
assigned_partitions = @group.assigned_partitions
|
151
|
+
|
152
|
+
send_heartbeat_if_necessary
|
153
|
+
|
154
|
+
raise "No partitions assigned!" if assigned_partitions.empty?
|
155
|
+
|
156
|
+
operation = FetchOperation.new(
|
157
|
+
cluster: @cluster,
|
158
|
+
logger: @logger,
|
159
|
+
min_bytes: 1,
|
160
|
+
max_wait_time: 5,
|
161
|
+
)
|
162
|
+
|
163
|
+
offset_response = @group.fetch_offsets
|
164
|
+
|
165
|
+
assigned_partitions.each do |topic, partitions|
|
166
|
+
partitions.each do |partition|
|
167
|
+
offset = @offsets.fetch(topic, {}).fetch(partition) {
|
168
|
+
offset_response.offset_for(topic, partition)
|
169
|
+
}
|
170
|
+
|
171
|
+
offset = @default_offsets.fetch(topic) if offset < 0
|
172
|
+
|
173
|
+
@logger.debug "Fetching from #{topic}/#{partition} starting at offset #{offset}"
|
174
|
+
|
175
|
+
operation.fetch_from_partition(topic, partition, offset: offset)
|
176
|
+
end
|
177
|
+
end
|
178
|
+
|
179
|
+
messages = operation.execute
|
180
|
+
|
181
|
+
@logger.debug "Fetched #{messages.count} messages"
|
182
|
+
|
183
|
+
messages
|
184
|
+
end
|
185
|
+
|
186
|
+
def commit_offsets
|
187
|
+
@logger.debug "Committing offsets"
|
188
|
+
@group.commit_offsets(@offsets)
|
189
|
+
end
|
190
|
+
|
191
|
+
# Sends a heartbeat if it would be necessary in order to avoid getting
|
192
|
+
# kicked out of the consumer group.
|
193
|
+
#
|
194
|
+
# Each consumer needs to send a heartbeat with a frequency defined by
|
195
|
+
# `session_timeout`.
|
196
|
+
#
|
197
|
+
def send_heartbeat_if_necessary
|
198
|
+
@last_heartbeat ||= Time.at(0)
|
199
|
+
|
200
|
+
if @last_heartbeat <= Time.now - @session_timeout + 2
|
201
|
+
@group.heartbeat
|
202
|
+
@last_heartbeat = Time.now
|
203
|
+
end
|
204
|
+
end
|
205
|
+
|
206
|
+
def mark_message_as_processed(message)
|
207
|
+
@offsets[message.topic] ||= {}
|
208
|
+
@offsets[message.topic][message.partition] = message.offset + 1
|
209
|
+
end
|
210
|
+
end
|
211
|
+
end
|