-
Notifications
You must be signed in to change notification settings - Fork 149
/
kafka_sarama.go
205 lines (174 loc) · 4.71 KB
/
kafka_sarama.go
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
package kafka
import (
"context"
"log"
"os"
"os/signal"
"sync"
"syscall"
"github.com/Shopify/sarama"
"github.com/reugn/go-streams"
"github.com/reugn/go-streams/flow"
)
// KafkaSource represents an Apache Kafka source connector.
type KafkaSource struct {
consumer sarama.ConsumerGroup
handler sarama.ConsumerGroupHandler
topics []string
out chan interface{}
ctx context.Context
cancelCtx context.CancelFunc
wg *sync.WaitGroup
}
// NewKafkaSource returns a new KafkaSource instance.
func NewKafkaSource(ctx context.Context, addrs []string, groupID string,
config *sarama.Config, topics ...string) (*KafkaSource, error) {
consumerGroup, err := sarama.NewConsumerGroup(addrs, groupID, config)
if err != nil {
return nil, err
}
out := make(chan interface{})
cctx, cancel := context.WithCancel(ctx)
sink := &KafkaSource{
consumer: consumerGroup,
handler: &GroupHandler{make(chan struct{}), out},
topics: topics,
out: out,
ctx: cctx,
cancelCtx: cancel,
wg: &sync.WaitGroup{},
}
go sink.init()
return sink, nil
}
func (ks *KafkaSource) claimLoop() {
ks.wg.Add(1)
defer func() {
ks.wg.Done()
log.Printf("Exiting Kafka claimLoop")
}()
for {
handler := ks.handler.(*GroupHandler)
// `Consume` should be called inside an infinite loop, when a
// server-side rebalance happens, the consumer session will need to be
// recreated to get the new claims
if err := ks.consumer.Consume(ks.ctx, ks.topics, handler); err != nil {
log.Printf("Kafka consumer.Consume failed with: %v", err)
}
select {
case <-ks.ctx.Done():
return
default:
}
handler.ready = make(chan struct{})
}
}
// init starts the main loop
func (ks *KafkaSource) init() {
sigchan := make(chan os.Signal, 1)
signal.Notify(sigchan, syscall.SIGINT, syscall.SIGTERM)
go ks.claimLoop()
select {
case <-sigchan:
ks.cancelCtx()
case <-ks.ctx.Done():
}
log.Printf("Closing Kafka consumer")
ks.wg.Wait()
close(ks.out)
ks.consumer.Close()
}
// Via streams data through the given flow
func (ks *KafkaSource) Via(_flow streams.Flow) streams.Flow {
flow.DoStream(ks, _flow)
return _flow
}
// Out returns an output channel for sending data
func (ks *KafkaSource) Out() <-chan interface{} {
return ks.out
}
// GroupHandler represents a Sarama consumer group handler
type GroupHandler struct {
ready chan struct{}
out chan interface{}
}
// Setup is run at the beginning of a new session, before ConsumeClaim
func (handler *GroupHandler) Setup(sarama.ConsumerGroupSession) error {
// Mark the consumer as ready
close(handler.ready)
return nil
}
// Cleanup is run at the end of a session, once all ConsumeClaim goroutines have exited
func (handler *GroupHandler) Cleanup(sarama.ConsumerGroupSession) error {
return nil
}
// ConsumeClaim must start a consumer loop of ConsumerGroupClaim's Messages().
func (handler *GroupHandler) ConsumeClaim(session sarama.ConsumerGroupSession, claim sarama.ConsumerGroupClaim) error {
for {
select {
case message := <-claim.Messages():
if message != nil {
log.Printf("Message claimed: value = %s, timestamp = %v, topic = %s",
string(message.Value), message.Timestamp, message.Topic)
session.MarkMessage(message, "")
handler.out <- message
}
case <-session.Context().Done():
return session.Context().Err()
}
}
}
// KafkaSink represents an Apache Kafka sink connector.
type KafkaSink struct {
producer sarama.SyncProducer
topic string
in chan interface{}
}
// NewKafkaSink returns a new KafkaSink instance.
func NewKafkaSink(addrs []string, config *sarama.Config, topic string) (*KafkaSink, error) {
producer, err := sarama.NewSyncProducer(addrs, config)
if err != nil {
return nil, err
}
sink := &KafkaSink{
producer: producer,
topic: topic,
in: make(chan interface{}),
}
go sink.init()
return sink, nil
}
// init starts the main loop
func (ks *KafkaSink) init() {
for msg := range ks.in {
var err error
switch m := msg.(type) {
case *sarama.ProducerMessage:
_, _, err = ks.producer.SendMessage(m)
case *sarama.ConsumerMessage:
sMsg := &sarama.ProducerMessage{
Topic: ks.topic,
Key: sarama.StringEncoder(m.Key),
Value: sarama.StringEncoder(m.Value),
}
_, _, err = ks.producer.SendMessage(sMsg)
case string:
sMsg := &sarama.ProducerMessage{
Topic: ks.topic,
Value: sarama.StringEncoder(m),
}
_, _, err = ks.producer.SendMessage(sMsg)
default:
log.Printf("Unsupported message type %v", m)
}
if err != nil {
log.Printf("Error processing Kafka message: %s", err)
}
}
log.Printf("Closing Kafka producer")
ks.producer.Close()
}
// In returns an input channel for receiving data
func (ks *KafkaSink) In() chan<- interface{} {
return ks.in
}