-
Notifications
You must be signed in to change notification settings - Fork 0
/
consistent_hash_router.go
466 lines (392 loc) · 11.4 KB
/
consistent_hash_router.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
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
package ring
import (
"crypto/md5"
"encoding/binary"
"errors"
"fmt"
"sort"
"strconv"
"sync"
)
var (
ErrStartupConfiguration = errors.New("startup configuration is not correct")
ErrNodeIsAlreadyOnRing = errors.New("node has already joined the ring")
ErrVirtualNodeCount = errors.New("virtual node count should be greater than zero")
)
type consistentHashRouter struct {
hashFunction HashFunction
// Real physical nodes on the ring.
realNodes map[uint64]*parentNode
// Nodes on the ring.
virtualNodes map[uint64]*virtualNode
loadBalancers map[string]*loadBalancer
sortedMap []uint64
lock sync.RWMutex
*shardChangeHandler
nodeName string
}
type loadBalancer struct {
rpcAddr string
}
type parentNode struct {
nodeKey string
virtualNodes map[uint64]*virtualNode
tags map[string]string
}
func (p *parentNode) GetKey() string {
return p.nodeKey
}
type routerConfig struct {
HashFunction HashFunction
NodeName string
startupConfig *StartupConfig
}
func newConsistentHashRouter(config routerConfig) (*consistentHashRouter, error) {
if config.HashFunction == nil {
// Default hash function
config.HashFunction = &MD5HashFunction{}
}
ch := &consistentHashRouter{
hashFunction: config.HashFunction,
realNodes: map[uint64]*parentNode{},
virtualNodes: map[uint64]*virtualNode{},
sortedMap: []uint64{},
loadBalancers: map[string]*loadBalancer{},
shardChangeHandler: &shardChangeHandler{
listeners: make(map[string]ShardResponsibilityHandler),
},
nodeName: config.NodeName,
}
if config.startupConfig == nil {
return ch, nil
}
// Set the configuration from the startup config.
for hash, node := range config.startupConfig.Nodes {
if ch.hashFunction.Hash(node.NodeKey) != hash {
return nil, ErrStartupConfiguration
}
pNode := &parentNode{
nodeKey: node.NodeKey,
tags: node.Tags,
virtualNodes: make(map[uint64]*virtualNode),
}
for vHash, vNodeIndex := range node.VirtualNodes {
vNode := &virtualNode{
parentNode: pNode,
index: vNodeIndex,
}
if ch.hashFunction.Hash(vNode.GetKey()) != vHash {
return nil, ErrStartupConfiguration
}
ch.sortedMap = append(ch.sortedMap, vHash)
pNode.virtualNodes[vHash] = vNode
ch.virtualNodes[vHash] = vNode
}
ch.realNodes[hash] = pNode
// Enables to do binary search
sort.Slice(ch.sortedMap, func(i, j int) bool {
return ch.sortedMap[i] < ch.sortedMap[j]
})
}
return ch, nil
}
func (c *consistentHashRouter) Join(nodeKey string, tags map[string]string) error {
if _, ok := c.realNodes[c.hashFunction.Hash(nodeKey)]; ok {
return ErrNodeIsAlreadyOnRing
}
vNodeCount, err := strconv.Atoi(tags[virtualNodesJSON])
if err != nil {
return err
}
memberTypeInt, err := strconv.ParseUint(tags[memberTypeJSON], 10, 64)
if err != nil {
return err
}
memberType := MemberType(uint8(memberTypeInt))
if vNodeCount <= 0 && memberType == ShardMember {
return ErrVirtualNodeCount
}
c.lock.Lock()
defer c.lock.Unlock()
// remove unnecessary information.
delete(tags, virtualNodesJSON)
delete(tags, ringRPCAddrJSON)
if memberType == ShardMember {
pNode := c.createOrGetParentNode(nodeKey)
pNode.tags = tags
startIndex := len(pNode.virtualNodes)
var newNodes []uint64
startup := len(c.sortedMap) == 0
// Nodes that will be re-sharded.
for i := startIndex; i < startIndex+vNodeCount; i++ {
vNode := virtualNode{
parentNode: pNode,
index: i,
}
newNodeHash := c.hashFunction.Hash(vNode.GetKey())
c.sortedMap = append(c.sortedMap, newNodeHash)
pNode.virtualNodes[newNodeHash] = &vNode
c.virtualNodes[newNodeHash] = &vNode
// Enables to do binary search
sort.Slice(c.sortedMap, func(i, j int) bool {
return c.sortedMap[i] < c.sortedMap[j]
})
newNodes = append(newNodes, newNodeHash)
}
// If the new node is not the only node on the ring.
if !startup && len(newNodes) > 0 {
c.handleResharding(newNodes)
}
return nil
} else if memberType == LoadBalancerMember {
c.loadBalancers[nodeKey] = &loadBalancer{
rpcAddr: nodeKey,
}
return nil
}
return nil
}
func (c *consistentHashRouter) handleResharding(newNodes []uint64) {
// Sort the new nodes.
sort.Slice(newNodes, func(i, j int) bool {
return newNodes[i] < newNodes[j]
})
nodeHashMap := make(map[uint64]uint64)
newNodeMap := make(map[uint64]bool)
for _, newNode := range newNodes {
newNodeMap[newNode] = true
}
for _, newNodeHash := range newNodes {
// Dichotomous lookup to find the previous node which will send its data to the new node.
previousNodeIndex := binarySearchUint64(c.sortedMap, 0, len(c.sortedMap), newNodeHash)
if previousNodeIndex == 0 {
previousNodeIndex = len(c.sortedMap) - 1
} else {
previousNodeIndex -= 1
}
nodeHashMap[newNodeHash] = c.sortedMap[previousNodeIndex]
}
affectedNodeMap := make(map[uint64][]uint64)
for nodeHash, previousNodeHash := range nodeHashMap {
for newNodeMap[previousNodeHash] {
// Find previous node that is not the new node.
previousNodeHash = nodeHashMap[previousNodeHash]
}
affectedNodeMap[previousNodeHash] = append(affectedNodeMap[previousNodeHash], nodeHash)
}
// Build a map to send the changes in a single batch.
var batch []ShardResponsibility
for currentNodeHash, newNodesOfCurrent := range affectedNodeMap {
if c.virtualNodes[currentNodeHash].getRealNode() == c.nodeName {
sort.Slice(newNodesOfCurrent, func(i, j int) bool {
return newNodesOfCurrent[i] < newNodesOfCurrent[j]
})
for i := 0; i < len(newNodesOfCurrent); i++ {
var endKey interface{}
if i+1 == len(newNodesOfCurrent) {
endKey = nil
} else {
endKey = newNodesOfCurrent[i+1]
}
batch = append(batch, newShardResponsibility(
newNodesOfCurrent[i],
endKey,
c.virtualNodes[newNodesOfCurrent[i]].getRealNode(),
c.virtualNodes[newNodesOfCurrent[i]].getTags(),
c.hashFunction,
))
}
}
}
//
sort.Slice(batch, func(i, j int) bool {
return batch[i].start < batch[j].start
})
// Notify listeners for the current node changes only.
c.notifyListeners(batch)
}
func (c *consistentHashRouter) Leave(nodeKey string, tags map[string]string) error {
c.lock.Lock()
defer c.lock.Unlock()
pNode, found := c.getParentNode(nodeKey)
if found {
for _, vNode := range pNode.virtualNodes {
hash := c.hashFunction.Hash(vNode.GetKey())
delete(pNode.virtualNodes, hash)
// Dichotomous search to the find the virtual node
index := sort.Search(len(c.sortedMap), func(i int) bool {
return c.sortedMap[i] == hash
})
if index < len(c.sortedMap) {
c.sortedMap = append(c.sortedMap[:index], c.sortedMap[index+1:]...)
}
}
delete(c.realNodes, c.hashFunction.Hash(pNode.GetKey()))
return nil
} else {
delete(c.loadBalancers, nodeKey)
}
return nil
}
func (c *consistentHashRouter) getConfig() *StartupConfig {
c.lock.RLock()
defer c.lock.RUnlock()
resp := &StartupConfig{
Nodes: make(map[uint64]ConfigurationNode),
}
for realNodeHash, realNode := range c.realNodes {
node := ConfigurationNode{
NodeKey: realNode.nodeKey,
VirtualNodes: make(map[uint64]int),
Tags: realNode.tags,
}
for hash, virtualNode := range realNode.virtualNodes {
node.VirtualNodes[hash] = virtualNode.index
}
resp.Nodes[realNodeHash] = node
}
return resp
}
func (c *consistentHashRouter) createOrGetParentNode(nodeKey string) *parentNode {
pNode, found := c.getParentNode(nodeKey)
if !found {
pNode = &parentNode{
nodeKey: nodeKey,
virtualNodes: map[uint64]*virtualNode{},
}
c.realNodes[c.hashFunction.Hash(pNode.GetKey())] = pNode
}
return pNode
}
func (c *consistentHashRouter) getParentNode(nodeKey string) (*parentNode, bool) {
hash := c.hashFunction.Hash(nodeKey)
if pNode, ok := c.realNodes[hash]; ok {
return pNode, true
}
return nil, false
}
// Get clockwise nearest real node based on the key
func (c *consistentHashRouter) Get(key string) (map[string]string, bool) {
c.lock.RLock()
defer c.lock.RUnlock()
// No real node currently
if len(c.sortedMap) == 0 {
return nil, false
}
// Calculate the hash value
hash := c.hashFunction.Hash(key)
// Dichotomous lookup
// because the virtual nodes are reordered each time a node is added
// so the first node queried is our target node
// remainder will give us a circular list effect, finding nodes clockwise
index := sort.Search(len(c.sortedMap), func(i int) bool {
return c.sortedMap[i] >= hash
}) % len(c.sortedMap)
// virtual nodes -> physical nodes mapping
return c.virtualNodes[c.sortedMap[index]].getTags(), true
}
func (c *consistentHashRouter) GetLoadBalancers() []string {
c.lock.Lock()
defer c.lock.Unlock()
var loadBalancers []string
for _, loadbalancer := range c.loadBalancers {
loadBalancers = append(loadBalancers, loadbalancer.rpcAddr)
}
return loadBalancers
}
func (c *consistentHashRouter) GetVirtualNodes(key string) ([]virtualNode, bool) {
if pNode, ok := c.realNodes[c.hashFunction.Hash(key)]; ok {
var virtualNodes []virtualNode
for _, vNode := range pNode.virtualNodes {
virtualNodes = append(virtualNodes, *vNode)
}
return virtualNodes, true
}
return nil, false
}
// virtualNode allows to distribute data across nodes at a finer granularity than can be easily achieved using a single-token architecture.
type virtualNode struct {
parentNode *parentNode
index int
}
func (v *virtualNode) GetKey() string {
return fmt.Sprintf("%s-%d", v.parentNode.GetKey(), v.index)
}
func (v *virtualNode) isVirtualNodeOf(key string) bool {
return v.parentNode.GetKey() == key
}
func (v *virtualNode) getRealNode() string {
return v.parentNode.GetKey()
}
func (v *virtualNode) getTags() map[string]string {
return v.parentNode.tags
}
// MD5HashFunction Default hash function
type MD5HashFunction struct {
HashFunction
}
func (m *MD5HashFunction) Hash(name string) uint64 {
data := []byte(name)
b := md5.Sum(data)
return binary.LittleEndian.Uint64(b[:])
}
type shardChangeHandler struct {
listeners map[string]ShardResponsibilityHandler
lock sync.Mutex
}
func (sch *shardChangeHandler) AddListener(listenerId string, listener ShardResponsibilityHandler) {
sch.lock.Lock()
defer sch.lock.Unlock()
sch.listeners[listenerId] = listener
}
func (sch *shardChangeHandler) RemoveListener(listenerId string) {
sch.lock.Lock()
defer sch.lock.Unlock()
delete(sch.listeners, listenerId)
}
func (sch *shardChangeHandler) notifyListeners(batch []ShardResponsibility) {
sch.lock.Lock()
defer sch.lock.Unlock()
for _, listener := range sch.listeners {
listener.OnChange(batch)
}
}
// ShardResponsibility to determine if an object should get be transferred to the given node.
type ShardResponsibility struct {
start uint64
end interface{}
newNode string
tags map[string]string
hashFunction HashFunction
}
func newShardResponsibility(
start interface{},
end interface{},
newNode string,
tags map[string]string,
hashFunction HashFunction,
) ShardResponsibility {
return ShardResponsibility{
start: start.(uint64),
end: end,
newNode: newNode,
tags: tags,
hashFunction: hashFunction,
}
}
func (s *ShardResponsibility) Transfer(objectKey string) bool {
objectHash := s.hashFunction.Hash(objectKey)
if objectHash <= s.start {
if s.end == nil {
return true
}
return objectHash > s.end.(uint64)
}
return false
}
func (s *ShardResponsibility) ResponsibleNodeTags() map[string]string {
return s.tags
}
func (s *ShardResponsibility) ResponsibleNode() string {
return s.newNode
}