github.com/muhammadn/cortex@v1.9.1-0.20220510110439-46bb7000d03d/pkg/chunk/cassandra/storage_client.go (about) 1 package cassandra 2 3 import ( 4 "bytes" 5 "context" 6 "crypto/tls" 7 "flag" 8 "fmt" 9 "io/ioutil" 10 "strings" 11 "time" 12 13 "github.com/go-kit/log" 14 "github.com/go-kit/log/level" 15 "github.com/gocql/gocql" 16 "github.com/grafana/dskit/flagext" 17 "github.com/pkg/errors" 18 "github.com/prometheus/client_golang/prometheus" 19 "golang.org/x/sync/semaphore" 20 21 "github.com/cortexproject/cortex/pkg/chunk" 22 "github.com/cortexproject/cortex/pkg/chunk/util" 23 util_log "github.com/cortexproject/cortex/pkg/util/log" 24 ) 25 26 // Config for a StorageClient 27 type Config struct { 28 Addresses string `yaml:"addresses"` 29 Port int `yaml:"port"` 30 Keyspace string `yaml:"keyspace"` 31 Consistency string `yaml:"consistency"` 32 ReplicationFactor int `yaml:"replication_factor"` 33 DisableInitialHostLookup bool `yaml:"disable_initial_host_lookup"` 34 SSL bool `yaml:"SSL"` 35 HostVerification bool `yaml:"host_verification"` 36 HostSelectionPolicy string `yaml:"host_selection_policy"` 37 CAPath string `yaml:"CA_path"` 38 CertPath string `yaml:"tls_cert_path"` 39 KeyPath string `yaml:"tls_key_path"` 40 Auth bool `yaml:"auth"` 41 Username string `yaml:"username"` 42 Password flagext.Secret `yaml:"password"` 43 PasswordFile string `yaml:"password_file"` 44 CustomAuthenticators flagext.StringSlice `yaml:"custom_authenticators"` 45 Timeout time.Duration `yaml:"timeout"` 46 ConnectTimeout time.Duration `yaml:"connect_timeout"` 47 ReconnectInterval time.Duration `yaml:"reconnect_interval"` 48 Retries int `yaml:"max_retries"` 49 MaxBackoff time.Duration `yaml:"retry_max_backoff"` 50 MinBackoff time.Duration `yaml:"retry_min_backoff"` 51 QueryConcurrency int `yaml:"query_concurrency"` 52 NumConnections int `yaml:"num_connections"` 53 ConvictHosts bool `yaml:"convict_hosts_on_failure"` 54 TableOptions string `yaml:"table_options"` 55 } 56 57 const ( 58 HostPolicyRoundRobin = "round-robin" 59 HostPolicyTokenAware = "token-aware" 60 ) 61 62 // RegisterFlags adds the flags required to config this to the given FlagSet 63 func (cfg *Config) RegisterFlags(f *flag.FlagSet) { 64 f.StringVar(&cfg.Addresses, "cassandra.addresses", "", "Comma-separated hostnames or IPs of Cassandra instances.") 65 f.IntVar(&cfg.Port, "cassandra.port", 9042, "Port that Cassandra is running on") 66 f.StringVar(&cfg.Keyspace, "cassandra.keyspace", "", "Keyspace to use in Cassandra.") 67 f.StringVar(&cfg.Consistency, "cassandra.consistency", "QUORUM", "Consistency level for Cassandra.") 68 f.IntVar(&cfg.ReplicationFactor, "cassandra.replication-factor", 3, "Replication factor to use in Cassandra.") 69 f.BoolVar(&cfg.DisableInitialHostLookup, "cassandra.disable-initial-host-lookup", false, "Instruct the cassandra driver to not attempt to get host info from the system.peers table.") 70 f.BoolVar(&cfg.SSL, "cassandra.ssl", false, "Use SSL when connecting to cassandra instances.") 71 f.BoolVar(&cfg.HostVerification, "cassandra.host-verification", true, "Require SSL certificate validation.") 72 f.StringVar(&cfg.HostSelectionPolicy, "cassandra.host-selection-policy", HostPolicyRoundRobin, "Policy for selecting Cassandra host. Supported values are: round-robin, token-aware.") 73 f.StringVar(&cfg.CAPath, "cassandra.ca-path", "", "Path to certificate file to verify the peer.") 74 f.StringVar(&cfg.CertPath, "cassandra.tls-cert-path", "", "Path to certificate file used by TLS.") 75 f.StringVar(&cfg.KeyPath, "cassandra.tls-key-path", "", "Path to private key file used by TLS.") 76 f.BoolVar(&cfg.Auth, "cassandra.auth", false, "Enable password authentication when connecting to cassandra.") 77 f.StringVar(&cfg.Username, "cassandra.username", "", "Username to use when connecting to cassandra.") 78 f.Var(&cfg.Password, "cassandra.password", "Password to use when connecting to cassandra.") 79 f.StringVar(&cfg.PasswordFile, "cassandra.password-file", "", "File containing password to use when connecting to cassandra.") 80 f.Var(&cfg.CustomAuthenticators, "cassandra.custom-authenticator", "If set, when authenticating with cassandra a custom authenticator will be expected during the handshake. This flag can be set multiple times.") 81 f.DurationVar(&cfg.Timeout, "cassandra.timeout", 2*time.Second, "Timeout when connecting to cassandra.") 82 f.DurationVar(&cfg.ConnectTimeout, "cassandra.connect-timeout", 5*time.Second, "Initial connection timeout, used during initial dial to server.") 83 f.DurationVar(&cfg.ReconnectInterval, "cassandra.reconnent-interval", 1*time.Second, "Interval to retry connecting to cassandra nodes marked as DOWN.") 84 f.IntVar(&cfg.Retries, "cassandra.max-retries", 0, "Number of retries to perform on a request. Set to 0 to disable retries.") 85 f.DurationVar(&cfg.MinBackoff, "cassandra.retry-min-backoff", 100*time.Millisecond, "Minimum time to wait before retrying a failed request.") 86 f.DurationVar(&cfg.MaxBackoff, "cassandra.retry-max-backoff", 10*time.Second, "Maximum time to wait before retrying a failed request.") 87 f.IntVar(&cfg.QueryConcurrency, "cassandra.query-concurrency", 0, "Limit number of concurrent queries to Cassandra. Set to 0 to disable the limit.") 88 f.IntVar(&cfg.NumConnections, "cassandra.num-connections", 2, "Number of TCP connections per host.") 89 f.BoolVar(&cfg.ConvictHosts, "cassandra.convict-hosts-on-failure", true, "Convict hosts of being down on failure.") 90 f.StringVar(&cfg.TableOptions, "cassandra.table-options", "", "Table options used to create index or chunk tables. This value is used as plain text in the table `WITH` like this, \"CREATE TABLE <generated_by_cortex> (...) WITH <cassandra.table-options>\". For details, see https://cortexmetrics.io/docs/production/cassandra. By default it will use the default table options of your Cassandra cluster.") 91 } 92 93 func (cfg *Config) Validate() error { 94 if cfg.Password.Value != "" && cfg.PasswordFile != "" { 95 return errors.Errorf("The password and password_file config options are mutually exclusive.") 96 } 97 if cfg.SSL && cfg.HostVerification && len(strings.Split(cfg.Addresses, ",")) != 1 { 98 return errors.Errorf("Host verification is only possible for a single host.") 99 } 100 if cfg.SSL && cfg.CertPath != "" && cfg.KeyPath == "" { 101 return errors.Errorf("TLS certificate specified, but private key configuration is missing.") 102 } 103 if cfg.SSL && cfg.KeyPath != "" && cfg.CertPath == "" { 104 return errors.Errorf("TLS private key specified, but certificate configuration is missing.") 105 } 106 return nil 107 } 108 109 func (cfg *Config) session(name string, reg prometheus.Registerer) (*gocql.Session, error) { 110 cluster := gocql.NewCluster(strings.Split(cfg.Addresses, ",")...) 111 cluster.Port = cfg.Port 112 cluster.Keyspace = cfg.Keyspace 113 cluster.BatchObserver = observer{} 114 cluster.QueryObserver = observer{} 115 cluster.Timeout = cfg.Timeout 116 cluster.ConnectTimeout = cfg.ConnectTimeout 117 cluster.ReconnectInterval = cfg.ReconnectInterval 118 cluster.NumConns = cfg.NumConnections 119 cluster.Logger = log.With(util_log.Logger, "module", "gocql", "client", name) 120 cluster.Registerer = prometheus.WrapRegistererWith( 121 prometheus.Labels{"client": name}, reg) 122 if cfg.Retries > 0 { 123 cluster.RetryPolicy = &gocql.ExponentialBackoffRetryPolicy{ 124 NumRetries: cfg.Retries, 125 Min: cfg.MinBackoff, 126 Max: cfg.MaxBackoff, 127 } 128 } 129 if !cfg.ConvictHosts { 130 cluster.ConvictionPolicy = noopConvictionPolicy{} 131 } 132 if err := cfg.setClusterConfig(cluster); err != nil { 133 return nil, errors.WithStack(err) 134 } 135 136 session, err := cluster.CreateSession() 137 if err == nil { 138 return session, nil 139 } 140 // ErrNoConnectionsStarted will be returned if keyspace don't exist or is invalid. 141 // ref. https://github.com/gocql/gocql/blob/07ace3bab0f84bb88477bab5d79ba1f7e1da0169/cassandra_test.go#L85-L97 142 if err != gocql.ErrNoConnectionsStarted { 143 return nil, errors.WithStack(err) 144 } 145 // keyspace not exist 146 if err := cfg.createKeyspace(); err != nil { 147 return nil, errors.WithStack(err) 148 } 149 session, err = cluster.CreateSession() 150 return session, errors.WithStack(err) 151 } 152 153 // apply config settings to a cassandra ClusterConfig 154 func (cfg *Config) setClusterConfig(cluster *gocql.ClusterConfig) error { 155 consistency, err := gocql.ParseConsistencyWrapper(cfg.Consistency) 156 if err != nil { 157 return errors.Wrap(err, "unable to parse the configured consistency") 158 } 159 160 cluster.Consistency = consistency 161 cluster.DisableInitialHostLookup = cfg.DisableInitialHostLookup 162 163 if cfg.SSL { 164 tlsConfig := &tls.Config{} 165 166 if cfg.CertPath != "" { 167 cert, err := tls.LoadX509KeyPair(cfg.CertPath, cfg.KeyPath) 168 if err != nil { 169 return errors.Wrap(err, "Unable to load TLS certificate and private key") 170 } 171 172 tlsConfig.Certificates = []tls.Certificate{cert} 173 } 174 175 if cfg.HostVerification { 176 tlsConfig.ServerName = strings.Split(cfg.Addresses, ",")[0] 177 178 cluster.SslOpts = &gocql.SslOptions{ 179 CaPath: cfg.CAPath, 180 EnableHostVerification: true, 181 Config: tlsConfig, 182 } 183 } else { 184 cluster.SslOpts = &gocql.SslOptions{ 185 EnableHostVerification: false, 186 Config: tlsConfig, 187 } 188 } 189 } 190 191 if cfg.HostSelectionPolicy == HostPolicyRoundRobin { 192 cluster.PoolConfig.HostSelectionPolicy = gocql.RoundRobinHostPolicy() 193 } else if cfg.HostSelectionPolicy == HostPolicyTokenAware { 194 cluster.PoolConfig.HostSelectionPolicy = gocql.TokenAwareHostPolicy(gocql.RoundRobinHostPolicy()) 195 } else { 196 return errors.New("Unknown host selection policy") 197 } 198 199 if cfg.Auth { 200 password := cfg.Password.Value 201 if cfg.PasswordFile != "" { 202 passwordBytes, err := ioutil.ReadFile(cfg.PasswordFile) 203 if err != nil { 204 return errors.Errorf("Could not read Cassandra password file: %v", err) 205 } 206 passwordBytes = bytes.TrimRight(passwordBytes, "\n") 207 password = string(passwordBytes) 208 } 209 if len(cfg.CustomAuthenticators) != 0 { 210 cluster.Authenticator = CustomPasswordAuthenticator{ 211 ApprovedAuthenticators: cfg.CustomAuthenticators, 212 Username: cfg.Username, 213 Password: password, 214 } 215 return nil 216 } 217 cluster.Authenticator = gocql.PasswordAuthenticator{ 218 Username: cfg.Username, 219 Password: password, 220 } 221 } 222 return nil 223 } 224 225 // createKeyspace will create the desired keyspace if it doesn't exist. 226 func (cfg *Config) createKeyspace() error { 227 cluster := gocql.NewCluster(strings.Split(cfg.Addresses, ",")...) 228 cluster.Port = cfg.Port 229 cluster.Keyspace = "system" 230 cluster.Timeout = 20 * time.Second 231 cluster.ConnectTimeout = 20 * time.Second 232 233 if err := cfg.setClusterConfig(cluster); err != nil { 234 return errors.WithStack(err) 235 } 236 237 session, err := cluster.CreateSession() 238 if err != nil { 239 return errors.WithStack(err) 240 } 241 defer session.Close() 242 243 err = session.Query(fmt.Sprintf( 244 `CREATE KEYSPACE IF NOT EXISTS %s 245 WITH replication = { 246 'class' : 'SimpleStrategy', 247 'replication_factor' : %d 248 }`, 249 cfg.Keyspace, cfg.ReplicationFactor)).Exec() 250 return errors.WithStack(err) 251 } 252 253 // StorageClient implements chunk.IndexClient and chunk.ObjectClient for Cassandra. 254 type StorageClient struct { 255 cfg Config 256 schemaCfg chunk.SchemaConfig 257 readSession *gocql.Session 258 writeSession *gocql.Session 259 querySemaphore *semaphore.Weighted 260 } 261 262 // NewStorageClient returns a new StorageClient. 263 func NewStorageClient(cfg Config, schemaCfg chunk.SchemaConfig, registerer prometheus.Registerer) (*StorageClient, error) { 264 readSession, err := cfg.session("index-read", registerer) 265 if err != nil { 266 return nil, errors.WithStack(err) 267 } 268 269 writeSession, err := cfg.session("index-write", registerer) 270 if err != nil { 271 return nil, errors.WithStack(err) 272 } 273 274 var querySemaphore *semaphore.Weighted 275 if cfg.QueryConcurrency > 0 { 276 querySemaphore = semaphore.NewWeighted(int64(cfg.QueryConcurrency)) 277 } 278 279 client := &StorageClient{ 280 cfg: cfg, 281 schemaCfg: schemaCfg, 282 readSession: readSession, 283 writeSession: writeSession, 284 querySemaphore: querySemaphore, 285 } 286 return client, nil 287 } 288 289 // Stop implement chunk.IndexClient. 290 func (s *StorageClient) Stop() { 291 s.readSession.Close() 292 s.writeSession.Close() 293 } 294 295 // Cassandra batching isn't really useful in this case, its more to do multiple 296 // atomic writes. Therefore we just do a bunch of writes in parallel. 297 type writeBatch struct { 298 entries []chunk.IndexEntry 299 deletes []chunk.IndexEntry 300 } 301 302 // NewWriteBatch implement chunk.IndexClient. 303 func (s *StorageClient) NewWriteBatch() chunk.WriteBatch { 304 return &writeBatch{} 305 } 306 307 func (b *writeBatch) Add(tableName, hashValue string, rangeValue []byte, value []byte) { 308 b.entries = append(b.entries, chunk.IndexEntry{ 309 TableName: tableName, 310 HashValue: hashValue, 311 RangeValue: rangeValue, 312 Value: value, 313 }) 314 } 315 316 func (b *writeBatch) Delete(tableName, hashValue string, rangeValue []byte) { 317 b.deletes = append(b.deletes, chunk.IndexEntry{ 318 TableName: tableName, 319 HashValue: hashValue, 320 RangeValue: rangeValue, 321 }) 322 } 323 324 // BatchWrite implement chunk.IndexClient. 325 func (s *StorageClient) BatchWrite(ctx context.Context, batch chunk.WriteBatch) error { 326 b := batch.(*writeBatch) 327 328 for _, entry := range b.entries { 329 err := s.writeSession.Query(fmt.Sprintf("INSERT INTO %s (hash, range, value) VALUES (?, ?, ?)", 330 entry.TableName), entry.HashValue, entry.RangeValue, entry.Value).WithContext(ctx).Exec() 331 if err != nil { 332 return errors.WithStack(err) 333 } 334 } 335 336 for _, entry := range b.deletes { 337 err := s.writeSession.Query(fmt.Sprintf("DELETE FROM %s WHERE hash = ? and range = ?", 338 entry.TableName), entry.HashValue, entry.RangeValue).WithContext(ctx).Exec() 339 if err != nil { 340 return errors.WithStack(err) 341 } 342 } 343 344 return nil 345 } 346 347 // QueryPages implement chunk.IndexClient. 348 func (s *StorageClient) QueryPages(ctx context.Context, queries []chunk.IndexQuery, callback func(chunk.IndexQuery, chunk.ReadBatch) bool) error { 349 return util.DoParallelQueries(ctx, s.query, queries, callback) 350 } 351 352 func (s *StorageClient) query(ctx context.Context, query chunk.IndexQuery, callback util.Callback) error { 353 if s.querySemaphore != nil { 354 if err := s.querySemaphore.Acquire(ctx, 1); err != nil { 355 return err 356 } 357 defer s.querySemaphore.Release(1) 358 } 359 360 var q *gocql.Query 361 362 switch { 363 case len(query.RangeValuePrefix) > 0 && query.ValueEqual == nil: 364 q = s.readSession.Query(fmt.Sprintf("SELECT range, value FROM %s WHERE hash = ? AND range >= ? AND range < ?", 365 query.TableName), query.HashValue, query.RangeValuePrefix, append(query.RangeValuePrefix, '\xff')) 366 367 case len(query.RangeValuePrefix) > 0 && query.ValueEqual != nil: 368 q = s.readSession.Query(fmt.Sprintf("SELECT range, value FROM %s WHERE hash = ? AND range >= ? AND range < ? AND value = ? ALLOW FILTERING", 369 query.TableName), query.HashValue, query.RangeValuePrefix, append(query.RangeValuePrefix, '\xff'), query.ValueEqual) 370 371 case len(query.RangeValueStart) > 0 && query.ValueEqual == nil: 372 q = s.readSession.Query(fmt.Sprintf("SELECT range, value FROM %s WHERE hash = ? AND range >= ?", 373 query.TableName), query.HashValue, query.RangeValueStart) 374 375 case len(query.RangeValueStart) > 0 && query.ValueEqual != nil: 376 q = s.readSession.Query(fmt.Sprintf("SELECT range, value FROM %s WHERE hash = ? AND range >= ? AND value = ? ALLOW FILTERING", 377 query.TableName), query.HashValue, query.RangeValueStart, query.ValueEqual) 378 379 case query.ValueEqual == nil: 380 q = s.readSession.Query(fmt.Sprintf("SELECT range, value FROM %s WHERE hash = ?", 381 query.TableName), query.HashValue) 382 383 case query.ValueEqual != nil: 384 q = s.readSession.Query(fmt.Sprintf("SELECT range, value FROM %s WHERE hash = ? AND value = ? ALLOW FILTERING", 385 query.TableName), query.HashValue, query.ValueEqual) 386 } 387 388 iter := q.WithContext(ctx).Iter() 389 defer iter.Close() 390 scanner := iter.Scanner() 391 for scanner.Next() { 392 b := &readBatch{} 393 if err := scanner.Scan(&b.rangeValue, &b.value); err != nil { 394 return errors.WithStack(err) 395 } 396 if !callback(query, b) { 397 return nil 398 } 399 } 400 return errors.WithStack(scanner.Err()) 401 } 402 403 // Allow other packages to interact with Cassandra directly 404 func (s *StorageClient) GetReadSession() *gocql.Session { 405 return s.readSession 406 } 407 408 // readBatch represents a batch of rows read from Cassandra. 409 type readBatch struct { 410 rangeValue []byte 411 value []byte 412 } 413 414 func (r *readBatch) Iterator() chunk.ReadBatchIterator { 415 return &readBatchIter{ 416 readBatch: r, 417 } 418 } 419 420 type readBatchIter struct { 421 consumed bool 422 *readBatch 423 } 424 425 func (b *readBatchIter) Next() bool { 426 if b.consumed { 427 return false 428 } 429 b.consumed = true 430 return true 431 } 432 433 func (b *readBatchIter) RangeValue() []byte { 434 return b.rangeValue 435 } 436 437 func (b *readBatchIter) Value() []byte { 438 return b.value 439 } 440 441 // ObjectClient implements chunk.ObjectClient for Cassandra. 442 type ObjectClient struct { 443 cfg Config 444 schemaCfg chunk.SchemaConfig 445 readSession *gocql.Session 446 writeSession *gocql.Session 447 querySemaphore *semaphore.Weighted 448 } 449 450 // NewObjectClient returns a new ObjectClient. 451 func NewObjectClient(cfg Config, schemaCfg chunk.SchemaConfig, registerer prometheus.Registerer) (*ObjectClient, error) { 452 readSession, err := cfg.session("chunks-read", registerer) 453 if err != nil { 454 return nil, errors.WithStack(err) 455 } 456 457 writeSession, err := cfg.session("chunks-write", registerer) 458 if err != nil { 459 return nil, errors.WithStack(err) 460 } 461 462 var querySemaphore *semaphore.Weighted 463 if cfg.QueryConcurrency > 0 { 464 querySemaphore = semaphore.NewWeighted(int64(cfg.QueryConcurrency)) 465 } 466 467 client := &ObjectClient{ 468 cfg: cfg, 469 schemaCfg: schemaCfg, 470 readSession: readSession, 471 writeSession: writeSession, 472 querySemaphore: querySemaphore, 473 } 474 return client, nil 475 } 476 477 // PutChunks implements chunk.ObjectClient. 478 func (s *ObjectClient) PutChunks(ctx context.Context, chunks []chunk.Chunk) error { 479 for i := range chunks { 480 buf, err := chunks[i].Encoded() 481 if err != nil { 482 return errors.WithStack(err) 483 } 484 key := chunks[i].ExternalKey() 485 tableName, err := s.schemaCfg.ChunkTableFor(chunks[i].From) 486 if err != nil { 487 return err 488 } 489 490 // Must provide a range key, even though its not useds - hence 0x00. 491 q := s.writeSession.Query(fmt.Sprintf("INSERT INTO %s (hash, range, value) VALUES (?, 0x00, ?)", 492 tableName), key, buf) 493 if err := q.WithContext(ctx).Exec(); err != nil { 494 return errors.WithStack(err) 495 } 496 } 497 498 return nil 499 } 500 501 // GetChunks implements chunk.ObjectClient. 502 func (s *ObjectClient) GetChunks(ctx context.Context, input []chunk.Chunk) ([]chunk.Chunk, error) { 503 return util.GetParallelChunks(ctx, input, s.getChunk) 504 } 505 506 func (s *ObjectClient) getChunk(ctx context.Context, decodeContext *chunk.DecodeContext, input chunk.Chunk) (chunk.Chunk, error) { 507 if s.querySemaphore != nil { 508 if err := s.querySemaphore.Acquire(ctx, 1); err != nil { 509 return input, err 510 } 511 defer s.querySemaphore.Release(1) 512 } 513 514 tableName, err := s.schemaCfg.ChunkTableFor(input.From) 515 if err != nil { 516 return input, err 517 } 518 519 var buf []byte 520 if err := s.readSession.Query(fmt.Sprintf("SELECT value FROM %s WHERE hash = ?", tableName), input.ExternalKey()). 521 WithContext(ctx).Scan(&buf); err != nil { 522 return input, errors.WithStack(err) 523 } 524 err = input.Decode(decodeContext, buf) 525 return input, err 526 } 527 528 func (s *ObjectClient) DeleteChunk(ctx context.Context, userID, chunkID string) error { 529 chunkRef, err := chunk.ParseExternalKey(userID, chunkID) 530 if err != nil { 531 return err 532 } 533 534 tableName, err := s.schemaCfg.ChunkTableFor(chunkRef.From) 535 if err != nil { 536 return err 537 } 538 539 q := s.writeSession.Query(fmt.Sprintf("DELETE FROM %s WHERE hash = ?", 540 tableName), chunkID) 541 if err := q.WithContext(ctx).Exec(); err != nil { 542 return errors.WithStack(err) 543 } 544 545 return nil 546 } 547 548 // Stop implement chunk.ObjectClient. 549 func (s *ObjectClient) Stop() { 550 s.readSession.Close() 551 s.writeSession.Close() 552 } 553 554 type noopConvictionPolicy struct{} 555 556 // AddFailure should return `true` if the host should be convicted, `false` otherwise. 557 // Convicted means connections are removed - we don't want that. 558 // Implementats gocql.ConvictionPolicy. 559 func (noopConvictionPolicy) AddFailure(err error, host *gocql.HostInfo) bool { 560 level.Error(util_log.Logger).Log("msg", "Cassandra host failure", "err", err, "host", host.String()) 561 return false 562 } 563 564 // Implementats gocql.ConvictionPolicy. 565 func (noopConvictionPolicy) Reset(host *gocql.HostInfo) {}