karafka 2.0.0.beta2 → 2.0.0.beta5
Sign up to get free protection for your applications and to get access to all the features.
- checksums.yaml +4 -4
- checksums.yaml.gz.sig +0 -0
- data/.github/workflows/ci.yml +18 -15
- data/CHANGELOG.md +49 -0
- data/Gemfile.lock +8 -8
- data/bin/benchmarks +2 -2
- data/bin/integrations +44 -15
- data/bin/scenario +29 -0
- data/bin/{stress → stress_many} +0 -0
- data/bin/stress_one +13 -0
- data/bin/wait_for_kafka +20 -0
- data/docker-compose.yml +28 -11
- data/karafka.gemspec +2 -2
- data/lib/karafka/active_job/routing/extensions.rb +12 -2
- data/lib/karafka/app.rb +2 -1
- data/lib/karafka/base_consumer.rb +75 -45
- data/lib/karafka/connection/client.rb +88 -22
- data/lib/karafka/connection/listener.rb +60 -18
- data/lib/karafka/connection/pauses_manager.rb +8 -0
- data/lib/karafka/connection/rebalance_manager.rb +20 -19
- data/lib/karafka/contracts/config.rb +17 -3
- data/lib/karafka/contracts/server_cli_options.rb +1 -1
- data/lib/karafka/errors.rb +3 -0
- data/lib/karafka/instrumentation/logger_listener.rb +34 -10
- data/lib/karafka/instrumentation/monitor.rb +3 -1
- data/lib/karafka/licenser.rb +26 -7
- data/lib/karafka/pro/active_job/consumer.rb +30 -9
- data/lib/karafka/pro/active_job/dispatcher.rb +9 -9
- data/lib/karafka/pro/active_job/job_options_contract.rb +9 -9
- data/lib/karafka/pro/base_consumer.rb +73 -0
- data/lib/karafka/pro/loader.rb +38 -20
- data/lib/karafka/pro/performance_tracker.rb +9 -9
- data/lib/karafka/pro/processing/coordinator.rb +12 -0
- data/lib/karafka/pro/processing/jobs/consume_non_blocking.rb +10 -11
- data/lib/karafka/pro/processing/jobs_builder.rb +32 -0
- data/lib/karafka/pro/processing/scheduler.rb +56 -0
- data/lib/karafka/pro/routing/extensions.rb +32 -0
- data/lib/karafka/processing/coordinator.rb +84 -0
- data/lib/karafka/processing/coordinators_buffer.rb +58 -0
- data/lib/karafka/processing/executor.rb +23 -9
- data/lib/karafka/processing/executors_buffer.rb +46 -15
- data/lib/karafka/processing/jobs/base.rb +8 -3
- data/lib/karafka/processing/jobs/consume.rb +11 -4
- data/lib/karafka/processing/jobs_builder.rb +29 -0
- data/lib/karafka/processing/result.rb +29 -0
- data/lib/karafka/processing/scheduler.rb +22 -0
- data/lib/karafka/processing/worker.rb +17 -9
- data/lib/karafka/routing/consumer_group.rb +1 -1
- data/lib/karafka/routing/subscription_group.rb +1 -1
- data/lib/karafka/routing/topic.rb +14 -0
- data/lib/karafka/setup/config.rb +19 -9
- data/lib/karafka/status.rb +1 -3
- data/lib/karafka/version.rb +1 -1
- data.tar.gz.sig +0 -0
- metadata +19 -7
- metadata.gz.sig +0 -0
- data/lib/karafka/pro/scheduler.rb +0 -54
- data/lib/karafka/scheduler.rb +0 -20
checksums.yaml
CHANGED
@@ -1,7 +1,7 @@
|
|
1
1
|
---
|
2
2
|
SHA256:
|
3
|
-
metadata.gz:
|
4
|
-
data.tar.gz:
|
3
|
+
metadata.gz: 2c8e680ffdf69f88899a715c84cc484e8f568f4a93da9284195f4bf55a283ee1
|
4
|
+
data.tar.gz: 974356226a10ba2c77de770351a47180716533021a89040bcdc1aae57f452121
|
5
5
|
SHA512:
|
6
|
-
metadata.gz:
|
7
|
-
data.tar.gz:
|
6
|
+
metadata.gz: 2427aaae1b1b07430df7c9f042d290bbae8380fb1f6ec7c26eecee92b8fe79e13ea9f3a99a36bf89b314ffba809c556618b22c0a87f0c0c83bb73cf8af72321b
|
7
|
+
data.tar.gz: 55e18448b5645acd38c4194967ea7df657c142d82a105699f7b204f222f8dfb2dbd14cce82b1f424ec177afb78049b3e7588642013674a3c2923a8848b6b87e7
|
checksums.yaml.gz.sig
CHANGED
Binary file
|
data/.github/workflows/ci.yml
CHANGED
@@ -8,6 +8,10 @@ on:
|
|
8
8
|
schedule:
|
9
9
|
- cron: '0 1 * * *'
|
10
10
|
|
11
|
+
env:
|
12
|
+
BUNDLE_RETRY: 6
|
13
|
+
BUNDLE_JOBS: 4
|
14
|
+
|
11
15
|
jobs:
|
12
16
|
diffend:
|
13
17
|
runs-on: ubuntu-latest
|
@@ -17,14 +21,16 @@ jobs:
|
|
17
21
|
- uses: actions/checkout@v2
|
18
22
|
with:
|
19
23
|
fetch-depth: 0
|
24
|
+
|
20
25
|
- name: Set up Ruby
|
21
26
|
uses: ruby/setup-ruby@v1
|
22
27
|
with:
|
23
28
|
ruby-version: 3.1
|
24
|
-
|
25
|
-
|
29
|
+
bundler-cache: true
|
30
|
+
|
26
31
|
- name: Install Diffend plugin
|
27
32
|
run: bundle plugin install diffend
|
33
|
+
|
28
34
|
- name: Bundle Secure
|
29
35
|
run: bundle secure
|
30
36
|
|
@@ -57,25 +63,19 @@ jobs:
|
|
57
63
|
- name: Install package dependencies
|
58
64
|
run: "[ -e $APT_DEPS ] || sudo apt-get install -y --no-install-recommends $APT_DEPS"
|
59
65
|
|
60
|
-
- name:
|
66
|
+
- name: Start Kafka with docker-compose
|
61
67
|
run: |
|
62
68
|
docker-compose up -d
|
63
|
-
sleep 10
|
64
69
|
|
65
70
|
- name: Set up Ruby
|
66
71
|
uses: ruby/setup-ruby@v1
|
67
72
|
with:
|
68
73
|
ruby-version: ${{matrix.ruby}}
|
74
|
+
bundler-cache: true
|
69
75
|
|
70
|
-
- name:
|
76
|
+
- name: Ensure all needed Kafka topics are created and wait if not
|
71
77
|
run: |
|
72
|
-
|
73
|
-
bundle config set without 'tools benchmarks docs'
|
74
|
-
|
75
|
-
- name: Bundle install
|
76
|
-
run: |
|
77
|
-
bundle config set without development
|
78
|
-
bundle install --jobs 4 --retry 3
|
78
|
+
bin/wait_for_kafka
|
79
79
|
|
80
80
|
- name: Run all specs
|
81
81
|
env:
|
@@ -100,10 +100,9 @@ jobs:
|
|
100
100
|
- name: Install package dependencies
|
101
101
|
run: "[ -e $APT_DEPS ] || sudo apt-get install -y --no-install-recommends $APT_DEPS"
|
102
102
|
|
103
|
-
- name:
|
103
|
+
- name: Start Kafka with docker-compose
|
104
104
|
run: |
|
105
105
|
docker-compose up -d
|
106
|
-
sleep 5
|
107
106
|
|
108
107
|
- name: Set up Ruby
|
109
108
|
uses: ruby/setup-ruby@v1
|
@@ -119,7 +118,11 @@ jobs:
|
|
119
118
|
- name: Bundle install
|
120
119
|
run: |
|
121
120
|
bundle config set without development
|
122
|
-
bundle install
|
121
|
+
bundle install
|
122
|
+
|
123
|
+
- name: Ensure all needed Kafka topics are created and wait if not
|
124
|
+
run: |
|
125
|
+
bin/wait_for_kafka
|
123
126
|
|
124
127
|
- name: Run integration tests
|
125
128
|
env:
|
data/CHANGELOG.md
CHANGED
@@ -1,5 +1,54 @@
|
|
1
1
|
# Karafka framework changelog
|
2
2
|
|
3
|
+
## 2.0.0-beta5 (2022-07-05)
|
4
|
+
- Always resume processing of a revoked partition upon assignment.
|
5
|
+
- Improve specs stability.
|
6
|
+
- Fix a case where revocation job would be executed on partition for which we never did any work.
|
7
|
+
- Introduce a jobs group coordinator for easier jobs management.
|
8
|
+
- Improve stability of resuming paused partitions that were revoked and re-assigned.
|
9
|
+
- Optimize reaction time on partition ownership changes.
|
10
|
+
- Fix a bug where despite setting long max wait time, we would return messages prior to it while not reaching the desired max messages count.
|
11
|
+
- Add more integration specs related to polling limits.
|
12
|
+
- Remove auto-detection of re-assigned partitions upon rebalance as for too fast rebalances it could not be accurate enough. It would also mess up in case of rebalances that would happen right after a `#seek` was issued for a partition.
|
13
|
+
- Optimize the removal of pre-buffered lost partitions data.
|
14
|
+
- Always rune `#revoked` when rebalance with revocation happens.
|
15
|
+
- Evict executors upon rebalance, to prevent race-conditions.
|
16
|
+
- Align topics names for integration specs.
|
17
|
+
|
18
|
+
## 2.0.0-beta4 (2022-06-20)
|
19
|
+
- Rename job internal api methods from `#prepare` to `#before_call` and from `#teardown` to `#after_call` to abstract away jobs execution from any type of executors and consumers logic
|
20
|
+
- Remove ability of running `before_consume` and `after_consume` completely. Those should be for internal usage only.
|
21
|
+
- Reorganize how Pro consumer and Pro AJ consumers inherit.
|
22
|
+
- Require WaterDrop `2.3.1`.
|
23
|
+
- Add more integration specs for rebalancing and max poll exceeded.
|
24
|
+
- Move `revoked?` state from PRO to regular Karafka.
|
25
|
+
- Use return value of `mark_as_consumed!` and `mark_as_consumed` as indicator of partition ownership + use it to switch the ownership state.
|
26
|
+
- Do not remove rebalance manager upon client reset and recovery. This will allow us to keep the notion of lost partitions, so we can run revocation jobs for blocking jobs that exceeded the max poll interval.
|
27
|
+
- Run revocation jobs upon reaching max poll interval for blocking jobs.
|
28
|
+
- Early exit `poll` operation upon partition lost or max poll exceeded event.
|
29
|
+
- Always reset consumer instances on timeout exceeded.
|
30
|
+
- Wait for Kafka to create all the needed topics before running specs in CI.
|
31
|
+
|
32
|
+
## 2.0.0-beta3 (2022-06-14)
|
33
|
+
- Jobs building responsibility extracted out of the listener code base.
|
34
|
+
- Fix a case where specs supervisor would try to kill no longer running process (#868)
|
35
|
+
- Fix an instable integration spec that could misbehave under load
|
36
|
+
- Commit offsets prior to pausing partitions to ensure that the latest offset is always committed
|
37
|
+
- Fix a case where consecutive CTRL+C (non-stop) would case an exception during forced shutdown
|
38
|
+
- Add missing `consumer.prepared.error` into `LoggerListener`
|
39
|
+
- Delegate partition resuming from the consumers to listeners threads.
|
40
|
+
- Add support for Long Running Jobs (LRJ) for ActiveJob [PRO]
|
41
|
+
- Add support for Long Running Jobs for consumers [PRO]
|
42
|
+
- Allow `active_job_topic` to accept a block for extra topic related settings
|
43
|
+
- Remove no longer needed logger threads
|
44
|
+
- Auto-adapt number of processes for integration specs based on the number of CPUs
|
45
|
+
- Introduce an integration spec runner that prints everything to stdout (better for development)
|
46
|
+
- Introduce extra integration specs for various ActiveJob usage scenarios
|
47
|
+
- Rename consumer method `#prepared` to `#prepare` to reflect better its use-case
|
48
|
+
- For test and dev raise an error when expired license key is used (never for non dev)
|
49
|
+
- Add worker related monitor events (`worker.process` and `worker.processed`)
|
50
|
+
- Update `LoggerListener` to include more useful information about processing and polling messages
|
51
|
+
|
3
52
|
## 2.0.0-beta2 (2022-06-07)
|
4
53
|
- Abstract away notion of topics groups (until now it was just an array)
|
5
54
|
- Optimize how jobs queue is closed. Since we enqueue jobs only from the listeners, we can safely close jobs queue once listeners are done. By extracting this responsibility from listeners, we remove corner cases and race conditions. Note here: for non-blocking jobs we do wait for them to finish while running the `poll`. This ensures, that for async jobs that are long-living, we do not reach `max.poll.interval`.
|
data/Gemfile.lock
CHANGED
@@ -1,13 +1,13 @@
|
|
1
1
|
PATH
|
2
2
|
remote: .
|
3
3
|
specs:
|
4
|
-
karafka (2.0.0.
|
4
|
+
karafka (2.0.0.beta5)
|
5
5
|
dry-configurable (~> 0.13)
|
6
6
|
dry-monitor (~> 0.5)
|
7
7
|
dry-validation (~> 1.7)
|
8
8
|
rdkafka (>= 0.10)
|
9
9
|
thor (>= 0.20)
|
10
|
-
waterdrop (>= 2.3.
|
10
|
+
waterdrop (>= 2.3.1, < 3.0.0)
|
11
11
|
zeitwerk (~> 2.3)
|
12
12
|
|
13
13
|
GEM
|
@@ -45,7 +45,7 @@ GEM
|
|
45
45
|
dry-configurable (~> 0.13, >= 0.13.0)
|
46
46
|
dry-core (~> 0.5, >= 0.5)
|
47
47
|
dry-events (~> 0.2)
|
48
|
-
dry-schema (1.9.
|
48
|
+
dry-schema (1.9.2)
|
49
49
|
concurrent-ruby (~> 1.0)
|
50
50
|
dry-configurable (~> 0.13, >= 0.13.0)
|
51
51
|
dry-core (~> 0.5, >= 0.5)
|
@@ -58,12 +58,12 @@ GEM
|
|
58
58
|
dry-core (~> 0.5, >= 0.5)
|
59
59
|
dry-inflector (~> 0.1, >= 0.1.2)
|
60
60
|
dry-logic (~> 1.0, >= 1.0.2)
|
61
|
-
dry-validation (1.8.
|
61
|
+
dry-validation (1.8.1)
|
62
62
|
concurrent-ruby (~> 1.0)
|
63
63
|
dry-container (~> 0.7, >= 0.7.1)
|
64
64
|
dry-core (~> 0.5, >= 0.5)
|
65
65
|
dry-initializer (~> 3.0)
|
66
|
-
dry-schema (~> 1.
|
66
|
+
dry-schema (~> 1.8, >= 1.8.0)
|
67
67
|
factory_bot (6.2.1)
|
68
68
|
activesupport (>= 5.0.0)
|
69
69
|
ffi (1.15.5)
|
@@ -74,7 +74,7 @@ GEM
|
|
74
74
|
mini_portile2 (2.8.0)
|
75
75
|
minitest (5.15.0)
|
76
76
|
rake (13.0.6)
|
77
|
-
rdkafka (0.
|
77
|
+
rdkafka (0.12.0)
|
78
78
|
ffi (~> 1.15)
|
79
79
|
mini_portile2 (~> 2.6)
|
80
80
|
rake (> 12)
|
@@ -100,14 +100,14 @@ GEM
|
|
100
100
|
thor (1.2.1)
|
101
101
|
tzinfo (2.0.4)
|
102
102
|
concurrent-ruby (~> 1.0)
|
103
|
-
waterdrop (2.3.
|
103
|
+
waterdrop (2.3.1)
|
104
104
|
concurrent-ruby (>= 1.1)
|
105
105
|
dry-configurable (~> 0.13)
|
106
106
|
dry-monitor (~> 0.5)
|
107
107
|
dry-validation (~> 1.7)
|
108
108
|
rdkafka (>= 0.10)
|
109
109
|
zeitwerk (~> 2.3)
|
110
|
-
zeitwerk (2.
|
110
|
+
zeitwerk (2.6.0)
|
111
111
|
|
112
112
|
PLATFORMS
|
113
113
|
x86_64-linux
|
data/bin/benchmarks
CHANGED
@@ -39,8 +39,8 @@ if ENV['SEED']
|
|
39
39
|
|
40
40
|
# We do not populate data of benchmarks_0_10 as we use it with life-stream data only
|
41
41
|
%w[
|
42
|
-
|
43
|
-
|
42
|
+
benchmarks_00_01
|
43
|
+
benchmarks_00_05
|
44
44
|
].each do |topic_name|
|
45
45
|
partitions_count = topic_name.split('_').last.to_i
|
46
46
|
|
data/bin/integrations
CHANGED
@@ -11,21 +11,24 @@ require 'open3'
|
|
11
11
|
require 'fileutils'
|
12
12
|
require 'pathname'
|
13
13
|
require 'tmpdir'
|
14
|
+
require 'etc'
|
14
15
|
|
15
16
|
ROOT_PATH = Pathname.new(File.expand_path(File.join(File.dirname(__FILE__), '../')))
|
16
17
|
|
17
|
-
# Raised from the parent process if any of the integration tests fails
|
18
|
-
IntegrationTestError = Class.new(StandardError)
|
19
|
-
|
20
18
|
# How many child processes with integration specs do we want to run in parallel
|
21
|
-
# When the value is high, there's a problem with thread allocation on Github
|
22
|
-
|
19
|
+
# When the value is high, there's a problem with thread allocation on Github CI, tht is why
|
20
|
+
# we limit it. Locally we can run a lot of those, as many of them have sleeps and do not use a lot
|
21
|
+
# of CPU
|
22
|
+
CONCURRENCY = ENV.key?('CI') ? 5 : Etc.nprocessors * 2
|
23
|
+
|
24
|
+
# How may bytes do we want to keep from the stdout in the buffer for when we need to print it
|
25
|
+
MAX_BUFFER_OUTPUT = 10_240
|
23
26
|
|
24
27
|
# Abstraction around a single test scenario execution process
|
25
28
|
class Scenario
|
26
29
|
# How long a scenario can run before we kill it
|
27
30
|
# This is a fail-safe just in case something would hang
|
28
|
-
MAX_RUN_TIME =
|
31
|
+
MAX_RUN_TIME = 3 * 60 # 3 minutes tops
|
29
32
|
|
30
33
|
# There are rare cases where Karafka may force shutdown for some of the integration cases
|
31
34
|
# This includes exactly those
|
@@ -73,15 +76,20 @@ class Scenario
|
|
73
76
|
# If the thread is running too long, kill it
|
74
77
|
if current_time - @started_at > MAX_RUN_TIME
|
75
78
|
@wait_thr.kill
|
76
|
-
|
79
|
+
|
80
|
+
begin
|
81
|
+
Process.kill('TERM', pid)
|
82
|
+
# It may finish right after we want to kill it, that's why we ignore this
|
83
|
+
rescue Errno::ESRCH
|
84
|
+
end
|
77
85
|
end
|
78
86
|
|
79
87
|
# We read it so it won't grow as we use our default logger that prints to both test.log and
|
80
88
|
# to stdout. Otherwise after reaching the buffer size, it would hang
|
81
89
|
buffer = ''
|
82
|
-
@stdout.read_nonblock(
|
90
|
+
@stdout.read_nonblock(MAX_BUFFER_OUTPUT, buffer, exception: false)
|
83
91
|
@stdout_tail << buffer
|
84
|
-
@stdout_tail = @stdout_tail[-
|
92
|
+
@stdout_tail = @stdout_tail[-MAX_BUFFER_OUTPUT..-1] || @stdout_tail
|
85
93
|
|
86
94
|
!@wait_thr.alive?
|
87
95
|
end
|
@@ -106,14 +114,19 @@ class Scenario
|
|
106
114
|
|
107
115
|
# Prints a status report when scenario is finished and stdout if it failed
|
108
116
|
def report
|
109
|
-
|
117
|
+
if success?
|
118
|
+
print "\e[#{32}m#{'.'}\e[0m"
|
119
|
+
else
|
120
|
+
buffer = ''
|
110
121
|
|
111
|
-
|
122
|
+
@stderr.read_nonblock(MAX_BUFFER_OUTPUT, buffer, exception: false)
|
112
123
|
|
113
|
-
|
124
|
+
puts
|
125
|
+
puts "\e[#{31}m#{'[FAILED]'}\e[0m #{name}"
|
114
126
|
puts "Exit code: #{exit_code}"
|
115
127
|
puts @stdout_tail
|
116
|
-
puts
|
128
|
+
puts buffer
|
129
|
+
puts
|
117
130
|
end
|
118
131
|
end
|
119
132
|
|
@@ -204,5 +217,21 @@ while finished_scenarios.size < scenarios.size
|
|
204
217
|
sleep(0.1)
|
205
218
|
end
|
206
219
|
|
207
|
-
|
208
|
-
|
220
|
+
failed_scenarios = finished_scenarios.reject(&:success?)
|
221
|
+
|
222
|
+
# Report once more on the failed jobs
|
223
|
+
# This will only list scenarios that failed without printing their stdout here.
|
224
|
+
if failed_scenarios.empty?
|
225
|
+
puts
|
226
|
+
else
|
227
|
+
puts "\nFailed scenarios:\n\n"
|
228
|
+
|
229
|
+
failed_scenarios.each do |scenario|
|
230
|
+
puts "\e[#{31}m#{'[FAILED]'}\e[0m #{scenario.name}"
|
231
|
+
end
|
232
|
+
|
233
|
+
puts
|
234
|
+
|
235
|
+
# Exit with 1 if not all scenarios were successful
|
236
|
+
exit 1
|
237
|
+
end
|
data/bin/scenario
ADDED
@@ -0,0 +1,29 @@
|
|
1
|
+
#!/usr/bin/env ruby
|
2
|
+
|
3
|
+
# Runner for non-parallel execution of a single scenario.
|
4
|
+
# It prints all the info stdout, etc and basically replaces itself with the scenario execution.
|
5
|
+
# It is useful when we work with a single spec and we need all the debug info
|
6
|
+
|
7
|
+
raise 'This code needs to be executed WITHOUT bundle exec' if Kernel.const_defined?(:Bundler)
|
8
|
+
|
9
|
+
require 'open3'
|
10
|
+
require 'fileutils'
|
11
|
+
require 'pathname'
|
12
|
+
require 'tmpdir'
|
13
|
+
require 'etc'
|
14
|
+
|
15
|
+
ROOT_PATH = Pathname.new(File.expand_path(File.join(File.dirname(__FILE__), '../')))
|
16
|
+
|
17
|
+
# Load all the specs
|
18
|
+
specs = Dir[ROOT_PATH.join('spec/integrations/**/*.rb')]
|
19
|
+
|
20
|
+
# If filters is provided, apply
|
21
|
+
# Allows to provide several filters one after another and applies all of them
|
22
|
+
ARGV.each do |filter|
|
23
|
+
specs.delete_if { |name| !name.include?(filter) }
|
24
|
+
end
|
25
|
+
|
26
|
+
raise ArgumentError, "No integration specs with filters: #{ARGV.join(', ')}" if specs.empty?
|
27
|
+
raise ArgumentError, "Many specs found with filters: #{ARGV.join(', ')}" if specs.size != 1
|
28
|
+
|
29
|
+
exec("bundle exec ruby -r #{ROOT_PATH}/spec/integrations_helper.rb #{specs[0]}")
|
data/bin/{stress → stress_many}
RENAMED
File without changes
|
data/bin/stress_one
ADDED
@@ -0,0 +1,13 @@
|
|
1
|
+
#!/bin/bash
|
2
|
+
|
3
|
+
# Runs a single integration spec in an endless loop
|
4
|
+
# This allows us to ensure (after long enough time) that the integration spec is stable and
|
5
|
+
# that there are no anomalies when running it for a long period of time
|
6
|
+
|
7
|
+
set -e
|
8
|
+
|
9
|
+
while :
|
10
|
+
do
|
11
|
+
reset
|
12
|
+
bin/scenario $1
|
13
|
+
done
|
data/bin/wait_for_kafka
ADDED
@@ -0,0 +1,20 @@
|
|
1
|
+
#!/bin/bash
|
2
|
+
|
3
|
+
# This script allows us to wait for Kafka docker to fully be ready
|
4
|
+
# We consider it fully ready when all our topics that need to be created are created as expected
|
5
|
+
|
6
|
+
KAFKA_NAME='karafka_20_kafka'
|
7
|
+
ZOOKEEPER='zookeeper:2181'
|
8
|
+
LIST_CMD="kafka-topics.sh --list --zookeeper $ZOOKEEPER"
|
9
|
+
|
10
|
+
# Take the number of topics that we need to create prior to running anything
|
11
|
+
TOPICS_COUNT=`cat docker-compose.yml | grep -E -i 'integrations_|benchmarks_' | wc -l`
|
12
|
+
|
13
|
+
# And wait until all of them are created
|
14
|
+
until (((`docker exec $KAFKA_NAME $LIST_CMD | wc -l`) >= $TOPICS_COUNT));
|
15
|
+
do
|
16
|
+
echo "Waiting for Kafka to create all the needed topics..."
|
17
|
+
sleep 1
|
18
|
+
done
|
19
|
+
|
20
|
+
echo "All the needed topics created."
|
data/docker-compose.yml
CHANGED
@@ -1,10 +1,12 @@
|
|
1
1
|
version: '2'
|
2
2
|
services:
|
3
3
|
zookeeper:
|
4
|
+
container_name: karafka_20_zookeeper
|
4
5
|
image: wurstmeister/zookeeper
|
5
6
|
ports:
|
6
7
|
- '2181:2181'
|
7
8
|
kafka:
|
9
|
+
container_name: karafka_20_kafka
|
8
10
|
image: wurstmeister/kafka
|
9
11
|
ports:
|
10
12
|
- '9092:9092'
|
@@ -14,16 +16,31 @@ services:
|
|
14
16
|
KAFKA_ZOOKEEPER_CONNECT: zookeeper:2181
|
15
17
|
KAFKA_AUTO_CREATE_TOPICS_ENABLE: 'true'
|
16
18
|
KAFKA_CREATE_TOPICS:
|
17
|
-
"
|
18
|
-
|
19
|
-
|
20
|
-
|
21
|
-
|
22
|
-
|
23
|
-
|
24
|
-
|
25
|
-
|
26
|
-
|
27
|
-
|
19
|
+
"integrations_00_02:2:1,\
|
20
|
+
integrations_01_02:2:1,\
|
21
|
+
integrations_02_02:2:1,\
|
22
|
+
integrations_03_02:2:1,\
|
23
|
+
integrations_04_02:2:1,\
|
24
|
+
integrations_05_02:2:1,\
|
25
|
+
integrations_06_02:2:1,\
|
26
|
+
integrations_07_02:2:1,\
|
27
|
+
integrations_08_02:2:1,\
|
28
|
+
integrations_09_02:2:1,\
|
29
|
+
integrations_10_02:2:1,\
|
30
|
+
integrations_11_02:2:1,\
|
31
|
+
integrations_12_02:2:1,\
|
32
|
+
integrations_13_02:2:1,\
|
33
|
+
integrations_14_02:2:1,\
|
34
|
+
integrations_15_02:2:1,\
|
35
|
+
integrations_16_02:2:1,\
|
36
|
+
integrations_00_03:3:1,\
|
37
|
+
integrations_01_03:3:1,\
|
38
|
+
integrations_02_03:3:1,\
|
39
|
+
integrations_03_03:3:1,\
|
40
|
+
integrations_00_10:10:1,\
|
41
|
+
integrations_01_10:10:1,\
|
42
|
+
benchmarks_00_01:1:1,\
|
43
|
+
benchmarks_00_05:5:1,\
|
44
|
+
benchmarks_00_10:10:1"
|
28
45
|
volumes:
|
29
46
|
- /var/run/docker.sock:/var/run/docker.sock
|
data/karafka.gemspec
CHANGED
@@ -14,14 +14,14 @@ Gem::Specification.new do |spec|
|
|
14
14
|
spec.homepage = 'https://karafka.io'
|
15
15
|
spec.summary = 'Ruby based framework for working with Apache Kafka'
|
16
16
|
spec.description = 'Framework used to simplify Apache Kafka based Ruby applications development'
|
17
|
-
spec.
|
17
|
+
spec.licenses = ['LGPL-3.0', 'Commercial']
|
18
18
|
|
19
19
|
spec.add_dependency 'dry-configurable', '~> 0.13'
|
20
20
|
spec.add_dependency 'dry-monitor', '~> 0.5'
|
21
21
|
spec.add_dependency 'dry-validation', '~> 1.7'
|
22
22
|
spec.add_dependency 'rdkafka', '>= 0.10'
|
23
23
|
spec.add_dependency 'thor', '>= 0.20'
|
24
|
-
spec.add_dependency 'waterdrop', '>= 2.3.
|
24
|
+
spec.add_dependency 'waterdrop', '>= 2.3.1', '< 3.0.0'
|
25
25
|
spec.add_dependency 'zeitwerk', '~> 2.3'
|
26
26
|
|
27
27
|
spec.required_ruby_version = '>= 2.6.0'
|
@@ -10,9 +10,19 @@ module Karafka
|
|
10
10
|
# This method simplifies routes definition for ActiveJob topics / queues by auto-injecting
|
11
11
|
# the consumer class
|
12
12
|
# @param name [String, Symbol] name of the topic where ActiveJobs jobs should go
|
13
|
-
|
13
|
+
# @param block [Proc] block that we can use for some extra configuration
|
14
|
+
def active_job_topic(name, &block)
|
14
15
|
topic(name) do
|
15
|
-
consumer App.config.internal.active_job.
|
16
|
+
consumer App.config.internal.active_job.consumer_class
|
17
|
+
|
18
|
+
next unless block
|
19
|
+
|
20
|
+
instance_eval(&block)
|
21
|
+
|
22
|
+
# This is handled by our custom ActiveJob consumer
|
23
|
+
# Without this, default behaviour would cause messages to skip upon shutdown as the
|
24
|
+
# offset would be committed for the last message
|
25
|
+
manual_offset_management true
|
16
26
|
end
|
17
27
|
end
|
18
28
|
end
|