From 015a564cc784139c806a7004f25bf5f7a4b4a29d Mon Sep 17 00:00:00 2001 From: Alena Rybakina Date: Fri, 2 Feb 2024 22:01:09 +0300 Subject: [PATCH 1/2] Transform OR clause to ANY expressions. Replace (expr op C1) OR (expr op C2) ... with expr op ANY(C1, C2, ...) on the preliminary stage of optimization when we are still working with the tree expression. Here C is a constant expression, 'expr' is non-constant expression, 'op' is an operator which returns boolean result and has a commuter (for the case of reverse order of constant and non-constant parts of the expression, like 'CX op expr'). Sometimes it can lead to not optimal plan. But we think it is better to have array of elements instead of a lot of OR clauses. Here is a room for further optimizations on decomposing that array into more optimal parts. Authors: Alena Rybakina , Andrey Lepikhov Reviewed-by: Peter Geoghegan , Ranier Vilela Reviewed-by: Alexander Korotkov , Robert Haas Reviewed-by: jian he --- .../postgres_fdw/expected/postgres_fdw.out | 16 +- doc/src/sgml/config.sgml | 18 + src/backend/nodes/queryjumblefuncs.c | 27 ++ src/backend/parser/parse_expr.c | 340 ++++++++++++++++++ src/backend/utils/misc/guc_tables.c | 11 + src/backend/utils/misc/postgresql.conf.sample | 1 + src/bin/pg_dump/t/002_pg_dump.pl | 6 +- src/include/nodes/queryjumble.h | 1 + src/include/optimizer/optimizer.h | 1 + src/test/regress/expected/create_index.out | 156 +++++++- src/test/regress/expected/inherit.out | 2 +- src/test/regress/expected/join.out | 62 +++- src/test/regress/expected/partition_prune.out | 219 +++++++++-- src/test/regress/expected/rules.out | 4 +- src/test/regress/expected/stats_ext.out | 12 +- src/test/regress/expected/sysviews.out | 3 +- src/test/regress/expected/tidscan.out | 23 +- src/test/regress/sql/create_index.sql | 35 ++ src/test/regress/sql/join.sql | 10 + src/test/regress/sql/partition_prune.sql | 22 ++ src/test/regress/sql/tidscan.sql | 6 + src/tools/pgindent/typedefs.list | 2 + 22 files changed, 908 insertions(+), 69 deletions(-) diff --git a/contrib/postgres_fdw/expected/postgres_fdw.out b/contrib/postgres_fdw/expected/postgres_fdw.out index c355e8f3f7..0523bbd8f7 100644 --- a/contrib/postgres_fdw/expected/postgres_fdw.out +++ b/contrib/postgres_fdw/expected/postgres_fdw.out @@ -1349,7 +1349,7 @@ SELECT t1.c1, t1.c2, t2.c1, t2.c2 FROM ft4 t1 LEFT JOIN (SELECT * FROM ft5 WHERE Foreign Scan Output: t1.c1, t1.c2, ft5.c1, ft5.c2 Relations: (public.ft4 t1) LEFT JOIN (public.ft5) - Remote SQL: SELECT r1.c1, r1.c2, r4.c1, r4.c2 FROM ("S 1"."T 3" r1 LEFT JOIN "S 1"."T 4" r4 ON (((r1.c1 = r4.c1)) AND ((r4.c1 < 10)))) WHERE (((r4.c1 < 10) OR (r4.c1 IS NULL))) AND ((r1.c1 < 10)) + Remote SQL: SELECT r1.c1, r1.c2, r4.c1, r4.c2 FROM ("S 1"."T 3" r1 LEFT JOIN "S 1"."T 4" r4 ON (((r1.c1 = r4.c1)) AND ((r4.c1 < 10)))) WHERE (((r4.c1 IS NULL) OR (r4.c1 < 10))) AND ((r1.c1 < 10)) (4 rows) SELECT t1.c1, t1.c2, t2.c1, t2.c2 FROM ft4 t1 LEFT JOIN (SELECT * FROM ft5 WHERE c1 < 10) t2 ON (t1.c1 = t2.c1) @@ -3105,7 +3105,7 @@ select array_agg(distinct (t1.c1)%5) from ft4 t1 full join ft5 t2 on (t1.c1 = t2 Foreign Scan Output: (array_agg(DISTINCT (t1.c1 % 5))), ((t2.c1 % 3)) Relations: Aggregate on ((public.ft4 t1) FULL JOIN (public.ft5 t2)) - Remote SQL: SELECT array_agg(DISTINCT (r1.c1 % 5)), (r2.c1 % 3) FROM ("S 1"."T 3" r1 FULL JOIN "S 1"."T 4" r2 ON (((r1.c1 = r2.c1)))) WHERE (((r1.c1 < 20) OR ((r1.c1 IS NULL) AND (r2.c1 < 5)))) GROUP BY 2 ORDER BY array_agg(DISTINCT (r1.c1 % 5)) ASC NULLS LAST + Remote SQL: SELECT array_agg(DISTINCT (r1.c1 % 5)), (r2.c1 % 3) FROM ("S 1"."T 3" r1 FULL JOIN "S 1"."T 4" r2 ON (((r1.c1 = r2.c1)))) WHERE ((((r1.c1 IS NULL) AND (r2.c1 < 5)) OR (r1.c1 < 20))) GROUP BY 2 ORDER BY array_agg(DISTINCT (r1.c1 % 5)) ASC NULLS LAST (4 rows) select array_agg(distinct (t1.c1)%5) from ft4 t1 full join ft5 t2 on (t1.c1 = t2.c1) where t1.c1 < 20 or (t1.c1 is null and t2.c1 < 5) group by (t2.c1)%3 order by 1; @@ -3123,7 +3123,7 @@ select array_agg(distinct (t1.c1)%5 order by (t1.c1)%5) from ft4 t1 full join ft Foreign Scan Output: (array_agg(DISTINCT (t1.c1 % 5) ORDER BY (t1.c1 % 5))), ((t2.c1 % 3)) Relations: Aggregate on ((public.ft4 t1) FULL JOIN (public.ft5 t2)) - Remote SQL: SELECT array_agg(DISTINCT (r1.c1 % 5) ORDER BY ((r1.c1 % 5)) ASC NULLS LAST), (r2.c1 % 3) FROM ("S 1"."T 3" r1 FULL JOIN "S 1"."T 4" r2 ON (((r1.c1 = r2.c1)))) WHERE (((r1.c1 < 20) OR ((r1.c1 IS NULL) AND (r2.c1 < 5)))) GROUP BY 2 ORDER BY array_agg(DISTINCT (r1.c1 % 5) ORDER BY ((r1.c1 % 5)) ASC NULLS LAST) ASC NULLS LAST + Remote SQL: SELECT array_agg(DISTINCT (r1.c1 % 5) ORDER BY ((r1.c1 % 5)) ASC NULLS LAST), (r2.c1 % 3) FROM ("S 1"."T 3" r1 FULL JOIN "S 1"."T 4" r2 ON (((r1.c1 = r2.c1)))) WHERE ((((r1.c1 IS NULL) AND (r2.c1 < 5)) OR (r1.c1 < 20))) GROUP BY 2 ORDER BY array_agg(DISTINCT (r1.c1 % 5) ORDER BY ((r1.c1 % 5)) ASC NULLS LAST) ASC NULLS LAST (4 rows) select array_agg(distinct (t1.c1)%5 order by (t1.c1)%5) from ft4 t1 full join ft5 t2 on (t1.c1 = t2.c1) where t1.c1 < 20 or (t1.c1 is null and t2.c1 < 5) group by (t2.c1)%3 order by 1; @@ -3140,7 +3140,7 @@ select array_agg(distinct (t1.c1)%5 order by (t1.c1)%5 desc nulls last) from ft4 Foreign Scan Output: (array_agg(DISTINCT (t1.c1 % 5) ORDER BY (t1.c1 % 5) DESC NULLS LAST)), ((t2.c1 % 3)) Relations: Aggregate on ((public.ft4 t1) FULL JOIN (public.ft5 t2)) - Remote SQL: SELECT array_agg(DISTINCT (r1.c1 % 5) ORDER BY ((r1.c1 % 5)) DESC NULLS LAST), (r2.c1 % 3) FROM ("S 1"."T 3" r1 FULL JOIN "S 1"."T 4" r2 ON (((r1.c1 = r2.c1)))) WHERE (((r1.c1 < 20) OR ((r1.c1 IS NULL) AND (r2.c1 < 5)))) GROUP BY 2 ORDER BY array_agg(DISTINCT (r1.c1 % 5) ORDER BY ((r1.c1 % 5)) DESC NULLS LAST) ASC NULLS LAST + Remote SQL: SELECT array_agg(DISTINCT (r1.c1 % 5) ORDER BY ((r1.c1 % 5)) DESC NULLS LAST), (r2.c1 % 3) FROM ("S 1"."T 3" r1 FULL JOIN "S 1"."T 4" r2 ON (((r1.c1 = r2.c1)))) WHERE ((((r1.c1 IS NULL) AND (r2.c1 < 5)) OR (r1.c1 < 20))) GROUP BY 2 ORDER BY array_agg(DISTINCT (r1.c1 % 5) ORDER BY ((r1.c1 % 5)) DESC NULLS LAST) ASC NULLS LAST (4 rows) select array_agg(distinct (t1.c1)%5 order by (t1.c1)%5 desc nulls last) from ft4 t1 full join ft5 t2 on (t1.c1 = t2.c1) where t1.c1 < 20 or (t1.c1 is null and t2.c1 < 5) group by (t2.c1)%3 order by 1; @@ -8797,18 +8797,18 @@ insert into utrtest values (2, 'qux'); -- Check case where the foreign partition is a subplan target rel explain (verbose, costs off) update utrtest set a = 1 where a = 1 or a = 2 returning *; - QUERY PLAN ----------------------------------------------------------------------------------------------------- + QUERY PLAN +-------------------------------------------------------------------------------------------------------------- Update on public.utrtest Output: utrtest_1.a, utrtest_1.b Foreign Update on public.remp utrtest_1 Update on public.locp utrtest_2 -> Append -> Foreign Update on public.remp utrtest_1 - Remote SQL: UPDATE public.loct SET a = 1 WHERE (((a = 1) OR (a = 2))) RETURNING a, b + Remote SQL: UPDATE public.loct SET a = 1 WHERE ((a = ANY ('{1,2}'::integer[]))) RETURNING a, b -> Seq Scan on public.locp utrtest_2 Output: 1, utrtest_2.tableoid, utrtest_2.ctid, NULL::record - Filter: ((utrtest_2.a = 1) OR (utrtest_2.a = 2)) + Filter: (utrtest_2.a = ANY ('{1,2}'::integer[])) (10 rows) -- The new values are concatenated with ' triggered !' diff --git a/doc/src/sgml/config.sgml b/doc/src/sgml/config.sgml index 43b1a132a2..abad02f92c 100644 --- a/doc/src/sgml/config.sgml +++ b/doc/src/sgml/config.sgml @@ -5433,6 +5433,24 @@ ANY num_sync ( + enable_or_transformation (boolean) + + enable_or_transformation configuration parameter + + + + + Enables or disables the query planner's ability to lookup and group multiple + similar OR expressions to ANY () expressions. + The grouping technique of this transformation is based on the similarity of variable sides. + It applies to equality expressions only. One side of such an expression + must be a constant clause, and the other must contain a variable clause. + The default is on. + + + + enable_parallel_append (boolean) diff --git a/src/backend/nodes/queryjumblefuncs.c b/src/backend/nodes/queryjumblefuncs.c index 82f725baaa..b203746be4 100644 --- a/src/backend/nodes/queryjumblefuncs.c +++ b/src/backend/nodes/queryjumblefuncs.c @@ -141,6 +141,33 @@ JumbleQuery(Query *query) return jstate; } +JumbleState * +JumbleExpr(Expr *expr, uint64 *queryId) +{ + JumbleState *jstate = NULL; + + Assert(queryId != NULL); + + jstate = (JumbleState *) palloc(sizeof(JumbleState)); + + /* Set up workspace for query jumbling */ + jstate->jumble = (unsigned char *) palloc(JUMBLE_SIZE); + jstate->jumble_len = 0; + jstate->clocations_buf_size = 32; + jstate->clocations = (LocationLen *) + palloc(jstate->clocations_buf_size * sizeof(LocationLen)); + jstate->clocations_count = 0; + jstate->highest_extern_param_id = 0; + + /* Compute query ID */ + _jumbleNode(jstate, (Node *) expr); + *queryId = DatumGetUInt64(hash_any_extended(jstate->jumble, + jstate->jumble_len, + 0)); + + return jstate; +} + /* * Enables query identifier computation. * diff --git a/src/backend/parser/parse_expr.c b/src/backend/parser/parse_expr.c index 9300c7b9ab..10ceccc80c 100644 --- a/src/backend/parser/parse_expr.c +++ b/src/backend/parser/parse_expr.c @@ -16,12 +16,14 @@ #include "postgres.h" #include "catalog/pg_aggregate.h" +#include "catalog/pg_operator.h" #include "catalog/pg_proc.h" #include "catalog/pg_type.h" #include "commands/dbcommands.h" #include "miscadmin.h" #include "nodes/makefuncs.h" #include "nodes/nodeFuncs.h" +#include "nodes/queryjumble.h" #include "optimizer/optimizer.h" #include "parser/analyze.h" #include "parser/parse_agg.h" @@ -38,11 +40,13 @@ #include "utils/date.h" #include "utils/fmgroids.h" #include "utils/lsyscache.h" +#include "utils/syscache.h" #include "utils/timestamp.h" #include "utils/xml.h" /* GUC parameters */ bool Transform_null_equals = false; +bool enable_or_transformation = true; static Node *transformExprRecurse(ParseState *pstate, Node *expr); @@ -99,6 +103,337 @@ static Expr *make_distinct_op(ParseState *pstate, List *opname, static Node *make_nulltest_from_distinct(ParseState *pstate, A_Expr *distincta, Node *arg); +typedef struct OrClauseGroupKey +{ + Expr *expr; /* Pointer to the expression tree which has been a source for + the hashkey value */ + Oid opno; + Oid exprtype; +} OrClauseGroupKey; + +typedef struct OrClauseGroupEntry +{ + OrClauseGroupKey key; + + Node *node; + List *consts; + List *exprs; +} OrClauseGroupEntry; + +/* + * Hash function to find candidate clauses. + */ +static uint32 +orclause_hash(const void *data, Size keysize) +{ + OrClauseGroupKey *key = (OrClauseGroupKey *) data; + uint64 hash; + + Assert(keysize == sizeof(OrClauseGroupKey)); + + (void) JumbleExpr(key->expr, &hash); + hash += ((uint64) key->opno + (uint64) key->exprtype); + return (uint32) (hash % UINT32_MAX); +} + +static void * +orclause_keycopy(void *dest, const void *src, Size keysize) +{ + OrClauseGroupKey *src_key = (OrClauseGroupKey *) src; + OrClauseGroupKey *dst_key = (OrClauseGroupKey *) dest; + + Assert(sizeof(OrClauseGroupKey) == keysize); + + dst_key->expr = src_key->expr; + dst_key->opno = src_key->opno; + dst_key->exprtype = src_key->exprtype; + return dst_key; +} + +/* + * Dynahash match function to use in guc_hashtab + */ +static int +orclause_match(const void *data1, const void *data2, Size keysize) +{ + OrClauseGroupKey *key1 = (OrClauseGroupKey *) data1; + OrClauseGroupKey *key2 = (OrClauseGroupKey *) data2; + + Assert(sizeof(OrClauseGroupKey) == keysize); + + if (key1->opno == key2->opno && key1->exprtype == key2->exprtype && + equal(key1->expr, key2->expr)) + return 0; + + return 1; +} + +/* + * TransformOrExprToANY - + * Discover the args of an OR expression and try to group similar OR + * expressions to an ANY operation. + * Transformation must be already done on input args list before the call. + * Transformation groups two-sided equality operations. One side of such an + * operation must be plain constant or constant expression. The other side of + * the clause must be a variable expression without volatile functions. + * The grouping technique is based on the similarity of variable sides of the + * expression: using queryId and equal() routine, it groups constant sides of + * similar clauses into an array. After the grouping procedure, each couple + * ('variable expression' and 'constant array') form a new SAOP operation, + * which is added to the args list of the returning expression. + * + * NOTE: function returns OR BoolExpr if more than one clause are detected in + * the final args list, or ScalarArrayOpExpr if all args were grouped into + * the single SAOP expression. + */ +static Node * +TransformOrExprToANY(ParseState *pstate, List *args, int location) +{ + List *or_list = NIL; + List *entries = NIL; + ListCell *lc; + HASHCTL info; + HTAB *or_group_htab = NULL; + int len_ors = list_length(args); + OrClauseGroupEntry *entry = NULL; + + Assert(enable_or_transformation && len_ors > 1); + + MemSet(&info, 0, sizeof(info)); + info.keysize = sizeof(OrClauseGroupKey); + info.entrysize = sizeof(OrClauseGroupEntry); + info.hash = orclause_hash; + info.keycopy = orclause_keycopy; + info.match = orclause_match; + or_group_htab = hash_create("OR Groups", + len_ors, + &info, + HASH_ELEM | HASH_FUNCTION | HASH_COMPARE | HASH_KEYCOPY); + + foreach(lc, args) + { + Node *orqual = lfirst(lc); + Node *const_expr; + Node *nconst_expr; + OrClauseGroupKey hashkey; + bool found; + Oid opno; + Oid exprtype; + Node *leftop, *rightop; + + if (!IsA(orqual, OpExpr)) + { + or_list = lappend(or_list, orqual); + continue; + } + + opno = ((OpExpr *) orqual)->opno; + if (get_op_rettype(opno) != BOOLOID) + { + /* Only operator returning boolean suits OR -> ANY transformation */ + or_list = lappend(or_list, orqual); + continue; + } + + /* + * Detect the constant side of the clause. Recall non-constant + * expression can be made not only with Vars, but also with Params, + * which is not bonded with any relation. Thus, we detect the const + * side - if another side is constant too, the orqual couldn't be + * an OpExpr. + * Get pointers to constant and expression sides of the qual. + */ + leftop = get_leftop(orqual); + if (IsA(leftop, RelabelType)) + leftop = (Node *) ((RelabelType *) leftop)->arg; + rightop = get_rightop(orqual); + if (IsA(rightop, RelabelType)) + rightop = (Node *) ((RelabelType *) rightop)->arg; + + if (IsA(leftop, Const)) + { + opno = get_commutator(opno); + + if (!OidIsValid(opno)) + { + /* commutator doesn't exist, we can't reverse the order */ + or_list = lappend(or_list, orqual); + continue; + } + + nconst_expr = get_rightop(orqual); + const_expr = get_leftop(orqual); + } + else if (IsA(rightop, Const)) + { + const_expr = get_rightop(orqual); + nconst_expr = get_leftop(orqual); + } + else + { + or_list = lappend(or_list, orqual); + continue; + } + + /* + * Transformation only works with both side type is not + * { array | composite | domain | record }. + * Also, forbid it for volatile expressions. + */ + exprtype = exprType(nconst_expr); + if (type_is_rowtype(exprType(const_expr)) || + type_is_rowtype(exprtype) || + contain_volatile_functions((Node *) nconst_expr)) + { + or_list = lappend(or_list, orqual); + continue; + } + + /* + * At this point we definitely have a transformable clause. + * Classify it and add into specific group of clauses, or create new + * group. + */ + hashkey.expr = (Expr *) nconst_expr; + hashkey.opno = opno; + hashkey.exprtype = exprtype; + entry = hash_search(or_group_htab, &hashkey, HASH_ENTER, &found); + + if (unlikely(found)) + { + entry->consts = lappend(entry->consts, const_expr); + entry->exprs = lappend(entry->exprs, orqual); + } + else + { + entry->node = nconst_expr; + entry->consts = list_make1(const_expr); + entry->exprs = list_make1(orqual); + + /* + * Add the entry to the list. It is needed exclusively to manage the + * problem with the order of transformed clauses in explain. + * Hash value can depend on the platform and version. Hence, + * sequental scan of the hash table would prone to change the order + * of clauses in lists and, as a result, break regression tests + * accidentially. + */ + entries = lappend(entries, entry); + } + } + + /* Let's convert each group of clauses to an IN operation. */ + + /* + * Go through the list of groups and convert each, where number of + * consts more than 1. trivial groups move to OR-list again + */ + + foreach (lc, entries) + { + Oid scalar_type; + Oid array_type; + + entry = (OrClauseGroupEntry *) lfirst(lc); + + Assert(list_length(entry->consts) > 0); + Assert(list_length(entry->exprs) == list_length(entry->consts)); + + if (list_length(entry->consts) == 1) + { + /* + * Only one element in the class. Return origin expression into + * the BoolExpr args list unchanged. + */ + list_free(entry->consts); + or_list = list_concat(or_list, entry->exprs); + continue; + } + + /* + * Do the transformation. + */ + + scalar_type = entry->key.exprtype; + array_type = OidIsValid(scalar_type) ? get_array_type(scalar_type) : + InvalidOid; + + if (OidIsValid(array_type)) + { + /* + * OK: coerce all the right-hand non-Var inputs to the common + * type and build an ArrayExpr for them. + */ + List *aexprs = NIL; + ArrayExpr *newa = NULL; + ScalarArrayOpExpr *saopexpr = NULL; + HeapTuple opertup; + Form_pg_operator operform; + List *namelist = NIL; + ListCell *lc1; + + foreach(lc1, entry->consts) + { + Node *rexpr = (Node *) lfirst(lc1); + + rexpr = coerce_to_common_type(pstate, rexpr, + scalar_type, + "IN"); + aexprs = lappend(aexprs, rexpr); + } + + newa = makeNode(ArrayExpr); + /* array_collid will be set by parse_collate.c */ + newa->element_typeid = scalar_type; + newa->array_typeid = array_type; + newa->multidims = false; + newa->elements = aexprs; + newa->location = -1; + + opertup = SearchSysCache1(OPEROID, + ObjectIdGetDatum(entry->key.opno)); + if (!HeapTupleIsValid(opertup)) + elog(ERROR, "cache lookup failed for operator %u", + entry->key.opno); + + operform = (Form_pg_operator) GETSTRUCT(opertup); + if (!OperatorIsVisible(entry->key.opno)) + namelist = lappend(namelist, makeString(get_namespace_name(operform->oprnamespace))); + + namelist = lappend(namelist, makeString(pstrdup(NameStr(operform->oprname)))); + ReleaseSysCache(opertup); + + saopexpr = + (ScalarArrayOpExpr *) + make_scalar_array_op(pstate, + namelist, + true, + entry->node, + (Node *) newa, + -1); + + or_list = lappend(or_list, (void *) saopexpr); + } + else + { + /* + * If the const node (right side of operator expression) 's type + * don't have “true” array type, then we cannnot do the transformation. + * We simply concatenate the expression node. + * + */ + list_free(entry->consts); + or_list = list_concat(or_list, entry->exprs); + } + } + hash_destroy(or_group_htab); + list_free(entries); + + /* One more trick: assemble correct clause */ + return (Node *) ((list_length(or_list) > 1) ? + makeBoolExpr(OR_EXPR, or_list, location) : + linitial(or_list)); +} /* * transformExpr - @@ -1386,6 +1721,11 @@ transformBoolExpr(ParseState *pstate, BoolExpr *a) args = lappend(args, arg); } + /* Make an attempt to group similar OR clauses into ANY operation */ + if (enable_or_transformation && a->boolop == OR_EXPR && + list_length(args) > 1) + return TransformOrExprToANY(pstate, args, a->location); + return (Node *) makeBoolExpr(a->boolop, args, a->location); } diff --git a/src/backend/utils/misc/guc_tables.c b/src/backend/utils/misc/guc_tables.c index 93ded31ed9..7d3a1ca238 100644 --- a/src/backend/utils/misc/guc_tables.c +++ b/src/backend/utils/misc/guc_tables.c @@ -1026,6 +1026,17 @@ struct config_bool ConfigureNamesBool[] = true, NULL, NULL, NULL }, + { + {"enable_or_transformation", PGC_USERSET, QUERY_TUNING_OTHER, + gettext_noop("Transform a sequence of OR clauses to an IN expression."), + gettext_noop("The planner will replace clauses like 'x=c1 OR x=c2 .." + "to the clause 'x IN (c1,c2,...)'"), + GUC_EXPLAIN + }, + &enable_or_transformation, + true, + NULL, NULL, NULL + }, { /* * Not for general use --- used by SET SESSION AUTHORIZATION and SET diff --git a/src/backend/utils/misc/postgresql.conf.sample b/src/backend/utils/misc/postgresql.conf.sample index edcc0282b2..d30dc6d51c 100644 --- a/src/backend/utils/misc/postgresql.conf.sample +++ b/src/backend/utils/misc/postgresql.conf.sample @@ -389,6 +389,7 @@ # - Planner Method Configuration - #enable_async_append = on +#enable_or_transformation = on #enable_bitmapscan = on #enable_gathermerge = on #enable_hashagg = on diff --git a/src/bin/pg_dump/t/002_pg_dump.pl b/src/bin/pg_dump/t/002_pg_dump.pl index 00b5092713..d28bf617db 100644 --- a/src/bin/pg_dump/t/002_pg_dump.pl +++ b/src/bin/pg_dump/t/002_pg_dump.pl @@ -2095,9 +2095,9 @@ my %tests = ( regexp => qr/^ \QCREATE DOMAIN dump_test.us_postal_code AS text COLLATE pg_catalog."C" DEFAULT '10014'::text\E\n\s+ \QCONSTRAINT us_postal_code_check CHECK \E - \Q(((VALUE ~ '^\d{5}\E - \$\Q'::text) OR (VALUE ~ '^\d{5}-\d{4}\E\$ - \Q'::text)));\E(.|\n)* + \Q((VALUE ~ ANY (ARRAY['^\d{5}\E + \$\Q'::text, '^\d{5}-\d{4}\E\$ + \Q'::text])));\E(.|\n)* \QCOMMENT ON CONSTRAINT us_postal_code_check ON DOMAIN dump_test.us_postal_code IS 'check it';\E /xm, like => diff --git a/src/include/nodes/queryjumble.h b/src/include/nodes/queryjumble.h index f1c55c8067..a9ae048af5 100644 --- a/src/include/nodes/queryjumble.h +++ b/src/include/nodes/queryjumble.h @@ -65,6 +65,7 @@ extern PGDLLIMPORT int compute_query_id; extern const char *CleanQuerytext(const char *query, int *location, int *len); extern JumbleState *JumbleQuery(Query *query); +extern JumbleState *JumbleExpr(Expr *expr, uint64 *queryId); extern void EnableQueryId(void); extern PGDLLIMPORT bool query_id_enabled; diff --git a/src/include/optimizer/optimizer.h b/src/include/optimizer/optimizer.h index 7b63c5cf71..35ab577501 100644 --- a/src/include/optimizer/optimizer.h +++ b/src/include/optimizer/optimizer.h @@ -50,6 +50,7 @@ struct PlannedStmt; struct ParamListInfoData; struct HeapTupleData; +extern PGDLLIMPORT bool enable_or_transformation; /* in path/clausesel.c: */ diff --git a/src/test/regress/expected/create_index.out b/src/test/regress/expected/create_index.out index 79fa117cb5..b8653c09ea 100644 --- a/src/test/regress/expected/create_index.out +++ b/src/test/regress/expected/create_index.out @@ -1838,18 +1838,50 @@ DROP TABLE onek_with_null; EXPLAIN (COSTS OFF) SELECT * FROM tenk1 WHERE thousand = 42 AND (tenthous = 1 OR tenthous = 3 OR tenthous = 42); - QUERY PLAN ------------------------------------------------------------------------------------------------------------------------------------------ - Bitmap Heap Scan on tenk1 - Recheck Cond: (((thousand = 42) AND (tenthous = 1)) OR ((thousand = 42) AND (tenthous = 3)) OR ((thousand = 42) AND (tenthous = 42))) - -> BitmapOr - -> Bitmap Index Scan on tenk1_thous_tenthous - Index Cond: ((thousand = 42) AND (tenthous = 1)) - -> Bitmap Index Scan on tenk1_thous_tenthous - Index Cond: ((thousand = 42) AND (tenthous = 3)) - -> Bitmap Index Scan on tenk1_thous_tenthous - Index Cond: ((thousand = 42) AND (tenthous = 42)) -(9 rows) + QUERY PLAN +------------------------------------------------------------------------------ + Index Scan using tenk1_thous_tenthous on tenk1 + Index Cond: ((thousand = 42) AND (tenthous = ANY ('{1,3,42}'::integer[]))) +(2 rows) + +SELECT * FROM tenk1 + WHERE thousand = 42 AND (tenthous = 1 OR tenthous = 3 OR tenthous = 42); + unique1 | unique2 | two | four | ten | twenty | hundred | thousand | twothousand | fivethous | tenthous | odd | even | stringu1 | stringu2 | string4 +---------+---------+-----+------+-----+--------+---------+----------+-------------+-----------+----------+-----+------+----------+----------+--------- + 42 | 5530 | 0 | 2 | 2 | 2 | 42 | 42 | 42 | 42 | 42 | 84 | 85 | QBAAAA | SEIAAA | OOOOxx +(1 row) + +EXPLAIN (COSTS OFF) +SELECT count(*) FROM tenk1 + WHERE hundred = 42 AND (thousand = 42 OR thousand = 99); + QUERY PLAN +------------------------------------------------------------------------------------ + Aggregate + -> Bitmap Heap Scan on tenk1 + Recheck Cond: ((hundred = 42) AND (thousand = ANY ('{42,99}'::integer[]))) + -> BitmapAnd + -> Bitmap Index Scan on tenk1_hundred + Index Cond: (hundred = 42) + -> Bitmap Index Scan on tenk1_thous_tenthous + Index Cond: (thousand = ANY ('{42,99}'::integer[])) +(8 rows) + +SELECT count(*) FROM tenk1 + WHERE hundred = 42 AND (thousand = 42 OR thousand = 99); + count +------- + 10 +(1 row) + +SET enable_or_transformation = on; +EXPLAIN (COSTS OFF) +SELECT * FROM tenk1 + WHERE thousand = 42 AND (tenthous = 1 OR tenthous = 3 OR tenthous = 42); + QUERY PLAN +------------------------------------------------------------------------------ + Index Scan using tenk1_thous_tenthous on tenk1 + Index Cond: ((thousand = 42) AND (tenthous = ANY ('{1,3,42}'::integer[]))) +(2 rows) SELECT * FROM tenk1 WHERE thousand = 42 AND (tenthous = 1 OR tenthous = 3 OR tenthous = 42); @@ -1861,28 +1893,116 @@ SELECT * FROM tenk1 EXPLAIN (COSTS OFF) SELECT count(*) FROM tenk1 WHERE hundred = 42 AND (thousand = 42 OR thousand = 99); - QUERY PLAN ---------------------------------------------------------------------------------- + QUERY PLAN +------------------------------------------------------------------------------------ + Aggregate + -> Bitmap Heap Scan on tenk1 + Recheck Cond: ((hundred = 42) AND (thousand = ANY ('{42,99}'::integer[]))) + -> BitmapAnd + -> Bitmap Index Scan on tenk1_hundred + Index Cond: (hundred = 42) + -> Bitmap Index Scan on tenk1_thous_tenthous + Index Cond: (thousand = ANY ('{42,99}'::integer[])) +(8 rows) + +SELECT count(*) FROM tenk1 + WHERE hundred = 42 AND (thousand = 42 OR thousand = 99); + count +------- + 10 +(1 row) + +EXPLAIN (COSTS OFF) +SELECT count(*) FROM tenk1 + WHERE hundred = 42 AND (thousand < 42 OR thousand < 99 OR 43 > thousand OR 42 > thousand); + QUERY PLAN +------------------------------------------------------------------------------------------ + Aggregate + -> Bitmap Heap Scan on tenk1 + Recheck Cond: ((hundred = 42) AND (thousand < ANY ('{42,99,43,42}'::integer[]))) + -> BitmapAnd + -> Bitmap Index Scan on tenk1_hundred + Index Cond: (hundred = 42) + -> Bitmap Index Scan on tenk1_thous_tenthous + Index Cond: (thousand < ANY ('{42,99,43,42}'::integer[])) +(8 rows) + +EXPLAIN (COSTS OFF) +SELECT count(*) FROM tenk1 + WHERE thousand = 42 AND (tenthous = 1 OR tenthous = 3) OR thousand = 41; + QUERY PLAN +-------------------------------------------------------------------------------------------------------- + Aggregate + -> Bitmap Heap Scan on tenk1 + Recheck Cond: (((thousand = 42) AND (tenthous = ANY ('{1,3}'::integer[]))) OR (thousand = 41)) + -> BitmapOr + -> Bitmap Index Scan on tenk1_thous_tenthous + Index Cond: ((thousand = 42) AND (tenthous = ANY ('{1,3}'::integer[]))) + -> Bitmap Index Scan on tenk1_thous_tenthous + Index Cond: (thousand = 41) +(8 rows) + +SELECT count(*) FROM tenk1 + WHERE thousand = 42 AND (tenthous = 1 OR tenthous = 3) OR thousand = 41; + count +------- + 10 +(1 row) + +EXPLAIN (COSTS OFF) +SELECT count(*) FROM tenk1 + WHERE hundred = 42 AND (thousand = 42 OR thousand = 99 OR tenthous < 2) OR thousand = 41; + QUERY PLAN +----------------------------------------------------------------------------------------------------------------------------- + Aggregate + -> Bitmap Heap Scan on tenk1 + Recheck Cond: (((hundred = 42) AND ((thousand = ANY ('{42,99}'::integer[])) OR (tenthous < 2))) OR (thousand = 41)) + -> BitmapOr + -> BitmapAnd + -> Bitmap Index Scan on tenk1_hundred + Index Cond: (hundred = 42) + -> BitmapOr + -> Bitmap Index Scan on tenk1_thous_tenthous + Index Cond: (thousand = ANY ('{42,99}'::integer[])) + -> Bitmap Index Scan on tenk1_thous_tenthous + Index Cond: (tenthous < 2) + -> Bitmap Index Scan on tenk1_thous_tenthous + Index Cond: (thousand = 41) +(14 rows) + +SELECT count(*) FROM tenk1 + WHERE hundred = 42 AND (thousand = 42 OR thousand = 99 OR tenthous < 2) OR thousand = 41; + count +------- + 20 +(1 row) + +EXPLAIN (COSTS OFF) +SELECT count(*) FROM tenk1 + WHERE hundred = 42 AND (thousand = 42 OR thousand = 41 OR thousand = 99 AND tenthous = 2); + QUERY PLAN +------------------------------------------------------------------------------------------------------------------------------ Aggregate -> Bitmap Heap Scan on tenk1 - Recheck Cond: ((hundred = 42) AND ((thousand = 42) OR (thousand = 99))) + Recheck Cond: ((hundred = 42) AND (((thousand = 99) AND (tenthous = 2)) OR (thousand = ANY ('{42,41}'::integer[])))) -> BitmapAnd -> Bitmap Index Scan on tenk1_hundred Index Cond: (hundred = 42) -> BitmapOr -> Bitmap Index Scan on tenk1_thous_tenthous - Index Cond: (thousand = 42) + Index Cond: ((thousand = 99) AND (tenthous = 2)) -> Bitmap Index Scan on tenk1_thous_tenthous - Index Cond: (thousand = 99) + Index Cond: (thousand = ANY ('{42,41}'::integer[])) (11 rows) SELECT count(*) FROM tenk1 - WHERE hundred = 42 AND (thousand = 42 OR thousand = 99); + WHERE hundred = 42 AND (thousand = 42 OR thousand = 41 OR thousand = 99 AND tenthous = 2); count ------- 10 (1 row) +RESET enable_or_transformation; -- -- Check behavior with duplicate index column contents -- diff --git a/src/test/regress/expected/inherit.out b/src/test/regress/expected/inherit.out index 130a924228..684886336c 100644 --- a/src/test/regress/expected/inherit.out +++ b/src/test/regress/expected/inherit.out @@ -2600,7 +2600,7 @@ explain (costs off) select * from list_parted where a = 'ab' or a in (null, 'cd' QUERY PLAN --------------------------------------------------------------------------------- Seq Scan on part_ab_cd list_parted - Filter: (((a)::text = 'ab'::text) OR ((a)::text = ANY ('{NULL,cd}'::text[]))) + Filter: (((a)::text = ANY ('{NULL,cd}'::text[])) OR ((a)::text = 'ab'::text)) (2 rows) explain (costs off) select * from list_parted where a = 'ab'; diff --git a/src/test/regress/expected/join.out b/src/test/regress/expected/join.out index 9605400021..cd2d78a636 100644 --- a/src/test/regress/expected/join.out +++ b/src/test/regress/expected/join.out @@ -4210,10 +4210,10 @@ explain (costs off) select * from tenk1 a join tenk1 b on (a.unique1 = 1 and b.unique1 = 2) or ((a.unique2 = 3 or a.unique2 = 7) and b.hundred = 4); - QUERY PLAN ----------------------------------------------------------------------------------------------------------------------- + QUERY PLAN +------------------------------------------------------------------------------------------------------------------------ Nested Loop - Join Filter: (((a.unique1 = 1) AND (b.unique1 = 2)) OR (((a.unique2 = 3) OR (a.unique2 = 7)) AND (b.hundred = 4))) + Join Filter: (((a.unique1 = 1) AND (b.unique1 = 2)) OR ((a.unique2 = ANY ('{3,7}'::integer[])) AND (b.hundred = 4))) -> Bitmap Heap Scan on tenk1 b Recheck Cond: ((unique1 = 2) OR (hundred = 4)) -> BitmapOr @@ -4223,16 +4223,64 @@ select * from tenk1 a join tenk1 b on Index Cond: (hundred = 4) -> Materialize -> Bitmap Heap Scan on tenk1 a - Recheck Cond: ((unique1 = 1) OR (unique2 = 3) OR (unique2 = 7)) + Recheck Cond: ((unique1 = 1) OR (unique2 = ANY ('{3,7}'::integer[]))) -> BitmapOr -> Bitmap Index Scan on tenk1_unique1 Index Cond: (unique1 = 1) -> Bitmap Index Scan on tenk1_unique2 - Index Cond: (unique2 = 3) + Index Cond: (unique2 = ANY ('{3,7}'::integer[])) +(17 rows) + +SET enable_or_transformation = on; +explain (costs off) +select * from tenk1 a join tenk1 b on + (a.unique1 = 1 and b.unique1 = 2) or + ((a.unique2 = 3 or a.unique2 = 7) and b.hundred = 4); + QUERY PLAN +------------------------------------------------------------------------------------------------------------------------ + Nested Loop + Join Filter: (((a.unique1 = 1) AND (b.unique1 = 2)) OR ((a.unique2 = ANY ('{3,7}'::integer[])) AND (b.hundred = 4))) + -> Bitmap Heap Scan on tenk1 b + Recheck Cond: ((unique1 = 2) OR (hundred = 4)) + -> BitmapOr + -> Bitmap Index Scan on tenk1_unique1 + Index Cond: (unique1 = 2) + -> Bitmap Index Scan on tenk1_hundred + Index Cond: (hundred = 4) + -> Materialize + -> Bitmap Heap Scan on tenk1 a + Recheck Cond: ((unique1 = 1) OR (unique2 = ANY ('{3,7}'::integer[]))) + -> BitmapOr + -> Bitmap Index Scan on tenk1_unique1 + Index Cond: (unique1 = 1) -> Bitmap Index Scan on tenk1_unique2 - Index Cond: (unique2 = 7) -(19 rows) + Index Cond: (unique2 = ANY ('{3,7}'::integer[])) +(17 rows) + +explain (costs off) +select * from tenk1 a join tenk1 b on + (a.unique1 < 20 or a.unique1 = 3 or a.unique1 = 1 and b.unique1 = 2) or + ((a.unique2 = 3 or a.unique2 = 7) and b.hundred = 4); + QUERY PLAN +--------------------------------------------------------------------------------------------------------------------------------------------------------------- + Nested Loop + Join Filter: (((a.unique1 = 1) AND (b.unique1 = 2)) OR ((a.unique2 = ANY ('{3,7}'::integer[])) AND (b.hundred = 4)) OR (a.unique1 < 20) OR (a.unique1 = 3)) + -> Seq Scan on tenk1 b + -> Materialize + -> Bitmap Heap Scan on tenk1 a + Recheck Cond: ((unique1 = 1) OR (unique2 = ANY ('{3,7}'::integer[])) OR (unique1 < 20) OR (unique1 = 3)) + -> BitmapOr + -> Bitmap Index Scan on tenk1_unique1 + Index Cond: (unique1 = 1) + -> Bitmap Index Scan on tenk1_unique2 + Index Cond: (unique2 = ANY ('{3,7}'::integer[])) + -> Bitmap Index Scan on tenk1_unique1 + Index Cond: (unique1 < 20) + -> Bitmap Index Scan on tenk1_unique1 + Index Cond: (unique1 = 3) +(15 rows) +RESET enable_or_transformation; -- -- test placement of movable quals in a parameterized join tree -- diff --git a/src/test/regress/expected/partition_prune.out b/src/test/regress/expected/partition_prune.out index b41950d923..26b05d7a5a 100644 --- a/src/test/regress/expected/partition_prune.out +++ b/src/test/regress/expected/partition_prune.out @@ -82,25 +82,47 @@ explain (costs off) select * from lp where a is null; (2 rows) explain (costs off) select * from lp where a = 'a' or a = 'c'; - QUERY PLAN ----------------------------------------------------------- + QUERY PLAN +----------------------------------------------- Append -> Seq Scan on lp_ad lp_1 - Filter: ((a = 'a'::bpchar) OR (a = 'c'::bpchar)) + Filter: (a = ANY ('{a,c}'::bpchar[])) -> Seq Scan on lp_bc lp_2 - Filter: ((a = 'a'::bpchar) OR (a = 'c'::bpchar)) + Filter: (a = ANY ('{a,c}'::bpchar[])) (5 rows) explain (costs off) select * from lp where a is not null and (a = 'a' or a = 'c'); - QUERY PLAN --------------------------------------------------------------------------------- + QUERY PLAN +--------------------------------------------------------------------- Append -> Seq Scan on lp_ad lp_1 - Filter: ((a IS NOT NULL) AND ((a = 'a'::bpchar) OR (a = 'c'::bpchar))) + Filter: ((a IS NOT NULL) AND (a = ANY ('{a,c}'::bpchar[]))) -> Seq Scan on lp_bc lp_2 - Filter: ((a IS NOT NULL) AND ((a = 'a'::bpchar) OR (a = 'c'::bpchar))) + Filter: ((a IS NOT NULL) AND (a = ANY ('{a,c}'::bpchar[]))) (5 rows) +SET enable_or_transformation = on; +explain (costs off) select * from lp where a = 'a' or a = 'c'; + QUERY PLAN +----------------------------------------------- + Append + -> Seq Scan on lp_ad lp_1 + Filter: (a = ANY ('{a,c}'::bpchar[])) + -> Seq Scan on lp_bc lp_2 + Filter: (a = ANY ('{a,c}'::bpchar[])) +(5 rows) + +explain (costs off) select * from lp where a is not null and (a = 'a' or a = 'c'); + QUERY PLAN +--------------------------------------------------------------------- + Append + -> Seq Scan on lp_ad lp_1 + Filter: ((a IS NOT NULL) AND (a = ANY ('{a,c}'::bpchar[]))) + -> Seq Scan on lp_bc lp_2 + Filter: ((a IS NOT NULL) AND (a = ANY ('{a,c}'::bpchar[]))) +(5 rows) + +RESET enable_or_transformation; explain (costs off) select * from lp where a <> 'g'; QUERY PLAN ------------------------------------ @@ -515,10 +537,10 @@ explain (costs off) select * from rlp where a <= 31; (27 rows) explain (costs off) select * from rlp where a = 1 or a = 7; - QUERY PLAN --------------------------------- + QUERY PLAN +------------------------------------------ Seq Scan on rlp2 rlp - Filter: ((a = 1) OR (a = 7)) + Filter: (a = ANY ('{1,7}'::integer[])) (2 rows) explain (costs off) select * from rlp where a = 1 or b = 'ab'; @@ -589,20 +611,20 @@ explain (costs off) select * from rlp where a < 1 or (a > 20 and a < 25); ------------------------------------------------------ Append -> Seq Scan on rlp1 rlp_1 - Filter: ((a < 1) OR ((a > 20) AND (a < 25))) + Filter: (((a > 20) AND (a < 25)) OR (a < 1)) -> Seq Scan on rlp4_1 rlp_2 - Filter: ((a < 1) OR ((a > 20) AND (a < 25))) + Filter: (((a > 20) AND (a < 25)) OR (a < 1)) (5 rows) -- where clause contradicts sub-partition's constraint explain (costs off) select * from rlp where a = 20 or a = 40; - QUERY PLAN ----------------------------------------- + QUERY PLAN +-------------------------------------------------- Append -> Seq Scan on rlp4_1 rlp_1 - Filter: ((a = 20) OR (a = 40)) + Filter: (a = ANY ('{20,40}'::integer[])) -> Seq Scan on rlp5_default rlp_2 - Filter: ((a = 20) OR (a = 40)) + Filter: (a = ANY ('{20,40}'::integer[])) (5 rows) explain (costs off) select * from rlp3 where a = 20; /* empty */ @@ -671,6 +693,163 @@ explain (costs off) select * from rlp where (a = 1 and a = 3) or (a > 1 and a = Filter: (((a = 1) AND (a = 3)) OR ((a > 1) AND (a = 15))) (11 rows) +SET enable_or_transformation = on; +explain (costs off) select * from rlp where a = 1 or a = 7; + QUERY PLAN +------------------------------------------ + Seq Scan on rlp2 rlp + Filter: (a = ANY ('{1,7}'::integer[])) +(2 rows) + +explain (costs off) select * from rlp where a = 1 or b = 'ab'; + QUERY PLAN +------------------------------------------------------- + Append + -> Seq Scan on rlp1 rlp_1 + Filter: ((a = 1) OR ((b)::text = 'ab'::text)) + -> Seq Scan on rlp2 rlp_2 + Filter: ((a = 1) OR ((b)::text = 'ab'::text)) + -> Seq Scan on rlp3abcd rlp_3 + Filter: ((a = 1) OR ((b)::text = 'ab'::text)) + -> Seq Scan on rlp4_1 rlp_4 + Filter: ((a = 1) OR ((b)::text = 'ab'::text)) + -> Seq Scan on rlp4_2 rlp_5 + Filter: ((a = 1) OR ((b)::text = 'ab'::text)) + -> Seq Scan on rlp4_default rlp_6 + Filter: ((a = 1) OR ((b)::text = 'ab'::text)) + -> Seq Scan on rlp5_1 rlp_7 + Filter: ((a = 1) OR ((b)::text = 'ab'::text)) + -> Seq Scan on rlp5_default rlp_8 + Filter: ((a = 1) OR ((b)::text = 'ab'::text)) + -> Seq Scan on rlp_default_10 rlp_9 + Filter: ((a = 1) OR ((b)::text = 'ab'::text)) + -> Seq Scan on rlp_default_30 rlp_10 + Filter: ((a = 1) OR ((b)::text = 'ab'::text)) + -> Seq Scan on rlp_default_null rlp_11 + Filter: ((a = 1) OR ((b)::text = 'ab'::text)) + -> Seq Scan on rlp_default_default rlp_12 + Filter: ((a = 1) OR ((b)::text = 'ab'::text)) +(25 rows) + +explain (costs off) select * from rlp where a > 20 and a < 27; + QUERY PLAN +----------------------------------------- + Append + -> Seq Scan on rlp4_1 rlp_1 + Filter: ((a > 20) AND (a < 27)) + -> Seq Scan on rlp4_2 rlp_2 + Filter: ((a > 20) AND (a < 27)) +(5 rows) + +explain (costs off) select * from rlp where a = 29; + QUERY PLAN +------------------------------ + Seq Scan on rlp4_default rlp + Filter: (a = 29) +(2 rows) + +explain (costs off) select * from rlp where a >= 29; + QUERY PLAN +--------------------------------------------- + Append + -> Seq Scan on rlp4_default rlp_1 + Filter: (a >= 29) + -> Seq Scan on rlp5_1 rlp_2 + Filter: (a >= 29) + -> Seq Scan on rlp5_default rlp_3 + Filter: (a >= 29) + -> Seq Scan on rlp_default_30 rlp_4 + Filter: (a >= 29) + -> Seq Scan on rlp_default_default rlp_5 + Filter: (a >= 29) +(11 rows) + +explain (costs off) select * from rlp where a < 1 or (a > 20 and a < 25); + QUERY PLAN +------------------------------------------------------ + Append + -> Seq Scan on rlp1 rlp_1 + Filter: (((a > 20) AND (a < 25)) OR (a < 1)) + -> Seq Scan on rlp4_1 rlp_2 + Filter: (((a > 20) AND (a < 25)) OR (a < 1)) +(5 rows) + +explain (costs off) select * from rlp where a = 20 or a = 40; + QUERY PLAN +-------------------------------------------------- + Append + -> Seq Scan on rlp4_1 rlp_1 + Filter: (a = ANY ('{20,40}'::integer[])) + -> Seq Scan on rlp5_default rlp_2 + Filter: (a = ANY ('{20,40}'::integer[])) +(5 rows) + +explain (costs off) select * from rlp3 where a = 20; /* empty */ + QUERY PLAN +-------------------------- + Result + One-Time Filter: false +(2 rows) + +explain (costs off) select * from rlp where a > 1 and a = 10; /* only default */ + QUERY PLAN +---------------------------------- + Seq Scan on rlp_default_10 rlp + Filter: ((a > 1) AND (a = 10)) +(2 rows) + +explain (costs off) select * from rlp where a > 1 and a >=15; /* rlp3 onwards, including default */ + QUERY PLAN +---------------------------------------------- + Append + -> Seq Scan on rlp3abcd rlp_1 + Filter: ((a > 1) AND (a >= 15)) + -> Seq Scan on rlp3efgh rlp_2 + Filter: ((a > 1) AND (a >= 15)) + -> Seq Scan on rlp3nullxy rlp_3 + Filter: ((a > 1) AND (a >= 15)) + -> Seq Scan on rlp3_default rlp_4 + Filter: ((a > 1) AND (a >= 15)) + -> Seq Scan on rlp4_1 rlp_5 + Filter: ((a > 1) AND (a >= 15)) + -> Seq Scan on rlp4_2 rlp_6 + Filter: ((a > 1) AND (a >= 15)) + -> Seq Scan on rlp4_default rlp_7 + Filter: ((a > 1) AND (a >= 15)) + -> Seq Scan on rlp5_1 rlp_8 + Filter: ((a > 1) AND (a >= 15)) + -> Seq Scan on rlp5_default rlp_9 + Filter: ((a > 1) AND (a >= 15)) + -> Seq Scan on rlp_default_30 rlp_10 + Filter: ((a > 1) AND (a >= 15)) + -> Seq Scan on rlp_default_default rlp_11 + Filter: ((a > 1) AND (a >= 15)) +(23 rows) + +explain (costs off) select * from rlp where a = 1 and a = 3; /* empty */ + QUERY PLAN +-------------------------- + Result + One-Time Filter: false +(2 rows) + +explain (costs off) select * from rlp where (a = 1 and a = 3) or (a > 1 and a = 15); + QUERY PLAN +------------------------------------------------------------------- + Append + -> Seq Scan on rlp2 rlp_1 + Filter: (((a = 1) AND (a = 3)) OR ((a > 1) AND (a = 15))) + -> Seq Scan on rlp3abcd rlp_2 + Filter: (((a = 1) AND (a = 3)) OR ((a > 1) AND (a = 15))) + -> Seq Scan on rlp3efgh rlp_3 + Filter: (((a = 1) AND (a = 3)) OR ((a > 1) AND (a = 15))) + -> Seq Scan on rlp3nullxy rlp_4 + Filter: (((a = 1) AND (a = 3)) OR ((a > 1) AND (a = 15))) + -> Seq Scan on rlp3_default rlp_5 + Filter: (((a = 1) AND (a = 3)) OR ((a > 1) AND (a = 15))) +(11 rows) + +RESET enable_or_transformation; -- multi-column keys create table mc3p (a int, b int, c int) partition by range (a, abs(b), c); create table mc3p_default partition of mc3p default; @@ -2036,10 +2215,10 @@ explain (costs off) select * from hp where a = 1 and b = 'abcde'; explain (costs off) select * from hp where a = 1 and b = 'abcde' and (c = 2 or c = 3); - QUERY PLAN ----------------------------------------------------------------------- + QUERY PLAN +-------------------------------------------------------------------------------- Seq Scan on hp2 hp - Filter: ((a = 1) AND (b = 'abcde'::text) AND ((c = 2) OR (c = 3))) + Filter: ((c = ANY ('{2,3}'::integer[])) AND (a = 1) AND (b = 'abcde'::text)) (2 rows) drop table hp2; diff --git a/src/test/regress/expected/rules.out b/src/test/regress/expected/rules.out index b7488d760e..b424ad3154 100644 --- a/src/test/regress/expected/rules.out +++ b/src/test/regress/expected/rules.out @@ -2501,7 +2501,7 @@ pg_stats| SELECT n.nspname AS schemaname, JOIN pg_class c ON ((c.oid = s.starelid))) JOIN pg_attribute a ON (((c.oid = a.attrelid) AND (a.attnum = s.staattnum)))) LEFT JOIN pg_namespace n ON ((n.oid = c.relnamespace))) - WHERE ((NOT a.attisdropped) AND has_column_privilege(c.oid, a.attnum, 'select'::text) AND ((c.relrowsecurity = false) OR (NOT row_security_active(c.oid)))); + WHERE ((NOT a.attisdropped) AND has_column_privilege(c.oid, a.attnum, 'select'::text) AND ((NOT row_security_active(c.oid)) OR (c.relrowsecurity = false))); pg_stats_ext| SELECT cn.nspname AS schemaname, c.relname AS tablename, sn.nspname AS statistics_schemaname, @@ -2532,7 +2532,7 @@ pg_stats_ext| SELECT cn.nspname AS schemaname, WHERE ((NOT (EXISTS ( SELECT 1 FROM (unnest(s.stxkeys) k(k) JOIN pg_attribute a ON (((a.attrelid = s.stxrelid) AND (a.attnum = k.k)))) - WHERE (NOT has_column_privilege(c.oid, a.attnum, 'select'::text))))) AND ((c.relrowsecurity = false) OR (NOT row_security_active(c.oid)))); + WHERE (NOT has_column_privilege(c.oid, a.attnum, 'select'::text))))) AND ((NOT row_security_active(c.oid)) OR (c.relrowsecurity = false))); pg_stats_ext_exprs| SELECT cn.nspname AS schemaname, c.relname AS tablename, sn.nspname AS statistics_schemaname, diff --git a/src/test/regress/expected/stats_ext.out b/src/test/regress/expected/stats_ext.out index 10903bdab0..6f55b9e3ec 100644 --- a/src/test/regress/expected/stats_ext.out +++ b/src/test/regress/expected/stats_ext.out @@ -1322,19 +1322,19 @@ SELECT * FROM check_estimated_rows('SELECT * FROM functional_dependencies WHERE SELECT * FROM check_estimated_rows('SELECT * FROM functional_dependencies WHERE (a = 1 OR a = 51) AND b = ''1'''); estimated | actual -----------+-------- - 99 | 100 + 100 | 100 (1 row) SELECT * FROM check_estimated_rows('SELECT * FROM functional_dependencies WHERE (a = 1 OR a = 51) AND (b = ''1'' OR b = ''2'')'); estimated | actual -----------+-------- - 99 | 100 + 100 | 100 (1 row) SELECT * FROM check_estimated_rows('SELECT * FROM functional_dependencies WHERE (a = 1 OR a = 2 OR a = 51 OR a = 52) AND (b = ''1'' OR b = ''2'')'); estimated | actual -----------+-------- - 197 | 200 + 200 | 200 (1 row) -- OR clauses referencing different attributes are incompatible @@ -1664,19 +1664,19 @@ SELECT * FROM check_estimated_rows('SELECT * FROM functional_dependencies WHERE SELECT * FROM check_estimated_rows('SELECT * FROM functional_dependencies WHERE ((a * 2) = 2 OR (a * 2) = 102) AND upper(b) = ''1'''); estimated | actual -----------+-------- - 99 | 100 + 100 | 100 (1 row) SELECT * FROM check_estimated_rows('SELECT * FROM functional_dependencies WHERE ((a * 2) = 2 OR (a * 2) = 102) AND (upper(b) = ''1'' OR upper(b) = ''2'')'); estimated | actual -----------+-------- - 99 | 100 + 100 | 100 (1 row) SELECT * FROM check_estimated_rows('SELECT * FROM functional_dependencies WHERE ((a * 2) = 2 OR (a * 2) = 4 OR (a * 2) = 102 OR (a * 2) = 104) AND (upper(b) = ''1'' OR upper(b) = ''2'')'); estimated | actual -----------+-------- - 197 | 200 + 200 | 200 (1 row) -- OR clauses referencing different attributes diff --git a/src/test/regress/expected/sysviews.out b/src/test/regress/expected/sysviews.out index 9be7aca2b8..1f9029b5b2 100644 --- a/src/test/regress/expected/sysviews.out +++ b/src/test/regress/expected/sysviews.out @@ -124,6 +124,7 @@ select name, setting from pg_settings where name like 'enable%'; enable_memoize | on enable_mergejoin | on enable_nestloop | on + enable_or_transformation | on enable_parallel_append | on enable_parallel_hash | on enable_partition_pruning | on @@ -134,7 +135,7 @@ select name, setting from pg_settings where name like 'enable%'; enable_seqscan | on enable_sort | on enable_tidscan | on -(23 rows) +(24 rows) -- There are always wait event descriptions for various types. select type, count(*) > 0 as ok FROM pg_wait_events diff --git a/src/test/regress/expected/tidscan.out b/src/test/regress/expected/tidscan.out index f133b5a4ac..2a079e996b 100644 --- a/src/test/regress/expected/tidscan.out +++ b/src/test/regress/expected/tidscan.out @@ -43,10 +43,26 @@ SELECT ctid, * FROM tidscan WHERE '(0,1)' = ctid; -- OR'd clauses EXPLAIN (COSTS OFF) SELECT ctid, * FROM tidscan WHERE ctid = '(0,2)' OR '(0,1)' = ctid; - QUERY PLAN --------------------------------------------------------------- + QUERY PLAN +------------------------------------------------------- + Tid Scan on tidscan + TID Cond: (ctid = ANY ('{"(0,2)","(0,1)"}'::tid[])) +(2 rows) + +SELECT ctid, * FROM tidscan WHERE ctid = '(0,2)' OR '(0,1)' = ctid; + ctid | id +-------+---- + (0,1) | 1 + (0,2) | 2 +(2 rows) + +SET enable_or_transformation = on; +EXPLAIN (COSTS OFF) +SELECT ctid, * FROM tidscan WHERE ctid = '(0,2)' OR '(0,1)' = ctid; + QUERY PLAN +------------------------------------------------------- Tid Scan on tidscan - TID Cond: ((ctid = '(0,2)'::tid) OR ('(0,1)'::tid = ctid)) + TID Cond: (ctid = ANY ('{"(0,2)","(0,1)"}'::tid[])) (2 rows) SELECT ctid, * FROM tidscan WHERE ctid = '(0,2)' OR '(0,1)' = ctid; @@ -56,6 +72,7 @@ SELECT ctid, * FROM tidscan WHERE ctid = '(0,2)' OR '(0,1)' = ctid; (0,2) | 2 (2 rows) +RESET enable_or_transformation; -- ctid = ScalarArrayOp - implemented as tidscan EXPLAIN (COSTS OFF) SELECT ctid, * FROM tidscan WHERE ctid = ANY(ARRAY['(0,1)', '(0,2)']::tid[]); diff --git a/src/test/regress/sql/create_index.sql b/src/test/regress/sql/create_index.sql index d49ce9f300..56fde15bc1 100644 --- a/src/test/regress/sql/create_index.sql +++ b/src/test/regress/sql/create_index.sql @@ -737,6 +737,41 @@ SELECT count(*) FROM tenk1 SELECT count(*) FROM tenk1 WHERE hundred = 42 AND (thousand = 42 OR thousand = 99); +SET enable_or_transformation = on; +EXPLAIN (COSTS OFF) +SELECT * FROM tenk1 + WHERE thousand = 42 AND (tenthous = 1 OR tenthous = 3 OR tenthous = 42); +SELECT * FROM tenk1 + WHERE thousand = 42 AND (tenthous = 1 OR tenthous = 3 OR tenthous = 42); + +EXPLAIN (COSTS OFF) +SELECT count(*) FROM tenk1 + WHERE hundred = 42 AND (thousand = 42 OR thousand = 99); +SELECT count(*) FROM tenk1 + WHERE hundred = 42 AND (thousand = 42 OR thousand = 99); +EXPLAIN (COSTS OFF) +SELECT count(*) FROM tenk1 + WHERE hundred = 42 AND (thousand < 42 OR thousand < 99 OR 43 > thousand OR 42 > thousand); + +EXPLAIN (COSTS OFF) +SELECT count(*) FROM tenk1 + WHERE thousand = 42 AND (tenthous = 1 OR tenthous = 3) OR thousand = 41; +SELECT count(*) FROM tenk1 + WHERE thousand = 42 AND (tenthous = 1 OR tenthous = 3) OR thousand = 41; + +EXPLAIN (COSTS OFF) +SELECT count(*) FROM tenk1 + WHERE hundred = 42 AND (thousand = 42 OR thousand = 99 OR tenthous < 2) OR thousand = 41; +SELECT count(*) FROM tenk1 + WHERE hundred = 42 AND (thousand = 42 OR thousand = 99 OR tenthous < 2) OR thousand = 41; + +EXPLAIN (COSTS OFF) +SELECT count(*) FROM tenk1 + WHERE hundred = 42 AND (thousand = 42 OR thousand = 41 OR thousand = 99 AND tenthous = 2); +SELECT count(*) FROM tenk1 + WHERE hundred = 42 AND (thousand = 42 OR thousand = 41 OR thousand = 99 AND tenthous = 2); +RESET enable_or_transformation; + -- -- Check behavior with duplicate index column contents -- diff --git a/src/test/regress/sql/join.sql b/src/test/regress/sql/join.sql index c4c6c7b8ba..1663608043 100644 --- a/src/test/regress/sql/join.sql +++ b/src/test/regress/sql/join.sql @@ -1408,6 +1408,16 @@ explain (costs off) select * from tenk1 a join tenk1 b on (a.unique1 = 1 and b.unique1 = 2) or ((a.unique2 = 3 or a.unique2 = 7) and b.hundred = 4); +SET enable_or_transformation = on; +explain (costs off) +select * from tenk1 a join tenk1 b on + (a.unique1 = 1 and b.unique1 = 2) or + ((a.unique2 = 3 or a.unique2 = 7) and b.hundred = 4); +explain (costs off) +select * from tenk1 a join tenk1 b on + (a.unique1 < 20 or a.unique1 = 3 or a.unique1 = 1 and b.unique1 = 2) or + ((a.unique2 = 3 or a.unique2 = 7) and b.hundred = 4); +RESET enable_or_transformation; -- -- test placement of movable quals in a parameterized join tree diff --git a/src/test/regress/sql/partition_prune.sql b/src/test/regress/sql/partition_prune.sql index 7ba6a9ff37..31ad4166d5 100644 --- a/src/test/regress/sql/partition_prune.sql +++ b/src/test/regress/sql/partition_prune.sql @@ -21,6 +21,12 @@ explain (costs off) select * from lp where a is not null; explain (costs off) select * from lp where a is null; explain (costs off) select * from lp where a = 'a' or a = 'c'; explain (costs off) select * from lp where a is not null and (a = 'a' or a = 'c'); + +SET enable_or_transformation = on; +explain (costs off) select * from lp where a = 'a' or a = 'c'; +explain (costs off) select * from lp where a is not null and (a = 'a' or a = 'c'); +RESET enable_or_transformation; + explain (costs off) select * from lp where a <> 'g'; explain (costs off) select * from lp where a <> 'a' and a <> 'd'; explain (costs off) select * from lp where a not in ('a', 'd'); @@ -99,6 +105,22 @@ explain (costs off) select * from rlp where a > 1 and a >=15; /* rlp3 onwards, i explain (costs off) select * from rlp where a = 1 and a = 3; /* empty */ explain (costs off) select * from rlp where (a = 1 and a = 3) or (a > 1 and a = 15); + +SET enable_or_transformation = on; +explain (costs off) select * from rlp where a = 1 or a = 7; +explain (costs off) select * from rlp where a = 1 or b = 'ab'; +explain (costs off) select * from rlp where a > 20 and a < 27; +explain (costs off) select * from rlp where a = 29; +explain (costs off) select * from rlp where a >= 29; +explain (costs off) select * from rlp where a < 1 or (a > 20 and a < 25); +explain (costs off) select * from rlp where a = 20 or a = 40; +explain (costs off) select * from rlp3 where a = 20; /* empty */ +explain (costs off) select * from rlp where a > 1 and a = 10; /* only default */ +explain (costs off) select * from rlp where a > 1 and a >=15; /* rlp3 onwards, including default */ +explain (costs off) select * from rlp where a = 1 and a = 3; /* empty */ +explain (costs off) select * from rlp where (a = 1 and a = 3) or (a > 1 and a = 15); +RESET enable_or_transformation; + -- multi-column keys create table mc3p (a int, b int, c int) partition by range (a, abs(b), c); create table mc3p_default partition of mc3p default; diff --git a/src/test/regress/sql/tidscan.sql b/src/test/regress/sql/tidscan.sql index 313e0fb9b6..0499bedb9e 100644 --- a/src/test/regress/sql/tidscan.sql +++ b/src/test/regress/sql/tidscan.sql @@ -22,6 +22,12 @@ EXPLAIN (COSTS OFF) SELECT ctid, * FROM tidscan WHERE ctid = '(0,2)' OR '(0,1)' = ctid; SELECT ctid, * FROM tidscan WHERE ctid = '(0,2)' OR '(0,1)' = ctid; +SET enable_or_transformation = on; +EXPLAIN (COSTS OFF) +SELECT ctid, * FROM tidscan WHERE ctid = '(0,2)' OR '(0,1)' = ctid; +SELECT ctid, * FROM tidscan WHERE ctid = '(0,2)' OR '(0,1)' = ctid; +RESET enable_or_transformation; + -- ctid = ScalarArrayOp - implemented as tidscan EXPLAIN (COSTS OFF) SELECT ctid, * FROM tidscan WHERE ctid = ANY(ARRAY['(0,1)', '(0,2)']::tid[]); diff --git a/src/tools/pgindent/typedefs.list b/src/tools/pgindent/typedefs.list index fc8b15d0cf..5cf444e95b 100644 --- a/src/tools/pgindent/typedefs.list +++ b/src/tools/pgindent/typedefs.list @@ -1656,6 +1656,8 @@ NumericVar OM_uint32 OP OSAPerGroupState +OrClauseGroupEntry +OrClauseGroupKey OSAPerQueryState OSInfo OSSLCipher -- 2.43.2