github.com/pingcap/tiflow@v0.0.0-20240520035814-5bf52d54e205/pkg/sink/codec/avro/confluent_schema_registry.go (about) 1 // Copyright 2020 PingCAP, Inc. 2 // 3 // Licensed under the Apache License, Version 2.0 (the "License"); 4 // you may not use this file except in compliance with the License. 5 // You may obtain a copy of the License at 6 // 7 // http://www.apache.org/licenses/LICENSE-2.0 8 // 9 // Unless required by applicable law or agreed to in writing, software 10 // distributed under the License is distributed on an "AS IS" BASIS, 11 // See the License for the specific language governing permissions and 12 // limitations under the License. 13 14 package avro 15 16 import ( 17 "bytes" 18 "context" 19 "encoding/binary" 20 "encoding/json" 21 "io" 22 "net/http" 23 "net/url" 24 "strconv" 25 "strings" 26 "sync" 27 "time" 28 29 "github.com/cenkalti/backoff/v4" 30 "github.com/linkedin/goavro/v2" 31 "github.com/pingcap/errors" 32 "github.com/pingcap/log" 33 cerror "github.com/pingcap/tiflow/pkg/errors" 34 "github.com/pingcap/tiflow/pkg/httputil" 35 "github.com/pingcap/tiflow/pkg/security" 36 "github.com/pingcap/tiflow/pkg/sink/codec/common" 37 "go.uber.org/zap" 38 ) 39 40 // confluent avro wire format, the first byte is always 0 41 // https://docs.confluent.io/platform/current/schema-registry/fundamentals/serdes-develop/index.html#wire-format 42 const magicByte = uint8(0) 43 44 // confluentSchemaManager is used to register Avro Schemas to the confluent Registry server, 45 // look up local cache according to the table's name, and fetch from the Registry 46 // in cache the local cache entry is missing. 47 type confluentSchemaManager struct { 48 registryURL string 49 50 credential *security.Credential // placeholder, currently always nil 51 52 cacheRWLock sync.RWMutex 53 cache map[string]*schemaCacheEntry 54 registryType string 55 } 56 57 type registerRequest struct { 58 Schema string `json:"schema"` 59 // Commented out for compatibility with Confluent 5.4.x 60 // SchemaType string `json:"schemaType"` 61 } 62 63 type registerResponse struct { 64 SchemaID int `json:"id"` 65 } 66 67 type lookupResponse struct { 68 Name string `json:"name"` 69 SchemaID int `json:"id"` 70 Schema string `json:"schema"` 71 } 72 73 // NewConfluentSchemaManager create schema managers, 74 // and test connectivity to the schema registry 75 func NewConfluentSchemaManager( 76 ctx context.Context, 77 registryURL string, 78 credential *security.Credential, 79 ) (SchemaManager, error) { 80 registryURL = strings.TrimRight(registryURL, "/") 81 httpCli, err := httputil.NewClient(credential) 82 if err != nil { 83 return nil, errors.Trace(err) 84 } 85 resp, err := httpCli.Get(ctx, registryURL) 86 if err != nil { 87 log.Error("Test connection to Schema Registry failed", zap.Error(err)) 88 return nil, cerror.WrapError(cerror.ErrAvroSchemaAPIError, err) 89 } 90 defer resp.Body.Close() 91 92 text, err := io.ReadAll(resp.Body) 93 if err != nil { 94 log.Error("Reading response from Schema Registry failed", zap.Error(err)) 95 return nil, cerror.WrapError(cerror.ErrAvroSchemaAPIError, err) 96 } 97 98 if string(text[:]) != "{}" { 99 log.Error("Unexpected response from Schema Registry", zap.ByteString("response", text)) 100 return nil, cerror.ErrAvroSchemaAPIError.GenWithStack( 101 "Unexpected response from Schema Registry", 102 ) 103 } 104 105 log.Info( 106 "Successfully tested connectivity to Schema Registry", 107 zap.String("registryURL", registryURL), 108 ) 109 110 return &confluentSchemaManager{ 111 registryURL: registryURL, 112 cache: make(map[string]*schemaCacheEntry, 1), 113 registryType: common.SchemaRegistryTypeConfluent, 114 }, nil 115 } 116 117 // Register a schema in schema registry, no cache 118 func (m *confluentSchemaManager) Register( 119 ctx context.Context, 120 schemaName string, 121 schemaDefinition string, 122 ) (schemaID, error) { 123 // The Schema Registry expects the JSON to be without newline characters 124 id := schemaID{} 125 log.Info("confluentSchemaManager", zap.String("schemaDefinition", schemaDefinition), zap.String("schemaName", schemaName)) 126 127 buffer := new(bytes.Buffer) 128 err := json.Compact(buffer, []byte(schemaDefinition)) 129 if err != nil { 130 log.Error("Could not compact schema", zap.Error(err)) 131 return id, cerror.WrapError(cerror.ErrAvroSchemaAPIError, err) 132 } 133 reqBody := registerRequest{ 134 Schema: buffer.String(), 135 } 136 payload, err := json.Marshal(&reqBody) 137 if err != nil { 138 log.Error("Could not marshal request to the Registry", zap.Error(err)) 139 return id, cerror.WrapError(cerror.ErrAvroSchemaAPIError, err) 140 } 141 uri := m.registryURL + "/subjects/" + url.QueryEscape(schemaName) + "/versions" 142 log.Info("Registering schema", zap.String("uri", uri), zap.ByteString("payload", payload)) 143 144 req, err := http.NewRequestWithContext(ctx, "POST", uri, bytes.NewReader(payload)) 145 if err != nil { 146 log.Error("Failed to NewRequestWithContext", zap.Error(err)) 147 return id, cerror.WrapError(cerror.ErrAvroSchemaAPIError, err) 148 } 149 req.Header.Add( 150 "Accept", 151 "application/vnd.schemaregistry.v1+json, application/vnd.schemaregistry+json, "+ 152 "application/json", 153 ) 154 req.Header.Add("Content-Type", "application/vnd.schemaregistry.v1+json") 155 resp, err := httpRetry(ctx, m.credential, req) 156 if err != nil { 157 return id, err 158 } 159 defer resp.Body.Close() 160 161 body, err := io.ReadAll(resp.Body) 162 if err != nil { 163 log.Error("Failed to read response from Registry", zap.Error(err)) 164 return id, cerror.WrapError(cerror.ErrAvroSchemaAPIError, err) 165 } 166 167 if resp.StatusCode != 200 { 168 // https://docs.confluent.io/platform/current/schema-registry/develop/api.html \ 169 // #post--subjects-(string-%20subject)-versions 170 // 409 for incompatible schema 171 log.Error( 172 "Failed to register schema to the Registry, HTTP error", 173 zap.Int("status", resp.StatusCode), 174 zap.String("uri", uri), 175 zap.ByteString("requestBody", payload), 176 zap.ByteString("responseBody", body), 177 ) 178 return id, cerror.ErrAvroSchemaAPIError.GenWithStackByArgs() 179 } 180 181 var jsonResp registerResponse 182 err = json.Unmarshal(body, &jsonResp) 183 184 if err != nil { 185 log.Error("Failed to parse result from Registry", zap.Error(err)) 186 return id, cerror.WrapError(cerror.ErrAvroSchemaAPIError, err) 187 } 188 189 if jsonResp.SchemaID == 0 { 190 return id, cerror.ErrAvroSchemaAPIError.GenWithStack( 191 "Illegal schema ID returned from Registry %d", 192 jsonResp.SchemaID, 193 ) 194 } 195 196 log.Info("Registered schema successfully", 197 zap.Int("schemaID", jsonResp.SchemaID), 198 zap.String("uri", uri), 199 zap.ByteString("body", body)) 200 201 id.confluentSchemaID = jsonResp.SchemaID 202 return id, nil 203 } 204 205 // Lookup the cached schema entry first, if not found, fetch from the Registry server. 206 func (m *confluentSchemaManager) Lookup( 207 ctx context.Context, 208 schemaName string, 209 schemaID schemaID, 210 ) (*goavro.Codec, error) { 211 m.cacheRWLock.RLock() 212 entry, exists := m.cache[schemaName] 213 if exists && entry.schemaID.confluentSchemaID == schemaID.confluentSchemaID { 214 m.cacheRWLock.RUnlock() 215 return entry.codec, nil 216 } 217 m.cacheRWLock.RUnlock() 218 219 uri := m.registryURL + "/schemas/ids/" + strconv.Itoa(schemaID.confluentSchemaID) 220 221 req, err := http.NewRequestWithContext(ctx, "GET", uri, nil) 222 if err != nil { 223 log.Error("Error constructing request for Registry lookup", zap.Error(err)) 224 return nil, cerror.WrapError(cerror.ErrAvroSchemaAPIError, err) 225 } 226 req.Header.Add( 227 "Accept", 228 "application/vnd.schemaregistry.v1+json, application/vnd.schemaregistry+json, "+ 229 "application/json", 230 ) 231 232 resp, err := httpRetry(ctx, m.credential, req) 233 if err != nil { 234 return nil, err 235 } 236 defer resp.Body.Close() 237 238 body, err := io.ReadAll(resp.Body) 239 if err != nil { 240 log.Error("Failed to parse result from Registry", zap.Error(err)) 241 return nil, cerror.WrapError(cerror.ErrAvroSchemaAPIError, err) 242 } 243 244 if resp.StatusCode != 200 && resp.StatusCode != 404 { 245 log.Error("Failed to query schema from the Registry, HTTP error", 246 zap.Int("status", resp.StatusCode), 247 zap.String("uri", uri), 248 zap.ByteString("responseBody", body)) 249 return nil, cerror.ErrAvroSchemaAPIError.GenWithStack( 250 "Failed to query schema from the Registry, HTTP error", 251 ) 252 } 253 254 if resp.StatusCode == 404 { 255 log.Warn("Specified schema not found in Registry", 256 zap.String("key", schemaName), 257 zap.Int("schemaID", schemaID.confluentSchemaID)) 258 return nil, cerror.ErrAvroSchemaAPIError.GenWithStackByArgs( 259 "Schema not found in Registry", 260 ) 261 } 262 263 var jsonResp lookupResponse 264 err = json.Unmarshal(body, &jsonResp) 265 if err != nil { 266 log.Error("Failed to parse result from Registry", zap.Error(err)) 267 return nil, cerror.WrapError(cerror.ErrAvroSchemaAPIError, err) 268 } 269 270 cacheEntry := new(schemaCacheEntry) 271 cacheEntry.codec, err = goavro.NewCodec(jsonResp.Schema) 272 if err != nil { 273 log.Error("Creating Avro codec failed", zap.Error(err)) 274 return nil, cerror.WrapError(cerror.ErrAvroSchemaAPIError, err) 275 } 276 cacheEntry.schemaID.confluentSchemaID = schemaID.confluentSchemaID 277 cacheEntry.header, err = m.getMsgHeader(schemaID.confluentSchemaID) 278 if err != nil { 279 return nil, err 280 } 281 282 m.cacheRWLock.Lock() 283 m.cache[schemaName] = cacheEntry 284 m.cacheRWLock.Unlock() 285 return cacheEntry.codec, nil 286 } 287 288 // GetCachedOrRegister checks if the suitable Avro schema has been cached. 289 // If not, a new schema is generated, registered and cached. 290 // Re-registering an existing schema shall return the same id(and version), so even if the 291 // cache is out-of-sync with schema registry, we could reload it. 292 func (m *confluentSchemaManager) GetCachedOrRegister( 293 ctx context.Context, 294 schemaSubject string, 295 tableVersion uint64, 296 schemaGen SchemaGenerator, 297 ) (*goavro.Codec, []byte, error) { 298 m.cacheRWLock.RLock() 299 if entry, exists := m.cache[schemaSubject]; exists && entry.tableVersion == tableVersion { 300 log.Debug("Avro schema GetCachedOrRegister cache hit", 301 zap.String("key", schemaSubject), 302 zap.Uint64("tableVersion", tableVersion), 303 zap.Int("schemaID", entry.schemaID.confluentSchemaID)) 304 m.cacheRWLock.RUnlock() 305 return entry.codec, entry.header, nil 306 } 307 m.cacheRWLock.RUnlock() 308 309 log.Info("Avro schema lookup cache miss", 310 zap.String("key", schemaSubject), 311 zap.Uint64("tableVersion", tableVersion)) 312 313 schema, err := schemaGen() 314 if err != nil { 315 return nil, nil, err 316 } 317 318 codec, err := goavro.NewCodec(schema) 319 if err != nil { 320 log.Error("GetCachedOrRegister: Could not make goavro codec", zap.Error(err)) 321 return nil, nil, cerror.WrapError(cerror.ErrAvroSchemaAPIError, err) 322 } 323 324 id, err := m.Register(ctx, schemaSubject, schema) 325 if err != nil { 326 log.Error("GetCachedOrRegister: Could not register schema", zap.Error(err)) 327 return nil, nil, errors.Trace(err) 328 } 329 330 cacheEntry := new(schemaCacheEntry) 331 cacheEntry.codec = codec 332 cacheEntry.schemaID = id 333 cacheEntry.tableVersion = tableVersion 334 header, err := m.getMsgHeader(cacheEntry.schemaID.confluentSchemaID) 335 if err != nil { 336 return nil, nil, err 337 } 338 cacheEntry.header = header 339 340 m.cacheRWLock.Lock() 341 m.cache[schemaSubject] = cacheEntry 342 m.cacheRWLock.Unlock() 343 344 log.Info("Avro schema GetCachedOrRegister successful with cache miss", 345 zap.Uint64("tableVersion", cacheEntry.tableVersion), 346 zap.Int("schemaID", cacheEntry.schemaID.confluentSchemaID), 347 zap.String("schema", cacheEntry.codec.Schema())) 348 349 return codec, cacheEntry.header, nil 350 } 351 352 // ClearRegistry clears the Registry subject for the given table. Should be idempotent. 353 // Exported for testing. 354 // NOT USED for now, reserved for future use. 355 func (m *confluentSchemaManager) ClearRegistry(ctx context.Context, schemaSubject string) error { 356 uri := m.registryURL + "/subjects/" + url.QueryEscape(schemaSubject) 357 req, err := http.NewRequestWithContext(ctx, "DELETE", uri, nil) 358 if err != nil { 359 log.Error("Could not construct request for clearRegistry", zap.Error(err)) 360 return cerror.WrapError(cerror.ErrAvroSchemaAPIError, err) 361 } 362 req.Header.Add( 363 "Accept", 364 "application/vnd.schemaregistry.v1+json, application/vnd.schemaregistry+json, "+ 365 "application/json", 366 ) 367 resp, err := httpRetry(ctx, m.credential, req) 368 if err != nil { 369 return err 370 } 371 defer func() { 372 _, _ = io.Copy(io.Discard, resp.Body) 373 _ = resp.Body.Close() 374 }() 375 376 if resp.StatusCode == 200 { 377 log.Info("Clearing Registry successful") 378 return nil 379 } 380 381 if resp.StatusCode == 404 { 382 log.Info("Registry already cleaned") 383 return nil 384 } 385 386 log.Error("Error when clearing Registry", zap.Int("status", resp.StatusCode)) 387 return cerror.ErrAvroSchemaAPIError.GenWithStack( 388 "Error when clearing Registry, status = %d", 389 resp.StatusCode, 390 ) 391 } 392 393 func (m *confluentSchemaManager) RegistryType() string { 394 return m.registryType 395 } 396 397 // confluent avro wire format, confluent avro is not same as apache avro 398 // https://rmoff.net/2020/07/03/why-json-isnt-the-same-as-json-schema-in-kafka-connect-converters \ 399 // -and-ksqldb-viewing-kafka-messages-bytes-as-hex/ 400 func (m *confluentSchemaManager) getMsgHeader(schemaID int) ([]byte, error) { 401 head := new(bytes.Buffer) 402 err := head.WriteByte(magicByte) 403 if err != nil { 404 return nil, cerror.WrapError(cerror.ErrEncodeFailed, err) 405 } 406 err = binary.Write(head, binary.BigEndian, int32(schemaID)) 407 if err != nil { 408 return nil, cerror.WrapError(cerror.ErrEncodeFailed, err) 409 } 410 return head.Bytes(), nil 411 } 412 413 func httpRetry( 414 ctx context.Context, 415 credential *security.Credential, 416 r *http.Request, 417 ) (*http.Response, error) { 418 var ( 419 err error 420 resp *http.Response 421 data []byte 422 ) 423 424 expBackoff := backoff.NewExponentialBackOff() 425 expBackoff.MaxInterval = time.Second * 30 426 httpCli, err := httputil.NewClient(credential) 427 428 if r.Body != nil { 429 data, err = io.ReadAll(r.Body) 430 _ = r.Body.Close() 431 } 432 433 if err != nil { 434 log.Error("Failed to parse response", zap.Error(err)) 435 return nil, cerror.WrapError(cerror.ErrAvroSchemaAPIError, err) 436 } 437 for { 438 if data != nil { 439 r.Body = io.NopCloser(bytes.NewReader(data)) 440 } 441 resp, err = httpCli.Do(r) 442 443 if err != nil { 444 log.Warn("HTTP request failed", zap.String("msg", err.Error())) 445 goto checkCtx 446 } 447 448 // retry 4xx codes like 409 & 422 has no meaning since it's non-recoverable 449 if resp.StatusCode >= 200 && resp.StatusCode < 300 || 450 (resp.StatusCode >= 400 && resp.StatusCode < 500) { 451 break 452 } 453 log.Warn("HTTP server returned with error", zap.Int("status", resp.StatusCode)) 454 _, _ = io.Copy(io.Discard, resp.Body) 455 _ = resp.Body.Close() 456 457 checkCtx: 458 select { 459 case <-ctx.Done(): 460 return nil, errors.New("HTTP retry cancelled") 461 default: 462 } 463 464 time.Sleep(expBackoff.NextBackOff()) 465 } 466 467 return resp, nil 468 } 469 470 func getConfluentSchemaIDFromHeader(header []byte) (uint32, error) { 471 if len(header) < 5 { 472 return 0, cerror.ErrDecodeFailed.GenWithStackByArgs("header too short") 473 } 474 return binary.BigEndian.Uint32(header[1:5]), nil 475 }