karafka 2.1.5 → 2.1.7

Sign up to get free protection for your applications and to get access to all the features.
Files changed (38) hide show
  1. checksums.yaml +4 -4
  2. checksums.yaml.gz.sig +2 -2
  3. data/CHANGELOG.md +26 -1
  4. data/Gemfile.lock +15 -15
  5. data/karafka.gemspec +2 -2
  6. data/lib/karafka/admin.rb +35 -9
  7. data/lib/karafka/base_consumer.rb +10 -2
  8. data/lib/karafka/connection/client.rb +103 -86
  9. data/lib/karafka/errors.rb +4 -1
  10. data/lib/karafka/instrumentation/logger_listener.rb +3 -0
  11. data/lib/karafka/messages/seek.rb +3 -0
  12. data/lib/karafka/pro/iterator/expander.rb +95 -0
  13. data/lib/karafka/pro/iterator/tpl_builder.rb +160 -0
  14. data/lib/karafka/pro/iterator.rb +9 -92
  15. data/lib/karafka/pro/processing/filters_applier.rb +1 -0
  16. data/lib/karafka/pro/processing/strategies/aj/dlq_ftr_lrj_mom.rb +3 -1
  17. data/lib/karafka/pro/processing/strategies/aj/dlq_ftr_lrj_mom_vp.rb +3 -1
  18. data/lib/karafka/pro/processing/strategies/aj/dlq_lrj_mom.rb +3 -1
  19. data/lib/karafka/pro/processing/strategies/aj/dlq_lrj_mom_vp.rb +3 -1
  20. data/lib/karafka/pro/processing/strategies/aj/ftr_lrj_mom_vp.rb +3 -1
  21. data/lib/karafka/pro/processing/strategies/aj/lrj_mom_vp.rb +4 -1
  22. data/lib/karafka/pro/processing/strategies/dlq/ftr_lrj.rb +2 -2
  23. data/lib/karafka/pro/processing/strategies/dlq/ftr_lrj_mom.rb +2 -2
  24. data/lib/karafka/pro/processing/strategies/dlq/lrj.rb +2 -1
  25. data/lib/karafka/pro/processing/strategies/dlq/lrj_mom.rb +3 -1
  26. data/lib/karafka/pro/processing/strategies/ftr/default.rb +8 -1
  27. data/lib/karafka/pro/processing/strategies/lrj/default.rb +1 -1
  28. data/lib/karafka/pro/processing/strategies/lrj/ftr.rb +2 -2
  29. data/lib/karafka/pro/processing/strategies/lrj/ftr_mom.rb +2 -2
  30. data/lib/karafka/pro/processing/strategies/lrj/mom.rb +3 -1
  31. data/lib/karafka/pro/processing/virtual_offset_manager.rb +1 -1
  32. data/lib/karafka/processing/coordinator.rb +14 -0
  33. data/lib/karafka/railtie.rb +2 -2
  34. data/lib/karafka/setup/attributes_map.rb +2 -0
  35. data/lib/karafka/version.rb +1 -1
  36. data.tar.gz.sig +0 -0
  37. metadata +8 -6
  38. metadata.gz.sig +6 -1
