github.com/Jeffail/benthos/v3@v3.65.0/internal/impl/confluent/processor_schema_registry_encode.go (about) 1 package confluent 2 3 import ( 4 "context" 5 "crypto/tls" 6 "encoding/binary" 7 "encoding/json" 8 "errors" 9 "fmt" 10 "io" 11 "net/http" 12 "net/url" 13 "path" 14 "sync" 15 "sync/atomic" 16 "time" 17 18 "github.com/Jeffail/benthos/v3/internal/shutdown" 19 "github.com/Jeffail/benthos/v3/public/service" 20 "github.com/linkedin/goavro/v2" 21 ) 22 23 func schemaRegistryEncoderConfig() *service.ConfigSpec { 24 return service.NewConfigSpec(). 25 // Stable(). TODO 26 Categories("Parsing", "Integration"). 27 Summary("Automatically encodes and validates messages with schemas from a Confluent Schema Registry service."). 28 Description(` 29 Encodes messages automatically from schemas obtains from a [Confluent Schema Registry service](https://docs.confluent.io/platform/current/schema-registry/index.html) by polling the service for the latest schema version for target subjects. 30 31 If a message fails to encode under the schema then it will remain unchanged and the error can be caught using error handling methods outlined [here](/docs/configuration/error_handling). 32 33 Currently only Avro schemas are supported. 34 35 ### Avro JSON Format 36 37 By default this processor expects documents formatted as [Avro JSON](https://avro.apache.org/docs/current/spec.html#json_encoding) when encoding Avro schemas. In this format the value of a union is encoded in JSON as follows: 38 39 - if its type is ` + "`null`, then it is encoded as a JSON `null`" + `; 40 - otherwise it is encoded as a JSON object with one name/value pair whose name is the type's name and whose value is the recursively encoded value. For Avro's named types (record, fixed or enum) the user-specified name is used, for other types the type name is used. 41 42 For example, the union schema ` + "`[\"null\",\"string\",\"Foo\"]`, where `Foo`" + ` is a record name, would encode: 43 44 - ` + "`null` as `null`" + `; 45 - the string ` + "`\"a\"` as `{\"string\": \"a\"}`" + `; and 46 - a ` + "`Foo` instance as `{\"Foo\": {...}}`, where `{...}` indicates the JSON encoding of a `Foo`" + ` instance. 47 48 However, it is possible to instead consume documents in raw JSON format (that match the schema) by setting the field ` + "[`avro_raw_json`](#avro_raw_json) to `true`" + `.`). 49 Field(service.NewStringField("url").Description("The base URL of the schema registry service.")). 50 Field(service.NewInterpolatedStringField("subject").Description("The schema subject to derive schemas from."). 51 Example("foo"). 52 Example(`${! meta("kafka_topic") }`)). 53 Field(service.NewStringField("refresh_period"). 54 Description("The period after which a schema is refreshed for each subject, this is done by polling the schema registry service."). 55 Default("10m"). 56 Example("60s"). 57 Example("1h")). 58 Field(service.NewBoolField("avro_raw_json"). 59 Description("Whether messages encoded in Avro format should be parsed as raw JSON documents rather than [Avro JSON](https://avro.apache.org/docs/current/spec.html#json_encoding)."). 60 Advanced().Default(false).Version("3.59.0")). 61 Field(service.NewTLSField("tls")). 62 Version("3.58.0") 63 } 64 65 func init() { 66 err := service.RegisterBatchProcessor( 67 "schema_registry_encode", schemaRegistryEncoderConfig(), 68 func(conf *service.ParsedConfig, mgr *service.Resources) (service.BatchProcessor, error) { 69 return newSchemaRegistryEncoderFromConfig(conf, mgr.Logger()) 70 }) 71 72 if err != nil { 73 panic(err) 74 } 75 } 76 77 //------------------------------------------------------------------------------ 78 79 type schemaRegistryEncoder struct { 80 client *http.Client 81 subject *service.InterpolatedString 82 avroRawJSON bool 83 schemaRefreshAfter time.Duration 84 85 schemaRegistryBaseURL *url.URL 86 87 schemas map[string]*cachedSchemaEncoder 88 cacheMut sync.RWMutex 89 requestMut sync.Mutex 90 shutSig *shutdown.Signaller 91 92 logger *service.Logger 93 nowFn func() time.Time 94 } 95 96 func newSchemaRegistryEncoderFromConfig(conf *service.ParsedConfig, logger *service.Logger) (*schemaRegistryEncoder, error) { 97 urlStr, err := conf.FieldString("url") 98 if err != nil { 99 return nil, err 100 } 101 subject, err := conf.FieldInterpolatedString("subject") 102 if err != nil { 103 return nil, err 104 } 105 avroRawJSON, err := conf.FieldBool("avro_raw_json") 106 if err != nil { 107 return nil, err 108 } 109 refreshPeriodStr, err := conf.FieldString("refresh_period") 110 if err != nil { 111 return nil, err 112 } 113 refreshPeriod, err := time.ParseDuration(refreshPeriodStr) 114 if err != nil { 115 return nil, fmt.Errorf("failed to parse refresh period: %v", err) 116 } 117 refreshTicker := refreshPeriod / 10 118 if refreshTicker < time.Second { 119 refreshTicker = time.Second 120 } 121 tlsConf, err := conf.FieldTLS("tls") 122 if err != nil { 123 return nil, err 124 } 125 return newSchemaRegistryEncoder(urlStr, tlsConf, subject, avroRawJSON, refreshPeriod, refreshTicker, logger) 126 } 127 128 func newSchemaRegistryEncoder( 129 urlStr string, 130 tlsConf *tls.Config, 131 subject *service.InterpolatedString, 132 avroRawJSON bool, 133 schemaRefreshAfter, schemaRefreshTicker time.Duration, 134 logger *service.Logger, 135 ) (*schemaRegistryEncoder, error) { 136 u, err := url.Parse(urlStr) 137 if err != nil { 138 return nil, fmt.Errorf("failed to parse url: %w", err) 139 } 140 141 s := &schemaRegistryEncoder{ 142 schemaRegistryBaseURL: u, 143 subject: subject, 144 avroRawJSON: avroRawJSON, 145 schemaRefreshAfter: schemaRefreshAfter, 146 schemas: map[string]*cachedSchemaEncoder{}, 147 shutSig: shutdown.NewSignaller(), 148 logger: logger, 149 nowFn: time.Now, 150 } 151 152 s.client = http.DefaultClient 153 if tlsConf != nil { 154 s.client = &http.Client{} 155 if c, ok := http.DefaultTransport.(*http.Transport); ok { 156 cloned := c.Clone() 157 cloned.TLSClientConfig = tlsConf 158 s.client.Transport = cloned 159 } else { 160 s.client.Transport = &http.Transport{ 161 TLSClientConfig: tlsConf, 162 } 163 } 164 } 165 166 go func() { 167 for { 168 select { 169 case <-time.After(schemaRefreshTicker): 170 s.refreshEncoders() 171 case <-s.shutSig.CloseAtLeisureChan(): 172 return 173 } 174 } 175 }() 176 return s, nil 177 } 178 179 func (s *schemaRegistryEncoder) ProcessBatch(ctx context.Context, batch service.MessageBatch) ([]service.MessageBatch, error) { 180 batch = batch.Copy() 181 for i, msg := range batch { 182 encoder, id, err := s.getEncoder(batch.InterpolatedString(i, s.subject)) 183 if err != nil { 184 msg.SetError(err) 185 continue 186 } 187 188 if err := encoder(msg); err != nil { 189 msg.SetError(err) 190 continue 191 } 192 193 rawBytes, err := msg.AsBytes() 194 if err != nil { 195 msg.SetError(errors.New("unable to reference encoded message as bytes")) 196 continue 197 } 198 199 if rawBytes, err = insertID(id, rawBytes); err != nil { 200 msg.SetError(err) 201 continue 202 } 203 msg.SetBytes(rawBytes) 204 } 205 return []service.MessageBatch{batch}, nil 206 } 207 208 func (s *schemaRegistryEncoder) Close(ctx context.Context) error { 209 s.shutSig.CloseNow() 210 s.cacheMut.Lock() 211 defer s.cacheMut.Unlock() 212 if ctx.Err() != nil { 213 return ctx.Err() 214 } 215 for k := range s.schemas { 216 delete(s.schemas, k) 217 } 218 return nil 219 } 220 221 //------------------------------------------------------------------------------ 222 223 type schemaEncoder func(m *service.Message) error 224 225 type cachedSchemaEncoder struct { 226 lastUsedUnixSeconds int64 227 lastUpdatedUnixSeconds int64 228 id int 229 encoder schemaEncoder 230 } 231 232 func insertID(id int, content []byte) ([]byte, error) { 233 newBytes := make([]byte, len(content)+5) 234 235 binary.BigEndian.PutUint32(newBytes[1:], uint32(id)) 236 copy(newBytes[5:], content) 237 238 return newBytes, nil 239 } 240 241 func (s *schemaRegistryEncoder) refreshEncoders() { 242 // First pass in read only mode to gather purge candidates and refresh 243 // candidates 244 s.cacheMut.RLock() 245 purgeTargetTime := s.nowFn().Add(-schemaStaleAfter).Unix() 246 updateTargetTime := s.nowFn().Add(-s.schemaRefreshAfter).Unix() 247 var purgeTargets, refreshTargets []string 248 for k, v := range s.schemas { 249 if atomic.LoadInt64(&v.lastUsedUnixSeconds) < purgeTargetTime { 250 purgeTargets = append(purgeTargets, k) 251 } else if atomic.LoadInt64(&v.lastUpdatedUnixSeconds) < updateTargetTime { 252 refreshTargets = append(refreshTargets, k) 253 } 254 } 255 s.cacheMut.RUnlock() 256 257 // Second pass fully locks schemas and removes stale decoders 258 if len(purgeTargets) > 0 { 259 s.cacheMut.Lock() 260 for _, k := range purgeTargets { 261 if s.schemas[k].lastUsedUnixSeconds < purgeTargetTime { 262 delete(s.schemas, k) 263 } 264 } 265 s.cacheMut.Unlock() 266 } 267 268 // Each refresh target gets updated passively 269 if len(refreshTargets) > 0 { 270 s.requestMut.Lock() 271 for _, k := range refreshTargets { 272 encoder, id, err := s.getLatestEncoder(k) 273 if err != nil { 274 s.logger.Errorf("Failed to refresh schema subject '%v': %v", k, err) 275 } else { 276 s.cacheMut.Lock() 277 s.schemas[k].encoder = encoder 278 s.schemas[k].id = id 279 s.schemas[k].lastUpdatedUnixSeconds = s.nowFn().Unix() 280 s.cacheMut.Unlock() 281 } 282 } 283 s.requestMut.Unlock() 284 } 285 } 286 287 func (s *schemaRegistryEncoder) getLatestEncoder(subject string) (schemaEncoder, int, error) { 288 ctx, done := context.WithTimeout(context.Background(), time.Second*5) 289 defer done() 290 291 reqURL := *s.schemaRegistryBaseURL 292 reqURL.Path = path.Join(reqURL.Path, fmt.Sprintf("/subjects/%s/versions/latest", subject)) 293 294 req, err := http.NewRequestWithContext(ctx, "GET", reqURL.String(), http.NoBody) 295 if err != nil { 296 return nil, 0, err 297 } 298 req.Header.Add("Accept", "application/vnd.schemaregistry.v1+json") 299 300 var resBytes []byte 301 for i := 0; i < 3; i++ { 302 var res *http.Response 303 if res, err = s.client.Do(req); err != nil { 304 s.logger.Errorf("request failed for schema subject '%v': %v", subject, err) 305 continue 306 } 307 308 if res.StatusCode == http.StatusNotFound { 309 err = fmt.Errorf("schema subject '%v' not found by registry", subject) 310 s.logger.Errorf(err.Error()) 311 break 312 } 313 314 if res.StatusCode != http.StatusOK { 315 err = fmt.Errorf("request failed for schema subject '%v'", subject) 316 s.logger.Errorf(err.Error()) 317 // TODO: Best attempt at parsing out the body 318 continue 319 } 320 321 if res.Body == nil { 322 s.logger.Errorf("request for schema subject latest '%v' returned an empty body", subject) 323 err = errors.New("schema request returned an empty body") 324 continue 325 } 326 327 resBytes, err = io.ReadAll(res.Body) 328 res.Body.Close() 329 if err != nil { 330 s.logger.Errorf("failed to read response for schema subject '%v': %v", subject, err) 331 continue 332 } 333 334 break 335 } 336 if err != nil { 337 return nil, 0, err 338 } 339 340 resPayload := struct { 341 Schema string `json:"schema"` 342 ID int `json:"id"` 343 }{} 344 if err = json.Unmarshal(resBytes, &resPayload); err != nil { 345 s.logger.Errorf("failed to parse response for schema subject '%v': %v", subject, err) 346 return nil, 0, err 347 } 348 349 var codec *goavro.Codec 350 if codec, err = goavro.NewCodecForStandardJSON(resPayload.Schema); err != nil { 351 s.logger.Errorf("failed to parse response for schema subject '%v': %v", subject, err) 352 return nil, 0, err 353 } 354 355 return func(m *service.Message) error { 356 var datum interface{} 357 if s.avroRawJSON { 358 b, err := m.AsBytes() 359 if err != nil { 360 return err 361 } 362 363 if datum, _, err = codec.NativeFromTextual(b); err != nil { 364 return err 365 } 366 } else if datum, err = m.AsStructured(); err != nil { 367 return err 368 } 369 370 binary, err := codec.BinaryFromNative(nil, datum) 371 if err != nil { 372 return err 373 } 374 375 m.SetBytes(binary) 376 return nil 377 }, resPayload.ID, nil 378 } 379 380 func (s *schemaRegistryEncoder) getEncoder(subject string) (schemaEncoder, int, error) { 381 s.cacheMut.RLock() 382 c, ok := s.schemas[subject] 383 s.cacheMut.RUnlock() 384 if ok { 385 atomic.StoreInt64(&c.lastUsedUnixSeconds, s.nowFn().Unix()) 386 return c.encoder, c.id, nil 387 } 388 389 s.requestMut.Lock() 390 defer s.requestMut.Unlock() 391 392 // We might've been beaten to making the request, so check once more whilst 393 // within the request lock. 394 s.cacheMut.RLock() 395 c, ok = s.schemas[subject] 396 s.cacheMut.RUnlock() 397 if ok { 398 atomic.StoreInt64(&c.lastUsedUnixSeconds, s.nowFn().Unix()) 399 return c.encoder, c.id, nil 400 } 401 402 encoder, id, err := s.getLatestEncoder(subject) 403 if err != nil { 404 return nil, 0, err 405 } 406 407 s.cacheMut.Lock() 408 s.schemas[subject] = &cachedSchemaEncoder{ 409 lastUsedUnixSeconds: s.nowFn().Unix(), 410 lastUpdatedUnixSeconds: s.nowFn().Unix(), 411 id: id, 412 encoder: encoder, 413 } 414 s.cacheMut.Unlock() 415 416 return encoder, id, nil 417 }