Modernizing SQL functions' result type coercions - Mailing list pgsql-hackers

From Tom Lane
Subject Modernizing SQL functions' result type coercions
Date
Msg-id 18929.1574895430@sss.pgh.pa.us
Whole thread Raw
Responses Re: Modernizing SQL functions' result type coercions  (Heikki Linnakangas <hlinnaka@iki.fi>)
List pgsql-hackers
I got interested in $SUBJECT as a result of the complaint at [1]
about typmods not being checked/enforced in places where they
reasonably should be.  The cause is that executor/functions.c's
check_sql_fn_retval() only worries about types not typmods.
Another thing not to like is that it only supports cases where
the target type is binary-compatible with the source.  It's not
great that user-visible semantics depend on an implementation
detail like binary compatibility.  (Amusingly, our user docs
ignore this altogether and claim that the types must be identical.)

Hence, the attached patch rearranges things so that we'll allow
any case where the parser's standard coercion logic can find an
assignment-level coercion, including typmod coercion if needed.
In a green field I might've argued for restricting this to
implicit coercions; but since some of the standard binary-compatible
casts are assignment-level, that would risk breaking applications
that work today.  It's really safe enough though, just as assignment
coercions are fine in INSERT: there's no possible confusion about
which conversion is appropriate.

This required some adjustments of check_sql_fn_retval's API.
I found that pulling out the determination of the result tupdesc
and making the callers do that was advisable: in most cases, the
caller has more information and can produce a more accurate tupdesc
(eg by calling get_call_result_type not get_func_result_type).
I also pulled out creation of the JunkFilter that functions.c
wants (but none of the other callers do); having it in just one
place seems simpler.  A nice side-effect of these changes is that
we can inline SQL functions in some cases where that wasn't
possible before.

This could use review/testing, so I'll add it to the next CF.

            regards, tom lane

[1] https://www.postgresql.org/message-id/b2dbfbd0-8f12-4cca-a0b5-9ee3f00385b4%40phystech.edu

diff --git a/doc/src/sgml/xfunc.sgml b/doc/src/sgml/xfunc.sgml
index d9afd3b..5616524 100644
--- a/doc/src/sgml/xfunc.sgml
+++ b/doc/src/sgml/xfunc.sgml
@@ -388,11 +388,15 @@ $$ LANGUAGE SQL;
     </para>

     <para>
-     A <acronym>SQL</acronym> function must return exactly its declared
-     result type.  This may require inserting an explicit cast.
+     If the final <literal>SELECT</literal> or <literal>RETURNING</literal>
+     clause in a <acronym>SQL</acronym> function does not return exactly
+     the function's declared result
+     type, <productname>PostgreSQL</productname> will automatically cast
+     the value to the required type, if that is possible with an implicit
+     or assignment cast.  Otherwise, you must write an explicit cast.
      For example, suppose we wanted the
      previous <function>add_em</function> function to return
-     type <type>float8</type> instead.  This won't work:
+     type <type>float8</type> instead.  It's sufficient to write

 <programlisting>
 CREATE FUNCTION add_em(integer, integer) RETURNS float8 AS $$
@@ -400,16 +404,10 @@ CREATE FUNCTION add_em(integer, integer) RETURNS float8 AS $$
 $$ LANGUAGE SQL;
 </programlisting>

-     even though in other contexts <productname>PostgreSQL</productname>
-     would be willing to insert an implicit cast to
-     convert <type>integer</type> to <type>float8</type>.
-     We need to write it as
-
-<programlisting>
-CREATE FUNCTION add_em(integer, integer) RETURNS float8 AS $$
-    SELECT ($1 + $2)::float8;
-$$ LANGUAGE SQL;
-</programlisting>
+     since the <type>integer</type> sum can be implicitly cast
+     to <type>float8</type>.
+     (See <xref linkend="typeconv"/> or <xref linkend="sql-createcast"/>
+     for more about casts.)
     </para>
    </sect2>

@@ -503,23 +501,24 @@ $$ LANGUAGE SQL;
       <listitem>
        <para>
         The select list order in the query must be exactly the same as
-        that in which the columns appear in the table associated
-        with the composite type.  (Naming the columns, as we did above,
+        that in which the columns appear in the composite type.
+        (Naming the columns, as we did above,
         is irrelevant to the system.)
        </para>
       </listitem>
       <listitem>
        <para>
-        We must ensure each expression's type matches the corresponding
-        column of the composite type, inserting a cast if necessary.
+        We must ensure each expression's type can be cast to that of
+        the corresponding column of the composite type.
         Otherwise we'll get errors like this:
 <screen>
 <computeroutput>
-ERROR:  function declared to return emp returns varchar instead of text at column 1
+ERROR:  return type mismatch in function declared to return emp
+DETAIL:  Final statement returns text instead of point at column 4.
 </computeroutput>
 </screen>
-        As with the base-type case, the function will not insert any casts
-        automatically.
+        As with the base-type case, the system will not insert explicit
+        casts automatically, only implicit or assignment casts.
        </para>
       </listitem>
      </itemizedlist>
@@ -542,8 +541,7 @@ $$ LANGUAGE SQL;
      Another example is that if we are trying to write a function that
      returns a domain over composite, rather than a plain composite type,
      it is always necessary to write it as returning a single column,
-     since there is no other way to produce a value that is exactly of
-     the domain type.
+     since there is no way to cause a coercion of the whole row result.
     </para>

     <para>
@@ -1263,7 +1261,7 @@ SELECT make_array(1, 2) AS intarray, make_array('a'::text, 'b') AS textarray;
      Without the typecast, you will get errors like this:
 <screen>
 <computeroutput>
-ERROR:  could not determine polymorphic type because input has type "unknown"
+ERROR:  could not determine polymorphic type because input has type unknown
 </computeroutput>
 </screen>
     </para>
diff --git a/src/backend/catalog/pg_proc.c b/src/backend/catalog/pg_proc.c
index ef009ad..dd13a8e 100644
--- a/src/backend/catalog/pg_proc.c
+++ b/src/backend/catalog/pg_proc.c
@@ -923,6 +923,8 @@ fmgr_sql_validator(PG_FUNCTION_ARGS)
              * verify the result type.
              */
             SQLFunctionParseInfoPtr pinfo;
+            Oid            rettype;
+            TupleDesc    rettupdesc;

             /* But first, set up parameter information */
             pinfo = prepare_sql_fn_parse_info(tuple, NULL, InvalidOid);
@@ -943,9 +945,12 @@ fmgr_sql_validator(PG_FUNCTION_ARGS)
             }

             check_sql_fn_statements(querytree_list);
-            (void) check_sql_fn_retval(funcoid, proc->prorettype,
-                                       querytree_list,
-                                       NULL, NULL);
+
+            (void) get_func_result_type(funcoid, &rettype, &rettupdesc);
+
+            (void) check_sql_fn_retval(querytree_list,
+                                       rettype, rettupdesc,
+                                       false, NULL);
         }

         error_context_stack = sqlerrcontext.previous;