@@ -0,0 +1,95 @@
1
+ # frozen_string_literal: true
2
+
3
+ # This Karafka component is a Pro component under a commercial license.
4
+ # This Karafka component is NOT licensed under LGPL.
5
+ #
6
+ # All of the commercial components are present in the lib/karafka/pro directory of this
7
+ # repository and their usage requires commercial license agreement.
8
+ #
9
+ # Karafka has also commercial-friendly license, commercial support and commercial components.
10
+ #
11
+ # By sending a pull request to the pro components, you are agreeing to transfer the copyright of
12
+ # your code to Maciej Mensfeld.
13
+
14
+ module Karafka
15
+ module Pro
16
+ class Iterator
17
+ # There are various ways you can provide topics information for iterating.
18
+ #
19
+ # This mapper normalizes this data, resolves offsets and maps the time based offsets into
20
+ # appropriate once
21
+ #
22
+ # Following formats are accepted:
23
+ #
24
+ # - 'topic1' - just a string with one topic name
25
+ # - ['topic1', 'topic2'] - just the names
26
+ # - { 'topic1' => -100 } - names with negative lookup offset
27
+ # - { 'topic1' => { 0 => 5 } } - names with exact partitions offsets
28
+ # - { 'topic1' => { 0 => -5 }, 'topic2' => { 1 => 5 } } - with per partition negative offsets
29
+ # - { 'topic1' => 100 } - means we run all partitions from the offset 100
30
+ # - { 'topic1' => Time.now - 60 } - we run all partitions from the message from 60s ago
31
+ # - { 'topic1' => { 1 => Time.now - 60 } } - partition1 from message 60s ago
32
+ #
33
+ class Expander
34
+ # Expands topics to which we want to subscribe with partitions information in case this
35
+ # info is not provided.
36
+ #
37
+ # @param topics [Array, Hash, String] topics definitions
38
+ # @return [Hash] expanded and normalized requested topics and partitions data
39
+ def call(topics)
40
+ expanded = Hash.new { |h, k| h[k] = {} }
41
+
42
+ normalize_format(topics).map do |topic, details|
43
+ if details.is_a?(Hash)
44
+ details.each do |partition, offset|
45
+ expanded[topic][partition] = offset
46
+ end
47
+ else
48
+ partition_count(topic).times do |partition|
49
+ # If no offsets are provided, we just start from zero
50
+ expanded[topic][partition] = details || 0
51
+ end
52
+ end
53
+ end
54
+
55
+ expanded
56
+ end
57
+
58
+ private
59
+
60
+ # Input can be provided in multiple formats. Here we normalize it to one (hash).
61
+ #
62
+ # @param topics [Array, Hash, String] requested topics
63
+ # @return [Hash] normalized hash with topics data
64
+ def normalize_format(topics)
65
+ # Simplification for the single topic case
66
+ topics = [topics] if topics.is_a?(String)
67
+
68
+ # If we've got just array with topics, we need to convert that into a representation
69
+ # that we can expand with offsets
70
+ topics = topics.map { |name| [name, false] }.to_h if topics.is_a?(Array)
71
+ # We remap by creating new hash, just in case the hash came as the argument for this
72
+ # expanded. We do not want to modify user provided hash
73
+ topics.transform_keys(&:to_s)
74
+ end
75
+
76
+ # List of topics with their partition information for expansion
77
+ # We cache it so we do not have to run consecutive requests to obtain data about multiple
78
+ # topics
79
+ def topics
80
+ @topics ||= Admin.cluster_info.topics
81
+ end
82
+
83
+ # @param name [String] topic name
84
+ # @return [Integer] number of partitions of the topic we want to iterate over
85
+ def partition_count(name)
86
+ topics
87
+ .find { |topic| topic.fetch(:topic_name) == name }
88
+ .tap { |topic| topic || raise(Errors::TopicNotFoundError, name) }
89
+ .fetch(:partitions)
90
+ .count
91
+ end
92
+ end
93
+ end
94
+ end
95
+ end
@@ -0,0 +1,160 @@
1
+ # frozen_string_literal: true
2
+
3
+ # This Karafka component is a Pro component under a commercial license.
4
+ # This Karafka component is NOT licensed under LGPL.
5
+ #
6
+ # All of the commercial components are present in the lib/karafka/pro directory of this
7
+ # repository and their usage requires commercial license agreement.
8
+ #
9
+ # Karafka has also commercial-friendly license, commercial support and commercial components.
10
+ #
11
+ # By sending a pull request to the pro components, you are agreeing to transfer the copyright of
12
+ # your code to Maciej Mensfeld.
13
+
14
+ module Karafka
15
+ module Pro
16
+ class Iterator
17
+ # Max time for a TPL request. We increase it to compensate for remote clusters latency
18
+ TPL_REQUEST_TIMEOUT = 2_000
19
+
20
+ private_constant :TPL_REQUEST_TIMEOUT
21
+
22
+ # Because we have various formats in which we can provide the offsets, before we can
23
+ # subscribe to them, there needs to be a bit of normalization.
24
+ #
25
+ # For some of the cases, we need to go to Kafka and get the real offsets or watermarks.
26
+ #
27
+ # This builder resolves that and builds a tpl to which we can safely subscribe the way
28
+ # we want it.
29
+ class TplBuilder
30
+ # @param consumer [::Rdkafka::Consumer] consumer instance needed to talk with Kafka
31
+ # @param expanded_topics [Hash] hash with expanded and normalized topics data
32
+ def initialize(consumer, expanded_topics)
33
+ @consumer = consumer
34
+ @expanded_topics = expanded_topics
35
+ @mapped_topics = Hash.new { |h, k| h[k] = {} }
36
+ end
37
+
38
+ # @return [Rdkafka::Consumer::TopicPartitionList] final tpl we can use to subscribe
39
+ def call
40
+ resolve_partitions_without_offsets
41
+ resolve_partitions_with_exact_offsets
42
+ resolve_partitions_with_negative_offsets
43
+ resolve_partitions_with_time_offsets
44
+
45
+ # Final tpl with all the data
46
+ tpl = Rdkafka::Consumer::TopicPartitionList.new
47
+
48
+ @mapped_topics.each do |name, partitions|
49
+ tpl.add_topic_and_partitions_with_offsets(name, partitions)
50
+ end
51
+
52
+ tpl
53
+ end
54
+
55
+ private
56
+
57
+ # First we expand on those partitions that do not have offsets defined.
58
+ # When we operate in case like this, we just start from beginning
59
+ def resolve_partitions_without_offsets
60
+ @expanded_topics.each do |name, partitions|
61
+ # We can here only about the case where we have partitions without offsets
62
+ next unless partitions.is_a?(Array) || partitions.is_a?(Range)
63
+
64
+ # When no offsets defined, we just start from zero
65
+ @mapped_topics[name] = partitions.map { |partition| [partition, 0] }.to_h
66
+ end
67
+ end
68
+
69
+ # If we get exact numeric offsets, we can just start from them without any extra work
70
+ def resolve_partitions_with_exact_offsets
71
+ @expanded_topics.each do |name, partitions|
72
+ next unless partitions.is_a?(Hash)
73
+
74
+ partitions.each do |partition, offset|
75
+ # Skip negative and time based offsets
76
+ next unless offset.is_a?(Integer) && offset >= 0
77
+
78
+ # Exact offsets can be used as they are
79
+ # No need for extra operations
80
+ @mapped_topics[name][partition] = offset
81
+ end
82
+ end
83
+ end
84
+
85
+ # If the offsets are negative, it means we want to fetch N last messages and we need to
86
+ # figure out the appropriate offsets
87
+ #
88
+ # We do it by getting the watermark offsets and just calculating it. This means that for
89
+ # heavily compacted topics, this may return less than the desired number but it is a
90
+ # limitation that is documented.
91
+ def resolve_partitions_with_negative_offsets
92
+ @expanded_topics.each do |name, partitions|
93
+ next unless partitions.is_a?(Hash)
94
+
95
+ partitions.each do |partition, offset|
96
+ # Care only about numerical offsets
97
+ #
98
+ # For time based we already resolve them via librdkafka lookup API
99
+ next unless offset.is_a?(Integer)
100
+
101
+ low_offset, high_offset = @consumer.query_watermark_offsets(name, partition)
102
+
103
+ # Care only about negative offsets (last n messages)
104
+ #
105
+ # We reject the above results but we **NEED** to run the `#query_watermark_offsets`
106
+ # for each topic partition nonetheless. Without this, librdkafka fetches a lot more
107
+ # metadata about each topic and each partition and this takes much more time than
108
+ # just getting watermarks. If we do not run watermark, at least an extra second
109
+ # is added at the beginning of iterator flow
110
+ #
111
+ # This may not be significant when this runs in the background but in case of
112
+ # using iterator in thins like Puma, it heavily impacts the end user experience
113
+ next unless offset.negative?
114
+
115
+ # We add because this offset is negative
116
+ @mapped_topics[name][partition] = [high_offset + offset, low_offset].max
117
+ end
118
+ end
119
+ end
120
+
121
+ # For time based offsets we first need to aggregate them and request the proper offsets.
122
+ # We want to get all times in one go for all tpls defined with times, so we accumulate
123
+ # them here and we will make one sync request to kafka for all.
124
+ def resolve_partitions_with_time_offsets
125
+ time_tpl = Rdkafka::Consumer::TopicPartitionList.new
126
+
127
+ # First we need to collect the time based once
128
+ @expanded_topics.each do |name, partitions|
129
+ next unless partitions.is_a?(Hash)
130
+
131
+ time_based = {}
132
+
133
+ partitions.each do |partition, offset|
134
+ next unless offset.is_a?(Time)
135
+
136
+ time_based[partition] = offset
137
+ end
138
+
139
+ next if time_based.empty?
140
+
141
+ time_tpl.add_topic_and_partitions_with_offsets(name, time_based)
142
+ end
143
+
144
+ # If there were no time-based, no need to query Kafka
145
+ return if time_tpl.empty?
146
+
147
+ real_offsets = @consumer.offsets_for_times(time_tpl, TPL_REQUEST_TIMEOUT)
148
+
149
+ real_offsets.to_h.each do |name, results|
150
+ results.each do |result|
151
+ raise(Errors::InvalidTimeBasedOffsetError) unless result
152
+
153
+ @mapped_topics[name][result.partition] = result.offset
154
+ end
155
+ end
156
+ end
157
+ end
158
+ end
159
+ end
160
+ end
@@ -39,6 +39,7 @@ module Karafka
39
39
  # overwritten, you may want to include `auto.offset.reset` to match your case.
