diff --git a/pkg/gen/protobuf.bzl b/pkg/gen/protobuf.bzl index df87c11dff58..92ea6d55516c 100644 --- a/pkg/gen/protobuf.bzl +++ b/pkg/gen/protobuf.bzl @@ -38,6 +38,7 @@ PROTOBUF_SRCS = [ "//pkg/sql/contentionpb:contentionpb_go_proto", "//pkg/sql/execinfrapb:execinfrapb_go_proto", "//pkg/sql/inverted:inverted_go_proto", + "//pkg/sql/lex:lex_go_proto", "//pkg/sql/pgwire/pgerror:pgerror_go_proto", "//pkg/sql/protoreflect/test:protoreflecttest_go_proto", "//pkg/sql/rowenc/rowencpb:rowencpb_go_proto", diff --git a/pkg/sql/BUILD.bazel b/pkg/sql/BUILD.bazel index 76d8272c53be..1837c8679619 100644 --- a/pkg/sql/BUILD.bazel +++ b/pkg/sql/BUILD.bazel @@ -328,6 +328,7 @@ go_library( "//pkg/sql/gcjob/gcjobnotifier", "//pkg/sql/idxusage", "//pkg/sql/inverted", + "//pkg/sql/lex", "//pkg/sql/lexbase", "//pkg/sql/memsize", "//pkg/sql/mutations", diff --git a/pkg/sql/add_column.go b/pkg/sql/add_column.go index f492056a8c1f..b15e9a4d6c75 100644 --- a/pkg/sql/add_column.go +++ b/pkg/sql/add_column.go @@ -49,10 +49,6 @@ func (p *planner) addColumnImpl( ) } - if err := checkTypeIsSupported(params.ctx, params.ExecCfg().Settings, toType); err != nil { - return err - } - var colOwnedSeqDesc *tabledesc.Mutable newDef, seqPrefix, seqName, seqOpts, err := params.p.processSerialLikeInColumnDef(params.ctx, d, tn) if err != nil { diff --git a/pkg/sql/alter_column_type.go b/pkg/sql/alter_column_type.go index 7b6fa2eb8173..27f6a6a22ed9 100644 --- a/pkg/sql/alter_column_type.go +++ b/pkg/sql/alter_column_type.go @@ -82,10 +82,6 @@ func AlterColumnType( return err } - if err := checkTypeIsSupported(ctx, params.ExecCfg().Settings, typ); err != nil { - return err - } - // Special handling for STRING COLLATE xy to verify that we recognize the language. if t.Collation != "" { if types.IsStringType(typ) { diff --git a/pkg/sql/create_table.go b/pkg/sql/create_table.go index aa25162b61f6..366419052d97 100644 --- a/pkg/sql/create_table.go +++ b/pkg/sql/create_table.go @@ -1529,9 +1529,6 @@ func NewTableDesc( ) } } - if err := checkTypeIsSupported(ctx, st, defType); err != nil { - return nil, err - } if d.PrimaryKey.Sharded { if n.PartitionByTable.ContainsPartitions() && !n.PartitionByTable.All { return nil, pgerror.New(pgcode.FeatureNotSupported, "hash sharded indexes cannot be explicitly partitioned") @@ -2763,18 +2760,6 @@ func regionalByRowDefaultColDef( return c } -func checkTypeIsSupported(ctx context.Context, settings *cluster.Settings, typ *types.T) error { - version := settings.Version.ActiveVersionOrEmpty(ctx) - if supported := types.IsTypeSupportedInVersion(version, typ); !supported { - return pgerror.Newf( - pgcode.FeatureNotSupported, - "type %s is not supported until version upgrade is finalized", - typ.SQLString(), - ) - } - return nil -} - // setSequenceOwner adds sequence id to the sequence id list owned by a column // and set ownership values of sequence options. func setSequenceOwner( diff --git a/pkg/sql/exec_util.go b/pkg/sql/exec_util.go index def33aefaa57..8bee02199c27 100644 --- a/pkg/sql/exec_util.go +++ b/pkg/sql/exec_util.go @@ -64,6 +64,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/sql/gcjob/gcjobnotifier" + "github.com/cockroachdb/cockroach/pkg/sql/lex" "github.com/cockroachdb/cockroach/pkg/sql/opt" "github.com/cockroachdb/cockroach/pkg/sql/optionalnodeliveness" "github.com/cockroachdb/cockroach/pkg/sql/parser" @@ -2806,7 +2807,7 @@ func (m *sessionDataMutator) SetAvoidBuffering(b bool) { m.data.AvoidBuffering = b } -func (m *sessionDataMutator) SetBytesEncodeFormat(val sessiondatapb.BytesEncodeFormat) { +func (m *sessionDataMutator) SetBytesEncodeFormat(val lex.BytesEncodeFormat) { m.data.DataConversionConfig.BytesEncodeFormat = val } diff --git a/pkg/sql/lex/BUILD.bazel b/pkg/sql/lex/BUILD.bazel index c5523afe3b6a..25b7b0463f4e 100644 --- a/pkg/sql/lex/BUILD.bazel +++ b/pkg/sql/lex/BUILD.bazel @@ -1,14 +1,16 @@ +load("@rules_proto//proto:defs.bzl", "proto_library") +load("@io_bazel_rules_go//proto:def.bzl", "go_proto_library") load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") go_library( name = "lex", srcs = ["encode.go"], + embed = [":lex_go_proto"], importpath = "github.com/cockroachdb/cockroach/pkg/sql/lex", visibility = ["//visibility:public"], deps = [ "//pkg/sql/pgwire/pgcode", "//pkg/sql/pgwire/pgerror", - "//pkg/sql/sessiondatapb", "@com_github_cockroachdb_errors//:errors", "@org_golang_x_text//language", ], @@ -16,9 +18,27 @@ go_library( go_test( name = "lex_test", - srcs = ["encode_test.go"], - deps = [ - ":lex", - "//pkg/sql/sessiondatapb", + srcs = [ + "dep_test.go", + "encode_test.go", ], + embed = [":lex"], + deps = ["//pkg/testutils/buildutil"], +) + +proto_library( + name = "lex_proto", + srcs = ["encode.proto"], + strip_import_prefix = "/pkg", + visibility = ["//visibility:public"], + deps = ["@com_github_gogo_protobuf//gogoproto:gogo_proto"], +) + +go_proto_library( + name = "lex_go_proto", + compilers = ["//pkg/cmd/protoc-gen-gogoroach:protoc-gen-gogoroach_compiler"], + importpath = "github.com/cockroachdb/cockroach/pkg/sql/lex", + proto = ":lex_proto", + visibility = ["//visibility:public"], + deps = ["@com_github_gogo_protobuf//gogoproto"], ) diff --git a/pkg/sql/lex/dep_test.go b/pkg/sql/lex/dep_test.go new file mode 100644 index 000000000000..b6f5d9b4ed7a --- /dev/null +++ b/pkg/sql/lex/dep_test.go @@ -0,0 +1,26 @@ +// Copyright 2022 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 lex + +import ( + "testing" + + "github.com/cockroachdb/cockroach/pkg/testutils/buildutil" +) + +func TestNoLinkForbidden(t *testing.T) { + buildutil.VerifyNoImports(t, + "github.com/cockroachdb/cockroach/pkg/sql/lex", true, + []string{ + "github.com/cockroachdb/cockroach/pkg/sql/sessiondatapb", + }, nil, + ) +} diff --git a/pkg/sql/lex/encode.go b/pkg/sql/lex/encode.go index ee0fa03bdcc9..e02b9effdd93 100644 --- a/pkg/sql/lex/encode.go +++ b/pkg/sql/lex/encode.go @@ -23,11 +23,12 @@ import ( "bytes" "encoding/base64" "encoding/hex" + "fmt" + "strings" "unicode" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" - "github.com/cockroachdb/cockroach/pkg/sql/sessiondatapb" "github.com/cockroachdb/errors" "golang.org/x/text/language" ) @@ -92,11 +93,9 @@ func LocaleNamesAreEqual(a, b string) bool { // If the skipHexPrefix argument is set, the hexadecimal encoding does not // prefix the output with "\x". This is suitable e.g. for the encode() // built-in. -func EncodeByteArrayToRawBytes( - data string, be sessiondatapb.BytesEncodeFormat, skipHexPrefix bool, -) string { +func EncodeByteArrayToRawBytes(data string, be BytesEncodeFormat, skipHexPrefix bool) string { switch be { - case sessiondatapb.BytesEncodeHex: + case BytesEncodeHex: head := 2 if skipHexPrefix { head = 0 @@ -109,7 +108,7 @@ func EncodeByteArrayToRawBytes( hex.Encode(res[head:], []byte(data)) return string(res) - case sessiondatapb.BytesEncodeEscape: + case BytesEncodeEscape: // PostgreSQL does not allow all the escapes formats recognized by // CockroachDB's scanner. It only recognizes octal and \\ for the // backslash itself. @@ -131,7 +130,7 @@ func EncodeByteArrayToRawBytes( } return string(res) - case sessiondatapb.BytesEncodeBase64: + case BytesEncodeBase64: return base64.StdEncoding.EncodeToString([]byte(data)) default: @@ -144,12 +143,12 @@ func EncodeByteArrayToRawBytes( // When using the Hex format, the caller is responsible for skipping the // "\x" prefix, if any. See DecodeRawBytesToByteArrayAuto() below for // an alternative. -func DecodeRawBytesToByteArray(data string, be sessiondatapb.BytesEncodeFormat) ([]byte, error) { +func DecodeRawBytesToByteArray(data string, be BytesEncodeFormat) ([]byte, error) { switch be { - case sessiondatapb.BytesEncodeHex: + case BytesEncodeHex: return hex.DecodeString(data) - case sessiondatapb.BytesEncodeEscape: + case BytesEncodeEscape: // PostgreSQL does not allow all the escapes formats recognized by // CockroachDB's scanner. It only recognizes octal and \\ for the // backslash itself. @@ -188,7 +187,7 @@ func DecodeRawBytesToByteArray(data string, be sessiondatapb.BytesEncodeFormat) } return res, nil - case sessiondatapb.BytesEncodeBase64: + case BytesEncodeBase64: return base64.StdEncoding.DecodeString(data) default: @@ -201,7 +200,34 @@ func DecodeRawBytesToByteArray(data string, be sessiondatapb.BytesEncodeFormat) // and escape. func DecodeRawBytesToByteArrayAuto(data []byte) ([]byte, error) { if len(data) >= 2 && data[0] == '\\' && (data[1] == 'x' || data[1] == 'X') { - return DecodeRawBytesToByteArray(string(data[2:]), sessiondatapb.BytesEncodeHex) + return DecodeRawBytesToByteArray(string(data[2:]), BytesEncodeHex) + } + return DecodeRawBytesToByteArray(string(data), BytesEncodeEscape) +} + +func (f BytesEncodeFormat) String() string { + switch f { + case BytesEncodeHex: + return "hex" + case BytesEncodeEscape: + return "escape" + case BytesEncodeBase64: + return "base64" + default: + return fmt.Sprintf("invalid (%d)", f) + } +} + +// BytesEncodeFormatFromString converts a string into a BytesEncodeFormat. +func BytesEncodeFormatFromString(val string) (_ BytesEncodeFormat, ok bool) { + switch strings.ToUpper(val) { + case "HEX": + return BytesEncodeHex, true + case "ESCAPE": + return BytesEncodeEscape, true + case "BASE64": + return BytesEncodeBase64, true + default: + return -1, false } - return DecodeRawBytesToByteArray(string(data), sessiondatapb.BytesEncodeEscape) } diff --git a/pkg/sql/lex/encode.proto b/pkg/sql/lex/encode.proto new file mode 100644 index 000000000000..732872675c71 --- /dev/null +++ b/pkg/sql/lex/encode.proto @@ -0,0 +1,29 @@ +// Copyright 2022 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. + +syntax = "proto3"; +package cockroach.sql.sessiondatapb; +option go_package = "lex"; + +import "gogoproto/gogo.proto"; + +// BytesEncodeFormat is the configuration for bytes to string conversions. +enum BytesEncodeFormat { + option (gogoproto.goproto_enum_prefix) = false; + option (gogoproto.goproto_enum_stringer) = false; + + // BytesEncodeHex uses the hex format: e'abc\n'::BYTES::STRING -> '\x61626312'. + // This is the default, for compatibility with PostgreSQL. + BytesEncodeHex = 0; + // BytesEncodeEscape uses the escaped format: e'abc\n'::BYTES::STRING -> 'abc\012'. + BytesEncodeEscape = 1; + // BytesEncodeBase64 uses base64 encoding. + BytesEncodeBase64 = 2; +} diff --git a/pkg/sql/lex/encode_test.go b/pkg/sql/lex/encode_test.go index 4f22f8be454c..f36bfb82e683 100644 --- a/pkg/sql/lex/encode_test.go +++ b/pkg/sql/lex/encode_test.go @@ -15,19 +15,18 @@ import ( "testing" "github.com/cockroachdb/cockroach/pkg/sql/lex" - "github.com/cockroachdb/cockroach/pkg/sql/sessiondatapb" ) func TestByteArrayDecoding(t *testing.T) { const ( - fmtHex = sessiondatapb.BytesEncodeHex - fmtEsc = sessiondatapb.BytesEncodeEscape - fmtB64 = sessiondatapb.BytesEncodeBase64 + fmtHex = lex.BytesEncodeHex + fmtEsc = lex.BytesEncodeEscape + fmtB64 = lex.BytesEncodeBase64 ) testData := []struct { in string auto bool - inFmt sessiondatapb.BytesEncodeFormat + inFmt lex.BytesEncodeFormat out string err string }{ @@ -103,10 +102,10 @@ func TestByteArrayEncoding(t *testing.T) { for _, s := range testData { t.Run(s.in, func(t *testing.T) { - for _, format := range []sessiondatapb.BytesEncodeFormat{ - sessiondatapb.BytesEncodeHex, - sessiondatapb.BytesEncodeEscape, - sessiondatapb.BytesEncodeBase64, + for _, format := range []lex.BytesEncodeFormat{ + lex.BytesEncodeHex, + lex.BytesEncodeEscape, + lex.BytesEncodeBase64, } { t.Run(format.String(), func(t *testing.T) { enc := lex.EncodeByteArrayToRawBytes(s.in, format, false) @@ -116,7 +115,7 @@ func TestByteArrayEncoding(t *testing.T) { t.Fatalf("encoded %q, expected %q", enc, expEnc) } - if format == sessiondatapb.BytesEncodeHex { + if format == lex.BytesEncodeHex { // Check that the \x also can be skipped. enc2 := lex.EncodeByteArrayToRawBytes(s.in, format, true) if enc[2:] != enc2 { diff --git a/pkg/sql/schemachanger/scbuild/builder_state.go b/pkg/sql/schemachanger/scbuild/builder_state.go index 09ee68a093db..4a2515aa3781 100644 --- a/pkg/sql/schemachanger/scbuild/builder_state.go +++ b/pkg/sql/schemachanger/scbuild/builder_state.go @@ -11,11 +11,9 @@ package scbuild import ( - "context" "strings" "github.com/cockroachdb/cockroach/pkg/keys" - "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/catconstants" "github.com/cockroachdb/cockroach/pkg/sql/catalog/catpb" @@ -298,18 +296,10 @@ func (b *builderState) ResolveTypeRef(ref tree.ResolvableTypeReference) scpb.Typ if err != nil { panic(err) } - return newTypeT(b.ctx, b.clusterSettings, toType) + return newTypeT(toType) } -func newTypeT(ctx context.Context, settings *cluster.Settings, t *types.T) scpb.TypeT { - version := settings.Version.ActiveVersionOrEmpty(ctx) - supported := types.IsTypeSupportedInVersion(version, t) - if !supported { - panic(pgerror.Newf(pgcode.FeatureNotSupported, - "type %s is not supported until version upgrade is finalized", t.SQLString(), - )) - } - +func newTypeT(t *types.T) scpb.TypeT { m, err := typedesc.GetTypeDescriptorClosure(t) if err != nil { panic(err) @@ -416,7 +406,7 @@ func (b *builderState) ComputedColumnExpression( if err != nil { panic(err) } - return parsedExpr, newTypeT(b.ctx, b.clusterSettings, typ) + return parsedExpr, newTypeT(typ) } var _ scbuildstmt.ElementReferences = (*builderState)(nil) diff --git a/pkg/sql/sem/builtins/builtins.go b/pkg/sql/sem/builtins/builtins.go index d59f26e797eb..6b208ed3ce6e 100644 --- a/pkg/sql/sem/builtins/builtins.go +++ b/pkg/sql/sem/builtins/builtins.go @@ -1098,7 +1098,7 @@ var builtins = map[string]builtinDefinition{ ReturnType: tree.FixedReturnType(types.String), Fn: func(evalCtx *tree.EvalContext, args tree.Datums) (_ tree.Datum, err error) { data, format := *args[0].(*tree.DBytes), string(tree.MustBeDString(args[1])) - be, ok := sessiondatapb.BytesEncodeFormatFromString(format) + be, ok := lex.BytesEncodeFormatFromString(format) if !ok { return nil, pgerror.New(pgcode.InvalidParameterValue, "only 'hex', 'escape', and 'base64' formats are supported for encode()") @@ -1117,7 +1117,7 @@ var builtins = map[string]builtinDefinition{ ReturnType: tree.FixedReturnType(types.Bytes), Fn: func(evalCtx *tree.EvalContext, args tree.Datums) (_ tree.Datum, err error) { data, format := string(tree.MustBeDString(args[0])), string(tree.MustBeDString(args[1])) - be, ok := sessiondatapb.BytesEncodeFormatFromString(format) + be, ok := lex.BytesEncodeFormatFromString(format) if !ok { return nil, pgerror.New(pgcode.InvalidParameterValue, "only 'hex', 'escape', and 'base64' formats are supported for decode()") diff --git a/pkg/sql/sessiondatapb/BUILD.bazel b/pkg/sql/sessiondatapb/BUILD.bazel index 5f618bec4bad..e27929266ef5 100644 --- a/pkg/sql/sessiondatapb/BUILD.bazel +++ b/pkg/sql/sessiondatapb/BUILD.bazel @@ -31,6 +31,7 @@ proto_library( strip_import_prefix = "/pkg", visibility = ["//visibility:public"], deps = [ + "//pkg/sql/lex:lex_proto", "//pkg/util/duration:duration_proto", "//pkg/util/timeutil/pgdate:pgdate_proto", "@com_github_gogo_protobuf//gogoproto:gogo_proto", @@ -46,6 +47,7 @@ go_proto_library( proto = ":sessiondatapb_proto", visibility = ["//visibility:public"], deps = [ + "//pkg/sql/lex", "//pkg/util/duration", "//pkg/util/timeutil/pgdate", "@com_github_gogo_protobuf//gogoproto", diff --git a/pkg/sql/sessiondatapb/session_data.go b/pkg/sql/sessiondatapb/session_data.go index eaf8dbec3fa4..e895bbb70e3d 100644 --- a/pkg/sql/sessiondatapb/session_data.go +++ b/pkg/sql/sessiondatapb/session_data.go @@ -51,33 +51,6 @@ func (c DataConversionConfig) GetFloatPrec() int { return int(nDigits) } -func (f BytesEncodeFormat) String() string { - switch f { - case BytesEncodeHex: - return "hex" - case BytesEncodeEscape: - return "escape" - case BytesEncodeBase64: - return "base64" - default: - return fmt.Sprintf("invalid (%d)", f) - } -} - -// BytesEncodeFormatFromString converts a string into a BytesEncodeFormat. -func BytesEncodeFormatFromString(val string) (_ BytesEncodeFormat, ok bool) { - switch strings.ToUpper(val) { - case "HEX": - return BytesEncodeHex, true - case "ESCAPE": - return BytesEncodeEscape, true - case "BASE64": - return BytesEncodeBase64, true - default: - return -1, false - } -} - func (m VectorizeExecMode) String() string { switch m { case VectorizeOn, VectorizeUnset: diff --git a/pkg/sql/sessiondatapb/session_data.proto b/pkg/sql/sessiondatapb/session_data.proto index eff5503e026f..8e4529c034ea 100644 --- a/pkg/sql/sessiondatapb/session_data.proto +++ b/pkg/sql/sessiondatapb/session_data.proto @@ -12,6 +12,7 @@ syntax = "proto3"; package cockroach.sql.sessiondatapb; option go_package = "sessiondatapb"; +import "sql/lex/encode.proto"; import "util/duration/duration.proto"; import "util/timeutil/pgdate/pgdate.proto"; import "gogoproto/gogo.proto"; @@ -106,20 +107,6 @@ message DataConversionConfig { util.timeutil.pgdate.DateStyle date_style = 4 [(gogoproto.nullable) = false]; } -// BytesEncodeFormat is the configuration for bytes to string conversions. -enum BytesEncodeFormat { - option (gogoproto.goproto_enum_prefix) = false; - option (gogoproto.goproto_enum_stringer) = false; - - // BytesEncodeHex uses the hex format: e'abc\n'::BYTES::STRING -> '\x61626312'. - // This is the default, for compatibility with PostgreSQL. - BytesEncodeHex = 0; - // BytesEncodeEscape uses the escaped format: e'abc\n'::BYTES::STRING -> 'abc\012'. - BytesEncodeEscape = 1; - // BytesEncodeBase64 uses base64 encoding. - BytesEncodeBase64 = 2; -} - // VectorizeExecMode controls if an when the Executor executes queries using // the columnar execution engine. enum VectorizeExecMode { diff --git a/pkg/sql/types/BUILD.bazel b/pkg/sql/types/BUILD.bazel index 5b3a6610b486..09b896c1103e 100644 --- a/pkg/sql/types/BUILD.bazel +++ b/pkg/sql/types/BUILD.bazel @@ -6,7 +6,6 @@ go_library( name = "types", srcs = [ "alias.go", - "minimum_type_version.go", "oid.go", "testutils.go", "types.go", @@ -16,7 +15,6 @@ go_library( importpath = "github.com/cockroachdb/cockroach/pkg/sql/types", visibility = ["//visibility:public"], deps = [ - "//pkg/clusterversion", "//pkg/geo/geopb", "//pkg/sql/lex", "//pkg/sql/oidext", @@ -35,19 +33,17 @@ go_test( name = "types_test", size = "small", srcs = [ - "minimum_type_version_test.go", + "dep_test.go", "types_test.go", "types_text_marshal_test.go", ], embed = [":types"], deps = [ - "//pkg/clusterversion", "//pkg/geo/geopb", "//pkg/sql/catalog/descpb", "//pkg/sql/catalog/typedesc", "//pkg/sql/oidext", - "//pkg/util/leaktest", - "//pkg/util/log", + "//pkg/testutils/buildutil", "//pkg/util/protoutil", "@com_github_lib_pq//oid", "@com_github_stretchr_testify//assert", diff --git a/pkg/sql/types/dep_test.go b/pkg/sql/types/dep_test.go new file mode 100644 index 000000000000..152831cad951 --- /dev/null +++ b/pkg/sql/types/dep_test.go @@ -0,0 +1,26 @@ +// Copyright 2022 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 types + +import ( + "testing" + + "github.com/cockroachdb/cockroach/pkg/testutils/buildutil" +) + +func TestNoLinkForbidden(t *testing.T) { + buildutil.VerifyNoImports(t, + "github.com/cockroachdb/cockroach/pkg/sql/types", true, + []string{ + "github.com/cockroachdb/cockroach/pkg/clusterversion", + }, nil, + ) +} diff --git a/pkg/sql/types/minimum_type_version.go b/pkg/sql/types/minimum_type_version.go deleted file mode 100644 index 9c0a1b19af52..000000000000 --- a/pkg/sql/types/minimum_type_version.go +++ /dev/null @@ -1,34 +0,0 @@ -// Copyright 2021 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 types - -import "github.com/cockroachdb/cockroach/pkg/clusterversion" - -// minimumTypeUsageVersions defines the minimum version needed for a new -// data type. -// Note: please do not remove this map or IsTypeSupportedInVersion even -// if the map becomes empty temporarily. -var minimumTypeUsageVersions = map[*T]clusterversion.Key{} - -// IsTypeSupportedInVersion returns whether a given type is supported in the given version. -func IsTypeSupportedInVersion(v clusterversion.ClusterVersion, t *T) bool { - // For these checks, if we have an array, we only want to find whether - // we support the array contents. - if t.Family() == ArrayFamily { - t = t.ArrayContents() - } - - minVersion, ok := minimumTypeUsageVersions[t] - if !ok { - return true - } - return v.IsActive(minVersion) -} diff --git a/pkg/sql/types/minimum_type_version_test.go b/pkg/sql/types/minimum_type_version_test.go deleted file mode 100644 index 0d77b86b6971..000000000000 --- a/pkg/sql/types/minimum_type_version_test.go +++ /dev/null @@ -1,46 +0,0 @@ -// 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 types - -import ( - "fmt" - "testing" - - "github.com/cockroachdb/cockroach/pkg/clusterversion" - "github.com/cockroachdb/cockroach/pkg/util/leaktest" - "github.com/cockroachdb/cockroach/pkg/util/log" - "github.com/stretchr/testify/require" -) - -func TestIsTypeSupportedInVersion(t *testing.T) { - defer leaktest.AfterTest(t)() - defer log.Scope(t).Close(t) - - testCases := []struct { - v clusterversion.Key - t *T - - ok bool - }{ - {clusterversion.TODOPreV21_2, RegRole, true}, - {clusterversion.TODOPreV21_2, MakeArray(RegRole), true}, - } - - for _, tc := range testCases { - t.Run(fmt.Sprintf("%s:%s", tc.v, tc.t.SQLString()), func(t *testing.T) { - ok := IsTypeSupportedInVersion( - clusterversion.ClusterVersion{Version: clusterversion.ByKey(tc.v)}, - tc.t, - ) - require.Equal(t, tc.ok, ok) - }) - } -} diff --git a/pkg/sql/vars.go b/pkg/sql/vars.go index 775c84eda3ab..3b4d867ed1eb 100644 --- a/pkg/sql/vars.go +++ b/pkg/sql/vars.go @@ -26,6 +26,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/sql/catalog/schemaexpr" "github.com/cockroachdb/cockroach/pkg/sql/delegate" + "github.com/cockroachdb/cockroach/pkg/sql/lex" "github.com/cockroachdb/cockroach/pkg/sql/paramparse" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" @@ -199,7 +200,7 @@ var varGen = map[string]sessionVar{ Set: func( _ context.Context, m sessionDataMutator, s string, ) error { - mode, ok := sessiondatapb.BytesEncodeFormatFromString(s) + mode, ok := lex.BytesEncodeFormatFromString(s) if !ok { return newVarValueError(`bytea_output`, s, "hex", "escape", "base64") } @@ -209,7 +210,7 @@ var varGen = map[string]sessionVar{ Get: func(evalCtx *extendedEvalContext) (string, error) { return evalCtx.SessionData().DataConversionConfig.BytesEncodeFormat.String(), nil }, - GlobalDefault: func(sv *settings.Values) string { return sessiondatapb.BytesEncodeHex.String() }, + GlobalDefault: func(sv *settings.Values) string { return lex.BytesEncodeHex.String() }, }, `client_min_messages`: {