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 
30 #include "rdkafka_int.h"
31 #include "rdkafka_aux.h"
32 #include "rdkafka_error.h"
33 
34 rd_kafka_resp_err_t
rd_kafka_topic_result_error(const rd_kafka_topic_result_t * topicres)35 rd_kafka_topic_result_error (const rd_kafka_topic_result_t *topicres) {
36         return topicres->err;
37 }
38 
39 const char *
rd_kafka_topic_result_error_string(const rd_kafka_topic_result_t * topicres)40 rd_kafka_topic_result_error_string (const rd_kafka_topic_result_t *topicres) {
41         return topicres->errstr;
42 }
43 
44 const char *
rd_kafka_topic_result_name(const rd_kafka_topic_result_t * topicres)45 rd_kafka_topic_result_name (const rd_kafka_topic_result_t *topicres) {
46         return topicres->topic;
47 }
48 
49 /**
50  * @brief Create new topic_result (single allocation).
51  *
52  * @param topic Topic string, if topic_size is != -1 it does not have to
53  *              be nul-terminated.
54  * @param topic_size Size of topic, or -1 to perform automatic strlen()
55  * @param err Error code
56  * @param errstr Optional error string.
57  *
58  * All input arguments are copied.
59  */
60 
61 rd_kafka_topic_result_t *
rd_kafka_topic_result_new(const char * topic,ssize_t topic_size,rd_kafka_resp_err_t err,const char * errstr)62 rd_kafka_topic_result_new (const char *topic, ssize_t topic_size,
63                            rd_kafka_resp_err_t err,
64                            const char *errstr) {
65         size_t tlen = topic_size != -1 ? (size_t)topic_size : strlen(topic);
66         size_t elen = errstr ? strlen(errstr) + 1 : 0;
67         rd_kafka_topic_result_t *terr;
68 
69         terr = rd_malloc(sizeof(*terr) + tlen + 1 + elen);
70 
71         terr->err = err;
72 
73         terr->topic = terr->data;
74         memcpy(terr->topic, topic, tlen);
75         terr->topic[tlen] = '\0';
76 
77         if (errstr) {
78                 terr->errstr = terr->topic + tlen + 1;
79                 memcpy(terr->errstr, errstr, elen);
80         } else {
81                 terr->errstr = NULL;
82         }
83 
84         return terr;
85 }
86 
87 
88 /**
89  * @brief Destroy topic_result
90  */
rd_kafka_topic_result_destroy(rd_kafka_topic_result_t * terr)91 void rd_kafka_topic_result_destroy (rd_kafka_topic_result_t *terr) {
92         rd_free(terr);
93 }
94 
95 /**
96  * @brief Destroy-variant suitable for rd_list free_cb use.
97  */
rd_kafka_topic_result_free(void * ptr)98 void rd_kafka_topic_result_free (void *ptr) {
99         rd_kafka_topic_result_destroy((rd_kafka_topic_result_t *)ptr);
100 }
101 
102 const rd_kafka_error_t *
rd_kafka_group_result_error(const rd_kafka_group_result_t * groupres)103 rd_kafka_group_result_error (const rd_kafka_group_result_t *groupres) {
104         return groupres->error;
105 }
106 
107 const char *
rd_kafka_group_result_name(const rd_kafka_group_result_t * groupres)108 rd_kafka_group_result_name (const rd_kafka_group_result_t *groupres) {
109         return groupres->group;
110 }
111 
112 const rd_kafka_topic_partition_list_t *
rd_kafka_group_result_partitions(const rd_kafka_group_result_t * groupres)113 rd_kafka_group_result_partitions (const rd_kafka_group_result_t *groupres) {
114         return groupres->partitions;
115 }
116 
117 rd_kafka_group_result_t *
rd_kafka_group_result_copy(const rd_kafka_group_result_t * groupres)118 rd_kafka_group_result_copy (const rd_kafka_group_result_t *groupres) {
119         return rd_kafka_group_result_new(groupres->group,
120                                          -1,
121                                          groupres->partitions,
122                                          groupres->error ?
123                                          rd_kafka_error_copy(groupres->error) :
124                                          NULL);
125 }
126 
127 /**
128  * @brief Same as rd_kafka_group_result_copy() but suitable for
129  *        rd_list_copy(). The \p opaque is ignored.
130  */
131 void *
rd_kafka_group_result_copy_opaque(const void * src_groupres,void * opaque)132 rd_kafka_group_result_copy_opaque (const void *src_groupres,
133                                    void *opaque) {
134        return rd_kafka_group_result_copy(src_groupres);
135 }
136 
137 
138 /**
139  * @brief Create new group_result (single allocation).
140  *
141  * @param group Group string, if group_size is != -1 it does not have to
142  *              be nul-terminated.
143  * @param group_size Size of group, or -1 to perform automatic strlen()
144  * @param error Error object, or NULL on success. Takes ownership of \p error.
145  *
146  * All input arguments are copied.
147  */
148 
149 rd_kafka_group_result_t *
rd_kafka_group_result_new(const char * group,ssize_t group_size,const rd_kafka_topic_partition_list_t * partitions,rd_kafka_error_t * error)150 rd_kafka_group_result_new (const char *group, ssize_t group_size,
151                            const rd_kafka_topic_partition_list_t *partitions,
152                            rd_kafka_error_t *error) {
153         size_t glen = group_size != -1 ? (size_t)group_size : strlen(group);
154         rd_kafka_group_result_t *groupres;
155 
156         groupres = rd_calloc(1, sizeof(*groupres) + glen + 1);
157 
158 
159         groupres->group = groupres->data;
160         memcpy(groupres->group, group, glen);
161         groupres->group[glen] = '\0';
162 
163         if (partitions)
164                 groupres->partitions = rd_kafka_topic_partition_list_copy(
165                         partitions);
166 
167         groupres->error = error;
168 
169         return groupres;
170 }
171 
172 
173  /**
174  * @brief Destroy group_result
175  */
rd_kafka_group_result_destroy(rd_kafka_group_result_t * groupres)176 void rd_kafka_group_result_destroy (rd_kafka_group_result_t *groupres) {
177         if (groupres->partitions)
178                 rd_kafka_topic_partition_list_destroy(groupres->partitions);
179         if (groupres->error)
180                 rd_kafka_error_destroy(groupres->error);
181         rd_free(groupres);
182 }
183 
184  /**
185  * @brief Destroy-variant suitable for rd_list free_cb use.
186  */
rd_kafka_group_result_free(void * ptr)187 void rd_kafka_group_result_free (void *ptr) {
188         rd_kafka_group_result_destroy((rd_kafka_group_result_t *)ptr);
189 }
190