forked from apache/cassandra-gocql-driver
-
Notifications
You must be signed in to change notification settings - Fork 59
/
ring_describer.go
275 lines (232 loc) · 6.47 KB
/
ring_describer.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
package gocql
import (
"context"
"errors"
"fmt"
"net"
"sync"
)
// Polls system.peers at a specific interval to find new hosts
type ringDescriber struct {
control controlConnection
cfg *ClusterConfig
logger StdLogger
mu sync.RWMutex
prevHosts []*HostInfo
prevPartitioner string
// hosts are the set of all hosts in the cassandra ring that we know of.
// key of map is host_id.
hosts map[string]*HostInfo
// hostIPToUUID maps host native address to host_id.
hostIPToUUID map[string]string
}
func (r *ringDescriber) setControlConn(c controlConnection) {
r.mu.Lock()
defer r.mu.Unlock()
r.control = c
}
// Ask the control node for the local host info
func (r *ringDescriber) getLocalHostInfo(conn ConnInterface) (*HostInfo, error) {
iter := conn.querySystem(context.TODO(), qrySystemLocal)
if iter == nil {
return nil, errNoControl
}
host, err := hostInfoFromIter(iter, nil, r.cfg.Port, r.cfg.translateAddressPort)
if err != nil {
return nil, fmt.Errorf("could not retrieve local host info: %w", err)
}
return host, nil
}
// Ask the control node for host info on all it's known peers
func (r *ringDescriber) getClusterPeerInfo(localHost *HostInfo, c ConnInterface) ([]*HostInfo, error) {
var iter *Iter
if c.getIsSchemaV2() {
iter = c.querySystem(context.TODO(), qrySystemPeersV2)
} else {
iter = c.querySystem(context.TODO(), qrySystemPeers)
}
if iter == nil {
return nil, errNoControl
}
rows, err := iter.SliceMap()
if err != nil {
// TODO(zariel): make typed error
return nil, fmt.Errorf("unable to fetch peer host info: %s", err)
}
return getPeersFromQuerySystemPeers(rows, r.cfg.Port, r.cfg.translateAddressPort, r.logger)
}
func getPeersFromQuerySystemPeers(querySystemPeerRows []map[string]interface{}, port int, translateAddressPort func(addr net.IP, port int) (net.IP, int), logger StdLogger) ([]*HostInfo, error) {
var peers []*HostInfo
for _, row := range querySystemPeerRows {
// extract all available info about the peer
host, err := hostInfoFromMap(row, &HostInfo{port: port}, translateAddressPort)
if err != nil {
return nil, err
} else if !isValidPeer(host) {
// If it's not a valid peer
logger.Printf("Found invalid peer '%s' "+
"Likely due to a gossip or snitch issue, this host will be ignored", host)
continue
} else if isZeroToken(host) {
continue
}
peers = append(peers, host)
}
return peers, nil
}
// Return true if the host is a valid peer
func isValidPeer(host *HostInfo) bool {
return !(len(host.RPCAddress()) == 0 ||
host.hostId == "" ||
host.dataCenter == "" ||
host.rack == "")
}
func isZeroToken(host *HostInfo) bool {
return len(host.tokens) == 0
}
// GetHostsFromSystem returns a list of hosts found via queries to system.local and system.peers
func (r *ringDescriber) GetHostsFromSystem() ([]*HostInfo, string, error) {
r.mu.Lock()
defer r.mu.Unlock()
if r.control == nil {
return r.prevHosts, r.prevPartitioner, errNoControl
}
ch := r.control.getConn()
localHost, err := r.getLocalHostInfo(ch.conn)
if err != nil {
return r.prevHosts, r.prevPartitioner, err
}
peerHosts, err := r.getClusterPeerInfo(localHost, ch.conn)
if err != nil {
return r.prevHosts, r.prevPartitioner, err
}
var hosts []*HostInfo
if !isZeroToken(localHost) {
hosts = []*HostInfo{localHost}
}
hosts = append(hosts, peerHosts...)
var partitioner string
if len(hosts) > 0 {
partitioner = hosts[0].Partitioner()
}
r.prevHosts = hosts
r.prevPartitioner = partitioner
return hosts, partitioner, nil
}
// Given an ip/port return HostInfo for the specified ip/port
func (r *ringDescriber) getHostInfo(hostID UUID) (*HostInfo, error) {
var host *HostInfo
for _, table := range []string{"system.peers", "system.local"} {
ch := r.control.getConn()
var iter *Iter
if ch.host.HostID() == hostID.String() {
host = ch.host
iter = nil
}
if table == "system.peers" {
if ch.conn.getIsSchemaV2() {
iter = ch.conn.querySystem(context.TODO(), qrySystemPeersV2)
} else {
iter = ch.conn.querySystem(context.TODO(), qrySystemPeers)
}
} else {
iter = ch.conn.query(context.TODO(), fmt.Sprintf("SELECT * FROM %s", table))
}
if iter != nil {
rows, err := iter.SliceMap()
if err != nil {
return nil, err
}
for _, row := range rows {
h, err := hostInfoFromMap(row, &HostInfo{port: r.cfg.Port}, r.cfg.translateAddressPort)
if err != nil {
return nil, err
}
if h.HostID() == hostID.String() {
host = h
break
}
}
}
}
if host == nil {
return nil, errors.New("unable to fetch host info: invalid control connection")
} else if host.invalidConnectAddr() {
return nil, fmt.Errorf("host ConnectAddress invalid ip=%v: %v", host.connectAddress, host)
}
return host, nil
}
func (r *ringDescriber) getHostByIP(ip string) (*HostInfo, bool) {
r.mu.RLock()
defer r.mu.RUnlock()
hi, ok := r.hostIPToUUID[ip]
return r.hosts[hi], ok
}
func (r *ringDescriber) getHost(hostID string) *HostInfo {
r.mu.RLock()
host := r.hosts[hostID]
r.mu.RUnlock()
return host
}
func (r *ringDescriber) getHostsList() []*HostInfo {
r.mu.RLock()
hosts := make([]*HostInfo, 0, len(r.hosts))
for _, host := range r.hosts {
hosts = append(hosts, host)
}
r.mu.RUnlock()
return hosts
}
func (r *ringDescriber) getHostsMap() map[string]*HostInfo {
r.mu.RLock()
hosts := make(map[string]*HostInfo, len(r.hosts))
for k, v := range r.hosts {
hosts[k] = v
}
r.mu.RUnlock()
return hosts
}
func (r *ringDescriber) addOrUpdate(host *HostInfo) *HostInfo {
if existingHost, ok := r.addHostIfMissing(host); ok {
existingHost.update(host)
host = existingHost
}
return host
}
func (r *ringDescriber) addHostIfMissing(host *HostInfo) (*HostInfo, bool) {
if host.invalidConnectAddr() {
panic(fmt.Sprintf("invalid host: %v", host))
}
hostID := host.HostID()
r.mu.Lock()
if r.hosts == nil {
r.hosts = make(map[string]*HostInfo)
}
if r.hostIPToUUID == nil {
r.hostIPToUUID = make(map[string]string)
}
existing, ok := r.hosts[hostID]
if !ok {
r.hosts[hostID] = host
r.hostIPToUUID[host.nodeToNodeAddress().String()] = hostID
existing = host
}
r.mu.Unlock()
return existing, ok
}
func (r *ringDescriber) removeHost(hostID string) bool {
r.mu.Lock()
if r.hosts == nil {
r.hosts = make(map[string]*HostInfo)
}
if r.hostIPToUUID == nil {
r.hostIPToUUID = make(map[string]string)
}
h, ok := r.hosts[hostID]
if ok {
delete(r.hostIPToUUID, h.nodeToNodeAddress().String())
}
delete(r.hosts, hostID)
r.mu.Unlock()
return ok
}