Skip to content

Commit

Permalink
Merge #57066
Browse files Browse the repository at this point in the history
57066: sql: allow for pre-19.2 foreign keys in table validation r=lucy-zhang a=lucy-zhang


sql: allow for pre-19.2 foreign keys in table validation

We introduced a bug in v20.2 where we failed to upgrade referenced
descriptors' FK representations from pre-19.2 descriptors when
validating cross-references for tables, leading to validation failures
that would make the table unusable. This PR gets rid of the validation
errors by having `Validate()` account for pre-19.2-style foreign keys on
referenced table descriptors.

Fixes #57032.

Release note (bug fix): Fixes a bug where tables and metadata were
unavailable due to spurious `missing fk back reference` validation
errors.

---

catalogkv: add testing of descriptor unwrapping and validation

This commit adds a framework to test that descriptors validate properly.
It also adds testdata and a bash script to generate that test data.

This test is to exercise a problematic path whereby tables constructed
in v19.1 would fail to validate in v20.2.

Release note: None

Co-authored-by: Lucy Zhang <lucy@cockroachlabs.com>
Co-authored-by: Andrew Werner <ajwerner@cockroachlabs.com>
  • Loading branch information
3 people committed Nov 24, 2020
2 parents 61bd94b + bb95a38 commit c9fb63a
Show file tree
Hide file tree
Showing 6 changed files with 514 additions and 20 deletions.
19 changes: 18 additions & 1 deletion pkg/sql/catalog/catalogkv/BUILD.bazel
Original file line number Diff line number Diff line change
@@ -1,4 +1,4 @@
load("@io_bazel_rules_go//go:def.bzl", "go_library")
load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test")

go_library(
name = "catalogkv",
Expand Down Expand Up @@ -32,3 +32,20 @@ go_library(
"//vendor/github.com/cockroachdb/errors",
],
)

go_test(
name = "catalogkv_test",
srcs = ["unwrap_validation_test.go"],
data = glob(["testdata/**"]),
embed = [":catalogkv"],
deps = [
"//pkg/sql/catalog",
"//pkg/sql/catalog/descpb",
"//pkg/testutils",
"//pkg/util/encoding/csv",
"//pkg/util/hlc",
"//pkg/util/protoutil",
"//vendor/github.com/cockroachdb/errors",
"//vendor/github.com/stretchr/testify/require",
],
)

Large diffs are not rendered by default.

Original file line number Diff line number Diff line change
@@ -0,0 +1,100 @@
#!/bin/bash

set -e

declare -a VERSIONS=( v19.1.11 v19.2.11 v20.1.8 v20.2.1 )

declare -A VERSION_STATEMENTS=(
[v19.1.11]="
CREATE DATABASE db1;
USE db1;
CREATE TABLE a (i INT PRIMARY KEY, j INT UNIQUE);
CREATE TABLE b (i INT PRIMARY KEY, j INT UNIQUE REFERENCES a (j));
-- c.j will get an auto-created index.
CREATE TABLE c (i INT PRIMARY KEY, j INT REFERENCES b (j));
CREATE DATABASE db2;
USE db2;
CREATE TABLE a (i INT PRIMARY KEY, j INT UNIQUE, INDEX(j, i));
CREATE TABLE b (i INT PRIMARY KEY, j INT UNIQUE REFERENCES a (j));
-- c.j will get an auto-created index.
CREATE TABLE c (i INT PRIMARY KEY, j INT REFERENCES b (j), k INT, INDEX idx_k (k));
-- Create some extra indexes.
CREATE INDEX extra_idx_1 ON c(j);
CREATE INDEX extra_idx_2 ON c(j, k);
CREATE DATABASE db3;
USE db3;
CREATE TABLE a (i INT PRIMARY KEY, j INT);
CREATE UNIQUE INDEX a_idx_j ON a(j);
CREATE TABLE b (i INT PRIMARY KEY, j INT, k INT);
CREATE INDEX b_idx_j ON b(j);
ALTER TABLE b ADD FOREIGN KEY (j) REFERENCES a(j);
CREATE TABLE c (i INT PRIMARY KEY, j INT, k INT);
CREATE INDEX c_idx_j ON c(j);
ALTER TABLE c ADD FOREIGN KEY (j) REFERENCES a(j);
"
[v19.2.11]="
-- These schema changes will upgrade the FKs on the respective descriptors.
-- Keep db1.b unupgraded, but upgrade the other tables in its reference graph.
ALTER TABLE db1.a RENAME COLUMN j TO k;
ALTER TABLE db1.c RENAME COLUMN j TO k;
-- Unrelated DROP INDEX.
DROP INDEX db2.c@idx_k;
"
[v20.1.8]="
-- Swap out a referencing-side index. This should upgrade db3.a, but we leave db3.c alone.
USE db3;
CREATE INDEX b_idx_j_new ON b(j, k);
DROP INDEX b_idx_j;
"
[v20.2.1]=''
)


main() {
check_local_does_not_exist
install_cockroach_versions
run_version_statements
dump_descriptors
}

check_local_does_not_exist() {
out="$( { roachprod run local true 2>&1 || true ; } | head )"
if [[ "${out}" =~ "unknown cluster: local" ]]; then
return 0
fi
echo >&2 "make sure the local cluster does not exist"
exit 1
}

