-
Notifications
You must be signed in to change notification settings - Fork 3.2k
/
TopicImpl.cpp
124 lines (103 loc) · 4.9 KB
/
TopicImpl.cpp
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
/*
* librdkafka - Apache Kafka C/C++ library
*
* Copyright (c) 2014-2022, Magnus Edenhill
* All rights reserved.
*
* Redistribution and use in source and binary forms, with or without
* modification, are permitted provided that the following conditions are met:
*
* 1. Redistributions of source code must retain the above copyright notice,
* this list of conditions and the following disclaimer.
* 2. Redistributions in binary form must reproduce the above copyright notice,
* this list of conditions and the following disclaimer in the documentation
* and/or other materials provided with the distribution.
*
* THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS"
* AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE
* IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE
* ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE
* LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR
* CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF
* SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS
* INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN
* CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE)
* ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE
* POSSIBILITY OF SUCH DAMAGE.
*/
#include <iostream>
#include <string>
#include <list>
#include <cerrno>
#include "rdkafkacpp_int.h"
const int32_t RdKafka::Topic::PARTITION_UA = RD_KAFKA_PARTITION_UA;
const int64_t RdKafka::Topic::OFFSET_BEGINNING = RD_KAFKA_OFFSET_BEGINNING;
const int64_t RdKafka::Topic::OFFSET_END = RD_KAFKA_OFFSET_END;
const int64_t RdKafka::Topic::OFFSET_STORED = RD_KAFKA_OFFSET_STORED;
const int64_t RdKafka::Topic::OFFSET_INVALID = RD_KAFKA_OFFSET_INVALID;
RdKafka::Topic::~Topic() {
}
static int32_t partitioner_cb_trampoline(const rd_kafka_topic_t *rkt,
const void *keydata,
size_t keylen,
int32_t partition_cnt,
void *rkt_opaque,
void *msg_opaque) {
RdKafka::TopicImpl *topicimpl = static_cast<RdKafka::TopicImpl *>(rkt_opaque);
std::string key(static_cast<const char *>(keydata), keylen);
return topicimpl->partitioner_cb_->partitioner_cb(topicimpl, &key,
partition_cnt, msg_opaque);
}
static int32_t partitioner_kp_cb_trampoline(const rd_kafka_topic_t *rkt,
const void *keydata,
size_t keylen,
int32_t partition_cnt,
void *rkt_opaque,
void *msg_opaque) {
RdKafka::TopicImpl *topicimpl = static_cast<RdKafka::TopicImpl *>(rkt_opaque);
return topicimpl->partitioner_kp_cb_->partitioner_cb(
topicimpl, keydata, keylen, partition_cnt, msg_opaque);
}
RdKafka::Topic *RdKafka::Topic::create(Handle *base,
const std::string &topic_str,
const Conf *conf,
std::string &errstr) {
const RdKafka::ConfImpl *confimpl =
static_cast<const RdKafka::ConfImpl *>(conf);
rd_kafka_topic_t *rkt;
rd_kafka_topic_conf_t *rkt_conf;
rd_kafka_t *rk = dynamic_cast<HandleImpl *>(base)->rk_;
RdKafka::TopicImpl *topic = new RdKafka::TopicImpl();
if (!confimpl) {
/* Reuse default topic config, but we need our own copy to
* set the topic opaque. */
rkt_conf = rd_kafka_default_topic_conf_dup(rk);
} else {
/* Make a copy of conf struct to allow Conf reuse. */
rkt_conf = rd_kafka_topic_conf_dup(confimpl->rkt_conf_);
}
/* Set topic opaque to the topic so that we can reach our topic object
* from whatever callbacks get registered.
* The application itself will not need these opaques since their
* callbacks are class based. */
rd_kafka_topic_conf_set_opaque(rkt_conf, static_cast<void *>(topic));
if (confimpl) {
if (confimpl->partitioner_cb_) {
rd_kafka_topic_conf_set_partitioner_cb(rkt_conf,
partitioner_cb_trampoline);
topic->partitioner_cb_ = confimpl->partitioner_cb_;
} else if (confimpl->partitioner_kp_cb_) {
rd_kafka_topic_conf_set_partitioner_cb(rkt_conf,
partitioner_kp_cb_trampoline);
topic->partitioner_kp_cb_ = confimpl->partitioner_kp_cb_;
}
}
if (!(rkt = rd_kafka_topic_new(rk, topic_str.c_str(), rkt_conf))) {
errstr = rd_kafka_err2str(rd_kafka_last_error());
delete topic;
rd_kafka_topic_conf_destroy(rkt_conf);
return NULL;
}
topic->rkt_ = rkt;
return topic;
}