| 1 | /* |
| 2 | * librdkafka - Apache Kafka C library |
| 3 | * |
| 4 | * Copyright (c) 2012-2015, 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 | #ifndef _RDKAFKA_REQUEST_H_ |
| 29 | #define _RDKAFKA_REQUEST_H_ |
| 30 | |
| 31 | #include "rdkafka_cgrp.h" |
| 32 | #include "rdkafka_feature.h" |
| 33 | |
| 34 | |
| 35 | #define RD_KAFKA_ERR_ACTION_PERMANENT 0x1 /* Permanent error */ |
| 36 | #define RD_KAFKA_ERR_ACTION_IGNORE 0x2 /* Error can be ignored */ |
| 37 | #define RD_KAFKA_ERR_ACTION_REFRESH 0x4 /* Refresh state (e.g., metadata) */ |
| 38 | #define RD_KAFKA_ERR_ACTION_RETRY 0x8 /* Retry request after backoff */ |
| 39 | #define RD_KAFKA_ERR_ACTION_INFORM 0x10 /* Inform application about err */ |
| 40 | #define RD_KAFKA_ERR_ACTION_SPECIAL 0x20 /* Special-purpose, depends on context */ |
| 41 | #define RD_KAFKA_ERR_ACTION_MSG_NOT_PERSISTED 0x40 /* ProduceReq msg status */ |
| 42 | #define RD_KAFKA_ERR_ACTION_MSG_POSSIBLY_PERSISTED 0x80 /* ProduceReq msg status */ |
| 43 | #define RD_KAFKA_ERR_ACTION_MSG_PERSISTED 0x100 /* ProduceReq msg status */ |
| 44 | #define RD_KAFKA_ERR_ACTION_END 0 /* var-arg sentinel */ |
| 45 | |
| 46 | /** @macro bitmask of the message persistence flags */ |
| 47 | #define RD_KAFKA_ERR_ACTION_MSG_FLAGS \ |
| 48 | (RD_KAFKA_ERR_ACTION_MSG_NOT_PERSISTED | \ |
| 49 | RD_KAFKA_ERR_ACTION_MSG_POSSIBLY_PERSISTED | \ |
| 50 | RD_KAFKA_ERR_ACTION_MSG_PERSISTED) |
| 51 | |
| 52 | int rd_kafka_err_action (rd_kafka_broker_t *rkb, |
| 53 | rd_kafka_resp_err_t err, |
| 54 | const rd_kafka_buf_t *request, ...); |
| 55 | |
| 56 | |
| 57 | void rd_kafka_GroupCoordinatorRequest (rd_kafka_broker_t *rkb, |
| 58 | const rd_kafkap_str_t *cgrp, |
| 59 | rd_kafka_replyq_t replyq, |
| 60 | rd_kafka_resp_cb_t *resp_cb, |
| 61 | void *opaque); |
| 62 | |
| 63 | rd_kafka_resp_err_t rd_kafka_handle_Offset (rd_kafka_t *rk, |
| 64 | rd_kafka_broker_t *rkb, |
| 65 | rd_kafka_resp_err_t err, |
| 66 | rd_kafka_buf_t *rkbuf, |
| 67 | rd_kafka_buf_t *request, |
| 68 | rd_kafka_topic_partition_list_t |
| 69 | *offsets); |
| 70 | |
| 71 | void rd_kafka_OffsetRequest (rd_kafka_broker_t *rkb, |
| 72 | rd_kafka_topic_partition_list_t *offsets, |
| 73 | int16_t api_version, |
| 74 | rd_kafka_replyq_t replyq, |
| 75 | rd_kafka_resp_cb_t *resp_cb, |
| 76 | void *opaque); |
| 77 | |
| 78 | rd_kafka_resp_err_t |
| 79 | rd_kafka_handle_OffsetFetch (rd_kafka_t *rk, |
| 80 | rd_kafka_broker_t *rkb, |
| 81 | rd_kafka_resp_err_t err, |
| 82 | rd_kafka_buf_t *rkbuf, |
| 83 | rd_kafka_buf_t *request, |
| 84 | rd_kafka_topic_partition_list_t *offsets, |
| 85 | int update_toppar); |
| 86 | |
| 87 | void rd_kafka_op_handle_OffsetFetch (rd_kafka_t *rk, |
| 88 | rd_kafka_broker_t *rkb, |
| 89 | rd_kafka_resp_err_t err, |
| 90 | rd_kafka_buf_t *rkbuf, |
| 91 | rd_kafka_buf_t *request, |
| 92 | void *opaque); |
| 93 | |
| 94 | void rd_kafka_OffsetFetchRequest (rd_kafka_broker_t *rkb, |
| 95 | int16_t api_version, |
| 96 | rd_kafka_topic_partition_list_t *parts, |
| 97 | rd_kafka_replyq_t replyq, |
| 98 | rd_kafka_resp_cb_t *resp_cb, |
| 99 | void *opaque); |
| 100 | |
| 101 | |
| 102 | |
| 103 | rd_kafka_resp_err_t |
| 104 | rd_kafka_handle_OffsetCommit (rd_kafka_t *rk, |
| 105 | rd_kafka_broker_t *rkb, |
| 106 | rd_kafka_resp_err_t err, |
| 107 | rd_kafka_buf_t *rkbuf, |
| 108 | rd_kafka_buf_t *request, |
| 109 | rd_kafka_topic_partition_list_t *offsets); |
| 110 | int rd_kafka_OffsetCommitRequest (rd_kafka_broker_t *rkb, |
| 111 | rd_kafka_cgrp_t *rkcg, |
| 112 | int16_t api_version, |
| 113 | rd_kafka_topic_partition_list_t *offsets, |
| 114 | rd_kafka_replyq_t replyq, |
| 115 | rd_kafka_resp_cb_t *resp_cb, |
| 116 | void *opaque, const char *reason); |
| 117 | |
| 118 | |
| 119 | |
| 120 | void rd_kafka_JoinGroupRequest (rd_kafka_broker_t *rkb, |
| 121 | const rd_kafkap_str_t *group_id, |
| 122 | const rd_kafkap_str_t *member_id, |
| 123 | const rd_kafkap_str_t *protocol_type, |
| 124 | const rd_list_t *topics, |
| 125 | rd_kafka_replyq_t replyq, |
| 126 | rd_kafka_resp_cb_t *resp_cb, |
| 127 | void *opaque); |
| 128 | |
| 129 | |
| 130 | void rd_kafka_LeaveGroupRequest (rd_kafka_broker_t *rkb, |
| 131 | const rd_kafkap_str_t *group_id, |
| 132 | const rd_kafkap_str_t *member_id, |
| 133 | rd_kafka_replyq_t replyq, |
| 134 | rd_kafka_resp_cb_t *resp_cb, |
| 135 | void *opaque); |
| 136 | void rd_kafka_handle_LeaveGroup (rd_kafka_t *rk, |
| 137 | rd_kafka_broker_t *rkb, |
| 138 | rd_kafka_resp_err_t err, |
| 139 | rd_kafka_buf_t *rkbuf, |
| 140 | rd_kafka_buf_t *request, |
| 141 | void *opaque); |
| 142 | |
| 143 | void rd_kafka_SyncGroupRequest (rd_kafka_broker_t *rkb, |
| 144 | const rd_kafkap_str_t *group_id, |
| 145 | int32_t generation_id, |
| 146 | const rd_kafkap_str_t *member_id, |
| 147 | const rd_kafka_group_member_t |
| 148 | *assignments, |
| 149 | int assignment_cnt, |
| 150 | rd_kafka_replyq_t replyq, |
| 151 | rd_kafka_resp_cb_t *resp_cb, |
| 152 | void *opaque); |
| 153 | void rd_kafka_handle_SyncGroup (rd_kafka_t *rk, |
| 154 | rd_kafka_broker_t *rkb, |
| 155 | rd_kafka_resp_err_t err, |
| 156 | rd_kafka_buf_t *rkbuf, |
| 157 | rd_kafka_buf_t *request, |
| 158 | void *opaque); |
| 159 | |
| 160 | void rd_kafka_ListGroupsRequest (rd_kafka_broker_t *rkb, |
| 161 | rd_kafka_replyq_t replyq, |
| 162 | rd_kafka_resp_cb_t *resp_cb, |
| 163 | void *opaque); |
| 164 | |
| 165 | void rd_kafka_DescribeGroupsRequest (rd_kafka_broker_t *rkb, |
| 166 | const char **groups, int group_cnt, |
| 167 | rd_kafka_replyq_t replyq, |
| 168 | rd_kafka_resp_cb_t *resp_cb, |
| 169 | void *opaque); |
| 170 | |
| 171 | |
| 172 | void rd_kafka_HeartbeatRequest (rd_kafka_broker_t *rkb, |
| 173 | const rd_kafkap_str_t *group_id, |
| 174 | int32_t generation_id, |
| 175 | const rd_kafkap_str_t *member_id, |
| 176 | rd_kafka_replyq_t replyq, |
| 177 | rd_kafka_resp_cb_t *resp_cb, |
| 178 | void *opaque); |
| 179 | |
| 180 | rd_kafka_resp_err_t |
| 181 | rd_kafka_MetadataRequest (rd_kafka_broker_t *rkb, |
| 182 | const rd_list_t *topics, const char *reason, |
| 183 | rd_kafka_op_t *rko); |
| 184 | |
| 185 | rd_kafka_resp_err_t |
| 186 | rd_kafka_handle_ApiVersion (rd_kafka_t *rk, |
| 187 | rd_kafka_broker_t *rkb, |
| 188 | rd_kafka_resp_err_t err, |
| 189 | rd_kafka_buf_t *rkbuf, |
| 190 | rd_kafka_buf_t *request, |
| 191 | struct rd_kafka_ApiVersion **apis, |
| 192 | size_t *api_cnt); |
| 193 | void rd_kafka_ApiVersionRequest (rd_kafka_broker_t *rkb, |
| 194 | rd_kafka_replyq_t replyq, |
| 195 | rd_kafka_resp_cb_t *resp_cb, |
| 196 | void *opaque); |
| 197 | |
| 198 | void rd_kafka_SaslHandshakeRequest (rd_kafka_broker_t *rkb, |
| 199 | const char *mechanism, |
| 200 | rd_kafka_replyq_t replyq, |
| 201 | rd_kafka_resp_cb_t *resp_cb, |
| 202 | void *opaque); |
| 203 | |
| 204 | int rd_kafka_ProduceRequest (rd_kafka_broker_t *rkb, rd_kafka_toppar_t *rktp, |
| 205 | const rd_kafka_pid_t pid); |
| 206 | |
| 207 | rd_kafka_resp_err_t |
| 208 | rd_kafka_CreateTopicsRequest (rd_kafka_broker_t *rkb, |
| 209 | const rd_list_t *new_topics /*(NewTopic_t*)*/, |
| 210 | rd_kafka_AdminOptions_t *options, |
| 211 | char *errstr, size_t errstr_size, |
| 212 | rd_kafka_replyq_t replyq, |
| 213 | rd_kafka_resp_cb_t *resp_cb, |
| 214 | void *opaque); |
| 215 | |
| 216 | rd_kafka_resp_err_t |
| 217 | rd_kafka_DeleteTopicsRequest (rd_kafka_broker_t *rkb, |
| 218 | const rd_list_t *del_topics /*(DeleteTopic_t*)*/, |
| 219 | rd_kafka_AdminOptions_t *options, |
| 220 | char *errstr, size_t errstr_size, |
| 221 | rd_kafka_replyq_t replyq, |
| 222 | rd_kafka_resp_cb_t *resp_cb, |
| 223 | void *opaque); |
| 224 | |
| 225 | rd_kafka_resp_err_t |
| 226 | rd_kafka_CreatePartitionsRequest (rd_kafka_broker_t *rkb, |
| 227 | const rd_list_t *new_parts /*(NewPartitions_t*)*/, |
| 228 | rd_kafka_AdminOptions_t *options, |
| 229 | char *errstr, size_t errstr_size, |
| 230 | rd_kafka_replyq_t replyq, |
| 231 | rd_kafka_resp_cb_t *resp_cb, |
| 232 | void *opaque); |
| 233 | |
| 234 | rd_kafka_resp_err_t |
| 235 | rd_kafka_AlterConfigsRequest (rd_kafka_broker_t *rkb, |
| 236 | const rd_list_t *configs /*(ConfigResource_t*)*/, |
| 237 | rd_kafka_AdminOptions_t *options, |
| 238 | char *errstr, size_t errstr_size, |
| 239 | rd_kafka_replyq_t replyq, |
| 240 | rd_kafka_resp_cb_t *resp_cb, |
| 241 | void *opaque); |
| 242 | |
| 243 | rd_kafka_resp_err_t |
| 244 | rd_kafka_DescribeConfigsRequest (rd_kafka_broker_t *rkb, |
| 245 | const rd_list_t *configs /*(ConfigResource_t*)*/, |
| 246 | rd_kafka_AdminOptions_t *options, |
| 247 | char *errstr, size_t errstr_size, |
| 248 | rd_kafka_replyq_t replyq, |
| 249 | rd_kafka_resp_cb_t *resp_cb, |
| 250 | void *opaque); |
| 251 | |
| 252 | void |
| 253 | rd_kafka_handle_InitProducerId (rd_kafka_t *rk, |
| 254 | rd_kafka_broker_t *rkb, |
| 255 | rd_kafka_resp_err_t err, |
| 256 | rd_kafka_buf_t *rkbuf, |
| 257 | rd_kafka_buf_t *request, |
| 258 | void *opaque); |
| 259 | |
| 260 | rd_kafka_resp_err_t |
| 261 | rd_kafka_InitProducerIdRequest (rd_kafka_broker_t *rkb, |
| 262 | const char *transactional_id, |
| 263 | int transaction_timeout_ms, |
| 264 | char *errstr, size_t errstr_size, |
| 265 | rd_kafka_replyq_t replyq, |
| 266 | rd_kafka_resp_cb_t *resp_cb, |
| 267 | void *opaque); |
| 268 | |
| 269 | |
| 270 | int unittest_request (void); |
| 271 | |
| 272 | #endif /* _RDKAFKA_REQUEST_H_ */ |
| 273 | |