github.com/cockroachdb/cockroach@v20.2.0-alpha.1+incompatible/pkg/ccl/changefeedccl/avro.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 "encoding/json" 13 "fmt" 14 "math/big" 15 "time" 16 17 "github.com/cockroachdb/apd" 18 "github.com/cockroachdb/cockroach/pkg/geo" 19 "github.com/cockroachdb/cockroach/pkg/geo/geopb" 20 "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" 21 "github.com/cockroachdb/cockroach/pkg/sql/sqlbase" 22 "github.com/cockroachdb/cockroach/pkg/sql/types" 23 "github.com/cockroachdb/cockroach/pkg/util/hlc" 24 "github.com/cockroachdb/cockroach/pkg/util/timeofday" 25 "github.com/cockroachdb/errors" 26 "github.com/linkedin/goavro" 27 ) 28 29 // The file contains a very specific marriage between avro and our SQL schemas. 30 // It's not intended to be a general purpose avro utility. 31 // 32 // Avro is a spec for data schemas, a binary format for encoding a record 33 // conforming to a given schema, and various container formats for those encoded 34 // records. It also has rules for determining backward and forward compatibility 35 // of schemas as they evolve. 36 // 37 // The Confluent ecosystem, Kafka plus other things, has first-class support for 38 // Avro, including a server for registering schemas and referencing which 39 // registered schema a Kafka record conforms to. 40 // 41 // We map a SQL table schema to an Avro record with 1:1 mapping between table 42 // columns and Avro fields. The type of the column is mapped to a native Avro 43 // type as faithfully as possible. This is then used to make an "optional" Avro 44 // field for that column by unioning with null and explicitly specifying null as 45 // default, regardless of whether the sql column allows NULLs. This may seem an 46 // odd choice, but it allows for all adjacent Avro schemas for a given SQL table 47 // to be backward and forward compatible with each other. Forward and backward 48 // compatibility drastically eases use of the resulting data by downstream 49 // systems, especially when working with long histories of archived data across 50 // many schema changes (such as a data lake). 51 // 52 // One downside of the above is that it's not possible to recover the original 53 // SQL table schema from an Avro one. (This is also true for other reasons, such 54 // as lossy mappings from sql types to avro types.) To partially address this, 55 // the SQL column type is embedded as metadata in the Avro field schema in a way 56 // that Avro ignores it but passes it along. 57 58 // avroSchemaType is one of the set of avro primitive types. 59 type avroSchemaType interface{} 60 61 const ( 62 avroSchemaBoolean = `boolean` 63 avroSchemaBytes = `bytes` 64 avroSchemaDouble = `double` 65 avroSchemaInt = `int` 66 avroSchemaLong = `long` 67 avroSchemaNull = `null` 68 avroSchemaString = `string` 69 ) 70 71 type avroLogicalType struct { 72 SchemaType avroSchemaType `json:"type"` 73 LogicalType string `json:"logicalType"` 74 Precision int `json:"precision,omitempty"` 75 Scale int `json:"scale,omitempty"` 76 } 77 78 func avroUnionKey(t avroSchemaType) string { 79 switch s := t.(type) { 80 case string: 81 return s 82 case avroLogicalType: 83 return avroUnionKey(s.SchemaType) + `.` + s.LogicalType 84 case *avroRecord: 85 return s.Name 86 default: 87 panic(fmt.Sprintf(`unsupported type %T %v`, t, t)) 88 } 89 } 90 91 // avroSchemaField is our representation of the schema of a field in an avro 92 // record. Serializing it to JSON gives the standard schema representation. 93 type avroSchemaField struct { 94 SchemaType avroSchemaType `json:"type"` 95 Name string `json:"name"` 96 Default *string `json:"default"` 97 Metadata string `json:"__crdb__,omitempty"` 98 99 typ *types.T 100 101 encodeFn func(tree.Datum) (interface{}, error) 102 decodeFn func(interface{}) (tree.Datum, error) 103 } 104 105 // avroRecord is our representation of the schema of an avro record. Serializing 106 // it to JSON gives the standard schema representation. 107 type avroRecord struct { 108 SchemaType string `json:"type"` 109 Name string `json:"name"` 110 Fields []*avroSchemaField `json:"fields"` 111 codec *goavro.Codec 112 } 113 114 // avroDataRecord is an `avroRecord` that represents the schema of a SQL table 115 // or index. 116 type avroDataRecord struct { 117 avroRecord 118 119 colIdxByFieldIdx map[int]int 120 fieldIdxByName map[string]int 121 alloc sqlbase.DatumAlloc 122 } 123 124 // avroMetadata is the `avroEnvelopeRecord` metadata. 125 type avroMetadata map[string]interface{} 126 127 // avroEnvelopeOpts controls which fields in avroEnvelopeRecord are set. 128 type avroEnvelopeOpts struct { 129 beforeField, afterField bool 130 updatedField, resolvedField bool 131 } 132 133 // avroEnvelopeRecord is an `avroRecord` that wraps a changed SQL row and some 134 // metadata. 135 type avroEnvelopeRecord struct { 136 avroRecord 137 138 opts avroEnvelopeOpts 139 before, after *avroDataRecord 140 } 141 142 // columnDescToAvroSchema converts a column descriptor into its corresponding 143 // avro field schema. 144 func columnDescToAvroSchema(colDesc *sqlbase.ColumnDescriptor) (*avroSchemaField, error) { 145 schema := &avroSchemaField{ 146 Name: SQLNameToAvroName(colDesc.Name), 147 Metadata: colDesc.SQLString(), 148 Default: nil, 149 typ: colDesc.Type, 150 } 151 152 var avroType avroSchemaType 153 switch colDesc.Type.Family() { 154 case types.IntFamily: 155 avroType = avroSchemaLong 156 schema.encodeFn = func(d tree.Datum) (interface{}, error) { 157 return int64(*d.(*tree.DInt)), nil 158 } 159 schema.decodeFn = func(x interface{}) (tree.Datum, error) { 160 return tree.NewDInt(tree.DInt(x.(int64))), nil 161 } 162 case types.BoolFamily: 163 avroType = avroSchemaBoolean 164 schema.encodeFn = func(d tree.Datum) (interface{}, error) { 165 return bool(*d.(*tree.DBool)), nil 166 } 167 schema.decodeFn = func(x interface{}) (tree.Datum, error) { 168 return tree.MakeDBool(tree.DBool(x.(bool))), nil 169 } 170 case types.FloatFamily: 171 avroType = avroSchemaDouble 172 schema.encodeFn = func(d tree.Datum) (interface{}, error) { 173 return float64(*d.(*tree.DFloat)), nil 174 } 175 schema.decodeFn = func(x interface{}) (tree.Datum, error) { 176 return tree.NewDFloat(tree.DFloat(x.(float64))), nil 177 } 178 case types.GeographyFamily: 179 avroType = avroSchemaBytes 180 schema.encodeFn = func(d tree.Datum) (interface{}, error) { 181 return []byte(d.(*tree.DGeography).EWKB()), nil 182 } 183 schema.decodeFn = func(x interface{}) (tree.Datum, error) { 184 g, err := geo.ParseGeographyFromEWKBRaw(geopb.EWKB(x.([]byte))) 185 if err != nil { 186 return nil, err 187 } 188 return &tree.DGeography{Geography: g}, nil 189 } 190 case types.GeometryFamily: 191 avroType = avroSchemaBytes 192 schema.encodeFn = func(d tree.Datum) (interface{}, error) { 193 return []byte(d.(*tree.DGeometry).EWKB()), nil 194 } 195 schema.decodeFn = func(x interface{}) (tree.Datum, error) { 196 g, err := geo.ParseGeometryFromEWKBRaw(geopb.EWKB(x.([]byte))) 197 if err != nil { 198 return nil, err 199 } 200 return &tree.DGeometry{Geometry: g}, nil 201 } 202 case types.StringFamily: 203 avroType = avroSchemaString 204 schema.encodeFn = func(d tree.Datum) (interface{}, error) { 205 return string(*d.(*tree.DString)), nil 206 } 207 schema.decodeFn = func(x interface{}) (tree.Datum, error) { 208 return tree.NewDString(x.(string)), nil 209 } 210 case types.BytesFamily: 211 avroType = avroSchemaBytes 212 schema.encodeFn = func(d tree.Datum) (interface{}, error) { 213 return []byte(*d.(*tree.DBytes)), nil 214 } 215 schema.decodeFn = func(x interface{}) (tree.Datum, error) { 216 return tree.NewDBytes(tree.DBytes(x.([]byte))), nil 217 } 218 case types.DateFamily: 219 avroType = avroLogicalType{ 220 SchemaType: avroSchemaInt, 221 LogicalType: `date`, 222 } 223 schema.encodeFn = func(d tree.Datum) (interface{}, error) { 224 date := *d.(*tree.DDate) 225 if !date.IsFinite() { 226 return nil, errors.Errorf( 227 `column %s: infinite date not yet supported with avro`, colDesc.Name) 228 } 229 // The avro library requires us to return this as a time.Time. 230 return date.ToTime() 231 } 232 schema.decodeFn = func(x interface{}) (tree.Datum, error) { 233 // The avro library hands this back as a time.Time. 234 return tree.NewDDateFromTime(x.(time.Time)) 235 } 236 case types.TimeFamily: 237 avroType = avroLogicalType{ 238 SchemaType: avroSchemaLong, 239 LogicalType: `time-micros`, 240 } 241 schema.encodeFn = func(d tree.Datum) (interface{}, error) { 242 // The avro library requires us to return this as a time.Duration. 243 duration := time.Duration(*d.(*tree.DTime)) * time.Microsecond 244 return duration, nil 245 } 246 schema.decodeFn = func(x interface{}) (tree.Datum, error) { 247 // The avro library hands this back as a time.Duration. 248 micros := x.(time.Duration) / time.Microsecond 249 return tree.MakeDTime(timeofday.TimeOfDay(micros)), nil 250 } 251 case types.TimeTZFamily: 252 avroType = avroSchemaString 253 // We cannot encode this as a long, as it does not encode 254 // timezone correctly. 255 schema.encodeFn = func(d tree.Datum) (interface{}, error) { 256 return d.(*tree.DTimeTZ).TimeTZ.String(), nil 257 } 258 schema.decodeFn = func(x interface{}) (tree.Datum, error) { 259 return tree.ParseDTimeTZ(nil, x.(string), time.Microsecond) 260 } 261 case types.TimestampFamily: 262 avroType = avroLogicalType{ 263 SchemaType: avroSchemaLong, 264 LogicalType: `timestamp-micros`, 265 } 266 schema.encodeFn = func(d tree.Datum) (interface{}, error) { 267 return d.(*tree.DTimestamp).Time, nil 268 } 269 schema.decodeFn = func(x interface{}) (tree.Datum, error) { 270 return tree.MakeDTimestamp(x.(time.Time), time.Microsecond) 271 } 272 case types.TimestampTZFamily: 273 avroType = avroLogicalType{ 274 SchemaType: avroSchemaLong, 275 LogicalType: `timestamp-micros`, 276 } 277 schema.encodeFn = func(d tree.Datum) (interface{}, error) { 278 return d.(*tree.DTimestampTZ).Time, nil 279 } 280 schema.decodeFn = func(x interface{}) (tree.Datum, error) { 281 return tree.MakeDTimestampTZ(x.(time.Time), time.Microsecond) 282 } 283 case types.DecimalFamily: 284 if colDesc.Type.Precision() == 0 { 285 return nil, errors.Errorf( 286 `column %s: decimal with no precision not yet supported with avro`, colDesc.Name) 287 } 288 avroType = avroLogicalType{ 289 SchemaType: avroSchemaBytes, 290 LogicalType: `decimal`, 291 Precision: int(colDesc.Type.Precision()), 292 Scale: int(colDesc.Type.Width()), 293 } 294 schema.encodeFn = func(d tree.Datum) (interface{}, error) { 295 dec := d.(*tree.DDecimal).Decimal 296 // TODO(dan): For the cases that the avro defined decimal format 297 // would not roundtrip, serialize the decimal as a string. Also 298 // support the unspecified precision/scale case in this branch. We 299 // can't currently do this without surgery to the avro library we're 300 // using and that's too scary leading up to 2.1.0. 301 rat, err := decimalToRat(dec, colDesc.Type.Width()) 302 if err != nil { 303 return nil, err 304 } 305 return &rat, nil 306 } 307 schema.decodeFn = func(x interface{}) (tree.Datum, error) { 308 return &tree.DDecimal{Decimal: ratToDecimal(*x.(*big.Rat), colDesc.Type.Width())}, nil 309 } 310 case types.UuidFamily: 311 // Should be logical type of "uuid", but the avro library doesn't support 312 // that yet. 313 avroType = avroSchemaString 314 schema.encodeFn = func(d tree.Datum) (interface{}, error) { 315 return d.(*tree.DUuid).UUID.String(), nil 316 } 317 schema.decodeFn = func(x interface{}) (tree.Datum, error) { 318 return tree.ParseDUuidFromString(x.(string)) 319 } 320 case types.INetFamily: 321 avroType = avroSchemaString 322 schema.encodeFn = func(d tree.Datum) (interface{}, error) { 323 return d.(*tree.DIPAddr).IPAddr.String(), nil 324 } 325 schema.decodeFn = func(x interface{}) (tree.Datum, error) { 326 return tree.ParseDIPAddrFromINetString(x.(string)) 327 } 328 case types.JsonFamily: 329 avroType = avroSchemaString 330 schema.encodeFn = func(d tree.Datum) (interface{}, error) { 331 return d.(*tree.DJSON).JSON.String(), nil 332 } 333 schema.decodeFn = func(x interface{}) (tree.Datum, error) { 334 return tree.ParseDJSON(x.(string)) 335 } 336 default: 337 return nil, errors.Errorf(`column %s: type %s not yet supported with avro`, 338 colDesc.Name, colDesc.Type.SQLString()) 339 } 340 schema.SchemaType = avroType 341 342 // Make every field optional by unioning it with null, so that all schema 343 // evolutions for a table are considered "backward compatible" by avro. This 344 // means that the Avro type doesn't mirror the column's nullability, but it 345 // makes it much easier to work with long histories of table data afterward, 346 // especially for things like loading into analytics databases. 347 { 348 // The default for a union type is the default for the first element of 349 // the union. 350 schema.SchemaType = []avroSchemaType{avroSchemaNull, avroType} 351 encodeFn := schema.encodeFn 352 decodeFn := schema.decodeFn 353 unionKey := avroUnionKey(avroType) 354 schema.encodeFn = func(d tree.Datum) (interface{}, error) { 355 if d == tree.DNull { 356 return goavro.Union(avroSchemaNull, nil), nil 357 } 358 encoded, err := encodeFn(d) 359 if err != nil { 360 return nil, err 361 } 362 return goavro.Union(unionKey, encoded), nil 363 } 364 schema.decodeFn = func(x interface{}) (tree.Datum, error) { 365 if x == nil { 366 return tree.DNull, nil 367 } 368 return decodeFn(x.(map[string]interface{})[unionKey]) 369 } 370 } 371 372 return schema, nil 373 } 374 375 // indexToAvroSchema converts a column descriptor into its corresponding avro 376 // record schema. The fields are kept in the same order as columns in the index. 377 func indexToAvroSchema( 378 tableDesc *sqlbase.TableDescriptor, indexDesc *sqlbase.IndexDescriptor, 379 ) (*avroDataRecord, error) { 380 schema := &avroDataRecord{ 381 avroRecord: avroRecord{ 382 Name: SQLNameToAvroName(tableDesc.Name), 383 SchemaType: `record`, 384 }, 385 fieldIdxByName: make(map[string]int), 386 colIdxByFieldIdx: make(map[int]int), 387 } 388 colIdxByID := tableDesc.ColumnIdxMap() 389 for _, colID := range indexDesc.ColumnIDs { 390 colIdx, ok := colIdxByID[colID] 391 if !ok { 392 return nil, errors.Errorf(`unknown column id: %d`, colID) 393 } 394 col := &tableDesc.Columns[colIdx] 395 field, err := columnDescToAvroSchema(col) 396 if err != nil { 397 return nil, err 398 } 399 schema.colIdxByFieldIdx[len(schema.Fields)] = colIdx 400 schema.fieldIdxByName[field.Name] = len(schema.Fields) 401 schema.Fields = append(schema.Fields, field) 402 } 403 schemaJSON, err := json.Marshal(schema) 404 if err != nil { 405 return nil, err 406 } 407 schema.codec, err = goavro.NewCodec(string(schemaJSON)) 408 if err != nil { 409 return nil, err 410 } 411 return schema, nil 412 } 413 414 const ( 415 // avroSchemaNoSuffix can be passed to tableToAvroSchema to indicate that 416 // no suffix should be appended to the avro record's name. 417 avroSchemaNoSuffix = `` 418 ) 419 420 // tableToAvroSchema converts a column descriptor into its corresponding avro 421 // record schema. The fields are kept in the same order as `tableDesc.Columns`. 422 // If a name suffix is provided (as opposed to avroSchemaNoSuffix), it will be 423 // appended to the end of the avro record's name. 424 func tableToAvroSchema( 425 tableDesc *sqlbase.TableDescriptor, nameSuffix string, 426 ) (*avroDataRecord, error) { 427 name := SQLNameToAvroName(tableDesc.Name) 428 if nameSuffix != avroSchemaNoSuffix { 429 name = name + `_` + nameSuffix 430 } 431 schema := &avroDataRecord{ 432 avroRecord: avroRecord{ 433 Name: name, 434 SchemaType: `record`, 435 }, 436 fieldIdxByName: make(map[string]int), 437 colIdxByFieldIdx: make(map[int]int), 438 } 439 for colIdx := range tableDesc.Columns { 440 col := &tableDesc.Columns[colIdx] 441 field, err := columnDescToAvroSchema(col) 442 if err != nil { 443 return nil, err 444 } 445 schema.colIdxByFieldIdx[len(schema.Fields)] = colIdx 446 schema.fieldIdxByName[field.Name] = len(schema.Fields) 447 schema.Fields = append(schema.Fields, field) 448 } 449 schemaJSON, err := json.Marshal(schema) 450 if err != nil { 451 return nil, err 452 } 453 schema.codec, err = goavro.NewCodec(string(schemaJSON)) 454 if err != nil { 455 return nil, err 456 } 457 return schema, nil 458 } 459 460 // textualFromRow encodes the given row data into avro's defined JSON format. 461 func (r *avroDataRecord) textualFromRow(row sqlbase.EncDatumRow) ([]byte, error) { 462 native, err := r.nativeFromRow(row) 463 if err != nil { 464 return nil, err 465 } 466 return r.codec.TextualFromNative(nil /* buf */, native) 467 } 468 469 // BinaryFromRow encodes the given row data into avro's defined binary format. 470 func (r *avroDataRecord) BinaryFromRow(buf []byte, row sqlbase.EncDatumRow) ([]byte, error) { 471 native, err := r.nativeFromRow(row) 472 if err != nil { 473 return nil, err 474 } 475 return r.codec.BinaryFromNative(buf, native) 476 } 477 478 // rowFromTextual decodes the given row data from avro's defined JSON format. 479 func (r *avroDataRecord) rowFromTextual(buf []byte) (sqlbase.EncDatumRow, error) { 480 native, newBuf, err := r.codec.NativeFromTextual(buf) 481 if err != nil { 482 return nil, err 483 } 484 if len(newBuf) > 0 { 485 return nil, errors.New(`only one row was expected`) 486 } 487 return r.rowFromNative(native) 488 } 489 490 // RowFromBinary decodes the given row data from avro's defined binary format. 491 func (r *avroDataRecord) RowFromBinary(buf []byte) (sqlbase.EncDatumRow, error) { 492 native, newBuf, err := r.codec.NativeFromBinary(buf) 493 if err != nil { 494 return nil, err 495 } 496 if len(newBuf) > 0 { 497 return nil, errors.New(`only one row was expected`) 498 } 499 return r.rowFromNative(native) 500 } 501 502 func (r *avroDataRecord) nativeFromRow(row sqlbase.EncDatumRow) (interface{}, error) { 503 avroDatums := make(map[string]interface{}, len(row)) 504 for fieldIdx, field := range r.Fields { 505 d := row[r.colIdxByFieldIdx[fieldIdx]] 506 if err := d.EnsureDecoded(field.typ, &r.alloc); err != nil { 507 return nil, err 508 } 509 var err error 510 if avroDatums[field.Name], err = field.encodeFn(d.Datum); err != nil { 511 return nil, err 512 } 513 } 514 return avroDatums, nil 515 } 516 517 func (r *avroDataRecord) rowFromNative(native interface{}) (sqlbase.EncDatumRow, error) { 518 avroDatums, ok := native.(map[string]interface{}) 519 if !ok { 520 return nil, errors.Errorf(`unknown avro native type: %T`, native) 521 } 522 if len(r.Fields) != len(avroDatums) { 523 return nil, errors.Errorf( 524 `expected row with %d columns got %d`, len(r.Fields), len(avroDatums)) 525 } 526 row := make(sqlbase.EncDatumRow, len(r.Fields)) 527 for fieldName, avroDatum := range avroDatums { 528 fieldIdx := r.fieldIdxByName[fieldName] 529 field := r.Fields[fieldIdx] 530 decoded, err := field.decodeFn(avroDatum) 531 if err != nil { 532 return nil, err 533 } 534 row[r.colIdxByFieldIdx[fieldIdx]] = sqlbase.DatumToEncDatum(field.typ, decoded) 535 } 536 return row, nil 537 } 538 539 // envelopeToAvroSchema creates an avro record schema for an envelope containing 540 // before and after versions of a row change and metadata about that row change. 541 func envelopeToAvroSchema( 542 topic string, opts avroEnvelopeOpts, before, after *avroDataRecord, 543 ) (*avroEnvelopeRecord, error) { 544 schema := &avroEnvelopeRecord{ 545 avroRecord: avroRecord{ 546 Name: SQLNameToAvroName(topic) + `_envelope`, 547 SchemaType: `record`, 548 }, 549 opts: opts, 550 } 551 552 if opts.beforeField { 553 schema.before = before 554 beforeField := &avroSchemaField{ 555 Name: `before`, 556 SchemaType: []avroSchemaType{avroSchemaNull, before}, 557 Default: nil, 558 } 559 schema.Fields = append(schema.Fields, beforeField) 560 } 561 if opts.afterField { 562 schema.after = after 563 afterField := &avroSchemaField{ 564 Name: `after`, 565 SchemaType: []avroSchemaType{avroSchemaNull, after}, 566 Default: nil, 567 } 568 schema.Fields = append(schema.Fields, afterField) 569 } 570 if opts.updatedField { 571 updatedField := &avroSchemaField{ 572 SchemaType: []avroSchemaType{avroSchemaNull, avroSchemaString}, 573 Name: `updated`, 574 Default: nil, 575 } 576 schema.Fields = append(schema.Fields, updatedField) 577 } 578 if opts.resolvedField { 579 resolvedField := &avroSchemaField{ 580 SchemaType: []avroSchemaType{avroSchemaNull, avroSchemaString}, 581 Name: `resolved`, 582 Default: nil, 583 } 584 schema.Fields = append(schema.Fields, resolvedField) 585 } 586 587 schemaJSON, err := json.Marshal(schema) 588 if err != nil { 589 return nil, err 590 } 591 schema.codec, err = goavro.NewCodec(string(schemaJSON)) 592 if err != nil { 593 return nil, err 594 } 595 return schema, nil 596 } 597 598 // BinaryFromRow encodes the given metadata and row data into avro's defined 599 // binary format. 600 func (r *avroEnvelopeRecord) BinaryFromRow( 601 buf []byte, meta avroMetadata, beforeRow, afterRow sqlbase.EncDatumRow, 602 ) ([]byte, error) { 603 native := map[string]interface{}{} 604 if r.opts.beforeField { 605 if beforeRow == nil { 606 native[`before`] = nil 607 } else { 608 beforeNative, err := r.before.nativeFromRow(beforeRow) 609 if err != nil { 610 return nil, err 611 } 612 native[`before`] = goavro.Union(avroUnionKey(&r.before.avroRecord), beforeNative) 613 } 614 } 615 if r.opts.afterField { 616 if afterRow == nil { 617 native[`after`] = nil 618 } else { 619 afterNative, err := r.after.nativeFromRow(afterRow) 620 if err != nil { 621 return nil, err 622 } 623 native[`after`] = goavro.Union(avroUnionKey(&r.after.avroRecord), afterNative) 624 } 625 } 626 if r.opts.updatedField { 627 native[`updated`] = nil 628 if u, ok := meta[`updated`]; ok { 629 delete(meta, `updated`) 630 ts, ok := u.(hlc.Timestamp) 631 if !ok { 632 return nil, errors.Errorf(`unknown metadata timestamp type: %T`, u) 633 } 634 native[`updated`] = goavro.Union(avroUnionKey(avroSchemaString), ts.AsOfSystemTime()) 635 } 636 } 637 if r.opts.resolvedField { 638 native[`resolved`] = nil 639 if u, ok := meta[`resolved`]; ok { 640 delete(meta, `resolved`) 641 ts, ok := u.(hlc.Timestamp) 642 if !ok { 643 return nil, errors.Errorf(`unknown metadata timestamp type: %T`, u) 644 } 645 native[`resolved`] = goavro.Union(avroUnionKey(avroSchemaString), ts.AsOfSystemTime()) 646 } 647 } 648 for k := range meta { 649 return nil, errors.AssertionFailedf(`unhandled meta key: %s`, k) 650 } 651 return r.codec.BinaryFromNative(buf, native) 652 } 653 654 // decimalToRat converts one of our apd decimals to the format expected by the 655 // avro library we use. If the column has a fixed scale (which is always true if 656 // precision is set) this is roundtripable without information loss. 657 func decimalToRat(dec apd.Decimal, scale int32) (big.Rat, error) { 658 if dec.Form != apd.Finite { 659 return big.Rat{}, errors.Errorf(`cannot convert %s form decimal`, dec.Form) 660 } 661 if scale > 0 && scale != -dec.Exponent { 662 return big.Rat{}, errors.Errorf(`%s will not roundtrip at scale %d`, &dec, scale) 663 } 664 var r big.Rat 665 if dec.Exponent >= 0 { 666 exp := big.NewInt(10) 667 exp = exp.Exp(exp, big.NewInt(int64(dec.Exponent)), nil) 668 var coeff big.Int 669 r.SetFrac(coeff.Mul(&dec.Coeff, exp), big.NewInt(1)) 670 } else { 671 exp := big.NewInt(10) 672 exp = exp.Exp(exp, big.NewInt(int64(-dec.Exponent)), nil) 673 r.SetFrac(&dec.Coeff, exp) 674 } 675 if dec.Negative { 676 r.Mul(&r, big.NewRat(-1, 1)) 677 } 678 return r, nil 679 } 680 681 // ratToDecimal converts the output of decimalToRat back into the original apd 682 // decimal, given a fixed column scale. NB: big.Rat is lossy-compared to apd 683 // decimal, so this is not possible when the scale is not fixed. 684 func ratToDecimal(rat big.Rat, scale int32) apd.Decimal { 685 num, denom := rat.Num(), rat.Denom() 686 exp := big.NewInt(10) 687 exp = exp.Exp(exp, big.NewInt(int64(scale)), nil) 688 sf := denom.Div(exp, denom) 689 coeff := num.Mul(num, sf) 690 dec := apd.NewWithBigInt(coeff, -scale) 691 return *dec 692 }