1/*
2 * librdkafka - Apache Kafka C library
3 *
4 * Copyright (c) 2018 Magnus Edenhill
5 * All rights reserved.
6 *
7 * Redistribution and use in source and binary forms, with or without
8 * modification, are permitted provided that the following conditions are met:
9 *
10 * 1. Redistributions of source code must retain the above copyright notice,
11 * this list of conditions and the following disclaimer.
12 * 2. Redistributions in binary form must reproduce the above copyright notice,
13 * this list of conditions and the following disclaimer in the documentation
14 * and/or other materials provided with the distribution.
15 *
16 * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS"
17 * AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE
18 * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE
19 * ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE
20 * LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR
21 * CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF
22 * SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS
23 * INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN
24 * CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE)
25 * ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE
26 * POSSIBILITY OF SUCH DAMAGE.
27 */
28
29#ifndef _RDKAFKA_ADMIN_H_
30#define _RDKAFKA_ADMIN_H_
31
32
33#include "rdstring.h"
34#include "rdkafka_confval.h"
35
36
37
38/**
39 * @brief Common AdminOptions type used for all admin APIs.
40 *
41 * @remark Visit AdminOptions_use() when you change this struct
42 * to make sure it is copied properly.
43 */
44struct rd_kafka_AdminOptions_s {
45 rd_kafka_admin_op_t for_api; /**< Limit allowed options to
46 * this API (optional) */
47
48 /* Generic */
49 rd_kafka_confval_t request_timeout;/**< I32: Full request timeout,
50 * includes looking up leader
51 * broker,
52 * waiting for req/response,
53 * etc. */
54 rd_ts_t abs_timeout; /**< Absolute timeout calculated
55 * from .timeout */
56
57 /* Specific for one or more APIs */
58 rd_kafka_confval_t operation_timeout; /**< I32: Timeout on broker.
59 * Valid for:
60 * CreateTopics
61 * DeleteTopics
62 */
63 rd_kafka_confval_t validate_only; /**< BOOL: Only validate (on broker),
64 * but don't perform action.
65 * Valid for:
66 * CreateTopics
67 * CreatePartitions
68 * AlterConfigs
69 */
70
71 rd_kafka_confval_t incremental; /**< BOOL: Incremental rather than
72 * absolute application
73 * of config.
74 * Valid for:
75 * AlterConfigs
76 */
77
78 rd_kafka_confval_t broker; /**< INT: Explicitly override
79 * broker id to send
80 * requests to.
81 * Valid for:
82 * all
83 */
84
85 rd_kafka_confval_t opaque; /**< PTR: Application opaque.
86 * Valid for all. */
87};
88
89
90
91
92
93/**
94 * @name CreateTopics
95 * @{
96 */
97
98/**
99 * @brief NewTopic type, used with CreateTopics.
100 */
101struct rd_kafka_NewTopic_s {
102 /* Required */
103 char *topic; /**< Topic to be created */
104 int num_partitions; /**< Number of partitions to create */
105 int replication_factor; /**< Replication factor */
106
107 /* Optional */
108 rd_list_t replicas; /**< Type (rd_list_t (int32_t)):
109 * Array of replica lists indexed by
110 * partition, size num_partitions. */
111 rd_list_t config; /**< Type (rd_kafka_ConfigEntry_t *):
112 * List of configuration entries */
113};
114
115/**@}*/
116
117
118/**
119 * @name DeleteTopics
120 * @{
121 */
122
123/**
124 * @brief DeleteTopics result
125 */
126struct rd_kafka_DeleteTopics_result_s {
127 rd_list_t topics; /**< Type (rd_kafka_topic_result_t *) */
128};
129
130struct rd_kafka_DeleteTopic_s {
131 char *topic; /**< Points to data */
132 char data[1]; /**< The topic name is allocated along with
133 * the struct here. */
134};
135
136/**@}*/
137
138
139
140/**
141 * @name CreatePartitions
142 * @{
143 */
144
145
146/**
147 * @brief CreatePartitions result
148 */
149struct rd_kafka_CreatePartitions_result_s {
150 rd_list_t topics; /**< Type (rd_kafka_topic_result_t *) */
151};
152
153struct rd_kafka_NewPartitions_s {
154 char *topic; /**< Points to data */
155 size_t total_cnt; /**< New total partition count */
156
157 /* Optional */
158 rd_list_t replicas; /**< Type (rd_list_t (int32_t)):
159 * Array of replica lists indexed by
160 * new partition relative index.
161 * Size is dynamic since we don't
162 * know how many partitions are actually
163 * being added by total_cnt */
164
165 char data[1]; /**< The topic name is allocated along with
166 * the struct here. */
167};
168
169/**@}*/
170
171
172
173/**
174 * @name ConfigEntry
175 * @{
176 */
177
178/* KIP-248 */
179typedef enum rd_kafka_AlterOperation_t {
180 RD_KAFKA_ALTER_OP_ADD = 0,
181 RD_KAFKA_ALTER_OP_SET = 1,
182 RD_KAFKA_ALTER_OP_DELETE = 2,
183} rd_kafka_AlterOperation_t;
184
185struct rd_kafka_ConfigEntry_s {
186 rd_strtup_t *kv; /**< Name/Value pair */
187
188 /* Response */
189
190 /* Attributes: this is a struct for easy copying */
191 struct {
192 rd_kafka_AlterOperation_t operation; /**< Operation */
193 rd_kafka_ConfigSource_t source; /**< Config source */
194 rd_bool_t is_readonly; /**< Value is read-only (on broker) */
195 rd_bool_t is_default; /**< Value is at its default */
196 rd_bool_t is_sensitive; /**< Value is sensitive */
197 rd_bool_t is_synonym; /**< Value is synonym */
198 } a;
199
200 rd_list_t synonyms; /**< Type (rd_kafka_configEntry *) */
201};
202
203/**
204 * @brief A cluster ConfigResource constisting of:
205 * - resource type (BROKER, TOPIC)
206 * - configuration property name
207 * - configuration property value
208 *
209 * https://cwiki.apache.org/confluence/display/KAFKA/KIP-133%3A+Describe+and+Alter+Configs+Admin+APIs
210 */
211struct rd_kafka_ConfigResource_s {
212 rd_kafka_ResourceType_t restype; /**< Resource type */
213 char *name; /**< Resource name, points to .data*/
214 rd_list_t config; /**< Type (rd_kafka_ConfigEntry_t *):
215 * List of config props */
216
217 /* Response */
218 rd_kafka_resp_err_t err; /**< Response error code */
219 char *errstr; /**< Response error string */
220
221 char data[1]; /**< The name is allocated along with
222 * the struct here. */
223};
224
225
226
227
228/**@}*/
229
230/**
231 * @name AlterConfigs
232 * @{
233 */
234
235
236
237
238struct rd_kafka_AlterConfigs_result_s {
239 rd_list_t resources; /**< Type (rd_kafka_ConfigResource_t *) */
240};
241
242struct rd_kafka_ConfigResource_result_s {
243 rd_list_t resources; /**< Type (struct rd_kafka_ConfigResource *):
244 * List of config resources, sans config
245 * but with response error values. */
246};
247
248/**@}*/
249
250
251
252/**
253 * @name DescribeConfigs
254 * @{
255 */
256
257struct rd_kafka_DescribeConfigs_result_s {
258 rd_list_t configs; /**< Type (rd_kafka_ConfigResource_t *) */
259};
260
261/**@}*/
262
263
264/**@}*/
265
266#endif /* _RDKAFKA_ADMIN_H_ */
267