40
40
  # @param yield_nil [Boolean] should we yield also `nil` values when poll returns nothing.
41
41
  # Useful in particular for long-living iterators.
42
+ # @param max_wait_time [Integer] max wait in ms when iterator did not receive any messages
42
43
  #
43
44
  # @note It is worth keeping in mind, that this API also needs to operate within
44
45
  # `max.poll.interval.ms` limitations on each iteration
@@ -48,9 +49,10 @@ module Karafka
48
49
  def initialize(
49
50
  topics,
50
51
  settings: { 'auto.offset.reset': 'beginning' },
51
- yield_nil: false
52
+ yield_nil: false,
53
+ max_wait_time: 200
52
54
  )
53
- @topics_with_partitions = expand_topics_with_partitions(topics)
55
+ @topics_with_partitions = Expander.new.call(topics)
54
56
 
55
57
  @routing_topics = @topics_with_partitions.map do |name, _|
56
58
  [name, ::Karafka::Routing::Router.find_or_initialize_by_name(name)]
@@ -62,6 +64,7 @@ module Karafka
62
64
 
63
65
  @settings = settings
64
66
  @yield_nil = yield_nil
67
+ @max_wait_time = max_wait_time
65
68
  end
66
69
 
67
70
  # Iterates over requested topic partitions and yields the results with the iterator itself
