From 59c8bb34b200509b32d0a3ef6e20c73ecfc4744b Mon Sep 17 00:00:00 2001 From: Yahor Yuzefovich Date: Mon, 6 Feb 2023 16:07:01 -0800 Subject: [PATCH] parser: fix GetTypeFromValidSQLSyntax for collated strings Previously, `GetTypeFromValidSQLSyntax` would result in an error when attempting to parse a collated string. This is due to an expression like `1::STRING COLLATE en` being parsed as `CollateExpr(CastExpr)` whereas the function expected just `CastExpr`. This is now fixed by having a special case for the collated strings. Additionally, this commit adds a couple of testing improvements around the collated strings: - a representative collated string type is now included into `randgen.SeedTypes` - a new test is introduced that asserts that for all (with a few exceptions) type families at least one representative type is included into `randgen.SeedTypes`. Release note: None --- pkg/internal/sqlsmith/BUILD.bazel | 1 + pkg/internal/sqlsmith/sqlsmith_test.go | 4 ++ pkg/sql/parser/BUILD.bazel | 3 ++ pkg/sql/parser/parse.go | 13 +++++ pkg/sql/parser/parse_test.go | 18 +++++++ pkg/sql/randgen/BUILD.bazel | 5 +- pkg/sql/randgen/mutator.go | 6 +++ pkg/sql/randgen/type.go | 8 +++ pkg/sql/randgen/types_test.go | 54 +++++++++++++++++++ .../schemachange/operation_generator.go | 10 ++-- 10 files changed, 116 insertions(+), 6 deletions(-) create mode 100644 pkg/sql/randgen/types_test.go diff --git a/pkg/internal/sqlsmith/BUILD.bazel b/pkg/internal/sqlsmith/BUILD.bazel index 76c60cdcd6c2..fce8b0fe6fe4 100644 --- a/pkg/internal/sqlsmith/BUILD.bazel +++ b/pkg/internal/sqlsmith/BUILD.bazel @@ -64,6 +64,7 @@ go_test( "//pkg/testutils/sqlutils", "//pkg/testutils/testcluster", "//pkg/util/leaktest", + "//pkg/util/log", "//pkg/util/randutil", ], ) diff --git a/pkg/internal/sqlsmith/sqlsmith_test.go b/pkg/internal/sqlsmith/sqlsmith_test.go index 96f8445e07c8..186c6a8e937a 100644 --- a/pkg/internal/sqlsmith/sqlsmith_test.go +++ b/pkg/internal/sqlsmith/sqlsmith_test.go @@ -23,6 +23,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" "github.com/cockroachdb/cockroach/pkg/testutils/sqlutils" "github.com/cockroachdb/cockroach/pkg/util/leaktest" + "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/randutil" ) @@ -36,6 +37,7 @@ var ( // TestSetups verifies that all setups generate executable SQL. func TestSetups(t *testing.T) { defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) defer ccl.TestingEnableEnterprise()() for name, setup := range Setups { @@ -81,6 +83,7 @@ func TestSetups(t *testing.T) { // false-negative. func TestRandTableInserts(t *testing.T) { defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) ctx := context.Background() s, sqlDB, _ := serverutils.StartServer(t, base.TestServerArgs{}) @@ -144,6 +147,7 @@ func TestRandTableInserts(t *testing.T) { // sometimes put them into bad states that the parser would never do. func TestGenerateParse(t *testing.T) { defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) defer ccl.TestingEnableEnterprise()() ctx := context.Background() diff --git a/pkg/sql/parser/BUILD.bazel b/pkg/sql/parser/BUILD.bazel index 37c06531521d..1545c3df7ed4 100644 --- a/pkg/sql/parser/BUILD.bazel +++ b/pkg/sql/parser/BUILD.bazel @@ -61,6 +61,7 @@ go_test( deps = [ "//pkg/sql/lexbase", "//pkg/sql/pgwire/pgerror", + "//pkg/sql/randgen", "//pkg/sql/sem/builtins", "//pkg/sql/sem/tree", "//pkg/sql/sem/tree/treebin", @@ -70,9 +71,11 @@ go_test( "//pkg/testutils/sqlutils", "//pkg/util/leaktest", "//pkg/util/log", + "//pkg/util/randutil", "@com_github_cockroachdb_datadriven//:datadriven", "@com_github_cockroachdb_errors//:errors", "@com_github_stretchr_testify//assert", + "@com_github_stretchr_testify//require", ], ) diff --git a/pkg/sql/parser/parse.go b/pkg/sql/parser/parse.go index 563479d6def5..ad73aa6abc1c 100644 --- a/pkg/sql/parser/parse.go +++ b/pkg/sql/parser/parse.go @@ -458,6 +458,19 @@ func GetTypeFromValidSQLSyntax(sql string) (tree.ResolvableTypeReference, error) if err != nil { return nil, err } + return GetTypeFromCastOrCollate(expr) +} + +// GetTypeFromCastOrCollate returns the type of the given tree.Expr. The method +// assumes that the expression is either tree.CastExpr or tree.CollateExpr +// (which wraps the tree.CastExpr). +func GetTypeFromCastOrCollate(expr tree.Expr) (tree.ResolvableTypeReference, error) { + // COLLATE clause has lower precedence than the cast, so if we have + // something like `1::STRING COLLATE en`, it'll be parsed as + // CollateExpr(CastExpr). + if collate, ok := expr.(*tree.CollateExpr); ok { + return types.MakeCollatedString(types.String, collate.Locale), nil + } cast, ok := expr.(*tree.CastExpr) if !ok { diff --git a/pkg/sql/parser/parse_test.go b/pkg/sql/parser/parse_test.go index d26ff87f2aef..3ac9f3d64f83 100644 --- a/pkg/sql/parser/parse_test.go +++ b/pkg/sql/parser/parse_test.go @@ -21,6 +21,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/parser" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" + "github.com/cockroachdb/cockroach/pkg/sql/randgen" _ "github.com/cockroachdb/cockroach/pkg/sql/sem/builtins" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree/treebin" @@ -29,9 +30,11 @@ import ( "github.com/cockroachdb/cockroach/pkg/testutils/datapathutils" "github.com/cockroachdb/cockroach/pkg/testutils/sqlutils" _ "github.com/cockroachdb/cockroach/pkg/util/log" // for flags + "github.com/cockroachdb/cockroach/pkg/util/randutil" "github.com/cockroachdb/datadriven" "github.com/cockroachdb/errors" "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" ) var issueLinkRE = regexp.MustCompile("https://go.crdb.dev/issue-v/([0-9]+)/.*") @@ -746,3 +749,18 @@ func BenchmarkParse(b *testing.B) { }) } } + +func TestGetTypeFromValidSQLSyntax(t *testing.T) { + rng, _ := randutil.NewTestRand() + + const numRuns = 1000 + for i := 0; i < numRuns; i++ { + orig := randgen.RandType(rng) + // TODO(yuzefovich): ideally, we'd assert that the returned type is + // equal to the original one; however, there are some subtle differences + // at the moment (like the width might only be set on the returned + // type), so we simply assert that no error is returned. + _, err := parser.GetTypeFromValidSQLSyntax(orig.SQLString()) + require.NoError(t, err) + } +} diff --git a/pkg/sql/randgen/BUILD.bazel b/pkg/sql/randgen/BUILD.bazel index d94870765905..e92aaabed07f 100644 --- a/pkg/sql/randgen/BUILD.bazel +++ b/pkg/sql/randgen/BUILD.bazel @@ -65,21 +65,24 @@ go_test( "main_test.go", "mutator_test.go", "schema_test.go", + "types_test.go", ], args = ["-test.timeout=295s"], + embed = [":randgen"], deps = [ - ":randgen", "//pkg/base", "//pkg/ccl", "//pkg/security/securityassets", "//pkg/security/securitytest", "//pkg/server", "//pkg/sql/sem/tree", + "//pkg/sql/types", "//pkg/testutils/serverutils", "//pkg/testutils/sqlutils", "//pkg/testutils/testcluster", "//pkg/util/leaktest", "//pkg/util/randutil", + "@com_github_cockroachdb_errors//:errors", "@com_github_stretchr_testify//require", ], ) diff --git a/pkg/sql/randgen/mutator.go b/pkg/sql/randgen/mutator.go index 9cc4986fc80a..259bc7c394ac 100644 --- a/pkg/sql/randgen/mutator.go +++ b/pkg/sql/randgen/mutator.go @@ -218,6 +218,12 @@ func statisticsMutator( return } colType := tree.MustBeStaticallyKnownType(col.Type) + if colType.Family() == types.CollatedStringFamily { + // Collated strings are not roundtrippable during + // encoding/decoding, so we cannot always make a valid + // histogram. + return + } h := randHistogram(rng, colType) stat := colStats[col.Name] if err := stat.SetHistogram(&h); err != nil { diff --git a/pkg/sql/randgen/type.go b/pkg/sql/randgen/type.go index 76eb0e731aea..fd02ba64e37f 100644 --- a/pkg/sql/randgen/type.go +++ b/pkg/sql/randgen/type.go @@ -59,6 +59,14 @@ func init() { } } + // Add a collated string separately (since it shares the oid with the STRING + // type and, thus, wasn't included above). + collatedStringType := types.MakeCollatedString(types.String, "en" /* locale */) + SeedTypes = append(SeedTypes, collatedStringType) + if IsAllowedForArray(collatedStringType) { + arrayContentsTypes = append(arrayContentsTypes, collatedStringType) + } + // Sort these so randomly chosen indexes always point to the same element. sort.Slice(SeedTypes, func(i, j int) bool { return SeedTypes[i].String() < SeedTypes[j].String() diff --git a/pkg/sql/randgen/types_test.go b/pkg/sql/randgen/types_test.go new file mode 100644 index 000000000000..fc1ca32b5249 --- /dev/null +++ b/pkg/sql/randgen/types_test.go @@ -0,0 +1,54 @@ +// Copyright 2023 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 randgen + +import ( + "fmt" + "testing" + + "github.com/cockroachdb/cockroach/pkg/sql/types" + "github.com/cockroachdb/cockroach/pkg/util/leaktest" + "github.com/cockroachdb/errors" +) + +// TestSeedTypes verifies that at least one representative type is included into +// SeedTypes for all (with a few exceptions) type families. +func TestSeedTypes(t *testing.T) { + defer leaktest.AfterTest(t)() + + noFamilyRepresentative := make(map[types.Family]struct{}) +loop: + for id := range types.Family_name { + familyID := types.Family(id) + switch familyID { + case types.EnumFamily: + // Enums need to created separately. + continue loop + case types.EncodedKeyFamily: + // It's not a real type. + continue loop + case types.UnknownFamily, types.AnyFamily: + // These are not included on purpose. + continue loop + } + noFamilyRepresentative[familyID] = struct{}{} + } + for _, typ := range SeedTypes { + delete(noFamilyRepresentative, typ.Family()) + } + if len(noFamilyRepresentative) > 0 { + s := "no representative for " + for f := range noFamilyRepresentative { + s += fmt.Sprintf("%s (%d) ", types.Family_name[int32(f)], f) + } + t.Fatal(errors.Errorf("%s", s)) + } +} diff --git a/pkg/workload/schemachange/operation_generator.go b/pkg/workload/schemachange/operation_generator.go index ea4a49de2737..8400cff155bb 100644 --- a/pkg/workload/schemachange/operation_generator.go +++ b/pkg/workload/schemachange/operation_generator.go @@ -3582,11 +3582,11 @@ func (og *operationGenerator) typeFromTypeName( if err != nil { return nil, errors.Wrapf(err, "typeFromTypeName: %s", typeName) } - typ, err := tree.ResolveType( - ctx, - stmt.AST.(*tree.Select).Select.(*tree.SelectClause).Exprs[0].Expr.(*tree.CastExpr).Type, - &txTypeResolver{tx: tx}, - ) + typRef, err := parser.GetTypeFromCastOrCollate(stmt.AST.(*tree.Select).Select.(*tree.SelectClause).Exprs[0].Expr) + if err != nil { + return nil, errors.Wrapf(err, "GetTypeFromCastOrCollate: %s", typeName) + } + typ, err := tree.ResolveType(ctx, typRef, &txTypeResolver{tx: tx}) if err != nil { return nil, errors.Wrapf(err, "ResolveType: %v", typeName) }