diff --git a/src/backend/executor/functions.c b/src/backend/executor/functions.c
index 83337c2..93b786c 100644
--- a/src/backend/executor/functions.c
+++ b/src/backend/executor/functions.c
@@ -154,7 +154,7 @@ static Node *sql_fn_resolve_param_name(SQLFunctionParseInfoPtr pinfo,
 static List *init_execution_state(List *queryTree_list,
                                   SQLFunctionCachePtr fcache,
                                   bool lazyEvalOK);
-static void init_sql_fcache(FmgrInfo *finfo, Oid collation, bool lazyEvalOK);
+static void init_sql_fcache(FunctionCallInfo fcinfo, Oid collation, bool lazyEvalOK);
 static void postquel_start(execution_state *es, SQLFunctionCachePtr fcache);
 static bool postquel_getnext(execution_state *es, SQLFunctionCachePtr fcache);
 static void postquel_end(execution_state *es);
@@ -166,6 +166,11 @@ static Datum postquel_get_single_result(TupleTableSlot *slot,
                                         MemoryContext resultcontext);
 static void sql_exec_error_callback(void *arg);
 static void ShutdownSQLFunction(Datum arg);
+static bool coerce_fn_result_column(TargetEntry *src_tle,
+                                    Oid res_type, int32 res_typmod,
+                                    bool tlist_is_modifiable,
+                                    List **upper_tlist,
+                                    bool *upper_tlist_nontrivial);
 static void sqlfunction_startup(DestReceiver *self, int operation, TupleDesc typeinfo);
 static bool sqlfunction_receive(TupleTableSlot *slot, DestReceiver *self);
 static void sqlfunction_shutdown(DestReceiver *self);
@@ -591,18 +596,21 @@ init_execution_state(List *queryTree_list,
  * Initialize the SQLFunctionCache for a SQL function
  */
 static void
-init_sql_fcache(FmgrInfo *finfo, Oid collation, bool lazyEvalOK)
+init_sql_fcache(FunctionCallInfo fcinfo, Oid collation, bool lazyEvalOK)
 {
+    FmgrInfo   *finfo = fcinfo->flinfo;
     Oid            foid = finfo->fn_oid;
     MemoryContext fcontext;
     MemoryContext oldcontext;
     Oid            rettype;
+    TupleDesc    rettupdesc;
     HeapTuple    procedureTuple;
     Form_pg_proc procedureStruct;
     SQLFunctionCachePtr fcache;
     List       *raw_parsetree_list;
     List       *queryTree_list;
     List       *flat_query_list;
+    List       *resulttlist;
     ListCell   *lc;
     Datum        tmp;
     bool        isNull;
@@ -642,20 +650,10 @@ init_sql_fcache(FmgrInfo *finfo, Oid collation, bool lazyEvalOK)
     MemoryContextSetIdentifier(fcontext, fcache->fname);

     /*
-     * get the result type from the procedure tuple, and check for polymorphic
-     * result type; if so, find out the actual result type.
+     * Resolve any polymorphism, obtaining the actual result type, and the
+     * corresponding tupdesc if it's a rowtype.
      */
-    rettype = procedureStruct->prorettype;
-
-    if (IsPolymorphicType(rettype))
-    {
-        rettype = get_fn_expr_rettype(finfo);
-        if (rettype == InvalidOid)    /* this probably should not happen */
-            ereport(ERROR,
-                    (errcode(ERRCODE_DATATYPE_MISMATCH),
-                     errmsg("could not determine actual result type for function declared to return type %s",
-                            format_type_be(procedureStruct->prorettype))));
-    }
+    (void) get_call_result_type(fcinfo, &rettype, &rettupdesc);

     fcache->rettype = rettype;

@@ -728,8 +726,11 @@ init_sql_fcache(FmgrInfo *finfo, Oid collation, bool lazyEvalOK)
      * Check that the function returns the type it claims to.  Although in
      * simple cases this was already done when the function was defined, we
      * have to recheck because database objects used in the function's queries
-     * might have changed type.  We'd have to do it anyway if the function had
-     * any polymorphic arguments.
+     * might have changed type.  We'd have to recheck anyway if the function
+     * had any polymorphic arguments.  Moreover, check_sql_fn_retval takes
+     * care of injecting any required column type coercions.  (But we don't
+     * ask it to insert nulls for dropped columns; the junkfilter handles
+     * that.)
      *
      * Note: we set fcache->returnsTuple according to whether we are returning
      * the whole tuple result or just a single column.  In the latter case we
@@ -738,16 +739,38 @@ init_sql_fcache(FmgrInfo *finfo, Oid collation, bool lazyEvalOK)
      * lazy eval mode in that case; otherwise we'd need extra code to expand
      * the rowtype column into multiple columns, since we have no way to
      * notify the caller that it should do that.)
-     *
-     * check_sql_fn_retval will also construct a JunkFilter we can use to
-     * coerce the returned rowtype to the desired form (unless the result type
-     * is VOID, in which case there's nothing to coerce to).
      */
-    fcache->returnsTuple = check_sql_fn_retval(foid,
+    fcache->returnsTuple = check_sql_fn_retval(flat_query_list,
                                                rettype,
-                                               flat_query_list,
-                                               NULL,
-                                               &fcache->junkFilter);
+                                               rettupdesc,
+                                               false,
+                                               &resulttlist);
+
+    /*
+     * Construct a JunkFilter we can use to coerce the returned rowtype to the
+     * desired form, unless the result type is VOID, in which case there's
+     * nothing to coerce to.  (XXX Frequently, the JunkFilter isn't doing
+     * anything very interesting, but much of this module expects it to be
+     * there anyway.)
+     */
+    if (rettype != VOIDOID)
+    {
+        TupleTableSlot *slot = MakeSingleTupleTableSlot(NULL,
+                                                        &TTSOpsMinimalTuple);
+
+        /*
+         * If the result is composite, *and* we are returning the whole tuple
+         * result, we need to get rid of any dropped columns.  In the
+         * single-column-result case, there might be dropped columns within
+         * the composite column value, but it's not our problem here.
+         */
+        if (rettupdesc && fcache->returnsTuple)
+            fcache->junkFilter = ExecInitJunkFilterConversion(resulttlist,
+                                                              rettupdesc,
+                                                              slot);
+        else
+            fcache->junkFilter = ExecInitJunkFilter(resulttlist, slot);
+    }

     if (fcache->returnsTuple)
     {
@@ -1049,7 +1072,7 @@ fmgr_sql(PG_FUNCTION_ARGS)

     if (fcache == NULL)
     {
-        init_sql_fcache(fcinfo->flinfo, PG_GET_COLLATION(), lazyEvalOK);
+        init_sql_fcache(fcinfo, PG_GET_COLLATION(), lazyEvalOK);
         fcache = (SQLFunctionCachePtr) fcinfo->flinfo->fn_extra;
     }

@@ -1532,15 +1555,9 @@ check_sql_fn_statements(List *queryTreeList)
  * check_sql_fn_retval() -- check return value of a list of sql parse trees.
  *
  * The return value of a sql function is the value returned by the last
- * canSetTag query in the function.  We do some ad-hoc type checking here
- * to be sure that the user is returning the type he claims.  There are
- * also a couple of strange-looking features to assist callers in dealing
- * with allowed special cases, such as binary-compatible result types.
- *
- * For a polymorphic function the passed rettype must be the actual resolved
- * output type of the function; we should never see a polymorphic pseudotype
- * such as ANYELEMENT as rettype.  (This means we can't check the type during
- * function definition of a polymorphic function.)
+ * canSetTag query in the function.  We do some ad-hoc type checking and
+ * coercion here to ensure that the function returns what it's supposed to.
+ * Note that we may actually modify the last query to make it match!
  *
  * This function returns true if the sql function returns the entire tuple
  * result of its final statement, or false if it returns just the first column
@@ -1550,45 +1567,47 @@ check_sql_fn_statements(List *queryTreeList)
  * Note that because we allow "SELECT rowtype_expression", the result can be
  * false even when the declared function return type is a rowtype.
  *
- * If modifyTargetList isn't NULL, the function will modify the final
- * statement's targetlist in two cases:
- * (1) if the tlist returns values that are binary-coercible to the expected
- * type rather than being exactly the expected type.  RelabelType nodes will
- * be inserted to make the result types match exactly.
- * (2) if there are dropped columns in the declared result rowtype.  NULL
- * output columns will be inserted in the tlist to match them.
- * (Obviously the caller must pass a parsetree that is okay to modify when
- * using this flag.)  Note that this flag does not affect whether the tlist is
- * considered to be a legal match to the result type, only how we react to
- * allowed not-exact-match cases.  *modifyTargetList will be set true iff
- * we had to make any "dangerous" changes that could modify the semantics of
- * the statement.  If it is set true, the caller should not use the modified
- * statement, but for simplicity we apply the changes anyway.
+ * For a polymorphic function the passed rettype must be the actual resolved
+ * output type of the function; we should never see a polymorphic pseudotype
+ * such as ANYELEMENT as rettype.  (This means we can't check the type during
+ * function definition of a polymorphic function.)  If the function returns
+ * composite, the passed rettupdesc should describe the expected output.
+ * If rettupdesc is NULL, we can't verify that the output matches; that
+ * should only happen in fmgr_sql_validator(), or when the function returns
+ * RECORD and the caller doesn't actually care which composite type it is.
+ * (Typically, rettype and rettupdesc are computed by get_call_result_type
+ * or a sibling function.)
+ *
+ * In addition to coercing individual output columns, we can modify the
+ * output to include dummy NULL columns for any dropped columns appearing
+ * in rettupdesc.  This is done only if the caller asks for it.
  *
- * If junkFilter isn't NULL, then *junkFilter is set to a JunkFilter defined
- * to convert the function's tuple result to the correct output tuple type.
- * Exception: if the function is defined to return VOID then *junkFilter is
- * set to NULL.
+ * If resultTargetList isn't NULL, then *resultTargetList is set to the
+ * targetlist that defines the final statement's result.  Exception: if the
+ * function is defined to return VOID then *resultTargetList is set to NIL.
  */
 bool
-check_sql_fn_retval(Oid func_id, Oid rettype, List *queryTreeList,
-                    bool *modifyTargetList,
-                    JunkFilter **junkFilter)
+check_sql_fn_retval(List *queryTreeList,
+                    Oid rettype, TupleDesc rettupdesc,
+                    bool insertDroppedCols,
+                    List **resultTargetList)
 {
+    bool        is_tuple_result = false;
     Query       *parse;
-    List      **tlist_ptr;
+    ListCell   *parse_cell;
     List       *tlist;
     int            tlistlen;
+    bool        tlist_is_modifiable;
     char        fn_typtype;
-    Oid            restype;
+    List       *upper_tlist = NIL;
+    bool        upper_tlist_nontrivial = false;
     ListCell   *lc;

+    /* Caller must have resolved any polymorphism */
     AssertArg(!IsPolymorphicType(rettype));

-    if (modifyTargetList)
-        *modifyTargetList = false;    /* initialize for no change */
-    if (junkFilter)
-        *junkFilter = NULL;        /* initialize in case of VOID result */
+    if (resultTargetList)
+        *resultTargetList = NIL;    /* initialize in case of VOID result */

     /*
      * If it's declared to return VOID, we don't care what's in the function.
@@ -1603,12 +1622,16 @@ check_sql_fn_retval(Oid func_id, Oid rettype, List *queryTreeList,
      * the user wrote.
      */
     parse = NULL;
+    parse_cell = NULL;
     foreach(lc, queryTreeList)
     {
         Query       *q = lfirst_node(Query, lc);

         if (q->canSetTag)
+        {
             parse = q;
+            parse_cell = lc;
+        }
     }

     /*
@@ -1625,8 +1648,9 @@ check_sql_fn_retval(Oid func_id, Oid rettype, List *queryTreeList,
     if (parse &&
         parse->commandType == CMD_SELECT)
     {
-        tlist_ptr = &parse->targetList;
         tlist = parse->targetList;
+        /* tlist is modifiable unless it's a dummy in a setop query */
+        tlist_is_modifiable = (parse->setOperations == NULL);
     }
     else if (parse &&
              (parse->commandType == CMD_INSERT ||
@@ -1634,8 +1658,9 @@ check_sql_fn_retval(Oid func_id, Oid rettype, List *queryTreeList,
               parse->commandType == CMD_DELETE) &&
              parse->returningList)
     {
-        tlist_ptr = &parse->returningList;
         tlist = parse->returningList;
+        /* returningList can always be modified */
+        tlist_is_modifiable = true;
     }
     else
     {
@@ -1650,7 +1675,12 @@ check_sql_fn_retval(Oid func_id, Oid rettype, List *queryTreeList,

     /*
      * OK, check that the targetlist returns something matching the declared
-     * type.
+     * type, and modify it if necessary.  If possible, we insert any coercion
+     * steps right into the final statement's targetlist.  However, that might
+     * risk changes in the statement's semantics --- we can't safely change
+     * the output type of a grouping column, for instance.  In such cases we
+     * handle coercions by inserting an extra level of Query that effectively
+     * just does a projection.
      */

     /*
@@ -1667,8 +1697,7 @@ check_sql_fn_retval(Oid func_id, Oid rettype, List *queryTreeList,
     {
         /*
          * For scalar-type returns, the target list must have exactly one
-         * non-junk entry, and its type must agree with what the user
-         * declared; except we allow binary-compatible types too.
+         * non-junk entry, and its type must be coercible to rettype.
          */
         TargetEntry *tle;

@@ -1683,30 +1712,16 @@ check_sql_fn_retval(Oid func_id, Oid rettype, List *queryTreeList,
         tle = (TargetEntry *) linitial(tlist);
         Assert(!tle->resjunk);

-        restype = exprType((Node *) tle->expr);
-        if (!IsBinaryCoercible(restype, rettype))
+        if (!coerce_fn_result_column(tle, rettype, -1,
+                                     tlist_is_modifiable,
+                                     &upper_tlist,
+                                     &upper_tlist_nontrivial))
             ereport(ERROR,
                     (errcode(ERRCODE_INVALID_FUNCTION_DEFINITION),
                      errmsg("return type mismatch in function declared to return %s",
                             format_type_be(rettype)),
                      errdetail("Actual return type is %s.",
-                               format_type_be(restype))));
-        if (modifyTargetList && restype != rettype)
-        {
-            tle->expr = (Expr *) makeRelabelType(tle->expr,
-                                                 rettype,
-                                                 -1,
-                                                 get_typcollation(rettype),
-                                                 COERCE_IMPLICIT_CAST);
-            /* Relabel is dangerous if TLE is a sort/group or setop column */
-            if (tle->ressortgroupref != 0 || parse->setOperations)
-                *modifyTargetList = true;
-        }
-
-        /* Set up junk filter if needed */
-        if (junkFilter)
-            *junkFilter = ExecInitJunkFilter(tlist,
-                                             MakeSingleTupleTableSlot(NULL, &TTSOpsMinimalTuple));
+                               format_type_be(exprType((Node *) tle->expr)))));
     }
     else if (fn_typtype == TYPTYPE_COMPOSITE || rettype == RECORDOID)
     {
@@ -1715,26 +1730,29 @@ check_sql_fn_retval(Oid func_id, Oid rettype, List *queryTreeList,
          *
          * Note that we will not consider a domain over composite to be a
          * "rowtype" return type; it goes through the scalar case above.  This
-         * is because SQL functions don't provide any implicit casting to the
-         * result type, so there is no way to produce a domain-over-composite
-         * result except by computing it as an explicit single-column result.
+         * is because we only provide column-by-column implicit casting, and
+         * will not cast the complete record result.  So the only way to
+         * produce a domain-over-composite result is to compute it as an
+         * explicit single-column result.  The single-composite-column code
+         * path just below could handle such cases, but it won't be reached.
          */
-        TupleDesc    tupdesc;
         int            tupnatts;    /* physical number of columns in tuple */
         int            tuplogcols; /* # of nondeleted columns in tuple */
         int            colindex;    /* physical column index */
-        List       *newtlist;    /* new non-junk tlist entries */
-        List       *junkattrs;    /* new junk tlist entries */

         /*
-         * If the target list is of length 1, and the type of the varnode in
-         * the target list matches the declared return type, this is okay.
-         * This can happen, for example, where the body of the function is
-         * 'SELECT func2()', where func2 has the same composite return type as
-         * the function that's calling it.
+         * If the target list has one non-junk entry, and that expression has
+         * or can be coerced to the declared return type, take it as the
+         * result.  This allows, for example, 'SELECT func2()', where func2
+         * has the same composite return type as the function that's calling
+         * it.  This provision creates some ambiguity --- maybe the expression
+         * was meant to be the lone field of the composite result --- but it
+         * works well enough as long as we don't get too enthusiastic about
+         * inventing coercions from scalar to composite types.
          *
-         * XXX Note that if rettype is RECORD, the IsBinaryCoercible check
-         * will succeed for any composite restype.  For the moment we rely on
+         * XXX Note that if rettype is RECORD and the expression is of a named
+         * composite type, or vice versa, this coercion will succeed, whether
+         * or not the record type really matches.  For the moment we rely on
          * runtime type checking to catch any discrepancy, but it'd be nice to
          * do better at parse time.
          */
@@ -1743,78 +1761,46 @@ check_sql_fn_retval(Oid func_id, Oid rettype, List *queryTreeList,
             TargetEntry *tle = (TargetEntry *) linitial(tlist);

             Assert(!tle->resjunk);
-            restype = exprType((Node *) tle->expr);
-            if (IsBinaryCoercible(restype, rettype))
+            if (coerce_fn_result_column(tle, rettype, -1,
+                                        tlist_is_modifiable,
+                                        &upper_tlist,
+                                        &upper_tlist_nontrivial))
             {
-                if (modifyTargetList && restype != rettype)
-                {
-                    tle->expr = (Expr *) makeRelabelType(tle->expr,
-                                                         rettype,
-                                                         -1,
-                                                         get_typcollation(rettype),
-                                                         COERCE_IMPLICIT_CAST);
-                    /* Relabel is dangerous if sort/group or setop column */
-                    if (tle->ressortgroupref != 0 || parse->setOperations)
-                        *modifyTargetList = true;
-                }
-                /* Set up junk filter if needed */
-                if (junkFilter)
-                {
-                    TupleTableSlot *slot =
-                    MakeSingleTupleTableSlot(NULL, &TTSOpsMinimalTuple);
-
-                    *junkFilter = ExecInitJunkFilter(tlist, slot);
-                }
-                return false;    /* NOT returning whole tuple */
+                /* Note that we're NOT setting is_tuple_result */
+                goto tlist_coercion_finished;
             }
         }

         /*
-         * Is the rowtype fixed, or determined only at runtime?  (Note we
-         * cannot see TYPEFUNC_COMPOSITE_DOMAIN here.)
+         * If the caller didn't provide an expected tupdesc, we can't do any
+         * further checking.  Assume we're returning the whole tuple.
          */
-        if (get_func_result_type(func_id, NULL, &tupdesc) != TYPEFUNC_COMPOSITE)
+        if (rettupdesc == NULL)
         {
-            /*
-             * Assume we are returning the whole tuple. Crosschecking against
-             * what the caller expects will happen at runtime.
-             */
-            if (junkFilter)
-            {
-                TupleTableSlot *slot;
-
-                slot = MakeSingleTupleTableSlot(NULL, &TTSOpsMinimalTuple);
-                *junkFilter = ExecInitJunkFilter(tlist, slot);
-            }
+            /* Return tlist if requested */
+            if (resultTargetList)
+                *resultTargetList = tlist;
             return true;
         }
-        Assert(tupdesc);

         /*
-         * Verify that the targetlist matches the return tuple type. We scan
-         * the non-deleted attributes to ensure that they match the datatypes
-         * of the non-resjunk columns.  For deleted attributes, insert NULL
-         * result columns if the caller asked for that.
+         * Verify that the targetlist matches the return tuple type.  We scan
+         * the non-resjunk columns, and coerce them if necessary to match the
+         * datatypes of the non-deleted attributes.  For deleted attributes,
+         * insert NULL result columns if the caller asked for that.
          */
-        tupnatts = tupdesc->natts;
+        tupnatts = rettupdesc->natts;
         tuplogcols = 0;            /* we'll count nondeleted cols as we go */
         colindex = 0;
-        newtlist = NIL;            /* these are only used if modifyTargetList */
-        junkattrs = NIL;

         foreach(lc, tlist)
         {
             TargetEntry *tle = (TargetEntry *) lfirst(lc);
             Form_pg_attribute attr;
-            Oid            tletype;
-            Oid            atttype;

+            /* resjunk columns can simply be ignored */
             if (tle->resjunk)
-            {
-                if (modifyTargetList)
-                    junkattrs = lappend(junkattrs, tle);
                 continue;
-            }

             do
             {
@@ -1825,8 +1811,8 @@ check_sql_fn_retval(Oid func_id, Oid rettype, List *queryTreeList,
                              errmsg("return type mismatch in function declared to return %s",
                                     format_type_be(rettype)),
                              errdetail("Final statement returns too many columns.")));
-                attr = TupleDescAttr(tupdesc, colindex - 1);
-                if (attr->attisdropped && modifyTargetList)
+                attr = TupleDescAttr(rettupdesc, colindex - 1);
+                if (attr->attisdropped && insertDroppedCols)
                 {
                     Expr       *null_expr;

@@ -1838,57 +1824,41 @@ check_sql_fn_retval(Oid func_id, Oid rettype, List *queryTreeList,
                                                    (Datum) 0,
                                                    true,    /* isnull */
                                                    true /* byval */ );
-                    newtlist = lappend(newtlist,
-                                       makeTargetEntry(null_expr,
-                                                       colindex,
-                                                       NULL,
-                                                       false));
-                    /* NULL insertion is dangerous in a setop */
-                    if (parse->setOperations)
-                        *modifyTargetList = true;
+                    upper_tlist = lappend(upper_tlist,
+                                          makeTargetEntry(null_expr,
+                                                          list_length(upper_tlist) + 1,
+                                                          NULL,
+                                                          false));
+                    upper_tlist_nontrivial = true;
                 }
             } while (attr->attisdropped);
             tuplogcols++;

-            tletype = exprType((Node *) tle->expr);
-            atttype = attr->atttypid;
-            if (!IsBinaryCoercible(tletype, atttype))
+            if (!coerce_fn_result_column(tle,
+                                         attr->atttypid, attr->atttypmod,
+                                         tlist_is_modifiable,
+                                         &upper_tlist,
+                                         &upper_tlist_nontrivial))
                 ereport(ERROR,
                         (errcode(ERRCODE_INVALID_FUNCTION_DEFINITION),
                          errmsg("return type mismatch in function declared to return %s",
                                 format_type_be(rettype)),
                          errdetail("Final statement returns %s instead of %s at column %d.",
-                                   format_type_be(tletype),
-                                   format_type_be(atttype),
+                                   format_type_be(exprType((Node *) tle->expr)),
+                                   format_type_be(attr->atttypid),
                                    tuplogcols)));
-            if (modifyTargetList)
-            {
-                if (tletype != atttype)
-                {
-                    tle->expr = (Expr *) makeRelabelType(tle->expr,
-                                                         atttype,
-                                                         -1,
-                                                         get_typcollation(atttype),
-                                                         COERCE_IMPLICIT_CAST);
-                    /* Relabel is dangerous if sort/group or setop column */
-                    if (tle->ressortgroupref != 0 || parse->setOperations)
-                        *modifyTargetList = true;
-                }
-                tle->resno = colindex;
-                newtlist = lappend(newtlist, tle);
-            }
         }

-        /* remaining columns in tupdesc had better all be dropped */
+        /* remaining columns in rettupdesc had better all be dropped */
         for (colindex++; colindex <= tupnatts; colindex++)
         {
-            if (!TupleDescAttr(tupdesc, colindex - 1)->attisdropped)
+            if (!TupleDescAttr(rettupdesc, colindex - 1)->attisdropped)
                 ereport(ERROR,
                         (errcode(ERRCODE_INVALID_FUNCTION_DEFINITION),
                          errmsg("return type mismatch in function declared to return %s",
                                 format_type_be(rettype)),
                          errdetail("Final statement returns too few columns.")));
-            if (modifyTargetList)
+            if (insertDroppedCols)
             {
                 Expr       *null_expr;

@@ -1900,43 +1870,17 @@ check_sql_fn_retval(Oid func_id, Oid rettype, List *queryTreeList,
                                                (Datum) 0,
                                                true,    /* isnull */
                                                true /* byval */ );
-                newtlist = lappend(newtlist,
-                                   makeTargetEntry(null_expr,
-                                                   colindex,
-                                                   NULL,
-                                                   false));
-                /* NULL insertion is dangerous in a setop */
-                if (parse->setOperations)
-                    *modifyTargetList = true;
+                upper_tlist = lappend(upper_tlist,
+                                      makeTargetEntry(null_expr,
+                                                      list_length(upper_tlist) + 1,
+                                                      NULL,
+                                                      false));
+                upper_tlist_nontrivial = true;
             }
         }

-        if (modifyTargetList)
-        {
-            /* ensure resjunk columns are numbered correctly */
-            foreach(lc, junkattrs)
-            {
-                TargetEntry *tle = (TargetEntry *) lfirst(lc);
-
-                tle->resno = colindex++;
-            }
-            /* replace the tlist with the modified one */
-            *tlist_ptr = list_concat(newtlist, junkattrs);
-        }
-
-        /* Set up junk filter if needed */
-        if (junkFilter)
-        {
-            TupleTableSlot *slot =
-            MakeSingleTupleTableSlot(NULL, &TTSOpsMinimalTuple);
-
-            *junkFilter = ExecInitJunkFilterConversion(tlist,
-                                                       CreateTupleDescCopy(tupdesc),
-                                                       slot);
-        }
-
         /* Report that we are returning entire tuple result */
-        return true;
+        is_tuple_result = true;
     }
     else
         ereport(ERROR,
@@ -1944,7 +1888,135 @@ check_sql_fn_retval(Oid func_id, Oid rettype, List *queryTreeList,
                  errmsg("return type %s is not supported for SQL functions",
                         format_type_be(rettype))));

-    return false;
+tlist_coercion_finished:
+
+    /*
+     * If necessary, modify the final Query by injecting an extra Query level
+     * that just performs a projection.  (It'd be dubious to do this to a
+     * non-SELECT query, but we never have to; RETURNING lists can always be
+     * modified in-place.)
+     */
+    if (upper_tlist_nontrivial)
+    {
+        Query       *newquery;
+        List       *colnames;
+        RangeTblEntry *rte;
+        RangeTblRef *rtr;
+
+        Assert(parse->commandType == CMD_SELECT);
+
+        /* Most of the upper Query struct can be left as zeroes/nulls */
+        newquery = makeNode(Query);
+        newquery->commandType = CMD_SELECT;
+        newquery->querySource = parse->querySource;
+        newquery->canSetTag = true;
+        newquery->targetList = upper_tlist;
+
+        /* We need a moderately realistic colnames list for the subquery RTE */
+        colnames = NIL;
+        foreach(lc, parse->targetList)
+        {
+            TargetEntry *tle = (TargetEntry *) lfirst(lc);
+
+            if (tle->resjunk)
+                continue;
+            colnames = lappend(colnames,
+                               makeString(tle->resname ? tle->resname : ""));
+        }
+
+        /* Build a suitable RTE for the subquery */
+        rte = makeNode(RangeTblEntry);
+        rte->rtekind = RTE_SUBQUERY;
+        rte->subquery = parse;
+        rte->eref = rte->alias = makeAlias("*SELECT*", colnames);
+        rte->lateral = false;
+        rte->inh = false;
+        rte->inFromCl = true;
+        newquery->rtable = list_make1(rte);
+
+        rtr = makeNode(RangeTblRef);
+        rtr->rtindex = 1;
+        newquery->jointree = makeFromExpr(list_make1(rtr), NULL);
+
+        /* Replace original query in the correct element of the query list */
+        lfirst(parse_cell) = newquery;
+    }
+
+    /* Return tlist (possibly modified) if requested */
+    if (resultTargetList)
+        *resultTargetList = upper_tlist;
+
+    return is_tuple_result;
+}
+
+/*
+ * Process one function result column for check_sql_fn_retval
+ *
+ * Coerce the output value to the required type/typmod, and add a column
+ * to *upper_tlist for it.  Set *upper_tlist_nontrivial to true if we
+ * add an upper tlist item that's not just a Var.
+ *
+ * Returns true if OK, false if could not coerce to required type
+ * (in which case, no changes have been made)
+ */
+static bool
+coerce_fn_result_column(TargetEntry *src_tle,
+                        Oid res_type,
+                        int32 res_typmod,
+                        bool tlist_is_modifiable,
+                        List **upper_tlist,
+                        bool *upper_tlist_nontrivial)
+{
+    TargetEntry *new_tle;
+    Expr       *new_tle_expr;
+    Node       *cast_result;
+
+    /*
+     * If the TLE has a sortgroupref marking, don't change it, as it probably
+     * is referenced by ORDER BY, DISTINCT, etc, and changing its type would
+     * break query semantics.  Otherwise, it's safe to modify in-place unless
+     * the query as a whole has issues with that.
+     */
+    if (tlist_is_modifiable && src_tle->ressortgroupref == 0)
+    {
+        /* OK to modify src_tle in place, if necessary */
+        cast_result = coerce_to_target_type(NULL,
+                                            (Node *) src_tle->expr,
+                                            exprType((Node *) src_tle->expr),
+                                            res_type, res_typmod,
+                                            COERCION_ASSIGNMENT,
+                                            COERCE_IMPLICIT_CAST,
+                                            -1);
+        if (cast_result == NULL)
+            return false;
+        src_tle->expr = (Expr *) cast_result;
+        /* Make a Var referencing the possibly-modified TLE */
+        new_tle_expr = (Expr *) makeVarFromTargetEntry(1, src_tle);
+    }
+    else
+    {
+        /* Any casting must happen in the upper tlist */
+        Var           *var = makeVarFromTargetEntry(1, src_tle);
+
+        cast_result = coerce_to_target_type(NULL,
+                                            (Node *) var,
+                                            var->vartype,
+                                            res_type, res_typmod,
+                                            COERCION_ASSIGNMENT,
+                                            COERCE_IMPLICIT_CAST,
+                                            -1);
+        if (cast_result == NULL)
+            return false;
+        /* Did the coercion actually do anything? */
+        if (cast_result != (Node *) var)
+            *upper_tlist_nontrivial = true;
+        new_tle_expr = (Expr *) cast_result;
+    }
+    new_tle = makeTargetEntry(new_tle_expr,
+                              list_length(*upper_tlist) + 1,
+                              src_tle->resname, false);
+    *upper_tlist = lappend(*upper_tlist, new_tle);
+    return true;
 }


diff --git a/src/backend/optimizer/util/clauses.c b/src/backend/optimizer/util/clauses.c
index a04b622..e6f787f 100644
--- a/src/backend/optimizer/util/clauses.c
+++ b/src/backend/optimizer/util/clauses.c
@@ -155,7 +155,6 @@ static Query *substitute_actual_srf_parameters(Query *expr,
                                                int nargs, List *args);
 static Node *substitute_actual_srf_parameters_mutator(Node *node,
                                                       substitute_actual_srf_parameters_context *context);
-static bool tlist_matches_coltypelist(List *tlist, List *coltypelist);


 /*****************************************************************************
@@ -4399,15 +4398,16 @@ inline_function(Oid funcid, Oid result_type, Oid result_collid,
     char       *src;
     Datum        tmp;
     bool        isNull;
-    bool        modifyTargetList;
     MemoryContext oldcxt;
     MemoryContext mycxt;
     inline_error_callback_arg callback_arg;
     ErrorContextCallback sqlerrcontext;
     FuncExpr   *fexpr;
     SQLFunctionParseInfoPtr pinfo;
+    TupleDesc    rettupdesc;
     ParseState *pstate;
     List       *raw_parsetree_list;
+    List       *querytree_list;
     Query       *querytree;
     Node       *newexpr;
     int           *usecounts;
@@ -4472,8 +4472,8 @@ inline_function(Oid funcid, Oid result_type, Oid result_collid,
     /*
      * Set up to handle parameters while parsing the function body.  We need a
      * dummy FuncExpr node containing the already-simplified arguments to pass
-     * to prepare_sql_fn_parse_info.  (It is really only needed if there are
-     * some polymorphic arguments, but for simplicity we always build it.)
+     * to prepare_sql_fn_parse_info.  (In some cases we don't really need
+     * that, but for simplicity we always build it.)
      */
     fexpr = makeNode(FuncExpr);
     fexpr->funcid = funcid;
@@ -4490,6 +4490,11 @@ inline_function(Oid funcid, Oid result_type, Oid result_collid,
                                       (Node *) fexpr,
                                       input_collid);

+    /* fexpr also provides a convenient way to resolve a composite result */
+    (void) get_expr_result_type((Node *) fexpr,
+                                NULL,
+                                &rettupdesc);
+
     /*
      * We just do parsing and parse analysis, not rewriting, because rewriting
      * will not affect table-free-SELECT-only queries, which is all that we
@@ -4542,16 +4547,24 @@ inline_function(Oid funcid, Oid result_type, Oid result_collid,
      * Make sure the function (still) returns what it's declared to.  This
      * will raise an error if wrong, but that's okay since the function would
      * fail at runtime anyway.  Note that check_sql_fn_retval will also insert
-     * a RelabelType if needed to make the tlist expression match the declared
+     * a coercion if needed to make the tlist expression match the declared
      * type of the function.
      *
      * Note: we do not try this until we have verified that no rewriting was
      * needed; that's probably not important, but let's be careful.
      */
-    if (check_sql_fn_retval(funcid, result_type, list_make1(querytree),
-                            &modifyTargetList, NULL))
+    querytree_list = list_make1(querytree);
+    if (check_sql_fn_retval(querytree_list, result_type, rettupdesc,
+                            false, NULL))
         goto fail;                /* reject whole-tuple-result cases */

+    /*
+     * Given the tests above, check_sql_fn_retval shouldn't have decided to
+     * inject a projection step, but let's just make sure.
+     */
+    if (querytree != linitial(querytree_list))
+        goto fail;
+
     /* Now we can grab the tlist expression */
     newexpr = (Node *) ((TargetEntry *) linitial(querytree->targetList))->expr;

@@ -4566,9 +4579,6 @@ inline_function(Oid funcid, Oid result_type, Oid result_collid,
     if (exprType(newexpr) != result_type)
         goto fail;

-    /* check_sql_fn_retval couldn't have made any dangerous tlist changes */
-    Assert(!modifyTargetList);
-
     /*
      * Additional validity checks on the expression.  It mustn't be more
      * volatile than the surrounding function (this is to avoid breaking hacks
@@ -4877,12 +4887,13 @@ inline_set_returning_function(PlannerInfo *root, RangeTblEntry *rte)
     char       *src;
     Datum        tmp;
     bool        isNull;
-    bool        modifyTargetList;
     MemoryContext oldcxt;
     MemoryContext mycxt;
     inline_error_callback_arg callback_arg;
     ErrorContextCallback sqlerrcontext;
     SQLFunctionParseInfoPtr pinfo;
+    TypeFuncClass functypclass;
+    TupleDesc    rettupdesc;
     List       *raw_parsetree_list;
     List       *querytree_list;
     Query       *querytree;
@@ -5013,6 +5024,18 @@ inline_set_returning_function(PlannerInfo *root, RangeTblEntry *rte)
                                       fexpr->inputcollid);

     /*
+     * Also resolve the actual function result tupdesc, if composite.  If the
+     * function is just declared to return RECORD, dig the info out of the AS
+     * clause.
+     */
+    functypclass = get_expr_result_type((Node *) fexpr, NULL, &rettupdesc);
+    if (functypclass == TYPEFUNC_RECORD)
+        rettupdesc = BuildDescFromLists(rtfunc->funccolnames,
+                                        rtfunc->funccoltypes,
+                                        rtfunc->funccoltypmods,
+                                        rtfunc->funccolcollations);
+
+    /*
      * Parse, analyze, and rewrite (unlike inline_function(), we can't skip
      * rewriting here).  We can fail as soon as we find more than one query,
      * though.
@@ -5040,43 +5063,28 @@ inline_set_returning_function(PlannerInfo *root, RangeTblEntry *rte)
      * Make sure the function (still) returns what it's declared to.  This
      * will raise an error if wrong, but that's okay since the function would
      * fail at runtime anyway.  Note that check_sql_fn_retval will also insert
-     * RelabelType(s) and/or NULL columns if needed to make the tlist
-     * expression(s) match the declared type of the function.
+     * coercions if needed to make the tlist expression(s) match the declared
+     * type of the function.  We also ask it to insert dummy NULL columns for
+     * any dropped columns in rettupdesc, so that the elements of the modified
+     * tlist match up to the attribute numbers.
      *
      * If the function returns a composite type, don't inline unless the check
      * shows it's returning a whole tuple result; otherwise what it's
-     * returning is a single composite column which is not what we need. (Like
-     * check_sql_fn_retval, we deliberately exclude domains over composite
-     * here.)
+     * returning is a single composite column which is not what we need.
      */
-    if (!check_sql_fn_retval(func_oid, fexpr->funcresulttype,
-                             querytree_list,
-                             &modifyTargetList, NULL) &&
-        (get_typtype(fexpr->funcresulttype) == TYPTYPE_COMPOSITE ||
-         fexpr->funcresulttype == RECORDOID))
+    if (!check_sql_fn_retval(querytree_list,
+                             fexpr->funcresulttype, rettupdesc,
+                             true, NULL) &&
+        (functypclass == TYPEFUNC_COMPOSITE ||
+         functypclass == TYPEFUNC_COMPOSITE_DOMAIN ||
+         functypclass == TYPEFUNC_RECORD))
         goto fail;                /* reject not-whole-tuple-result cases */

     /*
-     * If we had to modify the tlist to make it match, and the statement is
-     * one in which changing the tlist contents could change semantics, we
-     * have to punt and not inline.
-     */
-    if (modifyTargetList)
-        goto fail;
-
-    /*
-     * If it returns RECORD, we have to check against the column type list
-     * provided in the RTE; check_sql_fn_retval can't do that.  (If no match,
-     * we just fail to inline, rather than complaining; see notes for
-     * tlist_matches_coltypelist.)    We don't have to do this for functions
-     * with declared OUT parameters, even though their funcresulttype is
-     * RECORDOID, so check get_func_result_type too.
+     * check_sql_fn_retval might've inserted a projection step, but that's
+     * fine; just make sure we use the upper Query.
      */
-    if (fexpr->funcresulttype == RECORDOID &&
-        get_func_result_type(func_oid, NULL, NULL) == TYPEFUNC_RECORD &&
-        !tlist_matches_coltypelist(querytree->targetList,
-                                   rtfunc->funccoltypes))
-        goto fail;
+    querytree = linitial(querytree_list);

     /*
      * Looks good --- substitute parameters into the query.
@@ -5181,46 +5189,3 @@ substitute_actual_srf_parameters_mutator(Node *node,
                                    substitute_actual_srf_parameters_mutator,
                                    (void *) context);
 }
-
-/*
- * Check whether a SELECT targetlist emits the specified column types,
- * to see if it's safe to inline a function returning record.
- *
- * We insist on exact match here.  The executor allows binary-coercible
- * cases too, but we don't have a way to preserve the correct column types
- * in the correct places if we inline the function in such a case.
- *
- * Note that we only check type OIDs not typmods; this agrees with what the
- * executor would do at runtime, and attributing a specific typmod to a
- * function result is largely wishful thinking anyway.
- */
-static bool
-tlist_matches_coltypelist(List *tlist, List *coltypelist)
-{
-    ListCell   *tlistitem;
-    ListCell   *clistitem;
-
-    clistitem = list_head(coltypelist);
-    foreach(tlistitem, tlist)
-    {
-        TargetEntry *tle = (TargetEntry *) lfirst(tlistitem);
-        Oid            coltype;
-
-        if (tle->resjunk)
-            continue;            /* ignore junk columns */
-
-        if (clistitem == NULL)
-            return false;        /* too many tlist items */
-
-        coltype = lfirst_oid(clistitem);
-        clistitem = lnext(coltypelist, clistitem);
-
-        if (exprType((Node *) tle->expr) != coltype)
-            return false;        /* column type mismatch */
-    }
-
-    if (clistitem != NULL)
-        return false;            /* too few tlist items */
-
-    return true;
-}
diff --git a/src/include/executor/functions.h b/src/include/executor/functions.h
index 99131bf..b7b6342 100644
--- a/src/include/executor/functions.h
+++ b/src/include/executor/functions.h
@@ -31,10 +31,10 @@ extern void sql_fn_parser_setup(struct ParseState *pstate,

 extern void check_sql_fn_statements(List *queryTreeList);

-extern bool check_sql_fn_retval(Oid func_id, Oid rettype,
-                                List *queryTreeList,
-                                bool *modifyTargetList,
-                                JunkFilter **junkFilter);
+extern bool check_sql_fn_retval(List *queryTreeList,
+                                Oid rettype, TupleDesc rettupdesc,
+                                bool insertDroppedCols,
+                                List **resultTargetList);

 extern DestReceiver *CreateSQLFunctionDestReceiver(void);

diff --git a/src/test/regress/expected/rangefuncs.out b/src/test/regress/expected/rangefuncs.out
index 36a5929..a70060b 100644
--- a/src/test/regress/expected/rangefuncs.out
+++ b/src/test/regress/expected/rangefuncs.out
@@ -1820,6 +1820,67 @@ select * from array_to_set(array['one', 'two']); -- fail
 ERROR:  a column definition list is required for functions returning "record"
 LINE 1: select * from array_to_set(array['one', 'two']);
                       ^
+-- after-the-fact coercion of the columns is now possible, too
+select * from array_to_set(array['one', 'two']) as t(f1 numeric(4,2),f2 text);
+  f1  | f2
+------+-----
+ 1.00 | one
+ 2.00 | two
+(2 rows)
+
+-- and if it doesn't work, you get a compile-time not run-time error
+select * from array_to_set(array['one', 'two']) as t(f1 point,f2 text);
+ERROR:  return type mismatch in function declared to return record
+DETAIL:  Final statement returns integer instead of point at column 1.
+CONTEXT:  SQL function "array_to_set" during startup
+-- with "strict", this function can't be inlined in FROM
+explain (verbose, costs off)
+  select * from array_to_set(array['one', 'two']) as t(f1 numeric(4,2),f2 text);
+                     QUERY PLAN
+----------------------------------------------------
+ Function Scan on public.array_to_set t
+   Output: f1, f2
+   Function Call: array_to_set('{one,two}'::text[])
+(3 rows)
+
+-- but without, it can be:
+create or replace function array_to_set(anyarray) returns setof record as $$
+  select i AS "index", $1[i] AS "value" from generate_subscripts($1, 1) i
+$$ language sql immutable;
+select array_to_set(array['one', 'two']);
+ array_to_set
+--------------
+ (1,one)
+ (2,two)
+(2 rows)
+
+select * from array_to_set(array['one', 'two']) as t(f1 int,f2 text);
+ f1 | f2
+----+-----
+  1 | one
+  2 | two
+(2 rows)
+
+select * from array_to_set(array['one', 'two']) as t(f1 numeric(4,2),f2 text);
+  f1  | f2
+------+-----
+ 1.00 | one
+ 2.00 | two
+(2 rows)
+
+select * from array_to_set(array['one', 'two']) as t(f1 point,f2 text);
+ERROR:  return type mismatch in function declared to return record
+DETAIL:  Final statement returns integer instead of point at column 1.
+CONTEXT:  SQL function "array_to_set" during inlining
+explain (verbose, costs off)
+  select * from array_to_set(array['one', 'two']) as t(f1 numeric(4,2),f2 text);
+                          QUERY PLAN
+--------------------------------------------------------------
+ Function Scan on pg_catalog.generate_subscripts i
+   Output: i.i, ('{one,two}'::text[])[i.i]
+   Function Call: generate_subscripts('{one,two}'::text[], 1)
+(3 rows)
+
 create temp table rngfunc(f1 int8, f2 int8);
 create function testrngfunc() returns record as $$
   insert into rngfunc values (1,2) returning *;
@@ -1863,6 +1924,140 @@ ERROR:  a column definition list is required for functions returning "record"
 LINE 1: select * from testrngfunc();
                       ^
 drop function testrngfunc();
+-- Check that typmod imposed by a composite type is honored
+create type rngfunc_type as (f1 numeric(35,6), f2 numeric(35,2));
+create function testrngfunc() returns rngfunc_type as $$
+  select 7.136178319899999964, 7.136178319899999964;
+$$ language sql immutable;
+explain (verbose, costs off)
+select testrngfunc();
+                QUERY PLAN
+-------------------------------------------
+ Result
+   Output: '(7.136178,7.14)'::rngfunc_type
+(2 rows)
+
+select testrngfunc();
+   testrngfunc
+-----------------
+ (7.136178,7.14)
+(1 row)
+
+explain (verbose, costs off)
+select * from testrngfunc();
+                    QUERY PLAN
+--------------------------------------------------
+ Function Scan on testrngfunc
+   Output: f1, f2
+   Function Call: '(7.136178,7.14)'::rngfunc_type
+(3 rows)
+
+select * from testrngfunc();
+    f1    |  f2
+----------+------
+ 7.136178 | 7.14
+(1 row)
+
+create or replace function testrngfunc() returns rngfunc_type as $$
+  select 7.136178319899999964, 7.136178319899999964;
+$$ language sql volatile;
+explain (verbose, costs off)
+select testrngfunc();
+       QUERY PLAN
+-------------------------
+ Result
+   Output: testrngfunc()
+(2 rows)
+
+select testrngfunc();
+   testrngfunc
+-----------------
+ (7.136178,7.14)
+(1 row)
+
+explain (verbose, costs off)
+select * from testrngfunc();
+             QUERY PLAN
+-------------------------------------
+ Function Scan on public.testrngfunc
+   Output: f1, f2
+   Function Call: testrngfunc()
+(3 rows)
+
+select * from testrngfunc();
+    f1    |  f2
+----------+------
+ 7.136178 | 7.14
+(1 row)
+
+drop function testrngfunc();
+create function testrngfunc() returns setof rngfunc_type as $$
+  select 7.136178319899999964, 7.136178319899999964;
+$$ language sql immutable;
+explain (verbose, costs off)
+select testrngfunc();
+       QUERY PLAN
+-------------------------
+ ProjectSet
+   Output: testrngfunc()
+   ->  Result
+(3 rows)
+
+select testrngfunc();
+   testrngfunc
+-----------------
+ (7.136178,7.14)
+(1 row)
+
+explain (verbose, costs off)
+select * from testrngfunc();
+                       QUERY PLAN
+--------------------------------------------------------
+ Result
+   Output: 7.136178::numeric(35,6), 7.14::numeric(35,2)
+(2 rows)
+
+select * from testrngfunc();
+    f1    |  f2
+----------+------
+ 7.136178 | 7.14
+(1 row)
+
+create or replace function testrngfunc() returns setof rngfunc_type as $$
+  select 7.136178319899999964, 7.136178319899999964;
+$$ language sql volatile;
+explain (verbose, costs off)
+select testrngfunc();
+       QUERY PLAN
+-------------------------
+ ProjectSet
+   Output: testrngfunc()
+   ->  Result
+(3 rows)
+
+select testrngfunc();
+   testrngfunc
+-----------------
+ (7.136178,7.14)
+(1 row)
+
+explain (verbose, costs off)
+select * from testrngfunc();
+             QUERY PLAN
+-------------------------------------
+ Function Scan on public.testrngfunc
+   Output: f1, f2
+   Function Call: testrngfunc()
+(3 rows)
+
+select * from testrngfunc();
+    f1    |  f2
+----------+------
+ 7.136178 | 7.14
+(1 row)
+
+drop type rngfunc_type cascade;
+NOTICE:  drop cascades to function testrngfunc()
 --
 -- Check some cases involving added/dropped columns in a rowtype result
 --
@@ -1955,7 +2150,7 @@ drop view usersview;
 drop function get_first_user();
 drop function get_users();
 drop table users;
--- this won't get inlined because of type coercion, but it shouldn't fail
+-- check behavior with type coercion required for a set-op
 create or replace function rngfuncbar() returns setof text as
 $$ select 'foo'::varchar union all select 'bar'::varchar ; $$
 language sql stable;
@@ -1973,6 +2168,19 @@ select * from rngfuncbar();
  bar
 (2 rows)

+-- this function is now inlinable, too:
+explain (verbose, costs off) select * from rngfuncbar();
+                   QUERY PLAN
+------------------------------------------------
+ Result
+   Output: ('foo'::character varying)
+   ->  Append
+         ->  Result
+               Output: 'foo'::character varying
+         ->  Result
+               Output: 'bar'::character varying
+(7 rows)
+
 drop function rngfuncbar();
 -- check handling of a SQL function with multiple OUT params (bug #5777)
 create or replace function rngfuncbar(out integer, out numeric) as
diff --git a/src/test/regress/sql/rangefuncs.sql b/src/test/regress/sql/rangefuncs.sql
index 5d29d2e..476b4f2 100644
--- a/src/test/regress/sql/rangefuncs.sql
+++ b/src/test/regress/sql/rangefuncs.sql
@@ -515,6 +515,27 @@ $$ language sql strict immutable;
 select array_to_set(array['one', 'two']);
 select * from array_to_set(array['one', 'two']) as t(f1 int,f2 text);
 select * from array_to_set(array['one', 'two']); -- fail
+-- after-the-fact coercion of the columns is now possible, too
+select * from array_to_set(array['one', 'two']) as t(f1 numeric(4,2),f2 text);
+-- and if it doesn't work, you get a compile-time not run-time error
+select * from array_to_set(array['one', 'two']) as t(f1 point,f2 text);
+
+-- with "strict", this function can't be inlined in FROM
+explain (verbose, costs off)
+  select * from array_to_set(array['one', 'two']) as t(f1 numeric(4,2),f2 text);
+
+-- but without, it can be:
+
+create or replace function array_to_set(anyarray) returns setof record as $$
+  select i AS "index", $1[i] AS "value" from generate_subscripts($1, 1) i
+$$ language sql immutable;
+
+select array_to_set(array['one', 'two']);
+select * from array_to_set(array['one', 'two']) as t(f1 int,f2 text);
+select * from array_to_set(array['one', 'two']) as t(f1 numeric(4,2),f2 text);
+select * from array_to_set(array['one', 'two']) as t(f1 point,f2 text);
+explain (verbose, costs off)
+  select * from array_to_set(array['one', 'two']) as t(f1 numeric(4,2),f2 text);

 create temp table rngfunc(f1 int8, f2 int8);

@@ -538,6 +559,57 @@ select * from testrngfunc(); -- fail

 drop function testrngfunc();

+-- Check that typmod imposed by a composite type is honored
+create type rngfunc_type as (f1 numeric(35,6), f2 numeric(35,2));
+
+create function testrngfunc() returns rngfunc_type as $$
+  select 7.136178319899999964, 7.136178319899999964;
+$$ language sql immutable;
+
+explain (verbose, costs off)
+select testrngfunc();
+select testrngfunc();
+explain (verbose, costs off)
+select * from testrngfunc();
+select * from testrngfunc();
+
+create or replace function testrngfunc() returns rngfunc_type as $$
+  select 7.136178319899999964, 7.136178319899999964;
+$$ language sql volatile;
+
+explain (verbose, costs off)
+select testrngfunc();
+select testrngfunc();
+explain (verbose, costs off)
+select * from testrngfunc();
+select * from testrngfunc();
+
+drop function testrngfunc();
+
+create function testrngfunc() returns setof rngfunc_type as $$
+  select 7.136178319899999964, 7.136178319899999964;
+$$ language sql immutable;
+
+explain (verbose, costs off)
+select testrngfunc();
+select testrngfunc();
+explain (verbose, costs off)
+select * from testrngfunc();
+select * from testrngfunc();
+
+create or replace function testrngfunc() returns setof rngfunc_type as $$
+  select 7.136178319899999964, 7.136178319899999964;
+$$ language sql volatile;
+
+explain (verbose, costs off)
+select testrngfunc();
+select testrngfunc();
+explain (verbose, costs off)
+select * from testrngfunc();
+select * from testrngfunc();
+
+drop type rngfunc_type cascade;
+
 --
 -- Check some cases involving added/dropped columns in a rowtype result
 --
@@ -585,7 +657,7 @@ drop function get_first_user();
 drop function get_users();
 drop table users;

--- this won't get inlined because of type coercion, but it shouldn't fail
+-- check behavior with type coercion required for a set-op

 create or replace function rngfuncbar() returns setof text as
 $$ select 'foo'::varchar union all select 'bar'::varchar ; $$
@@ -593,6 +665,8 @@ language sql stable;

 select rngfuncbar();
 select * from rngfuncbar();
+-- this function is now inlinable, too:
+explain (verbose, costs off) select * from rngfuncbar();

 drop function rngfuncbar();


pgsql-hackers by date:

Previous
From: Alvaro Herrera
Date:
Subject: Re: FETCH FIRST clause WITH TIES option
Next
From: Jeff Davis
Date:
Subject: Re: Memory-Bounded Hash Aggregation