install_cockroach_versions() {
roachprod create local -n 1
for v in "${VERSIONS[@]}"; do
roachprod stage local release "${v}"
roachprod run local cp ./cockroach "./cockroach-${v}"
done
}

run_version_statements() {
for v in "${VERSIONS[@]}"; do
roachprod stop local
roachprod start local --binary cockroach-$v
sleep 1 # wait for the upgrade to happen
stmts="${VERSION_STATEMENTS[$v]}"
if [[ -z "${stmts}" ]]; then
continue
fi
roachprod sql local -- -e "${stmts}"
done
}

dump_descriptors() {
roachprod sql local -- \
--format csv \
-e "SELECT id, encode(descriptor, 'hex') AS descriptor FROM system.descriptor" \
> descriptors.csv
}

main
127 changes: 127 additions & 0 deletions pkg/sql/catalog/catalogkv/unwrap_validation_test.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,127 @@
// Copyright 2020 The Cockroach Authors.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt.
//
// As of the Change Date specified in that file, in accordance with
// the Business Source License, use of this software will be governed
// by the Apache License, Version 2.0, included in the file
// licenses/APL.txt.

package catalogkv

import (
"context"
"encoding/hex"
"io/ioutil"
"os"
"path/filepath"
"strconv"
"testing"

"github.com/cockroachdb/cockroach/pkg/sql/catalog"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb"
"github.com/cockroachdb/cockroach/pkg/testutils"
"github.com/cockroachdb/cockroach/pkg/util/encoding/csv"
"github.com/cockroachdb/cockroach/pkg/util/hlc"
"github.com/cockroachdb/cockroach/pkg/util/protoutil"
"github.com/cockroachdb/errors"
"github.com/stretchr/testify/require"
)

// TestUnwrapValidation uses testdata to find issues validating descriptors.
// The test is driven by serialized testdata. The expected testdata directories
// will hold a file "descriptors.csv" which is a csv of id,descriptor where
// descriptor is hex encoded.
func TestUnwrapValidation(t *testing.T) {
testdata := testutils.TestDataPath("testdata", "unwrap_validation")
const descriptorsCSVFilename = "descriptors.csv"
dirs, err := ioutil.ReadDir(testdata)
require.NoError(t, err)
for _, dir := range dirs {
if !dir.IsDir() {
continue
}
dp := filepath.Join(testdata, dir.Name(), descriptorsCSVFilename)
if _, err := os.Stat(dp); errors.Is(err, os.ErrNotExist) {
continue
}
t.Run(dir.Name(), func(t *testing.T) {
unwrapValidationTest(t, dp)
})
}
}

func unwrapValidationTest(t *testing.T, descriptorCSVPath string) {
m := decodeDescriptorDSV(t, descriptorCSVPath)
for id, data := range m {
var desc descpb.Descriptor
require.NoError(t, protoutil.Unmarshal(data, &desc))
ts := descpb.GetDescriptorModificationTime(&desc)
if ts == (hlc.Timestamp{}) {
ts = hlc.Timestamp{WallTime: 1}
}
_, err := unwrapDescriptor(context.Background(), m, ts, &desc, true)
require.NoErrorf(t, err, "id: %d", id)
}
}

// oneLevelMapDescGetter exists to mirror the behavior of the
// oneLevelTxnDescGetter but instead of reading from the key-value store, it
// reads from the map.
type oneLevelMapDescGetter map[descpb.ID][]byte

var _ catalog.DescGetter = (oneLevelMapDescGetter)(nil)

func (o oneLevelMapDescGetter) GetDesc(
ctx context.Context, id descpb.ID,
) (catalog.Descriptor, error) {
var desc descpb.Descriptor
if err := protoutil.Unmarshal(o[id], &desc); err != nil {
return nil, err
}
mt := descpb.GetDescriptorModificationTime(&desc)
if mt == (hlc.Timestamp{}) {
mt = hlc.Timestamp{WallTime: 1}
}
return unwrapDescriptorMutable(ctx, nil, mt, &desc)
}

func (o oneLevelMapDescGetter) GetDescs(
ctx context.Context, reqs []descpb.ID,
) ([]catalog.Descriptor, error) {
resps := make([]catalog.Descriptor, len(reqs))
for i, r := range reqs {
var err error
resps[i], err = o.GetDesc(ctx, r)
if err != nil {
return nil, err
}
}
return resps, nil
}

func decodeDescriptorDSV(t *testing.T, descriptorCSVPath string) oneLevelMapDescGetter {
f, err := os.Open(descriptorCSVPath)
require.NoError(t, err)
defer f.Close()
r := csv.NewReader(f)
records, err := r.ReadAll()
require.NoError(t, err)
require.Equal(t, records[0], []string{"id", "descriptor"})
records = records[1:]
m := decodeCSVRecordsToDescGetter(t, records)
return m
}

func decodeCSVRecordsToDescGetter(t *testing.T, records [][]string) oneLevelMapDescGetter {
m := oneLevelMapDescGetter{}
for _, rec := range records {
id, err := strconv.Atoi(rec[0])
require.NoError(t, err)
decoded, err := hex.DecodeString(rec[1])
require.NoError(t, err)
m[descpb.ID(id)] = decoded
}
return m
}
Loading

0 comments on commit c9fb63a

Please sign in to comment.