Skip to content

Commit 267367c

Browse files
authored
Topic partition and Uuid additional common functions (#4621)
1 parent 91a423a commit 267367c

File tree

6 files changed

+247
-1
lines changed

6 files changed

+247
-1
lines changed

src/rdkafka.c

Lines changed: 12 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -64,6 +64,7 @@
6464
#endif
6565

6666
#include "rdtime.h"
67+
#include "rdmap.h"
6768
#include "crc32c.h"
6869
#include "rdunittest.h"
6970

@@ -5178,6 +5179,17 @@ const char *rd_kafka_Uuid_base64str(const rd_kafka_Uuid_t *uuid) {
51785179
return uuid->base64str;
51795180
}
51805181

5182+
unsigned int rd_kafka_Uuid_hash(const rd_kafka_Uuid_t *uuid) {
5183+
unsigned char bytes[16];
5184+
memcpy(bytes, &uuid->most_significant_bits, 8);
5185+
memcpy(&bytes[8], &uuid->least_significant_bits, 8);
5186+
return rd_bytes_hash(bytes, 16);
5187+
}
5188+
5189+
unsigned int rd_kafka_Uuid_map_hash(const void *key) {
5190+
return rd_kafka_Uuid_hash(key);
5191+
}
5192+
51815193
int64_t rd_kafka_Uuid_least_significant_bits(const rd_kafka_Uuid_t *uuid) {
51825194
return uuid->least_significant_bits;
51835195
}

src/rdkafka_partition.c

Lines changed: 180 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -3087,6 +3087,12 @@ int rd_kafka_topic_partition_by_id_cmp(const void *_a, const void *_b) {
30873087
return are_topic_ids_different || RD_CMP(a->partition, b->partition);
30883088
}
30893089

3090+
static int rd_kafka_topic_partition_by_id_cmp_opaque(const void *_a,
3091+
const void *_b,
3092+
void *opaque) {
3093+
return rd_kafka_topic_partition_by_id_cmp(_a, _b);
3094+
}
3095+
30903096
/** @brief Compare only the topic */
30913097
int rd_kafka_topic_partition_cmp_topic(const void *_a, const void *_b) {
30923098
const rd_kafka_topic_partition_t *a = _a;
@@ -3100,13 +3106,22 @@ static int rd_kafka_topic_partition_cmp_opaque(const void *_a,
31003106
return rd_kafka_topic_partition_cmp(_a, _b);
31013107
}
31023108

3103-
/** @returns a hash of the topic and partition */
3109+
/** @returns a hash of the topic name and partition */
31043110
unsigned int rd_kafka_topic_partition_hash(const void *_a) {
31053111
const rd_kafka_topic_partition_t *a = _a;
31063112
int r = 31 * 17 + a->partition;
31073113
return 31 * r + rd_string_hash(a->topic, -1);
31083114
}
31093115

3116+
/** @returns a hash of the topic id and partition */
3117+
unsigned int rd_kafka_topic_partition_hash_by_id(const void *_a) {
3118+
const rd_kafka_topic_partition_t *a = _a;
3119+
const rd_kafka_Uuid_t topic_id =
3120+
rd_kafka_topic_partition_get_topic_id(a);
3121+
int r = 31 * 17 + a->partition;
3122+
return 31 * r + rd_kafka_Uuid_hash(&topic_id);
3123+
}
3124+
31103125

31113126

31123127
/**
@@ -3313,6 +3328,12 @@ void rd_kafka_topic_partition_list_sort_by_topic(
33133328
rktparlist, rd_kafka_topic_partition_cmp_opaque, NULL);
33143329
}
33153330

3331+
void rd_kafka_topic_partition_list_sort_by_topic_id(
3332+
rd_kafka_topic_partition_list_t *rktparlist) {
3333+
rd_kafka_topic_partition_list_sort(
3334+
rktparlist, rd_kafka_topic_partition_by_id_cmp_opaque, NULL);
3335+
}
3336+
33163337
rd_kafka_resp_err_t rd_kafka_topic_partition_list_set_offset(
33173338
rd_kafka_topic_partition_list_t *rktparlist,
33183339
const char *topic,
@@ -4479,3 +4500,161 @@ const char *rd_kafka_fetch_pos2str(const rd_kafka_fetch_pos_t fetchpos) {
44794500

44804501
return ret[idx];
44814502
}
4503+
4504+
typedef RD_MAP_TYPE(const rd_kafka_topic_partition_t *,
4505+
void *) map_toppar_void_t;
4506+
4507+
/**
4508+
* @brief Calculates \p a ∩ \p b using \p cmp and \p hash .
4509+
* Ordered following \p a order. Elements are copied from \p a.
4510+
*/
4511+
static rd_kafka_topic_partition_list_t *
4512+
rd_kafka_topic_partition_list_intersection0(
4513+
rd_kafka_topic_partition_list_t *a,
4514+
rd_kafka_topic_partition_list_t *b,
4515+
int(cmp)(const void *_a, const void *_b),
4516+
unsigned int(hash)(const void *_a)) {
4517+
rd_kafka_topic_partition_t *rktpar;
4518+
rd_kafka_topic_partition_list_t *ret =
4519+
rd_kafka_topic_partition_list_new(a->cnt < b->cnt ? a->cnt
4520+
: b->cnt);
4521+
map_toppar_void_t b_map =
4522+
RD_MAP_INITIALIZER(b->cnt, cmp, hash, NULL, NULL);
4523+
RD_KAFKA_TPLIST_FOREACH(rktpar, b) {
4524+
RD_MAP_SET(&b_map, rktpar, rktpar);
4525+
}
4526+
RD_KAFKA_TPLIST_FOREACH(rktpar, a) {
4527+
if ((RD_MAP_GET(&b_map, rktpar) != NULL) == 1) {
4528+
rd_kafka_topic_partition_list_add_copy(ret, rktpar);
4529+
}
4530+
}
4531+
RD_MAP_DESTROY(&b_map);
4532+
return ret;
4533+
}
4534+
4535+
/**
4536+
* @brief Calculates \p a - \p b using \p cmp and \p hash .
4537+
* Ordered following \p a order. Elements are copied from \p a.
4538+
*/
4539+
static rd_kafka_topic_partition_list_t *
4540+
rd_kafka_topic_partition_list_difference0(rd_kafka_topic_partition_list_t *a,
4541+
rd_kafka_topic_partition_list_t *b,
4542+
int(cmp)(const void *_a,
4543+
const void *_b),
4544+
unsigned int(hash)(const void *_a)) {
4545+
rd_kafka_topic_partition_t *rktpar;
4546+
rd_kafka_topic_partition_list_t *ret =
4547+
rd_kafka_topic_partition_list_new(a->cnt);
4548+
map_toppar_void_t b_map =
4549+
RD_MAP_INITIALIZER(b->cnt, cmp, hash, NULL, NULL);
4550+
RD_KAFKA_TPLIST_FOREACH(rktpar, b) {
4551+
RD_MAP_SET(&b_map, rktpar, rktpar);
4552+
}
4553+
RD_KAFKA_TPLIST_FOREACH(rktpar, a) {
4554+
if ((RD_MAP_GET(&b_map, rktpar) != NULL) == 0) {
4555+
rd_kafka_topic_partition_list_add_copy(ret, rktpar);
4556+
}
4557+
}
4558+
RD_MAP_DESTROY(&b_map);
4559+
return ret;
4560+
}
4561+
4562+
/**
4563+
* @brief Calculates \p a ∪ \p b using \p cmp and \p hash .
4564+
* Ordered following \p a order for elements in \p a
4565+
* and \p b order for elements only in \p b.
4566+
* Elements are copied the same way.
4567+
*/
4568+
static rd_kafka_topic_partition_list_t *
4569+
rd_kafka_topic_partition_list_union0(rd_kafka_topic_partition_list_t *a,
4570+
rd_kafka_topic_partition_list_t *b,
4571+
int(cmp)(const void *_a, const void *_b),
4572+
unsigned int(hash)(const void *_a)) {
4573+
4574+
rd_kafka_topic_partition_list_t *b_minus_a =
4575+
rd_kafka_topic_partition_list_difference0(b, a, cmp, hash);
4576+
rd_kafka_topic_partition_list_t *ret =
4577+
rd_kafka_topic_partition_list_new(a->cnt + b_minus_a->cnt);
4578+
4579+
rd_kafka_topic_partition_list_add_list(ret, a);
4580+
rd_kafka_topic_partition_list_add_list(ret, b_minus_a);
4581+
4582+
rd_kafka_topic_partition_list_destroy(b_minus_a);
4583+
return ret;
4584+
}
4585+
4586+
/**
4587+
* @brief Calculates \p a ∩ \p b using topic name and partition id.
4588+
* Ordered following \p a order. Elements are copied from \p a.
4589+
*/
4590+
rd_kafka_topic_partition_list_t *
4591+
rd_kafka_topic_partition_list_intersection_by_name(
4592+
rd_kafka_topic_partition_list_t *a,
4593+
rd_kafka_topic_partition_list_t *b) {
4594+
return rd_kafka_topic_partition_list_intersection0(
4595+
a, b, rd_kafka_topic_partition_cmp, rd_kafka_topic_partition_hash);
4596+
}
4597+
4598+
/**
4599+
* @brief Calculates \p a - \p b using topic name and partition id.
4600+
* Ordered following \p a order. Elements are copied from \p a.
4601+
*/
4602+
rd_kafka_topic_partition_list_t *
4603+
rd_kafka_topic_partition_list_difference_by_name(
4604+
rd_kafka_topic_partition_list_t *a,
4605+
rd_kafka_topic_partition_list_t *b) {
4606+
return rd_kafka_topic_partition_list_difference0(
4607+
a, b, rd_kafka_topic_partition_cmp, rd_kafka_topic_partition_hash);
4608+
}
4609+
4610+
/**
4611+
* @brief Calculates \p a ∪ \p b using topic name and partition id.
4612+
* Ordered following \p a order for elements in \p a
4613+
* and \p b order for elements only in \p b.
4614+
* Elements are copied the same way.
4615+
*/
4616+
rd_kafka_topic_partition_list_t *rd_kafka_topic_partition_list_union_by_name(
4617+
rd_kafka_topic_partition_list_t *a,
4618+
rd_kafka_topic_partition_list_t *b) {
4619+
return rd_kafka_topic_partition_list_union0(
4620+
a, b, rd_kafka_topic_partition_cmp, rd_kafka_topic_partition_hash);
4621+
}
4622+
4623+
/**
4624+
* @brief Calculates \p a ∩ \p b using topic id and partition id.
4625+
* Ordered following \p a order. Elements are copied from \p a.
4626+
*/
4627+
rd_kafka_topic_partition_list_t *
4628+
rd_kafka_topic_partition_list_intersection_by_id(
4629+
rd_kafka_topic_partition_list_t *a,
4630+
rd_kafka_topic_partition_list_t *b) {
4631+
return rd_kafka_topic_partition_list_intersection0(
4632+
a, b, rd_kafka_topic_partition_by_id_cmp,
4633+
rd_kafka_topic_partition_hash_by_id);
4634+
}
4635+
4636+
/**
4637+
* @brief Calculates \p a - \p b using topic id and partition id.
4638+
* Ordered following \p a order. Elements are copied from \p a.
4639+
*/
4640+
rd_kafka_topic_partition_list_t *rd_kafka_topic_partition_list_difference_by_id(
4641+
rd_kafka_topic_partition_list_t *a,
4642+
rd_kafka_topic_partition_list_t *b) {
4643+
return rd_kafka_topic_partition_list_difference0(
4644+
a, b, rd_kafka_topic_partition_by_id_cmp,
4645+
rd_kafka_topic_partition_hash_by_id);
4646+
}
4647+
4648+
/**
4649+
* @brief Calculates \p a ∪ \p b using topic id and partition id.
4650+
* Ordered following \p a order for elements in \p a
4651+
* and \p b order for elements only in \p b.
4652+
* Elements are copied the same way.
4653+
*/
4654+
rd_kafka_topic_partition_list_t *
4655+
rd_kafka_topic_partition_list_union_by_id(rd_kafka_topic_partition_list_t *a,
4656+
rd_kafka_topic_partition_list_t *b) {
4657+
return rd_kafka_topic_partition_list_union0(
4658+
a, b, rd_kafka_topic_partition_by_id_cmp,
4659+
rd_kafka_topic_partition_hash_by_id);
4660+
}

src/rdkafka_partition.h

Lines changed: 30 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -773,6 +773,9 @@ rd_kafka_topic_partition_t *rd_kafka_topic_partition_list_find_topic(
773773
void rd_kafka_topic_partition_list_sort_by_topic(
774774
rd_kafka_topic_partition_list_t *rktparlist);
775775

776+
void rd_kafka_topic_partition_list_sort_by_topic_id(
777+
rd_kafka_topic_partition_list_t *rktparlist);
778+
776779
void rd_kafka_topic_partition_list_reset_offsets(
777780
rd_kafka_topic_partition_list_t *rktparlist,
778781
int64_t offset);
@@ -1122,4 +1125,31 @@ static RD_UNUSED RD_INLINE void rd_kafka_toppar_set_offset_validation_position(
11221125
rktp->rktp_offset_validation_pos = offset_validation_pos;
11231126
}
11241127

1128+
rd_kafka_topic_partition_list_t *
1129+
rd_kafka_topic_partition_list_intersection_by_name(
1130+
rd_kafka_topic_partition_list_t *a,
1131+
rd_kafka_topic_partition_list_t *b);
1132+
1133+
rd_kafka_topic_partition_list_t *
1134+
rd_kafka_topic_partition_list_difference_by_name(
1135+
rd_kafka_topic_partition_list_t *a,
1136+
rd_kafka_topic_partition_list_t *b);
1137+
1138+
rd_kafka_topic_partition_list_t *
1139+
rd_kafka_topic_partition_list_union_by_name(rd_kafka_topic_partition_list_t *a,
1140+
rd_kafka_topic_partition_list_t *b);
1141+
1142+
rd_kafka_topic_partition_list_t *
1143+
rd_kafka_topic_partition_list_intersection_by_id(
1144+
rd_kafka_topic_partition_list_t *a,
1145+
rd_kafka_topic_partition_list_t *b);
1146+
1147+
rd_kafka_topic_partition_list_t *rd_kafka_topic_partition_list_difference_by_id(
1148+
rd_kafka_topic_partition_list_t *a,
1149+
rd_kafka_topic_partition_list_t *b);
1150+
1151+
rd_kafka_topic_partition_list_t *
1152+
rd_kafka_topic_partition_list_union_by_id(rd_kafka_topic_partition_list_t *a,
1153+
rd_kafka_topic_partition_list_t *b);
1154+
11251155
#endif /* _RDKAFKA_PARTITION_H_ */

src/rdkafka_proto.h

Lines changed: 4 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -603,6 +603,10 @@ rd_kafka_Uuid_t rd_kafka_Uuid_random();
603603

604604
const char *rd_kafka_Uuid_str(const rd_kafka_Uuid_t *uuid);
605605

606+
unsigned int rd_kafka_Uuid_hash(const rd_kafka_Uuid_t *uuid);
607+
608+
unsigned int rd_kafka_Uuid_map_hash(const void *key);
609+
606610
/**
607611
* @name Producer ID and Epoch for the Idempotent Producer
608612
* @{

src/rdmap.c

Lines changed: 16 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -2,6 +2,7 @@
22
* librdkafka - The Apache Kafka C/C++ library
33
*
44
* Copyright (c) 2020-2022, Magnus Edenhill
5+
* 2023, Confluent Inc.
56
* All rights reserved.
67
*
78
* Redistribution and use in source and binary forms, with or without
@@ -237,6 +238,21 @@ unsigned int rd_map_str_hash(const void *key) {
237238
}
238239

239240

241+
/**
242+
* @returns a djb2 hash of \p bytes.
243+
*
244+
* @param len \p bytes will be hashed up to \p len.
245+
*/
246+
unsigned int rd_bytes_hash(unsigned char *bytes, size_t len) {
247+
unsigned int hash = 5381;
248+
size_t i;
249+
250+
for (i = 0; i < len; i++)
251+
hash = ((hash << 5) + hash) + bytes[i];
252+
253+
return hash;
254+
}
255+
240256

241257
/**
242258
* @name Unit tests

src/rdmap.h

Lines changed: 5 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -2,6 +2,7 @@
22
* librdkafka - The Apache Kafka C/C++ library
33
*
44
* Copyright (c) 2020-2022, Magnus Edenhill
5+
* 2023, Confluent Inc.
56
* All rights reserved.
67
*
78
* Redistribution and use in source and binary forms, with or without
@@ -249,6 +250,10 @@ int rd_map_str_cmp(const void *a, const void *b);
249250
*/
250251
unsigned int rd_map_str_hash(const void *a);
251252

253+
/**
254+
* @brief Bytes hash function (djb2).
255+
*/
256+
unsigned int rd_bytes_hash(unsigned char *bytes, size_t len);
252257

253258

254259
/**

0 commit comments

Comments
 (0)