This repository has been archived by the owner on Aug 23, 2023. It is now read-only.
-
Notifications
You must be signed in to change notification settings - Fork 105
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
use node priority for routing requests
- add a priority attribute to the cluster nodes - when getting peers needed to handle a query, use the nodes with the lowest priority. - update kafkamdm to track kafka lag, and use this lag as the node priority. So if a node has a large lag (either due to proecessing the backlog at startup, or because of a fault) it will stop recieving queries. But if all nodes in the cluster have high lag, then queries will still be processed and the user will get all of the data available. - closes #519
- Loading branch information
woodsaj
committed
Feb 22, 2017
1 parent
3a1fdf5
commit a81c6b3
Showing
6 changed files
with
218 additions
and
17 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,89 @@ | ||
package kafkamdm | ||
|
||
import ( | ||
"sync" | ||
) | ||
|
||
type lagLogger struct { | ||
sync.Mutex | ||
pos int | ||
measurements []int | ||
} | ||
|
||
func newLagLogger(size int) *lagLogger { | ||
return &lagLogger{ | ||
pos: 0, | ||
measurements: make([]int, 0, size), | ||
} | ||
} | ||
|
||
func (l *lagLogger) Store(lag int) { | ||
l.Lock() | ||
defer l.Unlock() | ||
l.pos++ | ||
if len(l.measurements) < cap(l.measurements) { | ||
l.measurements = append(l.measurements, lag) | ||
return | ||
} | ||
|
||
if l.pos >= cap(l.measurements) { | ||
l.pos = 0 | ||
} | ||
l.measurements[l.pos] = lag | ||
} | ||
|
||
func (l *lagLogger) Min() int { | ||
l.Lock() | ||
defer l.Unlock() | ||
min := -1 | ||
for _, m := range l.measurements { | ||
if min < 0 || m < min { | ||
min = m | ||
} | ||
} | ||
if min < 0 { | ||
min = 0 | ||
} | ||
return min | ||
} | ||
|
||
/* | ||
LagMonitor is used to determine how upToDate this node is. | ||
We periodically collect the lag for each partition, keeping the last N | ||
measurements in a moving window. Using those measurements we can then | ||
compute a overall score for this node. The score is just the maximum minimum | ||
lag of all partitions. | ||
For each partition we get the minimum lag seen in the last N measurements. | ||
Using minimum ensures that transient issues dont affect the health score. | ||
From each of those per-partition values, we then get the maximum. This | ||
ensures that overall health is based on the worst performing partition. | ||
*/ | ||
type LagMonitor struct { | ||
lag map[int32]*lagLogger | ||
} | ||
|
||
func NewLagMonitor(size int, partitions []int32) *LagMonitor { | ||
m := &LagMonitor{ | ||
lag: make(map[int32]*lagLogger), | ||
} | ||
for _, p := range partitions { | ||
m.lag[p] = newLagLogger(size) | ||
} | ||
return m | ||
} | ||
|
||
func (l *LagMonitor) Metric() int { | ||
max := 0 | ||
for _, lag := range l.lag { | ||
val := lag.Min() | ||
if val > max { | ||
max = val | ||
} | ||
} | ||
return max | ||
} | ||
|
||
func (l *LagMonitor) Store(partition int32, val int) { | ||
l.lag[partition].Store(val) | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,49 @@ | ||
package kafkamdm | ||
|
||
import ( | ||
"testing" | ||
|
||
. "github.com/smartystreets/goconvey/convey" | ||
) | ||
|
||
func TestLagLogger(t *testing.T) { | ||
logger := newLagLogger(5) | ||
Convey("with 0 measurements", t, func() { | ||
So(logger.Min(), ShouldEqual, 0) | ||
}) | ||
Convey("with 1 measurements", t, func() { | ||
logger.Store(10) | ||
So(logger.Min(), ShouldEqual, 10) | ||
}) | ||
Convey("with 2 measurements", t, func() { | ||
logger.Store(5) | ||
So(logger.Min(), ShouldEqual, 5) | ||
}) | ||
Convey("with lots of measurements", t, func() { | ||
for i := 0; i < 100; i++ { | ||
logger.Store(i) | ||
} | ||
So(logger.Min(), ShouldEqual, 95) | ||
}) | ||
} | ||
|
||
func TestLagMonitor(t *testing.T) { | ||
mon := NewLagMonitor(10, []int32{0, 1, 2, 3}) | ||
Convey("with 0 measurements", t, func() { | ||
So(mon.Metric(), ShouldEqual, 0) | ||
}) | ||
Convey("with lots of measurements", t, func() { | ||
for part := range mon.lag { | ||
for i := 0; i < 100; i++ { | ||
mon.Store(part, i) | ||
} | ||
} | ||
So(mon.Metric(), ShouldEqual, 90) | ||
}) | ||
Convey("metric should be worst partition", t, func() { | ||
for part := range mon.lag { | ||
mon.Store(part, 10+int(part)) | ||
} | ||
So(mon.Metric(), ShouldEqual, 13) | ||
}) | ||
} |