Skip to content

Commit b4a989c

Browse files
committed
Make timeout handling consistent in the C extension
1 parent 4c8c905 commit b4a989c

File tree

3 files changed

+12
-16
lines changed

3 files changed

+12
-16
lines changed

confluent_kafka/src/Consumer.c

Lines changed: 7 additions & 11 deletions
Original file line numberDiff line numberDiff line change
@@ -596,8 +596,7 @@ static PyObject *Consumer_committed (Handle *self, PyObject *args,
596596
return NULL;
597597

598598
Py_BEGIN_ALLOW_THREADS;
599-
err = rd_kafka_committed(self->rk, c_parts,
600-
tmout >= 0 ? (int)(tmout * 1000.0f) : -1);
599+
err = rd_kafka_committed(self->rk, c_parts, cfl_timeout_ms(tmout));
601600
Py_END_ALLOW_THREADS;
602601

603602
if (err) {
@@ -793,7 +792,7 @@ static PyObject *Consumer_get_watermark_offsets (Handle *self, PyObject *args,
793792
err = rd_kafka_query_watermark_offsets(self->rk,
794793
tp->topic, tp->partition,
795794
&low, &high,
796-
tmout >= 0 ? (int)(tmout * 1000.0f) : -1);
795+
cfl_timeout_ms(tmout));
797796
Py_END_ALLOW_THREADS;
798797
}
799798

@@ -844,9 +843,8 @@ static PyObject *Consumer_offsets_for_times (Handle *self, PyObject *args,
844843
return NULL;
845844

846845
Py_BEGIN_ALLOW_THREADS;
847-
err = rd_kafka_offsets_for_times(self->rk,
848-
c_parts,
849-
tmout >= 0 ? (int)(tmout * 1000.0f) : -1);
846+
err = rd_kafka_offsets_for_times(self->rk, c_parts,
847+
cfl_timeout_ms(tmout));
850848
Py_END_ALLOW_THREADS;
851849

852850
if (err) {
@@ -884,8 +882,7 @@ static PyObject *Consumer_poll (Handle *self, PyObject *args,
884882

885883
CallState_begin(self, &cs);
886884

887-
rkm = rd_kafka_consumer_poll(self->rk, tmout >= 0 ?
888-
(int)(tmout * 1000.0f) : -1);
885+
rkm = rd_kafka_consumer_poll(self->rk, cfl_timeout_ms(tmout));
889886

890887
if (!CallState_end(self, &cs)) {
891888
if (rkm)
@@ -940,9 +937,8 @@ static PyObject *Consumer_consume (Handle *self, PyObject *args,
940937
rkmessages = malloc(num_messages * sizeof(rd_kafka_message_t *));
941938

942939
n = (Py_ssize_t)rd_kafka_consume_batch_queue(rkqu,
943-
tmout >= 0 ? (int)(tmout * 1000.0f) : -1,
944-
rkmessages,
945-
num_messages);
940+
cfl_timeout_ms(tmout),
941+
rkmessages, num_messages);
946942

947943
if (!CallState_end(self, &cs)) {
948944
for (i = 0; i < n; i++) {

confluent_kafka/src/Metadata.c

Lines changed: 3 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -343,11 +343,11 @@ list_topics (Handle *self, PyObject *args, PyObject *kwargs) {
343343
const rd_kafka_metadata_t *metadata = NULL;
344344
rd_kafka_topic_t *only_rkt = NULL;
345345
const char *topic = NULL;
346-
double timeout = -1.0f;
346+
double tmout = -1.0f;
347347
static char *kws[] = {"topic", "timeout", NULL};
348348

349349
if (!PyArg_ParseTupleAndKeywords(args, kwargs, "|zd", kws,
350-
&topic, &timeout))
350+
&topic, &tmout))
351351
return NULL;
352352

353353
if (topic != NULL) {
@@ -364,7 +364,7 @@ list_topics (Handle *self, PyObject *args, PyObject *kwargs) {
364364
CallState_begin(self, &cs);
365365

366366
err = rd_kafka_metadata(self->rk, !only_rkt, only_rkt, &metadata,
367-
timeout >= 0 ? (int)(timeout * 1000.0f) : -1);
367+
cfl_timeout_ms(tmout));
368368

369369
if (!CallState_end(self, &cs)) {
370370
/* Exception raised */

confluent_kafka/src/Producer.c

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -363,7 +363,7 @@ static PyObject *Producer_poll (Handle *self, PyObject *args,
363363
if (!PyArg_ParseTupleAndKeywords(args, kwargs, "|d", kws, &tmout))
364364
return NULL;
365365

366-
r = Producer_poll0(self, (int)(tmout * 1000));
366+
r = Producer_poll0(self, cfl_timeout_ms(tmout));
367367
if (r == -1)
368368
return NULL;
369369

@@ -385,7 +385,7 @@ static PyObject *Producer_flush (Handle *self, PyObject *args,
385385

386386
#if RD_KAFKA_VERSION >= 0x00090300
387387
CallState_begin(self, &cs);
388-
rd_kafka_flush(self->rk, tmout < 0 ? -1 : (int)(tmout * 1000));
388+
rd_kafka_flush(self->rk, cfl_timeout_ms(tmout));
389389
if (!CallState_end(self, &cs))
390390
return NULL;
391391
qlen = rd_kafka_outq_len(self->rk);

0 commit comments

Comments
 (0)
pFad - Phonifier reborn

Pfad - The Proxy pFad of © 2024 Garber Painting. All rights reserved.

Note: This service is not intended for secure transactions such as banking, social media, email, or purchasing. Use at your own risk. We assume no liability whatsoever for broken pages.


Alternative Proxies:

Alternative Proxy

pFad Proxy

pFad v3 Proxy

pFad v4 Proxy