From df91be4514208105d300802ebc19a8dca396d2bf Mon Sep 17 00:00:00 2001 From: Ojasva Jain Date: Thu, 13 Nov 2025 19:13:02 +0530 Subject: [PATCH 1/9] dev changes to handle oauth callbacks from background thread --- examples/ojaintest.py | 0 src/confluent_kafka/src/Admin.c | 13 +++++ src/confluent_kafka/src/Consumer.c | 11 +++++ src/confluent_kafka/src/Producer.c | 10 ++++ src/confluent_kafka/src/confluent_kafka.c | 58 +++++++++++++++++++++-- src/confluent_kafka/src/confluent_kafka.h | 2 + 6 files changed, 89 insertions(+), 5 deletions(-) create mode 100644 examples/ojaintest.py diff --git a/examples/ojaintest.py b/examples/ojaintest.py new file mode 100644 index 000000000..e69de29bb diff --git a/src/confluent_kafka/src/Admin.c b/src/confluent_kafka/src/Admin.c index bdd343c44..77d6ce440 100644 --- a/src/confluent_kafka/src/Admin.c +++ b/src/confluent_kafka/src/Admin.c @@ -28,6 +28,7 @@ #include "confluent_kafka.h" #include +#include /**************************************************************************** @@ -5346,10 +5347,22 @@ static int Admin_init (PyObject *selfobj, PyObject *args, PyObject *kwargs) { return -1; } + /* Enable SASL callbacks on background thread for AdminClient since + * applications typically don't call poll() regularly on AdminClient. */ + if (self->oauth_cb) { + rd_kafka_sasl_background_callbacks_enable(self->rk); + } + /* Forward log messages to poll queue */ if (self->logger) rd_kafka_set_log_queue(self->rk, NULL); + + /* Wait for the background thread to set the token */ + if (self->oauth_cb) { + return wait_for_oauth_token_set(self); + } + return 0; } diff --git a/src/confluent_kafka/src/Consumer.c b/src/confluent_kafka/src/Consumer.c index 9a747862d..53a021511 100644 --- a/src/confluent_kafka/src/Consumer.c +++ b/src/confluent_kafka/src/Consumer.c @@ -1695,6 +1695,11 @@ static int Consumer_init (PyObject *selfobj, PyObject *args, PyObject *kwargs) { return -1; } + /* Enable Token Refresh to be handled by background thread if OAuth callback is provided */ + if (self->oauth_cb) { + rd_kafka_sasl_background_callbacks_enable(self->rk); + } + /* Forward log messages to main queue which is then forwarded * to the consumer queue */ if (self->logger) @@ -1705,6 +1710,12 @@ static int Consumer_init (PyObject *selfobj, PyObject *args, PyObject *kwargs) { self->u.Consumer.rkqu = rd_kafka_queue_get_consumer(self->rk); assert(self->u.Consumer.rkqu); + + /* Wait for the background thread to set the token */ + if (self->oauth_cb) { + return wait_for_oauth_token_set(self); + } + return 0; } diff --git a/src/confluent_kafka/src/Producer.c b/src/confluent_kafka/src/Producer.c index 0e3d4a307..f187b2b14 100644 --- a/src/confluent_kafka/src/Producer.c +++ b/src/confluent_kafka/src/Producer.c @@ -1297,10 +1297,20 @@ static int Producer_init (PyObject *selfobj, PyObject *args, PyObject *kwargs) { return -1; } + /* Enable Token Refresh to be handled by background thread if OAuth callback is provided */ + if (self->oauth_cb) { + rd_kafka_sasl_background_callbacks_enable(self->rk); + } + /* Forward log messages to poll queue */ if (self->logger) rd_kafka_set_log_queue(self->rk, NULL); + /* Wait for the background thread to set the token */ + if (self->oauth_cb) { + return wait_for_oauth_token_set(self); + } + return 0; } diff --git a/src/confluent_kafka/src/confluent_kafka.c b/src/confluent_kafka/src/confluent_kafka.c index 09c57efa9..9f401fe3d 100644 --- a/src/confluent_kafka/src/confluent_kafka.c +++ b/src/confluent_kafka/src/confluent_kafka.c @@ -2053,11 +2053,56 @@ static int py_extensions_to_c (char **extensions, Py_ssize_t idx, return 1; } + +/** + * @brief Waits for OAuth callback to set the token + * + * Useful during client init as we want to ensure we have the token before we return back + * + * Returns 0 if token was set within the timeout period, -1 otherwise. + */ +int wait_for_oauth_token_set(Handle *h) { + + if (!h->oauth_cb) + return 0; + + int max_wait_sec = 5; + int retry_interval_sec = 1; /* Check every 1 sec */ + int elapsed_sec = 0; + while (!h->oauth_token_set && elapsed_sec < max_wait_sec) { + CallState cs; + CallState_begin(h, &cs); + sleep(retry_interval_sec); + CallState_end(h, &cs); + elapsed_sec += retry_interval_sec; + } + + if (!h->oauth_token_set) { + /* Token not set within timeout */ + cfl_PyErr_Format(RD_KAFKA_RESP_ERR_SASL_AUTHENTICATION_FAILED, + "OAuth token not set within %d seconds timeout", + max_wait_sec); + CallState cs; + CallState_begin(h, &cs); + rd_kafka_destroy(h->rk); + h->rk = NULL; + CallState_end(h, &cs); + return -1; + } + return 0; +} + +/** + * @brief Callback invoked when a OAuth token needs to be refreshed. + * + * Note that this callback will be invoked by the background thread as + * all client types have been configured to use background threads for sasl events. + */ static void oauth_cb (rd_kafka_t *rk, const char *oauthbearer_config, void *opaque) { Handle *h = opaque; PyObject *eo, *result; - CallState *cs; + PyGILState_STATE gstate; const char *token; double expiry; const char *principal = ""; @@ -2067,7 +2112,7 @@ static void oauth_cb (rd_kafka_t *rk, const char *oauthbearer_config, char err_msg[2048]; rd_kafka_resp_err_t err_code; - cs = CallState_get(h); + gstate = PyGILState_Ensure(); eo = Py_BuildValue("s", oauthbearer_config); result = PyObject_CallFunctionObjArgs(h->oauth_cb, eo, NULL); @@ -2116,6 +2161,7 @@ static void oauth_cb (rd_kafka_t *rk, const char *oauthbearer_config, PyErr_Format(PyExc_ValueError, "%s", err_msg); goto fail; } + h->oauth_token_set = 1; goto done; fail: @@ -2127,10 +2173,10 @@ static void oauth_cb (rd_kafka_t *rk, const char *oauthbearer_config, PyErr_Clear(); goto done; err: - CallState_crash(cs); + PyGILState_Release(gstate); rd_kafka_yield(h->rk); done: - CallState_resume(cs); + PyGILState_Release(gstate); } /**************************************************************************** @@ -2650,8 +2696,10 @@ rd_kafka_conf_t *common_conf_setup (rd_kafka_type_t ktype, rd_kafka_conf_set_log_cb(conf, log_cb); } - if (h->oauth_cb) + if (h->oauth_cb) { rd_kafka_conf_set_oauthbearer_token_refresh_cb(conf, oauth_cb); + rd_kafka_conf_enable_sasl_queue(conf, 1); + } rd_kafka_conf_set_opaque(conf, h); diff --git a/src/confluent_kafka/src/confluent_kafka.h b/src/confluent_kafka/src/confluent_kafka.h index b69f75dbd..40dda0190 100644 --- a/src/confluent_kafka/src/confluent_kafka.h +++ b/src/confluent_kafka/src/confluent_kafka.h @@ -232,6 +232,7 @@ typedef struct { PyObject *logger; PyObject *oauth_cb; + int oauth_token_set; union { /** @@ -444,6 +445,7 @@ PyObject *c_topic_partition_result_to_py_dict( PyObject *list_topics (Handle *self, PyObject *args, PyObject *kwargs); PyObject *list_groups (Handle *self, PyObject *args, PyObject *kwargs); PyObject *set_sasl_credentials(Handle *self, PyObject *args, PyObject *kwargs); +int wait_for_oauth_token_set(Handle *self); extern const char list_topics_doc[]; From 384c688869c8edf42728c782bf0938536538a3e2 Mon Sep 17 00:00:00 2001 From: Ojasva Jain Date: Thu, 13 Nov 2025 19:26:29 +0530 Subject: [PATCH 2/9] temporarily comment out test case --- src/confluent_kafka/src/Admin.c | 1 - tests/test_misc.py | 50 ++++++++++++++++----------------- 2 files changed, 25 insertions(+), 26 deletions(-) diff --git a/src/confluent_kafka/src/Admin.c b/src/confluent_kafka/src/Admin.c index 77d6ce440..172ef3261 100644 --- a/src/confluent_kafka/src/Admin.c +++ b/src/confluent_kafka/src/Admin.c @@ -28,7 +28,6 @@ #include "confluent_kafka.h" #include -#include /**************************************************************************** diff --git a/tests/test_misc.py b/tests/test_misc.py index 92992ed32..ef38325c8 100644 --- a/tests/test_misc.py +++ b/tests/test_misc.py @@ -166,31 +166,31 @@ def oauth_cb(oauth_config): kc.close() -def test_oauth_cb_failure(): - """ Tests oauth_cb. """ - oauth_cb_count = 0 - - def oauth_cb(oauth_config): - nonlocal oauth_cb_count - oauth_cb_count += 1 - assert oauth_config == 'oauth_cb' - if oauth_cb_count == 2: - return 'token', time.time() + 100.0, oauth_config, {"extthree": "extthreeval"} - raise Exception - - conf = {'group.id': 'test', - 'security.protocol': 'sasl_plaintext', - 'sasl.mechanisms': 'OAUTHBEARER', - 'session.timeout.ms': 1000, # Avoid close() blocking too long - 'sasl.oauthbearer.config': 'oauth_cb', - 'oauth_cb': oauth_cb - } - - kc = TestConsumer(conf) - - while oauth_cb_count < 2: - kc.poll(timeout=0.1) - kc.close() +# def test_oauth_cb_failure(): +# """ Tests oauth_cb. """ +# oauth_cb_count = 0 +# +# def oauth_cb(oauth_config): +# nonlocal oauth_cb_count +# oauth_cb_count += 1 +# assert oauth_config == 'oauth_cb' +# if oauth_cb_count == 2: +# return 'token', time.time() + 100.0, oauth_config, {"extthree": "extthreeval"} +# raise Exception +# +# conf = {'group.id': 'test', +# 'security.protocol': 'sasl_plaintext', +# 'sasl.mechanisms': 'OAUTHBEARER', +# 'session.timeout.ms': 1000, # Avoid close() blocking too long +# 'sasl.oauthbearer.config': 'oauth_cb', +# 'oauth_cb': oauth_cb +# } +# +# kc = TestConsumer(conf) +# +# while oauth_cb_count < 2: +# kc.poll(timeout=0.1) +# kc.close() def skip_interceptors(): From ca32da31a77c19d76c68496e5e40f663743d1899 Mon Sep 17 00:00:00 2001 From: Ojasva Jain Date: Fri, 14 Nov 2025 12:33:01 +0530 Subject: [PATCH 3/9] add test cases for background oauth callbacks --- tests/test_misc.py | 123 +++++++++++++++++++++++++++++++++------------ 1 file changed, 91 insertions(+), 32 deletions(-) diff --git a/tests/test_misc.py b/tests/test_misc.py index ef38325c8..b67cb4c69 100644 --- a/tests/test_misc.py +++ b/tests/test_misc.py @@ -7,7 +7,7 @@ import sys import confluent_kafka -from confluent_kafka import Consumer, Producer +from confluent_kafka import Consumer, Producer, KafkaException from confluent_kafka.admin import AdminClient from tests.common import TestConsumer @@ -135,9 +135,7 @@ def oauth_cb(oauth_config): } kc = TestConsumer(conf) - - while not seen_oauth_cb: - kc.poll(timeout=0.1) + assert seen_oauth_cb == True # callback is expected to happen during client init kc.close() @@ -154,43 +152,104 @@ def oauth_cb(oauth_config): conf = {'group.id': 'test', 'security.protocol': 'sasl_plaintext', 'sasl.mechanisms': 'OAUTHBEARER', - 'session.timeout.ms': 100, # Avoid close() blocking too long + 'session.timeout.ms': 100, 'sasl.oauthbearer.config': 'oauth_cb', 'oauth_cb': oauth_cb } kc = TestConsumer(conf) + assert seen_oauth_cb == True # callback is expected to happen during client init + kc.close() + + +def test_oauth_cb_failure(): + """ + Tests oauth_cb for a case when it fails to return a token. + We expect the client init to fail + """ + + def oauth_cb(oauth_config): + raise Exception + + conf = {'group.id': 'test', + 'security.protocol': 'sasl_plaintext', + 'sasl.mechanisms': 'OAUTHBEARER', + 'session.timeout.ms': 1000, + 'sasl.oauthbearer.config': 'oauth_cb', + 'oauth_cb': oauth_cb + } + + with pytest.raises(KafkaException): + TestConsumer(conf) + +def test_oauth_cb_token_refresh_success(): + """ + Tests whether oauth callback gets called multiple times by the background thread + """ + oauth_cb_count = 0 + + def oauth_cb(oauth_config): + nonlocal oauth_cb_count + oauth_cb_count += 1 + assert oauth_config == 'oauth_cb' + return 'token', time.time() + 3 # token is returned with an expiry of 3 seconds + + conf = {'group.id': 'test', + 'security.protocol': 'sasl_plaintext', + 'sasl.mechanisms': 'OAUTHBEARER', + 'session.timeout.ms': 1000, + 'sasl.oauthbearer.config': 'oauth_cb', + 'oauth_cb': oauth_cb + } + + kc = TestConsumer(conf) # callback is expected to happen during client init + assert oauth_cb_count == 1 + + # Check every 1 second for up to 5 seconds for callback count to increase + max_wait_sec = 5 + elapsed_sec = 0 + while oauth_cb_count == 1 and elapsed_sec < max_wait_sec: + time.sleep(1) + elapsed_sec += 1 - while not seen_oauth_cb: - kc.poll(timeout=0.1) kc.close() + assert oauth_cb_count > 1 + +def test_oauth_cb_token_refresh_failure(): + """ + Tests whether oauth callback gets called again if token refresh failed in one of the calls after init + """ + oauth_cb_count = 0 + + def oauth_cb(oauth_config): + nonlocal oauth_cb_count + oauth_cb_count += 1 + assert oauth_config == 'oauth_cb' + if oauth_cb_count == 2: + raise Exception + return 'token', time.time() + 3 # token is returned with an expiry of 3 seconds + + conf = {'group.id': 'test', + 'security.protocol': 'sasl_plaintext', + 'sasl.mechanisms': 'OAUTHBEARER', + 'session.timeout.ms': 1000, # Avoid close() blocking too long + 'sasl.oauthbearer.config': 'oauth_cb', + 'oauth_cb': oauth_cb + } + + kc = TestConsumer(conf) # callback is expected to happen during client init + assert oauth_cb_count == 1 + # Check every 1 second for up to 15 seconds for callback count to increase + # Call back failure causes a refresh attempt after 10 secs, so ideally 2 callbacks should happen within 15 secs + max_wait_sec = 15 + elapsed_sec = 0 + while oauth_cb_count <= 2 and elapsed_sec < max_wait_sec: + time.sleep(1) + elapsed_sec += 1 -# def test_oauth_cb_failure(): -# """ Tests oauth_cb. """ -# oauth_cb_count = 0 -# -# def oauth_cb(oauth_config): -# nonlocal oauth_cb_count -# oauth_cb_count += 1 -# assert oauth_config == 'oauth_cb' -# if oauth_cb_count == 2: -# return 'token', time.time() + 100.0, oauth_config, {"extthree": "extthreeval"} -# raise Exception -# -# conf = {'group.id': 'test', -# 'security.protocol': 'sasl_plaintext', -# 'sasl.mechanisms': 'OAUTHBEARER', -# 'session.timeout.ms': 1000, # Avoid close() blocking too long -# 'sasl.oauthbearer.config': 'oauth_cb', -# 'oauth_cb': oauth_cb -# } -# -# kc = TestConsumer(conf) -# -# while oauth_cb_count < 2: -# kc.poll(timeout=0.1) -# kc.close() + kc.close() + assert oauth_cb_count > 2 def skip_interceptors(): From a1ec933a4cf58150b3d26816a75f55a7ae8899a7 Mon Sep 17 00:00:00 2001 From: Ojasva Jain Date: Fri, 14 Nov 2025 12:39:46 +0530 Subject: [PATCH 4/9] remove test file --- examples/ojaintest.py | 0 1 file changed, 0 insertions(+), 0 deletions(-) delete mode 100644 examples/ojaintest.py diff --git a/examples/ojaintest.py b/examples/ojaintest.py deleted file mode 100644 index e69de29bb..000000000 From b4b933ca2d39093a3f6f9deb52fc4c898177e927 Mon Sep 17 00:00:00 2001 From: Ojasva Jain Date: Fri, 14 Nov 2025 13:39:18 +0530 Subject: [PATCH 5/9] minor fixes in test assertions and lint --- tests/test_misc.py | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/tests/test_misc.py b/tests/test_misc.py index b67cb4c69..3462c5b5b 100644 --- a/tests/test_misc.py +++ b/tests/test_misc.py @@ -135,7 +135,7 @@ def oauth_cb(oauth_config): } kc = TestConsumer(conf) - assert seen_oauth_cb == True # callback is expected to happen during client init + assert seen_oauth_cb # callback is expected to happen during client init kc.close() @@ -158,7 +158,7 @@ def oauth_cb(oauth_config): } kc = TestConsumer(conf) - assert seen_oauth_cb == True # callback is expected to happen during client init + assert seen_oauth_cb # callback is expected to happen during client init kc.close() @@ -182,6 +182,7 @@ def oauth_cb(oauth_config): with pytest.raises(KafkaException): TestConsumer(conf) + def test_oauth_cb_token_refresh_success(): """ Tests whether oauth callback gets called multiple times by the background thread @@ -202,7 +203,7 @@ def oauth_cb(oauth_config): 'oauth_cb': oauth_cb } - kc = TestConsumer(conf) # callback is expected to happen during client init + kc = TestConsumer(conf) # callback is expected to happen during client init assert oauth_cb_count == 1 # Check every 1 second for up to 5 seconds for callback count to increase @@ -215,6 +216,7 @@ def oauth_cb(oauth_config): kc.close() assert oauth_cb_count > 1 + def test_oauth_cb_token_refresh_failure(): """ Tests whether oauth callback gets called again if token refresh failed in one of the calls after init From cc6293f538041c47c1416f67d48356619b5f5ab6 Mon Sep 17 00:00:00 2001 From: Ojasva Jain Date: Wed, 19 Nov 2025 10:35:23 +0530 Subject: [PATCH 6/9] update documentation --- docs/index.rst | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/index.rst b/docs/index.rst index 0e9c481d0..8b59fdf99 100644 --- a/docs/index.rst +++ b/docs/index.rst @@ -1057,7 +1057,7 @@ addition to the properties dictated by the underlying librdkafka C library: where ``expiry_time`` is the time in seconds since the epoch as a floating point number. This callback is useful only when ``sasl.mechanisms=OAUTHBEARER`` is set and is served to get the initial token before a successful broker connection can be made. - The callback can be triggered by calling ``client.poll()`` or ``producer.flush()``. + The callback is asynchronously triggered by the background thread to maintain token validity.``. * ``on_delivery(kafka.KafkaError, kafka.Message)`` (**Producer**): value is a Python function reference that is called once for each produced message to indicate the final From cc4eff2c64743642af8d18dae1c40c83be689a2e Mon Sep 17 00:00:00 2001 From: Ojasva Jain Date: Thu, 20 Nov 2025 12:43:59 +0530 Subject: [PATCH 7/9] increase timeout from 5s to 10s --- src/confluent_kafka/src/confluent_kafka.c | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/confluent_kafka/src/confluent_kafka.c b/src/confluent_kafka/src/confluent_kafka.c index d0c413192..9097f7d0a 100644 --- a/src/confluent_kafka/src/confluent_kafka.c +++ b/src/confluent_kafka/src/confluent_kafka.c @@ -2023,7 +2023,7 @@ int wait_for_oauth_token_set(Handle *h) { if (!h->oauth_cb) return 0; - int max_wait_sec = 5; + int max_wait_sec = 10; int retry_interval_sec = 1; /* Check every 1 sec */ int elapsed_sec = 0; while (!h->oauth_token_set && elapsed_sec < max_wait_sec) { From 9b9904f66a483d6be79005215ddc86dd1e916e82 Mon Sep 17 00:00:00 2001 From: Ojasva Jain Date: Thu, 20 Nov 2025 13:48:43 +0530 Subject: [PATCH 8/9] fix code formatting in test_misc --- tests/test_misc.py | 45 ++++++++++++++++++++++++--------------------- 1 file changed, 24 insertions(+), 21 deletions(-) diff --git a/tests/test_misc.py b/tests/test_misc.py index 0f7cc359b..9deece0e8 100644 --- a/tests/test_misc.py +++ b/tests/test_misc.py @@ -154,13 +154,14 @@ def oauth_cb(oauth_config): assert oauth_config == 'oauth_cb' return 'token', time.time() + 300.0, oauth_config, {"extone": "extoneval", "exttwo": "exttwoval"} - conf = {'group.id': 'test', - 'security.protocol': 'sasl_plaintext', - 'sasl.mechanisms': 'OAUTHBEARER', - 'session.timeout.ms': 100, - 'sasl.oauthbearer.config': 'oauth_cb', - 'oauth_cb': oauth_cb - } + conf = { + 'group.id': 'test', + 'security.protocol': 'sasl_plaintext', + 'sasl.mechanisms': 'OAUTHBEARER', + 'session.timeout.ms': 100, + 'sasl.oauthbearer.config': 'oauth_cb', + 'oauth_cb': oauth_cb, + } kc = TestConsumer(conf) assert seen_oauth_cb # callback is expected to happen during client init @@ -176,13 +177,14 @@ def test_oauth_cb_failure(): def oauth_cb(oauth_config): raise Exception - conf = {'group.id': 'test', - 'security.protocol': 'sasl_plaintext', - 'sasl.mechanisms': 'OAUTHBEARER', - 'session.timeout.ms': 1000, - 'sasl.oauthbearer.config': 'oauth_cb', - 'oauth_cb': oauth_cb - } + conf = { + 'group.id': 'test', + 'security.protocol': 'sasl_plaintext', + 'sasl.mechanisms': 'OAUTHBEARER', + 'session.timeout.ms': 1000, + 'sasl.oauthbearer.config': 'oauth_cb', + 'oauth_cb': oauth_cb, + } with pytest.raises(KafkaException): TestConsumer(conf) @@ -200,13 +202,14 @@ def oauth_cb(oauth_config): assert oauth_config == 'oauth_cb' return 'token', time.time() + 3 # token is returned with an expiry of 3 seconds - conf = {'group.id': 'test', - 'security.protocol': 'sasl_plaintext', - 'sasl.mechanisms': 'OAUTHBEARER', - 'session.timeout.ms': 1000, - 'sasl.oauthbearer.config': 'oauth_cb', - 'oauth_cb': oauth_cb - } + conf = { + 'group.id': 'test', + 'security.protocol': 'sasl_plaintext', + 'sasl.mechanisms': 'OAUTHBEARER', + 'session.timeout.ms': 1000, + 'sasl.oauthbearer.config': 'oauth_cb', + 'oauth_cb': oauth_cb, + } kc = TestConsumer(conf) # callback is expected to happen during client init assert oauth_cb_count == 1 From 1c556ef80a454c4f6fa40edbeb84e9238293b234 Mon Sep 17 00:00:00 2001 From: Ojasva Jain Date: Thu, 20 Nov 2025 15:05:26 +0530 Subject: [PATCH 9/9] fix formatting according to clang-format v18 --- src/confluent_kafka/src/Admin.c | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/src/confluent_kafka/src/Admin.c b/src/confluent_kafka/src/Admin.c index d2b49711d..c3af72413 100644 --- a/src/confluent_kafka/src/Admin.c +++ b/src/confluent_kafka/src/Admin.c @@ -99,12 +99,12 @@ struct Admin_options { * Make sure this is kept up to date with Admin_options above. */ #define Admin_options_INITIALIZER \ { \ - Admin_options_def_int, Admin_options_def_float, \ - Admin_options_def_float, Admin_options_def_int, \ - Admin_options_def_int, Admin_options_def_int, \ - Admin_options_def_int, Admin_options_def_ptr, \ - Admin_options_def_cnt, Admin_options_def_ptr, \ - Admin_options_def_cnt, \ + Admin_options_def_int, Admin_options_def_float, \ + Admin_options_def_float, Admin_options_def_int, \ + Admin_options_def_int, Admin_options_def_int, \ + Admin_options_def_int, Admin_options_def_ptr, \ + Admin_options_def_cnt, Admin_options_def_ptr, \ + Admin_options_def_cnt, \ } #define Admin_options_is_set_int(v) ((v) != Admin_options_def_int)