github.com/Jeffail/benthos/v3@v3.65.0/internal/impl/confluent/processor_schema_registry_decode.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 schemaRegistryDecoderConfig() *service.ConfigSpec { 24 return service.NewConfigSpec(). 25 // Stable(). TODO 26 Categories("Parsing", "Integration"). 27 Summary("Automatically decodes and validates messages with schemas from a Confluent Schema Registry service."). 28 Description(` 29 Decodes messages automatically from a schema stored within a [Confluent Schema Registry service](https://docs.confluent.io/platform/current/schema-registry/index.html) by extracting a schema ID from the message and obtaining the associated schema from the registry. If a message fails to match against the schema then it will remain unchanged and the error can be caught using error handling methods outlined [here](/docs/configuration/error_handling). 30 31 Currently only Avro schemas are supported. 32 33 ### Avro JSON Format 34 35 This processor creates documents formatted as [Avro JSON](https://avro.apache.org/docs/current/spec.html#json_encoding) when decoding Avro schemas. In this format the value of a union is encoded in JSON as follows: 36 37 - if its type is ` + "`null`, then it is encoded as a JSON `null`" + `; 38 - 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. 39 40 For example, the union schema ` + "`[\"null\",\"string\",\"Foo\"]`, where `Foo`" + ` is a record name, would encode: 41 42 - ` + "`null` as `null`" + `; 43 - the string ` + "`\"a\"` as `{\"string\": \"a\"}`" + `; and 44 - a ` + "`Foo` instance as `{\"Foo\": {...}}`, where `{...}` indicates the JSON encoding of a `Foo`" + ` instance.`). 45 // Field(service.NewBoolField("avro_raw_json"). 46 // Description("Whether Avro messages should be decoded into raw JSON documents rather than [Avro JSON](https://avro.apache.org/docs/current/spec.html#json_encoding). Avro JSON contains namespaced objects for any typed or non-nil union values, e.g. a union `[\"null\",\"string\"]` field with a string value would be represented as `{\"string\":\"foo\"}`."). 47 // Advanced().Default(false)). 48 Field(service.NewStringField("url").Description("The base URL of the schema registry service.")). 49 Field(service.NewTLSField("tls")) 50 } 51 52 func init() { 53 err := service.RegisterProcessor( 54 "schema_registry_decode", schemaRegistryDecoderConfig(), 55 func(conf *service.ParsedConfig, mgr *service.Resources) (service.Processor, error) { 56 return newSchemaRegistryDecoderFromConfig(conf, mgr.Logger()) 57 }) 58 59 if err != nil { 60 panic(err) 61 } 62 } 63 64 //------------------------------------------------------------------------------ 65 66 type schemaRegistryDecoder struct { 67 client *http.Client 68 avroRawJSON bool 69 70 schemaRegistryBaseURL *url.URL 71 72 schemas map[int]*cachedSchemaDecoder 73 cacheMut sync.RWMutex 74 requestMut sync.Mutex 75 shutSig *shutdown.Signaller 76 77 logger *service.Logger 78 } 79 80 func newSchemaRegistryDecoderFromConfig(conf *service.ParsedConfig, logger *service.Logger) (*schemaRegistryDecoder, error) { 81 urlStr, err := conf.FieldString("url") 82 if err != nil { 83 return nil, err 84 } 85 tlsConf, err := conf.FieldTLS("tls") 86 if err != nil { 87 return nil, err 88 } 89 return newSchemaRegistryDecoder(urlStr, tlsConf, false, logger) 90 } 91 92 func newSchemaRegistryDecoder(urlStr string, tlsConf *tls.Config, avroRawJSON bool, logger *service.Logger) (*schemaRegistryDecoder, error) { 93 u, err := url.Parse(urlStr) 94 if err != nil { 95 return nil, fmt.Errorf("failed to parse url: %w", err) 96 } 97 98 s := &schemaRegistryDecoder{ 99 avroRawJSON: avroRawJSON, 100 schemaRegistryBaseURL: u, 101 schemas: map[int]*cachedSchemaDecoder{}, 102 shutSig: shutdown.NewSignaller(), 103 logger: logger, 104 } 105 106 s.client = http.DefaultClient 107 if tlsConf != nil { 108 s.client = &http.Client{} 109 if c, ok := http.DefaultTransport.(*http.Transport); ok { 110 cloned := c.Clone() 111 cloned.TLSClientConfig = tlsConf 112 s.client.Transport = cloned 113 } else { 114 s.client.Transport = &http.Transport{ 115 TLSClientConfig: tlsConf, 116 } 117 } 118 } 119 120 go func() { 121 for { 122 select { 123 case <-time.After(schemaCachePurgePeriod): 124 s.clearExpired() 125 case <-s.shutSig.CloseAtLeisureChan(): 126 return 127 } 128 } 129 }() 130 return s, nil 131 } 132 133 func (s *schemaRegistryDecoder) Process(ctx context.Context, msg *service.Message) (service.MessageBatch, error) { 134 b, err := msg.AsBytes() 135 if err != nil { 136 return nil, errors.New("unable to reference message as bytes") 137 } 138 139 id, remaining, err := extractID(b) 140 if err != nil { 141 return nil, err 142 } 143 144 decoder, err := s.getDecoder(id) 145 if err != nil { 146 return nil, err 147 } 148 149 newMsg := msg.Copy() 150 newMsg.SetBytes(remaining) 151 if err := decoder(newMsg); err != nil { 152 return nil, err 153 } 154 155 return service.MessageBatch{newMsg}, nil 156 } 157 158 func (s *schemaRegistryDecoder) Close(ctx context.Context) error { 159 s.shutSig.CloseNow() 160 s.cacheMut.Lock() 161 defer s.cacheMut.Unlock() 162 if ctx.Err() != nil { 163 return ctx.Err() 164 } 165 for k := range s.schemas { 166 delete(s.schemas, k) 167 } 168 return nil 169 } 170 171 //------------------------------------------------------------------------------ 172 173 type schemaDecoder func(m *service.Message) error 174 175 type cachedSchemaDecoder struct { 176 lastUsedUnixSeconds int64 177 decoder schemaDecoder 178 } 179 180 func extractID(b []byte) (id int, remaining []byte, err error) { 181 if len(b) == 0 { 182 err = errors.New("message is empty") 183 return 184 } 185 if b[0] != 0 { 186 err = fmt.Errorf("serialization format version number %v not supported", b[0]) 187 return 188 } 189 id = int(binary.BigEndian.Uint32(b[1:5])) 190 remaining = b[5:] 191 return 192 } 193 194 const ( 195 schemaStaleAfter = time.Minute * 10 196 schemaCachePurgePeriod = time.Minute 197 ) 198 199 func (s *schemaRegistryDecoder) clearExpired() { 200 // First pass in read only mode to gather candidates 201 s.cacheMut.RLock() 202 targetTime := time.Now().Add(-schemaStaleAfter).Unix() 203 var targets []int 204 for k, v := range s.schemas { 205 if atomic.LoadInt64(&v.lastUsedUnixSeconds) < targetTime { 206 targets = append(targets, k) 207 } 208 } 209 s.cacheMut.RUnlock() 210 211 // Second pass fully locks schemas and removes stale decoders 212 if len(targets) > 0 { 213 s.cacheMut.Lock() 214 for _, k := range targets { 215 if s.schemas[k].lastUsedUnixSeconds < targetTime { 216 delete(s.schemas, k) 217 } 218 } 219 s.cacheMut.Unlock() 220 } 221 } 222 223 func (s *schemaRegistryDecoder) getDecoder(id int) (schemaDecoder, error) { 224 s.cacheMut.RLock() 225 c, ok := s.schemas[id] 226 s.cacheMut.RUnlock() 227 if ok { 228 atomic.StoreInt64(&c.lastUsedUnixSeconds, time.Now().Unix()) 229 return c.decoder, nil 230 } 231 232 s.requestMut.Lock() 233 defer s.requestMut.Unlock() 234 235 // We might've been beaten to making the request, so check once more whilst 236 // within the request lock. 237 s.cacheMut.RLock() 238 c, ok = s.schemas[id] 239 s.cacheMut.RUnlock() 240 if ok { 241 atomic.StoreInt64(&c.lastUsedUnixSeconds, time.Now().Unix()) 242 return c.decoder, nil 243 } 244 245 ctx, done := context.WithTimeout(context.Background(), time.Second*5) 246 defer done() 247 248 reqURL := *s.schemaRegistryBaseURL 249 reqURL.Path = path.Join(reqURL.Path, fmt.Sprintf("/schemas/ids/%v", id)) 250 251 req, err := http.NewRequestWithContext(ctx, "GET", reqURL.String(), http.NoBody) 252 if err != nil { 253 return nil, err 254 } 255 req.Header.Add("Accept", "application/vnd.schemaregistry.v1+json") 256 257 var resBytes []byte 258 for i := 0; i < 3; i++ { 259 var res *http.Response 260 if res, err = s.client.Do(req); err != nil { 261 s.logger.Errorf("request failed for schema '%v': %v", id, err) 262 continue 263 } 264 265 if res.StatusCode == http.StatusNotFound { 266 err = fmt.Errorf("schema '%v' not found by registry", id) 267 s.logger.Errorf(err.Error()) 268 break 269 } 270 271 if res.StatusCode != http.StatusOK { 272 err = fmt.Errorf("request failed for schema '%v'", id) 273 s.logger.Errorf(err.Error()) 274 // TODO: Best attempt at parsing out the body 275 continue 276 } 277 278 if res.Body == nil { 279 s.logger.Errorf("request for schema '%v' returned an empty body", id) 280 err = errors.New("schema request returned an empty body") 281 continue 282 } 283 284 resBytes, err = io.ReadAll(res.Body) 285 res.Body.Close() 286 if err != nil { 287 s.logger.Errorf("failed to read response for schema '%v': %v", id, err) 288 continue 289 } 290 291 break 292 } 293 if err != nil { 294 return nil, err 295 } 296 297 resPayload := struct { 298 Schema string `json:"schema"` 299 }{} 300 if err = json.Unmarshal(resBytes, &resPayload); err != nil { 301 s.logger.Errorf("failed to parse response for schema '%v': %v", id, err) 302 return nil, err 303 } 304 305 var codec *goavro.Codec 306 if codec, err = goavro.NewCodecForStandardJSON(resPayload.Schema); err != nil { 307 s.logger.Errorf("failed to parse response for schema '%v': %v", id, err) 308 return nil, err 309 } 310 311 decoder := func(m *service.Message) error { 312 b, err := m.AsBytes() 313 if err != nil { 314 return err 315 } 316 317 native, _, err := codec.NativeFromBinary(b) 318 if err != nil { 319 return err 320 } 321 322 if s.avroRawJSON { 323 // TODO: This still encodes with Avro JSON format, needs 324 // investigation as to whether this is possible. 325 jb, err := codec.TextualFromNative(nil, native) 326 if err != nil { 327 return err 328 } 329 m.SetBytes(jb) 330 } else { 331 m.SetStructured(native) 332 } 333 return nil 334 } 335 336 s.cacheMut.Lock() 337 s.schemas[id] = &cachedSchemaDecoder{ 338 lastUsedUnixSeconds: time.Now().Unix(), 339 decoder: decoder, 340 } 341 s.cacheMut.Unlock() 342 343 return decoder, nil 344 }