From b0ffa290563f90cf8221dcdef427c7b3b2d37d3b Mon Sep 17 00:00:00 2001 From: Oliver Tan Date: Thu, 23 Feb 2023 10:20:27 +1100 Subject: [PATCH 1/3] sql: fix inet cast to string PG adds the mask if it doesn't exist - but only on casts (not pgwire formatting). Release note (bug fix): Previously, casting an `inet` to a string type omitted the mask if a mask was not provided. This didn't match postgresql and is now resolved. --- pkg/sql/logictest/testdata/logic_test/inet | 10 ++++++++++ pkg/sql/sem/eval/cast.go | 7 ++++++- pkg/sql/sem/eval/testdata/eval/concat | 2 +- 3 files changed, 17 insertions(+), 2 deletions(-) diff --git a/pkg/sql/logictest/testdata/logic_test/inet b/pkg/sql/logictest/testdata/logic_test/inet index 1ab2d69ca617..1bef04f58893 100644 --- a/pkg/sql/logictest/testdata/logic_test/inet +++ b/pkg/sql/logictest/testdata/logic_test/inet @@ -840,11 +840,21 @@ SELECT text('10.1.0.0/16'::INET) ---- 10.1.0.0/16 +query T +SELECT '192.168.0.1'::INET::TEXT +---- +192.168.0.1/32 + query T SELECT text('192.168.0.1/16'::INET) ---- 192.168.0.1/16 +query T +SELECT '192.168.0.1/16'::INET::TEXT +---- +192.168.0.1/16 + query T SELECT text('192.168.0.1'::INET) ---- diff --git a/pkg/sql/sem/eval/cast.go b/pkg/sql/sem/eval/cast.go index 14912a92d747..2d64828d838a 100644 --- a/pkg/sql/sem/eval/cast.go +++ b/pkg/sql/sem/eval/cast.go @@ -13,6 +13,7 @@ package eval import ( "context" "math" + "strconv" "strings" "time" @@ -466,7 +467,11 @@ func performCastWithoutPrecisionTruncation( case *tree.DUuid: s = t.UUID.String() case *tree.DIPAddr: - s = tree.AsStringWithFlags(d, tree.FmtBareStrings) + s = t.IPAddr.String() + // Ensure the string has a "/mask" suffix. + if strings.IndexByte(s, '/') == -1 { + s += "/" + strconv.Itoa(int(t.IPAddr.Mask)) + } case *tree.DString: s = string(*t) case *tree.DCollatedString: diff --git a/pkg/sql/sem/eval/testdata/eval/concat b/pkg/sql/sem/eval/testdata/eval/concat index c7cbdc2b84c4..1c4943e9a27f 100644 --- a/pkg/sql/sem/eval/testdata/eval/concat +++ b/pkg/sql/sem/eval/testdata/eval/concat @@ -75,7 +75,7 @@ eval eval '3 days 2 hours'::interval || ' a ' || '127.0.0.1'::inet ---- -'3 days 02:00:00 a 127.0.0.1' +'3 days 02:00:00 a 127.0.0.1/32' eval '22:30:00'::time || ' a ' || '22:30:00-05'::timetz From 564960561edde92455da9b1613442d98484c17da Mon Sep 17 00:00:00 2001 From: Oliver Tan Date: Wed, 22 Feb 2023 12:39:29 +1100 Subject: [PATCH 2/3] builtins: automatically add builtins for each type cast In PG, casts from one type to another can also be use the function syntax, e.g. `date(now())` = `now()::date`. This is done at type resolution time. Unfortunately we do not support that in type resolution, and from experience long ago it was tricky to do so (happy to be proven wrong). This change instead defines a builtin for each castable type, which emulates the same behavior. We already kind of do this for `oid` and `inet`, so this isn't much worse right? Release note (sql change): Each type cast is now expressable as a function, e.g. `now()::date` can be expressed as `date(now())`. --- docs/generated/sql/functions.md | 6 - pkg/internal/sqlsmith/schema.go | 8 + pkg/sql/sem/builtins/BUILD.bazel | 3 + .../builtins/builtinconstants/constants.go | 1 + pkg/sql/sem/builtins/builtins.go | 34 --- pkg/sql/sem/builtins/cast_test.go | 63 ++++ pkg/sql/sem/builtins/fixed_oids.go | 286 +++++++++++++++++- pkg/sql/sem/builtins/help_test.go | 6 +- pkg/sql/sem/builtins/pg_builtins.go | 103 ++++++- pkg/sql/sem/cast/BUILD.bazel | 1 + pkg/sql/sem/cast/cast.go | 8 + pkg/sql/sem/cast/type_name.go | 38 +++ pkg/sql/sem/eval/testdata/eval/cast | 55 ++++ pkg/testutils/lint/lint_test.go | 3 + 14 files changed, 561 insertions(+), 54 deletions(-) create mode 100644 pkg/sql/sem/builtins/cast_test.go create mode 100644 pkg/sql/sem/cast/type_name.go diff --git a/docs/generated/sql/functions.md b/docs/generated/sql/functions.md index 97a71432d98e..9e80ab11bd17 100644 --- a/docs/generated/sql/functions.md +++ b/docs/generated/sql/functions.md @@ -980,8 +980,6 @@ available replica will error.

Immutable set_masklen(val: inet, prefixlen: int) → inet

Sets the prefix length of val to prefixlen.

For example, set_masklen('192.168.1.2', 16) returns '192.168.1.2/16'.

-
Immutable -text(val: inet) → string

Converts the IP address and prefix length to text.

Immutable @@ -2784,8 +2782,6 @@ The output can be used to recreate a database.’

Immutable get_byte(byte_string: bytes, index: int) → int

Extracts a byte at the given index in the byte array.

Immutable -inet(val: string) → inet

If possible, converts input to that of type inet.

-
Immutable initcap(val: string) → string

Capitalizes the first letter of val.

Immutable left(input: bytes, return_set: int) → bytes

Returns the first return_set bytes from input.

@@ -3537,8 +3533,6 @@ table. Returns an error if validation fails.

Stable obj_description(object_oid: oid, catalog_name: string) → string

Returns the comment for a database object specified by its OID and the name of the containing system catalog. For example, obj_description(123456, ‘pg_class’) would retrieve the comment for the table with OID 123456.

Stable -oid(int: int) → oid

Converts an integer to an OID.

-
Immutable pg_backend_pid() → int

Returns a numerical ID attached to this session. This ID is part of the query cancellation key used by the wire protocol. This function was only added for compatibility, and unlike in Postgres, the returned value does not correspond to a real process ID.

Stable pg_collation_for(str: anyelement) → string

Returns the collation of the argument

diff --git a/pkg/internal/sqlsmith/schema.go b/pkg/internal/sqlsmith/schema.go index c14be1661efa..5d5b82c4f4e4 100644 --- a/pkg/internal/sqlsmith/schema.go +++ b/pkg/internal/sqlsmith/schema.go @@ -495,6 +495,14 @@ var functions = func() map[tree.FunctionClass]map[oid.Oid][]function { // See #69213. continue } + + if n := tree.Name(def.Name); n.String() != def.Name { + // sqlsmith doesn't know how to quote function names, e.g. for + // the numeric cast, we need to use `"numeric"(val)`, but sqlsmith + // makes it `numeric(val)` which is incorrect. + continue + } + skip := false for _, substr := range []string{ // crdb_internal.complete_stream_ingestion_job is a stateful diff --git a/pkg/sql/sem/builtins/BUILD.bazel b/pkg/sql/sem/builtins/BUILD.bazel index 41b764f4d186..848210c71b2c 100644 --- a/pkg/sql/sem/builtins/BUILD.bazel +++ b/pkg/sql/sem/builtins/BUILD.bazel @@ -82,6 +82,7 @@ go_library( "//pkg/sql/sem/builtins/builtinconstants", "//pkg/sql/sem/builtins/builtinsregistry", "//pkg/sql/sem/builtins/pgformat", + "//pkg/sql/sem/cast", "//pkg/sql/sem/catconstants", "//pkg/sql/sem/catid", "//pkg/sql/sem/eval", @@ -155,6 +156,7 @@ go_test( "aggregate_builtins_test.go", "all_builtins_test.go", "builtins_test.go", + "cast_test.go", "datums_to_bytes_builtin_test.go", "fingerprint_builtin_test.go", "generator_builtins_test.go", @@ -187,6 +189,7 @@ go_test( "//pkg/sql/randgen", "//pkg/sql/sem/builtins/builtinconstants", "//pkg/sql/sem/builtins/builtinsregistry", + "//pkg/sql/sem/cast", "//pkg/sql/sem/eval", "//pkg/sql/sem/tree", "//pkg/sql/sem/tree/treewindow", diff --git a/pkg/sql/sem/builtins/builtinconstants/constants.go b/pkg/sql/sem/builtins/builtinconstants/constants.go index 3e9f30735e1b..e9474062119d 100644 --- a/pkg/sql/sem/builtins/builtinconstants/constants.go +++ b/pkg/sql/sem/builtins/builtinconstants/constants.go @@ -39,6 +39,7 @@ const ( CategoryArray = "Array" CategoryComparison = "Comparison" CategoryCompatibility = "Compatibility" + CategoryCast = "Cast" CategoryCrypto = "Cryptographic" CategoryDateAndTime = "Date and time" CategoryEnum = "Enum" diff --git a/pkg/sql/sem/builtins/builtins.go b/pkg/sql/sem/builtins/builtins.go index 67fe1d9e235a..21f0bde21549 100644 --- a/pkg/sql/sem/builtins/builtins.go +++ b/pkg/sql/sem/builtins/builtins.go @@ -983,24 +983,6 @@ var regularBuiltins = map[string]builtinDefinition{ }, ), - "text": makeBuiltin(defProps(), - tree.Overload{ - Types: tree.ParamTypes{{Name: "val", Typ: types.INet}}, - ReturnType: tree.FixedReturnType(types.String), - Fn: func(_ context.Context, _ *eval.Context, args tree.Datums) (tree.Datum, error) { - dIPAddr := tree.MustBeDIPAddr(args[0]) - s := dIPAddr.IPAddr.String() - // Ensure the string has a "/mask" suffix. - if strings.IndexByte(s, '/') == -1 { - s += "/" + strconv.Itoa(int(dIPAddr.Mask)) - } - return tree.NewDString(s), nil - }, - Info: "Converts the IP address and prefix length to text.", - Volatility: volatility.Immutable, - }, - ), - "inet_same_family": makeBuiltin(defProps(), tree.Overload{ Types: tree.ParamTypes{ @@ -1054,22 +1036,6 @@ var regularBuiltins = map[string]builtinDefinition{ }, ), - "inet": makeBuiltin(defProps(), - tree.Overload{ - Types: tree.ParamTypes{{Name: "val", Typ: types.String}}, - ReturnType: tree.FixedReturnType(types.INet), - Fn: func(ctx context.Context, evalCtx *eval.Context, args tree.Datums) (tree.Datum, error) { - inet, err := eval.PerformCast(ctx, evalCtx, args[0], types.INet) - if err != nil { - return nil, pgerror.WithCandidateCode(err, pgcode.InvalidTextRepresentation) - } - return inet, nil - }, - Info: "If possible, converts input to that of type inet.", - Volatility: volatility.Immutable, - }, - ), - "from_ip": makeBuiltin(defProps(), tree.Overload{ Types: tree.ParamTypes{{Name: "val", Typ: types.Bytes}}, diff --git a/pkg/sql/sem/builtins/cast_test.go b/pkg/sql/sem/builtins/cast_test.go new file mode 100644 index 000000000000..6e95b72ce4b6 --- /dev/null +++ b/pkg/sql/sem/builtins/cast_test.go @@ -0,0 +1,63 @@ +// 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 builtins_test + +import ( + "context" + "fmt" + "testing" + + "github.com/cockroachdb/cockroach/pkg/base" + "github.com/cockroachdb/cockroach/pkg/sql/sem/cast" + "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" + "github.com/cockroachdb/cockroach/pkg/sql/sem/volatility" + "github.com/cockroachdb/cockroach/pkg/sql/types" + "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" + "github.com/cockroachdb/cockroach/pkg/util/leaktest" + "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/lib/pq/oid" + "github.com/stretchr/testify/require" +) + +// TestCastBuiltins sanity checks all casts for the cast map work. +// Note we don't have to check for families or anything crazy like we do +// for shouldMakeFromCastBuiltin. +func TestCastBuiltins(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + ctx := context.Background() + serv, sqlDB, _ := serverutils.StartServer(t, base.TestServerArgs{}) + defer serv.Stopper().Stop(ctx) + + cast.ForEachCast(func( + fromOID oid.Oid, toOID oid.Oid, castCtx cast.Context, ctxOrigin cast.ContextOrigin, v volatility.V, + ) { + fromTyp, ok := types.OidToType[fromOID] + if !ok { + return + } + // Cannot cast as tuple. + if fromTyp.Family() == types.TupleFamily { + return + } + toTyp, ok := types.OidToType[toOID] + if !ok { + return + } + toName := tree.Name(cast.CastTypeName(toTyp)) + q := fmt.Sprintf("SELECT %s(NULL::%s)", toName.String(), fromTyp.String()) + t.Run(q, func(t *testing.T) { + _, err := sqlDB.Exec(q) + require.NoError(t, err) + }) + }) +} diff --git a/pkg/sql/sem/builtins/fixed_oids.go b/pkg/sql/sem/builtins/fixed_oids.go index 837f8c75b782..852696068d9b 100644 --- a/pkg/sql/sem/builtins/fixed_oids.go +++ b/pkg/sql/sem/builtins/fixed_oids.go @@ -859,7 +859,7 @@ var builtinOidsArray = []string{ 878: `masklen(val: inet) -> int`, 879: `netmask(val: inet) -> inet`, 880: `set_masklen(val: inet, prefixlen: int) -> inet`, - 881: `text(val: inet) -> string`, + 881: `text(inet: inet) -> string`, 882: `inet_same_family(val: inet, val: inet) -> bool`, 883: `inet_contained_by_or_equals(val: inet, container: inet) -> bool`, 884: `inet_contains_or_equals(container: inet, val: inet) -> bool`, @@ -2051,6 +2051,290 @@ var builtinOidsArray = []string{ 2076: `plainto_tsquery(config: string, text: string) -> tsquery`, 2077: `ts_parse(parser_name: string, document: string) -> tuple{int AS tokid, string AS token}`, 2078: `crdb_internal.humanize_bytes(val: int) -> string`, + 2079: `geography(jsonb: jsonb) -> geography`, + 2080: `geography(string: string) -> geography`, + 2081: `timetz(time: time) -> timetz`, + 2082: `timetz(timestamptz: timestamptz) -> timetz`, + 2083: `timetz(timetz: timetz) -> timetz`, + 2084: `timetz(string: string) -> timetz`, + 2085: `int8(bit: bit) -> int`, + 2086: `int8(interval: interval) -> int`, + 2087: `int8(decimal: decimal) -> int`, + 2088: `int8(date: date) -> int`, + 2089: `int8(oid: oid) -> int`, + 2090: `int8(bool: bool) -> int`, + 2091: `int8(int: int) -> int`, + 2092: `int8(timestamp: timestamp) -> int`, + 2093: `int8(timestamptz: timestamptz) -> int`, + 2094: `int8(float: float) -> int`, + 2095: `int8(jsonb: jsonb) -> int`, + 2096: `int8(string: string) -> int`, + 2097: `timestamp(date: date) -> timestamp`, + 2098: `timestamp(int: int) -> timestamp`, + 2099: `timestamp(timestamp: timestamp) -> timestamp`, + 2100: `timestamp(timestamptz: timestamptz) -> timestamp`, + 2101: `timestamp(string: string) -> timestamp`, + 2102: `bit(bit: bit) -> bit`, + 2103: `bit(int: int) -> bit`, + 2104: `bit(string: string) -> bit`, + 2105: `float8(interval: interval) -> float`, + 2106: `float8(decimal: decimal) -> float`, + 2107: `float8(date: date) -> float`, + 2108: `float8(bool: bool) -> float`, + 2109: `float8(int: int) -> float`, + 2110: `float8(timestamp: timestamp) -> float`, + 2111: `float8(timestamptz: timestamptz) -> float`, + 2112: `float8(float: float) -> float`, + 2113: `float8(jsonb: jsonb) -> float`, + 2114: `float8(string: string) -> float`, + 2115: `tsquery(string: string) -> tsquery`, + 2116: `tsquery(tsquery: tsquery) -> tsquery`, + 2117: `inet(string: string) -> inet`, + 2118: `inet(inet: inet) -> inet`, + 2119: `char(uuid: uuid) -> "char"`, + 2120: `char(bit: bit) -> "char"`, + 2121: `char(box2d: box2d) -> "char"`, + 2122: `char(interval: interval) -> "char"`, + 2123: `char(decimal: decimal) -> "char"`, + 2124: `char(tuple: tuple) -> "char"`, + 2125: `char(date: date) -> "char"`, + 2126: `char(oid: oid) -> "char"`, + 2127: `char(geometry: geometry) -> "char"`, + 2128: `char(time: time) -> "char"`, + 2129: `char(tsvector: tsvector) -> "char"`, + 2130: `char(bool: bool) -> "char"`, + 2131: `char(int: int) -> "char"`, + 2132: `char(timestamp: timestamp) -> "char"`, + 2133: `char(timestamptz: timestamptz) -> "char"`, + 2134: `char(tsquery: tsquery) -> "char"`, + 2135: `char(float: float) -> "char"`, + 2136: `char(geography: geography) -> "char"`, + 2137: `char(inet: inet) -> "char"`, + 2138: `char(timetz: timetz) -> "char"`, + 2139: `char(void: void) -> "char"`, + 2140: `char(bytes: bytes) -> "char"`, + 2141: `char(jsonb: jsonb) -> "char"`, + 2142: `char(string: string) -> "char"`, + 2143: `bytes(uuid: uuid) -> bytes`, + 2144: `bytes(geometry: geometry) -> bytes`, + 2145: `bytes(geography: geography) -> bytes`, + 2146: `bytes(string: string) -> bytes`, + 2147: `bytes(bytes: bytes) -> bytes`, + 2148: `int4(bit: bit) -> int4`, + 2149: `int4(interval: interval) -> int4`, + 2150: `int4(decimal: decimal) -> int4`, + 2151: `int4(date: date) -> int4`, + 2152: `int4(oid: oid) -> int4`, + 2153: `int4(bool: bool) -> int4`, + 2154: `int4(timestamp: timestamp) -> int4`, + 2155: `int4(timestamptz: timestamptz) -> int4`, + 2156: `int4(float: float) -> int4`, + 2157: `int4(jsonb: jsonb) -> int4`, + 2158: `int4(string: string) -> int4`, + 2159: `int4(int: int) -> int4`, + 2160: `geometry(box2d: box2d) -> geometry`, + 2161: `geometry(geometry: geometry) -> geometry`, + 2162: `geometry(geography: geography) -> geometry`, + 2163: `geometry(bytes: bytes) -> geometry`, + 2164: `geometry(jsonb: jsonb) -> geometry`, + 2165: `geometry(string: string) -> geometry`, + 2166: `float4(interval: interval) -> float4`, + 2167: `float4(decimal: decimal) -> float4`, + 2168: `float4(date: date) -> float4`, + 2169: `float4(bool: bool) -> float4`, + 2170: `float4(int: int) -> float4`, + 2171: `float4(timestamp: timestamp) -> float4`, + 2172: `float4(timestamptz: timestamptz) -> float4`, + 2173: `float4(jsonb: jsonb) -> float4`, + 2174: `float4(string: string) -> float4`, + 2175: `float4(float: float) -> float4`, + 2176: `regprocedure(oid: oid) -> regprocedure`, + 2177: `regprocedure(int: int) -> regprocedure`, + 2178: `regprocedure(string: string) -> regprocedure`, + 2179: `regtype(oid: oid) -> regtype`, + 2180: `regtype(int: int) -> regtype`, + 2181: `regtype(string: string) -> regtype`, + 2182: `text(uuid: uuid) -> string`, + 2183: `text(bit: bit) -> string`, + 2184: `text(box2d: box2d) -> string`, + 2185: `text(interval: interval) -> string`, + 2186: `text(decimal: decimal) -> string`, + 2187: `text(tuple: tuple) -> string`, + 2188: `text(date: date) -> string`, + 2189: `text(oid: oid) -> string`, + 2190: `text(geometry: geometry) -> string`, + 2191: `text(time: time) -> string`, + 2192: `text(tsvector: tsvector) -> string`, + 2193: `text(bool: bool) -> string`, + 2194: `text(int: int) -> string`, + 2195: `text(timestamp: timestamp) -> string`, + 2196: `text(timestamptz: timestamptz) -> string`, + 2197: `text(tsquery: tsquery) -> string`, + 2198: `text(float: float) -> string`, + 2199: `text(geography: geography) -> string`, + 2201: `text(timetz: timetz) -> string`, + 2202: `text(void: void) -> string`, + 2203: `text(bytes: bytes) -> string`, + 2204: `text(jsonb: jsonb) -> string`, + 2205: `text(string: string) -> string`, + 2206: `varchar(uuid: uuid) -> varchar`, + 2207: `varchar(bit: bit) -> varchar`, + 2208: `varchar(box2d: box2d) -> varchar`, + 2209: `varchar(interval: interval) -> varchar`, + 2210: `varchar(decimal: decimal) -> varchar`, + 2211: `varchar(tuple: tuple) -> varchar`, + 2212: `varchar(date: date) -> varchar`, + 2213: `varchar(oid: oid) -> varchar`, + 2214: `varchar(geometry: geometry) -> varchar`, + 2215: `varchar(time: time) -> varchar`, + 2216: `varchar(tsvector: tsvector) -> varchar`, + 2217: `varchar(bool: bool) -> varchar`, + 2218: `varchar(int: int) -> varchar`, + 2219: `varchar(timestamp: timestamp) -> varchar`, + 2220: `varchar(timestamptz: timestamptz) -> varchar`, + 2221: `varchar(tsquery: tsquery) -> varchar`, + 2222: `varchar(float: float) -> varchar`, + 2223: `varchar(geography: geography) -> varchar`, + 2224: `varchar(inet: inet) -> varchar`, + 2225: `varchar(timetz: timetz) -> varchar`, + 2226: `varchar(void: void) -> varchar`, + 2227: `varchar(bytes: bytes) -> varchar`, + 2228: `varchar(jsonb: jsonb) -> varchar`, + 2229: `varchar(string: string) -> varchar`, + 2230: `void(string: string) -> void`, + 2231: `void(void: void) -> void`, + 2232: `regnamespace(oid: oid) -> regnamespace`, + 2233: `regnamespace(int: int) -> regnamespace`, + 2234: `regnamespace(string: string) -> regnamespace`, + 2235: `regclass(oid: oid) -> regclass`, + 2236: `regclass(int: int) -> regclass`, + 2237: `regclass(string: string) -> regclass`, + 2238: `tsvector(string: string) -> tsvector`, + 2239: `tsvector(tsvector: tsvector) -> tsvector`, + 2240: `record(string: string) -> anyelement`, + 2241: `record(tuple: tuple) -> anyelement`, + 2242: `varbit(bit: bit) -> varbit`, + 2243: `varbit(string: string) -> varbit`, + 2244: `timestamptz(date: date) -> timestamptz`, + 2245: `timestamptz(int: int) -> timestamptz`, + 2246: `timestamptz(timestamp: timestamp) -> timestamptz`, + 2247: `timestamptz(timestamptz: timestamptz) -> timestamptz`, + 2248: `timestamptz(string: string) -> timestamptz`, + 2249: `regrole(oid: oid) -> regrole`, + 2250: `regrole(int: int) -> regrole`, + 2251: `regrole(string: string) -> regrole`, + 2252: `jsonb(geometry: geometry) -> jsonb`, + 2253: `jsonb(geography: geography) -> jsonb`, + 2254: `jsonb(string: string) -> jsonb`, + 2255: `jsonb(jsonb: jsonb) -> jsonb`, + 2256: `oid(int: int) -> oid`, + 2257: `oid(string: string) -> oid`, + 2258: `oid(oid: oid) -> oid`, + 2259: `name(uuid: uuid) -> name`, + 2260: `name(bit: bit) -> name`, + 2261: `name(box2d: box2d) -> name`, + 2262: `name(interval: interval) -> name`, + 2263: `name(decimal: decimal) -> name`, + 2264: `name(tuple: tuple) -> name`, + 2265: `name(date: date) -> name`, + 2266: `name(oid: oid) -> name`, + 2267: `name(geometry: geometry) -> name`, + 2268: `name(time: time) -> name`, + 2269: `name(tsvector: tsvector) -> name`, + 2270: `name(bool: bool) -> name`, + 2271: `name(int: int) -> name`, + 2272: `name(timestamp: timestamp) -> name`, + 2273: `name(timestamptz: timestamptz) -> name`, + 2274: `name(tsquery: tsquery) -> name`, + 2275: `name(float: float) -> name`, + 2276: `name(geography: geography) -> name`, + 2277: `name(inet: inet) -> name`, + 2278: `name(timetz: timetz) -> name`, + 2279: `name(void: void) -> name`, + 2280: `name(bytes: bytes) -> name`, + 2281: `name(jsonb: jsonb) -> name`, + 2282: `name(string: string) -> name`, + 2283: `time(interval: interval) -> time`, + 2284: `time(time: time) -> time`, + 2285: `time(timestamp: timestamp) -> time`, + 2286: `time(timestamptz: timestamptz) -> time`, + 2287: `time(timetz: timetz) -> time`, + 2288: `time(string: string) -> time`, + 2289: `bool(decimal: decimal) -> bool`, + 2290: `bool(int: int) -> bool`, + 2291: `bool(float: float) -> bool`, + 2292: `bool(jsonb: jsonb) -> bool`, + 2293: `bool(string: string) -> bool`, + 2294: `bool(bool: bool) -> bool`, + 2295: `uuid(bytes: bytes) -> uuid`, + 2296: `uuid(string: string) -> uuid`, + 2297: `uuid(uuid: uuid) -> uuid`, + 2298: `int2(bit: bit) -> int2`, + 2299: `int2(interval: interval) -> int2`, + 2300: `int2(decimal: decimal) -> int2`, + 2301: `int2(date: date) -> int2`, + 2302: `int2(oid: oid) -> int2`, + 2303: `int2(bool: bool) -> int2`, + 2304: `int2(int: int) -> int2`, + 2305: `int2(timestamp: timestamp) -> int2`, + 2306: `int2(timestamptz: timestamptz) -> int2`, + 2307: `int2(float: float) -> int2`, + 2308: `int2(jsonb: jsonb) -> int2`, + 2309: `int2(string: string) -> int2`, + 2310: `interval(interval: interval) -> interval`, + 2311: `interval(decimal: decimal) -> interval`, + 2312: `interval(time: time) -> interval`, + 2313: `interval(int: int) -> interval`, + 2314: `interval(float: float) -> interval`, + 2315: `interval(string: string) -> interval`, + 2316: `regproc(oid: oid) -> regproc`, + 2317: `regproc(int: int) -> regproc`, + 2318: `regproc(string: string) -> regproc`, + 2319: `date(int: int) -> date`, + 2320: `date(timestamp: timestamp) -> date`, + 2321: `date(timestamptz: timestamptz) -> date`, + 2322: `date(string: string) -> date`, + 2323: `date(date: date) -> date`, + 2324: `bpchar(uuid: uuid) -> char`, + 2325: `bpchar(bit: bit) -> char`, + 2326: `bpchar(box2d: box2d) -> char`, + 2327: `bpchar(interval: interval) -> char`, + 2328: `bpchar(decimal: decimal) -> char`, + 2329: `bpchar(tuple: tuple) -> char`, + 2330: `bpchar(date: date) -> char`, + 2331: `bpchar(oid: oid) -> char`, + 2332: `bpchar(geometry: geometry) -> char`, + 2333: `bpchar(time: time) -> char`, + 2334: `bpchar(tsvector: tsvector) -> char`, + 2335: `bpchar(bool: bool) -> char`, + 2336: `bpchar(int: int) -> char`, + 2337: `bpchar(timestamp: timestamp) -> char`, + 2338: `bpchar(timestamptz: timestamptz) -> char`, + 2339: `bpchar(tsquery: tsquery) -> char`, + 2340: `bpchar(float: float) -> char`, + 2341: `bpchar(geography: geography) -> char`, + 2342: `bpchar(inet: inet) -> char`, + 2343: `bpchar(timetz: timetz) -> char`, + 2344: `bpchar(void: void) -> char`, + 2345: `bpchar(bytes: bytes) -> char`, + 2346: `bpchar(jsonb: jsonb) -> char`, + 2347: `bpchar(string: string) -> char`, + 2348: `numeric(interval: interval) -> decimal`, + 2349: `numeric(decimal: decimal) -> decimal`, + 2350: `numeric(date: date) -> decimal`, + 2351: `numeric(bool: bool) -> decimal`, + 2352: `numeric(int: int) -> decimal`, + 2353: `numeric(timestamp: timestamp) -> decimal`, + 2354: `numeric(timestamptz: timestamptz) -> decimal`, + 2355: `numeric(float: float) -> decimal`, + 2356: `numeric(jsonb: jsonb) -> decimal`, + 2357: `numeric(string: string) -> decimal`, + 2358: `box2d(geometry: geometry) -> box2d`, + 2359: `box2d(string: string) -> box2d`, + 2360: `box2d(box2d: box2d) -> box2d`, + 2361: `geography(geometry: geometry) -> geography`, + 2362: `geography(geography: geography) -> geography`, + 2363: `geography(bytes: bytes) -> geography`, } var builtinOidsBySignature map[string]oid.Oid diff --git a/pkg/sql/sem/builtins/help_test.go b/pkg/sql/sem/builtins/help_test.go index b41a4c145d09..54530c652692 100644 --- a/pkg/sql/sem/builtins/help_test.go +++ b/pkg/sql/sem/builtins/help_test.go @@ -18,6 +18,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/parser" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" + "github.com/cockroachdb/cockroach/pkg/sql/sem/builtins/builtinconstants" "github.com/cockroachdb/cockroach/pkg/sql/sem/builtins/builtinsregistry" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/util/leaktest" @@ -27,10 +28,13 @@ func TestHelpFunctions(t *testing.T) { defer leaktest.AfterTest(t)() numTestsRun := 0 // This test checks that all the built-in functions receive contextual help. - builtinsregistry.AddSubscription(func(f string, _ *tree.FunctionProperties, _ []tree.Overload) { + builtinsregistry.AddSubscription(func(f string, prop *tree.FunctionProperties, _ []tree.Overload) { if unicode.IsUpper(rune(f[0])) { return } + if prop.Category == builtinconstants.CategoryCast { + return + } t.Run(f, func(t *testing.T) { numTestsRun++ _, err := parser.Parse("select " + f + "(??") diff --git a/pkg/sql/sem/builtins/pg_builtins.go b/pkg/sql/sem/builtins/pg_builtins.go index b01c09522535..c42654fdea24 100644 --- a/pkg/sql/sem/builtins/pg_builtins.go +++ b/pkg/sql/sem/builtins/pg_builtins.go @@ -27,6 +27,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/privilege" "github.com/cockroachdb/cockroach/pkg/sql/rowenc/valueside" "github.com/cockroachdb/cockroach/pkg/sql/sem/builtins/builtinconstants" + "github.com/cockroachdb/cockroach/pkg/sql/sem/cast" "github.com/cockroachdb/cockroach/pkg/sql/sem/catconstants" "github.com/cockroachdb/cockroach/pkg/sql/sem/catid" "github.com/cockroachdb/cockroach/pkg/sql/sem/eval" @@ -132,6 +133,76 @@ func init() { registerBuiltin(name, builtin) } + // Make type cast builtins. + // In postgresql, this is done at type resolution type - if a valid cast exists + // but used as a function, make it a cast. + // e.g. date(ts) is the same as ts::date. + castBuiltins := make(map[oid.Oid]*builtinDefinition) + cast.ForEachCast(func(fromOID oid.Oid, toOID oid.Oid, _ cast.Context, _ cast.ContextOrigin, v volatility.V) { + fromTyp, ok := types.OidToType[fromOID] + if !ok || !shouldMakeFromCastBuiltin(fromTyp) { + return + } + toType, ok := types.OidToType[toOID] + if !ok { + return + } + if _, ok := castBuiltins[toOID]; !ok { + castBuiltins[toOID] = &builtinDefinition{ + props: tree.FunctionProperties{ + Category: builtinconstants.CategoryCast, + Undocumented: true, + }, + } + } + castBuiltins[toOID].overloads = append( + castBuiltins[toOID].overloads, + tree.Overload{ + Types: tree.ParamTypes{{Name: fromTyp.String(), Typ: fromTyp}}, + ReturnType: tree.FixedReturnType(toType), + Fn: func(ctx context.Context, evalCtx *eval.Context, args tree.Datums) (tree.Datum, error) { + return eval.PerformCast(ctx, evalCtx, args[0], toType) + }, + Class: tree.NormalClass, + Info: fmt.Sprintf("Cast from %s to %s.", fromTyp.SQLString(), toType.SQLString()), + Volatility: v, + // The one for name casts differ. + // Since we're using the same one as cast, ignore that from now. + IgnoreVolatilityCheck: true, + }, + ) + }) + // Add casts between the same type. + for typOID, def := range castBuiltins { + typ := types.OidToType[typOID] + if !shouldMakeFromCastBuiltin(typ) { + continue + } + // Some casts already have been defined to deal with typmod coercion. + // Do not double add them. + if cast.OIDInCastMap(typOID, typOID) { + continue + } + def.overloads = append( + def.overloads, + tree.Overload{ + Types: tree.ParamTypes{{Name: typ.String(), Typ: typ}}, + ReturnType: tree.FixedReturnType(typ), + Fn: func(ctx context.Context, evalCtx *eval.Context, args tree.Datums) (tree.Datum, error) { + return eval.PerformCast(ctx, evalCtx, args[0], typ) + }, + Class: tree.NormalClass, + Info: fmt.Sprintf("Cast from %s to %s.", typ.SQLString(), typ.SQLString()), + Volatility: volatility.Immutable, + }, + ) + } + for toOID, def := range castBuiltins { + n := cast.CastTypeName(types.OidToType[toOID]) + CastBuiltinNames[n] = struct{}{} + registerBuiltin(n, *def) + } + // Make crdb_internal.create_regfoo and to_regfoo builtins. for _, b := range []struct { toRegOverloadHelpText string @@ -148,7 +219,27 @@ func init() { registerBuiltin("crdb_internal.create_"+typName, makeCreateRegDef(b.typ)) registerBuiltin("to_"+typName, makeToRegOverload(b.typ, b.toRegOverloadHelpText)) } +} +// CastBuiltinNames contains all cast builtin names. +var CastBuiltinNames = make(map[string]struct{}) + +func shouldMakeFromCastBuiltin(in *types.T) bool { + // Since type resolutions are based on families, prevent ambiguity where + // possible by using the "preferred" type for the family. + switch { + case in.Family() == types.OidFamily && in.Oid() != oid.T_oid: + return false + case in.Family() == types.BitFamily && in.Oid() != oid.T_bit: + return false + case in.Family() == types.StringFamily && in.Oid() != oid.T_text: + return false + case in.Family() == types.IntFamily && in.Oid() != oid.T_int8: + return false + case in.Family() == types.FloatFamily && in.Oid() != oid.T_float8: + return false + } + return true } var errUnimplemented = pgerror.New(pgcode.FeatureNotSupported, "unimplemented") @@ -998,18 +1089,6 @@ var pgBuiltins = map[string]builtinDefinition{ }, ), - "oid": makeBuiltin(defProps(), - tree.Overload{ - Types: tree.ParamTypes{{Name: "int", Typ: types.Int}}, - ReturnType: tree.FixedReturnType(types.Oid), - Fn: func(ctx context.Context, evalCtx *eval.Context, args tree.Datums) (tree.Datum, error) { - return eval.PerformCast(ctx, evalCtx, args[0], types.Oid) - }, - Info: "Converts an integer to an OID.", - Volatility: volatility.Immutable, - }, - ), - "shobj_description": makeBuiltin(defProps(), tree.Overload{ IsUDF: true, diff --git a/pkg/sql/sem/cast/BUILD.bazel b/pkg/sql/sem/cast/BUILD.bazel index 073c2ba8c2a8..873f50b187d7 100644 --- a/pkg/sql/sem/cast/BUILD.bazel +++ b/pkg/sql/sem/cast/BUILD.bazel @@ -6,6 +6,7 @@ go_library( srcs = [ "cast.go", "cast_map.go", + "type_name.go", ], importpath = "github.com/cockroachdb/cockroach/pkg/sql/sem/cast", visibility = ["//visibility:public"], diff --git a/pkg/sql/sem/cast/cast.go b/pkg/sql/sem/cast/cast.go index 5f0ea9c82b1d..85d16deb21d8 100644 --- a/pkg/sql/sem/cast/cast.go +++ b/pkg/sql/sem/cast/cast.go @@ -189,6 +189,14 @@ func ValidCast(src, tgt *types.T, ctx Context) bool { return false } +// OIDInCastMap checks to see if the cast is in the cast map. This bypasses +// a few false equivalences found in LookupCast. +// You are more likely using to use LookupCast. +func OIDInCastMap(src, tgt oid.Oid) bool { + _, ok := castMap[src][tgt] + return ok +} + // LookupCast returns a cast that describes the cast from src to tgt if it // exists. If it does not exist, ok=false is returned. func LookupCast(src, tgt *types.T) (Cast, bool) { diff --git a/pkg/sql/sem/cast/type_name.go b/pkg/sql/sem/cast/type_name.go new file mode 100644 index 000000000000..7eeb54af3cc1 --- /dev/null +++ b/pkg/sql/sem/cast/type_name.go @@ -0,0 +1,38 @@ +// 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 cast + +import ( + "strings" + + "github.com/cockroachdb/cockroach/pkg/sql/types" + "github.com/lib/pq/oid" +) + +// CastTypeName returns the name of the type used for casting. +func CastTypeName(t *types.T) string { + // SQLString is wrong for these types. + switch t.Oid() { + case oid.T_numeric: + // SQLString returns `decimal` + return "numeric" + case oid.T_char: + // SQLString returns `"char"` + return "char" + case oid.T_bpchar: + // SQLString returns `char`. + return "bpchar" + case oid.T_text: + // SQLString returns `string` + return "text" + } + return strings.ToLower(t.SQLString()) +} diff --git a/pkg/sql/sem/eval/testdata/eval/cast b/pkg/sql/sem/eval/testdata/eval/cast index 60acd46136b6..e462a3343fdd 100644 --- a/pkg/sql/sem/eval/testdata/eval/cast +++ b/pkg/sql/sem/eval/testdata/eval/cast @@ -1405,3 +1405,58 @@ eval (4294967296)::oid ---- OID out of range: 4294967296 + +eval +date('2020-05-03 01:02:03') +---- +'2020-05-03' + +eval +date('2020-05-03 01:02:03'::timestamptz) +---- +'2020-05-03' + +eval +int2(12.1) +---- +12 + +eval +int4(12.1) +---- +12 + +eval +int8(12.1) +---- +12 + +eval +"numeric"('1.1'::string) +---- +1.1 + +eval +"char"(90::int) +---- +'Z' + +eval +"bpchar"(90::int) +---- +'90' + +eval +"varbit"('010') +---- +B'010' + +eval +"bit"('010') +---- +B'010' + +eval +text(1.11::float8) +---- +'1.11' diff --git a/pkg/testutils/lint/lint_test.go b/pkg/testutils/lint/lint_test.go index 8c93b93098ad..cb7e68bd1c5a 100644 --- a/pkg/testutils/lint/lint_test.go +++ b/pkg/testutils/lint/lint_test.go @@ -137,6 +137,9 @@ func TestLint(t *testing.T) { var names []string for _, name := range builtins.AllBuiltinNames() { + if _, ok := builtins.CastBuiltinNames[name]; ok { + continue + } switch name { case "extract", "trim", "overlay", "position", "substring", "st_x", "st_y": // Exempt special forms: EXTRACT(... FROM ...), etc. From d10c3dd42c3dc40cad82792a30ae47fd2a663f43 Mon Sep 17 00:00:00 2001 From: Oliver Tan Date: Thu, 2 Mar 2023 14:15:12 +1100 Subject: [PATCH 3/3] sql: define castfunc oid in pg_cast This commit correct fills in the castfunc column in pg_catalog.pg_cast now that we have all the builtins defined. Release note: None --- .../logictest/testdata/logic_test/pg_catalog | 324 +++++++++--------- pkg/sql/pg_catalog.go | 8 +- pkg/sql/sem/builtins/all_builtins.go | 7 + pkg/sql/sem/builtins/pg_builtins.go | 5 + 4 files changed, 184 insertions(+), 160 deletions(-) diff --git a/pkg/sql/logictest/testdata/logic_test/pg_catalog b/pkg/sql/logictest/testdata/logic_test/pg_catalog index dfa2825c2b19..01cdd889c977 100644 --- a/pkg/sql/logictest/testdata/logic_test/pg_catalog +++ b/pkg/sql/logictest/testdata/logic_test/pg_catalog @@ -4273,170 +4273,176 @@ INNER LOOKUP JOIN pg_type ON pg_type.oid=a.b ## pg_cast +# This should always be empty. +query OO +SELECT castsource, casttarget FROM pg_cast WHERE castfunc IS NULL +ORDER BY oid +---- + query OOOOTT colnames SELECT * FROM pg_cast ORDER BY oid ---- oid castsource casttarget castfunc castcontext castmethod -140679991 1042 25 NULL i NULL -140679996 1042 18 NULL a NULL -140679997 1042 19 NULL i NULL -186882866 869 1042 NULL a NULL -186882867 869 1043 NULL a NULL -207790440 1042 1042 NULL i NULL -207790441 1042 1043 NULL i NULL -253993333 869 25 NULL a NULL -398529196 90002 90002 NULL i NULL -398529198 90002 90000 NULL e NULL -486164264 1266 1266 NULL i NULL -486164449 1266 1083 NULL a NULL -519779712 25 25 NULL i NULL -519779722 25 19 NULL i NULL -519779723 25 18 NULL a NULL -586890230 25 1043 NULL i NULL -586890231 25 1042 NULL i NULL -637806108 700 1700 NULL a NULL -641069276 1700 700 NULL i NULL -641069277 1700 701 NULL i NULL -654000812 25 2205 NULL i NULL -674624401 1700 23 NULL a NULL -674624402 1700 20 NULL a NULL -674624403 1700 21 NULL a NULL -704916921 700 701 NULL i NULL -705151184 3802 16 NULL e NULL -705151188 3802 20 NULL e NULL -705151189 3802 21 NULL e NULL -705151191 3802 23 NULL e NULL -708179464 1700 1700 NULL i NULL -718142737 1186 1083 NULL a NULL -718142856 1186 1186 NULL i NULL -738471997 700 23 NULL a NULL -738471998 700 20 NULL a NULL -738471999 700 21 NULL a NULL -738706266 3802 700 NULL e NULL -738706267 3802 701 NULL e NULL -805816982 3802 1700 NULL e NULL -939073229 25 90000 NULL i NULL -1019153911 90002 17 NULL i NULL -1039252266 19 25 NULL i NULL -1106362732 19 1043 NULL a NULL -1106362733 19 1042 NULL a NULL -1298988567 16 23 NULL e NULL -1298988569 16 25 NULL a NULL -1366099038 16 1042 NULL a NULL -1366099039 16 1043 NULL a NULL -1418646496 1043 1043 NULL i NULL -1418646497 1043 1042 NULL i NULL -1485756966 1043 25 NULL i NULL -1485756972 1043 19 NULL i NULL -1485756973 1043 18 NULL a NULL -1619977802 1043 2205 NULL i NULL -1646747850 26 4089 NULL i NULL -1730635912 26 2202 NULL i NULL -1730635916 26 2206 NULL i NULL -1730635919 26 2205 NULL i NULL -1864856754 26 24 NULL i NULL -1864856765 26 23 NULL a NULL -1864856766 26 20 NULL a NULL -1940039391 23 4089 NULL i NULL -1984069072 1560 1560 NULL i NULL -1984069074 1560 1562 NULL i NULL -1990371905 23 2206 NULL i NULL -1990371906 23 2205 NULL i NULL -1990371909 23 2202 NULL i NULL -2083548972 701 23 NULL a NULL -2083548974 701 21 NULL a NULL -2083548975 701 20 NULL a NULL -2084734765 1560 23 NULL e NULL -2084734766 1560 20 NULL e NULL -2091037769 23 1560 NULL e NULL -2091037941 23 1700 NULL i NULL -2117104349 701 700 NULL a NULL -2124592994 23 21 NULL a NULL -2124592995 23 20 NULL i NULL -2124592997 23 18 NULL e NULL -2124592999 23 16 NULL e NULL -2124593005 23 26 NULL i NULL -2124593007 23 24 NULL i NULL -2133298650 26 4096 NULL i NULL -2146493458 1114 1184 NULL i NULL -2146493576 1114 1082 NULL a NULL -2146493577 1114 1083 NULL a NULL -2146493672 1114 1114 NULL i NULL -2158148128 23 701 NULL i NULL -2158148129 23 700 NULL i NULL -2173360273 2205 20 NULL a NULL -2173360274 2205 23 NULL a NULL -2173360287 2205 26 NULL i NULL -2184214537 701 1700 NULL a NULL -2188735018 1184 1114 NULL a NULL -2188735050 1184 1082 NULL a NULL -2188735051 1184 1083 NULL a NULL -2188735106 1184 1266 NULL a NULL -2188735184 1184 1184 NULL i NULL -2233330912 20 4089 NULL i NULL -2250108209 20 2205 NULL i NULL -2250108210 20 2206 NULL i NULL -2250108214 20 2202 NULL i NULL -2350774022 20 1560 NULL e NULL -2350774202 20 1700 NULL i NULL -2384329297 20 21 NULL a NULL -2384329299 20 23 NULL a NULL -2384329308 20 24 NULL i NULL -2384329310 20 26 NULL i NULL -2393034919 23 4096 NULL i NULL -2417884630 20 700 NULL i NULL -2417884631 20 701 NULL i NULL -2424867656 1082 1114 NULL i NULL -2424867762 1082 1184 NULL i NULL -2433096517 2202 23 NULL a NULL -2433096518 2202 20 NULL a NULL -2433096520 2202 26 NULL i NULL -2433096522 2202 24 NULL i NULL -2456191196 90000 3802 NULL e NULL -2493265536 4089 20 NULL a NULL -2493265539 4089 23 NULL a NULL -2493265550 4089 26 NULL i NULL -2623967189 90000 17 NULL i NULL -2623967197 90000 25 NULL i NULL -2652771188 20 4096 NULL i NULL -2794916917 17 90000 NULL i NULL -2794916919 17 90002 NULL i NULL -3132647220 90004 90000 NULL i NULL -3335448938 24 2202 NULL i NULL -3460964389 21 4096 NULL i NULL -3469670034 24 26 NULL i NULL -3469670044 24 20 NULL a NULL -3469670047 24 23 NULL a NULL -3518437249 2206 23 NULL a NULL -3518437250 2206 20 NULL a NULL -3518437260 2206 26 NULL i NULL -3628740423 21 1700 NULL i NULL -3635723561 1083 1186 NULL i NULL -3635723641 1083 1266 NULL i NULL -3635723696 1083 1083 NULL i NULL -3695851154 21 701 NULL i NULL -3695851155 21 700 NULL i NULL -3729406273 21 20 NULL i NULL -3729406274 21 23 NULL i NULL -3729406285 21 24 NULL i NULL -3729406287 21 26 NULL i NULL -3746183697 21 4089 NULL i NULL -3790213776 1562 1562 NULL i NULL -3790213778 1562 1560 NULL i NULL -3815325360 90000 90000 NULL i NULL -3815325362 90000 90002 NULL i NULL -3815325364 90000 90004 NULL i NULL -3854332660 4096 20 NULL a NULL -3854332663 4096 23 NULL a NULL -3854332666 4096 26 NULL i NULL -3863627120 21 2205 NULL i NULL -3863627123 21 2206 NULL i NULL -3863627127 21 2202 NULL i NULL -3922032068 18 1042 NULL a NULL -3922032069 18 1043 NULL a NULL -3989142581 18 23 NULL e NULL -3989142587 18 25 NULL i NULL +140679991 1042 25 2205 i NULL +140679996 1042 18 2142 a NULL +140679997 1042 19 2282 i NULL +186882866 869 1042 2342 a NULL +186882867 869 1043 2224 a NULL +207790440 1042 1042 2347 i NULL +207790441 1042 1043 2229 i NULL +253993333 869 25 881 a NULL +398529196 90002 90002 2362 i NULL +398529198 90002 90000 2162 e NULL +486164264 1266 1266 2083 i NULL +486164449 1266 1083 2287 a NULL +519779712 25 25 2205 i NULL +519779722 25 19 2282 i NULL +519779723 25 18 2142 a NULL +586890230 25 1043 2229 i NULL +586890231 25 1042 2347 i NULL +637806108 700 1700 2355 a NULL +641069276 1700 700 2167 i NULL +641069277 1700 701 2106 i NULL +654000812 25 2205 2237 i NULL +674624401 1700 23 2150 a NULL +674624402 1700 20 2087 a NULL +674624403 1700 21 2300 a NULL +704916921 700 701 2112 i NULL +705151184 3802 16 2292 e NULL +705151188 3802 20 2095 e NULL +705151189 3802 21 2308 e NULL +705151191 3802 23 2157 e NULL +708179464 1700 1700 2349 i NULL +718142737 1186 1083 2283 a NULL +718142856 1186 1186 2310 i NULL +738471997 700 23 2156 a NULL +738471998 700 20 2094 a NULL +738471999 700 21 2307 a NULL +738706266 3802 700 2173 e NULL +738706267 3802 701 2113 e NULL +805816982 3802 1700 2356 e NULL +939073229 25 90000 2165 i NULL +1019153911 90002 17 2145 i NULL +1039252266 19 25 2205 i NULL +1106362732 19 1043 2229 a NULL +1106362733 19 1042 2347 a NULL +1298988567 16 23 2153 e NULL +1298988569 16 25 2193 a NULL +1366099038 16 1042 2335 a NULL +1366099039 16 1043 2217 a NULL +1418646496 1043 1043 2229 i NULL +1418646497 1043 1042 2347 i NULL +1485756966 1043 25 2205 i NULL +1485756972 1043 19 2282 i NULL +1485756973 1043 18 2142 a NULL +1619977802 1043 2205 2237 i NULL +1646747850 26 4089 2232 i NULL +1730635912 26 2202 2176 i NULL +1730635916 26 2206 2179 i NULL +1730635919 26 2205 2235 i NULL +1864856754 26 24 2316 i NULL +1864856765 26 23 2152 a NULL +1864856766 26 20 2089 a NULL +1940039391 23 4089 2233 i NULL +1984069072 1560 1560 2102 i NULL +1984069074 1560 1562 2242 i NULL +1990371905 23 2206 2180 i NULL +1990371906 23 2205 2236 i NULL +1990371909 23 2202 2177 i NULL +2083548972 701 23 2156 a NULL +2083548974 701 21 2307 a NULL +2083548975 701 20 2094 a NULL +2084734765 1560 23 2148 e NULL +2084734766 1560 20 2085 e NULL +2091037769 23 1560 2103 e NULL +2091037941 23 1700 2352 i NULL +2117104349 701 700 2175 a NULL +2124592994 23 21 2304 a NULL +2124592995 23 20 2091 i NULL +2124592997 23 18 2131 e NULL +2124592999 23 16 2290 e NULL +2124593005 23 26 2256 i NULL +2124593007 23 24 2317 i NULL +2133298650 26 4096 2249 i NULL +2146493458 1114 1184 2246 i NULL +2146493576 1114 1082 2320 a NULL +2146493577 1114 1083 2285 a NULL +2146493672 1114 1114 2099 i NULL +2158148128 23 701 2109 i NULL +2158148129 23 700 2170 i NULL +2173360273 2205 20 2089 a NULL +2173360274 2205 23 2152 a NULL +2173360287 2205 26 2258 i NULL +2184214537 701 1700 2355 a NULL +2188735018 1184 1114 2100 a NULL +2188735050 1184 1082 2321 a NULL +2188735051 1184 1083 2286 a NULL +2188735106 1184 1266 2082 a NULL +2188735184 1184 1184 2247 i NULL +2233330912 20 4089 2233 i NULL +2250108209 20 2205 2236 i NULL +2250108210 20 2206 2180 i NULL +2250108214 20 2202 2177 i NULL +2350774022 20 1560 2103 e NULL +2350774202 20 1700 2352 i NULL +2384329297 20 21 2304 a NULL +2384329299 20 23 2159 a NULL +2384329308 20 24 2317 i NULL +2384329310 20 26 2256 i NULL +2393034919 23 4096 2250 i NULL +2417884630 20 700 2170 i NULL +2417884631 20 701 2109 i NULL +2424867656 1082 1114 2097 i NULL +2424867762 1082 1184 2244 i NULL +2433096517 2202 23 2152 a NULL +2433096518 2202 20 2089 a NULL +2433096520 2202 26 2258 i NULL +2433096522 2202 24 2316 i NULL +2456191196 90000 3802 2252 e NULL +2493265536 4089 20 2089 a NULL +2493265539 4089 23 2152 a NULL +2493265550 4089 26 2258 i NULL +2623967189 90000 17 2144 i NULL +2623967197 90000 25 2190 i NULL +2652771188 20 4096 2250 i NULL +2794916917 17 90000 2163 i NULL +2794916919 17 90002 2363 i NULL +3132647220 90004 90000 2160 i NULL +3335448938 24 2202 2176 i NULL +3460964389 21 4096 2250 i NULL +3469670034 24 26 2258 i NULL +3469670044 24 20 2089 a NULL +3469670047 24 23 2152 a NULL +3518437249 2206 23 2152 a NULL +3518437250 2206 20 2089 a NULL +3518437260 2206 26 2258 i NULL +3628740423 21 1700 2352 i NULL +3635723561 1083 1186 2312 i NULL +3635723641 1083 1266 2081 i NULL +3635723696 1083 1083 2284 i NULL +3695851154 21 701 2109 i NULL +3695851155 21 700 2170 i NULL +3729406273 21 20 2091 i NULL +3729406274 21 23 2159 i NULL +3729406285 21 24 2317 i NULL +3729406287 21 26 2256 i NULL +3746183697 21 4089 2233 i NULL +3790213776 1562 1562 2242 i NULL +3790213778 1562 1560 2102 i NULL +3815325360 90000 90000 2161 i NULL +3815325362 90000 90002 2361 i NULL +3815325364 90000 90004 2358 i NULL +3854332660 4096 20 2089 a NULL +3854332663 4096 23 2152 a NULL +3854332666 4096 26 2258 i NULL +3863627120 21 2205 2236 i NULL +3863627123 21 2206 2180 i NULL +3863627127 21 2202 2177 i NULL +3922032068 18 1042 2347 a NULL +3922032069 18 1043 2229 a NULL +3989142581 18 23 2158 e NULL +3989142587 18 25 2205 i NULL subtest seq_bound_should_consistent_with_session_var diff --git a/pkg/sql/pg_catalog.go b/pkg/sql/pg_catalog.go index 96d644d67269..97edc12bdbcb 100644 --- a/pkg/sql/pg_catalog.go +++ b/pkg/sql/pg_catalog.go @@ -515,11 +515,17 @@ https://www.postgresql.org/docs/9.6/catalog-pg-cast.html`, if ctxOrigin == cast.ContextOriginPgCast { castCtx := cCtx.PGString() + castFunc := tree.DNull + if srcTyp, ok := types.OidToType[src]; ok { + if v, ok := builtins.CastBuiltinOIDs[tgt][srcTyp.Family()]; ok { + castFunc = tree.NewDOid(v) + } + } _ = addRow( h.CastOid(src, tgt), // oid tree.NewDOid(src), // cast source tree.NewDOid(tgt), // casttarget - tree.DNull, // castfunc + castFunc, // castfunc tree.NewDString(castCtx), // castcontext tree.DNull, // castmethod ) diff --git a/pkg/sql/sem/builtins/all_builtins.go b/pkg/sql/sem/builtins/all_builtins.go index b32ecff1adb3..77e2d2d54bf2 100644 --- a/pkg/sql/sem/builtins/all_builtins.go +++ b/pkg/sql/sem/builtins/all_builtins.go @@ -57,6 +57,13 @@ func init() { for i, fn := range overloads { signature := name + fn.Signature(true) overloads[i].Oid = signatureMustHaveHardcodedOID(signature) + if _, ok := CastBuiltinNames[name]; ok { + retOid := fn.ReturnType(nil).Oid() + if _, ok := CastBuiltinOIDs[retOid]; !ok { + CastBuiltinOIDs[retOid] = make(map[types.Family]oid.Oid, len(overloads)) + } + CastBuiltinOIDs[retOid][fn.Types.GetAt(0).Family()] = overloads[i].Oid + } } fDef := tree.NewFunctionDefinition(name, props, overloads) addResolvedFuncDef(tree.ResolvedBuiltinFuncDefs, tree.OidToQualifiedBuiltinOverload, fDef) diff --git a/pkg/sql/sem/builtins/pg_builtins.go b/pkg/sql/sem/builtins/pg_builtins.go index c42654fdea24..fb619322dbaa 100644 --- a/pkg/sql/sem/builtins/pg_builtins.go +++ b/pkg/sql/sem/builtins/pg_builtins.go @@ -224,6 +224,11 @@ func init() { // CastBuiltinNames contains all cast builtin names. var CastBuiltinNames = make(map[string]struct{}) +// CastBuiltinOIDs maps casts from tgt oid to src family to OIDs. +// We base the second on family as casts are only defined once per family +// in order to make type resolution non-ambiguous. +var CastBuiltinOIDs = make(map[oid.Oid]map[types.Family]oid.Oid) + func shouldMakeFromCastBuiltin(in *types.T) bool { // Since type resolutions are based on families, prevent ambiguity where // possible by using the "preferred" type for the family.