@@ -16,6 +16,7 @@ import (
16
16
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode"
17
17
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror"
18
18
"github.com/cockroachdb/cockroach/pkg/sql/types"
19
+ "github.com/cockroachdb/cockroach/pkg/util/json"
19
20
"github.com/cockroachdb/errors"
20
21
)
21
22
@@ -117,6 +118,7 @@ const (
117
118
// TODO(#142273): look into whether we want to add headers to pub/sub, and other
118
119
// sinks as well (eg cloudstorage, webhook, ..). Currently it's kafka-only.
119
120
OptHeadersJSONColumnName = `headers_json_column_name`
121
+ OptExtraHeaders = `extra_headers`
120
122
121
123
OptVirtualColumnsOmitted VirtualColumnVisibility = `omitted`
122
124
OptVirtualColumnsNull VirtualColumnVisibility = `null`
@@ -408,6 +410,7 @@ var ChangefeedOptionExpectValues = map[string]OptionPermittedValues{
408
410
OptEncodeJSONValueNullAsObject : flagOption ,
409
411
OptEnrichedProperties : csv (string (EnrichedPropertySource ), string (EnrichedPropertySchema )),
410
412
OptHeadersJSONColumnName : stringOption ,
413
+ OptExtraHeaders : jsonOption ,
411
414
}
412
415
413
416
// CommonOptions is options common to all sinks
@@ -428,13 +431,13 @@ var CommonOptions = makeStringSet(OptCursor, OptEndTime, OptEnvelope,
428
431
var SQLValidOptions map [string ]struct {} = nil
429
432
430
433
// KafkaValidOptions is options exclusive to Kafka sink
431
- var KafkaValidOptions = makeStringSet (OptAvroSchemaPrefix , OptConfluentSchemaRegistry , OptKafkaSinkConfig , OptHeadersJSONColumnName )
434
+ var KafkaValidOptions = makeStringSet (OptAvroSchemaPrefix , OptConfluentSchemaRegistry , OptKafkaSinkConfig , OptHeadersJSONColumnName , OptExtraHeaders )
432
435
433
436
// CloudStorageValidOptions is options exclusive to cloud storage sink
434
437
var CloudStorageValidOptions = makeStringSet (OptCompression )
435
438
436
439
// WebhookValidOptions is options exclusive to webhook sink
437
- var WebhookValidOptions = makeStringSet (OptWebhookAuthHeader , OptWebhookClientTimeout , OptWebhookSinkConfig , OptCompression )
440
+ var WebhookValidOptions = makeStringSet (OptWebhookAuthHeader , OptWebhookClientTimeout , OptWebhookSinkConfig , OptCompression , OptExtraHeaders )
438
441
439
442
// PubsubValidOptions is options exclusive to pubsub sink
440
443
var PubsubValidOptions = makeStringSet (OptPubsubSinkConfig )
@@ -477,6 +480,7 @@ func RedactUserFromURI(uri string) (string, error) {
477
480
// RedactedOptions are options whose values should be replaced with "redacted" in job descriptions and errors.
478
481
var RedactedOptions = map [string ]redactionFunc {
479
482
OptWebhookAuthHeader : redactSimple ,
483
+ OptExtraHeaders : redactSimple ,
480
484
SinkParamClientKey : redactSimple ,
481
485
OptConfluentSchemaRegistry : RedactUserFromURI ,
482
486
}
@@ -1032,6 +1036,7 @@ func (s StatementOptions) GetFilters() Filters {
1032
1036
type WebhookSinkOptions struct {
1033
1037
JSONConfig SinkSpecificJSONConfig
1034
1038
AuthHeader string
1039
+ ExtraHeaders map [string ]string
1035
1040
ClientTimeout * time.Duration
1036
1041
Compression string
1037
1042
}
@@ -1049,13 +1054,66 @@ func (s StatementOptions) GetWebhookSinkOptions() (WebhookSinkOptions, error) {
1049
1054
return o , err
1050
1055
}
1051
1056
o .ClientTimeout = timeout
1057
+
1058
+ headersMap , err := parseHeaders [string ](s .m [OptExtraHeaders ])
1059
+ if err != nil {
1060
+ return o , err
1061
+ }
1062
+ o .ExtraHeaders = headersMap
1052
1063
return o , nil
1053
1064
}
1054
1065
1055
- // GetKafkaConfigJSON returns arbitrary json to be interpreted
1056
- // by the kafka sink.
1057
- func (s StatementOptions ) GetKafkaConfigJSON () SinkSpecificJSONConfig {
1058
- return s .getJSONValue (OptKafkaSinkConfig )
1066
+ func parseHeaders [S interface { string | []byte }](headers string ) (map [string ]S , error ) {
1067
+ if headers == "" {
1068
+ return nil , nil
1069
+ }
1070
+ headersJ , err := json .ParseJSON (headers )
1071
+ if err != nil {
1072
+ return nil , errors .Wrap (err , "parsing headers" )
1073
+ }
1074
+ it , err := headersJ .ObjectIter ()
1075
+ if err != nil {
1076
+ return nil , errors .Wrap (err , "parsing headers as object" )
1077
+ }
1078
+ if it == nil {
1079
+ return nil , errors .Newf ("headers is not a JSON object: %s" , headers )
1080
+ }
1081
+ headersMap := make (map [string ]S , headersJ .Len ())
1082
+ for it .Next () {
1083
+ k := it .Key ()
1084
+ v := it .Value ()
1085
+ s , err := v .AsText ()
1086
+ if err != nil {
1087
+ return nil , errors .Wrap (err , "parsing header value as text" )
1088
+ }
1089
+ if s == nil {
1090
+ continue
1091
+ }
1092
+ headersMap [k ] = S (* s )
1093
+ }
1094
+ return headersMap , nil
1095
+ }
1096
+
1097
+ type KafkaSinkOptions struct {
1098
+ // JSONConfig is arbitrary json to be interpreted
1099
+ // by the kafka sink.
1100
+ JSONConfig SinkSpecificJSONConfig
1101
+
1102
+ // Headers is a map of header names to values.
1103
+ Headers map [string ][]byte
1104
+ }
1105
+
1106
+ func (s StatementOptions ) GetKafkaSinkOptions () (KafkaSinkOptions , error ) {
1107
+ headersMap , err := parseHeaders [[]byte ](s .m [OptExtraHeaders ])
1108
+ if err != nil {
1109
+ return KafkaSinkOptions {}, err
1110
+ }
1111
+
1112
+ o := KafkaSinkOptions {
1113
+ JSONConfig : s .getJSONValue (OptKafkaSinkConfig ),
1114
+ Headers : headersMap ,
1115
+ }
1116
+ return o , nil
1059
1117
}
1060
1118
1061
1119
// GetPubsubConfigJSON returns arbitrary json to be interpreted
0 commit comments