forked from apache/cassandra-gocql-driver
-
Notifications
You must be signed in to change notification settings - Fork 2
/
connectionpool.go
393 lines (331 loc) · 9.28 KB
/
connectionpool.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
package gocql
import (
"fmt"
"log"
"strings"
"sync"
"time"
)
/*ConnectionPool represents the interface gocql will use to work with a collection of connections.
Purpose
The connection pool in gocql opens and closes connections as well as selects an available connection
for gocql to execute a query against. The pool is also respnsible for handling connection errors that
are caught by the connection experiencing the error.
A connection pool should make a copy of the variables used from the ClusterConfig provided to the pool
upon creation. ClusterConfig is a pointer and can be modified after the creation of the pool. This can
lead to issues with variables being modified outside the expectations of the ConnectionPool type.
Example of Single Connection Pool:
type SingleConnection struct {
conn *Conn
cfg *ClusterConfig
}
func NewSingleConnection(cfg *ClusterConfig) ConnectionPool {
addr := strings.TrimSpace(cfg.Hosts[0])
if strings.Index(addr, ":") < 0 {
addr = fmt.Sprintf("%s:%d", addr, cfg.Port)
}
connCfg := ConnConfig{
ProtoVersion: cfg.ProtoVersion,
CQLVersion: cfg.CQLVersion,
Timeout: cfg.Timeout,
NumStreams: cfg.NumStreams,
Compressor: cfg.Compressor,
Authenticator: cfg.Authenticator,
Keepalive: cfg.SocketKeepalive,
}
pool := SingleConnection{cfg:cfg}
pool.conn = Connect(addr,connCfg,pool)
return &pool
}
func (s *SingleConnection) HandleError(conn *Conn, err error, closed bool) {
if closed {
connCfg := ConnConfig{
ProtoVersion: cfg.ProtoVersion,
CQLVersion: cfg.CQLVersion,
Timeout: cfg.Timeout,
NumStreams: cfg.NumStreams,
Compressor: cfg.Compressor,
Authenticator: cfg.Authenticator,
Keepalive: cfg.SocketKeepalive,
}
s.conn = Connect(conn.Address(),connCfg,s)
}
}
func (s *SingleConnection) Pick(qry *Query) *Conn {
if s.conn.isClosed {
return nil
}
return s.conn
}
func (s *SingleConnection) Size() int {
return 1
}
func (s *SingleConnection) Close() {
s.conn.Close()
}
This is a very simple example of a type that exposes the connection pool interface. To assign
this type as the connection pool to use you would assign it to the ClusterConfig like so:
cluster := NewCluster("127.0.0.1")
cluster.ConnPoolType = NewSingleConnection
...
session, err := cluster.CreateSession()
To see a more complete example of a ConnectionPool implementation please see the SimplePool type.
*/
type ConnectionPool interface {
Pick(*Query) *Conn
Size() int
HandleError(*Conn, error, bool)
Close()
SetHosts(host []HostInfo)
}
//NewPoolFunc is the type used by ClusterConfig to create a pool of a specific type.
type NewPoolFunc func(*ClusterConfig) ConnectionPool
//SimplePool is the current implementation of the connection pool inside gocql. This
//pool is meant to be a simple default used by gocql so users can get up and running
//quickly.
type SimplePool struct {
cfg *ClusterConfig
hostPool *RoundRobin
connPool map[string]*RoundRobin
conns map[*Conn]struct{}
keyspace string
hostMu sync.RWMutex
// this is the set of current hosts which the pool will attempt to connect to
hosts map[string]*HostInfo
// protects hostpool, connPoll, conns, quit
mu sync.Mutex
cFillingPool chan int
quit bool
quitWait chan bool
quitOnce sync.Once
}
//NewSimplePool is the function used by gocql to create the simple connection pool.
//This is the default if no other pool type is specified.
func NewSimplePool(cfg *ClusterConfig) ConnectionPool {
pool := &SimplePool{
cfg: cfg,
hostPool: NewRoundRobin(),
connPool: make(map[string]*RoundRobin),
conns: make(map[*Conn]struct{}),
quitWait: make(chan bool),
cFillingPool: make(chan int, 1),
keyspace: cfg.Keyspace,
hosts: make(map[string]*HostInfo),
}
for _, host := range cfg.Hosts {
// seed hosts have unknown topology
// TODO: Handle populating this during SetHosts
pool.hosts[host] = &HostInfo{Peer: host}
}
//Walk through connecting to hosts. As soon as one host connects
//defer the remaining connections to cluster.fillPool()
for i := 0; i < len(cfg.Hosts); i++ {
addr := strings.TrimSpace(cfg.Hosts[i])
if strings.Index(addr, ":") < 0 {
addr = fmt.Sprintf("%s:%d", addr, cfg.Port)
}
if pool.connect(addr) == nil {
pool.cFillingPool <- 1
go pool.fillPool()
break
}
}
return pool
}
func (c *SimplePool) connect(addr string) error {
cfg := ConnConfig{
ProtoVersion: c.cfg.ProtoVersion,
CQLVersion: c.cfg.CQLVersion,
Timeout: c.cfg.Timeout,
NumStreams: c.cfg.NumStreams,
Compressor: c.cfg.Compressor,
Authenticator: c.cfg.Authenticator,
Keepalive: c.cfg.SocketKeepalive,
}
conn, err := Connect(addr, cfg, c)
if err != nil {
log.Printf("connect: failed to connect to %q: %v", addr, err)
return err
}
return c.addConn(conn)
}
func (c *SimplePool) addConn(conn *Conn) error {
c.mu.Lock()
defer c.mu.Unlock()
if c.quit {
conn.Close()
return nil
}
//Set the connection's keyspace if any before adding it to the pool
if c.keyspace != "" {
if err := conn.UseKeyspace(c.keyspace); err != nil {
log.Printf("error setting connection keyspace. %v", err)
conn.Close()
return err
}
}
connPool := c.connPool[conn.Address()]
if connPool == nil {
connPool = NewRoundRobin()
c.connPool[conn.Address()] = connPool
c.hostPool.AddNode(connPool)
}
connPool.AddNode(conn)
c.conns[conn] = struct{}{}
return nil
}
//fillPool manages the pool of connections making sure that each host has the correct
//amount of connections defined. Also the method will test a host with one connection
//instead of flooding the host with number of connections defined in the cluster config
func (c *SimplePool) fillPool() {
//Debounce large amounts of requests to fill pool
select {
case <-time.After(1 * time.Millisecond):
return
case <-c.cFillingPool:
defer func() { c.cFillingPool <- 1 }()
}
c.mu.Lock()
isClosed := c.quit
c.mu.Unlock()
//Exit if cluster(session) is closed
if isClosed {
return
}
c.hostMu.RLock()
//Walk through list of defined hosts
for host := range c.hosts {
addr := strings.TrimSpace(host)
if strings.Index(addr, ":") < 0 {
addr = fmt.Sprintf("%s:%d", addr, c.cfg.Port)
}
numConns := 1
//See if the host already has connections in the pool
c.mu.Lock()
conns, ok := c.connPool[addr]
c.mu.Unlock()
if ok {
//if the host has enough connections just exit
numConns = conns.Size()
if numConns >= c.cfg.NumConns {
continue
}
} else {
//See if the host is reachable
if err := c.connect(addr); err != nil {
continue
}
}
//This is reached if the host is responsive and needs more connections
//Create connections for host synchronously to mitigate flooding the host.
go func(a string, conns int) {
for ; conns < c.cfg.NumConns; conns++ {
c.connect(addr)
}
}(addr, numConns)
}
c.hostMu.RUnlock()
}
// Should only be called if c.mu is locked
func (c *SimplePool) removeConnLocked(conn *Conn) {
conn.Close()
connPool := c.connPool[conn.addr]
if connPool == nil {
return
}
connPool.RemoveNode(conn)
if connPool.Size() == 0 {
c.hostPool.RemoveNode(connPool)
delete(c.connPool, conn.addr)
}
delete(c.conns, conn)
}
func (c *SimplePool) removeConn(conn *Conn) {
c.mu.Lock()
defer c.mu.Unlock()
c.removeConnLocked(conn)
}
//HandleError is called by a Connection object to report to the pool an error has occured.
//Logic is then executed within the pool to clean up the erroroneous connection and try to
//top off the pool.
func (c *SimplePool) HandleError(conn *Conn, err error, closed bool) {
if !closed {
// ignore all non-fatal errors
return
}
c.removeConn(conn)
if !c.quit {
go c.fillPool() // top off pool.
}
}
//Pick selects a connection to be used by the query.
func (c *SimplePool) Pick(qry *Query) *Conn {
//Check if connections are available
c.mu.Lock()
conns := len(c.conns)
c.mu.Unlock()
if conns == 0 {
//try to populate the pool before returning.
c.fillPool()
}
return c.hostPool.Pick(qry)
}
//Size returns the number of connections currently active in the pool
func (p *SimplePool) Size() int {
p.mu.Lock()
conns := len(p.conns)
p.mu.Unlock()
return conns
}
//Close kills the pool and all associated connections.
func (c *SimplePool) Close() {
c.quitOnce.Do(func() {
c.mu.Lock()
defer c.mu.Unlock()
c.quit = true
close(c.quitWait)
for conn := range c.conns {
c.removeConnLocked(conn)
}
})
}
func (c *SimplePool) SetHosts(hosts []HostInfo) {
c.hostMu.Lock()
toRemove := make(map[string]struct{})
for k := range c.hosts {
toRemove[k] = struct{}{}
}
for _, host := range hosts {
host := host
delete(toRemove, host.Peer)
// we already have it
if _, ok := c.hosts[host.Peer]; ok {
// TODO: Check rack, dc, token range is consistent, trigger topology change
// update stored host
continue
}
c.hosts[host.Peer] = &host
}
// can we hold c.mu whilst iterating this loop?
for addr := range toRemove {
c.removeHostLocked(addr)
}
c.hostMu.Unlock()
c.fillPool()
}
func (c *SimplePool) removeHostLocked(addr string) {
if _, ok := c.hosts[addr]; !ok {
return
}
delete(c.hosts, addr)
c.mu.Lock()
defer c.mu.Unlock()
if _, ok := c.connPool[addr]; !ok {
return
}
for conn := range c.conns {
if conn.Address() == addr {
c.removeConnLocked(conn)
}
}
}