Skip to content

Commit 45caa41

Browse files
committed
kafka adapter:
- Major chagne is to allow multiple consumers / consumer threads to service the same topic - Each consumer can now get assigned a subset of partitions for a topic. Consumers need to keep track of their own partitions - Since we dont know which keys are on which partition, the subscriber map is moved from Consumer to AdapterManager which will have the full set of subscribers for a given topic - Removed complexity of not subscribing to Rebalance events if not using start offset, always sub and wait for EOF even if not using start offset - exposed kafka consumer and producer specific config options Signed-off-by: Rob Ambalu <robert.ambalu@point72.com>
1 parent e24e2e2 commit 45caa41

6 files changed

Lines changed: 220 additions & 175 deletions

File tree

cpp/csp/adapters/kafka/KafkaAdapterManager.cpp

Lines changed: 105 additions & 19 deletions
Original file line numberDiff line numberDiff line change
@@ -136,7 +136,9 @@ void KafkaAdapterManager::setConfProperties( RdKafka::Conf * conf, const Diction
136136
void KafkaAdapterManager::forceShutdown( const std::string & err )
137137
{
138138
m_unrecoverableError = true; // So we can alert the producer to stop trying to flush
139-
forceConsumerReplayComplete();
139+
//Force all adapters replay complete so they dont stay blocked
140+
for( auto &[_,topicData] : m_topics )
141+
topicData.markReplayComplete();
140142
try
141143
{
142144
CSP_THROW( RuntimeException, "Kafka fatal error. " + err );
@@ -147,12 +149,6 @@ void KafkaAdapterManager::forceShutdown( const std::string & err )
147149
}
148150
}
149151

150-
void KafkaAdapterManager::forceConsumerReplayComplete()
151-
{
152-
for( auto & consumer : m_consumerVector )
153-
consumer -> forceReplayCompleted();
154-
}
155-
156152
void KafkaAdapterManager::start( DateTime starttime, DateTime endtime )
157153
{
158154
std::string errstr;
@@ -166,6 +162,14 @@ void KafkaAdapterManager::start( DateTime starttime, DateTime endtime )
166162
}
167163
}
168164

165+
// wildcard subscription has no guarantee of being in order
166+
// we flag replay complete as soon as we identify it.
167+
for( auto &[_,topicData] : m_topics )
168+
{
169+
if( topicData.wildcardSubscriber )
170+
topicData.wildcardSubscriber -> flagReplayComplete();
171+
}
172+
169173
// start all consumers
170174
for( auto & it : m_consumerVector )
171175
it -> start( starttime );
@@ -244,6 +248,95 @@ void KafkaAdapterManager::pollProducers()
244248
}
245249
}
246250