@@ -71,7 +74,7 @@ module Karafka
71
74
  # only eat up resources.
72
75
  def each
73
76
  Admin.with_consumer(@settings) do |consumer|
74
- tpl = tpl_with_expanded_offsets(consumer)
77
+ tpl = TplBuilder.new(consumer, @topics_with_partitions).call
75
78
  consumer.assign(tpl)
76
79
 
77
80
  # We need this for self-referenced APIs like pausing
@@ -80,7 +83,7 @@ module Karafka
80
83
  # Stream data until we reach the end of all the partitions or until the end user
81
84
  # indicates that they are done
82
85
  until done?
83
- message = poll(200)
86
+ message = poll
84
87
 
85
88
  # Skip nils if not explicitly required
86
89
  next if message.nil? && !@yield_nil
@@ -131,47 +134,9 @@ module Karafka
131
134
 
132
135
  private
133
136
 
134
- # Expands topics to which we want to subscribe with partitions information in case this
135
- # info is not provided. For our convenience we want to support 5 formats of defining
136
- # the subscribed topics:
137
- #
138
- # - 'topic1' - just a string with one topic name
139
- # - ['topic1', 'topic2'] - just the names
140
- # - { 'topic1' => -100 } - names with negative lookup offset
141
- # - { 'topic1' => { 0 => 5 } } - names with exact partitions offsets
142
- # - { 'topic1' => { 0 => -5 }, 'topic2' => { 1 => 5 } } - with per partition negative offsets
143
- #
144
- # @param topics [Array, Hash] topics definitions
145
- # @return [Hash] hash with topics containing partitions definitions
146
- def expand_topics_with_partitions(topics)
147
- # Simplification for the single topic case
148
- topics = [topics] if topics.is_a?(String)
149
- # If we've got just array with topics, we need to convert that into a representation
150
- # that we can expand with offsets
151
- topics = topics.map { |name| [name, false] }.to_h if topics.is_a?(Array)
152
-
153
- expanded = Hash.new { |h, k| h[k] = {} }
154
-
155
- topics.map do |topic, details|
156
- if details.is_a?(Hash)
157
- details.each do |partition, offset|
158
- expanded[topic][partition] = offset
159
- end
160
- else
161
- partition_count(topic.to_s).times do |partition|
162
- # If no offsets are provided, we just start from zero
163
- expanded[topic][partition] = details || 0
164
- end
165
- end
166
- end
167
-
168
- expanded
169
- end
170
-
171
- # @param timeout [Integer] timeout in ms
172
137
  # @return [Rdkafka::Consumer::Message, nil] message or nil if nothing to do
