Skip to content

Commit

Permalink
start replacing host discovery with events
Browse files Browse the repository at this point in the history
  • Loading branch information
Zariel committed Dec 25, 2015
1 parent 732282f commit 5c1118a
Show file tree
Hide file tree
Showing 10 changed files with 239 additions and 148 deletions.
8 changes: 0 additions & 8 deletions cassandra_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -165,7 +165,6 @@ func TestAuthentication(t *testing.T) {
func TestRingDiscovery(t *testing.T) {
cluster := createCluster()
cluster.Hosts = clusterHosts[:1]
cluster.DiscoverHosts = true

session := createSessionFromCluster(cluster, t)
defer session.Close()
Expand Down Expand Up @@ -649,10 +648,6 @@ func TestCreateSessionTimeout(t *testing.T) {
session.Close()
t.Fatal("expected ErrNoConnectionsStarted, but no error was returned.")
}

if err != ErrNoConnectionsStarted {
t.Fatalf("expected ErrNoConnectionsStarted, but received %v", err)
}
}

type FullName struct {
Expand Down Expand Up @@ -2001,7 +1996,6 @@ func TestRoutingKey(t *testing.T) {
func TestTokenAwareConnPool(t *testing.T) {
cluster := createCluster()
cluster.PoolConfig.HostSelectionPolicy = TokenAwareHostPolicy(RoundRobinHostPolicy())
cluster.DiscoverHosts = true

session := createSessionFromCluster(cluster, t)
defer session.Close()
Expand Down Expand Up @@ -2379,9 +2373,7 @@ func TestDiscoverViaProxy(t *testing.T) {
proxyAddr := proxy.Addr().String()

cluster := createCluster()
cluster.DiscoverHosts = true
cluster.NumConns = 1
cluster.Discovery.Sleep = 100 * time.Millisecond
// initial host is the proxy address
cluster.Hosts = []string{proxyAddr}

Expand Down
15 changes: 2 additions & 13 deletions cluster.go
Original file line number Diff line number Diff line change
Expand Up @@ -40,16 +40,6 @@ func initStmtsLRU(max int) {
}
}

// To enable periodic node discovery enable DiscoverHosts in ClusterConfig
type DiscoveryConfig struct {
// If not empty will filter all discoverred hosts to a single Data Centre (default: "")
DcFilter string
// If not empty will filter all discoverred hosts to a single Rack (default: "")
RackFilter string
// The interval to check for new hosts (default: 30s)
Sleep time.Duration
}

// PoolConfig configures the connection pool used by the driver, it defaults to
// using a round robbin host selection policy and a round robbin connection selection
// policy for each host.
Expand Down Expand Up @@ -94,12 +84,10 @@ type ClusterConfig struct {
Authenticator Authenticator // authenticator (default: nil)
RetryPolicy RetryPolicy // Default retry policy to use for queries (default: 0)
SocketKeepalive time.Duration // The keepalive period to use, enabled if > 0 (default: 0)
DiscoverHosts bool // If set, gocql will attempt to automatically discover other members of the Cassandra cluster (default: false)
MaxPreparedStmts int // Sets the maximum cache size for prepared statements globally for gocql (default: 1000)
MaxRoutingKeyInfo int // Sets the maximum cache size for query info about statements for each session (default: 1000)
PageSize int // Default page size to use for created sessions (default: 5000)
SerialConsistency SerialConsistency // Sets the consistency for the serial part of queries, values can be either SERIAL or LOCAL_SERIAL (default: unset)
Discovery DiscoveryConfig
SslOpts *SslOptions
DefaultTimestamp bool // Sends a client side timestamp for all requests which overrides the timestamp at which it arrives at the server. (default: true, only enabled for protocol 3 and above)
// PoolConfig configures the underlying connection pool, allowing the
Expand All @@ -110,6 +98,8 @@ type ClusterConfig struct {
// receiving a schema change frame. (deault: 60s)
MaxWaitSchemaAgreement time.Duration

HostFilter

// internal config for testing
disableControlConn bool
}
Expand All @@ -124,7 +114,6 @@ func NewCluster(hosts ...string) *ClusterConfig {
Port: 9042,
NumConns: 2,
Consistency: Quorum,
DiscoverHosts: false,
MaxPreparedStmts: defaultMaxPreparedStmts,
MaxRoutingKeyInfo: 1000,
PageSize: 5000,
Expand Down
1 change: 0 additions & 1 deletion conn.go
Original file line number Diff line number Diff line change
Expand Up @@ -141,7 +141,6 @@ type Conn struct {
}

// Connect establishes a connection to a Cassandra node.
// You must also call the Serve method before you can execute any queries.
func Connect(addr string, cfg *ConnConfig, errorHandler ConnErrorHandler, session *Session) (*Conn, error) {
var (
err error
Expand Down
43 changes: 22 additions & 21 deletions conn_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -50,12 +50,18 @@ func TestJoinHostPort(t *testing.T) {
}
}

func testCluster(addr string, proto protoVersion) *ClusterConfig {
cluster := NewCluster(addr)
cluster.ProtoVersion = int(proto)
cluster.disableControlConn = true
return cluster
}

func TestSimple(t *testing.T) {
srv := NewTestServer(t, defaultProto)
defer srv.Stop()

cluster := NewCluster(srv.Address)
cluster.ProtoVersion = int(defaultProto)
cluster := testCluster(srv.Address, defaultProto)
db, err := cluster.CreateSession()
if err != nil {
t.Fatalf("0x%x: NewCluster: %v", defaultProto, err)
Expand Down Expand Up @@ -94,18 +100,19 @@ func TestSSLSimpleNoClientCert(t *testing.T) {
}
}

func createTestSslCluster(hosts string, proto uint8, useClientCert bool) *ClusterConfig {
cluster := NewCluster(hosts)
func createTestSslCluster(addr string, proto protoVersion, useClientCert bool) *ClusterConfig {
cluster := testCluster(addr, proto)
sslOpts := &SslOptions{
CaPath: "testdata/pki/ca.crt",
EnableHostVerification: false,
}

if useClientCert {
sslOpts.CertPath = "testdata/pki/gocql.crt"
sslOpts.KeyPath = "testdata/pki/gocql.key"
}

cluster.SslOpts = sslOpts
cluster.ProtoVersion = int(proto)
return cluster
}

Expand All @@ -115,28 +122,23 @@ func TestClosed(t *testing.T) {
srv := NewTestServer(t, defaultProto)
defer srv.Stop()

cluster := NewCluster(srv.Address)
cluster.ProtoVersion = int(defaultProto)

session, err := cluster.CreateSession()
defer session.Close()
session, err := newTestSession(srv.Address, defaultProto)
if err != nil {
t.Fatalf("0x%x: NewCluster: %v", defaultProto, err)
}

session.Close()

if err := session.Query("void").Exec(); err != ErrSessionClosed {
t.Fatalf("0x%x: expected %#v, got %#v", defaultProto, ErrSessionClosed, err)
}
}

func newTestSession(addr string, proto uint8) (*Session, error) {
cluster := NewCluster(addr)
cluster.ProtoVersion = int(proto)
return cluster.CreateSession()
func newTestSession(addr string, proto protoVersion) (*Session, error) {
return testCluster(addr, proto).CreateSession()
}

func TestTimeout(t *testing.T) {

srv := NewTestServer(t, defaultProto)
defer srv.Stop()

Expand Down Expand Up @@ -197,7 +199,7 @@ func TestStreams_Protocol1(t *testing.T) {

// TODO: these are more like session tests and should instead operate
// on a single Conn
cluster := NewCluster(srv.Address)
cluster := testCluster(srv.Address, protoVersion1)
cluster.NumConns = 1
cluster.ProtoVersion = 1

Expand Down Expand Up @@ -229,7 +231,7 @@ func TestStreams_Protocol3(t *testing.T) {

// TODO: these are more like session tests and should instead operate
// on a single Conn
cluster := NewCluster(srv.Address)
cluster := testCluster(srv.Address, protoVersion3)
cluster.NumConns = 1
cluster.ProtoVersion = 3

Expand Down Expand Up @@ -356,7 +358,6 @@ func TestPolicyConnPoolSSL(t *testing.T) {

db, err := cluster.CreateSession()
if err != nil {
db.Close()
t.Fatalf("failed to create new session: %v", err)
}

Expand All @@ -377,7 +378,7 @@ func TestQueryTimeout(t *testing.T) {
srv := NewTestServer(t, defaultProto)
defer srv.Stop()

cluster := NewCluster(srv.Address)
cluster := testCluster(srv.Address, defaultProto)
// Set the timeout arbitrarily low so that the query hits the timeout in a
// timely manner.
cluster.Timeout = 1 * time.Millisecond
Expand Down Expand Up @@ -418,7 +419,7 @@ func TestQueryTimeoutReuseStream(t *testing.T) {
srv := NewTestServer(t, defaultProto)
defer srv.Stop()

cluster := NewCluster(srv.Address)
cluster := testCluster(srv.Address, defaultProto)
// Set the timeout arbitrarily low so that the query hits the timeout in a
// timely manner.
cluster.Timeout = 1 * time.Millisecond
Expand All @@ -442,7 +443,7 @@ func TestQueryTimeoutClose(t *testing.T) {
srv := NewTestServer(t, defaultProto)
defer srv.Stop()

cluster := NewCluster(srv.Address)
cluster := testCluster(srv.Address, defaultProto)
// Set the timeout arbitrarily low so that the query hits the timeout in a
// timely manner.
cluster.Timeout = 1000 * time.Millisecond
Expand Down
63 changes: 44 additions & 19 deletions connectionpool.go
Original file line number Diff line number Diff line change
Expand Up @@ -71,45 +71,57 @@ type policyConnPool struct {
hostConnPools map[string]*hostConnPool
}

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 {
return nil, err
}
}

return &ConnConfig{
ProtoVersion: cfg.ProtoVersion,
CQLVersion: cfg.CQLVersion,
Timeout: cfg.Timeout,
NumStreams: cfg.NumStreams,
Compressor: cfg.Compressor,
Authenticator: cfg.Authenticator,
Keepalive: cfg.SocketKeepalive,
tlsConfig: tlsConfig,
}, nil
}

func newPolicyConnPool(session *Session, hostPolicy HostSelectionPolicy,
connPolicy func() ConnSelectionPolicy) (*policyConnPool, error) {

connCfg, err := connConfig(session)
if err != nil {
return nil, err
}

// 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,
connCfg: connCfg,
keyspace: session.cfg.Keyspace,
hostPolicy: hostPolicy,
connPolicy: connPolicy,
hostConnPools: map[string]*hostConnPool{},
}

hosts := make([]HostInfo, len(cfg.Hosts))
for i, hostAddr := range cfg.Hosts {
// TODO(zariel): fetch this from session metadata.
hosts := make([]HostInfo, len(session.cfg.Hosts))
for i, hostAddr := range session.cfg.Hosts {
hosts[i].Peer = hostAddr
}

Expand Down Expand Up @@ -241,6 +253,7 @@ func (p *policyConnPool) addHost(host *HostInfo) {
}

func (p *policyConnPool) removeHost(addr string) {
p.hostPolicy.RemoveHost(addr)
p.mu.Lock()

pool, ok := p.hostConnPools[addr]
Expand All @@ -255,6 +268,18 @@ func (p *policyConnPool) removeHost(addr string) {
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 {
Expand Down
Loading

0 comments on commit 5c1118a

Please sign in to comment.