File: test_transactions.py

package info (click to toggle)
python-confluent-kafka 1.7.0-4
  • links: PTS, VCS
  • area: main
  • in suites: bookworm, forky, sid, trixie
  • size: 1,900 kB
  • sloc: python: 8,335; ansic: 6,065; sh: 1,203; makefile: 178
file content (214 lines) | stat: -rw-r--r-- 6,533 bytes parent folder | download
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
#!/usr/bin/env python
# -*- coding: utf-8 -*-
#
# Copyright 2020 Confluent Inc.
#
# Licensed under the Apache License, Version 2.0 (the "License");
# you may not use this file except in compliance with the License.
# You may obtain a copy of the License at
#
# http://www.apache.org/licenses/LICENSE-2.0
#
# Unless required by applicable law or agreed to in writing, software
# distributed under the License is distributed on an "AS IS" BASIS,
# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
# See the License for the specific language governing permissions and
# limit
#
import inspect
import sys
from uuid import uuid1

from confluent_kafka import Consumer, KafkaError


def called_by():
    if sys.version_info < (3, 5):
        return inspect.stack()[1][3]

    return inspect.stack()[1].function


def prefixed_error_cb(prefix):
    def error_cb(err):
        """ Reports global/generic errors to aid in troubleshooting test failures. """
        print("[{}]: {}".format(prefix, err))

    return error_cb


def prefixed_delivery_cb(prefix):
    def delivery_err(err, msg):
        """ Reports failed message delivery to aid in troubleshooting test failures. """
        if err:
            print("[{}]: Message delivery failed ({} [{}]): {}".format(
                prefix, msg.topic(), str(msg.partition()), err))
            return

    return delivery_err


def test_commit_transaction(kafka_cluster):
    output_topic = kafka_cluster.create_topic("output_topic")

    producer = kafka_cluster.producer({
        'transactional.id': 'example_transactional_id',
        'error_cb': prefixed_error_cb('test_commit_transaction'),
    })

    producer.init_transactions()
    transactional_produce(producer, output_topic, 100)
    producer.commit_transaction()

    assert consume_committed(kafka_cluster.client_conf(), output_topic) == 100


def test_abort_transaction(kafka_cluster):
    output_topic = kafka_cluster.create_topic("output_topic")

    producer = kafka_cluster.producer({
        'transactional.id': 'example_transactional_id',
        'error_cb': prefixed_error_cb('test_abort_transaction'),
    })

    producer.init_transactions()
    transactional_produce(producer, output_topic, 100)
    producer.abort_transaction()

    assert consume_committed(kafka_cluster.client_conf(), output_topic) == 0


def test_abort_retry_commit_transaction(kafka_cluster):
    output_topic = kafka_cluster.create_topic("output_topic")

    producer = kafka_cluster.producer({
        'transactional.id': 'example_transactional_id',
        'error_cb': prefixed_error_cb('test_abort_retry_commit_transaction'),
    })

    producer.init_transactions()
    transactional_produce(producer, output_topic, 100)
    producer.abort_transaction()

    transactional_produce(producer, output_topic, 25)
    producer.commit_transaction()

    assert consume_committed(kafka_cluster.client_conf(), output_topic) == 25


def test_send_offsets_committed_transaction(kafka_cluster):
    input_topic = kafka_cluster.create_topic("input_topic")
    output_topic = kafka_cluster.create_topic("output_topic")
    error_cb = prefixed_error_cb('test_send_offsets_committed_transaction')
    producer = kafka_cluster.producer({
        'client.id': 'producer1',
        'transactional.id': 'example_transactional_id',
        'error_cb': error_cb,
    })

    consumer_conf = {
        'group.id': str(uuid1()),
        'auto.offset.reset': 'earliest',
        'enable.auto.commit': False,
        'enable.partition.eof': True,
        'error_cb': error_cb
    }
    consumer_conf.update(kafka_cluster.client_conf())
    consumer = Consumer(consumer_conf)

    kafka_cluster.seed_topic(input_topic)
    consumer.subscribe([input_topic])

    read_all_msgs(consumer)

    producer.init_transactions()
    transactional_produce(producer, output_topic, 100)

    consumer_position = consumer.position(consumer.assignment())
    group_metadata = consumer.consumer_group_metadata()
    print("=== Sending offsets {} to transaction ===".format(consumer_position))
    producer.send_offsets_to_transaction(consumer_position, group_metadata)
    producer.commit_transaction()

    producer2 = kafka_cluster.producer({
        'client.id': 'producer2',
        'transactional.id': 'example_transactional_id',
        'error_cb': error_cb
    })

    # ensure offset commits are visible prior to sending FetchOffsets request
    producer2.init_transactions()

    committed_offsets = consumer.committed(consumer.assignment())
    print("=== Committed offsets for {} ===".format(committed_offsets))

    assert [tp.offset for tp in committed_offsets] == [100]

    consumer.close()


def transactional_produce(producer, topic, num_messages):
    print("=== Producing {} transactional messages to topic {}. ===".format(
        num_messages, topic))

    producer.begin_transaction()

    for value in ['test-data{}'.format(i) for i in range(0, num_messages)]:
        producer.produce(topic, value, on_delivery=prefixed_delivery_cb(called_by()))
        producer.poll(0.0)

    producer.flush()


def read_all_msgs(consumer):
    """
    Consumes all messages in the consumer assignment.

    This method assumes the consumer has not already read all of the
    messages available in a partition.

    :param consumer:
    :returns: total messages read
    :rtype: int
    """
    msg_cnt = 0
    eof = {}
    print("=== Draining {} ===".format(consumer.assignment()))
    while (True):
        msg = consumer.poll(timeout=1.0)

        if msg is None:
            continue

        topic, partition = msg.topic(), msg.partition()
        if msg.error():
            if msg.error().code() == KafkaError._PARTITION_EOF:
                eof[(topic, partition)] = True
                if len(eof) == len(consumer.assignment()):
                    break
            continue

        eof.pop((topic, partition), None)
        msg_cnt += 1

    return msg_cnt


def consume_committed(conf, topic):
    print("=== Consuming transactional messages from topic {}. ===".format(topic))

    consumer_conf = {'group.id': str(uuid1()),
                     'auto.offset.reset': 'earliest',
                     'enable.auto.commit': False,
                     'enable.partition.eof': True,
                     'error_cb': prefixed_error_cb(called_by()), }

    consumer_conf.update(conf)
    consumer = Consumer(consumer_conf)
    consumer.subscribe([topic])

    msg_cnt = read_all_msgs(consumer)

    consumer.close()

    return msg_cnt