173
- def poll(timeout)
174
- @current_consumer.poll(timeout)
138
+ def poll
139
+ @current_consumer.poll(@max_wait_time)
175
140
  rescue Rdkafka::RdkafkaError => e
176
141
  # End of partition
177
142
  if e.code == :partition_eof
@@ -200,54 +165,6 @@ module Karafka
200
165
  def done?
201
166
  @stopped_partitions >= @total_partitions
202
167
  end
203
-
204
- # Builds the tpl representing all the subscriptions we want to run
205
- #
206
- # Additionally for negative offsets, does the watermark calculation where to start
207
- #
208
- # @param consumer [Rdkafka::Consumer] consumer we need in case of negative offsets as
209
- # negative are going to be used to do "give me last X". We use the already initialized
210
- # consumer instance, not to start another one again.
211
- # @return [Rdkafka::Consumer::TopicPartitionList]
212
- def tpl_with_expanded_offsets(consumer)
213
- tpl = Rdkafka::Consumer::TopicPartitionList.new
214
-
215
- @topics_with_partitions.each do |name, partitions|
216
- partitions_with_offsets = {}
217
-
218
- # When no offsets defined, we just start from zero
219
- if partitions.is_a?(Array) || partitions.is_a?(Range)
220
- partitions_with_offsets = partitions.map { |partition| [partition, 0] }.to_h
221
- else
222
- # When offsets defined, we can either use them if positive or expand and move back
223
- # in case of negative (-1000 means last 1000 messages, etc)
224
- partitions.each do |partition, offset|
225
- if offset.negative?
226
- _, high_watermark_offset = consumer.query_watermark_offsets(name, partition)
227
- # We add because this offset is negative
228
- partitions_with_offsets[partition] = high_watermark_offset + offset
229
- else
230
- partitions_with_offsets[partition] = offset
231
- end
232
- end
233
- end
234
-
235
- tpl.add_topic_and_partitions_with_offsets(name, partitions_with_offsets)
236
- end
237
-
238
- tpl
239
- end
240
-
241
- # @param name [String] topic name
242
- # @return [Integer] number of partitions of the topic we want to iterate over
243
- def partition_count(name)
244
- Admin
245
- .cluster_info
246
- .topics
247
- .find { |topic| topic.fetch(:topic_name) == name }
248
- .fetch(:partitions)
249
- .count
250
- end
251
168
  end
252
169
  end
253
170
  end
@@ -81,6 +81,7 @@ module Karafka
81
81
  # The first message we do need to get next time we poll. We use the minimum not to jump
82
82
  # accidentally by over any.
83
83
  # @return [Karafka::Messages::Message, nil] cursor message or nil if none
84
+ # @note Cursor message can also return the offset in the time format
84
85
  def cursor
85
86
  return nil unless active?
86
87
 
@@ -44,7 +44,9 @@ module Karafka
44
44
  if coordinator.filtered? && !revoked?
45
45
  handle_post_filtering
46
46
  elsif !revoked?
47
- seek(coordinator.seek_offset)
47
+ # no need to check for manual seek because AJ consumer is internal and
48
+ # fully controlled by us
49
+ seek(coordinator.seek_offset, false)
48
50
  resume
49
51
  else
50
52
  resume
@@ -50,7 +50,9 @@ module Karafka
50
50
  if coordinator.filtered? && !revoked?
51
51
  handle_post_filtering
52
52
  elsif !revoked?
53
- seek(coordinator.seek_offset)
53
+ # no need to check for manual seek because AJ consumer is internal and
54
+ # fully controlled by us
55
+ seek(coordinator.seek_offset, false)
54
56
  resume
