forked from confluentinc/librdkafka
-
Notifications
You must be signed in to change notification settings - Fork 0
/
rdkafka_proto.h
245 lines (197 loc) · 6.85 KB
/
rdkafka_proto.h
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
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
/*
* librdkafka - Apache Kafka C library
*
* Copyright (c) 2012,2013 Magnus Edenhill
* All rights reserved.
*
* Redistribution and use in source and binary forms, with or without
* modification, are permitted provided that the following conditions are met:
*
* 1. Redistributions of source code must retain the above copyright notice,
* this list of conditions and the following disclaimer.
* 2. Redistributions in binary form must reproduce the above copyright notice,
* this list of conditions and the following disclaimer in the documentation
* and/or other materials provided with the distribution.
*
* THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS"
* AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE
* IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE
* ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE
* LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR
* CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF
* SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS
* INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN
* CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE)
* ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE
* POSSIBILITY OF SUCH DAMAGE.
*/
#pragma once
/*
* Kafka protocol definitions.
*/
#define RD_KAFKA_PORT 9092
#define RD_KAFKA_PORT_STR "9092"
/**
* Request header
*/
struct rd_kafkap_reqhdr {
int32_t Size;
int16_t ApiKey;
#define RD_KAFKAP_Produce 0
#define RD_KAFKAP_Fetch 1
#define RD_KAFKAP_Offset 2
#define RD_KAFKAP_Metadata 3
#define RD_KAFKAP_LeaderAndIsr 4
#define RD_KAFKAP_StopReplica 5
#define RD_KAFKAP_OffsetCommit 6
#define RD_KAFKAP_OffsetFetch 7
int16_t ApiVersion;
int32_t CorrId;
/* ClientId follows */
} RD_PACKED;
/**
* Response header
*/
struct rd_kafkap_reshdr {
int32_t Size;
int32_t CorrId;
} RD_PACKED;
/**
*
* Kafka protocol string representation: { uint16, data.. }
*
*/
typedef struct rd_kafkap_str_s {
int16_t len; /* big endian */
char str[0]; /* allocated dynamically */
} RD_PACKED rd_kafkap_str_t;
#define RD_KAFKAP_STR_LEN_NULL -1
/* Returns the actual size of a kafka protocol string representation. */
#define RD_KAFKAP_STR_SIZE(kstr) (int16_t)(sizeof((kstr)->len) + \
((int16_t)ntohs((kstr)->len) == \
RD_KAFKAP_STR_LEN_NULL ? \
0 : ntohs((kstr)->len)))
/* Returns the length of the string of a kafka protocol string representation */
#define RD_KAFKAP_STR_LEN(kstr) (int)((ntohs((kstr)->len) == \
RD_KAFKAP_STR_LEN_NULL ? \
0 : (int16_t)ntohs((kstr)->len)))
/* Macro suitable for "%.*s" printing. */
#define RD_KAFKAP_STR_PR(kstr) \
((int16_t)ntohs((kstr)->len) == RD_KAFKAP_STR_LEN_NULL ? \
0 : (int)ntohs((kstr)->len)), (kstr)->str
#define RD_KAFKAP_STR_IS_NULL(kstr) \
((int16_t)ntohs((kstr)->len) == RD_KAFKAP_STR_LEN_NULL)
static inline int rd_kafkap_str_cmp (const rd_kafkap_str_t *a,
const rd_kafkap_str_t *b) RD_UNUSED;
static inline int rd_kafkap_str_cmp (const rd_kafkap_str_t *a,
const rd_kafkap_str_t *b) {
if (a->len != b->len)
return -1;
return memcmp(a->str, b->str, ntohs(a->len));
}
static inline int rd_kafkap_str_cmp_str (const rd_kafkap_str_t *a,
const char *str) RD_UNUSED;
static inline int rd_kafkap_str_cmp_str (const rd_kafkap_str_t *a,
const char *str) {
int len = strlen(str);
if (ntohs(a->len) != len)
return -1;
return memcmp(a->str, str, ntohs(a->len));
}
static inline rd_kafkap_str_t *rd_kafkap_str_new (const char *str) RD_UNUSED;
static inline rd_kafkap_str_t *rd_kafkap_str_new (const char *str) {
rd_kafkap_str_t *kstr;
int len = 0;
if (str)
len = strlen(str);
else
len = 0;
/* We allocate one more byte so we can null-terminate the string.
* This null-termination is not included in the length so it
* is not sent over the wire. */
kstr = malloc(sizeof(*kstr) + len + 1);
if (str) {
kstr->len = ntohs(len);
memcpy(kstr->str, str, len+1);
} else
kstr->len = (int16_t)ntohs(RD_KAFKAP_STR_LEN_NULL);
return kstr;
}
#define rd_kafkap_str_destroy(kstr) free(kstr)
#define rd_kafkap_strdupa(kstr) strndupa((kstr)->str,\
RD_KAFKAP_STR_SIZE((kstr)))
/**
*
* Kafka protocol bytes representation: { uint32, data.. }
*
*/
typedef struct rd_kafkap_bytes_s {
int32_t len; /* big endian */
char data[0]; /* allocated dynamically */
} RD_PACKED rd_kafkap_bytes_t;
#define RD_KAFKAP_BYTES_LEN_NULL -1
/* Returns the actual size of a kafka protocol bytes representation. */
#define RD_KAFKAP_BYTES_SIZE(kbytes) (int32_t)(sizeof((kbytes)->len) + \
((int32_t)ntohl((kbytes)->len)==\
RD_KAFKAP_BYTES_LEN_NULL ? \
0 : ntohl((kbytes)->len)))
/* Returns the length of the string of a kafka protocol bytes representation */
#define RD_KAFKAP_BYTES_LEN(kbytes) (int32_t)(((int32_t)ntohl((kbytes)->len) ==\
RD_KAFKAP_BYTES_LEN_NULL ? \
0 : \
(int32_t)ntohl((kbytes)->len)))
#define RD_KAFKAP_BYTES_IS_NULL(kbytes) \
((int32_t)ntohl((kbytes)->len) == RD_KAFKAP_STR_LEN_NULL)
static inline int rd_kafkap_bytes_cmp (const rd_kafkap_bytes_t *a,
const rd_kafkap_bytes_t *b) RD_UNUSED;
static inline int rd_kafkap_bytes_cmp (const rd_kafkap_bytes_t *a,
const rd_kafkap_bytes_t *b) {
if (a->len != b->len)
return -1;
return memcmp(a->data, b->data, ntohl(a->len));
}
static inline int rd_kafkap_bytes_cmp_bytes (const rd_kafkap_bytes_t *a,
const void *data, size_t datalen)
RD_UNUSED;
static inline int rd_kafkap_bytes_cmp_bytes (const rd_kafkap_bytes_t *a,
const void *data, size_t datalen) {
if (a->len != datalen)
return -1;
return memcmp(a->data, data, ntohl(a->len));
}
static inline rd_kafkap_bytes_t *rd_kafkap_bytes_new (const void *data,
size_t datalen) RD_UNUSED;
static inline rd_kafkap_bytes_t *rd_kafkap_bytes_new (const void *data,
size_t datalen) {
rd_kafkap_bytes_t *kbytes;
kbytes = malloc(sizeof(*kbytes) + datalen);
if (data) {
kbytes->len = ntohl(datalen);
memcpy(kbytes->data, data, datalen);
} else
kbytes->len = (int32_t)ntohl(RD_KAFKAP_BYTES_LEN_NULL);
return kbytes;
}
#define rd_kafkap_bytes_destroy(kbytes) free(kbytes)
struct rd_kafkap_FetchRequest {
int32_t ReplicaId;
int32_t MaxWaitTime;
int32_t MinBytes;
int32_t TopicArrayCnt;
} RD_PACKED;
/* Non-protocol representation of a topic's metadata. */
struct rd_kafka_TopicMetadata {
int16_t ErrorCode;
rd_kafkap_str_t *Name;
struct {
int16_t ErrorCode;
int32_t PartitionId;
int32_t Leader;
struct rd_kafka_broker_s *rkb;
int32_t *Replicas;
int32_t Replicas_cnt;
int32_t *Isr;
int32_t Isr_cnt;
} *PartitionMetadata;
int32_t PartitionMetadata_cnt;
};