|
|
@@ -19,7 +19,7 @@ import (
|
|
|
|
|
|
// interface to implement to receive the host information
|
|
|
type SetHosts interface {
|
|
|
- SetHosts(hosts []HostInfo)
|
|
|
+ SetHosts(hosts []*HostInfo)
|
|
|
}
|
|
|
|
|
|
// interface to implement to receive the partitioner value
|
|
|
@@ -62,25 +62,25 @@ type policyConnPool struct {
|
|
|
|
|
|
port int
|
|
|
numConns int
|
|
|
- connCfg *ConnConfig
|
|
|
keyspace string
|
|
|
|
|
|
mu sync.RWMutex
|
|
|
hostPolicy HostSelectionPolicy
|
|
|
connPolicy func() ConnSelectionPolicy
|
|
|
hostConnPools map[string]*hostConnPool
|
|
|
+
|
|
|
+ endpoints []string
|
|
|
}
|
|
|
|
|
|
-func newPolicyConnPool(session *Session, hostPolicy HostSelectionPolicy,
|
|
|
- connPolicy func() ConnSelectionPolicy) (*policyConnPool, error) {
|
|
|
+func connConfig(session *Session) (*ConnConfig, error) {
|
|
|
+ cfg := session.cfg
|
|
|
|
|
|
var (
|
|
|
err error
|
|
|
tlsConfig *tls.Config
|
|
|
)
|
|
|
|
|
|
- cfg := session.cfg
|
|
|
-
|
|
|
+ // TODO(zariel): move tls config setup into session init.
|
|
|
if cfg.SslOpts != nil {
|
|
|
tlsConfig, err = setupTLSConfig(cfg.SslOpts)
|
|
|
if err != nil {
|
|
|
@@ -88,37 +88,38 @@ func newPolicyConnPool(session *Session, hostPolicy HostSelectionPolicy,
|
|
|
}
|
|
|
}
|
|
|
|
|
|
+ return &ConnConfig{
|
|
|
+ ProtoVersion: cfg.ProtoVersion,
|
|
|
+ CQLVersion: cfg.CQLVersion,
|
|
|
+ Timeout: cfg.Timeout,
|
|
|
+ Compressor: cfg.Compressor,
|
|
|
+ Authenticator: cfg.Authenticator,
|
|
|
+ Keepalive: cfg.SocketKeepalive,
|
|
|
+ tlsConfig: tlsConfig,
|
|
|
+ }, nil
|
|
|
+}
|
|
|
+
|
|
|
+func newPolicyConnPool(session *Session, hostPolicy HostSelectionPolicy,
|
|
|
+ connPolicy func() ConnSelectionPolicy) *policyConnPool {
|
|
|
+
|
|
|
// create the pool
|
|
|
pool := &policyConnPool{
|
|
|
- session: session,
|
|
|
- port: cfg.Port,
|
|
|
- numConns: cfg.NumConns,
|
|
|
- connCfg: &ConnConfig{
|
|
|
- ProtoVersion: cfg.ProtoVersion,
|
|
|
- CQLVersion: cfg.CQLVersion,
|
|
|
- Timeout: cfg.Timeout,
|
|
|
- Compressor: cfg.Compressor,
|
|
|
- Authenticator: cfg.Authenticator,
|
|
|
- Keepalive: cfg.SocketKeepalive,
|
|
|
- tlsConfig: tlsConfig,
|
|
|
- },
|
|
|
- keyspace: cfg.Keyspace,
|
|
|
+ session: session,
|
|
|
+ port: session.cfg.Port,
|
|
|
+ numConns: session.cfg.NumConns,
|
|
|
+ keyspace: session.cfg.Keyspace,
|
|
|
hostPolicy: hostPolicy,
|
|
|
connPolicy: connPolicy,
|
|
|
hostConnPools: map[string]*hostConnPool{},
|
|
|
}
|
|
|
|
|
|
- hosts := make([]HostInfo, len(cfg.Hosts))
|
|
|
- for i, hostAddr := range cfg.Hosts {
|
|
|
- hosts[i].Peer = hostAddr
|
|
|
- }
|
|
|
-
|
|
|
- pool.SetHosts(hosts)
|
|
|
+ pool.endpoints = make([]string, len(session.cfg.Hosts))
|
|
|
+ copy(pool.endpoints, session.cfg.Hosts)
|
|
|
|
|
|
- return pool, nil
|
|
|
+ return pool
|
|
|
}
|
|
|
|
|
|
-func (p *policyConnPool) SetHosts(hosts []HostInfo) {
|
|
|
+func (p *policyConnPool) SetHosts(hosts []*HostInfo) {
|
|
|
p.mu.Lock()
|
|
|
defer p.mu.Unlock()
|
|
|
|
|
|
@@ -129,24 +130,22 @@ func (p *policyConnPool) SetHosts(hosts []HostInfo) {
|
|
|
|
|
|
// TODO connect to hosts in parallel, but wait for pools to be
|
|
|
// created before returning
|
|
|
-
|
|
|
- for i := range hosts {
|
|
|
- pool, exists := p.hostConnPools[hosts[i].Peer]
|
|
|
- if !exists {
|
|
|
+ for _, host := range hosts {
|
|
|
+ pool, exists := p.hostConnPools[host.Peer()]
|
|
|
+ if !exists && host.IsUp() {
|
|
|
// create a connection pool for the host
|
|
|
pool = newHostConnPool(
|
|
|
p.session,
|
|
|
- hosts[i].Peer,
|
|
|
+ host,
|
|
|
p.port,
|
|
|
p.numConns,
|
|
|
- p.connCfg,
|
|
|
p.keyspace,
|
|
|
p.connPolicy(),
|
|
|
)
|
|
|
- p.hostConnPools[hosts[i].Peer] = pool
|
|
|
+ p.hostConnPools[host.Peer()] = pool
|
|
|
} else {
|
|
|
// still have this host, so don't remove it
|
|
|
- delete(toRemove, hosts[i].Peer)
|
|
|
+ delete(toRemove, host.Peer())
|
|
|
}
|
|
|
}
|
|
|
|
|
|
@@ -158,7 +157,6 @@ func (p *policyConnPool) SetHosts(hosts []HostInfo) {
|
|
|
|
|
|
// update the policy
|
|
|
p.hostPolicy.SetHosts(hosts)
|
|
|
-
|
|
|
}
|
|
|
|
|
|
func (p *policyConnPool) SetPartitioner(partitioner string) {
|
|
|
@@ -194,7 +192,7 @@ func (p *policyConnPool) Pick(qry *Query) (SelectedHost, *Conn) {
|
|
|
panic(fmt.Sprintf("policy %T returned no host info: %+v", p.hostPolicy, host))
|
|
|
}
|
|
|
|
|
|
- pool, ok := p.hostConnPools[host.Info().Peer]
|
|
|
+ pool, ok := p.hostConnPools[host.Info().Peer()]
|
|
|
if !ok {
|
|
|
continue
|
|
|
}
|
|
|
@@ -209,7 +207,7 @@ func (p *policyConnPool) Close() {
|
|
|
defer p.mu.Unlock()
|
|
|
|
|
|
// remove the hosts from the policy
|
|
|
- p.hostPolicy.SetHosts([]HostInfo{})
|
|
|
+ p.hostPolicy.SetHosts(nil)
|
|
|
|
|
|
// close the pools
|
|
|
for addr, pool := range p.hostConnPools {
|
|
|
@@ -218,15 +216,69 @@ func (p *policyConnPool) Close() {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
+func (p *policyConnPool) addHost(host *HostInfo) {
|
|
|
+ p.mu.Lock()
|
|
|
+ defer p.mu.Unlock()
|
|
|
+
|
|
|
+ pool, ok := p.hostConnPools[host.Peer()]
|
|
|
+ if ok {
|
|
|
+ go pool.fill()
|
|
|
+ return
|
|
|
+ }
|
|
|
+
|
|
|
+ pool = newHostConnPool(
|
|
|
+ p.session,
|
|
|
+ host,
|
|
|
+ host.Port(),
|
|
|
+ p.numConns,
|
|
|
+ p.keyspace,
|
|
|
+ p.connPolicy(),
|
|
|
+ )
|
|
|
+
|
|
|
+ p.hostConnPools[host.Peer()] = pool
|
|
|
+
|
|
|
+ // update policy
|
|
|
+ // TODO: policy should not have conns, it should have hosts and return a host
|
|
|
+ // iter which the pool will use to serve conns
|
|
|
+ p.hostPolicy.AddHost(host)
|
|
|
+}
|
|
|
+
|
|
|
+func (p *policyConnPool) removeHost(addr string) {
|
|
|
+ p.hostPolicy.RemoveHost(addr)
|
|
|
+ p.mu.Lock()
|
|
|
+
|
|
|
+ pool, ok := p.hostConnPools[addr]
|
|
|
+ if !ok {
|
|
|
+ p.mu.Unlock()
|
|
|
+ return
|
|
|
+ }
|
|
|
+
|
|
|
+ delete(p.hostConnPools, addr)
|
|
|
+ p.mu.Unlock()
|
|
|
+
|
|
|
+ pool.Close()
|
|
|
+}
|
|
|
+
|
|
|
+func (p *policyConnPool) hostUp(host *HostInfo) {
|
|
|
+ // TODO(zariel): have a set of up hosts and down hosts, we can internally
|
|
|
+ // detect down hosts, then try to reconnect to them.
|
|
|
+ p.addHost(host)
|
|
|
+}
|
|
|
+
|
|
|
+func (p *policyConnPool) hostDown(addr string) {
|
|
|
+ // TODO(zariel): mark host as down so we can try to connect to it later, for
|
|
|
+ // now just treat it has removed.
|
|
|
+ p.removeHost(addr)
|
|
|
+}
|
|
|
+
|
|
|
// hostConnPool is a connection pool for a single host.
|
|
|
// Connection selection is based on a provided ConnSelectionPolicy
|
|
|
type hostConnPool struct {
|
|
|
session *Session
|
|
|
- host string
|
|
|
+ host *HostInfo
|
|
|
port int
|
|
|
addr string
|
|
|
size int
|
|
|
- connCfg *ConnConfig
|
|
|
keyspace string
|
|
|
policy ConnSelectionPolicy
|
|
|
// protection for conns, closed, filling
|
|
|
@@ -236,16 +288,22 @@ type hostConnPool struct {
|
|
|
filling bool
|
|
|
}
|
|
|
|
|
|
-func newHostConnPool(session *Session, host string, port, size int, connCfg *ConnConfig,
|
|
|
+func (h *hostConnPool) String() string {
|
|
|
+ h.mu.RLock()
|
|
|
+ defer h.mu.RUnlock()
|
|
|
+ return fmt.Sprintf("[filling=%v closed=%v conns=%v size=%v host=%v]",
|
|
|
+ h.filling, h.closed, len(h.conns), h.size, h.host)
|
|
|
+}
|
|
|
+
|
|
|
+func newHostConnPool(session *Session, host *HostInfo, port, size int,
|
|
|
keyspace string, policy ConnSelectionPolicy) *hostConnPool {
|
|
|
|
|
|
pool := &hostConnPool{
|
|
|
session: session,
|
|
|
host: host,
|
|
|
port: port,
|
|
|
- addr: JoinHostPort(host, port),
|
|
|
+ addr: JoinHostPort(host.Peer(), port),
|
|
|
size: size,
|
|
|
- connCfg: connCfg,
|
|
|
keyspace: keyspace,
|
|
|
policy: policy,
|
|
|
conns: make([]*Conn, 0, size),
|
|
|
@@ -267,13 +325,16 @@ func (pool *hostConnPool) Pick(qry *Query) *Conn {
|
|
|
return nil
|
|
|
}
|
|
|
|
|
|
- empty := len(pool.conns) == 0
|
|
|
+ size := len(pool.conns)
|
|
|
pool.mu.RUnlock()
|
|
|
|
|
|
- if empty {
|
|
|
- // try to fill the empty pool
|
|
|
+ if size < pool.size {
|
|
|
+ // try to fill the pool
|
|
|
go pool.fill()
|
|
|
- return nil
|
|
|
+
|
|
|
+ if size == 0 {
|
|
|
+ return nil
|
|
|
+ }
|
|
|
}
|
|
|
|
|
|
return pool.policy.Pick(qry)
|
|
|
@@ -350,7 +411,11 @@ func (pool *hostConnPool) fill() {
|
|
|
|
|
|
if err != nil {
|
|
|
// probably unreachable host
|
|
|
- go pool.fillingStopped()
|
|
|
+ pool.fillingStopped()
|
|
|
+
|
|
|
+ // this is calle with the connetion pool mutex held, this call will
|
|
|
+ // then recursivly try to lock it again. FIXME
|
|
|
+ go pool.session.handleNodeDown(net.ParseIP(pool.host.Peer()), pool.port)
|
|
|
return
|
|
|
}
|
|
|
|
|
|
@@ -366,7 +431,7 @@ func (pool *hostConnPool) fill() {
|
|
|
fillCount--
|
|
|
}
|
|
|
|
|
|
- go pool.fillingStopped()
|
|
|
+ pool.fillingStopped()
|
|
|
return
|
|
|
}
|
|
|
|
|
|
@@ -410,7 +475,7 @@ func (pool *hostConnPool) fillingStopped() {
|
|
|
// create a new connection to the host and add it to the pool
|
|
|
func (pool *hostConnPool) connect() error {
|
|
|
// try to connect
|
|
|
- conn, err := Connect(pool.addr, pool.connCfg, pool, pool.session)
|
|
|
+ conn, err := pool.session.connect(pool.addr, pool)
|
|
|
if err != nil {
|
|
|
return err
|
|
|
}
|
|
|
@@ -433,7 +498,11 @@ func (pool *hostConnPool) connect() error {
|
|
|
}
|
|
|
|
|
|
pool.conns = append(pool.conns, conn)
|
|
|
- pool.policy.SetConns(pool.conns)
|
|
|
+
|
|
|
+ conns := make([]*Conn, len(pool.conns))
|
|
|
+ copy(conns, pool.conns)
|
|
|
+ pool.policy.SetConns(conns)
|
|
|
+
|
|
|
return nil
|
|
|
}
|
|
|
|
|
|
@@ -444,6 +513,8 @@ func (pool *hostConnPool) HandleError(conn *Conn, err error, closed bool) {
|
|
|
return
|
|
|
}
|
|
|
|
|
|
+ // TODO: track the number of errors per host and detect when a host is dead,
|
|
|
+ // then also have something which can detect when a host comes back.
|
|
|
pool.mu.Lock()
|
|
|
defer pool.mu.Unlock()
|
|
|
|
|
|
@@ -459,7 +530,9 @@ func (pool *hostConnPool) HandleError(conn *Conn, err error, closed bool) {
|
|
|
pool.conns[i], pool.conns = pool.conns[len(pool.conns)-1], pool.conns[:len(pool.conns)-1]
|
|
|
|
|
|
// update the policy
|
|
|
- pool.policy.SetConns(pool.conns)
|
|
|
+ conns := make([]*Conn, len(pool.conns))
|
|
|
+ copy(conns, pool.conns)
|
|
|
+ pool.policy.SetConns(conns)
|
|
|
|
|
|
// lost a connection, so fill the pool
|
|
|
go pool.fill()
|
|
|
@@ -475,10 +548,10 @@ func (pool *hostConnPool) drain() {
|
|
|
|
|
|
// empty the pool
|
|
|
conns := pool.conns
|
|
|
- pool.conns = pool.conns[:0]
|
|
|
+ pool.conns = pool.conns[:0:0]
|
|
|
|
|
|
// update the policy
|
|
|
- pool.policy.SetConns(pool.conns)
|
|
|
+ pool.policy.SetConns(nil)
|
|
|
|
|
|
// close the connections
|
|
|
for _, conn := range conns {
|