-
Notifications
You must be signed in to change notification settings - Fork 2.4k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Deadlock detector hack for Kafka driver instability #1087
Changes from 5 commits
493329c
ba9e170
7dba654
0f870db
01d65a6
c818b70
bed08c4
f243e40
4085d54
52f1a16
4573d51
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -16,6 +16,7 @@ package consumer | |
|
||
import ( | ||
"sync" | ||
"time" | ||
|
||
"github.com/Shopify/sarama" | ||
sc "github.com/bsm/sarama-cluster" | ||
|
@@ -42,6 +43,9 @@ type Consumer struct { | |
internalConsumer consumer.Consumer | ||
processorFactory ProcessorFactory | ||
|
||
deadlockDetectorFactory deadlockDetectorFactory | ||
allPartitionDeadlockDetector *deadlockDetector | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. nit: allPartitions... There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. 👍 |
||
|
||
partitionIDToState map[int32]*consumerState | ||
} | ||
|
||
|
@@ -52,12 +56,15 @@ type consumerState struct { | |
|
||
// New is a constructor for a Consumer | ||
func New(params Params) (*Consumer, error) { | ||
deadlockDetectorFactory := newDeadlockDetectorFactory(params.Factory, params.Logger, time.Minute) | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. separate issue: s/params.Factory/params.MetricsFactory/ There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I'll address this separately - it'll only add noise to this PR |
||
return &Consumer{ | ||
metricsFactory: params.Factory, | ||
logger: params.Logger, | ||
internalConsumer: params.InternalConsumer, | ||
processorFactory: params.ProcessorFactory, | ||
partitionIDToState: make(map[int32]*consumerState), | ||
metricsFactory: params.Factory, | ||
logger: params.Logger, | ||
internalConsumer: params.InternalConsumer, | ||
processorFactory: params.ProcessorFactory, | ||
deadlockDetectorFactory: deadlockDetectorFactory, | ||
allPartitionDeadlockDetector: deadlockDetectorFactory.startMonitoring(), | ||
partitionIDToState: make(map[int32]*consumerState), | ||
}, nil | ||
} | ||
|
||
|
@@ -73,6 +80,7 @@ func (c *Consumer) Start() { | |
// to the cleanup process not completing | ||
p.wg.Wait() | ||
} | ||
c.newPartitionMetrics(pc.Partition()).startCounter.Inc(1) | ||
c.partitionIDToState[pc.Partition()] = &consumerState{partitionConsumer: pc} | ||
go c.handleMessages(pc) | ||
go c.handleErrors(pc.Partition(), pc.Errors()) | ||
|
@@ -86,6 +94,7 @@ func (c *Consumer) Close() error { | |
c.closePartition(p.partitionConsumer) | ||
p.wg.Wait() | ||
} | ||
c.allPartitionDeadlockDetector.close() | ||
c.logger.Info("Closing parent consumer") | ||
return c.internalConsumer.Close() | ||
} | ||
|
@@ -97,27 +106,44 @@ func (c *Consumer) handleMessages(pc sc.PartitionConsumer) { | |
defer c.closePartition(pc) | ||
|
||
msgMetrics := c.newMsgMetrics(pc.Partition()) | ||
|
||
var msgProcessor processor.SpanProcessor | ||
|
||
for msg := range pc.Messages() { | ||
c.logger.Debug("Got msg", zap.Any("msg", msg)) | ||
msgMetrics.counter.Inc(1) | ||
msgMetrics.offsetGauge.Update(msg.Offset) | ||
msgMetrics.lagGauge.Update(pc.HighWaterMarkOffset() - msg.Offset - 1) | ||
deadlockDetector := c.deadlockDetectorFactory.startMonitoringForPartition(pc.Partition()) | ||
defer deadlockDetector.close() | ||
|
||
if msgProcessor == nil { | ||
msgProcessor = c.processorFactory.new(pc.Partition(), msg.Offset-1) | ||
defer msgProcessor.Close() | ||
} | ||
for { | ||
select { | ||
case msg, ok := <-pc.Messages(): | ||
if !ok { | ||
c.logger.Info("Message channel closed. ", zap.Int32("partition", pc.Partition())) | ||
return | ||
} | ||
c.logger.Debug("Got msg", zap.Any("msg", msg)) | ||
msgMetrics.counter.Inc(1) | ||
msgMetrics.offsetGauge.Update(msg.Offset) | ||
msgMetrics.lagGauge.Update(pc.HighWaterMarkOffset() - msg.Offset - 1) | ||
deadlockDetector.incrementMsgCount() | ||
c.allPartitionDeadlockDetector.incrementMsgCount() | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I don't follow what the purpose of this allPartitionDeadlockDetector is. It looks like it's only used to increment this counter - why do we even need it? We can always sum the time series to get total counter. |
||
|
||
if msgProcessor == nil { | ||
msgProcessor = c.processorFactory.new(pc.Partition(), msg.Offset-1) | ||
defer msgProcessor.Close() | ||
yurishkuro marked this conversation as resolved.
Show resolved
Hide resolved
|
||
} | ||
|
||
msgProcessor.Process(&saramaMessageWrapper{msg}) | ||
|
||
msgProcessor.Process(&saramaMessageWrapper{msg}) | ||
case <-deadlockDetector.getClosePartition(): | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. s/getClosePartition/closePartitionChannel/ |
||
c.logger.Info("Closing partition due to inactivity", zap.Int32("partition", pc.Partition())) | ||
return | ||
} | ||
} | ||
c.logger.Info("Finished handling messages", zap.Int32("partition", pc.Partition())) | ||
} | ||
|
||
func (c *Consumer) closePartition(partitionConsumer sc.PartitionConsumer) { | ||
c.logger.Info("Closing partition consumer", zap.Int32("partition", partitionConsumer.Partition())) | ||
partitionConsumer.Close() // blocks until messages channel is drained | ||
vprithvi marked this conversation as resolved.
Show resolved
Hide resolved
|
||
c.newPartitionMetrics(partitionConsumer.Partition()).closeCounter.Inc(1) | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. why do we call |
||
c.logger.Info("Closed partition consumer", zap.Int32("partition", partitionConsumer.Partition())) | ||
} | ||
|
||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -30,8 +30,17 @@ type errMetrics struct { | |
errCounter metrics.Counter | ||
} | ||
|
||
type partitionMetrics struct { | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. nit: we should clean-up these metrics. All 3 structs are per-partition, could we not combine them into one? Then we won't have all these small functions on the Consumer. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I agree - I'll do it as a separate commit |
||
startCounter metrics.Counter | ||
closeCounter metrics.Counter | ||
} | ||
|
||
func (c *Consumer) getNamespace(partition int32) metrics.Factory { | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. s/getNamespace/metricsFactoryForPartition/ There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. NB: we don't use "get" in Go. |
||
return c.metricsFactory.Namespace("sarama-consumer", map[string]string{"partition": strconv.Itoa(int(partition))}) | ||
} | ||
|
||
func (c *Consumer) newMsgMetrics(partition int32) msgMetrics { | ||
f := c.metricsFactory.Namespace("sarama-consumer", map[string]string{"partition": strconv.Itoa(int(partition))}) | ||
f := c.getNamespace(partition) | ||
return msgMetrics{ | ||
counter: f.Counter("messages", nil), | ||
offsetGauge: f.Gauge("current-offset", nil), | ||
|
@@ -40,7 +49,12 @@ func (c *Consumer) newMsgMetrics(partition int32) msgMetrics { | |
} | ||
|
||
func (c *Consumer) newErrMetrics(partition int32) errMetrics { | ||
f := c.metricsFactory.Namespace("sarama-consumer", map[string]string{"partition": strconv.Itoa(int(partition))}) | ||
return errMetrics{errCounter: f.Counter("errors", nil)} | ||
return errMetrics{errCounter: c.getNamespace(partition).Counter("errors", nil)} | ||
} | ||
|
||
func (c *Consumer) newPartitionMetrics(partition int32) partitionMetrics { | ||
f := c.getNamespace(partition) | ||
return partitionMetrics{ | ||
closeCounter: f.Counter("partition-close", nil), | ||
startCounter: f.Counter("partition-start", nil)} | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. This still bothers me. Some metrics factories are not happy if you try to create a metric with the same name twice. So if we re-acquire the same partition, this could cause a panic, e.g. if someone is using expvar-based metrics (unless we implemented protection in the factory, which I had to do for Prometheus). There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I added a test to jaeger-lib/metrics which shows that calling |
||
} |
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -89,11 +89,14 @@ func newConsumer( | |
consumer consumer.Consumer) *Consumer { | ||
|
||
logger, _ := zap.NewDevelopment() | ||
deadlockDetectorFactory := newDeadlockDetectorFactory(factory, logger, time.Second) | ||
return &Consumer{ | ||
metricsFactory: factory, | ||
logger: logger, | ||
internalConsumer: consumer, | ||
partitionIDToState: make(map[int32]*consumerState), | ||
metricsFactory: factory, | ||
logger: logger, | ||
internalConsumer: consumer, | ||
partitionIDToState: make(map[int32]*consumerState), | ||
deadlockDetectorFactory: deadlockDetectorFactory, | ||
allPartitionDeadlockDetector: deadlockDetectorFactory.startMonitoringForPartition(-1), | ||
|
||
processorFactory: ProcessorFactory{ | ||
topic: topic, | ||
|
@@ -173,6 +176,11 @@ func TestSaramaConsumerWrapper_start_Messages(t *testing.T) { | |
Tags: partitionTag, | ||
Value: 0, | ||
}) | ||
testutils.AssertCounterMetrics(t, localFactory, testutils.ExpectedMetric{ | ||
Name: "sarama-consumer.partition-start", | ||
Tags: partitionTag, | ||
Value: 1, | ||
}) | ||
} | ||
|
||
func TestSaramaConsumerWrapper_start_Errors(t *testing.T) { | ||
|
@@ -210,3 +218,28 @@ func TestSaramaConsumerWrapper_start_Errors(t *testing.T) { | |
|
||
t.Fail() | ||
} | ||
|
||
func TestHandleClosePartition(t *testing.T) { | ||
localFactory := metrics.NewLocalFactory(0) | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. s/localFactory/metricsFactory/ |
||
|
||
mp := &pmocks.SpanProcessor{} | ||
saramaConsumer := smocks.NewConsumer(t, &sarama.Config{}) | ||
mc := saramaConsumer.ExpectConsumePartition(topic, partition, msgOffset) | ||
mc.ExpectErrorsDrainedOnClose() | ||
saramaPartitionConsumer, e := saramaConsumer.ConsumePartition(topic, partition, msgOffset) | ||
require.NoError(t, e) | ||
|
||
undertest := newConsumer(localFactory, topic, mp, newSaramaClusterConsumer(saramaPartitionConsumer)) | ||
undertest.deadlockDetectorFactory = newDeadlockDetectorFactory(localFactory, zap.NewNop(), 10*time.Millisecond) | ||
undertest.Start() | ||
defer undertest.Close() | ||
|
||
for i := 0; i < 10; i++ { | ||
time.Sleep(20 * time.Millisecond) | ||
c, _ := localFactory.Snapshot() | ||
if c["sarama-consumer.partition-close|partition=316"] == 1 { | ||
return | ||
} | ||
} | ||
assert.Fail(t, "Did not close partition") | ||
} |
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,127 @@ | ||
// Copyright (c) 2018 The Jaeger Authors. | ||
// | ||
// 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. | ||
|
||
package consumer | ||
|
||
import ( | ||
"runtime" | ||
"strconv" | ||
"sync/atomic" | ||
"time" | ||
|
||
"github.com/uber/jaeger-lib/metrics" | ||
"go.uber.org/zap" | ||
) | ||
|
||
// deadlockDetectorFactory is a factory for deadlockDetectors | ||
type deadlockDetectorFactory struct { | ||
metricsFactory metrics.Factory | ||
logger *zap.Logger | ||
interval time.Duration | ||
panicFunc func(int32) | ||
} | ||
|
||
type deadlockDetector struct { | ||
msgConsumed *uint64 | ||
ticker *time.Ticker | ||
logger *zap.Logger | ||
closePartition chan struct{} | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I assume this does not apply to all-partitions detector? There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. It does not |
||
done chan struct{} | ||
} | ||
|
||
func newDeadlockDetectorFactory(factory metrics.Factory, logger *zap.Logger, interval time.Duration) deadlockDetectorFactory { | ||
return deadlockDetectorFactory{ | ||
metricsFactory: factory, | ||
logger: logger, | ||
interval: interval, | ||
panicFunc: func(partition int32) { | ||
factory.Counter("deadlockdetector.panic-issued", map[string]string{"partition": strconv.Itoa(int(partition))}).Inc(1) | ||
time.Sleep(time.Second) // Allow time to flush metric | ||
|
||
buf := make([]byte, 1<<20) | ||
logger.Panic("No messages processed in the last check interval", | ||
zap.Int32("partition", partition), | ||
zap.String("stack", string(buf[:runtime.Stack(buf, true)]))) | ||
}, | ||
} | ||
} | ||
|
||
// startMonitoringForPartition monitors the messages consumed by the partition and signals for the partition to by | ||
// closed by sending a message on the closePartition channel. | ||
// | ||
// Closing the partition should result in a rebalance, which alleviates the condition. This means that rebalances can | ||
// happen frequently if there is no traffic on the Kafka topic. This shouldn't affect normal operations. | ||
// | ||
// If the message send isn't processed within the next check interval, a panic is issued.This hack relies on a | ||
// container management system (k8s, aurora, marathon, etc) to reschedule | ||
// the dead instance. | ||
// | ||
// This hack protects jaeger-ingester from issues described in https://github.com/jaegertracing/jaeger/issues/1052 | ||
vprithvi marked this conversation as resolved.
Show resolved
Hide resolved
|
||
// | ||
func (s *deadlockDetectorFactory) startMonitoringForPartition(partition int32) *deadlockDetector { | ||
var msgConsumed uint64 | ||
w := &deadlockDetector{ | ||
msgConsumed: &msgConsumed, | ||
ticker: time.NewTicker(s.interval), | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. nit: you do not need to leak the ticker. Create it inside goroutine as local var with defer close. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. 👍 |
||
closePartition: make(chan struct{}, 1), | ||
done: make(chan struct{}), | ||
logger: s.logger, | ||
} | ||
|
||
go func() { | ||
vprithvi marked this conversation as resolved.
Show resolved
Hide resolved
|
||
for range w.ticker.C { | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. this is an odd pattern. If you close(done), nothing will happen till the ticker fires. You probably want unconditional for{} and then select across all channels, including the ticker. |
||
select { | ||
case <-w.done: | ||
s.logger.Info("Closing ticker routine", zap.Int32("partition", partition)) | ||
return | ||
default: | ||
if atomic.LoadUint64(w.msgConsumed) == 0 { | ||
select { | ||
case w.closePartition <- struct{}{}: | ||
s.logger.Warn("Signalling partition close due to inactivity", zap.Int32("partition", partition)) | ||
vprithvi marked this conversation as resolved.
Show resolved
Hide resolved
|
||
default: | ||
// If closePartition is blocked, the consumer might have deadlocked - kill the process | ||
s.panicFunc(partition) | ||
} | ||
} else { | ||
atomic.StoreUint64(w.msgConsumed, 0) | ||
} | ||
} | ||
} | ||
}() | ||
|
||
return w | ||
} | ||
|
||
// startMonitoring is to monitor that the sum of messages consumed across all partitions is non zero for the given interval | ||
// If it is zero when there are producers producing messages on the topic, it means that sarama-cluster hasn't | ||
// retrieved partition assignments. (This case will not be caught by startMonitoringForPartition because no partitions | ||
// were retrieved). | ||
func (s *deadlockDetectorFactory) startMonitoring() *deadlockDetector { | ||
return s.startMonitoringForPartition(-1) | ||
} | ||
|
||
func (w *deadlockDetector) getClosePartition() chan struct{} { | ||
return w.closePartition | ||
} | ||
|
||
func (w *deadlockDetector) close() { | ||
w.logger.Info("Closing deadlock detector") | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. nit: better to capture partition # in the detector and include it in the log stmt |
||
w.done <- struct{}{} | ||
w.ticker.Stop() | ||
} | ||
|
||
func (w *deadlockDetector) incrementMsgCount() { | ||
atomic.AddUint64(w.msgConsumed, 1) | ||
} |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I think it would be cleaner and easier to understand if you had a top-level
deadlockDetector
, which can createpartitionDeadlockDetector
as needed (implementation-wide the former may contain the latter for pId=-1). So factory is only used once to create the top-level detector, and the factory does not need to be stored in the Consumer.There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Doesn't this mean that the top level
deadlockDetector
also has the responsibilities of the Factor? (That being said, I think it might be cleaner design)There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
it does, in a way, but there's nothing wrong with that, especially considering that it happens at runtime and many times, whereas the top-level factory will be used only once on startup and not needed afterwards.
Separating top-level from individual detectors will also allow clean separation in some implementation details, e.g. where some features are not used.
And best of all, you'll be able to move a lot of methods into detectors, away from Consumer.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
👍