|
| 1 | +package gocql |
| 2 | + |
| 3 | +import ( |
| 4 | + "fmt" |
| 5 | + "math" |
| 6 | + "strconv" |
| 7 | + "sync/atomic" |
| 8 | +) |
| 9 | + |
| 10 | +// scyllaSupported represents Scylla connection options as sent in SUPPORTED |
| 11 | +// frame. |
| 12 | +type scyllaSupported struct { |
| 13 | + shard int |
| 14 | + nrShards int |
| 15 | + msbIgnore uint64 |
| 16 | +} |
| 17 | + |
| 18 | +func parseSupported(supported map[string][]string) scyllaSupported { |
| 19 | + const ( |
| 20 | + scyllaShard = "SCYLLA_SHARD" |
| 21 | + scyllaNrShards = "SCYLLA_NR_SHARDS" |
| 22 | + scyllaPartitioner = "SCYLLA_PARTITIONER" |
| 23 | + scyllaShardingAlgorithm = "SCYLLA_SHARDING_ALGORITHM" |
| 24 | + scyllaShardingIgnoreMSB = "SCYLLA_SHARDING_IGNORE_MSB" |
| 25 | + ) |
| 26 | + |
| 27 | + var ( |
| 28 | + si scyllaSupported |
| 29 | + err error |
| 30 | + ) |
| 31 | + |
| 32 | + if s, ok := supported[scyllaShard]; ok { |
| 33 | + if si.shard, err = strconv.Atoi(s[0]); err != nil { |
| 34 | + if gocqlDebug { |
| 35 | + Logger.Printf("scylla: failed to parse %s value %v: %s", scyllaShard, s, err) |
| 36 | + } |
| 37 | + } |
| 38 | + } |
| 39 | + if s, ok := supported[scyllaNrShards]; ok { |
| 40 | + if si.nrShards, err = strconv.Atoi(s[0]); err != nil { |
| 41 | + if gocqlDebug { |
| 42 | + Logger.Printf("scylla: failed to parse %s value %v: %s", scyllaNrShards, s, err) |
| 43 | + } |
| 44 | + } |
| 45 | + } |
| 46 | + if s, ok := supported[scyllaShardingIgnoreMSB]; ok { |
| 47 | + if si.msbIgnore, err = strconv.ParseUint(s[0], 10, 64); err != nil { |
| 48 | + if gocqlDebug { |
| 49 | + Logger.Printf("scylla: failed to parse %s value %v: %s", scyllaShardingIgnoreMSB, s, err) |
| 50 | + } |
| 51 | + } |
| 52 | + } |
| 53 | + |
| 54 | + var ( |
| 55 | + partitioner string |
| 56 | + algorithm string |
| 57 | + ) |
| 58 | + if s, ok := supported[scyllaPartitioner]; ok { |
| 59 | + partitioner = s[0] |
| 60 | + } |
| 61 | + if s, ok := supported[scyllaShardingAlgorithm]; ok { |
| 62 | + algorithm = s[0] |
| 63 | + } |
| 64 | + |
| 65 | + if partitioner != "org.apache.cassandra.dht.Murmur3Partitioner" || algorithm != "biased-token-round-robin" || si.nrShards == 0 || si.msbIgnore == 0 { |
| 66 | + if gocqlDebug { |
| 67 | + Logger.Printf("scylla: unsupported sharding configuration") |
| 68 | + } |
| 69 | + return scyllaSupported{} |
| 70 | + } |
| 71 | + |
| 72 | + return si |
| 73 | +} |
| 74 | + |
| 75 | +// isScyllaConn checks if conn is suitable for scyllaConnPicker. |
| 76 | +func isScyllaConn(conn *Conn) bool { |
| 77 | + s := parseSupported(conn.supported) |
| 78 | + return s.nrShards != 0 |
| 79 | +} |
| 80 | + |
| 81 | +// scyllaConnPicker is a specialised ConnPicker that selects connections based |
| 82 | +// on token trying to get connection to a shard containing the given token. |
| 83 | +type scyllaConnPicker struct { |
| 84 | + conns []*Conn |
| 85 | + nrConns int |
| 86 | + nrShards int |
| 87 | + msbIgnore uint64 |
| 88 | + pos int32 |
| 89 | +} |
| 90 | + |
| 91 | +func newScyllaConnPicker(conn *Conn) *scyllaConnPicker { |
| 92 | + s := parseSupported(conn.supported) |
| 93 | + if s.nrShards == 0 { |
| 94 | + panic(fmt.Sprintf("scylla: %s not a sharded connection", conn.Address())) |
| 95 | + } |
| 96 | + |
| 97 | + if gocqlDebug { |
| 98 | + Logger.Printf("scylla: %s sharding options %+v", conn.Address(), s) |
| 99 | + } |
| 100 | + |
| 101 | + return &scyllaConnPicker{ |
| 102 | + nrShards: s.nrShards, |
| 103 | + msbIgnore: s.msbIgnore, |
| 104 | + } |
| 105 | +} |
| 106 | + |
| 107 | +func (p *scyllaConnPicker) Remove(conn *Conn) { |
| 108 | + s := parseSupported(conn.supported) |
| 109 | + if s.nrShards == 0 { |
| 110 | + panic(fmt.Sprintf("scylla: %s not a sharded connection", conn.Address())) |
| 111 | + } |
| 112 | + if gocqlDebug { |
| 113 | + Logger.Printf("scylla: %s remove shard %d connection", conn.Address(), s.shard) |
| 114 | + } |
| 115 | + p.conns[s.shard] = nil |
| 116 | +} |
| 117 | + |
| 118 | +func (p *scyllaConnPicker) Close() { |
| 119 | + conns := p.conns |
| 120 | + p.conns = nil |
| 121 | + for _, conn := range conns { |
| 122 | + if conn != nil { |
| 123 | + conn.Close() |
| 124 | + } |
| 125 | + } |
| 126 | +} |
| 127 | + |
| 128 | +func (p *scyllaConnPicker) Size() (int, int) { |
| 129 | + return p.nrConns, p.nrShards - p.nrConns |
| 130 | +} |
| 131 | + |
| 132 | +func (p *scyllaConnPicker) Pick(t token) *Conn { |
| 133 | + if len(p.conns) == 0 { |
| 134 | + return nil |
| 135 | + } |
| 136 | + |
| 137 | + if t == nil { |
| 138 | + idx := int(atomic.AddInt32(&p.pos, 1)) |
| 139 | + for i := 0; i < len(p.conns); i++ { |
| 140 | + if conn := p.conns[(idx+i)%len(p.conns)]; conn != nil { |
| 141 | + return conn |
| 142 | + } |
| 143 | + } |
| 144 | + return nil |
| 145 | + } |
| 146 | + |
| 147 | + mmt, ok := t.(murmur3Token) |
| 148 | + // double check if that's murmur3 token |
| 149 | + if !ok { |
| 150 | + return nil |
| 151 | + } |
| 152 | + |
| 153 | + idx := p.shardOf(mmt) |
| 154 | + return p.conns[idx] |
| 155 | +} |
| 156 | + |
| 157 | +func (p *scyllaConnPicker) shardOf(token murmur3Token) int { |
| 158 | + shards := uint64(p.nrShards) |
| 159 | + z := uint64(token+math.MinInt64) << p.msbIgnore |
| 160 | + lo := z & 0xffffffff |
| 161 | + hi := (z >> 32) & 0xffffffff |
| 162 | + mul1 := lo * shards |
| 163 | + mul2 := hi * shards |
| 164 | + sum := (mul1 >> 32) + mul2 |
| 165 | + return int(sum >> 32) |
| 166 | +} |
| 167 | + |
| 168 | +func (p *scyllaConnPicker) Put(conn *Conn) { |
| 169 | + s := parseSupported(conn.supported) |
| 170 | + if s.nrShards == 0 { |
| 171 | + panic(fmt.Sprintf("scylla: %s not a sharded connection", conn.Address())) |
| 172 | + } |
| 173 | + |
| 174 | + if s.nrShards != len(p.conns) { |
| 175 | + if s.nrShards != p.nrShards { |
| 176 | + panic(fmt.Sprintf("scylla: %s invalid number of shards", conn.Address())) |
| 177 | + } |
| 178 | + conns := p.conns |
| 179 | + p.conns = make([]*Conn, s.nrShards, s.nrShards) |
| 180 | + copy(p.conns, conns) |
| 181 | + } |
| 182 | + if c := p.conns[s.shard]; c != nil { |
| 183 | + conn.Close() |
| 184 | + return |
| 185 | + } |
| 186 | + p.conns[s.shard] = conn |
| 187 | + p.nrConns++ |
| 188 | + if gocqlDebug { |
| 189 | + Logger.Printf("scylla: %s put shard %d connection total: %d missing: %d", conn.Address(), s.shard, p.nrConns, p.nrShards-p.nrConns) |
| 190 | + } |
| 191 | +} |
0 commit comments