55
57
  else
56
58
  resume
@@ -42,7 +42,9 @@ module Karafka
42
42
  if coordinator.success?
43
43
  coordinator.pause_tracker.reset
44
44
 
45
- seek(coordinator.seek_offset) unless revoked?
45
+ # no need to check for manual seek because AJ consumer is internal and
46
+ # fully controlled by us
47
+ seek(coordinator.seek_offset, false) unless revoked?
46
48
 
47
49
  resume
48
50
  elsif coordinator.pause_tracker.attempt <= topic.dead_letter_queue.max_retries
@@ -46,7 +46,9 @@ module Karafka
46
46
  # Since we have VP here we do not commit intermediate offsets and need to commit
47
47
  # them here. We do commit in collapsed mode but this is generalized.
48
48
  mark_as_consumed(last_group_message) unless revoked?
49
- seek(coordinator.seek_offset) unless revoked?
49
+ # no need to check for manual seek because AJ consumer is internal and
50
+ # fully controlled by us
51
+ seek(coordinator.seek_offset, false) unless revoked?
50
52
 
51
53
  resume
52
54
  elsif coordinator.pause_tracker.attempt <= topic.dead_letter_queue.max_retries
@@ -46,7 +46,9 @@ module Karafka
46
46
  if coordinator.filtered? && !revoked?
47
47
  handle_post_filtering
48
48
  elsif !revoked?
49
- seek(coordinator.seek_offset)
49
+ # no need to check for manual seek because AJ consumer is internal and
50
+ # fully controlled by us
51
+ seek(coordinator.seek_offset, false)
50
52
  resume
51
53
  else
52
54
  resume
@@ -48,7 +48,10 @@ module Karafka
48
48
  coordinator.pause_tracker.reset
49
49
 
50
50
  mark_as_consumed(last_group_message) unless revoked?
51
- seek(coordinator.seek_offset) unless revoked?
51
+
52
+ # no need to check for manual seek because AJ consumer is internal and
53
+ # fully controlled by us
54
+ seek(coordinator.seek_offset, false) unless revoked?
52
55
 
53
56
  resume
54
57
  else
@@ -47,8 +47,8 @@ module Karafka
47
47
 
48
48
  if coordinator.filtered? && !revoked?
49
49
  handle_post_filtering
50
- elsif !revoked?
51
- seek(coordinator.seek_offset)
50
+ elsif !revoked? && !coordinator.manual_seek?
51
+ seek(coordinator.seek_offset, false)
52
52
  resume
53
53
  else
54
54
  resume
@@ -42,8 +42,8 @@ module Karafka
42
42
 
43
43
  if coordinator.filtered? && !revoked?
44
44
  handle_post_filtering
45
- elsif !revoked?
46
- seek(last_group_message.offset + 1)
45
+ elsif !revoked? && !coordinator.manual_seek?
46
+ seek(last_group_message.offset + 1, false)
47
47
  resume
48
48
  else
49
49
  resume
@@ -38,7 +38,8 @@ module Karafka
38
38
  return if coordinator.manual_pause?
39
39
 
40
40
  mark_as_consumed(last_group_message) unless revoked?
41
- seek(coordinator.seek_offset) unless revoked?
41
+ # We should not overwrite user manual seel request with our seek
42
+ seek(coordinator.seek_offset, false) unless revoked? || coordinator.manual_seek?
42
43
 
43
44
  resume
44
45
  elsif coordinator.pause_tracker.attempt <= topic.dead_letter_queue.max_retries
@@ -37,7 +37,9 @@ module Karafka
37
37
 
38
38
  return if coordinator.manual_pause?
39
39
 
40
- seek(last_group_message.offset + 1) unless revoked?
40
+ unless revoked? || coordinator.manual_seek?
41
+ seek(last_group_message.offset + 1, false)
42
+ end
41
43
 
42
44
  resume
43
45
  elsif coordinator.pause_tracker.attempt <= topic.dead_letter_queue.max_retries
@@ -70,6 +70,10 @@ module Karafka
70
70
  when :skip
71
71
  nil
72
72
  when :seek
73
+ # User direct actions take priority over automatic operations
74
+ # If we've already seeked we can just resume operations, nothing extra needed
75
+ return resume if coordinator.manual_seek?
76
+
73
77
  throttle_message = filter.cursor
