forked from redpanda-data/connect
-
Notifications
You must be signed in to change notification settings - Fork 0
/
sql.go
558 lines (494 loc) · 16 KB
/
sql.go
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
package processor
import (
"database/sql"
"errors"
"fmt"
"strings"
"sync"
"time"
"github.com/dafanshu/benthos/v3/internal/bloblang/field"
"github.com/dafanshu/benthos/v3/internal/bloblang/mapping"
"github.com/dafanshu/benthos/v3/internal/docs"
"github.com/dafanshu/benthos/v3/internal/interop"
"github.com/dafanshu/benthos/v3/internal/tracing"
"github.com/dafanshu/benthos/v3/lib/log"
"github.com/dafanshu/benthos/v3/lib/metrics"
"github.com/dafanshu/benthos/v3/lib/types"
// SQL Drivers
_ "github.com/ClickHouse/clickhouse-go"
_ "github.com/denisenkom/go-mssqldb"
_ "github.com/go-sql-driver/mysql"
)
//------------------------------------------------------------------------------
func init() {
Constructors[TypeSQL] = TypeSpec{
constructor: NewSQL,
Categories: []Category{
CategoryIntegration,
},
Status: docs.StatusStable,
Summary: `
Runs an SQL prepared query against a target database for each message and, for
queries that return rows, replaces it with the result according to a
[codec](#result-codecs).`,
Description: `
## Alternatives
For basic inserts or select queries use use either the ` + "[`sql_insert`](/docs/components/processors/sql_insert)" + ` or the ` + "[`sql_select`](/docs/components/processors/sql_select)" + ` processor.
For more complex queries use the ` + "[`sql_raw`](/docs/components/processors/sql_raw)" + ` processor.`,
Examples: []docs.AnnotatedExample{
{
Title: "Table Insert (MySQL)",
Summary: `
The following example inserts rows into the table footable with the columns foo,
bar and baz populated with values extracted from messages:`,
Config: `
pipeline:
processors:
- sql:
driver: mysql
data_source_name: foouser:foopassword@tcp(localhost:3306)/foodb
query: "INSERT INTO footable (foo, bar, baz) VALUES (?, ?, ?);"
args_mapping: '[ document.foo, document.bar, meta("kafka_topic") ]'
`,
},
{
Title: "Table Query (PostgreSQL)",
Summary: `
Here we query a database for columns of footable that share a ` + "`user_id`" + `
with the message ` + "`user.id`" + `. The ` + "`result_codec`" + ` is set to
` + "`json_array`" + ` and a ` + "[`branch` processor](/docs/components/processors/branch)" + `
is used in order to insert the resulting array into the original message at the
path ` + "`foo_rows`" + `:`,
Config: `
pipeline:
processors:
- branch:
processors:
- sql:
driver: postgres
result_codec: json_array
data_source_name: postgres://foouser:foopass@localhost:5432/testdb?sslmode=disable
query: "SELECT * FROM footable WHERE user_id = $1;"
args_mapping: '[ this.user.id ]'
result_map: 'root.foo_rows = this'
`,
},
},
FieldSpecs: docs.FieldSpecs{
docs.FieldCommon(
"driver",
"A database [driver](#drivers) to use.",
).HasOptions("mysql", "postgres", "clickhouse", "mssql"),
docs.FieldCommon(
"data_source_name", "A Data Source Name to identify the target database.",
"tcp://host1:9000?username=user&password=qwerty&database=clicks&read_timeout=10&write_timeout=20&alt_hosts=host2:9000,host3:9000",
"foouser:foopassword@tcp(localhost:3306)/foodb",
"postgres://foouser:foopass@localhost:5432/foodb?sslmode=disable",
),
docs.FieldDeprecated("dsn", ""),
docs.FieldCommon(
"query", "The query to run against the database.",
"INSERT INTO footable (foo, bar, baz) VALUES (?, ?, ?);",
),
docs.FieldBool(
"unsafe_dynamic_query",
"Whether to enable dynamic queries that support interpolation functions. WARNING: This feature opens up the possibility of SQL injection attacks and is considered unsafe.",
).Advanced().HasDefault(false),
docs.FieldDeprecated(
"args",
"A list of arguments for the query to be resolved for each message.",
).IsInterpolated().Array(),
docs.FieldBloblang(
"args_mapping",
"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.",
`[ this.foo, this.bar.not_empty().catch(null), meta("baz") ]`,
`root = [ uuid_v4() ].merge(this.document.args)`,
).AtVersion("3.47.0"),
docs.FieldCommon(
"result_codec",
"A [codec](#result-codecs) to determine how resulting rows are converted into messages.",
).HasOptions("none", "json_array"),
},
Footnotes: `
## Result Codecs
When a query returns rows they are serialised according to a chosen codec, and
the message contents are replaced with the serialised result.
### ` + "`none`" + `
The result of the query is ignored and the message remains unchanged. If your
query does not return rows then this is the appropriate codec.
### ` + "`json_array`" + `
The resulting rows are serialised into an array of JSON objects, where each
object represents a row, where the key is the column name and the value is that
columns value in the row.`,
}
}
//------------------------------------------------------------------------------
// SQLConfig contains configuration fields for the SQL processor.
type SQLConfig struct {
Driver string `json:"driver" yaml:"driver"`
DataSourceName string `json:"data_source_name" yaml:"data_source_name"`
DSN string `json:"dsn" yaml:"dsn"`
Query string `json:"query" yaml:"query"`
UnsafeDynamicQuery bool `json:"unsafe_dynamic_query" yaml:"unsafe_dynamic_query"`
Args []string `json:"args" yaml:"args"`
ArgsMapping string `json:"args_mapping" yaml:"args_mapping"`
ResultCodec string `json:"result_codec" yaml:"result_codec"`
}
// NewSQLConfig returns a SQLConfig with default values.
func NewSQLConfig() SQLConfig {
return SQLConfig{
Driver: "mysql",
DataSourceName: "",
DSN: "",
Query: "",
UnsafeDynamicQuery: false,
Args: []string{},
ArgsMapping: "",
ResultCodec: "none",
}
}
//------------------------------------------------------------------------------
// Some SQL drivers (such as clickhouse) require prepared inserts to be local to
// a transaction, rather than general.
func insertRequiresTransactionPrepare(driver string) bool {
_, exists := map[string]struct{}{
"clickhouse": {},
}[driver]
return exists
}
//------------------------------------------------------------------------------
// SQL is a processor that executes an SQL query for each message.
type SQL struct {
log log.Modular
stats metrics.Type
conf SQLConfig
db *sql.DB
dbMux sync.RWMutex
args []*field.Expression
argsMapping *mapping.Executor
resCodec sqlResultCodec
// TODO: V4 Remove this
deprecated bool
resCodecDeprecated sqlResultCodecDeprecated
queryStr string
dynQuery *field.Expression
query *sql.Stmt
closeChan chan struct{}
closedChan chan struct{}
closeOnce sync.Once
mCount metrics.StatCounter
mErr metrics.StatCounter
mSent metrics.StatCounter
mBatchSent metrics.StatCounter
}
// NewSQL returns a SQL processor.
func NewSQL(
conf Config, mgr types.Manager, log log.Modular, stats metrics.Type,
) (Type, error) {
deprecated := false
dsn := conf.SQL.DataSourceName
if len(conf.SQL.DSN) > 0 {
if len(dsn) > 0 {
return nil, errors.New("specified both a deprecated `dsn` as well as a `data_source_name`")
}
dsn = conf.SQL.DSN
deprecated = true
}
if len(conf.SQL.Args) > 0 && conf.SQL.ArgsMapping != "" {
return nil, errors.New("cannot specify both `args` and an `args_mapping` in the same processor")
}
var argsMapping *mapping.Executor
if conf.SQL.ArgsMapping != "" {
if deprecated {
return nil, errors.New("the field `args_mapping` cannot be used when running the `sql` processor in deprecated mode (using the `dsn` field), use the `data_source_name` field instead")
}
log.Warnln("using unsafe_dynamic_query leaves you vulnerable to SQL injection attacks")
var err error
if argsMapping, err = interop.NewBloblangMapping(mgr, conf.SQL.ArgsMapping); err != nil {
return nil, fmt.Errorf("failed to parse `args_mapping`: %w", err)
}
}
var args []*field.Expression
for i, v := range conf.SQL.Args {
expr, err := interop.NewBloblangField(mgr, v)
if err != nil {
return nil, fmt.Errorf("failed to parse arg %v expression: %v", i, err)
}
args = append(args, expr)
}
if conf.SQL.Driver == "mssql" {
// For MSSQL, if the user part of the connection string is in the
// `DOMAIN\username` format, then the backslash character needs to be
// URL-encoded.
conf.SQL.DataSourceName = strings.ReplaceAll(conf.SQL.DataSourceName, `\`, "%5C")
}
s := &SQL{
log: log,
stats: stats,
conf: conf.SQL,
args: args,
argsMapping: argsMapping,
queryStr: conf.SQL.Query,
deprecated: deprecated,
closeChan: make(chan struct{}),
closedChan: make(chan struct{}),
mCount: stats.GetCounter("count"),
mErr: stats.GetCounter("error"),
mSent: stats.GetCounter("sent"),
mBatchSent: stats.GetCounter("batch.sent"),
}
var err error
if deprecated {
s.log.Warnln("Using deprecated SQL functionality due to use of field 'dsn'. To switch to the new processor use the field 'data_source_name' instead. The new processor is not backwards compatible due to differences in how message batches are processed. For more information check out the docs at https://www.benthos.dev/docs/components/processors/sql.")
if conf.SQL.Driver != "mysql" && conf.SQL.Driver != "postgres" && conf.SQL.Driver != "mssql" {
return nil, fmt.Errorf("driver '%v' is not supported with deprecated SQL features (using field 'dsn')", conf.SQL.Driver)
}
if s.resCodecDeprecated, err = strToSQLResultCodecDeprecated(conf.SQL.ResultCodec); err != nil {
return nil, err
}
} else if s.resCodec, err = strToSQLResultCodec(conf.SQL.ResultCodec); err != nil {
return nil, err
}
if s.db, err = sql.Open(conf.SQL.Driver, dsn); err != nil {
return nil, err
}
if conf.SQL.UnsafeDynamicQuery {
if deprecated {
return nil, errors.New("cannot use dynamic queries when running in deprecated mode")
}
if s.dynQuery, err = interop.NewBloblangField(mgr, s.queryStr); err != nil {
return nil, fmt.Errorf("failed to parse dynamic query expression: %v", err)
}
}
isSelectQuery := s.resCodecDeprecated != nil || s.resCodec != nil
// Some drivers only support transactional prepared inserts.
if s.dynQuery == nil && (isSelectQuery || !insertRequiresTransactionPrepare(conf.SQL.Driver)) {
if s.query, err = s.db.Prepare(s.queryStr); err != nil {
s.db.Close()
return nil, fmt.Errorf("failed to prepare query: %v", err)
}
}
go func() {
defer func() {
s.dbMux.Lock()
s.db.Close()
if s.query != nil {
s.query.Close()
}
s.dbMux.Unlock()
close(s.closedChan)
}()
<-s.closeChan
}()
return s, nil
}
//------------------------------------------------------------------------------
type sqlResultCodec func(rows *sql.Rows, part types.Part) error
func sqlResultJSONArrayCodec(rows *sql.Rows, part types.Part) error {
columnNames, err := rows.Columns()
if err != nil {
return err
}
jArray := []interface{}{}
for rows.Next() {
values := make([]interface{}, len(columnNames))
valuesWrapped := make([]interface{}, len(columnNames))
for i := range values {
valuesWrapped[i] = &values[i]
}
if err := rows.Scan(valuesWrapped...); err != nil {
return err
}
jObj := map[string]interface{}{}
for i, v := range values {
switch t := v.(type) {
case string:
jObj[columnNames[i]] = t
case []byte:
jObj[columnNames[i]] = string(t)
case int, int8, int16, int32, int64, uint, uint8, uint16, uint32, uint64:
jObj[columnNames[i]] = t
case float32, float64:
jObj[columnNames[i]] = t
case bool:
jObj[columnNames[i]] = t
default:
jObj[columnNames[i]] = t
}
}
jArray = append(jArray, jObj)
}
if err := rows.Err(); err != nil {
return err
}
return part.SetJSON(jArray)
}
func strToSQLResultCodec(codec string) (sqlResultCodec, error) {
switch codec {
case "json_array":
return sqlResultJSONArrayCodec, nil
case "none":
return nil, nil
}
return nil, fmt.Errorf("unrecognised result codec: %v", codec)
}
//------------------------------------------------------------------------------
func (s *SQL) doExecute(argSets [][]interface{}) (errs []error) {
var err error
defer func() {
if err != nil {
if len(errs) == 0 {
errs = make([]error, len(argSets))
}
for i := range errs {
if errs[i] == nil {
errs[i] = err
}
}
}
}()
var tx *sql.Tx
if tx, err = s.db.Begin(); err != nil {
return
}
stmt := s.query
if stmt == nil {
if stmt, err = tx.Prepare(s.queryStr); err != nil {
return
}
defer stmt.Close()
} else {
stmt = tx.Stmt(stmt)
}
for i, args := range argSets {
if len(args) == 0 {
continue
}
if _, serr := stmt.Exec(args...); serr != nil {
if len(errs) == 0 {
errs = make([]error, len(argSets))
}
errs[i] = serr
}
}
err = tx.Commit()
return
}
func (s *SQL) getArgs(index int, msg types.Message) ([]interface{}, error) {
if len(s.args) > 0 {
args := make([]interface{}, len(s.args))
for i, v := range s.args {
args[i] = v.String(index, msg)
}
return args, nil
}
if s.argsMapping == nil {
return nil, nil
}
pargs, err := s.argsMapping.MapPart(index, msg)
if err != nil {
return nil, err
}
iargs, err := pargs.JSON()
if err != nil {
return nil, fmt.Errorf("mapping returned non-structured result: %w", err)
}
args, ok := iargs.([]interface{})
if !ok {
return nil, fmt.Errorf("mapping returned non-array result: %T", iargs)
}
return args, nil
}
// ProcessMessage logs an event and returns the message unchanged.
func (s *SQL) ProcessMessage(msg types.Message) ([]types.Message, types.Response) {
s.dbMux.RLock()
defer s.dbMux.RUnlock()
if s.deprecated {
return s.processMessageDeprecated(msg)
}
s.mCount.Incr(1)
newMsg := msg.Copy()
if s.resCodec == nil && s.dynQuery == nil {
argSets := make([][]interface{}, newMsg.Len())
newMsg.Iter(func(index int, p types.Part) error {
args, err := s.getArgs(index, msg)
if err != nil {
s.mErr.Incr(1)
s.log.Errorf("Args mapping error: %v\n", err)
FlagErr(newMsg.Get(index), err)
return nil
}
argSets[index] = args
return nil
})
for i, err := range s.doExecute(argSets) {
if err != nil {
s.mErr.Incr(1)
s.log.Errorf("SQL error: %v\n", err)
FlagErr(newMsg.Get(i), err)
}
}
} else {
IteratePartsWithSpanV2(TypeSQL, nil, newMsg, func(index int, span *tracing.Span, part types.Part) error {
args, err := s.getArgs(index, msg)
if err != nil {
s.mErr.Incr(1)
s.log.Errorf("Args mapping error: %v\n", err)
return err
}
if s.resCodec == nil {
if s.dynQuery != nil {
queryStr := s.dynQuery.String(index, msg)
_, err = s.db.Exec(queryStr, args...)
} else {
_, err = s.query.Exec(args...)
}
if err != nil {
return fmt.Errorf("failed to execute query: %w", err)
}
return nil
}
var rows *sql.Rows
if s.dynQuery != nil {
queryStr := s.dynQuery.String(index, msg)
rows, err = s.db.Query(queryStr, args...)
} else {
rows, err = s.query.Query(args...)
}
if err == nil {
defer rows.Close()
if err = s.resCodec(rows, part); err != nil {
err = fmt.Errorf("failed to apply result codec: %v", err)
}
} else {
err = fmt.Errorf("failed to execute query: %v", err)
}
if err != nil {
s.mErr.Incr(1)
s.log.Errorf("SQL error: %v\n", err)
return err
}
return nil
})
}
s.mBatchSent.Incr(1)
s.mSent.Incr(int64(newMsg.Len()))
msgs := [1]types.Message{newMsg}
return msgs[:], nil
}
// CloseAsync shuts down the processor and stops processing requests.
func (s *SQL) CloseAsync() {
s.closeOnce.Do(func() {
close(s.closeChan)
})
}
// WaitForClose blocks until the processor has closed down.
func (s *SQL) WaitForClose(timeout time.Duration) error {
select {
case <-time.After(timeout):
return types.ErrTimeout
case <-s.closedChan:
}
return nil
}
//------------------------------------------------------------------------------