Support domains over composite types.

This is the last major omission in our domains feature: you can now
make a domain over anything that's not a pseudotype.

The major complication from an implementation standpoint is that places
that might be creating tuples of a domain type now need to be prepared
to apply domain_check().  It seems better that unprepared code fail
with an error like "<type> is not composite" than that it silently fail
to apply domain constraints.  Therefore, relevant infrastructure like
get_func_result_type() and lookup_rowtype_tupdesc() has been adjusted
to treat domain-over-composite as a distinct case that unprepared code
won't recognize, rather than just transparently treating it the same
as plain composite.  This isn't a 100% solution to the possibility of
overlooked domain checks, but it catches most places.

In passing, improve typcache.c's support for domains (it can now cache
the identity of a domain's base type), and rewrite the argument handling
logic in jsonfuncs.c's populate_record[set]_worker to reduce duplicative
per-call lookups.

I believe this is code-complete so far as the core and contrib code go.
The PLs need varying amounts of work, which will be tackled in followup
patches.

Discussion: https://postgr.es/m/4206.1499798337@sss.pgh.pa.us
This commit is contained in:
Tom Lane 2017-10-26 13:47:45 -04:00
parent 08f1e1f0a4
commit 37a795a60b
37 changed files with 1085 additions and 293 deletions

View File

