|
|
@@ -200,11 +200,18 @@ type HostStateNotifier interface {
|
|
|
HostDown(host *HostInfo)
|
|
|
}
|
|
|
|
|
|
+type KeyspaceUpdateEvent struct {
|
|
|
+ Keyspace string
|
|
|
+ Change string
|
|
|
+}
|
|
|
+
|
|
|
// HostSelectionPolicy is an interface for selecting
|
|
|
// the most appropriate host to execute a given query.
|
|
|
type HostSelectionPolicy interface {
|
|
|
HostStateNotifier
|
|
|
SetPartitioner
|
|
|
+ KeyspaceChanged(KeyspaceUpdateEvent)
|
|
|
+ Init(*Session)
|
|
|
//Pick returns an iteration function over selected hosts
|
|
|
Pick(ExecutableQuery) NextHost
|
|
|
}
|
|
|
@@ -239,9 +246,9 @@ type roundRobinHostPolicy struct {
|
|
|
mu sync.RWMutex
|
|
|
}
|
|
|
|
|
|
-func (r *roundRobinHostPolicy) SetPartitioner(partitioner string) {
|
|
|
- // noop
|
|
|
-}
|
|
|
+func (r *roundRobinHostPolicy) KeyspaceChanged(KeyspaceUpdateEvent) {}
|
|
|
+func (r *roundRobinHostPolicy) SetPartitioner(partitioner string) {}
|
|
|
+func (r *roundRobinHostPolicy) Init(*Session) {}
|
|
|
|
|
|
func (r *roundRobinHostPolicy) Pick(qry ExecutableQuery) NextHost {
|
|
|
// i is used to limit the number of attempts to find a host
|
|
|
@@ -281,19 +288,69 @@ func (r *roundRobinHostPolicy) HostDown(host *HostInfo) {
|
|
|
r.RemoveHost(host)
|
|
|
}
|
|
|
|
|
|
+func ShuffleReplicas() func(*tokenAwareHostPolicy) {
|
|
|
+ return func(t *tokenAwareHostPolicy) {
|
|
|
+ t.shuffleReplicas = true
|
|
|
+ }
|
|
|
+}
|
|
|
+
|
|
|
// TokenAwareHostPolicy is a token aware host selection policy, where hosts are
|
|
|
// selected based on the partition key, so queries are sent to the host which
|
|
|
// owns the partition. Fallback is used when routing information is not available.
|
|
|
-func TokenAwareHostPolicy(fallback HostSelectionPolicy) HostSelectionPolicy {
|
|
|
- return &tokenAwareHostPolicy{fallback: fallback}
|
|
|
+func TokenAwareHostPolicy(fallback HostSelectionPolicy, opts ...func(*tokenAwareHostPolicy)) HostSelectionPolicy {
|
|
|
+ p := &tokenAwareHostPolicy{fallback: fallback}
|
|
|
+ for _, opt := range opts {
|
|
|
+ opt(p)
|
|
|
+ }
|
|
|
+ return p
|
|
|
+}
|
|
|
+
|
|
|
+type keyspaceMeta struct {
|
|
|
+ replicas map[string]map[token][]*HostInfo
|
|
|
}
|
|
|
|
|
|
type tokenAwareHostPolicy struct {
|
|
|
hosts cowHostList
|
|
|
mu sync.RWMutex
|
|
|
partitioner string
|
|
|
- tokenRing *tokenRing
|
|
|
fallback HostSelectionPolicy
|
|
|
+ session *Session
|
|
|
+
|
|
|
+ tokenRing atomic.Value // *tokenRing
|
|
|
+ keyspaces atomic.Value // *keyspaceMeta
|
|
|
+
|
|
|
+ shuffleReplicas bool
|
|
|
+}
|
|
|
+
|
|
|
+func (t *tokenAwareHostPolicy) Init(s *Session) {
|
|
|
+ t.session = s
|
|
|
+}
|
|
|
+
|
|
|
+func (t *tokenAwareHostPolicy) KeyspaceChanged(update KeyspaceUpdateEvent) {
|
|
|
+ meta, _ := t.keyspaces.Load().(*keyspaceMeta)
|
|
|
+ // TODO: avoid recaulating things which havnt changed
|
|
|
+ newMeta := &keyspaceMeta{
|
|
|
+ replicas: make(map[string]map[token][]*HostInfo, len(meta.replicas)),
|
|
|
+ }
|
|
|
+
|
|
|
+ ks, err := t.session.KeyspaceMetadata(update.Keyspace)
|
|
|
+ if err == nil {
|
|
|
+ strat := getStrategy(ks)
|
|
|
+ tr := t.tokenRing.Load().(*tokenRing)
|
|
|
+ if tr != nil {
|
|
|
+ newMeta.replicas[update.Keyspace] = strat.replicaMap(t.hosts.get(), tr.tokens)
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ if meta != nil {
|
|
|
+ for ks, replicas := range meta.replicas {
|
|
|
+ if ks != update.Keyspace {
|
|
|
+ newMeta.replicas[ks] = replicas
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ t.keyspaces.Store(newMeta)
|
|
|
}
|
|
|
|
|
|
func (t *tokenAwareHostPolicy) SetPartitioner(partitioner string) {
|
|
|
@@ -304,31 +361,34 @@ func (t *tokenAwareHostPolicy) SetPartitioner(partitioner string) {
|
|
|
t.fallback.SetPartitioner(partitioner)
|
|
|
t.partitioner = partitioner
|
|
|
|
|
|
- t.resetTokenRing()
|
|
|
+ t.resetTokenRing(partitioner)
|
|
|
}
|
|
|
}
|
|
|
|
|
|
func (t *tokenAwareHostPolicy) AddHost(host *HostInfo) {
|
|
|
- t.mu.Lock()
|
|
|
- defer t.mu.Unlock()
|
|
|
-
|
|
|
t.hosts.add(host)
|
|
|
t.fallback.AddHost(host)
|
|
|
|
|
|
- t.resetTokenRing()
|
|
|
+ t.mu.RLock()
|
|
|
+ partitioner := t.partitioner
|
|
|
+ t.mu.RUnlock()
|
|
|
+ t.resetTokenRing(partitioner)
|
|
|
}
|
|
|
|
|
|
func (t *tokenAwareHostPolicy) RemoveHost(host *HostInfo) {
|
|
|
- t.mu.Lock()
|
|
|
- defer t.mu.Unlock()
|
|
|
-
|
|
|
t.hosts.remove(host.ConnectAddress())
|
|
|
t.fallback.RemoveHost(host)
|
|
|
|
|
|
- t.resetTokenRing()
|
|
|
+ t.mu.RLock()
|
|
|
+ partitioner := t.partitioner
|
|
|
+ t.mu.RUnlock()
|
|
|
+ t.resetTokenRing(partitioner)
|
|
|
}
|
|
|
|
|
|
func (t *tokenAwareHostPolicy) HostUp(host *HostInfo) {
|
|
|
+ // TODO: need to avoid doing all the work on AddHost on hostup/down
|
|
|
+ // because it now expensive to calculate the replica map for each
|
|
|
+ // token
|
|
|
t.AddHost(host)
|
|
|
}
|
|
|
|
|
|
@@ -336,22 +396,31 @@ func (t *tokenAwareHostPolicy) HostDown(host *HostInfo) {
|
|
|
t.RemoveHost(host)
|
|
|
}
|
|
|
|
|
|
-func (t *tokenAwareHostPolicy) resetTokenRing() {
|
|
|
- if t.partitioner == "" {
|
|
|
+func (t *tokenAwareHostPolicy) resetTokenRing(partitioner string) {
|
|
|
+ if partitioner == "" {
|
|
|
// partitioner not yet set
|
|
|
return
|
|
|
}
|
|
|
|
|
|
// create a new token ring
|
|
|
hosts := t.hosts.get()
|
|
|
- tokenRing, err := newTokenRing(t.partitioner, hosts)
|
|
|
+ tokenRing, err := newTokenRing(partitioner, hosts)
|
|
|
if err != nil {
|
|
|
Logger.Printf("Unable to update the token ring due to error: %s", err)
|
|
|
return
|
|
|
}
|
|
|
|
|
|
// replace the token ring
|
|
|
- t.tokenRing = tokenRing
|
|
|
+ t.tokenRing.Store(tokenRing)
|
|
|
+}
|
|
|
+
|
|
|
+func (t *tokenAwareHostPolicy) getReplicas(keyspace string, token token) ([]*HostInfo, bool) {
|
|
|
+ meta, _ := t.keyspaces.Load().(*keyspaceMeta)
|
|
|
+ if meta == nil {
|
|
|
+ return nil, false
|
|
|
+ }
|
|
|
+ tokens, ok := meta.replicas[keyspace][token]
|
|
|
+ return tokens, ok
|
|
|
}
|
|
|
|
|
|
func (t *tokenAwareHostPolicy) Pick(qry ExecutableQuery) NextHost {
|
|
|
@@ -362,45 +431,62 @@ func (t *tokenAwareHostPolicy) Pick(qry ExecutableQuery) NextHost {
|
|
|
routingKey, err := qry.GetRoutingKey()
|
|
|
if err != nil {
|
|
|
return t.fallback.Pick(qry)
|
|
|
+ } else if routingKey == nil {
|
|
|
+ return t.fallback.Pick(qry)
|
|
|
}
|
|
|
- if routingKey == nil {
|
|
|
+
|
|
|
+ tr, _ := t.tokenRing.Load().(*tokenRing)
|
|
|
+ if tr == nil {
|
|
|
return t.fallback.Pick(qry)
|
|
|
}
|
|
|
|
|
|
- t.mu.RLock()
|
|
|
- // TODO retrieve a list of hosts based on the replication strategy
|
|
|
- host := t.tokenRing.GetHostForPartitionKey(routingKey)
|
|
|
- t.mu.RUnlock()
|
|
|
+ token := tr.partitioner.Hash(routingKey)
|
|
|
+ primaryEndpoint := tr.GetHostForToken(token)
|
|
|
|
|
|
- if host == nil {
|
|
|
+ if primaryEndpoint == nil || token == nil {
|
|
|
return t.fallback.Pick(qry)
|
|
|
}
|
|
|
|
|
|
- // scope these variables for the same lifetime as the iterator function
|
|
|
+ replicas, ok := t.getReplicas(qry.Keyspace(), token)
|
|
|
+ if !ok {
|
|
|
+ replicas = []*HostInfo{primaryEndpoint}
|
|
|
+ } else if t.shuffleReplicas {
|
|
|
+ replicas = shuffleHosts(replicas)
|
|
|
+ }
|
|
|
+
|
|
|
var (
|
|
|
- hostReturned bool
|
|
|
fallbackIter NextHost
|
|
|
+ i int
|
|
|
)
|
|
|
|
|
|
+ used := make(map[*HostInfo]bool)
|
|
|
return func() SelectedHost {
|
|
|
- if !hostReturned {
|
|
|
- hostReturned = true
|
|
|
- return (*selectedHost)(host)
|
|
|
+ for i < len(replicas) {
|
|
|
+ h := replicas[i]
|
|
|
+ i++
|
|
|
+
|
|
|
+ if !h.IsUp() {
|
|
|
+ // TODO: need a way to handle host distance, as we may want to not
|
|
|
+ // use hosts in specific DC's
|
|
|
+ continue
|
|
|
+ }
|
|
|
+ used[h] = true
|
|
|
+
|
|
|
+ return (*selectedHost)(h)
|
|
|
}
|
|
|
|
|
|
- // fallback
|
|
|
if fallbackIter == nil {
|
|
|
+ // fallback
|
|
|
fallbackIter = t.fallback.Pick(qry)
|
|
|
}
|
|
|
|
|
|
- fallbackHost := fallbackIter()
|
|
|
-
|
|
|
// filter the token aware selected hosts from the fallback hosts
|
|
|
- if fallbackHost != nil && fallbackHost.Info() == host {
|
|
|
- fallbackHost = fallbackIter()
|
|
|
+ for fallbackHost := fallbackIter(); fallbackHost != nil; fallbackHost = fallbackIter() {
|
|
|
+ if !used[fallbackHost.Info()] {
|
|
|
+ return fallbackHost
|
|
|
+ }
|
|
|
}
|
|
|
-
|
|
|
- return fallbackHost
|
|
|
+ return nil
|
|
|
}
|
|
|
}
|
|
|
|
|
|
@@ -428,6 +514,10 @@ type hostPoolHostPolicy struct {
|
|
|
hostMap map[string]*HostInfo
|
|
|
}
|
|
|
|
|
|
+func (r *hostPoolHostPolicy) Init(*Session) {}
|
|
|
+func (r *hostPoolHostPolicy) KeyspaceChanged(KeyspaceUpdateEvent) {}
|
|
|
+func (r *hostPoolHostPolicy) SetPartitioner(string) {}
|
|
|
+
|
|
|
func (r *hostPoolHostPolicy) SetHosts(hosts []*HostInfo) {
|
|
|
peers := make([]string, len(hosts))
|
|
|
hostMap := make(map[string]*HostInfo, len(hosts))
|
|
|
@@ -492,10 +582,6 @@ func (r *hostPoolHostPolicy) HostDown(host *HostInfo) {
|
|
|
r.RemoveHost(host)
|
|
|
}
|
|
|
|
|
|
-func (r *hostPoolHostPolicy) SetPartitioner(partitioner string) {
|
|
|
- // noop
|
|
|
-}
|
|
|
-
|
|
|
func (r *hostPoolHostPolicy) Pick(qry ExecutableQuery) NextHost {
|
|
|
return func() SelectedHost {
|
|
|
r.mu.RLock()
|
|
|
@@ -557,11 +643,13 @@ type dcAwareRR struct {
|
|
|
// return hosts which are in the local datacentre before returning hosts in all
|
|
|
// other datercentres
|
|
|
func DCAwareRoundRobinPolicy(localDC string) HostSelectionPolicy {
|
|
|
- return &dcAwareRR{
|
|
|
- local: localDC,
|
|
|
- }
|
|
|
+ return &dcAwareRR{local: localDC}
|
|
|
}
|
|
|
|
|
|
+func (r *dcAwareRR) Init(*Session) {}
|
|
|
+func (r *dcAwareRR) KeyspaceChanged(KeyspaceUpdateEvent) {}
|
|
|
+func (d *dcAwareRR) SetPartitioner(p string) {}
|
|
|
+
|
|
|
func (d *dcAwareRR) AddHost(host *HostInfo) {
|
|
|
if host.DataCenter() == d.local {
|
|
|
d.localHosts.add(host)
|
|
|
@@ -578,15 +666,8 @@ func (d *dcAwareRR) RemoveHost(host *HostInfo) {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
-func (d *dcAwareRR) HostUp(host *HostInfo) {
|
|
|
- d.AddHost(host)
|
|
|
-}
|
|
|
-
|
|
|
-func (d *dcAwareRR) HostDown(host *HostInfo) {
|
|
|
- d.RemoveHost(host)
|
|
|
-}
|
|
|
-
|
|
|
-func (d *dcAwareRR) SetPartitioner(p string) {}
|
|
|
+func (d *dcAwareRR) HostUp(host *HostInfo) { d.AddHost(host) }
|
|
|
+func (d *dcAwareRR) HostDown(host *HostInfo) { d.RemoveHost(host) }
|
|
|
|
|
|
func (d *dcAwareRR) Pick(q ExecutableQuery) NextHost {
|
|
|
var i int
|