From 3e19b06be29ea0ede9f0c240183b81e6070f53a9 Mon Sep 17 00:00:00 2001 From: Oliver Tan Date: Tue, 29 Sep 2020 20:46:22 -0700 Subject: [PATCH 1/7] sql: fix unescaped casts to regclass types Release note (bug fix): Fix a bug where casts to regclass were not escaped (i.e. when the type or table name had " characters). --- .../logictest/testdata/logic_test/pgoidtype | 16 ++ pkg/sql/sem/tree/casts.go | 143 ++++++++++++++++-- pkg/sql/sem/tree/casts_test.go | 74 +++++++++ 3 files changed, 222 insertions(+), 11 deletions(-) diff --git a/pkg/sql/logictest/testdata/logic_test/pgoidtype b/pkg/sql/logictest/testdata/logic_test/pgoidtype index 475df7ffc9ea..383385f0e6e3 100644 --- a/pkg/sql/logictest/testdata/logic_test/pgoidtype +++ b/pkg/sql/logictest/testdata/logic_test/pgoidtype @@ -347,3 +347,19 @@ FROM (d.adrelid = a.attrelid AND d.adnum = a.attnum) JOIN (SELECT 1 AS oid, 1 AS attnum) AS vals ON (c.oid = vals.oid AND a.attnum = vals.attnum); + +statement error relation ".*"regression_53686.*"" does not exist +SELECT '\"regression_53686\"'::regclass + +statement ok +CREATE TABLE "regression_53686""" (a int) + +query T +SELECT 'regression_53686"'::regclass +---- +"regression_53686""" + +query T +SELECT 'public.regression_53686"'::regclass +---- +"regression_53686""" diff --git a/pkg/sql/sem/tree/casts.go b/pkg/sql/sem/tree/casts.go index 669a08bd2a7e..7de301790d8b 100644 --- a/pkg/sql/sem/tree/casts.go +++ b/pkg/sql/sem/tree/casts.go @@ -1076,17 +1076,10 @@ func PerformCast(ctx *EvalContext, d Datum, t *types.T) (Datum, error) { } case *DString: s := string(*v) - // Trim whitespace and unwrap outer quotes if necessary. - // This is required to mimic postgres. - s = strings.TrimSpace(s) - origS := s - if len(s) > 1 && s[0] == '"' && s[len(s)-1] == '"' { - s = s[1 : len(s)-1] - } switch t.Oid() { case oid.T_oid: - i, err := ParseDInt(s) + i, err := ParseDInt(strings.TrimSpace(s)) if err != nil { return nil, err } @@ -1097,7 +1090,7 @@ func PerformCast(ctx *EvalContext, d Datum, t *types.T) (Datum, error) { // going to always do it. // We additionally do not yet implement disambiguation based on type // parameters: we return the match iff there is exactly one. - s = pgSignatureRegexp.ReplaceAllString(s, "$1") + s = pgSignatureRegexp.ReplaceAllString(formatIdentifierForOidFamilyCast(s), "$1") // Resolve function name. substrs := strings.Split(s, ".") if len(substrs) > 3 { @@ -1118,6 +1111,7 @@ func PerformCast(ctx *EvalContext, d Datum, t *types.T) (Datum, error) { } return queryOid(ctx, t, NewDString(funcDef.Name)) case oid.T_regtype: + s = formatIdentifierForOidFamilyCast(s) parsedTyp, err := ctx.Planner.ParseType(s) if err == nil { return &DOid{ @@ -1155,11 +1149,11 @@ func PerformCast(ctx *EvalContext, d Datum, t *types.T) (Datum, error) { }, nil case oid.T_regclass: - tn, err := ctx.Planner.ParseQualifiedTableName(origS) + tn, err := castStringToRegClassTableName(s) if err != nil { return nil, err } - id, err := ctx.Planner.ResolveTableName(ctx.Ctx(), tn) + id, err := ctx.Planner.ResolveTableName(ctx.Ctx(), &tn) if err != nil { return nil, err } @@ -1177,3 +1171,130 @@ func PerformCast(ctx *EvalContext, d Datum, t *types.T) (Datum, error) { return nil, pgerror.Newf( pgcode.CannotCoerce, "invalid cast: %s -> %s", d.ResolvedType(), t) } + +// castStringToRegClassTableName normalizes a TableName from a string. +func castStringToRegClassTableName(s string) (TableName, error) { + components, err := splitIdentifierList(s) + if err != nil { + return TableName{}, err + } + + if len(components) > 3 { + return TableName{}, pgerror.Newf( + pgcode.InvalidName, + "too many components: %s", + s, + ) + } + var retComponents [3]string + for i := 0; i < len(components); i++ { + retComponents[len(components)-1-i] = components[i] + } + u, err := NewUnresolvedObjectName( + len(components), + retComponents, + 0, + ) + if err != nil { + return TableName{}, err + } + return u.ToTableName(), nil +} + +// formatIdentifierForOidFamilyCast cleans the identifier by removing whitespace and removing +// surrounding quotes. +// TODO(#sql-features): this does not work for split types, e.g. +// "asdf"."asdf" is incorrectly handled. +func formatIdentifierForOidFamilyCast(s string) string { + s = strings.TrimSpace(s) + // Trim whitespace and unwrap outer quotes if necessary. + // This is required to mimic postgres. + if len(s) > 1 && s[0] == '"' && s[len(s)-1] == '"' { + s = s[1 : len(s)-1] + } + return s +} + +// splitIdentifierList splits identifiers to individual components, lower +// casing non-quoted identifiers and escaping quoted identifiers as appropriate. +// It is based on PostgreSQL's SplitIdentifier. +func splitIdentifierList(in string) ([]string, error) { + var pos int + var ret []string + const separator = '.' + + for pos < len(in) { + if isWhitespace(in[pos]) { + pos++ + continue + } + if in[pos] == '"' { + var b strings.Builder + // Attempt to find the ending quote. If the quote is double "", + // fold it into a " character for the str (e.g. "a""" means a"). + for { + pos++ + endIdx := strings.IndexByte(in[pos:], '"') + if endIdx == -1 { + return nil, pgerror.Newf( + pgcode.InvalidName, + `invalid name: unclosed ": %s`, + in, + ) + } + b.WriteString(in[pos : pos+endIdx]) + pos += endIdx + 1 + // If we reached the end, or the following character is not ", + // we can break and assume this is one identifier. + // There are checks below to ensure EOF or whitespace comes + // afterward. + if pos == len(in) || in[pos] != '"' { + break + } + b.WriteByte('"') + } + ret = append(ret, b.String()) + } else { + var b strings.Builder + for pos < len(in) && in[pos] != separator && !isWhitespace(in[pos]) { + b.WriteByte(in[pos]) + pos++ + } + // Anything with no quotations should be lowered. + ret = append(ret, strings.ToLower(b.String())) + } + + // Further ignore all white space. + for pos < len(in) && isWhitespace(in[pos]) { + pos++ + } + + // At this stage, we expect separator or end of string. + if pos == len(in) { + break + } + + if in[pos] != separator { + return nil, pgerror.Newf( + pgcode.InvalidName, + "invalid name: expected separator %c: %s", + separator, + in, + ) + } + + pos++ + } + + return ret, nil +} + +// isWhitespace returns true if the given character is a space. +// This must match parser.SkipWhitespace above. +func isWhitespace(ch byte) bool { + switch ch { + case ' ', '\t', '\r', '\f', '\n': + return true + } + return false +} diff --git a/pkg/sql/sem/tree/casts_test.go b/pkg/sql/sem/tree/casts_test.go index c8e03adbe76a..f6760d74677b 100644 --- a/pkg/sql/sem/tree/casts_test.go +++ b/pkg/sql/sem/tree/casts_test.go @@ -203,3 +203,77 @@ func TestTupleCastVolatility(t *testing.T) { } } } + +func TestCastStringToRegClassTableName(t *testing.T) { + defer leaktest.AfterTest(t)() + + testCases := []struct { + in string + expected TableName + }{ + {"a", MakeUnqualifiedTableName("a")}, + {`a"`, MakeUnqualifiedTableName(`a"`)}, + {`"a""".bB."cD" `, MakeTableNameWithSchema(`a"`, "bb", "cD")}, + } + + for _, tc := range testCases { + t.Run(tc.in, func(t *testing.T) { + out, err := castStringToRegClassTableName(tc.in) + require.NoError(t, err) + require.Equal(t, tc.expected, out) + }) + } + + errorTestCases := []struct { + in string + expectedError string + }{ + {"a.b.c.d", "too many components: a.b.c.d"}, + {"", `invalid table name: `}, + } + + for _, tc := range errorTestCases { + t.Run(tc.in, func(t *testing.T) { + _, err := castStringToRegClassTableName(tc.in) + require.EqualError(t, err, tc.expectedError) + }) + } + +} + +func TestSplitIdentifierList(t *testing.T) { + defer leaktest.AfterTest(t)() + + testCases := []struct { + in string + expected []string + }{ + {`abc`, []string{"abc"}}, + {`abc.dEf `, []string{"abc", "def"}}, + {` "aBc" . d ."HeLLo"""`, []string{"aBc", "d", `HeLLo"`}}, + } + + for _, tc := range testCases { + t.Run(tc.in, func(t *testing.T) { + out, err := splitIdentifierList(tc.in) + require.NoError(t, err) + require.Equal(t, tc.expected, out) + }) + } + + errorTestCases := []struct { + in string + expectedError string + }{ + {`"unclosed`, `invalid name: unclosed ": "unclosed`}, + {`"unclosed""`, `invalid name: unclosed ": "unclosed""`}, + {`hello !`, `invalid name: expected separator .: hello !`}, + } + + for _, tc := range errorTestCases { + t.Run(tc.in, func(t *testing.T) { + _, err := splitIdentifierList(tc.in) + require.EqualError(t, err, tc.expectedError) + }) + } +} From d497df9eeddd3e7f7b2273268971e8a2f865da80 Mon Sep 17 00:00:00 2001 From: Oliver Tan Date: Thu, 15 Oct 2020 13:58:58 -0700 Subject: [PATCH 2/7] tree: add more casting fixes for text to reg* Release note (sql change): Added ability to cast a string containing all integers to be cast into a given regtype, e.g. '1234'::regproc is now valid. Release note (bug fix): Fixed a bug where casts from string to regproc, regtype or regprocedure would not work if they contained " characters at the beginning or at the end. --- .../logictest/testdata/logic_test/pgoidtype | 25 +++++++++- pkg/sql/sem/tree/casts.go | 50 ++++++++++--------- pkg/sql/sem/tree/eval.go | 2 +- 3 files changed, 52 insertions(+), 25 deletions(-) diff --git a/pkg/sql/logictest/testdata/logic_test/pgoidtype b/pkg/sql/logictest/testdata/logic_test/pgoidtype index 383385f0e6e3..1b92e621be74 100644 --- a/pkg/sql/logictest/testdata/logic_test/pgoidtype +++ b/pkg/sql/logictest/testdata/logic_test/pgoidtype @@ -100,7 +100,7 @@ SELECT 'blah ()'::REGPROC query error unknown function: blah\(\) SELECT 'blah( )'::REGPROC -query error unknown function: blah\(, \)\(\) +query error invalid name: expected separator \.: blah\(, \) SELECT 'blah(, )'::REGPROC query error more than one function named 'sqrt' @@ -363,3 +363,26 @@ query T SELECT 'public.regression_53686"'::regclass ---- "regression_53686""" + +query T +SELECT 'pg_catalog."radians"'::regproc +---- +radians + +query T +SELECT 'pg_catalog."radians"("float4")'::regproc +---- +radians + +statement error unknown function: pg_catalog.radians"\(\) +SELECT 'pg_catalog."radians"""'::regproc + +query TTTTT +SELECT + '12345'::regclass::string, + '12345'::regtype::string, + '12345'::oid::string, + '12345'::regproc::string, + '12345'::regprocedure::string +---- +12345 12345 12345 12345 12345 diff --git a/pkg/sql/sem/tree/casts.go b/pkg/sql/sem/tree/casts.go index 7de301790d8b..028da58909f6 100644 --- a/pkg/sql/sem/tree/casts.go +++ b/pkg/sql/sem/tree/casts.go @@ -1077,22 +1077,30 @@ func PerformCast(ctx *EvalContext, d Datum, t *types.T) (Datum, error) { case *DString: s := string(*v) - switch t.Oid() { - case oid.T_oid: - i, err := ParseDInt(strings.TrimSpace(s)) + // If it is an integer in string form, convert it as an int. + if val, err := ParseDInt(strings.TrimSpace(s)); err == nil { + tmpOid := NewDOid(*val) + oid, err := queryOid(ctx, t, tmpOid) if err != nil { - return nil, err + oid = tmpOid + oid.semanticType = t } - return &DOid{semanticType: t, DInt: *i}, nil + return oid, nil + } + + switch t.Oid() { case oid.T_regproc, oid.T_regprocedure: // Trim procedure type parameters, e.g. `max(int)` becomes `max`. // Postgres only does this when the cast is ::regprocedure, but we're // going to always do it. // We additionally do not yet implement disambiguation based on type // parameters: we return the match iff there is exactly one. - s = pgSignatureRegexp.ReplaceAllString(formatIdentifierForOidFamilyCast(s), "$1") - // Resolve function name. - substrs := strings.Split(s, ".") + s = pgSignatureRegexp.ReplaceAllString(s, "$1") + + substrs, err := splitIdentifierList(s) + if err != nil { + return nil, err + } if len(substrs) > 3 { // A fully qualified function name in pg's dialect can contain // at most 3 parts: db.schema.funname. @@ -1111,7 +1119,6 @@ func PerformCast(ctx *EvalContext, d Datum, t *types.T) (Datum, error) { } return queryOid(ctx, t, NewDString(funcDef.Name)) case oid.T_regtype: - s = formatIdentifierForOidFamilyCast(s) parsedTyp, err := ctx.Planner.ParseType(s) if err == nil { return &DOid{ @@ -1120,10 +1127,21 @@ func PerformCast(ctx *EvalContext, d Datum, t *types.T) (Datum, error) { name: parsedTyp.SQLStandardName(), }, nil } + // Fall back to searching pg_type, since we don't provide syntax for // every postgres type that we understand OIDs for. + // Note this section does *not* work if there is a schema in front of the + // type, e.g. "pg_catalog"."int4" (if int4 was not defined). + + // Trim whitespace and unwrap outer quotes if necessary. + // This is required to mimic postgres. + s = strings.TrimSpace(s) + if len(s) > 1 && s[0] == '"' && s[len(s)-1] == '"' { + s = s[1 : len(s)-1] + } // Trim type modifiers, e.g. `numeric(10,3)` becomes `numeric`. s = pgSignatureRegexp.ReplaceAllString(s, "$1") + dOid, missingTypeErr := queryOid(ctx, t, NewDString(s)) if missingTypeErr == nil { return dOid, missingTypeErr @@ -1201,20 +1219,6 @@ func castStringToRegClassTableName(s string) (TableName, error) { return u.ToTableName(), nil } -// formatIdentifierForOidFamilyCast cleans the identifier by removing whitespace and removing -// surrounding quotes. -// TODO(#sql-features): this does not work for split types, e.g. -// "asdf"."asdf" is incorrectly handled. -func formatIdentifierForOidFamilyCast(s string) string { - s = strings.TrimSpace(s) - // Trim whitespace and unwrap outer quotes if necessary. - // This is required to mimic postgres. - if len(s) > 1 && s[0] == '"' && s[len(s)-1] == '"' { - s = s[1 : len(s)-1] - } - return s -} - // splitIdentifierList splits identifiers to individual components, lower // casing non-quoted identifiers and escaping quoted identifiers as appropriate. // It is based on PostgreSQL's SplitIdentifier. diff --git a/pkg/sql/sem/tree/eval.go b/pkg/sql/sem/tree/eval.go index 12a17f94e4c5..a7e38538e8d1 100644 --- a/pkg/sql/sem/tree/eval.go +++ b/pkg/sql/sem/tree/eval.go @@ -3616,7 +3616,7 @@ func (expr *CaseExpr) Eval(ctx *EvalContext) (Datum, error) { // pgSignatureRegexp matches a Postgres function type signature, capturing the // name of the function into group 1. // e.g. function(a, b, c) or function( a ) -var pgSignatureRegexp = regexp.MustCompile(`^\s*([\w\.]+)\s*\((?:(?:\s*\w+\s*,)*\s*\w+)?\s*\)\s*$`) +var pgSignatureRegexp = regexp.MustCompile(`^\s*([\w\."]+)\s*\((?:(?:\s*[\w"]+\s*,)*\s*[\w"]+)?\s*\)\s*$`) // regTypeInfo contains details on a pg_catalog table that has a reg* type. type regTypeInfo struct { From b0a1069d3fc846e841783120ac985697ddd3e1dc Mon Sep 17 00:00:00 2001 From: Andrei Matei Date: Thu, 15 Oct 2020 16:48:53 -0400 Subject: [PATCH 3/7] server: improve error message on drain timeout When draining times out, we print out a warning. Commonly that warning tells you that the leases could not be drained in time. In this case, the patch improves the message: - don't print a useless stack trace any more - refer to the cluster setting controlling the timeout - refer to leases, not "Raft leadership". Draining deals with both, but leases are the concept that's somewhat exposed to users. Release note: None --- pkg/kv/kvserver/store.go | 16 ++++++++++++---- pkg/util/contextutil/context.go | 5 +++++ 2 files changed, 17 insertions(+), 4 deletions(-) diff --git a/pkg/kv/kvserver/store.go b/pkg/kv/kvserver/store.go index 3095c2f75d86..4d0269622b3d 100644 --- a/pkg/kv/kvserver/store.go +++ b/pkg/kv/kvserver/store.go @@ -1220,7 +1220,8 @@ func (s *Store) SetDraining(drain bool, reporter func(int, redact.SafeString)) { // until they're all gone, up to the configured timeout. transferTimeout := raftLeadershipTransferWait.Get(&s.cfg.Settings.SV) - if err := contextutil.RunWithTimeout(ctx, "wait for raft leadership transfer", transferTimeout, + drainLeasesOp := "transfer range leases" + if err := contextutil.RunWithTimeout(ctx, drainLeasesOp, transferTimeout, func(ctx context.Context) error { opts := retry.Options{ InitialBackoff: 10 * time.Millisecond, @@ -1251,9 +1252,16 @@ func (s *Store) SetDraining(drain bool, reporter func(int, redact.SafeString)) { // err, take it into account here. return errors.CombineErrors(err, ctx.Err()) }); err != nil { - // You expect this message when shutting down a server in an unhealthy - // cluster. If we see it on healthy ones, there's likely something to fix. - log.Warningf(ctx, "unable to drain cleanly within %s, service might briefly deteriorate: %+v", transferTimeout, err) + if tErr := (*contextutil.TimeoutError)(nil); errors.As(err, &tErr) && tErr.Operation() == drainLeasesOp { + // You expect this message when shutting down a server in an unhealthy + // cluster, or when draining all nodes with replicas for some range at the + // same time. If we see it on healthy ones, there's likely something to fix. + log.Warningf(ctx, "unable to drain cleanly within %s (cluster setting %s), "+ + "service might briefly deteriorate if the node is terminated: %s", + transferTimeout, raftLeadershipTransferWaitKey, tErr.Cause()) + } else { + log.Warningf(ctx, "drain error: %+v", err) + } } } diff --git a/pkg/util/contextutil/context.go b/pkg/util/contextutil/context.go index b6c1eec62b3c..37035be7d62c 100644 --- a/pkg/util/contextutil/context.go +++ b/pkg/util/contextutil/context.go @@ -96,6 +96,11 @@ var _ errors.Formatter = (*TimeoutError)(nil) // that people looking for net.Error attributes will still find them. var _ net.Error = (*TimeoutError)(nil) +// Operation returns the name of the operation that timed out. +func (t *TimeoutError) Operation() string { + return t.operation +} + func (t *TimeoutError) Error() string { return fmt.Sprintf("%v", t) } // Format implements fmt.Formatter. From 63ad5403f9899dd7deb4f05d155fe2ff5e2e0101 Mon Sep 17 00:00:00 2001 From: Andrei Matei Date: Thu, 15 Oct 2020 17:35:33 -0400 Subject: [PATCH 4/7] kvserver: minor renames Rename some variables to match the cluster setting controlling them. Release note: None --- pkg/kv/kvserver/store.go | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/pkg/kv/kvserver/store.go b/pkg/kv/kvserver/store.go index 4d0269622b3d..75f5896f3414 100644 --- a/pkg/kv/kvserver/store.go +++ b/pkg/kv/kvserver/store.go @@ -149,17 +149,17 @@ var queueAdditionOnSystemConfigUpdateBurst = settings.RegisterNonNegativeIntSett "the burst rate at which the store will add all replicas to the split and merge queue due to system config gossip", 32) -// raftLeadershipTransferTimeout limits the amount of time a drain command -// waits for lease transfers. -var raftLeadershipTransferWait = func() *settings.DurationSetting { +// leaseTransferWait limits the amount of time a drain command waits for lease +// and Raft leadership transfers. +var leaseTransferWait = func() *settings.DurationSetting { s := settings.RegisterValidatedDurationSetting( - raftLeadershipTransferWaitKey, + leaseTransferWaitSettingName, "the amount of time a server waits to transfer range leases before proceeding with the rest of the shutdown process", 5*time.Second, func(v time.Duration) error { if v < 0 { return errors.Errorf("cannot set %s to a negative duration: %s", - raftLeadershipTransferWaitKey, v) + leaseTransferWaitSettingName, v) } return nil }, @@ -168,7 +168,7 @@ var raftLeadershipTransferWait = func() *settings.DurationSetting { return s }() -const raftLeadershipTransferWaitKey = "server.shutdown.lease_transfer_wait" +const leaseTransferWaitSettingName = "server.shutdown.lease_transfer_wait" // ExportRequestsLimit is the number of Export requests that can run at once. // Each extracts data from RocksDB to a temp file and then uploads it to cloud @@ -1218,7 +1218,7 @@ func (s *Store) SetDraining(drain bool, reporter func(int, redact.SafeString)) { // We've seen all the replicas once. Now we're going to iterate // until they're all gone, up to the configured timeout. - transferTimeout := raftLeadershipTransferWait.Get(&s.cfg.Settings.SV) + transferTimeout := leaseTransferWait.Get(&s.cfg.Settings.SV) drainLeasesOp := "transfer range leases" if err := contextutil.RunWithTimeout(ctx, drainLeasesOp, transferTimeout, @@ -1258,7 +1258,7 @@ func (s *Store) SetDraining(drain bool, reporter func(int, redact.SafeString)) { // same time. If we see it on healthy ones, there's likely something to fix. log.Warningf(ctx, "unable to drain cleanly within %s (cluster setting %s), "+ "service might briefly deteriorate if the node is terminated: %s", - transferTimeout, raftLeadershipTransferWaitKey, tErr.Cause()) + transferTimeout, leaseTransferWaitSettingName, tErr.Cause()) } else { log.Warningf(ctx, "drain error: %+v", err) } From 149b583b5881dc191c95d27847ca51887e0011f5 Mon Sep 17 00:00:00 2001 From: Oliver Tan Date: Fri, 16 Oct 2020 16:12:21 -0700 Subject: [PATCH 5/7] *: include bazel in .gitignore Release note: None --- .gitignore | 3 +++ 1 file changed, 3 insertions(+) diff --git a/.gitignore b/.gitignore index 7d25e6ff1090..33b8e1e2cd9f 100644 --- a/.gitignore +++ b/.gitignore @@ -32,3 +32,6 @@ customenv.mk # Generated files containing include paths. zcgo_flags*.go build/Railroad.jar + +# Bazel generated symlinks +/bazel-* From a0da596bb12875c6056b04a89fc141260c80bc48 Mon Sep 17 00:00:00 2001 From: Nathan VanBenschoten Date: Fri, 16 Oct 2020 14:20:14 -0400 Subject: [PATCH 6/7] tpcc: remove unused and unpartitionable stock_item_fk_idx This was discussed in #50911. We ended up not making the change because it didn't appear to improve performance. Given the fact that the index is only used in a single place and can easily be replaced by the primary key of the stock table, it doesn't seem possible that this would actually hurt performance. However, it does seem possible for the index to hurt performance in multi-region clusters, where the fk validation was not guaranteed to be local because it was not using the partitioned primary key of the stock table. This allows us to remove a scary comment around partitioning code, because the index was not partitionable by warehouse id. --- pkg/ccl/importccl/read_import_avro_test.go | 5 ++--- pkg/sql/opt/bench/bench_test.go | 3 +-- pkg/sql/opt/testutils/opttester/testfixtures/tpcc_schema | 1 - pkg/sql/opt/xform/testdata/external/tpcc | 4 ++-- pkg/sql/opt/xform/testdata/external/tpcc-later-stats | 4 ++-- pkg/sql/opt/xform/testdata/external/tpcc-no-stats | 4 ++-- pkg/workload/tpcc/ddls.go | 2 +- pkg/workload/tpcc/partition.go | 3 --- pkg/workload/tpcc/tpcc.go | 4 ++-- 9 files changed, 12 insertions(+), 18 deletions(-) diff --git a/pkg/ccl/importccl/read_import_avro_test.go b/pkg/ccl/importccl/read_import_avro_test.go index 5d3fbba55dfa..c818b185d57a 100644 --- a/pkg/ccl/importccl/read_import_avro_test.go +++ b/pkg/ccl/importccl/read_import_avro_test.go @@ -570,9 +570,8 @@ func benchmarkAvroImport(b *testing.B, avroOpts roachpb.AvroOptions, testData st s_order_cnt integer, s_remote_cnt integer, s_data varchar(50), - primary key (s_w_id, s_i_id), - index stock_item_fk_idx (s_i_id)) - `) + primary key (s_w_id, s_i_id) +)`) require.NoError(b, err) diff --git a/pkg/sql/opt/bench/bench_test.go b/pkg/sql/opt/bench/bench_test.go index 94a6e0738d5b..78af7d553cb0 100644 --- a/pkg/sql/opt/bench/bench_test.go +++ b/pkg/sql/opt/bench/bench_test.go @@ -146,8 +146,7 @@ var schemas = [...]string{ s_order_cnt integer, s_remote_cnt integer, s_data varchar(50), - primary key (s_w_id, s_i_id), - index stock_item_fk_idx (s_i_id) + primary key (s_w_id, s_i_id) ) `, ` diff --git a/pkg/sql/opt/testutils/opttester/testfixtures/tpcc_schema b/pkg/sql/opt/testutils/opttester/testfixtures/tpcc_schema index dbee7b0c1fa2..66c38d98449e 100644 --- a/pkg/sql/opt/testutils/opttester/testfixtures/tpcc_schema +++ b/pkg/sql/opt/testutils/opttester/testfixtures/tpcc_schema @@ -141,7 +141,6 @@ CREATE TABLE stock s_remote_cnt integer, s_data varchar(50), primary key (s_w_id, s_i_id), - index stock_item_fk_idx (s_i_id), foreign key (s_w_id) references warehouse (w_id), foreign key (s_i_id) references item (i_id) ) interleave in parent warehouse (s_w_id) diff --git a/pkg/sql/opt/xform/testdata/external/tpcc b/pkg/sql/opt/xform/testdata/external/tpcc index b07f66394f66..0b3e5f9be6f1 100644 --- a/pkg/sql/opt/xform/testdata/external/tpcc +++ b/pkg/sql/opt/xform/testdata/external/tpcc @@ -404,9 +404,9 @@ insert order_line │ │ └── cardinality: [6 - 6] │ └── filters (true) └── f-k-checks-item: order_line(ol_supply_w_id,ol_i_id) -> stock(s_w_id,s_i_id) - └── anti-join (lookup stock@stock_item_fk_idx) + └── anti-join (lookup stock) ├── columns: column6:35!null column5:36!null - ├── key columns: [36 35] = [37 38] + ├── key columns: [35 36] = [38 37] ├── lookup columns are key ├── cardinality: [0 - 6] ├── with-scan &1 diff --git a/pkg/sql/opt/xform/testdata/external/tpcc-later-stats b/pkg/sql/opt/xform/testdata/external/tpcc-later-stats index 24a63fc01906..358aa551aba7 100644 --- a/pkg/sql/opt/xform/testdata/external/tpcc-later-stats +++ b/pkg/sql/opt/xform/testdata/external/tpcc-later-stats @@ -407,9 +407,9 @@ insert order_line │ │ └── cardinality: [6 - 6] │ └── filters (true) └── f-k-checks-item: order_line(ol_supply_w_id,ol_i_id) -> stock(s_w_id,s_i_id) - └── anti-join (lookup stock@stock_item_fk_idx) + └── anti-join (lookup stock) ├── columns: column6:35!null column5:36!null - ├── key columns: [36 35] = [37 38] + ├── key columns: [35 36] = [38 37] ├── lookup columns are key ├── cardinality: [0 - 6] ├── with-scan &1 diff --git a/pkg/sql/opt/xform/testdata/external/tpcc-no-stats b/pkg/sql/opt/xform/testdata/external/tpcc-no-stats index 777c7247a5f1..25e469a6aa4a 100644 --- a/pkg/sql/opt/xform/testdata/external/tpcc-no-stats +++ b/pkg/sql/opt/xform/testdata/external/tpcc-no-stats @@ -401,9 +401,9 @@ insert order_line │ │ └── cardinality: [6 - 6] │ └── filters (true) └── f-k-checks-item: order_line(ol_supply_w_id,ol_i_id) -> stock(s_w_id,s_i_id) - └── anti-join (lookup stock@stock_item_fk_idx) + └── anti-join (lookup stock) ├── columns: column6:35!null column5:36!null - ├── key columns: [36 35] = [37 38] + ├── key columns: [35 36] = [38 37] ├── lookup columns are key ├── cardinality: [0 - 6] ├── with-scan &1 diff --git a/pkg/workload/tpcc/ddls.go b/pkg/workload/tpcc/ddls.go index 64d394c906ac..01874030fbdd 100644 --- a/pkg/workload/tpcc/ddls.go +++ b/pkg/workload/tpcc/ddls.go @@ -154,7 +154,7 @@ const ( s_remote_cnt integer, s_data varchar(50), primary key (s_w_id, s_i_id)` - tpccStockSchemaFkSuffix = ` + deprecatedTpccStockSchemaFkSuffix = ` index stock_item_fk_idx (s_i_id)` tpccStockSchemaInterleaveSuffix = ` interleave in parent warehouse (s_w_id)` diff --git a/pkg/workload/tpcc/partition.go b/pkg/workload/tpcc/partition.go index cd46cc7f6881..dcdd2bcc1e5e 100644 --- a/pkg/workload/tpcc/partition.go +++ b/pkg/workload/tpcc/partition.go @@ -347,9 +347,6 @@ func partitionOrderLine(db *gosql.DB, cfg zoneConfig, wPart *partitioner) error } func partitionStock(db *gosql.DB, cfg zoneConfig, wPart *partitioner) error { - // The stock_item_fk_idx can't be partitioned because it doesn't have a - // warehouse prefix. It's an all-around unfortunate index that we only - // need because of a restriction in SQL. See #36859 and #37255. return partitionTable(db, cfg, wPart, "stock", "s_w_id", 0) } diff --git a/pkg/workload/tpcc/tpcc.go b/pkg/workload/tpcc/tpcc.go index f5f1c2fec563..a50b97d369b8 100644 --- a/pkg/workload/tpcc/tpcc.go +++ b/pkg/workload/tpcc/tpcc.go @@ -540,9 +540,9 @@ func (w *tpcc) Tables() []workload.Table { Schema: maybeAddInterleaveSuffix( w.interleaved, maybeAddFkSuffix( - w.fks, + w.deprecatedFkIndexes, tpccStockSchemaBase, - tpccStockSchemaFkSuffix, + deprecatedTpccStockSchemaFkSuffix, ), tpccStockSchemaInterleaveSuffix, ), From a42f02614939cb7c8520c455cd48121b9e900fcc Mon Sep 17 00:00:00 2001 From: Lucy Zhang Date: Mon, 19 Oct 2020 12:15:39 -0400 Subject: [PATCH 7/7] clusterversion: remove unused timestamp-related cluster versions Remove `VersionTimeTZType` and `VersionTimePrecision`, which are unused. Release note: None --- pkg/clusterversion/cockroach_versions.go | 12 ------ pkg/clusterversion/versionkey_string.go | 54 ++++++++++++------------ 2 files changed, 26 insertions(+), 40 deletions(-) diff --git a/pkg/clusterversion/cockroach_versions.go b/pkg/clusterversion/cockroach_versions.go index 732740f176a9..6c6383580666 100644 --- a/pkg/clusterversion/cockroach_versions.go +++ b/pkg/clusterversion/cockroach_versions.go @@ -52,8 +52,6 @@ const ( VersionStatementDiagnosticsSystemTables VersionSchemaChangeJob VersionSavepoints - VersionTimeTZType - VersionTimePrecision Version20_1 VersionStart20_2 VersionGeospatialType @@ -293,16 +291,6 @@ var versionsSingleton = keyedVersions([]keyedVersion{ Key: VersionSavepoints, Version: roachpb.Version{Major: 19, Minor: 2, Unstable: 16}, }, - { - // VersionTimeTZType enables the use of the TimeTZ data type. - Key: VersionTimeTZType, - Version: roachpb.Version{Major: 19, Minor: 2, Unstable: 17}, - }, - { - // VersionTimePrecision enables the use of precision with time/intervals. - Key: VersionTimePrecision, - Version: roachpb.Version{Major: 19, Minor: 2, Unstable: 18}, - }, { // Version20_1 is CockroachDB v20.1. It's used for all v20.1.x patch releases. Key: Version20_1, diff --git a/pkg/clusterversion/versionkey_string.go b/pkg/clusterversion/versionkey_string.go index d51390813a63..e910d0f23e65 100644 --- a/pkg/clusterversion/versionkey_string.go +++ b/pkg/clusterversion/versionkey_string.go @@ -28,37 +28,35 @@ func _() { _ = x[VersionStatementDiagnosticsSystemTables-17] _ = x[VersionSchemaChangeJob-18] _ = x[VersionSavepoints-19] - _ = x[VersionTimeTZType-20] - _ = x[VersionTimePrecision-21] - _ = x[Version20_1-22] - _ = x[VersionStart20_2-23] - _ = x[VersionGeospatialType-24] - _ = x[VersionEnums-25] - _ = x[VersionRangefeedLeases-26] - _ = x[VersionAlterColumnTypeGeneral-27] - _ = x[VersionAlterSystemJobsAddCreatedByColumns-28] - _ = x[VersionAddScheduledJobsTable-29] - _ = x[VersionUserDefinedSchemas-30] - _ = x[VersionNoOriginFKIndexes-31] - _ = x[VersionClientRangeInfosOnBatchResponse-32] - _ = x[VersionNodeMembershipStatus-33] - _ = x[VersionRangeStatsRespHasDesc-34] - _ = x[VersionMinPasswordLength-35] - _ = x[VersionAbortSpanBytes-36] - _ = x[VersionAlterSystemJobsAddSqllivenessColumnsAddNewSystemSqllivenessTable-37] - _ = x[VersionMaterializedViews-38] - _ = x[VersionBox2DType-39] - _ = x[VersionLeasedDatabaseDescriptors-40] - _ = x[VersionUpdateScheduledJobsSchema-41] - _ = x[VersionCreateLoginPrivilege-42] - _ = x[VersionHBAForNonTLS-43] - _ = x[Version20_2-44] - _ = x[VersionStart21_1-45] + _ = x[Version20_1-20] + _ = x[VersionStart20_2-21] + _ = x[VersionGeospatialType-22] + _ = x[VersionEnums-23] + _ = x[VersionRangefeedLeases-24] + _ = x[VersionAlterColumnTypeGeneral-25] + _ = x[VersionAlterSystemJobsAddCreatedByColumns-26] + _ = x[VersionAddScheduledJobsTable-27] + _ = x[VersionUserDefinedSchemas-28] + _ = x[VersionNoOriginFKIndexes-29] + _ = x[VersionClientRangeInfosOnBatchResponse-30] + _ = x[VersionNodeMembershipStatus-31] + _ = x[VersionRangeStatsRespHasDesc-32] + _ = x[VersionMinPasswordLength-33] + _ = x[VersionAbortSpanBytes-34] + _ = x[VersionAlterSystemJobsAddSqllivenessColumnsAddNewSystemSqllivenessTable-35] + _ = x[VersionMaterializedViews-36] + _ = x[VersionBox2DType-37] + _ = x[VersionLeasedDatabaseDescriptors-38] + _ = x[VersionUpdateScheduledJobsSchema-39] + _ = x[VersionCreateLoginPrivilege-40] + _ = x[VersionHBAForNonTLS-41] + _ = x[Version20_2-42] + _ = x[VersionStart21_1-43] } -const _VersionKey_name = "Version19_1VersionAtomicChangeReplicasTriggerVersionAtomicChangeReplicasVersionPartitionedBackupVersion19_2VersionStart20_1VersionContainsEstimatesCounterVersionChangeReplicasDemotionVersionSecondaryIndexColumnFamiliesVersionNamespaceTableWithSchemasVersionProtectedTimestampsVersionPrimaryKeyChangesVersionAuthLocalAndTrustRejectMethodsVersionPrimaryKeyColumnsOutOfFamilyZeroVersionNoExplicitForeignKeyIndexIDsVersionHashShardedIndexesVersionCreateRolePrivilegeVersionStatementDiagnosticsSystemTablesVersionSchemaChangeJobVersionSavepointsVersionTimeTZTypeVersionTimePrecisionVersion20_1VersionStart20_2VersionGeospatialTypeVersionEnumsVersionRangefeedLeasesVersionAlterColumnTypeGeneralVersionAlterSystemJobsAddCreatedByColumnsVersionAddScheduledJobsTableVersionUserDefinedSchemasVersionNoOriginFKIndexesVersionClientRangeInfosOnBatchResponseVersionNodeMembershipStatusVersionRangeStatsRespHasDescVersionMinPasswordLengthVersionAbortSpanBytesVersionAlterSystemJobsAddSqllivenessColumnsAddNewSystemSqllivenessTableVersionMaterializedViewsVersionBox2DTypeVersionLeasedDatabaseDescriptorsVersionUpdateScheduledJobsSchemaVersionCreateLoginPrivilegeVersionHBAForNonTLSVersion20_2VersionStart21_1" +const _VersionKey_name = "Version19_1VersionAtomicChangeReplicasTriggerVersionAtomicChangeReplicasVersionPartitionedBackupVersion19_2VersionStart20_1VersionContainsEstimatesCounterVersionChangeReplicasDemotionVersionSecondaryIndexColumnFamiliesVersionNamespaceTableWithSchemasVersionProtectedTimestampsVersionPrimaryKeyChangesVersionAuthLocalAndTrustRejectMethodsVersionPrimaryKeyColumnsOutOfFamilyZeroVersionNoExplicitForeignKeyIndexIDsVersionHashShardedIndexesVersionCreateRolePrivilegeVersionStatementDiagnosticsSystemTablesVersionSchemaChangeJobVersionSavepointsVersion20_1VersionStart20_2VersionGeospatialTypeVersionEnumsVersionRangefeedLeasesVersionAlterColumnTypeGeneralVersionAlterSystemJobsAddCreatedByColumnsVersionAddScheduledJobsTableVersionUserDefinedSchemasVersionNoOriginFKIndexesVersionClientRangeInfosOnBatchResponseVersionNodeMembershipStatusVersionRangeStatsRespHasDescVersionMinPasswordLengthVersionAbortSpanBytesVersionAlterSystemJobsAddSqllivenessColumnsAddNewSystemSqllivenessTableVersionMaterializedViewsVersionBox2DTypeVersionLeasedDatabaseDescriptorsVersionUpdateScheduledJobsSchemaVersionCreateLoginPrivilegeVersionHBAForNonTLSVersion20_2VersionStart21_1" -var _VersionKey_index = [...]uint16{0, 11, 45, 72, 96, 107, 123, 154, 183, 218, 250, 276, 300, 337, 376, 411, 436, 462, 501, 523, 540, 557, 577, 588, 604, 625, 637, 659, 688, 729, 757, 782, 806, 844, 871, 899, 923, 944, 1015, 1039, 1055, 1087, 1119, 1146, 1165, 1176, 1192} +var _VersionKey_index = [...]uint16{0, 11, 45, 72, 96, 107, 123, 154, 183, 218, 250, 276, 300, 337, 376, 411, 436, 462, 501, 523, 540, 551, 567, 588, 600, 622, 651, 692, 720, 745, 769, 807, 834, 862, 886, 907, 978, 1002, 1018, 1050, 1082, 1109, 1128, 1139, 1155} func (i VersionKey) String() string { if i < 0 || i >= VersionKey(len(_VersionKey_index)-1) {