Open Quuxplusone opened 11 months ago
@Quuxplusone thanks for sharing this. Hope will help people being aware of the changes to apply before switching to murmur2_random
, that we cannot set as default for the same reason.
Could you make murmur2_random
the default and bump up the version of librdkafka, noting this as a breaking change, warning users to explicitly choose consistent_random
if they were relying on it up til now (for the reasons listed here)?
I appreciate not wanting to surprise existing users. But for new users, it would be great if they inherited murmur2_random
by default, so they don't run into issues the moment they ever start to share a topic between Java and non-Java producers.
See https://www.confluent.io/blog/standardized-hashing-across-java-and-non-java-producers/
@benissimo we have a list of breaking changes that we want to do in a planned major release, and this is already included. We should also deprecate them before removing in next version and give time to developers to know that. In case we do a major release we should also backport fixes to previous major version until it goes out of support.
2.x was needed more than planned because of the upgrade of OSSL to 3.x but for the rest it's compatible.
This is to say that at some point it will be needed but we have to plan it and we're currently focused on these features: KIP-848, KIP-714 and KIP-951
(Reporting this as an issue mainly so that there is a canonical place it's documented, for internal reference and for posterity.)
librdkafka provides several partitioners out-of-the-box:
rd_kafka_msg_partitioner_murmur2_random
rd_kafka_msg_partitioner_fnv1a_random
rd_kafka_msg_partitioner_consistent_random
They have the following behaviors:
murmur2_random
: Ifkey
is null, pick a random partition. Otherwise, partition based onmurmur2(key, keylen)
.fnv1a_random
: Ifkey
is null, pick a random partition. Otherwise, partition based onfnv1a(key, keylen)
.consistent_random
: Ifkey
is null orkeylen
is zero, pick a random partition. Otherwise, partition based oncrc32(key, keylen)
.Notice the inconsistency between
consistent_random
(the default) andmurmur2_random
(which various Internet folks recommend switching to: 1, 2).consistent_random
will distribute zero-length keys among random partitions;murmur2_hash
will send all zero-length keys to a single partition. There are two ways this could bite someone who switches partitioning schemes:The way it bit us in real life: Suppose we're currently using
consistent_random
, and storing the Kafkakey
in astd::string
. (If there's no key — our common case — then thestd::string
is just empty). We unconditionally callrd_kafka_producev(..., RD_KAFKA_V_KEY(key.data(), key.size()), ...)
. When the key is absent,key.size() == 0
, andconsistent_random
distributes our data to random partitions. Tomorrow, we switch tomurmur2_random
— and suddenly all our keyless data ends up in the same single partition! (Partition 21 mod 30, in our case.) To restore the behavior we depended on before, we must change our calling code tord_kafka_producev(..., RD_KAFKA_V_KEY(key.empty() ? nullptr : key.data(), key.size()), ...)
.Or, vice versa: Suppose we're currently using
murmur2_random
orfnv1a_random
. We always provide a key, and sometimes the key is zero-length. (Maybe we use "customer name" as the key, and one smart-aleck customer set their name to the empty string, I dunno.) Usingmurmur2_random
, all records with a zero-length key get hashed to one consistent partition. Tomorrow, we switch to the default partitionerconsistent_random
— and suddenly all those records are distributed randomly, such that they aren't in the same partition anymore! To restore the old behavior, we must change our calling code to check for the zero-length key and map it to something non-zero-length.Because changing the behavior in either direction has the potential to break someone, I don't think there's any way for librdkafka to fix this inconsistency between how
consistent_random
andmurmur2_random
handle empty keys. I just wish we had known about it before it bit us.