bool enable_mergejoin = true;
bool enable_hashjoin = true;
#ifdef PGXC
+bool enable_fast_query_shipping = true;
bool enable_remotejoin = true;
bool enable_remotegroup = true;
#endif
if (planner_hook)
result = (*planner_hook) (parse, cursorOptions, boundParams);
else
- result = standard_planner(parse, cursorOptions, boundParams);
+#ifdef PGXC
+ /*
+ * A Coordinator receiving a query from another Coordinator
+ * is not allowed to go into PGXC planner.
+ */
+ if (IS_PGXC_COORDINATOR && !IsConnFromCoord())
+ result = pgxc_planner(parse, cursorOptions, boundParams);
+ else
+#endif
+ result = standard_planner(parse, cursorOptions, boundParams);
return result;
}
fix_scan_list(root, splan->scan.plan.qual, rtoffset);
}
break;
+#ifdef PGXC
+ case T_RemoteQuery:
+ {
+#if 0
+ RemoteQuery *splan = (RemoteQuery *) plan;
+
+ /*
+ * If base_tlist is set, it means that we have a reduced remote
+ * query plan. So need to set the var references accordingly.
+ */
+ if (splan->base_tlist)
+ set_remote_references(root, splan, rtoffset);
+ splan->scan.plan.targetlist =
+ fix_scan_list(root, splan->scan.plan.targetlist, rtoffset);
+ splan->scan.plan.qual =
+ fix_scan_list(root, splan->scan.plan.qual, rtoffset);
+ splan->base_tlist =
+ fix_scan_list(root, splan->base_tlist, rtoffset);
+ splan->scan.scanrelid += rtoffset;
+#endif
+ }
+ break;
+#endif
case T_ForeignScan:
set_foreignscan_references(root, (ForeignScan *) plan, rtoffset);
break;
include $(top_builddir)/src/Makefile.global
OBJS = clauses.o joininfo.o orclauses.o pathnode.o placeholder.o \
- plancat.o predtest.o relnode.o restrictinfo.o tlist.o var.o
+ plancat.o predtest.o relnode.o restrictinfo.o tlist.o var.o pgxcship.o
include $(top_srcdir)/src/backend/common.mk
#include "catalog/pg_inherits_fn.h"
#include "catalog/pg_namespace.h"
#include "catalog/pg_proc.h"
+#ifdef PGXC
+#include "catalog/pg_trigger.h"
+#endif
#include "catalog/pg_type.h"
#include "catalog/pgxc_node.h"
#include "commands/trigger.h"
#include "parser/parsetree.h"
#include "parser/parse_coerce.h"
#include "parser/parse_type.h"
+#include "pgxc/locator.h"
#include "pgxc/pgxcnode.h"
#include "utils/lsyscache.h"
#include "utils/rel.h"
*/
SS_HAS_AGG_EXPR, /* it has aggregate expressions */
SS_UNSHIPPABLE_TYPE, /* the type of expression is unshippable */
- SS_UNSHIPPABLE_TRIGGER /* the type of trigger is unshippable */
+ SS_UNSHIPPABLE_TRIGGER, /* the type of trigger is unshippable */
+ SS_UPDATES_DISTRIBUTION_COLUMN /* query updates the distribution column */
} ShippabilityStat;
/* Manipulation of shippability reason */
static bool pgxc_query_contains_only_pg_catalog(List *rtable);
static bool pgxc_is_var_distrib_column(Var *var, List *rtable);
static bool pgxc_distinct_has_distcol(Query *query);
+static bool pgxc_targetlist_has_distcol(Query *query);
static ExecNodes *pgxc_FQS_find_datanodes_recurse(Node *node, Query *query,
Bitmapset **relids);
static ExecNodes *pgxc_FQS_datanodes_for_rtr(Index varno, Query *query);
* replicated JOIN, choose only one of them. If one of them is a
* preferred node choose that one, otherwise choose the first one.
*/
- if (IsExecNodesReplicated(exec_nodes) &&
+ if (IsLocatorReplicated(exec_nodes->baselocatortype) &&
exec_nodes->accesstype == RELATION_ACCESS_READ)
{
List *tmp_list = exec_nodes->nodeList;
query->jointree->quals, rel_access);
else
rel_exec_nodes = GetRelationNodes(rel_loc_info, (Datum) 0,
- true, InvalidOid, rel_access);
+ true, rel_access);
if (!rel_exec_nodes)
return NULL;
if (query->distinctClause && !pgxc_distinct_has_distcol(query))
pgxc_set_shippability_reason(sc_context, SS_NEED_SINGLENODE);
+
+ if ((query->commandType == CMD_UPDATE) &&
+ pgxc_targetlist_has_distcol(query))
+ pgxc_set_shippability_reason(sc_context, SS_UPDATES_DISTRIBUTION_COLUMN);
+
/*
* walk the entire query tree to analyse the query. We will walk the
if (sc_context->sc_max_varlevelsup != 0)
pgxc_set_shippability_reason(sc_context, SS_VARLEVEL);
- /* Check shippability of triggers on this query */
- if (query->commandType == CMD_UPDATE ||
- query->commandType == CMD_INSERT ||
- query->commandType == CMD_DELETE)
- {
- RangeTblEntry *rte = (RangeTblEntry *)
- list_nth(query->rtable, query->resultRelation - 1);
-
- if (!pgxc_check_triggers_shippability(rte->relid,
- query->commandType))
- pgxc_set_shippability_reason(sc_context,
- SS_UNSHIPPABLE_TRIGGER);
-
- /*
- * PGXCTODO: For the time being Postgres-XC does not support
- * global constraints, but once it does it will be necessary
- * to add here evaluation of the shippability of indexes and
- * constraints of the relation used for INSERT/UPDATE/DELETE.
- */
- }
-
/*
* Walk the join tree of the query and find the
* Datanodes needed for evaluating this query
case T_PlaceHolderVar:
case T_AppendRelInfo:
case T_PlaceHolderInfo:
+ case T_OnConflictExpr:
+ case T_WithCheckOption:
{
/* PGXCTODO: till we exhaust this list */
pgxc_set_shippability_reason(sc_context, SS_UNSUPPORTED_EXPR);
static bool
pgxc_query_needs_coord(Query *query)
{
- /*
- * If the query is an EXEC DIRECT on the same Coordinator where it's fired,
- * it should not be shipped
- */
- if (query->is_local)
- return true;
/*
* If the query involves just the catalog tables, and is not an EXEC DIRECT
* statement, it can be evaluated completely on the Coordinator. No need to
else
return NULL;
}
+
+static
+bool pgxc_targetlist_has_distcol(Query *query)
+{
+ RangeTblEntry *rte = rt_fetch(query->resultRelation, query->rtable);
+ RelationLocInfo *rel_loc_info;
+ ListCell *lc;
+ const char *distcol;
+
+ /* distribution column only applies to the relations */
+ if (rte->rtekind != RTE_RELATION ||
+ rte->relkind != RELKIND_RELATION)
+ return false;
+ rel_loc_info = GetRelationLocInfo(rte->relid);
+ if (!rel_loc_info)
+ return false;
+
+ distcol = GetRelationDistribColumn(rel_loc_info);
+ if (!distcol)
+ return false;
+
+ foreach(lc, query->targetList)
+ {
+ TargetEntry *tle = (TargetEntry *) lfirst(lc);
+
+ if (tle->resjunk)
+ continue;
+ if (strcmp(tle->resname, distcol) == 0)
+ return true;
+ }
+ return false;
+}
bool *hasprimary);
static int locate_modulo_select(Locator *self, Datum value, bool isnull,
bool *hasprimary);
+static Expr * pgxc_find_distcol_expr(Index varno,
+ AttrNumber attrNum,
+ Node *quals);
#endif
static const unsigned int xc_mod_m[] =
{0x7fffffff, 0x7fffffff, 0x7fffffff, 0x7fffffff, 0x7fffffff, 0x7fffffff}
};
+/*
+ * GetPreferredReplicationNode
+ * Pick any Datanode from given list, however fetch a preferred node first.
+ */
+List *
+GetPreferredReplicationNode(List *relNodes)
+{
+ ListCell *item;
+ int nodeid = -1;
+
+ if (list_length(relNodes) <= 0)
+ elog(ERROR, "a list of nodes should have at least one node");
+
+ foreach(item, relNodes)
+ {
+ int cnt_nodes;
+ char nodetype = PGXC_NODE_DATANODE;
+ for (cnt_nodes = 0;
+ cnt_nodes < num_preferred_data_nodes && nodeid < 0;
+ cnt_nodes++)
+ {
+ if (PGXCNodeGetNodeId(preferred_data_node[cnt_nodes],
+ &nodetype) == lfirst_int(item))
+ nodeid = lfirst_int(item);
+ }
+ if (nodeid >= 0)
+ break;
+ }
+ if (nodeid < 0)
+ return list_make1_int(linitial_int(relNodes));
+
+ return list_make1_int(nodeid);
+}
+
/*
* GetAnyDataNode
* Pick any data node from given set, but try a preferred node
return self->nodeCount;
}
#endif
+
+/*
+ * GetRelationNodes
+ *
+ * Get list of relation nodes
+ * If the table is replicated and we are reading, we can just pick one.
+ * If the table is partitioned, we apply partitioning column value, if possible.
+ *
+ * If the relation is partitioned, partValue will be applied if present
+ * (indicating a value appears for partitioning column), otherwise it
+ * is ignored.
+ *
+ * preferredNodes is only used when for replicated tables. If set, it will
+ * use one of the nodes specified if the table is replicated on it.
+ * This helps optimize for avoiding introducing additional nodes into the
+ * transaction.
+ *
+ * The returned List is a copy, so it should be freed when finished.
+ */
+ExecNodes *
+GetRelationNodes(RelationLocInfo *rel_loc_info, Datum valueForDistCol,
+ bool isValueNull,
+ RelationAccessType accessType)
+{
+ ExecNodes *exec_nodes;
+ int *nodenums;
+ int i, count;
+ Locator *locator;
+ Oid typeOfValueForDistCol = InvalidOid;
+
+ if (rel_loc_info == NULL)
+ return NULL;
+
+
+ if (IsLocatorDistributedByValue(rel_loc_info->locatorType))
+ {
+ /* A sufficient lock level needs to be taken at a higher level */
+ Relation rel = relation_open(rel_loc_info->relid, NoLock);
+ TupleDesc tupDesc = RelationGetDescr(rel);
+ Form_pg_attribute *attr = tupDesc->attrs;
+ /* Get the hash type of relation */
+ typeOfValueForDistCol = attr[rel_loc_info->partAttrNum - 1]->atttypid;
+ relation_close(rel, NoLock);
+ }
+
+ exec_nodes = makeNode(ExecNodes);
+ exec_nodes->baselocatortype = rel_loc_info->locatorType;
+ exec_nodes->accesstype = accessType;
+
+ locator = createLocator(rel_loc_info->locatorType,
+ accessType,
+ typeOfValueForDistCol,
+ LOCATOR_LIST_LIST,
+ 0,
+ (void *)rel_loc_info->nodeList,
+ (void **)&nodenums,
+ false);
+ count = GET_NODES(locator, valueForDistCol, isValueNull, NULL);
+
+ for (i = 0; i < count; i++)
+ exec_nodes->nodeList = lappend_int(exec_nodes->nodeList, nodenums[i]);
+
+ freeLocator(locator);
+ return exec_nodes;
+}
+
+/*
+ * GetRelationNodesByQuals
+ * A wrapper around GetRelationNodes to reduce the node list by looking at the
+ * quals. varno is assumed to be the varno of reloid inside the quals. No check
+ * is made to see if that's correct.
+ */
+ExecNodes *
+GetRelationNodesByQuals(Oid reloid, Index varno, Node *quals,
+ RelationAccessType relaccess)
+{
+ RelationLocInfo *rel_loc_info = GetRelationLocInfo(reloid);
+ Expr *distcol_expr = NULL;
+ ExecNodes *exec_nodes;
+ Datum distcol_value;
+ bool distcol_isnull;
+
+ if (!rel_loc_info)
+ return NULL;
+ /*
+ * If the table distributed by value, check if we can reduce the Datanodes
+ * by looking at the qualifiers for this relation
+ */
+ if (IsRelationDistributedByValue(rel_loc_info))
+ {
+ Oid disttype = get_atttype(reloid, rel_loc_info->partAttrNum);
+ int32 disttypmod = get_atttypmod(reloid, rel_loc_info->partAttrNum);
+ distcol_expr = pgxc_find_distcol_expr(varno, rel_loc_info->partAttrNum,
+ quals);
+ /*
+ * If the type of expression used to find the Datanode, is not same as
+ * the distribution column type, try casting it. This is same as what
+ * will happen in case of inserting that type of expression value as the
+ * distribution column value.
+ */
+ if (distcol_expr)
+ {
+ distcol_expr = (Expr *)coerce_to_target_type(NULL,
+ (Node *)distcol_expr,
+ exprType((Node *)distcol_expr),
+ disttype, disttypmod,
+ COERCION_ASSIGNMENT,
+ COERCE_IMPLICIT_CAST, -1);
+ /*
+ * PGXC_FQS_TODO: We should set the bound parameters here, but we don't have
+ * PlannerInfo struct and we don't handle them right now.
+ * Even if constant expression mutator changes the expression, it will
+ * only simplify it, keeping the semantics same
+ */
+ distcol_expr = (Expr *)eval_const_expressions(NULL,
+ (Node *)distcol_expr);
+ }
+ }
+
+ if (distcol_expr && IsA(distcol_expr, Const))
+ {
+ Const *const_expr = (Const *)distcol_expr;
+ distcol_value = const_expr->constvalue;
+ distcol_isnull = const_expr->constisnull;
+ }
+ else
+ {
+ distcol_value = (Datum) 0;
+ distcol_isnull = true;
+ }
+
+ exec_nodes = GetRelationNodes(rel_loc_info, distcol_value,
+ distcol_isnull,
+ relaccess);
+ return exec_nodes;
+}
+
+/*
+ * GetRelationDistribColumn
+ * Return hash column name for relation or NULL if relation is not distributed.
+ */
+char *
+GetRelationDistribColumn(RelationLocInfo *locInfo)
+{
+ /* No relation, so simply leave */
+ if (!locInfo)
+ return NULL;
+
+ /* No distribution column if relation is not distributed with a key */
+ if (!IsRelationDistributedByValue(locInfo))
+ return NULL;
+
+ /* Return column name */
+ return get_attname(locInfo->relid, locInfo->partAttrNum);
+}
+
+/*
+ * pgxc_find_distcol_expr
+ * Search through the quals provided and find out an expression which will give
+ * us value of distribution column if exists in the quals. Say for a table
+ * tab1 (val int, val2 int) distributed by hash(val), a query "SELECT * FROM
+ * tab1 WHERE val = fn(x, y, z) and val2 = 3", fn(x,y,z) is the expression which
+ * decides the distribution column value in the rows qualified by this query.
+ * Hence return fn(x, y, z). But for a query "SELECT * FROM tab1 WHERE val =
+ * fn(x, y, z) || val2 = 3", there is no expression which decides the values
+ * distribution column val can take in the qualified rows. So, in such cases
+ * this function returns NULL.
+ */
+static Expr *
+pgxc_find_distcol_expr(Index varno,
+ AttrNumber attrNum,
+ Node *quals)
+{
+ List *lquals;
+ ListCell *qual_cell;
+
+ /* If no quals, no distribution column expression */
+ if (!quals)
+ return NULL;
+
+ /* Convert the qualification into List if it's not already so */
+ if (!IsA(quals, List))
+ lquals = make_ands_implicit((Expr *)quals);
+ else
+ lquals = (List *)quals;
+
+ /*
+ * For every ANDed expression, check if that expression is of the form
+ * <distribution_col> = <expr>. If so return expr.
+ */
+ foreach(qual_cell, lquals)
+ {
+ Expr *qual_expr = (Expr *)lfirst(qual_cell);
+ OpExpr *op;
+ Expr *lexpr;
+ Expr *rexpr;
+ Var *var_expr;
+ Expr *distcol_expr;
+
+ if (!IsA(qual_expr, OpExpr))
+ continue;
+ op = (OpExpr *)qual_expr;
+ /* If not a binary operator, it can not be '='. */
+ if (list_length(op->args) != 2)
+ continue;
+
+ lexpr = linitial(op->args);
+ rexpr = lsecond(op->args);
+
+ /*
+ * If either of the operands is a RelabelType, extract the Var in the RelabelType.
+ * A RelabelType represents a "dummy" type coercion between two binary compatible datatypes.
+ * If we do not handle these then our optimization does not work in case of varchar
+ * For example if col is of type varchar and is the dist key then
+ * select * from vc_tab where col = 'abcdefghijklmnopqrstuvwxyz';
+ * should be shipped to one of the nodes only
+ */
+ if (IsA(lexpr, RelabelType))
+ lexpr = ((RelabelType*)lexpr)->arg;
+ if (IsA(rexpr, RelabelType))
+ rexpr = ((RelabelType*)rexpr)->arg;
+
+ /*
+ * If either of the operands is a Var expression, assume the other
+ * one is distribution column expression. If none is Var check next
+ * qual.
+ */
+ if (IsA(lexpr, Var))
+ {
+ var_expr = (Var *)lexpr;
+ distcol_expr = rexpr;
+ }
+ else if (IsA(rexpr, Var))
+ {
+ var_expr = (Var *)rexpr;
+ distcol_expr = lexpr;
+ }
+ else
+ continue;
+ /*
+ * If Var found is not the distribution column of required relation,
+ * check next qual
+ */
+ if (var_expr->varno != varno || var_expr->varattno != attrNum)
+ continue;
+ /*
+ * If the operator is not an assignment operator, check next
+ * constraint. An operator is an assignment operator if it's
+ * mergejoinable or hashjoinable. Beware that not every assignment
+ * operator is mergejoinable or hashjoinable, so we might leave some
+ * oportunity. But then we have to rely on the opname which may not
+ * be something we know to be equality operator as well.
+ */
+ if (!op_mergejoinable(op->opno, exprType((Node *)lexpr)) &&
+ !op_hashjoinable(op->opno, exprType((Node *)lexpr)))
+ continue;
+ /* Found the distribution column expression return it */
+ return distcol_expr;
+ }
+ /* Exhausted all quals, but no distribution column expression */
+ return NULL;
+}
#include "nodes/nodes.h"
#include "nodes/parsenodes.h"
#include "optimizer/clauses.h"
+#include "optimizer/cost.h"
#include "optimizer/planmain.h"
#include "optimizer/planner.h"
#include "optimizer/tlist.h"
/* Forbid unsafe SQL statements */
bool StrictStatementChecking = true;
+static void validate_part_col_updatable(const Query *query);
+static bool contains_temp_tables(List *rtable);
+static PlannedStmt *pgxc_FQS_planner(Query *query, ParamListInfo boundParams);
+static RemoteQuery *pgxc_FQS_create_remote_plan(Query *query,
+ ExecNodes *exec_nodes,
+ bool is_exec_direct);
+static CombineType get_plan_combine_type(CmdType commandType, char baselocatortype);
+
#ifdef XCP
/*
* AddRemoteQueryNode
return result;
}
+/*
+ * Returns true if at least one temporary table is in use
+ * in query (and its subqueries)
+ */
+static bool
+contains_temp_tables(List *rtable)
+{
+ ListCell *item;
+
+ foreach(item, rtable)
+ {
+ RangeTblEntry *rte = (RangeTblEntry *) lfirst(item);
+
+ if (rte->rtekind == RTE_RELATION)
+ {
+ if (IsTempTable(rte->relid))
+ return true;
+ }
+ else if (rte->rtekind == RTE_SUBQUERY &&
+ contains_temp_tables(rte->subquery->rtable))
+ return true;
+ }
+
+ return false;
+}
+
+/*
+ * get_plan_combine_type - determine combine type
+ *
+ * COMBINE_TYPE_SAME - for replicated updates
+ * COMBINE_TYPE_SUM - for hash and round robin updates
+ * COMBINE_TYPE_NONE - for operations where row_count is not applicable
+ *
+ * return NULL if it is not safe to be done in a single step.
+ */
+static CombineType
+get_plan_combine_type(CmdType commandType, char baselocatortype)
+{
+
+ switch (commandType)
+ {
+ case CMD_INSERT:
+ case CMD_UPDATE:
+ case CMD_DELETE:
+ return baselocatortype == LOCATOR_TYPE_REPLICATED ?
+ COMBINE_TYPE_SAME : COMBINE_TYPE_SUM;
+
+ default:
+ return COMBINE_TYPE_NONE;
+ }
+ /* quiet compiler warning */
+ return COMBINE_TYPE_NONE;
+}
+
+
+/*
+ * Build up a QueryPlan to execute on.
+ *
+ * This functions tries to find out whether
+ * 1. The statement can be shipped to the Datanode and Coordinator is needed
+ * only as a proxy - in which case, it creates a single node plan.
+ * 2. The statement can be evaluated on the Coordinator completely - thus no
+ * query shipping is involved and standard_planner() is invoked to plan the
+ * statement
+ * 3. The statement needs Coordinator as well as Datanode for evaluation -
+ * again we use standard_planner() to plan the statement.
+ *
+ * The plan generated in either of the above cases is returned.
+ */
+PlannedStmt *
+pgxc_planner(Query *query, int cursorOptions, ParamListInfo boundParams)
+{
+ PlannedStmt *result;
+
+ /* see if can ship the query completely */
+ result = pgxc_FQS_planner(query, boundParams);
+ if (result)
+ return result;
+
+ /* we need Coordinator for evaluation, invoke standard planner */
+ result = standard_planner(query, cursorOptions, boundParams);
+ return result;
+}
+
+/*
+ * pgxc_FQS_planner
+ * The routine tries to see if the statement can be completely evaluated on the
+ * Datanodes. In such cases Coordinator is not needed to evaluate the statement,
+ * and just acts as a proxy. A statement can be completely shipped to the remote
+ * node if every row of the result can be evaluated on a single Datanode.
+ * For example:
+ *
+ * 1. SELECT * FROM tab1; where tab1 is a distributed table - Every row of the
+ * result set can be evaluated at a single Datanode. Hence this statement is
+ * completely shippable even though many Datanodes are involved in evaluating
+ * complete result set. In such case Coordinator will be able to gather rows
+ * arisign from individual Datanodes and proxy the result to the client.
+ *
+ * 2. SELECT count(*) FROM tab1; where tab1 is a distributed table - there is
+ * only one row in the result but it needs input from all the Datanodes. Hence
+ * this is not completely shippable.
+ *
+ * 3. SELECT count(*) FROM tab1; where tab1 is replicated table - since result
+ * can be obtained from a single Datanode, this is a completely shippable
+ * statement.
+ *
+ * fqs in the name of function is acronym for fast query shipping.
+ */
+static PlannedStmt *
+pgxc_FQS_planner(Query *query, ParamListInfo boundParams)
+{
+ PlannedStmt *result;
+ PlannerGlobal *glob;
+ PlannerInfo *root;
+ ExecNodes *exec_nodes;
+ Plan *top_plan;
+
+ /* Try by-passing standard planner, if fast query shipping is enabled */
+ if (!enable_fast_query_shipping)
+ return NULL;
+
+ /* Do not FQS cursor statements */
+ if (query->utilityStmt &&
+ IsA(query->utilityStmt, DeclareCursorStmt))
+ return NULL;
+ /*
+ * If the query can not be or need not be shipped to the Datanodes, don't
+ * create any plan here. standard_planner() will take care of it.
+ */
+ exec_nodes = pgxc_is_query_shippable(query, 0);
+ if (exec_nodes == NULL)
+ return NULL;
+
+ glob = makeNode(PlannerGlobal);
+ glob->boundParams = boundParams;
+ /* Create a PlannerInfo data structure, usually it is done for a subquery */
+ root = makeNode(PlannerInfo);
+ root->parse = query;
+ root->glob = glob;
+ root->query_level = 1;
+ root->planner_cxt = CurrentMemoryContext;
+
+ /*
+ * We decided to ship the query to the Datanode/s, create a RemoteQuery node
+ * for the same.
+ */
+ top_plan = (Plan *)pgxc_FQS_create_remote_plan(query, exec_nodes, false);
+
+ /*
+ * Just before creating the PlannedStmt, do some final cleanup
+ * We need to save plan dependencies, so that dropping objects will
+ * invalidate the cached plan if it depends on those objects. Table
+ * dependencies are available in glob->relationOids and all other
+ * dependencies are in glob->invalItems. These fields can be retrieved
+ * through set_plan_references().
+ */
+ top_plan = set_plan_references(root, top_plan);
+
+ /* build the PlannedStmt result */
+ result = makeNode(PlannedStmt);
+ /* Try and set what we can, rest must have been zeroed out by makeNode() */
+ result->commandType = query->commandType;
+ result->canSetTag = query->canSetTag;
+ result->utilityStmt = query->utilityStmt;
+
+ /* Set result relations */
+ if (query->commandType != CMD_SELECT)
+ result->resultRelations = list_make1_int(query->resultRelation);
+ result->planTree = top_plan;
+ result->rtable = query->rtable;
+ result->relationOids = glob->relationOids;
+ result->invalItems = glob->invalItems;
+
+ return result;
+}
+
+static RemoteQuery *
+pgxc_FQS_create_remote_plan(Query *query, ExecNodes *exec_nodes, bool is_exec_direct)
+{
+ RemoteQuery *query_step;
+ StringInfoData buf;
+ RangeTblEntry *dummy_rte;
+
+ /* EXECUTE DIRECT statements have their RemoteQuery node already built when analyzing */
+ if (is_exec_direct)
+ {
+ Assert(IsA(query->utilityStmt, RemoteQuery));
+ query_step = (RemoteQuery *)query->utilityStmt;
+ query->utilityStmt = NULL;
+ }
+ else
+ {
+ query_step = makeNode(RemoteQuery);
+ query_step->combine_type = COMBINE_TYPE_NONE;
+ query_step->exec_type = EXEC_ON_DATANODES;
+ query_step->exec_direct_type = EXEC_DIRECT_NONE;
+ query_step->exec_nodes = exec_nodes;
+ }
+
+ Assert(query_step->exec_nodes);
+
+ /* Deparse query tree to get step query. */
+ if (query_step->sql_statement == NULL)
+ {
+ initStringInfo(&buf);
+ /*
+ * We always finalise aggregates on datanodes for FQS.
+ * Use the expressions for ORDER BY or GROUP BY clauses.
+ */
+ deparse_query(query, &buf, NIL, true, false);
+ query_step->sql_statement = pstrdup(buf.data);
+ pfree(buf.data);
+ }
+
+ /* Optimize multi-node handling */
+ query_step->read_only = (query->commandType == CMD_SELECT && !query->hasForUpdate);
+ query_step->has_row_marks = query->hasForUpdate;
+
+ /* Check if temporary tables are in use in query */
+ /* PGXC_FQS_TODO: scanning the rtable again for the queries should not be
+ * needed. We should be able to find out if the query has a temporary object
+ * while finding nodes for the objects. But there is no way we can convey
+ * that information here. Till such a connection is available, this is it.
+ */
+ if (contains_temp_tables(query->rtable))
+ query_step->is_temp = true;
+
+ /*
+ * We need to evaluate some expressions like the ExecNodes->en_expr at
+ * Coordinator, prepare those for evaluation. Ideally we should call
+ * preprocess_expression, but it needs PlannerInfo structure for the same
+ */
+ fix_opfuncids((Node *)(query_step->exec_nodes->en_expr));
+ /*
+ * PGXCTODO
+ * When Postgres runs insert into t (a) values (1); against table
+ * defined as create table t (a int, b int); the plan is looking
+ * like insert into t (a,b) values (1,null);
+ * Later executor is verifying plan, to make sure table has not
+ * been altered since plan has been created and comparing table
+ * definition with plan target list and output error if they do
+ * not match.
+ * I could not find better way to generate targetList for pgxc plan
+ * then call standard planner and take targetList from the plan
+ * generated by Postgres.
+ */
+ query_step->combine_type = get_plan_combine_type(
+ query->commandType, query_step->exec_nodes->baselocatortype);
+
+ /*
+ * Create a dummy RTE for the remote query being created. Append the dummy
+ * range table entry to the range table. Note that this modifies the master
+ * copy the caller passed us, otherwise e.g EXPLAIN VERBOSE will fail to
+ * find the rte the Vars built below refer to. Also create the tuple
+ * descriptor for the result of this query from the base_tlist (targetlist
+ * we used to generate the remote node query).
+ */
+ dummy_rte = makeNode(RangeTblEntry);
+ dummy_rte->rtekind = RTE_REMOTE_DUMMY;
+ /* Use a dummy relname... */
+ if (is_exec_direct)
+ dummy_rte->relname = "__EXECUTE_DIRECT__";
+ else
+ dummy_rte->relname = "__REMOTE_FQS_QUERY__";
+ dummy_rte->eref = makeAlias("__REMOTE_FQS_QUERY__", NIL);
+ /* Rest will be zeroed out in makeNode() */
+
+ query->rtable = lappend(query->rtable, dummy_rte);
+ query_step->scan.scanrelid = list_length(query->rtable);
+ query_step->scan.plan.targetlist = query->targetList;
+ query_step->base_tlist = query->targetList;
+
+ return query_step;
+}
+
+/*
+ * validate whether partition column of a table is being updated
+ */
+static void
+validate_part_col_updatable(const Query *query)
+{
+ RangeTblEntry *rte;
+ RelationLocInfo *rel_loc_info;
+ ListCell *lc;
+
+ /* Make sure there is one table at least */
+ if (query->rtable == NULL)
+ return;
+
+ rte = (RangeTblEntry *) list_nth(query->rtable, query->resultRelation - 1);
+
+
+ if (rte != NULL && rte->relkind != RELKIND_RELATION)
+ /* Bad relation type */
+ return;
+
+ /* See if we have the partitioned case. */
+ rel_loc_info = GetRelationLocInfo(rte->relid);
+
+ /* Any column updation on local relations is fine */
+ if (!rel_loc_info)
+ return;
+
+ /* Only relations distributed by value can be checked */
+ if (IsRelationDistributedByValue(rel_loc_info))
+ {
+ /* It is a partitioned table, check partition column in targetList */
+ foreach(lc, query->targetList)
+ {
+ TargetEntry *tle = (TargetEntry *) lfirst(lc);
+
+ /* Nothing to do for a junk entry */
+ if (tle->resjunk)
+ continue;
+
+ /*
+ * See if we have a constant expression comparing against the
+ * designated partitioned column
+ */
+ if (strcmp(tle->resname, GetRelationDistribColumn(rel_loc_info)) == 0)
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_COLUMN_REFERENCE),
+ (errmsg("Partition column can't be updated in current version"))));
+ }
+ }
+}
+
if (exec_nodes)
{
+ if (exec_nodes->en_expr)
+ {
+ /* execution time determining of target Datanodes */
+ bool isnull;
+ ExprState *estate = ExecInitExpr(exec_nodes->en_expr,
+ (PlanState *) planstate);
+ Datum partvalue = ExecEvalExpr(estate,
+ planstate->combiner.ss.ps.ps_ExprContext,
+ &isnull,
+ NULL);
+ RelationLocInfo *rel_loc_info = GetRelationLocInfo(exec_nodes->en_relid);
+ /* PGXCTODO what is the type of partvalue here */
+ ExecNodes *nodes = GetRelationNodes(rel_loc_info,
+ partvalue,
+ isnull,
+ exec_nodes->accesstype);
+ /*
+ * en_expr is set by pgxc_set_en_expr only for distributed
+ * relations while planning DMLs, hence a select for update
+ * on a replicated table here is an assertion
+ */
+ Assert(!(exec_nodes->accesstype == RELATION_ACCESS_READ_FOR_UPDATE &&
+ IsRelationReplicated(rel_loc_info)));
+
+ if (nodes)
+ {
+ nodelist = nodes->nodeList;
+ primarynode = nodes->primarynodelist;
+ pfree(nodes);
+ }
+ FreeRelationLocInfo(rel_loc_info);
+ }
+ else if (OidIsValid(exec_nodes->en_relid))
+ {
+ RelationLocInfo *rel_loc_info = GetRelationLocInfo(exec_nodes->en_relid);
+ ExecNodes *nodes = GetRelationNodes(rel_loc_info, 0, true, exec_nodes->accesstype);
+
+ /*
+ * en_relid is set only for DMLs, hence a select for update on a
+ * replicated table here is an assertion
+ */
+ Assert(!(exec_nodes->accesstype == RELATION_ACCESS_READ_FOR_UPDATE &&
+ IsRelationReplicated(rel_loc_info)));
+
+ /* Use the obtained list for given table */
+ if (nodes)
+ nodelist = nodes->nodeList;
+
+ /*
+ * Special handling for ROUND ROBIN distributed tables. The target
+ * node must be determined at the execution time
+ */
+ if (rel_loc_info->locatorType == LOCATOR_TYPE_RROBIN && nodes)
+ {
+ nodelist = nodes->nodeList;
+ primarynode = nodes->primarynodelist;
+ }
+ else if (nodes)
+ {
+ if (exec_type == EXEC_ON_DATANODES || exec_type == EXEC_ON_ALL_NODES)
+ {
+ nodelist = exec_nodes->nodeList;
+ primarynode = exec_nodes->primarynodelist;
+ }
+ }
+
+ if (nodes)
+ pfree(nodes);
+ FreeRelationLocInfo(rel_loc_info);
+ }
+ else
{
if (exec_type == EXEC_ON_DATANODES || exec_type == EXEC_ON_ALL_NODES)
nodelist = exec_nodes->nodeList;
/* need to use Extended Query Protocol */
int fetch = 0;
bool prepared = false;
+ char nodetype = PGXC_NODE_DATANODE;
+
+ /* if prepared statement is referenced see if it is already
+ * exist */
+ if (step->statement)
+ prepared =
+ ActivateDatanodeStatementOnNode(step->statement,
+ PGXCNodeGetNodeId(connection->nodeoid,
+ &nodetype));
/*
* execute and fetch rows only if they will be consumed
remotestate = makeNode(RemoteQueryState);
combiner = (ResponseCombiner *) remotestate;
- InitResponseCombiner(combiner, 0, COMBINE_TYPE_NONE);
+ InitResponseCombiner(combiner, 0, node->combine_type);
combiner->ss.ps.plan = (Plan *) node;
combiner->ss.ps.state = estate;
combiner->request_type = REQUEST_TYPE_QUERY;
ExecInitResultTupleSlot(estate, &combiner->ss.ps);
- if (node->scan.plan.targetlist)
- ExecAssignResultTypeFromTL((PlanState *) remotestate);
+ ExecAssignResultTypeFromTL((PlanState *) remotestate);
/*
* If there are parameters supplied, get them into a form to be sent to the
return (DatumGetChar(datum) == PROVOLATILE_IMMUTABLE);
}
-/*
- * Check whether the ExprState node should be evaluated in foreign server.
- *
- * An expression which consists of expressions below will be evaluated in
- * the foreign server.
- * - constant value
- * - variable (foreign table column)
- * - external parameter (parameter of prepared statement)
- * - array
- * - bool expression (AND/OR/NOT)
- * - NULL test (IS [NOT] NULL)
- * - operator
- * - IMMUTABLE only
- * - It is required that the meaning of the operator be the same as the
- * local server in the foreign server.
- * - function
- * - IMMUTABLE only
- * - It is required that the meaning of the operator be the same as the
- * local server in the foreign server.
- * - scalar array operator (ANY/ALL)
- */
-bool
-pgxc_is_expr_shippable(Expr *node, bool *has_aggs)
-{
- return false;
-}
cparsetree->utilityStmt = (Node *) create_stmt;
initStringInfo(&cquery);
- deparse_query(cparsetree, &cquery, NIL);
+ deparse_query(cparsetree, &cquery, NIL, false, false);
/* Finally, fire off the query to run the DDL */
ProcessUtility(cparsetree->utilityStmt, cquery.data, PROCESS_UTILITY_QUERY,
/* Get the SELECT query string */
initStringInfo(&cquery);
- deparse_query((Query *)stmt->query, &cquery, NIL);
+ deparse_query((Query *)stmt->query, &cquery, NIL, false, false);
selectstr = pstrdup(cquery.data);
/* Now, finally build the INSERT INTO statement */
bool varprefix; /* TRUE to print prefixes on Vars */
ParseExprKind special_exprkind; /* set only for exprkinds needing
* special handling */
+#ifdef PGXC
+ bool finalise_aggs; /* should Datanode finalise the aggregates? */
+ bool sortgroup_colno;/* instead of expression use resno for
+ * sortgrouprefs.
+ */
+#endif /* PGXC */
} deparse_context;
/*
deparse_context *context);
static void get_query_def(Query *query, StringInfo buf, List *parentnamespace,
TupleDesc resultDesc,
- int prettyFlags, int wrapColumn, int startIndent);
+ int prettyFlags, int wrapColumn, int startIndent
+#ifdef PGXC
+ , bool finalise_aggregates, bool sortgroup_colno
+#endif /* PGXC */
+ );
static void get_values_def(List *values_lists, deparse_context *context);
static void get_with_clause(Query *query, deparse_context *context);
static void get_select_query_def(Query *query, deparse_context *context,
{
query = (Query *) lfirst(action);
get_query_def(query, buf, NIL, NULL,
- prettyFlags, WRAP_COLUMN_DEFAULT, 0);
+ prettyFlags, WRAP_COLUMN_DEFAULT, 0
+#ifdef PGXC
+ , false, false
+#endif /* PGXC */
+ );
if (prettyFlags)
appendStringInfoString(buf, ";\n");
else
query = (Query *) linitial(actions);
get_query_def(query, buf, NIL, NULL,
- prettyFlags, WRAP_COLUMN_DEFAULT, 0);
- appendStringInfoChar(buf, ';');
+ prettyFlags, WRAP_COLUMN_DEFAULT, 0
+#ifdef PGXC
+ , false, false
+#endif /* PGXC */
+ );
+ appendStringInfo(buf, ";");
}
}
ev_relation = heap_open(ev_class, AccessShareLock);
get_query_def(query, buf, NIL, RelationGetDescr(ev_relation),
- prettyFlags, wrapColumn, 0);
- appendStringInfoChar(buf, ';');
+ prettyFlags, wrapColumn, 0
+#ifdef PGXC
+ , false, false
+#endif /* PGXC */
+ );
+ appendStringInfo(buf, ";");
heap_close(ev_relation, AccessShareLock);
}
* ----------
*/
void
-deparse_query(Query *query, StringInfo buf, List *parentnamespace)
+deparse_query(Query *query, StringInfo buf, List *parentnamespace,
+ bool finalise_aggs, bool sortgroup_colno)
{
- get_query_def(query, buf, parentnamespace, NULL, 0, 0, 0);
+ get_query_def(query, buf, parentnamespace, NULL, 0, 0, 0, finalise_aggs,
+ sortgroup_colno);
}
/* code borrowed from get_insert_query_def */
/* Add the SELECT */
get_query_def(select_rte->subquery, buf, NIL, NULL,
context.prettyFlags, context.wrapColumn,
- context.indentLevel);
+ context.indentLevel,
+ context.finalise_aggs, context.sortgroup_colno);
}
else if (values_rte)
{
static void
get_query_def(Query *query, StringInfo buf, List *parentnamespace,
TupleDesc resultDesc,
- int prettyFlags, int wrapColumn, int startIndent)
+ int prettyFlags, int wrapColumn, int startIndent,
+ bool finalise_aggs, bool sortgroup_colno)
{
deparse_context context;
deparse_namespace dpns;
context.wrapColumn = wrapColumn;
context.indentLevel = startIndent;
context.special_exprkind = EXPR_KIND_NONE;
+ context.finalise_aggs = finalise_aggs;
+ context.sortgroup_colno = sortgroup_colno;
set_deparse_for_query(&dpns, query, parentnamespace);
appendContextKeyword(context, "", 0, 0, 0);
get_query_def((Query *) cte->ctequery, buf, context->namespaces, NULL,
context->prettyFlags, context->wrapColumn,
- context->indentLevel);
+ context->indentLevel,
+ context->finalise_aggs,
+ context->sortgroup_colno);
if (PRETTY_INDENT(context))
appendContextKeyword(context, "", 0, 0, 0);
appendStringInfoChar(buf, ')');
appendStringInfoChar(buf, '(');
get_query_def(subquery, buf, context->namespaces, resultDesc,
context->prettyFlags, context->wrapColumn,
- context->indentLevel);
+ context->indentLevel,
+ context->finalise_aggs,
+ context->sortgroup_colno);
if (need_paren)
appendStringInfoChar(buf, ')');
}
/* Add the SELECT */
get_query_def(select_rte->subquery, buf, NIL, NULL,
context->prettyFlags, context->wrapColumn,
- context->indentLevel);
+ context->indentLevel,
+ context->finalise_aggs,
+ context->sortgroup_colno);
}
else if (values_rte)
{
get_query_def(query, buf, context->namespaces, NULL,
context->prettyFlags, context->wrapColumn,
- context->indentLevel);
+ context->indentLevel,
+ context->finalise_aggs,
+ context->sortgroup_colno);
if (need_paren)
appendStringInfoString(buf, "))");
appendStringInfoChar(buf, '(');
get_query_def(rte->subquery, buf, context->namespaces, NULL,
context->prettyFlags, context->wrapColumn,
- context->indentLevel);
+ context->indentLevel,
+ context->finalise_aggs,
+ context->sortgroup_colno);
appendStringInfoChar(buf, ')');
break;
case RTE_FUNCTION:
plansource->parserSetupArg = parserSetupArg;
plansource->cursor_options = cursor_options;
plansource->fixed_result = fixed_result;
+#ifdef PGXC
+ //plansource->stmt_name = NULL;
+#endif
plansource->resultDesc = PlanCacheComputeResultDesc(querytree_list);
MemoryContextSwitchTo(oldcxt);
* If this plansource belongs to a named prepared statement, store the stmt
* name for the Datanode queries.
*/
- if (IS_PGXC_LOCAL_COORDINATOR && plansource->stmt_name)
+ if (IS_PGXC_LOCAL_COORDINATOR && plansource->stmt_name &&
+ plansource->stmt_name[0] != '\0')
{
ListCell *lc;
int n;
* code that is being actively worked on. FIXME someday.
*/
+#ifdef PGXC
+ char *relname;
+#endif
+
/*
* Fields valid for a plain relation RTE (else zero):
*/
Oid aggtype; /* type Oid of result of the aggregate */
Oid aggcollid; /* OID of collation of result */
Oid inputcollid; /* OID of collation that function should use */
+#ifdef PGXC
+ Oid aggtrantype; /* type Oid of transition results */
+ bool agghas_collectfn; /* is collection function available */
+#endif /* PGXC */
List *aggdirectargs; /* direct arguments, if an ordered-set agg */
List *args; /* aggregated arguments and sort expressions */
List *aggorder; /* ORDER BY (list of SortGroupClause) */
int num_batches; /* number of batches expected */
} HashPath;
+#ifdef PGXC
+/*
+ * A remotequery path represents the queries to be sent to the datanode/s
+ *
+ * When RemoteQuery plan is created from RemoteQueryPath, we build the query to
+ * be executed at the datanode. For building such a query, it's important to get
+ * the RHS relation and LHS relation of the JOIN clause. So, instead of storing
+ * the outer and inner paths, we find out the RHS and LHS paths and store those
+ * here.
+ */
+
+typedef struct RemoteQueryPath
+{
+ Path path;
+ ExecNodes *rqpath_en; /* List of datanodes to execute the query on */
+ /*
+ * If the path represents a JOIN rel, leftpath and rightpath represent the
+ * RemoteQuery paths for left (outer) and right (inner) side of the JOIN
+ * resp. jointype and join_restrictlist pertains to such JOINs.
+ */
+ struct RemoteQueryPath *leftpath;
+ struct RemoteQueryPath *rightpath;
+ JoinType jointype;
+ List *join_restrictlist; /* restrict list corresponding to JOINs,
+ * only considered if rest of
+ * the JOIN information is
+ * available
+ */
+ bool rqhas_unshippable_qual; /* TRUE if there is at least
+ * one qual which can not be
+ * shipped to the datanodes
+ */
+ bool rqhas_temp_rel; /* TRUE if one of the base relations
+ * involved in this path is a temporary
+ * table.
+ */
+ bool rqhas_unshippable_tlist;/* TRUE if there is at least one
+ * targetlist entry which is
+ * not completely shippable.
+ */
+} RemoteQueryPath;
+#endif /* PGXC */
+
/*
* Restriction clause info.
*
extern bool enable_mergejoin;
extern bool enable_hashjoin;
#ifdef PGXC
+extern bool enable_fast_query_shipping;
extern bool enable_remotejoin;
extern bool enable_remotegroup;
#endif
ListCell *roundRobinNode; /* index of the next one to use */
} RelationLocInfo;
+#define IsRelationReplicated(rel_loc) IsLocatorReplicated((rel_loc)->locatorType)
+#define IsRelationColumnDistributed(rel_loc) IsLocatorColumnDistributed((rel_loc)->locatorType)
+#define IsRelationDistributedByValue(rel_loc) IsLocatorDistributedByValue((rel_loc)->locatorType)
/*
* Nodes to execute on
* primarynodelist is for replicated table writes, where to execute first.
} ExecNodes;
+#define IsExecNodesReplicated(en) IsLocatorReplicated((en)->baselocatortype)
+#define IsExecNodesColumnDistributed(en) IsLocatorColumnDistributed((en)->baselocatortype)
+#define IsExecNodesDistributedByValue(en) IsLocatorDistributedByValue((en)->baselocatortype)
+
typedef enum
{
LOCATOR_LIST_NONE, /* locator returns integers in range 0..NodeCount-1,
extern bool IsHashColumn(RelationLocInfo *rel_loc_info, char *part_col_name);
extern bool IsHashColumnForRelId(Oid relid, char *part_col_name);
extern int GetRoundRobinNode(Oid relid);
+extern ExecNodes *GetRelationNodes(RelationLocInfo *rel_loc_info,
+ Datum valueForDistCol,
+ bool isValueNull,
+ RelationAccessType accessType);
+extern ExecNodes *GetRelationNodesByQuals(Oid reloid,
+ Index varno,
+ Node *quals,
+ RelationAccessType relaccess);
extern bool IsTypeHashDistributable(Oid col_type);
extern List *GetAllDataNodes(void);
extern char *GetRelationDistColumn(RelationLocInfo *rel_loc_info);
extern bool IsDistColumnForRelId(Oid relid, char *part_col_name);
extern void FreeExecNodes(ExecNodes **exec_nodes);
+extern List *GetPreferredReplicationNode(List *relNodes);
+extern char *GetRelationDistribColumn(RelationLocInfo *locInfo);
#endif /* LOCATOR_H */
*/
RemoteQueryExecType exec_type;
int reduce_level; /* in case of reduced JOIN, it's level */
- List *base_tlist; /* in case of isReduced, the base tlist */
char *outer_alias;
char *inner_alias;
int outer_reduce_level;
bool has_row_marks; /* Did SELECT had FOR UPDATE/SHARE? */
bool has_ins_child_sel_parent; /* This node is part of an INSERT SELECT that
* inserts into child by selecting from its parent */
+
+ bool rq_finalise_aggs; /* Aggregates should be finalised at
+ the
+ * Datanode */
+ bool rq_sortgroup_colno; /* Use resno for sort group references
+ * instead of expressions */
+ Query *remote_query; /* Query structure representing the query
+ to be
+ * sent to the datanodes */
+ List *base_tlist; /* the targetlist representing the result
+ of
+ * the query to be sent to the datanode */
+
+ /*
+ * Reference targetlist of Vars to match the Vars in the plan nodes on
+ * coordinator to the corresponding Vars in the remote_query. These
+ * targetlists are used to while replacing/adding targetlist and quals in
+ * the remote_query.
+ */
+ List *coord_var_tlist;
+ List *query_var_tlist;
+ bool is_temp;
+
} RemoteQuery;
/*
* by FQS, but such expressions might be
* supported by FQS in future
*/
- SS_HAS_AGG_EXPR /* it has aggregate expressions */
+ SS_HAS_AGG_EXPR, /* it has aggregate expressions */
+ SS_UPDATES_DISTRIBUTION_COLUMN /* query updates distribution column */
} ShippabilityStat;
/* forbid SQL if unsafe, useful to turn off for development */
ParamListInfo boundParams);
extern List *AddRemoteQueryNode(List *stmts, const char *queryString,
RemoteQueryExecType remoteExecType);
+extern PlannedStmt *pgxc_planner(Query *query, int cursorOptions,
+ ParamListInfo boundParams);
+extern ExecNodes *pgxc_is_query_shippable(Query *query, int query_level);
+
#endif /* PGXCPLANNER_H */
#ifdef PGXC
extern List *deparse_context_for_remotequery(Alias *aliasname, Oid relid);
extern void get_query_def_from_valuesList(Query *query, StringInfo buf);
-extern void deparse_query(Query *query, StringInfo buf, List *parentnamespace);
+extern void deparse_query(Query *query, StringInfo buf, List *parentnamespace,
+ bool finalise_aggs, bool sortgroup_colno);
#endif
#ifdef PGXC
extern List *deparse_context_for_plan(Node *plan, List *ancestors,