diff --git a/connectionpool.go b/connectionpool.go index c34448326..9e5fa109b 100644 --- a/connectionpool.go +++ b/connectionpool.go @@ -329,7 +329,7 @@ func newHostConnPool(session *Session, host *HostInfo, port, size int, } // Pick a connection from this connection pool for the given query. -func (pool *hostConnPool) Pick(token token, keyspace string, table string) *Conn { +func (pool *hostConnPool) Pick(token Token, keyspace string, table string) *Conn { pool.mu.RLock() defer pool.mu.RUnlock() diff --git a/connpicker.go b/connpicker.go index c6c65f7d3..daadfe1c1 100644 --- a/connpicker.go +++ b/connpicker.go @@ -7,7 +7,7 @@ import ( ) type ConnPicker interface { - Pick(token, string, string) *Conn + Pick(Token, string, string) *Conn Put(*Conn) Remove(conn *Conn) InFlight() int @@ -71,7 +71,7 @@ func (p *defaultConnPicker) Size() (int, int) { return size, p.size - size } -func (p *defaultConnPicker) Pick(token, string, string) *Conn { +func (p *defaultConnPicker) Pick(Token, string, string) *Conn { pos := int(atomic.AddUint32(&p.pos, 1) - 1) size := len(p.conns) @@ -110,7 +110,7 @@ func (*defaultConnPicker) NextShard() (shardID, nrShards int) { // to the point where we have first connection. type nopConnPicker struct{} -func (nopConnPicker) Pick(token, string, string) *Conn { +func (nopConnPicker) Pick(Token, string, string) *Conn { return nil } diff --git a/host_source.go b/host_source.go index 8dcf371ae..551833735 100644 --- a/host_source.go +++ b/host_source.go @@ -596,7 +596,7 @@ func addTabletToTabletsList(tablets []*TabletInfo, tablet *TabletInfo) []*Tablet } // Search for place in tablets table for token starting from index l to index r -func findTabletForToken(tablets []*TabletInfo, token token, l int, r int) *TabletInfo { +func findTabletForToken(tablets []*TabletInfo, token Token, l int, r int) *TabletInfo { for l < r { var m int if r*l > 0 { diff --git a/policies.go b/policies.go index 98867d2fa..eda40fc4a 100644 --- a/policies.go +++ b/policies.go @@ -323,20 +323,20 @@ type HostSelectionPolicy interface { // selection policy. type SelectedHost interface { Info() *HostInfo - Token() token + Token() Token Mark(error) } type selectedHost struct { info *HostInfo - token token + token Token } func (host selectedHost) Info() *HostInfo { return host.info } -func (host selectedHost) Token() token { +func (host selectedHost) Token() Token { return host.token } @@ -928,7 +928,7 @@ func (host selectedHostPoolHost) Info() *HostInfo { return host.info } -func (host selectedHostPoolHost) Token() token { +func (host selectedHostPoolHost) Token() Token { return nil } diff --git a/query_executor.go b/query_executor.go index f0d4e761f..02b0ac84a 100644 --- a/query_executor.go +++ b/query_executor.go @@ -18,7 +18,7 @@ type ExecutableQuery interface { Table() string IsIdempotent() bool IsLWT() bool - GetCustomPartitioner() partitioner + GetCustomPartitioner() Partitioner withContext(context.Context) ExecutableQuery diff --git a/scylla.go b/scylla.go index 54e00a9e4..66b580f1d 100644 --- a/scylla.go +++ b/scylla.go @@ -353,7 +353,7 @@ func newScyllaConnPicker(conn *Conn) *scyllaConnPicker { } } -func (p *scyllaConnPicker) Pick(t token, keyspace string, table string) *Conn { +func (p *scyllaConnPicker) Pick(t Token, keyspace string, table string) *Conn { if len(p.conns) == 0 { return nil } @@ -860,7 +860,7 @@ func ScyllaGetSourcePort(ctx context.Context) uint16 { // Returns a partitioner specific to the table, or "nil" // if the cluster-global partitioner should be used -func scyllaGetTablePartitioner(session *Session, keyspaceName, tableName string) (partitioner, error) { +func scyllaGetTablePartitioner(session *Session, keyspaceName, tableName string) (Partitioner, error) { isCdc, err := scyllaIsCdcTable(session, keyspaceName, tableName) if err != nil { return nil, err diff --git a/scylla_cdc.go b/scylla_cdc.go index 5762567ac..c4f8d5730 100644 --- a/scylla_cdc.go +++ b/scylla_cdc.go @@ -24,13 +24,13 @@ const ( type scyllaCDCPartitioner struct{} -var _ partitioner = scyllaCDCPartitioner{} +var _ Partitioner = scyllaCDCPartitioner{} func (p scyllaCDCPartitioner) Name() string { return scyllaCDCPartitionerName } -func (p scyllaCDCPartitioner) Hash(partitionKey []byte) token { +func (p scyllaCDCPartitioner) Hash(partitionKey []byte) Token { if len(partitionKey) < 8 { // The key is too short to extract any sensible token, // so return the min token instead @@ -68,7 +68,7 @@ func (p scyllaCDCPartitioner) Hash(partitionKey []byte) token { return int64Token(upperQword) } -func (p scyllaCDCPartitioner) ParseString(str string) token { +func (p scyllaCDCPartitioner) ParseString(str string) Token { return parseInt64Token(str) } diff --git a/scylla_test.go b/scylla_test.go index a00c06d4a..daaa3a54a 100644 --- a/scylla_test.go +++ b/scylla_test.go @@ -24,7 +24,7 @@ func TestScyllaConnPickerPickNilToken(t *testing.T) { s.conns = []*Conn{{ streams: streams.New(protoVersion4), }} - if s.Pick(token(nil), "", "") != s.conns[0] { + if s.Pick(Token(nil), "", "") != s.conns[0] { t.Fatal("expected connection") } }) @@ -33,7 +33,7 @@ func TestScyllaConnPickerPickNilToken(t *testing.T) { s.conns = []*Conn{{ streams: streams.New(protoVersion4), }} - if s.Pick(token(nil), "", "") != s.conns[0] { + if s.Pick(Token(nil), "", "") != s.conns[0] { t.Fatal("expected connection") } }) @@ -42,20 +42,20 @@ func TestScyllaConnPickerPickNilToken(t *testing.T) { s.conns = []*Conn{nil, { streams: streams.New(protoVersion4), }} - if s.Pick(token(nil), "", "") != s.conns[1] { + if s.Pick(Token(nil), "", "") != s.conns[1] { t.Fatal("expected connection") } - if s.Pick(token(nil), "", "") != s.conns[1] { + if s.Pick(Token(nil), "", "") != s.conns[1] { t.Fatal("expected connection") } }) t.Run("multiple shards no conns", func(t *testing.T) { s.conns = []*Conn{nil, nil} - if s.Pick(token(nil), "", "") != nil { + if s.Pick(Token(nil), "", "") != nil { t.Fatal("expected nil") } - if s.Pick(token(nil), "", "") != nil { + if s.Pick(Token(nil), "", "") != nil { t.Fatal("expected nil") } }) @@ -163,7 +163,7 @@ func TestScyllaRandomConnPIcker(t *testing.T) { conns: []*Conn{nil, mockConn(1)}, } - if s.Pick(token(nil), "", "") == nil { + if s.Pick(Token(nil), "", "") == nil { t.Fatal("expected connection") } }) @@ -187,7 +187,7 @@ func TestScyllaRandomConnPIcker(t *testing.T) { defer wg.Done() for i := 0; i < 3; i++ { select { - case connCh <- s.Pick(token(nil), "", ""): + case connCh <- s.Pick(Token(nil), "", ""): case <-ctx.Done(): } } diff --git a/session.go b/session.go index 6bdfb8873..23b661f47 100644 --- a/session.go +++ b/session.go @@ -986,7 +986,7 @@ type queryRoutingInfo struct { lwt bool // If not nil, represents a custom partitioner for the table. - partitioner partitioner + partitioner Partitioner keyspace string @@ -999,7 +999,7 @@ func (qri *queryRoutingInfo) isLWT() bool { return qri.lwt } -func (qri *queryRoutingInfo) getPartitioner() partitioner { +func (qri *queryRoutingInfo) getPartitioner() Partitioner { qri.mu.RLock() defer qri.mu.RUnlock() return qri.partitioner @@ -1310,7 +1310,7 @@ func (q *Query) IsLWT() bool { return q.routingInfo.isLWT() } -func (q *Query) GetCustomPartitioner() partitioner { +func (q *Query) GetCustomPartitioner() Partitioner { return q.routingInfo.getPartitioner() } @@ -1933,7 +1933,7 @@ func (b *Batch) IsLWT() bool { return b.routingInfo.isLWT() } -func (b *Batch) GetCustomPartitioner() partitioner { +func (b *Batch) GetCustomPartitioner() Partitioner { return b.routingInfo.getPartitioner() } @@ -2176,7 +2176,7 @@ type routingKeyInfo struct { keyspace string table string lwt bool - partitioner partitioner + partitioner Partitioner } func (r *routingKeyInfo) String() string { diff --git a/token.go b/token.go index c201cfb3e..72d0ce4eb 100644 --- a/token.go +++ b/token.go @@ -17,16 +17,16 @@ import ( ) // a token partitioner -type partitioner interface { +type Partitioner interface { Name() string - Hash([]byte) token - ParseString(string) token + Hash([]byte) Token + ParseString(string) Token } -// a token -type token interface { +// a Token +type Token interface { fmt.Stringer - Less(token) bool + Less(Token) bool } // murmur3 partitioner @@ -36,13 +36,13 @@ func (p murmur3Partitioner) Name() string { return "Murmur3Partitioner" } -func (p murmur3Partitioner) Hash(partitionKey []byte) token { +func (p murmur3Partitioner) Hash(partitionKey []byte) Token { h1 := murmur.Murmur3H1(partitionKey) return int64Token(h1) } // murmur3 little-endian, 128-bit hash, but returns only h1 -func (p murmur3Partitioner) ParseString(str string) token { +func (p murmur3Partitioner) ParseString(str string) Token { return parseInt64Token(str) } @@ -58,7 +58,7 @@ func (m int64Token) String() string { return strconv.FormatInt(int64(m), 10) } -func (m int64Token) Less(token token) bool { +func (m int64Token) Less(token Token) bool { return m < token.(int64Token) } @@ -70,12 +70,12 @@ func (p orderedPartitioner) Name() string { return "OrderedPartitioner" } -func (p orderedPartitioner) Hash(partitionKey []byte) token { +func (p orderedPartitioner) Hash(partitionKey []byte) Token { // the partition key is the token return orderedToken(partitionKey) } -func (p orderedPartitioner) ParseString(str string) token { +func (p orderedPartitioner) ParseString(str string) Token { return orderedToken(str) } @@ -83,7 +83,7 @@ func (o orderedToken) String() string { return string(o) } -func (o orderedToken) Less(token token) bool { +func (o orderedToken) Less(token Token) bool { return o < token.(orderedToken) } @@ -98,7 +98,7 @@ func (r randomPartitioner) Name() string { // 2 ** 128 var maxHashInt, _ = new(big.Int).SetString("340282366920938463463374607431768211456", 10) -func (p randomPartitioner) Hash(partitionKey []byte) token { +func (p randomPartitioner) Hash(partitionKey []byte) Token { sum := md5.Sum(partitionKey) val := new(big.Int) val.SetBytes(sum[:]) @@ -110,7 +110,7 @@ func (p randomPartitioner) Hash(partitionKey []byte) token { return (*randomToken)(val) } -func (p randomPartitioner) ParseString(str string) token { +func (p randomPartitioner) ParseString(str string) Token { val := new(big.Int) val.SetString(str, 10) return (*randomToken)(val) @@ -120,12 +120,12 @@ func (r *randomToken) String() string { return (*big.Int)(r).String() } -func (r *randomToken) Less(token token) bool { +func (r *randomToken) Less(token Token) bool { return -1 == (*big.Int)(r).Cmp((*big.Int)(token.(*randomToken))) } type hostToken struct { - token token + token Token host *HostInfo } @@ -135,7 +135,7 @@ func (ht hostToken) String() string { // a data structure for organizing the relationship between tokens and hosts type tokenRing struct { - partitioner partitioner + partitioner Partitioner // tokens map token range to primary replica. // The elements in tokens are sorted by token ascending. @@ -212,7 +212,7 @@ func (t *tokenRing) String() string { // // It returns two tokens. First is token that exactly corresponds to the partition key (and could be used to // determine shard, for example), second token is the endToken that corresponds to the host. -func (t *tokenRing) GetHostForPartitionKey(partitionKey []byte) (host *HostInfo, token token, endToken token) { +func (t *tokenRing) GetHostForPartitionKey(partitionKey []byte) (host *HostInfo, token Token, endToken Token) { if t == nil { return nil, nil, nil } @@ -222,7 +222,7 @@ func (t *tokenRing) GetHostForPartitionKey(partitionKey []byte) (host *HostInfo, return host, token, endToken } -func (t *tokenRing) GetHostForToken(token token) (host *HostInfo, endToken token) { +func (t *tokenRing) GetHostForToken(token Token) (host *HostInfo, endToken Token) { if t == nil || len(t.tokens) == 0 { return nil, nil } diff --git a/token_test.go b/token_test.go index bb75942cb..af11eb146 100644 --- a/token_test.go +++ b/token_test.go @@ -133,7 +133,7 @@ func TestRandomToken(t *testing.T) { type intToken int func (i intToken) String() string { return strconv.Itoa(int(i)) } -func (i intToken) Less(token token) bool { return i < token.(intToken) } +func (i intToken) Less(token Token) bool { return i < token.(intToken) } // Test of the token ring implementation based on example at the start of this // page of documentation: diff --git a/topology.go b/topology.go index a154f0fb9..1048ee34e 100644 --- a/topology.go +++ b/topology.go @@ -9,7 +9,7 @@ import ( type hostTokens struct { // token is end (inclusive) of token range these hosts belong to - token token + token Token hosts []*HostInfo } @@ -24,7 +24,7 @@ func (h tokenRingReplicas) Less(i, j int) bool { return h[i].token.Less(h[j].tok func (h tokenRingReplicas) Len() int { return len(h) } func (h tokenRingReplicas) Swap(i, j int) { h[i], h[j] = h[j], h[i] } -func (h tokenRingReplicas) replicasFor(t token) *hostTokens { +func (h tokenRingReplicas) replicasFor(t Token) *hostTokens { if len(h) == 0 { return nil }