Drop no-op CoerceToDomain nodes from expressions at planning time.

If a domain has no constraints, then CoerceToDomain doesn't really do
anything and can be simplified to a RelabelType.  This not only
eliminates cycles at execution, but allows the planner to optimize better
(for instance, match the coerced expression to an index on the underlying
column).  However, we do have to support invalidating the plan later if
a constraint gets added to the domain.  That's comparable to the case of
a change to a SQL function that had been inlined into a plan, so all the
necessary logic already exists for plans depending on functions.  We
need only duplicate or share that logic for domains.

ALTER DOMAIN ADD/DROP CONSTRAINT need to be taught to send out sinval
messages for the domain's pg_type entry, since those operations don't
update that row.  (ALTER DOMAIN SET/DROP NOT NULL do update that row,
so no code change is needed for them.)

Testing this revealed what's really a pre-existing bug in plpgsql:
it caches the SQL-expression-tree expansion of type coercions and
had no provision for invalidating entries in that cache.  Up to now
that was only a problem if such an expression had inlined a SQL
function that got changed, which is unlikely though not impossible.
But failing to track changes of domain constraints breaks an existing
regression test case and would likely cause practical problems too.

We could fix that locally in plpgsql, but what seems like a better
idea is to build some generic infrastructure in plancache.c to store
standalone expressions and track invalidation events for them.
(It's tempting to wonder whether plpgsql's "simple expression" stuff
could use this code with lower overhead than its current use of the
heavyweight plancache APIs.  But I've left that idea for later.)

Other stuff fixed in passing:

* Allow estimate_expression_value() to drop CoerceToDomain
unconditionally, effectively assuming that the coercion will succeed.
This will improve planner selectivity estimates for cases involving
estimatable expressions that are coerced to domains.  We could have
done this independently of everything else here, but there wasn't
previously any need for eval_const_expressions_mutator to know about
CoerceToDomain at all.

* Use a dlist for plancache.c's list of cached plans, rather than a
manually threaded singly-linked list.  That eliminates a potential
performance problem in DropCachedPlan.

* Fix a couple of inconsistencies in typecmds.c about whether
operations on domains drop RowExclusiveLock on pg_type.  Our common
practice is that DDL operations do drop catalog locks, so standardize
on that choice.

Discussion: https://postgr.es/m/19958.1544122124@sss.pgh.pa.us
This commit is contained in:
Tom Lane 2018-12-13 13:24:43 -05:00
parent 52ac6cd2d0
commit 04fe805a17
12 changed files with 510 additions and 92 deletions

View File

@ -62,6 +62,7 @@
#include "parser/parse_type.h"
#include "utils/builtins.h"
#include "utils/fmgroids.h"
#include "utils/inval.h"
#include "utils/lsyscache.h"
#include "utils/memutils.h"
#include "utils/rel.h"
@ -2297,7 +2298,7 @@ AlterDomainDefault(List *names, Node *defaultRaw)
ObjectAddressSet(address, TypeRelationId, domainoid);
/* Clean up */
heap_close(rel, NoLock);
heap_close(rel, RowExclusiveLock);
heap_freetuple(newtuple);
return address;
@ -2494,8 +2495,6 @@ AlterDomainDropConstraint(List *names, const char *constrName,
systable_endscan(conscan);
heap_close(conrel, RowExclusiveLock);
heap_close(rel, NoLock);
if (!found)
{
if (!missing_ok)
@ -2509,8 +2508,18 @@ AlterDomainDropConstraint(List *names, const char *constrName,
constrName, TypeNameToString(typename))));
}
/*
* We must send out an sinval message for the domain, to ensure that any
* dependent plans get rebuilt. Since this command doesn't change the
* domain's pg_type row, that won't happen automatically; do it manually.
*/
CacheInvalidateHeapTuple(rel, tup, NULL);
ObjectAddressSet(address, TypeRelationId, domainoid);
/* Clean up */
heap_close(rel, RowExclusiveLock);
return address;
}
@ -2615,6 +2624,13 @@ AlterDomainAddConstraint(List *names, Node *newConstraint,
if (!constr->skip_validation)
validateDomainConstraint(domainoid, ccbin);
/*
* We must send out an sinval message for the domain, to ensure that any
* dependent plans get rebuilt. Since this command doesn't change the
* domain's pg_type row, that won't happen automatically; do it manually.
*/
CacheInvalidateHeapTuple(typrel, tup, NULL);
ObjectAddressSet(address, TypeRelationId, domainoid);
/* Clean up */

View File

@ -5923,10 +5923,16 @@ adjust_paths_for_srfs(PlannerInfo *root, RelOptInfo *rel,
* side-effect that is useful when the expression will get evaluated more than
* once. Also, we must fix operator function IDs.
*
* This does not return any information about dependencies of the expression.
* Hence callers should use the results only for the duration of the current
* query. Callers that would like to cache the results for longer should use
* expression_planner_with_deps, probably via the plancache.
*
* Note: this must not make any damaging changes to the passed-in expression
* tree. (It would actually be okay to apply fix_opfuncids to it, but since
* we first do an expression_tree_mutator-based walk, what is returned will
* be a new node tree.)
* be a new node tree.) The result is constructed in the current memory
* context; beware that this can leak a lot of additional stuff there, too.
*/
Expr *
expression_planner(Expr *expr)
@ -5945,6 +5951,57 @@ expression_planner(Expr *expr)
return (Expr *) result;
}
/*
* expression_planner_with_deps
* Perform planner's transformations on a standalone expression,
* returning expression dependency information along with the result.
*
* This is identical to expression_planner() except that it also returns
* information about possible dependencies of the expression, ie identities of
* objects whose definitions affect the result. As in a PlannedStmt, these
* are expressed as a list of relation Oids and a list of PlanInvalItems.
*/
Expr *
expression_planner_with_deps(Expr *expr,
List **relationOids,
List **invalItems)
{
Node *result;
PlannerGlobal glob;
PlannerInfo root;
/* Make up dummy planner state so we can use setrefs machinery */
MemSet(&glob, 0, sizeof(glob));
glob.type = T_PlannerGlobal;
glob.relationOids = NIL;
glob.invalItems = NIL;
MemSet(&root, 0, sizeof(root));
root.type = T_PlannerInfo;
root.glob = &glob;
/*
* Convert named-argument function calls, insert default arguments and
* simplify constant subexprs. Collect identities of inlined functions
* and elided domains, too.
*/
result = eval_const_expressions(&root, (Node *) expr);
/* Fill in opfuncid values if missing */
fix_opfuncids(result);
/*
* Now walk the finished expression to find anything else we ought to
* record as an expression dependency.
*/
(void) extract_query_dependencies_walker(result, &root);
*relationOids = glob.relationOids;
*invalItems = glob.invalItems;
return (Expr *) result;
}
/*
* plan_cluster_use_sort

View File

@ -138,8 +138,7 @@ static List *set_returning_clause_references(PlannerInfo *root,
Plan *topplan,
Index resultRelation,
int rtoffset);
static bool extract_query_dependencies_walker(Node *node,
PlannerInfo *context);
/*****************************************************************************
*
@ -175,8 +174,8 @@ static bool extract_query_dependencies_walker(Node *node,
* This will be used by plancache.c to drive invalidation of cached plans.
* Relation dependencies are represented by OIDs, and everything else by
* PlanInvalItems (this distinction is motivated by the shared-inval APIs).
* Currently, relations and user-defined functions are the only types of
* objects that are explicitly tracked this way.
* Currently, relations, user-defined functions, and domains are the only
* types of objects that are explicitly tracked this way.
*
* 8. We assign every plan node in the tree a unique ID.
*
@ -2577,6 +2576,42 @@ record_plan_function_dependency(PlannerInfo *root, Oid funcid)
}
}
/*
* record_plan_type_dependency
* Mark the current plan as depending on a particular type.
*
* This is exported so that eval_const_expressions can record a
* dependency on a domain that it's removed a CoerceToDomain node for.
*
* We don't currently need to record dependencies on domains that the
* plan contains CoerceToDomain nodes for, though that might change in
* future. Hence, this isn't actually called in this module, though
* someday fix_expr_common might call it.
*/
void
record_plan_type_dependency(PlannerInfo *root, Oid typeid)
{
/*
* As in record_plan_function_dependency, ignore the possibility that
* someone would change a built-in domain.
*/
if (typeid >= (Oid) FirstBootstrapObjectId)
{
PlanInvalItem *inval_item = makeNode(PlanInvalItem);
/*
* It would work to use any syscache on pg_type, but the easiest is
* TYPEOID since we already have the type's OID at hand. Note that
* plancache.c knows we use TYPEOID.
*/
inval_item->cacheId = TYPEOID;
inval_item->hashValue = GetSysCacheHashValue1(TYPEOID,
ObjectIdGetDatum(typeid));
root->glob->invalItems = lappend(root->glob->invalItems, inval_item);
}
}
/*
* extract_query_dependencies
* Given a rewritten, but not yet planned, query or queries
@ -2586,6 +2621,13 @@ record_plan_function_dependency(PlannerInfo *root, Oid funcid)
*
* This is needed by plancache.c to handle invalidation of cached unplanned
* queries.
*
* Note: this does not go through eval_const_expressions, and hence doesn't
* reflect its additions of inlined functions and elided CoerceToDomain nodes
* to the invalItems list. This is obviously OK for functions, since we'll
* see them in the original query tree anyway. For domains, it's OK because
* we don't care about domains unless they get elided. That is, a plan might
* have domain dependencies that the query tree doesn't.
*/
void
extract_query_dependencies(Node *query,
@ -2615,14 +2657,20 @@ extract_query_dependencies(Node *query,
*hasRowSecurity = glob.dependsOnRole;
}
static bool
/*
* Tree walker for extract_query_dependencies.
*
* This is exported so that expression_planner_with_deps can call it on
* simple expressions (post-planning, not before planning, in that case).
* In that usage, glob.dependsOnRole isn't meaningful, but the relationOids
* and invalItems lists are added to as needed.
*/
bool
extract_query_dependencies_walker(Node *node, PlannerInfo *context)
{
if (node == NULL)
return false;
Assert(!IsA(node, PlaceHolderVar));
/* Extract function dependencies and check for regclass Consts */
fix_expr_common(context, node);
if (IsA(node, Query))
{
Query *query = (Query *) node;
@ -2662,6 +2710,8 @@ extract_query_dependencies_walker(Node *node, PlannerInfo *context)
return query_tree_walker(query, extract_query_dependencies_walker,
(void *) context, 0);
}
/* Extract function dependencies and check for regclass Consts */
fix_expr_common(context, node);
return expression_tree_walker(node, extract_query_dependencies_walker,
(void *) context);
}

View File

@ -3699,6 +3699,70 @@ eval_const_expressions_mutator(Node *node,
newbtest->location = btest->location;
return (Node *) newbtest;
}
case T_CoerceToDomain:
{
/*
* If the domain currently has no constraints, we replace the
* CoerceToDomain node with a simple RelabelType, which is
* both far faster to execute and more amenable to later
* optimization. We must then mark the plan as needing to be
* rebuilt if the domain's constraints change.
*
* Also, in estimation mode, always replace CoerceToDomain
* nodes, effectively assuming that the coercion will succeed.
*/
CoerceToDomain *cdomain = (CoerceToDomain *) node;
CoerceToDomain *newcdomain;
Node *arg;
arg = eval_const_expressions_mutator((Node *) cdomain->arg,
context);
if (context->estimate ||
!DomainHasConstraints(cdomain->resulttype))
{
/* Record dependency, if this isn't estimation mode */
if (context->root && !context->estimate)
record_plan_type_dependency(context->root,
cdomain->resulttype);
/* Generate RelabelType to substitute for CoerceToDomain */
/* This should match the RelabelType logic above */
while (arg && IsA(arg, RelabelType))
arg = (Node *) ((RelabelType *) arg)->arg;
if (arg && IsA(arg, Const))
{
Const *con = (Const *) arg;
con->consttype = cdomain->resulttype;
con->consttypmod = cdomain->resulttypmod;
con->constcollid = cdomain->resultcollid;
return (Node *) con;
}
else
{
RelabelType *newrelabel = makeNode(RelabelType);
newrelabel->arg = (Expr *) arg;
newrelabel->resulttype = cdomain->resulttype;
newrelabel->resulttypmod = cdomain->resulttypmod;
newrelabel->resultcollid = cdomain->resultcollid;
newrelabel->relabelformat = cdomain->coercionformat;
newrelabel->location = cdomain->location;
return (Node *) newrelabel;
}
}
newcdomain = makeNode(CoerceToDomain);
newcdomain->arg = (Expr *) arg;
newcdomain->resulttype = cdomain->resulttype;
newcdomain->resulttypmod = cdomain->resulttypmod;
newcdomain->resultcollid = cdomain->resultcollid;
newcdomain->coercionformat = cdomain->coercionformat;
newcdomain->location = cdomain->location;
return (Node *) newcdomain;
}
case T_PlaceHolderVar:
/*
@ -3770,7 +3834,7 @@ eval_const_expressions_mutator(Node *node,
* For any node type not handled above, copy the node unchanged but
* const-simplify its subexpressions. This is the correct thing for node
* types whose behavior might change between planning and execution, such
* as CoerceToDomain. It's also a safe default for new node types not
* as CurrentOfExpr. It's also a safe default for new node types not
* known to this routine.
*/
return ece_generic_processing(node);

View File

@ -27,15 +27,21 @@
* query to change output tupdesc on replan --- if so, it's up to the
* caller to notice changes and cope with them.
*
* Currently, we track exactly the dependencies of plans on relations and
* user-defined functions. On relcache invalidation events or pg_proc
* syscache invalidation events, we invalidate just those plans that depend
* on the particular object being modified. (Note: this scheme assumes
* that any table modification that requires replanning will generate a
* relcache inval event.) We also watch for inval events on certain other
* system catalogs, such as pg_namespace; but for them, our response is
* just to invalidate all plans. We expect updates on those catalogs to
* be infrequent enough that more-detailed tracking is not worth the effort.
* Currently, we track exactly the dependencies of plans on relations,
* user-defined functions, and domains. On relcache invalidation events or
* pg_proc or pg_type syscache invalidation events, we invalidate just those
* plans that depend on the particular object being modified. (Note: this
* scheme assumes that any table modification that requires replanning will
* generate a relcache inval event.) We also watch for inval events on
* certain other system catalogs, such as pg_namespace; but for them, our
* response is just to invalidate all plans. We expect updates on those
* catalogs to be infrequent enough that more-detailed tracking is not worth
* the effort.
*
* In addition to full-fledged query plans, we provide a facility for
* detecting invalidations of simple scalar expressions. This is fairly
* bare-bones; it's the caller's responsibility to build a new expression
* if the old one gets invalidated.
*
*
* Portions Copyright (c) 1996-2018, PostgreSQL Global Development Group
@ -57,6 +63,7 @@
#include "nodes/nodeFuncs.h"
#include "optimizer/cost.h"
#include "optimizer/planmain.h"
#include "optimizer/planner.h"
#include "optimizer/prep.h"
#include "parser/analyze.h"
#include "parser/parsetree.h"
@ -82,10 +89,15 @@
/*
* This is the head of the backend's list of "saved" CachedPlanSources (i.e.,
* those that are in long-lived storage and are examined for sinval events).
* We thread the structs manually instead of using List cells so that we can
* guarantee to save a CachedPlanSource without error.
* We use a dlist instead of separate List cells so that we can guarantee
* to save a CachedPlanSource without error.
*/
static CachedPlanSource *first_saved_plan = NULL;
static dlist_head saved_plan_list = DLIST_STATIC_INIT(saved_plan_list);
/*
* This is the head of the backend's list of CachedExpressions.
*/
static dlist_head cached_expression_list = DLIST_STATIC_INIT(cached_expression_list);
static void ReleaseGenericPlan(CachedPlanSource *plansource);
static List *RevalidateCachedQuery(CachedPlanSource *plansource,
@ -103,7 +115,7 @@ static void ScanQueryForLocks(Query *parsetree, bool acquire);
static bool ScanQueryWalker(Node *node, bool *acquire);
static TupleDesc PlanCacheComputeResultDesc(List *stmt_list);
static void PlanCacheRelCallback(Datum arg, Oid relid);
static void PlanCacheFuncCallback(Datum arg, int cacheid, uint32 hashvalue);
static void PlanCacheObjectCallback(Datum arg, int cacheid, uint32 hashvalue);
static void PlanCacheSysCallback(Datum arg, int cacheid, uint32 hashvalue);
/* GUC parameter */
@ -118,7 +130,8 @@ void
InitPlanCache(void)
{
CacheRegisterRelcacheCallback(PlanCacheRelCallback, (Datum) 0);
CacheRegisterSyscacheCallback(PROCOID, PlanCacheFuncCallback, (Datum) 0);
CacheRegisterSyscacheCallback(PROCOID, PlanCacheObjectCallback, (Datum) 0);
CacheRegisterSyscacheCallback(TYPEOID, PlanCacheObjectCallback, (Datum) 0);
CacheRegisterSyscacheCallback(NAMESPACEOID, PlanCacheSysCallback, (Datum) 0);
CacheRegisterSyscacheCallback(OPEROID, PlanCacheSysCallback, (Datum) 0);
CacheRegisterSyscacheCallback(AMOPOPID, PlanCacheSysCallback, (Datum) 0);
@ -206,7 +219,6 @@ CreateCachedPlan(RawStmt *raw_parse_tree,
plansource->is_saved = false;
plansource->is_valid = false;
plansource->generation = 0;
plansource->next_saved = NULL;
plansource->generic_cost = -1;
plansource->total_custom_cost = 0;
plansource->num_custom_plans = 0;
@ -274,7 +286,6 @@ CreateOneShotCachedPlan(RawStmt *raw_parse_tree,
plansource->is_saved = false;
plansource->is_valid = false;
plansource->generation = 0;
plansource->next_saved = NULL;
plansource->generic_cost = -1;
plansource->total_custom_cost = 0;
plansource->num_custom_plans = 0;
@ -471,8 +482,7 @@ SaveCachedPlan(CachedPlanSource *plansource)
/*
* Add the entry to the global list of cached plans.
*/
plansource->next_saved = first_saved_plan;
first_saved_plan = plansource;
dlist_push_tail(&saved_plan_list, &plansource->node);
plansource->is_saved = true;
}
@ -493,21 +503,7 @@ DropCachedPlan(CachedPlanSource *plansource)
/* If it's been saved, remove it from the list */
if (plansource->is_saved)
{
if (first_saved_plan == plansource)
first_saved_plan = plansource->next_saved;
else
{
CachedPlanSource *psrc;
for (psrc = first_saved_plan; psrc; psrc = psrc->next_saved)
{
if (psrc->next_saved == plansource)
{
psrc->next_saved = plansource->next_saved;
break;
}
}
}
dlist_delete(&plansource->node);
plansource->is_saved = false;
}
@ -1399,7 +1395,6 @@ CopyCachedPlan(CachedPlanSource *plansource)
newsource->is_saved = false;
newsource->is_valid = plansource->is_valid;
newsource->generation = plansource->generation;
newsource->next_saved = NULL;
/* We may as well copy any acquired cost knowledge */
newsource->generic_cost = plansource->generic_cost;
@ -1458,6 +1453,85 @@ CachedPlanGetTargetList(CachedPlanSource *plansource,
return FetchStatementTargetList((Node *) pstmt);
}
/*
* GetCachedExpression: construct a CachedExpression for an expression.
*
* This performs the same transformations on the expression as
* expression_planner(), ie, convert an expression as emitted by parse
* analysis to be ready to pass to the executor.
*
* The result is stashed in a private, long-lived memory context.
* (Note that this might leak a good deal of memory in the caller's
* context before that.) The passed-in expr tree is not modified.
*/
CachedExpression *
GetCachedExpression(Node *expr)
{
CachedExpression *cexpr;
List *relationOids;
List *invalItems;
MemoryContext cexpr_context;
MemoryContext oldcxt;
/*
* Pass the expression through the planner, and collect dependencies.
* Everything built here is leaked in the caller's context; that's
* intentional to minimize the size of the permanent data structure.
*/
expr = (Node *) expression_planner_with_deps((Expr *) expr,
&relationOids,
&invalItems);
/*
* Make a private memory context, and copy what we need into that. To
* avoid leaking a long-lived context if we fail while copying data, we
* initially make the context under the caller's context.
*/
cexpr_context = AllocSetContextCreate(CurrentMemoryContext,
"CachedExpression",
ALLOCSET_SMALL_SIZES);
oldcxt = MemoryContextSwitchTo(cexpr_context);
cexpr = (CachedExpression *) palloc(sizeof(CachedExpression));
cexpr->magic = CACHEDEXPR_MAGIC;
cexpr->expr = copyObject(expr);
cexpr->is_valid = true;
cexpr->relationOids = copyObject(relationOids);
cexpr->invalItems = copyObject(invalItems);
cexpr->context = cexpr_context;
MemoryContextSwitchTo(oldcxt);
/*
* Reparent the expr's memory context under CacheMemoryContext so that it
* will live indefinitely.
*/
MemoryContextSetParent(cexpr_context, CacheMemoryContext);
/*
* Add the entry to the global list of cached expressions.
*/
dlist_push_tail(&cached_expression_list, &cexpr->node);
return cexpr;
}
/*
* FreeCachedExpression
* Delete a CachedExpression.
*/
void
FreeCachedExpression(CachedExpression *cexpr)
{
/* Sanity check */
Assert(cexpr->magic == CACHEDEXPR_MAGIC);
/* Unlink from global list */
dlist_delete(&cexpr->node);
/* Free all storage associated with CachedExpression */
MemoryContextDelete(cexpr->context);
}
/*
* QueryListGetPrimaryStmt
* Get the "primary" stmt within a list, ie, the one marked canSetTag.
@ -1692,10 +1766,13 @@ PlanCacheComputeResultDesc(List *stmt_list)
static void
PlanCacheRelCallback(Datum arg, Oid relid)
{
CachedPlanSource *plansource;
dlist_iter iter;
for (plansource = first_saved_plan; plansource; plansource = plansource->next_saved)
dlist_foreach(iter, &saved_plan_list)
{
CachedPlanSource *plansource = dlist_container(CachedPlanSource,
node, iter.cur);
Assert(plansource->magic == CACHEDPLANSOURCE_MAGIC);
/* No work if it's already invalidated */
@ -1742,25 +1819,43 @@ PlanCacheRelCallback(Datum arg, Oid relid)
}
}
}
/* Likewise check cached expressions */
dlist_foreach(iter, &cached_expression_list)
{
CachedExpression *cexpr = dlist_container(CachedExpression,
node, iter.cur);
Assert(cexpr->magic == CACHEDEXPR_MAGIC);
/* No work if it's already invalidated */
if (!cexpr->is_valid)
continue;
if ((relid == InvalidOid) ? cexpr->relationOids != NIL :
list_member_oid(cexpr->relationOids, relid))
{
cexpr->is_valid = false;
}
}
}
/*
* PlanCacheFuncCallback
* Syscache inval callback function for PROCOID cache
* PlanCacheObjectCallback
* Syscache inval callback function for PROCOID and TYPEOID caches
*
* Invalidate all plans mentioning the object with the specified hash value,
* or all plans mentioning any member of this cache if hashvalue == 0.
*
* Note that the coding would support use for multiple caches, but right
* now only user-defined functions are tracked this way.
*/
static void
PlanCacheFuncCallback(Datum arg, int cacheid, uint32 hashvalue)
PlanCacheObjectCallback(Datum arg, int cacheid, uint32 hashvalue)
{
CachedPlanSource *plansource;
dlist_iter iter;
for (plansource = first_saved_plan; plansource; plansource = plansource->next_saved)
dlist_foreach(iter, &saved_plan_list)
{
CachedPlanSource *plansource = dlist_container(CachedPlanSource,
node, iter.cur);
ListCell *lc;
Assert(plansource->magic == CACHEDPLANSOURCE_MAGIC);
@ -1825,6 +1920,34 @@ PlanCacheFuncCallback(Datum arg, int cacheid, uint32 hashvalue)
}
}
}
/* Likewise check cached expressions */
dlist_foreach(iter, &cached_expression_list)
{
CachedExpression *cexpr = dlist_container(CachedExpression,
node, iter.cur);
ListCell *lc;
Assert(cexpr->magic == CACHEDEXPR_MAGIC);
/* No work if it's already invalidated */
if (!cexpr->is_valid)
continue;
foreach(lc, cexpr->invalItems)
{
PlanInvalItem *item = (PlanInvalItem *) lfirst(lc);
if (item->cacheId != cacheid)
continue;
if (hashvalue == 0 ||
item->hashValue == hashvalue)
{
cexpr->is_valid = false;
break;
}
}
}
}
/*
@ -1845,10 +1968,12 @@ PlanCacheSysCallback(Datum arg, int cacheid, uint32 hashvalue)
void
ResetPlanCache(void)
{
CachedPlanSource *plansource;
dlist_iter iter;
for (plansource = first_saved_plan; plansource; plansource = plansource->next_saved)
dlist_foreach(iter, &saved_plan_list)
{
CachedPlanSource *plansource = dlist_container(CachedPlanSource,
node, iter.cur);
ListCell *lc;
Assert(plansource->magic == CACHEDPLANSOURCE_MAGIC);
@ -1888,4 +2013,15 @@ ResetPlanCache(void)
}
}
}
/* Likewise invalidate cached expressions */
dlist_foreach(iter, &cached_expression_list)
{
CachedExpression *cexpr = dlist_container(CachedExpression,
node, iter.cur);
Assert(cexpr->magic == CACHEDEXPR_MAGIC);
cexpr->is_valid = false;
}
}

