-
Notifications
You must be signed in to change notification settings - Fork 659
/
cooperative_consumer_example.go
161 lines (140 loc) · 4.68 KB
/
cooperative_consumer_example.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
/**
* Copyright 2020 Confluent Inc.
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
// Example high-level Apache Kafka consumer using the
// cooperative incremental rebalancing protocol which allows
// seamless reassignment of partitions to other group members.
package main
import (
"fmt"
"os"
"os/signal"
"syscall"
"github.com/confluentinc/confluent-kafka-go/v2/kafka"
)
func main() {
if len(os.Args) < 4 {
fmt.Fprintf(os.Stderr, "Usage: %s <bootstrap-servers> <group> <topics..>\n",
os.Args[0])
os.Exit(1)
}
bootstrapServers := os.Args[1]
group := os.Args[2]
topics := os.Args[3:]
sigchan := make(chan os.Signal, 1)
signal.Notify(sigchan, syscall.SIGINT, syscall.SIGTERM)
c, err := kafka.NewConsumer(&kafka.ConfigMap{
"bootstrap.servers": bootstrapServers,
// Avoid connecting to IPv6 brokers:
// This is needed for the ErrAllBrokersDown show-case below
// when using localhost brokers on OSX, since the OSX resolver
// will return the IPv6 addresses first.
// You typically don't need to specify this configuration
// property.
"broker.address.family": "v4",
// Consumer group ID
"group.id": group,
// Use the cooperative incremental rebalance protocol.
"partition.assignment.strategy": "cooperative-sticky",
// Start reading from the first message of each assigned
// partition if there are no previously committed offsets
// for this group.
"auto.offset.reset": "earliest"})
if err != nil {
fmt.Fprintf(os.Stderr, "Failed to create consumer: %s\n", err)
os.Exit(1)
}
fmt.Printf("Created Consumer %v\n", c)
// Subscribe to topics, call the rebalanceCallback on assignment/revoke.
// The rebalanceCallback is triggered from c.Poll() and c.Close() below.
err = c.SubscribeTopics(topics, rebalanceCallback)
if err != nil {
fmt.Fprintf(os.Stderr, "Failed to subscribe to topics: %s\n", err)
os.Exit(1)
}
run := true
for run {
select {
case sig := <-sigchan:
fmt.Printf("Caught signal %v: terminating\n", sig)
run = false
default:
ev := c.Poll(100)
if ev == nil {
continue
}
switch e := ev.(type) {
case *kafka.Message:
fmt.Printf("%% Message on %s:\n%s\n",
e.TopicPartition, string(e.Value))
if e.Headers != nil {
fmt.Printf("%% Headers: %v\n", e.Headers)
}
case kafka.Error:
// Errors should generally be considered
// informational, the client will try to
// automatically recover.
// But in this example we choose to terminate
// the application if all brokers are down.
fmt.Fprintf(os.Stderr, "%% Error: %v: %v\n",
e.Code(), e)
if e.Code() == kafka.ErrAllBrokersDown {
run = false
}
default:
fmt.Printf("Ignored %v\n", e)
}
}
}
fmt.Printf("Closing consumer\n")
c.Close()
}
// rebalanceCallback is called on each group rebalance to assign additional
// partitions, or remove existing partitions, from the consumer's current
// assignment.
//
// The application may use this optional callback to inspect the assignment,
// alter the initial start offset (the .Offset field of each assigned partition),
// and read/write offsets to commit to an alternative store outside of Kafka.
func rebalanceCallback(c *kafka.Consumer, event kafka.Event) error {
switch ev := event.(type) {
case kafka.AssignedPartitions:
fmt.Fprintf(os.Stderr,
"%% %s rebalance: %d new partition(s) assigned: %v\n",
c.GetRebalanceProtocol(), len(ev.Partitions),
ev.Partitions)
// The application may update the start .Offset of each
// assigned partition and then call IncrementalAssign().
// Even though this example does not alter the offsets we
// provide the call to IncrementalAssign() as an example.
err := c.IncrementalAssign(ev.Partitions)
if err != nil {
panic(err)
}
case kafka.RevokedPartitions:
fmt.Fprintf(os.Stderr,
"%% %s rebalance: %d partition(s) revoked: %v\n",
c.GetRebalanceProtocol(), len(ev.Partitions),
ev.Partitions)
if c.AssignmentLost() {
// Our consumer has been kicked out of the group and the
// entire assignment is thus lost.
fmt.Fprintf(os.Stderr, "%% Current assignment lost!\n")
}
// The client automatically calls IncrementalUnassign() unless
// the callback has already called that method.
}
return nil
}