1
0
mirror of https://github.com/postgres/postgres.git synced 2025-07-30 11:03:19 +03:00

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

@ -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}';