File: event.go

package info (click to toggle)
golang-github-confluentinc-confluent-kafka-go 0.11.6-1
  • links: PTS, VCS
  • area: main
  • in suites: bookworm, bullseye, buster, forky, sid, trixie
  • size: 604 kB
  • sloc: sh: 23; python: 15; ansic: 13; makefile: 9
file content (330 lines) | stat: -rw-r--r-- 9,116 bytes parent folder | download
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
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
package kafka

/**
 * Copyright 2016 Confluent Inc.
 *
 * Licensed under the Apache License, Version 2.0 (the "License");
 * you may not use this file except in compliance with the License.
 * You may obtain a copy of the License at
 *
 * http://www.apache.org/licenses/LICENSE-2.0
 *
 * Unless required by applicable law or agreed to in writing, software
 * distributed under the License is distributed on an "AS IS" BASIS,
 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 * See the License for the specific language governing permissions and
 * limitations under the License.
 */

import (
	"fmt"
	"os"
	"unsafe"
)

/*
#include <stdlib.h>
#include <librdkafka/rdkafka.h>
#include "glue_rdkafka.h"


#ifdef RD_KAFKA_V_HEADERS
void chdrs_to_tmphdrs (rd_kafka_headers_t *chdrs, tmphdr_t *tmphdrs) {
   size_t i = 0;
   const char *name;
   const void *val;
   size_t size;

   while (!rd_kafka_header_get_all(chdrs, i,
                                   &tmphdrs[i].key,
                                   &tmphdrs[i].val,
                                   (size_t *)&tmphdrs[i].size))
     i++;
}
#endif

rd_kafka_event_t *_rk_queue_poll (rd_kafka_queue_t *rkq, int timeoutMs,
                                  rd_kafka_event_type_t *evtype,
                                  fetched_c_msg_t *fcMsg,
                                  rd_kafka_event_t *prev_rkev) {
    rd_kafka_event_t *rkev;

    if (prev_rkev)
      rd_kafka_event_destroy(prev_rkev);

    rkev = rd_kafka_queue_poll(rkq, timeoutMs);
    *evtype = rd_kafka_event_type(rkev);

    if (*evtype == RD_KAFKA_EVENT_FETCH) {
#ifdef RD_KAFKA_V_HEADERS
        rd_kafka_headers_t *hdrs;
#endif

        fcMsg->msg = (rd_kafka_message_t *)rd_kafka_event_message_next(rkev);
        fcMsg->ts = rd_kafka_message_timestamp(fcMsg->msg, &fcMsg->tstype);

#ifdef RD_KAFKA_V_HEADERS
        if (!rd_kafka_message_headers(fcMsg->msg, &hdrs)) {
           fcMsg->tmphdrsCnt = rd_kafka_header_cnt(hdrs);
           fcMsg->tmphdrs = malloc(sizeof(*fcMsg->tmphdrs) * fcMsg->tmphdrsCnt);
           chdrs_to_tmphdrs(hdrs, fcMsg->tmphdrs);
        } else {
#else
        if (1) {
#endif
           fcMsg->tmphdrs = NULL;
           fcMsg->tmphdrsCnt = 0;
        }
    }
    return rkev;
}
*/
import "C"

// Event generic interface
type Event interface {
	// String returns a human-readable representation of the event
	String() string
}

// Specific event types

// Stats statistics event
type Stats struct {
	statsJSON string
}

func (e Stats) String() string {
	return e.statsJSON
}

// AssignedPartitions consumer group rebalance event: assigned partition set
type AssignedPartitions struct {
	Partitions []TopicPartition
}

func (e AssignedPartitions) String() string {
	return fmt.Sprintf("AssignedPartitions: %v", e.Partitions)
}

// RevokedPartitions consumer group rebalance event: revoked partition set
type RevokedPartitions struct {
	Partitions []TopicPartition
}

func (e RevokedPartitions) String() string {
	return fmt.Sprintf("RevokedPartitions: %v", e.Partitions)
}

// PartitionEOF consumer reached end of partition
type PartitionEOF TopicPartition

func (p PartitionEOF) String() string {
	return fmt.Sprintf("EOF at %s", TopicPartition(p))
}

// OffsetsCommitted reports committed offsets
type OffsetsCommitted struct {
	Error   error
	Offsets []TopicPartition
}

func (o OffsetsCommitted) String() string {
	return fmt.Sprintf("OffsetsCommitted (%v, %v)", o.Error, o.Offsets)
}

