|
|
@@ -463,24 +463,23 @@ type consumerGroupSession struct {
|
|
|
generationID int32
|
|
|
handler ConsumerGroupHandler
|
|
|
|
|
|
- offsets OffsetManager
|
|
|
- claims map[string][]int32
|
|
|
- poms map[string]map[int32]PartitionOffsetManager
|
|
|
- done chan none
|
|
|
-
|
|
|
+ claims map[string][]int32
|
|
|
+ offsets *offsetManager
|
|
|
+ done chan none
|
|
|
waitGroup sync.WaitGroup
|
|
|
|
|
|
cancelOnce, releaseOnce sync.Once
|
|
|
}
|
|
|
|
|
|
func newConsumerGroupSession(parent *consumerGroup, claims map[string][]int32, memberID string, generationID int32, handler ConsumerGroupHandler) (*consumerGroupSession, error) {
|
|
|
- // create a new offset manager for the session
|
|
|
- offsets, err := NewOffsetManagerFromClient(parent.groupID, parent.client)
|
|
|
+ // init offset manager
|
|
|
+ om, err := NewOffsetManagerFromClient(parent.groupID, parent.client)
|
|
|
if err != nil {
|
|
|
return nil, err
|
|
|
}
|
|
|
- offsets.(*offsetManager).memberID = memberID
|
|
|
- offsets.(*offsetManager).generation = generationID
|
|
|
+ offsets := om.(*offsetManager)
|
|
|
+ offsets.memberID = memberID
|
|
|
+ offsets.generation = generationID
|
|
|
|
|
|
// init session
|
|
|
sess := &consumerGroupSession{
|
|
|
@@ -488,21 +487,17 @@ func newConsumerGroupSession(parent *consumerGroup, claims map[string][]int32, m
|
|
|
memberID: memberID,
|
|
|
generationID: generationID,
|
|
|
handler: handler,
|
|
|
-
|
|
|
- offsets: offsets,
|
|
|
- claims: claims,
|
|
|
- poms: make(map[string]map[int32]PartitionOffsetManager, len(claims)),
|
|
|
- done: make(chan none),
|
|
|
+ offsets: offsets,
|
|
|
+ claims: claims,
|
|
|
+ done: make(chan none),
|
|
|
}
|
|
|
|
|
|
// start heartbeat loop
|
|
|
sess.waitGroup.Add(1)
|
|
|
go sess.heartbeatLoop()
|
|
|
|
|
|
- // for each claim, create a PartitionOffsetManager
|
|
|
+ // create a POM for each claim
|
|
|
for topic, partitions := range claims {
|
|
|
- sess.poms[topic] = make(map[int32]PartitionOffsetManager, len(partitions))
|
|
|
-
|
|
|
for _, partition := range partitions {
|
|
|
pom, err := offsets.ManagePartition(topic, partition)
|
|
|
if err != nil {
|
|
|
@@ -510,13 +505,12 @@ func newConsumerGroupSession(parent *consumerGroup, claims map[string][]int32, m
|
|
|
return nil, err
|
|
|
}
|
|
|
|
|
|
+ // handle POM errors
|
|
|
go func(topic string, partition int32) {
|
|
|
for err := range pom.Errors() {
|
|
|
sess.parent.handleError(err, topic, partition)
|
|
|
}
|
|
|
}(topic, partition)
|
|
|
-
|
|
|
- sess.poms[topic][partition] = pom
|
|
|
}
|
|
|
}
|
|
|
|
|
|
@@ -551,18 +545,14 @@ func (s *consumerGroupSession) MemberID() string { return s.memberID }
|
|
|
func (s *consumerGroupSession) GenerationID() int32 { return s.generationID }
|
|
|
|
|
|
func (s *consumerGroupSession) MarkOffset(topic string, partition int32, offset int64, metadata string) {
|
|
|
- if partitions, ok := s.poms[topic]; ok {
|
|
|
- if pom, ok := partitions[partition]; ok {
|
|
|
- pom.MarkOffset(offset, metadata)
|
|
|
- }
|
|
|
+ if pom := s.offsets.findPOM(topic, partition); pom != nil {
|
|
|
+ pom.MarkOffset(offset, metadata)
|
|
|
}
|
|
|
}
|
|
|
|
|
|
func (s *consumerGroupSession) ResetOffset(topic string, partition int32, offset int64, metadata string) {
|
|
|
- if partitions, ok := s.poms[topic]; ok {
|
|
|
- if pom, ok := partitions[partition]; ok {
|
|
|
- pom.ResetOffset(offset, metadata)
|
|
|
- }
|
|
|
+ if pom := s.offsets.findPOM(topic, partition); pom != nil {
|
|
|
+ pom.ResetOffset(offset, metadata)
|
|
|
}
|
|
|
}
|
|
|
|
|
|
@@ -579,7 +569,10 @@ func (s *consumerGroupSession) consume(topic string, partition int32) {
|
|
|
}
|
|
|
|
|
|
// get next offset
|
|
|
- offset, _ := s.poms[topic][partition].NextOffset()
|
|
|
+ offset := s.parent.config.Consumer.Offsets.Initial
|
|
|
+ if pom := s.offsets.findPOM(topic, partition); pom != nil {
|
|
|
+ offset, _ = pom.NextOffset()
|
|
|
+ }
|
|
|
|
|
|
// create new claim
|
|
|
claim, err := newConsumerGroupClaim(s, topic, partition, offset)
|
|
|
@@ -620,35 +613,22 @@ func (s *consumerGroupSession) Cancel() {
|
|
|
})
|
|
|
}
|
|
|
|
|
|
-func (s *consumerGroupSession) release(cleanup bool) (err error) {
|
|
|
+func (s *consumerGroupSession) release(withCleanup bool) (err error) {
|
|
|
// signal release, stop heartbeat
|
|
|
s.Cancel()
|
|
|
|
|
|
// wait for consumers to exit
|
|
|
s.waitGroup.Wait()
|
|
|
|
|
|
- // perform release steps
|
|
|
+ // perform release
|
|
|
s.releaseOnce.Do(func() {
|
|
|
-
|
|
|
- // close partition offset managers
|
|
|
- for topic, partitions := range s.poms {
|
|
|
- for partition, pom := range partitions {
|
|
|
- if e := pom.Close(); e != nil {
|
|
|
- s.parent.handleError(err, topic, partition)
|
|
|
- err = e
|
|
|
- }
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
- // perform cleanup if this is a clean release
|
|
|
- if cleanup {
|
|
|
+ if withCleanup {
|
|
|
if e := s.handler.Cleanup(s); e != nil {
|
|
|
s.parent.handleError(err, "", -1)
|
|
|
err = e
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- // close offset manager
|
|
|
if e := s.offsets.Close(); e != nil {
|
|
|
err = e
|
|
|
}
|