74
78
 
75
79
  Karafka.monitor.instrument(
@@ -77,11 +81,14 @@ module Karafka
77
81
  caller: self,
78
82
  message: throttle_message
79
83
  ) do
80
- seek(throttle_message.offset)
84
+ seek(throttle_message.offset, false)
81
85
  end
82
86
 
83
87
  resume
84
88
  when :pause
89
+ # User direct actions take priority over automatic operations
90
+ return nil if coordinator.manual_pause?
91
+
85
92
  throttle_message = filter.cursor
86
93
 
87
94
  Karafka.monitor.instrument(
@@ -53,7 +53,7 @@ module Karafka
53
53
  return if coordinator.manual_pause?
54
54
 
55
55
  mark_as_consumed(last_group_message) unless revoked?
56
- seek(coordinator.seek_offset) unless revoked?
56
+ seek(coordinator.seek_offset, false) unless revoked? || coordinator.manual_seek?
57
57
 
58
58
  resume
59
59
  else
@@ -45,10 +45,10 @@ module Karafka
45
45
  # If still not revoked and was throttled, we need to apply throttling logic
46
46
  if coordinator.filtered? && !revoked?
47
47
  handle_post_filtering
48
- elsif !revoked?
48
+ elsif !revoked? && !coordinator.manual_seek?
49
49
  # If not revoked and not throttled, we move to where we were suppose to and
50
50
  # resume
51
- seek(coordinator.seek_offset)
51
+ seek(coordinator.seek_offset, false)
52
52
  resume
53
53
  else
54
54
  resume
@@ -43,10 +43,10 @@ module Karafka
43
43
  # If still not revoked and was throttled, we need to apply filtering logic
44
44
  if coordinator.filtered? && !revoked?
45
45
  handle_post_filtering
46
- elsif !revoked?
46
+ elsif !revoked? && !coordinator.manual_seek?
47
47
  # If not revoked and not throttled, we move to where we were suppose to and
48
48
  # resume
49
- seek(last_group_message.offset + 1)
49
+ seek(last_group_message.offset + 1, false)
50
50
  resume
51
51
  else
52
52
  resume
@@ -52,7 +52,9 @@ module Karafka
52
52
 
53
53
  return if coordinator.manual_pause?
54
54
 
55
- seek(last_group_message.offset + 1) unless revoked?
55
+ unless revoked? || coordinator.manual_seek?
56
+ seek(last_group_message.offset + 1, false)
57
+ end
56
58
 
57
59
  resume
58
60
  else
@@ -118,7 +118,7 @@ module Karafka
118
118
 
119
119
  # @return [Messages::Seek] markable message for real offset marking
120
120
  def markable
121
- raise Errors::InvalidRealOffsetUsage unless markable?
121
+ raise Errors::InvalidRealOffsetUsageError unless markable?
122
122
 
123
123
  Messages::Seek.new(
124
124
  @topic,
@@ -23,6 +23,7 @@ module Karafka
23
23
  @consumptions = {}
24
24
  @running_jobs = 0
25
25
  @manual_pause = false
26
+ @manual_seek = false
26
27
  @mutex = Mutex.new
27
28
  @marked = false
28
29
  @failure = false
@@ -41,6 +42,9 @@ module Karafka
41
42
  # When starting to run, no pause is expected and no manual pause as well
42
43
  @manual_pause = false
43
44
 
45
+ # No user invoked seeks on a new run
46
+ @manual_seek = false
47
+
44
48
  # We set it on the first encounter and never again, because then the offset setting
45
49
  # should be up to the consumers logic (our or the end user)
46
50
  # Seek offset needs to be always initialized as for case where manual offset management
@@ -148,6 +152,16 @@ module Karafka
148
152
  @pause_tracker.paused? && @manual_pause
149
153
  end
150
154
 
155
+ # Marks seek as manual for coordination purposes
156
+ def manual_seek
157
+ @manual_seek = true
158
+ end
159
+
160
+ # @return [Boolean] did a user invoke seek in the current operations scope
161
+ def manual_seek?
162
+ @manual_seek
163
+ end
164
+
151
165
  # Allows to run synchronized (locked) code that can operate in between virtual partitions
152
166
  # @param block [Proc] code we want to run in the synchronized mode
153
167
  def synchronize(&block)