forked from apache/cassandra-gocql-driver
-
Notifications
You must be signed in to change notification settings - Fork 0
/
host_source.go
395 lines (326 loc) · 8.46 KB
/
host_source.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
package gocql
import (
"fmt"
"log"
"net"
"strconv"
"strings"
"sync"
"time"
)
type nodeState int32
func (n nodeState) String() string {
if n == NodeUp {
return "UP"
} else if n == NodeDown {
return "DOWN"
}
return fmt.Sprintf("UNKNOWN_%d", n)
}
const (
NodeUp nodeState = iota
NodeDown
)
type cassVersion struct {
Major, Minor, Patch int
}
func (c *cassVersion) Set(v string) error {
if v == "" {
return nil
}
return c.UnmarshalCQL(nil, []byte(v))
}
func (c *cassVersion) UnmarshalCQL(info TypeInfo, data []byte) error {
return c.unmarshal(data)
}
func (c *cassVersion) unmarshal(data []byte) error {
version := strings.TrimSuffix(string(data), "-SNAPSHOT")
version = strings.TrimPrefix(version, "v")
v := strings.Split(version, ".")
if len(v) < 2 {
return fmt.Errorf("invalid version string: %s", data)
}
var err error
c.Major, err = strconv.Atoi(v[0])
if err != nil {
return fmt.Errorf("invalid major version %v: %v", v[0], err)
}
c.Minor, err = strconv.Atoi(v[1])
if err != nil {
return fmt.Errorf("invalid minor version %v: %v", v[1], err)
}
if len(v) > 2 {
c.Patch, err = strconv.Atoi(v[2])
if err != nil {
return fmt.Errorf("invalid patch version %v: %v", v[2], err)
}
}
return nil
}
func (c cassVersion) Before(major, minor, patch int) bool {
if c.Major > major {
return true
} else if c.Minor > minor {
return true
} else if c.Patch > patch {
return true
}
return false
}
func (c cassVersion) String() string {
return fmt.Sprintf("v%d.%d.%d", c.Major, c.Minor, c.Patch)
}
func (c cassVersion) nodeUpDelay() time.Duration {
if c.Major >= 2 && c.Minor >= 2 {
// CASSANDRA-8236
return 0
}
return 10 * time.Second
}
type HostInfo struct {
// TODO(zariel): reduce locking maybe, not all values will change, but to ensure
// that we are thread safe use a mutex to access all fields.
mu sync.RWMutex
peer string
port int
dataCenter string
rack string
hostId string
version cassVersion
state nodeState
tokens []string
}
func (h *HostInfo) Equal(host *HostInfo) bool {
h.mu.RLock()
defer h.mu.RUnlock()
host.mu.RLock()
defer host.mu.RUnlock()
return h.peer == host.peer && h.hostId == host.hostId
}
func (h *HostInfo) Peer() string {
h.mu.RLock()
defer h.mu.RUnlock()
return h.peer
}
func (h *HostInfo) setPeer(peer string) *HostInfo {
h.mu.Lock()
defer h.mu.Unlock()
h.peer = peer
return h
}
func (h *HostInfo) DataCenter() string {
h.mu.RLock()
defer h.mu.RUnlock()
return h.dataCenter
}
func (h *HostInfo) setDataCenter(dataCenter string) *HostInfo {
h.mu.Lock()
defer h.mu.Unlock()
h.dataCenter = dataCenter
return h
}
func (h *HostInfo) Rack() string {
h.mu.RLock()
defer h.mu.RUnlock()
return h.rack
}
func (h *HostInfo) setRack(rack string) *HostInfo {
h.mu.Lock()
defer h.mu.Unlock()
h.rack = rack
return h
}
func (h *HostInfo) HostID() string {
h.mu.RLock()
defer h.mu.RUnlock()
return h.hostId
}
func (h *HostInfo) setHostID(hostID string) *HostInfo {
h.mu.Lock()
defer h.mu.Unlock()
h.hostId = hostID
return h
}
func (h *HostInfo) Version() cassVersion {
h.mu.RLock()
defer h.mu.RUnlock()
return h.version
}
func (h *HostInfo) setVersion(major, minor, patch int) *HostInfo {
h.mu.Lock()
defer h.mu.Unlock()
h.version = cassVersion{major, minor, patch}
return h
}
func (h *HostInfo) State() nodeState {
h.mu.RLock()
defer h.mu.RUnlock()
return h.state
}
func (h *HostInfo) setState(state nodeState) *HostInfo {
h.mu.Lock()
defer h.mu.Unlock()
h.state = state
return h
}
func (h *HostInfo) Tokens() []string {
h.mu.RLock()
defer h.mu.RUnlock()
return h.tokens
}
func (h *HostInfo) setTokens(tokens []string) *HostInfo {
h.mu.Lock()
defer h.mu.Unlock()
h.tokens = tokens
return h
}
func (h *HostInfo) Port() int {
h.mu.RLock()
defer h.mu.RUnlock()
return h.port
}
func (h *HostInfo) setPort(port int) *HostInfo {
h.mu.Lock()
defer h.mu.Unlock()
h.port = port
return h
}
func (h *HostInfo) update(from *HostInfo) {
h.mu.Lock()
defer h.mu.Unlock()
h.tokens = from.tokens
h.version = from.version
h.hostId = from.hostId
h.dataCenter = from.dataCenter
}
func (h *HostInfo) IsUp() bool {
return h.State() == NodeUp
}
func (h *HostInfo) String() string {
h.mu.RLock()
defer h.mu.RUnlock()
return fmt.Sprintf("[hostinfo peer=%q port=%d data_centre=%q rack=%q host_id=%q version=%q state=%s num_tokens=%d]", h.peer, h.port, h.dataCenter, h.rack, h.hostId, h.version, h.state, len(h.tokens))
}
// Polls system.peers at a specific interval to find new hosts
type ringDescriber struct {
dcFilter string
rackFilter string
session *Session
closeChan chan bool
// indicates that we can use system.local to get the connections remote address
localHasRpcAddr bool
mu sync.Mutex
prevHosts []*HostInfo
prevPartitioner string
}
func checkSystemLocal(control *controlConn) (bool, error) {
iter := control.query("SELECT broadcast_address FROM system.local")
if err := iter.err; err != nil {
if errf, ok := err.(*errorFrame); ok {
if errf.code == errSyntax {
return false, nil
}
}
return false, err
}
return true, nil
}
func (r *ringDescriber) GetHosts() (hosts []*HostInfo, partitioner string, err error) {
r.mu.Lock()
defer r.mu.Unlock()
// we need conn to be the same because we need to query system.peers and system.local
// on the same node to get the whole cluster
const (
legacyLocalQuery = "SELECT data_center, rack, host_id, tokens, partitioner, release_version FROM system.local"
// only supported in 2.2.0, 2.1.6, 2.0.16
localQuery = "SELECT broadcast_address, data_center, rack, host_id, tokens, partitioner, release_version FROM system.local"
)
localHost := &HostInfo{}
if r.localHasRpcAddr {
iter := r.session.control.query(localQuery)
if iter == nil {
return r.prevHosts, r.prevPartitioner, nil
}
iter.Scan(&localHost.peer, &localHost.dataCenter, &localHost.rack,
&localHost.hostId, &localHost.tokens, &partitioner, &localHost.version)
if err = iter.Close(); err != nil {
return nil, "", err
}
} else {
iter := r.session.control.query(legacyLocalQuery)
if iter == nil {
return r.prevHosts, r.prevPartitioner, nil
}
iter.Scan(&localHost.dataCenter, &localHost.rack, &localHost.hostId, &localHost.tokens, &partitioner, &localHost.version)
if err = iter.Close(); err != nil {
return nil, "", err
}
addr, _, err := net.SplitHostPort(r.session.control.addr())
if err != nil {
// this should not happen, ever, as this is the address that was dialed by conn, here
// a panic makes sense, please report a bug if it occurs.
panic(err)
}
localHost.peer = addr
}
localHost.port = r.session.cfg.Port
hosts = []*HostInfo{localHost}
iter := r.session.control.query("SELECT rpc_address, data_center, rack, host_id, tokens, release_version FROM system.peers")
if iter == nil {
return r.prevHosts, r.prevPartitioner, nil
}
var (
host = &HostInfo{port: r.session.cfg.Port}
versionBytes []byte
)
for iter.Scan(&host.peer, &host.dataCenter, &host.rack, &host.hostId, &host.tokens, &versionBytes) {
if err = host.version.unmarshal(versionBytes); err != nil {
log.Printf("invalid peer entry: peer=%s host_id=%s tokens=%v version=%s\n", host.peer, host.hostId, host.tokens, versionBytes)
continue
}
if r.matchFilter(host) {
hosts = append(hosts, host)
}
host = &HostInfo{
port: r.session.cfg.Port,
}
}
if err = iter.Close(); err != nil {
return nil, "", err
}
r.prevHosts = hosts
r.prevPartitioner = partitioner
return hosts, partitioner, nil
}
func (r *ringDescriber) matchFilter(host *HostInfo) bool {
if r.dcFilter != "" && r.dcFilter != host.DataCenter() {
return false
}
if r.rackFilter != "" && r.rackFilter != host.Rack() {
return false
}
return true
}
func (r *ringDescriber) refreshRing() error {
// if we have 0 hosts this will return the previous list of hosts to
// attempt to reconnect to the cluster otherwise we would never find
// downed hosts again, could possibly have an optimisation to only
// try to add new hosts if GetHosts didnt error and the hosts didnt change.
hosts, partitioner, err := r.GetHosts()
if err != nil {
return err
}
// TODO: move this to session
// TODO: handle removing hosts here
for _, h := range hosts {
if r.session.cfg.HostFilter == nil || r.session.cfg.HostFilter.Accept(h) {
if host, ok := r.session.ring.addHostIfMissing(h); !ok {
r.session.pool.addHost(h)
} else {
host.update(h)
}
}
}
r.session.metadata.setPartitioner(partitioner)
return nil
}