kafka/protocol/
offset.rs

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
use std::io::{Read, Write};

use super::{HeaderRequest, HeaderResponse};
use super::{API_KEY_OFFSET, API_VERSION};
use crate::codecs::{FromByte, ToByte};
use crate::error::{KafkaCode, Result};
use crate::utils::PartitionOffset;
use std;

#[derive(Debug)]
pub struct OffsetRequest<'a> {
    pub header: HeaderRequest<'a>,
    pub replica: i32,
    pub topic_partitions: Vec<TopicPartitionOffsetRequest<'a>>,
}

#[derive(Default, Debug)]
pub struct TopicPartitionOffsetRequest<'a> {
    pub topic: &'a str,
    pub partitions: Vec<PartitionOffsetRequest>,
}

#[derive(Default, Debug)]
pub struct PartitionOffsetRequest {
    pub partition: i32,
    pub max_offsets: i32,
    pub time: i64,
}

impl<'a> OffsetRequest<'a> {
    pub fn new(correlation_id: i32, client_id: &'a str) -> OffsetRequest<'a> {
        OffsetRequest {
            header: HeaderRequest::new(API_KEY_OFFSET, API_VERSION, correlation_id, client_id),
            replica: -1,
            topic_partitions: vec![],
        }
    }

    pub fn add(&mut self, topic: &'a str, partition: i32, time: i64) {
        for tp in &mut self.topic_partitions {
            if tp.topic == topic {
                tp.add(partition, time);
                return;
            }
        }
        let mut tp = TopicPartitionOffsetRequest::new(topic);
        tp.add(partition, time);
        self.topic_partitions.push(tp);
    }
}

impl<'a> TopicPartitionOffsetRequest<'a> {
    pub fn new(topic: &'a str) -> TopicPartitionOffsetRequest<'a> {
        TopicPartitionOffsetRequest {
            topic,
            partitions: vec![],
        }
    }

    pub fn add(&mut self, partition: i32, time: i64) {
        self.partitions
            .push(PartitionOffsetRequest::new(partition, time));
    }
}

impl PartitionOffsetRequest {
    pub fn new(partition: i32, time: i64) -> PartitionOffsetRequest {
        PartitionOffsetRequest {
            partition,
            max_offsets: 1,
            time,
        }
    }
}

impl<'a> ToByte for OffsetRequest<'a> {
    fn encode<T: Write>(&self, buffer: &mut T) -> Result<()> {
        try_multi!(
            self.header.encode(buffer),
            self.replica.encode(buffer),
            self.topic_partitions.encode(buffer)
        )
    }
}

impl<'a> ToByte for TopicPartitionOffsetRequest<'a> {
    fn encode<T: Write>(&self, buffer: &mut T) -> Result<()> {
        try_multi!(self.topic.encode(buffer), self.partitions.encode(buffer))
    }
}

impl ToByte for PartitionOffsetRequest {
    fn encode<T: Write>(&self, buffer: &mut T) -> Result<()> {
        try_multi!(
            self.partition.encode(buffer),
            self.time.encode(buffer),
            self.max_offsets.encode(buffer)
        )
    }
}

// --------------------------------------------------------------------

#[derive(Default, Debug)]
pub struct OffsetResponse {
    pub header: HeaderResponse,
    pub topic_partitions: Vec<TopicPartitionOffsetResponse>,
}

#[derive(Default, Debug)]
pub struct TopicPartitionOffsetResponse {
    pub topic: String,
    pub partitions: Vec<PartitionOffsetResponse>,
}

#[derive(Default, Debug)]
pub struct PartitionOffsetResponse {
    pub partition: i32,
    pub error: i16,
    pub offset: Vec<i64>,
}

impl PartitionOffsetResponse {
    pub fn to_offset(&self) -> std::result::Result<PartitionOffset, KafkaCode> {
        match KafkaCode::from_protocol(self.error) {
            Some(code) => Err(code),
            None => {
                let offset = match self.offset.first() {
                    Some(offs) => *offs,
                    None => -1,
                };

                Ok(PartitionOffset {
                    partition: self.partition,
                    offset,
                })
            }
        }
    }
}

impl FromByte for OffsetResponse {
    type R = OffsetResponse;

    #[allow(unused_must_use)]
    fn decode<T: Read>(&mut self, buffer: &mut T) -> Result<()> {
        try_multi!(
            self.header.decode(buffer),
            self.topic_partitions.decode(buffer)
        )
    }
}

impl FromByte for TopicPartitionOffsetResponse {
    type R = TopicPartitionOffsetResponse;

    #[allow(unused_must_use)]
    fn decode<T: Read>(&mut self, buffer: &mut T) -> Result<()> {
        try_multi!(self.topic.decode(buffer), self.partitions.decode(buffer))
    }
}

impl FromByte for PartitionOffsetResponse {
    type R = PartitionOffsetResponse;

    #[allow(unused_must_use)]
    fn decode<T: Read>(&mut self, buffer: &mut T) -> Result<()> {
        try_multi!(
            self.partition.decode(buffer),
            self.error.decode(buffer),
            self.offset.decode(buffer)
        )
    }
}