github.com/pingcap/tiflow@v0.0.0-20240520035814-5bf52d54e205/pkg/sink/codec/avro/glue_schema_registry.go (about) 1 // Copyright 2023 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 "context" 18 "fmt" 19 "strings" 20 "sync" 21 "time" 22 23 "github.com/aws/aws-sdk-go-v2/aws" 24 awsconfig "github.com/aws/aws-sdk-go-v2/config" 25 "github.com/aws/aws-sdk-go-v2/credentials" 26 "github.com/aws/aws-sdk-go-v2/service/glue" 27 "github.com/aws/aws-sdk-go-v2/service/glue/types" 28 "github.com/google/uuid" 29 "github.com/linkedin/goavro/v2" 30 "github.com/pingcap/errors" 31 "github.com/pingcap/log" 32 "github.com/pingcap/tiflow/pkg/config" 33 cerror "github.com/pingcap/tiflow/pkg/errors" 34 "github.com/pingcap/tiflow/pkg/sink/codec/common" 35 "go.uber.org/zap" 36 ) 37 38 // ---------- Glue schema manager 39 // schemaManager is used to register Avro Schemas to the Registry server, 40 // look up local cache according to the table's name, and fetch from the Registry 41 // in cache the local cache entry is missing. 42 type glueSchemaManager struct { 43 registryName string 44 client glueClient 45 46 cacheRWLock sync.RWMutex 47 cache map[string]*schemaCacheEntry 48 registryType string 49 } 50 51 // NewGlueSchemaManager creates a new schema manager for AWS Glue Schema Registry 52 // It will load the default AWS credentials if no credentials are provided. 53 // It will check if the registry exists, if not, it will return an error. 54 func NewGlueSchemaManager( 55 ctx context.Context, 56 cfg *config.GlueSchemaRegistryConfig, 57 ) (SchemaManager, error) { 58 var awsCfg aws.Config 59 var err error 60 if cfg.NoCredentials() { 61 awsCfg, err = awsconfig.LoadDefaultConfig(ctx) 62 if err != nil { 63 log.Info("LoadDefaultConfig failed", zap.Error(err)) 64 return nil, errors.Trace(err) 65 } 66 } else { 67 awsCfg = *aws.NewConfig() 68 awsCfg.Region = cfg.Region 69 awsCfg.Credentials = credentials. 70 NewStaticCredentialsProvider(cfg.AccessKey, cfg.SecretAccessKey, cfg.Token) 71 } 72 client := glue.NewFromConfig(awsCfg) 73 res := &glueSchemaManager{ 74 registryName: cfg.RegistryName, 75 client: client, 76 cache: make(map[string]*schemaCacheEntry), 77 registryType: common.SchemaRegistryTypeGlue, 78 } 79 ctx, cancel := context.WithTimeout(ctx, 30*time.Second) 80 defer cancel() 81 registry, err := res.client.GetRegistry( 82 ctx, 83 &glue.GetRegistryInput{ 84 RegistryId: &types.RegistryId{ 85 RegistryName: &cfg.RegistryName, 86 }, 87 }, 88 ) 89 if err != nil { 90 return nil, errors.Trace(err) 91 } 92 log.Info("Glue schema registry initialized", zap.Any("registry", registry)) 93 return res, nil 94 } 95 96 // Register a schema into schema registry, no cache 97 func (m *glueSchemaManager) Register( 98 ctx context.Context, 99 schemaName string, 100 schemaDefinition string, 101 ) (schemaID, error) { 102 id := schemaID{} 103 ok, _, err := m.getSchemaByName(ctx, schemaName) 104 if err != nil { 105 return id, errors.Trace(err) 106 } 107 if ok { 108 log.Info("Schema already exists in registry, update it", zap.String("schemaName", schemaName)) 109 schemaID, err := m.updateSchema(ctx, schemaName, schemaDefinition) 110 if err != nil { 111 return id, errors.Trace(err) 112 } 113 log.Info("Schema updated", zap.String("schemaName", schemaName), 114 zap.String("schemaID", schemaID)) 115 id.glueSchemaID = schemaID 116 return id, nil 117 } 118 log.Info("Schema does not exist, create it", zap.String("schemaName", schemaName)) 119 schemaID, err := m.createSchema(ctx, schemaName, schemaDefinition) 120 if err != nil { 121 return id, errors.Trace(err) 122 } 123 id.glueSchemaID = schemaID 124 return id, nil 125 } 126 127 func (m *glueSchemaManager) Lookup( 128 ctx context.Context, 129 schemaName string, 130 schemaID schemaID, 131 ) (*goavro.Codec, error) { 132 m.cacheRWLock.RLock() 133 entry, exists := m.cache[schemaName] 134 if exists && entry.schemaID.confluentSchemaID == schemaID.confluentSchemaID { 135 log.Debug("Avro schema lookup cache hit", 136 zap.String("key", schemaName), 137 zap.Int("schemaID", entry.schemaID.confluentSchemaID)) 138 m.cacheRWLock.RUnlock() 139 return entry.codec, nil 140 } 141 m.cacheRWLock.RUnlock() 142 143 log.Info("Avro schema lookup cache miss", 144 zap.String("key", schemaName), 145 zap.Int("schemaID", schemaID.confluentSchemaID)) 146 147 ok, schema, err := m.getSchemaByID(ctx, schemaID.glueSchemaID) 148 if err != nil { 149 return nil, errors.Trace(err) 150 } 151 if !ok { 152 return nil, cerror.ErrAvroSchemaAPIError. 153 GenWithStackByArgs("schema not found in registry, name: %s, id: %s", schemaName, schemaID.glueSchemaID) 154 } 155 156 codec, err := goavro.NewCodec(schema) 157 if err != nil { 158 log.Error("could not make goavro codec", zap.Error(err)) 159 return nil, cerror.WrapError(cerror.ErrAvroSchemaAPIError, err) 160 } 161 162 header, err := m.getMsgHeader(schemaID.glueSchemaID) 163 if err != nil { 164 log.Error("could not get message header", zap.Error(err)) 165 return nil, cerror.WrapError(cerror.ErrAvroSchemaAPIError, err) 166 } 167 168 m.cacheRWLock.Lock() 169 defer m.cacheRWLock.Unlock() 170 m.cache[schemaName] = &schemaCacheEntry{ 171 schemaID: schemaID, 172 codec: codec, 173 header: header, 174 } 175 176 return codec, nil 177 } 178 179 // GetCachedOrRegister checks if the suitable Avro schema has been cached. 180 // If not, a new schema is generated, registered and cached. 181 // Re-registering an existing schema shall return the same id(and version), so even if the 182 // cache is out-of-sync with schema registry, we could reload it. 183 func (m *glueSchemaManager) GetCachedOrRegister( 184 ctx context.Context, 185 schemaName string, 186 tableVersion uint64, 187 schemaGen SchemaGenerator, 188 ) (*goavro.Codec, []byte, error) { 189 m.cacheRWLock.RLock() 190 if entry, exists := m.cache[schemaName]; exists && entry.tableVersion == tableVersion { 191 log.Debug("Avro schema GetCachedOrRegister cache hit", 192 zap.String("schemaName", schemaName), 193 zap.Uint64("tableVersion", tableVersion), 194 zap.String("schemaID", entry.schemaID.glueSchemaID)) 195 m.cacheRWLock.RUnlock() 196 return entry.codec, entry.header, nil 197 } 198 m.cacheRWLock.RUnlock() 199 200 log.Info("Avro schema lookup cache miss", 201 zap.String("schemaName", schemaName), 202 zap.Uint64("tableVersion", tableVersion)) 203 204 schema, err := schemaGen() 205 if err != nil { 206 return nil, nil, err 207 } 208 209 codec, err := goavro.NewCodec(schema) 210 if err != nil { 211 log.Error("GetCachedOrRegister: Could not make goavro codec", zap.Error(err)) 212 return nil, nil, cerror.WrapError(cerror.ErrAvroSchemaAPIError, err) 213 } 214 215 log.Info(fmt.Sprintf("The code to be registered: %#v", schema)) 216 217 id, err := m.Register(ctx, schemaName, schema) 218 if err != nil { 219 log.Error("GetCachedOrRegister: Could not register schema", zap.Error(err)) 220 return nil, nil, errors.Trace(err) 221 } 222 223 header, err := m.getMsgHeader(id.glueSchemaID) 224 if err != nil { 225 log.Error("GetCachedOrRegister: Could not get message header", zap.Error(err)) 226 return nil, nil, errors.Trace(err) 227 } 228 229 cacheEntry := &schemaCacheEntry{ 230 tableVersion: tableVersion, 231 schemaID: id, 232 codec: codec, 233 header: header, 234 } 235 236 m.cacheRWLock.Lock() 237 m.cache[schemaName] = cacheEntry 238 m.cacheRWLock.Unlock() 239 240 log.Info("Avro schema GetCachedOrRegister successful with cache miss", 241 zap.String("schemaName", schemaName), 242 zap.Uint64("tableVersion", tableVersion), 243 zap.String("schemaID", id.glueSchemaID)) 244 245 return codec, header, nil 246 } 247 248 // ClearRegistry implements SchemaManager, it is not used. 249 func (m *glueSchemaManager) ClearRegistry(ctx context.Context, schemaSubject string) error { 250 return nil 251 } 252 253 func (m *glueSchemaManager) RegistryType() string { 254 return m.registryType 255 } 256 257 func (m *glueSchemaManager) createSchema(ctx context.Context, schemaName, schemaDefinition string) (string, error) { 258 createSchemaInput := &glue.CreateSchemaInput{ 259 RegistryId: &types.RegistryId{ 260 RegistryName: &m.registryName, 261 }, 262 SchemaName: aws.String(schemaName), 263 DataFormat: types.DataFormatAvro, 264 SchemaDefinition: aws.String(schemaDefinition), 265 // cdc don't need to set compatibility check for schema registry 266 // TiDB do the schema compatibility check for us, we need to accept all schema changes 267 // otherwise, we some schema changes will be failed 268 Compatibility: types.CompatibilityNone, 269 } 270 271 output, err := m.client.CreateSchema(ctx, createSchemaInput) 272 if err != nil { 273 return "", errors.Trace(err) 274 } 275 return *output.SchemaVersionId, nil 276 } 277 278 func (m *glueSchemaManager) updateSchema(ctx context.Context, schemaName, schemaDefinition string) (string, error) { 279 input := &glue.RegisterSchemaVersionInput{ 280 SchemaId: &types.SchemaId{ 281 RegistryName: aws.String(m.registryName), 282 SchemaName: &schemaName, 283 }, 284 SchemaDefinition: aws.String(schemaDefinition), 285 } 286 287 resp, err := m.client.RegisterSchemaVersion(ctx, input) 288 if err != nil { 289 return "", errors.Trace(err) 290 } 291 return *resp.SchemaVersionId, nil 292 } 293 294 func (m *glueSchemaManager) getSchemaByName(ctx context.Context, schemaNAme string) (bool, string, error) { 295 input := &glue.GetSchemaVersionInput{ 296 SchemaId: &types.SchemaId{ 297 RegistryName: aws.String(m.registryName), 298 SchemaName: aws.String(schemaNAme), 299 }, 300 SchemaVersionNumber: &types.SchemaVersionNumber{LatestVersion: true}, 301 } 302 result, err := m.client.GetSchemaVersion(ctx, input) 303 if err != nil { 304 if strings.Contains(err.Error(), "EntityNotFoundException") { 305 return false, "", nil 306 } 307 return false, "", errors.Trace(err) 308 } 309 return true, *result.SchemaDefinition, nil 310 } 311 312 func (m *glueSchemaManager) getSchemaByID(ctx context.Context, schemaID string) (bool, string, error) { 313 input := &glue.GetSchemaVersionInput{ 314 SchemaVersionId: aws.String(schemaID), 315 } 316 result, err := m.client.GetSchemaVersion(ctx, input) 317 if err != nil { 318 if strings.Contains(err.Error(), "EntityNotFoundException") { 319 return false, "", nil 320 } 321 return false, "", errors.Trace(err) 322 } 323 return true, *result.SchemaDefinition, nil 324 } 325 326 // This is the header of the glue message, ref: 327 // https://github.com/awslabs/aws-glue-schema-registry/blob/ 328 // master/common/src/main/java/com/amazonaws/services/ 329 // schemaregistry/utils/AWSSchemaRegistryConstants.java 330 const ( 331 headerVersionByte = uint8(3) // 3 is fixed for the glue message 332 compressionDefaultByte = uint8(0) // 0 no compression 333 ) 334 335 func (m *glueSchemaManager) getMsgHeader(schemaID string) ([]byte, error) { 336 header := []byte{} 337 header = append(header, headerVersionByte) 338 header = append(header, compressionDefaultByte) 339 uuid, err := uuid.ParseBytes([]byte(schemaID)) 340 if err != nil { 341 return nil, cerror.WrapError(cerror.ErrEncodeFailed, err) 342 } 343 header = append(header, uuid[:]...) 344 return header, nil 345 } 346 347 func getGlueSchemaIDFromHeader(header []byte) (string, error) { 348 if len(header) < 18 { 349 return "", cerror.ErrDecodeFailed.GenWithStackByArgs("header is too short") 350 } 351 uuid := uuid.UUID(header[2:18]) 352 return uuid.String(), nil 353 }