forked from pipelinedb/pipeline_kafka
-
Notifications
You must be signed in to change notification settings - Fork 0
/
pipeline_kafka.c
2443 lines (1983 loc) · 61 KB
/
pipeline_kafka.c
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
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864
865
866
867
868
869
870
871
872
873
874
875
876
877
878
879
880
881
882
883
884
885
886
887
888
889
890
891
892
893
894
895
896
897
898
899
900
901
902
903
904
905
906
907
908
909
910
911
912
913
914
915
916
917
918
919
920
921
922
923
924
925
926
927
928
929
930
931
932
933
934
935
936
937
938
939
940
941
942
943
944
945
946
947
948
949
950
951
952
953
954
955
956
957
958
959
960
961
962
963
964
965
966
967
968
969
970
971
972
973
974
975
976
977
978
979
980
981
982
983
984
985
986
987
988
989
990
991
992
993
994
995
996
997
998
999
1000
/*-------------------------------------------------------------------------
*
* pipeline_kafka.c
*
* PipelineDB support for Kafka
*
* Copyright (c) 2016, PipelineDB
*
* contrib/pipeline_kafka.c
*
*-------------------------------------------------------------------------
*/
#include <stdlib.h>
#include "postgres.h"
#include "funcapi.h"
#include "access/heapam.h"
#include "access/htup_details.h"
#include "access/skey.h"
#include "access/xact.h"
#include "access/xlog.h"
#include "catalog/catalog.h"
#include "catalog/namespace.h"
#include "catalog/pipeline_stream_fn.h"
#include "executor/executor.h"
#include "executor/spi.h"
#include "commands/copy.h"
#include "commands/dbcommands.h"
#include "commands/sequence.h"
#include "catalog/pg_type.h"
#include "commands/trigger.h"
#include "lib/stringinfo.h"
#include "librdkafka/rdkafka.h"
#include "miscadmin.h"
#include "nodes/makefuncs.h"
#include "nodes/print.h"
#include "pipeline_kafka.h"
#include "pipeline/stream.h"
#include "port/atomics.h"
#include "postmaster/bgworker.h"
#include "storage/ipc.h"
#include "storage/latch.h"
#include "storage/lwlock.h"
#include "storage/proc.h"
#include "storage/shmem.h"
#include "utils/builtins.h"
#include "utils/fmgroids.h"
#include "utils/guc.h"
#include "utils/json.h"
#include "utils/memutils.h"
#include "utils/rel.h"
#include "utils/relcache.h"
#include "utils/snapmgr.h"
PG_MODULE_MAGIC;
#define RETURN_SUCCESS() PG_RETURN_DATUM(CStringGetTextDatum("success"))
#define RETURN_FAILURE() PG_RETURN_DATUM(CStringGetTextDatum("failure"))
#define KAFKA_CONSUME_MAIN "kafka_consume_main"
#define PIPELINE_KAFKA_LIB "pipeline_kafka"
#define PIPELINE_KAFKA_SCHEMA "pipeline_kafka"
#define CONSUMER_RELATION "consumers"
#define CONSUMER_RELATION_NATTS 12
#define CONSUMER_ATTR_ID 1
#define CONSUMER_ATTR_TOPIC 2
#define CONSUMER_ATTR_RELATION 3
#define CONSUMER_ATTR_GROUP_ID 4
#define CONSUMER_ATTR_FORMAT 5
#define CONSUMER_ATTR_DELIMITER 6
#define CONSUMER_ATTR_QUOTE 7
#define CONSUMER_ATTR_ESCAPE 8
#define CONSUMER_ATTR_BATCH_SIZE 9
#define CONSUMER_ATTR_MAX_BYTES 10
#define CONSUMER_ATTR_PARALLELISM 11
#define CONSUMER_ATTR_TIMEOUT 12
#define OFFSETS_RELATION "offsets"
#define OFFSETS_RELATION_NATTS 3
#define OFFSETS_ATTR_CONSUMER 1
#define OFFSETS_ATTR_PARTITION 2
#define OFFSETS_ATTR_OFFSET 3
#define BROKER_RELATION "brokers"
#define BROKER_RELATION_NATTS 1
#define BROKER_ATTR_HOST 1
#define NUM_CONSUMERS_INIT 4
#define NUM_CONSUMERS_MAX 1024
#define DEFAULT_PARALLELISM 1
#define MAX_CONSUMER_PROCS 32
#define KAFKA_META_TIMEOUT 1000 /* 1s */
#define OPTION_DELIMITER "delimiter"
#define OPTION_FORMAT "format"
#define FORMAT_CSV "csv"
#define FORMAT_JSON "json"
#define OPTION_QUOTE "quote"
#define OPTION_ESCAPE "escape"
#define FORMAT_JSON_QUOTE "\x01"
#define FORMAT_JSON_DELIMITER "\x02"
#define RD_KAFKA_OFFSET_NULL INT64_MIN
#define CONSUMER_LOG_PREFIX "[pipeline_kafka] %s <- %s (PID %d): "
#define CONSUMER_LOG_PREFIX_PARAMS(consumer) \
((consumer)->rel ? (consumer)->rel->relname : "*"), (consumer)->topic_name, MyProcPid
#define CONSUMER_WORKER_RESTART_TIME 1
static volatile sig_atomic_t got_SIGTERM = false;
static rd_kafka_t *MyKafka = NULL;
static shmem_startup_hook_type prev_shmem_startup_hook = NULL;
static char *broker_version = NULL;
static char *consumer_config = NULL;
void _PG_init(void);
typedef struct error_buf_t
{
Size size;
pg_atomic_uint32 offset;
char *bytes;
slock_t mutex;
} error_buf_t;
#define ERROR_BUF_SIZE 4096
static error_buf_t my_error_buf;
static void
error_buf_init(error_buf_t *ebuf, Size size)
{
MemSet(ebuf, 0, sizeof(error_buf_t));
ebuf->size = size;
ebuf->bytes = palloc0(size);
pg_atomic_init_u32(&ebuf->offset, 0);
SpinLockInit(&ebuf->mutex);
}
static bool
error_buf_push(error_buf_t *ebuf, const char *str)
{
bool success = false;
int len = strlen(str) + 1; /* for \n */
uint32 offset;
SpinLockAcquire(&ebuf->mutex);
offset = pg_atomic_read_u32(&ebuf->offset);
if (ebuf->size - offset > len)
{
char *pos = &ebuf->bytes[offset];
memcpy(pos, str, len);
offset += len;
Assert(offset <= ebuf->size);
pg_atomic_write_u32(&ebuf->offset, offset);
success = true;
}
SpinLockRelease(&ebuf->mutex);
return success;
}
static char *
error_buf_pop(error_buf_t *ebuf)
{
char *err;
uint32 offset = pg_atomic_read_u32(&ebuf->offset);
if (!offset)
return NULL;
SpinLockAcquire(&ebuf->mutex);
offset = pg_atomic_read_u32(&ebuf->offset);
err = palloc(offset);
memcpy(err, ebuf->bytes, offset);
pg_atomic_write_u32(&ebuf->offset, 0);
SpinLockRelease(&ebuf->mutex);
return err;
}
/*
* Shared-memory state for each consumer process
*/
typedef struct KafkaConsumerProc
{
int32 id;
Oid db;
int32 consumer_id;
int64 start_offset;
int partition_group;
BackgroundWorkerHandle worker;
} KafkaConsumerProc;
typedef struct KafkaConsumerGroupKey
{
Oid db;
int32 consumer_id;
} KafkaConsumerGroupKey;
/*
* Shared-memory state for each consumer process group
*/
typedef struct KafkaConsumerGroup
{
KafkaConsumerGroupKey key;
int parallelism;
} KafkaConsumerGroup;
/*
* Local-memory configuration for a consumer
*/
typedef struct KafkaConsumer
{
int32 id;
List *brokers;
char *topic_name;
RangeVar *rel;
int32_t partition;
int64_t offset;
int batch_size;
int max_bytes;
int parallelism;
int timeout;
char *group_id;
char *format;
char *delimiter;
char *quote;
char *escape;
int num_partitions;
int64_t *offsets;
rd_kafka_t *kafka;
rd_kafka_topic_t *topic;
} KafkaConsumer;
/* Shared-memory hashtable for storing consumer process group information */
static HTAB *consumer_groups = NULL;
/* Shared-memory hashtable storing all individual consumer process information */
static HTAB *consumer_procs = NULL;
static void
pipeline_kafka_shmem_startup(void)
{
HASHCTL ctl;
if (prev_shmem_startup_hook)
prev_shmem_startup_hook();
LWLockAcquire(AddinShmemInitLock, LW_EXCLUSIVE);
MemSet(&ctl, 0, sizeof(HASHCTL));
ctl.keysize = sizeof(int32);
ctl.entrysize = sizeof(KafkaConsumerProc);
consumer_procs = ShmemInitHash("KafkaConsumerProcs", NUM_CONSUMERS_INIT,
NUM_CONSUMERS_MAX, &ctl, HASH_ELEM | HASH_BLOBS);
MemSet(&ctl, 0, sizeof(HASHCTL));
ctl.keysize = sizeof(KafkaConsumerGroupKey);
ctl.entrysize = sizeof(KafkaConsumerGroup);
consumer_groups = ShmemInitHash("KafkaConsumerGroups", 2 * NUM_CONSUMERS_INIT,
2 * NUM_CONSUMERS_MAX, &ctl, HASH_ELEM | HASH_BLOBS);
LWLockRelease(AddinShmemInitLock);
}
static Size
pipeline_kafka_shmem_size(void)
{
Size size;
size = hash_estimate_size(max_worker_processes, sizeof(KafkaConsumerGroup));
size = add_size(size, hash_estimate_size(max_worker_processes, sizeof(KafkaConsumerProc)));
return size;
}
static void
check_pipeline_kafka_preloaded(void)
{
if (!consumer_groups)
{
Assert(!consumer_procs);
ereport(ERROR,
(errmsg("%s wasn't loaded as a shared library", PIPELINE_KAFKA_LIB),
errhint("Add %s to the shared_preload_libraries configuration parameter.", PIPELINE_KAFKA_LIB)));
}
}
/*
* Initialization performed at module-load time
*/
void
_PG_init(void)
{
if (!process_shared_preload_libraries_in_progress)
{
elog(WARNING, "%s must be loaded via shared_preload_libraries", PIPELINE_KAFKA_LIB);
return;
}
DefineCustomStringVariable("pipeline_kafka.broker_version",
gettext_noop("Specifies the Kafka broker version for cases in which it can't be detected."),
NULL,
&broker_version,
NULL,
PGC_POSTMASTER, 0,
NULL, NULL, NULL);
DefineCustomStringVariable("pipeline_kafka.consumer_config",
gettext_noop("Comma-separated list of key-value pairs to override the default librdkafka configuration with."),
NULL,
&consumer_config,
NULL,
PGC_POSTMASTER, 0,
NULL, NULL, NULL);
prev_shmem_startup_hook = shmem_startup_hook;
shmem_startup_hook = pipeline_kafka_shmem_startup;
RequestAddinShmemSpace(MAXALIGN(pipeline_kafka_shmem_size()));
}
/*
* Signal handler for SIGTERM
* Set a flag to let the main loop to terminate, and set our latch to wake
* it up.
*/
static void
kafka_consume_main_sigterm(SIGNAL_ARGS)
{
int save_errno = errno;
got_SIGTERM = true;
if (MyLatch)
SetLatch(MyLatch);
errno = save_errno;
}
static RangeVar *
get_rangevar(char *relname)
{
return makeRangeVar(PIPELINE_KAFKA_SCHEMA, relname, -1);
}
static ResultRelInfo *
relinfo_open(RangeVar *rv, LOCKMODE mode)
{
Relation rel = relation_openrv(rv, mode);
ResultRelInfo *rinfo = makeNode(ResultRelInfo);
rinfo->ri_RangeTableIndex = 1; /* dummy */
rinfo->ri_RelationDesc = rel;
rinfo->ri_TrigDesc = NULL;
ExecOpenIndices(rinfo, false);
return rinfo;
}
static void
relinfo_close(ResultRelInfo *rinfo, LOCKMODE mode)
{
ExecCloseIndices(rinfo);
relation_close(rinfo->ri_RelationDesc, mode);
pfree(rinfo);
}
static void
update_indices(ResultRelInfo *rinfo, HeapTuple tup)
{
EState *estate = CreateExecutorState();
TupleTableSlot *slot = MakeSingleTupleTableSlot(RelationGetDescr(rinfo->ri_RelationDesc));
ExecStoreTuple(tup, slot, InvalidBuffer, false);
estate->es_result_relation_info = rinfo;
ExecInsertIndexTuples(slot, &tup->t_self, estate, false, NULL, NIL);
ExecDropSingleTupleTableSlot(slot);
FreeExecutorState(estate);
}
static void
relinfo_insert(ResultRelInfo *rinfo, HeapTuple tup)
{
simple_heap_insert(rinfo->ri_RelationDesc, tup);
update_indices(rinfo, tup);
}
static void
relinfo_update(ResultRelInfo *rinfo, ItemPointer tid, HeapTuple tup)
{
simple_heap_update(rinfo->ri_RelationDesc, tid, tup);
update_indices(rinfo, tup);
}
static void
relinfo_delete(ResultRelInfo *rinfo, ItemPointer tid)
{
simple_heap_delete(rinfo->ri_RelationDesc, tid);
}
/*
* librdkafka consumer logger function
*/
static void
consumer_logger(const rd_kafka_t *rk, int level, const char *fac, const char *buf)
{
error_buf_push(&my_error_buf, buf);
}
/*
* get_all_brokers
*
* Return a list of all brokers in pipeline_kafka_brokers
*/
static List *
get_all_brokers(void)
{
HeapTuple tup = NULL;
HeapScanDesc scan;
ResultRelInfo *brokers = relinfo_open(get_rangevar(BROKER_RELATION), AccessShareLock);
TupleTableSlot *slot = MakeSingleTupleTableSlot(RelationGetDescr(brokers->ri_RelationDesc));
List *result = NIL;
scan = heap_beginscan(brokers->ri_RelationDesc, GetTransactionSnapshot(), 0, NULL);
while ((tup = heap_getnext(scan, ForwardScanDirection)) != NULL)
{
char *host;
Datum d;
bool isnull;
ExecStoreTuple(tup, slot, InvalidBuffer, false);
d = slot_getattr(slot, BROKER_ATTR_HOST, &isnull);
host = TextDatumGetCString(d);
result = lappend(result, host);
}
ExecDropSingleTupleTableSlot(slot);
heap_endscan(scan);
relinfo_close(brokers, NoLock);
return result;
}
/*
* load_consumer_offsets
*
* Load all offsets for all of this consumer's partitions
*/
static void
load_consumer_offsets(KafkaConsumer *consumer, struct rd_kafka_metadata_topic *meta, int64_t start_offset)
{
MemoryContext old;
ScanKeyData skey[1];
HeapTuple tup = NULL;
IndexScanDesc scan;
ResultRelInfo *offsets;
TupleTableSlot *slot;
int i;
old = MemoryContextSwitchTo(CacheMemoryContext);
consumer->offsets = palloc0(meta->partition_cnt * sizeof(int64_t));
MemoryContextSwitchTo(old);
/* by default, begin consuming from the end of a stream */
for (i = 0; i < meta->partition_cnt; i++)
consumer->offsets[i] = start_offset;
/*
* Consumers with a group get their offsets from brokers
*/
if (consumer->group_id)
return;
offsets = relinfo_open(get_rangevar(OFFSETS_RELATION), RowExclusiveLock);
slot = MakeSingleTupleTableSlot(RelationGetDescr(offsets->ri_RelationDesc));
ScanKeyInit(&skey[0], 1, BTEqualStrategyNumber, F_OIDEQ, ObjectIdGetDatum(consumer->id));
scan = index_beginscan(offsets->ri_RelationDesc, offsets->ri_IndexRelationDescs[1],
GetTransactionSnapshot(), 1, 0);
index_rescan(scan, skey, 1, NULL, 0);
while ((tup = index_getnext(scan, ForwardScanDirection)) != NULL)
{
Datum d;
bool isnull;
int partition;
int64_t offset;
ExecStoreTuple(tup, slot, InvalidBuffer, false);
d = slot_getattr(slot, OFFSETS_ATTR_PARTITION, &isnull);
partition = DatumGetInt32(d);
if (partition > consumer->num_partitions)
elog(ERROR, "invalid partition id: %d", partition);
if (start_offset == RD_KAFKA_OFFSET_NULL)
{
d = slot_getattr(slot, OFFSETS_ATTR_OFFSET, &isnull);
if (isnull)
offset = RD_KAFKA_OFFSET_END;
else
{
/*
* Add one so that we start consuming from the message after the one we consumed
* most recently.
*/
offset = DatumGetInt64(d) + 1;
}
}
else
offset = start_offset;
consumer->offsets[partition] = DatumGetInt64(offset);
}
/* If no offset was saved and we passed it a NULL start_offset, set it to END */
for (i = 0; i < meta->partition_cnt; i++)
{
if (consumer->offsets[i] == RD_KAFKA_OFFSET_NULL)
consumer->offsets[i] = RD_KAFKA_OFFSET_END;
}
ExecDropSingleTupleTableSlot(slot);
index_endscan(scan);
relinfo_close(offsets, NoLock);
}
/*
* load_consumer_state
*
* Read consumer state from pipeline_kafka_consumers into the given struct
*/
static void
load_consumer_state(int32 consumer_id, KafkaConsumer *consumer)
{
ScanKeyData skey[1];
HeapTuple tup = NULL;
IndexScanDesc scan;
ResultRelInfo *consumers = relinfo_open(get_rangevar(CONSUMER_RELATION), ExclusiveLock);
TupleTableSlot *slot = MakeSingleTupleTableSlot(RelationGetDescr(consumers->ri_RelationDesc));
Datum d;
bool isnull;
text *qualified;
MemoryContext old;
char *relname;
MemSet(consumer, 0, sizeof(KafkaConsumer));
ScanKeyInit(&skey[0], 1, BTEqualStrategyNumber, F_INT4EQ, Int32GetDatum(consumer_id));
scan = index_beginscan(consumers->ri_RelationDesc, consumers->ri_IndexRelationDescs[0],
GetTransactionSnapshot(), 1, 0);
index_rescan(scan, skey, 1, NULL, 0);
tup = index_getnext(scan, ForwardScanDirection);
if (!HeapTupleIsValid(tup))
elog(ERROR, "kafka consumer %d not found", consumer_id);
ExecStoreTuple(tup, slot, InvalidBuffer, false);
/* we don't want anything that's palloc'd to get freed when we commit */
old = MemoryContextSwitchTo(CacheMemoryContext);
d = slot_getattr(slot, CONSUMER_ATTR_ID, &isnull);
Assert(!isnull);
consumer->id = DatumGetInt32(d);
/* target relation */
d = slot_getattr(slot, CONSUMER_ATTR_RELATION, &isnull);
Assert(!isnull);
qualified = (text *) DatumGetPointer(d);
relname = text_to_cstring(qualified);
if (strlen(relname))
consumer->rel = makeRangeVarFromNameList(textToQualifiedNameList(qualified));
else
consumer->rel = NULL;
/* topic */
d = slot_getattr(slot, CONSUMER_ATTR_TOPIC, &isnull);
Assert(!isnull);
consumer->topic_name = TextDatumGetCString(d);
/* consumer group id */
d = slot_getattr(slot, CONSUMER_ATTR_GROUP_ID, &isnull);
if (!isnull)
consumer->group_id = TextDatumGetCString(d);
else
consumer->group_id = NULL;
/* format */
d = slot_getattr(slot, CONSUMER_ATTR_FORMAT, &isnull);
consumer->format = TextDatumGetCString(d);
/* delimiter */
d = slot_getattr(slot, CONSUMER_ATTR_DELIMITER, &isnull);
if (!isnull)
consumer->delimiter = TextDatumGetCString(d);
else
consumer->delimiter = NULL;
/* quote character */
d = slot_getattr(slot, CONSUMER_ATTR_QUOTE, &isnull);
if (!isnull)
consumer->quote = TextDatumGetCString(d);
else
consumer->quote = NULL;
/* escape character */
d = slot_getattr(slot, CONSUMER_ATTR_ESCAPE, &isnull);
if (!isnull)
consumer->escape = TextDatumGetCString(d);
else
consumer->escape = NULL;
/* now load all brokers */
consumer->brokers = get_all_brokers();
MemoryContextSwitchTo(old);
d = slot_getattr(slot, CONSUMER_ATTR_PARALLELISM, &isnull);
Assert(!isnull);
consumer->parallelism = DatumGetInt32(d);
/* batch size */
d = slot_getattr(slot, CONSUMER_ATTR_BATCH_SIZE, &isnull);
Assert(!isnull);
consumer->batch_size = DatumGetInt32(d);
/* max bytes */
d = slot_getattr(slot, CONSUMER_ATTR_MAX_BYTES, &isnull);
Assert(!isnull);
consumer->max_bytes = DatumGetInt32(d);
/* timeout */
d = slot_getattr(slot, CONSUMER_ATTR_TIMEOUT, &isnull);
Assert(!isnull);
consumer->timeout = DatumGetInt32(d);
ExecDropSingleTupleTableSlot(slot);
index_endscan(scan);
relinfo_close(consumers, NoLock);
}
/*
* copy_next
*/
static int
copy_next(void *args, void *buf, int minread, int maxread)
{
StringInfo messages = (StringInfo) args;
int remaining = messages->len - messages->cursor;
int read = 0;
if (maxread <= remaining)
read = maxread;
else
read = remaining;
if (read == 0)
return 0;
memcpy(buf, messages->data + messages->cursor, read);
messages->cursor += read;
return read;
}
/*
* save_consumer_offsets
*/
static void
save_consumer_offsets(KafkaConsumer *consumer, int partition_group)
{
ScanKeyData skey[1];
HeapTuple tup = NULL;
IndexScanDesc scan;
ResultRelInfo *offsets;
Datum values[OFFSETS_RELATION_NATTS];
bool nulls[OFFSETS_RELATION_NATTS];
bool replace[OFFSETS_RELATION_NATTS];
bool updated[consumer->num_partitions];
TupleTableSlot *slot;
int partition;
/*
* Consumers with a group store offsets in brokers
*/
if (consumer->group_id)
{
for (partition = 0; partition < consumer->num_partitions; partition++)
{
rd_kafka_resp_err_t err;
if (partition % consumer->parallelism != partition_group)
continue;
err = rd_kafka_offset_store(consumer->topic, partition, consumer->offsets[partition]);
if (err)
{
elog(LOG, CONSUMER_LOG_PREFIX "librdkafka error: %s",
CONSUMER_LOG_PREFIX_PARAMS(consumer), rd_kafka_err2str(err));
}
}
}
offsets = relinfo_open(get_rangevar(OFFSETS_RELATION), RowExclusiveLock);
slot = MakeSingleTupleTableSlot(RelationGetDescr(offsets->ri_RelationDesc));
MemSet(updated, false, sizeof(updated));
ScanKeyInit(&skey[0], 1, BTEqualStrategyNumber, F_OIDEQ, ObjectIdGetDatum(consumer->id));
scan = index_beginscan(offsets->ri_RelationDesc, offsets->ri_IndexRelationDescs[1],
GetTransactionSnapshot(), 1, 0);
index_rescan(scan, skey, 1, NULL, 0);
/* update any existing offset rows */
while ((tup = index_getnext(scan, ForwardScanDirection)) != NULL)
{
Datum d;
bool isnull;
int partition;
int offset;
HeapTuple modified;
ExecStoreTuple(tup, slot, InvalidBuffer, false);
d = slot_getattr(slot, OFFSETS_ATTR_PARTITION, &isnull);
partition = DatumGetInt32(d);
/* we only want to update the offsets we're responsible for */
if (partition % consumer->parallelism != partition_group)
continue;
d = slot_getattr(slot, OFFSETS_ATTR_OFFSET, &isnull);
offset = DatumGetInt64(d);
updated[partition] = true;
/* No need to update offset if its unchanged */
if (offset == consumer->offsets[partition])
continue;
MemSet(nulls, false, sizeof(nulls));
MemSet(replace, false, sizeof(nulls));
values[OFFSETS_ATTR_OFFSET - 1] = Int64GetDatum(consumer->offsets[partition]);
replace[OFFSETS_ATTR_OFFSET - 1] = true;
if (consumer->offsets[partition] == RD_KAFKA_OFFSET_NULL)
replace[OFFSETS_ATTR_OFFSET - 1] = false;
modified = heap_modify_tuple(tup, RelationGetDescr(offsets->ri_RelationDesc), values, nulls, replace);
relinfo_update(offsets, &modified->t_self, modified);
CommandCounterIncrement();
}
index_endscan(scan);
/* now insert any offset rows that didn't already exist */
for (partition = 0; partition < consumer->num_partitions; partition++)
{
if (updated[partition])
continue;
if (partition % consumer->parallelism != partition_group)
continue;
values[OFFSETS_ATTR_CONSUMER - 1] = ObjectIdGetDatum(consumer->id);
values[OFFSETS_ATTR_PARTITION - 1] = Int32GetDatum(partition);
values[OFFSETS_ATTR_OFFSET - 1] = Int64GetDatum(consumer->offsets[partition]);
MemSet(nulls, false, sizeof(nulls));
if (consumer->offsets[partition] == RD_KAFKA_OFFSET_NULL)
nulls[OFFSETS_ATTR_OFFSET - 1] = true;
tup = heap_form_tuple(RelationGetDescr(offsets->ri_RelationDesc), values, nulls);
relinfo_insert(offsets, tup);
}
ExecDropSingleTupleTableSlot(slot);
relinfo_close(offsets, NoLock);
}
/*
* get_copy_statement
*
* Get the COPY statement that will be used to write messages to a stream
*/
static CopyStmt *
get_copy_statement(KafkaConsumer *consumer, bool missing_ok)
{
MemoryContext old;
CopyStmt *stmt;
Relation rel;
TupleDesc desc;
DefElem *format;
int i;
rel = heap_openrv_extended(consumer->rel, AccessShareLock, missing_ok);
if (!rel)
return NULL;
old = MemoryContextSwitchTo(CacheMemoryContext);
stmt = makeNode(CopyStmt);
stmt->relation = copyObject(consumer->rel);
stmt->filename = NULL;
stmt->options = NIL;
stmt->is_from = true;
stmt->query = NULL;
stmt->attlist = NIL;
format = makeNode(DefElem);
desc = RelationGetDescr(rel);
for (i = 0; i < desc->natts; i++)
{
/*
* Users can't supply values for arrival_timestamp, so make
* sure we exclude it from the copy attr list
*/
char *name = pstrdup(NameStr(desc->attrs[i]->attname));
if (IsStream(RelationGetRelid(rel)) && pg_strcasecmp(name, ARRIVAL_TIMESTAMP) == 0)
continue;
stmt->attlist = lappend(stmt->attlist, makeString(name));
}
if (consumer->delimiter)
{
DefElem *delim = makeNode(DefElem);
delim->defname = OPTION_DELIMITER;
delim->arg = (Node *) makeString(consumer->delimiter);
stmt->options = lappend(stmt->options, delim);
}
format->defname = OPTION_FORMAT;
format->arg = (Node *) makeString(consumer->format);
stmt->options = lappend(stmt->options, format);
if (consumer->quote)
{
DefElem *quote = makeNode(DefElem);
quote->defname = OPTION_QUOTE;
quote->arg = (Node *) makeString(consumer->quote);
stmt->options = lappend(stmt->options, quote);
}
if (consumer->escape)
{
DefElem *escape = makeNode(DefElem);
escape->defname = OPTION_ESCAPE;
escape->arg = (Node *) makeString(consumer->escape);
stmt->options = lappend(stmt->options, escape);
}
heap_close(rel, NoLock);
MemoryContextSwitchTo(old);
return stmt;
}
/*
* execute_copy
*
* Write messages to stream
*/
static void
execute_copy(KafkaConsumer *consumer, KafkaConsumerProc *proc, CopyStmt *stmt, StringInfo buf, int num_messages)
{
MemoryContext old = CurrentMemoryContext;
StartTransactionCommand();
/* we don't want to die in the event of any errors */
PG_TRY();
{
uint64 processed;
copy_iter_arg = buf;
DoCopy(stmt, "COPY", &processed);
}
PG_CATCH();
{
elog(LOG, CONSUMER_LOG_PREFIX "failed to process batch, dropped %d message%s",
CONSUMER_LOG_PREFIX_PARAMS(consumer), num_messages, (num_messages == 1 ? "" : "s"));
EmitErrorReport();
FlushErrorState();
AbortCurrentTransaction();
}
PG_END_TRY();
if (!IsTransactionState())
StartTransactionCommand();
/*
* We only store offsets if we're not part of a consumer group.
* Consumer groups store their offsets in Kafka.
*/
save_consumer_offsets(consumer, proc->partition_group);
CommitTransactionCommand();
MemoryContextSwitchTo(old);
}
static void
consume_topic_into_relation(KafkaConsumer *consumer, KafkaConsumerProc *proc, rd_kafka_topic_t *topic)
{
CopyStmt *copy;
rd_kafka_message_t **messages;
MemoryContext work_ctx = CurrentMemoryContext;
StartTransactionCommand();
copy = get_copy_statement(consumer, false);
CommitTransactionCommand();
messages = MemoryContextAlloc(CacheMemoryContext, sizeof(rd_kafka_message_t *) * consumer->batch_size);
/*
* Consume messages until we are terminated
*/
while (!got_SIGTERM)
{
int num_consumed;
int i;
int messages_buffered = 0;
int partition;
char *librdkerrs;
StringInfo buf;
MemoryContextSwitchTo(work_ctx);
MemoryContextReset(work_ctx);
buf = makeStringInfo();
for (partition = 0; partition < consumer->num_partitions; partition++)
{
if (partition % consumer->parallelism != proc->partition_group)
continue;
num_consumed = rd_kafka_consume_batch(topic, partition,
consumer->timeout, messages, consumer->batch_size);
if (num_consumed <= 0)
continue;
for (i = 0; i < num_consumed; i++)
{
rd_kafka_message_t *message = messages[i];
Assert(message);
if (message->err)
{
/* Ignore partition EOF internal error */
if (message->err != RD_KAFKA_RESP_ERR__PARTITION_EOF)
elog(LOG, CONSUMER_LOG_PREFIX "librdkafka error: %s",
CONSUMER_LOG_PREFIX_PARAMS(consumer), rd_kafka_err2str(message->err));
}
else if (message->len)
{
appendBinaryStringInfo(buf, message->payload, message->len);
/* COPY expects a newline after each tuple, so add one if missing. */
if (buf->data[buf->len - 1] != '\n')
appendStringInfoChar(buf, '\n');
messages_buffered++;
Assert(message->offset >= consumer->offsets[partition]);
consumer->offsets[partition] = message->offset;
}
rd_kafka_message_destroy(message);
messages[i] = NULL;
}
/* Flush if we've buffered enough messages or space used by messages has exceeded buffer size threshold */
if (messages_buffered >= consumer->batch_size || buf->len >= consumer->max_bytes)
{
execute_copy(consumer, proc, copy, buf, messages_buffered);
resetStringInfo(buf);