View File

@ -992,7 +992,16 @@ load_domaintype_info(TypeCacheEntry *typentry)
check_expr = (Expr *) stringToNode(constring);
/* ExecInitExpr will assume we've planned the expression */
/*
* Plan the expression, since ExecInitExpr will expect that.
*
* Note: caching the result of expression_planner() is not very
* good practice. Ideally we'd use a CachedExpression here so
* that we would react promptly to, eg, changes in inlined
* functions. However, because we don't support mutable domain
* CHECK constraints, it's not really clear that it's worth the
* extra overhead to do that.
*/
check_expr = expression_planner(check_expr);
r = makeNode(DomainConstraintState);

View File

@ -118,9 +118,11 @@ extern bool innerrel_is_unique(PlannerInfo *root,
*/
extern Plan *set_plan_references(PlannerInfo *root, Plan *plan);
extern void record_plan_function_dependency(PlannerInfo *root, Oid funcid);
extern void record_plan_type_dependency(PlannerInfo *root, Oid typeid);
extern void extract_query_dependencies(Node *query,
List **relationOids,
List **invalItems,
bool *hasRowSecurity);
extern bool extract_query_dependencies_walker(Node *node, PlannerInfo *root);
#endif /* PLANMAIN_H */

View File

@ -55,6 +55,9 @@ extern Path *get_cheapest_fractional_path(RelOptInfo *rel,
double tuple_fraction);
extern Expr *expression_planner(Expr *expr);
extern Expr *expression_planner_with_deps(Expr *expr,
List **relationOids,
List **invalItems);
extern Expr *preprocess_phv_expression(PlannerInfo *root, Expr *expr);

View File

@ -16,14 +16,27 @@
#define PLANCACHE_H
#include "access/tupdesc.h"
#include "lib/ilist.h"
#include "nodes/params.h"
#include "utils/queryenvironment.h"
/* Forward declaration, to avoid including parsenodes.h here */
struct RawStmt;
/* possible values for plan_cache_mode */
typedef enum
{
PLAN_CACHE_MODE_AUTO,
PLAN_CACHE_MODE_FORCE_GENERIC_PLAN,
PLAN_CACHE_MODE_FORCE_CUSTOM_PLAN
} PlanCacheMode;
/* GUC parameter */
extern int plan_cache_mode;
#define CACHEDPLANSOURCE_MAGIC 195726186
#define CACHEDPLAN_MAGIC 953717834
#define CACHEDEXPR_MAGIC 838275847
/*
* CachedPlanSource (which might better have been called CachedQuery)
@ -110,7 +123,7 @@ typedef struct CachedPlanSource
bool is_valid; /* is the query_list currently valid? */
int generation; /* increments each time we create a plan */
/* If CachedPlanSource has been saved, it is a member of a global list */
struct CachedPlanSource *next_saved; /* list link, if so */
dlist_node node; /* list link, if is_saved */
/* State kept to help decide whether to use custom or generic plans: */
double generic_cost; /* cost of generic plan, or -1 if not known */
double total_custom_cost; /* total cost of custom plans so far */
@ -143,6 +156,30 @@ typedef struct CachedPlan
MemoryContext context; /* context containing this CachedPlan */
} CachedPlan;
/*
* CachedExpression is a low-overhead mechanism for caching the planned form
* of standalone scalar expressions. While such expressions are not usually
* subject to cache invalidation events, that can happen, for example because
* of replacement of a SQL function that was inlined into the expression.
* The plancache takes care of storing the expression tree and marking it
* invalid if a cache invalidation occurs, but the caller must notice the
* !is_valid status and discard the obsolete expression without reusing it.
* We do not store the original parse tree, only the planned expression;
* this is an optimization based on the assumption that we usually will not
* need to replan for the life of the session.
*/
typedef struct CachedExpression
{
int magic; /* should equal CACHEDEXPR_MAGIC */
Node *expr; /* planned form of expression */
bool is_valid; /* is the expression still valid? */
/* remaining fields should be treated as private to plancache.c: */
List *relationOids; /* OIDs of relations the expr depends on */
List *invalItems; /* other dependencies, as PlanInvalItems */
MemoryContext context; /* context containing this CachedExpression */
dlist_node node; /* link in global list of CachedExpressions */
} CachedExpression;
extern void InitPlanCache(void);
extern void ResetPlanCache(void);
@ -182,15 +219,7 @@ extern CachedPlan *GetCachedPlan(CachedPlanSource *plansource,
QueryEnvironment *queryEnv);
extern void ReleaseCachedPlan(CachedPlan *plan, bool useResOwner);
/* possible values for plan_cache_mode */
typedef enum
{
PLAN_CACHE_MODE_AUTO,
PLAN_CACHE_MODE_FORCE_GENERIC_PLAN,
PLAN_CACHE_MODE_FORCE_CUSTOM_PLAN
} PlanCacheMode;
/* GUC parameter */
extern int plan_cache_mode;
extern CachedExpression *GetCachedExpression(Node *expr);
extern void FreeCachedExpression(CachedExpression *cexpr);
#endif /* PLANCACHE_H */

