github.com/Jeffail/benthos/v3@v3.65.0/lib/output/sql.go (about) 1 package output 2 3 import ( 4 "context" 5 "database/sql" 6 "errors" 7 "fmt" 8 "strings" 9 "sync" 10 "time" 11 12 "github.com/Jeffail/benthos/v3/internal/bloblang/field" 13 "github.com/Jeffail/benthos/v3/internal/bloblang/mapping" 14 "github.com/Jeffail/benthos/v3/internal/docs" 15 "github.com/Jeffail/benthos/v3/internal/interop" 16 "github.com/Jeffail/benthos/v3/lib/log" 17 "github.com/Jeffail/benthos/v3/lib/message/batch" 18 "github.com/Jeffail/benthos/v3/lib/metrics" 19 "github.com/Jeffail/benthos/v3/lib/output/writer" 20 "github.com/Jeffail/benthos/v3/lib/types" 21 22 // SQL Drivers 23 _ "github.com/ClickHouse/clickhouse-go" 24 _ "github.com/denisenkom/go-mssqldb" 25 _ "github.com/go-sql-driver/mysql" 26 ) 27 28 //------------------------------------------------------------------------------ 29 30 func init() { 31 Constructors[TypeSQL] = TypeSpec{ 32 constructor: fromSimpleConstructor(func(conf Config, mgr types.Manager, log log.Modular, stats metrics.Type) (Type, error) { 33 if conf.SQL.Driver == "mssql" { 34 // For MSSQL, if the user part of the connection string is in the 35 // `DOMAIN\username` format, then the backslash character needs 36 // to be URL-encoded. 37 conf.SQL.DataSourceName = strings.ReplaceAll(conf.SQL.DataSourceName, `\`, "%5C") 38 } 39 40 s, err := newSQLWriter(conf.SQL, mgr, log) 41 if err != nil { 42 return nil, err 43 } 44 w, err := NewAsyncWriter(TypeSQL, conf.SQL.MaxInFlight, s, log, stats) 45 if err != nil { 46 return nil, err 47 } 48 return NewBatcherFromConfig(conf.SQL.Batching, w, mgr, log, stats) 49 }), 50 Status: docs.StatusStable, 51 Batches: true, 52 Async: true, 53 Version: "3.33.0", 54 Categories: []Category{ 55 CategoryServices, 56 }, 57 Summary: ` 58 Runs an SQL prepared query against a target database for each message.`, 59 Description: ` 60 ## Alternatives 61 62 For basic inserts use the ` + "[`sql_insert`](/docs/components/outputs/sql_insert)" + ` output instead. For more complex queries use the ` + "[`sql_raw`](/docs/components/outputs/sql_raw)" + ` output.`, 63 Examples: []docs.AnnotatedExample{ 64 { 65 Title: "Table Insert (MySQL)", 66 Summary: ` 67 The following example inserts rows into the table footable with the columns foo, 68 bar and baz populated with values extracted from messages:`, 69 Config: ` 70 output: 71 sql: 72 driver: mysql 73 data_source_name: foouser:foopassword@tcp(localhost:3306)/foodb 74 query: "INSERT INTO footable (foo, bar, baz) VALUES (?, ?, ?);" 75 args_mapping: '[ this.document.foo, this.document.bar, meta("kafka_topic") ]' 76 batching: 77 count: 500 78 `, 79 }, 80 { 81 Title: "Table Insert (PostgreSQL)", 82 Summary: ` 83 The following example inserts rows into the table footable with the columns foo, 84 bar and baz populated with values extracted from messages:`, 85 Config: ` 86 output: 87 sql: 88 driver: postgres 89 data_source_name: postgres://foouser:foopassword@localhost:5432/foodb?sslmode=disable 90 query: "INSERT INTO footable (foo, bar, baz) VALUES ($1, $2, $3);" 91 args_mapping: '[ this.document.foo, this.document.bar, meta("kafka_topic") ]' 92 batching: 93 count: 500 94 `, 95 }, 96 }, 97 FieldSpecs: docs.FieldSpecs{ 98 docs.FieldCommon( 99 "driver", 100 "A database [driver](#drivers) to use.", 101 ).HasOptions("mysql", "postgres", "clickhouse", "mssql"), 102 docs.FieldCommon( 103 "data_source_name", "A Data Source Name to identify the target database.", 104 "tcp://host1:9000?username=user&password=qwerty&database=clicks&read_timeout=10&write_timeout=20&alt_hosts=host2:9000,host3:9000", 105 "foouser:foopassword@tcp(localhost:3306)/foodb", 106 "postgres://foouser:foopass@localhost:5432/foodb?sslmode=disable", 107 ), 108 docs.FieldCommon( 109 "query", "The query to run against the database.", 110 "INSERT INTO footable (foo, bar, baz) VALUES (?, ?, ?);", 111 ), 112 docs.FieldDeprecated( 113 "args", 114 "A list of arguments for the query to be resolved for each message.", 115 ).IsInterpolated().Array(), 116 docs.FieldBloblang( 117 "args_mapping", 118 "A [Bloblang mapping](/docs/guides/bloblang/about) that produces the arguments for the query. The mapping must return an array containing the number of arguments in the query.", 119 `[ this.foo, this.bar.not_empty().catch(null), meta("baz") ]`, 120 `root = [ uuid_v4() ].merge(this.document.args)`, 121 ).AtVersion("3.47.0"), 122 docs.FieldCommon("max_in_flight", "The maximum number of messages to have in flight at a given time. Increase this to improve throughput."), 123 batch.FieldSpec(), 124 }, 125 } 126 } 127 128 //------------------------------------------------------------------------------ 129 130 // SQLConfig contains configuration fields for the SQL processor. 131 type SQLConfig struct { 132 Driver string `json:"driver" yaml:"driver"` 133 DataSourceName string `json:"data_source_name" yaml:"data_source_name"` 134 Query string `json:"query" yaml:"query"` 135 Args []string `json:"args" yaml:"args"` 136 ArgsMapping string `json:"args_mapping" yaml:"args_mapping"` 137 MaxInFlight int `json:"max_in_flight" yaml:"max_in_flight"` 138 Batching batch.PolicyConfig `json:"batching" yaml:"batching"` 139 } 140 141 // NewSQLConfig returns a SQLConfig with default values. 142 func NewSQLConfig() SQLConfig { 143 return SQLConfig{ 144 Driver: "mysql", 145 DataSourceName: "", 146 Query: "", 147 Args: []string{}, 148 ArgsMapping: "", 149 MaxInFlight: 1, 150 Batching: batch.NewPolicyConfig(), 151 } 152 } 153 154 //------------------------------------------------------------------------------ 155 156 func insertOnlyBatchDriver(driver string) bool { 157 _, exists := map[string]struct{}{ 158 "clickhouse": {}, 159 }[driver] 160 return exists 161 } 162 163 //------------------------------------------------------------------------------ 164 165 type sqlWriter struct { 166 log log.Modular 167 conf SQLConfig 168 169 db *sql.DB 170 dbMut sync.Mutex 171 args []*field.Expression 172 argsMapping *mapping.Executor 173 174 query *sql.Stmt 175 } 176 177 func newSQLWriter(conf SQLConfig, mgr types.Manager, log log.Modular) (*sqlWriter, error) { 178 if len(conf.Args) > 0 && conf.ArgsMapping != "" { 179 return nil, errors.New("cannot specify both `args` and an `args_mapping` in the same output") 180 } 181 182 var args []*field.Expression 183 for i, v := range conf.Args { 184 expr, err := interop.NewBloblangField(mgr, v) 185 if err != nil { 186 return nil, fmt.Errorf("failed to parse arg %v expression: %v", i, err) 187 } 188 args = append(args, expr) 189 } 190 191 var argsMapping *mapping.Executor 192 if conf.ArgsMapping != "" { 193 var err error 194 if argsMapping, err = interop.NewBloblangMapping(mgr, conf.ArgsMapping); err != nil { 195 return nil, fmt.Errorf("failed to parse `args_mapping`: %w", err) 196 } 197 } 198 199 s := &sqlWriter{ 200 log: log, 201 conf: conf, 202 args: args, 203 argsMapping: argsMapping, 204 } 205 206 return s, nil 207 } 208 209 //------------------------------------------------------------------------------ 210 211 // ConnectWithContext attempts to establish a connection to the target database. 212 func (s *sqlWriter) ConnectWithContext(ctx context.Context) error { 213 s.dbMut.Lock() 214 defer s.dbMut.Unlock() 215 216 if s.db != nil { 217 return nil 218 } 219 220 var err error 221 db, err := sql.Open(s.conf.Driver, s.conf.DataSourceName) 222 if err != nil { 223 return err 224 } 225 226 // Some drivers only support transactional prepared inserts. 227 if !insertOnlyBatchDriver(s.conf.Driver) { 228 if s.query, err = db.Prepare(s.conf.Query); err != nil { 229 db.Close() 230 return fmt.Errorf("failed to prepare query: %v", err) 231 } 232 } 233 234 s.log.Infof("Writing messages to %v database.\n", s.conf.Driver) 235 s.db = db 236 return nil 237 } 238 239 func (s *sqlWriter) doExecute(argSets [][]interface{}) (errs []error) { 240 s.dbMut.Lock() 241 db := s.db 242 stmt := s.query 243 s.dbMut.Unlock() 244 245 var err error 246 defer func() { 247 if err != nil { 248 if len(errs) == 0 { 249 errs = make([]error, len(argSets)) 250 } 251 for i := range errs { 252 if errs[i] == nil { 253 errs[i] = err 254 } 255 } 256 } 257 }() 258 259 if db == nil { 260 err = types.ErrNotConnected 261 return 262 } 263 264 var tx *sql.Tx 265 if tx, err = db.Begin(); err != nil { 266 return 267 } 268 269 if stmt == nil { 270 if stmt, err = tx.Prepare(s.conf.Query); err != nil { 271 return 272 } 273 defer stmt.Close() 274 } else { 275 stmt = tx.Stmt(stmt) 276 } 277 278 for i, args := range argSets { 279 if _, serr := stmt.Exec(args...); serr != nil { 280 if len(errs) == 0 { 281 errs = make([]error, len(argSets)) 282 } 283 errs[i] = serr 284 } 285 } 286 287 err = tx.Commit() 288 return 289 } 290 291 func (s *sqlWriter) getArgs(index int, msg types.Message) ([]interface{}, error) { 292 if len(s.args) > 0 { 293 args := make([]interface{}, len(s.args)) 294 for i, v := range s.args { 295 args[i] = v.String(index, msg) 296 } 297 return args, nil 298 } 299 300 if s.argsMapping == nil { 301 return nil, nil 302 } 303 304 pargs, err := s.argsMapping.MapPart(index, msg) 305 if err != nil { 306 return nil, err 307 } 308 309 iargs, err := pargs.JSON() 310 if err != nil { 311 return nil, fmt.Errorf("mapping returned non-structured result: %w", err) 312 } 313 314 args, ok := iargs.([]interface{}) 315 if !ok { 316 return nil, fmt.Errorf("mapping returned non-array result: %T", iargs) 317 } 318 return args, nil 319 } 320 321 // WriteWithContext attempts to write a message to the database. 322 func (s *sqlWriter) WriteWithContext(ctx context.Context, msg types.Message) error { 323 argSets := make([][]interface{}, msg.Len()) 324 if err := msg.Iter(func(index int, p types.Part) error { 325 args, err := s.getArgs(index, msg) 326 if err != nil { 327 return err 328 } 329 argSets[index] = args 330 return nil 331 }); err != nil { 332 return err 333 } 334 335 errs := s.doExecute(argSets) 336 return writer.IterateBatchedSend(msg, func(i int, _ types.Part) error { 337 if len(errs) > i { 338 return errs[i] 339 } 340 return nil 341 }) 342 } 343 344 // CloseAsync shuts down the processor and stops processing requests. 345 func (s *sqlWriter) CloseAsync() { 346 go func() { 347 s.dbMut.Lock() 348 if s.db != nil { 349 s.db.Close() 350 } 351 if s.query != nil { 352 s.query.Close() 353 } 354 s.db = nil 355 s.query = nil 356 s.dbMut.Unlock() 357 }() 358 } 359 360 // WaitForClose blocks until the processor has closed down. 361 func (s *sqlWriter) WaitForClose(timeout time.Duration) error { 362 return nil 363 } 364 365 //------------------------------------------------------------------------------