-
Notifications
You must be signed in to change notification settings - Fork 4.9k
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
Kafka module: query each broker all the partitions it is a leader for #16556
Changes from all commits
32fdbab
c4557fe
ee5275b
67f0cc6
d9128f8
73bf9f3
bf55b0d
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 |
---|---|---|
|
@@ -24,6 +24,7 @@ import ( | |
"net" | ||
"os" | ||
"strings" | ||
"sync" | ||
"time" | ||
|
||
"github.com/pkg/errors" | ||
|
@@ -73,6 +74,18 @@ type MemberDescription struct { | |
Topics map[string][]int32 | ||
} | ||
|
||
type PartitionOffsets struct { | ||
Err error | ||
Offset int64 | ||
} | ||
|
||
type topicPartition struct { | ||
topic string | ||
partition int32 | ||
} | ||
|
||
type brokerTopicPartitionsMap map[int32][]topicPartition | ||
|
||
const noID = -1 | ||
|
||
// NewBroker creates a new unconnected kafka Broker connection instance. | ||
|
@@ -141,7 +154,7 @@ func (b *Broker) Connect() error { | |
return fmt.Errorf("No advertised broker with address %v found", b.Addr()) | ||
} | ||
|
||
debugf("found matching broker %v with id %v", other.Addr(), other.ID()) | ||
log.Debugf("found matching broker %v with id %v", other.Addr(), other.ID()) | ||
b.id = other.ID() | ||
b.advertisedAddr = other.Addr() | ||
|
||
|
@@ -281,7 +294,7 @@ func (b *Broker) FetchGroupOffsets(group string, partitions map[string][]int32) | |
return b.broker.FetchOffset(requ) | ||
} | ||
|
||
// FetchPartitionOffsetFromTheLeader fetches the OffsetNewest from the leader. | ||
// FetchPartitionOffsetFromTheLeader fetches the Newest from the leader. | ||
func (b *Broker) FetchPartitionOffsetFromTheLeader(topic string, partitionID int32) (int64, error) { | ||
offset, err := b.client.GetOffset(topic, partitionID, sarama.OffsetNewest) | ||
if err != nil { | ||
|
@@ -290,6 +303,114 @@ func (b *Broker) FetchPartitionOffsetFromTheLeader(topic string, partitionID int | |
return offset, nil | ||
} | ||
|
||
// FetchPartitionOffsetsForTopics fetches offsets of all partitions from its leaders. | ||
func (b *Broker) FetchPartitionOffsetsForTopics(topics []*sarama.TopicMetadata, time int64) map[string]map[int32]PartitionOffsets { | ||
leaderTopicPartition := b.groupBrokersPerTopicPartitions(topics) | ||
topicPartitionPartitionOffsets := b.fetchGroupedPartitionOffsetsPerBroker(leaderTopicPartition, time) | ||
return topicPartitionPartitionOffsets | ||
} | ||
|
||
func (b *Broker) groupBrokersPerTopicPartitions(topics []*sarama.TopicMetadata) brokerTopicPartitionsMap { | ||
leaderTopicPartition := brokerTopicPartitionsMap{} | ||
for _, topic := range topics { | ||
for _, partition := range topic.Partitions { | ||
if _, ok := leaderTopicPartition[partition.Leader]; !ok { | ||
leaderTopicPartition[partition.Leader] = []topicPartition{} | ||
} | ||
leaderTopicPartition[partition.Leader] = append(leaderTopicPartition[partition.Leader], | ||
topicPartition{topic: topic.Name, partition: partition.ID}) | ||
} | ||
} | ||
return leaderTopicPartition | ||
} | ||
|
||
func (b *Broker) fetchGroupedPartitionOffsetsPerBroker(leaderTopicPartition brokerTopicPartitionsMap, | ||
time int64) map[string]map[int32]PartitionOffsets { | ||
|
||
var wg sync.WaitGroup | ||
wg.Add(len(leaderTopicPartition)) | ||
|
||
queryResults := make(chan map[string]map[int32]PartitionOffsets, len(leaderTopicPartition)) | ||
defer close(queryResults) | ||
|
||
for leader, topicPartitions := range leaderTopicPartition { | ||
thisLeader := leader | ||
thisTopicPartitions := topicPartitions | ||
go func() { | ||
queryResults <- b.queryBrokerForPartitionOffsets(thisLeader, thisTopicPartitions, time) | ||
wg.Done() | ||
}() | ||
} | ||
wg.Wait() | ||
|
||
topicPartitionPartitionOffsets := map[string]map[int32]PartitionOffsets{} | ||
for i := 0; i < len(leaderTopicPartition); i++ { | ||
queryResult := <-queryResults | ||
for topic := range queryResult { | ||
topicPartitionPartitionOffsets[topic] = queryResult[topic] | ||
} | ||
} | ||
return topicPartitionPartitionOffsets | ||
} | ||
|
||
func (b *Broker) queryBrokerForPartitionOffsets(brokerID int32, topicPartitions []topicPartition, time int64) map[string]map[int32]PartitionOffsets { | ||
req := new(sarama.OffsetRequest) | ||
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. Something I am missing here is that in previous implementation we were making a request per replica (the request done in 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 seems that this may work (replicaID = -1 means a leader), with I believe this is getting too complex. |
||
for _, topicPartition := range topicPartitions { | ||
req.AddBlock(topicPartition.topic, topicPartition.partition, time, 1) | ||
} | ||
|
||
broker, err := b.client.Leader(topicPartitions[0].topic, topicPartitions[0].partition) | ||
if err != nil { | ||
return b.handleBrokerQueryError(topicPartitions, err) | ||
} | ||
|
||
defer func() { | ||
err := broker.Close() | ||
if err != nil { | ||
log.Debugf("closing broker (ID: %d) failed: ", brokerID, err) | ||
} | ||
}() | ||
|
||
resp, err := broker.GetAvailableOffsets(req) | ||
if err != nil { | ||
err = fmt.Errorf("get available offsets failed by leader (ID: %d): %v", brokerID, err) | ||
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 could use |
||
return b.handleBrokerQueryError(topicPartitions, err) | ||
} | ||
|
||
topicPartitionPartitionOffsets := map[string]map[int32]PartitionOffsets{} | ||
for _, topicPartition := range topicPartitions { | ||
if _, ok := topicPartitionPartitionOffsets[topicPartition.topic]; !ok { | ||
topicPartitionPartitionOffsets[topicPartition.topic] = map[int32]PartitionOffsets{} | ||
} | ||
|
||
block := resp.GetBlock(topicPartition.topic, topicPartition.partition) | ||
if len(block.Offsets) == 0 || block.Err != 0 { | ||
err = fmt.Errorf("offsets block is invalid (topicName: %s, partitionID: %d, leaderID: %d): %v", | ||
topicPartition.topic, topicPartition.partition, brokerID, block.Err.Error()) | ||
topicPartitionPartitionOffsets[topicPartition.topic][topicPartition.partition] = PartitionOffsets{Err: err, Offset: -1} | ||
continue | ||
} | ||
topicPartitionPartitionOffsets[topicPartition.topic][topicPartition.partition] = PartitionOffsets{Offset: block.Offsets[0]} | ||
} | ||
return topicPartitionPartitionOffsets | ||
} | ||
|
||
func (b *Broker) handleBrokerQueryError(topicPartitions []topicPartition, err error) map[string]map[int32]PartitionOffsets { | ||
topicPartitionPartitionOffsets := map[string]map[int32]PartitionOffsets{} | ||
for _, topicPartition := range topicPartitions { | ||
if err != nil { | ||
if _, ok := topicPartitionPartitionOffsets[topicPartition.topic]; ok { | ||
topicPartitionPartitionOffsets[topicPartition.topic] = map[int32]PartitionOffsets{} | ||
} | ||
topicPartitionPartitionOffsets[topicPartition.topic][topicPartition.partition] = PartitionOffsets{ | ||
Err: err, | ||
Offset: -1, | ||
} | ||
} | ||
} | ||
return topicPartitionPartitionOffsets | ||
} | ||
|
||
// ID returns the broker ID or -1 if the broker id is unknown. | ||
func (b *Broker) ID() int32 { | ||
if b.id == noID { | ||
|
@@ -418,7 +539,7 @@ func (m *brokerFinder) findBroker(addr string, brokers []*sarama.Broker) *sarama | |
} | ||
|
||
func (m *brokerFinder) findAddress(addr string, brokers []string) (int, bool) { | ||
debugf("Try to match broker to: %v", addr) | ||
log.Debugf("Try to match broker to: %v", addr) | ||
|
||
// get connection 'port' | ||
host, port, err := net.SplitHostPort(addr) | ||
|
@@ -437,14 +558,14 @@ func (m *brokerFinder) findAddress(addr string, brokers []string) (int, bool) { | |
if err != nil || len(localIPs) == 0 { | ||
return -1, false | ||
} | ||
debugf("local machine ips: %v", localIPs) | ||
log.Debugf("local machine ips: %v", localIPs) | ||
|
||
// try to find broker by comparing the fqdn for each known ip to list of | ||
// brokers | ||
localHosts := m.lookupHosts(localIPs) | ||
debugf("local machine addresses: %v", localHosts) | ||
log.Debugf("local machine addresses: %v", localHosts) | ||
for _, host := range localHosts { | ||
debugf("try to match with fqdn: %v (%v)", host, port) | ||
log.Debugf("try to match with fqdn: %v (%v)", host, port) | ||
if i, found := indexOf(net.JoinHostPort(host, port), brokers); found { | ||
return i, true | ||
} | ||
|
@@ -466,7 +587,7 @@ func (m *brokerFinder) findAddress(addr string, brokers []string) (int, bool) { | |
// try to find broker id by comparing the machines local hostname to | ||
// broker hostnames in metadata | ||
if host, err := m.Net.Hostname(); err == nil { | ||
debugf("try to match with hostname only: %v (%v)", host, port) | ||
log.Debugf("try to match with hostname only: %v (%v)", host, port) | ||
|
||
tmp := net.JoinHostPort(strings.ToLower(host), port) | ||
if i, found := indexOf(tmp, brokers); found { | ||
|
@@ -475,9 +596,9 @@ func (m *brokerFinder) findAddress(addr string, brokers []string) (int, bool) { | |
} | ||
|
||
// lookup ips for all brokers | ||
debugf("match by ips") | ||
log.Debugf("match by ips") | ||
for i, b := range brokers { | ||
debugf("test broker address: %v", b) | ||
log.Debugf("test broker address: %v", b) | ||
bh, bp, err := net.SplitHostPort(b) | ||
if err != nil { | ||
continue | ||
|
@@ -490,12 +611,12 @@ func (m *brokerFinder) findAddress(addr string, brokers []string) (int, bool) { | |
|
||
// lookup all ips for brokers host: | ||
ips, err := m.Net.LookupIP(bh) | ||
debugf("broker %v ips: %v, %v", bh, ips, err) | ||
log.Debugf("broker %v ips: %v, %v", bh, ips, err) | ||
if err != nil { | ||
continue | ||
} | ||
|
||
debugf("broker (%v) ips: %v", bh, ips) | ||
log.Debugf("broker (%v) ips: %v", bh, ips) | ||
|
||
// check if ip is known | ||
if anyIPsMatch(ips, localIPs) { | ||
|
@@ -515,7 +636,7 @@ func (m *brokerFinder) lookupHosts(ips []net.IP) []string { | |
} | ||
|
||
hosts, err := m.Net.LookupAddr(string(txt)) | ||
debugf("lookup %v => %v, %v", string(txt), hosts, err) | ||
log.Debugf("lookup %v => %v, %v", string(txt), hosts, err) | ||
if err != nil { | ||
continue | ||
} | ||
|
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.
Nit. This initialization is not needed,
append
will initialize it if needed.