251+
void KafkaAdapterManager::onMessage( RdKafka::Message * msg ) const
252+
{
253+
auto topicIt = m_topics.find( msg -> topic_name() );
254+
if( topicIt == m_topics.end() )
255+
{
256+
std::string errmsg = "KafkaAdapterManager: Message received on unknown topic: " + msg -> topic_name() +
257+
" errcode: " + RdKafka::err2str( msg -> err() ) + " error: " + msg -> errstr();
258+
pushStatus( StatusLevel::ERROR, KafkaStatusMessageType::MSG_RECV_ERROR, errmsg );
259+
return;
260+
}
261+
auto & topicData = topicIt -> second;
262+
263+
if( !msg -> key() )
264+
{
265+
std::string errmsg = "KafkaAdapterManager: Message received with null key on topic " + msg -> topic_name() + ".";
266+
pushStatus( StatusLevel::ERROR, KafkaStatusMessageType::MSG_RECV_ERROR, errmsg );
267+
return;
268+
}
269+
270+
auto subscribersIt = topicData.subscribers.find( *msg -> key() );
271+
if( subscribersIt != topicData.subscribers.end() )
272+
{
273+
bool live = topicData.flaggedReplayComplete;
274+
for( auto it : subscribersIt -> second )
275+
it -> onMessage( msg, live );
276+
}
277+
278+
//Note we always have to tick wildcard as live because it can get messages from multiple
279+
//partitions, some which may have done replaying and some not ( not to mention that data can be out of order )
280+
if( topicData.wildcardSubscriber )
281+
topicData.wildcardSubscriber -> onMessage( msg, true );
282+
}
283+
284+
//Called from individual consumers once that partitions they are servicing for a given topic have all hit EOF
285+
void KafkaAdapterManager::markConsumerReplayDone( KafkaConsumer * consumer, const std::string & topic )
286+
{
287+
auto topicIt = m_topics.find( topic );
288+
assert( topicIt != m_topics.end() );
289+
if( topicIt == m_topics.end() )
290+
return;
291+
292+
topicIt -> second.markConsumerReplayDone( consumer );
293+
}
294+
295+
/*** TopicData ***/
296+
void KafkaAdapterManager::TopicData::addSubscriber( KafkaConsumer * consumer, const std::string & key, KafkaSubscriber * subscriber )
297+
{
298+
consumers.emplace( consumer, false );
299+
if( key.empty() )
300+
{
301+
assert( wildcardSubscriber == nullptr );
302+
wildcardSubscriber = subscriber;
303+
}
304+
else
305+
subscribers[key].emplace_back( subscriber );
306+
}
307+
308+
void KafkaAdapterManager::TopicData::markConsumerReplayDone( KafkaConsumer * consumer )
309+
{
310+
auto it = consumers.find( consumer );
311+
assert( it != consumers.end() );
312+
it -> second = true;
313+
314+
for( auto [_,done] : consumers )
315+
{
316+
if( !done )
317+
return;
318+
}
319+
320+
//All consumer partitions for the given topic are done replaying
321+
markReplayComplete();
322+
}
323+
324+
void KafkaAdapterManager::TopicData::markReplayComplete()
325+
{
326+
//this can be called from multiple consumer threads
327+
bool prevVal = flaggedReplayComplete.exchange( true );
328+
if( !prevVal )
329+
{
330+
// Flag all regular subscribers
331+
for( auto& subscriberEntry : subscribers )
332+
{
333+
for( auto* subscriber : subscriberEntry.second )
334+
subscriber -> flagReplayComplete();
335+
}
336+
}
337+
}
338+
/*** end TopicData ***/
339+
247340
PushInputAdapter * KafkaAdapterManager::getInputAdapter( CspTypePtr & type, PushMode pushMode, const Dictionary & properties )
248341
{
249342
std::string topic = properties.get<std::string>( "topic" );
@@ -274,25 +367,16 @@ OutputAdapter * KafkaAdapterManager::getOutputAdapter( CspTypePtr & type, const
274367
}
275368
}
276369

