5757#include "rdkafka_interceptor.h"
5858#include "rdkafka_idempotence.h"
5959#include "rdkafka_sasl_oauthbearer.h"
60+ #include "rdmurmur2.h"
6061#if WITH_OAUTHBEARER_OIDC
6162#include "rdkafka_sasl_oauthbearer_oidc.h"
6263#endif
8283#endif
8384
8485
85- static once_flag rd_kafka_global_init_once = ONCE_FLAG_INIT ;
86- static once_flag rd_kafka_global_srand_once = ONCE_FLAG_INIT ;
86+ static once_flag rd_kafka_global_init_once = ONCE_FLAG_INIT ;
87+ #ifdef _WIN32
88+ /* On Windows srand needs to be called on each thread. */
89+ static RD_TLS once_flag rd_kafka_srand_once = ONCE_FLAG_INIT ;
90+ #else
91+ static once_flag rd_kafka_srand_once = ONCE_FLAG_INIT ;
92+ #endif
93+
8794
8895/**
8996 * @brief Global counter+lock for all active librdkafka instances
@@ -130,6 +137,22 @@ void rd_kafka_set_thread_name(const char *fmt, ...) {
130137 */
131138static char RD_TLS rd_kafka_thread_sysname [16 ] = "app" ;
132139
140+ /**
141+ * @brief Seed the PRNG with current microseconds and thread ID.
142+ */
143+ static void rd_kafka_srand (void ) {
144+ unsigned int seed = 0 ;
145+ struct timeval tv ;
146+ rd_gettimeofday (& tv , NULL );
147+ seed = (unsigned int )(tv .tv_usec );
148+ seed ^= thrd_current_id ();
149+
150+ /* Apply the murmur2 hash to distribute entropy to
151+ * the whole seed. */
152+ seed = (unsigned int )rd_murmur2 (& seed , sizeof (seed ));
153+ srand (seed );
154+ }
155+
133156void rd_kafka_set_thread_sysname (const char * fmt , ...) {
134157 va_list ap ;
135158
@@ -141,6 +164,30 @@ void rd_kafka_set_thread_sysname(const char *fmt, ...) {
141164 thrd_setname (rd_kafka_thread_sysname );
142165}
143166
167+ /**
168+ * @brief Seed the PRNG for the current thread or for the whole process.
169+ * Depending on the platform implementation of srand() the seed can
170+ * be a thread local or global one. In case it's thread local we
171+ * need to call it on each thread.
172+ *
173+ * @param rk Client instance.
174+ * @param internal_thread If true, seed the PRNG if
175+ * it's required per-thread.
176+ */
177+ void rd_kafka_thread_srand (rd_kafka_t * rk , rd_bool_t internal_thread ) {
178+ #ifdef _WIN32
179+ rd_bool_t required_per_thread = rd_true ;
180+ #else
181+ rd_bool_t required_per_thread = rd_false ;
182+ #endif
183+ if ((required_per_thread &&
184+ (rk -> rk_conf .enable_random_seed || internal_thread )) ||
185+ (!required_per_thread && rk -> rk_conf .enable_random_seed &&
186+ !internal_thread )) {
187+ call_once (& rd_kafka_srand_once , rd_kafka_srand );
188+ }
189+ }
190+
144191static void rd_kafka_global_init0 (void ) {
145192 cJSON_Hooks json_hooks = {.malloc_fn = rd_malloc , .free_fn = rd_free };
146193
@@ -171,18 +218,6 @@ void rd_kafka_global_init(void) {
171218}
172219
173220
174- /**
175- * @brief Seed the PRNG with current_time.milliseconds
176- */
177- static void rd_kafka_global_srand (void ) {
178- struct timeval tv ;
179-
180- rd_gettimeofday (& tv , NULL );
181-
182- srand ((unsigned int )(tv .tv_usec / 1000 ));
183- }
184-
185-
186221/**
187222 * @returns the current number of active librdkafka instances
188223 */
@@ -2218,6 +2253,7 @@ static int rd_kafka_thread_main(void *arg) {
22182253
22192254 rd_kafka_set_thread_name ("main" );
22202255 rd_kafka_set_thread_sysname ("rdk:main" );
2256+ rd_kafka_thread_srand (rk , rd_true /* we're in an internal thread */ );
22212257
22222258 rd_kafka_interceptors_on_thread_start (rk , RD_KAFKA_THREAD_MAIN );
22232259
@@ -2367,10 +2403,7 @@ rd_kafka_t *rd_kafka_new(rd_kafka_type_t type,
23672403 * freed from rd_kafka_destroy_internal()
23682404 * as the rk itself is destroyed. */
23692405
2370- /* Seed PRNG, don't bother about HAVE_RAND_R, since it is pretty cheap.
2371- */
2372- if (rk -> rk_conf .enable_random_seed )
2373- call_once (& rd_kafka_global_srand_once , rd_kafka_global_srand );
2406+ rd_kafka_thread_srand (rk , rd_false /* we're on an app thread */ );
23742407
23752408 /* Call on_new() interceptors */
23762409 rd_kafka_interceptors_on_new (rk , & rk -> rk_conf );
0 commit comments