Skip to content

Commit 086d5c0

Browse files
committed
Rename KafkaError .is_txn_abortable() to .txn_requires_abort()
1 parent b01932a commit 086d5c0

File tree

4 files changed

+27
-23
lines changed

4 files changed

+27
-23
lines changed

confluent_kafka/src/Producer.c

Lines changed: 5 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -698,8 +698,8 @@ static PyMethodDef Producer_methods[] = {
698698
"\n"
699699
" :raises: KafkaError: Use exc.args[0].retriable() to check if the\n"
700700
" operation may be retried, or\n"
701-
" exc.args[0].txn_abortable() if the current transaction\n"
702-
" has failed and must be aborted by calling\n"
701+
" exc.args[0].txn_requires_abort() if the current\n"
702+
" transaction has failed and must be aborted by calling\n"
703703
" abort_transaction() and then start a new transaction\n"
704704
" with begin_transaction().\n"
705705
" Treat any other error as a fatal error.\n"
@@ -734,8 +734,8 @@ static PyMethodDef Producer_methods[] = {
734734
"\n"
735735
" :raises: KafkaError: Use exc.args[0].retriable() to check if the\n"
736736
" operation may be retried, or\n"
737-
" exc.args[0].txn_abortable() if the current transaction\n"
738-
" has failed and must be aborted by calling\n"
737+
" exc.args[0].txn_requires_abort() if the current\n"
738+
" transaction has failed and must be aborted by calling\n"
739739
" abort_transaction() and then start a new transaction\n"
740740
" with begin_transaction().\n"
741741
" Treat any other error as a fatal error.\n"
@@ -747,7 +747,7 @@ static PyMethodDef Producer_methods[] = {
747747
"\n"
748748
" Aborts the current transaction.\n"
749749
" This function should also be used to recover from non-fatal\n"
750-
" abortable transaction errors, when KafkaError.txn_abortable()\n"
750+
" abortable transaction errors when KafkaError.txn_requires_abort()\n"
751751
" is True.\n"
752752
" \n"
753753
" Any outstanding messages will be purged and fail with\n"

confluent_kafka/src/confluent_kafka.c

Lines changed: 15 additions & 11 deletions
Original file line numberDiff line numberDiff line change
@@ -64,8 +64,8 @@ typedef struct {
6464
* Else falls back on err2str(). */
6565
int fatal; /**< Set to true if a fatal error. */
6666
int retriable; /**< Set to true if operation is retriable. */
67-
int txn_abortable; /**< Set to true if this is an abortable
68-
* transaction error. */
67+
int txn_requires_abort; /**< Set to true if this is an abortable
68+
* transaction error. */
6969
} KafkaError;
7070

7171

@@ -99,8 +99,9 @@ static PyObject *KafkaError_retriable (KafkaError *self, PyObject *ignore) {
9999
return ret;
100100
}
101101

102-
static PyObject *KafkaError_txn_abortable (KafkaError *self, PyObject *ignore) {
103-
PyObject *ret = self->txn_abortable ? Py_True : Py_False;
102+
static PyObject *
103+
KafkaError_txn_requires_abort (KafkaError *self, PyObject *ignore) {
104+
PyObject *ret = self->txn_requires_abort ? Py_True : Py_False;
104105
Py_INCREF(ret);
105106
return ret;
106107
}
@@ -151,12 +152,15 @@ static PyMethodDef KafkaError_methods[] = {
151152
" :rtype: bool\n"
152153
"\n"
153154
},
154-
{ "txn_abortable", (PyCFunction)KafkaError_txn_abortable, METH_NOARGS,
155-
" :returns: True if the error is an abortable transaction error, "
156-
"allowing the application to abort the current transaction with "
155+
{ "txn_requires_abort", (PyCFunction)KafkaError_txn_requires_abort,
156+
METH_NOARGS,
157+
" :returns: True if the error is an abortable transaction error "
158+
"in which case application must abort the current transaction with "
157159
"abort_transaction() and start a new transaction with "
158-
"begin_transaction(). This will only return true for errors from "
159-
"the transactional producer API.\n"
160+
"begin_transaction() if it wishes to proceed with "
161+
"transactional operations. "
162+
"This will only return true for errors from the transactional\n"
163+
"producer API.\n"
160164
" :rtype: bool\n"
161165
"\n"
162166
},
@@ -305,7 +309,7 @@ static void KafkaError_init (KafkaError *self,
305309
self->code = code;
306310
self->fatal = 0;
307311
self->retriable = 0;
308-
self->txn_abortable = 0;
312+
self->txn_requires_abort = 0;
309313
if (str)
310314
self->str = strdup(str);
311315
else
@@ -364,7 +368,7 @@ PyObject *KafkaError_new_from_error_destroy (rd_kafka_error_t *error) {
364368

365369
kerr->fatal = rd_kafka_error_is_fatal(error);
366370
kerr->retriable = rd_kafka_error_is_retriable(error);
367-
kerr->txn_abortable = rd_kafka_error_is_txn_abortable(error);
371+
kerr->txn_requires_abort = rd_kafka_error_txn_requires_abort(error);
368372
rd_kafka_error_destroy(error);
369373

370374
return (PyObject *)kerr;

docs/index.rst

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -257,7 +257,7 @@ using :py:meth:`confluent_kafka.Producer.abort_transaction()` and optionally
257257
start a new transaction by calling
258258
:py:meth:`confluent_kafka.Producer.begin_transaction()`.
259259
Whether an error is abortable or not is detected by calling
260-
:py:meth:`confluent_kafka.KafkaError.txn_abortable()`.
260+
:py:meth:`confluent_kafka.KafkaError.txn_requires_abort()`.
261261

262262
**Fatal errors**
263263

@@ -295,7 +295,7 @@ neither the retriable or abortable flags set, as fatal.
295295
if e.args[0].retriable():
296296
# retriable error, try again
297297
continue
298-
elif e.args[0].txn_abortable():
298+
elif e.args[0].txn_requires_abort():
299299
# abort current transaction, begin a new transaction,
300300
# and rewind the consumer to start over.
301301
producer.abort_transaction()

tests/test_Producer.py

Lines changed: 5 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -200,15 +200,15 @@ def test_transaction_api():
200200
assert ex.value.args[0].code() == KafkaError._TIMED_OUT
201201
assert ex.value.args[0].retriable() is True
202202
assert ex.value.args[0].fatal() is False
203-
assert ex.value.args[0].txn_abortable() is False
203+
assert ex.value.args[0].txn_requires_abort() is False
204204

205205
# Any subsequent APIs will fail since init did not succeed.
206206
with pytest.raises(KafkaException) as ex:
207207
p.begin_transaction()
208208
assert ex.value.args[0].code() == KafkaError._STATE
209209
assert ex.value.args[0].retriable() is False
210210
assert ex.value.args[0].fatal() is False
211-
assert ex.value.args[0].txn_abortable() is False
211+
assert ex.value.args[0].txn_requires_abort() is False
212212

213213
consumer = Consumer({"group.id": "testgroup"})
214214
group_metadata = consumer.consumer_group_metadata()
@@ -220,18 +220,18 @@ def test_transaction_api():
220220
assert ex.value.args[0].code() == KafkaError._STATE
221221
assert ex.value.args[0].retriable() is False
222222
assert ex.value.args[0].fatal() is False
223-
assert ex.value.args[0].txn_abortable() is False
223+
assert ex.value.args[0].txn_requires_abort() is False
224224

225225
with pytest.raises(KafkaException) as ex:
226226
p.commit_transaction(0.5)
227227
assert ex.value.args[0].code() == KafkaError._STATE
228228
assert ex.value.args[0].retriable() is False
229229
assert ex.value.args[0].fatal() is False
230-
assert ex.value.args[0].txn_abortable() is False
230+
assert ex.value.args[0].txn_requires_abort() is False
231231

232232
with pytest.raises(KafkaException) as ex:
233233
p.abort_transaction(0.5)
234234
assert ex.value.args[0].code() == KafkaError._STATE
235235
assert ex.value.args[0].retriable() is False
236236
assert ex.value.args[0].fatal() is False
237-
assert ex.value.args[0].txn_abortable() is False
237+
assert ex.value.args[0].txn_requires_abort() is False

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