diff --git a/pkg/internal/sqlsmith/plpgsql.go b/pkg/internal/sqlsmith/plpgsql.go index 0d35b72f5f4c..47fcbae7f8dd 100644 --- a/pkg/internal/sqlsmith/plpgsql.go +++ b/pkg/internal/sqlsmith/plpgsql.go @@ -60,7 +60,7 @@ func (s *Smither) makePLpgSQLDeclarations( varName = s.name("decl") } varTyp := s.randType() - for types.IsRecordType(varTyp) || varTyp.Family() == types.CollatedStringFamily { + for varTyp.Identical(types.AnyTuple) || varTyp.Family() == types.CollatedStringFamily { // TODO(#114874): allow record types here when they are supported. // TODO(#105245): allow collated strings when they are supported. varTyp = s.randType() diff --git a/pkg/internal/sqlsmith/relational.go b/pkg/internal/sqlsmith/relational.go index 825928a4a1d8..fd89a4b3bdee 100644 --- a/pkg/internal/sqlsmith/relational.go +++ b/pkg/internal/sqlsmith/relational.go @@ -960,7 +960,7 @@ func (s *Smither) makeCreateFunc() (cf *tree.CreateRoutine, ok bool) { // TODO(#105713): lift the RECORD-type restriction. ptyp := s.randType() for ptyp.Family() == types.CollatedStringFamily || - (lang == tree.RoutineLangPLpgSQL && types.IsRecordType(ptyp)) { + (lang == tree.RoutineLangPLpgSQL && ptyp.Identical(types.AnyTuple)) { ptyp = s.randType() } pname := fmt.Sprintf("p%d", i) diff --git a/pkg/sql/catalog/funcdesc/func_desc.go b/pkg/sql/catalog/funcdesc/func_desc.go index 440f0b0721a2..5fb1c2bd8b09 100644 --- a/pkg/sql/catalog/funcdesc/func_desc.go +++ b/pkg/sql/catalog/funcdesc/func_desc.go @@ -809,9 +809,7 @@ func (desc *immutable) ToOverload() (ret *tree.Overload, err error) { ret.RoutineParams = append(ret.RoutineParams, routineParam) } ret.ReturnType = tree.FixedReturnType(desc.ReturnType.Type) - // TODO(yuzefovich): we should not be setting ReturnsRecordType to 'true' - // when the return type is based on output parameters. - ret.ReturnsRecordType = types.IsRecordType(desc.ReturnType.Type) + ret.ReturnsRecordType = desc.ReturnType.Type.Identical(types.AnyTuple) ret.Types = signatureTypes ret.Volatility, err = desc.getOverloadVolatility() if err != nil { diff --git a/pkg/sql/opt/optbuilder/create_function.go b/pkg/sql/opt/optbuilder/create_function.go index 8b660788a2cf..93d2bb129eaa 100644 --- a/pkg/sql/opt/optbuilder/create_function.go +++ b/pkg/sql/opt/optbuilder/create_function.go @@ -202,7 +202,7 @@ func (b *Builder) buildCreateFunction(cf *tree.CreateRoutine, inScope *scope) (o } // The parameter type must be supported by the current cluster version. checkUnsupportedType(b.ctx, b.semaCtx, typ) - if types.IsRecordType(typ) { + if typ.Identical(types.AnyTuple) { if language == tree.RoutineLangSQL { panic(pgerror.Newf(pgcode.InvalidFunctionDefinition, "SQL functions cannot have arguments of type record")) @@ -478,15 +478,15 @@ func validateReturnType( // If return type is RECORD and the tuple content types unspecified by OUT // parameters, any column types are valid. This is the case when we have - // RETURNS RECORD without OUT params - we don't need to check the types + // RETURNS RECORD without OUT-params - we don't need to check the types // below. - if types.IsRecordType(expected) && types.IsWildcardTupleType(expected) { + if expected.Identical(types.AnyTuple) { return nil } if len(cols) == 1 { typeToCheck := expected - if isSQLProcedure && types.IsRecordType(expected) && len(expected.TupleContents()) == 1 { + if isSQLProcedure && len(expected.TupleContents()) == 1 { // For SQL procedures with output parameters we get a record type // even with a single column. typeToCheck = expected.TupleContents()[0] diff --git a/pkg/sql/opt/optbuilder/plpgsql.go b/pkg/sql/opt/optbuilder/plpgsql.go index ec9749939847..949cf8736d99 100644 --- a/pkg/sql/opt/optbuilder/plpgsql.go +++ b/pkg/sql/opt/optbuilder/plpgsql.go @@ -337,7 +337,7 @@ func (b *plpgsqlBuilder) buildBlock(astBlock *ast.Block, s *scope) *scope { if err != nil { panic(err) } - if types.IsRecordType(typ) { + if typ.Identical(types.AnyTuple) { panic(recordVarErr) } b.addVariable(dec.Var, typ) @@ -360,7 +360,7 @@ func (b *plpgsqlBuilder) buildBlock(astBlock *ast.Block, s *scope) *scope { block.cursors[dec.Name] = *dec } } - if types.IsRecordType(b.returnType) && types.IsWildcardTupleType(b.returnType) { + if b.returnType.Identical(types.AnyTuple) { // For a RECORD-returning routine, infer the concrete type by examining the // RETURN statements. This has to happen after building the declaration // block because RETURN statements can reference declared variables. Only diff --git a/pkg/sql/opt/optbuilder/routine.go b/pkg/sql/opt/optbuilder/routine.go index 9dfa3d0ae2f3..b9133e99434c 100644 --- a/pkg/sql/opt/optbuilder/routine.go +++ b/pkg/sql/opt/optbuilder/routine.go @@ -237,7 +237,7 @@ func (b *Builder) buildRoutine( // be concrete in order to decode them correctly. We can determine the types // from the result columns or tuple of the last statement. finishResolveType := func(lastStmtScope *scope) { - if types.IsWildcardTupleType(rtyp) { + if rtyp.Identical(types.AnyTuple) { if len(lastStmtScope.cols) == 1 && lastStmtScope.cols[0].typ.Family() == types.TupleFamily { // When the final statement returns a single tuple, we can use @@ -502,6 +502,10 @@ func (b *Builder) finishBuildLastStmt( expr, physProps = stmtScope.expr, stmtScope.makePhysicalProps() rtyp := f.ResolvedType() + // Note: since the final return type has already been resolved by this point, + // we can't check if this is a RECORD-returning routine by examining rTyp. + isRecordReturning := f.ResolvedOverload().ReturnsRecordType + // Add a LIMIT 1 to the last statement if the UDF is not // set-returning. This is valid because any other rows after the // first can simply be ignored. The limit could be beneficial @@ -542,21 +546,32 @@ func (b *Builder) finishBuildLastStmt( expr = b.constructProject(expr, elems) physProps = stmtScope.makePhysicalProps() } - } else if len(cols) > 1 || (types.IsRecordType(rtyp) && !isSingleTupleResult) { - // Only a single column can be returned from a UDF, unless it is used as a - // data source (see comment above). If there are multiple columns, combine - // them into a tuple. If the last statement is already returning a tuple - // and the function has a record return type, then do not wrap the - // output in another tuple. - elems := make(memo.ScalarListExpr, len(cols)) - for i := range cols { - elems[i] = b.factory.ConstructVariable(cols[i].ID) + } else { + // Only a single column can be returned from a routine, unless it is a UDF + // used as a data source (see comment above). There are three cases in which + // we must wrap the column(s) from the last statement into a single tuple: + // 1. The last statement has multiple result columns. + // 2. The routine returns RECORD, and the last statement does not already + // return a tuple column. + // 3. The routine is a stored procedure that returns a non-VOID type, and + // the last statement does not already return a tuple column. + overload := f.ResolvedOverload() + mustWrapColsInTuple := len(cols) > 1 + if len(cols) == 1 && !isSingleTupleResult { + mustWrapColsInTuple = mustWrapColsInTuple || isRecordReturning || + (rtyp.Family() != types.VoidFamily && overload.Type == tree.ProcedureRoutine) + } + if mustWrapColsInTuple { + elems := make(memo.ScalarListExpr, len(cols)) + for i := range cols { + elems[i] = b.factory.ConstructVariable(cols[i].ID) + } + tup := b.factory.ConstructTuple(elems, rtyp) + stmtScope = bodyScope.push() + col := b.synthesizeColumn(stmtScope, scopeColName(""), rtyp, nil /* expr */, tup) + expr = b.constructProject(expr, []scopeColumn{*col}) + physProps = stmtScope.makePhysicalProps() } - tup := b.factory.ConstructTuple(elems, rtyp) - stmtScope = bodyScope.push() - col := b.synthesizeColumn(stmtScope, scopeColName(""), rtyp, nil /* expr */, tup) - expr = b.constructProject(expr, []scopeColumn{*col}) - physProps = stmtScope.makePhysicalProps() } // We must preserve the presentation of columns as physical @@ -569,17 +584,18 @@ func (b *Builder) finishBuildLastStmt( if len(cols) > 0 { returnCol := physProps.Presentation[0].ID returnColMeta := b.factory.Metadata().ColumnMeta(returnCol) - if !types.IsRecordType(rtyp) && !isMultiColDataSource && !returnColMeta.Type.Identical(rtyp) { + if !isRecordReturning && !isMultiColDataSource && + !returnColMeta.Type.Identical(rtyp) { if !cast.ValidCast(returnColMeta.Type, rtyp, cast.ContextAssignment) { panic(sqlerrors.NewInvalidAssignmentCastError( returnColMeta.Type, rtyp, returnColMeta.Alias)) } - cast := b.factory.ConstructAssignmentCast( + assignCast := b.factory.ConstructAssignmentCast( b.factory.ConstructVariable(physProps.Presentation[0].ID), rtyp, ) stmtScope = bodyScope.push() - col := b.synthesizeColumn(stmtScope, scopeColName(""), rtyp, nil /* expr */, cast) + col := b.synthesizeColumn(stmtScope, scopeColName(""), rtyp, nil /* expr */, assignCast) expr = b.constructProject(expr, []scopeColumn{*col}) physProps = stmtScope.makePhysicalProps() } diff --git a/pkg/sql/opt/testutils/testcat/function.go b/pkg/sql/opt/testutils/testcat/function.go index bf90085e358c..c16cffab2aa2 100644 --- a/pkg/sql/opt/testutils/testcat/function.go +++ b/pkg/sql/opt/testutils/testcat/function.go @@ -181,7 +181,7 @@ func (tc *Catalog) CreateRoutine(c *tree.CreateRoutine) { OutParamTypes: outParams, DefaultExprs: defaultExprs, } - overload.ReturnsRecordType = types.IsRecordType(retType) + overload.ReturnsRecordType = retType.Identical(types.AnyTuple) if c.ReturnType != nil && c.ReturnType.SetOf { overload.Class = tree.GeneratorClass } diff --git a/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/create_function.go b/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/create_function.go index b334519a067e..cc597ff1cfcb 100644 --- a/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/create_function.go +++ b/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/create_function.go @@ -25,6 +25,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/sqlerrors" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/errors" + "github.com/lib/pq/oid" ) func CreateFunction(b BuildCtx, n *tree.CreateRoutine) { @@ -66,7 +67,7 @@ func CreateFunction(b BuildCtx, n *tree.CreateRoutine) { if n.IsProcedure { if n.ReturnType != nil { returnType := b.ResolveTypeRef(n.ReturnType.Type) - if returnType.Type.Family() != types.VoidFamily && !types.IsRecordType(returnType.Type) { + if returnType.Type.Family() != types.VoidFamily && returnType.Type.Oid() != oid.T_record { panic(errors.AssertionFailedf( "CreateRoutine.ReturnType is expected to be empty, VOID, or RECORD for procedures", )) @@ -79,7 +80,7 @@ func CreateFunction(b BuildCtx, n *tree.CreateRoutine) { } } else if n.ReturnType != nil { typ = n.ReturnType.Type - if returnType := b.ResolveTypeRef(typ); types.IsRecordType(returnType.Type) { + if returnType := b.ResolveTypeRef(typ); returnType.Type.Oid() == oid.T_record { // If the function returns a RECORD type, then we need to check // whether its OUT parameters specify labels for the return type. outParamTypes, outParamNames := getOutputParameters(b, n.Params) diff --git a/pkg/sql/types/types.go b/pkg/sql/types/types.go index d369072759e0..f196842dcf50 100644 --- a/pkg/sql/types/types.go +++ b/pkg/sql/types/types.go @@ -2836,13 +2836,6 @@ func IsWildcardTupleType(t *T) bool { return len(t.TupleContents()) == 1 && t.TupleContents()[0].Family() == AnyFamily } -// IsRecordType returns true if this is a RECORD type. This should only be used -// when processing UDFs. A record differs from AnyTuple in that the tuple -// contents may contain types other than Any. -func IsRecordType(typ *T) bool { - return typ.Family() == TupleFamily && typ.Oid() == oid.T_record -} - // collatedStringTypeSQL returns the string representation of a COLLATEDSTRING // or []COLLATEDSTRING type. This is tricky in the case of an array of collated // string, since brackets must precede the COLLATE identifier: