github.com/cockroachdb/cockroach@v20.2.0-alpha.1+incompatible/pkg/ccl/changefeedccl/cdctest/validator.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 cdctest 10 11 import ( 12 "bytes" 13 gosql "database/sql" 14 gojson "encoding/json" 15 "fmt" 16 "sort" 17 "strings" 18 19 "github.com/cockroachdb/cockroach/pkg/sql" 20 "github.com/cockroachdb/cockroach/pkg/util/hlc" 21 "github.com/cockroachdb/errors" 22 ) 23 24 // Validator checks for violations of our changefeed ordering and delivery 25 // guarantees in a single table. 26 type Validator interface { 27 // NoteRow accepts a changed row entry. 28 NoteRow(partition string, key, value string, updated hlc.Timestamp) error 29 // NoteResolved accepts a resolved timestamp entry. 30 NoteResolved(partition string, resolved hlc.Timestamp) error 31 // Failures returns any violations seen so far. 32 Failures() []string 33 } 34 35 type orderValidator struct { 36 topic string 37 partitionForKey map[string]string 38 keyTimestamps map[string][]hlc.Timestamp 39 resolved map[string]hlc.Timestamp 40 41 failures []string 42 } 43 44 // NewOrderValidator returns a Validator that checks the row and resolved 45 // timestamp ordering guarantees. It also asserts that keys have an affinity to 46 // a single partition. 47 // 48 // Once a row with has been emitted with some timestamp, no previously unseen 49 // versions of that row will be emitted with a lower timestamp. 50 // 51 // Once a resolved timestamp has been emitted, no previously unseen rows with a 52 // lower update timestamp will be emitted on that partition. 53 func NewOrderValidator(topic string) Validator { 54 return &orderValidator{ 55 topic: topic, 56 partitionForKey: make(map[string]string), 57 keyTimestamps: make(map[string][]hlc.Timestamp), 58 resolved: make(map[string]hlc.Timestamp), 59 } 60 } 61 62 // NoteRow implements the Validator interface. 63 func (v *orderValidator) NoteRow( 64 partition string, key, ignoredValue string, updated hlc.Timestamp, 65 ) error { 66 if prev, ok := v.partitionForKey[key]; ok && prev != partition { 67 v.failures = append(v.failures, fmt.Sprintf( 68 `key [%s] received on two partitions: %s and %s`, key, prev, partition, 69 )) 70 return nil 71 } 72 v.partitionForKey[key] = partition 73 74 timestamps := v.keyTimestamps[key] 75 timestampsIdx := sort.Search(len(timestamps), func(i int) bool { 76 return updated.LessEq(timestamps[i]) 77 }) 78 seen := timestampsIdx < len(timestamps) && timestamps[timestampsIdx] == updated 79 80 if !seen && len(timestamps) > 0 && updated.Less(timestamps[len(timestamps)-1]) { 81 v.failures = append(v.failures, fmt.Sprintf( 82 `topic %s partition %s: saw new row timestamp %s after %s was seen`, 83 v.topic, partition, 84 updated.AsOfSystemTime(), timestamps[len(timestamps)-1].AsOfSystemTime(), 85 )) 86 } 87 if !seen && updated.Less(v.resolved[partition]) { 88 v.failures = append(v.failures, fmt.Sprintf( 89 `topic %s partition %s: saw new row timestamp %s after %s was resolved`, 90 v.topic, partition, updated.AsOfSystemTime(), v.resolved[partition].AsOfSystemTime(), 91 )) 92 } 93 94 if !seen { 95 v.keyTimestamps[key] = append( 96 append(timestamps[:timestampsIdx], updated), timestamps[timestampsIdx:]...) 97 } 98 return nil 99 } 100 101 // NoteResolved implements the Validator interface. 102 func (v *orderValidator) NoteResolved(partition string, resolved hlc.Timestamp) error { 103 prev := v.resolved[partition] 104 if prev.Less(resolved) { 105 v.resolved[partition] = resolved 106 } 107 return nil 108 } 109 110 // Failures implements the Validator interface. 111 func (v *orderValidator) Failures() []string { 112 return v.failures 113 } 114 115 type beforeAfterValidator struct { 116 sqlDB *gosql.DB 117 table string 118 primaryKeyCols []string 119 resolved map[string]hlc.Timestamp 120 121 failures []string 122 } 123 124 // NewBeforeAfterValidator returns a Validator verifies that the "before" and 125 // "after" fields in each row agree with the source table when performing AS OF 126 // SYSTEM TIME lookups before and at the row's timestamp. 127 func NewBeforeAfterValidator(sqlDB *gosql.DB, table string) (Validator, error) { 128 primaryKeyCols, err := fetchPrimaryKeyCols(sqlDB, table) 129 if err != nil { 130 return nil, errors.Wrap(err, "fetchPrimaryKeyCols failed") 131 } 132 133 return &beforeAfterValidator{ 134 sqlDB: sqlDB, 135 table: table, 136 primaryKeyCols: primaryKeyCols, 137 resolved: make(map[string]hlc.Timestamp), 138 }, nil 139 } 140 141 // NoteRow implements the Validator interface. 142 func (v *beforeAfterValidator) NoteRow( 143 partition string, key, value string, updated hlc.Timestamp, 144 ) error { 145 var primaryKeyDatums []interface{} 146 if err := gojson.Unmarshal([]byte(key), &primaryKeyDatums); err != nil { 147 return err 148 } 149 if len(primaryKeyDatums) != len(v.primaryKeyCols) { 150 return errors.Errorf( 151 `expected primary key columns %s got datums %s`, v.primaryKeyCols, primaryKeyDatums) 152 } 153 154 type wrapper struct { 155 After map[string]interface{} `json:"after"` 156 Before map[string]interface{} `json:"before"` 157 } 158 var rowJSON wrapper 159 if err := gojson.Unmarshal([]byte(value), &rowJSON); err != nil { 160 return err 161 } 162 163 // Check that the "after" field agrees with the row in the table at the 164 // updated timestamp. 165 if err := v.checkRowAt("after", primaryKeyDatums, rowJSON.After, updated); err != nil { 166 return err 167 } 168 169 if v.resolved[partition].IsEmpty() && rowJSON.Before == nil { 170 // If the initial scan hasn't completed for this partition, 171 // we don't require the rows to contain a "before" field. 172 return nil 173 } 174 175 // Check that the "before" field agrees with the row in the table immediately 176 // before the updated timestamp. 177 return v.checkRowAt("before", primaryKeyDatums, rowJSON.Before, updated.Prev()) 178 } 179 180 func (v *beforeAfterValidator) checkRowAt( 181 field string, primaryKeyDatums []interface{}, rowDatums map[string]interface{}, ts hlc.Timestamp, 182 ) error { 183 var stmtBuf bytes.Buffer 184 var args []interface{} 185 if rowDatums == nil { 186 // We expect the row to be missing ... 187 stmtBuf.WriteString(`SELECT count(*) = 0 `) 188 } else { 189 // We expect the row to be present ... 190 stmtBuf.WriteString(`SELECT count(*) = 1 `) 191 } 192 fmt.Fprintf(&stmtBuf, `FROM %s AS OF SYSTEM TIME '%s' WHERE `, v.table, ts.AsOfSystemTime()) 193 if rowDatums == nil { 194 // ... with the primary key. 195 for i, datum := range primaryKeyDatums { 196 if len(args) != 0 { 197 stmtBuf.WriteString(` AND `) 198 } 199 fmt.Fprintf(&stmtBuf, `%s = $%d`, v.primaryKeyCols[i], i+1) 200 args = append(args, datum) 201 } 202 } else { 203 // ... and match the specified datums. 204 colNames := make([]string, 0, len(rowDatums)) 205 for col := range rowDatums { 206 colNames = append(colNames, col) 207 } 208 sort.Strings(colNames) 209 for i, col := range colNames { 210 if len(args) != 0 { 211 stmtBuf.WriteString(` AND `) 212 } 213 fmt.Fprintf(&stmtBuf, `%s = $%d`, col, i+1) 214 args = append(args, rowDatums[col]) 215 } 216 } 217 218 var valid bool 219 stmt := stmtBuf.String() 220 if err := v.sqlDB.QueryRow(stmt, args...).Scan(&valid); err != nil { 221 return errors.Wrapf(err, "while executing %s", stmt) 222 } 223 if !valid { 224 v.failures = append(v.failures, fmt.Sprintf( 225 "%q field did not agree with row at %s: %s %v", 226 field, ts.AsOfSystemTime(), stmt, args)) 227 } 228 return nil 229 } 230 231 // NoteResolved implements the Validator interface. 232 func (v *beforeAfterValidator) NoteResolved(partition string, resolved hlc.Timestamp) error { 233 prev := v.resolved[partition] 234 if prev.Less(resolved) { 235 v.resolved[partition] = resolved 236 } 237 return nil 238 } 239 240 // Failures implements the Validator interface. 241 func (v *beforeAfterValidator) Failures() []string { 242 return v.failures 243 } 244 245 type validatorRow struct { 246 key, value string 247 updated hlc.Timestamp 248 } 249 250 // fingerprintValidator verifies that recreating a table from its changefeed 251 // will fingerprint the same at all "interesting" points in time. 252 type fingerprintValidator struct { 253 sqlDB *gosql.DB 254 origTable, fprintTable string 255 primaryKeyCols []string 256 partitionResolved map[string]hlc.Timestamp 257 resolved hlc.Timestamp 258 // It's possible to get a resolved timestamp from before the table even 259 // exists, which is valid but complicates the way fingerprintValidator works. 260 // Don't create a fingerprint earlier than the first seen row. 261 firstRowTimestamp hlc.Timestamp 262 // previousRowUpdateTs keeps track of the timestamp of the most recently processed row 263 // update. Before starting to process row updates belonging to a particular timestamp 264 // X, we want to fingerprint at `X.Prev()` to catch any "missed" row updates. 265 // Maintaining `previousRowUpdateTs` allows us to do this. See `NoteResolved()` for 266 // more details. 267 previousRowUpdateTs hlc.Timestamp 268 269 // `fprintOrigColumns` keeps track of the number of non test columns in `fprint`. 270 fprintOrigColumns int 271 fprintTestColumns int 272 buffer []validatorRow 273 274 failures []string 275 } 276 277 // NewFingerprintValidator returns a new FingerprintValidator that uses `fprintTable` as 278 // scratch space to recreate `origTable`. `fprintTable` must exist before calling this 279 // constructor. `maxTestColumnCount` indicates the maximum number of columns that can be 280 // expected in `origTable` due to test-related schema changes. This fingerprint validator 281 // will modify `fprint`'s schema to add `maxTestColumnCount` columns to avoid having to 282 // accommodate schema changes on the fly. 283 func NewFingerprintValidator( 284 sqlDB *gosql.DB, origTable, fprintTable string, partitions []string, maxTestColumnCount int, 285 ) (Validator, error) { 286 // Fetch the primary keys though information_schema schema inspections so we 287 // can use them to construct the SQL for DELETEs and also so we can verify 288 // that the key in a message matches what's expected for the value. 289 primaryKeyCols, err := fetchPrimaryKeyCols(sqlDB, fprintTable) 290 if err != nil { 291 return nil, err 292 } 293 294 // Record the non-test%d columns in `fprint`. 295 var fprintOrigColumns int 296 if err := sqlDB.QueryRow(` 297 SELECT count(column_name) 298 FROM information_schema.columns 299 WHERE table_name=$1 300 `, fprintTable).Scan(&fprintOrigColumns); err != nil { 301 return nil, err 302 } 303 304 // Add test columns to fprint. 305 if maxTestColumnCount > 0 { 306 var addColumnStmt bytes.Buffer 307 addColumnStmt.WriteString(`ALTER TABLE fprint `) 308 for i := 0; i < maxTestColumnCount; i++ { 309 if i != 0 { 310 addColumnStmt.WriteString(`, `) 311 } 312 fmt.Fprintf(&addColumnStmt, `ADD COLUMN test%d STRING`, i) 313 } 314 if _, err := sqlDB.Query(addColumnStmt.String()); err != nil { 315 return nil, err 316 } 317 } 318 319 v := &fingerprintValidator{ 320 sqlDB: sqlDB, 321 origTable: origTable, 322 fprintTable: fprintTable, 323 primaryKeyCols: primaryKeyCols, 324 fprintOrigColumns: fprintOrigColumns, 325 fprintTestColumns: maxTestColumnCount, 326 } 327 v.partitionResolved = make(map[string]hlc.Timestamp) 328 for _, partition := range partitions { 329 v.partitionResolved[partition] = hlc.Timestamp{} 330 } 331 return v, nil 332 } 333 334 // NoteRow implements the Validator interface. 335 func (v *fingerprintValidator) NoteRow( 336 ignoredPartition string, key, value string, updated hlc.Timestamp, 337 ) error { 338 if v.firstRowTimestamp.IsEmpty() || updated.Less(v.firstRowTimestamp) { 339 v.firstRowTimestamp = updated 340 } 341 v.buffer = append(v.buffer, validatorRow{ 342 key: key, 343 value: value, 344 updated: updated, 345 }) 346 return nil 347 } 348 349 // applyRowUpdate applies the update represented by `row` to the scratch table. 350 func (v *fingerprintValidator) applyRowUpdate(row validatorRow) (_err error) { 351 defer func() { 352 _err = errors.Wrap(_err, "fingerprintValidator failed") 353 }() 354 355 var args []interface{} 356 var primaryKeyDatums []interface{} 357 if err := gojson.Unmarshal([]byte(row.key), &primaryKeyDatums); err != nil { 358 return err 359 } 360 if len(primaryKeyDatums) != len(v.primaryKeyCols) { 361 return errors.Errorf(`expected primary key columns %s got datums %s`, 362 v.primaryKeyCols, primaryKeyDatums) 363 } 364 365 var stmtBuf bytes.Buffer 366 type wrapper struct { 367 After map[string]interface{} `json:"after"` 368 } 369 var value wrapper 370 if err := gojson.Unmarshal([]byte(row.value), &value); err != nil { 371 return err 372 } 373 if value.After != nil { 374 // UPDATE or INSERT 375 fmt.Fprintf(&stmtBuf, `UPSERT INTO %s (`, v.fprintTable) 376 for col, colValue := range value.After { 377 if len(args) != 0 { 378 stmtBuf.WriteString(`,`) 379 } 380 stmtBuf.WriteString(col) 381 args = append(args, colValue) 382 } 383 for i := len(value.After) - v.fprintOrigColumns; i < v.fprintTestColumns; i++ { 384 fmt.Fprintf(&stmtBuf, `, test%d`, i) 385 args = append(args, nil) 386 } 387 stmtBuf.WriteString(`) VALUES (`) 388 for i := range args { 389 if i != 0 { 390 stmtBuf.WriteString(`,`) 391 } 392 fmt.Fprintf(&stmtBuf, `$%d`, i+1) 393 } 394 stmtBuf.WriteString(`)`) 395 396 // Also verify that the key matches the value. 397 primaryKeyDatums = make([]interface{}, len(v.primaryKeyCols)) 398 for idx, primaryKeyCol := range v.primaryKeyCols { 399 primaryKeyDatums[idx] = value.After[primaryKeyCol] 400 } 401 primaryKeyJSON, err := gojson.Marshal(primaryKeyDatums) 402 if err != nil { 403 return err 404 } 405 406 if string(primaryKeyJSON) != row.key { 407 v.failures = append(v.failures, 408 fmt.Sprintf(`key %s did not match expected key %s for value %s`, 409 row.key, primaryKeyJSON, row.value)) 410 } 411 } else { 412 // DELETE 413 fmt.Fprintf(&stmtBuf, `DELETE FROM %s WHERE `, v.fprintTable) 414 for i, datum := range primaryKeyDatums { 415 if len(args) != 0 { 416 stmtBuf.WriteString(` AND `) 417 } 418 fmt.Fprintf(&stmtBuf, `%s = $%d`, v.primaryKeyCols[i], i+1) 419 args = append(args, datum) 420 } 421 } 422 _, err := v.sqlDB.Exec(stmtBuf.String(), args...) 423 return err 424 } 425 426 // NoteResolved implements the Validator interface. 427 func (v *fingerprintValidator) NoteResolved(partition string, resolved hlc.Timestamp) error { 428 if r, ok := v.partitionResolved[partition]; !ok { 429 return errors.Errorf(`unknown partition: %s`, partition) 430 } else if resolved.LessEq(r) { 431 return nil 432 } 433 v.partitionResolved[partition] = resolved 434 435 // Check if this partition's resolved timestamp advancing has advanced the 436 // overall topic resolved timestamp. This is O(n^2) but could be better with 437 // a heap, if necessary. 438 newResolved := resolved 439 for _, r := range v.partitionResolved { 440 if r.Less(newResolved) { 441 newResolved = r 442 } 443 } 444 if newResolved.LessEq(v.resolved) { 445 return nil 446 } 447 v.resolved = newResolved 448 449 // NB: Intentionally not stable sort because it shouldn't matter. 450 sort.Slice(v.buffer, func(i, j int) bool { 451 return v.buffer[i].updated.Less(v.buffer[j].updated) 452 }) 453 454 var lastFingerprintedAt hlc.Timestamp 455 // We apply all the row updates we received in the time window between the last 456 // resolved timestamp and this one. We process all row updates belonging to a given 457 // timestamp and then `fingerprint` to ensure the scratch table and the original table 458 // match. 459 for len(v.buffer) > 0 { 460 if v.resolved.Less(v.buffer[0].updated) { 461 break 462 } 463 row := v.buffer[0] 464 v.buffer = v.buffer[1:] 465 466 // If we've processed all row updates belonging to the previous row's timestamp, 467 // we fingerprint at `updated.Prev()` since we want to catch cases where one or 468 // more row updates are missed. For example: If k1 was written at t1, t2, t3 and 469 // the update for t2 was missed. 470 if v.previousRowUpdateTs != (hlc.Timestamp{}) && v.previousRowUpdateTs.Less(row.updated) { 471 if err := v.fingerprint(row.updated.Prev()); err != nil { 472 return err 473 } 474 } 475 if err := v.applyRowUpdate(row); err != nil { 476 return err 477 } 478 479 // If any updates have exactly the same timestamp, we have to apply them all 480 // before fingerprinting. 481 if len(v.buffer) == 0 || v.buffer[0].updated != row.updated { 482 lastFingerprintedAt = row.updated 483 if err := v.fingerprint(row.updated); err != nil { 484 return err 485 } 486 } 487 v.previousRowUpdateTs = row.updated 488 } 489 490 if !v.firstRowTimestamp.IsEmpty() && v.firstRowTimestamp.LessEq(resolved) && 491 lastFingerprintedAt != resolved { 492 return v.fingerprint(resolved) 493 } 494 return nil 495 } 496 497 func (v *fingerprintValidator) fingerprint(ts hlc.Timestamp) error { 498 var orig string 499 if err := v.sqlDB.QueryRow(`SELECT IFNULL(fingerprint, 'EMPTY') FROM [ 500 SHOW EXPERIMENTAL_FINGERPRINTS FROM TABLE ` + v.origTable + ` 501 ] AS OF SYSTEM TIME '` + ts.AsOfSystemTime() + `'`).Scan(&orig); err != nil { 502 return err 503 } 504 var check string 505 if err := v.sqlDB.QueryRow(`SELECT IFNULL(fingerprint, 'EMPTY') FROM [ 506 SHOW EXPERIMENTAL_FINGERPRINTS FROM TABLE ` + v.fprintTable + ` 507 ]`).Scan(&check); err != nil { 508 return err 509 } 510 if orig != check { 511 v.failures = append(v.failures, fmt.Sprintf( 512 `fingerprints did not match at %s: %s vs %s`, ts.AsOfSystemTime(), orig, check)) 513 } 514 return nil 515 } 516 517 // Failures implements the Validator interface. 518 func (v *fingerprintValidator) Failures() []string { 519 return v.failures 520 } 521 522 // Validators abstracts over running multiple `Validator`s at once on the same 523 // feed. 524 type Validators []Validator 525 526 // NoteRow implements the Validator interface. 527 func (vs Validators) NoteRow(partition string, key, value string, updated hlc.Timestamp) error { 528 for _, v := range vs { 529 if err := v.NoteRow(partition, key, value, updated); err != nil { 530 return err 531 } 532 } 533 return nil 534 } 535 536 // NoteResolved implements the Validator interface. 537 func (vs Validators) NoteResolved(partition string, resolved hlc.Timestamp) error { 538 for _, v := range vs { 539 if err := v.NoteResolved(partition, resolved); err != nil { 540 return err 541 } 542 } 543 return nil 544 } 545 546 // Failures implements the Validator interface. 547 func (vs Validators) Failures() []string { 548 var f []string 549 for _, v := range vs { 550 f = append(f, v.Failures()...) 551 } 552 return f 553 } 554 555 // CountValidator wraps a Validator and keeps count of how many rows and 556 // resolved timestamps have been seen. 557 type CountValidator struct { 558 v Validator 559 560 NumRows, NumResolved int 561 NumResolvedRows, NumResolvedWithRows int 562 rowsSinceResolved int 563 } 564 565 // MakeCountValidator returns a CountValidator wrapping the given Validator. 566 func MakeCountValidator(v Validator) *CountValidator { 567 return &CountValidator{v: v} 568 } 569 570 // NoteRow implements the Validator interface. 571 func (v *CountValidator) NoteRow(partition string, key, value string, updated hlc.Timestamp) error { 572 v.NumRows++ 573 v.rowsSinceResolved++ 574 return v.v.NoteRow(partition, key, value, updated) 575 } 576 577 // NoteResolved implements the Validator interface. 578 func (v *CountValidator) NoteResolved(partition string, resolved hlc.Timestamp) error { 579 v.NumResolved++ 580 if v.rowsSinceResolved > 0 { 581 v.NumResolvedWithRows++ 582 v.NumResolvedRows += v.rowsSinceResolved 583 v.rowsSinceResolved = 0 584 } 585 return v.v.NoteResolved(partition, resolved) 586 } 587 588 // Failures implements the Validator interface. 589 func (v *CountValidator) Failures() []string { 590 return v.v.Failures() 591 } 592 593 // ParseJSONValueTimestamps returns the updated or resolved timestamp set in the 594 // provided `format=json` value. Exported for acceptance testing. 595 func ParseJSONValueTimestamps(v []byte) (updated, resolved hlc.Timestamp, err error) { 596 var valueRaw struct { 597 Resolved string `json:"resolved"` 598 Updated string `json:"updated"` 599 } 600 if err := gojson.Unmarshal(v, &valueRaw); err != nil { 601 return hlc.Timestamp{}, hlc.Timestamp{}, errors.Wrapf(err, "parsing [%s] as json", v) 602 } 603 if valueRaw.Updated != `` { 604 var err error 605 updated, err = sql.ParseHLC(valueRaw.Updated) 606 if err != nil { 607 return hlc.Timestamp{}, hlc.Timestamp{}, err 608 } 609 } 610 if valueRaw.Resolved != `` { 611 var err error 612 resolved, err = sql.ParseHLC(valueRaw.Resolved) 613 if err != nil { 614 return hlc.Timestamp{}, hlc.Timestamp{}, err 615 } 616 } 617 return updated, resolved, nil 618 } 619 620 // fetchPrimaryKeyCols fetches the names of the primary key columns for the 621 // specified table. 622 func fetchPrimaryKeyCols(sqlDB *gosql.DB, tableStr string) ([]string, error) { 623 parts := strings.Split(tableStr, ".") 624 var db, table string 625 switch len(parts) { 626 case 1: 627 table = parts[0] 628 case 2: 629 db = parts[0] + "." 630 table = parts[1] 631 default: 632 return nil, errors.Errorf("could not parse table %s", parts) 633 } 634 rows, err := sqlDB.Query(fmt.Sprintf(` 635 SELECT column_name 636 FROM %sinformation_schema.key_column_usage 637 WHERE table_name=$1 638 AND constraint_name='primary' 639 ORDER BY ordinal_position`, db), 640 table, 641 ) 642 if err != nil { 643 return nil, err 644 } 645 defer func() { _ = rows.Close() }() 646 var primaryKeyCols []string 647 for rows.Next() { 648 var primaryKeyCol string 649 if err := rows.Scan(&primaryKeyCol); err != nil { 650 return nil, err 651 } 652 primaryKeyCols = append(primaryKeyCols, primaryKeyCol) 653 } 654 if len(primaryKeyCols) == 0 { 655 return nil, errors.Errorf("no primary key information found for %s", tableStr) 656 } 657 return primaryKeyCols, nil 658 }