@@ -177,6 +177,31 @@ Baton KafkaConsumer::Assign(std::vector<RdKafka::TopicPartition*> partitions) {
177
177
return Baton (errcode);
178
178
}
179
179
180
+ Baton KafkaConsumer::IncrementalAssign (std::vector<RdKafka::TopicPartition*> partitions) {
181
+ if (!IsConnected ()) {
182
+ return Baton (RdKafka::ERR__STATE, " KafkaConsumer is disconnected" );
183
+ }
184
+
185
+ RdKafka::KafkaConsumer* consumer =
186
+ dynamic_cast <RdKafka::KafkaConsumer*>(m_client);
187
+
188
+ RdKafka::Error *e = consumer->incremental_assign (partitions);
189
+
190
+ if (e) {
191
+ RdKafka::ErrorCode errcode = e->code ();
192
+ delete e;
193
+ return Baton (errcode);
194
+ }
195
+
196
+ m_partition_cnt += partitions.size ();
197
+ for (auto i = partitions.begin (); i != partitions.end (); ++i) {
198
+ m_partitions.push_back (*i);
199
+ }
200
+ partitions.clear ();
201
+
202
+ return Baton (RdKafka::ERR_NO_ERROR);
203
+ }
204
+
180
205
Baton KafkaConsumer::Unassign () {
181
206
if (!IsClosing () && !IsConnected ()) {
182
207
return Baton (RdKafka::ERR__STATE);
@@ -193,12 +218,41 @@ Baton KafkaConsumer::Unassign() {
193
218
194
219
// Destroy the old list of partitions since we are no longer using it
195
220
RdKafka::TopicPartition::destroy (m_partitions);
221
+ m_partitions.clear ();
196
222
197
223
m_partition_cnt = 0 ;
198
224
199
225
return Baton (RdKafka::ERR_NO_ERROR);
200
226
}
201
227
228
+ Baton KafkaConsumer::IncrementalUnassign (std::vector<RdKafka::TopicPartition*> partitions) {
229
+ if (!IsClosing () && !IsConnected ()) {
230
+ return Baton (RdKafka::ERR__STATE);
231
+ }
232
+
233
+ RdKafka::KafkaConsumer* consumer =
234
+ dynamic_cast <RdKafka::KafkaConsumer*>(m_client);
235
+
236
+ RdKafka::Error *e = consumer->incremental_unassign (partitions);
237
+ if (e) {
238
+ RdKafka::ErrorCode errcode = e->code ();
239
+ delete e;
240
+ return Baton (errcode);
241
+ }
242
+
243
+ // Destroy the old list of partitions since we are no longer using it
244
+ RdKafka::TopicPartition::destroy (partitions);
245
+
246
+ m_partitions.erase (
247
+ std::remove_if (m_partitions.begin (), m_partitions.end (), [&partitions](RdKafka::TopicPartition *x) -> bool {
248
+ return std::find (partitions.begin (), partitions.end (), x) != partitions.end ();
249
+ }),
250
+ m_partitions.end ()
251
+ );
252
+ m_partition_cnt -= partitions.size ();
253
+ return Baton (RdKafka::ERR_NO_ERROR);
254
+ }
255
+
202
256
Baton KafkaConsumer::Commit (std::vector<RdKafka::TopicPartition*> toppars) {
203
257
if (!IsConnected ()) {
204
258
return Baton (RdKafka::ERR__STATE);
@@ -525,7 +579,9 @@ void KafkaConsumer::Init(v8::Local<v8::Object> exports) {
525
579
Nan::SetPrototypeMethod (tpl, " committed" , NodeCommitted);
526
580
Nan::SetPrototypeMethod (tpl, " position" , NodePosition);
527
581
Nan::SetPrototypeMethod (tpl, " assign" , NodeAssign);
582
+ Nan::SetPrototypeMethod (tpl, " incrementalAssign" , NodeIncrementalAssign);
528
583
Nan::SetPrototypeMethod (tpl, " unassign" , NodeUnassign);
584
+ Nan::SetPrototypeMethod (tpl, " incrementalUnassign" , NodeIncrementalUnassign);
529
585
Nan::SetPrototypeMethod (tpl, " assignments" , NodeAssignments);
530
586
531
587
Nan::SetPrototypeMethod (tpl, " commit" , NodeCommit);
@@ -757,6 +813,64 @@ NAN_METHOD(KafkaConsumer::NodeAssign) {
757
813
info.GetReturnValue ().Set (Nan::True ());
758
814
}
759
815
816
+ NAN_METHOD (KafkaConsumer::NodeIncrementalAssign) {
817
+ Nan::HandleScope scope;
818
+
819
+ if (info.Length () < 1 || !info[0 ]->IsArray ()) {
820
+ // Just throw an exception
821
+ return Nan::ThrowError (" Need to specify an array of partitions" );
822
+ }
823
+
824
+ v8::Local<v8::Array> partitions = info[0 ].As <v8::Array>();
825
+ std::vector<RdKafka::TopicPartition*> topic_partitions;
826
+
827
+ for (unsigned int i = 0 ; i < partitions->Length (); ++i) {
828
+ v8::Local<v8::Value> partition_obj_value;
829
+ if (!(
830
+ Nan::Get (partitions, i).ToLocal (&partition_obj_value) &&
831
+ partition_obj_value->IsObject ())) {
832
+ Nan::ThrowError (" Must pass topic-partition objects" );
833
+ }
834
+
835
+ v8::Local<v8::Object> partition_obj = partition_obj_value.As <v8::Object>();
836
+
837
+ // Got the object
838
+ int64_t partition = GetParameter<int64_t >(partition_obj, " partition" , -1 );
839
+ std::string topic = GetParameter<std::string>(partition_obj, " topic" , " " );
840
+
841
+ if (!topic.empty ()) {
842
+ RdKafka::TopicPartition* part;
843
+
844
+ if (partition < 0 ) {
845
+ part = Connection::GetPartition (topic);
846
+ } else {
847
+ part = Connection::GetPartition (topic, partition);
848
+ }
849
+
850
+ // Set the default value to offset invalid. If provided, we will not set
851
+ // the offset.
852
+ int64_t offset = GetParameter<int64_t >(
853
+ partition_obj, " offset" , RdKafka::Topic::OFFSET_INVALID);
854
+ if (offset != RdKafka::Topic::OFFSET_INVALID) {
855
+ part->set_offset (offset);
856
+ }
857
+
858
+ topic_partitions.push_back (part);
859
+ }
860
+ }
861
+
862
+ KafkaConsumer* consumer = ObjectWrap::Unwrap<KafkaConsumer>(info.This ());
863
+
864
+ // Hand over the partitions to the consumer.
865
+ Baton b = consumer->IncrementalAssign (topic_partitions);
866
+
867
+ if (b.err () != RdKafka::ERR_NO_ERROR) {
868
+ Nan::ThrowError (RdKafka::err2str (b.err ()).c_str ());
869
+ }
870
+
871
+ info.GetReturnValue ().Set (Nan::True ());
872
+ }
873
+
760
874
NAN_METHOD (KafkaConsumer::NodeUnassign) {
761
875
Nan::HandleScope scope;
762
876
@@ -777,6 +891,63 @@ NAN_METHOD(KafkaConsumer::NodeUnassign) {
777
891
info.GetReturnValue ().Set (Nan::True ());
778
892
}
779
893
894
+ NAN_METHOD (KafkaConsumer::NodeIncrementalUnassign) {
895
+ Nan::HandleScope scope;
896
+
897
+ if (info.Length () < 1 || !info[0 ]->IsArray ()) {
898
+ // Just throw an exception
899
+ return Nan::ThrowError (" Need to specify an array of partitions" );
900
+ }
901
+
902
+ v8::Local<v8::Array> partitions = info[0 ].As <v8::Array>();
903
+ std::vector<RdKafka::TopicPartition*> topic_partitions;
904
+
905
+ for (unsigned int i = 0 ; i < partitions->Length (); ++i) {
906
+ v8::Local<v8::Value> partition_obj_value;
907
+ if (!(
908
+ Nan::Get (partitions, i).ToLocal (&partition_obj_value) &&
909
+ partition_obj_value->IsObject ())) {
910
+ Nan::ThrowError (" Must pass topic-partition objects" );
911
+ }
912
+
913
+ v8::Local<v8::Object> partition_obj = partition_obj_value.As <v8::Object>();
914
+
915
+ // Got the object
916
+ int64_t partition = GetParameter<int64_t >(partition_obj, " partition" , -1 );
917
+ std::string topic = GetParameter<std::string>(partition_obj, " topic" , " " );
918
+
919
+ if (!topic.empty ()) {
920
+ RdKafka::TopicPartition* part;
921
+
922
+ if (partition < 0 ) {
923
+ part = Connection::GetPartition (topic);
924
+ } else {
925
+ part = Connection::GetPartition (topic, partition);
926
+ }
927
+
928
+ // Set the default value to offset invalid. If provided, we will not set
929
+ // the offset.
930
+ int64_t offset = GetParameter<int64_t >(
931
+ partition_obj, " offset" , RdKafka::Topic::OFFSET_INVALID);
932
+ if (offset != RdKafka::Topic::OFFSET_INVALID) {
933
+ part->set_offset (offset);
934
+ }
935
+
936
+ topic_partitions.push_back (part);
937
+ }
938
+ }
939
+
940
+ KafkaConsumer* consumer = ObjectWrap::Unwrap<KafkaConsumer>(info.This ());
941
+ // Hand over the partitions to the consumer.
942
+ Baton b = consumer->IncrementalUnassign (topic_partitions);
943
+
944
+ if (b.err () != RdKafka::ERR_NO_ERROR) {
945
+ Nan::ThrowError (RdKafka::err2str (b.err ()).c_str ());
946
+ }
947
+
948
+ info.GetReturnValue ().Set (Nan::True ());
949
+ }
950
+
780
951
NAN_METHOD (KafkaConsumer::NodeUnsubscribe) {
781
952
Nan::HandleScope scope;
782
953
0 commit comments