@ -752,6 +752,8 @@ typedef struct RecordIOData
{
Oid record_type;
int32 record_typmod;
/* this field is used only if target type is domain over composite: */
void *domain_info; /* opaque cache for domain checks */
int ncolumns;
ColumnIOData columns[FLEXIBLE_ARRAY_MEMBER];
} RecordIOData;
@ -780,9 +782,11 @@ hstore_from_record(PG_FUNCTION_ARGS)
Oid argtype = get_fn_expr_argtype(fcinfo->flinfo, 0);
/*
* have no tuple to look at, so the only source of type info is the
* argtype. The lookup_rowtype_tupdesc call below will error out if we
* don't have a known composite type oid here.
* We have no tuple to look at, so the only source of type info is the
* argtype --- which might be domain over composite, but we don't care
* here, since we have no need to be concerned about domain
* constraints. The lookup_rowtype_tupdesc_domain call below will
* error out if we don't have a known composite type oid here.
*/
tupType = argtype;
tupTypmod = -1;
@ -793,12 +797,15 @@ hstore_from_record(PG_FUNCTION_ARGS)
{
rec = PG_GETARG_HEAPTUPLEHEADER(0);
/* Extract type info from the tuple itself */
/*
* Extract type info from the tuple itself -- this will work even for
* anonymous record types.
*/
tupType = HeapTupleHeaderGetTypeId(rec);
tupTypmod = HeapTupleHeaderGetTypMod(rec);
}
tupdesc = lookup_rowtype_tupdesc(tupType, tupTypmod);
tupdesc = lookup_rowtype_tupdesc_domain(tupType, tupTypmod, false);
ncolumns = tupdesc->natts;
/*
@ -943,9 +950,9 @@ hstore_populate_record(PG_FUNCTION_ARGS)
rec = NULL;
/*
* have no tuple to look at, so the only source of type info is the
* argtype. The lookup_rowtype_tupdesc call below will error out if we
* don't have a known composite type oid here.
* We have no tuple to look at, so the only source of type info is the
* argtype. The lookup_rowtype_tupdesc_domain call below will error
* out if we don't have a known composite type oid here.
*/
tupType = argtype;
tupTypmod = -1;
@ -957,7 +964,10 @@ hstore_populate_record(PG_FUNCTION_ARGS)
if (PG_ARGISNULL(1))
PG_RETURN_POINTER(rec);
/* Extract type info from the tuple itself */
/*
* Extract type info from the tuple itself -- this will work even for
* anonymous record types.
*/
tupType = HeapTupleHeaderGetTypeId(rec);
tupTypmod = HeapTupleHeaderGetTypMod(rec);
}
@ -975,7 +985,11 @@ hstore_populate_record(PG_FUNCTION_ARGS)
if (HS_COUNT(hs) == 0 && rec)
PG_RETURN_POINTER(rec);
tupdesc = lookup_rowtype_tupdesc(tupType, tupTypmod);
/*
* Lookup the input record's tupdesc. For the moment, we don't worry
* about whether it is a domain over composite.
*/
tupdesc = lookup_rowtype_tupdesc_domain(tupType, tupTypmod, false);
ncolumns = tupdesc->natts;
if (rec)
@ -1002,6 +1016,7 @@ hstore_populate_record(PG_FUNCTION_ARGS)
my_extra = (RecordIOData *) fcinfo->flinfo->fn_extra;
my_extra->record_type = InvalidOid;
my_extra->record_typmod = 0;
my_extra->domain_info = NULL;
}
if (my_extra->record_type != tupType ||
@ -1103,6 +1118,17 @@ hstore_populate_record(PG_FUNCTION_ARGS)
rettuple = heap_form_tuple(tupdesc, values, nulls);
/*
* If the target type is domain over composite, all we know at this point
* is that we've made a valid value of the base composite type. Must
* check domain constraints before deciding we're done.
*/
if (argtype != tupdesc->tdtypeid)
domain_check(HeapTupleGetDatum(rettuple), false,
argtype,
&my_extra->domain_info,
fcinfo->flinfo->fn_mcxt);
ReleaseTupleDesc(tupdesc);
PG_RETURN_DATUM(HeapTupleGetDatum(rettuple));

View File

@ -4379,8 +4379,7 @@ SET xmloption TO { DOCUMENT | CONTENT };
underlying type &mdash; for example, any operator or function that
can be applied to the underlying type will work on the domain type.
The underlying type can be any built-in or user-defined base type,
enum type, array or range type, or another domain. (Currently, domains
over composite types are not implemented.)
enum type, array type, composite type, range type, or another domain.
</para>
<para>

View File

@ -84,8 +84,9 @@ CREATE TABLE inventory_item (
restriction of the current implementation: since no constraints are
associated with a composite type, the constraints shown in the table
definition <emphasis>do not apply</emphasis> to values of the composite type
outside the table. (A partial workaround is to use domain
types as members of composite types.)
outside the table. (To work around this, create a domain over the composite
type, and apply the desired constraints as <literal>CHECK</literal>
constraints of the domain.)
</para>
</sect2>

View File

@ -351,6 +351,31 @@ CREATE FUNCTION tf1 (accountno integer, debit numeric) RETURNS numeric AS $$
WHERE accountno = tf1.accountno
RETURNING balance;
$$ LANGUAGE SQL;
</programlisting>
</para>
<para>
A <acronym>SQL</acronym> function must return exactly its declared
result type. This may require inserting 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:
<programlisting>
CREATE FUNCTION add_em(integer, integer) RETURNS float8 AS $$
SELECT $1 + $2;
$$ 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>
</para>
</sect2>
@ -452,13 +477,16 @@ $$ LANGUAGE SQL;
</listitem>
<listitem>
<para>
You must typecast the expressions to match the
definition of the composite type, or you will get errors like this:
We must ensure each expression's type matches the corresponding
column of the composite type, inserting a cast if necessary.
Otherwise we'll get errors like this:
<screen>
<computeroutput>
ERROR: function declared to return emp returns varchar instead of text at column 1
</computeroutput>
</screen>
As with the base-type case, the function will not insert any casts
automatically.
</para>
</listitem>
</itemizedlist>
@ -478,6 +506,11 @@ $$ LANGUAGE SQL;
in this situation, but it is a handy alternative in some cases
&mdash; for example, if we need to compute the result by calling
another function that returns the desired composite value.
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.
</para>
<para>

View File

@ -301,6 +301,11 @@ has_superclass(Oid relationId)
/*
* Given two type OIDs, determine whether the first is a complex type
* (class type) that inherits from the second.
*
* This essentially asks whether the first type is guaranteed to be coercible
* to the second. Therefore, we allow the first type to be a domain over a
* complex type that inherits from the second; that creates no difficulties.
* But the second type cannot be a domain.
*/
bool
typeInheritsFrom(Oid subclassTypeId, Oid superclassTypeId)
@ -314,9 +319,9 @@ typeInheritsFrom(Oid subclassTypeId, Oid superclassTypeId)
ListCell *queue_item;
/* We need to work with the associated relation OIDs */
subclassRelid = typeidTypeRelid(subclassTypeId);
subclassRelid = typeOrDomainTypeRelid(subclassTypeId);
if (subclassRelid == InvalidOid)
return false; /* not a complex type */
return false; /* not a complex type or domain over one */
superclassRelid = typeidTypeRelid(superclassTypeId);
if (superclassRelid == InvalidOid)
return false; /* not a complex type */

View File

@ -262,7 +262,7 @@ ProcedureCreate(const char *procedureName,
*/
if (parameterCount == 1 &&
OidIsValid(parameterTypes->values[0]) &&
(relid = typeidTypeRelid(parameterTypes->values[0])) != InvalidOid &&
(relid = typeOrDomainTypeRelid(parameterTypes->values[0])) != InvalidOid &&
get_attnum(relid, procedureName) != InvalidAttrNumber)
ereport(ERROR,
(errcode(ERRCODE_DUPLICATE_COLUMN),

View File

@ -5091,6 +5091,8 @@ find_typed_table_dependencies(Oid typeOid, const char *typeName, DropBehavior be
* isn't suitable, throw an error. Currently, we require that the type
* originated with CREATE TYPE AS. We could support any row type, but doing so
* would require handling a number of extra corner cases in the DDL commands.
* (Also, allowing domain-over-composite would open up a can of worms about
* whether and how the domain's constraints should apply to derived tables.)
*/
void
check_of_type(HeapTuple typetuple)
@ -6190,8 +6192,8 @@ ATPrepSetStatistics(Relation rel, const char *colName, int16 colNum, Node *newVa
RelationGetRelationName(rel))));
/*
* We allow referencing columns by numbers only for indexes, since
* table column numbers could contain gaps if columns are later dropped.
* We allow referencing columns by numbers only for indexes, since table
* column numbers could contain gaps if columns are later dropped.
*/
if (rel->rd_rel->relkind != RELKIND_INDEX && !colName)
ereport(ERROR,

View File

@ -798,13 +798,16 @@ DefineDomain(CreateDomainStmt *stmt)
basetypeoid = HeapTupleGetOid(typeTup);
/*
* Base type must be a plain base type, another domain, an enum or a range
* type. Domains over pseudotypes would create a security hole. Domains
* over composite types might be made to work in the future, but not
* today.
* Base type must be a plain base type, a composite type, another domain,
* an enum or a range type. Domains over pseudotypes would create a
* security hole. (It would be shorter to code this to just check for
* pseudotypes; but it seems safer to call out the specific typtypes that
* are supported, rather than assume that all future typtypes would be
* automatically supported.)
*/
typtype = baseType->typtype;
if (typtype != TYPTYPE_BASE &&
typtype != TYPTYPE_COMPOSITE &&
typtype != TYPTYPE_DOMAIN &&
typtype != TYPTYPE_ENUM &&
typtype != TYPTYPE_RANGE)

View File

@ -3469,8 +3469,12 @@ ExecEvalWholeRowVar(ExprState *state, ExprEvalStep *op, ExprContext *econtext)
* generates an INT4 NULL regardless of the dropped column type).
* If we find a dropped column and cannot verify that case (1)
* holds, we have to use the slow path to check (2) for each row.
*
* If vartype is a domain over composite, just look through that
* to the base composite type.
*/
var_tupdesc = lookup_rowtype_tupdesc(variable->vartype, -1);
var_tupdesc = lookup_rowtype_tupdesc_domain(variable->vartype,
-1, false);
slot_tupdesc = slot->tts_tupleDescriptor;

View File

@ -502,7 +502,7 @@ restart:
{
TupleTableSlot *slot = fcache->funcResultSlot;
MemoryContext oldContext;
bool foundTup;
bool foundTup;
/*
* Have to make sure tuple in slot lives long enough, otherwise
@ -734,7 +734,8 @@ init_sexpr(Oid foid, Oid input_collation, Expr *node,
/* Must save tupdesc in sexpr's context */
oldcontext = MemoryContextSwitchTo(sexprCxt);
if (functypclass == TYPEFUNC_COMPOSITE)
if (functypclass == TYPEFUNC_COMPOSITE ||
functypclass == TYPEFUNC_COMPOSITE_DOMAIN)
{
/* Composite data type, e.g. a table's row type */
Assert(tupdesc);

View File

@ -1665,7 +1665,15 @@ check_sql_fn_retval(Oid func_id, Oid rettype, List *queryTreeList,
}
else if (fn_typtype == TYPTYPE_COMPOSITE || rettype == RECORDOID)
{
/* Returns a rowtype */
/*
* Returns a rowtype.
*
* 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.
*/
TupleDesc tupdesc;
int tupnatts; /* physical number of columns in tuple */
int tuplogcols; /* # of nondeleted columns in tuple */
@ -1711,7 +1719,10 @@ check_sql_fn_retval(Oid func_id, Oid rettype, List *queryTreeList,
}
}
/* Is the rowtype fixed, or determined only at runtime? */
/*
* Is the rowtype fixed, or determined only at runtime? (Note we
* cannot see TYPEFUNC_COMPOSITE_DOMAIN here.)
*/
if (get_func_result_type(func_id, NULL, &tupdesc) != TYPEFUNC_COMPOSITE)
{
/*

View File

@ -383,7 +383,8 @@ ExecInitFunctionScan(FunctionScan *node, EState *estate, int eflags)
&funcrettype,
&tupdesc);
if (functypclass == TYPEFUNC_COMPOSITE)
if (functypclass == TYPEFUNC_COMPOSITE ||
functypclass == TYPEFUNC_COMPOSITE_DOMAIN)
{
/* Composite data type, e.g. a table's row type */
Assert(tupdesc);

View File

@ -120,8 +120,10 @@ makeVarFromTargetEntry(Index varno,
* table entry, and varattno == 0 to signal that it references the whole
* tuple. (Use of zero here is unclean, since it could easily be confused
* with error cases, but it's not worth changing now.) The vartype indicates
* a rowtype; either a named composite type, or RECORD. This function
* encapsulates the logic for determining the correct rowtype OID to use.
* a rowtype; either a named composite type, or a domain over a named
* composite type (only possible if the RTE is a function returning that),
* or RECORD. This function encapsulates the logic for determining the
* correct rowtype OID to use.
*
* If allowScalar is true, then for the case where the RTE is a single function
* returning a non-composite result type, we produce a normal Var referencing

View File

@ -2356,6 +2356,10 @@ CommuteRowCompareExpr(RowCompareExpr *clause)
* is still what it was when the expression was parsed. This is needed to
* guard against improper simplification after ALTER COLUMN TYPE. (XXX we
* may well need to make similar checks elsewhere?)
*
* rowtypeid may come from a whole-row Var, and therefore it can be a domain
* over composite, but for this purpose we only care about checking the type
* of a contained field.
*/
static bool
rowtype_field_matches(Oid rowtypeid, int fieldnum,
@ -2368,7 +2372,7 @@ rowtype_field_matches(Oid rowtypeid, int fieldnum,
/* No issue for RECORD, since there is no way to ALTER such a type */
if (rowtypeid == RECORDOID)
return true;
tupdesc = lookup_rowtype_tupdesc(rowtypeid, -1);
tupdesc = lookup_rowtype_tupdesc_domain(rowtypeid, -1, false);
if (fieldnum <= 0 || fieldnum > tupdesc->natts)
{
ReleaseTupleDesc(tupdesc);
@ -5005,7 +5009,9 @@ inline_set_returning_function(PlannerInfo *root, RangeTblEntry *rte)
*
* 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.
* returning is a single composite column which is not what we need. (Like
* check_sql_fn_retval, we deliberately exclude domains over composite
* here.)
*/
if (!check_sql_fn_retval(func_oid, fexpr->funcresulttype,
querytree_list,

View File

@ -499,9 +499,26 @@ coerce_type(ParseState *pstate, Node *node,
* Input class type is a subclass of target, so generate an
* appropriate runtime conversion (removing unneeded columns and
* possibly rearranging the ones that are wanted).
*
* We will also get here when the input is a domain over a subclass of
* the target type. To keep life simple for the executor, we define
* ConvertRowtypeExpr as only working between regular composite types;
* therefore, in such cases insert a RelabelType to smash the input
* expression down to its base type.
*/
Oid baseTypeId = getBaseType(inputTypeId);
ConvertRowtypeExpr *r = makeNode(ConvertRowtypeExpr);
if (baseTypeId != inputTypeId)
{
RelabelType *rt = makeRelabelType((Expr *) node,
baseTypeId, -1,
InvalidOid,
COERCE_IMPLICIT_CAST);
rt->location = location;
node = (Node *) rt;
}
r->arg = (Expr *) node;
r->resulttype = targetTypeId;
r->convertformat = cformat;
@ -966,6 +983,8 @@ coerce_record_to_complex(ParseState *pstate, Node *node,
int location)
{
RowExpr *rowexpr;
Oid baseTypeId;
int32 baseTypeMod = -1;
TupleDesc tupdesc;
List *args = NIL;
List *newargs;
@ -1001,7 +1020,14 @@ coerce_record_to_complex(ParseState *pstate, Node *node,
format_type_be(targetTypeId)),
parser_coercion_errposition(pstate, location, node)));
tupdesc = lookup_rowtype_tupdesc(targetTypeId, -1);
/*
* Look up the composite type, accounting for possibility that what we are
* given is a domain over composite.
*/
baseTypeId = getBaseTypeAndTypmod(targetTypeId, &baseTypeMod);
tupdesc = lookup_rowtype_tupdesc(baseTypeId, baseTypeMod);
/* Process the fields */
newargs = NIL;
ucolno = 1;
arg = list_head(args);
@ -1070,10 +1096,22 @@ coerce_record_to_complex(ParseState *pstate, Node *node,
rowexpr = makeNode(RowExpr);
rowexpr->args = newargs;
rowexpr->row_typeid = targetTypeId;
rowexpr->row_typeid = baseTypeId;
rowexpr->row_format = cformat;
rowexpr->colnames = NIL; /* not needed for named target type */
rowexpr->location = location;
/* If target is a domain, apply constraints */
if (baseTypeId != targetTypeId)
{
rowexpr->row_format = COERCE_IMPLICIT_CAST;
return coerce_to_domain((Node *) rowexpr,
baseTypeId, baseTypeMod,
targetTypeId,
ccontext, cformat, location,
false);
}
return (Node *) rowexpr;
}
@ -2401,13 +2439,13 @@ is_complex_array(Oid typid)
/*
* Check whether reltypeId is the row type of a typed table of type
* reloftypeId. (This is conceptually similar to the subtype
* relationship checked by typeInheritsFrom().)
* reloftypeId, or is a domain over such a row type. (This is conceptually
* similar to the subtype relationship checked by typeInheritsFrom().)
*/
static bool
typeIsOfTypedTable(Oid reltypeId, Oid reloftypeId)
{
Oid relid = typeidTypeRelid(reltypeId);
Oid relid = typeOrDomainTypeRelid(reltypeId);
bool result = false;
if (relid)

View File

@ -1819,18 +1819,19 @@ ParseComplexProjection(ParseState *pstate, char *funcname, Node *first_arg,
}
/*
* Else do it the hard way with get_expr_result_type().
* Else do it the hard way with get_expr_result_tupdesc().
*
* If it's a Var of type RECORD, we have to work even harder: we have to
* find what the Var refers to, and pass that to get_expr_result_type.
* find what the Var refers to, and pass that to get_expr_result_tupdesc.
* That task is handled by expandRecordVariable().
*/
if (IsA(first_arg, Var) &&
((Var *) first_arg)->vartype == RECORDOID)
tupdesc = expandRecordVariable(pstate, (Var *) first_arg, 0);
else if (get_expr_result_type(first_arg, NULL, &tupdesc) != TYPEFUNC_COMPOSITE)
else
tupdesc = get_expr_result_tupdesc(first_arg, true);
if (!tupdesc)
return NULL; /* unresolvable RECORD type */
Assert(tupdesc);
for (i = 0; i < tupdesc->natts; i++)
{

View File

@ -1496,7 +1496,8 @@ addRangeTableEntryForFunction(ParseState *pstate,
parser_errposition(pstate, exprLocation(funcexpr))));
}
if (functypclass == TYPEFUNC_COMPOSITE)
if (functypclass == TYPEFUNC_COMPOSITE ||
functypclass == TYPEFUNC_COMPOSITE_DOMAIN)
{
/* Composite data type, e.g. a table's row type */
Assert(tupdesc);
@ -2245,7 +2246,8 @@ expandRTE(RangeTblEntry *rte, int rtindex, int sublevels_up,
functypclass = get_expr_result_type(rtfunc->funcexpr,
&funcrettype,
&tupdesc);
if (functypclass == TYPEFUNC_COMPOSITE)
if (functypclass == TYPEFUNC_COMPOSITE ||
functypclass == TYPEFUNC_COMPOSITE_DOMAIN)
{
/* Composite data type, e.g. a table's row type */
Assert(tupdesc);
@ -2765,7 +2767,8 @@ get_rte_attribute_type(RangeTblEntry *rte, AttrNumber attnum,
&funcrettype,
&tupdesc);
if (functypclass == TYPEFUNC_COMPOSITE)
if (functypclass == TYPEFUNC_COMPOSITE ||
functypclass == TYPEFUNC_COMPOSITE_DOMAIN)
{
/* Composite data type, e.g. a table's row type */
Form_pg_attribute att_tup;
@ -2966,14 +2969,11 @@ get_rte_attribute_is_dropped(RangeTblEntry *rte, AttrNumber attnum)
if (attnum > atts_done &&
attnum <= atts_done + rtfunc->funccolcount)
{
TypeFuncClass functypclass;
Oid funcrettype;
TupleDesc tupdesc;
functypclass = get_expr_result_type(rtfunc->funcexpr,
&funcrettype,
&tupdesc);
if (functypclass == TYPEFUNC_COMPOSITE)
tupdesc = get_expr_result_tupdesc(rtfunc->funcexpr,
true);
if (tupdesc)
{
/* Composite data type, e.g. a table's row type */
Form_pg_attribute att_tup;

View File

@ -725,6 +725,8 @@ transformAssignmentIndirection(ParseState *pstate,
else
{
FieldStore *fstore;
Oid baseTypeId;
int32 baseTypeMod;
Oid typrelid;
AttrNumber attnum;
Oid fieldTypeId;
@ -752,7 +754,14 @@ transformAssignmentIndirection(ParseState *pstate,
/* No subscripts, so can process field selection here */
typrelid = typeidTypeRelid(targetTypeId);
/*
* Look up the composite type, accounting for possibility that
* what we are given is a domain over composite.
*/
baseTypeMod = targetTypMod;
baseTypeId = getBaseTypeAndTypmod(targetTypeId, &baseTypeMod);
typrelid = typeidTypeRelid(baseTypeId);
if (!typrelid)
ereport(ERROR,
(errcode(ERRCODE_DATATYPE_MISMATCH),
@ -796,7 +805,17 @@ transformAssignmentIndirection(ParseState *pstate,
fstore->arg = (Expr *) basenode;
fstore->newvals = list_make1(rhs);
fstore->fieldnums = list_make1_int(attnum);
fstore->resulttype = targetTypeId;
fstore->resulttype = baseTypeId;
/* If target is a domain, apply constraints */
if (baseTypeId != targetTypeId)
return coerce_to_domain((Node *) fstore,
baseTypeId, baseTypeMod,
targetTypeId,
COERCION_IMPLICIT,
COERCE_IMPLICIT_CAST,
location,
false);
return (Node *) fstore;
}
@ -1164,7 +1183,7 @@ ExpandColumnRefStar(ParseState *pstate, ColumnRef *cref,
Node *node;
node = pstate->p_post_columnref_hook(pstate, cref,
(Node *) rte);
(Node *) rte);
if (node != NULL)
{
if (rte != NULL)
@ -1387,22 +1406,18 @@ ExpandRowReference(ParseState *pstate, Node *expr,
* (This can be pretty inefficient if the expression involves nontrivial
* computation :-(.)
*
* Verify it's a composite type, and get the tupdesc. We use
* get_expr_result_type() because that can handle references to functions
* returning anonymous record types. If that fails, use
* lookup_rowtype_tupdesc(), which will almost certainly fail as well, but
* it will give an appropriate error message.
* Verify it's a composite type, and get the tupdesc.
* get_expr_result_tupdesc() handles this conveniently.
*
* If it's a Var of type RECORD, we have to work even harder: we have to
* find what the Var refers to, and pass that to get_expr_result_type.
* find what the Var refers to, and pass that to get_expr_result_tupdesc.
* That task is handled by expandRecordVariable().
*/
if (IsA(expr, Var) &&
((Var *) expr)->vartype == RECORDOID)
tupleDesc = expandRecordVariable(pstate, (Var *) expr, 0);
else if (get_expr_result_type(expr, NULL, &tupleDesc) != TYPEFUNC_COMPOSITE)
tupleDesc = lookup_rowtype_tupdesc_copy(exprType(expr),
exprTypmod(expr));
else
tupleDesc = get_expr_result_tupdesc(expr, false);
Assert(tupleDesc);
/* Generate a list of references to the individual fields */
@ -1610,15 +1625,9 @@ expandRecordVariable(ParseState *pstate, Var *var, int levelsup)
/*
* We now have an expression we can't expand any more, so see if
* get_expr_result_type() can do anything with it. If not, pass to
* lookup_rowtype_tupdesc() which will probably fail, but will give an
* appropriate error message while failing.
* get_expr_result_tupdesc() can do anything with it.
*/
if (get_expr_result_type(expr, NULL, &tupleDesc) != TYPEFUNC_COMPOSITE)
tupleDesc = lookup_rowtype_tupdesc_copy(exprType(expr),
exprTypmod(expr));
return tupleDesc;
return get_expr_result_tupdesc(expr, false);
}

View File

@ -641,7 +641,10 @@ stringTypeDatum(Type tp, char *string, int32 atttypmod)
return OidInputFunctionCall(typinput, string, typioparam, atttypmod);
}
/* given a typeid, return the type's typrelid (associated relation, if any) */
/*
* Given a typeid, return the type's typrelid (associated relation), if any.
* Returns InvalidOid if type is not a composite type.
*/
Oid
typeidTypeRelid(Oid type_id)
{
@ -652,13 +655,44 @@ typeidTypeRelid(Oid type_id)
typeTuple = SearchSysCache1(TYPEOID, ObjectIdGetDatum(type_id));
if (!HeapTupleIsValid(typeTuple))
elog(ERROR, "cache lookup failed for type %u", type_id);
type = (Form_pg_type) GETSTRUCT(typeTuple);
result = type->typrelid;
ReleaseSysCache(typeTuple);
return result;
}
/*
* Given a typeid, return the type's typrelid (associated relation), if any.
* Returns InvalidOid if type is not a composite type or a domain over one.
* This is the same as typeidTypeRelid(getBaseType(type_id)), but faster.
*/
Oid
typeOrDomainTypeRelid(Oid type_id)
{
HeapTuple typeTuple;
Form_pg_type type;
Oid result;
for (;;)
{
typeTuple = SearchSysCache1(TYPEOID, ObjectIdGetDatum(type_id));
if (!HeapTupleIsValid(typeTuple))
elog(ERROR, "cache lookup failed for type %u", type_id);
type = (Form_pg_type) GETSTRUCT(typeTuple);
if (type->typtype != TYPTYPE_DOMAIN)
{
/* Not a domain, so done looking through domains */
break;
}
/* It is a domain, so examine the base type instead */
type_id = type->typbasetype;
ReleaseSysCache(typeTuple);
}
result = type->typrelid;
ReleaseSysCache(typeTuple);
return result;
}
/*
* error context callback for parse failure during parseTypeString()
*/

View File

@ -82,9 +82,10 @@ domain_state_setup(Oid domainType, bool binary, MemoryContext mcxt)
* Verify that domainType represents a valid domain type. We need to be
* careful here because domain_in and domain_recv can be called from SQL,
* possibly with incorrect arguments. We use lookup_type_cache mainly
* because it will throw a clean user-facing error for a bad OID.
* because it will throw a clean user-facing error for a bad OID; but also
* it can cache the underlying base type info.
*/
typentry = lookup_type_cache(domainType, 0);
typentry = lookup_type_cache(domainType, TYPECACHE_DOMAIN_BASE_INFO);
if (typentry->typtype != TYPTYPE_DOMAIN)
ereport(ERROR,
(errcode(ERRCODE_DATATYPE_MISMATCH),
@ -92,8 +93,8 @@ domain_state_setup(Oid domainType, bool binary, MemoryContext mcxt)
format_type_be(domainType))));
/* Find out the base type */
my_extra->typtypmod = -1;
baseType = getBaseTypeAndTypmod(domainType, &my_extra->typtypmod);
baseType = typentry->domainBaseType;
my_extra->typtypmod = typentry->domainBaseTypmod;
/* Look up underlying I/O function */
if (binary)

View File

@ -169,6 +169,11 @@ typedef struct CompositeIOData
*/
RecordIOData *record_io; /* metadata cache for populate_record() */
TupleDesc tupdesc; /* cached tuple descriptor */
/* these fields differ from target type only if domain over composite: */
Oid base_typid; /* base type id */
int32 base_typmod; /* base type modifier */
/* this field is used only if target type is domain over composite: */
void *domain_info; /* opaque cache for domain checks */
} CompositeIOData;
/* structure to cache metadata needed for populate_domain() */
@ -186,6 +191,7 @@ typedef enum TypeCat
TYPECAT_SCALAR = 's',
TYPECAT_ARRAY = 'a',
TYPECAT_COMPOSITE = 'c',
TYPECAT_COMPOSITE_DOMAIN = 'C',
TYPECAT_DOMAIN = 'd'
} TypeCat;
@ -217,7 +223,15 @@ struct RecordIOData
ColumnIOData columns[FLEXIBLE_ARRAY_MEMBER];
};
/* state for populate_recordset */
/* per-query cache for populate_recordset */
typedef struct PopulateRecordsetCache
{
Oid argtype; /* declared type of the record argument */
ColumnIOData c; /* metadata cache for populate_composite() */
MemoryContext fn_mcxt; /* where this is stored */
} PopulateRecordsetCache;
/* per-call state for populate_recordset */
typedef struct PopulateRecordsetState
{
JsonLexContext *lex;
@ -227,17 +241,15 @@ typedef struct PopulateRecordsetState
char *save_json_start;
JsonTokenType saved_token_type;
Tuplestorestate *tuple_store;
TupleDesc ret_tdesc;
HeapTupleHeader rec;
RecordIOData **my_extra;
MemoryContext fn_mcxt; /* used to stash IO funcs */
PopulateRecordsetCache *cache;
} PopulateRecordsetState;
/* structure to cache metadata needed for populate_record_worker() */
typedef struct PopulateRecordCache
{
Oid argtype; /* verified row type of the first argument */
CompositeIOData io; /* metadata cache for populate_composite() */
Oid argtype; /* declared type of the record argument */
ColumnIOData c; /* metadata cache for populate_composite() */
} PopulateRecordCache;
/* common data for populate_array_json() and populate_array_dim_jsonb() */
@ -415,16 +427,13 @@ static Datum populate_record_worker(FunctionCallInfo fcinfo, const char *funcnam
static HeapTupleHeader populate_record(TupleDesc tupdesc, RecordIOData **record_p,
HeapTupleHeader defaultval, MemoryContext mcxt,
JsObject *obj);
static Datum populate_record_field(ColumnIOData *col, Oid typid, int32 typmod,
const char *colname, MemoryContext mcxt,
Datum defaultval, JsValue *jsv, bool *isnull);
static void JsValueToJsObject(JsValue *jsv, JsObject *jso);
static Datum populate_composite(CompositeIOData *io, Oid typid, int32 typmod,
static Datum populate_composite(CompositeIOData *io, Oid typid,
const char *colname, MemoryContext mcxt,
HeapTupleHeader defaultval, JsValue *jsv);
HeapTupleHeader defaultval, JsValue *jsv, bool isnull);
static Datum populate_scalar(ScalarIOData *io, Oid typid, int32 typmod, JsValue *jsv);
static void prepare_column_cache(ColumnIOData *column, Oid typid, int32 typmod,
MemoryContext mcxt, bool json);
MemoryContext mcxt, bool need_scalar);
static Datum populate_record_field(ColumnIOData *col, Oid typid, int32 typmod,
const char *colname, MemoryContext mcxt, Datum defaultval,
JsValue *jsv, bool *isnull);
@ -2704,25 +2713,16 @@ JsValueToJsObject(JsValue *jsv, JsObject *jso)
}
}
/* recursively populate a composite (row type) value from json/jsonb */
static Datum
populate_composite(CompositeIOData *io,
Oid typid,
int32 typmod,
const char *colname,
MemoryContext mcxt,
HeapTupleHeader defaultval,
JsValue *jsv)
/* acquire or update cached tuple descriptor for a composite type */
static void
update_cached_tupdesc(CompositeIOData *io, MemoryContext mcxt)
{
HeapTupleHeader tuple;
JsObject jso;
/* acquire cached tuple descriptor */
if (!io->tupdesc ||
io->tupdesc->tdtypeid != typid ||
io->tupdesc->tdtypmod != typmod)
io->tupdesc->tdtypeid != io->base_typid ||
io->tupdesc->tdtypmod != io->base_typmod)
{
TupleDesc tupdesc = lookup_rowtype_tupdesc(typid, typmod);
TupleDesc tupdesc = lookup_rowtype_tupdesc(io->base_typid,
io->base_typmod);
MemoryContext oldcxt;
if (io->tupdesc)
@ -2735,17 +2735,50 @@ populate_composite(CompositeIOData *io,
ReleaseTupleDesc(tupdesc);
}
}
/* prepare input value */
JsValueToJsObject(jsv, &jso);
/* recursively populate a composite (row type) value from json/jsonb */
static Datum
populate_composite(CompositeIOData *io,
Oid typid,
const char *colname,
MemoryContext mcxt,
HeapTupleHeader defaultval,
JsValue *jsv,
bool isnull)
{
Datum result;
/* populate resulting record tuple */
tuple = populate_record(io->tupdesc, &io->record_io,
defaultval, mcxt, &jso);
/* acquire/update cached tuple descriptor */
update_cached_tupdesc(io, mcxt);
JsObjectFree(&jso);
if (isnull)
result = (Datum) 0;
else
{
HeapTupleHeader tuple;
JsObject jso;
return HeapTupleHeaderGetDatum(tuple);
/* prepare input value */
JsValueToJsObject(jsv, &jso);
/* populate resulting record tuple */
tuple = populate_record(io->tupdesc, &io->record_io,
defaultval, mcxt, &jso);
result = HeapTupleHeaderGetDatum(tuple);
JsObjectFree(&jso);
}
/*
* If it's domain over composite, check domain constraints. (This should
* probably get refactored so that we can see the TYPECAT value, but for
* now, we can tell by comparing typid to base_typid.)
*/
if (typid != io->base_typid && typid != RECORDOID)
domain_check(result, isnull, typid, &io->domain_info, mcxt);
return result;
}
/* populate non-null scalar value from json/jsonb value */
@ -2867,7 +2900,7 @@ prepare_column_cache(ColumnIOData *column,
Oid typid,
int32 typmod,
MemoryContext mcxt,
bool json)
bool need_scalar)
{
HeapTuple tup;
Form_pg_type type;
@ -2883,18 +2916,43 @@ prepare_column_cache(ColumnIOData *column,
if (type->typtype == TYPTYPE_DOMAIN)
{
column->typcat = TYPECAT_DOMAIN;
column->io.domain.base_typid = type->typbasetype;
column->io.domain.base_typmod = type->typtypmod;
column->io.domain.base_io = MemoryContextAllocZero(mcxt,
sizeof(ColumnIOData));
column->io.domain.domain_info = NULL;
/*
* We can move directly to the bottom base type; domain_check() will
* take care of checking all constraints for a stack of domains.
*/
Oid base_typid;
int32 base_typmod = typmod;
base_typid = getBaseTypeAndTypmod(typid, &base_typmod);
if (get_typtype(base_typid) == TYPTYPE_COMPOSITE)
{
/* domain over composite has its own code path */
column->typcat = TYPECAT_COMPOSITE_DOMAIN;
column->io.composite.record_io = NULL;
column->io.composite.tupdesc = NULL;
column->io.composite.base_typid = base_typid;
column->io.composite.base_typmod = base_typmod;
column->io.composite.domain_info = NULL;
}
else
{
/* domain over anything else */
column->typcat = TYPECAT_DOMAIN;
column->io.domain.base_typid = base_typid;
column->io.domain.base_typmod = base_typmod;
column->io.domain.base_io =
MemoryContextAllocZero(mcxt, sizeof(ColumnIOData));
column->io.domain.domain_info = NULL;
}
}
else if (type->typtype == TYPTYPE_COMPOSITE || typid == RECORDOID)
{
column->typcat = TYPECAT_COMPOSITE;
column->io.composite.record_io = NULL;
column->io.composite.tupdesc = NULL;
column->io.composite.base_typid = typid;
column->io.composite.base_typmod = typmod;
column->io.composite.domain_info = NULL;
}
else if (type->typlen == -1 && OidIsValid(type->typelem))
{
@ -2906,10 +2964,13 @@ prepare_column_cache(ColumnIOData *column,
column->io.array.element_typmod = typmod;
}
else
{
column->typcat = TYPECAT_SCALAR;
need_scalar = true;
}
/* don't need input function when converting from jsonb to jsonb */
if (json || typid != JSONBOID)
/* caller can force us to look up scalar_io info even for non-scalars */
if (need_scalar)
{
Oid typioproc;
@ -2935,9 +2996,12 @@ populate_record_field(ColumnIOData *col,
check_stack_depth();
/* prepare column metadata cache for the given type */
/*
* Prepare column metadata cache for the given type. Force lookup of the
* scalar_io data so that the json string hack below will work.
*/
if (col->typid != typid || col->typmod != typmod)
prepare_column_cache(col, typid, typmod, mcxt, jsv->is_json);
prepare_column_cache(col, typid, typmod, mcxt, true);
*isnull = JsValueIsNull(jsv);
@ -2945,11 +3009,15 @@ populate_record_field(ColumnIOData *col,
/* try to convert json string to a non-scalar type through input function */
if (JsValueIsString(jsv) &&
(typcat == TYPECAT_ARRAY || typcat == TYPECAT_COMPOSITE))
(typcat == TYPECAT_ARRAY ||
typcat == TYPECAT_COMPOSITE ||
typcat == TYPECAT_COMPOSITE_DOMAIN))
typcat = TYPECAT_SCALAR;
/* we must perform domain checks for NULLs */
if (*isnull && typcat != TYPECAT_DOMAIN)
/* we must perform domain checks for NULLs, otherwise exit immediately */
if (*isnull &&
typcat != TYPECAT_DOMAIN &&
typcat != TYPECAT_COMPOSITE_DOMAIN)
return (Datum) 0;
switch (typcat)
@ -2961,12 +3029,13 @@ populate_record_field(ColumnIOData *col,
return populate_array(&col->io.array, colname, mcxt, jsv);
case TYPECAT_COMPOSITE:
return populate_composite(&col->io.composite, typid, typmod,
case TYPECAT_COMPOSITE_DOMAIN:
return populate_composite(&col->io.composite, typid,
colname, mcxt,
DatumGetPointer(defaultval)
? DatumGetHeapTupleHeader(defaultval)
: NULL,
jsv);
jsv, *isnull);
case TYPECAT_DOMAIN:
return populate_domain(&col->io.domain, typid, colname, mcxt,
@ -3137,10 +3206,7 @@ populate_record_worker(FunctionCallInfo fcinfo, const char *funcname,
int json_arg_num = have_record_arg ? 1 : 0;
Oid jtype = get_fn_expr_argtype(fcinfo->flinfo, json_arg_num);
JsValue jsv = {0};
HeapTupleHeader rec = NULL;
Oid tupType;
int32 tupTypmod;
TupleDesc tupdesc = NULL;
HeapTupleHeader rec;
Datum rettuple;
JsonbValue jbv;
MemoryContext fnmcxt = fcinfo->flinfo->fn_mcxt;
@ -3149,77 +3215,88 @@ populate_record_worker(FunctionCallInfo fcinfo, const char *funcname,
Assert(jtype == JSONOID || jtype == JSONBOID);
/*
* We arrange to look up the needed I/O info just once per series of
* calls, assuming the record type doesn't change underneath us.
* If first time through, identify input/result record type. Note that
* this stanza looks only at fcinfo context, which can't change during the
* query; so we may not be able to fully resolve a RECORD input type yet.
*/
if (!cache)
{
fcinfo->flinfo->fn_extra = cache =
MemoryContextAllocZero(fnmcxt, sizeof(*cache));
if (have_record_arg)
{
Oid argtype = get_fn_expr_argtype(fcinfo->flinfo, 0);
if (cache->argtype != argtype)
if (have_record_arg)
{
if (!type_is_rowtype(argtype))
/*
* json{b}_populate_record case: result type will be same as first
* argument's.
*/
cache->argtype = get_fn_expr_argtype(fcinfo->flinfo, 0);
prepare_column_cache(&cache->c,
cache->argtype, -1,
fnmcxt, false);
if (cache->c.typcat != TYPECAT_COMPOSITE &&
cache->c.typcat != TYPECAT_COMPOSITE_DOMAIN)
ereport(ERROR,
(errcode(ERRCODE_DATATYPE_MISMATCH),
errmsg("first argument of %s must be a row type",
funcname)));
cache->argtype = argtype;
}
if (PG_ARGISNULL(0))
{
if (PG_ARGISNULL(1))
PG_RETURN_NULL();
/*
* We have no tuple to look at, so the only source of type info is
* the argtype. The lookup_rowtype_tupdesc call below will error
* out if we don't have a known composite type oid here.
*/
tupType = argtype;
tupTypmod = -1;
}
else
{
rec = PG_GETARG_HEAPTUPLEHEADER(0);
/*
* json{b}_to_record case: result type is specified by calling
* query. Here it is syntactically impossible to specify the
* target type as domain-over-composite.
*/
TupleDesc tupdesc;
MemoryContext old_cxt;
if (PG_ARGISNULL(1))
PG_RETURN_POINTER(rec);
if (get_call_result_type(fcinfo, NULL, &tupdesc) != TYPEFUNC_COMPOSITE)
ereport(ERROR,
(errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
errmsg("function returning record called in context "
"that cannot accept type record"),
errhint("Try calling the function in the FROM clause "
"using a column definition list.")));
/* Extract type info from the tuple itself */
tupType = HeapTupleHeaderGetTypeId(rec);
tupTypmod = HeapTupleHeaderGetTypMod(rec);
Assert(tupdesc);
cache->argtype = tupdesc->tdtypeid;
/* Save identified tupdesc */
old_cxt = MemoryContextSwitchTo(fnmcxt);
cache->c.io.composite.tupdesc = CreateTupleDescCopy(tupdesc);
cache->c.io.composite.base_typid = tupdesc->tdtypeid;
cache->c.io.composite.base_typmod = tupdesc->tdtypmod;
MemoryContextSwitchTo(old_cxt);
}
}
/* Collect record arg if we have one */
if (have_record_arg && !PG_ARGISNULL(0))
{
rec = PG_GETARG_HEAPTUPLEHEADER(0);
/*
* When declared arg type is RECORD, identify actual record type from
* the tuple itself. Note the lookup_rowtype_tupdesc call in
* update_cached_tupdesc will fail if we're unable to do this.
*/
if (cache->argtype == RECORDOID)
{
cache->c.io.composite.base_typid = HeapTupleHeaderGetTypeId(rec);
cache->c.io.composite.base_typmod = HeapTupleHeaderGetTypMod(rec);
}
}
else
rec = NULL;
/* If no JSON argument, just return the record (if any) unchanged */
if (PG_ARGISNULL(json_arg_num))
{
/* json{b}_to_record case */
if (PG_ARGISNULL(0))
if (rec)
PG_RETURN_POINTER(rec);
else
PG_RETURN_NULL();
if (get_call_result_type(fcinfo, NULL, &tupdesc) != TYPEFUNC_COMPOSITE)
ereport(ERROR,
(errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
errmsg("function returning record called in context "
"that cannot accept type record"),
errhint("Try calling the function in the FROM clause "
"using a column definition list.")));
Assert(tupdesc);
/*
* Add tupdesc to the cache and set the appropriate values of
* tupType/tupTypmod for proper cache usage in populate_composite().
*/
cache->io.tupdesc = tupdesc;
tupType = tupdesc->tdtypeid;
tupTypmod = tupdesc->tdtypmod;
}
jsv.is_json = jtype == JSONOID;
@ -3245,14 +3322,8 @@ populate_record_worker(FunctionCallInfo fcinfo, const char *funcname,
jbv.val.binary.len = VARSIZE(jb) - VARHDRSZ;
}
rettuple = populate_composite(&cache->io, tupType, tupTypmod,
NULL, fnmcxt, rec, &jsv);
if (tupdesc)
{
cache->io.tupdesc = NULL;
ReleaseTupleDesc(tupdesc);
}
rettuple = populate_composite(&cache->c.io.composite, cache->argtype,
NULL, fnmcxt, rec, &jsv, false);
PG_RETURN_DATUM(rettuple);
}
@ -3438,13 +3509,28 @@ json_to_recordset(PG_FUNCTION_ARGS)
static void
populate_recordset_record(PopulateRecordsetState *state, JsObject *obj)
{
PopulateRecordsetCache *cache = state->cache;
HeapTupleHeader tuphead;
HeapTupleData tuple;
HeapTupleHeader tuphead = populate_record(state->ret_tdesc,
state->my_extra,
state->rec,
state->fn_mcxt,
obj);
/* acquire/update cached tuple descriptor */
update_cached_tupdesc(&cache->c.io.composite, cache->fn_mcxt);
/* replace record fields from json */
tuphead = populate_record(cache->c.io.composite.tupdesc,
&cache->c.io.composite.record_io,
state->rec,
cache->fn_mcxt,
obj);
/* if it's domain over composite, check domain constraints */
if (cache->c.typcat == TYPECAT_COMPOSITE_DOMAIN)
domain_check(HeapTupleHeaderGetDatum(tuphead), false,
cache->argtype,
&cache->c.io.composite.domain_info,
cache->fn_mcxt);
/* ok, save into tuplestore */
tuple.t_len = HeapTupleHeaderGetDatumLength(tuphead);
ItemPointerSetInvalid(&(tuple.t_self));
tuple.t_tableOid = InvalidOid;
@ -3465,25 +3551,13 @@ populate_recordset_worker(FunctionCallInfo fcinfo, const char *funcname,
ReturnSetInfo *rsi;
MemoryContext old_cxt;
HeapTupleHeader rec;
TupleDesc tupdesc;
PopulateRecordsetCache *cache = fcinfo->flinfo->fn_extra;
PopulateRecordsetState *state;
if (have_record_arg)
{
Oid argtype = get_fn_expr_argtype(fcinfo->flinfo, 0);
if (!type_is_rowtype(argtype))
ereport(ERROR,
(errcode(ERRCODE_DATATYPE_MISMATCH),
errmsg("first argument of %s must be a row type",
funcname)));
}
rsi = (ReturnSetInfo *) fcinfo->resultinfo;
if (!rsi || !IsA(rsi, ReturnSetInfo) ||
(rsi->allowedModes & SFRM_Materialize) == 0 ||
rsi->expectedDesc == NULL)
(rsi->allowedModes & SFRM_Materialize) == 0)
ereport(ERROR,
(errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
errmsg("set-valued function called in context that "
@ -3492,40 +3566,97 @@ populate_recordset_worker(FunctionCallInfo fcinfo, const char *funcname,
rsi->returnMode = SFRM_Materialize;
/*
* get the tupdesc from the result set info - it must be a record type
* because we already checked that arg1 is a record type, or we're in a
* to_record function which returns a setof record.
* If first time through, identify input/result record type. Note that
* this stanza looks only at fcinfo context, which can't change during the
* query; so we may not be able to fully resolve a RECORD input type yet.
*/
if (get_call_result_type(fcinfo, NULL, &tupdesc) != TYPEFUNC_COMPOSITE)
ereport(ERROR,
(errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
errmsg("function returning record called in context "
"that cannot accept type record")));
if (!cache)
{
fcinfo->flinfo->fn_extra = cache =
MemoryContextAllocZero(fcinfo->flinfo->fn_mcxt, sizeof(*cache));
cache->fn_mcxt = fcinfo->flinfo->fn_mcxt;
if (have_record_arg)
{
/*
* json{b}_populate_recordset case: result type will be same as
* first argument's.
*/
cache->argtype = get_fn_expr_argtype(fcinfo->flinfo, 0);
prepare_column_cache(&cache->c,
cache->argtype, -1,
cache->fn_mcxt, false);
if (cache->c.typcat != TYPECAT_COMPOSITE &&
cache->c.typcat != TYPECAT_COMPOSITE_DOMAIN)
ereport(ERROR,
(errcode(ERRCODE_DATATYPE_MISMATCH),
errmsg("first argument of %s must be a row type",
funcname)));
}
else
{
/*
* json{b}_to_recordset case: result type is specified by calling
* query. Here it is syntactically impossible to specify the
* target type as domain-over-composite.
*/
TupleDesc tupdesc;
if (get_call_result_type(fcinfo, NULL, &tupdesc) != TYPEFUNC_COMPOSITE)
ereport(ERROR,
(errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
errmsg("function returning record called in context "
"that cannot accept type record"),
errhint("Try calling the function in the FROM clause "
"using a column definition list.")));
Assert(tupdesc);
cache->argtype = tupdesc->tdtypeid;
/* Save identified tupdesc */
old_cxt = MemoryContextSwitchTo(cache->fn_mcxt);
cache->c.io.composite.tupdesc = CreateTupleDescCopy(tupdesc);
cache->c.io.composite.base_typid = tupdesc->tdtypeid;
cache->c.io.composite.base_typmod = tupdesc->tdtypmod;
MemoryContextSwitchTo(old_cxt);
}
}
/* Collect record arg if we have one */
if (have_record_arg && !PG_ARGISNULL(0))
{
rec = PG_GETARG_HEAPTUPLEHEADER(0);
/*
* When declared arg type is RECORD, identify actual record type from
* the tuple itself. Note the lookup_rowtype_tupdesc call in
* update_cached_tupdesc will fail if we're unable to do this.
*/
if (cache->argtype == RECORDOID)
{
cache->c.io.composite.base_typid = HeapTupleHeaderGetTypeId(rec);
cache->c.io.composite.base_typmod = HeapTupleHeaderGetTypMod(rec);
}
}
else
rec = NULL;
/* if the json is null send back an empty set */
if (PG_ARGISNULL(json_arg_num))
PG_RETURN_NULL();
if (!have_record_arg || PG_ARGISNULL(0))
rec = NULL;
else
rec = PG_GETARG_HEAPTUPLEHEADER(0);
state = palloc0(sizeof(PopulateRecordsetState));
/* make these in a sufficiently long-lived memory context */
/* make tuplestore in a sufficiently long-lived memory context */
old_cxt = MemoryContextSwitchTo(rsi->econtext->ecxt_per_query_memory);
state->ret_tdesc = CreateTupleDescCopy(tupdesc);
BlessTupleDesc(state->ret_tdesc);
state->tuple_store = tuplestore_begin_heap(rsi->allowedModes &
SFRM_Materialize_Random,
false, work_mem);
MemoryContextSwitchTo(old_cxt);
state->function_name = funcname;
state->my_extra = (RecordIOData **) &fcinfo->flinfo->fn_extra;
state->cache = cache;
state->rec = rec;
state->fn_mcxt = fcinfo->flinfo->fn_mcxt;
if (jtype == JSONOID)
{
@ -3592,7 +3723,7 @@ populate_recordset_worker(FunctionCallInfo fcinfo, const char *funcname,
}
rsi->setResult = state->tuple_store;
rsi->setDesc = state->ret_tdesc;
rsi->setDesc = cache->c.io.composite.tupdesc;
PG_RETURN_NULL();
}

View File

@ -6731,17 +6731,12 @@ get_name_for_var_field(Var *var, int fieldno,
/*
* If it's a Var of type RECORD, we have to find what the Var refers to;
* if not, we can use get_expr_result_type. If that fails, we try
* lookup_rowtype_tupdesc, which will probably fail too, but will ereport
* an acceptable message.
* if not, we can use get_expr_result_tupdesc().
*/
if (!IsA(var, Var) ||
var->vartype != RECORDOID)
{
if (get_expr_result_type((Node *) var, NULL, &tupleDesc) != TYPEFUNC_COMPOSITE)
tupleDesc = lookup_rowtype_tupdesc_copy(exprType((Node *) var),
exprTypmod((Node *) var));
Assert(tupleDesc);
tupleDesc = get_expr_result_tupdesc((Node *) var, false);
/* Got the tupdesc, so we can extract the field name */
Assert(fieldno >= 1 && fieldno <= tupleDesc->natts);
return NameStr(TupleDescAttr(tupleDesc, fieldno - 1)->attname);
@ -7044,14 +7039,9 @@ get_name_for_var_field(Var *var, int fieldno,
/*
* We now have an expression we can't expand any more, so see if
* get_expr_result_type() can do anything with it. If not, pass to
* lookup_rowtype_tupdesc() which will probably fail, but will give an
* appropriate error message while failing.
* get_expr_result_tupdesc() can do anything with it.
*/
if (get_expr_result_type(expr, NULL, &tupleDesc) != TYPEFUNC_COMPOSITE)
tupleDesc = lookup_rowtype_tupdesc_copy(exprType(expr),
exprTypmod(expr));
Assert(tupleDesc);
tupleDesc = get_expr_result_tupdesc(expr, false);
/* Got the tupdesc, so we can extract the field name */
Assert(fieldno >= 1 && fieldno <= tupleDesc->natts);
return NameStr(TupleDescAttr(tupleDesc, fieldno - 1)->attname);

View File

@ -2398,12 +2398,26 @@ get_typtype(Oid typid)
* type_is_rowtype
*
* Convenience function to determine whether a type OID represents
* a "rowtype" type --- either RECORD or a named composite type.
* a "rowtype" type --- either RECORD or a named composite type
* (including a domain over a named composite type).
*/
bool
type_is_rowtype(Oid typid)
{
return (typid == RECORDOID || get_typtype(typid) == TYPTYPE_COMPOSITE);
if (typid == RECORDOID)
return true; /* easy case */
switch (get_typtype(typid))
{
case TYPTYPE_COMPOSITE:
return true;
case TYPTYPE_DOMAIN:
if (get_typtype(getBaseType(typid)) == TYPTYPE_COMPOSITE)
return true;
break;
default:
break;
}
return false;
}
/*

View File

@ -96,6 +96,7 @@ static TypeCacheEntry *firstDomainTypeEntry = NULL;
#define TCFLAGS_HAVE_FIELD_EQUALITY 0x004000
#define TCFLAGS_HAVE_FIELD_COMPARE 0x008000
#define TCFLAGS_CHECKED_DOMAIN_CONSTRAINTS 0x010000
#define TCFLAGS_DOMAIN_BASE_IS_COMPOSITE 0x020000
/*
* Data stored about a domain type's constraints. Note that we do not create
@ -747,7 +748,15 @@ lookup_type_cache(Oid type_id, int flags)
/*
* If requested, get information about a domain type
*/
if ((flags & TYPECACHE_DOMAIN_INFO) &&
if ((flags & TYPECACHE_DOMAIN_BASE_INFO) &&
typentry->domainBaseType == InvalidOid &&
typentry->typtype == TYPTYPE_DOMAIN)
{
typentry->domainBaseTypmod = -1;
typentry->domainBaseType =
getBaseTypeAndTypmod(type_id, &typentry->domainBaseTypmod);
}
if ((flags & TYPECACHE_DOMAIN_CONSTR_INFO) &&
(typentry->flags & TCFLAGS_CHECKED_DOMAIN_CONSTRAINTS) == 0 &&
typentry->typtype == TYPTYPE_DOMAIN)
{
@ -1166,7 +1175,7 @@ InitDomainConstraintRef(Oid type_id, DomainConstraintRef *ref,
MemoryContext refctx, bool need_exprstate)
{
/* Look up the typcache entry --- we assume it survives indefinitely */
ref->tcache = lookup_type_cache(type_id, TYPECACHE_DOMAIN_INFO);
ref->tcache = lookup_type_cache(type_id, TYPECACHE_DOMAIN_CONSTR_INFO);
ref->need_exprstate = need_exprstate;
/* For safety, establish the callback before acquiring a refcount */
ref->refctx = refctx;
@ -1257,7 +1266,7 @@ DomainHasConstraints(Oid type_id)
* Note: a side effect is to cause the typcache's domain data to become
* valid. This is fine since we'll likely need it soon if there is any.
*/
typentry = lookup_type_cache(type_id, TYPECACHE_DOMAIN_INFO);
typentry = lookup_type_cache(type_id, TYPECACHE_DOMAIN_CONSTR_INFO);
return (typentry->domainData != NULL);
}
@ -1405,6 +1414,29 @@ cache_record_field_properties(TypeCacheEntry *typentry)
DecrTupleDescRefCount(tupdesc);
}
else if (typentry->typtype == TYPTYPE_DOMAIN)
{
/* If it's domain over composite, copy base type's properties */
TypeCacheEntry *baseentry;
/* load up basetype info if we didn't already */
if (typentry->domainBaseType == InvalidOid)
{
typentry->domainBaseTypmod = -1;
typentry->domainBaseType =
getBaseTypeAndTypmod(typentry->type_id,
&typentry->domainBaseTypmod);
}
baseentry = lookup_type_cache(typentry->domainBaseType,
TYPECACHE_EQ_OPR |
TYPECACHE_CMP_PROC);
if (baseentry->typtype == TYPTYPE_COMPOSITE)
{
typentry->flags |= TCFLAGS_DOMAIN_BASE_IS_COMPOSITE;
typentry->flags |= baseentry->flags & (TCFLAGS_HAVE_FIELD_EQUALITY |
TCFLAGS_HAVE_FIELD_COMPARE);
}
}
typentry->flags |= TCFLAGS_CHECKED_FIELD_PROPERTIES;
}
@ -1618,6 +1650,53 @@ lookup_rowtype_tupdesc_copy(Oid type_id, int32 typmod)
return CreateTupleDescCopyConstr(tmp);
}
/*
* lookup_rowtype_tupdesc_domain
*
* Same as lookup_rowtype_tupdesc_noerror(), except that the type can also be
* a domain over a named composite type; so this is effectively equivalent to
* lookup_rowtype_tupdesc_noerror(getBaseType(type_id), typmod, noError)
* except for being a tad faster.
*
* Note: the reason we don't fold the look-through-domain behavior into plain
* lookup_rowtype_tupdesc() is that we want callers to know they might be
* dealing with a domain. Otherwise they might construct a tuple that should
* be of the domain type, but not apply domain constraints.
*/
TupleDesc
lookup_rowtype_tupdesc_domain(Oid type_id, int32 typmod, bool noError)
{
TupleDesc tupDesc;
if (type_id != RECORDOID)
{
/*
* Check for domain or named composite type. We might as well load
* whichever data is needed.
*/
TypeCacheEntry *typentry;
typentry = lookup_type_cache(type_id,
TYPECACHE_TUPDESC |
TYPECACHE_DOMAIN_BASE_INFO);
if (typentry->typtype == TYPTYPE_DOMAIN)
return lookup_rowtype_tupdesc_noerror(typentry->domainBaseType,
typentry->domainBaseTypmod,
noError);
if (typentry->tupDesc == NULL && !noError)
ereport(ERROR,
(errcode(ERRCODE_WRONG_OBJECT_TYPE),
errmsg("type %s is not composite",
format_type_be(type_id))));
tupDesc = typentry->tupDesc;
}
else
tupDesc = lookup_rowtype_tupdesc_internal(type_id, typmod, noError);
if (tupDesc != NULL)
PinTupleDesc(tupDesc);
return tupDesc;
}
/*
* Hash function for the hash table of RecordCacheEntry.
*/
@ -1929,29 +2008,40 @@ TypeCacheRelCallback(Datum arg, Oid relid)
hash_seq_init(&status, TypeCacheHash);
while ((typentry = (TypeCacheEntry *) hash_seq_search(&status)) != NULL)
{
if (typentry->typtype != TYPTYPE_COMPOSITE)
continue; /* skip non-composites */
if (typentry->typtype == TYPTYPE_COMPOSITE)
{
/* Skip if no match, unless we're zapping all composite types */
if (relid != typentry->typrelid && relid != InvalidOid)
continue;
/* Skip if no match, unless we're zapping all composite types */
if (relid != typentry->typrelid && relid != InvalidOid)
continue;
/* Delete tupdesc if we have it */
if (typentry->tupDesc != NULL)
{
/*
* Release our refcount, and free the tupdesc if none remain.
* (Can't use DecrTupleDescRefCount because this reference is
* not logged in current resource owner.)
*/
Assert(typentry->tupDesc->tdrefcount > 0);
if (--typentry->tupDesc->tdrefcount == 0)
FreeTupleDesc(typentry->tupDesc);
typentry->tupDesc = NULL;
}
/* Delete tupdesc if we have it */
if (typentry->tupDesc != NULL)
/* Reset equality/comparison/hashing validity information */
typentry->flags = 0;
}
else if (typentry->typtype == TYPTYPE_DOMAIN)
{
/*
* Release our refcount, and free the tupdesc if none remain.
* (Can't use DecrTupleDescRefCount because this reference is not
* logged in current resource owner.)
* If it's domain over composite, reset flags. (We don't bother
* trying to determine whether the specific base type needs a
* reset.) Note that if we haven't determined whether the base
* type is composite, we don't need to reset anything.
*/
Assert(typentry->tupDesc->tdrefcount > 0);
if (--typentry->tupDesc->tdrefcount == 0)
FreeTupleDesc(typentry->tupDesc);
typentry->tupDesc = NULL;
if (typentry->flags & TCFLAGS_DOMAIN_BASE_IS_COMPOSITE)
typentry->flags = 0;
}
/* Reset equality/comparison/hashing validity information */
typentry->flags = 0;
}
}

View File

@ -39,7 +39,7 @@ static TypeFuncClass internal_get_result_type(Oid funcid,
static bool resolve_polymorphic_tupdesc(TupleDesc tupdesc,
oidvector *declared_args,
Node *call_expr);
static TypeFuncClass get_type_func_class(Oid typid);
static TypeFuncClass get_type_func_class(Oid typid, Oid *base_typeid);
/*
@ -246,14 +246,17 @@ get_expr_result_type(Node *expr,
{
/* handle as a generic expression; no chance to resolve RECORD */
Oid typid = exprType(expr);
Oid base_typid;
if (resultTypeId)
*resultTypeId = typid;
if (resultTupleDesc)
*resultTupleDesc = NULL;
result = get_type_func_class(typid);
if (result == TYPEFUNC_COMPOSITE && resultTupleDesc)
*resultTupleDesc = lookup_rowtype_tupdesc_copy(typid, -1);
result = get_type_func_class(typid, &base_typid);
if ((result == TYPEFUNC_COMPOSITE ||
result == TYPEFUNC_COMPOSITE_DOMAIN) &&
resultTupleDesc)
*resultTupleDesc = lookup_rowtype_tupdesc_copy(base_typid, -1);
}
return result;
@ -296,6 +299,7 @@ internal_get_result_type(Oid funcid,
HeapTuple tp;
Form_pg_proc procform;
Oid rettype;
Oid base_rettype;
TupleDesc tupdesc;
/* First fetch the function's pg_proc row to inspect its rettype */
@ -363,12 +367,13 @@ internal_get_result_type(Oid funcid,
*resultTupleDesc = NULL; /* default result */
/* Classify the result type */
result = get_type_func_class(rettype);
result = get_type_func_class(rettype, &base_rettype);
switch (result)
{
case TYPEFUNC_COMPOSITE:
case TYPEFUNC_COMPOSITE_DOMAIN:
if (resultTupleDesc)
*resultTupleDesc = lookup_rowtype_tupdesc_copy(rettype, -1);
*resultTupleDesc = lookup_rowtype_tupdesc_copy(base_rettype, -1);
/* Named composite types can't have any polymorphic columns */
break;
case TYPEFUNC_SCALAR:
@ -393,6 +398,46 @@ internal_get_result_type(Oid funcid,
return result;
}
/*
* get_expr_result_tupdesc
* Get a tupdesc describing the result of a composite-valued expression
*
* If expression is not composite or rowtype can't be determined, returns NULL
* if noError is true, else throws error.
*
* This is a simpler version of get_expr_result_type() for use when the caller
* is only interested in determinate rowtype results.
*/
TupleDesc
get_expr_result_tupdesc(Node *expr, bool noError)
{
TupleDesc tupleDesc;
TypeFuncClass functypclass;
functypclass = get_expr_result_type(expr, NULL, &tupleDesc);
if (functypclass == TYPEFUNC_COMPOSITE ||
functypclass == TYPEFUNC_COMPOSITE_DOMAIN)
return tupleDesc;
if (!noError)
{
Oid exprTypeId = exprType(expr);
if (exprTypeId != RECORDOID)
ereport(ERROR,
(errcode(ERRCODE_WRONG_OBJECT_TYPE),
errmsg("type %s is not composite",
format_type_be(exprTypeId))));
else
ereport(ERROR,
(errcode(ERRCODE_WRONG_OBJECT_TYPE),
errmsg("record type has not been registered")));
}
return NULL;
}
/*
* Given the result tuple descriptor for a function with OUT parameters,
* replace any polymorphic columns (ANYELEMENT etc) with correct data types
@ -741,23 +786,31 @@ resolve_polymorphic_argtypes(int numargs, Oid *argtypes, char *argmodes,
/*
* get_type_func_class
* Given the type OID, obtain its TYPEFUNC classification.
* Also, if it's a domain, return the base type OID.
*
* This is intended to centralize a bunch of formerly ad-hoc code for
* classifying types. The categories used here are useful for deciding
* how to handle functions returning the datatype.
*/
static TypeFuncClass
get_type_func_class(Oid typid)
get_type_func_class(Oid typid, Oid *base_typeid)
{
*base_typeid = typid;
switch (get_typtype(typid))
{
case TYPTYPE_COMPOSITE:
return TYPEFUNC_COMPOSITE;
case TYPTYPE_BASE:
case TYPTYPE_DOMAIN:
case TYPTYPE_ENUM:
case TYPTYPE_RANGE:
return TYPEFUNC_SCALAR;
case TYPTYPE_DOMAIN:
*base_typeid = typid = getBaseType(typid);
if (get_typtype(typid) == TYPTYPE_COMPOSITE)
return TYPEFUNC_COMPOSITE_DOMAIN;
else /* domain base type can't be a pseudotype */
return TYPEFUNC_SCALAR;
case TYPTYPE_PSEUDO:
if (typid == RECORDOID)
return TYPEFUNC_RECORD;
@ -1320,16 +1373,20 @@ RelationNameGetTupleDesc(const char *relname)
TupleDesc
TypeGetTupleDesc(Oid typeoid, List *colaliases)
{
TypeFuncClass functypclass = get_type_func_class(typeoid);
Oid base_typeoid;
TypeFuncClass functypclass = get_type_func_class(typeoid, &base_typeoid);
TupleDesc tupdesc = NULL;
/*
* Build a suitable tupledesc representing the output rows
* Build a suitable tupledesc representing the output rows. We
* intentionally do not support TYPEFUNC_COMPOSITE_DOMAIN here, as it's
* unlikely that legacy callers of this obsolete function would be
* prepared to apply domain constraints.
*/
if (functypclass == TYPEFUNC_COMPOSITE)
{
/* Composite data type, e.g. a table's row type */
tupdesc = lookup_rowtype_tupdesc_copy(typeoid, -1);
tupdesc = lookup_rowtype_tupdesc_copy(base_typeoid, -1);
if (colaliases != NIL)
{
@ -1424,7 +1481,8 @@ extract_variadic_args(FunctionCallInfo fcinfo, int variadic_start,
Datum *args_res;
bool *nulls_res;
Oid *types_res;
int nargs, i;
int nargs,
i;
*args = NULL;
*types = NULL;
@ -1460,7 +1518,7 @@ extract_variadic_args(FunctionCallInfo fcinfo, int variadic_start,
else
{
nargs = PG_NARGS() - variadic_start;
Assert (nargs > 0);
Assert(nargs > 0);
nulls_res = (bool *) palloc0(nargs * sizeof(bool));
args_res = (Datum *) palloc0(nargs * sizeof(Datum));
types_res = (Oid *) palloc0(nargs * sizeof(Oid));
@ -1473,11 +1531,10 @@ extract_variadic_args(FunctionCallInfo fcinfo, int variadic_start,
/*
* Turn a constant (more or less literal) value that's of unknown
* type into text if required . Unknowns come in as a cstring
* pointer.
* Note: for functions declared as taking type "any", the parser
* will not do any type conversion on unknown-type literals (that
* is, undecorated strings or NULLs).
* type into text if required. Unknowns come in as a cstring
* pointer. Note: for functions declared as taking type "any", the
* parser will not do any type conversion on unknown-type literals
* (that is, undecorated strings or NULLs).
*/
if (convert_unknown &&
types_res[i] == UNKNOWNOID &&

View File

@ -134,6 +134,11 @@ typedef struct DatumTupleFields
Oid datum_typeid; /* composite type OID, or RECORDOID */
/*
* datum_typeid cannot be a domain over composite, only plain composite,
* even if the datum is meant as a value of a domain-over-composite type.
* This is in line with the general principle that CoerceToDomain does not
* change the physical representation of the base type value.
*
* Note: field ordering is chosen with thought that Oid might someday
* widen to 64 bits.
*/

View File

@ -60,6 +60,12 @@ typedef struct tupleConstr
* row type, or a value >= 0 to allow the rowtype to be looked up in the
* typcache.c type cache.
*
* Note that tdtypeid is never the OID of a domain over composite, even if
* we are dealing with values that are known (at some higher level) to be of
* a domain-over-composite type. This is because tdtypeid/tdtypmod need to
* match up with the type labeling of composite Datums, and those are never
* explicitly marked as being of a domain type, either.
*
* Tuple descriptors that live in caches (relcache or typcache, at present)
* are reference-counted: they can be deleted when their reference count goes
* to zero. Tuple descriptors created by the executor need no reference

View File

@ -144,6 +144,10 @@ typedef struct FuncCallContext
* get_call_result_type. Note: the cases in which rowtypes cannot be
* determined are different from the cases for get_call_result_type.
* Do *not* use this if you can use one of the others.
*
* See also get_expr_result_tupdesc(), which is a convenient wrapper around
* get_expr_result_type() for use when the caller only cares about
* determinable-rowtype cases.
*----------
*/
@ -152,6 +156,7 @@ typedef enum TypeFuncClass
{
TYPEFUNC_SCALAR, /* scalar result type */
TYPEFUNC_COMPOSITE, /* determinable rowtype result */
TYPEFUNC_COMPOSITE_DOMAIN, /* domain over determinable rowtype result */
TYPEFUNC_RECORD, /* indeterminate rowtype result */
TYPEFUNC_OTHER /* bogus type, eg pseudotype */
} TypeFuncClass;
@ -166,6 +171,8 @@ extern TypeFuncClass get_func_result_type(Oid functionId,
Oid *resultTypeId,
TupleDesc *resultTupleDesc);
extern TupleDesc get_expr_result_tupdesc(Node *expr, bool noError);
extern bool resolve_polymorphic_argtypes(int numargs, Oid *argtypes,
char *argmodes,
Node *call_expr);
@ -335,7 +342,7 @@ extern void end_MultiFuncCall(PG_FUNCTION_ARGS, FuncCallContext *funcctx);
* "VARIADIC NULL".
*/
extern int extract_variadic_args(FunctionCallInfo fcinfo, int variadic_start,
bool convert_unknown, Datum **values,
Oid **types, bool **nulls);
bool convert_unknown, Datum **values,
Oid **types, bool **nulls);
#endif /* FUNCAPI_H */

View File

@ -166,7 +166,7 @@ typedef struct Var
Index varno; /* index of this var's relation in the range
* table, or INNER_VAR/OUTER_VAR/INDEX_VAR */
AttrNumber varattno; /* attribute number of this var, or zero for
* all */
* all attrs ("whole-row Var") */
Oid vartype; /* pg_type OID for the type of this var */
int32 vartypmod; /* pg_attribute typmod value */
Oid varcollid; /* OID of collation, or InvalidOid if none */
@ -755,6 +755,9 @@ typedef struct FieldSelect
* the assign case of ArrayRef, this is used to implement UPDATE of a
* portion of a column.
*
* resulttype is always a named composite type (not a domain). To update
* a composite domain value, apply CoerceToDomain to the FieldStore.
*
* A single FieldStore can actually represent updates of several different
* fields. The parser only generates FieldStores with single-element lists,
* but the planner will collapse multiple updates of the same base column
@ -849,7 +852,8 @@ typedef struct ArrayCoerceExpr
* needed for the destination type plus possibly others; the columns need not
* be in the same positions, but are matched up by name. This is primarily
* used to convert a whole-row value of an inheritance child table into a
* valid whole-row value of its parent table's rowtype.
* valid whole-row value of its parent table's rowtype. Both resulttype
* and the exposed type of "arg" must be named composite types (not domains).
* ----------------
*/
@ -987,6 +991,9 @@ typedef struct RowExpr
Oid row_typeid; /* RECORDOID or a composite type's ID */
/*
* row_typeid cannot be a domain over composite, only plain composite. To
* create a composite domain value, apply CoerceToDomain to the RowExpr.
*
* Note: we deliberately do NOT store a typmod. Although a typmod will be
* associated with specific RECORD types at runtime, it will differ for
* different backends, and so cannot safely be stored in stored

View File

@ -46,10 +46,12 @@ extern Oid typeTypeCollation(Type typ);
extern Datum stringTypeDatum(Type tp, char *string, int32 atttypmod);
extern Oid typeidTypeRelid(Oid type_id);
extern Oid typeOrDomainTypeRelid(Oid type_id);
extern TypeName *typeStringToTypeName(const char *str);
extern void parseTypeString(const char *str, Oid *typeid_p, int32 *typmod_p, bool missing_ok);
#define ISCOMPLEX(typeid) (typeidTypeRelid(typeid) != InvalidOid)
/* true if typeid is composite, or domain over composite, but not RECORD */
#define ISCOMPLEX(typeid) (typeOrDomainTypeRelid(typeid) != InvalidOid)
#endif /* PARSE_TYPE_H */

View File

@ -91,6 +91,13 @@ typedef struct TypeCacheEntry
FmgrInfo rng_canonical_finfo; /* canonicalization function, if any */
FmgrInfo rng_subdiff_finfo; /* difference function, if any */
/*
* Domain's base type and typmod if it's a domain type. Zeroes if not
* domain, or if information hasn't been requested.
*/
Oid domainBaseType;
int32 domainBaseTypmod;
/*
* Domain constraint data if it's a domain type. NULL if not domain, or
* if domain has no constraints, or if information hasn't been requested.
@ -123,9 +130,10 @@ typedef struct TypeCacheEntry
#define TYPECACHE_BTREE_OPFAMILY 0x0200
#define TYPECACHE_HASH_OPFAMILY 0x0400
#define TYPECACHE_RANGE_INFO 0x0800
#define TYPECACHE_DOMAIN_INFO 0x1000
#define TYPECACHE_HASH_EXTENDED_PROC 0x2000
#define TYPECACHE_HASH_EXTENDED_PROC_FINFO 0x4000
#define TYPECACHE_DOMAIN_BASE_INFO 0x1000
#define TYPECACHE_DOMAIN_CONSTR_INFO 0x2000
#define TYPECACHE_HASH_EXTENDED_PROC 0x4000
#define TYPECACHE_HASH_EXTENDED_PROC_FINFO 0x8000
/*
* Callers wishing to maintain a long-lived reference to a domain's constraint
@ -163,6 +171,9 @@ extern TupleDesc lookup_rowtype_tupdesc_noerror(Oid type_id, int32 typmod,
extern TupleDesc lookup_rowtype_tupdesc_copy(Oid type_id, int32 typmod);
extern TupleDesc lookup_rowtype_tupdesc_domain(Oid type_id, int32 typmod,
bool noError);
extern void assign_record_type_typmod(TupleDesc tupDesc);
extern int compare_values_of_enum(TypeCacheEntry *tcache, Oid arg1, Oid arg2);

View File

@ -198,6 +198,94 @@ select pg_typeof('{1,2,3}'::dia || 42); -- should be int[] not dia
(1 row)
drop domain dia;
-- Test domains over composites
create type comptype as (r float8, i float8);
create domain dcomptype as comptype;
create table dcomptable (d1 dcomptype unique);
insert into dcomptable values (row(1,2)::dcomptype);
insert into dcomptable values (row(3,4)::comptype);
insert into dcomptable values (row(1,2)::dcomptype); -- fail on uniqueness
ERROR: duplicate key value violates unique constraint "dcomptable_d1_key"
DETAIL: Key (d1)=((1,2)) already exists.
insert into dcomptable (d1.r) values(11);
select * from dcomptable;
d1
-------
(1,2)
(3,4)
(11,)
(3 rows)
select (d1).r, (d1).i, (d1).* from dcomptable;
r | i | r | i
----+---+----+---
1 | 2 | 1 | 2
3 | 4 | 3 | 4
11 | | 11 |
(3 rows)
update dcomptable set d1.r = (d1).r + 1 where (d1).i > 0;
select * from dcomptable;
d1
-------
(11,)
(2,2)
(4,4)
(3 rows)
alter domain dcomptype add constraint c1 check ((value).r <= (value).i);
alter domain dcomptype add constraint c2 check ((value).r > (value).i); -- fail
ERROR: column "d1" of table "dcomptable" contains values that violate the new constraint
select row(2,1)::dcomptype; -- fail
ERROR: value for domain dcomptype violates check constraint "c1"
insert into dcomptable values (row(1,2)::comptype);
insert into dcomptable values (row(2,1)::comptype); -- fail
ERROR: value for domain dcomptype violates check constraint "c1"
insert into dcomptable (d1.r) values(99);
insert into dcomptable (d1.r, d1.i) values(99, 100);
insert into dcomptable (d1.r, d1.i) values(100, 99); -- fail
ERROR: value for domain dcomptype violates check constraint "c1"
update dcomptable set d1.r = (d1).r + 1 where (d1).i > 0; -- fail
ERROR: value for domain dcomptype violates check constraint "c1"
update dcomptable set d1.r = (d1).r - 1, d1.i = (d1).i + 1 where (d1).i > 0;
select * from dcomptable;
d1
----------
(11,)
(99,)
(1,3)
(3,5)
(0,3)
(98,101)
(6 rows)
explain (verbose, costs off)
update dcomptable set d1.r = (d1).r - 1, d1.i = (d1).i + 1 where (d1).i > 0;
QUERY PLAN
-----------------------------------------------------------------------------------------------
Update on public.dcomptable
-> Seq Scan on public.dcomptable
Output: ROW(((d1).r - '1'::double precision), ((d1).i + '1'::double precision)), ctid
Filter: ((dcomptable.d1).i > '0'::double precision)
(4 rows)
create rule silly as on delete to dcomptable do instead
update dcomptable set d1.r = (d1).r - 1, d1.i = (d1).i + 1 where (d1).i > 0;
\d+ dcomptable
Table "public.dcomptable"
Column | Type | Collation | Nullable | Default | Storage | Stats target | Description
--------+-----------+-----------+----------+---------+----------+--------------+-------------
d1 | dcomptype | | | | extended | |
Indexes:
"dcomptable_d1_key" UNIQUE CONSTRAINT, btree (d1)
Rules:
silly AS
ON DELETE TO dcomptable DO INSTEAD UPDATE dcomptable SET d1.r = (dcomptable.d1).r - 1::double precision, d1.i = (dcomptable.d1).i + 1::double precision
WHERE (dcomptable.d1).i > 0::double precision
drop table dcomptable;
drop type comptype cascade;
NOTICE: drop cascades to type dcomptype
-- Test domains over arrays of composite
create type comptype as (r float8, i float8);
create domain dcomptypea as comptype[];
@ -762,6 +850,14 @@ insert into ddtest2 values('{(-1)}');
alter domain posint add constraint c1 check(value >= 0);
ERROR: cannot alter type "posint" because column "ddtest2.f1" uses it
drop table ddtest2;
-- Likewise for domains within domains over composite
create domain ddtest1d as ddtest1;
create table ddtest2(f1 ddtest1d);
insert into ddtest2 values('(-1)');
alter domain posint add constraint c1 check(value >= 0);
ERROR: cannot alter type "posint" because column "ddtest2.f1" uses it
drop table ddtest2;
drop domain ddtest1d;
-- Likewise for domains within domains over array of composite
create domain ddtest1d as ddtest1[];
create table ddtest2(f1 ddtest1d);

View File

@ -1316,6 +1316,8 @@ create type jpop as (a text, b int, c timestamp);
CREATE DOMAIN js_int_not_null AS int NOT NULL;
CREATE DOMAIN js_int_array_1d AS int[] CHECK(array_length(VALUE, 1) = 3);
CREATE DOMAIN js_int_array_2d AS int[][] CHECK(array_length(VALUE, 2) = 3);
create type j_unordered_pair as (x int, y int);
create domain j_ordered_pair as j_unordered_pair check((value).x <= (value).y);
CREATE TYPE jsrec AS (
i int,
ia _int4,
@ -1740,6 +1742,30 @@ SELECT rec FROM json_populate_record(
(abc,3,"Thu Jan 02 00:00:00 2003")
(1 row)
-- anonymous record type
SELECT json_populate_record(null::record, '{"x": 0, "y": 1}');
ERROR: record type has not been registered
SELECT json_populate_record(row(1,2), '{"f1": 0, "f2": 1}');
json_populate_record
----------------------
(0,1)
(1 row)
-- composite domain
SELECT json_populate_record(null::j_ordered_pair, '{"x": 0, "y": 1}');
json_populate_record
----------------------
(0,1)
(1 row)
SELECT json_populate_record(row(1,2)::j_ordered_pair, '{"x": 0}');
json_populate_record
----------------------
(0,2)
(1 row)
SELECT json_populate_record(row(1,2)::j_ordered_pair, '{"x": 1, "y": 0}');
ERROR: value for domain j_ordered_pair violates check constraint "j_ordered_pair_check"
-- populate_recordset
select * from json_populate_recordset(null::jpop,'[{"a":"blurfl","x":43.2},{"b":3,"c":"2012-01-20 10:42:53"}]') q;
a | b | c
@ -1806,6 +1832,31 @@ select * from json_populate_recordset(row('def',99,null)::jpop,'[{"a":[100,200,3
{"z":true} | 3 | Fri Jan 20 10:42:53 2012
(2 rows)
-- anonymous record type
SELECT json_populate_recordset(null::record, '[{"x": 0, "y": 1}]');
ERROR: record type has not been registered
SELECT json_populate_recordset(row(1,2), '[{"f1": 0, "f2": 1}]');
json_populate_recordset
-------------------------
(0,1)
(1 row)
-- composite domain
SELECT json_populate_recordset(null::j_ordered_pair, '[{"x": 0, "y": 1}]');
json_populate_recordset
-------------------------
(0,1)
(1 row)
SELECT json_populate_recordset(row(1,2)::j_ordered_pair, '[{"x": 0}, {"y": 3}]');
json_populate_recordset
-------------------------
(0,2)
(1,3)
(2 rows)
SELECT json_populate_recordset(row(1,2)::j_ordered_pair, '[{"x": 1, "y": 0}]');
ERROR: value for domain j_ordered_pair violates check constraint "j_ordered_pair_check"
-- test type info caching in json_populate_record()
CREATE TEMP TABLE jspoptest (js json);
INSERT INTO jspoptest
@ -1828,6 +1879,8 @@ DROP TYPE jsrec_i_not_null;
DROP DOMAIN js_int_not_null;
DROP DOMAIN js_int_array_1d;
DROP DOMAIN js_int_array_2d;
DROP DOMAIN j_ordered_pair;
DROP TYPE j_unordered_pair;
--json_typeof() function
select value, json_typeof(value)
from (values (json '123.4'),

View File

@ -2005,6 +2005,8 @@ CREATE TYPE jbpop AS (a text, b int, c timestamp);
CREATE DOMAIN jsb_int_not_null AS int NOT NULL;
CREATE DOMAIN jsb_int_array_1d AS int[] CHECK(array_length(VALUE, 1) = 3);
CREATE DOMAIN jsb_int_array_2d AS int[][] CHECK(array_length(VALUE, 2) = 3);
create type jb_unordered_pair as (x int, y int);
create domain jb_ordered_pair as jb_unordered_pair check((value).x <= (value).y);
CREATE TYPE jsbrec AS (
i int,
ia _int4,
@ -2429,6 +2431,30 @@ SELECT rec FROM jsonb_populate_record(
(abc,3,"Thu Jan 02 00:00:00 2003")
(1 row)
-- anonymous record type
SELECT jsonb_populate_record(null::record, '{"x": 0, "y": 1}');
ERROR: record type has not been registered
SELECT jsonb_populate_record(row(1,2), '{"f1": 0, "f2": 1}');
jsonb_populate_record
-----------------------
(0,1)
(1 row)
-- composite domain
SELECT jsonb_populate_record(null::jb_ordered_pair, '{"x": 0, "y": 1}');
jsonb_populate_record
-----------------------
(0,1)
(1 row)
SELECT jsonb_populate_record(row(1,2)::jb_ordered_pair, '{"x": 0}');
jsonb_populate_record
-----------------------
(0,2)
(1 row)
SELECT jsonb_populate_record(row(1,2)::jb_ordered_pair, '{"x": 1, "y": 0}');
ERROR: value for domain jb_ordered_pair violates check constraint "jb_ordered_pair_check"
-- populate_recordset
SELECT * FROM jsonb_populate_recordset(NULL::jbpop,'[{"a":"blurfl","x":43.2},{"b":3,"c":"2012-01-20 10:42:53"}]') q;
a | b | c
@ -2488,6 +2514,31 @@ SELECT * FROM jsonb_populate_recordset(row('def',99,NULL)::jbpop,'[{"a":[100,200
{"z": true} | 3 | Fri Jan 20 10:42:53 2012
(2 rows)
-- anonymous record type
SELECT jsonb_populate_recordset(null::record, '[{"x": 0, "y": 1}]');
ERROR: record type has not been registered
SELECT jsonb_populate_recordset(row(1,2), '[{"f1": 0, "f2": 1}]');
jsonb_populate_recordset
--------------------------
(0,1)
(1 row)
-- composite domain
SELECT jsonb_populate_recordset(null::jb_ordered_pair, '[{"x": 0, "y": 1}]');
jsonb_populate_recordset
--------------------------
(0,1)
(1 row)
SELECT jsonb_populate_recordset(row(1,2)::jb_ordered_pair, '[{"x": 0}, {"y": 3}]');
jsonb_populate_recordset
--------------------------
(0,2)
(1,3)
(2 rows)
SELECT jsonb_populate_recordset(row(1,2)::jb_ordered_pair, '[{"x": 1, "y": 0}]');
ERROR: value for domain jb_ordered_pair violates check constraint "jb_ordered_pair_check"
-- jsonb_to_record and jsonb_to_recordset
select * from jsonb_to_record('{"a":1,"b":"foo","c":"bar"}')
as x(a int, b text, d text);
@ -2587,6 +2638,8 @@ DROP TYPE jsbrec_i_not_null;
DROP DOMAIN jsb_int_not_null;
DROP DOMAIN jsb_int_array_1d;
DROP DOMAIN jsb_int_array_2d;
DROP DOMAIN jb_ordered_pair;
DROP TYPE jb_unordered_pair;
-- indexing
SELECT count(*) FROM testjsonb WHERE j @> '{"wait":null}';
count

View File

@ -120,6 +120,45 @@ select pg_typeof('{1,2,3}'::dia || 42); -- should be int[] not dia
drop domain dia;
-- Test domains over composites
create type comptype as (r float8, i float8);
create domain dcomptype as comptype;
create table dcomptable (d1 dcomptype unique);
insert into dcomptable values (row(1,2)::dcomptype);
insert into dcomptable values (row(3,4)::comptype);
insert into dcomptable values (row(1,2)::dcomptype); -- fail on uniqueness
insert into dcomptable (d1.r) values(11);
select * from dcomptable;
select (d1).r, (d1).i, (d1).* from dcomptable;
update dcomptable set d1.r = (d1).r + 1 where (d1).i > 0;
select * from dcomptable;
alter domain dcomptype add constraint c1 check ((value).r <= (value).i);
alter domain dcomptype add constraint c2 check ((value).r > (value).i); -- fail
select row(2,1)::dcomptype; -- fail
insert into dcomptable values (row(1,2)::comptype);
insert into dcomptable values (row(2,1)::comptype); -- fail
insert into dcomptable (d1.r) values(99);
insert into dcomptable (d1.r, d1.i) values(99, 100);
insert into dcomptable (d1.r, d1.i) values(100, 99); -- fail
update dcomptable set d1.r = (d1).r + 1 where (d1).i > 0; -- fail
update dcomptable set d1.r = (d1).r - 1, d1.i = (d1).i + 1 where (d1).i > 0;
select * from dcomptable;
explain (verbose, costs off)
update dcomptable set d1.r = (d1).r - 1, d1.i = (d1).i + 1 where (d1).i > 0;
create rule silly as on delete to dcomptable do instead
update dcomptable set d1.r = (d1).r - 1, d1.i = (d1).i + 1 where (d1).i > 0;
\d+ dcomptable
drop table dcomptable;
drop type comptype cascade;
-- Test domains over arrays of composite
create type comptype as (r float8, i float8);
@ -500,6 +539,14 @@ insert into ddtest2 values('{(-1)}');
alter domain posint add constraint c1 check(value >= 0);
drop table ddtest2;
-- Likewise for domains within domains over composite
create domain ddtest1d as ddtest1;
create table ddtest2(f1 ddtest1d);
insert into ddtest2 values('(-1)');
alter domain posint add constraint c1 check(value >= 0);
drop table ddtest2;
drop domain ddtest1d;
-- Likewise for domains within domains over array of composite
create domain ddtest1d as ddtest1[];
create table ddtest2(f1 ddtest1d);

View File

@ -388,6 +388,9 @@ CREATE DOMAIN js_int_not_null AS int NOT NULL;
CREATE DOMAIN js_int_array_1d AS int[] CHECK(array_length(VALUE, 1) = 3);
CREATE DOMAIN js_int_array_2d AS int[][] CHECK(array_length(VALUE, 2) = 3);
create type j_unordered_pair as (x int, y int);
create domain j_ordered_pair as j_unordered_pair check((value).x <= (value).y);
CREATE TYPE jsrec AS (
i int,
ia _int4,
@ -516,6 +519,15 @@ SELECT rec FROM json_populate_record(
'{"rec": {"a": "abc", "c": "01.02.2003", "x": 43.2}}'
) q;
-- anonymous record type
SELECT json_populate_record(null::record, '{"x": 0, "y": 1}');
SELECT json_populate_record(row(1,2), '{"f1": 0, "f2": 1}');
-- composite domain
SELECT json_populate_record(null::j_ordered_pair, '{"x": 0, "y": 1}');
SELECT json_populate_record(row(1,2)::j_ordered_pair, '{"x": 0}');
SELECT json_populate_record(row(1,2)::j_ordered_pair, '{"x": 1, "y": 0}');
-- populate_recordset
select * from json_populate_recordset(null::jpop,'[{"a":"blurfl","x":43.2},{"b":3,"c":"2012-01-20 10:42:53"}]') q;
@ -532,6 +544,15 @@ select * from json_populate_recordset(null::jpop,'[{"a":"blurfl","x":43.2},{"b":
select * from json_populate_recordset(row('def',99,null)::jpop,'[{"a":"blurfl","x":43.2},{"b":3,"c":"2012-01-20 10:42:53"}]') q;
select * from json_populate_recordset(row('def',99,null)::jpop,'[{"a":[100,200,300],"x":43.2},{"a":{"z":true},"b":3,"c":"2012-01-20 10:42:53"}]') q;
-- anonymous record type
SELECT json_populate_recordset(null::record, '[{"x": 0, "y": 1}]');
SELECT json_populate_recordset(row(1,2), '[{"f1": 0, "f2": 1}]');
-- composite domain
SELECT json_populate_recordset(null::j_ordered_pair, '[{"x": 0, "y": 1}]');
SELECT json_populate_recordset(row(1,2)::j_ordered_pair, '[{"x": 0}, {"y": 3}]');
SELECT json_populate_recordset(row(1,2)::j_ordered_pair, '[{"x": 1, "y": 0}]');
-- test type info caching in json_populate_record()
CREATE TEMP TABLE jspoptest (js json);
@ -550,6 +571,8 @@ DROP TYPE jsrec_i_not_null;
DROP DOMAIN js_int_not_null;
DROP DOMAIN js_int_array_1d;
DROP DOMAIN js_int_array_2d;
DROP DOMAIN j_ordered_pair;
DROP TYPE j_unordered_pair;
--json_typeof() function
select value, json_typeof(value)

View File

@ -508,6 +508,9 @@ CREATE DOMAIN jsb_int_not_null AS int NOT NULL;
CREATE DOMAIN jsb_int_array_1d AS int[] CHECK(array_length(VALUE, 1) = 3);
CREATE DOMAIN jsb_int_array_2d AS int[][] CHECK(array_length(VALUE, 2) = 3);
create type jb_unordered_pair as (x int, y int);
create domain jb_ordered_pair as jb_unordered_pair check((value).x <= (value).y);
CREATE TYPE jsbrec AS (
i int,
ia _int4,
@ -636,6 +639,15 @@ SELECT rec FROM jsonb_populate_record(
'{"rec": {"a": "abc", "c": "01.02.2003", "x": 43.2}}'
) q;
-- anonymous record type
SELECT jsonb_populate_record(null::record, '{"x": 0, "y": 1}');
SELECT jsonb_populate_record(row(1,2), '{"f1": 0, "f2": 1}');
-- composite domain
SELECT jsonb_populate_record(null::jb_ordered_pair, '{"x": 0, "y": 1}');
SELECT jsonb_populate_record(row(1,2)::jb_ordered_pair, '{"x": 0}');
SELECT jsonb_populate_record(row(1,2)::jb_ordered_pair, '{"x": 1, "y": 0}');
-- populate_recordset
SELECT * FROM jsonb_populate_recordset(NULL::jbpop,'[{"a":"blurfl","x":43.2},{"b":3,"c":"2012-01-20 10:42:53"}]') q;
SELECT * FROM jsonb_populate_recordset(row('def',99,NULL)::jbpop,'[{"a":"blurfl","x":43.2},{"b":3,"c":"2012-01-20 10:42:53"}]') q;
@ -648,6 +660,15 @@ SELECT * FROM jsonb_populate_recordset(NULL::jbpop,'[{"a":"blurfl","x":43.2},{"b
SELECT * FROM jsonb_populate_recordset(row('def',99,NULL)::jbpop,'[{"a":"blurfl","x":43.2},{"b":3,"c":"2012-01-20 10:42:53"}]') q;
SELECT * FROM jsonb_populate_recordset(row('def',99,NULL)::jbpop,'[{"a":[100,200,300],"x":43.2},{"a":{"z":true},"b":3,"c":"2012-01-20 10:42:53"}]') q;
-- anonymous record type
SELECT jsonb_populate_recordset(null::record, '[{"x": 0, "y": 1}]');
SELECT jsonb_populate_recordset(row(1,2), '[{"f1": 0, "f2": 1}]');
-- composite domain
SELECT jsonb_populate_recordset(null::jb_ordered_pair, '[{"x": 0, "y": 1}]');
SELECT jsonb_populate_recordset(row(1,2)::jb_ordered_pair, '[{"x": 0}, {"y": 3}]');
SELECT jsonb_populate_recordset(row(1,2)::jb_ordered_pair, '[{"x": 1, "y": 0}]');
-- jsonb_to_record and jsonb_to_recordset
select * from jsonb_to_record('{"a":1,"b":"foo","c":"bar"}')
@ -693,6 +714,8 @@ DROP TYPE jsbrec_i_not_null;
DROP DOMAIN jsb_int_not_null;
DROP DOMAIN jsb_int_array_1d;
DROP DOMAIN jsb_int_array_2d;
DROP DOMAIN jb_ordered_pair;
DROP TYPE jb_unordered_pair;
-- indexing
SELECT count(*) FROM testjsonb WHERE j @> '{"wait":null}';