cockroachdb / cockroach

CockroachDB — the cloud native, distributed SQL database designed for high availability, effortless scale, and control over data placement.
https://www.cockroachlabs.com
Other
29.9k stars 3.78k forks source link

roachtest: backup-restore/online-restore failed [cluster backup missing sequence] #130778

Open cockroach-teamcity opened 3 days ago

cockroach-teamcity commented 3 days ago

roachtest.backup-restore/online-restore failed with artifacts on master @ 128fcab4c07413513a05aea1d1494943f4bc3092:

(monitor.go:154).Wait: monitor failure: backup 3_round-trip-test-backup_cluster: error in restore statement: pq: table descriptor rewrite failed: rewriting declarative schema changer state: rewriting descriptor ids: missing rewrite for id 255 in SequenceOwner:{DescID: 210, ColumnID: 5, ReferencedDescID: 255}
test artifacts and logs in: /artifacts/backup-restore/online-restore/run_1

Parameters:

See: roachtest README

See: How To Investigate (internal)

See: Grafana

Same failure on other branches

- #129187 roachtest: backup-restore/online-restore failed [relation "[239]" does not exist during schemachange workload] [C-test-failure O-roachtest O-robot P-2 T-sql-foundations branch-release-24.2]

/cc @cockroachdb/disaster-recovery

This test on roachdash | Improve this report!

Jira issue: CRDB-42230

msbutler commented 1 day ago

Looks like a full cluster restore failed to restore a full backup with an endtime of 2024-09-16 06:02:27.24754

2024/09/16 06:08:36 test_impl.go:427: test failure #1: full stack retained in failure_1.log: (monitor.go:154).Wait: monitor failure: backup 3_round-trip-test-backup_cluster: error in restore statement: pq: table descriptor rewrite failed:   rewriting declarative schema changer state: rewriting descriptor ids: missing rewrite for id 255 in SequenceOwner:{DescID: 210, ColumnID: 5, ReferencedDescID: 255}
msbutler commented 1 day ago

Reproduced this failure with regular restore. we've got a bug in our descriptor rewrite rules somewere. let's figure out what kind of descriptor we're seeking to restore.

RESTORE DATABASE schemachange FROM LATEST IN 'gs://cockroachdb-backup-testing-long-ttl/mixed-version/3_round-trip-test-backup_cluster_pGao?AUTH=implicit';
ERROR: table descriptor rewrite failed: rewriting declarative schema changer state: rewriting descriptor ids: missing rewrite for id 255 in SequenceOwner:{DescID: 210, ColumnID: 5, ReferencedDescID: 255}
msbutler commented 1 day ago

We can dump all the descriptor ids in the backup via:

roachprod sql $CLUSTER:1 -- -e "SHOW BACKUP FROM LATEST IN 'gs://cockroachdb-backup-testing-long-ttl/mixed-version/3_round-trip-test-backup_cluster_pGao?AUTH=implicit' with debug_ids;"

which reveals:

I see we added a sequence to this table in schema change logs:

{
 "workerId": 3,
 "clientTimestamp": "06:02:22.234199",
 "ops": [
  "BEGIN",
  {
   "sql": "CREATE SEQUENCE IF NOT EXISTS public.seq_w3_309 OWNED BY schema_w1_70.table_w0_59.col59_w0_64"
  },
  "COMMIT"
 ],
 "expectedExecErrors": "",
 "expectedCommitErrors": "",
 "message": ""
}

Looks like the owning column is indeed the 5th column in this table: col59_w0_64

   "sql": "CREATE TABLE schema_w1_70.table_w0_59 (col5͝9_w0_60 FLOAT8 NOT NULL, col59_w0_61 PG_LSN NOT NULL, c͎ol59_w0_62 UUID NOT NULL, col59_w0_63 \"char\" NOT NULL, col59_w0_64 BOX2D, col59_w0_65 INT8 NOT NULL, \"-col'59_w0_66\" FLOAT8 NOT NULL AS (abs(col5͝9_w0_60)) STORED, \"col\"\"59_w0_67\" FLOAT8 NOT NULL AS (abs(col5͝9_w0_60)) STORED, col59_w0_68 STRING AS (lower(CAST(col59_w0_64 AS STRING))) VIRTUAL, col59_w0_69 STRING NOT NULL AS (lower(CAST(col59_w0_61 AS STRING))) STORED, PRIMARY KEY (\"-col'59_w0_66\" ASC, col59_w0_61 DESC, col59_w0_65 ASC), UNIQUE (col59_w0_65, lower(CAST(c͎ol59_w0_62 AS STRING)) DESC, col59_w0_68, col59_w0_69 ASC, col59_w0_61 ASC), UNIQUE (c͎ol59_w0_62, \"-col'59_w0_66\" DESC), UNIQUE (col59_w0_64 DESC, col5͝9_w0_60, col59_w0_63), INDEX (\"col\"\"59_w0_67\"), FAMILY (col59_w0_69), FAMILY (\"-col'59_w0_66\"), FAMILY (\"col\"\"59_w0_67\", c͎ol59_w0_62, col59_w0_64, col5͝9_w0_60, col59_w0_61, col59_w0_65, col59_w0_63))"

