From c5bdee6cebe29c5e3b31388e2cf0959cc6c23bd0 Mon Sep 17 00:00:00 2001 From: Kaushik Raina Date: Tue, 12 Aug 2025 18:04:54 +0530 Subject: [PATCH 1/7] Fix error propagation rule for Python's C API --- src/confluent_kafka/src/Consumer.c | 1 + src/confluent_kafka/src/Producer.c | 1 + src/confluent_kafka/src/confluent_kafka.c | 22 +++++++++++++- src/confluent_kafka/src/confluent_kafka.h | 3 ++ tests/test_Consumer.py | 35 +++++++++++++++++++++++ tests/test_Producer.py | 33 +++++++++++++++++++++ 6 files changed, 94 insertions(+), 1 deletion(-) diff --git a/src/confluent_kafka/src/Consumer.c b/src/confluent_kafka/src/Consumer.c index 7376c6d84..047a4a36f 100644 --- a/src/confluent_kafka/src/Consumer.c +++ b/src/confluent_kafka/src/Consumer.c @@ -1575,6 +1575,7 @@ static void Consumer_rebalance_cb (rd_kafka_t *rk, rd_kafka_resp_err_t err, if (result) Py_DECREF(result); else { + PyErr_Fetch(&cs->exception_type, &cs->exception_value, &cs->exception_traceback); CallState_crash(cs); rd_kafka_yield(rk); } diff --git a/src/confluent_kafka/src/Producer.c b/src/confluent_kafka/src/Producer.c index b6a51f510..3f096a1b5 100644 --- a/src/confluent_kafka/src/Producer.c +++ b/src/confluent_kafka/src/Producer.c @@ -163,6 +163,7 @@ static void dr_msg_cb (rd_kafka_t *rk, const rd_kafka_message_t *rkm, if (result) Py_DECREF(result); else { + PyErr_Fetch(&cs->exception_type, &cs->exception_value, &cs->exception_traceback); CallState_crash(cs); rd_kafka_yield(rk); } diff --git a/src/confluent_kafka/src/confluent_kafka.c b/src/confluent_kafka/src/confluent_kafka.c index bbc335d46..97222b9fe 100644 --- a/src/confluent_kafka/src/confluent_kafka.c +++ b/src/confluent_kafka/src/confluent_kafka.c @@ -1755,6 +1755,8 @@ static void error_cb (rd_kafka_t *rk, int err, const char *reason, void *opaque) if (result) Py_DECREF(result); else { + + PyErr_Fetch(&cs->exception_type, &cs->exception_value, &cs->exception_traceback); crash: CallState_crash(cs); rd_kafka_yield(h->rk); @@ -1810,6 +1812,8 @@ static void throttle_cb (rd_kafka_t *rk, const char *broker_name, int32_t broker goto done; } + PyErr_Fetch(&cs->exception_type, &cs->exception_value, &cs->exception_traceback); + /** * Stop callback dispatcher, return err to application * fall-through to unlock GIL @@ -1839,6 +1843,7 @@ static int stats_cb(rd_kafka_t *rk, char *json, size_t json_len, void *opaque) { if (result) Py_DECREF(result); else { + PyErr_Fetch(&cs->exception_type, &cs->exception_value, &cs->exception_traceback); CallState_crash(cs); rd_kafka_yield(h->rk); } @@ -1874,6 +1879,7 @@ static void log_cb (const rd_kafka_t *rk, int level, if (result) Py_DECREF(result); else { + PyErr_Fetch(&cs->exception_type, &cs->exception_value, &cs->exception_traceback); CallState_crash(cs); rd_kafka_yield(h->rk); } @@ -2572,6 +2578,9 @@ void CallState_begin (Handle *h, CallState *cs) { cs->thread_state = PyEval_SaveThread(); assert(cs->thread_state != NULL); cs->crashed = 0; + cs->exception_type = NULL; + cs->exception_value = NULL; + cs->exception_traceback = NULL; #ifdef WITH_PY_TSS PyThread_tss_set(&h->tlskey, cs); #else @@ -2592,8 +2601,19 @@ int CallState_end (Handle *h, CallState *cs) { PyEval_RestoreThread(cs->thread_state); - if (PyErr_CheckSignals() == -1 || cs->crashed) + if (PyErr_CheckSignals() == -1) + return 0; + + if (cs->crashed) { + /* Restore the saved exception if we have one */ + if (cs->exception_type) { + PyErr_Restore(cs->exception_type, cs->exception_value, cs->exception_traceback); + cs->exception_type = NULL; + cs->exception_value = NULL; + cs->exception_traceback = NULL; + } return 0; + } return 1; } diff --git a/src/confluent_kafka/src/confluent_kafka.h b/src/confluent_kafka/src/confluent_kafka.h index ac7474c9e..e4eced986 100644 --- a/src/confluent_kafka/src/confluent_kafka.h +++ b/src/confluent_kafka/src/confluent_kafka.h @@ -275,6 +275,9 @@ int Handle_traverse (Handle *h, visitproc visit, void *arg); typedef struct { PyThreadState *thread_state; int crashed; /* Callback crashed */ + PyObject *exception_type; /* Stored exception type */ + PyObject *exception_value; /* Stored exception value */ + PyObject *exception_traceback; /* Stored exception traceback */ } CallState; /** diff --git a/tests/test_Consumer.py b/tests/test_Consumer.py index bfe03dffc..66306ffe9 100644 --- a/tests/test_Consumer.py +++ b/tests/test_Consumer.py @@ -324,3 +324,38 @@ def test_consumer_without_groupid(): with pytest.raises(ValueError) as ex: TestConsumer({'bootstrap.servers': "mybroker:9092"}) assert ex.match('group.id must be set') + + +def test_callback_exception_no_system_error(): + + exception_raised = [] + + def error_cb_that_raises(error): + """Error callback that raises an exception""" + exception_raised.append(error) + raise RuntimeError("Test exception from error_cb") + + # Create consumer with error callback that raises exception + consumer = TestConsumer({ + 'group.id': 'test-callback-systemerror-fix', + 'bootstrap.servers': 'nonexistent-broker:9092', # Will trigger error + 'socket.timeout.ms': 100, + 'session.timeout.ms': 1000, + 'error_cb': error_cb_that_raises + }) + + consumer.subscribe(['test-topic']) + + # This should trigger the error callback due to connection failure + # Before fix: Would get RuntimeError + SystemError + # After fix: Should only get RuntimeError (no SystemError) + with pytest.raises(RuntimeError) as exc_info: + consumer.consume(timeout=0.1) + + # Verify we got the expected exception message + assert "Test exception from error_cb" in str(exc_info.value) + + # Verify the error callback was actually called + assert len(exception_raised) > 0 + + consumer.close() diff --git a/tests/test_Producer.py b/tests/test_Producer.py index 0f0d69e1d..4cd8e4401 100644 --- a/tests/test_Producer.py +++ b/tests/test_Producer.py @@ -283,3 +283,36 @@ def test_producer_bool_value(): p = Producer({}) assert bool(p) + + +def test_callback_exception_no_system_error(): + delivery_reports = [] + + def delivery_cb_that_raises(err, msg): + """Delivery report callback that raises an exception""" + delivery_reports.append((err, msg)) + raise RuntimeError("Test exception from delivery_cb") + + producer = Producer({ + 'bootstrap.servers': 'nonexistent-broker:9092', # Will cause delivery failures + 'socket.timeout.ms': 100, + 'message.timeout.ms': 1000, + 'on_delivery': delivery_cb_that_raises + }) + + # Produce a message - this will trigger delivery report callback when it fails + producer.produce('test-topic', value='test-message') + + # Flush to ensure delivery reports are processed + # Before fix: Would get RuntimeError + SystemError + # After fix: Should only get RuntimeError (no SystemError) + with pytest.raises(RuntimeError) as exc_info: + producer.flush(timeout=1.0) + + # Verify we got an exception from our callback + assert "Test exception from delivery_cb" in str(exc_info.value) + + # Verify the delivery callback was actually called + assert len(delivery_reports) > 0 + + producer.close() From cb71833c7f2d1fc2f209b2802b253f444763ccab Mon Sep 17 00:00:00 2001 From: Kaushik Raina Date: Wed, 13 Aug 2025 18:39:53 +0530 Subject: [PATCH 2/7] Fix CI --- src/confluent_kafka/src/Consumer.c | 1 + src/confluent_kafka/src/Producer.c | 1 + tests/test_Producer.py | 6 ++---- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/src/confluent_kafka/src/Consumer.c b/src/confluent_kafka/src/Consumer.c index 047a4a36f..cdfc49015 100644 --- a/src/confluent_kafka/src/Consumer.c +++ b/src/confluent_kafka/src/Consumer.c @@ -1575,6 +1575,7 @@ static void Consumer_rebalance_cb (rd_kafka_t *rk, rd_kafka_resp_err_t err, if (result) Py_DECREF(result); else { + PyErr_Fetch(&cs->exception_type, &cs->exception_value, &cs->exception_traceback); CallState_crash(cs); rd_kafka_yield(rk); diff --git a/src/confluent_kafka/src/Producer.c b/src/confluent_kafka/src/Producer.c index 3f096a1b5..ba6e6e72c 100644 --- a/src/confluent_kafka/src/Producer.c +++ b/src/confluent_kafka/src/Producer.c @@ -163,6 +163,7 @@ static void dr_msg_cb (rd_kafka_t *rk, const rd_kafka_message_t *rkm, if (result) Py_DECREF(result); else { + PyErr_Fetch(&cs->exception_type, &cs->exception_value, &cs->exception_traceback); CallState_crash(cs); rd_kafka_yield(rk); diff --git a/tests/test_Producer.py b/tests/test_Producer.py index 4cd8e4401..045b9dce7 100644 --- a/tests/test_Producer.py +++ b/tests/test_Producer.py @@ -296,7 +296,7 @@ def delivery_cb_that_raises(err, msg): producer = Producer({ 'bootstrap.servers': 'nonexistent-broker:9092', # Will cause delivery failures 'socket.timeout.ms': 100, - 'message.timeout.ms': 1000, + 'message.timeout.ms': 10, # Very short timeout to trigger delivery failure quickly 'on_delivery': delivery_cb_that_raises }) @@ -307,12 +307,10 @@ def delivery_cb_that_raises(err, msg): # Before fix: Would get RuntimeError + SystemError # After fix: Should only get RuntimeError (no SystemError) with pytest.raises(RuntimeError) as exc_info: - producer.flush(timeout=1.0) + producer.flush(timeout=2.0) # Longer timeout to ensure delivery callback fires # Verify we got an exception from our callback assert "Test exception from delivery_cb" in str(exc_info.value) # Verify the delivery callback was actually called assert len(delivery_reports) > 0 - - producer.close() From 42714b5cbca06cf90eddb02aa49ab0752582af24 Mon Sep 17 00:00:00 2001 From: Kaushik Raina Date: Wed, 13 Aug 2025 19:00:20 +0530 Subject: [PATCH 3/7] Fix CI --- tests/test_Consumer.py | 14 +++++++------- tests/test_Producer.py | 14 +++++++------- 2 files changed, 14 insertions(+), 14 deletions(-) diff --git a/tests/test_Consumer.py b/tests/test_Consumer.py index 66306ffe9..22ff939dd 100644 --- a/tests/test_Consumer.py +++ b/tests/test_Consumer.py @@ -329,12 +329,12 @@ def test_consumer_without_groupid(): def test_callback_exception_no_system_error(): exception_raised = [] - + def error_cb_that_raises(error): """Error callback that raises an exception""" exception_raised.append(error) raise RuntimeError("Test exception from error_cb") - + # Create consumer with error callback that raises exception consumer = TestConsumer({ 'group.id': 'test-callback-systemerror-fix', @@ -343,19 +343,19 @@ def error_cb_that_raises(error): 'session.timeout.ms': 1000, 'error_cb': error_cb_that_raises }) - + consumer.subscribe(['test-topic']) - + # This should trigger the error callback due to connection failure # Before fix: Would get RuntimeError + SystemError # After fix: Should only get RuntimeError (no SystemError) with pytest.raises(RuntimeError) as exc_info: consumer.consume(timeout=0.1) - + # Verify we got the expected exception message assert "Test exception from error_cb" in str(exc_info.value) - + # Verify the error callback was actually called assert len(exception_raised) > 0 - + consumer.close() diff --git a/tests/test_Producer.py b/tests/test_Producer.py index 045b9dce7..7a039456d 100644 --- a/tests/test_Producer.py +++ b/tests/test_Producer.py @@ -287,30 +287,30 @@ def test_producer_bool_value(): def test_callback_exception_no_system_error(): delivery_reports = [] - + def delivery_cb_that_raises(err, msg): """Delivery report callback that raises an exception""" delivery_reports.append((err, msg)) raise RuntimeError("Test exception from delivery_cb") - + producer = Producer({ 'bootstrap.servers': 'nonexistent-broker:9092', # Will cause delivery failures 'socket.timeout.ms': 100, 'message.timeout.ms': 10, # Very short timeout to trigger delivery failure quickly 'on_delivery': delivery_cb_that_raises }) - + # Produce a message - this will trigger delivery report callback when it fails producer.produce('test-topic', value='test-message') - + # Flush to ensure delivery reports are processed - # Before fix: Would get RuntimeError + SystemError + # Before fix: Would get RuntimeError + SystemError # After fix: Should only get RuntimeError (no SystemError) with pytest.raises(RuntimeError) as exc_info: producer.flush(timeout=2.0) # Longer timeout to ensure delivery callback fires - + # Verify we got an exception from our callback assert "Test exception from delivery_cb" in str(exc_info.value) - + # Verify the delivery callback was actually called assert len(delivery_reports) > 0 From 5f29f1213e45524b9529451e5900bb0393966947 Mon Sep 17 00:00:00 2001 From: Kaushik Raina Date: Sat, 16 Aug 2025 12:50:34 +0530 Subject: [PATCH 4/7] Address PR review feedback - Add issue number #865 to test comments for better traceability - Move PyErr_Fetch into else condition for consistency in confluent_kafka.c - Add CHANGELOG entry documenting the fix for issue #865 --- CHANGELOG.md | 3 ++ src/confluent_kafka/src/confluent_kafka.c | 4 +- src/confluent_kafka/src/confluent_kafka.h | 49 +++++++++++++++++++++++ tests/test_Consumer.py | 2 +- tests/test_Producer.py | 2 +- 5 files changed, 56 insertions(+), 4 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index b5f8c123b..87fc15f2e 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -4,6 +4,9 @@ v2.11.0 is a feature release with the following enhancements: +### Fixes +- Fix error propagation rule for Python's C API to prevent SystemError when callbacks raise exceptions (#865) + confluent-kafka-python v2.11.0 is based on librdkafka v2.11.0, see the [librdkafka release notes](https://github.com/confluentinc/librdkafka/releases/tag/v2.11.0) for a complete list of changes, enhancements, fixes and upgrade considerations. diff --git a/src/confluent_kafka/src/confluent_kafka.c b/src/confluent_kafka/src/confluent_kafka.c index 97222b9fe..4db46ab37 100644 --- a/src/confluent_kafka/src/confluent_kafka.c +++ b/src/confluent_kafka/src/confluent_kafka.c @@ -1810,10 +1810,10 @@ static void throttle_cb (rd_kafka_t *rk, const char *broker_name, int32_t broker /* throttle_cb executed successfully */ Py_DECREF(result); goto done; + } else { + PyErr_Fetch(&cs->exception_type, &cs->exception_value, &cs->exception_traceback); } - PyErr_Fetch(&cs->exception_type, &cs->exception_value, &cs->exception_traceback); - /** * Stop callback dispatcher, return err to application * fall-through to unlock GIL diff --git a/src/confluent_kafka/src/confluent_kafka.h b/src/confluent_kafka/src/confluent_kafka.h index e4eced986..6fcb2bfb7 100644 --- a/src/confluent_kafka/src/confluent_kafka.h +++ b/src/confluent_kafka/src/confluent_kafka.h @@ -280,6 +280,55 @@ typedef struct { PyObject *exception_traceback; /* Stored exception traceback */ } CallState; +/** + * @brief Compatibility layer for Python exception handling API changes. + * PyErr_Fetch was deprecated in Python 3.12 in favor of PyErr_GetRaisedException. + */ +#if PY_VERSION_HEX >= 0x030c0000 +/* Python 3.12+ - use new API */ +static inline void +CallState_fetch_exception(CallState *cs) { + PyObject *exc = PyErr_GetRaisedException(); + if (exc) { + cs->exception_type = (PyObject *)Py_TYPE(exc); + Py_INCREF(cs->exception_type); + cs->exception_value = exc; + cs->exception_traceback = PyException_GetTraceback(exc); + } else { + cs->exception_type = NULL; + cs->exception_value = NULL; + cs->exception_traceback = NULL; + } +} + +static inline void +CallState_restore_exception(CallState *cs) { + if (cs->exception_value) { + PyErr_SetRaisedException(cs->exception_value); + /* PyErr_SetRaisedException steals the reference, so clear our pointer */ + cs->exception_value = NULL; + Py_XDECREF(cs->exception_type); + cs->exception_type = NULL; + Py_XDECREF(cs->exception_traceback); + cs->exception_traceback = NULL; + } +} +#else +/* Python < 3.12 - use legacy API */ +static inline void +CallState_fetch_exception(CallState *cs) { + PyErr_Fetch(&cs->exception_type, &cs->exception_value, &cs->exception_traceback); +} + +static inline void +CallState_restore_exception(CallState *cs) { + PyErr_Restore(cs->exception_type, cs->exception_value, cs->exception_traceback); + cs->exception_type = NULL; + cs->exception_value = NULL; + cs->exception_traceback = NULL; +} +#endif + /** * @brief Initialiase a CallState and unlock the GIL prior to a * possibly blocking external call. diff --git a/tests/test_Consumer.py b/tests/test_Consumer.py index 22ff939dd..850fa5ac5 100644 --- a/tests/test_Consumer.py +++ b/tests/test_Consumer.py @@ -347,7 +347,7 @@ def error_cb_that_raises(error): consumer.subscribe(['test-topic']) # This should trigger the error callback due to connection failure - # Before fix: Would get RuntimeError + SystemError + # Before fix: Would get RuntimeError + SystemError (Issue #865) # After fix: Should only get RuntimeError (no SystemError) with pytest.raises(RuntimeError) as exc_info: consumer.consume(timeout=0.1) diff --git a/tests/test_Producer.py b/tests/test_Producer.py index 7a039456d..11049e0a6 100644 --- a/tests/test_Producer.py +++ b/tests/test_Producer.py @@ -304,7 +304,7 @@ def delivery_cb_that_raises(err, msg): producer.produce('test-topic', value='test-message') # Flush to ensure delivery reports are processed - # Before fix: Would get RuntimeError + SystemError + # Before fix: Would get RuntimeError + SystemError (Issue #865) # After fix: Should only get RuntimeError (no SystemError) with pytest.raises(RuntimeError) as exc_info: producer.flush(timeout=2.0) # Longer timeout to ensure delivery callback fires From e693fac3fd6cb8482bfba0e7a978670a9a9caf2f Mon Sep 17 00:00:00 2001 From: Kaushik Raina Date: Wed, 20 Aug 2025 01:56:24 +0530 Subject: [PATCH 5/7] Add wrapper for depreciated API --- src/confluent_kafka/src/Admin.c | 2 +- src/confluent_kafka/src/Consumer.c | 2 +- src/confluent_kafka/src/Producer.c | 2 +- src/confluent_kafka/src/confluent_kafka.c | 13 ++--- src/confluent_kafka/src/confluent_kafka.h | 64 +++++++++++++---------- 5 files changed, 44 insertions(+), 39 deletions(-) diff --git a/src/confluent_kafka/src/Admin.c b/src/confluent_kafka/src/Admin.c index 451017ad6..e9fd53324 100644 --- a/src/confluent_kafka/src/Admin.c +++ b/src/confluent_kafka/src/Admin.c @@ -5082,7 +5082,7 @@ static void Admin_background_event_cb (rd_kafka_t *rk, rd_kafka_event_t *rkev, PyObject *trace = NULL; /* Fetch (and clear) currently raised exception */ - PyErr_Fetch(&exctype, &error, &trace); + cfl_exception_fetch(&exctype, &exc, &trace); Py_XDECREF(trace); } diff --git a/src/confluent_kafka/src/Consumer.c b/src/confluent_kafka/src/Consumer.c index cdfc49015..7dc8bb23d 100644 --- a/src/confluent_kafka/src/Consumer.c +++ b/src/confluent_kafka/src/Consumer.c @@ -1576,7 +1576,7 @@ static void Consumer_rebalance_cb (rd_kafka_t *rk, rd_kafka_resp_err_t err, Py_DECREF(result); else { - PyErr_Fetch(&cs->exception_type, &cs->exception_value, &cs->exception_traceback); + CallState_fetch_exception(cs); CallState_crash(cs); rd_kafka_yield(rk); } diff --git a/src/confluent_kafka/src/Producer.c b/src/confluent_kafka/src/Producer.c index ba6e6e72c..c1f5bfc4a 100644 --- a/src/confluent_kafka/src/Producer.c +++ b/src/confluent_kafka/src/Producer.c @@ -164,7 +164,7 @@ static void dr_msg_cb (rd_kafka_t *rk, const rd_kafka_message_t *rkm, Py_DECREF(result); else { - PyErr_Fetch(&cs->exception_type, &cs->exception_value, &cs->exception_traceback); + CallState_fetch_exception(cs); CallState_crash(cs); rd_kafka_yield(rk); } diff --git a/src/confluent_kafka/src/confluent_kafka.c b/src/confluent_kafka/src/confluent_kafka.c index 4db46ab37..78616436b 100644 --- a/src/confluent_kafka/src/confluent_kafka.c +++ b/src/confluent_kafka/src/confluent_kafka.c @@ -1756,7 +1756,7 @@ static void error_cb (rd_kafka_t *rk, int err, const char *reason, void *opaque) Py_DECREF(result); else { - PyErr_Fetch(&cs->exception_type, &cs->exception_value, &cs->exception_traceback); + CallState_fetch_exception(cs); crash: CallState_crash(cs); rd_kafka_yield(h->rk); @@ -1811,7 +1811,7 @@ static void throttle_cb (rd_kafka_t *rk, const char *broker_name, int32_t broker Py_DECREF(result); goto done; } else { - PyErr_Fetch(&cs->exception_type, &cs->exception_value, &cs->exception_traceback); + CallState_fetch_exception(cs); } /** @@ -1843,7 +1843,7 @@ static int stats_cb(rd_kafka_t *rk, char *json, size_t json_len, void *opaque) { if (result) Py_DECREF(result); else { - PyErr_Fetch(&cs->exception_type, &cs->exception_value, &cs->exception_traceback); + CallState_fetch_exception(cs); CallState_crash(cs); rd_kafka_yield(h->rk); } @@ -1879,7 +1879,7 @@ static void log_cb (const rd_kafka_t *rk, int level, if (result) Py_DECREF(result); else { - PyErr_Fetch(&cs->exception_type, &cs->exception_value, &cs->exception_traceback); + CallState_fetch_exception(cs); CallState_crash(cs); rd_kafka_yield(h->rk); } @@ -2607,10 +2607,7 @@ int CallState_end (Handle *h, CallState *cs) { if (cs->crashed) { /* Restore the saved exception if we have one */ if (cs->exception_type) { - PyErr_Restore(cs->exception_type, cs->exception_value, cs->exception_traceback); - cs->exception_type = NULL; - cs->exception_value = NULL; - cs->exception_traceback = NULL; + CallState_restore_exception(cs); } return 0; } diff --git a/src/confluent_kafka/src/confluent_kafka.h b/src/confluent_kafka/src/confluent_kafka.h index 6fcb2bfb7..7f7d4ac11 100644 --- a/src/confluent_kafka/src/confluent_kafka.h +++ b/src/confluent_kafka/src/confluent_kafka.h @@ -282,52 +282,60 @@ typedef struct { /** * @brief Compatibility layer for Python exception handling API changes. - * PyErr_Fetch was deprecated in Python 3.12 in favor of PyErr_GetRaisedException. + * PyErr_Fetch/PyErr_Restore were deprecated in Python 3.12 in favor of + * PyErr_GetRaisedException/PyErr_SetRaisedException. */ -#if PY_VERSION_HEX >= 0x030c0000 -/* Python 3.12+ - use new API */ + +/* General-purpose compatibility wrappers */ static inline void -CallState_fetch_exception(CallState *cs) { +cfl_exception_fetch(PyObject **exc_type, PyObject **exc_value, PyObject **exc_traceback) { +#if PY_VERSION_HEX >= 0x030c0000 + /* Python 3.12+ - use new API */ PyObject *exc = PyErr_GetRaisedException(); if (exc) { - cs->exception_type = (PyObject *)Py_TYPE(exc); - Py_INCREF(cs->exception_type); - cs->exception_value = exc; - cs->exception_traceback = PyException_GetTraceback(exc); + *exc_type = (PyObject *)Py_TYPE(exc); + Py_INCREF(*exc_type); + *exc_value = exc; + *exc_traceback = PyException_GetTraceback(exc); } else { - cs->exception_type = NULL; - cs->exception_value = NULL; - cs->exception_traceback = NULL; + *exc_type = *exc_value = *exc_traceback = NULL; } +#else + /* Python < 3.12 - use legacy API */ + PyErr_Fetch(exc_type, exc_value, exc_traceback); +#endif } static inline void -CallState_restore_exception(CallState *cs) { - if (cs->exception_value) { - PyErr_SetRaisedException(cs->exception_value); - /* PyErr_SetRaisedException steals the reference, so clear our pointer */ - cs->exception_value = NULL; - Py_XDECREF(cs->exception_type); - cs->exception_type = NULL; - Py_XDECREF(cs->exception_traceback); - cs->exception_traceback = NULL; +cfl_exception_restore(PyObject *exc_type, PyObject *exc_value, PyObject *exc_traceback) { +#if PY_VERSION_HEX >= 0x030c0000 + /* Python 3.12+ - use new API */ + if (exc_value) { + PyErr_SetRaisedException(exc_value); + Py_XDECREF(exc_type); + Py_XDECREF(exc_traceback); } -} #else -/* Python < 3.12 - use legacy API */ + /* Python < 3.12 - use legacy API */ + PyErr_Restore(exc_type, exc_value, exc_traceback); +#endif +} + +/* CallState-specific convenience wrappers */ static inline void CallState_fetch_exception(CallState *cs) { - PyErr_Fetch(&cs->exception_type, &cs->exception_value, &cs->exception_traceback); + cfl_exception_fetch(&cs->exception_type, &cs->exception_value, &cs->exception_traceback); } static inline void CallState_restore_exception(CallState *cs) { - PyErr_Restore(cs->exception_type, cs->exception_value, cs->exception_traceback); - cs->exception_type = NULL; - cs->exception_value = NULL; - cs->exception_traceback = NULL; + if (cs->exception_type) { + cfl_exception_restore(cs->exception_type, cs->exception_value, cs->exception_traceback); + cs->exception_type = NULL; + cs->exception_value = NULL; + cs->exception_traceback = NULL; + } } -#endif /** * @brief Initialiase a CallState and unlock the GIL prior to a From f8dc43a0e70e1b0091a5ca383fbf884cba8f2b51 Mon Sep 17 00:00:00 2001 From: Kaushik Raina Date: Wed, 20 Aug 2025 02:01:23 +0530 Subject: [PATCH 6/7] minor --- src/confluent_kafka/src/confluent_kafka.c | 2 +- src/confluent_kafka/src/confluent_kafka.h | 2 -- 2 files changed, 1 insertion(+), 3 deletions(-) diff --git a/src/confluent_kafka/src/confluent_kafka.c b/src/confluent_kafka/src/confluent_kafka.c index 78616436b..4bdebee14 100644 --- a/src/confluent_kafka/src/confluent_kafka.c +++ b/src/confluent_kafka/src/confluent_kafka.c @@ -1843,7 +1843,7 @@ static int stats_cb(rd_kafka_t *rk, char *json, size_t json_len, void *opaque) { if (result) Py_DECREF(result); else { - CallState_fetch_exception(cs); + CallState_fetch_exception(cs); CallState_crash(cs); rd_kafka_yield(h->rk); } diff --git a/src/confluent_kafka/src/confluent_kafka.h b/src/confluent_kafka/src/confluent_kafka.h index 7f7d4ac11..e4b1f3d54 100644 --- a/src/confluent_kafka/src/confluent_kafka.h +++ b/src/confluent_kafka/src/confluent_kafka.h @@ -286,7 +286,6 @@ typedef struct { * PyErr_GetRaisedException/PyErr_SetRaisedException. */ -/* General-purpose compatibility wrappers */ static inline void cfl_exception_fetch(PyObject **exc_type, PyObject **exc_value, PyObject **exc_traceback) { #if PY_VERSION_HEX >= 0x030c0000 @@ -321,7 +320,6 @@ cfl_exception_restore(PyObject *exc_type, PyObject *exc_value, PyObject *exc_tra #endif } -/* CallState-specific convenience wrappers */ static inline void CallState_fetch_exception(CallState *cs) { cfl_exception_fetch(&cs->exception_type, &cs->exception_value, &cs->exception_traceback); From 7d0822d6c104843a2aec2f2c799e2e77945c0461 Mon Sep 17 00:00:00 2001 From: Kaushik Raina Date: Fri, 22 Aug 2025 18:39:46 +0530 Subject: [PATCH 7/7] Revert error and exec in fetch --- src/confluent_kafka/src/Admin.c | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/confluent_kafka/src/Admin.c b/src/confluent_kafka/src/Admin.c index e9fd53324..f198c5404 100644 --- a/src/confluent_kafka/src/Admin.c +++ b/src/confluent_kafka/src/Admin.c @@ -5082,7 +5082,7 @@ static void Admin_background_event_cb (rd_kafka_t *rk, rd_kafka_event_t *rkev, PyObject *trace = NULL; /* Fetch (and clear) currently raised exception */ - cfl_exception_fetch(&exctype, &exc, &trace); + cfl_exception_fetch(&exctype, &error, &trace); Py_XDECREF(trace); }