github.com/confluentinc/confluent-kafka-go@v1.9.2/kafka/event.go (about) 1 /** 2 * Copyright 2016 Confluent Inc. 3 * 4 * Licensed under the Apache License, Version 2.0 (the "License"); 5 * you may not use this file except in compliance with the License. 6 * You may obtain a copy of the License at 7 * 8 * http://www.apache.org/licenses/LICENSE-2.0 9 * 10 * Unless required by applicable law or agreed to in writing, software 11 * distributed under the License is distributed on an "AS IS" BASIS, 12 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 * See the License for the specific language governing permissions and 14 * limitations under the License. 15 */ 16 17 package kafka 18 19 import ( 20 "fmt" 21 "os" 22 "unsafe" 23 ) 24 25 /* 26 #include <stdlib.h> 27 #include "select_rdkafka.h" 28 #include "glue_rdkafka.h" 29 30 31 void chdrs_to_tmphdrs (glue_msg_t *gMsg) { 32 size_t i = 0; 33 const char *name; 34 const void *val; 35 size_t size; 36 rd_kafka_headers_t *chdrs; 37 38 if (rd_kafka_message_headers(gMsg->msg, &chdrs)) { 39 gMsg->tmphdrs = NULL; 40 gMsg->tmphdrsCnt = 0; 41 return; 42 } 43 44 gMsg->tmphdrsCnt = rd_kafka_header_cnt(chdrs); 45 gMsg->tmphdrs = malloc(sizeof(*gMsg->tmphdrs) * gMsg->tmphdrsCnt); 46 47 while (!rd_kafka_header_get_all(chdrs, i, 48 &gMsg->tmphdrs[i].key, 49 &gMsg->tmphdrs[i].val, 50 (size_t *)&gMsg->tmphdrs[i].size)) 51 i++; 52 } 53 54 rd_kafka_event_t *_rk_queue_poll (rd_kafka_queue_t *rkq, int timeoutMs, 55 rd_kafka_event_type_t *evtype, 56 glue_msg_t *gMsg, 57 rd_kafka_event_t *prev_rkev) { 58 rd_kafka_event_t *rkev; 59 60 if (prev_rkev) 61 rd_kafka_event_destroy(prev_rkev); 62 63 rkev = rd_kafka_queue_poll(rkq, timeoutMs); 64 *evtype = rd_kafka_event_type(rkev); 65 66 if (*evtype == RD_KAFKA_EVENT_FETCH) { 67 gMsg->msg = (rd_kafka_message_t *)rd_kafka_event_message_next(rkev); 68 gMsg->ts = rd_kafka_message_timestamp(gMsg->msg, &gMsg->tstype); 69 70 if (gMsg->want_hdrs) 71 chdrs_to_tmphdrs(gMsg); 72 } 73 74 return rkev; 75 } 76 */ 77 import "C" 78 79 func chdrsToTmphdrs(gMsg *C.glue_msg_t) { 80 C.chdrs_to_tmphdrs(gMsg) 81 } 82 83 // Event generic interface 84 type Event interface { 85 // String returns a human-readable representation of the event 86 String() string 87 } 88 89 // Specific event types 90 91 // Stats statistics event 92 type Stats struct { 93 statsJSON string 94 } 95 96 func (e Stats) String() string { 97 return e.statsJSON 98 } 99 100 // AssignedPartitions consumer group rebalance event: assigned partition set 101 type AssignedPartitions struct { 102 Partitions []TopicPartition 103 } 104 105 func (e AssignedPartitions) String() string { 106 return fmt.Sprintf("AssignedPartitions: %v", e.Partitions) 107 } 108 109 // RevokedPartitions consumer group rebalance event: revoked partition set 110 type RevokedPartitions struct { 111 Partitions []TopicPartition 112 } 113 114 func (e RevokedPartitions) String() string { 115 return fmt.Sprintf("RevokedPartitions: %v", e.Partitions) 116 } 117 118 // PartitionEOF consumer reached end of partition 119 // Needs to be explicitly enabled by setting the `enable.partition.eof` 120 // configuration property to true. 121 type PartitionEOF TopicPartition 122 123 func (p PartitionEOF) String() string { 124 return fmt.Sprintf("EOF at %s", TopicPartition(p)) 125 } 126 127 // OffsetsCommitted reports committed offsets 128 type OffsetsCommitted struct { 129 Error error 130 Offsets []TopicPartition 131 } 132 133 func (o OffsetsCommitted) String() string { 134 return fmt.Sprintf("OffsetsCommitted (%v, %v)", o.Error, o.Offsets) 135 } 136 137 // OAuthBearerTokenRefresh indicates token refresh is required 138 type OAuthBearerTokenRefresh struct { 139 // Config is the value of the sasl.oauthbearer.config property 140 Config string 141 } 142 143 func (o OAuthBearerTokenRefresh) String() string { 144 return "OAuthBearerTokenRefresh" 145 } 146 147 // eventPoll polls an event from the handler's C rd_kafka_queue_t, 148 // translates it into an Event type and then sends on `channel` if non-nil, else returns the Event. 149 // term_chan is an optional channel to monitor along with producing to channel 150 // to indicate that `channel` is being terminated. 151 // returns (event Event, terminate Bool) tuple, where Terminate indicates 152 // if termChan received a termination event. 153 func (h *handle) eventPoll(channel chan Event, timeoutMs int, maxEvents int, termChan chan bool) (Event, bool) { 154 155 var prevRkev *C.rd_kafka_event_t 156 term := false 157 158 var retval Event 159 160 if channel == nil { 161 maxEvents = 1 162 } 163 out: 164 for evcnt := 0; evcnt < maxEvents; evcnt++ { 165 var evtype C.rd_kafka_event_type_t 166 var gMsg C.glue_msg_t 167 gMsg.want_hdrs = C.int8_t(bool2cint(h.msgFields.Headers)) 168 rkev := C._rk_queue_poll(h.rkq, C.int(timeoutMs), &evtype, &gMsg, prevRkev) 169 prevRkev = rkev 170 timeoutMs = 0 171 172 retval = nil 173 174 switch evtype { 175 case C.RD_KAFKA_EVENT_FETCH: 176 // Consumer fetch event, new message. 177 // Extracted into temporary gMsg for optimization 178 retval = h.newMessageFromGlueMsg(&gMsg) 179 180 case C.RD_KAFKA_EVENT_REBALANCE: 181 // Consumer rebalance event 182 retval = h.c.handleRebalanceEvent(channel, rkev) 183 184 case C.RD_KAFKA_EVENT_ERROR: 185 // Error event 186 cErr := C.rd_kafka_event_error(rkev) 187 if cErr == C.RD_KAFKA_RESP_ERR__PARTITION_EOF { 188 crktpar := C.rd_kafka_event_topic_partition(rkev) 189 if crktpar == nil { 190 break 191 } 192 193 defer C.rd_kafka_topic_partition_destroy(crktpar) 194 var peof PartitionEOF 195 setupTopicPartitionFromCrktpar((*TopicPartition)(&peof), crktpar) 196 197 retval = peof 198 199 } else if int(C.rd_kafka_event_error_is_fatal(rkev)) != 0 { 200 // A fatal error has been raised. 201 // Extract the actual error from the client 202 // instance and return a new Error with 203 // fatal set to true. 204 cFatalErrstrSize := C.size_t(512) 205 cFatalErrstr := (*C.char)(C.malloc(cFatalErrstrSize)) 206 defer C.free(unsafe.Pointer(cFatalErrstr)) 207 cFatalErr := C.rd_kafka_fatal_error(h.rk, cFatalErrstr, cFatalErrstrSize) 208 fatalErr := newErrorFromCString(cFatalErr, cFatalErrstr) 209 fatalErr.fatal = true 210 retval = fatalErr 211 212 } else { 213 retval = newErrorFromCString(cErr, C.rd_kafka_event_error_string(rkev)) 214 } 215 216 case C.RD_KAFKA_EVENT_STATS: 217 retval = &Stats{C.GoString(C.rd_kafka_event_stats(rkev))} 218 219 case C.RD_KAFKA_EVENT_DR: 220 // Producer Delivery Report event 221 // Each such event contains delivery reports for all 222 // messages in the produced batch. 223 // Forward delivery reports to per-message's response channel 224 // or to the global Producer.Events channel, or none. 225 rkmessages := make([]*C.rd_kafka_message_t, int(C.rd_kafka_event_message_count(rkev))) 226 227 cnt := int(C.rd_kafka_event_message_array(rkev, (**C.rd_kafka_message_t)(unsafe.Pointer(&rkmessages[0])), C.size_t(len(rkmessages)))) 228 229 for _, rkmessage := range rkmessages[:cnt] { 230 msg := h.newMessageFromC(rkmessage) 231 var ch *chan Event 232 233 if rkmessage._private != nil { 234 // Find cgoif by id 235 cg, found := h.cgoGet((int)((uintptr)(rkmessage._private))) 236 if found { 237 cdr := cg.(cgoDr) 238 239 if cdr.deliveryChan != nil { 240 ch = &cdr.deliveryChan 241 } 242 msg.Opaque = cdr.opaque 243 } 244 } 245 246 if ch == nil && h.fwdDr { 247 ch = &channel 248 } 249 250 if ch != nil { 251 select { 252 case *ch <- msg: 253 case <-termChan: 254 retval = nil 255 term = true 256 break out 257 } 258 259 } else { 260 retval = msg 261 break out 262 } 263 } 264 265 case C.RD_KAFKA_EVENT_OFFSET_COMMIT: 266 // Offsets committed 267 cErr := C.rd_kafka_event_error(rkev) 268 coffsets := C.rd_kafka_event_topic_partition_list(rkev) 269 var offsets []TopicPartition 270 if coffsets != nil { 271 offsets = newTopicPartitionsFromCparts(coffsets) 272 } 273 274 if cErr != C.RD_KAFKA_RESP_ERR_NO_ERROR { 275 retval = OffsetsCommitted{newErrorFromCString(cErr, C.rd_kafka_event_error_string(rkev)), offsets} 276 } else { 277 retval = OffsetsCommitted{nil, offsets} 278 } 279 280 case C.RD_KAFKA_EVENT_OAUTHBEARER_TOKEN_REFRESH: 281 ev := OAuthBearerTokenRefresh{C.GoString(C.rd_kafka_event_config_string(rkev))} 282 retval = ev 283 284 case C.RD_KAFKA_EVENT_NONE: 285 // poll timed out: no events available 286 break out 287 288 default: 289 if rkev != nil { 290 fmt.Fprintf(os.Stderr, "Ignored event %s\n", 291 C.GoString(C.rd_kafka_event_name(rkev))) 292 } 293 294 } 295 296 if retval != nil { 297 if channel != nil { 298 select { 299 case channel <- retval: 300 case <-termChan: 301 retval = nil 302 term = true 303 break out 304 } 305 } else { 306 break out 307 } 308 } 309 } 310 311 if prevRkev != nil { 312 C.rd_kafka_event_destroy(prevRkev) 313 } 314 315 return retval, term 316 }