-
Notifications
You must be signed in to change notification settings - Fork 339
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Add murmur2_random
support for message partitioning.
#884
Changes from 2 commits
b323c63
db7f5f4
0617573
4669c20
b2f3eed
d49e7c4
5a3374e
c52c238
f3e5078
fd89412
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,35 @@ | ||
# frozen_string_literal: true | ||
|
||
require 'digest/murmurhash' | ||
|
||
module Kafka | ||
|
||
# Java producer compatible message partitioner | ||
class Murmur2Partitioner | ||
SEED = [0x9747b28c].pack('L') | ||
|
||
# Assigns a partition number based on a partition key. If no explicit | ||
# partition key is provided, the message key will be used instead. | ||
# | ||
# If the key is nil, then a random partition is selected. Otherwise, a hash | ||
# of the key is used to deterministically find a partition. As long as the | ||
# number of partitions doesn't change, the same key will always be assigned | ||
# to the same partition. | ||
# | ||
# @param partition_count [Integer] the number of partitions in the topic. | ||
# @param message [Kafka::PendingMessage] the message that should be assigned | ||
# a partition. | ||
# @return [Integer] the partition number. | ||
def call(partition_count, message) | ||
raise ArgumentError if partition_count == 0 | ||
|
||
key = message.partition_key || message.key | ||
|
||
if key.nil? | ||
rand(partition_count) | ||
else | ||
(Digest::MurmurHash2.rawdigest(key, SEED) & 0x7fffffff) % partition_count | ||
end | ||
end | ||
end | ||
end |
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -28,6 +28,7 @@ Gem::Specification.new do |spec| | |
spec.require_paths = ["lib"] | ||
|
||
spec.add_dependency 'digest-crc' | ||
spec.add_dependency "digest-murmurhash" | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. What are the dependencies of this gem? The dependency policy for ruby-kafka is pretty strict, so for opt-in behavior the default would be to give an error message when the library cannot be detected and ask the user to install it themselves (would love optional dependencies in gems, but 🤷) If the gem is really trivial and doesn't include C code then we can add it as a hard dep, but only then. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Ah I see how the various compression codecs are used. Would a similar setup to that be ok here? The gem is fairly trivial but it's basically just C code. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. If it's C code then I think it should be opt-in, so that would be a good change 👍 There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Done, thanks! |
||
|
||
spec.add_development_dependency "bundler", ">= 1.9.5" | ||
spec.add_development_dependency "rake", "~> 10.0" | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -1,29 +1,81 @@ | ||
# frozen_string_literal: true | ||
|
||
describe Kafka::Partitioner, "#call" do | ||
let(:partitioner) { Kafka::Partitioner.new } | ||
let(:message) { double(:message, key: nil, partition_key: "yolo") } | ||
|
||
it "deterministically returns a partition number for a partition key and partition count" do | ||
partition = partitioner.call(3, message) | ||
expect(partition).to eq 0 | ||
end | ||
describe "default partitioner" do | ||
let(:partitioner) { Kafka::Partitioner.new } | ||
|
||
it "deterministically returns a partition number for a partition key and partition count" do | ||
partition = partitioner.call(3, message) | ||
expect(partition).to eq 0 | ||
end | ||
|
||
it "falls back to the message key if no partition key is available" do | ||
allow(message).to receive(:partition_key) { nil } | ||
allow(message).to receive(:key) { "hey" } | ||
|
||
it "falls back to the message key if no partition key is available" do | ||
allow(message).to receive(:partition_key) { nil } | ||
allow(message).to receive(:key) { "hey" } | ||
partition = partitioner.call(3, message) | ||
|
||
partition = partitioner.call(3, message) | ||
expect(partition).to eq 2 | ||
end | ||
|
||
expect(partition).to eq 2 | ||
it "randomly picks a partition if the key is nil" do | ||
allow(message).to receive(:key) { nil } | ||
allow(message).to receive(:partition_key) { nil } | ||
|
||
partitions = 30.times.map { partitioner.call(3, message) } | ||
|
||
expect(partitions.uniq).to contain_exactly(0, 1, 2) | ||
end | ||
end | ||
|
||
it "randomly picks a partition if the key is nil" do | ||
allow(message).to receive(:key) { nil } | ||
allow(message).to receive(:partition_key) { nil } | ||
describe "murmur2 partitioner" do | ||
let(:partitioner) { Kafka::Murmur2Partitioner.new } | ||
let(:message) { double(:message, key: nil, partition_key: "yolo") } | ||
|
||
it "deterministically returns a partition number for a partition key and partition count" do | ||
partition = partitioner.call(3, message) | ||
expect(partition).to eq 0 | ||
end | ||
|
||
it "falls back to the message key if no partition key is available" do | ||
allow(message).to receive(:partition_key) { nil } | ||
allow(message).to receive(:key) { "hey" } | ||
|
||
partition = partitioner.call(3, message) | ||
|
||
expect(partition).to eq 1 | ||
end | ||
|
||
it "randomly picks a partition if the key is nil" do | ||
allow(message).to receive(:key) { nil } | ||
allow(message).to receive(:partition_key) { nil } | ||
|
||
partitions = 30.times.map { partitioner.call(3, message) } | ||
|
||
partitions = 30.times.map { partitioner.call(3, message) } | ||
expect(partitions.uniq).to contain_exactly(0, 1, 2) | ||
end | ||
|
||
expect(partitions.uniq).to contain_exactly(0, 1, 2) | ||
it "picks a Java Kafka compatible partition" do | ||
partition_count = 100 | ||
{ | ||
# librdkafka test cases taken from tests/0048-partitioner.c | ||
"" => 0x106e08d9 % partition_count, | ||
"this is another string with more length to it perhaps" => 0x4f7703da % partition_count, | ||
"hejsan" => 0x5ec19395 % partition_count, | ||
# Java Kafka test cases taken from UtilsTest.java. | ||
# The Java tests check the result of murmur2 directly, | ||
# so have been ANDd with 0x7fffffff to work here | ||
"21" => (-973932308 & 0x7fffffff) % partition_count, | ||
"foobar" => (-790332482 & 0x7fffffff) % partition_count, | ||
"a-little-bit-long-string" => (-985981536 & 0x7fffffff) % partition_count, | ||
"a-little-bit-longer-string" => (-1486304829 & 0x7fffffff) % partition_count, | ||
"lkjh234lh9fiuh90y23oiuhsafujhadof229phr9h19h89h8" => (-58897971 & 0x7fffffff) % partition_count | ||
}.each do |key, partition| | ||
allow(message).to receive(:partition_key) { key } | ||
expect(partitioner.call(partition_count, message)).to eq partition | ||
end | ||
end | ||
end | ||
end |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I don't think we should introduce a new concept next to
Partitioner
. Instead,Partitioner
should be configured with the hash/digest algorithm, with the default beingcrc32
butmurmur2
being an option. The "random" part would just always be the behavior when no key is present.So: can we introduce a new parameter to
Partitioner#initialize
called e.g.hash_function
, which allows eithercrc32
(default) ormurmur2
. Clients that wish to usemurmur2
would need to instantiatePartitioner
with that value and pass the instance toproducer
.In Racecar and other high-level frameworks, we can add configuration keys for controlling this declaratively.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
That sounds good. I was taking inspiration from
librdkafka
but your idea will also work well with the opt-in behaviour below.Just to be clear, you would like the
Partitioner
instance to be passed in, alasasl_oauth_token_provider
, and not thehash_function
as a symbol alacompression_codec
? I'm only asking because the latter seems like a better fit here, but I don't have much experience writing gems.There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I'd prefer the former, since we're talking about varying the hash algo used by Partitioner, but not changing its fundamental strategy. Also, I prefer a minimal change in API – the high level libraries can easily make this a config key without any extra APIs.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Great, thanks for breaking it down for me. I've removed the API change and moved it inside the
Partitioner