github.com/cockroachdb/cockroach@v20.2.0-alpha.1+incompatible/pkg/ccl/changefeedccl/encoder.go (about) 1 // Copyright 2018 The Cockroach Authors. 2 // 3 // Licensed as a CockroachDB Enterprise file under the Cockroach Community 4 // License (the "License"); you may not use this file except in compliance with 5 // the License. You may obtain a copy of the License at 6 // 7 // https://github.com/cockroachdb/cockroach/blob/master/licenses/CCL.txt 8 9 package changefeedccl 10 11 import ( 12 "bytes" 13 "context" 14 "encoding/binary" 15 gojson "encoding/json" 16 "io/ioutil" 17 "net/url" 18 "path/filepath" 19 "time" 20 21 "github.com/cockroachdb/cockroach/pkg/base" 22 "github.com/cockroachdb/cockroach/pkg/ccl/changefeedccl/changefeedbase" 23 "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" 24 "github.com/cockroachdb/cockroach/pkg/sql/sqlbase" 25 "github.com/cockroachdb/cockroach/pkg/util/hlc" 26 "github.com/cockroachdb/cockroach/pkg/util/httputil" 27 "github.com/cockroachdb/cockroach/pkg/util/json" 28 "github.com/cockroachdb/cockroach/pkg/util/log" 29 "github.com/cockroachdb/cockroach/pkg/util/retry" 30 "github.com/cockroachdb/errors" 31 ) 32 33 const ( 34 confluentSchemaContentType = `application/vnd.schemaregistry.v1+json` 35 confluentSubjectSuffixKey = `-key` 36 confluentSubjectSuffixValue = `-value` 37 confluentAvroWireFormatMagic = byte(0) 38 ) 39 40 // encodeRow holds all the pieces necessary to encode a row change into a key or 41 // value. 42 type encodeRow struct { 43 // datums is the new value of a changed table row. 44 datums sqlbase.EncDatumRow 45 // updated is the mvcc timestamp corresponding to the latest update in 46 // `datums`. 47 updated hlc.Timestamp 48 // deleted is true if row is a deletion. In this case, only the primary 49 // key columns are guaranteed to be set in `datums`. 50 deleted bool 51 // tableDesc is a TableDescriptor for the table containing `datums`. 52 // It's valid for interpreting the row at `updated`. 53 tableDesc *sqlbase.TableDescriptor 54 // prevDatums is the old value of a changed table row. The field is set 55 // to nil if the before value for changes was not requested (OptDiff). 56 prevDatums sqlbase.EncDatumRow 57 // prevDeleted is true if prevDatums is missing or is a deletion. 58 prevDeleted bool 59 // prevTableDesc is a TableDescriptor for the table containing `prevDatums`. 60 // It's valid for interpreting the row at `updated.Prev()`. 61 prevTableDesc *sqlbase.TableDescriptor 62 } 63 64 // Encoder turns a row into a serialized changefeed key, value, or resolved 65 // timestamp. It represents one of the `format=` changefeed options. 66 type Encoder interface { 67 // EncodeKey encodes the primary key of the given row. The columns of the 68 // datums are expected to match 1:1 with the `Columns` field of the 69 // `TableDescriptor`, but only the primary key fields will be used. The 70 // returned bytes are only valid until the next call to Encode*. 71 EncodeKey(context.Context, encodeRow) ([]byte, error) 72 // EncodeValue encodes the primary key of the given row. The columns of the 73 // datums are expected to match 1:1 with the `Columns` field of the 74 // `TableDescriptor`. The returned bytes are only valid until the next call 75 // to Encode*. 76 EncodeValue(context.Context, encodeRow) ([]byte, error) 77 // EncodeResolvedTimestamp encodes a resolved timestamp payload for the 78 // given topic name. The returned bytes are only valid until the next call 79 // to Encode*. 80 EncodeResolvedTimestamp(context.Context, string, hlc.Timestamp) ([]byte, error) 81 } 82 83 func getEncoder(opts map[string]string) (Encoder, error) { 84 switch changefeedbase.FormatType(opts[changefeedbase.OptFormat]) { 85 case ``, changefeedbase.OptFormatJSON: 86 return makeJSONEncoder(opts) 87 case changefeedbase.OptFormatAvro: 88 return newConfluentAvroEncoder(opts) 89 default: 90 return nil, errors.Errorf(`unknown %s: %s`, changefeedbase.OptFormat, opts[changefeedbase.OptFormat]) 91 } 92 } 93 94 // jsonEncoder encodes changefeed entries as JSON. Keys are the primary key 95 // columns in a JSON array. Values are a JSON object mapping every column name 96 // to its value. Updated timestamps in rows and resolved timestamp payloads are 97 // stored in a sub-object under the `__crdb__` key in the top-level JSON object. 98 type jsonEncoder struct { 99 updatedField, beforeField, wrapped, keyOnly, keyInValue bool 100 101 alloc sqlbase.DatumAlloc 102 buf bytes.Buffer 103 } 104 105 var _ Encoder = &jsonEncoder{} 106 107 func makeJSONEncoder(opts map[string]string) (*jsonEncoder, error) { 108 e := &jsonEncoder{ 109 keyOnly: changefeedbase.EnvelopeType(opts[changefeedbase.OptEnvelope]) == changefeedbase.OptEnvelopeKeyOnly, 110 wrapped: changefeedbase.EnvelopeType(opts[changefeedbase.OptEnvelope]) == changefeedbase.OptEnvelopeWrapped, 111 } 112 _, e.updatedField = opts[changefeedbase.OptUpdatedTimestamps] 113 _, e.beforeField = opts[changefeedbase.OptDiff] 114 if e.beforeField && !e.wrapped { 115 return nil, errors.Errorf(`%s is only usable with %s=%s`, 116 changefeedbase.OptDiff, changefeedbase.OptEnvelope, changefeedbase.OptEnvelopeWrapped) 117 } 118 _, e.keyInValue = opts[changefeedbase.OptKeyInValue] 119 if e.keyInValue && !e.wrapped { 120 return nil, errors.Errorf(`%s is only usable with %s=%s`, 121 changefeedbase.OptKeyInValue, changefeedbase.OptEnvelope, changefeedbase.OptEnvelopeWrapped) 122 } 123 return e, nil 124 } 125 126 // EncodeKey implements the Encoder interface. 127 func (e *jsonEncoder) EncodeKey(_ context.Context, row encodeRow) ([]byte, error) { 128 jsonEntries, err := e.encodeKeyRaw(row) 129 if err != nil { 130 return nil, err 131 } 132 j, err := json.MakeJSON(jsonEntries) 133 if err != nil { 134 return nil, err 135 } 136 e.buf.Reset() 137 j.Format(&e.buf) 138 return e.buf.Bytes(), nil 139 } 140 141 func (e *jsonEncoder) encodeKeyRaw(row encodeRow) ([]interface{}, error) { 142 colIdxByID := row.tableDesc.ColumnIdxMap() 143 jsonEntries := make([]interface{}, len(row.tableDesc.PrimaryIndex.ColumnIDs)) 144 for i, colID := range row.tableDesc.PrimaryIndex.ColumnIDs { 145 idx, ok := colIdxByID[colID] 146 if !ok { 147 return nil, errors.Errorf(`unknown column id: %d`, colID) 148 } 149 datum, col := row.datums[idx], &row.tableDesc.Columns[idx] 150 if err := datum.EnsureDecoded(col.Type, &e.alloc); err != nil { 151 return nil, err 152 } 153 var err error 154 jsonEntries[i], err = tree.AsJSON(datum.Datum, time.UTC) 155 if err != nil { 156 return nil, err 157 } 158 } 159 return jsonEntries, nil 160 } 161 162 // EncodeValue implements the Encoder interface. 163 func (e *jsonEncoder) EncodeValue(_ context.Context, row encodeRow) ([]byte, error) { 164 if e.keyOnly || (!e.wrapped && row.deleted) { 165 return nil, nil 166 } 167 168 var after map[string]interface{} 169 if !row.deleted { 170 columns := row.tableDesc.Columns 171 after = make(map[string]interface{}, len(columns)) 172 for i := range columns { 173 col := &columns[i] 174 datum := row.datums[i] 175 if err := datum.EnsureDecoded(col.Type, &e.alloc); err != nil { 176 return nil, err 177 } 178 var err error 179 after[col.Name], err = tree.AsJSON(datum.Datum, time.UTC) 180 if err != nil { 181 return nil, err 182 } 183 } 184 } 185 186 var before map[string]interface{} 187 if row.prevDatums != nil && !row.prevDeleted { 188 columns := row.prevTableDesc.Columns 189 before = make(map[string]interface{}, len(columns)) 190 for i := range columns { 191 col := &columns[i] 192 datum := row.prevDatums[i] 193 if err := datum.EnsureDecoded(col.Type, &e.alloc); err != nil { 194 return nil, err 195 } 196 var err error 197 before[col.Name], err = tree.AsJSON(datum.Datum, time.UTC) 198 if err != nil { 199 return nil, err 200 } 201 } 202 } 203 204 var jsonEntries map[string]interface{} 205 if e.wrapped { 206 if after != nil { 207 jsonEntries = map[string]interface{}{`after`: after} 208 } else { 209 jsonEntries = map[string]interface{}{`after`: nil} 210 } 211 if e.beforeField { 212 if before != nil { 213 jsonEntries[`before`] = before 214 } else { 215 jsonEntries[`before`] = nil 216 } 217 } 218 if e.keyInValue { 219 keyEntries, err := e.encodeKeyRaw(row) 220 if err != nil { 221 return nil, err 222 } 223 jsonEntries[`key`] = keyEntries 224 } 225 } else { 226 jsonEntries = after 227 } 228 229 if e.updatedField { 230 var meta map[string]interface{} 231 if e.wrapped { 232 meta = jsonEntries 233 } else { 234 meta = make(map[string]interface{}, 1) 235 jsonEntries[jsonMetaSentinel] = meta 236 } 237 meta[`updated`] = row.updated.AsOfSystemTime() 238 } 239 240 j, err := json.MakeJSON(jsonEntries) 241 if err != nil { 242 return nil, err 243 } 244 e.buf.Reset() 245 j.Format(&e.buf) 246 return e.buf.Bytes(), nil 247 } 248 249 // EncodeResolvedTimestamp implements the Encoder interface. 250 func (e *jsonEncoder) EncodeResolvedTimestamp( 251 _ context.Context, _ string, resolved hlc.Timestamp, 252 ) ([]byte, error) { 253 meta := map[string]interface{}{ 254 `resolved`: tree.TimestampToDecimal(resolved).Decimal.String(), 255 } 256 var jsonEntries interface{} 257 if e.wrapped { 258 jsonEntries = meta 259 } else { 260 jsonEntries = map[string]interface{}{ 261 jsonMetaSentinel: meta, 262 } 263 } 264 return gojson.Marshal(jsonEntries) 265 } 266 267 // confluentAvroEncoder encodes changefeed entries as Avro's binary or textual 268 // JSON format. Keys are the primary key columns in a record. Values are all 269 // columns in a record. 270 type confluentAvroEncoder struct { 271 registryURL string 272 updatedField, beforeField, keyOnly bool 273 274 keyCache map[tableIDAndVersion]confluentRegisteredKeySchema 275 valueCache map[tableIDAndVersionPair]confluentRegisteredEnvelopeSchema 276 resolvedCache map[string]confluentRegisteredEnvelopeSchema 277 } 278 279 type tableIDAndVersion uint64 280 type tableIDAndVersionPair [2]tableIDAndVersion // [before, after] 281 282 func makeTableIDAndVersion(id sqlbase.ID, version sqlbase.DescriptorVersion) tableIDAndVersion { 283 return tableIDAndVersion(id)<<32 + tableIDAndVersion(version) 284 } 285 286 type confluentRegisteredKeySchema struct { 287 schema *avroDataRecord 288 registryID int32 289 } 290 291 type confluentRegisteredEnvelopeSchema struct { 292 schema *avroEnvelopeRecord 293 registryID int32 294 } 295 296 var _ Encoder = &confluentAvroEncoder{} 297 298 func newConfluentAvroEncoder(opts map[string]string) (*confluentAvroEncoder, error) { 299 e := &confluentAvroEncoder{registryURL: opts[changefeedbase.OptConfluentSchemaRegistry]} 300 301 switch opts[changefeedbase.OptEnvelope] { 302 case string(changefeedbase.OptEnvelopeKeyOnly): 303 e.keyOnly = true 304 case string(changefeedbase.OptEnvelopeWrapped): 305 default: 306 return nil, errors.Errorf(`%s=%s is not supported with %s=%s`, 307 changefeedbase.OptEnvelope, opts[changefeedbase.OptEnvelope], changefeedbase.OptFormat, changefeedbase.OptFormatAvro) 308 } 309 _, e.updatedField = opts[changefeedbase.OptUpdatedTimestamps] 310 if e.updatedField && e.keyOnly { 311 return nil, errors.Errorf(`%s is only usable with %s=%s`, 312 changefeedbase.OptUpdatedTimestamps, changefeedbase.OptEnvelope, changefeedbase.OptEnvelopeWrapped) 313 } 314 _, e.beforeField = opts[changefeedbase.OptDiff] 315 if e.beforeField && e.keyOnly { 316 return nil, errors.Errorf(`%s is only usable with %s=%s`, 317 changefeedbase.OptDiff, changefeedbase.OptEnvelope, changefeedbase.OptEnvelopeWrapped) 318 } 319 320 if _, ok := opts[changefeedbase.OptKeyInValue]; ok { 321 return nil, errors.Errorf(`%s is not supported with %s=%s`, 322 changefeedbase.OptKeyInValue, changefeedbase.OptFormat, changefeedbase.OptFormatAvro) 323 } 324 325 if len(e.registryURL) == 0 { 326 return nil, errors.Errorf(`WITH option %s is required for %s=%s`, 327 changefeedbase.OptConfluentSchemaRegistry, changefeedbase.OptFormat, changefeedbase.OptFormatAvro) 328 } 329 330 e.keyCache = make(map[tableIDAndVersion]confluentRegisteredKeySchema) 331 e.valueCache = make(map[tableIDAndVersionPair]confluentRegisteredEnvelopeSchema) 332 e.resolvedCache = make(map[string]confluentRegisteredEnvelopeSchema) 333 return e, nil 334 } 335 336 // EncodeKey implements the Encoder interface. 337 func (e *confluentAvroEncoder) EncodeKey(ctx context.Context, row encodeRow) ([]byte, error) { 338 cacheKey := makeTableIDAndVersion(row.tableDesc.ID, row.tableDesc.Version) 339 registered, ok := e.keyCache[cacheKey] 340 if !ok { 341 var err error 342 registered.schema, err = indexToAvroSchema(row.tableDesc, &row.tableDesc.PrimaryIndex) 343 if err != nil { 344 return nil, err 345 } 346 347 // NB: This uses the kafka name escaper because it has to match the name 348 // of the kafka topic. 349 subject := SQLNameToKafkaName(row.tableDesc.Name) + confluentSubjectSuffixKey 350 registered.registryID, err = e.register(ctx, ®istered.schema.avroRecord, subject) 351 if err != nil { 352 return nil, err 353 } 354 // TODO(dan): Bound the size of this cache. 355 e.keyCache[cacheKey] = registered 356 } 357 358 // https://docs.confluent.io/current/schema-registry/docs/serializer-formatter.html#wire-format 359 header := []byte{ 360 confluentAvroWireFormatMagic, 361 0, 0, 0, 0, // Placeholder for the ID. 362 } 363 binary.BigEndian.PutUint32(header[1:5], uint32(registered.registryID)) 364 return registered.schema.BinaryFromRow(header, row.datums) 365 } 366 367 // EncodeValue implements the Encoder interface. 368 func (e *confluentAvroEncoder) EncodeValue(ctx context.Context, row encodeRow) ([]byte, error) { 369 if e.keyOnly { 370 return nil, nil 371 } 372 373 var cacheKey tableIDAndVersionPair 374 if e.beforeField && row.prevTableDesc != nil { 375 cacheKey[0] = makeTableIDAndVersion(row.prevTableDesc.ID, row.prevTableDesc.Version) 376 } 377 cacheKey[1] = makeTableIDAndVersion(row.tableDesc.ID, row.tableDesc.Version) 378 registered, ok := e.valueCache[cacheKey] 379 if !ok { 380 var beforeDataSchema *avroDataRecord 381 if e.beforeField && row.prevTableDesc != nil { 382 var err error 383 beforeDataSchema, err = tableToAvroSchema(row.prevTableDesc, `before`) 384 if err != nil { 385 return nil, err 386 } 387 } 388 389 afterDataSchema, err := tableToAvroSchema(row.tableDesc, avroSchemaNoSuffix) 390 if err != nil { 391 return nil, err 392 } 393 394 opts := avroEnvelopeOpts{afterField: true, beforeField: e.beforeField, updatedField: e.updatedField} 395 registered.schema, err = envelopeToAvroSchema(row.tableDesc.Name, opts, beforeDataSchema, afterDataSchema) 396 if err != nil { 397 return nil, err 398 } 399 400 // NB: This uses the kafka name escaper because it has to match the name 401 // of the kafka topic. 402 subject := SQLNameToKafkaName(row.tableDesc.Name) + confluentSubjectSuffixValue 403 registered.registryID, err = e.register(ctx, ®istered.schema.avroRecord, subject) 404 if err != nil { 405 return nil, err 406 } 407 // TODO(dan): Bound the size of this cache. 408 e.valueCache[cacheKey] = registered 409 } 410 var meta avroMetadata 411 if registered.schema.opts.updatedField { 412 meta = map[string]interface{}{ 413 `updated`: row.updated, 414 } 415 } 416 var beforeDatums, afterDatums sqlbase.EncDatumRow 417 if row.prevDatums != nil && !row.prevDeleted { 418 beforeDatums = row.prevDatums 419 } 420 if !row.deleted { 421 afterDatums = row.datums 422 } 423 // https://docs.confluent.io/current/schema-registry/docs/serializer-formatter.html#wire-format 424 header := []byte{ 425 confluentAvroWireFormatMagic, 426 0, 0, 0, 0, // Placeholder for the ID. 427 } 428 binary.BigEndian.PutUint32(header[1:5], uint32(registered.registryID)) 429 return registered.schema.BinaryFromRow(header, meta, beforeDatums, afterDatums) 430 } 431 432 // EncodeResolvedTimestamp implements the Encoder interface. 433 func (e *confluentAvroEncoder) EncodeResolvedTimestamp( 434 ctx context.Context, topic string, resolved hlc.Timestamp, 435 ) ([]byte, error) { 436 registered, ok := e.resolvedCache[topic] 437 if !ok { 438 opts := avroEnvelopeOpts{resolvedField: true} 439 var err error 440 registered.schema, err = envelopeToAvroSchema(topic, opts, nil /* before */, nil /* after */) 441 if err != nil { 442 return nil, err 443 } 444 445 // NB: This uses the kafka name escaper because it has to match the name 446 // of the kafka topic. 447 subject := SQLNameToKafkaName(topic) + confluentSubjectSuffixValue 448 registered.registryID, err = e.register(ctx, ®istered.schema.avroRecord, subject) 449 if err != nil { 450 return nil, err 451 } 452 // TODO(dan): Bound the size of this cache. 453 e.resolvedCache[topic] = registered 454 } 455 var meta avroMetadata 456 if registered.schema.opts.resolvedField { 457 meta = map[string]interface{}{ 458 `resolved`: resolved, 459 } 460 } 461 // https://docs.confluent.io/current/schema-registry/docs/serializer-formatter.html#wire-format 462 header := []byte{ 463 confluentAvroWireFormatMagic, 464 0, 0, 0, 0, // Placeholder for the ID. 465 } 466 binary.BigEndian.PutUint32(header[1:5], uint32(registered.registryID)) 467 return registered.schema.BinaryFromRow(header, meta, nil /* beforeRow */, nil /* afterRow */) 468 } 469 470 func (e *confluentAvroEncoder) register( 471 ctx context.Context, schema *avroRecord, subject string, 472 ) (int32, error) { 473 type confluentSchemaVersionRequest struct { 474 Schema string `json:"schema"` 475 } 476 type confluentSchemaVersionResponse struct { 477 ID int32 `json:"id"` 478 } 479 480 url, err := url.Parse(e.registryURL) 481 if err != nil { 482 return 0, err 483 } 484 url.Path = filepath.Join(url.EscapedPath(), `subjects`, subject, `versions`) 485 486 schemaStr := schema.codec.Schema() 487 if log.V(1) { 488 log.Infof(ctx, "registering avro schema %s %s", url, schemaStr) 489 } 490 491 req := confluentSchemaVersionRequest{Schema: schemaStr} 492 var buf bytes.Buffer 493 if err := gojson.NewEncoder(&buf).Encode(req); err != nil { 494 return 0, err 495 } 496 497 var id int32 498 499 // Since network services are often a source of flakes, add a few retries here 500 // before we give up and return an error that will bubble up and tear down the 501 // entire changefeed, though that error is marked as retryable so that the job 502 // itself can attempt to start the changefeed again. TODO(dt): If the registry 503 // is down or constantly returning errors, we can't make progress. Continuing 504 // to indicate that we're "running" in this case can be misleading, as we 505 // really aren't anymore. Right now the MO in CDC is try and try again 506 // forever, so doing so here is consistent with the behavior elsewhere, but we 507 // should revisit this more broadly as this pattern can easily mask real, 508 // actionable issues in the operator's environment that which they might be 509 // able to resolve if we made them visible in a failure instead. 510 if err := retry.WithMaxAttempts(ctx, base.DefaultRetryOptions(), 3, func() error { 511 resp, err := httputil.Post(ctx, url.String(), confluentSchemaContentType, &buf) 512 if err != nil { 513 return errors.Wrap(err, "contacting confluent schema registry") 514 } 515 defer resp.Body.Close() 516 if resp.StatusCode < 200 || resp.StatusCode >= 300 { 517 body, _ := ioutil.ReadAll(resp.Body) 518 return errors.Errorf(`registering schema to %s %s: %s`, url.String(), resp.Status, body) 519 } 520 var res confluentSchemaVersionResponse 521 if err := gojson.NewDecoder(resp.Body).Decode(&res); err != nil { 522 return errors.Wrap(err, "decoding confluent schema registry reply") 523 } 524 id = res.ID 525 return nil 526 }); err != nil { 527 log.Warningf(ctx, "%+v", err) 528 return 0, MarkRetryableError(err) 529 } 530 531 return id, nil 532 }