View File

@ -152,6 +152,7 @@ typedef struct /* cast_hash table entry */
{
plpgsql_CastHashKey key; /* hash key --- MUST BE FIRST */
Expr *cast_expr; /* cast expression, or NULL if no-op cast */
CachedExpression *cast_cexpr; /* cached expression backing the above */
/* ExprState is valid only when cast_lxid matches current LXID */
ExprState *cast_exprstate; /* expression's eval tree */
bool cast_in_use; /* true while we're executing eval tree */
@ -7610,18 +7611,34 @@ get_cast_hashentry(PLpgSQL_execstate *estate,
cast_key.dsttypmod = dsttypmod;
cast_entry = (plpgsql_CastHashEntry *) hash_search(estate->cast_hash,
(void *) &cast_key,
HASH_FIND, NULL);
HASH_ENTER, &found);
if (!found) /* initialize if new entry */
cast_entry->cast_cexpr = NULL;
if (cast_entry == NULL)
if (cast_entry->cast_cexpr == NULL ||
!cast_entry->cast_cexpr->is_valid)
{
/* We've not looked up this coercion before */
/*
* We've not looked up this coercion before, or we have but the cached
* expression has been invalidated.
*/
Node *cast_expr;
CachedExpression *cast_cexpr;
CaseTestExpr *placeholder;
/*
* Drop old cached expression if there is one.
*/
if (cast_entry->cast_cexpr)
{
FreeCachedExpression(cast_entry->cast_cexpr);
cast_entry->cast_cexpr = NULL;
}
/*
* Since we could easily fail (no such coercion), construct a
* temporary coercion expression tree in the short-lived
* eval_mcontext, then if successful copy it to cast_hash_context.
* eval_mcontext, then if successful save it as a CachedExpression.
*/
oldcontext = MemoryContextSwitchTo(get_eval_mcontext(estate));
@ -7682,33 +7699,23 @@ get_cast_hashentry(PLpgSQL_execstate *estate,
/* Note: we don't bother labeling the expression tree with collation */
/* Plan the expression and build a CachedExpression */
cast_cexpr = GetCachedExpression(cast_expr);
cast_expr = cast_cexpr->expr;
/* Detect whether we have a no-op (RelabelType) coercion */
if (IsA(cast_expr, RelabelType) &&
((RelabelType *) cast_expr)->arg == (Expr *) placeholder)
cast_expr = NULL;
if (cast_expr)
{
/* ExecInitExpr assumes we've planned the expression */
cast_expr = (Node *) expression_planner((Expr *) cast_expr);
/* Now copy the tree into cast_hash_context */
MemoryContextSwitchTo(estate->cast_hash_context);
cast_expr = copyObject(cast_expr);
}
MemoryContextSwitchTo(oldcontext);
/* Now we can fill in a hashtable entry. */
cast_entry = (plpgsql_CastHashEntry *) hash_search(estate->cast_hash,
(void *) &cast_key,
HASH_ENTER, &found);
Assert(!found); /* wasn't there a moment ago */
/* Now we can fill in the hashtable entry. */
cast_entry->cast_cexpr = cast_cexpr;
cast_entry->cast_expr = (Expr *) cast_expr;
cast_entry->cast_exprstate = NULL;
cast_entry->cast_in_use = false;
cast_entry->cast_lxid = InvalidLocalTransactionId;
MemoryContextSwitchTo(oldcontext);
}
/* Done if we have determined that this is a no-op cast. */

View File

@ -1019,6 +1019,31 @@ drop function array_elem_check(int);
create domain di as int;
create function dom_check(int) returns di as $$
declare d di;
begin
d := $1::di;
return d;
end
$$ language plpgsql immutable;
select dom_check(0);
dom_check
-----------
0
(1 row)
alter domain di add constraint pos check (value > 0);
select dom_check(0); -- fail
ERROR: value for domain di violates check constraint "pos"
CONTEXT: PL/pgSQL function dom_check(integer) line 4 at assignment
alter domain di drop constraint pos;
select dom_check(0);
dom_check
-----------
0
(1 row)
-- implicit cast during assignment is a separate code path, test that too
create or replace function dom_check(int) returns di as $$
declare d di;
begin
d := $1;
return d;

View File

@ -679,6 +679,26 @@ create domain di as int;
create function dom_check(int) returns di as $$
declare d di;
begin
d := $1::di;
return d;
end
$$ language plpgsql immutable;
select dom_check(0);
alter domain di add constraint pos check (value > 0);
select dom_check(0); -- fail
alter domain di drop constraint pos;
select dom_check(0);
-- implicit cast during assignment is a separate code path, test that too
create or replace function dom_check(int) returns di as $$
declare d di;
begin
d := $1;
return d;