File: test_list_offsets.py

package info (click to toggle)
python-confluent-kafka 2.11.1-1
  • links: PTS, VCS
  • area: main
  • in suites: forky, sid
  • size: 3,660 kB
  • sloc: python: 30,428; ansic: 9,487; sh: 1,477; makefile: 192
file content (82 lines) | stat: -rw-r--r-- 3,394 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
# -*- coding: utf-8 -*-
# Copyright 2023 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
# limitations under the License.

from confluent_kafka import TopicPartition, IsolationLevel
from confluent_kafka.admin import ListOffsetsResultInfo, OffsetSpec


def test_list_offsets(kafka_cluster):
    """
    Test list offsets results when asking for the earliest offset (the first one),
    the latest offset (last one), max timestamp (second one)
    or after a specific timestamp (second one).
    """

    admin_client = kafka_cluster.admin()

    # Create a topic with a single partition
    topic = kafka_cluster.create_topic_and_wait_propogation("test-topic-verify-list-offsets",
                                                            {
                                                                "num_partitions": 1,
                                                                "replication_factor": 1,
                                                            })

    # Create Producer instance
    p = kafka_cluster.producer()
    base_timestamp = 1000000000
    p.produce(topic, "Message-1", timestamp=(base_timestamp + 100))
    p.produce(topic, "Message-2", timestamp=(base_timestamp + 400))
    p.produce(topic, "Message-3", timestamp=(base_timestamp + 200))
    p.flush()

    topic_partition = TopicPartition(topic, 0)

    for kwargs in [{},
                   {"isolation_level": IsolationLevel.READ_UNCOMMITTED},
                   {"request_timeout": 30},
                   {"isolation_level": IsolationLevel.READ_COMMITTED, "request_timeout": 30}]:

        requests = {topic_partition: OffsetSpec.earliest()}
        futmap = admin_client.list_offsets(requests, **kwargs)
        for _, fut in futmap.items():
            result = fut.result()
            assert isinstance(result, ListOffsetsResultInfo)
            assert (result.offset == 0)

        requests = {topic_partition: OffsetSpec.latest()}
        futmap = admin_client.list_offsets(requests, **kwargs)
        for _, fut in futmap.items():
            result = fut.result()
            assert isinstance(result, ListOffsetsResultInfo)
            assert (result.offset == 3)

        requests = {topic_partition: OffsetSpec.max_timestamp()}
        futmap = admin_client.list_offsets(requests, **kwargs)
        for _, fut in futmap.items():
            result = fut.result()
            assert isinstance(result, ListOffsetsResultInfo)
            assert (result.offset == 1)

        requests = {topic_partition: OffsetSpec.for_timestamp(base_timestamp + 150)}
        futmap = admin_client.list_offsets(requests, **kwargs)
        for _, fut in futmap.items():
            result = fut.result()
            assert isinstance(result, ListOffsetsResultInfo)
            assert (result.offset == 1)

    # Delete created topic
    fs = admin_client.delete_topics([topic])
    for topic, f in fs.items():
        f.result()