postgresql/src/backend/nodes/equalfuncs.c

Ignoring revisions in .git-blame-ignore-revs. Click here to bypass and see the normal blame view.

4443 lines
99 KiB
C
Raw Normal View History

/*-------------------------------------------------------------------------
*
* equalfuncs.c
* Equality functions to compare node trees.
*
* NOTE: we currently support comparing all node types found in parse
* trees. We do not support comparing executor state trees; there
* is no need for that, and no point in maintaining all the code that
* would be needed. We also do not support comparing Path trees, mainly
* because the circular linkages between RelOptInfo and Path nodes can't
* be handled easily in a simple depth-first traversal.
*
* Currently, in fact, equal() doesn't know how to compare Plan trees
* either. This might need to be fixed someday.
*
* NOTE: it is intentional that parse location fields (in nodes that have
* one) are not compared. This is because we want, for example, a variable
* "x" to be considered equal() to another reference to "x" in the query.
*
*
* Portions Copyright (c) 1996-2022, PostgreSQL Global Development Group
* Portions Copyright (c) 1994, Regents of the University of California
*
* IDENTIFICATION
2010-09-20 22:08:53 +02:00
* src/backend/nodes/equalfuncs.c
*
*-------------------------------------------------------------------------
*/
#include "postgres.h"
#include "miscadmin.h"
#include "nodes/extensible.h"
#include "nodes/pathnodes.h"
#include "utils/datum.h"
/*
* Macros to simplify comparison of different kinds of fields. Use these
* wherever possible to reduce the chance for silly typos. Note that these
* hard-wire the convention that the local variables in an Equal routine are
* named 'a' and 'b'.
*/
/* Compare a simple scalar field (int, float, bool, enum, etc) */
#define COMPARE_SCALAR_FIELD(fldname) \
do { \
if (a->fldname != b->fldname) \
return false; \
} while (0)
/* Compare a field that is a pointer to some kind of Node or Node tree */
#define COMPARE_NODE_FIELD(fldname) \
do { \
if (!equal(a->fldname, b->fldname)) \
return false; \
} while (0)
/* Compare a field that is a pointer to a Bitmapset */
#define COMPARE_BITMAPSET_FIELD(fldname) \
do { \
if (!bms_equal(a->fldname, b->fldname)) \
return false; \
} while (0)
/* Compare a field that is a pointer to a C string, or perhaps NULL */
#define COMPARE_STRING_FIELD(fldname) \
do { \
if (!equalstr(a->fldname, b->fldname)) \
return false; \
} while (0)
/* Macro for comparing string fields that might be NULL */
#define equalstr(a, b) \
(((a) != NULL && (b) != NULL) ? (strcmp(a, b) == 0) : (a) == (b))
/* Compare a field that is an inline array */
#define COMPARE_ARRAY_FIELD(fldname) \
do { \
if (memcmp(a->fldname, b->fldname, sizeof(a->fldname)) != 0) \
return false; \
} while (0)
/* Compare a field that is a pointer to a simple palloc'd object of size sz */
#define COMPARE_POINTER_FIELD(fldname, sz) \
do { \
if (memcmp(a->fldname, b->fldname, (sz)) != 0) \
return false; \
} while (0)
/* Compare a parse location field (this is a no-op, per note above) */
#define COMPARE_LOCATION_FIELD(fldname) \
((void) 0)
/* Compare a CoercionForm field (also a no-op, per comment in primnodes.h) */
#define COMPARE_COERCIONFORM_FIELD(fldname) \
((void) 0)
/*
* Stuff from primnodes.h
*/
static bool
_equalAlias(const Alias *a, const Alias *b)
{
COMPARE_STRING_FIELD(aliasname);
COMPARE_NODE_FIELD(colnames);
return true;
}
static bool
_equalRangeVar(const RangeVar *a, const RangeVar *b)
{
COMPARE_STRING_FIELD(catalogname);
COMPARE_STRING_FIELD(schemaname);
COMPARE_STRING_FIELD(relname);
COMPARE_SCALAR_FIELD(inh);
COMPARE_SCALAR_FIELD(relpersistence);
COMPARE_NODE_FIELD(alias);
COMPARE_LOCATION_FIELD(location);
return true;
}
static bool
_equalTableFunc(const TableFunc *a, const TableFunc *b)
{
COMPARE_SCALAR_FIELD(functype);
COMPARE_NODE_FIELD(ns_uris);
COMPARE_NODE_FIELD(ns_names);
COMPARE_NODE_FIELD(docexpr);
COMPARE_NODE_FIELD(rowexpr);
COMPARE_NODE_FIELD(colnames);
COMPARE_NODE_FIELD(coltypes);
COMPARE_NODE_FIELD(coltypmods);
COMPARE_NODE_FIELD(colcollations);
COMPARE_NODE_FIELD(colexprs);
COMPARE_NODE_FIELD(coldefexprs);
COMPARE_NODE_FIELD(colvalexprs);
COMPARE_BITMAPSET_FIELD(notnulls);
COMPARE_NODE_FIELD(plan);
COMPARE_SCALAR_FIELD(ordinalitycol);
COMPARE_LOCATION_FIELD(location);
return true;
}
static bool
_equalJsonTablePlan(const JsonTablePlan *a, const JsonTablePlan *b)
{
COMPARE_SCALAR_FIELD(plan_type);
COMPARE_SCALAR_FIELD(join_type);
COMPARE_NODE_FIELD(plan1);
COMPARE_NODE_FIELD(plan2);
COMPARE_STRING_FIELD(pathname);
COMPARE_LOCATION_FIELD(location);
return true;
}
static bool
_equalJsonTable(const JsonTable *a, const JsonTable *b)
{
COMPARE_NODE_FIELD(common);
COMPARE_NODE_FIELD(columns);
COMPARE_NODE_FIELD(plan);
COMPARE_NODE_FIELD(on_error);
COMPARE_NODE_FIELD(alias);
COMPARE_SCALAR_FIELD(lateral);
COMPARE_LOCATION_FIELD(location);
return true;
}
static bool
_equalJsonTableColumn(const JsonTableColumn *a, const JsonTableColumn *b)
{
COMPARE_SCALAR_FIELD(coltype);
COMPARE_STRING_FIELD(name);
COMPARE_NODE_FIELD(typeName);
COMPARE_STRING_FIELD(pathspec);
COMPARE_STRING_FIELD(pathname);
COMPARE_NODE_FIELD(format);
COMPARE_SCALAR_FIELD(wrapper);
COMPARE_SCALAR_FIELD(omit_quotes);
COMPARE_NODE_FIELD(columns);
COMPARE_NODE_FIELD(on_empty);
COMPARE_NODE_FIELD(on_error);
COMPARE_LOCATION_FIELD(location);
return true;
}
static bool
_equalJsonTableParent(const JsonTableParent *a, const JsonTableParent *b)
{
COMPARE_NODE_FIELD(path);
COMPARE_STRING_FIELD(name);
COMPARE_NODE_FIELD(child);
COMPARE_SCALAR_FIELD(outerJoin);
COMPARE_SCALAR_FIELD(colMin);
COMPARE_SCALAR_FIELD(colMax);
COMPARE_SCALAR_FIELD(errorOnError);
return true;
}
static bool
_equalJsonTableSibling(const JsonTableSibling *a, const JsonTableSibling *b)
{
COMPARE_NODE_FIELD(larg);
COMPARE_NODE_FIELD(rarg);
COMPARE_SCALAR_FIELD(cross);
return true;
}
static bool
_equalIntoClause(const IntoClause *a, const IntoClause *b)
{
COMPARE_NODE_FIELD(rel);
COMPARE_NODE_FIELD(colNames);
COMPARE_STRING_FIELD(accessMethod);
COMPARE_NODE_FIELD(options);
COMPARE_SCALAR_FIELD(onCommit);
COMPARE_STRING_FIELD(tableSpaceName);
COMPARE_NODE_FIELD(viewQuery);
COMPARE_SCALAR_FIELD(skipData);
return true;
}
/*
* We don't need an _equalExpr because Expr is an abstract supertype which
* should never actually get instantiated. Also, since it has no common
* fields except NodeTag, there's no need for a helper routine to factor
* out comparing the common fields...
*/
static bool
_equalVar(const Var *a, const Var *b)
{
COMPARE_SCALAR_FIELD(varno);
COMPARE_SCALAR_FIELD(varattno);
COMPARE_SCALAR_FIELD(vartype);
COMPARE_SCALAR_FIELD(vartypmod);
COMPARE_SCALAR_FIELD(varcollid);
COMPARE_SCALAR_FIELD(varlevelsup);
Reconsider the representation of join alias Vars. The core idea of this patch is to make the parser generate join alias Vars (that is, ones with varno pointing to a JOIN RTE) only when the alias Var is actually different from any raw join input, that is a type coercion and/or COALESCE is necessary to generate the join output value. Otherwise just generate varno/varattno pointing to the relevant join input column. In effect, this means that the planner's flatten_join_alias_vars() transformation is already done in the parser, for all cases except (a) columns that are merged by JOIN USING and are transformed in the process, and (b) whole-row join Vars. In principle that would allow us to skip doing flatten_join_alias_vars() in many more queries than we do now, but we don't have quite enough infrastructure to know that we can do so --- in particular there's no cheap way to know whether there are any whole-row join Vars. I'm not sure if it's worth the trouble to add a Query-level flag for that, and in any case it seems like fit material for a separate patch. But even without skipping the work entirely, this should make flatten_join_alias_vars() faster, particularly where there are nested joins that it previously had to flatten recursively. An essential part of this change is to replace Var nodes' varnoold/varoattno fields with varnosyn/varattnosyn, which have considerably more tightly-defined meanings than the old fields: when they differ from varno/varattno, they identify the Var's position in an aliased JOIN RTE, and the join alias is what ruleutils.c should print for the Var. This is necessary because the varno change destroyed ruleutils.c's ability to find the JOIN RTE from the Var's varno. Another way in which this change broke ruleutils.c is that it's no longer feasible to determine, from a JOIN RTE's joinaliasvars list, which join columns correspond to which columns of the join's immediate input relations. (If those are sub-joins, the joinaliasvars entries may point to columns of their base relations, not the sub-joins.) But that was a horrid mess requiring a lot of fragile assumptions already, so let's just bite the bullet and add some more JOIN RTE fields to make it more straightforward to figure that out. I added two integer-List fields containing the relevant column numbers from the left and right input rels, plus a count of how many merged columns there are. This patch depends on the ParseNamespaceColumn infrastructure that I added in commit 5815696bc. The biggest bit of code change is restructuring transformFromClauseItem's handling of JOINs so that the ParseNamespaceColumn data is propagated upward correctly. Other than that and the ruleutils fixes, everything pretty much just works, though some processing is now inessential. I grabbed two pieces of low-hanging fruit in that line: 1. In find_expr_references, we don't need to recurse into join alias Vars anymore. There aren't any except for references to merged USING columns, which are more properly handled when we scan the join's RTE. This change actually fixes an edge-case issue: we will now record a dependency on any type-coercion function present in a USING column's joinaliasvar, even if that join column has no references in the query text. The odds of the missing dependency causing a problem seem quite small: you'd have to posit somebody dropping an implicit cast between two data types, without removing the types themselves, and then having a stored rule containing a whole-row Var for a join whose USING merge depends on that cast. So I don't feel a great need to change this in the back branches. But in theory this way is more correct. 2. markRTEForSelectPriv and markTargetListOrigin don't need to recurse into join alias Vars either, because the cases they care about don't apply to alias Vars for USING columns that are semantically distinct from the underlying columns. This removes the only case in which markVarForSelectPriv could be called with NULL for the RTE, so adjust the comments to describe that hack as being strictly internal to markRTEForSelectPriv. catversion bump required due to changes in stored rules. Discussion: https://postgr.es/m/7115.1577986646@sss.pgh.pa.us
2020-01-09 17:56:59 +01:00
/*
* varnosyn/varattnosyn are intentionally ignored here, because Vars with
* different syntactic identifiers are semantically the same as long as
* their varno/varattno match.
*/
COMPARE_LOCATION_FIELD(location);
return true;
}
static bool
_equalConst(const Const *a, const Const *b)
{
COMPARE_SCALAR_FIELD(consttype);
COMPARE_SCALAR_FIELD(consttypmod);
COMPARE_SCALAR_FIELD(constcollid);
COMPARE_SCALAR_FIELD(constlen);
COMPARE_SCALAR_FIELD(constisnull);
COMPARE_SCALAR_FIELD(constbyval);
COMPARE_LOCATION_FIELD(location);
/*
* We treat all NULL constants of the same type as equal. Someday this
* might need to change? But datumIsEqual doesn't work on nulls, so...
*/
if (a->constisnull)
return true;
return datumIsEqual(a->constvalue, b->constvalue,
a->constbyval, a->constlen);
}
static bool
_equalParam(const Param *a, const Param *b)
{
COMPARE_SCALAR_FIELD(paramkind);
COMPARE_SCALAR_FIELD(paramid);
COMPARE_SCALAR_FIELD(paramtype);
COMPARE_SCALAR_FIELD(paramtypmod);
COMPARE_SCALAR_FIELD(paramcollid);
COMPARE_LOCATION_FIELD(location);
return true;
}
static bool
_equalAggref(const Aggref *a, const Aggref *b)
{
COMPARE_SCALAR_FIELD(aggfnoid);
COMPARE_SCALAR_FIELD(aggtype);
COMPARE_SCALAR_FIELD(aggcollid);
COMPARE_SCALAR_FIELD(inputcollid);
Fix handling of argument and result datatypes for partial aggregation. When doing partial aggregation, the args list of the upper (combining) Aggref node is replaced by a Var representing the output of the partial aggregation steps, which has either the aggregate's transition data type or a serialized representation of that. However, nodeAgg.c blindly continued to use the args list as an indication of the user-level argument types. This broke resolution of polymorphic transition datatypes at executor startup (though it accidentally failed to fail for the ANYARRAY case, which is likely the only one anyone had tested). Moreover, the constructed FuncExpr passed to the finalfunc contained completely wrong information, which would have led to bogus answers or crashes for any case where the finalfunc examined that information (which is only likely to be with polymorphic aggregates using a non-polymorphic transition type). As an independent bug, apply_partialaggref_adjustment neglected to resolve a polymorphic transition datatype before assigning it as the output type of the lower-level Aggref node. This again accidentally failed to fail for ANYARRAY but would be unlikely to work in other cases. To fix the first problem, record the user-level argument types in a separate OID-list field of Aggref, and look to that rather than the args list when asking what the argument types were. (It turns out to be convenient to include any "direct" arguments in this list too, although those are not currently subject to being overwritten.) Rather than adding yet another resolve_aggregate_transtype() call to fix the second problem, add an aggtranstype field to Aggref, and store the resolved transition type OID there when the planner first computes it. (By doing this in the planner and not the parser, we can allow the aggregate's transition type to change from time to time, although no DDL support yet exists for that.) This saves nothing of consequence for simple non-polymorphic aggregates, but for polymorphic transition types we save a catalog lookup during executor startup as well as several planner lookups that are new in 9.6 due to parallel query planning. In passing, fix an error that was introduced into count_agg_clauses_walker some time ago: it was applying exprTypmod() to something that wasn't an expression node at all, but a TargetEntry. exprTypmod silently returned -1 so that there was not an obvious failure, but this broke the intended sensitivity of aggregate space consumption estimates to the typmod of varchar and similar data types. This part needs to be back-patched. Catversion bump due to change of stored Aggref nodes. Discussion: <8229.1466109074@sss.pgh.pa.us>
2016-06-18 03:44:37 +02:00
/* ignore aggtranstype since it might not be set yet */
COMPARE_NODE_FIELD(aggargtypes);
Support ordered-set (WITHIN GROUP) aggregates. This patch introduces generic support for ordered-set and hypothetical-set aggregate functions, as well as implementations of the instances defined in SQL:2008 (percentile_cont(), percentile_disc(), rank(), dense_rank(), percent_rank(), cume_dist()). We also added mode() though it is not in the spec, as well as versions of percentile_cont() and percentile_disc() that can compute multiple percentile values in one pass over the data. Unlike the original submission, this patch puts full control of the sorting process in the hands of the aggregate's support functions. To allow the support functions to find out how they're supposed to sort, a new API function AggGetAggref() is added to nodeAgg.c. This allows retrieval of the aggregate call's Aggref node, which may have other uses beyond the immediate need. There is also support for ordered-set aggregates to install cleanup callback functions, so that they can be sure that infrastructure such as tuplesort objects gets cleaned up. In passing, make some fixes in the recently-added support for variadic aggregates, and make some editorial adjustments in the recent FILTER additions for aggregates. Also, simplify use of IsBinaryCoercible() by allowing it to succeed whenever the target type is ANY or ANYELEMENT. It was inconsistent that it dealt with other polymorphic target types but not these. Atri Sharma and Andrew Gierth; reviewed by Pavel Stehule and Vik Fearing, and rather heavily editorialized upon by Tom Lane
2013-12-23 22:11:35 +01:00
COMPARE_NODE_FIELD(aggdirectargs);
COMPARE_NODE_FIELD(args);
COMPARE_NODE_FIELD(aggorder);
COMPARE_NODE_FIELD(aggdistinct);
COMPARE_NODE_FIELD(aggfilter);
COMPARE_SCALAR_FIELD(aggstar);
COMPARE_SCALAR_FIELD(aggvariadic);
Support ordered-set (WITHIN GROUP) aggregates. This patch introduces generic support for ordered-set and hypothetical-set aggregate functions, as well as implementations of the instances defined in SQL:2008 (percentile_cont(), percentile_disc(), rank(), dense_rank(), percent_rank(), cume_dist()). We also added mode() though it is not in the spec, as well as versions of percentile_cont() and percentile_disc() that can compute multiple percentile values in one pass over the data. Unlike the original submission, this patch puts full control of the sorting process in the hands of the aggregate's support functions. To allow the support functions to find out how they're supposed to sort, a new API function AggGetAggref() is added to nodeAgg.c. This allows retrieval of the aggregate call's Aggref node, which may have other uses beyond the immediate need. There is also support for ordered-set aggregates to install cleanup callback functions, so that they can be sure that infrastructure such as tuplesort objects gets cleaned up. In passing, make some fixes in the recently-added support for variadic aggregates, and make some editorial adjustments in the recent FILTER additions for aggregates. Also, simplify use of IsBinaryCoercible() by allowing it to succeed whenever the target type is ANY or ANYELEMENT. It was inconsistent that it dealt with other polymorphic target types but not these. Atri Sharma and Andrew Gierth; reviewed by Pavel Stehule and Vik Fearing, and rather heavily editorialized upon by Tom Lane
2013-12-23 22:11:35 +01:00
COMPARE_SCALAR_FIELD(aggkind);
COMPARE_SCALAR_FIELD(agglevelsup);
COMPARE_SCALAR_FIELD(aggsplit);
COMPARE_SCALAR_FIELD(aggno);
COMPARE_SCALAR_FIELD(aggtransno);
COMPARE_LOCATION_FIELD(location);
return true;
}
Support GROUPING SETS, CUBE and ROLLUP. This SQL standard functionality allows to aggregate data by different GROUP BY clauses at once. Each grouping set returns rows with columns grouped by in other sets set to NULL. This could previously be achieved by doing each grouping as a separate query, conjoined by UNION ALLs. Besides being considerably more concise, grouping sets will in many cases be faster, requiring only one scan over the underlying data. The current implementation of grouping sets only supports using sorting for input. Individual sets that share a sort order are computed in one pass. If there are sets that don't share a sort order, additional sort & aggregation steps are performed. These additional passes are sourced by the previous sort step; thus avoiding repeated scans of the source data. The code is structured in a way that adding support for purely using hash aggregation or a mix of hashing and sorting is possible. Sorting was chosen to be supported first, as it is the most generic method of implementation. Instead of, as in an earlier versions of the patch, representing the chain of sort and aggregation steps as full blown planner and executor nodes, all but the first sort are performed inside the aggregation node itself. This avoids the need to do some unusual gymnastics to handle having to return aggregated and non-aggregated tuples from underlying nodes, as well as having to shut down underlying nodes early to limit memory usage. The optimizer still builds Sort/Agg node to describe each phase, but they're not part of the plan tree, but instead additional data for the aggregation node. They're a convenient and preexisting way to describe aggregation and sorting. The first (and possibly only) sort step is still performed as a separate execution step. That retains similarity with existing group by plans, makes rescans fairly simple, avoids very deep plans (leading to slow explains) and easily allows to avoid the sorting step if the underlying data is sorted by other means. A somewhat ugly side of this patch is having to deal with a grammar ambiguity between the new CUBE keyword and the cube extension/functions named cube (and rollup). To avoid breaking existing deployments of the cube extension it has not been renamed, neither has cube been made a reserved keyword. Instead precedence hacking is used to make GROUP BY cube(..) refer to the CUBE grouping sets feature, and not the function cube(). To actually group by a function cube(), unlikely as that might be, the function name has to be quoted. Needs a catversion bump because stored rules may change. Author: Andrew Gierth and Atri Sharma, with contributions from Andres Freund Reviewed-By: Andres Freund, Noah Misch, Tom Lane, Svenne Krap, Tomas Vondra, Erik Rijkers, Marti Raudsepp, Pavel Stehule Discussion: CAOeZVidmVRe2jU6aMk_5qkxnB7dfmPROzM7Ur8JPW5j8Y5X-Lw@mail.gmail.com
2015-05-16 03:40:59 +02:00
static bool
_equalGroupingFunc(const GroupingFunc *a, const GroupingFunc *b)
{
COMPARE_NODE_FIELD(args);
/*
* We must not compare the refs or cols field
*/
COMPARE_SCALAR_FIELD(agglevelsup);
COMPARE_LOCATION_FIELD(location);
return true;
}
static bool
_equalWindowFunc(const WindowFunc *a, const WindowFunc *b)
{
COMPARE_SCALAR_FIELD(winfnoid);
COMPARE_SCALAR_FIELD(wintype);
COMPARE_SCALAR_FIELD(wincollid);
COMPARE_SCALAR_FIELD(inputcollid);
COMPARE_NODE_FIELD(args);
COMPARE_NODE_FIELD(aggfilter);
COMPARE_SCALAR_FIELD(winref);
COMPARE_SCALAR_FIELD(winstar);
COMPARE_SCALAR_FIELD(winagg);
COMPARE_LOCATION_FIELD(location);
return true;
}
static bool
_equalSubscriptingRef(const SubscriptingRef *a, const SubscriptingRef *b)
{
COMPARE_SCALAR_FIELD(refcontainertype);
COMPARE_SCALAR_FIELD(refelemtype);
Support subscripting of arbitrary types, not only arrays. This patch generalizes the subscripting infrastructure so that any data type can be subscripted, if it provides a handler function to define what that means. Traditional variable-length (varlena) arrays all use array_subscript_handler(), while the existing fixed-length types that support subscripting use raw_array_subscript_handler(). It's expected that other types that want to use subscripting notation will define their own handlers. (This patch provides no such new features, though; it only lays the foundation for them.) To do this, move the parser's semantic processing of subscripts (including coercion to whatever data type is required) into a method callback supplied by the handler. On the execution side, replace the ExecEvalSubscriptingRef* layer of functions with direct calls to callback-supplied execution routines. (Thus, essentially no new run-time overhead should be caused by this patch. Indeed, there is room to remove some overhead by supplying specialized execution routines. This patch does a little bit in that line, but more could be done.) Additional work is required here and there to remove formerly hard-wired assumptions about the result type, collation, etc of a SubscriptingRef expression node; and to remove assumptions that the subscript values must be integers. One useful side-effect of this is that we now have a less squishy mechanism for identifying whether a data type is a "true" array: instead of wiring in weird rules about typlen, we can look to see if pg_type.typsubscript == F_ARRAY_SUBSCRIPT_HANDLER. For this to be bulletproof, we have to forbid user-defined types from using that handler directly; but there seems no good reason for them to do so. This patch also removes assumptions that the number of subscripts is limited to MAXDIM (6), or indeed has any hard-wired limit. That limit still applies to types handled by array_subscript_handler or raw_array_subscript_handler, but to discourage other dependencies on this constant, I've moved it from c.h to utils/array.h. Dmitry Dolgov, reviewed at various times by Tom Lane, Arthur Zakirov, Peter Eisentraut, Pavel Stehule Discussion: https://postgr.es/m/CA+q6zcVDuGBv=M0FqBYX8DPebS3F_0KQ6OVFobGJPM507_SZ_w@mail.gmail.com Discussion: https://postgr.es/m/CA+q6zcVovR+XY4mfk-7oNk-rF91gH0PebnNfuUjuuDsyHjOcVA@mail.gmail.com
2020-12-09 18:40:37 +01:00
COMPARE_SCALAR_FIELD(refrestype);
COMPARE_SCALAR_FIELD(reftypmod);
COMPARE_SCALAR_FIELD(refcollid);
COMPARE_NODE_FIELD(refupperindexpr);
COMPARE_NODE_FIELD(reflowerindexpr);
COMPARE_NODE_FIELD(refexpr);
COMPARE_NODE_FIELD(refassgnexpr);
return true;
}
static bool
_equalFuncExpr(const FuncExpr *a, const FuncExpr *b)
{
COMPARE_SCALAR_FIELD(funcid);
COMPARE_SCALAR_FIELD(funcresulttype);
COMPARE_SCALAR_FIELD(funcretset);
COMPARE_SCALAR_FIELD(funcvariadic);
COMPARE_COERCIONFORM_FIELD(funcformat);
COMPARE_SCALAR_FIELD(funccollid);
COMPARE_SCALAR_FIELD(inputcollid);
COMPARE_NODE_FIELD(args);
COMPARE_LOCATION_FIELD(location);
return true;
}
static bool
_equalNamedArgExpr(const NamedArgExpr *a, const NamedArgExpr *b)
{
COMPARE_NODE_FIELD(arg);
COMPARE_STRING_FIELD(name);
COMPARE_SCALAR_FIELD(argnumber);
COMPARE_LOCATION_FIELD(location);
return true;
}
static bool
_equalOpExpr(const OpExpr *a, const OpExpr *b)
{
COMPARE_SCALAR_FIELD(opno);
2003-08-04 02:43:34 +02:00
/*
* Special-case opfuncid: it is allowable for it to differ if one node
* contains zero and the other doesn't. This just means that the one node
* isn't as far along in the parse/plan pipeline and hasn't had the
* opfuncid cache filled yet.
*/
if (a->opfuncid != b->opfuncid &&
a->opfuncid != 0 &&
b->opfuncid != 0)
return false;
COMPARE_SCALAR_FIELD(opresulttype);
COMPARE_SCALAR_FIELD(opretset);
COMPARE_SCALAR_FIELD(opcollid);
COMPARE_SCALAR_FIELD(inputcollid);
COMPARE_NODE_FIELD(args);
COMPARE_LOCATION_FIELD(location);
return true;
}
static bool
_equalDistinctExpr(const DistinctExpr *a, const DistinctExpr *b)
{
COMPARE_SCALAR_FIELD(opno);
2003-08-04 02:43:34 +02:00
/*
* Special-case opfuncid: it is allowable for it to differ if one node
* contains zero and the other doesn't. This just means that the one node
* isn't as far along in the parse/plan pipeline and hasn't had the
* opfuncid cache filled yet.
*/
if (a->opfuncid != b->opfuncid &&
a->opfuncid != 0 &&
b->opfuncid != 0)
return false;
COMPARE_SCALAR_FIELD(opresulttype);
COMPARE_SCALAR_FIELD(opretset);
COMPARE_SCALAR_FIELD(opcollid);
COMPARE_SCALAR_FIELD(inputcollid);
COMPARE_NODE_FIELD(args);
COMPARE_LOCATION_FIELD(location);
return true;
}
static bool
_equalNullIfExpr(const NullIfExpr *a, const NullIfExpr *b)
{
COMPARE_SCALAR_FIELD(opno);
/*
* Special-case opfuncid: it is allowable for it to differ if one node
* contains zero and the other doesn't. This just means that the one node
* isn't as far along in the parse/plan pipeline and hasn't had the
* opfuncid cache filled yet.
*/
if (a->opfuncid != b->opfuncid &&
a->opfuncid != 0 &&
b->opfuncid != 0)
return false;
COMPARE_SCALAR_FIELD(opresulttype);
COMPARE_SCALAR_FIELD(opretset);
COMPARE_SCALAR_FIELD(opcollid);
COMPARE_SCALAR_FIELD(inputcollid);
COMPARE_NODE_FIELD(args);
COMPARE_LOCATION_FIELD(location);
return true;
}
static bool
_equalScalarArrayOpExpr(const ScalarArrayOpExpr *a, const ScalarArrayOpExpr *b)
{
COMPARE_SCALAR_FIELD(opno);
2003-08-04 02:43:34 +02:00
/*
* Special-case opfuncid: it is allowable for it to differ if one node
* contains zero and the other doesn't. This just means that the one node
* isn't as far along in the parse/plan pipeline and hasn't had the
* opfuncid cache filled yet.
*/
if (a->opfuncid != b->opfuncid &&
a->opfuncid != 0 &&
b->opfuncid != 0)
return false;
Speedup ScalarArrayOpExpr evaluation ScalarArrayOpExprs with "useOr=true" and a set of Consts on the righthand side have traditionally been evaluated by using a linear search over the array. When these arrays contain large numbers of elements then this linear search could become a significant part of execution time. Here we add a new method of evaluating ScalarArrayOpExpr expressions to allow them to be evaluated by first building a hash table containing each element, then on subsequent evaluations, we just probe that hash table to determine if there is a match. The planner is in charge of determining when this optimization is possible and it enables it by setting hashfuncid in the ScalarArrayOpExpr. The executor will only perform the hash table evaluation when the hashfuncid is set. This means that not all cases are optimized. For example CHECK constraints containing an IN clause won't go through the planner, so won't get the hashfuncid set. We could maybe do something about that at some later date. The reason we're not doing it now is from fear that we may slow down cases where the expression is evaluated only once. Those cases can be common, for example, a single row INSERT to a table with a CHECK constraint containing an IN clause. In the planner, we enable this when there are suitable hash functions for the ScalarArrayOpExpr's operator and only when there is at least MIN_ARRAY_SIZE_FOR_HASHED_SAOP elements in the array. The threshold is currently set to 9. Author: James Coleman, David Rowley Reviewed-by: David Rowley, Tomas Vondra, Heikki Linnakangas Discussion: https://postgr.es/m/CAAaqYe8x62+=wn0zvNKCj55tPpg-JBHzhZFFc6ANovdqFw7-dA@mail.gmail.com
2021-04-08 13:51:22 +02:00
/* As above, hashfuncid may differ too */
if (a->hashfuncid != b->hashfuncid &&
a->hashfuncid != 0 &&
b->hashfuncid != 0)
return false;
/* Likewise for the negfuncid */
if (a->negfuncid != b->negfuncid &&
a->negfuncid != 0 &&
b->negfuncid != 0)
return false;
COMPARE_SCALAR_FIELD(useOr);
COMPARE_SCALAR_FIELD(inputcollid);
COMPARE_NODE_FIELD(args);
COMPARE_LOCATION_FIELD(location);
return true;
}
static bool
_equalBoolExpr(const BoolExpr *a, const BoolExpr *b)
{
COMPARE_SCALAR_FIELD(boolop);
COMPARE_NODE_FIELD(args);
COMPARE_LOCATION_FIELD(location);
return true;
}
static bool
_equalSubLink(const SubLink *a, const SubLink *b)
{
COMPARE_SCALAR_FIELD(subLinkType);
COMPARE_SCALAR_FIELD(subLinkId);
COMPARE_NODE_FIELD(testexpr);
COMPARE_NODE_FIELD(operName);
COMPARE_NODE_FIELD(subselect);
COMPARE_LOCATION_FIELD(location);
return true;
}
static bool
_equalSubPlan(const SubPlan *a, const SubPlan *b)
{
COMPARE_SCALAR_FIELD(subLinkType);
COMPARE_NODE_FIELD(testexpr);
COMPARE_NODE_FIELD(paramIds);
COMPARE_SCALAR_FIELD(plan_id);
COMPARE_STRING_FIELD(plan_name);
COMPARE_SCALAR_FIELD(firstColType);
COMPARE_SCALAR_FIELD(firstColTypmod);
COMPARE_SCALAR_FIELD(firstColCollation);
COMPARE_SCALAR_FIELD(useHashTable);
COMPARE_SCALAR_FIELD(unknownEqFalse);
COMPARE_SCALAR_FIELD(parallel_safe);
COMPARE_NODE_FIELD(setParam);
COMPARE_NODE_FIELD(parParam);
COMPARE_NODE_FIELD(args);
COMPARE_SCALAR_FIELD(startup_cost);
COMPARE_SCALAR_FIELD(per_call_cost);
return true;
}
static bool
_equalAlternativeSubPlan(const AlternativeSubPlan *a, const AlternativeSubPlan *b)
{
COMPARE_NODE_FIELD(subplans);
return true;
}
static bool
_equalFieldSelect(const FieldSelect *a, const FieldSelect *b)
{
COMPARE_NODE_FIELD(arg);
COMPARE_SCALAR_FIELD(fieldnum);
COMPARE_SCALAR_FIELD(resulttype);
COMPARE_SCALAR_FIELD(resulttypmod);
COMPARE_SCALAR_FIELD(resultcollid);
return true;
}
static bool
_equalFieldStore(const FieldStore *a, const FieldStore *b)
{
COMPARE_NODE_FIELD(arg);
COMPARE_NODE_FIELD(newvals);
COMPARE_NODE_FIELD(fieldnums);
COMPARE_SCALAR_FIELD(resulttype);
return true;
}
static bool
_equalRelabelType(const RelabelType *a, const RelabelType *b)
{
COMPARE_NODE_FIELD(arg);
COMPARE_SCALAR_FIELD(resulttype);
COMPARE_SCALAR_FIELD(resulttypmod);
COMPARE_SCALAR_FIELD(resultcollid);
COMPARE_COERCIONFORM_FIELD(relabelformat);
COMPARE_LOCATION_FIELD(location);
return true;
}
static bool
_equalCoerceViaIO(const CoerceViaIO *a, const CoerceViaIO *b)
{
COMPARE_NODE_FIELD(arg);
COMPARE_SCALAR_FIELD(resulttype);
COMPARE_SCALAR_FIELD(resultcollid);
COMPARE_COERCIONFORM_FIELD(coerceformat);
COMPARE_LOCATION_FIELD(location);
return true;
}
static bool
_equalArrayCoerceExpr(const ArrayCoerceExpr *a, const ArrayCoerceExpr *b)
{
COMPARE_NODE_FIELD(arg);
Support arrays over domains. Allowing arrays with a domain type as their element type was left un-done in the original domain patch, but not for any very good reason. This omission leads to such surprising results as array_agg() not working on a domain column, because the parser can't identify a suitable output type for the polymorphic aggregate. In order to fix this, first clean up the APIs of coerce_to_domain() and some internal functions in parse_coerce.c so that we consistently pass around a CoercionContext along with CoercionForm. Previously, we sometimes passed an "isExplicit" boolean flag instead, which is strictly less information; and coerce_to_domain() didn't even get that, but instead had to reverse-engineer isExplicit from CoercionForm. That's contrary to the documentation in primnodes.h that says that CoercionForm only affects display and not semantics. I don't think this change fixes any live bugs, but it makes things more consistent. The main reason for doing it though is that now build_coercion_expression() receives ccontext, which it needs in order to be able to recursively invoke coerce_to_target_type(). Next, reimplement ArrayCoerceExpr so that the node does not directly know any details of what has to be done to the individual array elements while performing the array coercion. Instead, the per-element processing is represented by a sub-expression whose input is a source array element and whose output is a target array element. This simplifies life in parse_coerce.c, because it can build that sub-expression by a recursive invocation of coerce_to_target_type(). The executor now handles the per-element processing as a compiled expression instead of hard-wired code. The main advantage of this is that we can use a single ArrayCoerceExpr to handle as many as three successive steps per element: base type conversion, typmod coercion, and domain constraint checking. The old code used two stacked ArrayCoerceExprs to handle type + typmod coercion, which was pretty inefficient, and adding yet another array deconstruction to do domain constraint checking seemed very unappetizing. In the case where we just need a single, very simple coercion function, doing this straightforwardly leads to a noticeable increase in the per-array-element runtime cost. Hence, add an additional shortcut evalfunc in execExprInterp.c that skips unnecessary overhead for that specific form of expression. The runtime speed of simple cases is within 1% or so of where it was before, while cases that previously required two levels of array processing are significantly faster. Finally, create an implicit array type for every domain type, as we do for base types, enums, etc. Everything except the array-coercion case seems to just work without further effort. Tom Lane, reviewed by Andrew Dunstan Discussion: https://postgr.es/m/9852.1499791473@sss.pgh.pa.us
2017-09-30 19:40:56 +02:00
COMPARE_NODE_FIELD(elemexpr);
COMPARE_SCALAR_FIELD(resulttype);
COMPARE_SCALAR_FIELD(resulttypmod);
COMPARE_SCALAR_FIELD(resultcollid);
COMPARE_COERCIONFORM_FIELD(coerceformat);
COMPARE_LOCATION_FIELD(location);
return true;
}
static bool
_equalConvertRowtypeExpr(const ConvertRowtypeExpr *a, const ConvertRowtypeExpr *b)
{
COMPARE_NODE_FIELD(arg);
COMPARE_SCALAR_FIELD(resulttype);
COMPARE_COERCIONFORM_FIELD(convertformat);
COMPARE_LOCATION_FIELD(location);
return true;
}
static bool
_equalCollateExpr(const CollateExpr *a, const CollateExpr *b)
{
COMPARE_NODE_FIELD(arg);
COMPARE_SCALAR_FIELD(collOid);
COMPARE_LOCATION_FIELD(location);
return true;
}
static bool
_equalCaseExpr(const CaseExpr *a, const CaseExpr *b)
{
COMPARE_SCALAR_FIELD(casetype);
COMPARE_SCALAR_FIELD(casecollid);
COMPARE_NODE_FIELD(arg);
COMPARE_NODE_FIELD(args);
COMPARE_NODE_FIELD(defresult);
COMPARE_LOCATION_FIELD(location);
return true;
}
static bool
_equalCaseWhen(const CaseWhen *a, const CaseWhen *b)
{
COMPARE_NODE_FIELD(expr);
COMPARE_NODE_FIELD(result);
COMPARE_LOCATION_FIELD(location);
return true;
}
static bool
_equalCaseTestExpr(const CaseTestExpr *a, const CaseTestExpr *b)
{
COMPARE_SCALAR_FIELD(typeId);
COMPARE_SCALAR_FIELD(typeMod);
COMPARE_SCALAR_FIELD(collation);
return true;
}
static bool
_equalArrayExpr(const ArrayExpr *a, const ArrayExpr *b)
{
COMPARE_SCALAR_FIELD(array_typeid);
COMPARE_SCALAR_FIELD(array_collid);
COMPARE_SCALAR_FIELD(element_typeid);
COMPARE_NODE_FIELD(elements);
COMPARE_SCALAR_FIELD(multidims);
COMPARE_LOCATION_FIELD(location);
return true;
}
static bool
_equalRowExpr(const RowExpr *a, const RowExpr *b)
{
COMPARE_NODE_FIELD(args);
COMPARE_SCALAR_FIELD(row_typeid);
COMPARE_COERCIONFORM_FIELD(row_format);
COMPARE_NODE_FIELD(colnames);
COMPARE_LOCATION_FIELD(location);
return true;
}
static bool
_equalRowCompareExpr(const RowCompareExpr *a, const RowCompareExpr *b)
{
COMPARE_SCALAR_FIELD(rctype);
COMPARE_NODE_FIELD(opnos);
COMPARE_NODE_FIELD(opfamilies);
COMPARE_NODE_FIELD(inputcollids);
COMPARE_NODE_FIELD(largs);
COMPARE_NODE_FIELD(rargs);
return true;
}
static bool
_equalCoalesceExpr(const CoalesceExpr *a, const CoalesceExpr *b)
{
COMPARE_SCALAR_FIELD(coalescetype);
COMPARE_SCALAR_FIELD(coalescecollid);
COMPARE_NODE_FIELD(args);
COMPARE_LOCATION_FIELD(location);
return true;
}
static bool
_equalMinMaxExpr(const MinMaxExpr *a, const MinMaxExpr *b)
{
COMPARE_SCALAR_FIELD(minmaxtype);
COMPARE_SCALAR_FIELD(minmaxcollid);
COMPARE_SCALAR_FIELD(inputcollid);
COMPARE_SCALAR_FIELD(op);
COMPARE_NODE_FIELD(args);
COMPARE_LOCATION_FIELD(location);
return true;
}
static bool
_equalSQLValueFunction(const SQLValueFunction *a, const SQLValueFunction *b)
{
COMPARE_SCALAR_FIELD(op);
COMPARE_SCALAR_FIELD(type);
COMPARE_SCALAR_FIELD(typmod);
COMPARE_LOCATION_FIELD(location);
return true;
}
static bool
_equalXmlExpr(const XmlExpr *a, const XmlExpr *b)
{
COMPARE_SCALAR_FIELD(op);
COMPARE_STRING_FIELD(name);
COMPARE_NODE_FIELD(named_args);
COMPARE_NODE_FIELD(arg_names);
COMPARE_NODE_FIELD(args);
COMPARE_SCALAR_FIELD(xmloption);
COMPARE_SCALAR_FIELD(type);
COMPARE_SCALAR_FIELD(typmod);
COMPARE_LOCATION_FIELD(location);
return true;
}
static bool
_equalNullTest(const NullTest *a, const NullTest *b)
{
COMPARE_NODE_FIELD(arg);
COMPARE_SCALAR_FIELD(nulltesttype);
COMPARE_SCALAR_FIELD(argisrow);
COMPARE_LOCATION_FIELD(location);
return true;
}
static bool
_equalBooleanTest(const BooleanTest *a, const BooleanTest *b)
{
COMPARE_NODE_FIELD(arg);
COMPARE_SCALAR_FIELD(booltesttype);
COMPARE_LOCATION_FIELD(location);
return true;
}
static bool
_equalCoerceToDomain(const CoerceToDomain *a, const CoerceToDomain *b)
{
COMPARE_NODE_FIELD(arg);
COMPARE_SCALAR_FIELD(resulttype);
COMPARE_SCALAR_FIELD(resulttypmod);
COMPARE_SCALAR_FIELD(resultcollid);
COMPARE_COERCIONFORM_FIELD(coercionformat);
COMPARE_LOCATION_FIELD(location);
return true;
}
static bool
_equalCoerceToDomainValue(const CoerceToDomainValue *a, const CoerceToDomainValue *b)
{
COMPARE_SCALAR_FIELD(typeId);
COMPARE_SCALAR_FIELD(typeMod);
COMPARE_SCALAR_FIELD(collation);
COMPARE_LOCATION_FIELD(location);
return true;
}
static bool
_equalSetToDefault(const SetToDefault *a, const SetToDefault *b)
{
COMPARE_SCALAR_FIELD(typeId);
COMPARE_SCALAR_FIELD(typeMod);
COMPARE_SCALAR_FIELD(collation);
COMPARE_LOCATION_FIELD(location);
return true;
}
static bool
_equalCurrentOfExpr(const CurrentOfExpr *a, const CurrentOfExpr *b)
{
COMPARE_SCALAR_FIELD(cvarno);
COMPARE_STRING_FIELD(cursor_name);
COMPARE_SCALAR_FIELD(cursor_param);
return true;
}
static bool
_equalNextValueExpr(const NextValueExpr *a, const NextValueExpr *b)
{
COMPARE_SCALAR_FIELD(seqid);
COMPARE_SCALAR_FIELD(typeId);
return true;
}
Add support for INSERT ... ON CONFLICT DO NOTHING/UPDATE. The newly added ON CONFLICT clause allows to specify an alternative to raising a unique or exclusion constraint violation error when inserting. ON CONFLICT refers to constraints that can either be specified using a inference clause (by specifying the columns of a unique constraint) or by naming a unique or exclusion constraint. DO NOTHING avoids the constraint violation, without touching the pre-existing row. DO UPDATE SET ... [WHERE ...] updates the pre-existing tuple, and has access to both the tuple proposed for insertion and the existing tuple; the optional WHERE clause can be used to prevent an update from being executed. The UPDATE SET and WHERE clauses have access to the tuple proposed for insertion using the "magic" EXCLUDED alias, and to the pre-existing tuple using the table name or its alias. This feature is often referred to as upsert. This is implemented using a new infrastructure called "speculative insertion". It is an optimistic variant of regular insertion that first does a pre-check for existing tuples and then attempts an insert. If a violating tuple was inserted concurrently, the speculatively inserted tuple is deleted and a new attempt is made. If the pre-check finds a matching tuple the alternative DO NOTHING or DO UPDATE action is taken. If the insertion succeeds without detecting a conflict, the tuple is deemed inserted. To handle the possible ambiguity between the excluded alias and a table named excluded, and for convenience with long relation names, INSERT INTO now can alias its target table. Bumps catversion as stored rules change. Author: Peter Geoghegan, with significant contributions from Heikki Linnakangas and Andres Freund. Testing infrastructure by Jeff Janes. Reviewed-By: Heikki Linnakangas, Andres Freund, Robert Haas, Simon Riggs, Dean Rasheed, Stephen Frost and many others.
2015-05-08 05:31:36 +02:00
static bool
_equalInferenceElem(const InferenceElem *a, const InferenceElem *b)
{
COMPARE_NODE_FIELD(expr);
COMPARE_SCALAR_FIELD(infercollid);
COMPARE_SCALAR_FIELD(inferopclass);
Add support for INSERT ... ON CONFLICT DO NOTHING/UPDATE. The newly added ON CONFLICT clause allows to specify an alternative to raising a unique or exclusion constraint violation error when inserting. ON CONFLICT refers to constraints that can either be specified using a inference clause (by specifying the columns of a unique constraint) or by naming a unique or exclusion constraint. DO NOTHING avoids the constraint violation, without touching the pre-existing row. DO UPDATE SET ... [WHERE ...] updates the pre-existing tuple, and has access to both the tuple proposed for insertion and the existing tuple; the optional WHERE clause can be used to prevent an update from being executed. The UPDATE SET and WHERE clauses have access to the tuple proposed for insertion using the "magic" EXCLUDED alias, and to the pre-existing tuple using the table name or its alias. This feature is often referred to as upsert. This is implemented using a new infrastructure called "speculative insertion". It is an optimistic variant of regular insertion that first does a pre-check for existing tuples and then attempts an insert. If a violating tuple was inserted concurrently, the speculatively inserted tuple is deleted and a new attempt is made. If the pre-check finds a matching tuple the alternative DO NOTHING or DO UPDATE action is taken. If the insertion succeeds without detecting a conflict, the tuple is deemed inserted. To handle the possible ambiguity between the excluded alias and a table named excluded, and for convenience with long relation names, INSERT INTO now can alias its target table. Bumps catversion as stored rules change. Author: Peter Geoghegan, with significant contributions from Heikki Linnakangas and Andres Freund. Testing infrastructure by Jeff Janes. Reviewed-By: Heikki Linnakangas, Andres Freund, Robert Haas, Simon Riggs, Dean Rasheed, Stephen Frost and many others.
2015-05-08 05:31:36 +02:00
return true;
}
static bool
_equalTargetEntry(const TargetEntry *a, const TargetEntry *b)
{
COMPARE_NODE_FIELD(expr);
COMPARE_SCALAR_FIELD(resno);
COMPARE_STRING_FIELD(resname);
COMPARE_SCALAR_FIELD(ressortgroupref);
COMPARE_SCALAR_FIELD(resorigtbl);
COMPARE_SCALAR_FIELD(resorigcol);
COMPARE_SCALAR_FIELD(resjunk);
return true;
}
static bool
_equalRangeTblRef(const RangeTblRef *a, const RangeTblRef *b)
{
COMPARE_SCALAR_FIELD(rtindex);
return true;
}
static bool
_equalJoinExpr(const JoinExpr *a, const JoinExpr *b)
{
COMPARE_SCALAR_FIELD(jointype);
COMPARE_SCALAR_FIELD(isNatural);
COMPARE_NODE_FIELD(larg);
COMPARE_NODE_FIELD(rarg);
COMPARE_NODE_FIELD(usingClause);
COMPARE_NODE_FIELD(join_using_alias);
COMPARE_NODE_FIELD(quals);
COMPARE_NODE_FIELD(alias);
COMPARE_SCALAR_FIELD(rtindex);
return true;
}
static bool
_equalFromExpr(const FromExpr *a, const FromExpr *b)
{
COMPARE_NODE_FIELD(fromlist);
COMPARE_NODE_FIELD(quals);
return true;
}
Add support for INSERT ... ON CONFLICT DO NOTHING/UPDATE. The newly added ON CONFLICT clause allows to specify an alternative to raising a unique or exclusion constraint violation error when inserting. ON CONFLICT refers to constraints that can either be specified using a inference clause (by specifying the columns of a unique constraint) or by naming a unique or exclusion constraint. DO NOTHING avoids the constraint violation, without touching the pre-existing row. DO UPDATE SET ... [WHERE ...] updates the pre-existing tuple, and has access to both the tuple proposed for insertion and the existing tuple; the optional WHERE clause can be used to prevent an update from being executed. The UPDATE SET and WHERE clauses have access to the tuple proposed for insertion using the "magic" EXCLUDED alias, and to the pre-existing tuple using the table name or its alias. This feature is often referred to as upsert. This is implemented using a new infrastructure called "speculative insertion". It is an optimistic variant of regular insertion that first does a pre-check for existing tuples and then attempts an insert. If a violating tuple was inserted concurrently, the speculatively inserted tuple is deleted and a new attempt is made. If the pre-check finds a matching tuple the alternative DO NOTHING or DO UPDATE action is taken. If the insertion succeeds without detecting a conflict, the tuple is deemed inserted. To handle the possible ambiguity between the excluded alias and a table named excluded, and for convenience with long relation names, INSERT INTO now can alias its target table. Bumps catversion as stored rules change. Author: Peter Geoghegan, with significant contributions from Heikki Linnakangas and Andres Freund. Testing infrastructure by Jeff Janes. Reviewed-By: Heikki Linnakangas, Andres Freund, Robert Haas, Simon Riggs, Dean Rasheed, Stephen Frost and many others.
2015-05-08 05:31:36 +02:00
static bool
_equalOnConflictExpr(const OnConflictExpr *a, const OnConflictExpr *b)
{
COMPARE_SCALAR_FIELD(action);
COMPARE_NODE_FIELD(arbiterElems);
COMPARE_NODE_FIELD(arbiterWhere);
COMPARE_SCALAR_FIELD(constraint);
Add support for INSERT ... ON CONFLICT DO NOTHING/UPDATE. The newly added ON CONFLICT clause allows to specify an alternative to raising a unique or exclusion constraint violation error when inserting. ON CONFLICT refers to constraints that can either be specified using a inference clause (by specifying the columns of a unique constraint) or by naming a unique or exclusion constraint. DO NOTHING avoids the constraint violation, without touching the pre-existing row. DO UPDATE SET ... [WHERE ...] updates the pre-existing tuple, and has access to both the tuple proposed for insertion and the existing tuple; the optional WHERE clause can be used to prevent an update from being executed. The UPDATE SET and WHERE clauses have access to the tuple proposed for insertion using the "magic" EXCLUDED alias, and to the pre-existing tuple using the table name or its alias. This feature is often referred to as upsert. This is implemented using a new infrastructure called "speculative insertion". It is an optimistic variant of regular insertion that first does a pre-check for existing tuples and then attempts an insert. If a violating tuple was inserted concurrently, the speculatively inserted tuple is deleted and a new attempt is made. If the pre-check finds a matching tuple the alternative DO NOTHING or DO UPDATE action is taken. If the insertion succeeds without detecting a conflict, the tuple is deemed inserted. To handle the possible ambiguity between the excluded alias and a table named excluded, and for convenience with long relation names, INSERT INTO now can alias its target table. Bumps catversion as stored rules change. Author: Peter Geoghegan, with significant contributions from Heikki Linnakangas and Andres Freund. Testing infrastructure by Jeff Janes. Reviewed-By: Heikki Linnakangas, Andres Freund, Robert Haas, Simon Riggs, Dean Rasheed, Stephen Frost and many others.
2015-05-08 05:31:36 +02:00
COMPARE_NODE_FIELD(onConflictSet);
COMPARE_NODE_FIELD(onConflictWhere);
COMPARE_SCALAR_FIELD(exclRelIndex);
COMPARE_NODE_FIELD(exclRelTlist);
return true;
}
static bool
_equalJsonFormat(const JsonFormat *a, const JsonFormat *b)
{
COMPARE_SCALAR_FIELD(format_type);
COMPARE_SCALAR_FIELD(encoding);
COMPARE_LOCATION_FIELD(location);
return true;
}
static bool
_equalJsonReturning(const JsonReturning *a, const JsonReturning *b)
{
COMPARE_NODE_FIELD(format);
COMPARE_SCALAR_FIELD(typid);
COMPARE_SCALAR_FIELD(typmod);
return true;
}
static bool
_equalJsonValueExpr(const JsonValueExpr *a, const JsonValueExpr *b)
{
COMPARE_NODE_FIELD(raw_expr);
COMPARE_NODE_FIELD(formatted_expr);
COMPARE_NODE_FIELD(format);
return true;
}
static bool
_equalJsonParseExpr(const JsonParseExpr *a, const JsonParseExpr *b)
{
COMPARE_NODE_FIELD(expr);
COMPARE_NODE_FIELD(output);
COMPARE_SCALAR_FIELD(unique_keys);
COMPARE_LOCATION_FIELD(location);
return true;
}
static bool
_equalJsonScalarExpr(const JsonScalarExpr *a, const JsonScalarExpr *b)
{
COMPARE_NODE_FIELD(expr);
COMPARE_NODE_FIELD(output);
COMPARE_LOCATION_FIELD(location);
return true;
}
static bool
_equalJsonSerializeExpr(const JsonSerializeExpr *a, const JsonSerializeExpr *b)
{
COMPARE_NODE_FIELD(expr);
COMPARE_NODE_FIELD(output);
COMPARE_LOCATION_FIELD(location);
return true;
}
static bool
_equalJsonConstructorExpr(const JsonConstructorExpr *a, const JsonConstructorExpr *b)
{
COMPARE_SCALAR_FIELD(type);
COMPARE_NODE_FIELD(args);
COMPARE_NODE_FIELD(func);
COMPARE_NODE_FIELD(coercion);
COMPARE_NODE_FIELD(returning);
COMPARE_SCALAR_FIELD(absent_on_null);
COMPARE_SCALAR_FIELD(unique);
COMPARE_LOCATION_FIELD(location);
return true;
}
static bool
_equalJsonKeyValue(const JsonKeyValue *a, const JsonKeyValue *b)
{
COMPARE_NODE_FIELD(key);
COMPARE_NODE_FIELD(value);
return true;
}
static bool
_equalJsonObjectConstructor(const JsonObjectConstructor *a,
const JsonObjectConstructor *b)
{
COMPARE_NODE_FIELD(exprs);
COMPARE_NODE_FIELD(output);
COMPARE_SCALAR_FIELD(absent_on_null);
COMPARE_SCALAR_FIELD(unique);
COMPARE_LOCATION_FIELD(location);
return true;
}
static bool
_equalJsonAggConstructor(const JsonAggConstructor *a,
const JsonAggConstructor *b)
{
COMPARE_NODE_FIELD(output);
COMPARE_NODE_FIELD(agg_filter);
COMPARE_NODE_FIELD(agg_order);
COMPARE_NODE_FIELD(over);
COMPARE_LOCATION_FIELD(location);
return true;
}
static bool
_equalJsonObjectAgg(const JsonObjectAgg *a, const JsonObjectAgg *b)
{
COMPARE_NODE_FIELD(constructor);
COMPARE_NODE_FIELD(arg);
COMPARE_SCALAR_FIELD(absent_on_null);
COMPARE_SCALAR_FIELD(unique);
return true;
}
static bool
_equalJsonOutput(const JsonOutput *a, const JsonOutput *b)
{
COMPARE_NODE_FIELD(typeName);
COMPARE_NODE_FIELD(returning);
return true;
}
static bool
_equalJsonArrayConstructor(const JsonArrayConstructor *a,
const JsonArrayConstructor *b)
{
COMPARE_NODE_FIELD(exprs);
COMPARE_NODE_FIELD(output);
COMPARE_SCALAR_FIELD(absent_on_null);
COMPARE_LOCATION_FIELD(location);
return true;
}
static bool
_equalJsonArrayAgg(const JsonArrayAgg *a, const JsonArrayAgg *b)
{
COMPARE_NODE_FIELD(constructor);
COMPARE_NODE_FIELD(arg);
COMPARE_SCALAR_FIELD(absent_on_null);
return true;
}
static bool
_equalJsonArrayQueryConstructor(const JsonArrayQueryConstructor *a,
const JsonArrayQueryConstructor *b)
{
COMPARE_NODE_FIELD(query);
COMPARE_NODE_FIELD(output);
COMPARE_NODE_FIELD(format);
COMPARE_SCALAR_FIELD(absent_on_null);
COMPARE_LOCATION_FIELD(location);
return true;
}
static bool
_equalJsonFuncExpr(const JsonFuncExpr *a, const JsonFuncExpr *b)
{
COMPARE_SCALAR_FIELD(op);
COMPARE_NODE_FIELD(common);
COMPARE_NODE_FIELD(output);
COMPARE_NODE_FIELD(on_empty);
COMPARE_NODE_FIELD(on_error);
COMPARE_SCALAR_FIELD(wrapper);
COMPARE_SCALAR_FIELD(omit_quotes);
COMPARE_LOCATION_FIELD(location);
return true;
}
static bool
_equalJsonCommon(const JsonCommon *a, const JsonCommon *b)
{
COMPARE_NODE_FIELD(expr);
COMPARE_NODE_FIELD(pathspec);
COMPARE_STRING_FIELD(pathname);
COMPARE_NODE_FIELD(passing);
COMPARE_LOCATION_FIELD(location);
return true;
}
static bool
_equalJsonArgument(const JsonArgument *a, const JsonArgument *b)
{
COMPARE_NODE_FIELD(val);
COMPARE_STRING_FIELD(name);
return true;
}
static bool
_equalJsonIsPredicate(const JsonIsPredicate *a,
const JsonIsPredicate *b)
{
COMPARE_NODE_FIELD(expr);
COMPARE_NODE_FIELD(format);
COMPARE_SCALAR_FIELD(item_type);
COMPARE_SCALAR_FIELD(unique_keys);
COMPARE_LOCATION_FIELD(location);
return true;
}
/*
* _equalJsonBehavior
*/
static bool
_equalJsonBehavior(const JsonBehavior *a, const JsonBehavior *b)
{
COMPARE_SCALAR_FIELD(btype);
COMPARE_NODE_FIELD(default_expr);
return true;
}
/*
* _equalJsonExpr
*/
static bool
_equalJsonExpr(const JsonExpr *a, const JsonExpr *b)
{
COMPARE_SCALAR_FIELD(op);
COMPARE_NODE_FIELD(formatted_expr);
COMPARE_NODE_FIELD(result_coercion);
COMPARE_NODE_FIELD(format);
COMPARE_NODE_FIELD(path_spec);
COMPARE_NODE_FIELD(passing_names);
COMPARE_NODE_FIELD(passing_values);
COMPARE_NODE_FIELD(returning);
COMPARE_NODE_FIELD(on_empty);
COMPARE_NODE_FIELD(on_error);
COMPARE_NODE_FIELD(coercions);
COMPARE_SCALAR_FIELD(wrapper);
COMPARE_SCALAR_FIELD(omit_quotes);
COMPARE_LOCATION_FIELD(location);
return true;
}
/*
* _equalJsonCoercion
*/
static bool
_equalJsonCoercion(const JsonCoercion *a, const JsonCoercion *b)
{
COMPARE_NODE_FIELD(expr);
COMPARE_SCALAR_FIELD(via_populate);
COMPARE_SCALAR_FIELD(via_io);
COMPARE_SCALAR_FIELD(collation);
return true;
}
/*
* _equalJsonItemCoercions
*/
static bool
_equalJsonItemCoercions(const JsonItemCoercions *a, const JsonItemCoercions *b)
{
COMPARE_NODE_FIELD(null);
COMPARE_NODE_FIELD(string);
COMPARE_NODE_FIELD(numeric);
COMPARE_NODE_FIELD(boolean);
COMPARE_NODE_FIELD(date);
COMPARE_NODE_FIELD(time);
COMPARE_NODE_FIELD(timetz);
COMPARE_NODE_FIELD(timestamp);
COMPARE_NODE_FIELD(timestamptz);
COMPARE_NODE_FIELD(composite);
return true;
}
1998-08-02 00:12:13 +02:00
/*
* Stuff from pathnodes.h
1998-08-02 00:12:13 +02:00
*/
static bool
_equalPathKey(const PathKey *a, const PathKey *b)
{
Postpone creation of pathkeys lists to fix bug #8049. This patch gets rid of the concept of, and infrastructure for, non-canonical PathKeys; we now only ever create canonical pathkey lists. The need for non-canonical pathkeys came from the desire to have grouping_planner initialize query_pathkeys and related pathkey lists before calling query_planner. However, since query_planner didn't actually *do* anything with those lists before they'd been made canonical, we can get rid of the whole mess by just not creating the lists at all until the point where we formerly canonicalized them. There are several ways in which we could implement that without making query_planner itself deal with grouping/sorting features (which are supposed to be the province of grouping_planner). I chose to add a callback function to query_planner's API; other alternatives would have required adding more fields to PlannerInfo, which while not bad in itself would create an ABI break for planner-related plugins in the 9.2 release series. This still breaks ABI for anything that calls query_planner directly, but it seems somewhat unlikely that there are any such plugins. I had originally conceived of this change as merely a step on the way to fixing bug #8049 from Teun Hoogendoorn; but it turns out that this fixes that bug all by itself, as per the added regression test. The reason is that now get_eclass_for_sort_expr is adding the ORDER BY expression at the end of EquivalenceClass creation not the start, and so anything that is in a multi-member EquivalenceClass has already been created with correct em_nullable_relids. I am suspicious that there are related scenarios in which we still need to teach get_eclass_for_sort_expr to compute correct nullable_relids, but am not eager to risk destabilizing either 9.2 or 9.3 to fix bugs that are only hypothetical. So for the moment, do this and stop here. Back-patch to 9.2 but not to earlier branches, since they don't exhibit this bug for lack of join-clause-movement logic that depends on em_nullable_relids being correct. (We might have to revisit that choice if any related bugs turn up.) In 9.2, don't change the signature of make_pathkeys_for_sortclauses nor remove canonicalize_pathkeys, so as not to risk more plugin breakage than we have to.
2013-04-29 20:49:01 +02:00
/* We assume pointer equality is sufficient to compare the eclasses */
COMPARE_SCALAR_FIELD(pk_eclass);
COMPARE_SCALAR_FIELD(pk_opfamily);
COMPARE_SCALAR_FIELD(pk_strategy);
COMPARE_SCALAR_FIELD(pk_nulls_first);
return true;
}
static bool
_equalRestrictInfo(const RestrictInfo *a, const RestrictInfo *b)
{
COMPARE_NODE_FIELD(clause);
COMPARE_SCALAR_FIELD(is_pushed_down);
COMPARE_SCALAR_FIELD(outerjoin_delayed);
Improve RLS planning by marking individual quals with security levels. In an RLS query, we must ensure that security filter quals are evaluated before ordinary query quals, in case the latter contain "leaky" functions that could expose the contents of sensitive rows. The original implementation of RLS planning ensured this by pushing the scan of a secured table into a sub-query that it marked as a security-barrier view. Unfortunately this results in very inefficient plans in many cases, because the sub-query cannot be flattened and gets planned independently of the rest of the query. To fix, drop the use of sub-queries to enforce RLS qual order, and instead mark each qual (RestrictInfo) with a security_level field establishing its priority for evaluation. Quals must be evaluated in security_level order, except that "leakproof" quals can be allowed to go ahead of quals of lower security_level, if it's helpful to do so. This has to be enforced within the ordering of any one list of quals to be evaluated at a table scan node, and we also have to ensure that quals are not chosen for early evaluation (i.e., use as an index qual or TID scan qual) if they're not allowed to go ahead of other quals at the scan node. This is sufficient to fix the problem for RLS quals, since we only support RLS policies on simple tables and thus RLS quals will always exist at the table scan level only. Eventually these qual ordering rules should be enforced for join quals as well, which would permit improving planning for explicit security-barrier views; but that's a task for another patch. Note that FDWs would need to be aware of these rules --- and not, for example, send an insecure qual for remote execution --- but since we do not yet allow RLS policies on foreign tables, the case doesn't arise. This will need to be addressed before we can allow such policies. Patch by me, reviewed by Stephen Frost and Dean Rasheed. Discussion: https://postgr.es/m/8185.1477432701@sss.pgh.pa.us
2017-01-18 18:58:20 +01:00
COMPARE_SCALAR_FIELD(security_level);
COMPARE_BITMAPSET_FIELD(required_relids);
Revise parameterized-path mechanism to fix assorted issues. This patch adjusts the treatment of parameterized paths so that all paths with the same parameterization (same set of required outer rels) for the same relation will have the same rowcount estimate. We cache the rowcount estimates to ensure that property, and hopefully save a few cycles too. Doing this makes it practical for add_path_precheck to operate without a rowcount estimate: it need only assume that paths with different parameterizations never dominate each other, which is close enough to true anyway for coarse filtering, because normally a more-parameterized path should yield fewer rows thanks to having more join clauses to apply. In add_path, we do the full nine yards of comparing rowcount estimates along with everything else, so that we can discard parameterized paths that don't actually have an advantage. This fixes some issues I'd found with add_path rejecting parameterized paths on the grounds that they were more expensive than not-parameterized ones, even though they yielded many fewer rows and hence would be cheaper once subsequent joining was considered. To make the same-rowcounts assumption valid, we have to require that any parameterized path enforce *all* join clauses that could be obtained from the particular set of outer rels, even if not all of them are useful for indexing. This is required at both base scans and joins. It's a good thing anyway since the net impact is that join quals are checked at the lowest practical level in the join tree. Hence, discard the original rather ad-hoc mechanism for choosing parameterization joinquals, and build a better one that has a more principled rule for when clauses can be moved. The original rule was actually buggy anyway for lack of knowledge about which relations are part of an outer join's outer side; getting this right requires adding an outer_relids field to RestrictInfo.
2012-04-19 21:52:46 +02:00
COMPARE_BITMAPSET_FIELD(outer_relids);
COMPARE_BITMAPSET_FIELD(nullable_relids);
2003-08-04 02:43:34 +02:00
/*
* We ignore all the remaining fields, since they may not be set yet, and
* should be derivable from the clause anyway.
*/
return true;
}
static bool
_equalPlaceHolderVar(const PlaceHolderVar *a, const PlaceHolderVar *b)
{
/*
* We intentionally do not compare phexpr. Two PlaceHolderVars with the
* same ID and levelsup should be considered equal even if the contained
* expressions have managed to mutate to different states. This will
* happen during final plan construction when there are nested PHVs, since
* the inner PHV will get replaced by a Param in some copies of the outer
* PHV. Another way in which it can happen is that initplan sublinks
* could get replaced by differently-numbered Params when sublink folding
* is done. (The end result of such a situation would be some
* unreferenced initplans, which is annoying but not really a problem.) On
* the same reasoning, there is no need to examine phrels.
*
* COMPARE_NODE_FIELD(phexpr);
*
* COMPARE_BITMAPSET_FIELD(phrels);
*/
COMPARE_SCALAR_FIELD(phid);
COMPARE_SCALAR_FIELD(phlevelsup);
return true;
}
static bool
_equalSpecialJoinInfo(const SpecialJoinInfo *a, const SpecialJoinInfo *b)
{
COMPARE_BITMAPSET_FIELD(min_lefthand);
COMPARE_BITMAPSET_FIELD(min_righthand);
COMPARE_BITMAPSET_FIELD(syn_lefthand);
COMPARE_BITMAPSET_FIELD(syn_righthand);
COMPARE_SCALAR_FIELD(jointype);
COMPARE_SCALAR_FIELD(lhs_strict);
COMPARE_SCALAR_FIELD(delay_upper_joins);
Improve planner's cost estimation in the presence of semijoins. If we have a semijoin, say SELECT * FROM x WHERE x1 IN (SELECT y1 FROM y) and we're estimating the cost of a parameterized indexscan on x, the number of repetitions of the indexscan should not be taken as the size of y; it'll really only be the number of distinct values of y1, because the only valid plan with y on the outside of a nestloop would require y to be unique-ified before joining it to x. Most of the time this doesn't make that much difference, but sometimes it can lead to drastically underestimating the cost of the indexscan and hence choosing a bad plan, as pointed out by David Kubečka. Fixing this is a bit difficult because parameterized indexscans are costed out quite early in the planning process, before we have the information that would be needed to call estimate_num_groups() and thereby estimate the number of distinct values of the join column(s). However we can move the code that extracts a semijoin RHS's unique-ification columns, so that it's done in initsplan.c rather than on-the-fly in create_unique_path(). That shouldn't make any difference speed-wise and it's really a bit cleaner too. The other bit of information we need is the size of the semijoin RHS, which is easy if it's a single relation (we make those estimates before considering indexscan costs) but problematic if it's a join relation. The solution adopted here is just to use the product of the sizes of the join component rels. That will generally be an overestimate, but since estimate_num_groups() only uses this input as a clamp, an overestimate shouldn't hurt us too badly. In any case we don't allow this new logic to produce a value larger than we would have chosen before, so that at worst an overestimate leaves us no wiser than we were before.
2015-03-12 02:21:00 +01:00
COMPARE_SCALAR_FIELD(semi_can_btree);
COMPARE_SCALAR_FIELD(semi_can_hash);
COMPARE_NODE_FIELD(semi_operators);
COMPARE_NODE_FIELD(semi_rhs_exprs);
return true;
}
static bool
_equalAppendRelInfo(const AppendRelInfo *a, const AppendRelInfo *b)
{
COMPARE_SCALAR_FIELD(parent_relid);
COMPARE_SCALAR_FIELD(child_relid);
COMPARE_SCALAR_FIELD(parent_reltype);
COMPARE_SCALAR_FIELD(child_reltype);
COMPARE_NODE_FIELD(translated_vars);
COMPARE_SCALAR_FIELD(num_child_cols);
COMPARE_POINTER_FIELD(parent_colnos, a->num_child_cols * sizeof(AttrNumber));
COMPARE_SCALAR_FIELD(parent_reloid);
return true;
}
static bool
_equalPlaceHolderInfo(const PlaceHolderInfo *a, const PlaceHolderInfo *b)
{
COMPARE_SCALAR_FIELD(phid);
COMPARE_NODE_FIELD(ph_var); /* should be redundant */
COMPARE_BITMAPSET_FIELD(ph_eval_at);
COMPARE_BITMAPSET_FIELD(ph_lateral);
COMPARE_BITMAPSET_FIELD(ph_needed);
COMPARE_SCALAR_FIELD(ph_width);
return true;
}
/*
* Stuff from extensible.h
*/
static bool
_equalExtensibleNode(const ExtensibleNode *a, const ExtensibleNode *b)
{
const ExtensibleNodeMethods *methods;
COMPARE_STRING_FIELD(extnodename);
/* At this point, we know extnodename is the same for both nodes. */
methods = GetExtensibleNodeMethods(a->extnodename, false);
/* compare the private fields */
if (!methods->nodeEqual(a, b))
return false;
return true;
}
/*
* Stuff from parsenodes.h
*/
static bool
_equalQuery(const Query *a, const Query *b)
{
COMPARE_SCALAR_FIELD(commandType);
COMPARE_SCALAR_FIELD(querySource);
/* we intentionally ignore queryId, since it might not be set */
COMPARE_SCALAR_FIELD(canSetTag);
COMPARE_NODE_FIELD(utilityStmt);
COMPARE_SCALAR_FIELD(resultRelation);
COMPARE_SCALAR_FIELD(hasAggs);
COMPARE_SCALAR_FIELD(hasWindowFuncs);
Improve parser's and planner's handling of set-returning functions. Teach the parser to reject misplaced set-returning functions during parse analysis using p_expr_kind, in much the same way as we do for aggregates and window functions (cf commit eaccfded9). While this isn't complete (it misses nesting-based restrictions), it's much better than the previous error reporting for such cases, and it allows elimination of assorted ad-hoc expression_returns_set() error checks. We could add nesting checks later if it seems important to catch all cases at parse time. There is one case the parser will now throw error for although previous versions allowed it, which is SRFs in the tlist of an UPDATE. That never behaved sensibly (since it's ill-defined which generated row should be used to perform the update) and it's hard to see why it should not be treated as an error. It's a release-note-worthy change though. Also, add a new Query field hasTargetSRFs reporting whether there are any SRFs in the targetlist (including GROUP BY/ORDER BY expressions). The parser can now set that basically for free during parse analysis, and we can use it in a number of places to avoid expression_returns_set searches. (There will be more such checks soon.) In some places, this allows decontorting the logic since it's no longer expensive to check for SRFs in the tlist --- so I made the checks parallel to the handling of hasAggs/hasWindowFuncs wherever it seemed appropriate. catversion bump because adding a Query field changes stored rules. Andres Freund and Tom Lane Discussion: <24639.1473782855@sss.pgh.pa.us>
2016-09-13 19:54:24 +02:00
COMPARE_SCALAR_FIELD(hasTargetSRFs);
COMPARE_SCALAR_FIELD(hasSubLinks);
COMPARE_SCALAR_FIELD(hasDistinctOn);
COMPARE_SCALAR_FIELD(hasRecursive);
COMPARE_SCALAR_FIELD(hasModifyingCTE);
COMPARE_SCALAR_FIELD(hasForUpdate);
Row-Level Security Policies (RLS) Building on the updatable security-barrier views work, add the ability to define policies on tables to limit the set of rows which are returned from a query and which are allowed to be added to a table. Expressions defined by the policy for filtering are added to the security barrier quals of the query, while expressions defined to check records being added to a table are added to the with-check options of the query. New top-level commands are CREATE/ALTER/DROP POLICY and are controlled by the table owner. Row Security is able to be enabled and disabled by the owner on a per-table basis using ALTER TABLE .. ENABLE/DISABLE ROW SECURITY. Per discussion, ROW SECURITY is disabled on tables by default and must be enabled for policies on the table to be used. If no policies exist on a table with ROW SECURITY enabled, a default-deny policy is used and no records will be visible. By default, row security is applied at all times except for the table owner and the superuser. A new GUC, row_security, is added which can be set to ON, OFF, or FORCE. When set to FORCE, row security will be applied even for the table owner and superusers. When set to OFF, row security will be disabled when allowed and an error will be thrown if the user does not have rights to bypass row security. Per discussion, pg_dump sets row_security = OFF by default to ensure that exports and backups will have all data in the table or will error if there are insufficient privileges to bypass row security. A new option has been added to pg_dump, --enable-row-security, to ask pg_dump to export with row security enabled. A new role capability, BYPASSRLS, which can only be set by the superuser, is added to allow other users to be able to bypass row security using row_security = OFF. Many thanks to the various individuals who have helped with the design, particularly Robert Haas for his feedback. Authors include Craig Ringer, KaiGai Kohei, Adam Brightwell, Dean Rasheed, with additional changes and rework by me. Reviewers have included all of the above, Greg Smith, Jeff McCormick, and Robert Haas.
2014-09-19 17:18:35 +02:00
COMPARE_SCALAR_FIELD(hasRowSecurity);
SQL-standard function body This adds support for writing CREATE FUNCTION and CREATE PROCEDURE statements for language SQL with a function body that conforms to the SQL standard and is portable to other implementations. Instead of the PostgreSQL-specific AS $$ string literal $$ syntax, this allows writing out the SQL statements making up the body unquoted, either as a single statement: CREATE FUNCTION add(a integer, b integer) RETURNS integer LANGUAGE SQL RETURN a + b; or as a block CREATE PROCEDURE insert_data(a integer, b integer) LANGUAGE SQL BEGIN ATOMIC INSERT INTO tbl VALUES (a); INSERT INTO tbl VALUES (b); END; The function body is parsed at function definition time and stored as expression nodes in a new pg_proc column prosqlbody. So at run time, no further parsing is required. However, this form does not support polymorphic arguments, because there is no more parse analysis done at call time. Dependencies between the function and the objects it uses are fully tracked. A new RETURN statement is introduced. This can only be used inside function bodies. Internally, it is treated much like a SELECT statement. psql needs some new intelligence to keep track of function body boundaries so that it doesn't send off statements when it sees semicolons that are inside a function body. Tested-by: Jaime Casanova <jcasanov@systemguards.com.ec> Reviewed-by: Julien Rouhaud <rjuju123@gmail.com> Discussion: https://www.postgresql.org/message-id/flat/1c11f1eb-f00c-43b7-799d-2d44132c02d7@2ndquadrant.com
2021-04-07 21:30:08 +02:00
COMPARE_SCALAR_FIELD(isReturn);
COMPARE_NODE_FIELD(cteList);
COMPARE_NODE_FIELD(rtable);
COMPARE_NODE_FIELD(jointree);
COMPARE_NODE_FIELD(targetList);
COMPARE_SCALAR_FIELD(override);
Add support for INSERT ... ON CONFLICT DO NOTHING/UPDATE. The newly added ON CONFLICT clause allows to specify an alternative to raising a unique or exclusion constraint violation error when inserting. ON CONFLICT refers to constraints that can either be specified using a inference clause (by specifying the columns of a unique constraint) or by naming a unique or exclusion constraint. DO NOTHING avoids the constraint violation, without touching the pre-existing row. DO UPDATE SET ... [WHERE ...] updates the pre-existing tuple, and has access to both the tuple proposed for insertion and the existing tuple; the optional WHERE clause can be used to prevent an update from being executed. The UPDATE SET and WHERE clauses have access to the tuple proposed for insertion using the "magic" EXCLUDED alias, and to the pre-existing tuple using the table name or its alias. This feature is often referred to as upsert. This is implemented using a new infrastructure called "speculative insertion". It is an optimistic variant of regular insertion that first does a pre-check for existing tuples and then attempts an insert. If a violating tuple was inserted concurrently, the speculatively inserted tuple is deleted and a new attempt is made. If the pre-check finds a matching tuple the alternative DO NOTHING or DO UPDATE action is taken. If the insertion succeeds without detecting a conflict, the tuple is deemed inserted. To handle the possible ambiguity between the excluded alias and a table named excluded, and for convenience with long relation names, INSERT INTO now can alias its target table. Bumps catversion as stored rules change. Author: Peter Geoghegan, with significant contributions from Heikki Linnakangas and Andres Freund. Testing infrastructure by Jeff Janes. Reviewed-By: Heikki Linnakangas, Andres Freund, Robert Haas, Simon Riggs, Dean Rasheed, Stephen Frost and many others.
2015-05-08 05:31:36 +02:00
COMPARE_NODE_FIELD(onConflict);
COMPARE_NODE_FIELD(returningList);
COMPARE_NODE_FIELD(groupClause);
COMPARE_SCALAR_FIELD(groupDistinct);
Support GROUPING SETS, CUBE and ROLLUP. This SQL standard functionality allows to aggregate data by different GROUP BY clauses at once. Each grouping set returns rows with columns grouped by in other sets set to NULL. This could previously be achieved by doing each grouping as a separate query, conjoined by UNION ALLs. Besides being considerably more concise, grouping sets will in many cases be faster, requiring only one scan over the underlying data. The current implementation of grouping sets only supports using sorting for input. Individual sets that share a sort order are computed in one pass. If there are sets that don't share a sort order, additional sort & aggregation steps are performed. These additional passes are sourced by the previous sort step; thus avoiding repeated scans of the source data. The code is structured in a way that adding support for purely using hash aggregation or a mix of hashing and sorting is possible. Sorting was chosen to be supported first, as it is the most generic method of implementation. Instead of, as in an earlier versions of the patch, representing the chain of sort and aggregation steps as full blown planner and executor nodes, all but the first sort are performed inside the aggregation node itself. This avoids the need to do some unusual gymnastics to handle having to return aggregated and non-aggregated tuples from underlying nodes, as well as having to shut down underlying nodes early to limit memory usage. The optimizer still builds Sort/Agg node to describe each phase, but they're not part of the plan tree, but instead additional data for the aggregation node. They're a convenient and preexisting way to describe aggregation and sorting. The first (and possibly only) sort step is still performed as a separate execution step. That retains similarity with existing group by plans, makes rescans fairly simple, avoids very deep plans (leading to slow explains) and easily allows to avoid the sorting step if the underlying data is sorted by other means. A somewhat ugly side of this patch is having to deal with a grammar ambiguity between the new CUBE keyword and the cube extension/functions named cube (and rollup). To avoid breaking existing deployments of the cube extension it has not been renamed, neither has cube been made a reserved keyword. Instead precedence hacking is used to make GROUP BY cube(..) refer to the CUBE grouping sets feature, and not the function cube(). To actually group by a function cube(), unlikely as that might be, the function name has to be quoted. Needs a catversion bump because stored rules may change. Author: Andrew Gierth and Atri Sharma, with contributions from Andres Freund Reviewed-By: Andres Freund, Noah Misch, Tom Lane, Svenne Krap, Tomas Vondra, Erik Rijkers, Marti Raudsepp, Pavel Stehule Discussion: CAOeZVidmVRe2jU6aMk_5qkxnB7dfmPROzM7Ur8JPW5j8Y5X-Lw@mail.gmail.com
2015-05-16 03:40:59 +02:00
COMPARE_NODE_FIELD(groupingSets);
COMPARE_NODE_FIELD(havingQual);
COMPARE_NODE_FIELD(windowClause);
COMPARE_NODE_FIELD(distinctClause);
COMPARE_NODE_FIELD(sortClause);
COMPARE_NODE_FIELD(limitOffset);
COMPARE_NODE_FIELD(limitCount);
COMPARE_SCALAR_FIELD(limitOption);
COMPARE_NODE_FIELD(rowMarks);
COMPARE_NODE_FIELD(setOperations);
COMPARE_NODE_FIELD(constraintDeps);
COMPARE_NODE_FIELD(withCheckOptions);
Add support for MERGE SQL command MERGE performs actions that modify rows in the target table using a source table or query. MERGE provides a single SQL statement that can conditionally INSERT/UPDATE/DELETE rows -- a task that would otherwise require multiple PL statements. For example, MERGE INTO target AS t USING source AS s ON t.tid = s.sid WHEN MATCHED AND t.balance > s.delta THEN UPDATE SET balance = t.balance - s.delta WHEN MATCHED THEN DELETE WHEN NOT MATCHED AND s.delta > 0 THEN INSERT VALUES (s.sid, s.delta) WHEN NOT MATCHED THEN DO NOTHING; MERGE works with regular tables, partitioned tables and inheritance hierarchies, including column and row security enforcement, as well as support for row and statement triggers and transition tables therein. MERGE is optimized for OLTP and is parameterizable, though also useful for large scale ETL/ELT. MERGE is not intended to be used in preference to existing single SQL commands for INSERT, UPDATE or DELETE since there is some overhead. MERGE can be used from PL/pgSQL. MERGE does not support targetting updatable views or foreign tables, and RETURNING clauses are not allowed either. These limitations are likely fixable with sufficient effort. Rewrite rules are also not supported, but it's not clear that we'd want to support them. Author: Pavan Deolasee <pavan.deolasee@gmail.com> Author: Álvaro Herrera <alvherre@alvh.no-ip.org> Author: Amit Langote <amitlangote09@gmail.com> Author: Simon Riggs <simon.riggs@enterprisedb.com> Reviewed-by: Peter Eisentraut <peter.eisentraut@enterprisedb.com> Reviewed-by: Andres Freund <andres@anarazel.de> (earlier versions) Reviewed-by: Peter Geoghegan <pg@bowt.ie> (earlier versions) Reviewed-by: Robert Haas <robertmhaas@gmail.com> (earlier versions) Reviewed-by: Japin Li <japinli@hotmail.com> Reviewed-by: Justin Pryzby <pryzby@telsasoft.com> Reviewed-by: Tomas Vondra <tomas.vondra@enterprisedb.com> Reviewed-by: Zhihong Yu <zyu@yugabyte.com> Discussion: https://postgr.es/m/CANP8+jKitBSrB7oTgT9CY2i1ObfOt36z0XMraQc+Xrz8QB0nXA@mail.gmail.com Discussion: https://postgr.es/m/CAH2-WzkJdBuxj9PO=2QaO9-3h3xGbQPZ34kJH=HukRekwM-GZg@mail.gmail.com Discussion: https://postgr.es/m/20201231134736.GA25392@alvherre.pgsql
2022-03-28 16:45:58 +02:00
COMPARE_NODE_FIELD(mergeActionList);
COMPARE_SCALAR_FIELD(mergeUseOuterJoin);
Change representation of statement lists, and add statement location info. This patch makes several changes that improve the consistency of representation of lists of statements. It's always been the case that the output of parse analysis is a list of Query nodes, whatever the types of the individual statements in the list. This patch brings similar consistency to the outputs of raw parsing and planning steps: * The output of raw parsing is now always a list of RawStmt nodes; the statement-type-dependent nodes are one level down from that. * The output of pg_plan_queries() is now always a list of PlannedStmt nodes, even for utility statements. In the case of a utility statement, "planning" just consists of wrapping a CMD_UTILITY PlannedStmt around the utility node. This list representation is now used in Portal and CachedPlan plan lists, replacing the former convention of intermixing PlannedStmts with bare utility-statement nodes. Now, every list of statements has a consistent head-node type depending on how far along it is in processing. This allows changing many places that formerly used generic "Node *" pointers to use a more specific pointer type, thus reducing the number of IsA() tests and casts needed, as well as improving code clarity. Also, the post-parse-analysis representation of DECLARE CURSOR is changed so that it looks more like EXPLAIN, PREPARE, etc. That is, the contained SELECT remains a child of the DeclareCursorStmt rather than getting flipped around to be the other way. It's now true for both Query and PlannedStmt that utilityStmt is non-null if and only if commandType is CMD_UTILITY. That allows simplifying a lot of places that were testing both fields. (I think some of those were just defensive programming, but in many places, it was actually necessary to avoid confusing DECLARE CURSOR with SELECT.) Because PlannedStmt carries a canSetTag field, we're also able to get rid of some ad-hoc rules about how to reconstruct canSetTag for a bare utility statement; specifically, the assumption that a utility is canSetTag if and only if it's the only one in its list. While I see no near-term need for relaxing that restriction, it's nice to get rid of the ad-hocery. The API of ProcessUtility() is changed so that what it's passed is the wrapper PlannedStmt not just the bare utility statement. This will affect all users of ProcessUtility_hook, but the changes are pretty trivial; see the affected contrib modules for examples of the minimum change needed. (Most compilers should give pointer-type-mismatch warnings for uncorrected code.) There's also a change in the API of ExplainOneQuery_hook, to pass through cursorOptions instead of expecting hook functions to know what to pick. This is needed because of the DECLARE CURSOR changes, but really should have been done in 9.6; it's unlikely that any extant hook functions know about using CURSOR_OPT_PARALLEL_OK. Finally, teach gram.y to save statement boundary locations in RawStmt nodes, and pass those through to Query and PlannedStmt nodes. This allows more intelligent handling of cases where a source query string contains multiple statements. This patch doesn't actually do anything with the information, but a follow-on patch will. (Passing this information through cleanly is the true motivation for these changes; while I think this is all good cleanup, it's unlikely we'd have bothered without this end goal.) catversion bump because addition of location fields to struct Query affects stored rules. This patch is by me, but it owes a good deal to Fabien Coelho who did a lot of preliminary work on the problem, and also reviewed the patch. Discussion: https://postgr.es/m/alpine.DEB.2.20.1612200926310.29821@lancre
2017-01-14 22:02:35 +01:00
COMPARE_LOCATION_FIELD(stmt_location);
COMPARE_SCALAR_FIELD(stmt_len);
Change representation of statement lists, and add statement location info. This patch makes several changes that improve the consistency of representation of lists of statements. It's always been the case that the output of parse analysis is a list of Query nodes, whatever the types of the individual statements in the list. This patch brings similar consistency to the outputs of raw parsing and planning steps: * The output of raw parsing is now always a list of RawStmt nodes; the statement-type-dependent nodes are one level down from that. * The output of pg_plan_queries() is now always a list of PlannedStmt nodes, even for utility statements. In the case of a utility statement, "planning" just consists of wrapping a CMD_UTILITY PlannedStmt around the utility node. This list representation is now used in Portal and CachedPlan plan lists, replacing the former convention of intermixing PlannedStmts with bare utility-statement nodes. Now, every list of statements has a consistent head-node type depending on how far along it is in processing. This allows changing many places that formerly used generic "Node *" pointers to use a more specific pointer type, thus reducing the number of IsA() tests and casts needed, as well as improving code clarity. Also, the post-parse-analysis representation of DECLARE CURSOR is changed so that it looks more like EXPLAIN, PREPARE, etc. That is, the contained SELECT remains a child of the DeclareCursorStmt rather than getting flipped around to be the other way. It's now true for both Query and PlannedStmt that utilityStmt is non-null if and only if commandType is CMD_UTILITY. That allows simplifying a lot of places that were testing both fields. (I think some of those were just defensive programming, but in many places, it was actually necessary to avoid confusing DECLARE CURSOR with SELECT.) Because PlannedStmt carries a canSetTag field, we're also able to get rid of some ad-hoc rules about how to reconstruct canSetTag for a bare utility statement; specifically, the assumption that a utility is canSetTag if and only if it's the only one in its list. While I see no near-term need for relaxing that restriction, it's nice to get rid of the ad-hocery. The API of ProcessUtility() is changed so that what it's passed is the wrapper PlannedStmt not just the bare utility statement. This will affect all users of ProcessUtility_hook, but the changes are pretty trivial; see the affected contrib modules for examples of the minimum change needed. (Most compilers should give pointer-type-mismatch warnings for uncorrected code.) There's also a change in the API of ExplainOneQuery_hook, to pass through cursorOptions instead of expecting hook functions to know what to pick. This is needed because of the DECLARE CURSOR changes, but really should have been done in 9.6; it's unlikely that any extant hook functions know about using CURSOR_OPT_PARALLEL_OK. Finally, teach gram.y to save statement boundary locations in RawStmt nodes, and pass those through to Query and PlannedStmt nodes. This allows more intelligent handling of cases where a source query string contains multiple statements. This patch doesn't actually do anything with the information, but a follow-on patch will. (Passing this information through cleanly is the true motivation for these changes; while I think this is all good cleanup, it's unlikely we'd have bothered without this end goal.) catversion bump because addition of location fields to struct Query affects stored rules. This patch is by me, but it owes a good deal to Fabien Coelho who did a lot of preliminary work on the problem, and also reviewed the patch. Discussion: https://postgr.es/m/alpine.DEB.2.20.1612200926310.29821@lancre
2017-01-14 22:02:35 +01:00
return true;
}
static bool
_equalRawStmt(const RawStmt *a, const RawStmt *b)
{
COMPARE_NODE_FIELD(stmt);
COMPARE_LOCATION_FIELD(stmt_location);
COMPARE_SCALAR_FIELD(stmt_len);
return true;
}
static bool
_equalInsertStmt(const InsertStmt *a, const InsertStmt *b)
{
COMPARE_NODE_FIELD(relation);
COMPARE_NODE_FIELD(cols);
COMPARE_NODE_FIELD(selectStmt);
Add support for INSERT ... ON CONFLICT DO NOTHING/UPDATE. The newly added ON CONFLICT clause allows to specify an alternative to raising a unique or exclusion constraint violation error when inserting. ON CONFLICT refers to constraints that can either be specified using a inference clause (by specifying the columns of a unique constraint) or by naming a unique or exclusion constraint. DO NOTHING avoids the constraint violation, without touching the pre-existing row. DO UPDATE SET ... [WHERE ...] updates the pre-existing tuple, and has access to both the tuple proposed for insertion and the existing tuple; the optional WHERE clause can be used to prevent an update from being executed. The UPDATE SET and WHERE clauses have access to the tuple proposed for insertion using the "magic" EXCLUDED alias, and to the pre-existing tuple using the table name or its alias. This feature is often referred to as upsert. This is implemented using a new infrastructure called "speculative insertion". It is an optimistic variant of regular insertion that first does a pre-check for existing tuples and then attempts an insert. If a violating tuple was inserted concurrently, the speculatively inserted tuple is deleted and a new attempt is made. If the pre-check finds a matching tuple the alternative DO NOTHING or DO UPDATE action is taken. If the insertion succeeds without detecting a conflict, the tuple is deemed inserted. To handle the possible ambiguity between the excluded alias and a table named excluded, and for convenience with long relation names, INSERT INTO now can alias its target table. Bumps catversion as stored rules change. Author: Peter Geoghegan, with significant contributions from Heikki Linnakangas and Andres Freund. Testing infrastructure by Jeff Janes. Reviewed-By: Heikki Linnakangas, Andres Freund, Robert Haas, Simon Riggs, Dean Rasheed, Stephen Frost and many others.
2015-05-08 05:31:36 +02:00
COMPARE_NODE_FIELD(onConflictClause);
COMPARE_NODE_FIELD(returningList);
COMPARE_NODE_FIELD(withClause);
COMPARE_SCALAR_FIELD(override);
return true;
}
static bool
_equalDeleteStmt(const DeleteStmt *a, const DeleteStmt *b)
{
COMPARE_NODE_FIELD(relation);
COMPARE_NODE_FIELD(usingClause);
COMPARE_NODE_FIELD(whereClause);
COMPARE_NODE_FIELD(returningList);
COMPARE_NODE_FIELD(withClause);
return true;
}
static bool
_equalUpdateStmt(const UpdateStmt *a, const UpdateStmt *b)
{
COMPARE_NODE_FIELD(relation);
COMPARE_NODE_FIELD(targetList);
COMPARE_NODE_FIELD(whereClause);
COMPARE_NODE_FIELD(fromClause);
COMPARE_NODE_FIELD(returningList);
COMPARE_NODE_FIELD(withClause);
return true;
}
Add support for MERGE SQL command MERGE performs actions that modify rows in the target table using a source table or query. MERGE provides a single SQL statement that can conditionally INSERT/UPDATE/DELETE rows -- a task that would otherwise require multiple PL statements. For example, MERGE INTO target AS t USING source AS s ON t.tid = s.sid WHEN MATCHED AND t.balance > s.delta THEN UPDATE SET balance = t.balance - s.delta WHEN MATCHED THEN DELETE WHEN NOT MATCHED AND s.delta > 0 THEN INSERT VALUES (s.sid, s.delta) WHEN NOT MATCHED THEN DO NOTHING; MERGE works with regular tables, partitioned tables and inheritance hierarchies, including column and row security enforcement, as well as support for row and statement triggers and transition tables therein. MERGE is optimized for OLTP and is parameterizable, though also useful for large scale ETL/ELT. MERGE is not intended to be used in preference to existing single SQL commands for INSERT, UPDATE or DELETE since there is some overhead. MERGE can be used from PL/pgSQL. MERGE does not support targetting updatable views or foreign tables, and RETURNING clauses are not allowed either. These limitations are likely fixable with sufficient effort. Rewrite rules are also not supported, but it's not clear that we'd want to support them. Author: Pavan Deolasee <pavan.deolasee@gmail.com> Author: Álvaro Herrera <alvherre@alvh.no-ip.org> Author: Amit Langote <amitlangote09@gmail.com> Author: Simon Riggs <simon.riggs@enterprisedb.com> Reviewed-by: Peter Eisentraut <peter.eisentraut@enterprisedb.com> Reviewed-by: Andres Freund <andres@anarazel.de> (earlier versions) Reviewed-by: Peter Geoghegan <pg@bowt.ie> (earlier versions) Reviewed-by: Robert Haas <robertmhaas@gmail.com> (earlier versions) Reviewed-by: Japin Li <japinli@hotmail.com> Reviewed-by: Justin Pryzby <pryzby@telsasoft.com> Reviewed-by: Tomas Vondra <tomas.vondra@enterprisedb.com> Reviewed-by: Zhihong Yu <zyu@yugabyte.com> Discussion: https://postgr.es/m/CANP8+jKitBSrB7oTgT9CY2i1ObfOt36z0XMraQc+Xrz8QB0nXA@mail.gmail.com Discussion: https://postgr.es/m/CAH2-WzkJdBuxj9PO=2QaO9-3h3xGbQPZ34kJH=HukRekwM-GZg@mail.gmail.com Discussion: https://postgr.es/m/20201231134736.GA25392@alvherre.pgsql
2022-03-28 16:45:58 +02:00
static bool
_equalMergeStmt(const MergeStmt *a, const MergeStmt *b)
{
COMPARE_NODE_FIELD(relation);
COMPARE_NODE_FIELD(sourceRelation);
COMPARE_NODE_FIELD(joinCondition);
COMPARE_NODE_FIELD(mergeWhenClauses);
COMPARE_NODE_FIELD(withClause);
return true;
}
static bool
_equalSelectStmt(const SelectStmt *a, const SelectStmt *b)
{
COMPARE_NODE_FIELD(distinctClause);
COMPARE_NODE_FIELD(intoClause);
COMPARE_NODE_FIELD(targetList);
COMPARE_NODE_FIELD(fromClause);
COMPARE_NODE_FIELD(whereClause);
COMPARE_NODE_FIELD(groupClause);
COMPARE_SCALAR_FIELD(groupDistinct);
COMPARE_NODE_FIELD(havingClause);
COMPARE_NODE_FIELD(windowClause);
COMPARE_NODE_FIELD(valuesLists);
COMPARE_NODE_FIELD(sortClause);
COMPARE_NODE_FIELD(limitOffset);
COMPARE_NODE_FIELD(limitCount);
COMPARE_SCALAR_FIELD(limitOption);
COMPARE_NODE_FIELD(lockingClause);
COMPARE_NODE_FIELD(withClause);
COMPARE_SCALAR_FIELD(op);
COMPARE_SCALAR_FIELD(all);
COMPARE_NODE_FIELD(larg);
COMPARE_NODE_FIELD(rarg);
return true;
}
static bool
_equalSetOperationStmt(const SetOperationStmt *a, const SetOperationStmt *b)
{
COMPARE_SCALAR_FIELD(op);
COMPARE_SCALAR_FIELD(all);
COMPARE_NODE_FIELD(larg);
COMPARE_NODE_FIELD(rarg);
COMPARE_NODE_FIELD(colTypes);
COMPARE_NODE_FIELD(colTypmods);
COMPARE_NODE_FIELD(colCollations);
COMPARE_NODE_FIELD(groupClauses);
return true;
}
SQL-standard function body This adds support for writing CREATE FUNCTION and CREATE PROCEDURE statements for language SQL with a function body that conforms to the SQL standard and is portable to other implementations. Instead of the PostgreSQL-specific AS $$ string literal $$ syntax, this allows writing out the SQL statements making up the body unquoted, either as a single statement: CREATE FUNCTION add(a integer, b integer) RETURNS integer LANGUAGE SQL RETURN a + b; or as a block CREATE PROCEDURE insert_data(a integer, b integer) LANGUAGE SQL BEGIN ATOMIC INSERT INTO tbl VALUES (a); INSERT INTO tbl VALUES (b); END; The function body is parsed at function definition time and stored as expression nodes in a new pg_proc column prosqlbody. So at run time, no further parsing is required. However, this form does not support polymorphic arguments, because there is no more parse analysis done at call time. Dependencies between the function and the objects it uses are fully tracked. A new RETURN statement is introduced. This can only be used inside function bodies. Internally, it is treated much like a SELECT statement. psql needs some new intelligence to keep track of function body boundaries so that it doesn't send off statements when it sees semicolons that are inside a function body. Tested-by: Jaime Casanova <jcasanov@systemguards.com.ec> Reviewed-by: Julien Rouhaud <rjuju123@gmail.com> Discussion: https://www.postgresql.org/message-id/flat/1c11f1eb-f00c-43b7-799d-2d44132c02d7@2ndquadrant.com
2021-04-07 21:30:08 +02:00
static bool
_equalReturnStmt(const ReturnStmt *a, const ReturnStmt *b)
{
COMPARE_NODE_FIELD(returnval);
return true;
}
static bool
_equalPLAssignStmt(const PLAssignStmt *a, const PLAssignStmt *b)
{
COMPARE_STRING_FIELD(name);
COMPARE_NODE_FIELD(indirection);
COMPARE_SCALAR_FIELD(nnames);
COMPARE_NODE_FIELD(val);
COMPARE_LOCATION_FIELD(location);
return true;
}
static bool
_equalAlterTableStmt(const AlterTableStmt *a, const AlterTableStmt *b)
{
COMPARE_NODE_FIELD(relation);
COMPARE_NODE_FIELD(cmds);
COMPARE_SCALAR_FIELD(objtype);
COMPARE_SCALAR_FIELD(missing_ok);
return true;
}
static bool
_equalAlterTableCmd(const AlterTableCmd *a, const AlterTableCmd *b)
{
COMPARE_SCALAR_FIELD(subtype);
COMPARE_STRING_FIELD(name);
COMPARE_SCALAR_FIELD(num);
COMPARE_NODE_FIELD(newowner);
COMPARE_NODE_FIELD(def);
COMPARE_SCALAR_FIELD(behavior);
COMPARE_SCALAR_FIELD(missing_ok);
return true;
}
static bool
_equalAlterCollationStmt(const AlterCollationStmt *a, const AlterCollationStmt *b)
{
COMPARE_NODE_FIELD(collname);
return true;
}
2002-12-06 06:00:34 +01:00
static bool
_equalAlterDomainStmt(const AlterDomainStmt *a, const AlterDomainStmt *b)
2002-12-06 06:00:34 +01:00
{
COMPARE_SCALAR_FIELD(subtype);
COMPARE_NODE_FIELD(typeName);
2002-12-06 06:00:34 +01:00
COMPARE_STRING_FIELD(name);
COMPARE_NODE_FIELD(def);
COMPARE_SCALAR_FIELD(behavior);
COMPARE_SCALAR_FIELD(missing_ok);
2002-12-06 06:00:34 +01:00
return true;
}
static bool
_equalGrantStmt(const GrantStmt *a, const GrantStmt *b)
{
COMPARE_SCALAR_FIELD(is_grant);
COMPARE_SCALAR_FIELD(targtype);
COMPARE_SCALAR_FIELD(objtype);
COMPARE_NODE_FIELD(objects);
COMPARE_NODE_FIELD(privileges);
COMPARE_NODE_FIELD(grantees);
COMPARE_SCALAR_FIELD(grant_option);
COMPARE_NODE_FIELD(grantor);
COMPARE_SCALAR_FIELD(behavior);
return true;
}
static bool
_equalObjectWithArgs(const ObjectWithArgs *a, const ObjectWithArgs *b)
{
COMPARE_NODE_FIELD(objname);
COMPARE_NODE_FIELD(objargs);
Reconsider the handling of procedure OUT parameters. Commit 2453ea142 redefined pg_proc.proargtypes to include the types of OUT parameters, for procedures only. While that had some advantages for implementing the SQL-spec behavior of DROP PROCEDURE, it was pretty disastrous from a number of other perspectives. Notably, since the primary key of pg_proc is name + proargtypes, this made it possible to have multiple procedures with identical names + input arguments and differing output argument types. That would make it impossible to call any one of the procedures by writing just NULL (or "?", or any other data-type-free notation) for the output argument(s). The change also seems likely to cause grave confusion for client applications that examine pg_proc and expect the traditional definition of proargtypes. Hence, revert the definition of proargtypes to what it was, and undo a number of complications that had been added to support that. To support the SQL-spec behavior of DROP PROCEDURE, when there are no argmode markers in the command's parameter list, we perform the lookup both ways (that is, matching against both proargtypes and proallargtypes), succeeding if we get just one unique match. In principle this could result in ambiguous-function failures that would not happen when using only one of the two rules. However, overloading of procedure names is thought to be a pretty rare usage, so this shouldn't cause many problems in practice. Postgres-specific code such as pg_dump can defend against any possibility of such failures by being careful to specify argmodes for all procedure arguments. This also fixes a few other bugs in the area of CALL statements with named parameters, and improves the documentation a little. catversion bump forced because the representation of procedures with OUT arguments changes. Discussion: https://postgr.es/m/3742981.1621533210@sss.pgh.pa.us
2021-06-10 23:11:36 +02:00
COMPARE_NODE_FIELD(objfuncargs);
COMPARE_SCALAR_FIELD(args_unspecified);
return true;
}
static bool
_equalAccessPriv(const AccessPriv *a, const AccessPriv *b)
{
COMPARE_STRING_FIELD(priv_name);
COMPARE_NODE_FIELD(cols);
return true;
}
static bool
_equalGrantRoleStmt(const GrantRoleStmt *a, const GrantRoleStmt *b)
{
COMPARE_NODE_FIELD(granted_roles);
COMPARE_NODE_FIELD(grantee_roles);
COMPARE_SCALAR_FIELD(is_grant);
COMPARE_SCALAR_FIELD(admin_opt);
COMPARE_NODE_FIELD(grantor);
COMPARE_SCALAR_FIELD(behavior);
return true;
}
static bool
_equalAlterDefaultPrivilegesStmt(const AlterDefaultPrivilegesStmt *a, const AlterDefaultPrivilegesStmt *b)
{
COMPARE_NODE_FIELD(options);
COMPARE_NODE_FIELD(action);
return true;
}
static bool
_equalDeclareCursorStmt(const DeclareCursorStmt *a, const DeclareCursorStmt *b)
{
COMPARE_STRING_FIELD(portalname);
COMPARE_SCALAR_FIELD(options);
COMPARE_NODE_FIELD(query);
return true;
}
static bool
_equalClosePortalStmt(const ClosePortalStmt *a, const ClosePortalStmt *b)
{
COMPARE_STRING_FIELD(portalname);
return true;
}
static bool
_equalCallStmt(const CallStmt *a, const CallStmt *b)
{
COMPARE_NODE_FIELD(funccall);
COMPARE_NODE_FIELD(funcexpr);
Reconsider the handling of procedure OUT parameters. Commit 2453ea142 redefined pg_proc.proargtypes to include the types of OUT parameters, for procedures only. While that had some advantages for implementing the SQL-spec behavior of DROP PROCEDURE, it was pretty disastrous from a number of other perspectives. Notably, since the primary key of pg_proc is name + proargtypes, this made it possible to have multiple procedures with identical names + input arguments and differing output argument types. That would make it impossible to call any one of the procedures by writing just NULL (or "?", or any other data-type-free notation) for the output argument(s). The change also seems likely to cause grave confusion for client applications that examine pg_proc and expect the traditional definition of proargtypes. Hence, revert the definition of proargtypes to what it was, and undo a number of complications that had been added to support that. To support the SQL-spec behavior of DROP PROCEDURE, when there are no argmode markers in the command's parameter list, we perform the lookup both ways (that is, matching against both proargtypes and proallargtypes), succeeding if we get just one unique match. In principle this could result in ambiguous-function failures that would not happen when using only one of the two rules. However, overloading of procedure names is thought to be a pretty rare usage, so this shouldn't cause many problems in practice. Postgres-specific code such as pg_dump can defend against any possibility of such failures by being careful to specify argmodes for all procedure arguments. This also fixes a few other bugs in the area of CALL statements with named parameters, and improves the documentation a little. catversion bump forced because the representation of procedures with OUT arguments changes. Discussion: https://postgr.es/m/3742981.1621533210@sss.pgh.pa.us
2021-06-10 23:11:36 +02:00
COMPARE_NODE_FIELD(outargs);
return true;
}
static bool
_equalClusterStmt(const ClusterStmt *a, const ClusterStmt *b)
{
COMPARE_NODE_FIELD(relation);
COMPARE_STRING_FIELD(indexname);
COMPARE_NODE_FIELD(params);
return true;
}
static bool
_equalCopyStmt(const CopyStmt *a, const CopyStmt *b)
{
COMPARE_NODE_FIELD(relation);
COMPARE_NODE_FIELD(query);
COMPARE_NODE_FIELD(attlist);
COMPARE_SCALAR_FIELD(is_from);
COMPARE_SCALAR_FIELD(is_program);
COMPARE_STRING_FIELD(filename);
COMPARE_NODE_FIELD(options);
COMPARE_NODE_FIELD(whereClause);
return true;
}
static bool
_equalCreateStmt(const CreateStmt *a, const CreateStmt *b)
{
COMPARE_NODE_FIELD(relation);
COMPARE_NODE_FIELD(tableElts);
COMPARE_NODE_FIELD(inhRelations);
Implement table partitioning. Table partitioning is like table inheritance and reuses much of the existing infrastructure, but there are some important differences. The parent is called a partitioned table and is always empty; it may not have indexes or non-inherited constraints, since those make no sense for a relation with no data of its own. The children are called partitions and contain all of the actual data. Each partition has an implicit partitioning constraint. Multiple inheritance is not allowed, and partitioning and inheritance can't be mixed. Partitions can't have extra columns and may not allow nulls unless the parent does. Tuples inserted into the parent are automatically routed to the correct partition, so tuple-routing ON INSERT triggers are not needed. Tuple routing isn't yet supported for partitions which are foreign tables, and it doesn't handle updates that cross partition boundaries. Currently, tables can be range-partitioned or list-partitioned. List partitioning is limited to a single column, but range partitioning can involve multiple columns. A partitioning "column" can be an expression. Because table partitioning is less general than table inheritance, it is hoped that it will be easier to reason about properties of partitions, and therefore that this will serve as a better foundation for a variety of possible optimizations, including query planner optimizations. The tuple routing based which this patch does based on the implicit partitioning constraints is an example of this, but it seems likely that many other useful optimizations are also possible. Amit Langote, reviewed and tested by Robert Haas, Ashutosh Bapat, Amit Kapila, Rajkumar Raghuwanshi, Corey Huinker, Jaime Casanova, Rushabh Lathia, Erik Rijkers, among others. Minor revisions by me.
2016-12-07 19:17:43 +01:00
COMPARE_NODE_FIELD(partbound);
COMPARE_NODE_FIELD(partspec);
COMPARE_NODE_FIELD(ofTypename);
COMPARE_NODE_FIELD(constraints);
COMPARE_NODE_FIELD(options);
COMPARE_SCALAR_FIELD(oncommit);
COMPARE_STRING_FIELD(tablespacename);
COMPARE_STRING_FIELD(accessMethod);
COMPARE_SCALAR_FIELD(if_not_exists);
return true;
}
static bool
_equalTableLikeClause(const TableLikeClause *a, const TableLikeClause *b)
{
COMPARE_NODE_FIELD(relation);
COMPARE_SCALAR_FIELD(options);
COMPARE_SCALAR_FIELD(relationOid);
return true;
}
static bool
_equalDefineStmt(const DefineStmt *a, const DefineStmt *b)
{
COMPARE_SCALAR_FIELD(kind);
COMPARE_SCALAR_FIELD(oldstyle);
COMPARE_NODE_FIELD(defnames);
COMPARE_NODE_FIELD(args);
COMPARE_NODE_FIELD(definition);
COMPARE_SCALAR_FIELD(if_not_exists);
COMPARE_SCALAR_FIELD(replace);
return true;
}
static bool
_equalDropStmt(const DropStmt *a, const DropStmt *b)
{
COMPARE_NODE_FIELD(objects);
COMPARE_SCALAR_FIELD(removeType);
COMPARE_SCALAR_FIELD(behavior);
2005-11-21 00:24:12 +01:00
COMPARE_SCALAR_FIELD(missing_ok);
COMPARE_SCALAR_FIELD(concurrent);
return true;
}
static bool
_equalTruncateStmt(const TruncateStmt *a, const TruncateStmt *b)
{
COMPARE_NODE_FIELD(relations);
COMPARE_SCALAR_FIELD(restart_seqs);
COMPARE_SCALAR_FIELD(behavior);
return true;
}
static bool
_equalCommentStmt(const CommentStmt *a, const CommentStmt *b)
{
COMPARE_SCALAR_FIELD(objtype);
COMPARE_NODE_FIELD(object);
COMPARE_STRING_FIELD(comment);
return true;
}
static bool
_equalSecLabelStmt(const SecLabelStmt *a, const SecLabelStmt *b)
{
COMPARE_SCALAR_FIELD(objtype);
COMPARE_NODE_FIELD(object);
COMPARE_STRING_FIELD(provider);
COMPARE_STRING_FIELD(label);
return true;
}
static bool
_equalFetchStmt(const FetchStmt *a, const FetchStmt *b)
{
COMPARE_SCALAR_FIELD(direction);
COMPARE_SCALAR_FIELD(howMany);
COMPARE_STRING_FIELD(portalname);
COMPARE_SCALAR_FIELD(ismove);
return true;
}
static bool
_equalIndexStmt(const IndexStmt *a, const IndexStmt *b)
{
COMPARE_STRING_FIELD(idxname);
COMPARE_NODE_FIELD(relation);
COMPARE_STRING_FIELD(accessMethod);
COMPARE_STRING_FIELD(tableSpace);
COMPARE_NODE_FIELD(indexParams);
COMPARE_NODE_FIELD(indexIncludingParams);
COMPARE_NODE_FIELD(options);
COMPARE_NODE_FIELD(whereClause);
COMPARE_NODE_FIELD(excludeOpNames);
COMPARE_STRING_FIELD(idxcomment);
COMPARE_SCALAR_FIELD(indexOid);
Change internal RelFileNode references to RelFileNumber or RelFileLocator. We have been using the term RelFileNode to refer to either (1) the integer that is used to name the sequence of files for a certain relation within the directory set aside for that tablespace/database combination; or (2) that value plus the OIDs of the tablespace and database; or occasionally (3) the whole series of files created for a relation based on those values. Using the same name for more than one thing is confusing. Replace RelFileNode with RelFileNumber when we're talking about just the single number, i.e. (1) from above, and with RelFileLocator when we're talking about all the things that are needed to locate a relation's files on disk, i.e. (2) from above. In the places where we refer to (3) as a relfilenode, instead refer to "relation storage". Since there is a ton of SQL code in the world that knows about pg_class.relfilenode, don't change the name of that column, or of other SQL-facing things that derive their name from it. On the other hand, do adjust closely-related internal terminology. For example, the structure member names dbNode and spcNode appear to be derived from the fact that the structure itself was called RelFileNode, so change those to dbOid and spcOid. Likewise, various variables with names like rnode and relnode get renamed appropriately, according to how they're being used in context. Hopefully, this is clearer than before. It is also preparation for future patches that intend to widen the relfilenumber fields from its current width of 32 bits. Variables that store a relfilenumber are now declared as type RelFileNumber rather than type Oid; right now, these are the same, but that can now more easily be changed. Dilip Kumar, per an idea from me. Reviewed also by Andres Freund. I fixed some whitespace issues, changed a couple of words in a comment, and made one other minor correction. Discussion: http://postgr.es/m/CA+TgmoamOtXbVAQf9hWFzonUo6bhhjS6toZQd7HZ-pmojtAmag@mail.gmail.com Discussion: http://postgr.es/m/CA+Tgmobp7+7kmi4gkq7Y+4AM9fTvL+O1oQ4-5gFTT+6Ng-dQ=g@mail.gmail.com Discussion: http://postgr.es/m/CAFiTN-vTe79M8uDH1yprOU64MNFE+R3ODRuA+JWf27JbhY4hJw@mail.gmail.com
2022-07-06 17:39:09 +02:00
COMPARE_SCALAR_FIELD(oldNumber);
Skip WAL for new relfilenodes, under wal_level=minimal. Until now, only selected bulk operations (e.g. COPY) did this. If a given relfilenode received both a WAL-skipping COPY and a WAL-logged operation (e.g. INSERT), recovery could lose tuples from the COPY. See src/backend/access/transam/README section "Skipping WAL for New RelFileNode" for the new coding rules. Maintainers of table access methods should examine that section. To maintain data durability, just before commit, we choose between an fsync of the relfilenode and copying its contents to WAL. A new GUC, wal_skip_threshold, guides that choice. If this change slows a workload that creates small, permanent relfilenodes under wal_level=minimal, try adjusting wal_skip_threshold. Users setting a timeout on COMMIT may need to adjust that timeout, and log_min_duration_statement analysis will reflect time consumption moving to COMMIT from commands like COPY. Internally, this requires a reliable determination of whether RollbackAndReleaseCurrentSubTransaction() would unlink a relation's current relfilenode. Introduce rd_firstRelfilenodeSubid. Amend the specification of rd_createSubid such that the field is zero when a new rel has an old rd_node. Make relcache.c retain entries for certain dropped relations until end of transaction. Bump XLOG_PAGE_MAGIC, since this introduces XLOG_GIST_ASSIGN_LSN. Future servers accept older WAL, so this bump is discretionary. Kyotaro Horiguchi, reviewed (in earlier, similar versions) by Robert Haas. Heikki Linnakangas and Michael Paquier implemented earlier designs that materially clarified the problem. Reviewed, in earlier designs, by Andrew Dunstan, Andres Freund, Alvaro Herrera, Tom Lane, Fujii Masao, and Simon Riggs. Reported by Martijn van Oosterhout. Discussion: https://postgr.es/m/20150702220524.GA9392@svana.org
2020-04-04 21:25:34 +02:00
COMPARE_SCALAR_FIELD(oldCreateSubid);
Change internal RelFileNode references to RelFileNumber or RelFileLocator. We have been using the term RelFileNode to refer to either (1) the integer that is used to name the sequence of files for a certain relation within the directory set aside for that tablespace/database combination; or (2) that value plus the OIDs of the tablespace and database; or occasionally (3) the whole series of files created for a relation based on those values. Using the same name for more than one thing is confusing. Replace RelFileNode with RelFileNumber when we're talking about just the single number, i.e. (1) from above, and with RelFileLocator when we're talking about all the things that are needed to locate a relation's files on disk, i.e. (2) from above. In the places where we refer to (3) as a relfilenode, instead refer to "relation storage". Since there is a ton of SQL code in the world that knows about pg_class.relfilenode, don't change the name of that column, or of other SQL-facing things that derive their name from it. On the other hand, do adjust closely-related internal terminology. For example, the structure member names dbNode and spcNode appear to be derived from the fact that the structure itself was called RelFileNode, so change those to dbOid and spcOid. Likewise, various variables with names like rnode and relnode get renamed appropriately, according to how they're being used in context. Hopefully, this is clearer than before. It is also preparation for future patches that intend to widen the relfilenumber fields from its current width of 32 bits. Variables that store a relfilenumber are now declared as type RelFileNumber rather than type Oid; right now, these are the same, but that can now more easily be changed. Dilip Kumar, per an idea from me. Reviewed also by Andres Freund. I fixed some whitespace issues, changed a couple of words in a comment, and made one other minor correction. Discussion: http://postgr.es/m/CA+TgmoamOtXbVAQf9hWFzonUo6bhhjS6toZQd7HZ-pmojtAmag@mail.gmail.com Discussion: http://postgr.es/m/CA+Tgmobp7+7kmi4gkq7Y+4AM9fTvL+O1oQ4-5gFTT+6Ng-dQ=g@mail.gmail.com Discussion: http://postgr.es/m/CAFiTN-vTe79M8uDH1yprOU64MNFE+R3ODRuA+JWf27JbhY4hJw@mail.gmail.com
2022-07-06 17:39:09 +02:00
COMPARE_SCALAR_FIELD(oldFirstRelfilelocatorSubid);
COMPARE_SCALAR_FIELD(unique);
COMPARE_SCALAR_FIELD(nulls_not_distinct);
COMPARE_SCALAR_FIELD(primary);
COMPARE_SCALAR_FIELD(isconstraint);
COMPARE_SCALAR_FIELD(deferrable);
COMPARE_SCALAR_FIELD(initdeferred);
Get rid of multiple applications of transformExpr() to the same tree. transformExpr() has for many years had provisions to do nothing when applied to an already-transformed expression tree. However, this was always ugly and of dubious reliability, so we'd be much better off without it. The primary historical reason for it was that gram.y sometimes returned multiple links to the same subexpression, which is no longer true as of my BETWEEN fixes. We'd also grown some lazy hacks in CREATE TABLE LIKE (failing to distinguish between raw and already-transformed index specifications) and one or two other places. This patch removes the need for and support for re-transforming already transformed expressions. The index case is dealt with by adding a flag to struct IndexStmt to indicate that it's already been transformed; which has some benefit anyway in that tablecmds.c can now Assert that transformation has happened rather than just assuming. The other main reason was some rather sloppy code for array type coercion, which can be fixed (and its performance improved too) by refactoring. I did leave transformJoinUsingClause() still constructing expressions containing untransformed operator nodes being applied to Vars, so that transformExpr() still has to allow Var inputs. But that's a much narrower, and safer, special case than before, since Vars will never appear in a raw parse tree, and they don't have any substructure to worry about. In passing fix some oversights in the patch that added CREATE INDEX IF NOT EXISTS (missing processing of IndexStmt.if_not_exists). These appear relatively harmless, but still sloppy coding practice.
2015-02-22 19:59:09 +01:00
COMPARE_SCALAR_FIELD(transformed);
COMPARE_SCALAR_FIELD(concurrent);
COMPARE_SCALAR_FIELD(if_not_exists);
COMPARE_SCALAR_FIELD(reset_default_tblspc);
return true;
}
Implement multivariate n-distinct coefficients Add support for explicitly declared statistic objects (CREATE STATISTICS), allowing collection of statistics on more complex combinations that individual table columns. Companion commands DROP STATISTICS and ALTER STATISTICS ... OWNER TO / SET SCHEMA / RENAME are added too. All this DDL has been designed so that more statistic types can be added later on, such as multivariate most-common-values and multivariate histograms between columns of a single table, leaving room for permitting columns on multiple tables, too, as well as expressions. This commit only adds support for collection of n-distinct coefficient on user-specified sets of columns in a single table. This is useful to estimate number of distinct groups in GROUP BY and DISTINCT clauses; estimation errors there can cause over-allocation of memory in hashed aggregates, for instance, so it's a worthwhile problem to solve. A new special pseudo-type pg_ndistinct is used. (num-distinct estimation was deemed sufficiently useful by itself that this is worthwhile even if no further statistic types are added immediately; so much so that another version of essentially the same functionality was submitted by Kyotaro Horiguchi: https://postgr.es/m/20150828.173334.114731693.horiguchi.kyotaro@lab.ntt.co.jp though this commit does not use that code.) Author: Tomas Vondra. Some code rework by Álvaro. Reviewed-by: Dean Rasheed, David Rowley, Kyotaro Horiguchi, Jeff Janes, Ideriha Takeshi Discussion: https://postgr.es/m/543AFA15.4080608@fuzzy.cz https://postgr.es/m/20170320190220.ixlaueanxegqd5gr@alvherre.pgsql
2017-03-24 18:06:10 +01:00
static bool
_equalCreateStatsStmt(const CreateStatsStmt *a, const CreateStatsStmt *b)
{
COMPARE_NODE_FIELD(defnames);
COMPARE_NODE_FIELD(stat_types);
COMPARE_NODE_FIELD(exprs);
COMPARE_NODE_FIELD(relations);
COMPARE_STRING_FIELD(stxcomment);
COMPARE_SCALAR_FIELD(transformed);
Implement multivariate n-distinct coefficients Add support for explicitly declared statistic objects (CREATE STATISTICS), allowing collection of statistics on more complex combinations that individual table columns. Companion commands DROP STATISTICS and ALTER STATISTICS ... OWNER TO / SET SCHEMA / RENAME are added too. All this DDL has been designed so that more statistic types can be added later on, such as multivariate most-common-values and multivariate histograms between columns of a single table, leaving room for permitting columns on multiple tables, too, as well as expressions. This commit only adds support for collection of n-distinct coefficient on user-specified sets of columns in a single table. This is useful to estimate number of distinct groups in GROUP BY and DISTINCT clauses; estimation errors there can cause over-allocation of memory in hashed aggregates, for instance, so it's a worthwhile problem to solve. A new special pseudo-type pg_ndistinct is used. (num-distinct estimation was deemed sufficiently useful by itself that this is worthwhile even if no further statistic types are added immediately; so much so that another version of essentially the same functionality was submitted by Kyotaro Horiguchi: https://postgr.es/m/20150828.173334.114731693.horiguchi.kyotaro@lab.ntt.co.jp though this commit does not use that code.) Author: Tomas Vondra. Some code rework by Álvaro. Reviewed-by: Dean Rasheed, David Rowley, Kyotaro Horiguchi, Jeff Janes, Ideriha Takeshi Discussion: https://postgr.es/m/543AFA15.4080608@fuzzy.cz https://postgr.es/m/20170320190220.ixlaueanxegqd5gr@alvherre.pgsql
2017-03-24 18:06:10 +01:00
COMPARE_SCALAR_FIELD(if_not_exists);
return true;
}
static bool
_equalAlterStatsStmt(const AlterStatsStmt *a, const AlterStatsStmt *b)
{
COMPARE_NODE_FIELD(defnames);
COMPARE_SCALAR_FIELD(stxstattarget);
COMPARE_SCALAR_FIELD(missing_ok);
return true;
}
static bool
_equalCreateFunctionStmt(const CreateFunctionStmt *a, const CreateFunctionStmt *b)
{
COMPARE_SCALAR_FIELD(is_procedure);
COMPARE_SCALAR_FIELD(replace);
COMPARE_NODE_FIELD(funcname);
COMPARE_NODE_FIELD(parameters);
COMPARE_NODE_FIELD(returnType);
COMPARE_NODE_FIELD(options);
SQL-standard function body This adds support for writing CREATE FUNCTION and CREATE PROCEDURE statements for language SQL with a function body that conforms to the SQL standard and is portable to other implementations. Instead of the PostgreSQL-specific AS $$ string literal $$ syntax, this allows writing out the SQL statements making up the body unquoted, either as a single statement: CREATE FUNCTION add(a integer, b integer) RETURNS integer LANGUAGE SQL RETURN a + b; or as a block CREATE PROCEDURE insert_data(a integer, b integer) LANGUAGE SQL BEGIN ATOMIC INSERT INTO tbl VALUES (a); INSERT INTO tbl VALUES (b); END; The function body is parsed at function definition time and stored as expression nodes in a new pg_proc column prosqlbody. So at run time, no further parsing is required. However, this form does not support polymorphic arguments, because there is no more parse analysis done at call time. Dependencies between the function and the objects it uses are fully tracked. A new RETURN statement is introduced. This can only be used inside function bodies. Internally, it is treated much like a SELECT statement. psql needs some new intelligence to keep track of function body boundaries so that it doesn't send off statements when it sees semicolons that are inside a function body. Tested-by: Jaime Casanova <jcasanov@systemguards.com.ec> Reviewed-by: Julien Rouhaud <rjuju123@gmail.com> Discussion: https://www.postgresql.org/message-id/flat/1c11f1eb-f00c-43b7-799d-2d44132c02d7@2ndquadrant.com
2021-04-07 21:30:08 +02:00
COMPARE_NODE_FIELD(sql_body);
return true;
}
static bool
_equalFunctionParameter(const FunctionParameter *a, const FunctionParameter *b)
{
COMPARE_STRING_FIELD(name);
COMPARE_NODE_FIELD(argType);
COMPARE_SCALAR_FIELD(mode);
COMPARE_NODE_FIELD(defexpr);
return true;
}
static bool
_equalAlterFunctionStmt(const AlterFunctionStmt *a, const AlterFunctionStmt *b)
{
COMPARE_SCALAR_FIELD(objtype);
COMPARE_NODE_FIELD(func);
COMPARE_NODE_FIELD(actions);
return true;
}
static bool
_equalDoStmt(const DoStmt *a, const DoStmt *b)
{
COMPARE_NODE_FIELD(args);
return true;
}
static bool
_equalRenameStmt(const RenameStmt *a, const RenameStmt *b)
{
COMPARE_SCALAR_FIELD(renameType);
COMPARE_SCALAR_FIELD(relationType);
COMPARE_NODE_FIELD(relation);
2003-06-27 16:45:32 +02:00
COMPARE_NODE_FIELD(object);
COMPARE_STRING_FIELD(subname);
COMPARE_STRING_FIELD(newname);
COMPARE_SCALAR_FIELD(behavior);
COMPARE_SCALAR_FIELD(missing_ok);
return true;
}
static bool
_equalAlterObjectDependsStmt(const AlterObjectDependsStmt *a, const AlterObjectDependsStmt *b)
{
COMPARE_SCALAR_FIELD(objectType);
COMPARE_NODE_FIELD(relation);
COMPARE_NODE_FIELD(object);
COMPARE_NODE_FIELD(extname);
COMPARE_SCALAR_FIELD(remove);
return true;
}
static bool
_equalAlterObjectSchemaStmt(const AlterObjectSchemaStmt *a, const AlterObjectSchemaStmt *b)
{
COMPARE_SCALAR_FIELD(objectType);
COMPARE_NODE_FIELD(relation);
COMPARE_NODE_FIELD(object);
COMPARE_STRING_FIELD(newschema);
COMPARE_SCALAR_FIELD(missing_ok);
return true;
}
static bool
_equalAlterOwnerStmt(const AlterOwnerStmt *a, const AlterOwnerStmt *b)
{
COMPARE_SCALAR_FIELD(objectType);
COMPARE_NODE_FIELD(relation);
COMPARE_NODE_FIELD(object);
COMPARE_NODE_FIELD(newowner);
return true;
}
static bool
_equalAlterOperatorStmt(const AlterOperatorStmt *a, const AlterOperatorStmt *b)
{
COMPARE_NODE_FIELD(opername);
COMPARE_NODE_FIELD(options);
return true;
}
static bool
_equalAlterTypeStmt(const AlterTypeStmt *a, const AlterTypeStmt *b)
{
COMPARE_NODE_FIELD(typeName);
COMPARE_NODE_FIELD(options);
return true;
}
static bool
_equalRuleStmt(const RuleStmt *a, const RuleStmt *b)
{
COMPARE_NODE_FIELD(relation);
COMPARE_STRING_FIELD(rulename);
COMPARE_NODE_FIELD(whereClause);
COMPARE_SCALAR_FIELD(event);
COMPARE_SCALAR_FIELD(instead);
COMPARE_NODE_FIELD(actions);
COMPARE_SCALAR_FIELD(replace);
return true;
}
static bool
_equalNotifyStmt(const NotifyStmt *a, const NotifyStmt *b)
{
COMPARE_STRING_FIELD(conditionname);
COMPARE_STRING_FIELD(payload);
return true;
}
static bool
_equalListenStmt(const ListenStmt *a, const ListenStmt *b)
{
COMPARE_STRING_FIELD(conditionname);
return true;
}
static bool
_equalUnlistenStmt(const UnlistenStmt *a, const UnlistenStmt *b)
{
COMPARE_STRING_FIELD(conditionname);
return true;
}
static bool
_equalTransactionStmt(const TransactionStmt *a, const TransactionStmt *b)
{
COMPARE_SCALAR_FIELD(kind);
COMPARE_NODE_FIELD(options);
COMPARE_STRING_FIELD(savepoint_name);
COMPARE_STRING_FIELD(gid);
COMPARE_SCALAR_FIELD(chain);
return true;
}
static bool
_equalCompositeTypeStmt(const CompositeTypeStmt *a, const CompositeTypeStmt *b)
{
COMPARE_NODE_FIELD(typevar);
COMPARE_NODE_FIELD(coldeflist);
return true;
}
static bool
_equalCreateEnumStmt(const CreateEnumStmt *a, const CreateEnumStmt *b)
{
COMPARE_NODE_FIELD(typeName);
COMPARE_NODE_FIELD(vals);
return true;
}
static bool
_equalCreateRangeStmt(const CreateRangeStmt *a, const CreateRangeStmt *b)
{
COMPARE_NODE_FIELD(typeName);
COMPARE_NODE_FIELD(params);
return true;
}
static bool
_equalAlterEnumStmt(const AlterEnumStmt *a, const AlterEnumStmt *b)
{
COMPARE_NODE_FIELD(typeName);
COMPARE_STRING_FIELD(oldVal);
COMPARE_STRING_FIELD(newVal);
COMPARE_STRING_FIELD(newValNeighbor);
COMPARE_SCALAR_FIELD(newValIsAfter);
COMPARE_SCALAR_FIELD(skipIfNewValExists);
return true;
}
static bool
_equalViewStmt(const ViewStmt *a, const ViewStmt *b)
{
COMPARE_NODE_FIELD(view);
COMPARE_NODE_FIELD(aliases);
COMPARE_NODE_FIELD(query);
COMPARE_SCALAR_FIELD(replace);
COMPARE_NODE_FIELD(options);
COMPARE_SCALAR_FIELD(withCheckOption);
return true;
}
static bool
_equalLoadStmt(const LoadStmt *a, const LoadStmt *b)
{
COMPARE_STRING_FIELD(filename);
return true;
}
static bool
_equalCreateDomainStmt(const CreateDomainStmt *a, const CreateDomainStmt *b)
{
COMPARE_NODE_FIELD(domainname);
COMPARE_NODE_FIELD(typeName);
Remove collation information from TypeName, where it does not belong. The initial collations patch treated a COLLATE spec as part of a TypeName, following what can only be described as brain fade on the part of the SQL committee. It's a lot more reasonable to treat COLLATE as a syntactically separate object, so that it can be added in only the productions where it actually belongs, rather than needing to reject it in a boatload of places where it doesn't belong (something the original patch mostly failed to do). In addition this change lets us meet the spec's requirement to allow COLLATE anywhere in the clauses of a ColumnDef, and it avoids unfriendly behavior for constructs such as "foo::type COLLATE collation". To do this, pull collation information out of TypeName and put it in ColumnDef instead, thus reverting most of the collation-related changes in parse_type.c's API. I made one additional structural change, which was to use a ColumnDef as an intermediate node in AT_AlterColumnType AlterTableCmd nodes. This provides enough room to get rid of the "transform" wart in AlterTableCmd too, since the ColumnDef can carry the USING expression easily enough. Also fix some other minor bugs that have crept in in the same areas, like failure to copy recently-added fields of ColumnDef in copyfuncs.c. While at it, document the formerly secret ability to specify a collation in ALTER TABLE ALTER COLUMN TYPE, ALTER TYPE ADD ATTRIBUTE, and ALTER TYPE ALTER ATTRIBUTE TYPE; and correct some misstatements about what the default collation selection will be when COLLATE is omitted. BTW, the three-parameter form of format_type() should go away too, since it just contributes to the confusion in this area; but I'll do that in a separate patch.
2011-03-10 04:38:52 +01:00
COMPARE_NODE_FIELD(collClause);
COMPARE_NODE_FIELD(constraints);
return true;
}
static bool
_equalCreateOpClassStmt(const CreateOpClassStmt *a, const CreateOpClassStmt *b)
{
COMPARE_NODE_FIELD(opclassname);
COMPARE_NODE_FIELD(opfamilyname);
COMPARE_STRING_FIELD(amname);
COMPARE_NODE_FIELD(datatype);
COMPARE_NODE_FIELD(items);
COMPARE_SCALAR_FIELD(isDefault);
return true;
}
static bool
_equalCreateOpClassItem(const CreateOpClassItem *a, const CreateOpClassItem *b)
{
COMPARE_SCALAR_FIELD(itemtype);
COMPARE_NODE_FIELD(name);
COMPARE_SCALAR_FIELD(number);
COMPARE_NODE_FIELD(order_family);
COMPARE_NODE_FIELD(class_args);
COMPARE_NODE_FIELD(storedtype);
return true;
}
static bool
_equalCreateOpFamilyStmt(const CreateOpFamilyStmt *a, const CreateOpFamilyStmt *b)
{
COMPARE_NODE_FIELD(opfamilyname);
COMPARE_STRING_FIELD(amname);
return true;
}
static bool
_equalAlterOpFamilyStmt(const AlterOpFamilyStmt *a, const AlterOpFamilyStmt *b)
{
COMPARE_NODE_FIELD(opfamilyname);
COMPARE_STRING_FIELD(amname);
COMPARE_SCALAR_FIELD(isDrop);
COMPARE_NODE_FIELD(items);
return true;
}
static bool
_equalCreatedbStmt(const CreatedbStmt *a, const CreatedbStmt *b)
{
COMPARE_STRING_FIELD(dbname);
COMPARE_NODE_FIELD(options);
return true;
}
static bool
_equalAlterDatabaseStmt(const AlterDatabaseStmt *a, const AlterDatabaseStmt *b)
{
COMPARE_STRING_FIELD(dbname);
COMPARE_NODE_FIELD(options);
return true;
}
static bool
_equalAlterDatabaseRefreshCollStmt(const AlterDatabaseRefreshCollStmt *a, const AlterDatabaseRefreshCollStmt *b)
{
COMPARE_STRING_FIELD(dbname);
return true;
}
static bool
_equalAlterDatabaseSetStmt(const AlterDatabaseSetStmt *a, const AlterDatabaseSetStmt *b)
{
COMPARE_STRING_FIELD(dbname);
COMPARE_NODE_FIELD(setstmt);
return true;
}
static bool
_equalDropdbStmt(const DropdbStmt *a, const DropdbStmt *b)
{
COMPARE_STRING_FIELD(dbname);
2005-11-22 16:24:18 +01:00
COMPARE_SCALAR_FIELD(missing_ok);
COMPARE_NODE_FIELD(options);
return true;
}
static bool
_equalVacuumStmt(const VacuumStmt *a, const VacuumStmt *b)
{
COMPARE_NODE_FIELD(options);
COMPARE_NODE_FIELD(rels);
COMPARE_SCALAR_FIELD(is_vacuumcmd);
return true;
}
static bool
_equalVacuumRelation(const VacuumRelation *a, const VacuumRelation *b)
{
COMPARE_NODE_FIELD(relation);
COMPARE_SCALAR_FIELD(oid);
COMPARE_NODE_FIELD(va_cols);
return true;
}
static bool
_equalExplainStmt(const ExplainStmt *a, const ExplainStmt *b)
{
COMPARE_NODE_FIELD(query);
COMPARE_NODE_FIELD(options);
return true;
}
static bool
_equalCreateTableAsStmt(const CreateTableAsStmt *a, const CreateTableAsStmt *b)
{
COMPARE_NODE_FIELD(query);
COMPARE_NODE_FIELD(into);
COMPARE_SCALAR_FIELD(objtype);
COMPARE_SCALAR_FIELD(is_select_into);
COMPARE_SCALAR_FIELD(if_not_exists);
return true;
}
static bool
_equalRefreshMatViewStmt(const RefreshMatViewStmt *a, const RefreshMatViewStmt *b)
{
COMPARE_SCALAR_FIELD(concurrent);
COMPARE_SCALAR_FIELD(skipData);
COMPARE_NODE_FIELD(relation);
return true;
}
static bool
_equalReplicaIdentityStmt(const ReplicaIdentityStmt *a, const ReplicaIdentityStmt *b)
{
COMPARE_SCALAR_FIELD(identity_type);
COMPARE_STRING_FIELD(name);
return true;
}
static bool
_equalAlterSystemStmt(const AlterSystemStmt *a, const AlterSystemStmt *b)
{
COMPARE_NODE_FIELD(setstmt);
return true;
}
static bool
_equalCreateSeqStmt(const CreateSeqStmt *a, const CreateSeqStmt *b)
{
COMPARE_NODE_FIELD(sequence);
COMPARE_NODE_FIELD(options);
COMPARE_SCALAR_FIELD(ownerId);
COMPARE_SCALAR_FIELD(for_identity);
COMPARE_SCALAR_FIELD(if_not_exists);
return true;
}
static bool
_equalAlterSeqStmt(const AlterSeqStmt *a, const AlterSeqStmt *b)
{
COMPARE_NODE_FIELD(sequence);
COMPARE_NODE_FIELD(options);
COMPARE_SCALAR_FIELD(for_identity);
COMPARE_SCALAR_FIELD(missing_ok);
return true;
}
static bool
_equalVariableSetStmt(const VariableSetStmt *a, const VariableSetStmt *b)
{
COMPARE_SCALAR_FIELD(kind);
COMPARE_STRING_FIELD(name);
COMPARE_NODE_FIELD(args);
COMPARE_SCALAR_FIELD(is_local);
return true;
}
static bool
_equalVariableShowStmt(const VariableShowStmt *a, const VariableShowStmt *b)
{
COMPARE_STRING_FIELD(name);
return true;
}
static bool
_equalDiscardStmt(const DiscardStmt *a, const DiscardStmt *b)
{
COMPARE_SCALAR_FIELD(target);
return true;
}
static bool
_equalCreateTableSpaceStmt(const CreateTableSpaceStmt *a, const CreateTableSpaceStmt *b)
{
COMPARE_STRING_FIELD(tablespacename);
COMPARE_NODE_FIELD(owner);
COMPARE_STRING_FIELD(location);
COMPARE_NODE_FIELD(options);
return true;
}
static bool
_equalDropTableSpaceStmt(const DropTableSpaceStmt *a, const DropTableSpaceStmt *b)
{
COMPARE_STRING_FIELD(tablespacename);
COMPARE_SCALAR_FIELD(missing_ok);
return true;
}
static bool
_equalAlterTableSpaceOptionsStmt(const AlterTableSpaceOptionsStmt *a,
const AlterTableSpaceOptionsStmt *b)
{
COMPARE_STRING_FIELD(tablespacename);
COMPARE_NODE_FIELD(options);
COMPARE_SCALAR_FIELD(isReset);
return true;
}
static bool
_equalAlterTableMoveAllStmt(const AlterTableMoveAllStmt *a,
const AlterTableMoveAllStmt *b)
{
COMPARE_STRING_FIELD(orig_tablespacename);
COMPARE_SCALAR_FIELD(objtype);
COMPARE_NODE_FIELD(roles);
COMPARE_STRING_FIELD(new_tablespacename);
COMPARE_SCALAR_FIELD(nowait);
return true;
}
static bool
_equalCreateExtensionStmt(const CreateExtensionStmt *a, const CreateExtensionStmt *b)
{
COMPARE_STRING_FIELD(extname);
COMPARE_SCALAR_FIELD(if_not_exists);
COMPARE_NODE_FIELD(options);
return true;
}
static bool
_equalAlterExtensionStmt(const AlterExtensionStmt *a, const AlterExtensionStmt *b)
{
COMPARE_STRING_FIELD(extname);
COMPARE_NODE_FIELD(options);
return true;
}
static bool
_equalAlterExtensionContentsStmt(const AlterExtensionContentsStmt *a, const AlterExtensionContentsStmt *b)
{
COMPARE_STRING_FIELD(extname);
COMPARE_SCALAR_FIELD(action);
COMPARE_SCALAR_FIELD(objtype);
COMPARE_NODE_FIELD(object);
return true;
}
static bool
_equalCreateFdwStmt(const CreateFdwStmt *a, const CreateFdwStmt *b)
{
COMPARE_STRING_FIELD(fdwname);
COMPARE_NODE_FIELD(func_options);
COMPARE_NODE_FIELD(options);
return true;
}
static bool
_equalAlterFdwStmt(const AlterFdwStmt *a, const AlterFdwStmt *b)
{
COMPARE_STRING_FIELD(fdwname);
COMPARE_NODE_FIELD(func_options);
COMPARE_NODE_FIELD(options);
return true;
}
static bool
_equalCreateForeignServerStmt(const CreateForeignServerStmt *a, const CreateForeignServerStmt *b)
{
COMPARE_STRING_FIELD(servername);
COMPARE_STRING_FIELD(servertype);
COMPARE_STRING_FIELD(version);
COMPARE_STRING_FIELD(fdwname);
COMPARE_SCALAR_FIELD(if_not_exists);
COMPARE_NODE_FIELD(options);
return true;
}
static bool
_equalAlterForeignServerStmt(const AlterForeignServerStmt *a, const AlterForeignServerStmt *b)
{
COMPARE_STRING_FIELD(servername);
COMPARE_STRING_FIELD(version);
COMPARE_NODE_FIELD(options);
COMPARE_SCALAR_FIELD(has_version);
return true;
}
static bool
_equalCreateUserMappingStmt(const CreateUserMappingStmt *a, const CreateUserMappingStmt *b)
{
COMPARE_NODE_FIELD(user);
COMPARE_STRING_FIELD(servername);
COMPARE_SCALAR_FIELD(if_not_exists);
COMPARE_NODE_FIELD(options);
return true;
}
static bool
_equalAlterUserMappingStmt(const AlterUserMappingStmt *a, const AlterUserMappingStmt *b)
{
COMPARE_NODE_FIELD(user);
COMPARE_STRING_FIELD(servername);
COMPARE_NODE_FIELD(options);
return true;
}
static bool
_equalDropUserMappingStmt(const DropUserMappingStmt *a, const DropUserMappingStmt *b)
{
COMPARE_NODE_FIELD(user);
COMPARE_STRING_FIELD(servername);
COMPARE_SCALAR_FIELD(missing_ok);
return true;
}
static bool
_equalCreateForeignTableStmt(const CreateForeignTableStmt *a, const CreateForeignTableStmt *b)
{
if (!_equalCreateStmt(&a->base, &b->base))
return false;
COMPARE_STRING_FIELD(servername);
COMPARE_NODE_FIELD(options);
return true;
}
static bool
_equalImportForeignSchemaStmt(const ImportForeignSchemaStmt *a, const ImportForeignSchemaStmt *b)
{
COMPARE_STRING_FIELD(server_name);
COMPARE_STRING_FIELD(remote_schema);
COMPARE_STRING_FIELD(local_schema);
COMPARE_SCALAR_FIELD(list_type);
COMPARE_NODE_FIELD(table_list);
COMPARE_NODE_FIELD(options);
return true;
}
static bool
_equalCreateTransformStmt(const CreateTransformStmt *a, const CreateTransformStmt *b)
{
COMPARE_SCALAR_FIELD(replace);
COMPARE_NODE_FIELD(type_name);
COMPARE_STRING_FIELD(lang);
COMPARE_NODE_FIELD(fromsql);
COMPARE_NODE_FIELD(tosql);
return true;
}
static bool
_equalCreateAmStmt(const CreateAmStmt *a, const CreateAmStmt *b)
{
COMPARE_STRING_FIELD(amname);
COMPARE_NODE_FIELD(handler_name);
COMPARE_SCALAR_FIELD(amtype);
return true;
}
static bool
_equalCreateTrigStmt(const CreateTrigStmt *a, const CreateTrigStmt *b)
{
COMPARE_SCALAR_FIELD(replace);
COMPARE_SCALAR_FIELD(isconstraint);
COMPARE_STRING_FIELD(trigname);
COMPARE_NODE_FIELD(relation);
COMPARE_NODE_FIELD(funcname);
COMPARE_NODE_FIELD(args);
COMPARE_SCALAR_FIELD(row);
COMPARE_SCALAR_FIELD(timing);
COMPARE_SCALAR_FIELD(events);
COMPARE_NODE_FIELD(columns);
COMPARE_NODE_FIELD(whenClause);
COMPARE_NODE_FIELD(transitionRels);
COMPARE_SCALAR_FIELD(deferrable);
COMPARE_SCALAR_FIELD(initdeferred);
COMPARE_NODE_FIELD(constrrel);
return true;
}
static bool
_equalCreateEventTrigStmt(const CreateEventTrigStmt *a, const CreateEventTrigStmt *b)
{
COMPARE_STRING_FIELD(trigname);
COMPARE_STRING_FIELD(eventname);
COMPARE_NODE_FIELD(whenclause);
COMPARE_NODE_FIELD(funcname);
return true;
}
static bool
_equalAlterEventTrigStmt(const AlterEventTrigStmt *a, const AlterEventTrigStmt *b)
{
COMPARE_STRING_FIELD(trigname);
COMPARE_SCALAR_FIELD(tgenabled);
return true;
}
static bool
_equalCreatePLangStmt(const CreatePLangStmt *a, const CreatePLangStmt *b)
{
COMPARE_SCALAR_FIELD(replace);
COMPARE_STRING_FIELD(plname);
COMPARE_NODE_FIELD(plhandler);
COMPARE_NODE_FIELD(plinline);
COMPARE_NODE_FIELD(plvalidator);
COMPARE_SCALAR_FIELD(pltrusted);
return true;
}
static bool
_equalCreateRoleStmt(const CreateRoleStmt *a, const CreateRoleStmt *b)
{
COMPARE_SCALAR_FIELD(stmt_type);
COMPARE_STRING_FIELD(role);
COMPARE_NODE_FIELD(options);
return true;
}
static bool
_equalAlterRoleStmt(const AlterRoleStmt *a, const AlterRoleStmt *b)
{
COMPARE_NODE_FIELD(role);
COMPARE_NODE_FIELD(options);
COMPARE_SCALAR_FIELD(action);
return true;
}
static bool
_equalAlterRoleSetStmt(const AlterRoleSetStmt *a, const AlterRoleSetStmt *b)
{
COMPARE_NODE_FIELD(role);
COMPARE_STRING_FIELD(database);
COMPARE_NODE_FIELD(setstmt);
return true;
}
static bool
_equalDropRoleStmt(const DropRoleStmt *a, const DropRoleStmt *b)
{
COMPARE_NODE_FIELD(roles);
2006-02-04 20:06:47 +01:00
COMPARE_SCALAR_FIELD(missing_ok);
return true;
}
static bool
_equalLockStmt(const LockStmt *a, const LockStmt *b)
{
COMPARE_NODE_FIELD(relations);
COMPARE_SCALAR_FIELD(mode);
2004-03-11 02:47:41 +01:00
COMPARE_SCALAR_FIELD(nowait);
return true;
}
static bool
_equalConstraintsSetStmt(const ConstraintsSetStmt *a, const ConstraintsSetStmt *b)
{
COMPARE_NODE_FIELD(constraints);
COMPARE_SCALAR_FIELD(deferred);
return true;
}
static bool
_equalReindexStmt(const ReindexStmt *a, const ReindexStmt *b)
{
COMPARE_SCALAR_FIELD(kind);
COMPARE_NODE_FIELD(relation);
COMPARE_STRING_FIELD(name);
COMPARE_NODE_FIELD(params);
return true;
}
static bool
_equalCreateSchemaStmt(const CreateSchemaStmt *a, const CreateSchemaStmt *b)
{
COMPARE_STRING_FIELD(schemaname);
COMPARE_NODE_FIELD(authrole);
COMPARE_NODE_FIELD(schemaElts);
COMPARE_SCALAR_FIELD(if_not_exists);
return true;
}
static bool
_equalCreateConversionStmt(const CreateConversionStmt *a, const CreateConversionStmt *b)
{
COMPARE_NODE_FIELD(conversion_name);
COMPARE_STRING_FIELD(for_encoding_name);
COMPARE_STRING_FIELD(to_encoding_name);
COMPARE_NODE_FIELD(func_name);
COMPARE_SCALAR_FIELD(def);
return true;
}
static bool
_equalCreateCastStmt(const CreateCastStmt *a, const CreateCastStmt *b)
{
COMPARE_NODE_FIELD(sourcetype);
COMPARE_NODE_FIELD(targettype);
COMPARE_NODE_FIELD(func);
COMPARE_SCALAR_FIELD(context);
COMPARE_SCALAR_FIELD(inout);
return true;
}
static bool
_equalPrepareStmt(const PrepareStmt *a, const PrepareStmt *b)
{
COMPARE_STRING_FIELD(name);
COMPARE_NODE_FIELD(argtypes);
COMPARE_NODE_FIELD(query);
return true;
}
static bool
_equalExecuteStmt(const ExecuteStmt *a, const ExecuteStmt *b)
{
COMPARE_STRING_FIELD(name);
COMPARE_NODE_FIELD(params);
return true;
}
static bool
_equalDeallocateStmt(const DeallocateStmt *a, const DeallocateStmt *b)
{
COMPARE_STRING_FIELD(name);
return true;
}
static bool
_equalDropOwnedStmt(const DropOwnedStmt *a, const DropOwnedStmt *b)
{
COMPARE_NODE_FIELD(roles);
COMPARE_SCALAR_FIELD(behavior);
return true;
}
static bool
_equalReassignOwnedStmt(const ReassignOwnedStmt *a, const ReassignOwnedStmt *b)
{
COMPARE_NODE_FIELD(roles);
COMPARE_NODE_FIELD(newrole);
return true;
}
static bool
_equalAlterTSDictionaryStmt(const AlterTSDictionaryStmt *a, const AlterTSDictionaryStmt *b)
{
COMPARE_NODE_FIELD(dictname);
COMPARE_NODE_FIELD(options);
return true;
}
static bool
_equalAlterTSConfigurationStmt(const AlterTSConfigurationStmt *a,
const AlterTSConfigurationStmt *b)
{
Allow on-the-fly capture of DDL event details This feature lets user code inspect and take action on DDL events. Whenever a ddl_command_end event trigger is installed, DDL actions executed are saved to a list which can be inspected during execution of a function attached to ddl_command_end. The set-returning function pg_event_trigger_ddl_commands can be used to list actions so captured; it returns data about the type of command executed, as well as the affected object. This is sufficient for many uses of this feature. For the cases where it is not, we also provide a "command" column of a new pseudo-type pg_ddl_command, which is a pointer to a C structure that can be accessed by C code. The struct contains all the info necessary to completely inspect and even reconstruct the executed command. There is no actual deparse code here; that's expected to come later. What we have is enough infrastructure that the deparsing can be done in an external extension. The intention is that we will add some deparsing code in a later release, as an in-core extension. A new test module is included. It's probably insufficient as is, but it should be sufficient as a starting point for a more complete and future-proof approach. Authors: Álvaro Herrera, with some help from Andres Freund, Ian Barwick, Abhijit Menon-Sen. Reviews by Andres Freund, Robert Haas, Amit Kapila, Michael Paquier, Craig Ringer, David Steele. Additional input from Chris Browne, Dimitri Fontaine, Stephen Frost, Petr Jelínek, Tom Lane, Jim Nasby, Steven Singer, Pavel Stěhule. Based on original work by Dimitri Fontaine, though I didn't use his code. Discussion: https://www.postgresql.org/message-id/m2txrsdzxa.fsf@2ndQuadrant.fr https://www.postgresql.org/message-id/20131108153322.GU5809@eldon.alvh.no-ip.org https://www.postgresql.org/message-id/20150215044814.GL3391@alvh.no-ip.org
2015-05-12 00:14:31 +02:00
COMPARE_SCALAR_FIELD(kind);
COMPARE_NODE_FIELD(cfgname);
COMPARE_NODE_FIELD(tokentype);
COMPARE_NODE_FIELD(dicts);
COMPARE_SCALAR_FIELD(override);
COMPARE_SCALAR_FIELD(replace);
COMPARE_SCALAR_FIELD(missing_ok);
return true;
}
static bool
_equalPublicationObject(const PublicationObjSpec *a,
const PublicationObjSpec *b)
{
COMPARE_SCALAR_FIELD(pubobjtype);
COMPARE_STRING_FIELD(name);
COMPARE_NODE_FIELD(pubtable);
COMPARE_LOCATION_FIELD(location);
return true;
}
static bool
_equalPublicationTable(const PublicationTable *a, const PublicationTable *b)
{
COMPARE_NODE_FIELD(relation);
Allow specifying row filters for logical replication of tables. This feature adds row filtering for publication tables. When a publication is defined or modified, an optional WHERE clause can be specified. Rows that don't satisfy this WHERE clause will be filtered out. This allows a set of tables to be partially replicated. The row filter is per table. A new row filter can be added simply by specifying a WHERE clause after the table name. The WHERE clause must be enclosed by parentheses. The row filter WHERE clause for a table added to a publication that publishes UPDATE and/or DELETE operations must contain only columns that are covered by REPLICA IDENTITY. The row filter WHERE clause for a table added to a publication that publishes INSERT can use any column. If the row filter evaluates to NULL, it is regarded as "false". The WHERE clause only allows simple expressions that don't have user-defined functions, user-defined operators, user-defined types, user-defined collations, non-immutable built-in functions, or references to system columns. These restrictions could be addressed in the future. If you choose to do the initial table synchronization, only data that satisfies the row filters is copied to the subscriber. If the subscription has several publications in which a table has been published with different WHERE clauses, rows that satisfy ANY of the expressions will be copied. If a subscriber is a pre-15 version, the initial table synchronization won't use row filters even if they are defined in the publisher. The row filters are applied before publishing the changes. If the subscription has several publications in which the same table has been published with different filters (for the same publish operation), those expressions get OR'ed together so that rows satisfying any of the expressions will be replicated. This means all the other filters become redundant if (a) one of the publications have no filter at all, (b) one of the publications was created using FOR ALL TABLES, (c) one of the publications was created using FOR ALL TABLES IN SCHEMA and the table belongs to that same schema. If your publication contains a partitioned table, the publication parameter publish_via_partition_root determines if it uses the partition's row filter (if the parameter is false, the default) or the root partitioned table's row filter. Psql commands \dRp+ and \d <table-name> will display any row filters. Author: Hou Zhijie, Euler Taveira, Peter Smith, Ajin Cherian Reviewed-by: Greg Nancarrow, Haiying Tang, Amit Kapila, Tomas Vondra, Dilip Kumar, Vignesh C, Alvaro Herrera, Andres Freund, Wei Wang Discussion: https://www.postgresql.org/message-id/flat/CAHE3wggb715X%2BmK_DitLXF25B%3DjE6xyNCH4YOwM860JR7HarGQ%40mail.gmail.com
2022-02-22 03:24:12 +01:00
COMPARE_NODE_FIELD(whereClause);
Allow specifying column lists for logical replication This allows specifying an optional column list when adding a table to logical replication. The column list may be specified after the table name, enclosed in parentheses. Columns not included in this list are not sent to the subscriber, allowing the schema on the subscriber to be a subset of the publisher schema. For UPDATE/DELETE publications, the column list needs to cover all REPLICA IDENTITY columns. For INSERT publications, the column list is arbitrary and may omit some REPLICA IDENTITY columns. Furthermore, if the table uses REPLICA IDENTITY FULL, column list is not allowed. The column list can contain only simple column references. Complex expressions, function calls etc. are not allowed. This restriction could be relaxed in the future. During the initial table synchronization, only columns included in the column list are copied to the subscriber. If the subscription has several publications, containing the same table with different column lists, columns specified in any of the lists will be copied. This means all columns are replicated if the table has no column list at all (which is treated as column list with all columns), or when of the publications is defined as FOR ALL TABLES (possibly IN SCHEMA that matches the schema of the table). For partitioned tables, publish_via_partition_root determines whether the column list for the root or the leaf relation will be used. If the parameter is 'false' (the default), the list defined for the leaf relation is used. Otherwise, the column list for the root partition will be used. Psql commands \dRp+ and \d <table-name> now display any column lists. Author: Tomas Vondra, Alvaro Herrera, Rahila Syed Reviewed-by: Peter Eisentraut, Alvaro Herrera, Vignesh C, Ibrar Ahmed, Amit Kapila, Hou zj, Peter Smith, Wang wei, Tang, Shi yu Discussion: https://postgr.es/m/CAH2L28vddB_NFdRVpuyRBJEBWjz4BSyTB=_ektNRH8NJ1jf95g@mail.gmail.com
2022-03-26 00:45:21 +01:00
COMPARE_NODE_FIELD(columns);
return true;
}
static bool
_equalCreatePublicationStmt(const CreatePublicationStmt *a,
const CreatePublicationStmt *b)
{
COMPARE_STRING_FIELD(pubname);
COMPARE_NODE_FIELD(options);
COMPARE_NODE_FIELD(pubobjects);
COMPARE_SCALAR_FIELD(for_all_tables);
return true;
}
static bool
_equalAlterPublicationStmt(const AlterPublicationStmt *a,
const AlterPublicationStmt *b)
{
COMPARE_STRING_FIELD(pubname);
COMPARE_NODE_FIELD(options);
COMPARE_NODE_FIELD(pubobjects);
COMPARE_SCALAR_FIELD(for_all_tables);
COMPARE_SCALAR_FIELD(action);
return true;
}
static bool
_equalCreateSubscriptionStmt(const CreateSubscriptionStmt *a,
const CreateSubscriptionStmt *b)
{
COMPARE_STRING_FIELD(subname);
COMPARE_STRING_FIELD(conninfo);
COMPARE_NODE_FIELD(publication);
COMPARE_NODE_FIELD(options);
return true;
}
static bool
_equalAlterSubscriptionStmt(const AlterSubscriptionStmt *a,
const AlterSubscriptionStmt *b)
{
COMPARE_SCALAR_FIELD(kind);
COMPARE_STRING_FIELD(subname);
COMPARE_STRING_FIELD(conninfo);
COMPARE_NODE_FIELD(publication);
COMPARE_NODE_FIELD(options);
return true;
}
static bool
_equalDropSubscriptionStmt(const DropSubscriptionStmt *a,
const DropSubscriptionStmt *b)
{
COMPARE_STRING_FIELD(subname);
COMPARE_SCALAR_FIELD(missing_ok);
COMPARE_SCALAR_FIELD(behavior);
return true;
}
Row-Level Security Policies (RLS) Building on the updatable security-barrier views work, add the ability to define policies on tables to limit the set of rows which are returned from a query and which are allowed to be added to a table. Expressions defined by the policy for filtering are added to the security barrier quals of the query, while expressions defined to check records being added to a table are added to the with-check options of the query. New top-level commands are CREATE/ALTER/DROP POLICY and are controlled by the table owner. Row Security is able to be enabled and disabled by the owner on a per-table basis using ALTER TABLE .. ENABLE/DISABLE ROW SECURITY. Per discussion, ROW SECURITY is disabled on tables by default and must be enabled for policies on the table to be used. If no policies exist on a table with ROW SECURITY enabled, a default-deny policy is used and no records will be visible. By default, row security is applied at all times except for the table owner and the superuser. A new GUC, row_security, is added which can be set to ON, OFF, or FORCE. When set to FORCE, row security will be applied even for the table owner and superusers. When set to OFF, row security will be disabled when allowed and an error will be thrown if the user does not have rights to bypass row security. Per discussion, pg_dump sets row_security = OFF by default to ensure that exports and backups will have all data in the table or will error if there are insufficient privileges to bypass row security. A new option has been added to pg_dump, --enable-row-security, to ask pg_dump to export with row security enabled. A new role capability, BYPASSRLS, which can only be set by the superuser, is added to allow other users to be able to bypass row security using row_security = OFF. Many thanks to the various individuals who have helped with the design, particularly Robert Haas for his feedback. Authors include Craig Ringer, KaiGai Kohei, Adam Brightwell, Dean Rasheed, with additional changes and rework by me. Reviewers have included all of the above, Greg Smith, Jeff McCormick, and Robert Haas.
2014-09-19 17:18:35 +02:00
static bool
_equalCreatePolicyStmt(const CreatePolicyStmt *a, const CreatePolicyStmt *b)
{
COMPARE_STRING_FIELD(policy_name);
COMPARE_NODE_FIELD(table);
COMPARE_STRING_FIELD(cmd_name);
COMPARE_SCALAR_FIELD(permissive);
Row-Level Security Policies (RLS) Building on the updatable security-barrier views work, add the ability to define policies on tables to limit the set of rows which are returned from a query and which are allowed to be added to a table. Expressions defined by the policy for filtering are added to the security barrier quals of the query, while expressions defined to check records being added to a table are added to the with-check options of the query. New top-level commands are CREATE/ALTER/DROP POLICY and are controlled by the table owner. Row Security is able to be enabled and disabled by the owner on a per-table basis using ALTER TABLE .. ENABLE/DISABLE ROW SECURITY. Per discussion, ROW SECURITY is disabled on tables by default and must be enabled for policies on the table to be used. If no policies exist on a table with ROW SECURITY enabled, a default-deny policy is used and no records will be visible. By default, row security is applied at all times except for the table owner and the superuser. A new GUC, row_security, is added which can be set to ON, OFF, or FORCE. When set to FORCE, row security will be applied even for the table owner and superusers. When set to OFF, row security will be disabled when allowed and an error will be thrown if the user does not have rights to bypass row security. Per discussion, pg_dump sets row_security = OFF by default to ensure that exports and backups will have all data in the table or will error if there are insufficient privileges to bypass row security. A new option has been added to pg_dump, --enable-row-security, to ask pg_dump to export with row security enabled. A new role capability, BYPASSRLS, which can only be set by the superuser, is added to allow other users to be able to bypass row security using row_security = OFF. Many thanks to the various individuals who have helped with the design, particularly Robert Haas for his feedback. Authors include Craig Ringer, KaiGai Kohei, Adam Brightwell, Dean Rasheed, with additional changes and rework by me. Reviewers have included all of the above, Greg Smith, Jeff McCormick, and Robert Haas.
2014-09-19 17:18:35 +02:00
COMPARE_NODE_FIELD(roles);
COMPARE_NODE_FIELD(qual);
COMPARE_NODE_FIELD(with_check);
return true;
}
static bool
_equalAlterPolicyStmt(const AlterPolicyStmt *a, const AlterPolicyStmt *b)
{
COMPARE_STRING_FIELD(policy_name);
COMPARE_NODE_FIELD(table);
COMPARE_NODE_FIELD(roles);
COMPARE_NODE_FIELD(qual);
COMPARE_NODE_FIELD(with_check);
return true;
}
static bool
_equalA_Expr(const A_Expr *a, const A_Expr *b)
{
COMPARE_SCALAR_FIELD(kind);
COMPARE_NODE_FIELD(name);
COMPARE_NODE_FIELD(lexpr);
COMPARE_NODE_FIELD(rexpr);
COMPARE_LOCATION_FIELD(location);
return true;
}
static bool
_equalColumnRef(const ColumnRef *a, const ColumnRef *b)
{
COMPARE_NODE_FIELD(fields);
COMPARE_LOCATION_FIELD(location);
return true;
}
static bool
_equalParamRef(const ParamRef *a, const ParamRef *b)
{
COMPARE_SCALAR_FIELD(number);
COMPARE_LOCATION_FIELD(location);
return true;
}
static bool
_equalA_Const(const A_Const *a, const A_Const *b)
{
/*
* Hack for in-line val field. Also val is not valid is isnull is true.
*/
if (!a->isnull && !b->isnull &&
!equal(&a->val, &b->val))
return false;
COMPARE_SCALAR_FIELD(isnull);
COMPARE_LOCATION_FIELD(location);
return true;
}
static bool
_equalFuncCall(const FuncCall *a, const FuncCall *b)
{
COMPARE_NODE_FIELD(funcname);
COMPARE_NODE_FIELD(args);
COMPARE_NODE_FIELD(agg_order);
COMPARE_NODE_FIELD(agg_filter);
Improve our ability to regurgitate SQL-syntax function calls. The SQL spec calls out nonstandard syntax for certain function calls, for example substring() with numeric position info is supposed to be spelled "SUBSTRING(string FROM start FOR count)". We accept many of these things, but up to now would not print them in the same format, instead simplifying down to "substring"(string, start, count). That's long annoyed me because it creates an interoperability problem: we're gratuitously injecting Postgres-specific syntax into what might otherwise be a perfectly spec-compliant view definition. However, the real reason for addressing it right now is to support a planned change in the semantics of EXTRACT() a/k/a date_part(). When we switch that to returning numeric, we'll have the parser translate EXTRACT() to some new function name (might as well be "extract" if you ask me) and then teach ruleutils.c to reverse-list that per SQL spec. In this way existing calls to date_part() will continue to have the old semantics. To implement this, invent a new CoercionForm value COERCE_SQL_SYNTAX, and make the parser insert that rather than COERCE_EXPLICIT_CALL when the input has SQL-spec decoration. (But if the input has the form of a plain function call, continue to mark it COERCE_EXPLICIT_CALL, even if it's calling one of these functions.) Then ruleutils.c recognizes COERCE_SQL_SYNTAX as a cue to emit SQL call syntax. It can know which decoration to emit using hard-wired knowledge about the functions that could be called this way. (While this solution isn't extensible without manual additions, neither is the grammar, so this doesn't seem unmaintainable.) Notice that this solution will reverse-list a function call with SQL decoration only if it was entered that way; so dump-and-reload will not by itself produce any changes in the appearance of views. This requires adding a CoercionForm field to struct FuncCall. (I couldn't resist the temptation to rearrange that struct's field order a tad while I was at it.) FuncCall doesn't appear in stored rules, so that change isn't a reason for a catversion bump, but I did one anyway because the new enum value for CoercionForm fields could confuse old backend code. Possible future work: * Perhaps CoercionForm should now be renamed to DisplayForm, or something like that, to reflect its more general meaning. This'd require touching a couple hundred places, so it's not clear it's worth the code churn. * The SQLValueFunction node type, which was invented partly for the same goal of improving SQL-compatibility of view output, could perhaps be replaced with regular function calls marked with COERCE_SQL_SYNTAX. It's unclear if this would be a net code savings, however. Discussion: https://postgr.es/m/42b73d2d-da12-ba9f-570a-420e0cce19d9@phystech.edu
2020-11-04 18:34:50 +01:00
COMPARE_NODE_FIELD(over);
Support ordered-set (WITHIN GROUP) aggregates. This patch introduces generic support for ordered-set and hypothetical-set aggregate functions, as well as implementations of the instances defined in SQL:2008 (percentile_cont(), percentile_disc(), rank(), dense_rank(), percent_rank(), cume_dist()). We also added mode() though it is not in the spec, as well as versions of percentile_cont() and percentile_disc() that can compute multiple percentile values in one pass over the data. Unlike the original submission, this patch puts full control of the sorting process in the hands of the aggregate's support functions. To allow the support functions to find out how they're supposed to sort, a new API function AggGetAggref() is added to nodeAgg.c. This allows retrieval of the aggregate call's Aggref node, which may have other uses beyond the immediate need. There is also support for ordered-set aggregates to install cleanup callback functions, so that they can be sure that infrastructure such as tuplesort objects gets cleaned up. In passing, make some fixes in the recently-added support for variadic aggregates, and make some editorial adjustments in the recent FILTER additions for aggregates. Also, simplify use of IsBinaryCoercible() by allowing it to succeed whenever the target type is ANY or ANYELEMENT. It was inconsistent that it dealt with other polymorphic target types but not these. Atri Sharma and Andrew Gierth; reviewed by Pavel Stehule and Vik Fearing, and rather heavily editorialized upon by Tom Lane
2013-12-23 22:11:35 +01:00
COMPARE_SCALAR_FIELD(agg_within_group);
COMPARE_SCALAR_FIELD(agg_star);
COMPARE_SCALAR_FIELD(agg_distinct);
COMPARE_SCALAR_FIELD(func_variadic);
COMPARE_COERCIONFORM_FIELD(funcformat);
COMPARE_LOCATION_FIELD(location);
return true;
}
static bool
_equalA_Star(const A_Star *a, const A_Star *b)
{
return true;
}
static bool
_equalA_Indices(const A_Indices *a, const A_Indices *b)
{
COMPARE_SCALAR_FIELD(is_slice);
COMPARE_NODE_FIELD(lidx);
COMPARE_NODE_FIELD(uidx);
return true;
}
static bool
_equalA_Indirection(const A_Indirection *a, const A_Indirection *b)
{
COMPARE_NODE_FIELD(arg);
COMPARE_NODE_FIELD(indirection);
return true;
}
static bool
_equalA_ArrayExpr(const A_ArrayExpr *a, const A_ArrayExpr *b)
{
COMPARE_NODE_FIELD(elements);
COMPARE_LOCATION_FIELD(location);
return true;
}
static bool
_equalResTarget(const ResTarget *a, const ResTarget *b)
{
COMPARE_STRING_FIELD(name);
COMPARE_NODE_FIELD(indirection);
COMPARE_NODE_FIELD(val);
COMPARE_LOCATION_FIELD(location);
return true;
}
static bool
_equalMultiAssignRef(const MultiAssignRef *a, const MultiAssignRef *b)
{
COMPARE_NODE_FIELD(source);
COMPARE_SCALAR_FIELD(colno);
COMPARE_SCALAR_FIELD(ncolumns);
return true;
}
static bool
_equalTypeName(const TypeName *a, const TypeName *b)
{
COMPARE_NODE_FIELD(names);
COMPARE_SCALAR_FIELD(typeOid);
COMPARE_SCALAR_FIELD(setof);
COMPARE_SCALAR_FIELD(pct_type);
COMPARE_NODE_FIELD(typmods);
COMPARE_SCALAR_FIELD(typemod);
COMPARE_NODE_FIELD(arrayBounds);
COMPARE_LOCATION_FIELD(location);
return true;
}
static bool
_equalTypeCast(const TypeCast *a, const TypeCast *b)
{
COMPARE_NODE_FIELD(arg);
COMPARE_NODE_FIELD(typeName);
COMPARE_LOCATION_FIELD(location);
return true;
}
static bool
_equalCollateClause(const CollateClause *a, const CollateClause *b)
{
COMPARE_NODE_FIELD(arg);
COMPARE_NODE_FIELD(collname);
COMPARE_LOCATION_FIELD(location);
return true;
}
static bool
_equalSortBy(const SortBy *a, const SortBy *b)
{
COMPARE_NODE_FIELD(node);
COMPARE_SCALAR_FIELD(sortby_dir);
COMPARE_SCALAR_FIELD(sortby_nulls);
COMPARE_NODE_FIELD(useOp);
COMPARE_LOCATION_FIELD(location);
return true;
}
static bool
_equalWindowDef(const WindowDef *a, const WindowDef *b)
{
COMPARE_STRING_FIELD(name);
COMPARE_STRING_FIELD(refname);
COMPARE_NODE_FIELD(partitionClause);
COMPARE_NODE_FIELD(orderClause);
COMPARE_SCALAR_FIELD(frameOptions);
COMPARE_NODE_FIELD(startOffset);
COMPARE_NODE_FIELD(endOffset);
COMPARE_LOCATION_FIELD(location);
return true;
}
static bool
_equalRangeSubselect(const RangeSubselect *a, const RangeSubselect *b)
{
COMPARE_SCALAR_FIELD(lateral);
COMPARE_NODE_FIELD(subquery);
COMPARE_NODE_FIELD(alias);
return true;
}
static bool
_equalRangeFunction(const RangeFunction *a, const RangeFunction *b)
{
COMPARE_SCALAR_FIELD(lateral);
COMPARE_SCALAR_FIELD(ordinality);
COMPARE_SCALAR_FIELD(is_rowsfrom);
COMPARE_NODE_FIELD(functions);
COMPARE_NODE_FIELD(alias);
COMPARE_NODE_FIELD(coldeflist);
return true;
}
Redesign tablesample method API, and do extensive code review. The original implementation of TABLESAMPLE modeled the tablesample method API on index access methods, which wasn't a good choice because, without specialized DDL commands, there's no way to build an extension that can implement a TSM. (Raw inserts into system catalogs are not an acceptable thing to do, because we can't undo them during DROP EXTENSION, nor will pg_upgrade behave sanely.) Instead adopt an API more like procedural language handlers or foreign data wrappers, wherein the only SQL-level support object needed is a single handler function identified by having a special return type. This lets us get rid of the supporting catalog altogether, so that no custom DDL support is needed for the feature. Adjust the API so that it can support non-constant tablesample arguments (the original coding assumed we could evaluate the argument expressions at ExecInitSampleScan time, which is undesirable even if it weren't outright unsafe), and discourage sampling methods from looking at invisible tuples. Make sure that the BERNOULLI and SYSTEM methods are genuinely repeatable within and across queries, as required by the SQL standard, and deal more honestly with methods that can't support that requirement. Make a full code-review pass over the tablesample additions, and fix assorted bugs, omissions, infelicities, and cosmetic issues (such as failure to put the added code stanzas in a consistent ordering). Improve EXPLAIN's output of tablesample plans, too. Back-patch to 9.5 so that we don't have to support the original API in production.
2015-07-25 20:39:00 +02:00
static bool
_equalRangeTableSample(const RangeTableSample *a, const RangeTableSample *b)
{
COMPARE_NODE_FIELD(relation);
COMPARE_NODE_FIELD(method);
COMPARE_NODE_FIELD(args);
COMPARE_NODE_FIELD(repeatable);
COMPARE_LOCATION_FIELD(location);
return true;
}
static bool
_equalRangeTableFunc(const RangeTableFunc *a, const RangeTableFunc *b)
{
COMPARE_SCALAR_FIELD(lateral);
COMPARE_NODE_FIELD(docexpr);
COMPARE_NODE_FIELD(rowexpr);
COMPARE_NODE_FIELD(namespaces);
COMPARE_NODE_FIELD(columns);
COMPARE_NODE_FIELD(alias);
COMPARE_LOCATION_FIELD(location);
return true;
}
static bool
_equalRangeTableFuncCol(const RangeTableFuncCol *a, const RangeTableFuncCol *b)
{
COMPARE_STRING_FIELD(colname);
COMPARE_NODE_FIELD(typeName);
COMPARE_SCALAR_FIELD(for_ordinality);
COMPARE_SCALAR_FIELD(is_not_null);
COMPARE_NODE_FIELD(colexpr);
COMPARE_NODE_FIELD(coldefexpr);
COMPARE_LOCATION_FIELD(location);
return true;
}
static bool
_equalIndexElem(const IndexElem *a, const IndexElem *b)
{
COMPARE_STRING_FIELD(name);
COMPARE_NODE_FIELD(expr);
COMPARE_STRING_FIELD(indexcolname);
COMPARE_NODE_FIELD(collation);
COMPARE_NODE_FIELD(opclass);
Implement operator class parameters PostgreSQL provides set of template index access methods, where opclasses have much freedom in the semantics of indexing. These index AMs are GiST, GIN, SP-GiST and BRIN. There opclasses define representation of keys, operations on them and supported search strategies. So, it's natural that opclasses may be faced some tradeoffs, which require user-side decision. This commit implements opclass parameters allowing users to set some values, which tell opclass how to index the particular dataset. This commit doesn't introduce new storage in system catalog. Instead it uses pg_attribute.attoptions, which is used for table column storage options but unused for index attributes. In order to evade changing signature of each opclass support function, we implement unified way to pass options to opclass support functions. Options are set to fn_expr as the constant bytea expression. It's possible due to the fact that opclass support functions are executed outside of expressions, so fn_expr is unused for them. This commit comes with some examples of opclass options usage. We parametrize signature length in GiST. That applies to multiple opclasses: tsvector_ops, gist__intbig_ops, gist_ltree_ops, gist__ltree_ops, gist_trgm_ops and gist_hstore_ops. Also we parametrize maximum number of integer ranges for gist__int_ops. However, the main future usage of this feature is expected to be json, where users would be able to specify which way to index particular json parts. Catversion is bumped. Discussion: https://postgr.es/m/d22c3a18-31c7-1879-fc11-4c1ce2f5e5af%40postgrespro.ru Author: Nikita Glukhov, revised by me Reviwed-by: Nikolay Shaplov, Robert Haas, Tom Lane, Tomas Vondra, Alvaro Herrera
2020-03-30 18:17:11 +02:00
COMPARE_NODE_FIELD(opclassopts);
COMPARE_SCALAR_FIELD(ordering);
COMPARE_SCALAR_FIELD(nulls_ordering);
return true;
}
Extended statistics on expressions Allow defining extended statistics on expressions, not just just on simple column references. With this commit, expressions are supported by all existing extended statistics kinds, improving the same types of estimates. A simple example may look like this: CREATE TABLE t (a int); CREATE STATISTICS s ON mod(a,10), mod(a,20) FROM t; ANALYZE t; The collected statistics are useful e.g. to estimate queries with those expressions in WHERE or GROUP BY clauses: SELECT * FROM t WHERE mod(a,10) = 0 AND mod(a,20) = 0; SELECT 1 FROM t GROUP BY mod(a,10), mod(a,20); This introduces new internal statistics kind 'e' (expressions) which is built automatically when the statistics object definition includes any expressions. This represents single-expression statistics, as if there was an expression index (but without the index maintenance overhead). The statistics is stored in pg_statistics_ext_data as an array of composite types, which is possible thanks to 79f6a942bd. CREATE STATISTICS allows building statistics on a single expression, in which case in which case it's not possible to specify statistics kinds. A new system view pg_stats_ext_exprs can be used to display expression statistics, similarly to pg_stats and pg_stats_ext views. ALTER TABLE ... ALTER COLUMN ... TYPE now treats indexes the same way it treats indexes, i.e. it drops and recreates the statistics. This means all statistics are reset, and we no longer try to preserve at least the functional dependencies. This should not be a major issue in practice, as the functional dependencies actually rely on per-column statistics, which were always reset anyway. Author: Tomas Vondra Reviewed-by: Justin Pryzby, Dean Rasheed, Zhihong Yu Discussion: https://postgr.es/m/ad7891d2-e90c-b446-9fe2-7419143847d7%40enterprisedb.com
2021-03-26 23:22:01 +01:00
static bool
_equalStatsElem(const StatsElem *a, const StatsElem *b)
{
COMPARE_STRING_FIELD(name);
COMPARE_NODE_FIELD(expr);
return true;
}
static bool
_equalColumnDef(const ColumnDef *a, const ColumnDef *b)
{
COMPARE_STRING_FIELD(colname);
COMPARE_NODE_FIELD(typeName);
Allow configurable LZ4 TOAST compression. There is now a per-column COMPRESSION option which can be set to pglz (the default, and the only option in up until now) or lz4. Or, if you like, you can set the new default_toast_compression GUC to lz4, and then that will be the default for new table columns for which no value is specified. We don't have lz4 support in the PostgreSQL code, so to use lz4 compression, PostgreSQL must be built --with-lz4. In general, TOAST compression means compression of individual column values, not the whole tuple, and those values can either be compressed inline within the tuple or compressed and then stored externally in the TOAST table, so those properties also apply to this feature. Prior to this commit, a TOAST pointer has two unused bits as part of the va_extsize field, and a compessed datum has two unused bits as part of the va_rawsize field. These bits are unused because the length of a varlena is limited to 1GB; we now use them to indicate the compression type that was used. This means we only have bit space for 2 more built-in compresison types, but we could work around that problem, if necessary, by introducing a new vartag_external value for any further types we end up wanting to add. Hopefully, it won't be too important to offer a wide selection of algorithms here, since each one we add not only takes more coding but also adds a build dependency for every packager. Nevertheless, it seems worth doing at least this much, because LZ4 gets better compression than PGLZ with less CPU usage. It's possible for LZ4-compressed datums to leak into composite type values stored on disk, just as it is for PGLZ. It's also possible for LZ4-compressed attributes to be copied into a different table via SQL commands such as CREATE TABLE AS or INSERT .. SELECT. It would be expensive to force such values to be decompressed, so PostgreSQL has never done so. For the same reasons, we also don't force recompression of already-compressed values even if the target table prefers a different compression method than was used for the source data. These architectural decisions are perhaps arguable but revisiting them is well beyond the scope of what seemed possible to do as part of this project. However, it's relatively cheap to recompress as part of VACUUM FULL or CLUSTER, so this commit adjusts those commands to do so, if the configured compression method of the table happens not to match what was used for some column value stored therein. Dilip Kumar. The original patches on which this work was based were written by Ildus Kurbangaliev, and those were patches were based on even earlier work by Nikita Glukhov, but the design has since changed very substantially, since allow a potentially large number of compression methods that could be added and dropped on a running system proved too problematic given some of the architectural issues mentioned above; the choice of which specific compression method to add first is now different; and a lot of the code has been heavily refactored. More recently, Justin Przyby helped quite a bit with testing and reviewing and this version also includes some code contributions from him. Other design input and review from Tomas Vondra, Álvaro Herrera, Andres Freund, Oleg Bartunov, Alexander Korotkov, and me. Discussion: http://postgr.es/m/20170907194236.4cefce96%40wp.localdomain Discussion: http://postgr.es/m/CAFiTN-uUpX3ck%3DK0mLEk-G_kUQY%3DSNOTeqdaNRR9FMdQrHKebw%40mail.gmail.com
2021-03-19 20:10:38 +01:00
COMPARE_STRING_FIELD(compression);
COMPARE_SCALAR_FIELD(inhcount);
COMPARE_SCALAR_FIELD(is_local);
COMPARE_SCALAR_FIELD(is_not_null);
Remove collation information from TypeName, where it does not belong. The initial collations patch treated a COLLATE spec as part of a TypeName, following what can only be described as brain fade on the part of the SQL committee. It's a lot more reasonable to treat COLLATE as a syntactically separate object, so that it can be added in only the productions where it actually belongs, rather than needing to reject it in a boatload of places where it doesn't belong (something the original patch mostly failed to do). In addition this change lets us meet the spec's requirement to allow COLLATE anywhere in the clauses of a ColumnDef, and it avoids unfriendly behavior for constructs such as "foo::type COLLATE collation". To do this, pull collation information out of TypeName and put it in ColumnDef instead, thus reverting most of the collation-related changes in parse_type.c's API. I made one additional structural change, which was to use a ColumnDef as an intermediate node in AT_AlterColumnType AlterTableCmd nodes. This provides enough room to get rid of the "transform" wart in AlterTableCmd too, since the ColumnDef can carry the USING expression easily enough. Also fix some other minor bugs that have crept in in the same areas, like failure to copy recently-added fields of ColumnDef in copyfuncs.c. While at it, document the formerly secret ability to specify a collation in ALTER TABLE ALTER COLUMN TYPE, ALTER TYPE ADD ATTRIBUTE, and ALTER TYPE ALTER ATTRIBUTE TYPE; and correct some misstatements about what the default collation selection will be when COLLATE is omitted. BTW, the three-parameter form of format_type() should go away too, since it just contributes to the confusion in this area; but I'll do that in a separate patch.
2011-03-10 04:38:52 +01:00
COMPARE_SCALAR_FIELD(is_from_type);
COMPARE_SCALAR_FIELD(storage);
COMPARE_NODE_FIELD(raw_default);
COMPARE_NODE_FIELD(cooked_default);
COMPARE_SCALAR_FIELD(identity);
COMPARE_NODE_FIELD(identitySequence);
COMPARE_SCALAR_FIELD(generated);
Remove collation information from TypeName, where it does not belong. The initial collations patch treated a COLLATE spec as part of a TypeName, following what can only be described as brain fade on the part of the SQL committee. It's a lot more reasonable to treat COLLATE as a syntactically separate object, so that it can be added in only the productions where it actually belongs, rather than needing to reject it in a boatload of places where it doesn't belong (something the original patch mostly failed to do). In addition this change lets us meet the spec's requirement to allow COLLATE anywhere in the clauses of a ColumnDef, and it avoids unfriendly behavior for constructs such as "foo::type COLLATE collation". To do this, pull collation information out of TypeName and put it in ColumnDef instead, thus reverting most of the collation-related changes in parse_type.c's API. I made one additional structural change, which was to use a ColumnDef as an intermediate node in AT_AlterColumnType AlterTableCmd nodes. This provides enough room to get rid of the "transform" wart in AlterTableCmd too, since the ColumnDef can carry the USING expression easily enough. Also fix some other minor bugs that have crept in in the same areas, like failure to copy recently-added fields of ColumnDef in copyfuncs.c. While at it, document the formerly secret ability to specify a collation in ALTER TABLE ALTER COLUMN TYPE, ALTER TYPE ADD ATTRIBUTE, and ALTER TYPE ALTER ATTRIBUTE TYPE; and correct some misstatements about what the default collation selection will be when COLLATE is omitted. BTW, the three-parameter form of format_type() should go away too, since it just contributes to the confusion in this area; but I'll do that in a separate patch.
2011-03-10 04:38:52 +01:00
COMPARE_NODE_FIELD(collClause);
COMPARE_SCALAR_FIELD(collOid);
COMPARE_NODE_FIELD(constraints);
COMPARE_NODE_FIELD(fdwoptions);
COMPARE_LOCATION_FIELD(location);
return true;
}
static bool
_equalConstraint(const Constraint *a, const Constraint *b)
{
COMPARE_SCALAR_FIELD(contype);
COMPARE_STRING_FIELD(conname);
COMPARE_SCALAR_FIELD(deferrable);
COMPARE_SCALAR_FIELD(initdeferred);
COMPARE_LOCATION_FIELD(location);
COMPARE_SCALAR_FIELD(is_no_inherit);
COMPARE_NODE_FIELD(raw_expr);
COMPARE_STRING_FIELD(cooked_expr);
COMPARE_SCALAR_FIELD(generated_when);
COMPARE_SCALAR_FIELD(nulls_not_distinct);
COMPARE_NODE_FIELD(keys);
COMPARE_NODE_FIELD(including);
COMPARE_NODE_FIELD(exclusions);
COMPARE_NODE_FIELD(options);
COMPARE_STRING_FIELD(indexname);
COMPARE_STRING_FIELD(indexspace);
COMPARE_SCALAR_FIELD(reset_default_tblspc);
COMPARE_STRING_FIELD(access_method);
COMPARE_NODE_FIELD(where_clause);
COMPARE_NODE_FIELD(pktable);
COMPARE_NODE_FIELD(fk_attrs);
COMPARE_NODE_FIELD(pk_attrs);
COMPARE_SCALAR_FIELD(fk_matchtype);
COMPARE_SCALAR_FIELD(fk_upd_action);
COMPARE_SCALAR_FIELD(fk_del_action);
COMPARE_NODE_FIELD(fk_del_set_cols);
COMPARE_NODE_FIELD(old_conpfeqop);
COMPARE_SCALAR_FIELD(old_pktable_oid);
COMPARE_SCALAR_FIELD(skip_validation);
COMPARE_SCALAR_FIELD(initially_valid);
return true;
}
static bool
_equalDefElem(const DefElem *a, const DefElem *b)
{
COMPARE_STRING_FIELD(defnamespace);
COMPARE_STRING_FIELD(defname);
COMPARE_NODE_FIELD(arg);
COMPARE_SCALAR_FIELD(defaction);
COMPARE_LOCATION_FIELD(location);
return true;
}
static bool
_equalLockingClause(const LockingClause *a, const LockingClause *b)
{
COMPARE_NODE_FIELD(lockedRels);
Improve concurrency of foreign key locking This patch introduces two additional lock modes for tuples: "SELECT FOR KEY SHARE" and "SELECT FOR NO KEY UPDATE". These don't block each other, in contrast with already existing "SELECT FOR SHARE" and "SELECT FOR UPDATE". UPDATE commands that do not modify the values stored in the columns that are part of the key of the tuple now grab a SELECT FOR NO KEY UPDATE lock on the tuple, allowing them to proceed concurrently with tuple locks of the FOR KEY SHARE variety. Foreign key triggers now use FOR KEY SHARE instead of FOR SHARE; this means the concurrency improvement applies to them, which is the whole point of this patch. The added tuple lock semantics require some rejiggering of the multixact module, so that the locking level that each transaction is holding can be stored alongside its Xid. Also, multixacts now need to persist across server restarts and crashes, because they can now represent not only tuple locks, but also tuple updates. This means we need more careful tracking of lifetime of pg_multixact SLRU files; since they now persist longer, we require more infrastructure to figure out when they can be removed. pg_upgrade also needs to be careful to copy pg_multixact files over from the old server to the new, or at least part of multixact.c state, depending on the versions of the old and new servers. Tuple time qualification rules (HeapTupleSatisfies routines) need to be careful not to consider tuples with the "is multi" infomask bit set as being only locked; they might need to look up MultiXact values (i.e. possibly do pg_multixact I/O) to find out the Xid that updated a tuple, whereas they previously were assured to only use information readily available from the tuple header. This is considered acceptable, because the extra I/O would involve cases that would previously cause some commands to block waiting for concurrent transactions to finish. Another important change is the fact that locking tuples that have previously been updated causes the future versions to be marked as locked, too; this is essential for correctness of foreign key checks. This causes additional WAL-logging, also (there was previously a single WAL record for a locked tuple; now there are as many as updated copies of the tuple there exist.) With all this in place, contention related to tuples being checked by foreign key rules should be much reduced. As a bonus, the old behavior that a subtransaction grabbing a stronger tuple lock than the parent (sub)transaction held on a given tuple and later aborting caused the weaker lock to be lost, has been fixed. Many new spec files were added for isolation tester framework, to ensure overall behavior is sane. There's probably room for several more tests. There were several reviewers of this patch; in particular, Noah Misch and Andres Freund spent considerable time in it. Original idea for the patch came from Simon Riggs, after a problem report by Joel Jacobson. Most code is from me, with contributions from Marti Raudsepp, Alexander Shulgin, Noah Misch and Andres Freund. This patch was discussed in several pgsql-hackers threads; the most important start at the following message-ids: AANLkTimo9XVcEzfiBR-ut3KVNDkjm2Vxh+t8kAmWjPuv@mail.gmail.com 1290721684-sup-3951@alvh.no-ip.org 1294953201-sup-2099@alvh.no-ip.org 1320343602-sup-2290@alvh.no-ip.org 1339690386-sup-8927@alvh.no-ip.org 4FE5FF020200002500048A3D@gw.wicourts.gov 4FEAB90A0200002500048B7D@gw.wicourts.gov
2013-01-23 16:04:59 +01:00
COMPARE_SCALAR_FIELD(strength);
COMPARE_SCALAR_FIELD(waitPolicy);
return true;
}
static bool
_equalRangeTblEntry(const RangeTblEntry *a, const RangeTblEntry *b)
{
COMPARE_SCALAR_FIELD(rtekind);
COMPARE_SCALAR_FIELD(relid);
COMPARE_SCALAR_FIELD(relkind);
COMPARE_SCALAR_FIELD(rellockmode);
COMPARE_NODE_FIELD(tablesample);
COMPARE_NODE_FIELD(subquery);
COMPARE_SCALAR_FIELD(security_barrier);
COMPARE_SCALAR_FIELD(jointype);
Reconsider the representation of join alias Vars. The core idea of this patch is to make the parser generate join alias Vars (that is, ones with varno pointing to a JOIN RTE) only when the alias Var is actually different from any raw join input, that is a type coercion and/or COALESCE is necessary to generate the join output value. Otherwise just generate varno/varattno pointing to the relevant join input column. In effect, this means that the planner's flatten_join_alias_vars() transformation is already done in the parser, for all cases except (a) columns that are merged by JOIN USING and are transformed in the process, and (b) whole-row join Vars. In principle that would allow us to skip doing flatten_join_alias_vars() in many more queries than we do now, but we don't have quite enough infrastructure to know that we can do so --- in particular there's no cheap way to know whether there are any whole-row join Vars. I'm not sure if it's worth the trouble to add a Query-level flag for that, and in any case it seems like fit material for a separate patch. But even without skipping the work entirely, this should make flatten_join_alias_vars() faster, particularly where there are nested joins that it previously had to flatten recursively. An essential part of this change is to replace Var nodes' varnoold/varoattno fields with varnosyn/varattnosyn, which have considerably more tightly-defined meanings than the old fields: when they differ from varno/varattno, they identify the Var's position in an aliased JOIN RTE, and the join alias is what ruleutils.c should print for the Var. This is necessary because the varno change destroyed ruleutils.c's ability to find the JOIN RTE from the Var's varno. Another way in which this change broke ruleutils.c is that it's no longer feasible to determine, from a JOIN RTE's joinaliasvars list, which join columns correspond to which columns of the join's immediate input relations. (If those are sub-joins, the joinaliasvars entries may point to columns of their base relations, not the sub-joins.) But that was a horrid mess requiring a lot of fragile assumptions already, so let's just bite the bullet and add some more JOIN RTE fields to make it more straightforward to figure that out. I added two integer-List fields containing the relevant column numbers from the left and right input rels, plus a count of how many merged columns there are. This patch depends on the ParseNamespaceColumn infrastructure that I added in commit 5815696bc. The biggest bit of code change is restructuring transformFromClauseItem's handling of JOINs so that the ParseNamespaceColumn data is propagated upward correctly. Other than that and the ruleutils fixes, everything pretty much just works, though some processing is now inessential. I grabbed two pieces of low-hanging fruit in that line: 1. In find_expr_references, we don't need to recurse into join alias Vars anymore. There aren't any except for references to merged USING columns, which are more properly handled when we scan the join's RTE. This change actually fixes an edge-case issue: we will now record a dependency on any type-coercion function present in a USING column's joinaliasvar, even if that join column has no references in the query text. The odds of the missing dependency causing a problem seem quite small: you'd have to posit somebody dropping an implicit cast between two data types, without removing the types themselves, and then having a stored rule containing a whole-row Var for a join whose USING merge depends on that cast. So I don't feel a great need to change this in the back branches. But in theory this way is more correct. 2. markRTEForSelectPriv and markTargetListOrigin don't need to recurse into join alias Vars either, because the cases they care about don't apply to alias Vars for USING columns that are semantically distinct from the underlying columns. This removes the only case in which markVarForSelectPriv could be called with NULL for the RTE, so adjust the comments to describe that hack as being strictly internal to markRTEForSelectPriv. catversion bump required due to changes in stored rules. Discussion: https://postgr.es/m/7115.1577986646@sss.pgh.pa.us
2020-01-09 17:56:59 +01:00
COMPARE_SCALAR_FIELD(joinmergedcols);
COMPARE_NODE_FIELD(joinaliasvars);
Reconsider the representation of join alias Vars. The core idea of this patch is to make the parser generate join alias Vars (that is, ones with varno pointing to a JOIN RTE) only when the alias Var is actually different from any raw join input, that is a type coercion and/or COALESCE is necessary to generate the join output value. Otherwise just generate varno/varattno pointing to the relevant join input column. In effect, this means that the planner's flatten_join_alias_vars() transformation is already done in the parser, for all cases except (a) columns that are merged by JOIN USING and are transformed in the process, and (b) whole-row join Vars. In principle that would allow us to skip doing flatten_join_alias_vars() in many more queries than we do now, but we don't have quite enough infrastructure to know that we can do so --- in particular there's no cheap way to know whether there are any whole-row join Vars. I'm not sure if it's worth the trouble to add a Query-level flag for that, and in any case it seems like fit material for a separate patch. But even without skipping the work entirely, this should make flatten_join_alias_vars() faster, particularly where there are nested joins that it previously had to flatten recursively. An essential part of this change is to replace Var nodes' varnoold/varoattno fields with varnosyn/varattnosyn, which have considerably more tightly-defined meanings than the old fields: when they differ from varno/varattno, they identify the Var's position in an aliased JOIN RTE, and the join alias is what ruleutils.c should print for the Var. This is necessary because the varno change destroyed ruleutils.c's ability to find the JOIN RTE from the Var's varno. Another way in which this change broke ruleutils.c is that it's no longer feasible to determine, from a JOIN RTE's joinaliasvars list, which join columns correspond to which columns of the join's immediate input relations. (If those are sub-joins, the joinaliasvars entries may point to columns of their base relations, not the sub-joins.) But that was a horrid mess requiring a lot of fragile assumptions already, so let's just bite the bullet and add some more JOIN RTE fields to make it more straightforward to figure that out. I added two integer-List fields containing the relevant column numbers from the left and right input rels, plus a count of how many merged columns there are. This patch depends on the ParseNamespaceColumn infrastructure that I added in commit 5815696bc. The biggest bit of code change is restructuring transformFromClauseItem's handling of JOINs so that the ParseNamespaceColumn data is propagated upward correctly. Other than that and the ruleutils fixes, everything pretty much just works, though some processing is now inessential. I grabbed two pieces of low-hanging fruit in that line: 1. In find_expr_references, we don't need to recurse into join alias Vars anymore. There aren't any except for references to merged USING columns, which are more properly handled when we scan the join's RTE. This change actually fixes an edge-case issue: we will now record a dependency on any type-coercion function present in a USING column's joinaliasvar, even if that join column has no references in the query text. The odds of the missing dependency causing a problem seem quite small: you'd have to posit somebody dropping an implicit cast between two data types, without removing the types themselves, and then having a stored rule containing a whole-row Var for a join whose USING merge depends on that cast. So I don't feel a great need to change this in the back branches. But in theory this way is more correct. 2. markRTEForSelectPriv and markTargetListOrigin don't need to recurse into join alias Vars either, because the cases they care about don't apply to alias Vars for USING columns that are semantically distinct from the underlying columns. This removes the only case in which markVarForSelectPriv could be called with NULL for the RTE, so adjust the comments to describe that hack as being strictly internal to markRTEForSelectPriv. catversion bump required due to changes in stored rules. Discussion: https://postgr.es/m/7115.1577986646@sss.pgh.pa.us
2020-01-09 17:56:59 +01:00
COMPARE_NODE_FIELD(joinleftcols);
COMPARE_NODE_FIELD(joinrightcols);
COMPARE_NODE_FIELD(join_using_alias);
COMPARE_NODE_FIELD(functions);
COMPARE_SCALAR_FIELD(funcordinality);
COMPARE_NODE_FIELD(tablefunc);
COMPARE_NODE_FIELD(values_lists);
COMPARE_STRING_FIELD(ctename);
COMPARE_SCALAR_FIELD(ctelevelsup);
COMPARE_SCALAR_FIELD(self_reference);
Fix reporting of column typmods for multi-row VALUES constructs. expandRTE() and get_rte_attribute_type() reported the exprType() and exprTypmod() values of the expressions in the first row of the VALUES as being the column type/typmod returned by the VALUES RTE. That's fine for the data type, since we coerce all expressions in a column to have the same common type. But we don't coerce them to have a common typmod, so it was possible for rows after the first one to return values that violate the claimed column typmod. This leads to the incorrect result seen in bug #14448 from Hassan Mahmood, as well as some other corner-case misbehaviors. The desired behavior is the same as we use in other type-unification cases: report the common typmod if there is one, but otherwise return -1 indicating no particular constraint. It's cheap for transformValuesClause to determine the common typmod while transforming a multi-row VALUES, but it'd be less cheap for expandRTE() and get_rte_attribute_type() to re-determine that info every time they're asked --- possibly a lot less cheap, if the VALUES has many rows. Therefore, the best fix is to record the common typmods explicitly in a list in the VALUES RTE, as we were already doing for column collations. This looks quite a bit like what we're doing for CTE RTEs, so we can save a little bit of space and code by unifying the representation for those two RTE types. They both now share coltypes/coltypmods/colcollations fields. (At some point it might seem desirable to populate those fields for all RTE types; but right now it looks like constructing them for other RTE types would add more code and cycles than it would save.) The RTE change requires a catversion bump, so this fix is only usable in HEAD. If we fix this at all in the back branches, the patch will need to look quite different. Report: https://postgr.es/m/20161205143037.4377.60754@wrigleys.postgresql.org Discussion: https://postgr.es/m/27429.1480968538@sss.pgh.pa.us
2016-12-08 17:40:02 +01:00
COMPARE_NODE_FIELD(coltypes);
COMPARE_NODE_FIELD(coltypmods);
COMPARE_NODE_FIELD(colcollations);
COMPARE_STRING_FIELD(enrname);
COMPARE_SCALAR_FIELD(enrtuples);
COMPARE_NODE_FIELD(alias);
COMPARE_NODE_FIELD(eref);
COMPARE_SCALAR_FIELD(lateral);
COMPARE_SCALAR_FIELD(inh);
COMPARE_SCALAR_FIELD(inFromCl);
COMPARE_SCALAR_FIELD(requiredPerms);
COMPARE_SCALAR_FIELD(checkAsUser);
COMPARE_BITMAPSET_FIELD(selectedCols);
COMPARE_BITMAPSET_FIELD(insertedCols);
COMPARE_BITMAPSET_FIELD(updatedCols);
COMPARE_BITMAPSET_FIELD(extraUpdatedCols);
COMPARE_NODE_FIELD(securityQuals);
return true;
}
static bool
_equalRangeTblFunction(const RangeTblFunction *a, const RangeTblFunction *b)
{
COMPARE_NODE_FIELD(funcexpr);
COMPARE_SCALAR_FIELD(funccolcount);
COMPARE_NODE_FIELD(funccolnames);
COMPARE_NODE_FIELD(funccoltypes);
COMPARE_NODE_FIELD(funccoltypmods);
COMPARE_NODE_FIELD(funccolcollations);
COMPARE_BITMAPSET_FIELD(funcparams);
return true;
}
Redesign tablesample method API, and do extensive code review. The original implementation of TABLESAMPLE modeled the tablesample method API on index access methods, which wasn't a good choice because, without specialized DDL commands, there's no way to build an extension that can implement a TSM. (Raw inserts into system catalogs are not an acceptable thing to do, because we can't undo them during DROP EXTENSION, nor will pg_upgrade behave sanely.) Instead adopt an API more like procedural language handlers or foreign data wrappers, wherein the only SQL-level support object needed is a single handler function identified by having a special return type. This lets us get rid of the supporting catalog altogether, so that no custom DDL support is needed for the feature. Adjust the API so that it can support non-constant tablesample arguments (the original coding assumed we could evaluate the argument expressions at ExecInitSampleScan time, which is undesirable even if it weren't outright unsafe), and discourage sampling methods from looking at invisible tuples. Make sure that the BERNOULLI and SYSTEM methods are genuinely repeatable within and across queries, as required by the SQL standard, and deal more honestly with methods that can't support that requirement. Make a full code-review pass over the tablesample additions, and fix assorted bugs, omissions, infelicities, and cosmetic issues (such as failure to put the added code stanzas in a consistent ordering). Improve EXPLAIN's output of tablesample plans, too. Back-patch to 9.5 so that we don't have to support the original API in production.
2015-07-25 20:39:00 +02:00
static bool
_equalTableSampleClause(const TableSampleClause *a, const TableSampleClause *b)
{
COMPARE_SCALAR_FIELD(tsmhandler);
COMPARE_NODE_FIELD(args);
COMPARE_NODE_FIELD(repeatable);
return true;
}
static bool
_equalWithCheckOption(const WithCheckOption *a, const WithCheckOption *b)
{
COMPARE_SCALAR_FIELD(kind);
COMPARE_STRING_FIELD(relname);
COMPARE_STRING_FIELD(polname);
COMPARE_NODE_FIELD(qual);
COMPARE_SCALAR_FIELD(cascaded);
return true;
}
static bool
_equalSortGroupClause(const SortGroupClause *a, const SortGroupClause *b)
{
COMPARE_SCALAR_FIELD(tleSortGroupRef);
COMPARE_SCALAR_FIELD(eqop);
COMPARE_SCALAR_FIELD(sortop);
COMPARE_SCALAR_FIELD(nulls_first);
COMPARE_SCALAR_FIELD(hashable);
return true;
}
Support GROUPING SETS, CUBE and ROLLUP. This SQL standard functionality allows to aggregate data by different GROUP BY clauses at once. Each grouping set returns rows with columns grouped by in other sets set to NULL. This could previously be achieved by doing each grouping as a separate query, conjoined by UNION ALLs. Besides being considerably more concise, grouping sets will in many cases be faster, requiring only one scan over the underlying data. The current implementation of grouping sets only supports using sorting for input. Individual sets that share a sort order are computed in one pass. If there are sets that don't share a sort order, additional sort & aggregation steps are performed. These additional passes are sourced by the previous sort step; thus avoiding repeated scans of the source data. The code is structured in a way that adding support for purely using hash aggregation or a mix of hashing and sorting is possible. Sorting was chosen to be supported first, as it is the most generic method of implementation. Instead of, as in an earlier versions of the patch, representing the chain of sort and aggregation steps as full blown planner and executor nodes, all but the first sort are performed inside the aggregation node itself. This avoids the need to do some unusual gymnastics to handle having to return aggregated and non-aggregated tuples from underlying nodes, as well as having to shut down underlying nodes early to limit memory usage. The optimizer still builds Sort/Agg node to describe each phase, but they're not part of the plan tree, but instead additional data for the aggregation node. They're a convenient and preexisting way to describe aggregation and sorting. The first (and possibly only) sort step is still performed as a separate execution step. That retains similarity with existing group by plans, makes rescans fairly simple, avoids very deep plans (leading to slow explains) and easily allows to avoid the sorting step if the underlying data is sorted by other means. A somewhat ugly side of this patch is having to deal with a grammar ambiguity between the new CUBE keyword and the cube extension/functions named cube (and rollup). To avoid breaking existing deployments of the cube extension it has not been renamed, neither has cube been made a reserved keyword. Instead precedence hacking is used to make GROUP BY cube(..) refer to the CUBE grouping sets feature, and not the function cube(). To actually group by a function cube(), unlikely as that might be, the function name has to be quoted. Needs a catversion bump because stored rules may change. Author: Andrew Gierth and Atri Sharma, with contributions from Andres Freund Reviewed-By: Andres Freund, Noah Misch, Tom Lane, Svenne Krap, Tomas Vondra, Erik Rijkers, Marti Raudsepp, Pavel Stehule Discussion: CAOeZVidmVRe2jU6aMk_5qkxnB7dfmPROzM7Ur8JPW5j8Y5X-Lw@mail.gmail.com
2015-05-16 03:40:59 +02:00
static bool
_equalGroupingSet(const GroupingSet *a, const GroupingSet *b)
{
COMPARE_SCALAR_FIELD(kind);
COMPARE_NODE_FIELD(content);
COMPARE_LOCATION_FIELD(location);
return true;
}
static bool
_equalWindowClause(const WindowClause *a, const WindowClause *b)
{
COMPARE_STRING_FIELD(name);
COMPARE_STRING_FIELD(refname);
COMPARE_NODE_FIELD(partitionClause);
COMPARE_NODE_FIELD(orderClause);
COMPARE_SCALAR_FIELD(frameOptions);
COMPARE_NODE_FIELD(startOffset);
COMPARE_NODE_FIELD(endOffset);
Teach planner and executor about monotonic window funcs Window functions such as row_number() always return a value higher than the previously returned value for tuples in any given window partition. Traditionally queries such as; SELECT * FROM ( SELECT *, row_number() over (order by c) rn FROM t ) t WHERE rn <= 10; were executed fairly inefficiently. Neither the query planner nor the executor knew that once rn made it to 11 that nothing further would match the outer query's WHERE clause. It would blindly continue until all tuples were exhausted from the subquery. Here we implement means to make the above execute more efficiently. This is done by way of adding a pg_proc.prosupport function to various of the built-in window functions and adding supporting code to allow the support function to inform the planner if the window function is monotonically increasing, monotonically decreasing, both or neither. The planner is then able to make use of that information and possibly allow the executor to short-circuit execution by way of adding a "run condition" to the WindowAgg to allow it to determine if some of its execution work can be skipped. This "run condition" is not like a normal filter. These run conditions are only built using quals comparing values to monotonic window functions. For monotonic increasing functions, quals making use of the btree operators for <, <= and = can be used (assuming the window function column is on the left). You can see here that once such a condition becomes false that a monotonic increasing function could never make it subsequently true again. For monotonically decreasing functions the >, >= and = btree operators for the given type can be used for run conditions. The best-case situation for this is when there is a single WindowAgg node without a PARTITION BY clause. Here when the run condition becomes false the WindowAgg node can simply return NULL. No more tuples will ever match the run condition. It's a little more complex when there is a PARTITION BY clause. In this case, we cannot return NULL as we must still process other partitions. To speed this case up we pull tuples from the outer plan to check if they're from the same partition and simply discard them if they are. When we find a tuple belonging to another partition we start processing as normal again until the run condition becomes false or we run out of tuples to process. When there are multiple WindowAgg nodes to evaluate then this complicates the situation. For intermediate WindowAggs we must ensure we always return all tuples to the calling node. Any filtering done could lead to incorrect results in WindowAgg nodes above. For all intermediate nodes, we can still save some work when the run condition becomes false. We've no need to evaluate the WindowFuncs anymore. Other WindowAgg nodes cannot reference the value of these and these tuples will not appear in the final result anyway. The savings here are small in comparison to what can be saved in the top-level WingowAgg, but still worthwhile. Intermediate WindowAgg nodes never filter out tuples, but here we change WindowAgg so that the top-level WindowAgg filters out tuples that don't match the intermediate WindowAgg node's run condition. Such filters appear in the "Filter" clause in EXPLAIN for the top-level WindowAgg node. Here we add prosupport functions to allow the above to work for; row_number(), rank(), dense_rank(), count(*) and count(expr). It appears technically possible to do the same for min() and max(), however, it seems unlikely to be useful enough, so that's not done here. Bump catversion Author: David Rowley Reviewed-by: Andy Fan, Zhihong Yu Discussion: https://postgr.es/m/CAApHDvqvp3At8++yF8ij06sdcoo1S_b2YoaT9D4Nf+MObzsrLQ@mail.gmail.com
2022-04-08 00:34:36 +02:00
COMPARE_NODE_FIELD(runCondition);
Support all SQL:2011 options for window frame clauses. This patch adds the ability to use "RANGE offset PRECEDING/FOLLOWING" frame boundaries in window functions. We'd punted on that back in the original patch to add window functions, because it was not clear how to do it in a reasonably data-type-extensible fashion. That problem is resolved here by adding the ability for btree operator classes to provide an "in_range" support function that defines how to add or subtract the RANGE offset value. Factoring it this way also allows the operator class to avoid overflow problems near the ends of the datatype's range, if it wishes to expend effort on that. (In the committed patch, the integer opclasses handle that issue, but it did not seem worth the trouble to avoid overflow failures for datetime types.) The patch includes in_range support for the integer_ops opfamily (int2/int4/int8) as well as the standard datetime types. Support for other numeric types has been requested, but that seems like suitable material for a follow-on patch. In addition, the patch adds GROUPS mode which counts the offset in ORDER-BY peer groups rather than rows, and it adds the frame_exclusion options specified by SQL:2011. As far as I can see, we are now fully up to spec on window framing options. Existing behaviors remain unchanged, except that I changed the errcode for a couple of existing error reports to meet the SQL spec's expectation that negative "offset" values should be reported as SQLSTATE 22013. Internally and in relevant parts of the documentation, we now consistently use the terminology "offset PRECEDING/FOLLOWING" rather than "value PRECEDING/FOLLOWING", since the term "value" is confusingly vague. Oliver Ford, reviewed and whacked around some by me Discussion: https://postgr.es/m/CAGMVOdu9sivPAxbNN0X+q19Sfv9edEPv=HibOJhB14TJv_RCQg@mail.gmail.com
2018-02-07 06:06:50 +01:00
COMPARE_SCALAR_FIELD(startInRangeFunc);
COMPARE_SCALAR_FIELD(endInRangeFunc);
COMPARE_SCALAR_FIELD(inRangeColl);
COMPARE_SCALAR_FIELD(inRangeAsc);
COMPARE_SCALAR_FIELD(inRangeNullsFirst);
COMPARE_SCALAR_FIELD(winref);
COMPARE_SCALAR_FIELD(copiedOrder);
return true;
}
static bool
_equalRowMarkClause(const RowMarkClause *a, const RowMarkClause *b)
{
COMPARE_SCALAR_FIELD(rti);
Improve concurrency of foreign key locking This patch introduces two additional lock modes for tuples: "SELECT FOR KEY SHARE" and "SELECT FOR NO KEY UPDATE". These don't block each other, in contrast with already existing "SELECT FOR SHARE" and "SELECT FOR UPDATE". UPDATE commands that do not modify the values stored in the columns that are part of the key of the tuple now grab a SELECT FOR NO KEY UPDATE lock on the tuple, allowing them to proceed concurrently with tuple locks of the FOR KEY SHARE variety. Foreign key triggers now use FOR KEY SHARE instead of FOR SHARE; this means the concurrency improvement applies to them, which is the whole point of this patch. The added tuple lock semantics require some rejiggering of the multixact module, so that the locking level that each transaction is holding can be stored alongside its Xid. Also, multixacts now need to persist across server restarts and crashes, because they can now represent not only tuple locks, but also tuple updates. This means we need more careful tracking of lifetime of pg_multixact SLRU files; since they now persist longer, we require more infrastructure to figure out when they can be removed. pg_upgrade also needs to be careful to copy pg_multixact files over from the old server to the new, or at least part of multixact.c state, depending on the versions of the old and new servers. Tuple time qualification rules (HeapTupleSatisfies routines) need to be careful not to consider tuples with the "is multi" infomask bit set as being only locked; they might need to look up MultiXact values (i.e. possibly do pg_multixact I/O) to find out the Xid that updated a tuple, whereas they previously were assured to only use information readily available from the tuple header. This is considered acceptable, because the extra I/O would involve cases that would previously cause some commands to block waiting for concurrent transactions to finish. Another important change is the fact that locking tuples that have previously been updated causes the future versions to be marked as locked, too; this is essential for correctness of foreign key checks. This causes additional WAL-logging, also (there was previously a single WAL record for a locked tuple; now there are as many as updated copies of the tuple there exist.) With all this in place, contention related to tuples being checked by foreign key rules should be much reduced. As a bonus, the old behavior that a subtransaction grabbing a stronger tuple lock than the parent (sub)transaction held on a given tuple and later aborting caused the weaker lock to be lost, has been fixed. Many new spec files were added for isolation tester framework, to ensure overall behavior is sane. There's probably room for several more tests. There were several reviewers of this patch; in particular, Noah Misch and Andres Freund spent considerable time in it. Original idea for the patch came from Simon Riggs, after a problem report by Joel Jacobson. Most code is from me, with contributions from Marti Raudsepp, Alexander Shulgin, Noah Misch and Andres Freund. This patch was discussed in several pgsql-hackers threads; the most important start at the following message-ids: AANLkTimo9XVcEzfiBR-ut3KVNDkjm2Vxh+t8kAmWjPuv@mail.gmail.com 1290721684-sup-3951@alvh.no-ip.org 1294953201-sup-2099@alvh.no-ip.org 1320343602-sup-2290@alvh.no-ip.org 1339690386-sup-8927@alvh.no-ip.org 4FE5FF020200002500048A3D@gw.wicourts.gov 4FEAB90A0200002500048B7D@gw.wicourts.gov
2013-01-23 16:04:59 +01:00
COMPARE_SCALAR_FIELD(strength);
COMPARE_SCALAR_FIELD(waitPolicy);
COMPARE_SCALAR_FIELD(pushedDown);
return true;
}
static bool
_equalWithClause(const WithClause *a, const WithClause *b)
{
COMPARE_NODE_FIELD(ctes);
COMPARE_SCALAR_FIELD(recursive);
COMPARE_LOCATION_FIELD(location);
return true;
}
Add support for INSERT ... ON CONFLICT DO NOTHING/UPDATE. The newly added ON CONFLICT clause allows to specify an alternative to raising a unique or exclusion constraint violation error when inserting. ON CONFLICT refers to constraints that can either be specified using a inference clause (by specifying the columns of a unique constraint) or by naming a unique or exclusion constraint. DO NOTHING avoids the constraint violation, without touching the pre-existing row. DO UPDATE SET ... [WHERE ...] updates the pre-existing tuple, and has access to both the tuple proposed for insertion and the existing tuple; the optional WHERE clause can be used to prevent an update from being executed. The UPDATE SET and WHERE clauses have access to the tuple proposed for insertion using the "magic" EXCLUDED alias, and to the pre-existing tuple using the table name or its alias. This feature is often referred to as upsert. This is implemented using a new infrastructure called "speculative insertion". It is an optimistic variant of regular insertion that first does a pre-check for existing tuples and then attempts an insert. If a violating tuple was inserted concurrently, the speculatively inserted tuple is deleted and a new attempt is made. If the pre-check finds a matching tuple the alternative DO NOTHING or DO UPDATE action is taken. If the insertion succeeds without detecting a conflict, the tuple is deemed inserted. To handle the possible ambiguity between the excluded alias and a table named excluded, and for convenience with long relation names, INSERT INTO now can alias its target table. Bumps catversion as stored rules change. Author: Peter Geoghegan, with significant contributions from Heikki Linnakangas and Andres Freund. Testing infrastructure by Jeff Janes. Reviewed-By: Heikki Linnakangas, Andres Freund, Robert Haas, Simon Riggs, Dean Rasheed, Stephen Frost and many others.
2015-05-08 05:31:36 +02:00
static bool
_equalInferClause(const InferClause *a, const InferClause *b)
{
COMPARE_NODE_FIELD(indexElems);
COMPARE_NODE_FIELD(whereClause);
COMPARE_STRING_FIELD(conname);
COMPARE_LOCATION_FIELD(location);
return true;
}
static bool
_equalOnConflictClause(const OnConflictClause *a, const OnConflictClause *b)
{
COMPARE_SCALAR_FIELD(action);
COMPARE_NODE_FIELD(infer);
COMPARE_NODE_FIELD(targetList);
COMPARE_NODE_FIELD(whereClause);
COMPARE_LOCATION_FIELD(location);
return true;
}
static bool
_equalCTESearchClause(const CTESearchClause *a, const CTESearchClause *b)
{
COMPARE_NODE_FIELD(search_col_list);
COMPARE_SCALAR_FIELD(search_breadth_first);
COMPARE_STRING_FIELD(search_seq_column);
COMPARE_LOCATION_FIELD(location);
return true;
}
static bool
_equalCTECycleClause(const CTECycleClause *a, const CTECycleClause *b)
{
COMPARE_NODE_FIELD(cycle_col_list);
COMPARE_STRING_FIELD(cycle_mark_column);
COMPARE_NODE_FIELD(cycle_mark_value);
COMPARE_NODE_FIELD(cycle_mark_default);
COMPARE_STRING_FIELD(cycle_path_column);
COMPARE_LOCATION_FIELD(location);
COMPARE_SCALAR_FIELD(cycle_mark_type);
COMPARE_SCALAR_FIELD(cycle_mark_typmod);
COMPARE_SCALAR_FIELD(cycle_mark_collation);
COMPARE_SCALAR_FIELD(cycle_mark_neop);
return true;
}
static bool
_equalCommonTableExpr(const CommonTableExpr *a, const CommonTableExpr *b)
{
COMPARE_STRING_FIELD(ctename);
COMPARE_NODE_FIELD(aliascolnames);
Allow user control of CTE materialization, and change the default behavior. Historically we've always materialized the full output of a CTE query, treating WITH as an optimization fence (so that, for example, restrictions from the outer query cannot be pushed into it). This is appropriate when the CTE query is INSERT/UPDATE/DELETE, or is recursive; but when the CTE query is non-recursive and side-effect-free, there's no hazard of changing the query results by pushing restrictions down. Another argument for materialization is that it can avoid duplicate computation of an expensive WITH query --- but that only applies if the WITH query is called more than once in the outer query. Even then it could still be a net loss, if each call has restrictions that would allow just a small part of the WITH query to be computed. Hence, let's change the behavior for WITH queries that are non-recursive and side-effect-free. By default, we will inline them into the outer query (removing the optimization fence) if they are called just once. If they are called more than once, we will keep the old behavior by default, but the user can override this and force inlining by specifying NOT MATERIALIZED. Lastly, the user can force the old behavior by specifying MATERIALIZED; this would mainly be useful when the query had deliberately been employing WITH as an optimization fence to prevent a poor choice of plan. Andreas Karlsson, Andrew Gierth, David Fetter Discussion: https://postgr.es/m/87sh48ffhb.fsf@news-spur.riddles.org.uk
2019-02-16 22:11:12 +01:00
COMPARE_SCALAR_FIELD(ctematerialized);
COMPARE_NODE_FIELD(ctequery);
COMPARE_NODE_FIELD(search_clause);
COMPARE_NODE_FIELD(cycle_clause);
COMPARE_LOCATION_FIELD(location);
COMPARE_SCALAR_FIELD(cterecursive);
COMPARE_SCALAR_FIELD(cterefcount);
COMPARE_NODE_FIELD(ctecolnames);
COMPARE_NODE_FIELD(ctecoltypes);
COMPARE_NODE_FIELD(ctecoltypmods);
COMPARE_NODE_FIELD(ctecolcollations);
return true;
}
Add support for MERGE SQL command MERGE performs actions that modify rows in the target table using a source table or query. MERGE provides a single SQL statement that can conditionally INSERT/UPDATE/DELETE rows -- a task that would otherwise require multiple PL statements. For example, MERGE INTO target AS t USING source AS s ON t.tid = s.sid WHEN MATCHED AND t.balance > s.delta THEN UPDATE SET balance = t.balance - s.delta WHEN MATCHED THEN DELETE WHEN NOT MATCHED AND s.delta > 0 THEN INSERT VALUES (s.sid, s.delta) WHEN NOT MATCHED THEN DO NOTHING; MERGE works with regular tables, partitioned tables and inheritance hierarchies, including column and row security enforcement, as well as support for row and statement triggers and transition tables therein. MERGE is optimized for OLTP and is parameterizable, though also useful for large scale ETL/ELT. MERGE is not intended to be used in preference to existing single SQL commands for INSERT, UPDATE or DELETE since there is some overhead. MERGE can be used from PL/pgSQL. MERGE does not support targetting updatable views or foreign tables, and RETURNING clauses are not allowed either. These limitations are likely fixable with sufficient effort. Rewrite rules are also not supported, but it's not clear that we'd want to support them. Author: Pavan Deolasee <pavan.deolasee@gmail.com> Author: Álvaro Herrera <alvherre@alvh.no-ip.org> Author: Amit Langote <amitlangote09@gmail.com> Author: Simon Riggs <simon.riggs@enterprisedb.com> Reviewed-by: Peter Eisentraut <peter.eisentraut@enterprisedb.com> Reviewed-by: Andres Freund <andres@anarazel.de> (earlier versions) Reviewed-by: Peter Geoghegan <pg@bowt.ie> (earlier versions) Reviewed-by: Robert Haas <robertmhaas@gmail.com> (earlier versions) Reviewed-by: Japin Li <japinli@hotmail.com> Reviewed-by: Justin Pryzby <pryzby@telsasoft.com> Reviewed-by: Tomas Vondra <tomas.vondra@enterprisedb.com> Reviewed-by: Zhihong Yu <zyu@yugabyte.com> Discussion: https://postgr.es/m/CANP8+jKitBSrB7oTgT9CY2i1ObfOt36z0XMraQc+Xrz8QB0nXA@mail.gmail.com Discussion: https://postgr.es/m/CAH2-WzkJdBuxj9PO=2QaO9-3h3xGbQPZ34kJH=HukRekwM-GZg@mail.gmail.com Discussion: https://postgr.es/m/20201231134736.GA25392@alvherre.pgsql
2022-03-28 16:45:58 +02:00
static bool
_equalMergeWhenClause(const MergeWhenClause *a, const MergeWhenClause *b)
{
COMPARE_SCALAR_FIELD(matched);
COMPARE_SCALAR_FIELD(commandType);
COMPARE_SCALAR_FIELD(override);
COMPARE_NODE_FIELD(condition);
COMPARE_NODE_FIELD(targetList);
COMPARE_NODE_FIELD(values);
return true;
}
static bool
_equalMergeAction(const MergeAction *a, const MergeAction *b)
{
COMPARE_SCALAR_FIELD(matched);
COMPARE_SCALAR_FIELD(commandType);
COMPARE_SCALAR_FIELD(override);
COMPARE_NODE_FIELD(qual);
COMPARE_NODE_FIELD(targetList);
COMPARE_NODE_FIELD(updateColnos);
return true;
}
static bool
_equalXmlSerialize(const XmlSerialize *a, const XmlSerialize *b)
{
COMPARE_SCALAR_FIELD(xmloption);
COMPARE_NODE_FIELD(expr);
COMPARE_NODE_FIELD(typeName);
COMPARE_LOCATION_FIELD(location);
return true;
}
static bool
_equalRoleSpec(const RoleSpec *a, const RoleSpec *b)
{
COMPARE_SCALAR_FIELD(roletype);
COMPARE_STRING_FIELD(rolename);
COMPARE_LOCATION_FIELD(location);
return true;
}
static bool
_equalTriggerTransition(const TriggerTransition *a, const TriggerTransition *b)
{
COMPARE_STRING_FIELD(name);
COMPARE_SCALAR_FIELD(isNew);
COMPARE_SCALAR_FIELD(isTable);
return true;
}
Implement table partitioning. Table partitioning is like table inheritance and reuses much of the existing infrastructure, but there are some important differences. The parent is called a partitioned table and is always empty; it may not have indexes or non-inherited constraints, since those make no sense for a relation with no data of its own. The children are called partitions and contain all of the actual data. Each partition has an implicit partitioning constraint. Multiple inheritance is not allowed, and partitioning and inheritance can't be mixed. Partitions can't have extra columns and may not allow nulls unless the parent does. Tuples inserted into the parent are automatically routed to the correct partition, so tuple-routing ON INSERT triggers are not needed. Tuple routing isn't yet supported for partitions which are foreign tables, and it doesn't handle updates that cross partition boundaries. Currently, tables can be range-partitioned or list-partitioned. List partitioning is limited to a single column, but range partitioning can involve multiple columns. A partitioning "column" can be an expression. Because table partitioning is less general than table inheritance, it is hoped that it will be easier to reason about properties of partitions, and therefore that this will serve as a better foundation for a variety of possible optimizations, including query planner optimizations. The tuple routing based which this patch does based on the implicit partitioning constraints is an example of this, but it seems likely that many other useful optimizations are also possible. Amit Langote, reviewed and tested by Robert Haas, Ashutosh Bapat, Amit Kapila, Rajkumar Raghuwanshi, Corey Huinker, Jaime Casanova, Rushabh Lathia, Erik Rijkers, among others. Minor revisions by me.
2016-12-07 19:17:43 +01:00
static bool
Code review focused on new node types added by partitioning support. Fix failure to check that we got a plain Const from const-simplification of a coercion request. This is the cause of bug #14666 from Tian Bing: there is an int4 to money cast, but it's only stable not immutable (because of dependence on lc_monetary), resulting in a FuncExpr that the code was miserably unequipped to deal with, or indeed even to notice that it was failing to deal with. Add test cases around this coercion behavior. In view of the above, sprinkle the code liberally with castNode() macros, in hope of catching the next such bug a bit sooner. Also, change some functions that were randomly declared to take Node* to take more specific pointer types. And change some struct fields that were declared Node* but could be given more specific types, allowing removal of assorted explicit casts. Place PARTITION_MAX_KEYS check a bit closer to the code it's protecting. Likewise check only-one-key-for-list-partitioning restriction in a less random place. Avoid not-per-project-style usages like !strcmp(...). Fix assorted failures to avoid scribbling on the input of parse transformation. I'm not sure how necessary this is, but it's entirely silly for these functions to be expending cycles to avoid that and not getting it right. Add guards against partitioning on system columns. Put backend/nodes/ support code into an order that matches handling of these node types elsewhere. Annotate the fact that somebody added location fields to PartitionBoundSpec and PartitionRangeDatum but forgot to handle them in outfuncs.c/readfuncs.c. This is fairly harmless for production purposes (since readfuncs.c would just substitute -1 anyway) but it's still bogus. It's not worth forcing a post-beta1 initdb just to fix this, but if we have another reason to force initdb before 10.0, we should go back and clean this up. Contrariwise, somebody added location fields to PartitionElem and PartitionSpec but forgot to teach exprLocation() about them. Consolidate duplicative code in transformPartitionBound(). Improve a couple of error messages. Improve assorted commentary. Re-pgindent the files touched by this patch; this affects a few comment blocks that must have been added quite recently. Report: https://postgr.es/m/20170524024550.29935.14396@wrigleys.postgresql.org
2017-05-29 05:20:28 +02:00
_equalPartitionElem(const PartitionElem *a, const PartitionElem *b)
Implement table partitioning. Table partitioning is like table inheritance and reuses much of the existing infrastructure, but there are some important differences. The parent is called a partitioned table and is always empty; it may not have indexes or non-inherited constraints, since those make no sense for a relation with no data of its own. The children are called partitions and contain all of the actual data. Each partition has an implicit partitioning constraint. Multiple inheritance is not allowed, and partitioning and inheritance can't be mixed. Partitions can't have extra columns and may not allow nulls unless the parent does. Tuples inserted into the parent are automatically routed to the correct partition, so tuple-routing ON INSERT triggers are not needed. Tuple routing isn't yet supported for partitions which are foreign tables, and it doesn't handle updates that cross partition boundaries. Currently, tables can be range-partitioned or list-partitioned. List partitioning is limited to a single column, but range partitioning can involve multiple columns. A partitioning "column" can be an expression. Because table partitioning is less general than table inheritance, it is hoped that it will be easier to reason about properties of partitions, and therefore that this will serve as a better foundation for a variety of possible optimizations, including query planner optimizations. The tuple routing based which this patch does based on the implicit partitioning constraints is an example of this, but it seems likely that many other useful optimizations are also possible. Amit Langote, reviewed and tested by Robert Haas, Ashutosh Bapat, Amit Kapila, Rajkumar Raghuwanshi, Corey Huinker, Jaime Casanova, Rushabh Lathia, Erik Rijkers, among others. Minor revisions by me.
2016-12-07 19:17:43 +01:00
{
Code review focused on new node types added by partitioning support. Fix failure to check that we got a plain Const from const-simplification of a coercion request. This is the cause of bug #14666 from Tian Bing: there is an int4 to money cast, but it's only stable not immutable (because of dependence on lc_monetary), resulting in a FuncExpr that the code was miserably unequipped to deal with, or indeed even to notice that it was failing to deal with. Add test cases around this coercion behavior. In view of the above, sprinkle the code liberally with castNode() macros, in hope of catching the next such bug a bit sooner. Also, change some functions that were randomly declared to take Node* to take more specific pointer types. And change some struct fields that were declared Node* but could be given more specific types, allowing removal of assorted explicit casts. Place PARTITION_MAX_KEYS check a bit closer to the code it's protecting. Likewise check only-one-key-for-list-partitioning restriction in a less random place. Avoid not-per-project-style usages like !strcmp(...). Fix assorted failures to avoid scribbling on the input of parse transformation. I'm not sure how necessary this is, but it's entirely silly for these functions to be expending cycles to avoid that and not getting it right. Add guards against partitioning on system columns. Put backend/nodes/ support code into an order that matches handling of these node types elsewhere. Annotate the fact that somebody added location fields to PartitionBoundSpec and PartitionRangeDatum but forgot to handle them in outfuncs.c/readfuncs.c. This is fairly harmless for production purposes (since readfuncs.c would just substitute -1 anyway) but it's still bogus. It's not worth forcing a post-beta1 initdb just to fix this, but if we have another reason to force initdb before 10.0, we should go back and clean this up. Contrariwise, somebody added location fields to PartitionElem and PartitionSpec but forgot to teach exprLocation() about them. Consolidate duplicative code in transformPartitionBound(). Improve a couple of error messages. Improve assorted commentary. Re-pgindent the files touched by this patch; this affects a few comment blocks that must have been added quite recently. Report: https://postgr.es/m/20170524024550.29935.14396@wrigleys.postgresql.org
2017-05-29 05:20:28 +02:00
COMPARE_STRING_FIELD(name);
COMPARE_NODE_FIELD(expr);
COMPARE_NODE_FIELD(collation);
COMPARE_NODE_FIELD(opclass);
Implement table partitioning. Table partitioning is like table inheritance and reuses much of the existing infrastructure, but there are some important differences. The parent is called a partitioned table and is always empty; it may not have indexes or non-inherited constraints, since those make no sense for a relation with no data of its own. The children are called partitions and contain all of the actual data. Each partition has an implicit partitioning constraint. Multiple inheritance is not allowed, and partitioning and inheritance can't be mixed. Partitions can't have extra columns and may not allow nulls unless the parent does. Tuples inserted into the parent are automatically routed to the correct partition, so tuple-routing ON INSERT triggers are not needed. Tuple routing isn't yet supported for partitions which are foreign tables, and it doesn't handle updates that cross partition boundaries. Currently, tables can be range-partitioned or list-partitioned. List partitioning is limited to a single column, but range partitioning can involve multiple columns. A partitioning "column" can be an expression. Because table partitioning is less general than table inheritance, it is hoped that it will be easier to reason about properties of partitions, and therefore that this will serve as a better foundation for a variety of possible optimizations, including query planner optimizations. The tuple routing based which this patch does based on the implicit partitioning constraints is an example of this, but it seems likely that many other useful optimizations are also possible. Amit Langote, reviewed and tested by Robert Haas, Ashutosh Bapat, Amit Kapila, Rajkumar Raghuwanshi, Corey Huinker, Jaime Casanova, Rushabh Lathia, Erik Rijkers, among others. Minor revisions by me.
2016-12-07 19:17:43 +01:00
COMPARE_LOCATION_FIELD(location);
return true;
}
static bool
Code review focused on new node types added by partitioning support. Fix failure to check that we got a plain Const from const-simplification of a coercion request. This is the cause of bug #14666 from Tian Bing: there is an int4 to money cast, but it's only stable not immutable (because of dependence on lc_monetary), resulting in a FuncExpr that the code was miserably unequipped to deal with, or indeed even to notice that it was failing to deal with. Add test cases around this coercion behavior. In view of the above, sprinkle the code liberally with castNode() macros, in hope of catching the next such bug a bit sooner. Also, change some functions that were randomly declared to take Node* to take more specific pointer types. And change some struct fields that were declared Node* but could be given more specific types, allowing removal of assorted explicit casts. Place PARTITION_MAX_KEYS check a bit closer to the code it's protecting. Likewise check only-one-key-for-list-partitioning restriction in a less random place. Avoid not-per-project-style usages like !strcmp(...). Fix assorted failures to avoid scribbling on the input of parse transformation. I'm not sure how necessary this is, but it's entirely silly for these functions to be expending cycles to avoid that and not getting it right. Add guards against partitioning on system columns. Put backend/nodes/ support code into an order that matches handling of these node types elsewhere. Annotate the fact that somebody added location fields to PartitionBoundSpec and PartitionRangeDatum but forgot to handle them in outfuncs.c/readfuncs.c. This is fairly harmless for production purposes (since readfuncs.c would just substitute -1 anyway) but it's still bogus. It's not worth forcing a post-beta1 initdb just to fix this, but if we have another reason to force initdb before 10.0, we should go back and clean this up. Contrariwise, somebody added location fields to PartitionElem and PartitionSpec but forgot to teach exprLocation() about them. Consolidate duplicative code in transformPartitionBound(). Improve a couple of error messages. Improve assorted commentary. Re-pgindent the files touched by this patch; this affects a few comment blocks that must have been added quite recently. Report: https://postgr.es/m/20170524024550.29935.14396@wrigleys.postgresql.org
2017-05-29 05:20:28 +02:00
_equalPartitionSpec(const PartitionSpec *a, const PartitionSpec *b)
Implement table partitioning. Table partitioning is like table inheritance and reuses much of the existing infrastructure, but there are some important differences. The parent is called a partitioned table and is always empty; it may not have indexes or non-inherited constraints, since those make no sense for a relation with no data of its own. The children are called partitions and contain all of the actual data. Each partition has an implicit partitioning constraint. Multiple inheritance is not allowed, and partitioning and inheritance can't be mixed. Partitions can't have extra columns and may not allow nulls unless the parent does. Tuples inserted into the parent are automatically routed to the correct partition, so tuple-routing ON INSERT triggers are not needed. Tuple routing isn't yet supported for partitions which are foreign tables, and it doesn't handle updates that cross partition boundaries. Currently, tables can be range-partitioned or list-partitioned. List partitioning is limited to a single column, but range partitioning can involve multiple columns. A partitioning "column" can be an expression. Because table partitioning is less general than table inheritance, it is hoped that it will be easier to reason about properties of partitions, and therefore that this will serve as a better foundation for a variety of possible optimizations, including query planner optimizations. The tuple routing based which this patch does based on the implicit partitioning constraints is an example of this, but it seems likely that many other useful optimizations are also possible. Amit Langote, reviewed and tested by Robert Haas, Ashutosh Bapat, Amit Kapila, Rajkumar Raghuwanshi, Corey Huinker, Jaime Casanova, Rushabh Lathia, Erik Rijkers, among others. Minor revisions by me.
2016-12-07 19:17:43 +01:00
{
Code review focused on new node types added by partitioning support. Fix failure to check that we got a plain Const from const-simplification of a coercion request. This is the cause of bug #14666 from Tian Bing: there is an int4 to money cast, but it's only stable not immutable (because of dependence on lc_monetary), resulting in a FuncExpr that the code was miserably unequipped to deal with, or indeed even to notice that it was failing to deal with. Add test cases around this coercion behavior. In view of the above, sprinkle the code liberally with castNode() macros, in hope of catching the next such bug a bit sooner. Also, change some functions that were randomly declared to take Node* to take more specific pointer types. And change some struct fields that were declared Node* but could be given more specific types, allowing removal of assorted explicit casts. Place PARTITION_MAX_KEYS check a bit closer to the code it's protecting. Likewise check only-one-key-for-list-partitioning restriction in a less random place. Avoid not-per-project-style usages like !strcmp(...). Fix assorted failures to avoid scribbling on the input of parse transformation. I'm not sure how necessary this is, but it's entirely silly for these functions to be expending cycles to avoid that and not getting it right. Add guards against partitioning on system columns. Put backend/nodes/ support code into an order that matches handling of these node types elsewhere. Annotate the fact that somebody added location fields to PartitionBoundSpec and PartitionRangeDatum but forgot to handle them in outfuncs.c/readfuncs.c. This is fairly harmless for production purposes (since readfuncs.c would just substitute -1 anyway) but it's still bogus. It's not worth forcing a post-beta1 initdb just to fix this, but if we have another reason to force initdb before 10.0, we should go back and clean this up. Contrariwise, somebody added location fields to PartitionElem and PartitionSpec but forgot to teach exprLocation() about them. Consolidate duplicative code in transformPartitionBound(). Improve a couple of error messages. Improve assorted commentary. Re-pgindent the files touched by this patch; this affects a few comment blocks that must have been added quite recently. Report: https://postgr.es/m/20170524024550.29935.14396@wrigleys.postgresql.org
2017-05-29 05:20:28 +02:00
COMPARE_STRING_FIELD(strategy);
COMPARE_NODE_FIELD(partParams);
Implement table partitioning. Table partitioning is like table inheritance and reuses much of the existing infrastructure, but there are some important differences. The parent is called a partitioned table and is always empty; it may not have indexes or non-inherited constraints, since those make no sense for a relation with no data of its own. The children are called partitions and contain all of the actual data. Each partition has an implicit partitioning constraint. Multiple inheritance is not allowed, and partitioning and inheritance can't be mixed. Partitions can't have extra columns and may not allow nulls unless the parent does. Tuples inserted into the parent are automatically routed to the correct partition, so tuple-routing ON INSERT triggers are not needed. Tuple routing isn't yet supported for partitions which are foreign tables, and it doesn't handle updates that cross partition boundaries. Currently, tables can be range-partitioned or list-partitioned. List partitioning is limited to a single column, but range partitioning can involve multiple columns. A partitioning "column" can be an expression. Because table partitioning is less general than table inheritance, it is hoped that it will be easier to reason about properties of partitions, and therefore that this will serve as a better foundation for a variety of possible optimizations, including query planner optimizations. The tuple routing based which this patch does based on the implicit partitioning constraints is an example of this, but it seems likely that many other useful optimizations are also possible. Amit Langote, reviewed and tested by Robert Haas, Ashutosh Bapat, Amit Kapila, Rajkumar Raghuwanshi, Corey Huinker, Jaime Casanova, Rushabh Lathia, Erik Rijkers, among others. Minor revisions by me.
2016-12-07 19:17:43 +01:00
COMPARE_LOCATION_FIELD(location);
return true;
}
static bool
_equalPartitionBoundSpec(const PartitionBoundSpec *a, const PartitionBoundSpec *b)
{
COMPARE_SCALAR_FIELD(strategy);
COMPARE_SCALAR_FIELD(is_default);
COMPARE_SCALAR_FIELD(modulus);
COMPARE_SCALAR_FIELD(remainder);
Implement table partitioning. Table partitioning is like table inheritance and reuses much of the existing infrastructure, but there are some important differences. The parent is called a partitioned table and is always empty; it may not have indexes or non-inherited constraints, since those make no sense for a relation with no data of its own. The children are called partitions and contain all of the actual data. Each partition has an implicit partitioning constraint. Multiple inheritance is not allowed, and partitioning and inheritance can't be mixed. Partitions can't have extra columns and may not allow nulls unless the parent does. Tuples inserted into the parent are automatically routed to the correct partition, so tuple-routing ON INSERT triggers are not needed. Tuple routing isn't yet supported for partitions which are foreign tables, and it doesn't handle updates that cross partition boundaries. Currently, tables can be range-partitioned or list-partitioned. List partitioning is limited to a single column, but range partitioning can involve multiple columns. A partitioning "column" can be an expression. Because table partitioning is less general than table inheritance, it is hoped that it will be easier to reason about properties of partitions, and therefore that this will serve as a better foundation for a variety of possible optimizations, including query planner optimizations. The tuple routing based which this patch does based on the implicit partitioning constraints is an example of this, but it seems likely that many other useful optimizations are also possible. Amit Langote, reviewed and tested by Robert Haas, Ashutosh Bapat, Amit Kapila, Rajkumar Raghuwanshi, Corey Huinker, Jaime Casanova, Rushabh Lathia, Erik Rijkers, among others. Minor revisions by me.
2016-12-07 19:17:43 +01:00
COMPARE_NODE_FIELD(listdatums);
COMPARE_NODE_FIELD(lowerdatums);
COMPARE_NODE_FIELD(upperdatums);
COMPARE_LOCATION_FIELD(location);
return true;
}
static bool
_equalPartitionRangeDatum(const PartitionRangeDatum *a, const PartitionRangeDatum *b)
{
COMPARE_SCALAR_FIELD(kind);
Implement table partitioning. Table partitioning is like table inheritance and reuses much of the existing infrastructure, but there are some important differences. The parent is called a partitioned table and is always empty; it may not have indexes or non-inherited constraints, since those make no sense for a relation with no data of its own. The children are called partitions and contain all of the actual data. Each partition has an implicit partitioning constraint. Multiple inheritance is not allowed, and partitioning and inheritance can't be mixed. Partitions can't have extra columns and may not allow nulls unless the parent does. Tuples inserted into the parent are automatically routed to the correct partition, so tuple-routing ON INSERT triggers are not needed. Tuple routing isn't yet supported for partitions which are foreign tables, and it doesn't handle updates that cross partition boundaries. Currently, tables can be range-partitioned or list-partitioned. List partitioning is limited to a single column, but range partitioning can involve multiple columns. A partitioning "column" can be an expression. Because table partitioning is less general than table inheritance, it is hoped that it will be easier to reason about properties of partitions, and therefore that this will serve as a better foundation for a variety of possible optimizations, including query planner optimizations. The tuple routing based which this patch does based on the implicit partitioning constraints is an example of this, but it seems likely that many other useful optimizations are also possible. Amit Langote, reviewed and tested by Robert Haas, Ashutosh Bapat, Amit Kapila, Rajkumar Raghuwanshi, Corey Huinker, Jaime Casanova, Rushabh Lathia, Erik Rijkers, among others. Minor revisions by me.
2016-12-07 19:17:43 +01:00
COMPARE_NODE_FIELD(value);
COMPARE_LOCATION_FIELD(location);
return true;
}
static bool
_equalPartitionCmd(const PartitionCmd *a, const PartitionCmd *b)
{
COMPARE_NODE_FIELD(name);
COMPARE_NODE_FIELD(bound);
ALTER TABLE ... DETACH PARTITION ... CONCURRENTLY Allow a partition be detached from its partitioned table without blocking concurrent queries, by running in two transactions and only requiring ShareUpdateExclusive in the partitioned table. Because it runs in two transactions, it cannot be used in a transaction block. This is the main reason to use dedicated syntax: so that users can choose to use the original mode if they need it. But also, it doesn't work when a default partition exists (because an exclusive lock would still need to be obtained on it, in order to change its partition constraint.) In case the second transaction is cancelled or a crash occurs, there's ALTER TABLE .. DETACH PARTITION .. FINALIZE, which executes the final steps. The main trick to make this work is the addition of column pg_inherits.inhdetachpending, initially false; can only be set true in the first part of this command. Once that is committed, concurrent transactions that use a PartitionDirectory will include or ignore partitions so marked: in optimizer they are ignored if the row is marked committed for the snapshot; in executor they are always included. As a result, and because of the way PartitionDirectory caches partition descriptors, queries that were planned before the detach will see the rows in the detached partition and queries that are planned after the detach, won't. A CHECK constraint is created that duplicates the partition constraint. This is probably not strictly necessary, and some users will prefer to remove it afterwards, but if the partition is re-attached to a partitioned table, the constraint needn't be rechecked. Author: Álvaro Herrera <alvherre@alvh.no-ip.org> Reviewed-by: Amit Langote <amitlangote09@gmail.com> Reviewed-by: Justin Pryzby <pryzby@telsasoft.com> Discussion: https://postgr.es/m/20200803234854.GA24158@alvherre.pgsql
2021-03-25 22:00:28 +01:00
COMPARE_SCALAR_FIELD(concurrent);
Implement table partitioning. Table partitioning is like table inheritance and reuses much of the existing infrastructure, but there are some important differences. The parent is called a partitioned table and is always empty; it may not have indexes or non-inherited constraints, since those make no sense for a relation with no data of its own. The children are called partitions and contain all of the actual data. Each partition has an implicit partitioning constraint. Multiple inheritance is not allowed, and partitioning and inheritance can't be mixed. Partitions can't have extra columns and may not allow nulls unless the parent does. Tuples inserted into the parent are automatically routed to the correct partition, so tuple-routing ON INSERT triggers are not needed. Tuple routing isn't yet supported for partitions which are foreign tables, and it doesn't handle updates that cross partition boundaries. Currently, tables can be range-partitioned or list-partitioned. List partitioning is limited to a single column, but range partitioning can involve multiple columns. A partitioning "column" can be an expression. Because table partitioning is less general than table inheritance, it is hoped that it will be easier to reason about properties of partitions, and therefore that this will serve as a better foundation for a variety of possible optimizations, including query planner optimizations. The tuple routing based which this patch does based on the implicit partitioning constraints is an example of this, but it seems likely that many other useful optimizations are also possible. Amit Langote, reviewed and tested by Robert Haas, Ashutosh Bapat, Amit Kapila, Rajkumar Raghuwanshi, Corey Huinker, Jaime Casanova, Rushabh Lathia, Erik Rijkers, among others. Minor revisions by me.
2016-12-07 19:17:43 +01:00
return true;
}
/*
* Stuff from pg_list.h
*/
static bool
_equalList(const List *a, const List *b)
{
const ListCell *item_a;
const ListCell *item_b;
/*
* Try to reject by simple scalar checks before grovelling through all the
* list elements...
*/
COMPARE_SCALAR_FIELD(type);
COMPARE_SCALAR_FIELD(length);
/*
* We place the switch outside the loop for the sake of efficiency; this
* may not be worth doing...
*/
switch (a->type)
{
case T_List:
forboth(item_a, a, item_b, b)
{
if (!equal(lfirst(item_a), lfirst(item_b)))
return false;
}
break;
case T_IntList:
forboth(item_a, a, item_b, b)
{
if (lfirst_int(item_a) != lfirst_int(item_b))
return false;
}
break;
case T_OidList:
forboth(item_a, a, item_b, b)
{
if (lfirst_oid(item_a) != lfirst_oid(item_b))
return false;
}
break;
default:
elog(ERROR, "unrecognized list node type: %d",
(int) a->type);
return false; /* keep compiler quiet */
}
/*
* If we got here, we should have run out of elements of both lists
*/
Assert(item_a == NULL);
Assert(item_b == NULL);
return true;
}
/*
* Stuff from value.h
*/
static bool
_equalInteger(const Integer *a, const Integer *b)
{
COMPARE_SCALAR_FIELD(ival);
return true;
}
static bool
_equalFloat(const Float *a, const Float *b)
{
COMPARE_STRING_FIELD(fval);
return true;
}
static bool
_equalBoolean(const Boolean *a, const Boolean *b)
{
COMPARE_SCALAR_FIELD(boolval);
return true;
}
static bool
_equalString(const String *a, const String *b)
{
COMPARE_STRING_FIELD(sval);
return true;
}
static bool
_equalBitString(const BitString *a, const BitString *b)
{
COMPARE_STRING_FIELD(bsval);
return true;
}
/*
* equal
* returns whether two nodes are equal
*/
bool
equal(const void *a, const void *b)
{
bool retval;
if (a == b)
return true;
/*
* note that a!=b, so only one of them can be NULL
*/
if (a == NULL || b == NULL)
return false;
/*
* are they the same type of nodes?
*/
if (nodeTag(a) != nodeTag(b))
return false;
/* Guard against stack overflow due to overly complex expressions */
check_stack_depth();
switch (nodeTag(a))
{
/*
* PRIMITIVE NODES
*/
case T_Alias:
retval = _equalAlias(a, b);
break;
case T_RangeVar:
retval = _equalRangeVar(a, b);
break;
case T_TableFunc:
retval = _equalTableFunc(a, b);
break;
case T_IntoClause:
retval = _equalIntoClause(a, b);
break;
case T_Var:
retval = _equalVar(a, b);
break;
case T_Const:
retval = _equalConst(a, b);
break;
case T_Param:
retval = _equalParam(a, b);
break;
case T_Aggref:
retval = _equalAggref(a, b);
break;
Support GROUPING SETS, CUBE and ROLLUP. This SQL standard functionality allows to aggregate data by different GROUP BY clauses at once. Each grouping set returns rows with columns grouped by in other sets set to NULL. This could previously be achieved by doing each grouping as a separate query, conjoined by UNION ALLs. Besides being considerably more concise, grouping sets will in many cases be faster, requiring only one scan over the underlying data. The current implementation of grouping sets only supports using sorting for input. Individual sets that share a sort order are computed in one pass. If there are sets that don't share a sort order, additional sort & aggregation steps are performed. These additional passes are sourced by the previous sort step; thus avoiding repeated scans of the source data. The code is structured in a way that adding support for purely using hash aggregation or a mix of hashing and sorting is possible. Sorting was chosen to be supported first, as it is the most generic method of implementation. Instead of, as in an earlier versions of the patch, representing the chain of sort and aggregation steps as full blown planner and executor nodes, all but the first sort are performed inside the aggregation node itself. This avoids the need to do some unusual gymnastics to handle having to return aggregated and non-aggregated tuples from underlying nodes, as well as having to shut down underlying nodes early to limit memory usage. The optimizer still builds Sort/Agg node to describe each phase, but they're not part of the plan tree, but instead additional data for the aggregation node. They're a convenient and preexisting way to describe aggregation and sorting. The first (and possibly only) sort step is still performed as a separate execution step. That retains similarity with existing group by plans, makes rescans fairly simple, avoids very deep plans (leading to slow explains) and easily allows to avoid the sorting step if the underlying data is sorted by other means. A somewhat ugly side of this patch is having to deal with a grammar ambiguity between the new CUBE keyword and the cube extension/functions named cube (and rollup). To avoid breaking existing deployments of the cube extension it has not been renamed, neither has cube been made a reserved keyword. Instead precedence hacking is used to make GROUP BY cube(..) refer to the CUBE grouping sets feature, and not the function cube(). To actually group by a function cube(), unlikely as that might be, the function name has to be quoted. Needs a catversion bump because stored rules may change. Author: Andrew Gierth and Atri Sharma, with contributions from Andres Freund Reviewed-By: Andres Freund, Noah Misch, Tom Lane, Svenne Krap, Tomas Vondra, Erik Rijkers, Marti Raudsepp, Pavel Stehule Discussion: CAOeZVidmVRe2jU6aMk_5qkxnB7dfmPROzM7Ur8JPW5j8Y5X-Lw@mail.gmail.com
2015-05-16 03:40:59 +02:00
case T_GroupingFunc:
retval = _equalGroupingFunc(a, b);
break;
case T_WindowFunc:
retval = _equalWindowFunc(a, b);
break;
case T_SubscriptingRef:
retval = _equalSubscriptingRef(a, b);
break;
case T_FuncExpr:
retval = _equalFuncExpr(a, b);
break;
case T_NamedArgExpr:
retval = _equalNamedArgExpr(a, b);
break;
case T_OpExpr:
retval = _equalOpExpr(a, b);
break;
case T_DistinctExpr:
retval = _equalDistinctExpr(a, b);
break;
case T_NullIfExpr:
retval = _equalNullIfExpr(a, b);
break;
case T_ScalarArrayOpExpr:
retval = _equalScalarArrayOpExpr(a, b);
break;
case T_BoolExpr:
retval = _equalBoolExpr(a, b);
break;
case T_SubLink:
retval = _equalSubLink(a, b);
break;
case T_SubPlan:
retval = _equalSubPlan(a, b);
break;
case T_AlternativeSubPlan:
retval = _equalAlternativeSubPlan(a, b);
break;
case T_FieldSelect:
retval = _equalFieldSelect(a, b);
break;
case T_FieldStore:
retval = _equalFieldStore(a, b);
break;
case T_RelabelType:
retval = _equalRelabelType(a, b);
break;
case T_CoerceViaIO:
retval = _equalCoerceViaIO(a, b);
break;
case T_ArrayCoerceExpr:
retval = _equalArrayCoerceExpr(a, b);
break;
case T_ConvertRowtypeExpr:
retval = _equalConvertRowtypeExpr(a, b);
break;
case T_CollateExpr:
retval = _equalCollateExpr(a, b);
break;
case T_CaseExpr:
retval = _equalCaseExpr(a, b);
break;
case T_CaseWhen:
retval = _equalCaseWhen(a, b);
break;
case T_CaseTestExpr:
retval = _equalCaseTestExpr(a, b);
break;
case T_ArrayExpr:
retval = _equalArrayExpr(a, b);
break;
case T_RowExpr:
retval = _equalRowExpr(a, b);
break;
case T_RowCompareExpr:
retval = _equalRowCompareExpr(a, b);
break;
case T_CoalesceExpr:
retval = _equalCoalesceExpr(a, b);
break;
case T_MinMaxExpr:
retval = _equalMinMaxExpr(a, b);
break;
case T_SQLValueFunction:
retval = _equalSQLValueFunction(a, b);
break;
case T_XmlExpr:
retval = _equalXmlExpr(a, b);
break;
case T_NullTest:
retval = _equalNullTest(a, b);
break;
case T_BooleanTest:
retval = _equalBooleanTest(a, b);
break;
case T_CoerceToDomain:
retval = _equalCoerceToDomain(a, b);
break;
case T_CoerceToDomainValue:
retval = _equalCoerceToDomainValue(a, b);
break;
case T_SetToDefault:
retval = _equalSetToDefault(a, b);
break;
case T_CurrentOfExpr:
retval = _equalCurrentOfExpr(a, b);
break;
case T_NextValueExpr:
retval = _equalNextValueExpr(a, b);
break;
Add support for INSERT ... ON CONFLICT DO NOTHING/UPDATE. The newly added ON CONFLICT clause allows to specify an alternative to raising a unique or exclusion constraint violation error when inserting. ON CONFLICT refers to constraints that can either be specified using a inference clause (by specifying the columns of a unique constraint) or by naming a unique or exclusion constraint. DO NOTHING avoids the constraint violation, without touching the pre-existing row. DO UPDATE SET ... [WHERE ...] updates the pre-existing tuple, and has access to both the tuple proposed for insertion and the existing tuple; the optional WHERE clause can be used to prevent an update from being executed. The UPDATE SET and WHERE clauses have access to the tuple proposed for insertion using the "magic" EXCLUDED alias, and to the pre-existing tuple using the table name or its alias. This feature is often referred to as upsert. This is implemented using a new infrastructure called "speculative insertion". It is an optimistic variant of regular insertion that first does a pre-check for existing tuples and then attempts an insert. If a violating tuple was inserted concurrently, the speculatively inserted tuple is deleted and a new attempt is made. If the pre-check finds a matching tuple the alternative DO NOTHING or DO UPDATE action is taken. If the insertion succeeds without detecting a conflict, the tuple is deemed inserted. To handle the possible ambiguity between the excluded alias and a table named excluded, and for convenience with long relation names, INSERT INTO now can alias its target table. Bumps catversion as stored rules change. Author: Peter Geoghegan, with significant contributions from Heikki Linnakangas and Andres Freund. Testing infrastructure by Jeff Janes. Reviewed-By: Heikki Linnakangas, Andres Freund, Robert Haas, Simon Riggs, Dean Rasheed, Stephen Frost and many others.
2015-05-08 05:31:36 +02:00
case T_InferenceElem:
retval = _equalInferenceElem(a, b);
break;
case T_TargetEntry:
retval = _equalTargetEntry(a, b);
break;
case T_RangeTblRef:
retval = _equalRangeTblRef(a, b);
break;
case T_FromExpr:
retval = _equalFromExpr(a, b);
break;
Add support for INSERT ... ON CONFLICT DO NOTHING/UPDATE. The newly added ON CONFLICT clause allows to specify an alternative to raising a unique or exclusion constraint violation error when inserting. ON CONFLICT refers to constraints that can either be specified using a inference clause (by specifying the columns of a unique constraint) or by naming a unique or exclusion constraint. DO NOTHING avoids the constraint violation, without touching the pre-existing row. DO UPDATE SET ... [WHERE ...] updates the pre-existing tuple, and has access to both the tuple proposed for insertion and the existing tuple; the optional WHERE clause can be used to prevent an update from being executed. The UPDATE SET and WHERE clauses have access to the tuple proposed for insertion using the "magic" EXCLUDED alias, and to the pre-existing tuple using the table name or its alias. This feature is often referred to as upsert. This is implemented using a new infrastructure called "speculative insertion". It is an optimistic variant of regular insertion that first does a pre-check for existing tuples and then attempts an insert. If a violating tuple was inserted concurrently, the speculatively inserted tuple is deleted and a new attempt is made. If the pre-check finds a matching tuple the alternative DO NOTHING or DO UPDATE action is taken. If the insertion succeeds without detecting a conflict, the tuple is deemed inserted. To handle the possible ambiguity between the excluded alias and a table named excluded, and for convenience with long relation names, INSERT INTO now can alias its target table. Bumps catversion as stored rules change. Author: Peter Geoghegan, with significant contributions from Heikki Linnakangas and Andres Freund. Testing infrastructure by Jeff Janes. Reviewed-By: Heikki Linnakangas, Andres Freund, Robert Haas, Simon Riggs, Dean Rasheed, Stephen Frost and many others.
2015-05-08 05:31:36 +02:00
case T_OnConflictExpr:
retval = _equalOnConflictExpr(a, b);
break;
case T_JoinExpr:
retval = _equalJoinExpr(a, b);
break;
case T_JsonFormat:
retval = _equalJsonFormat(a, b);
break;
case T_JsonReturning:
retval = _equalJsonReturning(a, b);
break;
case T_JsonValueExpr:
retval = _equalJsonValueExpr(a, b);
break;
case T_JsonParseExpr:
retval = _equalJsonParseExpr(a, b);
break;
case T_JsonScalarExpr:
retval = _equalJsonScalarExpr(a, b);
break;
case T_JsonSerializeExpr:
retval = _equalJsonSerializeExpr(a, b);
break;
case T_JsonConstructorExpr:
retval = _equalJsonConstructorExpr(a, b);
break;
case T_JsonIsPredicate:
retval = _equalJsonIsPredicate(a, b);
break;
case T_JsonBehavior:
retval = _equalJsonBehavior(a, b);
break;
case T_JsonExpr:
retval = _equalJsonExpr(a, b);
break;
case T_JsonCoercion:
retval = _equalJsonCoercion(a, b);
break;
case T_JsonItemCoercions:
retval = _equalJsonItemCoercions(a, b);
break;
case T_JsonTableParent:
retval = _equalJsonTableParent(a, b);
break;
case T_JsonTableSibling:
retval = _equalJsonTableSibling(a, b);
break;
/*
* RELATION NODES
*/
case T_PathKey:
retval = _equalPathKey(a, b);
break;
case T_RestrictInfo:
retval = _equalRestrictInfo(a, b);
break;
case T_PlaceHolderVar:
retval = _equalPlaceHolderVar(a, b);
break;
case T_SpecialJoinInfo:
retval = _equalSpecialJoinInfo(a, b);
break;
case T_AppendRelInfo:
retval = _equalAppendRelInfo(a, b);
break;
case T_PlaceHolderInfo:
retval = _equalPlaceHolderInfo(a, b);
break;
case T_List:
case T_IntList:
case T_OidList:
retval = _equalList(a, b);
break;
case T_Integer:
retval = _equalInteger(a, b);
break;
case T_Float:
retval = _equalFloat(a, b);
break;
case T_Boolean:
retval = _equalBoolean(a, b);
break;
case T_String:
retval = _equalString(a, b);
break;
case T_BitString:
retval = _equalBitString(a, b);
break;
/*
* EXTENSIBLE NODES
*/
case T_ExtensibleNode:
retval = _equalExtensibleNode(a, b);
break;
/*
* PARSE NODES
*/
case T_Query:
retval = _equalQuery(a, b);
break;
Change representation of statement lists, and add statement location info. This patch makes several changes that improve the consistency of representation of lists of statements. It's always been the case that the output of parse analysis is a list of Query nodes, whatever the types of the individual statements in the list. This patch brings similar consistency to the outputs of raw parsing and planning steps: * The output of raw parsing is now always a list of RawStmt nodes; the statement-type-dependent nodes are one level down from that. * The output of pg_plan_queries() is now always a list of PlannedStmt nodes, even for utility statements. In the case of a utility statement, "planning" just consists of wrapping a CMD_UTILITY PlannedStmt around the utility node. This list representation is now used in Portal and CachedPlan plan lists, replacing the former convention of intermixing PlannedStmts with bare utility-statement nodes. Now, every list of statements has a consistent head-node type depending on how far along it is in processing. This allows changing many places that formerly used generic "Node *" pointers to use a more specific pointer type, thus reducing the number of IsA() tests and casts needed, as well as improving code clarity. Also, the post-parse-analysis representation of DECLARE CURSOR is changed so that it looks more like EXPLAIN, PREPARE, etc. That is, the contained SELECT remains a child of the DeclareCursorStmt rather than getting flipped around to be the other way. It's now true for both Query and PlannedStmt that utilityStmt is non-null if and only if commandType is CMD_UTILITY. That allows simplifying a lot of places that were testing both fields. (I think some of those were just defensive programming, but in many places, it was actually necessary to avoid confusing DECLARE CURSOR with SELECT.) Because PlannedStmt carries a canSetTag field, we're also able to get rid of some ad-hoc rules about how to reconstruct canSetTag for a bare utility statement; specifically, the assumption that a utility is canSetTag if and only if it's the only one in its list. While I see no near-term need for relaxing that restriction, it's nice to get rid of the ad-hocery. The API of ProcessUtility() is changed so that what it's passed is the wrapper PlannedStmt not just the bare utility statement. This will affect all users of ProcessUtility_hook, but the changes are pretty trivial; see the affected contrib modules for examples of the minimum change needed. (Most compilers should give pointer-type-mismatch warnings for uncorrected code.) There's also a change in the API of ExplainOneQuery_hook, to pass through cursorOptions instead of expecting hook functions to know what to pick. This is needed because of the DECLARE CURSOR changes, but really should have been done in 9.6; it's unlikely that any extant hook functions know about using CURSOR_OPT_PARALLEL_OK. Finally, teach gram.y to save statement boundary locations in RawStmt nodes, and pass those through to Query and PlannedStmt nodes. This allows more intelligent handling of cases where a source query string contains multiple statements. This patch doesn't actually do anything with the information, but a follow-on patch will. (Passing this information through cleanly is the true motivation for these changes; while I think this is all good cleanup, it's unlikely we'd have bothered without this end goal.) catversion bump because addition of location fields to struct Query affects stored rules. This patch is by me, but it owes a good deal to Fabien Coelho who did a lot of preliminary work on the problem, and also reviewed the patch. Discussion: https://postgr.es/m/alpine.DEB.2.20.1612200926310.29821@lancre
2017-01-14 22:02:35 +01:00
case T_RawStmt:
retval = _equalRawStmt(a, b);
break;
case T_InsertStmt:
retval = _equalInsertStmt(a, b);
break;
case T_DeleteStmt:
retval = _equalDeleteStmt(a, b);
break;
case T_UpdateStmt:
retval = _equalUpdateStmt(a, b);
break;
Add support for MERGE SQL command MERGE performs actions that modify rows in the target table using a source table or query. MERGE provides a single SQL statement that can conditionally INSERT/UPDATE/DELETE rows -- a task that would otherwise require multiple PL statements. For example, MERGE INTO target AS t USING source AS s ON t.tid = s.sid WHEN MATCHED AND t.balance > s.delta THEN UPDATE SET balance = t.balance - s.delta WHEN MATCHED THEN DELETE WHEN NOT MATCHED AND s.delta > 0 THEN INSERT VALUES (s.sid, s.delta) WHEN NOT MATCHED THEN DO NOTHING; MERGE works with regular tables, partitioned tables and inheritance hierarchies, including column and row security enforcement, as well as support for row and statement triggers and transition tables therein. MERGE is optimized for OLTP and is parameterizable, though also useful for large scale ETL/ELT. MERGE is not intended to be used in preference to existing single SQL commands for INSERT, UPDATE or DELETE since there is some overhead. MERGE can be used from PL/pgSQL. MERGE does not support targetting updatable views or foreign tables, and RETURNING clauses are not allowed either. These limitations are likely fixable with sufficient effort. Rewrite rules are also not supported, but it's not clear that we'd want to support them. Author: Pavan Deolasee <pavan.deolasee@gmail.com> Author: Álvaro Herrera <alvherre@alvh.no-ip.org> Author: Amit Langote <amitlangote09@gmail.com> Author: Simon Riggs <simon.riggs@enterprisedb.com> Reviewed-by: Peter Eisentraut <peter.eisentraut@enterprisedb.com> Reviewed-by: Andres Freund <andres@anarazel.de> (earlier versions) Reviewed-by: Peter Geoghegan <pg@bowt.ie> (earlier versions) Reviewed-by: Robert Haas <robertmhaas@gmail.com> (earlier versions) Reviewed-by: Japin Li <japinli@hotmail.com> Reviewed-by: Justin Pryzby <pryzby@telsasoft.com> Reviewed-by: Tomas Vondra <tomas.vondra@enterprisedb.com> Reviewed-by: Zhihong Yu <zyu@yugabyte.com> Discussion: https://postgr.es/m/CANP8+jKitBSrB7oTgT9CY2i1ObfOt36z0XMraQc+Xrz8QB0nXA@mail.gmail.com Discussion: https://postgr.es/m/CAH2-WzkJdBuxj9PO=2QaO9-3h3xGbQPZ34kJH=HukRekwM-GZg@mail.gmail.com Discussion: https://postgr.es/m/20201231134736.GA25392@alvherre.pgsql
2022-03-28 16:45:58 +02:00
case T_MergeStmt:
retval = _equalMergeStmt(a, b);
break;
case T_SelectStmt:
retval = _equalSelectStmt(a, b);
break;
case T_SetOperationStmt:
retval = _equalSetOperationStmt(a, b);
break;
SQL-standard function body This adds support for writing CREATE FUNCTION and CREATE PROCEDURE statements for language SQL with a function body that conforms to the SQL standard and is portable to other implementations. Instead of the PostgreSQL-specific AS $$ string literal $$ syntax, this allows writing out the SQL statements making up the body unquoted, either as a single statement: CREATE FUNCTION add(a integer, b integer) RETURNS integer LANGUAGE SQL RETURN a + b; or as a block CREATE PROCEDURE insert_data(a integer, b integer) LANGUAGE SQL BEGIN ATOMIC INSERT INTO tbl VALUES (a); INSERT INTO tbl VALUES (b); END; The function body is parsed at function definition time and stored as expression nodes in a new pg_proc column prosqlbody. So at run time, no further parsing is required. However, this form does not support polymorphic arguments, because there is no more parse analysis done at call time. Dependencies between the function and the objects it uses are fully tracked. A new RETURN statement is introduced. This can only be used inside function bodies. Internally, it is treated much like a SELECT statement. psql needs some new intelligence to keep track of function body boundaries so that it doesn't send off statements when it sees semicolons that are inside a function body. Tested-by: Jaime Casanova <jcasanov@systemguards.com.ec> Reviewed-by: Julien Rouhaud <rjuju123@gmail.com> Discussion: https://www.postgresql.org/message-id/flat/1c11f1eb-f00c-43b7-799d-2d44132c02d7@2ndquadrant.com
2021-04-07 21:30:08 +02:00
case T_ReturnStmt:
retval = _equalReturnStmt(a, b);
break;
case T_PLAssignStmt:
retval = _equalPLAssignStmt(a, b);
break;
case T_AlterTableStmt:
retval = _equalAlterTableStmt(a, b);
break;
case T_AlterTableCmd:
retval = _equalAlterTableCmd(a, b);
break;
case T_AlterCollationStmt:
retval = _equalAlterCollationStmt(a, b);
break;
2002-12-06 06:00:34 +01:00
case T_AlterDomainStmt:
retval = _equalAlterDomainStmt(a, b);
break;
case T_GrantStmt:
retval = _equalGrantStmt(a, b);
break;
case T_GrantRoleStmt:
retval = _equalGrantRoleStmt(a, b);
break;
case T_AlterDefaultPrivilegesStmt:
retval = _equalAlterDefaultPrivilegesStmt(a, b);
break;
case T_DeclareCursorStmt:
retval = _equalDeclareCursorStmt(a, b);
break;
case T_ClosePortalStmt:
retval = _equalClosePortalStmt(a, b);
break;
case T_CallStmt:
retval = _equalCallStmt(a, b);
break;
case T_ClusterStmt:
retval = _equalClusterStmt(a, b);
break;
case T_CopyStmt:
retval = _equalCopyStmt(a, b);
break;
case T_CreateStmt:
retval = _equalCreateStmt(a, b);
break;
case T_TableLikeClause:
retval = _equalTableLikeClause(a, b);
break;
case T_DefineStmt:
retval = _equalDefineStmt(a, b);
break;
case T_DropStmt:
retval = _equalDropStmt(a, b);
break;
case T_TruncateStmt:
retval = _equalTruncateStmt(a, b);
break;
case T_CommentStmt:
retval = _equalCommentStmt(a, b);
break;
case T_SecLabelStmt:
retval = _equalSecLabelStmt(a, b);
break;
case T_FetchStmt:
retval = _equalFetchStmt(a, b);
break;
case T_IndexStmt:
retval = _equalIndexStmt(a, b);
break;
Implement multivariate n-distinct coefficients Add support for explicitly declared statistic objects (CREATE STATISTICS), allowing collection of statistics on more complex combinations that individual table columns. Companion commands DROP STATISTICS and ALTER STATISTICS ... OWNER TO / SET SCHEMA / RENAME are added too. All this DDL has been designed so that more statistic types can be added later on, such as multivariate most-common-values and multivariate histograms between columns of a single table, leaving room for permitting columns on multiple tables, too, as well as expressions. This commit only adds support for collection of n-distinct coefficient on user-specified sets of columns in a single table. This is useful to estimate number of distinct groups in GROUP BY and DISTINCT clauses; estimation errors there can cause over-allocation of memory in hashed aggregates, for instance, so it's a worthwhile problem to solve. A new special pseudo-type pg_ndistinct is used. (num-distinct estimation was deemed sufficiently useful by itself that this is worthwhile even if no further statistic types are added immediately; so much so that another version of essentially the same functionality was submitted by Kyotaro Horiguchi: https://postgr.es/m/20150828.173334.114731693.horiguchi.kyotaro@lab.ntt.co.jp though this commit does not use that code.) Author: Tomas Vondra. Some code rework by Álvaro. Reviewed-by: Dean Rasheed, David Rowley, Kyotaro Horiguchi, Jeff Janes, Ideriha Takeshi Discussion: https://postgr.es/m/543AFA15.4080608@fuzzy.cz https://postgr.es/m/20170320190220.ixlaueanxegqd5gr@alvherre.pgsql
2017-03-24 18:06:10 +01:00
case T_CreateStatsStmt:
retval = _equalCreateStatsStmt(a, b);
break;
case T_AlterStatsStmt:
retval = _equalAlterStatsStmt(a, b);
break;
case T_CreateFunctionStmt:
retval = _equalCreateFunctionStmt(a, b);
break;
case T_FunctionParameter:
retval = _equalFunctionParameter(a, b);
break;
case T_AlterFunctionStmt:
retval = _equalAlterFunctionStmt(a, b);
break;
case T_DoStmt:
retval = _equalDoStmt(a, b);
break;
case T_RenameStmt:
retval = _equalRenameStmt(a, b);
break;
case T_AlterObjectDependsStmt:
retval = _equalAlterObjectDependsStmt(a, b);
break;
case T_AlterObjectSchemaStmt:
retval = _equalAlterObjectSchemaStmt(a, b);
break;
case T_AlterOwnerStmt:
retval = _equalAlterOwnerStmt(a, b);
break;
case T_AlterOperatorStmt:
retval = _equalAlterOperatorStmt(a, b);
break;
case T_AlterTypeStmt:
retval = _equalAlterTypeStmt(a, b);
break;
case T_RuleStmt:
retval = _equalRuleStmt(a, b);
break;
case T_NotifyStmt:
retval = _equalNotifyStmt(a, b);
break;
case T_ListenStmt:
retval = _equalListenStmt(a, b);
break;
case T_UnlistenStmt:
retval = _equalUnlistenStmt(a, b);
break;
case T_TransactionStmt:
retval = _equalTransactionStmt(a, b);
break;
case T_CompositeTypeStmt:
retval = _equalCompositeTypeStmt(a, b);
break;
case T_CreateEnumStmt:
retval = _equalCreateEnumStmt(a, b);
break;
case T_CreateRangeStmt:
retval = _equalCreateRangeStmt(a, b);
break;
case T_AlterEnumStmt:
retval = _equalAlterEnumStmt(a, b);
break;
case T_ViewStmt:
retval = _equalViewStmt(a, b);
break;
case T_LoadStmt:
retval = _equalLoadStmt(a, b);
break;
case T_CreateDomainStmt:
retval = _equalCreateDomainStmt(a, b);
break;
case T_CreateOpClassStmt:
retval = _equalCreateOpClassStmt(a, b);
break;
case T_CreateOpClassItem:
retval = _equalCreateOpClassItem(a, b);
break;
case T_CreateOpFamilyStmt:
retval = _equalCreateOpFamilyStmt(a, b);
break;
case T_AlterOpFamilyStmt:
retval = _equalAlterOpFamilyStmt(a, b);
break;
case T_CreatedbStmt:
retval = _equalCreatedbStmt(a, b);
break;
case T_AlterDatabaseStmt:
retval = _equalAlterDatabaseStmt(a, b);
break;
case T_AlterDatabaseRefreshCollStmt:
retval = _equalAlterDatabaseRefreshCollStmt(a, b);
break;
case T_AlterDatabaseSetStmt:
retval = _equalAlterDatabaseSetStmt(a, b);
break;
case T_DropdbStmt:
retval = _equalDropdbStmt(a, b);
break;
case T_VacuumStmt:
retval = _equalVacuumStmt(a, b);
break;
case T_VacuumRelation:
retval = _equalVacuumRelation(a, b);
break;
case T_ExplainStmt:
retval = _equalExplainStmt(a, b);
break;
case T_CreateTableAsStmt:
retval = _equalCreateTableAsStmt(a, b);
break;
case T_RefreshMatViewStmt:
retval = _equalRefreshMatViewStmt(a, b);
break;
case T_ReplicaIdentityStmt:
retval = _equalReplicaIdentityStmt(a, b);
break;
case T_AlterSystemStmt:
retval = _equalAlterSystemStmt(a, b);
break;
case T_CreateSeqStmt:
retval = _equalCreateSeqStmt(a, b);
break;
case T_AlterSeqStmt:
retval = _equalAlterSeqStmt(a, b);
break;
case T_VariableSetStmt:
retval = _equalVariableSetStmt(a, b);
break;
case T_VariableShowStmt:
retval = _equalVariableShowStmt(a, b);
break;
case T_DiscardStmt:
retval = _equalDiscardStmt(a, b);
break;
case T_CreateTableSpaceStmt:
retval = _equalCreateTableSpaceStmt(a, b);
break;
case T_DropTableSpaceStmt:
retval = _equalDropTableSpaceStmt(a, b);
break;
case T_AlterTableSpaceOptionsStmt:
retval = _equalAlterTableSpaceOptionsStmt(a, b);
break;
case T_AlterTableMoveAllStmt:
retval = _equalAlterTableMoveAllStmt(a, b);
break;
case T_CreateExtensionStmt:
retval = _equalCreateExtensionStmt(a, b);
break;
case T_AlterExtensionStmt:
retval = _equalAlterExtensionStmt(a, b);
break;
case T_AlterExtensionContentsStmt:
retval = _equalAlterExtensionContentsStmt(a, b);
break;
case T_CreateFdwStmt:
retval = _equalCreateFdwStmt(a, b);
break;
case T_AlterFdwStmt:
retval = _equalAlterFdwStmt(a, b);
break;
case T_CreateForeignServerStmt:
retval = _equalCreateForeignServerStmt(a, b);
break;
case T_AlterForeignServerStmt:
retval = _equalAlterForeignServerStmt(a, b);
break;
case T_CreateUserMappingStmt:
retval = _equalCreateUserMappingStmt(a, b);
break;
case T_AlterUserMappingStmt:
retval = _equalAlterUserMappingStmt(a, b);
break;
case T_DropUserMappingStmt:
retval = _equalDropUserMappingStmt(a, b);
break;
case T_CreateForeignTableStmt:
retval = _equalCreateForeignTableStmt(a, b);
break;
case T_ImportForeignSchemaStmt:
retval = _equalImportForeignSchemaStmt(a, b);
break;
case T_CreateTransformStmt:
retval = _equalCreateTransformStmt(a, b);
break;
case T_CreateAmStmt:
retval = _equalCreateAmStmt(a, b);
break;
case T_CreateTrigStmt:
retval = _equalCreateTrigStmt(a, b);
break;
case T_CreateEventTrigStmt:
retval = _equalCreateEventTrigStmt(a, b);
break;
case T_AlterEventTrigStmt:
retval = _equalAlterEventTrigStmt(a, b);
break;
case T_CreatePLangStmt:
retval = _equalCreatePLangStmt(a, b);
break;
case T_CreateRoleStmt:
retval = _equalCreateRoleStmt(a, b);
break;
case T_AlterRoleStmt:
retval = _equalAlterRoleStmt(a, b);
break;
case T_AlterRoleSetStmt:
retval = _equalAlterRoleSetStmt(a, b);
break;
case T_DropRoleStmt:
retval = _equalDropRoleStmt(a, b);
break;
case T_LockStmt:
retval = _equalLockStmt(a, b);
break;
case T_ConstraintsSetStmt:
retval = _equalConstraintsSetStmt(a, b);
break;
case T_ReindexStmt:
retval = _equalReindexStmt(a, b);
break;
case T_CheckPointStmt:
retval = true;
break;
case T_CreateSchemaStmt:
retval = _equalCreateSchemaStmt(a, b);
break;
case T_CreateConversionStmt:
retval = _equalCreateConversionStmt(a, b);
break;
case T_CreateCastStmt:
retval = _equalCreateCastStmt(a, b);
break;
case T_PrepareStmt:
retval = _equalPrepareStmt(a, b);
break;
case T_ExecuteStmt:
retval = _equalExecuteStmt(a, b);
break;
case T_DeallocateStmt:
retval = _equalDeallocateStmt(a, b);
break;
case T_DropOwnedStmt:
retval = _equalDropOwnedStmt(a, b);
break;
case T_ReassignOwnedStmt:
retval = _equalReassignOwnedStmt(a, b);
break;
case T_AlterTSDictionaryStmt:
retval = _equalAlterTSDictionaryStmt(a, b);
break;
case T_AlterTSConfigurationStmt:
retval = _equalAlterTSConfigurationStmt(a, b);
break;
Row-Level Security Policies (RLS) Building on the updatable security-barrier views work, add the ability to define policies on tables to limit the set of rows which are returned from a query and which are allowed to be added to a table. Expressions defined by the policy for filtering are added to the security barrier quals of the query, while expressions defined to check records being added to a table are added to the with-check options of the query. New top-level commands are CREATE/ALTER/DROP POLICY and are controlled by the table owner. Row Security is able to be enabled and disabled by the owner on a per-table basis using ALTER TABLE .. ENABLE/DISABLE ROW SECURITY. Per discussion, ROW SECURITY is disabled on tables by default and must be enabled for policies on the table to be used. If no policies exist on a table with ROW SECURITY enabled, a default-deny policy is used and no records will be visible. By default, row security is applied at all times except for the table owner and the superuser. A new GUC, row_security, is added which can be set to ON, OFF, or FORCE. When set to FORCE, row security will be applied even for the table owner and superusers. When set to OFF, row security will be disabled when allowed and an error will be thrown if the user does not have rights to bypass row security. Per discussion, pg_dump sets row_security = OFF by default to ensure that exports and backups will have all data in the table or will error if there are insufficient privileges to bypass row security. A new option has been added to pg_dump, --enable-row-security, to ask pg_dump to export with row security enabled. A new role capability, BYPASSRLS, which can only be set by the superuser, is added to allow other users to be able to bypass row security using row_security = OFF. Many thanks to the various individuals who have helped with the design, particularly Robert Haas for his feedback. Authors include Craig Ringer, KaiGai Kohei, Adam Brightwell, Dean Rasheed, with additional changes and rework by me. Reviewers have included all of the above, Greg Smith, Jeff McCormick, and Robert Haas.
2014-09-19 17:18:35 +02:00
case T_CreatePolicyStmt:
retval = _equalCreatePolicyStmt(a, b);
break;
case T_AlterPolicyStmt:
retval = _equalAlterPolicyStmt(a, b);
break;
case T_CreatePublicationStmt:
retval = _equalCreatePublicationStmt(a, b);
break;
case T_AlterPublicationStmt:
retval = _equalAlterPublicationStmt(a, b);
break;
case T_CreateSubscriptionStmt:
retval = _equalCreateSubscriptionStmt(a, b);
break;
case T_AlterSubscriptionStmt:
retval = _equalAlterSubscriptionStmt(a, b);
break;
case T_DropSubscriptionStmt:
retval = _equalDropSubscriptionStmt(a, b);
break;
case T_A_Expr:
retval = _equalA_Expr(a, b);
break;
case T_ColumnRef:
retval = _equalColumnRef(a, b);
break;
case T_ParamRef:
retval = _equalParamRef(a, b);
break;
case T_A_Const:
retval = _equalA_Const(a, b);
break;
case T_FuncCall:
retval = _equalFuncCall(a, b);
break;
case T_A_Star:
retval = _equalA_Star(a, b);
break;
case T_A_Indices:
retval = _equalA_Indices(a, b);
break;
case T_A_Indirection:
retval = _equalA_Indirection(a, b);
break;
case T_A_ArrayExpr:
retval = _equalA_ArrayExpr(a, b);
break;
case T_ResTarget:
retval = _equalResTarget(a, b);
break;
case T_MultiAssignRef:
retval = _equalMultiAssignRef(a, b);
break;
case T_TypeCast:
retval = _equalTypeCast(a, b);
break;
case T_CollateClause:
retval = _equalCollateClause(a, b);
break;
case T_SortBy:
retval = _equalSortBy(a, b);
break;
case T_WindowDef:
retval = _equalWindowDef(a, b);
break;
case T_RangeSubselect:
retval = _equalRangeSubselect(a, b);
break;
case T_RangeFunction:
retval = _equalRangeFunction(a, b);
break;
Redesign tablesample method API, and do extensive code review. The original implementation of TABLESAMPLE modeled the tablesample method API on index access methods, which wasn't a good choice because, without specialized DDL commands, there's no way to build an extension that can implement a TSM. (Raw inserts into system catalogs are not an acceptable thing to do, because we can't undo them during DROP EXTENSION, nor will pg_upgrade behave sanely.) Instead adopt an API more like procedural language handlers or foreign data wrappers, wherein the only SQL-level support object needed is a single handler function identified by having a special return type. This lets us get rid of the supporting catalog altogether, so that no custom DDL support is needed for the feature. Adjust the API so that it can support non-constant tablesample arguments (the original coding assumed we could evaluate the argument expressions at ExecInitSampleScan time, which is undesirable even if it weren't outright unsafe), and discourage sampling methods from looking at invisible tuples. Make sure that the BERNOULLI and SYSTEM methods are genuinely repeatable within and across queries, as required by the SQL standard, and deal more honestly with methods that can't support that requirement. Make a full code-review pass over the tablesample additions, and fix assorted bugs, omissions, infelicities, and cosmetic issues (such as failure to put the added code stanzas in a consistent ordering). Improve EXPLAIN's output of tablesample plans, too. Back-patch to 9.5 so that we don't have to support the original API in production.
2015-07-25 20:39:00 +02:00
case T_RangeTableSample:
retval = _equalRangeTableSample(a, b);
break;
case T_RangeTableFunc:
retval = _equalRangeTableFunc(a, b);
break;
case T_RangeTableFuncCol:
retval = _equalRangeTableFuncCol(a, b);
break;
case T_TypeName:
retval = _equalTypeName(a, b);
break;
case T_IndexElem:
retval = _equalIndexElem(a, b);
break;
Extended statistics on expressions Allow defining extended statistics on expressions, not just just on simple column references. With this commit, expressions are supported by all existing extended statistics kinds, improving the same types of estimates. A simple example may look like this: CREATE TABLE t (a int); CREATE STATISTICS s ON mod(a,10), mod(a,20) FROM t; ANALYZE t; The collected statistics are useful e.g. to estimate queries with those expressions in WHERE or GROUP BY clauses: SELECT * FROM t WHERE mod(a,10) = 0 AND mod(a,20) = 0; SELECT 1 FROM t GROUP BY mod(a,10), mod(a,20); This introduces new internal statistics kind 'e' (expressions) which is built automatically when the statistics object definition includes any expressions. This represents single-expression statistics, as if there was an expression index (but without the index maintenance overhead). The statistics is stored in pg_statistics_ext_data as an array of composite types, which is possible thanks to 79f6a942bd. CREATE STATISTICS allows building statistics on a single expression, in which case in which case it's not possible to specify statistics kinds. A new system view pg_stats_ext_exprs can be used to display expression statistics, similarly to pg_stats and pg_stats_ext views. ALTER TABLE ... ALTER COLUMN ... TYPE now treats indexes the same way it treats indexes, i.e. it drops and recreates the statistics. This means all statistics are reset, and we no longer try to preserve at least the functional dependencies. This should not be a major issue in practice, as the functional dependencies actually rely on per-column statistics, which were always reset anyway. Author: Tomas Vondra Reviewed-by: Justin Pryzby, Dean Rasheed, Zhihong Yu Discussion: https://postgr.es/m/ad7891d2-e90c-b446-9fe2-7419143847d7%40enterprisedb.com
2021-03-26 23:22:01 +01:00
case T_StatsElem:
retval = _equalStatsElem(a, b);
break;
case T_ColumnDef:
retval = _equalColumnDef(a, b);
break;
case T_Constraint:
retval = _equalConstraint(a, b);
break;
case T_DefElem:
retval = _equalDefElem(a, b);
break;
case T_LockingClause:
retval = _equalLockingClause(a, b);
break;
case T_RangeTblEntry:
retval = _equalRangeTblEntry(a, b);
break;
case T_RangeTblFunction:
retval = _equalRangeTblFunction(a, b);
break;
Redesign tablesample method API, and do extensive code review. The original implementation of TABLESAMPLE modeled the tablesample method API on index access methods, which wasn't a good choice because, without specialized DDL commands, there's no way to build an extension that can implement a TSM. (Raw inserts into system catalogs are not an acceptable thing to do, because we can't undo them during DROP EXTENSION, nor will pg_upgrade behave sanely.) Instead adopt an API more like procedural language handlers or foreign data wrappers, wherein the only SQL-level support object needed is a single handler function identified by having a special return type. This lets us get rid of the supporting catalog altogether, so that no custom DDL support is needed for the feature. Adjust the API so that it can support non-constant tablesample arguments (the original coding assumed we could evaluate the argument expressions at ExecInitSampleScan time, which is undesirable even if it weren't outright unsafe), and discourage sampling methods from looking at invisible tuples. Make sure that the BERNOULLI and SYSTEM methods are genuinely repeatable within and across queries, as required by the SQL standard, and deal more honestly with methods that can't support that requirement. Make a full code-review pass over the tablesample additions, and fix assorted bugs, omissions, infelicities, and cosmetic issues (such as failure to put the added code stanzas in a consistent ordering). Improve EXPLAIN's output of tablesample plans, too. Back-patch to 9.5 so that we don't have to support the original API in production.
2015-07-25 20:39:00 +02:00
case T_TableSampleClause:
retval = _equalTableSampleClause(a, b);
break;
case T_WithCheckOption:
retval = _equalWithCheckOption(a, b);
break;
case T_SortGroupClause:
retval = _equalSortGroupClause(a, b);
break;
Support GROUPING SETS, CUBE and ROLLUP. This SQL standard functionality allows to aggregate data by different GROUP BY clauses at once. Each grouping set returns rows with columns grouped by in other sets set to NULL. This could previously be achieved by doing each grouping as a separate query, conjoined by UNION ALLs. Besides being considerably more concise, grouping sets will in many cases be faster, requiring only one scan over the underlying data. The current implementation of grouping sets only supports using sorting for input. Individual sets that share a sort order are computed in one pass. If there are sets that don't share a sort order, additional sort & aggregation steps are performed. These additional passes are sourced by the previous sort step; thus avoiding repeated scans of the source data. The code is structured in a way that adding support for purely using hash aggregation or a mix of hashing and sorting is possible. Sorting was chosen to be supported first, as it is the most generic method of implementation. Instead of, as in an earlier versions of the patch, representing the chain of sort and aggregation steps as full blown planner and executor nodes, all but the first sort are performed inside the aggregation node itself. This avoids the need to do some unusual gymnastics to handle having to return aggregated and non-aggregated tuples from underlying nodes, as well as having to shut down underlying nodes early to limit memory usage. The optimizer still builds Sort/Agg node to describe each phase, but they're not part of the plan tree, but instead additional data for the aggregation node. They're a convenient and preexisting way to describe aggregation and sorting. The first (and possibly only) sort step is still performed as a separate execution step. That retains similarity with existing group by plans, makes rescans fairly simple, avoids very deep plans (leading to slow explains) and easily allows to avoid the sorting step if the underlying data is sorted by other means. A somewhat ugly side of this patch is having to deal with a grammar ambiguity between the new CUBE keyword and the cube extension/functions named cube (and rollup). To avoid breaking existing deployments of the cube extension it has not been renamed, neither has cube been made a reserved keyword. Instead precedence hacking is used to make GROUP BY cube(..) refer to the CUBE grouping sets feature, and not the function cube(). To actually group by a function cube(), unlikely as that might be, the function name has to be quoted. Needs a catversion bump because stored rules may change. Author: Andrew Gierth and Atri Sharma, with contributions from Andres Freund Reviewed-By: Andres Freund, Noah Misch, Tom Lane, Svenne Krap, Tomas Vondra, Erik Rijkers, Marti Raudsepp, Pavel Stehule Discussion: CAOeZVidmVRe2jU6aMk_5qkxnB7dfmPROzM7Ur8JPW5j8Y5X-Lw@mail.gmail.com
2015-05-16 03:40:59 +02:00
case T_GroupingSet:
retval = _equalGroupingSet(a, b);
break;
case T_WindowClause:
retval = _equalWindowClause(a, b);
break;
case T_RowMarkClause:
retval = _equalRowMarkClause(a, b);
break;
case T_WithClause:
retval = _equalWithClause(a, b);
break;
Add support for INSERT ... ON CONFLICT DO NOTHING/UPDATE. The newly added ON CONFLICT clause allows to specify an alternative to raising a unique or exclusion constraint violation error when inserting. ON CONFLICT refers to constraints that can either be specified using a inference clause (by specifying the columns of a unique constraint) or by naming a unique or exclusion constraint. DO NOTHING avoids the constraint violation, without touching the pre-existing row. DO UPDATE SET ... [WHERE ...] updates the pre-existing tuple, and has access to both the tuple proposed for insertion and the existing tuple; the optional WHERE clause can be used to prevent an update from being executed. The UPDATE SET and WHERE clauses have access to the tuple proposed for insertion using the "magic" EXCLUDED alias, and to the pre-existing tuple using the table name or its alias. This feature is often referred to as upsert. This is implemented using a new infrastructure called "speculative insertion". It is an optimistic variant of regular insertion that first does a pre-check for existing tuples and then attempts an insert. If a violating tuple was inserted concurrently, the speculatively inserted tuple is deleted and a new attempt is made. If the pre-check finds a matching tuple the alternative DO NOTHING or DO UPDATE action is taken. If the insertion succeeds without detecting a conflict, the tuple is deemed inserted. To handle the possible ambiguity between the excluded alias and a table named excluded, and for convenience with long relation names, INSERT INTO now can alias its target table. Bumps catversion as stored rules change. Author: Peter Geoghegan, with significant contributions from Heikki Linnakangas and Andres Freund. Testing infrastructure by Jeff Janes. Reviewed-By: Heikki Linnakangas, Andres Freund, Robert Haas, Simon Riggs, Dean Rasheed, Stephen Frost and many others.
2015-05-08 05:31:36 +02:00
case T_InferClause:
retval = _equalInferClause(a, b);
break;
case T_OnConflictClause:
retval = _equalOnConflictClause(a, b);
break;
case T_CTESearchClause:
retval = _equalCTESearchClause(a, b);
break;
case T_CTECycleClause:
retval = _equalCTECycleClause(a, b);
break;
case T_CommonTableExpr:
retval = _equalCommonTableExpr(a, b);
break;
Add support for MERGE SQL command MERGE performs actions that modify rows in the target table using a source table or query. MERGE provides a single SQL statement that can conditionally INSERT/UPDATE/DELETE rows -- a task that would otherwise require multiple PL statements. For example, MERGE INTO target AS t USING source AS s ON t.tid = s.sid WHEN MATCHED AND t.balance > s.delta THEN UPDATE SET balance = t.balance - s.delta WHEN MATCHED THEN DELETE WHEN NOT MATCHED AND s.delta > 0 THEN INSERT VALUES (s.sid, s.delta) WHEN NOT MATCHED THEN DO NOTHING; MERGE works with regular tables, partitioned tables and inheritance hierarchies, including column and row security enforcement, as well as support for row and statement triggers and transition tables therein. MERGE is optimized for OLTP and is parameterizable, though also useful for large scale ETL/ELT. MERGE is not intended to be used in preference to existing single SQL commands for INSERT, UPDATE or DELETE since there is some overhead. MERGE can be used from PL/pgSQL. MERGE does not support targetting updatable views or foreign tables, and RETURNING clauses are not allowed either. These limitations are likely fixable with sufficient effort. Rewrite rules are also not supported, but it's not clear that we'd want to support them. Author: Pavan Deolasee <pavan.deolasee@gmail.com> Author: Álvaro Herrera <alvherre@alvh.no-ip.org> Author: Amit Langote <amitlangote09@gmail.com> Author: Simon Riggs <simon.riggs@enterprisedb.com> Reviewed-by: Peter Eisentraut <peter.eisentraut@enterprisedb.com> Reviewed-by: Andres Freund <andres@anarazel.de> (earlier versions) Reviewed-by: Peter Geoghegan <pg@bowt.ie> (earlier versions) Reviewed-by: Robert Haas <robertmhaas@gmail.com> (earlier versions) Reviewed-by: Japin Li <japinli@hotmail.com> Reviewed-by: Justin Pryzby <pryzby@telsasoft.com> Reviewed-by: Tomas Vondra <tomas.vondra@enterprisedb.com> Reviewed-by: Zhihong Yu <zyu@yugabyte.com> Discussion: https://postgr.es/m/CANP8+jKitBSrB7oTgT9CY2i1ObfOt36z0XMraQc+Xrz8QB0nXA@mail.gmail.com Discussion: https://postgr.es/m/CAH2-WzkJdBuxj9PO=2QaO9-3h3xGbQPZ34kJH=HukRekwM-GZg@mail.gmail.com Discussion: https://postgr.es/m/20201231134736.GA25392@alvherre.pgsql
2022-03-28 16:45:58 +02:00
case T_MergeWhenClause:
retval = _equalMergeWhenClause(a, b);
break;
case T_MergeAction:
retval = _equalMergeAction(a, b);
break;
case T_ObjectWithArgs:
retval = _equalObjectWithArgs(a, b);
break;
case T_AccessPriv:
retval = _equalAccessPriv(a, b);
break;
case T_XmlSerialize:
retval = _equalXmlSerialize(a, b);
break;
case T_RoleSpec:
retval = _equalRoleSpec(a, b);
break;
case T_TriggerTransition:
retval = _equalTriggerTransition(a, b);
break;
Implement table partitioning. Table partitioning is like table inheritance and reuses much of the existing infrastructure, but there are some important differences. The parent is called a partitioned table and is always empty; it may not have indexes or non-inherited constraints, since those make no sense for a relation with no data of its own. The children are called partitions and contain all of the actual data. Each partition has an implicit partitioning constraint. Multiple inheritance is not allowed, and partitioning and inheritance can't be mixed. Partitions can't have extra columns and may not allow nulls unless the parent does. Tuples inserted into the parent are automatically routed to the correct partition, so tuple-routing ON INSERT triggers are not needed. Tuple routing isn't yet supported for partitions which are foreign tables, and it doesn't handle updates that cross partition boundaries. Currently, tables can be range-partitioned or list-partitioned. List partitioning is limited to a single column, but range partitioning can involve multiple columns. A partitioning "column" can be an expression. Because table partitioning is less general than table inheritance, it is hoped that it will be easier to reason about properties of partitions, and therefore that this will serve as a better foundation for a variety of possible optimizations, including query planner optimizations. The tuple routing based which this patch does based on the implicit partitioning constraints is an example of this, but it seems likely that many other useful optimizations are also possible. Amit Langote, reviewed and tested by Robert Haas, Ashutosh Bapat, Amit Kapila, Rajkumar Raghuwanshi, Corey Huinker, Jaime Casanova, Rushabh Lathia, Erik Rijkers, among others. Minor revisions by me.
2016-12-07 19:17:43 +01:00
case T_PartitionElem:
retval = _equalPartitionElem(a, b);
break;
Code review focused on new node types added by partitioning support. Fix failure to check that we got a plain Const from const-simplification of a coercion request. This is the cause of bug #14666 from Tian Bing: there is an int4 to money cast, but it's only stable not immutable (because of dependence on lc_monetary), resulting in a FuncExpr that the code was miserably unequipped to deal with, or indeed even to notice that it was failing to deal with. Add test cases around this coercion behavior. In view of the above, sprinkle the code liberally with castNode() macros, in hope of catching the next such bug a bit sooner. Also, change some functions that were randomly declared to take Node* to take more specific pointer types. And change some struct fields that were declared Node* but could be given more specific types, allowing removal of assorted explicit casts. Place PARTITION_MAX_KEYS check a bit closer to the code it's protecting. Likewise check only-one-key-for-list-partitioning restriction in a less random place. Avoid not-per-project-style usages like !strcmp(...). Fix assorted failures to avoid scribbling on the input of parse transformation. I'm not sure how necessary this is, but it's entirely silly for these functions to be expending cycles to avoid that and not getting it right. Add guards against partitioning on system columns. Put backend/nodes/ support code into an order that matches handling of these node types elsewhere. Annotate the fact that somebody added location fields to PartitionBoundSpec and PartitionRangeDatum but forgot to handle them in outfuncs.c/readfuncs.c. This is fairly harmless for production purposes (since readfuncs.c would just substitute -1 anyway) but it's still bogus. It's not worth forcing a post-beta1 initdb just to fix this, but if we have another reason to force initdb before 10.0, we should go back and clean this up. Contrariwise, somebody added location fields to PartitionElem and PartitionSpec but forgot to teach exprLocation() about them. Consolidate duplicative code in transformPartitionBound(). Improve a couple of error messages. Improve assorted commentary. Re-pgindent the files touched by this patch; this affects a few comment blocks that must have been added quite recently. Report: https://postgr.es/m/20170524024550.29935.14396@wrigleys.postgresql.org
2017-05-29 05:20:28 +02:00
case T_PartitionSpec:
retval = _equalPartitionSpec(a, b);
break;
Implement table partitioning. Table partitioning is like table inheritance and reuses much of the existing infrastructure, but there are some important differences. The parent is called a partitioned table and is always empty; it may not have indexes or non-inherited constraints, since those make no sense for a relation with no data of its own. The children are called partitions and contain all of the actual data. Each partition has an implicit partitioning constraint. Multiple inheritance is not allowed, and partitioning and inheritance can't be mixed. Partitions can't have extra columns and may not allow nulls unless the parent does. Tuples inserted into the parent are automatically routed to the correct partition, so tuple-routing ON INSERT triggers are not needed. Tuple routing isn't yet supported for partitions which are foreign tables, and it doesn't handle updates that cross partition boundaries. Currently, tables can be range-partitioned or list-partitioned. List partitioning is limited to a single column, but range partitioning can involve multiple columns. A partitioning "column" can be an expression. Because table partitioning is less general than table inheritance, it is hoped that it will be easier to reason about properties of partitions, and therefore that this will serve as a better foundation for a variety of possible optimizations, including query planner optimizations. The tuple routing based which this patch does based on the implicit partitioning constraints is an example of this, but it seems likely that many other useful optimizations are also possible. Amit Langote, reviewed and tested by Robert Haas, Ashutosh Bapat, Amit Kapila, Rajkumar Raghuwanshi, Corey Huinker, Jaime Casanova, Rushabh Lathia, Erik Rijkers, among others. Minor revisions by me.
2016-12-07 19:17:43 +01:00
case T_PartitionBoundSpec:
retval = _equalPartitionBoundSpec(a, b);
break;
case T_PartitionRangeDatum:
retval = _equalPartitionRangeDatum(a, b);
break;
case T_PartitionCmd:
retval = _equalPartitionCmd(a, b);
break;
case T_PublicationObjSpec:
retval = _equalPublicationObject(a, b);
break;
case T_PublicationTable:
retval = _equalPublicationTable(a, b);
break;
case T_JsonKeyValue:
retval = _equalJsonKeyValue(a, b);
break;
case T_JsonObjectConstructor:
retval = _equalJsonObjectConstructor(a, b);
break;
case T_JsonAggConstructor:
retval = _equalJsonAggConstructor(a, b);
break;
case T_JsonObjectAgg:
retval = _equalJsonObjectAgg(a, b);
break;
case T_JsonOutput:
retval = _equalJsonOutput(a, b);
break;
case T_JsonArrayConstructor:
retval = _equalJsonArrayConstructor(a, b);
break;
case T_JsonArrayQueryConstructor:
retval = _equalJsonArrayQueryConstructor(a, b);
break;
case T_JsonArrayAgg:
retval = _equalJsonArrayAgg(a, b);
break;
case T_JsonFuncExpr:
retval = _equalJsonFuncExpr(a, b);
break;
case T_JsonCommon:
retval = _equalJsonCommon(a, b);
break;
case T_JsonArgument:
retval = _equalJsonArgument(a, b);
break;
case T_JsonTablePlan:
retval = _equalJsonTablePlan(a, b);
break;
case T_JsonTable:
retval = _equalJsonTable(a, b);
break;
case T_JsonTableColumn:
retval = _equalJsonTableColumn(a, b);
break;
default:
elog(ERROR, "unrecognized node type: %d",
(int) nodeTag(a));
retval = false; /* keep compiler quiet */
break;
}
return retval;
}