277-
KafkaConsumer * KafkaAdapterManager::getConsumer( const std::string & topic, const Dictionary & properties )
370+
KafkaConsumer * KafkaAdapterManager::getConsumer( const Dictionary & properties )
278371
{
279-
// If we have seen this topic before, look up the consumer for it in the map
280-
// Otherwise, make a new consumer (and insert it into the map)
281-
// If we have reached m_maxThreads, then round-robin the topic onto a consumer (and insert it into the map)
282-
if( m_consumerMap.find( topic ) != m_consumerMap.end() )
283-
{
284-
return m_consumerMap[ topic ].get();
285-
}
286372
if( m_consumerVector.size() < m_maxThreads )
287373
{
288374
auto consumer = std::make_shared<KafkaConsumer>( this, properties );
289375
m_consumerVector.emplace_back( consumer );
290-
m_consumerMap.emplace( topic, consumer );
291-
return m_consumerMap[ topic ].get();
376+
return consumer.get();
292377
}
293378

294379
auto consumer = m_consumerVector[ m_consumerIdx++ ];
295-
m_consumerMap.emplace( topic, consumer );
296380
if( m_consumerIdx >= m_maxThreads )
297381
m_consumerIdx = 0;
298382
return consumer.get();
@@ -308,7 +392,9 @@ KafkaSubscriber * KafkaAdapterManager::getSubscriber( const std::string & topic,
308392
std::unique_ptr<KafkaSubscriber> subscriber( new KafkaSubscriber( this, properties ) );
309393
rv.first -> second = std::move( subscriber );
310394

311-
this -> getConsumer( topic, properties ) -> addSubscriber( topic, key, rv.first -> second.get() );
395+
auto * consumer = this -> getConsumer( properties );
396+
consumer -> addTopic( topic );
397+
m_topics[ topic ].addSubscriber( consumer, key, rv.first -> second.get() );
312398
}
313399

314400
return rv.first -> second.get();

cpp/csp/adapters/kafka/KafkaAdapterManager.h

Lines changed: 23 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -6,6 +6,7 @@
66
#include <csp/engine/AdapterManager.h>
77
#include <csp/engine/Dictionary.h>
88
#include <csp/engine/PushInputAdapter.h>
9+
#include <librdkafka/rdkafkacpp.h>
910
#include <atomic>
1011
#include <string>
1112
#include <thread>
@@ -66,8 +67,6 @@ class KafkaAdapterManager final : public csp::AdapterManager
6667
PushInputAdapter * getInputAdapter( CspTypePtr & type, PushMode pushMode, const Dictionary & properties );
6768
OutputAdapter * getOutputAdapter( CspTypePtr & type, const Dictionary & properties );
6869

69-
KafkaConsumer * getConsumer( const std::string & topic, const Dictionary & properties );
70-
7170
RdKafka::Conf * getConsumerConf() { return m_consumerConf.get(); }
7271

7372
const Dictionary::Value & startOffsetProperty() const { return m_startOffsetProperty; }
@@ -76,22 +75,41 @@ class KafkaAdapterManager final : public csp::AdapterManager
7675

7776
void forceShutdown( const std::string & err );
7877

78+
void markConsumerReplayDone( KafkaConsumer * consumer, const std::string & topic );
79+
void onMessage( RdKafka::Message * msg ) const;
80+
7981
private:
8082

8183
using TopicKeyPair = std::pair<std::string, std::string>;
8284

85+
KafkaConsumer * getConsumer( const Dictionary & properties );
8386
void setConfProperties( RdKafka::Conf * conf, const Dictionary & properties );
8487
void pollProducers();
85-
void forceConsumerReplayComplete();
8688

8789
KafkaSubscriber * getSubscriber( const std::string & topic, const std::string & key, const Dictionary & properties );
8890
KafkaPublisher * getStaticPublisher( const TopicKeyPair & pair, const Dictionary & properties );
8991
KafkaPublisher * getDynamicPublisher( const std::string & topic, const Dictionary & properties );
9092

93+
struct TopicData
94+
{
95+
//Key -> Subscriber
96+
using SubscriberMap = std::unordered_map<std::string, std::vector<KafkaSubscriber*>>;
97+
using ConsumerMap = std::unordered_map<KafkaConsumer *, bool>;
98+
ConsumerMap consumers;
99+
SubscriberMap subscribers;
100+
KafkaSubscriber * wildcardSubscriber = nullptr;
101+
std::atomic<bool> flaggedReplayComplete = false;
102+
103+
void addSubscriber( KafkaConsumer * consumer, const std::string & key, KafkaSubscriber * subscriber );
104+
void markConsumerReplayDone( KafkaConsumer * consumer );
105+
void markReplayComplete();
106+
};
107+
108+
std::unordered_map<std::string,TopicData> m_topics;
109+
91110
using ConsumerVector = std::vector<std::shared_ptr<KafkaConsumer>>;
92111
ConsumerVector m_consumerVector;
93-
using ConsumerMap = std::unordered_map<std::string, std::shared_ptr<KafkaConsumer>>;
94-
ConsumerMap m_consumerMap;
112+
95113

96114
using StaticPublishers = std::unordered_map<TopicKeyPair, std::unique_ptr<KafkaPublisher>, hash::hash_pair>;
97115
StaticPublishers m_staticPublishers;

0 commit comments

Comments
 (0)