In addition we altered this sequence at the client timestamp of 06:02:25.924582, which is slightly less than the AOST used by the full backup 2024-09-16 06:02:27.24754:

"workerId": 2,
 "clientTimestamp": "06:02:25.924582",
 "ops": [
  "BEGIN",
  {
   "sql": "CREATE TABLE IF NOT EXISTS schema_w2_103.table_w2_211 (\"co͟l21\\n1_w2_212\" GEOGRAPHY NOT NULL, col211_w2_213 FLOAT8 NOT NULL, c͒ol211_w2_214 DECIMAL NOT NULL, \"col211\"\"_w2_215\" REGCLASS, PRIMARY KEY (c͒ol211_w2_214 ASC), INVERTED INDEX (\"co͟l21\\n1_w2_212\"), INDEX (c͒ol211_w2_214 ASC, \"col211\"\"_w2_215\"), INVERTED INDEX (c͒ol211_w2_214 ASC, col211_w2_213 ASC, \"col211\"\"_w2_215\" DESC, \"co͟l21\\n1_w2_212\" ASC))"
  },
  {
   "sql": "ALTER TABLE public.table_w1_144 ALTER COLUMN col144_w1_152 SET DEFAULT B'0111111111111111111111111'"
  },
  {
   "sql": "CREATE TABLE schema_w4_134.table_w2_216 AS SELECT schema_w2_103.table_w3_233.col233_w3_235, schema_w2_103.table_w3_233.col233_w3_236, schema_w2_103.table_w3_233.\"col\"\"2😞33_w3_238\", schema_w2_103.table_w3_233.\"c\nol233_w3_241\" FROM schema_w2_103.table_w3_233 FETCH FIRST 300000 ROWS ONLY",
   "potentialExecErr": "22003,22P01,22P02"
  },
  {
   "sql": "ALTER SEQUENCE public.seq_w3_309 RENAME TO public.seq_w2_217"
  },
  {
   "sql": "CREATE TABLE IF NOT EXISTS schema_w3_308.table_w2_218 (col218_w2_219 VARCHAR NOT NULL, col218_w2_220 FLOAT8 NOT NULL, col218_w2_221 \"char\", col218_w2_222 STRING COLLATE de_DE NOT NULL, col218_w2_223 JSONB NOT NULL, col218_w2_224 TIMESTAMP, col218_w2_225 FLOAT8[] NOT NULL, 😑col218_w2_226 DECIMAL NOT NULL, col218_w2_227 DECIMAL NOT NULL AS (abs(😑col218_w2_226)) VIRTUAL, PRIMARY KEY (😑col218_w2_226, col218_w2_227 ASC), FAMILY (col218_w2_221), FAMILY (col218_w2_224), FAMILY (col218_w2_222, 😑col218_w2_226, col218_w2_219), FAMILY (col218_w2_223), FAMILY (col218_w2_225, col218_w2_220))"
  },
  "COMMIT"
 ],
 "expectedExecErrors": "",
 "expectedCommitErrors": "",
 "message": ""
}
msbutler commented 19 hours ago

I may to need input from foundations to proceed, but i can explain what i'm seeing:

On the cluster backup side, we simply grab all cluster descriptors with this helper function: https://github.com/cockroachdb/cockroach/blob/master/pkg/ccl/backupccl/targets.go#L333 which calls GetAllDescriptors, so it's unclear to me how we missed the grabbing the sequence, Perhaps the workload client timestamp has significant skew with the mvcc timestamp in the db???

Here's a timeline:

So the descriptorRewrites map is missing the sequence. The sequence also doesn't appear in show backup.