// eventPoll polls an event from the handler's C rd_kafka_queue_t,
// translates it into an Event type and then sends on `channel` if non-nil, else returns the Event.
// term_chan is an optional channel to monitor along with producing to channel
// to indicate that `channel` is being terminated.
// returns (event Event, terminate Bool) tuple, where Terminate indicates
// if termChan received a termination event.
func (h *handle) eventPoll(channel chan Event, timeoutMs int, maxEvents int, termChan chan bool) (Event, bool) {

	var prevRkev *C.rd_kafka_event_t
	term := false

	var retval Event

	if channel == nil {
		maxEvents = 1
	}
out:
	for evcnt := 0; evcnt < maxEvents; evcnt++ {
		var evtype C.rd_kafka_event_type_t
		var fcMsg C.fetched_c_msg_t
		rkev := C._rk_queue_poll(h.rkq, C.int(timeoutMs), &evtype, &fcMsg, prevRkev)
		prevRkev = rkev
		timeoutMs = 0

		retval = nil

		switch evtype {
		case C.RD_KAFKA_EVENT_FETCH:
			// Consumer fetch event, new message.
			// Extracted into temporary fcMsg for optimization
			retval = h.newMessageFromFcMsg(&fcMsg)

		case C.RD_KAFKA_EVENT_REBALANCE:
			// Consumer rebalance event
			// If the app provided a RebalanceCb to Subscribe*() or
			// has go.application.rebalance.enable=true we create an event
			// and forward it to the application thru the RebalanceCb or the
			// Events channel respectively.
			// Since librdkafka requires the rebalance event to be "acked" by
			// the application to synchronize state we keep track of if the
			// application performed Assign() or Unassign(), but this only works for
			// the non-channel case. For the channel case we assume the application
			// calls Assign() / Unassign().
			// Failure to do so will "hang" the consumer, e.g., it wont start consuming
			// and it wont close cleanly, so this error case should be visible
			// immediately to the application developer.
			appReassigned := false
			if C.rd_kafka_event_error(rkev) == C.RD_KAFKA_RESP_ERR__ASSIGN_PARTITIONS {
				if h.currAppRebalanceEnable {
					// Application must perform Assign() call
					var ev AssignedPartitions
					ev.Partitions = newTopicPartitionsFromCparts(C.rd_kafka_event_topic_partition_list(rkev))
					if channel != nil || h.c.rebalanceCb == nil {
						retval = ev
						appReassigned = true
					} else {
						appReassigned = h.c.rebalance(ev)
					}
				}

				if !appReassigned {
					C.rd_kafka_assign(h.rk, C.rd_kafka_event_topic_partition_list(rkev))
				}
			} else {
				if h.currAppRebalanceEnable {
					// Application must perform Unassign() call
					var ev RevokedPartitions
					ev.Partitions = newTopicPartitionsFromCparts(C.rd_kafka_event_topic_partition_list(rkev))
					if channel != nil || h.c.rebalanceCb == nil {
						retval = ev
						appReassigned = true
					} else {
						appReassigned = h.c.rebalance(ev)
					}
				}

				if !appReassigned {
					C.rd_kafka_assign(h.rk, nil)
				}
			}

		case C.RD_KAFKA_EVENT_ERROR:
			// Error event
			cErr := C.rd_kafka_event_error(rkev)
			switch cErr {
			case C.RD_KAFKA_RESP_ERR__PARTITION_EOF:
				crktpar := C.rd_kafka_event_topic_partition(rkev)
				if crktpar == nil {
					break
				}

				defer C.rd_kafka_topic_partition_destroy(crktpar)
				var peof PartitionEOF
				setupTopicPartitionFromCrktpar((*TopicPartition)(&peof), crktpar)

				retval = peof
			default:
				retval = newErrorFromCString(cErr, C.rd_kafka_event_error_string(rkev))
			}

		case C.RD_KAFKA_EVENT_STATS:
			retval = &Stats{C.GoString(C.rd_kafka_event_stats(rkev))}

		case C.RD_KAFKA_EVENT_DR:
			// Producer Delivery Report event
			// Each such event contains delivery reports for all
			// messages in the produced batch.
			// Forward delivery reports to per-message's response channel
			// or to the global Producer.Events channel, or none.
			rkmessages := make([]*C.rd_kafka_message_t, int(C.rd_kafka_event_message_count(rkev)))

			cnt := int(C.rd_kafka_event_message_array(rkev, (**C.rd_kafka_message_t)(unsafe.Pointer(&rkmessages[0])), C.size_t(len(rkmessages))))

			for _, rkmessage := range rkmessages[:cnt] {
				msg := h.newMessageFromC(rkmessage)
				var ch *chan Event

				if rkmessage._private != nil {
					// Find cgoif by id
					cg, found := h.cgoGet((int)((uintptr)(rkmessage._private)))
					if found {
						cdr := cg.(cgoDr)

						if cdr.deliveryChan != nil {
							ch = &cdr.deliveryChan
						}
						msg.Opaque = cdr.opaque
					}
				}

				if ch == nil && h.fwdDr {
					ch = &channel
				}

				if ch != nil {
					select {
					case *ch <- msg:
					case <-termChan:
						break out
					}

				} else {
					retval = msg
					break out
				}
			}

		case C.RD_KAFKA_EVENT_OFFSET_COMMIT:
			// Offsets committed
			cErr := C.rd_kafka_event_error(rkev)
			coffsets := C.rd_kafka_event_topic_partition_list(rkev)
			var offsets []TopicPartition
			if coffsets != nil {
				offsets = newTopicPartitionsFromCparts(coffsets)
			}

			if cErr != C.RD_KAFKA_RESP_ERR_NO_ERROR {
				retval = OffsetsCommitted{newErrorFromCString(cErr, C.rd_kafka_event_error_string(rkev)), offsets}
			} else {
				retval = OffsetsCommitted{nil, offsets}
			}

		case C.RD_KAFKA_EVENT_NONE:
			// poll timed out: no events available
			break out

		default:
			if rkev != nil {
				fmt.Fprintf(os.Stderr, "Ignored event %s\n",
					C.GoString(C.rd_kafka_event_name(rkev)))
			}

		}

		if retval != nil {
			if channel != nil {
				select {
				case channel <- retval:
				case <-termChan:
					retval = nil
					term = true
					break out
				}
			} else {
				break out
			}
		}
	}

	if prevRkev != nil {
		C.rd_kafka_event_destroy(prevRkev)
	}

	return retval, term
}