github.com/Jeffail/benthos/v3@v3.65.0/lib/output/cassandra.go (about) 1 package output 2 3 import ( 4 "context" 5 "crypto/tls" 6 "encoding/json" 7 "fmt" 8 "math" 9 "math/rand" 10 "strconv" 11 "sync" 12 "time" 13 14 "github.com/Jeffail/benthos/v3/internal/bloblang/field" 15 "github.com/Jeffail/benthos/v3/internal/bloblang/mapping" 16 "github.com/Jeffail/benthos/v3/internal/bloblang/query" 17 "github.com/Jeffail/benthos/v3/internal/docs" 18 "github.com/Jeffail/benthos/v3/internal/interop" 19 "github.com/Jeffail/benthos/v3/lib/log" 20 "github.com/Jeffail/benthos/v3/lib/message/batch" 21 "github.com/Jeffail/benthos/v3/lib/metrics" 22 "github.com/Jeffail/benthos/v3/lib/types" 23 "github.com/Jeffail/benthos/v3/lib/util/retries" 24 btls "github.com/Jeffail/benthos/v3/lib/util/tls" 25 "github.com/gocql/gocql" 26 ) 27 28 //------------------------------------------------------------------------------ 29 30 func init() { 31 Constructors[TypeCassandra] = TypeSpec{ 32 constructor: fromSimpleConstructor(func(conf Config, mgr types.Manager, log log.Modular, stats metrics.Type) (Type, error) { 33 c, err := newCassandraWriter(conf.Cassandra, mgr, log, stats) 34 if err != nil { 35 return nil, err 36 } 37 w, err := NewAsyncWriter( 38 TypeCassandra, conf.Cassandra.MaxInFlight, c, log, stats, 39 ) 40 if err != nil { 41 return nil, err 42 } 43 return NewBatcherFromConfig(conf.Cassandra.Batching, w, mgr, log, stats) 44 }), 45 Status: docs.StatusBeta, 46 Batches: true, 47 Async: true, 48 Summary: ` 49 Runs a query against a Cassandra database for each message in order to insert data.`, 50 Description: ` 51 Query arguments can be set using [interpolation functions](/docs/configuration/interpolation#bloblang-queries) in the ` + "`args`" + ` field or by creating a bloblang array for the fields using the ` + "`args_mapping`" + ` field. 52 53 When populating timestamp columns the value must either be a string in ISO 8601 format (2006-01-02T15:04:05Z07:00), or an integer representing unix time in seconds.`, 54 Examples: []docs.AnnotatedExample{ 55 { 56 Title: "Basic Inserts", 57 Summary: "If we were to create a table with some basic columns with `CREATE TABLE foo.bar (id int primary key, content text, created_at timestamp);`, and were processing JSON documents of the form `{\"id\":\"342354354\",\"content\":\"hello world\",\"timestamp\":1605219406}`, we could populate our table with the following config:", 58 Config: ` 59 output: 60 cassandra: 61 addresses: 62 - localhost:9042 63 query: 'INSERT INTO foo.bar (id, content, created_at) VALUES (?, ?, ?)' 64 args_mapping: | 65 root = [ 66 this.id, 67 this.content, 68 this.timestamp 69 ] 70 batching: 71 count: 500 72 `, 73 }, 74 { 75 Title: "Insert JSON Documents", 76 Summary: "The following example inserts JSON documents into the table `footable` of the keyspace `foospace` using INSERT JSON (https://cassandra.apache.org/doc/latest/cql/json.html#insert-json).", 77 Config: ` 78 output: 79 cassandra: 80 addresses: 81 - localhost:9042 82 query: 'INSERT INTO foospace.footable JSON ?' 83 args_mapping: 'root = [ this ]' 84 batching: 85 count: 500 86 `, 87 }, 88 }, 89 FieldSpecs: docs.FieldSpecs{ 90 docs.FieldString( 91 "addresses", 92 "A list of Cassandra nodes to connect to. Multiple comma separated addresses can be specified on a single line.", 93 []string{"localhost:9042"}, 94 []string{"foo:9042", "bar:9042"}, 95 []string{"foo:9042,bar:9042"}, 96 ).Array(), 97 btls.FieldSpec(), 98 docs.FieldAdvanced( 99 "password_authenticator", 100 "An object containing the username and password.", 101 ).WithChildren( 102 docs.FieldCommon("enabled", "Whether to use password authentication."), 103 docs.FieldCommon("username", "A username."), 104 docs.FieldCommon("password", "A password."), 105 ), 106 docs.FieldAdvanced( 107 "disable_initial_host_lookup", 108 "If enabled the driver will not attempt to get host info from the system.peers table. This can speed up queries but will mean that data_centre, rack and token information will not be available.", 109 ), 110 docs.FieldCommon("query", "A query to execute for each message."), 111 docs.FieldDeprecated( 112 "args", 113 "A list of arguments for the query to be resolved for each message.", 114 ).IsInterpolated().Array().HasType(docs.FieldTypeString), 115 docs.FieldBloblang( 116 "args_mapping", 117 "A [Bloblang mapping](/docs/guides/bloblang/about) that can be used to provide arguments to Cassandra queries. The result of the query must be an array containing a matching number of elements to the query arguments.").AtVersion("3.55.0"), 118 docs.FieldAdvanced( 119 "consistency", 120 "The consistency level to use.", 121 ).HasOptions( 122 "ANY", "ONE", "TWO", "THREE", "QUORUM", "ALL", "LOCAL_QUORUM", "EACH_QUORUM", "LOCAL_ONE", 123 ), 124 docs.FieldAdvanced("max_retries", "The maximum number of retries before giving up on a request."), 125 docs.FieldAdvanced("backoff", "Control time intervals between retry attempts.").WithChildren( 126 docs.FieldAdvanced("initial_interval", "The initial period to wait between retry attempts."), 127 docs.FieldAdvanced("max_interval", "The maximum period to wait between retry attempts."), 128 docs.FieldDeprecated("max_elapsed_time"), 129 ), 130 docs.FieldString("timeout", "The client connection timeout.").AtVersion("3.63.0"), 131 }.Merge(docs.FieldSpecs{ 132 docs.FieldCommon("max_in_flight", "The maximum number of messages to have in flight at a given time. Increase this to improve throughput."), 133 batch.FieldSpec(), 134 }), 135 } 136 } 137 138 //------------------------------------------------------------------------------ 139 140 // PasswordAuthenticator contains the fields that will be used to authenticate with 141 // the Cassandra cluster. 142 type PasswordAuthenticator struct { 143 Enabled bool `json:"enabled" yaml:"enabled"` 144 Username string `json:"username" yaml:"username"` 145 Password string `json:"password" yaml:"password"` 146 } 147 148 // CassandraConfig contains configuration fields for the Cassandra output type. 149 type CassandraConfig struct { 150 Addresses []string `json:"addresses" yaml:"addresses"` 151 TLS btls.Config `json:"tls" yaml:"tls"` 152 PasswordAuthenticator PasswordAuthenticator `json:"password_authenticator" yaml:"password_authenticator"` 153 DisableInitialHostLookup bool `json:"disable_initial_host_lookup" yaml:"disable_initial_host_lookup"` 154 Query string `json:"query" yaml:"query"` 155 Args []string `json:"args" yaml:"args"` 156 ArgsMapping string `json:"args_mapping" yaml:"args_mapping"` 157 Consistency string `json:"consistency" yaml:"consistency"` 158 Timeout string `json:"timeout" yaml:"timeout"` 159 // TODO: V4 Remove this and replace with explicit values. 160 retries.Config `json:",inline" yaml:",inline"` 161 MaxInFlight int `json:"max_in_flight" yaml:"max_in_flight"` 162 Batching batch.PolicyConfig `json:"batching" yaml:"batching"` 163 } 164 165 // NewCassandraConfig creates a new CassandraConfig with default values. 166 func NewCassandraConfig() CassandraConfig { 167 rConf := retries.NewConfig() 168 rConf.MaxRetries = 3 169 rConf.Backoff.InitialInterval = "1s" 170 rConf.Backoff.MaxInterval = "5s" 171 rConf.Backoff.MaxElapsedTime = "" 172 173 return CassandraConfig{ 174 Addresses: []string{}, 175 TLS: btls.NewConfig(), 176 PasswordAuthenticator: PasswordAuthenticator{ 177 Enabled: false, 178 Username: "", 179 Password: "", 180 }, 181 DisableInitialHostLookup: false, 182 Query: "", 183 Args: []string{}, 184 ArgsMapping: "", 185 Consistency: gocql.Quorum.String(), 186 Timeout: "600ms", 187 Config: rConf, 188 MaxInFlight: 1, 189 Batching: batch.NewPolicyConfig(), 190 } 191 } 192 193 type cassandraWriter struct { 194 conf CassandraConfig 195 log log.Modular 196 stats metrics.Type 197 tlsConf *tls.Config 198 199 backoffMin time.Duration 200 backoffMax time.Duration 201 202 session *gocql.Session 203 mQueryLatency metrics.StatTimer 204 connLock sync.RWMutex 205 206 args []*field.Expression 207 argsMapping *mapping.Executor 208 } 209 210 func newCassandraWriter(conf CassandraConfig, mgr types.Manager, log log.Modular, stats metrics.Type) (*cassandraWriter, error) { 211 c := cassandraWriter{ 212 log: log, 213 stats: stats, 214 conf: conf, 215 mQueryLatency: stats.GetTimer("query.latency"), 216 } 217 218 var err error 219 if conf.TLS.Enabled { 220 if c.tlsConf, err = conf.TLS.Get(); err != nil { 221 return nil, err 222 } 223 } 224 if c.backoffMin, err = time.ParseDuration(c.conf.Config.Backoff.InitialInterval); err != nil { 225 return nil, fmt.Errorf("parsing backoff initial interval: %w", err) 226 } 227 if c.backoffMax, err = time.ParseDuration(c.conf.Config.Backoff.MaxInterval); err != nil { 228 return nil, fmt.Errorf("parsing backoff max interval: %w", err) 229 } 230 if err = c.parseArgs(mgr); err != nil { 231 return nil, fmt.Errorf("parsing args: %w", err) 232 } 233 234 return &c, nil 235 } 236 237 func (c *cassandraWriter) parseArgs(mgr types.Manager) error { 238 // Allow only args or args_mapping for now. 239 if len(c.conf.Args) > 0 && c.conf.ArgsMapping != "" { 240 return fmt.Errorf("can only specify one of [args, args_mapping]") 241 } 242 243 if len(c.conf.Args) > 0 { 244 for i, v := range c.conf.Args { 245 expr, err := interop.NewBloblangField(mgr, v) 246 if err != nil { 247 return fmt.Errorf("failed to parse arg %v expression: %v", i, err) 248 } 249 c.args = append(c.args, expr) 250 } 251 } 252 253 if c.conf.ArgsMapping != "" { 254 var err error 255 if c.argsMapping, err = interop.NewBloblangMapping(mgr, c.conf.ArgsMapping); err != nil { 256 return fmt.Errorf("parsing args_mapping: %w", err) 257 } 258 } 259 260 return nil 261 } 262 263 // ConnectWithContext establishes a connection to Cassandra. 264 func (c *cassandraWriter) ConnectWithContext(ctx context.Context) error { 265 c.connLock.Lock() 266 defer c.connLock.Unlock() 267 if c.session != nil { 268 return nil 269 } 270 271 var err error 272 conn := gocql.NewCluster(c.conf.Addresses...) 273 if c.tlsConf != nil { 274 conn.SslOpts = &gocql.SslOptions{ 275 Config: c.tlsConf, 276 CaPath: c.conf.TLS.RootCAsFile, 277 } 278 conn.DisableInitialHostLookup = c.conf.TLS.InsecureSkipVerify 279 } 280 if c.conf.PasswordAuthenticator.Enabled { 281 conn.Authenticator = gocql.PasswordAuthenticator{ 282 Username: c.conf.PasswordAuthenticator.Username, 283 Password: c.conf.PasswordAuthenticator.Password, 284 } 285 } 286 conn.DisableInitialHostLookup = c.conf.DisableInitialHostLookup 287 if conn.Consistency, err = gocql.ParseConsistencyWrapper(c.conf.Consistency); err != nil { 288 return fmt.Errorf("parsing consistency: %w", err) 289 } 290 291 conn.RetryPolicy = &decorator{ 292 NumRetries: int(c.conf.Config.MaxRetries), 293 Min: c.backoffMin, 294 Max: c.backoffMax, 295 } 296 if tout := c.conf.Timeout; len(tout) > 0 { 297 var err error 298 if conn.Timeout, err = time.ParseDuration(tout); err != nil { 299 return fmt.Errorf("failed to parse timeout string: %v", err) 300 } 301 } 302 session, err := conn.CreateSession() 303 if err != nil { 304 return fmt.Errorf("creating Cassandra session: %w", err) 305 } 306 307 c.session = session 308 c.log.Infof("Sending messages to Cassandra: %v\n", c.conf.Addresses) 309 return nil 310 } 311 312 // WriteWithContext writes a message to Cassandra. 313 func (c *cassandraWriter) WriteWithContext(ctx context.Context, msg types.Message) error { 314 c.connLock.RLock() 315 session := c.session 316 c.connLock.RUnlock() 317 318 if c.session == nil { 319 return types.ErrNotConnected 320 } 321 322 if msg.Len() == 1 { 323 return c.writeRow(session, msg) 324 } 325 return c.writeBatch(session, msg) 326 } 327 328 func (c *cassandraWriter) writeRow(session *gocql.Session, msg types.Message) error { 329 t0 := time.Now() 330 331 values, err := c.mapArgs(msg, 0) 332 if err != nil { 333 return fmt.Errorf("parsing args: %w", err) 334 } 335 336 if err := session.Query(c.conf.Query, values...).Exec(); err != nil { 337 return err 338 } 339 340 c.mQueryLatency.Timing(time.Since(t0).Nanoseconds()) 341 return nil 342 } 343 344 func (c *cassandraWriter) writeBatch(session *gocql.Session, msg types.Message) error { 345 batch := session.NewBatch(gocql.UnloggedBatch) 346 t0 := time.Now() 347 348 if err := msg.Iter(func(i int, p types.Part) error { 349 values, err := c.mapArgs(msg, i) 350 if err != nil { 351 return fmt.Errorf("parsing args for part: %d: %w", i, err) 352 } 353 batch.Query(c.conf.Query, values...) 354 return nil 355 }); err != nil { 356 return err 357 } 358 359 err := session.ExecuteBatch(batch) 360 if err != nil { 361 return err 362 } 363 c.mQueryLatency.Timing(time.Since(t0).Nanoseconds()) 364 return nil 365 } 366 367 func (c *cassandraWriter) mapArgs(msg types.Message, index int) ([]interface{}, error) { 368 if c.argsMapping != nil { 369 // We've got an "args_mapping" field, extract values from there. 370 part, err := c.argsMapping.MapPart(index, msg) 371 if err != nil { 372 return nil, fmt.Errorf("executing bloblang mapping: %w", err) 373 } 374 375 jraw, err := part.JSON() 376 if err != nil { 377 return nil, fmt.Errorf("parsing bloblang mapping result as json: %w", err) 378 } 379 380 j, ok := jraw.([]interface{}) 381 if !ok { 382 return nil, fmt.Errorf("expected bloblang mapping result to be []interface{} but was %T", jraw) 383 } 384 385 for i, v := range j { 386 j[i] = genericValue{v: v} 387 } 388 return j, nil 389 } 390 391 // If we've been given the "args" field, extract values from there. 392 if len(c.args) > 0 { 393 values := make([]interface{}, 0, len(c.args)) 394 for _, arg := range c.args { 395 values = append(values, stringValue(arg.String(index, msg))) 396 } 397 return values, nil 398 } 399 400 return nil, nil 401 } 402 403 // CloseAsync shuts down the Cassandra output and stops processing messages. 404 func (c *cassandraWriter) CloseAsync() { 405 go func() { 406 c.connLock.Lock() 407 if c.session != nil { 408 c.session.Close() 409 c.session = nil 410 } 411 c.connLock.Unlock() 412 }() 413 } 414 415 // WaitForClose blocks until the Cassandra output has closed down. 416 func (c *cassandraWriter) WaitForClose(timeout time.Duration) error { 417 return nil 418 } 419 420 type decorator struct { 421 NumRetries int 422 Min, Max time.Duration 423 } 424 425 func (d *decorator) Attempt(q gocql.RetryableQuery) bool { 426 if q.Attempts() > d.NumRetries { 427 return false 428 } 429 time.Sleep(getExponentialTime(d.Min, d.Max, q.Attempts())) 430 return true 431 } 432 433 func getExponentialTime(min, max time.Duration, attempts int) time.Duration { 434 minFloat := float64(min) 435 napDuration := minFloat * math.Pow(2, float64(attempts-1)) 436 437 // Add some jitter 438 napDuration += rand.Float64()*minFloat - (minFloat / 2) 439 if napDuration > float64(max) { 440 return max 441 } 442 return time.Duration(napDuration) 443 } 444 445 func (d *decorator) GetRetryType(err error) gocql.RetryType { 446 switch t := err.(type) { 447 // not enough replica alive to perform query with required consistency 448 case *gocql.RequestErrUnavailable: 449 if t.Alive > 0 { 450 return gocql.RetryNextHost 451 } 452 return gocql.Retry 453 // write timeout - uncertain whetever write was succesful or not 454 case *gocql.RequestErrWriteTimeout: 455 if t.Received > 0 { 456 return gocql.Ignore 457 } 458 return gocql.Retry 459 default: 460 return gocql.Rethrow 461 } 462 } 463 464 func formatCassandraInt64(x int64) []byte { 465 return []byte{byte(x >> 56), byte(x >> 48), byte(x >> 40), byte(x >> 32), 466 byte(x >> 24), byte(x >> 16), byte(x >> 8), byte(x)} 467 } 468 469 func formatCassandraInt32(x int32) []byte { 470 return []byte{byte(x >> 24), byte(x >> 16), byte(x >> 8), byte(x)} 471 } 472 473 type stringValue string 474 475 // All of our argument values are string types due to interpolation. However, 476 // gocql performs type checking and unfortunately does not like timestamp and 477 // some other values as strings: 478 // https://github.com/gocql/gocql/blob/5913df4d474e0b2492a129d17bbb3c04537a15cd/marshal.go#L1160 479 // 480 // In order to work around this we manually marshal some types. 481 func (s stringValue) MarshalCQL(info gocql.TypeInfo) ([]byte, error) { 482 switch info.Type() { 483 case gocql.TypeTimestamp: 484 t, err := time.Parse(time.RFC3339Nano, string(s)) 485 if err == nil { 486 if t.IsZero() { 487 return []byte{}, nil 488 } 489 x := t.UTC().Unix()*1e3 + int64(t.UTC().Nanosecond()/1e6) 490 return formatCassandraInt64(x), nil 491 } 492 x, err := strconv.ParseInt(string(s), 10, 64) 493 if err != nil { 494 return nil, fmt.Errorf("failed to parse time value '%v': expected either an ISO 8601 string or unix epoch in seconds", s) 495 } 496 return formatCassandraInt64(x * 1e3), nil 497 case gocql.TypeTime: 498 x, err := strconv.ParseInt(string(s), 10, 64) 499 if err != nil { 500 return nil, fmt.Errorf("failed to parse time value '%v': expected milliseconds", s) 501 } 502 return formatCassandraInt64(x), nil 503 case gocql.TypeBoolean: 504 if s == "true" { 505 return []byte{1}, nil 506 } else if s == "false" { 507 return []byte{0}, nil 508 } 509 case gocql.TypeFloat: 510 f, err := strconv.ParseFloat(string(s), 32) 511 if err != nil { 512 return nil, err 513 } 514 return formatCassandraInt32(int32(math.Float32bits(float32(f)))), nil 515 case gocql.TypeDouble: 516 f, err := strconv.ParseFloat(string(s), 64) 517 if err != nil { 518 return nil, err 519 } 520 return formatCassandraInt64(int64(math.Float64bits(f))), nil 521 } 522 return gocql.Marshal(info, string(s)) 523 } 524 525 type genericValue struct { 526 v interface{} 527 } 528 529 // We get typed values out of mappings. However, gocql performs type checking 530 // and unfortunately does not like timestamp and some other values as strings: 531 // https://github.com/gocql/gocql/blob/5913df4d474e0b2492a129d17bbb3c04537a15cd/marshal.go#L1160 532 // it's also very strict on numerical types, so we need to do some magic here. 533 func (g genericValue) MarshalCQL(info gocql.TypeInfo) ([]byte, error) { 534 switch info.Type() { 535 case gocql.TypeTimestamp: 536 t, err := query.IGetTimestamp(g.v) 537 if err != nil { 538 return nil, err 539 } 540 return gocql.Marshal(info, t) 541 case gocql.TypeDouble: 542 f, err := query.IGetNumber(g.v) 543 if err != nil { 544 return nil, err 545 } 546 return gocql.Marshal(info, f) 547 case gocql.TypeFloat: 548 f, err := query.IGetFloat32(g.v) 549 if err != nil { 550 return nil, err 551 } 552 return gocql.Marshal(info, f) 553 case gocql.TypeVarchar: 554 return gocql.Marshal(info, query.IToString(g.v)) 555 } 556 if _, isJSONNum := g.v.(json.Number); isJSONNum { 557 i, err := query.IGetInt(g.v) 558 if err != nil { 559 return nil, err 560 } 561 return gocql.Marshal(info, i) 562 } 563 return gocql.Marshal(info, g.v) 564 }