diff --git a/src/backend/distributed/commands/call.c b/src/backend/distributed/commands/call.c index dda4eb3de..af319f0ce 100644 --- a/src/backend/distributed/commands/call.c +++ b/src/backend/distributed/commands/call.c @@ -24,6 +24,7 @@ #include "distributed/commands/utility_hook.h" #include "distributed/connection_management.h" #include "distributed/deparse_shard_query.h" +#include "distributed/function_call_delegation.h" #include "distributed/metadata_utility.h" #include "distributed/metadata_cache.h" #include "distributed/multi_executor.h" @@ -46,9 +47,10 @@ #include "utils/lsyscache.h" #include "utils/syscache.h" -static bool CallFuncExprRemotely(CallStmt *callStmt, - DistObjectCacheEntry *procedure, - FuncExpr *funcExpr, DestReceiver *dest); + +/* global variable tracking whether we are in a delegated procedure call */ +bool InDelegatedProcedureCall = false; + /* * CallDistributedProcedureRemotely calls a stored procedure on the worker if possible. @@ -61,28 +63,21 @@ CallDistributedProcedureRemotely(CallStmt *callStmt, DestReceiver *dest) DistObjectCacheEntry *procedure = LookupDistObjectCacheEntry(ProcedureRelationId, functionId, 0); - - /* - * If procedure is not distributed or already delegated from another - * node, do not call the procedure remotely. - */ - if (procedure == NULL || !procedure->isDistributed || - IsCitusInitiatedRemoteBackend()) + if (procedure == NULL || !procedure->isDistributed) { return false; } - return CallFuncExprRemotely(callStmt, procedure, funcExpr, dest); -} + if (IsCitusInitiatedRemoteBackend()) + { + /* + * We are in a citus-initiated backend handling a CALL to a distributed + * procedure. That means that this is the delegated call. + */ + InDelegatedProcedureCall = true; + return false; + } - -/* - * CallFuncExprRemotely calls a procedure of function on the worker if possible. - */ -static bool -CallFuncExprRemotely(CallStmt *callStmt, DistObjectCacheEntry *procedure, - FuncExpr *funcExpr, DestReceiver *dest) -{ if (IsMultiStatementTransaction()) { ereport(DEBUG1, (errmsg("cannot push down CALL in multi-statement transaction"))); @@ -102,6 +97,7 @@ CallFuncExprRemotely(CallStmt *callStmt, DistObjectCacheEntry *procedure, "be constant expressions"))); return false; } + CitusTableCacheEntry *distTable = GetCitusTableCacheEntry(colocatedRelationId); Var *partitionColumn = distTable->partitionColumn; bool colocatedWithReferenceTable = false; diff --git a/src/backend/distributed/commands/function.c b/src/backend/distributed/commands/function.c index e3b93b809..26a56ca69 100644 --- a/src/backend/distributed/commands/function.c +++ b/src/backend/distributed/commands/function.c @@ -91,6 +91,7 @@ static void DistributeFunctionWithDistributionArgument(RegProcedure funcOid, char *distributionArgumentName, Oid distributionArgumentOid, char *colocateWithTableName, + bool *forcePushdownAddress, const ObjectAddress * functionAddress); static void DistributeFunctionColocatedWithDistributedTable(RegProcedure funcOid, @@ -124,6 +125,8 @@ create_distributed_function(PG_FUNCTION_ARGS) char *distributionArgumentName = NULL; char *colocateWithTableName = NULL; + bool *forcePushdownAddress = NULL; + bool forcePushdown = false; /* if called on NULL input, error out */ if (funcOid == InvalidOid) @@ -169,6 +172,17 @@ create_distributed_function(PG_FUNCTION_ARGS) } } + /* check if the force_pushdown flag is explicitly set (default is NULL) */ + if (PG_ARGISNULL(3)) + { + forcePushdownAddress = NULL; + } + else + { + forcePushdown = PG_GETARG_BOOL(3); + forcePushdownAddress = &forcePushdown; + } + EnsureCoordinator(); EnsureFunctionOwner(funcOid); @@ -204,6 +218,7 @@ create_distributed_function(PG_FUNCTION_ARGS) DistributeFunctionWithDistributionArgument(funcOid, distributionArgumentName, distributionArgumentOid, colocateWithTableName, + forcePushdownAddress, &functionAddress); } else if (!colocatedWithReferenceTable) @@ -265,6 +280,7 @@ DistributeFunctionWithDistributionArgument(RegProcedure funcOid, char *distributionArgumentName, Oid distributionArgumentOid, char *colocateWithTableName, + bool *forcePushdownAddress, const ObjectAddress *functionAddress) { /* get the argument index, or error out if we cannot find a valid index */ @@ -279,7 +295,8 @@ DistributeFunctionWithDistributionArgument(RegProcedure funcOid, /* record the distribution argument and colocationId */ UpdateFunctionDistributionInfo(functionAddress, &distributionArgumentIndex, - &colocationId); + &colocationId, + forcePushdownAddress); } @@ -310,7 +327,7 @@ DistributeFunctionColocatedWithDistributedTable(RegProcedure funcOid, } /* set distribution argument and colocationId to NULL */ - UpdateFunctionDistributionInfo(functionAddress, NULL, NULL); + UpdateFunctionDistributionInfo(functionAddress, NULL, NULL, NULL); } @@ -327,7 +344,8 @@ DistributeFunctionColocatedWithReferenceTable(const ObjectAddress *functionAddre /* set distribution argument to NULL and colocationId to the reference table colocation id */ int *distributionArgumentIndex = NULL; UpdateFunctionDistributionInfo(functionAddress, distributionArgumentIndex, - &colocationId); + &colocationId, + NULL); } @@ -596,7 +614,8 @@ EnsureFunctionCanBeColocatedWithTable(Oid functionOid, Oid distributionColumnTyp void UpdateFunctionDistributionInfo(const ObjectAddress *distAddress, int *distribution_argument_index, - int *colocationId) + int *colocationId, + bool *forcePushdown) { const bool indexOK = true; @@ -655,6 +674,18 @@ UpdateFunctionDistributionInfo(const ObjectAddress *distAddress, isnull[Anum_pg_dist_object_colocationid - 1] = true; } + replace[Anum_pg_dist_object_force_pushdown - 1] = true; + if (forcePushdown != NULL) + { + values[Anum_pg_dist_object_force_pushdown - 1] = BoolGetDatum( + *forcePushdown); + isnull[Anum_pg_dist_object_force_pushdown - 1] = false; + } + else + { + isnull[Anum_pg_dist_object_force_pushdown - 1] = true; + } + heapTuple = heap_modify_tuple(heapTuple, tupleDescriptor, values, isnull, replace); CatalogTupleUpdate(pgDistObjectRel, &heapTuple->t_self, heapTuple); @@ -672,6 +703,7 @@ UpdateFunctionDistributionInfo(const ObjectAddress *distAddress, List *objectAddressList = list_make1((ObjectAddress *) distAddress); List *distArgumentIndexList = NIL; List *colocationIdList = NIL; + List *forcePushdownList = NIL; if (distribution_argument_index == NULL) { @@ -691,10 +723,20 @@ UpdateFunctionDistributionInfo(const ObjectAddress *distAddress, colocationIdList = list_make1_int(*colocationId); } + if (forcePushdown == NULL) + { + forcePushdownList = list_make1_int(NO_FORCE_PUSHDOWN); + } + else + { + forcePushdownList = list_make1_int(*forcePushdown); + } + char *workerPgDistObjectUpdateCommand = MarkObjectsDistributedCreateCommand(objectAddressList, distArgumentIndexList, - colocationIdList); + colocationIdList, + forcePushdownList); SendCommandToWorkersWithMetadata(workerPgDistObjectUpdateCommand); } } diff --git a/src/backend/distributed/commands/multi_copy.c b/src/backend/distributed/commands/multi_copy.c index d683a2792..d2d7d9b23 100644 --- a/src/backend/distributed/commands/multi_copy.c +++ b/src/backend/distributed/commands/multi_copy.c @@ -3340,6 +3340,7 @@ InitializeCopyShardState(CopyShardState *shardState, { ListCell *placementCell = NULL; int failedPlacementCount = 0; + bool hasRemoteCopy = false; MemoryContext localContext = AllocSetContextCreateExtended(CurrentMemoryContext, @@ -3383,6 +3384,8 @@ InitializeCopyShardState(CopyShardState *shardState, continue; } + hasRemoteCopy = true; + MultiConnection *connection = CopyGetPlacementConnection(connectionStateHash, placement, colocatedIntermediateResult); @@ -3427,6 +3430,11 @@ InitializeCopyShardState(CopyShardState *shardState, ereport(ERROR, (errmsg("could not connect to any active placements"))); } + if (hasRemoteCopy) + { + EnsureRemoteTaskExecutionAllowed(); + } + /* * We just error out and code execution should never reach to this * point. This is the case for all tables. diff --git a/src/backend/distributed/commands/utility_hook.c b/src/backend/distributed/commands/utility_hook.c index 672a7ce81..3e2a2d24b 100644 --- a/src/backend/distributed/commands/utility_hook.c +++ b/src/backend/distributed/commands/utility_hook.c @@ -227,10 +227,20 @@ multi_ProcessUtility(PlannedStmt *pstmt, params, queryEnv, dest, completionTag); StoredProcedureLevel -= 1; + + if (InDelegatedProcedureCall && StoredProcedureLevel == 0) + { + InDelegatedProcedureCall = false; + } } PG_CATCH(); { StoredProcedureLevel -= 1; + + if (InDelegatedProcedureCall && StoredProcedureLevel == 0) + { + InDelegatedProcedureCall = false; + } PG_RE_THROW(); } PG_END_TRY(); diff --git a/src/backend/distributed/executor/adaptive_executor.c b/src/backend/distributed/executor/adaptive_executor.c index 0e9d96fd5..ae4db9b49 100644 --- a/src/backend/distributed/executor/adaptive_executor.c +++ b/src/backend/distributed/executor/adaptive_executor.c @@ -161,9 +161,11 @@ #include "distributed/shared_connection_stats.h" #include "distributed/subplan_execution.h" #include "distributed/transaction_management.h" +#include "distributed/transaction_identifier.h" #include "distributed/tuple_destination.h" #include "distributed/version_compat.h" #include "distributed/worker_protocol.h" +#include "distributed/backend_data.h" #include "lib/ilist.h" #include "portability/instr_time.h" #include "storage/fd.h" @@ -1290,6 +1292,12 @@ StartDistributedExecution(DistributedExecution *execution) */ RecordParallelRelationAccessForTaskList(execution->remoteAndLocalTaskList); } + + /* make sure we are not doing remote execution from within a task */ + if (execution->remoteTaskList != NIL) + { + EnsureRemoteTaskExecutionAllowed(); + } } diff --git a/src/backend/distributed/executor/citus_custom_scan.c b/src/backend/distributed/executor/citus_custom_scan.c index a06f060f3..607662599 100644 --- a/src/backend/distributed/executor/citus_custom_scan.c +++ b/src/backend/distributed/executor/citus_custom_scan.c @@ -34,12 +34,15 @@ #include "distributed/subplan_execution.h" #include "distributed/worker_log_messages.h" #include "distributed/worker_protocol.h" +#include "distributed/colocation_utils.h" +#include "distributed/function_call_delegation.h" #include "executor/executor.h" #include "nodes/makefuncs.h" #include "optimizer/optimizer.h" #include "optimizer/clauses.h" #include "utils/memutils.h" #include "utils/rel.h" +#include "utils/datum.h" /* functions for creating custom scan nodes */ @@ -59,6 +62,7 @@ static DistributedPlan * CopyDistributedPlanWithoutCache( DistributedPlan *originalDistributedPlan); static void CitusEndScan(CustomScanState *node); static void CitusReScan(CustomScanState *node); +static void SetJobColocationId(Job *job); /* create custom scan methods for all executors */ @@ -190,6 +194,28 @@ CitusBeginScan(CustomScanState *node, EState *estate, int eflags) CitusBeginModifyScan(node, estate, eflags); } + Job *workerJob = scanState->distributedPlan->workerJob; + + /* + * If the Job has the partition key, check for a match with the force_pushdown + * distribution argument. + */ + if (workerJob->partitionKeyValue) + { + SetJobColocationId(workerJob); + + if (!IsShardKeyValueAllowed(workerJob->partitionKeyValue, + workerJob->colocationId)) + { + ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED), + errmsg( + "queries must filter by the distribution argument in the same " + "colocation group when using the forced function pushdown"), + errhint( + "consider turning off the flag force_pushdown instead"))); + } + } + /* * In case of a prepared statement, we will see this distributed plan again * on the next execution with a higher usage counter. @@ -801,3 +827,51 @@ IsCitusCustomScan(Plan *plan) return true; } + + +/* + * In a Job, given a list of relations, if all them belong to the same + * colocation group, the Job's colocation ID is set to the group ID, else, + * it will be set to INVALID_COLOCATION_ID. + */ +static void +SetJobColocationId(Job *job) +{ + uint32 jobColocationId = INVALID_COLOCATION_ID; + + if (!job->partitionKeyValue) + { + /* if the Job has no shard key, nothing to do */ + return; + } + + List *rangeTableList = ExtractRangeTableEntryList(job->jobQuery); + ListCell *rangeTableCell = NULL; + foreach(rangeTableCell, rangeTableList) + { + RangeTblEntry *rangeTableEntry = (RangeTblEntry *) lfirst(rangeTableCell); + Oid relationId = rangeTableEntry->relid; + + if (!IsCitusTable(relationId)) + { + /* ignore the non distributed table */ + continue; + } + + uint32 colocationId = TableColocationId(relationId); + + if (jobColocationId == INVALID_COLOCATION_ID) + { + /* Initialize the ID */ + jobColocationId = colocationId; + } + else if (jobColocationId != colocationId) + { + /* Tables' colocationId is not the same */ + jobColocationId = INVALID_COLOCATION_ID; + break; + } + } + + job->colocationId = jobColocationId; +} diff --git a/src/backend/distributed/executor/multi_executor.c b/src/backend/distributed/executor/multi_executor.c index 5a535043d..d2f488b60 100644 --- a/src/backend/distributed/executor/multi_executor.c +++ b/src/backend/distributed/executor/multi_executor.c @@ -21,9 +21,11 @@ #include "distributed/citus_custom_scan.h" #include "distributed/commands/multi_copy.h" #include "distributed/commands/utility_hook.h" +#include "distributed/function_call_delegation.h" #include "distributed/insert_select_executor.h" #include "distributed/insert_select_planner.h" #include "distributed/listutils.h" +#include "distributed/local_executor.h" #include "distributed/coordinator_protocol.h" #include "distributed/multi_executor.h" #include "distributed/combine_query_planner.h" @@ -35,6 +37,7 @@ #include "distributed/version_compat.h" #include "distributed/worker_shard_visibility.h" #include "distributed/worker_protocol.h" +#include "distributed/function_call_delegation.h" #include "executor/execdebug.h" #include "commands/copy.h" #include "nodes/execnodes.h" @@ -232,6 +235,15 @@ CitusExecutorRun(QueryDesc *queryDesc, * transactions. */ CitusTableCacheFlushInvalidatedEntries(); + + /* + * Within a 2PC, when a function is delegated to a remote node, we pin + * the distribution argument as the shard key for all the SQL in the + * function's block. The restriction is imposed to not to access other + * nodes from the current node and violate the transactional integrity + * of the 2PC. Now that the query is ending, reset the shard key to NULL. + */ + ResetAllowedShardKeyValue(); } } PG_CATCH(); @@ -244,6 +256,15 @@ CitusExecutorRun(QueryDesc *queryDesc, executorBoundParams = savedBoundParams; ExecutorLevel--; + if (ExecutorLevel == 0) + { + /* + * In case of an exception, reset the pinned shard-key, for more + * details see the function header. + */ + ResetAllowedShardKeyValue(); + } + PG_RE_THROW(); } PG_END_TRY(); @@ -719,3 +740,46 @@ ExecutorBoundParams(void) Assert(ExecutorLevel > 0); return executorBoundParams; } + + +/* + * EnsureRemoteTaskExecutionAllowed ensures that we do not perform remote + * execution from within a task. That could happen when the user calls + * a function in a query that gets pushed down to the worker, and the + * function performs a query on a distributed table. + */ +void +EnsureRemoteTaskExecutionAllowed(void) +{ + if (!InTaskExecution()) + { + /* we are not within a task, distributed execution is allowed */ + return; + } + + ereport(ERROR, (errmsg("cannot execute a distributed query from a query on a " + "shard"))); +} + + +/* + * InTaskExecution determines whether we are currently in a task execution. + */ +bool +InTaskExecution(void) +{ + if (LocalExecutorLevel > 0) + { + /* in a local task */ + return true; + } + + /* + * Normally, any query execution within a citus-initiated backend + * is considered a task execution, but an exception is when we + * are in a delegated function/procedure call. + */ + return IsCitusInitiatedRemoteBackend() && + !InDelegatedFunctionCall && + !InDelegatedProcedureCall; +} diff --git a/src/backend/distributed/metadata/distobject.c b/src/backend/distributed/metadata/distobject.c index 92b72d64d..3dcde64b6 100644 --- a/src/backend/distributed/metadata/distobject.c +++ b/src/backend/distributed/metadata/distobject.c @@ -178,11 +178,13 @@ MarkObjectDistributed(const ObjectAddress *distAddress) List *objectAddressList = list_make1((ObjectAddress *) distAddress); List *distArgumetIndexList = list_make1_int(INVALID_DISTRIBUTION_ARGUMENT_INDEX); List *colocationIdList = list_make1_int(INVALID_COLOCATION_ID); + List *forcePushdownList = list_make1_int(NO_FORCE_PUSHDOWN); char *workerPgDistObjectUpdateCommand = MarkObjectsDistributedCreateCommand(objectAddressList, distArgumetIndexList, - colocationIdList); + colocationIdList, + forcePushdownList); SendCommandToWorkersWithMetadata(workerPgDistObjectUpdateCommand); } } diff --git a/src/backend/distributed/metadata/metadata_cache.c b/src/backend/distributed/metadata/metadata_cache.c index ce05f7c28..f46e2c4a3 100644 --- a/src/backend/distributed/metadata/metadata_cache.c +++ b/src/backend/distributed/metadata/metadata_cache.c @@ -1362,6 +1362,9 @@ LookupDistObjectCacheEntry(Oid classid, Oid objid, int32 objsubid) 1]); cacheEntry->colocationId = DatumGetInt32(datumArray[Anum_pg_dist_object_colocationid - 1]); + + cacheEntry->forcePushdown = + DatumGetBool(datumArray[Anum_pg_dist_object_force_pushdown - 1]); } else { diff --git a/src/backend/distributed/metadata/metadata_sync.c b/src/backend/distributed/metadata/metadata_sync.c index eb3025679..ba8871ebd 100644 --- a/src/backend/distributed/metadata/metadata_sync.c +++ b/src/backend/distributed/metadata/metadata_sync.c @@ -732,6 +732,7 @@ DistributedObjectMetadataSyncCommandList(void) List *objectAddressList = NIL; List *distArgumentIndexList = NIL; List *colocationIdList = NIL; + List *forcePushdownList = NIL; /* It is not strictly necessary to read the tuples in order. * However, it is useful to get consistent behavior, both for regression @@ -767,6 +768,14 @@ DistributedObjectMetadataSyncCommandList(void) &colocationIdIsNull); int32 colocationId = DatumGetInt32(colocationIdDatum); + bool forcePushdownIsNull = false; + Datum forcePushdownDatum = + heap_getattr(pgDistObjectTup, + Anum_pg_dist_object_force_pushdown, + pgDistObjectDesc, + &forcePushdownIsNull); + bool forcePushdown = DatumGetBool(forcePushdownDatum); + objectAddressList = lappend(objectAddressList, address); if (distributionArgumentIndexIsNull) @@ -789,6 +798,15 @@ DistributedObjectMetadataSyncCommandList(void) { colocationIdList = lappend_int(colocationIdList, colocationId); } + + if (forcePushdownIsNull) + { + forcePushdownList = lappend_int(forcePushdownList, NO_FORCE_PUSHDOWN); + } + else + { + forcePushdownList = lappend_int(forcePushdownList, forcePushdown); + } } systable_endscan_ordered(pgDistObjectScan); @@ -798,7 +816,8 @@ DistributedObjectMetadataSyncCommandList(void) char *workerMetadataUpdateCommand = MarkObjectsDistributedCreateCommand(objectAddressList, distArgumentIndexList, - colocationIdList); + colocationIdList, + forcePushdownList); List *commandList = list_make1(workerMetadataUpdateCommand); return commandList; @@ -1000,7 +1019,8 @@ NodeListInsertCommand(List *workerNodeList) char * MarkObjectsDistributedCreateCommand(List *addresses, List *distributionArgumentIndexes, - List *colocationIds) + List *colocationIds, + List *forcePushdowns) { StringInfo insertDistributedObjectsCommand = makeStringInfo(); @@ -1009,7 +1029,7 @@ MarkObjectsDistributedCreateCommand(List *addresses, appendStringInfo(insertDistributedObjectsCommand, "WITH distributed_object_data(typetext, objnames, " - "objargs, distargumentindex, colocationid) AS (VALUES "); + "objargs, distargumentindex, colocationid, force_pushdown) AS (VALUES "); bool isFirstObject = true; for (int currentObjectCounter = 0; currentObjectCounter < list_length(addresses); @@ -1019,6 +1039,7 @@ MarkObjectsDistributedCreateCommand(List *addresses, int distributionArgumentIndex = list_nth_int(distributionArgumentIndexes, currentObjectCounter); int colocationId = list_nth_int(colocationIds, currentObjectCounter); + int forcePushdown = list_nth_int(forcePushdowns, currentObjectCounter); List *names = NIL; List *args = NIL; char *objectType = NULL; @@ -1074,15 +1095,18 @@ MarkObjectsDistributedCreateCommand(List *addresses, appendStringInfo(insertDistributedObjectsCommand, "%d, ", distributionArgumentIndex); - appendStringInfo(insertDistributedObjectsCommand, "%d)", + appendStringInfo(insertDistributedObjectsCommand, "%d, ", colocationId); + + appendStringInfo(insertDistributedObjectsCommand, "%s)", + forcePushdown ? "true" : "false"); } appendStringInfo(insertDistributedObjectsCommand, ") "); appendStringInfo(insertDistributedObjectsCommand, "SELECT citus_internal_add_object_metadata(" - "typetext, objnames, objargs, distargumentindex::int, colocationid::int) " + "typetext, objnames, objargs, distargumentindex::int, colocationid::int, force_pushdown::bool) " "FROM distributed_object_data;"); return insertDistributedObjectsCommand->data; @@ -1101,6 +1125,7 @@ citus_internal_add_object_metadata(PG_FUNCTION_ARGS) ArrayType *argsArray = PG_GETARG_ARRAYTYPE_P(2); int distributionArgumentIndex = PG_GETARG_INT32(3); int colocationId = PG_GETARG_INT32(4); + bool forcePushdown = PG_GETARG_INT32(5); if (!ShouldSkipMetadataChecks()) { @@ -1142,9 +1167,14 @@ citus_internal_add_object_metadata(PG_FUNCTION_ARGS) NULL : &colocationId; + bool *forcePushdownAddress = + forcePushdown == false ? + NULL : + &forcePushdown; UpdateFunctionDistributionInfo(&objectAddress, distributionArgumentIndexAddress, - colocationIdAddress); + colocationIdAddress, + forcePushdownAddress); } SetLocalEnableDependencyCreation(prevDependencyCreationValue); diff --git a/src/backend/distributed/planner/function_call_delegation.c b/src/backend/distributed/planner/function_call_delegation.c index 6618e49e1..c39ffb058 100644 --- a/src/backend/distributed/planner/function_call_delegation.c +++ b/src/backend/distributed/planner/function_call_delegation.c @@ -17,7 +17,7 @@ #include "catalog/pg_proc.h" #include "catalog/pg_type.h" #include "commands/defrem.h" -#include "distributed/citus_custom_scan.h" +#include "distributed/metadata_utility.h" #include "distributed/citus_ruleutils.h" #include "distributed/colocation_utils.h" #include "distributed/commands.h" @@ -26,7 +26,7 @@ #include "distributed/deparse_shard_query.h" #include "distributed/function_call_delegation.h" #include "distributed/insert_select_planner.h" -#include "distributed/metadata_utility.h" +#include "distributed/citus_custom_scan.h" #include "distributed/coordinator_protocol.h" #include "distributed/listutils.h" #include "distributed/metadata_cache.h" @@ -41,6 +41,7 @@ #include "nodes/nodeFuncs.h" #include "nodes/parsenodes.h" #include "nodes/primnodes.h" +#include "nodes/print.h" #include "optimizer/clauses.h" #include "parser/parse_coerce.h" #include "parser/parsetree.h" @@ -55,7 +56,19 @@ struct ParamWalkerContext ParamKind paramKind; }; +extern bool ShouldAllowRestricted2PC; +extern AllowedDistributionColumn *AllowedDistributionColumnValue; + static bool contain_param_walker(Node *node, void *context); +static void CheckDelegatedFunctionExecution(DistObjectCacheEntry *procedure, + FuncExpr *funcExpr); +static bool IsQuerySimple(Query *query); +static FuncExpr * FunctionInFromClause(List *fromlist, Query *query); +static void EnableInForceDelegatedFuncExecution(Const *distArgument, uint32 colocationId); + + +/* global variable keeping track of whether we are in a delegated function call */ +bool InDelegatedFunctionCall = false; /* @@ -102,8 +115,8 @@ TryToDelegateFunctionCall(DistributedPlanningContext *planContext) { bool colocatedWithReferenceTable = false; ShardPlacement *placement = NULL; - DistributedPlan *distributedPlan = CitusMakeNode(DistributedPlan); struct ParamWalkerContext walkerParamContext = { 0 }; + bool inTransactionBlock = false; if (!CitusHasBeenLoaded() || !CheckCitusVersion(DEBUG4)) { @@ -112,15 +125,6 @@ TryToDelegateFunctionCall(DistributedPlanningContext *planContext) } int32 localGroupId = GetLocalGroupId(); - if (localGroupId != COORDINATOR_GROUP_ID && IsCitusInitiatedRemoteBackend()) - { - /* - * Do not delegate from workers if it is initiated by Citus already. - * It means that this function has already been delegated to this node. - */ - return NULL; - } - if (localGroupId == GROUP_ID_UPGRADING) { /* do not delegate while upgrading */ @@ -152,41 +156,50 @@ TryToDelegateFunctionCall(DistributedPlanningContext *planContext) return NULL; } + FuncExpr *fromFuncExpr = NULL; if (joinTree->fromlist != NIL) { + if (list_length(joinTree->fromlist) != 1) + { + /* e.g. SELECT ... FROM rel1, rel2. */ + Assert(list_length(joinTree->fromlist) > 1); + return NULL; + } + /* * In pg12's planning phase empty FROMs are represented with an RTE_RESULT. * When we arrive here, standard_planner has already been called which calls * replace_empty_jointree() which replaces empty fromlist with a list of * single RTE_RESULT RangleTableRef node. */ - if (list_length(joinTree->fromlist) == 1) - { - RangeTblRef *reference = linitial(joinTree->fromlist); + RangeTblRef *reference = linitial(joinTree->fromlist); - if (IsA(reference, RangeTblRef)) - { - RangeTblEntry *rtentry = rt_fetch(reference->rtindex, - planContext->query->rtable); - if (rtentry->rtekind != RTE_RESULT) - { - /* e.g. SELECT f() FROM rel */ - return NULL; - } - } - else + if (IsA(reference, RangeTblRef)) + { + RangeTblEntry *rtentry = rt_fetch(reference->rtindex, + planContext->query->rtable); + + if (rtentry->rtekind == RTE_FUNCTION) { /* - * e.g. IsA(reference, JoinExpr). This is explicit join expressions - * like INNER JOIN, NATURAL JOIN, ... + * Look for a function in the FROM clause. */ + fromFuncExpr = FunctionInFromClause(joinTree->fromlist, + planContext->query); + } + else if (rtentry->rtekind != RTE_RESULT) + { + /* e.g. SELECT f() FROM rel */ + ereport(DEBUG4, (errmsg("FromList item is not empty"))); return NULL; } } else { - /* e.g. SELECT ... FROM rel1, rel2. */ - Assert(list_length(joinTree->fromlist) > 1); + /* + * e.g. IsA(reference, JoinExpr). This is explicit join expressions + * like INNER JOIN, NATURAL JOIN, ... + */ return NULL; } } @@ -198,14 +211,44 @@ TryToDelegateFunctionCall(DistributedPlanningContext *planContext) return NULL; } + FuncExpr *targetFuncExpr = NULL; TargetEntry *targetEntry = (TargetEntry *) linitial(targetList); - if (!IsA(targetEntry->expr, FuncExpr)) + if (IsA(targetEntry->expr, FuncExpr)) { - /* target list item is not a function call */ + /* function from the SELECT clause e.g. SELECT fn() FROM */ + targetFuncExpr = (FuncExpr *) targetEntry->expr; + } + + /* + * Look for one of: + * SELECT fn(...); + * SELECT ... FROM fn(...); + */ + FuncExpr *funcExpr = NULL; + if (targetFuncExpr != NULL) + { + if (fromFuncExpr != NULL) + { + /* query is of the form: SELECT fn() FROM fn() */ + return NULL; + } + + /* query is of the form: SELECT fn(); */ + funcExpr = targetFuncExpr; + } + else if (fromFuncExpr != NULL) + { + /* query is of the form: SELECT ... FROM fn(); */ + funcExpr = fromFuncExpr; + } + else + { + /* query does not have a function call in SELECT or FROM */ + ereport(DEBUG4, (errmsg("Either no function found or present" + " in both the SELECT and FROM list."))); return NULL; } - FuncExpr *funcExpr = (FuncExpr *) targetEntry->expr; DistObjectCacheEntry *procedure = LookupDistObjectCacheEntry(ProcedureRelationId, funcExpr->funcid, 0); if (procedure == NULL || !procedure->isDistributed) @@ -218,6 +261,39 @@ TryToDelegateFunctionCall(DistributedPlanningContext *planContext) ereport(DEBUG4, (errmsg("function is distributed"))); } + if (IsCitusInitiatedRemoteBackend()) + { + /* + * We are planning a call to a distributed function within a Citus backend, + * that means that this is the delegated call. If the function is forcefully + * delegated, capture the distribution argument. + */ + CheckDelegatedFunctionExecution(procedure, funcExpr); + + /* + * InDelegatedFunctionCall flag grants the levy to do remote tasks from a + * delegated function, we can do that only if the function is delegated + * without an explicit transaction on the client session. + */ + if (!IsMultiStatementTransaction()) + { + InDelegatedFunctionCall = true; + } + + return NULL; + } + + if (localGroupId != COORDINATOR_GROUP_ID) + { + /* + * We are calling a distributed function on a worker node. We currently + * only delegate from the coordinator. + * + * TODO: remove this restriction. + */ + return NULL; + } + /* * Cannot delegate functions for INSERT ... SELECT func(), since they require * coordinated transactions. @@ -228,12 +304,58 @@ TryToDelegateFunctionCall(DistributedPlanningContext *planContext) return NULL; } + if (fromFuncExpr && !IsMultiStatementTransaction()) + { + /* + * For now, let's not push the function from the FROM clause unless it's in a + * multistatement transaction with the forcePushdown flag ON. + */ + ereport(DEBUG4, (errmsg("function from the FROM clause is not pushed"))); + return NULL; + } + if (IsMultiStatementTransaction()) { - /* cannot delegate function calls in a multi-statement transaction */ - ereport(DEBUG1, (errmsg("not pushing down function calls in " - "a multi-statement transaction"))); - return NULL; + if (!procedure->forcePushdown) + { + /* cannot delegate function calls in a multi-statement transaction */ + ereport(DEBUG1, (errmsg("not pushing down function calls in " + "a multi-statement transaction"))); + return NULL; + } + else + { + Node *partitionValueNode = (Node *) list_nth(funcExpr->args, + procedure->distributionArgIndex); + + if (!IsA(partitionValueNode, Const)) + { + ereport(DEBUG1, (errmsg("distribution argument value must be a " + "constant when using force_pushdown flag"))); + return NULL; + } + + /* + * If the expression is simple, such as, SELECT fn() in + * PL/PgSQL code, PL engine is doing simple expression + * evaluation, which can't interpret the CustomScan Node. + * Function from FROM clause is not simple, so it's ok. + */ + if (MaybeExecutingUDF() && IsQuerySimple(planContext->query) && !fromFuncExpr) + { + ereport(DEBUG1, (errmsg("Skipping delegation of function " + "from a PL/PgSQL simple expression"))); + return NULL; + } + + /* + * When is flag is on, delegate the function call in a multi-statement + * transaction but with restrictions. + */ + ereport(DEBUG1, (errmsg("pushing down function call in " + "a multi-statement transaction"))); + inTransactionBlock = true; + } } if (contain_volatile_functions((Node *) funcExpr->args)) @@ -284,7 +406,7 @@ TryToDelegateFunctionCall(DistributedPlanningContext *planContext) /* return if we could not find a placement */ if (placement == NULL) { - return false; + return NULL; } WorkerNode *workerNode = FindWorkerNode(placement->nodeName, placement->nodePort); @@ -327,7 +449,25 @@ TryToDelegateFunctionCall(DistributedPlanningContext *planContext) ereport(DEBUG1, (errmsg("pushing down the function call"))); Task *task = CitusMakeNode(Task); - task->taskType = READ_TASK; + + /* + * In a multi-statement block the function should be part of the sorrounding + * transaction, at this time, not knowing the operations in the function, it + * is safe to assume that it's a write task. + * + * TODO: We should compile the function to see the internals of the function + * and find if this has read-only tasks, does it involve doing a remote task + * or queries involving non-distribution column, etc. + */ + if (inTransactionBlock) + { + task->taskType = MODIFY_TASK; + } + else + { + task->taskType = READ_TASK; + } + task->taskPlacementList = list_make1(placement); SetTaskQueryIfShouldLazyDeparse(task, planContext->query); task->anchorShardId = placement->shardId; @@ -338,7 +478,7 @@ TryToDelegateFunctionCall(DistributedPlanningContext *planContext) job->jobQuery = planContext->query; job->taskList = list_make1(task); - distributedPlan = CitusMakeNode(DistributedPlan); + DistributedPlan *distributedPlan = CitusMakeNode(DistributedPlan); distributedPlan->workerJob = job; distributedPlan->combineQuery = NULL; distributedPlan->expectResults = true; @@ -449,3 +589,204 @@ ShardPlacementForFunctionColocatedWithReferenceTable(CitusTableCacheEntry *cache return (ShardPlacement *) linitial(placementList); } + + +/* + * Checks to see if the procedure is being executed on a worker after delegated + * by the coordinator. If the flag forcePushdown is set, capture the distribution + * argument value, to be used by the planner to make sure that function uses only + * the colocated shards of the distribution argument. + */ +void +CheckDelegatedFunctionExecution(DistObjectCacheEntry *procedure, FuncExpr *funcExpr) +{ + /* if the flag is not set, nothing to enforce */ + if (!procedure->forcePushdown) + { + return; + } + + /* + * On the coordinator PartiallyEvaluateExpression() descends into an + * expression tree to evaluate expressions that can be resolved to a + * constant. Expressions containing a Var are skipped, since the value + * of the Var is not known on the coordinator. + */ + Node *partitionValueNode = (Node *) list_nth(funcExpr->args, + procedure->distributionArgIndex); + Assert(partitionValueNode); + partitionValueNode = strip_implicit_coercions(partitionValueNode); + + if (IsA(partitionValueNode, Param)) + { + Param *partitionParam = (Param *) partitionValueNode; + + if (partitionParam->paramkind == PARAM_EXTERN) + { + /* we should end up here again without a parameter */ + return; + } + } + + if (!IsA(partitionValueNode, Const)) + { + /* + * Var node e.g. select fn(col) from table where col=150; + * Param(PARAM_EXEC) node e.g. SELECT fn((SELECT col from test_nested where col=val)) + */ + ereport(ERROR, (errmsg("Non-constant arguments for forcePushdown " + "functions not supported"))); + } + else + { + Const *partitionValueConst = (Const *) partitionValueNode; + + ereport(DEBUG1, (errmsg("Pushdown argument: %s", + pretty_format_node_dump(nodeToString( + partitionValueNode))))); + + EnableInForceDelegatedFuncExecution(partitionValueConst, procedure->colocationId); + } +} + + +/* + * Function returns true if the query is simple enough to skip the full executor + * It checks only for expressions in the query clauses, and not WHERE and FROM + * lists. + */ +static bool +IsQuerySimple(Query *query) +{ + if (query->hasAggs || + query->hasWindowFuncs || + query->hasTargetSRFs || + query->hasSubLinks || + query->cteList || + query->groupClause || + query->groupingSets || + query->havingQual || + query->windowClause || + query->distinctClause || + query->sortClause || + query->limitOffset || + query->limitCount || + query->setOperations) + { + return false; + } + + return true; +} + + +/* + * Look for a function in the FROM clause. + */ +static FuncExpr * +FunctionInFromClause(List *fromlist, Query *query) +{ + if (list_length(fromlist) != 1) + { + /* We are looking for a single function */ + return NULL; + } + + RangeTblRef *reference = linitial(fromlist); + if (!IsA(reference, RangeTblRef)) + { + /* Skip if there is no RTE */ + return NULL; + } + + RangeTblEntry *rtentry = rt_fetch(reference->rtindex, query->rtable); + if (rtentry->rtekind != RTE_FUNCTION) + { + return NULL; + } + + if (list_length(rtentry->functions) != 1) + { + /* Skip if RTE isn't a single FuncExpr */ + return NULL; + } + + RangeTblFunction *rtfunc = (RangeTblFunction *) linitial(rtentry->functions); + if (!IsA(rtfunc->funcexpr, FuncExpr)) + { + /* Skip if RTE isn't a simple FuncExpr */ + return NULL; + } + + return (FuncExpr *) rtfunc->funcexpr; +} + + +/* + * Sets a flag to true indicating that the current node is executing a delegated + * function call, using forcePushdown, within a distributed transaction issued + * by the coordinator. Also, saves the distribution argument. + */ +static void +EnableInForceDelegatedFuncExecution(Const *distArgument, uint32 colocationId) +{ + /* + * The saved distribution argument need to persist through the life + * of the query, both during the planning (where we save) and execution + * (where we compare) + */ + MemoryContext oldcontext = MemoryContextSwitchTo(TopTransactionContext); + ereport(DEBUG1, errmsg("Saving Distribution Argument: %s:%d", + pretty_format_node_dump(nodeToString(distArgument)), + colocationId)); + AllowedDistributionColumnValue = (AllowedDistributionColumn *) palloc( + sizeof(AllowedDistributionColumn)); + AllowedDistributionColumnValue->distributionColumnValue = copyObject(distArgument); + AllowedDistributionColumnValue->colocationId = colocationId; + MemoryContextSwitchTo(oldcontext); + + ShouldAllowRestricted2PC = true; +} + + +/* + * Within a 2PC, when a function is delegated to a remote node, we pin + * the distribution argument as the shard key for all the SQL in the + * function's block. The restriction is imposed to not to access other + * nodes from the current node and violate the transactional integrity of + * the 2PC. Reset the distribution argument value once the function ends. + */ +void +ResetAllowedShardKeyValue(void) +{ + if (AllowedDistributionColumnValue) + { + pfree(AllowedDistributionColumnValue->distributionColumnValue); + pfree(AllowedDistributionColumnValue); + } + AllowedDistributionColumnValue = NULL; +} + + +/* + * Function returns true if the current shard key in the adaptive executor + * matches the saved distribution argument of a force_pushdown function. + */ +bool +IsShardKeyValueAllowed(Const *shardKey, uint32 colocationId) +{ + if (!AllowedDistributionColumnValue) + { + return true; + } + + ereport(LOG, errmsg("Comparing saved:%s with Shard key: %s colocationid:%d:%d", + pretty_format_node_dump( + nodeToString( + AllowedDistributionColumnValue->distributionColumnValue)), + pretty_format_node_dump(nodeToString(shardKey)), + AllowedDistributionColumnValue->colocationId, colocationId)); + + return (equal(AllowedDistributionColumnValue->distributionColumnValue, shardKey) && + (AllowedDistributionColumnValue->colocationId == colocationId)); +} diff --git a/src/backend/distributed/sql/citus--10.2-4--11.0-1.sql b/src/backend/distributed/sql/citus--10.2-4--11.0-1.sql index b4b6cc532..80dd339a1 100644 --- a/src/backend/distributed/sql/citus--10.2-4--11.0-1.sql +++ b/src/backend/distributed/sql/citus--10.2-4--11.0-1.sql @@ -2,6 +2,7 @@ -- bump version to 11.0-1 #include "udfs/citus_disable_node/11.0-1.sql" +#include "udfs/create_distributed_function/11.0-1.sql" #include "udfs/citus_check_connection_to_node/11.0-1.sql" #include "udfs/citus_check_cluster_node_health/11.0-1.sql" @@ -18,6 +19,7 @@ DROP FUNCTION pg_catalog.master_append_table_to_shard(bigint, text, text, intege -- all existing citus local tables are auto converted -- none of the other tables can have auto-converted as true ALTER TABLE pg_catalog.pg_dist_partition ADD COLUMN autoconverted boolean DEFAULT false; +ALTER TABLE citus.pg_dist_object ADD COLUMN force_pushdown bool DEFAULT NULL; UPDATE pg_catalog.pg_dist_partition SET autoconverted = TRUE WHERE partmethod = 'n' AND repmodel = 's'; REVOKE ALL ON FUNCTION start_metadata_sync_to_node(text, integer) FROM PUBLIC; diff --git a/src/backend/distributed/sql/downgrades/citus--11.0-1--10.2-4.sql b/src/backend/distributed/sql/downgrades/citus--11.0-1--10.2-4.sql index df823be93..5f2df035d 100644 --- a/src/backend/distributed/sql/downgrades/citus--11.0-1--10.2-4.sql +++ b/src/backend/distributed/sql/downgrades/citus--11.0-1--10.2-4.sql @@ -1,5 +1,6 @@ -- citus--11.0-1--10.2-4 +DROP FUNCTION pg_catalog.create_distributed_function(regprocedure, text, text, bool); CREATE FUNCTION pg_catalog.master_apply_delete_command(text) RETURNS integer LANGUAGE C STRICT @@ -43,6 +44,8 @@ COMMENT ON FUNCTION pg_catalog.citus_disable_node(nodename text, nodeport intege DROP FUNCTION pg_catalog.citus_check_connection_to_node (text, integer); DROP FUNCTION pg_catalog.citus_check_cluster_node_health (); -DROP FUNCTION pg_catalog.citus_internal_add_object_metadata(text, text[], text[], integer, integer); +DROP FUNCTION pg_catalog.citus_internal_add_object_metadata(text, text[], text[], integer, integer, boolean); DROP FUNCTION pg_catalog.citus_run_local_command(text); DROP FUNCTION pg_catalog.worker_drop_sequence_dependency(text); +#include "../udfs/create_distributed_function/9.0-1.sql" +ALTER TABLE citus.pg_dist_object DROP COLUMN force_pushdown; diff --git a/src/backend/distributed/sql/udfs/citus_internal_add_object_metadata/11.0-1.sql b/src/backend/distributed/sql/udfs/citus_internal_add_object_metadata/11.0-1.sql index 65d63d8b0..022b5677d 100644 --- a/src/backend/distributed/sql/udfs/citus_internal_add_object_metadata/11.0-1.sql +++ b/src/backend/distributed/sql/udfs/citus_internal_add_object_metadata/11.0-1.sql @@ -3,11 +3,12 @@ CREATE OR REPLACE FUNCTION pg_catalog.citus_internal_add_object_metadata( objNames text[], objArgs text[], distribution_argument_index int, - colocationid int) + colocationid int, + force_pushdown bool) RETURNS void LANGUAGE C STRICT AS 'MODULE_PATHNAME'; -COMMENT ON FUNCTION pg_catalog.citus_internal_add_object_metadata(text,text[],text[],int,int) IS +COMMENT ON FUNCTION pg_catalog.citus_internal_add_object_metadata(text,text[],text[],int,int,bool) IS 'Inserts distributed object into pg_dist_object'; diff --git a/src/backend/distributed/sql/udfs/citus_internal_add_object_metadata/latest.sql b/src/backend/distributed/sql/udfs/citus_internal_add_object_metadata/latest.sql index 65d63d8b0..022b5677d 100644 --- a/src/backend/distributed/sql/udfs/citus_internal_add_object_metadata/latest.sql +++ b/src/backend/distributed/sql/udfs/citus_internal_add_object_metadata/latest.sql @@ -3,11 +3,12 @@ CREATE OR REPLACE FUNCTION pg_catalog.citus_internal_add_object_metadata( objNames text[], objArgs text[], distribution_argument_index int, - colocationid int) + colocationid int, + force_pushdown bool) RETURNS void LANGUAGE C STRICT AS 'MODULE_PATHNAME'; -COMMENT ON FUNCTION pg_catalog.citus_internal_add_object_metadata(text,text[],text[],int,int) IS +COMMENT ON FUNCTION pg_catalog.citus_internal_add_object_metadata(text,text[],text[],int,int,bool) IS 'Inserts distributed object into pg_dist_object'; diff --git a/src/backend/distributed/sql/udfs/create_distributed_function/11.0-1.sql b/src/backend/distributed/sql/udfs/create_distributed_function/11.0-1.sql new file mode 100644 index 000000000..ac1c7d997 --- /dev/null +++ b/src/backend/distributed/sql/udfs/create_distributed_function/11.0-1.sql @@ -0,0 +1,15 @@ +DROP FUNCTION create_distributed_function(regprocedure, text, text); + +CREATE OR REPLACE FUNCTION create_distributed_function(function_name regprocedure, + distribution_arg_name text DEFAULT NULL, + colocate_with text DEFAULT 'default', + force_pushdown bool DEFAULT NULL) + RETURNS void + LANGUAGE C CALLED ON NULL INPUT + AS 'MODULE_PATHNAME', $$create_distributed_function$$; + +COMMENT ON FUNCTION create_distributed_function(function_name regprocedure, + distribution_arg_name text, + colocate_with text, + force_pushdown bool) + IS 'creates a distributed function'; diff --git a/src/backend/distributed/sql/udfs/create_distributed_function/latest.sql b/src/backend/distributed/sql/udfs/create_distributed_function/latest.sql index aa43ad287..ac1c7d997 100644 --- a/src/backend/distributed/sql/udfs/create_distributed_function/latest.sql +++ b/src/backend/distributed/sql/udfs/create_distributed_function/latest.sql @@ -1,11 +1,15 @@ +DROP FUNCTION create_distributed_function(regprocedure, text, text); + CREATE OR REPLACE FUNCTION create_distributed_function(function_name regprocedure, distribution_arg_name text DEFAULT NULL, - colocate_with text DEFAULT 'default') + colocate_with text DEFAULT 'default', + force_pushdown bool DEFAULT NULL) RETURNS void LANGUAGE C CALLED ON NULL INPUT AS 'MODULE_PATHNAME', $$create_distributed_function$$; COMMENT ON FUNCTION create_distributed_function(function_name regprocedure, distribution_arg_name text, - colocate_with text) + colocate_with text, + force_pushdown bool) IS 'creates a distributed function'; diff --git a/src/backend/distributed/test/run_from_same_connection.c b/src/backend/distributed/test/run_from_same_connection.c index e0b7d806c..3b5f804b4 100644 --- a/src/backend/distributed/test/run_from_same_connection.c +++ b/src/backend/distributed/test/run_from_same_connection.c @@ -113,6 +113,12 @@ start_session_level_connection_to_node(PG_FUNCTION_ARGS) elog(ERROR, "failed to connect to %s:%d", nodeNameString, (int) nodePort); } + /* pretend we are a regular client to avoid citus-initiated backend checks */ + const char *setAppName = + "SET application_name TO run_commands_on_session_level_connection_to_node"; + + ExecuteCriticalRemoteCommand(singleConnection, setAppName); + PG_RETURN_VOID(); } diff --git a/src/backend/distributed/transaction/transaction_management.c b/src/backend/distributed/transaction/transaction_management.c index ee9912fe9..57caaf852 100644 --- a/src/backend/distributed/transaction/transaction_management.c +++ b/src/backend/distributed/transaction/transaction_management.c @@ -23,6 +23,7 @@ #include "distributed/citus_safe_lib.h" #include "distributed/connection_management.h" #include "distributed/distributed_planner.h" +#include "distributed/function_call_delegation.h" #include "distributed/hash_helpers.h" #include "distributed/intermediate_results.h" #include "distributed/listutils.h" @@ -38,10 +39,13 @@ #include "distributed/subplan_execution.h" #include "distributed/version_compat.h" #include "distributed/worker_log_messages.h" +#include "distributed/commands.h" #include "utils/hsearch.h" #include "utils/guc.h" #include "utils/memutils.h" +#include "utils/datum.h" #include "storage/fd.h" +#include "nodes/print.h" CoordinatedTransactionState CurrentCoordinatedTransactionState = COORD_TRANS_NONE; @@ -102,6 +106,18 @@ MemoryContext CommitContext = NULL; */ bool ShouldCoordinatedTransactionUse2PC = false; +/* + * Distribution function argument (along with colocationId) when delegated + * using forcePushdown flag. + */ +AllowedDistributionColumn *AllowedDistributionColumnValue = NULL; + +/* + * Flag indicating if a distributed forcePushdown function is executed in + * the current transaction. + */ +bool ShouldAllowRestricted2PC = false; + /* if disabled, distributed statements in a function may run as separate transactions */ bool FunctionOpensTransactionBlock = true; @@ -118,10 +134,10 @@ static void CoordinatedSubTransactionCallback(SubXactEvent event, SubTransaction static void AdjustMaxPreparedTransactions(void); static void PushSubXact(SubTransactionId subId); static void PopSubXact(SubTransactionId subId); -static bool MaybeExecutingUDF(void); static void ResetGlobalVariables(void); static bool SwallowErrors(void (*func)(void)); static void ForceAllInProgressConnectionsToClose(void); +static void EnsurePrepareTransactionIsAllowed(void); /* @@ -185,6 +201,17 @@ InCoordinatedTransaction(void) } +/* + * Returns true if a distributed forcePushdown function is executed in + * the current transaction. + */ +bool +GetInForceDelegatedFuncExecution() +{ + return ShouldAllowRestricted2PC; +} + + /* * Use2PCForCoordinatedTransaction() signals that the current coordinated * transaction should use 2PC to commit. @@ -459,12 +486,7 @@ CoordinatedTransactionCallback(XactEvent event, void *arg) case XACT_EVENT_PARALLEL_PRE_COMMIT: case XACT_EVENT_PRE_PREPARE: { - if (InCoordinatedTransaction()) - { - ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED), - errmsg("cannot use 2PC in transactions involving " - "multiple servers"))); - } + EnsurePrepareTransactionIsAllowed(); break; } } @@ -550,7 +572,10 @@ ResetGlobalVariables() ShouldCoordinatedTransactionUse2PC = false; TransactionModifiedNodeMetadata = false; MetadataSyncOnCommit = false; + InDelegatedFunctionCall = false; ResetWorkerErrorIndication(); + AllowedDistributionColumnValue = NULL; + ShouldAllowRestricted2PC = false; } @@ -784,7 +809,7 @@ IsMultiStatementTransaction(void) * If the planner is being called from the executor, then we may also be in * a UDF. */ -static bool +bool MaybeExecutingUDF(void) { return ExecutorLevel > 1 || (ExecutorLevel == 1 && PlannerLevel > 0); @@ -801,3 +826,40 @@ TriggerMetadataSyncOnCommit(void) { MetadataSyncOnCommit = true; } + + +/* + * Function raises an exception, if the current backend started a coordinated + * transaction and got a PREPARE event to become a participant in a 2PC + * transaction coordinated by another node. + */ +static void +EnsurePrepareTransactionIsAllowed(void) +{ + if (!InCoordinatedTransaction()) + { + /* If the backend has not started a coordinated transaction */ + return; + } + + if (GetInForceDelegatedFuncExecution()) + { + /* + * If a distributed forcePushdown function is executed in the + * current coordinated transaction, allow restricted 2PC + */ + return; + } + + if (IsCitusInitiatedRemoteBackend()) + { + /* + * If this is a Citus-initiated backend + */ + return; + } + + ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED), + errmsg("cannot use 2PC in transactions involving " + "multiple servers"))); +} diff --git a/src/include/distributed/commands.h b/src/include/distributed/commands.h index d2a2bc4df..662eeaae7 100644 --- a/src/include/distributed/commands.h +++ b/src/include/distributed/commands.h @@ -506,7 +506,8 @@ extern char * GenerateBackupNameForProcCollision(const ObjectAddress *address); extern ObjectWithArgs * ObjectWithArgsFromOid(Oid funcOid); extern void UpdateFunctionDistributionInfo(const ObjectAddress *distAddress, int *distribution_argument_index, - int *colocationId); + int *colocationId, + bool *forcePushdown); /* vacuum.c - forward declarations */ extern void PostprocessVacuumStmt(VacuumStmt *vacuumStmt, const char *vacuumCommand); diff --git a/src/include/distributed/commands/utility_hook.h b/src/include/distributed/commands/utility_hook.h index 9ead0df8b..1ee18a206 100644 --- a/src/include/distributed/commands/utility_hook.h +++ b/src/include/distributed/commands/utility_hook.h @@ -18,6 +18,7 @@ #include "tcop/utility.h" #include "distributed/coordinator_protocol.h" +#include "distributed/function_call_delegation.h" #include "distributed/version_compat.h" #include "distributed/worker_transaction.h" @@ -37,6 +38,7 @@ extern bool EnableAlterRolePropagation; extern bool EnableAlterRoleSetPropagation; extern bool EnableAlterDatabaseOwner; extern int UtilityHookLevel; +extern bool InDelegatedProcedureCall; /* diff --git a/src/include/distributed/function_call_delegation.h b/src/include/distributed/function_call_delegation.h index 865ac0ce1..e0485ca5c 100644 --- a/src/include/distributed/function_call_delegation.h +++ b/src/include/distributed/function_call_delegation.h @@ -15,7 +15,16 @@ #include "distributed/multi_physical_planner.h" -PlannedStmt * TryToDelegateFunctionCall(DistributedPlanningContext *planContext); +/* + * These flags keep track of whether the process is currently in a delegated + * function or procedure call. + */ +extern bool InDelegatedFunctionCall; +extern bool InDelegatedProcedureCall; +PlannedStmt * TryToDelegateFunctionCall(DistributedPlanningContext *planContext); +extern void ResetAllowedShardKeyValue(void); +extern bool IsShardKeyValueAllowed(Const *shardKey, uint32 colocationId); + #endif /* FUNCTION_CALL_DELEGATION_H */ diff --git a/src/include/distributed/metadata/distobject.h b/src/include/distributed/metadata/distobject.h index ef7f21faa..659e8ab7f 100644 --- a/src/include/distributed/metadata/distobject.h +++ b/src/include/distributed/metadata/distobject.h @@ -16,6 +16,7 @@ #include "catalog/objectaddress.h" #define INVALID_DISTRIBUTION_ARGUMENT_INDEX -1 +#define NO_FORCE_PUSHDOWN 0 extern bool ObjectExists(const ObjectAddress *address); extern bool CitusExtensionObject(const ObjectAddress *objectAddress); diff --git a/src/include/distributed/metadata/pg_dist_object.h b/src/include/distributed/metadata/pg_dist_object.h index 735e20819..673b488c8 100644 --- a/src/include/distributed/metadata/pg_dist_object.h +++ b/src/include/distributed/metadata/pg_dist_object.h @@ -35,6 +35,7 @@ typedef struct FormData_pg_dist_object uint32 distribution_argument_index; /* only valid for distributed functions/procedures */ uint32 colocationid; /* only valid for distributed functions/procedures */ + boolean forced_pushdown; /* only valid for distributed functions */ #endif } FormData_pg_dist_object; @@ -49,7 +50,7 @@ typedef FormData_pg_dist_object *Form_pg_dist_object; * compiler constants for pg_dist_object * ---------------- */ -#define Natts_pg_dist_object 8 +#define Natts_pg_dist_object 9 #define Anum_pg_dist_object_classid 1 #define Anum_pg_dist_object_objid 2 #define Anum_pg_dist_object_objsubid 3 @@ -58,5 +59,6 @@ typedef FormData_pg_dist_object *Form_pg_dist_object; #define Anum_pg_dist_object_object_args 6 #define Anum_pg_dist_object_distribution_argument_index 7 #define Anum_pg_dist_object_colocationid 8 +#define Anum_pg_dist_object_force_pushdown 9 #endif /* PG_DIST_OBJECT_H */ diff --git a/src/include/distributed/metadata_cache.h b/src/include/distributed/metadata_cache.h index 4461cb1e9..fc80661a4 100644 --- a/src/include/distributed/metadata_cache.h +++ b/src/include/distributed/metadata_cache.h @@ -115,6 +115,7 @@ typedef struct DistObjectCacheEntry int distributionArgIndex; int colocationId; + bool forcePushdown; } DistObjectCacheEntry; typedef enum diff --git a/src/include/distributed/metadata_sync.h b/src/include/distributed/metadata_sync.h index 324c8a864..c66cbfe4d 100644 --- a/src/include/distributed/metadata_sync.h +++ b/src/include/distributed/metadata_sync.h @@ -36,7 +36,8 @@ extern List * MetadataCreateCommands(void); extern List * MetadataDropCommands(void); extern char * MarkObjectsDistributedCreateCommand(List *addresses, List *distributionArgumentIndexes, - List *colocationIds); + List *colocationIds, + List *forcePushdowns); extern char * DistributionCreateCommand(CitusTableCacheEntry *cacheEntry); extern char * DistributionDeleteCommand(const char *schemaName, const char *tableName); diff --git a/src/include/distributed/multi_executor.h b/src/include/distributed/multi_executor.h index 143f5a1c7..3648dbc1b 100644 --- a/src/include/distributed/multi_executor.h +++ b/src/include/distributed/multi_executor.h @@ -149,6 +149,8 @@ extern void ExtractParametersFromParamList(ParamListInfo paramListInfo, const char ***parameterValues, bool useOriginalCustomTypeOids); extern ParamListInfo ExecutorBoundParams(void); +extern void EnsureRemoteTaskExecutionAllowed(void); +extern bool InTaskExecution(void); #endif /* MULTI_EXECUTOR_H */ diff --git a/src/include/distributed/multi_physical_planner.h b/src/include/distributed/multi_physical_planner.h index 8757c149d..1a3630f81 100644 --- a/src/include/distributed/multi_physical_planner.h +++ b/src/include/distributed/multi_physical_planner.h @@ -161,6 +161,7 @@ typedef struct Job * query. */ bool parametersInJobQueryResolved; + uint32 colocationId; /* common colocation group ID of the relations */ } Job; diff --git a/src/include/distributed/transaction_management.h b/src/include/distributed/transaction_management.h index c3c57f244..cf9b30490 100644 --- a/src/include/distributed/transaction_management.h +++ b/src/include/distributed/transaction_management.h @@ -12,6 +12,12 @@ #include "lib/ilist.h" #include "lib/stringinfo.h" #include "nodes/pg_list.h" +#include "lib/stringinfo.h" +#include "nodes/primnodes.h" + +/* forward declare, to avoid recursive includes */ +struct DistObjectCacheEntry; + /* describes what kind of modifications have occurred in the current transaction */ typedef enum @@ -53,6 +59,18 @@ typedef struct SubXactContext StringInfo setLocalCmds; } SubXactContext; +/* + * Function delegated with force_pushdown call enforces the distribution argument + * along with the colocationId. The latter one is equally important to not allow + * the same partition key value into another distributed table which is not co-located + * and therefore might be on a different node. + */ +typedef struct AllowedDistributionColumn +{ + Const *distributionColumnValue; + uint32 colocationId; +} AllowedDistributionColumn; + /* * GUC that determines whether a SELECT in a transaction block should also run in * a transaction block on the worker. @@ -100,6 +118,9 @@ extern void Use2PCForCoordinatedTransaction(void); extern bool GetCoordinatedTransactionShouldUse2PC(void); extern bool IsMultiStatementTransaction(void); extern void EnsureDistributedTransactionId(void); +extern bool GetInForceDelegatedFuncExecution(void); +extern bool MaybeExecutingUDF(void); + /* initialization function(s) */ extern void InitializeTransactionManagement(void); diff --git a/src/test/regress/diff b/src/test/regress/diff new file mode 100644 index 000000000..5112bcdb7 --- /dev/null +++ b/src/test/regress/diff @@ -0,0 +1,27 @@ +1684a1685,1687 +> ALTER TABLE mx_testing_schema.mx_test_table ADD CONSTRAINT mx_test_table_col_1_key UNIQUE (col_1) +> ALTER TABLE mx_testing_schema.mx_test_table OWNER TO postgres +> ALTER TABLE mx_testing_schema.mx_test_table OWNER TO postgres +1693,1695d1695 +< ALTER TABLE mx_testing_schema.mx_test_table ADD CONSTRAINT mx_test_table_col_1_key UNIQUE (col_1) +< ALTER TABLE mx_testing_schema.mx_test_table OWNER TO postgres +< ALTER TABLE mx_testing_schema.mx_test_table OWNER TO postgres +1702d1701 +< CREATE INDEX mx_index ON mx_testing_schema.mx_test_table USING btree (col_2) +1704a1704,1705 +> CREATE INDEX mx_index ON mx_testing_schema.mx_test_table USING btree (col_2) +> CREATE TABLE mx_testing_schema.mx_test_table (col_1 integer, col_2 text NOT NULL, col_3 bigint DEFAULT nextval('mx_testing_schema.mx_test_table_col_3_seq'::regclass) NOT NULL, col_4 bigint DEFAULT nextval('public.user_defined_seq'::regclass)) +1707d1707 +< CREATE TABLE mx_testing_schema.mx_test_table (col_1 integer, col_2 text NOT NULL, col_3 bigint DEFAULT nextval('mx_testing_schema.mx_test_table_col_3_seq'::regclass) NOT NULL, col_4 bigint DEFAULT nextval('public.user_defined_seq'::regclass)) +1711a1712 +> SELECT citus_internal_add_partition_metadata ('mx_testing_schema.mx_test_table'::regclass, 'h', 'col_1', 0, 's') +1714d1714 +< SELECT citus_internal_add_partition_metadata ('mx_testing_schema.mx_test_table'::regclass, 'h', 'col_1', 0, 's') +1722a1723 +> SELECT worker_create_truncate_trigger('mx_testing_schema.mx_test_table') +1725d1725 +< SELECT worker_create_truncate_trigger('mx_testing_schema.mx_test_table') +1738a1739 +> WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) AS (VALUES ('mx_testing_schema.mx_test_table'::regclass, 1310000, 't'::"char", '-2147483648', '-1610612737'), ('mx_testing_schema.mx_test_table'::regclass, 1310001, 't'::"char", '-1610612736', '-1073741825'), ('mx_testing_schema.mx_test_table'::regclass, 1310002, 't'::"char", '-1073741824', '-536870913'), ('mx_testing_schema.mx_test_table'::regclass, 1310003, 't'::"char", '-536870912', '-1'), ('mx_testing_schema.mx_test_table'::regclass, 1310004, 't'::"char", '0', '536870911'), ('mx_testing_schema.mx_test_table'::regclass, 1310005, 't'::"char", '536870912', '1073741823'), ('mx_testing_schema.mx_test_table'::regclass, 1310006, 't'::"char", '1073741824', '1610612735'), ('mx_testing_schema.mx_test_table'::regclass, 1310007, 't'::"char", '1610612736', '2147483647')) SELECT citus_internal_add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data; +1741d1741 +< WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) AS (VALUES ('mx_testing_schema.mx_test_table'::regclass, 1310000, 't'::"char", '-2147483648', '-1610612737'), ('mx_testing_schema.mx_test_table'::regclass, 1310001, 't'::"char", '-1610612736', '-1073741825'), ('mx_testing_schema.mx_test_table'::regclass, 1310002, 't'::"char", '-1073741824', '-536870913'), ('mx_testing_schema.mx_test_table'::regclass, 1310003, 't'::"char", '-536870912', '-1'), ('mx_testing_schema.mx_test_table'::regclass, 1310004, 't'::"char", '0', '536870911'), ('mx_testing_schema.mx_test_table'::regclass, 1310005, 't'::"char", '536870912', '1073741823'), ('mx_testing_schema.mx_test_table'::regclass, 1310006, 't'::"char", '1073741824', '1610612735'), ('mx_testing_schema.mx_test_table'::regclass, 1310007, 't'::"char", '1610612736', '2147483647')) SELECT citus_internal_add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data; diff --git a/src/test/regress/expected/citus_local_table_triggers.out b/src/test/regress/expected/citus_local_table_triggers.out index ac6906282..1a269c649 100644 --- a/src/test/regress/expected/citus_local_table_triggers.out +++ b/src/test/regress/expected/citus_local_table_triggers.out @@ -392,6 +392,13 @@ BEGIN RETURN NEW; END; $insert_100$ LANGUAGE plpgsql; +CREATE TABLE local_table (value int); +CREATE FUNCTION insert_100_local() RETURNS trigger AS $insert_100$ +BEGIN + INSERT INTO local_table VALUES (100); + RETURN NEW; +END; +$insert_100$ LANGUAGE plpgsql; BEGIN; CREATE TRIGGER insert_100_trigger AFTER TRUNCATE ON another_citus_local_table @@ -416,6 +423,7 @@ NOTICE: executing the command locally: SELECT value FROM citus_local_table_trig (2 rows) ROLLBACK; +-- cannot perform remote execution from a trigger on a Citus local table BEGIN; -- update should actually update something to test ON UPDATE CASCADE logic INSERT INTO another_citus_local_table VALUES (600); @@ -436,11 +444,70 @@ NOTICE: executing the command locally: SELECT worker_apply_shard_ddl_command (1 FOR EACH STATEMENT EXECUTE FUNCTION insert_100();') UPDATE another_citus_local_table SET value=value-1;; NOTICE: executing the command locally: UPDATE citus_local_table_triggers.another_citus_local_table_1507009 another_citus_local_table SET value = (value OPERATOR(pg_catalog.-) 1) -NOTICE: executing the command locally: INSERT INTO citus_local_table_triggers.reference_table_1507010 (value) VALUES (100) -NOTICE: executing the command locally: INSERT INTO citus_local_table_triggers.reference_table_1507010 (value) VALUES (100) +ERROR: cannot execute a distributed query from a query on a shard +ROLLBACK; +-- can perform regular execution from a trigger on a Citus local table +BEGIN; + -- update should actually update something to test ON UPDATE CASCADE logic + INSERT INTO another_citus_local_table VALUES (600); +NOTICE: executing the command locally: INSERT INTO citus_local_table_triggers.another_citus_local_table_1507009 (value) VALUES (600) + INSERT INTO citus_local_table VALUES (600); +NOTICE: executing the command locally: INSERT INTO citus_local_table_triggers.citus_local_table_1507001 (value) VALUES (600) + CREATE TRIGGER insert_100_trigger + AFTER UPDATE ON another_citus_local_table + FOR EACH STATEMENT EXECUTE FUNCTION insert_100_local(); +NOTICE: executing the command locally: SELECT worker_apply_shard_ddl_command (1507009, 'citus_local_table_triggers', 'CREATE TRIGGER insert_100_trigger + AFTER UPDATE ON another_citus_local_table + FOR EACH STATEMENT EXECUTE FUNCTION insert_100_local();') + CREATE TRIGGER insert_100_trigger + AFTER UPDATE ON citus_local_table + FOR EACH STATEMENT EXECUTE FUNCTION insert_100_local(); +NOTICE: executing the command locally: SELECT worker_apply_shard_ddl_command (1507001, 'citus_local_table_triggers', 'CREATE TRIGGER insert_100_trigger + AFTER UPDATE ON citus_local_table + FOR EACH STATEMENT EXECUTE FUNCTION insert_100_local();') + UPDATE another_citus_local_table SET value=value-1;; +NOTICE: executing the command locally: UPDATE citus_local_table_triggers.another_citus_local_table_1507009 another_citus_local_table SET value = (value OPERATOR(pg_catalog.-) 1) -- we should see two rows with "100" - SELECT * FROM reference_table; -NOTICE: executing the command locally: SELECT value FROM citus_local_table_triggers.reference_table_1507010 reference_table + SELECT * FROM local_table; + value +--------------------------------------------------------------------- + 100 + 100 +(2 rows) + +ROLLBACK; +-- can perform local execution from a trigger on a Citus local table +BEGIN; + SELECT citus_add_local_table_to_metadata('local_table'); + citus_add_local_table_to_metadata +--------------------------------------------------------------------- + +(1 row) + + -- update should actually update something to test ON UPDATE CASCADE logic + INSERT INTO another_citus_local_table VALUES (600); +NOTICE: executing the command locally: INSERT INTO citus_local_table_triggers.another_citus_local_table_1507009 (value) VALUES (600) + INSERT INTO citus_local_table VALUES (600); +NOTICE: executing the command locally: INSERT INTO citus_local_table_triggers.citus_local_table_1507001 (value) VALUES (600) + CREATE TRIGGER insert_100_trigger + AFTER UPDATE ON another_citus_local_table + FOR EACH STATEMENT EXECUTE FUNCTION insert_100_local(); +NOTICE: executing the command locally: SELECT worker_apply_shard_ddl_command (1507009, 'citus_local_table_triggers', 'CREATE TRIGGER insert_100_trigger + AFTER UPDATE ON another_citus_local_table + FOR EACH STATEMENT EXECUTE FUNCTION insert_100_local();') + CREATE TRIGGER insert_100_trigger + AFTER UPDATE ON citus_local_table + FOR EACH STATEMENT EXECUTE FUNCTION insert_100_local(); +NOTICE: executing the command locally: SELECT worker_apply_shard_ddl_command (1507001, 'citus_local_table_triggers', 'CREATE TRIGGER insert_100_trigger + AFTER UPDATE ON citus_local_table + FOR EACH STATEMENT EXECUTE FUNCTION insert_100_local();') + UPDATE another_citus_local_table SET value=value-1;; +NOTICE: executing the command locally: UPDATE citus_local_table_triggers.another_citus_local_table_1507009 another_citus_local_table SET value = (value OPERATOR(pg_catalog.-) 1) +NOTICE: executing the command locally: INSERT INTO citus_local_table_triggers.local_table_1507011 (value) VALUES (100) +NOTICE: executing the command locally: INSERT INTO citus_local_table_triggers.local_table_1507011 (value) VALUES (100) + -- we should see two rows with "100" + SELECT * FROM local_table; +NOTICE: executing the command locally: SELECT value FROM citus_local_table_triggers.local_table_1507011 local_table value --------------------------------------------------------------------- 100 @@ -456,11 +523,11 @@ CREATE TABLE par_another_citus_local_table_1 PARTITION OF par_another_citus_loca ALTER TABLE par_another_citus_local_table ADD CONSTRAINT fkey_self FOREIGN KEY(val) REFERENCES par_another_citus_local_table(val); ALTER TABLE par_citus_local_table ADD CONSTRAINT fkey_c_to_c FOREIGN KEY(val) REFERENCES par_another_citus_local_table(val) ON UPDATE CASCADE; SELECT citus_add_local_table_to_metadata('par_another_citus_local_table', cascade_via_foreign_keys=>true); -NOTICE: executing the command locally: SELECT worker_apply_inter_shard_ddl_command (1507011, 'citus_local_table_triggers', 1507012, 'citus_local_table_triggers', 'ALTER TABLE citus_local_table_triggers.par_another_citus_local_table ATTACH PARTITION citus_local_table_triggers.par_another_citus_local_table_1 FOR VALUES FROM (1) TO (10000);') -NOTICE: executing the command locally: SELECT pg_catalog.citus_run_local_command($$SELECT worker_fix_partition_shard_index_names('citus_local_table_triggers.par_another_citus_local_table_val_key_1507011'::regclass, 'citus_local_table_triggers.par_another_citus_local_table_1_1507012', 'par_another_citus_local_table_1_val_key_1507012')$$) -NOTICE: executing the command locally: SELECT worker_apply_inter_shard_ddl_command (1507013, 'citus_local_table_triggers', 1507014, 'citus_local_table_triggers', 'ALTER TABLE citus_local_table_triggers.par_citus_local_table ATTACH PARTITION citus_local_table_triggers.par_citus_local_table_1 FOR VALUES FROM (1) TO (10000);') -NOTICE: executing the command locally: SELECT worker_apply_inter_shard_ddl_command (1507011, 'citus_local_table_triggers', 1507011, 'citus_local_table_triggers', 'ALTER TABLE citus_local_table_triggers.par_another_citus_local_table ADD CONSTRAINT fkey_self FOREIGN KEY (val) REFERENCES citus_local_table_triggers.par_another_citus_local_table(val)') -NOTICE: executing the command locally: SELECT worker_apply_inter_shard_ddl_command (1507013, 'citus_local_table_triggers', 1507011, 'citus_local_table_triggers', 'ALTER TABLE citus_local_table_triggers.par_citus_local_table ADD CONSTRAINT fkey_c_to_c FOREIGN KEY (val) REFERENCES citus_local_table_triggers.par_another_citus_local_table(val) ON UPDATE CASCADE') +NOTICE: executing the command locally: SELECT worker_apply_inter_shard_ddl_command (1507012, 'citus_local_table_triggers', 1507013, 'citus_local_table_triggers', 'ALTER TABLE citus_local_table_triggers.par_another_citus_local_table ATTACH PARTITION citus_local_table_triggers.par_another_citus_local_table_1 FOR VALUES FROM (1) TO (10000);') +NOTICE: executing the command locally: SELECT pg_catalog.citus_run_local_command($$SELECT worker_fix_partition_shard_index_names('citus_local_table_triggers.par_another_citus_local_table_val_key_1507012'::regclass, 'citus_local_table_triggers.par_another_citus_local_table_1_1507013', 'par_another_citus_local_table_1_val_key_1507013')$$) +NOTICE: executing the command locally: SELECT worker_apply_inter_shard_ddl_command (1507014, 'citus_local_table_triggers', 1507015, 'citus_local_table_triggers', 'ALTER TABLE citus_local_table_triggers.par_citus_local_table ATTACH PARTITION citus_local_table_triggers.par_citus_local_table_1 FOR VALUES FROM (1) TO (10000);') +NOTICE: executing the command locally: SELECT worker_apply_inter_shard_ddl_command (1507012, 'citus_local_table_triggers', 1507012, 'citus_local_table_triggers', 'ALTER TABLE citus_local_table_triggers.par_another_citus_local_table ADD CONSTRAINT fkey_self FOREIGN KEY (val) REFERENCES citus_local_table_triggers.par_another_citus_local_table(val)') +NOTICE: executing the command locally: SELECT worker_apply_inter_shard_ddl_command (1507014, 'citus_local_table_triggers', 1507012, 'citus_local_table_triggers', 'ALTER TABLE citus_local_table_triggers.par_citus_local_table ADD CONSTRAINT fkey_c_to_c FOREIGN KEY (val) REFERENCES citus_local_table_triggers.par_another_citus_local_table(val) ON UPDATE CASCADE') citus_add_local_table_to_metadata --------------------------------------------------------------------- @@ -489,7 +556,7 @@ BEGIN; TRUNCATE par_another_citus_local_table CASCADE; NOTICE: truncate cascades to table "par_citus_local_table" NOTICE: truncate cascades to table "par_citus_local_table_1" -NOTICE: executing the command locally: INSERT INTO citus_local_table_triggers.par_reference_table_1507015 (val) VALUES (100) +NOTICE: executing the command locally: INSERT INTO citus_local_table_triggers.par_reference_table_1507016 (val) VALUES (100) NOTICE: executing the command locally: TRUNCATE TABLE citus_local_table_triggers.par_another_citus_local_table_xxxxx CASCADE NOTICE: truncate cascades to table "par_citus_local_table_xxxxx" NOTICE: truncate cascades to table "par_citus_local_table_1_xxxxx" @@ -497,12 +564,12 @@ NOTICE: executing the command locally: TRUNCATE TABLE citus_local_table_trigger NOTICE: truncate cascades to table "par_citus_local_table_xxxxx" NOTICE: truncate cascades to table "par_citus_local_table_1_xxxxx" NOTICE: truncate cascades to table "par_another_citus_local_table_xxxxx" -NOTICE: executing the command locally: INSERT INTO citus_local_table_triggers.par_reference_table_1507015 (val) VALUES (100) +NOTICE: executing the command locally: INSERT INTO citus_local_table_triggers.par_reference_table_1507016 (val) VALUES (100) NOTICE: executing the command locally: TRUNCATE TABLE citus_local_table_triggers.par_citus_local_table_xxxxx CASCADE NOTICE: executing the command locally: TRUNCATE TABLE citus_local_table_triggers.par_citus_local_table_1_xxxxx CASCADE -- we should see two rows with "100" SELECT * FROM par_reference_table; -NOTICE: executing the command locally: SELECT val FROM citus_local_table_triggers.par_reference_table_1507015 par_reference_table +NOTICE: executing the command locally: SELECT val FROM citus_local_table_triggers.par_reference_table_1507016 par_reference_table val --------------------------------------------------------------------- 100 @@ -512,4 +579,4 @@ NOTICE: executing the command locally: SELECT val FROM citus_local_table_trigge ROLLBACK; -- cleanup at exit DROP SCHEMA citus_local_table_triggers, "interesting!schema" CASCADE; -NOTICE: drop cascades to 20 other objects +NOTICE: drop cascades to 22 other objects diff --git a/src/test/regress/expected/distributed_functions.out b/src/test/regress/expected/distributed_functions.out index 056c49f42..68612c7d0 100644 --- a/src/test/regress/expected/distributed_functions.out +++ b/src/test/regress/expected/distributed_functions.out @@ -599,7 +599,7 @@ SELECT create_distributed_function('eq_with_param_names(macaddr, macaddr)', dist SELECT * FROM (SELECT unnest(master_metadata_snapshot()) as metadata_command order by 1) as innerResult WHERE metadata_command like '%distributed_object_data%'; metadata_command --------------------------------------------------------------------- - WITH distributed_object_data(typetext, objnames, objargs, distargumentindex, colocationid) AS (VALUES ('type', ARRAY['public.usage_access_type']::text[], ARRAY[]::text[], -1, 0), ('type', ARRAY['function_tests.dup_result']::text[], ARRAY[]::text[], -1, 0), ('function', ARRAY['public', 'usage_access_func']::text[], ARRAY['public.usage_access_type', 'integer[]']::text[], -1, 0), ('function', ARRAY['public', 'usage_access_func_third']::text[], ARRAY['integer', 'integer[]']::text[], 0, 50), ('function', ARRAY['function_tests', 'notice']::text[], ARRAY['pg_catalog.text']::text[], -1, 0), ('function', ARRAY['function_tests', 'dup']::text[], ARRAY['pg_catalog.macaddr']::text[], 0, 52), ('function', ARRAY['function_tests', 'eq_with_param_names']::text[], ARRAY['pg_catalog.macaddr', 'pg_catalog.macaddr']::text[], 0, 52), ('function', ARRAY['function_tests', 'eq_mi''xed_param_names']::text[], ARRAY['pg_catalog.macaddr', 'pg_catalog.macaddr']::text[], -1, 0), ('function', ARRAY['function_tests', 'agg_sfunc']::text[], ARRAY['integer', 'integer']::text[], -1, 0), ('function', ARRAY['function_tests', 'agg_invfunc']::text[], ARRAY['integer', 'integer']::text[], -1, 0), ('function', ARRAY['function_tests', 'agg_finalfunc']::text[], ARRAY['integer', 'integer']::text[], -1, 0), ('aggregate', ARRAY['function_tests', 'my_rank']::text[], ARRAY['pg_catalog."any"']::text[], -1, 0), ('function', ARRAY['function_tests', 'agg_names_sfunc']::text[], ARRAY['function_tests.dup_result', 'function_tests.dup_result', 'function_tests.dup_result']::text[], -1, 0), ('function', ARRAY['function_tests', 'agg_names_finalfunc']::text[], ARRAY['function_tests.dup_result']::text[], -1, 0), ('aggregate', ARRAY['function_tests', 'agg_names']::text[], ARRAY['function_tests.dup_result', 'function_tests.dup_result']::text[], -1, 0), ('sequence', ARRAY['public', 'user_defined_seq']::text[], ARRAY[]::text[], -1, 0), ('role', ARRAY['postgres']::text[], ARRAY[]::text[], -1, 0), ('database', ARRAY['regression']::text[], ARRAY[]::text[], -1, 0), ('server', ARRAY['fake_fdw_server']::text[], ARRAY[]::text[], -1, 0), ('schema', ARRAY['public']::text[], ARRAY[]::text[], -1, 0), ('schema', ARRAY['mx_testing_schema']::text[], ARRAY[]::text[], -1, 0), ('schema', ARRAY['mx_testing_schema_2']::text[], ARRAY[]::text[], -1, 0), ('schema', ARRAY['mx_test_schema_1']::text[], ARRAY[]::text[], -1, 0), ('schema', ARRAY['mx_test_schema_2']::text[], ARRAY[]::text[], -1, 0), ('schema', ARRAY['schema_colocation']::text[], ARRAY[]::text[], -1, 0), ('schema', ARRAY['function_tests']::text[], ARRAY[]::text[], -1, 0), ('schema', ARRAY['function_tests2']::text[], ARRAY[]::text[], -1, 0), ('extension', ARRAY['plpgsql']::text[], ARRAY[]::text[], -1, 0)) SELECT citus_internal_add_object_metadata(typetext, objnames, objargs, distargumentindex::int, colocationid::int) FROM distributed_object_data; + WITH distributed_object_data(typetext, objnames, objargs, distargumentindex, colocationid, force_pushdown) AS (VALUES ('type', ARRAY['public.usage_access_type']::text[], ARRAY[]::text[], -1, 0, false), ('type', ARRAY['function_tests.dup_result']::text[], ARRAY[]::text[], -1, 0, false), ('function', ARRAY['public', 'usage_access_func']::text[], ARRAY['public.usage_access_type', 'integer[]']::text[], -1, 0, false), ('function', ARRAY['public', 'usage_access_func_third']::text[], ARRAY['integer', 'integer[]']::text[], 0, 50, false), ('function', ARRAY['function_tests', 'notice']::text[], ARRAY['pg_catalog.text']::text[], -1, 0, false), ('function', ARRAY['function_tests', 'dup']::text[], ARRAY['pg_catalog.macaddr']::text[], 0, 52, false), ('function', ARRAY['function_tests', 'eq_with_param_names']::text[], ARRAY['pg_catalog.macaddr', 'pg_catalog.macaddr']::text[], 0, 52, false), ('function', ARRAY['function_tests', 'eq_mi''xed_param_names']::text[], ARRAY['pg_catalog.macaddr', 'pg_catalog.macaddr']::text[], -1, 0, false), ('function', ARRAY['function_tests', 'agg_sfunc']::text[], ARRAY['integer', 'integer']::text[], -1, 0, false), ('function', ARRAY['function_tests', 'agg_invfunc']::text[], ARRAY['integer', 'integer']::text[], -1, 0, false), ('function', ARRAY['function_tests', 'agg_finalfunc']::text[], ARRAY['integer', 'integer']::text[], -1, 0, false), ('aggregate', ARRAY['function_tests', 'my_rank']::text[], ARRAY['pg_catalog."any"']::text[], -1, 0, false), ('function', ARRAY['function_tests', 'agg_names_sfunc']::text[], ARRAY['function_tests.dup_result', 'function_tests.dup_result', 'function_tests.dup_result']::text[], -1, 0, false), ('function', ARRAY['function_tests', 'agg_names_finalfunc']::text[], ARRAY['function_tests.dup_result']::text[], -1, 0, false), ('aggregate', ARRAY['function_tests', 'agg_names']::text[], ARRAY['function_tests.dup_result', 'function_tests.dup_result']::text[], -1, 0, false), ('sequence', ARRAY['public', 'user_defined_seq']::text[], ARRAY[]::text[], -1, 0, false), ('role', ARRAY['postgres']::text[], ARRAY[]::text[], -1, 0, false), ('database', ARRAY['regression']::text[], ARRAY[]::text[], -1, 0, false), ('server', ARRAY['fake_fdw_server']::text[], ARRAY[]::text[], -1, 0, false), ('schema', ARRAY['public']::text[], ARRAY[]::text[], -1, 0, false), ('schema', ARRAY['mx_testing_schema']::text[], ARRAY[]::text[], -1, 0, false), ('schema', ARRAY['mx_testing_schema_2']::text[], ARRAY[]::text[], -1, 0, false), ('schema', ARRAY['mx_test_schema_1']::text[], ARRAY[]::text[], -1, 0, false), ('schema', ARRAY['mx_test_schema_2']::text[], ARRAY[]::text[], -1, 0, false), ('schema', ARRAY['schema_colocation']::text[], ARRAY[]::text[], -1, 0, false), ('schema', ARRAY['function_tests']::text[], ARRAY[]::text[], -1, 0, false), ('schema', ARRAY['function_tests2']::text[], ARRAY[]::text[], -1, 0, false), ('extension', ARRAY['plpgsql']::text[], ARRAY[]::text[], -1, 0, false)) SELECT citus_internal_add_object_metadata(typetext, objnames, objargs, distargumentindex::int, colocationid::int, force_pushdown::bool) FROM distributed_object_data; (1 row) -- valid distribution with distribution_arg_index diff --git a/src/test/regress/expected/forcepushdown_functions.out b/src/test/regress/expected/forcepushdown_functions.out new file mode 100644 index 000000000..51db60898 --- /dev/null +++ b/src/test/regress/expected/forcepushdown_functions.out @@ -0,0 +1,1148 @@ +SET citus.log_remote_commands TO OFF; +DROP SCHEMA IF EXISTS forcepushdown_schema CASCADE; +NOTICE: schema "forcepushdown_schema" does not exist, skipping +CREATE SCHEMA forcepushdown_schema; +SET search_path TO 'forcepushdown_schema'; +SET citus.shard_replication_factor = 1; +SET citus.shard_count = 32; +SET citus.next_shard_id TO 900000; +CREATE TABLE test_forcepushdown(intcol int PRIMARY KEY, data char(50) default 'default'); +SELECT create_distributed_table('test_forcepushdown', 'intcol', colocate_with := 'none'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +-- +--Table in a different colocation group +-- +CREATE TABLE test_forcepushdown_noncolocate(intcol int PRIMARY KEY); +SELECT create_distributed_table('test_forcepushdown_noncolocate', 'intcol', colocate_with := 'none'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +CREATE FUNCTION insert_data(a integer) +RETURNS void LANGUAGE plpgsql AS $fn$ +BEGIN + INSERT INTO forcepushdown_schema.test_forcepushdown VALUES (a); +END; +$fn$; +CREATE FUNCTION insert_data_non_distarg(a integer) +RETURNS void LANGUAGE plpgsql AS $fn$ +BEGIN + INSERT INTO forcepushdown_schema.test_forcepushdown VALUES (a+1); +END; +$fn$; +CREATE FUNCTION update_data_nonlocal(a integer) +RETURNS void LANGUAGE plpgsql AS $fn$ +BEGIN + UPDATE forcepushdown_schema.test_forcepushdown SET data = 'non-default'; +END; +$fn$; +CREATE FUNCTION insert_data_noncolocation(a int) +RETURNS void LANGUAGE plpgsql AS $fn$ +BEGIN + -- Insert into a different table than the function is colocated with + INSERT INTO forcepushdown_schema.test_forcepushdown_noncolocate VALUES (a); +END; +$fn$; +SELECT create_distributed_function( + 'insert_data(int)', 'a', + colocate_with := 'test_forcepushdown', + force_pushdown := true +); + create_distributed_function +--------------------------------------------------------------------- + +(1 row) + +SELECT create_distributed_function( + 'insert_data_non_distarg(int)', 'a', + colocate_with := 'test_forcepushdown', + force_pushdown := true +); + create_distributed_function +--------------------------------------------------------------------- + +(1 row) + +SELECT create_distributed_function( + 'update_data_nonlocal(int)', 'a', + colocate_with := 'test_forcepushdown', + force_pushdown := true +); + create_distributed_function +--------------------------------------------------------------------- + +(1 row) + +SELECT create_distributed_function( + 'insert_data_noncolocation(int)', 'a', + colocate_with := 'test_forcepushdown', + force_pushdown := true +); + create_distributed_function +--------------------------------------------------------------------- + +(1 row) + +SET client_min_messages TO DEBUG1; +--SET citus.log_remote_commands TO on; +SELECT public.wait_until_metadata_sync(30000); + wait_until_metadata_sync +--------------------------------------------------------------------- + +(1 row) + +SELECT 'Transaction with no errors' Testing; + testing +--------------------------------------------------------------------- + Transaction with no errors +(1 row) + +BEGIN; +INSERT INTO forcepushdown_schema.test_forcepushdown VALUES (1); +-- This call will insert both the rows locally on the remote worker +SELECT insert_data(2); +DEBUG: pushing down function call in a multi-statement transaction +DEBUG: pushing down the function call + insert_data +--------------------------------------------------------------------- + +(1 row) + +INSERT INTO forcepushdown_schema.test_forcepushdown VALUES (3); +COMMIT; +SELECT 'Transaction with duplicate error in the remote function' Testing; + testing +--------------------------------------------------------------------- + Transaction with duplicate error in the remote function +(1 row) + +BEGIN; +INSERT INTO forcepushdown_schema.test_forcepushdown VALUES (4); +-- This call will fail with duplicate error on the remote worker +SELECT insert_data(3); +DEBUG: pushing down function call in a multi-statement transaction +DEBUG: pushing down the function call +ERROR: duplicate key value violates unique constraint "test_forcepushdown_pkey_900015" +DETAIL: Key (intcol)=(3) already exists. +CONTEXT: SQL statement "INSERT INTO forcepushdown_schema.test_forcepushdown VALUES (a)" +PL/pgSQL function forcepushdown_schema.insert_data(integer) line XX at SQL statement +while executing command on localhost:xxxxx +INSERT INTO forcepushdown_schema.test_forcepushdown VALUES (5); +ERROR: current transaction is aborted, commands ignored until end of transaction block +COMMIT; +SELECT 'Transaction with duplicate error in the local statement' Testing; + testing +--------------------------------------------------------------------- + Transaction with duplicate error in the local statement +(1 row) + +BEGIN; +INSERT INTO forcepushdown_schema.test_forcepushdown VALUES (6); +-- This call will insert both the rows locally on the remote worker +SELECT insert_data(7); +DEBUG: pushing down function call in a multi-statement transaction +DEBUG: pushing down the function call + insert_data +--------------------------------------------------------------------- + +(1 row) + +INSERT INTO forcepushdown_schema.test_forcepushdown VALUES (8); +-- This will fail +INSERT INTO forcepushdown_schema.test_forcepushdown VALUES (8); +ERROR: duplicate key value violates unique constraint "test_forcepushdown_pkey_900000" +DETAIL: Key (intcol)=(8) already exists. +CONTEXT: while executing command on localhost:xxxxx +COMMIT; +SELECT 'Transaction with function using non-distribution argument' Testing; + testing +--------------------------------------------------------------------- + Transaction with function using non-distribution argument +(1 row) + +BEGIN; +-- This should fail +SELECT insert_data_non_distarg(9); +DEBUG: pushing down function call in a multi-statement transaction +DEBUG: pushing down the function call +ERROR: queries must filter by the distribution argument in the same colocation group when using the forced function pushdown +HINT: consider turning off the flag force_pushdown instead +CONTEXT: SQL statement "INSERT INTO forcepushdown_schema.test_forcepushdown VALUES (a+1)" +PL/pgSQL function forcepushdown_schema.insert_data_non_distarg(integer) line XX at SQL statement +while executing command on localhost:xxxxx +COMMIT; +SELECT 'Transaction with function doing remote connection' Testing; + testing +--------------------------------------------------------------------- + Transaction with function doing remote connection +(1 row) + +BEGIN; +-- This statement will pass +INSERT INTO forcepushdown_schema.test_forcepushdown VALUES (11); +-- This call will try to update rows locally and on remote node(s) +SELECT update_data_nonlocal(12); +DEBUG: pushing down function call in a multi-statement transaction +DEBUG: pushing down the function call +ERROR: cannot execute a distributed query from a query on a shard +CONTEXT: SQL statement "UPDATE forcepushdown_schema.test_forcepushdown SET data = 'non-default'" +PL/pgSQL function forcepushdown_schema.update_data_nonlocal(integer) line XX at SQL statement +while executing command on localhost:xxxxx +INSERT INTO forcepushdown_schema.test_forcepushdown VALUES (13); +ERROR: current transaction is aborted, commands ignored until end of transaction block +COMMIT; +SELECT 'Transaction with no errors but with a rollback' Testing; + testing +--------------------------------------------------------------------- + Transaction with no errors but with a rollback +(1 row) + +BEGIN; +INSERT INTO forcepushdown_schema.test_forcepushdown VALUES (14); +-- This call will insert both the rows locally on the remote worker +SELECT insert_data(15); +DEBUG: pushing down function call in a multi-statement transaction +DEBUG: pushing down the function call + insert_data +--------------------------------------------------------------------- + +(1 row) + +INSERT INTO forcepushdown_schema.test_forcepushdown VALUES (16); +ROLLBACK; +-- +-- Add function with pushdown=true in the targetList of a query +-- +BEGIN; +-- Query gets delegated to the node of the shard xx_900001 for the key=1, +-- and the function inserts value (1+17) locally on the shard xx_900031 +SELECT insert_data(intcol+17) from test_forcepushdown where intcol = 1; + insert_data +--------------------------------------------------------------------- + +(1 row) + +-- This will fail with duplicate error as the function already inserted +-- the value(1+17) +SELECT insert_data(18); +DEBUG: pushing down function call in a multi-statement transaction +DEBUG: pushing down the function call +ERROR: duplicate key value violates unique constraint "test_forcepushdown_pkey_900031" +DETAIL: Key (intcol)=(18) already exists. +CONTEXT: SQL statement "INSERT INTO forcepushdown_schema.test_forcepushdown VALUES (a)" +PL/pgSQL function forcepushdown_schema.insert_data(integer) line XX at SQL statement +while executing command on localhost:xxxxx +COMMIT; +-- +-- Access a table with the same shard key as distribution argument but in a +-- different colocation group. +-- +BEGIN; +SELECT insert_data_noncolocation(19); +DEBUG: pushing down function call in a multi-statement transaction +DEBUG: pushing down the function call +ERROR: queries must filter by the distribution argument in the same colocation group when using the forced function pushdown +HINT: consider turning off the flag force_pushdown instead +CONTEXT: SQL statement "INSERT INTO forcepushdown_schema.test_forcepushdown_noncolocate VALUES (a)" +PL/pgSQL function forcepushdown_schema.insert_data_noncolocation(integer) line XX at SQL statement +while executing command on localhost:xxxxx +COMMIT; +SELECT insert_data_noncolocation(19); +DEBUG: pushing down the function call +ERROR: queries must filter by the distribution argument in the same colocation group when using the forced function pushdown +HINT: consider turning off the flag force_pushdown instead +CONTEXT: SQL statement "INSERT INTO forcepushdown_schema.test_forcepushdown_noncolocate VALUES (a)" +PL/pgSQL function forcepushdown_schema.insert_data_noncolocation(integer) line XX at SQL statement +while executing command on localhost:xxxxx +-- This should have only the first 3 rows as all other transactions were rolled back. +SELECT * FROM forcepushdown_schema.test_forcepushdown ORDER BY 1; + intcol | data +--------------------------------------------------------------------- + 1 | default + 2 | default + 3 | default +(3 rows) + +-- +-- Nested call, function with pushdown=false calling function with pushdown=true +-- +CREATE TABLE test_nested (id int, name text); +SELECT create_distributed_table('test_nested','id'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +INSERT INTO test_nested VALUES (100,'hundred'); +INSERT INTO test_nested VALUES (200,'twohundred'); +INSERT INTO test_nested VALUES (300,'threehundred'); +INSERT INTO test_nested VALUES (400,'fourhundred'); +CREATE OR REPLACE FUNCTION inner_force_pushdown_function(int) +RETURNS NUMERIC AS $$ +DECLARE ret_val NUMERIC; +BEGIN + SELECT max(id)::numeric+1 INTO ret_val FROM forcepushdown_schema.test_nested WHERE id = $1; + RAISE NOTICE 'inner_force_pushdown_function():%', ret_val; + RETURN ret_val; +END; +$$ LANGUAGE plpgsql; +CREATE OR REPLACE FUNCTION func_calls_forcepush_func() +RETURNS NUMERIC AS $$ +DECLARE incremented_val NUMERIC; +BEGIN + -- Constant distribution argument + SELECT inner_force_pushdown_function INTO incremented_val FROM inner_force_pushdown_function(100); + RETURN incremented_val; +END; +$$ LANGUAGE plpgsql; +SELECT create_distributed_function('func_calls_forcepush_func()'); +DEBUG: switching to sequential query execution mode +DETAIL: A distributed function is created. To make sure subsequent commands see the type correctly we need to make sure to use only one connection for all future commands + create_distributed_function +--------------------------------------------------------------------- + +(1 row) + +SELECT create_distributed_function('inner_force_pushdown_function(int)', '$1', colocate_with := 'test_nested', force_pushdown := true); +DEBUG: switching to sequential query execution mode +DETAIL: A distributed function is created. To make sure subsequent commands see the type correctly we need to make sure to use only one connection for all future commands + create_distributed_function +--------------------------------------------------------------------- + +(1 row) + +SELECT public.wait_until_metadata_sync(30000); + wait_until_metadata_sync +--------------------------------------------------------------------- + +(1 row) + +BEGIN; +SELECT func_calls_forcepush_func(); +DEBUG: not pushing down function calls in a multi-statement transaction +DEBUG: pushing down function call in a multi-statement transaction +CONTEXT: SQL statement "SELECT inner_force_pushdown_function FROM inner_force_pushdown_function(100)" +PL/pgSQL function func_calls_forcepush_func() line XX at SQL statement +DEBUG: pushing down the function call +CONTEXT: SQL statement "SELECT inner_force_pushdown_function FROM inner_force_pushdown_function(100)" +PL/pgSQL function func_calls_forcepush_func() line XX at SQL statement +NOTICE: inner_force_pushdown_function():101 +DETAIL: from localhost:xxxxx +CONTEXT: SQL statement "SELECT inner_force_pushdown_function FROM inner_force_pushdown_function(100)" +PL/pgSQL function func_calls_forcepush_func() line XX at SQL statement + func_calls_forcepush_func +--------------------------------------------------------------------- + 101 +(1 row) + +COMMIT; +SELECT func_calls_forcepush_func(); +DEBUG: function does not have co-located tables +NOTICE: inner_force_pushdown_function():101 +DETAIL: from localhost:xxxxx +CONTEXT: SQL statement "SELECT inner_force_pushdown_function FROM inner_force_pushdown_function(100)" +PL/pgSQL function func_calls_forcepush_func() line XX at SQL statement + func_calls_forcepush_func +--------------------------------------------------------------------- + 101 +(1 row) + +CREATE OR REPLACE FUNCTION get_val() +RETURNS INT AS $$ +BEGIN + RETURN 100::INT; +END; +$$ LANGUAGE plpgsql; +-- +-- UDF calling another UDF in a FROM clause +-- fn() +-- { +-- select res into var from fn(); +-- } +-- +CREATE OR REPLACE FUNCTION func_calls_forcepush_func_infrom() +RETURNS NUMERIC AS $$ +DECLARE incremented_val NUMERIC; +DECLARE add_val INT; +BEGIN + add_val := get_val(); + SELECT inner_force_pushdown_function INTO incremented_val FROM inner_force_pushdown_function(add_val + 100); + RETURN incremented_val; +END; +$$ LANGUAGE plpgsql; +SELECT func_calls_forcepush_func_infrom(); +DEBUG: pushing down function call in a multi-statement transaction +CONTEXT: SQL statement "SELECT inner_force_pushdown_function FROM inner_force_pushdown_function(add_val + 100)" +PL/pgSQL function func_calls_forcepush_func_infrom() line XX at SQL statement +DEBUG: pushing down the function call +CONTEXT: SQL statement "SELECT inner_force_pushdown_function FROM inner_force_pushdown_function(add_val + 100)" +PL/pgSQL function func_calls_forcepush_func_infrom() line XX at SQL statement +NOTICE: inner_force_pushdown_function():201 +DETAIL: from localhost:xxxxx +CONTEXT: SQL statement "SELECT inner_force_pushdown_function FROM inner_force_pushdown_function(add_val + 100)" +PL/pgSQL function func_calls_forcepush_func_infrom() line XX at SQL statement + func_calls_forcepush_func_infrom +--------------------------------------------------------------------- + 201 +(1 row) + +BEGIN; +SELECT func_calls_forcepush_func_infrom(); +DEBUG: pushing down function call in a multi-statement transaction +CONTEXT: SQL statement "SELECT inner_force_pushdown_function FROM inner_force_pushdown_function(add_val + 100)" +PL/pgSQL function func_calls_forcepush_func_infrom() line XX at SQL statement +DEBUG: pushing down the function call +CONTEXT: SQL statement "SELECT inner_force_pushdown_function FROM inner_force_pushdown_function(add_val + 100)" +PL/pgSQL function func_calls_forcepush_func_infrom() line XX at SQL statement +NOTICE: inner_force_pushdown_function():201 +DETAIL: from localhost:xxxxx +CONTEXT: SQL statement "SELECT inner_force_pushdown_function FROM inner_force_pushdown_function(add_val + 100)" +PL/pgSQL function func_calls_forcepush_func_infrom() line XX at SQL statement + func_calls_forcepush_func_infrom +--------------------------------------------------------------------- + 201 +(1 row) + +COMMIT; +-- +-- UDF calling another UDF in the SELECT targetList +-- fn() +-- { +-- select fn() into var; +-- } +-- +CREATE OR REPLACE FUNCTION func_calls_forcepush_func_intarget() +RETURNS NUMERIC AS $$ +DECLARE incremented_val NUMERIC; +DECLARE add_val INT; +BEGIN + add_val := get_val(); + SELECT inner_force_pushdown_function(100 + 100) INTO incremented_val OFFSET 0; + RETURN incremented_val; +END; +$$ LANGUAGE plpgsql; +SELECT func_calls_forcepush_func_intarget(); +DEBUG: pushing down function call in a multi-statement transaction +CONTEXT: SQL statement "SELECT inner_force_pushdown_function(100 + 100) OFFSET 0" +PL/pgSQL function func_calls_forcepush_func_intarget() line XX at SQL statement +DEBUG: pushing down the function call +CONTEXT: SQL statement "SELECT inner_force_pushdown_function(100 + 100) OFFSET 0" +PL/pgSQL function func_calls_forcepush_func_intarget() line XX at SQL statement +NOTICE: inner_force_pushdown_function():201 +DETAIL: from localhost:xxxxx +CONTEXT: SQL statement "SELECT inner_force_pushdown_function(100 + 100) OFFSET 0" +PL/pgSQL function func_calls_forcepush_func_intarget() line XX at SQL statement + func_calls_forcepush_func_intarget +--------------------------------------------------------------------- + 201 +(1 row) + +BEGIN; +SELECT func_calls_forcepush_func_intarget(); +NOTICE: inner_force_pushdown_function():201 +DETAIL: from localhost:xxxxx +CONTEXT: SQL statement "SELECT inner_force_pushdown_function(100 + 100) OFFSET 0" +PL/pgSQL function func_calls_forcepush_func_intarget() line XX at SQL statement + func_calls_forcepush_func_intarget +--------------------------------------------------------------------- + 201 +(1 row) + +COMMIT; +-- +-- Recursive function call with pushdown=true +-- +CREATE OR REPLACE FUNCTION test_recursive(inp integer) +RETURNS INT AS $$ +DECLARE var INT; +BEGIN + RAISE NOTICE 'input:%', inp; + if (inp > 1) then + inp := inp - 1; + var := forcepushdown_schema.test_recursive(inp); + RETURN var; + else + RETURN inp; + END if; +END; +$$ LANGUAGE plpgsql; +SELECT create_distributed_function('test_recursive(int)', '$1', colocate_with := 'test_nested', force_pushdown := true); +DEBUG: switching to sequential query execution mode +DETAIL: A distributed function is created. To make sure subsequent commands see the type correctly we need to make sure to use only one connection for all future commands + create_distributed_function +--------------------------------------------------------------------- + +(1 row) + +BEGIN; +SELECT test_recursive(5); +DEBUG: pushing down function call in a multi-statement transaction +DEBUG: pushing down the function call +NOTICE: input:5 +DETAIL: from localhost:xxxxx +NOTICE: input:4 +DETAIL: from localhost:xxxxx +NOTICE: input:3 +DETAIL: from localhost:xxxxx +NOTICE: input:2 +DETAIL: from localhost:xxxxx +NOTICE: input:1 +DETAIL: from localhost:xxxxx + test_recursive +--------------------------------------------------------------------- + 1 +(1 row) + +END; +-- +-- Distributed function gets delegated indirectly (as part of a query) +-- +BEGIN; +-- Query lands on the shard with key = 300(shard __900089) and the function inserts locally +SELECT inner_force_pushdown_function(id) FROM test_nested WHERE id = 300; +NOTICE: inner_force_pushdown_function():301 +DETAIL: from localhost:xxxxx + inner_force_pushdown_function +--------------------------------------------------------------------- + 301 +(1 row) + +-- Query lands on the shard with key = 300(shard __900089) and the function inserts remotely +SELECT insert_data_non_distarg(id) FROM test_nested WHERE id = 300; +ERROR: cannot execute a distributed query from a query on a shard +CONTEXT: SQL statement "INSERT INTO forcepushdown_schema.test_forcepushdown VALUES (a+1)" +PL/pgSQL function forcepushdown_schema.insert_data_non_distarg(integer) line XX at SQL statement +while executing command on localhost:xxxxx +END; +-- +-- Non constant distribution arguments +-- +-- Param(PARAM_EXEC) node e.g. SELECT fn((SELECT col from test_nested where col=val)) +BEGIN; +SELECT inner_force_pushdown_function((SELECT id FROM test_nested WHERE id=400)); +ERROR: Non-constant arguments for forcePushdown functions not supported +CONTEXT: while executing command on localhost:xxxxx +END; +CREATE TABLE emp ( + empname text NOT NULL, + salary integer +); +CREATE TABLE emp_audit( + operation char(1) NOT NULL, + stamp timestamp NOT NULL, + userid text NOT NULL, + empname text NOT NULL, + salary integer +); +SELECT create_distributed_table('emp','empname'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +SELECT create_distributed_table('emp_audit','empname'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +CREATE OR REPLACE FUNCTION inner_emp(empname text) +RETURNS void +AS $$ +DECLARE +BEGIN + INSERT INTO emp VALUES (empname, 33); +END; +$$ LANGUAGE plpgsql; +CREATE OR REPLACE FUNCTION outer_emp() +RETURNS void +AS $$ +DECLARE +BEGIN + PERFORM inner_emp('hello'); +END; +$$ LANGUAGE plpgsql; +SELECT create_distributed_function('inner_emp(text)','empname', force_pushdown := true); +DEBUG: switching to sequential query execution mode +DETAIL: A distributed function is created. To make sure subsequent commands see the type correctly we need to make sure to use only one connection for all future commands + create_distributed_function +--------------------------------------------------------------------- + +(1 row) + +SELECT outer_emp(); +DEBUG: Skipping delegation of function from a PL/PgSQL simple expression +CONTEXT: SQL statement "SELECT inner_emp('hello')" +PL/pgSQL function outer_emp() line XX at PERFORM + outer_emp +--------------------------------------------------------------------- + +(1 row) + +SELECT * from emp; + empname | salary +--------------------------------------------------------------------- + hello | 33 +(1 row) + +-- +-- INSERT..SELECT +-- +CREATE FUNCTION insert_select_data(a integer) +RETURNS void LANGUAGE plpgsql AS $fn$ +BEGIN + INSERT INTO forcepushdown_schema.test_forcepushdown SELECT(a+1); +END; +$fn$; +SELECT create_distributed_function( + 'insert_select_data(int)', 'a', + colocate_with := 'test_forcepushdown', + force_pushdown := true +); +DEBUG: switching to sequential query execution mode +DETAIL: A distributed function is created. To make sure subsequent commands see the type correctly we need to make sure to use only one connection for all future commands + create_distributed_function +--------------------------------------------------------------------- + +(1 row) + +-- Function lands on worker1 and issues COPY ... INSERT on the worker2 into the shard_900021 +BEGIN; +INSERT INTO forcepushdown_schema.test_forcepushdown VALUES (30); +-- This will fail +SELECT insert_select_data(20); +DEBUG: pushing down function call in a multi-statement transaction +DEBUG: pushing down the function call +ERROR: cannot execute a distributed query from a query on a shard +CONTEXT: SQL statement "INSERT INTO forcepushdown_schema.test_forcepushdown SELECT(a+1)" +PL/pgSQL function forcepushdown_schema.insert_select_data(integer) line XX at SQL statement +while executing command on localhost:xxxxx +COMMIT; +-- Function lands on worker2 and issues COPY ... INSERT on the same node into the shard_900029 +BEGIN; +-- This will pass +SELECT insert_select_data(21); +DEBUG: pushing down function call in a multi-statement transaction +DEBUG: pushing down the function call + insert_select_data +--------------------------------------------------------------------- + +(1 row) + +END; +-- Function lands on worker2 and issues COPY ... INSERT on the worker1 into the shard_900028 +BEGIN; +INSERT INTO forcepushdown_schema.test_forcepushdown VALUES (30); +-- This will fail +SELECT insert_select_data(22); +DEBUG: pushing down function call in a multi-statement transaction +DEBUG: pushing down the function call +ERROR: cannot execute a distributed query from a query on a shard +CONTEXT: SQL statement "INSERT INTO forcepushdown_schema.test_forcepushdown SELECT(a+1)" +PL/pgSQL function forcepushdown_schema.insert_select_data(integer) line XX at SQL statement +while executing command on localhost:xxxxx +END; +-- Functions lands on worker1 and issues COPY ... INSERT on the worker2 into the shard_900021 +-- This will pass as there is no surrounding transaction +SELECT insert_select_data(20); +DEBUG: pushing down the function call + insert_select_data +--------------------------------------------------------------------- + +(1 row) + +-- (21+1) and (20+1) should appear +SELECT * FROM forcepushdown_schema.test_forcepushdown ORDER BY 1; + intcol | data +--------------------------------------------------------------------- + 1 | default + 2 | default + 3 | default + 21 | default + 22 | default +(5 rows) + +CREATE FUNCTION insert_select_data_nonlocal(a integer) +RETURNS void LANGUAGE plpgsql AS $fn$ +BEGIN + INSERT INTO forcepushdown_schema.test_forcepushdown(intcol) + SELECT intcol FROM forcepushdown_schema.test_forcepushdown_noncolocate; +END; +$fn$; +SELECT create_distributed_function( + 'insert_select_data_nonlocal(int)', 'a', + colocate_with := 'test_forcepushdown', + force_pushdown := true +); +DEBUG: switching to sequential query execution mode +DETAIL: A distributed function is created. To make sure subsequent commands see the type correctly we need to make sure to use only one connection for all future commands + create_distributed_function +--------------------------------------------------------------------- + +(1 row) + +INSERT INTO forcepushdown_schema.test_forcepushdown_noncolocate VALUES (30); +INSERT INTO forcepushdown_schema.test_forcepushdown_noncolocate VALUES (31); +INSERT INTO forcepushdown_schema.test_forcepushdown_noncolocate VALUES (32); +BEGIN; +INSERT INTO forcepushdown_schema.test_forcepushdown VALUES (40); +-- This will fail +SELECT insert_select_data_nonlocal(41); +DEBUG: pushing down function call in a multi-statement transaction +DEBUG: pushing down the function call +ERROR: cannot execute a distributed query from a query on a shard +CONTEXT: SQL statement "INSERT INTO forcepushdown_schema.test_forcepushdown(intcol) + SELECT intcol FROM forcepushdown_schema.test_forcepushdown_noncolocate" +PL/pgSQL function forcepushdown_schema.insert_select_data_nonlocal(integer) line XX at SQL statement +while executing command on localhost:xxxxx +COMMIT; +-- Above 3 rows (30, 31, 32) should appear now +SELECT insert_select_data_nonlocal(40); +DEBUG: pushing down the function call + insert_select_data_nonlocal +--------------------------------------------------------------------- + +(1 row) + +SELECT * FROM forcepushdown_schema.test_forcepushdown ORDER BY 1; + intcol | data +--------------------------------------------------------------------- + 1 | default + 2 | default + 3 | default + 21 | default + 22 | default + 30 | default + 31 | default + 32 | default +(8 rows) + +CREATE TABLE test_forcepushdown_char(data char(50) PRIMARY KEY); +DEBUG: CREATE TABLE / PRIMARY KEY will create implicit index "test_forcepushdown_char_pkey" for table "test_forcepushdown_char" +SELECT create_distributed_table('test_forcepushdown_char', 'data', colocate_with := 'none'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +CREATE TABLE test_forcepushdown_varchar(data varchar PRIMARY KEY); +DEBUG: CREATE TABLE / PRIMARY KEY will create implicit index "test_forcepushdown_varchar_pkey" for table "test_forcepushdown_varchar" +SELECT create_distributed_table('test_forcepushdown_varchar', 'data', colocate_with := 'none'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +CREATE TABLE test_forcepushdown_text(data text PRIMARY KEY); +DEBUG: CREATE TABLE / PRIMARY KEY will create implicit index "test_forcepushdown_text_pkey" for table "test_forcepushdown_text" +SELECT create_distributed_table('test_forcepushdown_text', 'data', colocate_with := 'none'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +CREATE FUNCTION insert_data_char(a char(50)) +RETURNS void LANGUAGE plpgsql AS $fn$ +BEGIN + INSERT INTO forcepushdown_schema.test_forcepushdown_char VALUES (a); +END; +$fn$; +SELECT create_distributed_function( + 'insert_data_char(char)', 'a', + colocate_with := 'test_forcepushdown_char', + force_pushdown := true +); +DEBUG: switching to sequential query execution mode +DETAIL: A distributed function is created. To make sure subsequent commands see the type correctly we need to make sure to use only one connection for all future commands + create_distributed_function +--------------------------------------------------------------------- + +(1 row) + +CREATE FUNCTION insert_data_varchar(a varchar) +RETURNS void LANGUAGE plpgsql AS $fn$ +BEGIN + INSERT INTO forcepushdown_schema.test_forcepushdown_varchar VALUES (a); +END; +$fn$; +SELECT create_distributed_function( + 'insert_data_varchar(varchar)', 'a', + colocate_with := 'test_forcepushdown_varchar', + force_pushdown := true +); +DEBUG: switching to sequential query execution mode +DETAIL: A distributed function is created. To make sure subsequent commands see the type correctly we need to make sure to use only one connection for all future commands + create_distributed_function +--------------------------------------------------------------------- + +(1 row) + +CREATE FUNCTION insert_data_text(a text) +RETURNS void LANGUAGE plpgsql AS $fn$ +BEGIN + INSERT INTO forcepushdown_schema.test_forcepushdown_text VALUES (a); +END; +$fn$; +SELECT create_distributed_function( + 'insert_data_text(text)', 'a', + colocate_with := 'test_forcepushdown_text', + force_pushdown := true +); +DEBUG: switching to sequential query execution mode +DETAIL: A distributed function is created. To make sure subsequent commands see the type correctly we need to make sure to use only one connection for all future commands + create_distributed_function +--------------------------------------------------------------------- + +(1 row) + +SELECT insert_data_varchar('VARCHAR'); +DEBUG: pushing down the function call + insert_data_varchar +--------------------------------------------------------------------- + +(1 row) + +BEGIN; +SELECT insert_data_varchar('VARCHAR2'); +DEBUG: pushing down function call in a multi-statement transaction +DEBUG: pushing down the function call + insert_data_varchar +--------------------------------------------------------------------- + +(1 row) + +COMMIT; +SELECT insert_data_text('TEXT'); +DEBUG: pushing down the function call + insert_data_text +--------------------------------------------------------------------- + +(1 row) + +BEGIN; +SELECT insert_data_text('TEXT2'); +DEBUG: pushing down function call in a multi-statement transaction +DEBUG: pushing down the function call + insert_data_text +--------------------------------------------------------------------- + +(1 row) + +COMMIT; +-- Char is failing as the datatype is represented differently in the +-- PL/PgSQL and the exec engine. +SELECT insert_data_char('CHAR'); +DEBUG: pushing down the function call +ERROR: queries must filter by the distribution argument in the same colocation group when using the forced function pushdown +HINT: consider turning off the flag force_pushdown instead +CONTEXT: SQL statement "INSERT INTO forcepushdown_schema.test_forcepushdown_char VALUES (a)" +PL/pgSQL function forcepushdown_schema.insert_data_char(character) line XX at SQL statement +while executing command on localhost:xxxxx +BEGIN; +SELECT insert_data_char('CHAR'); +DEBUG: pushing down function call in a multi-statement transaction +DEBUG: pushing down the function call +ERROR: queries must filter by the distribution argument in the same colocation group when using the forced function pushdown +HINT: consider turning off the flag force_pushdown instead +CONTEXT: SQL statement "INSERT INTO forcepushdown_schema.test_forcepushdown_char VALUES (a)" +PL/pgSQL function forcepushdown_schema.insert_data_char(character) line XX at SQL statement +while executing command on localhost:xxxxx +COMMIT; +SELECT * from test_forcepushdown_char; + data +--------------------------------------------------------------------- +(0 rows) + +SELECT * from test_forcepushdown_varchar; + data +--------------------------------------------------------------------- + VARCHAR2 + VARCHAR +(2 rows) + +SELECT * from test_forcepushdown_text; + data +--------------------------------------------------------------------- + TEXT2 + TEXT +(2 rows) + +-- Test sub query +CREATE TABLE test_subquery(data int, result int); +SELECT create_distributed_table('test_subquery', 'data', colocate_with := 'none'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +CREATE TABLE test_non_colocated(id int); +SELECT create_distributed_table('test_non_colocated', 'id', colocate_with := 'none'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +CREATE FUNCTION select_data(a integer) +RETURNS void LANGUAGE plpgsql AS $fn$ +DECLARE var INT; +BEGIN + SELECT result INTO var FROM forcepushdown_schema.test_subquery WHERE data = + (SELECT data FROM forcepushdown_schema.test_subquery WHERE data = a); + RAISE NOTICE 'Result: %', var; +END; +$fn$; +SELECT create_distributed_function( + 'select_data(int)', 'a', + colocate_with := 'test_subquery', + force_pushdown := true +); +DEBUG: switching to sequential query execution mode +DETAIL: A distributed function is created. To make sure subsequent commands see the type correctly we need to make sure to use only one connection for all future commands + create_distributed_function +--------------------------------------------------------------------- + +(1 row) + +CREATE FUNCTION select_data_noncolocate(a integer) +RETURNS void LANGUAGE plpgsql AS $fn$ +DECLARE var INT; +BEGIN + -- Key is the same but colocation ID is different + SELECT data INTO var FROM forcepushdown_schema.test_subquery WHERE data = + (SELECT id FROM forcepushdown_schema.test_non_colocated WHERE id = a); + RAISE NOTICE 'Result: %', var; +END; +$fn$; +SELECT create_distributed_function( + 'select_data_noncolocate(int)', 'a', + colocate_with := 'test_subquery', + force_pushdown := true +); +DEBUG: switching to sequential query execution mode +DETAIL: A distributed function is created. To make sure subsequent commands see the type correctly we need to make sure to use only one connection for all future commands + create_distributed_function +--------------------------------------------------------------------- + +(1 row) + +CREATE FUNCTION insert_select_data_cte1(a integer) +RETURNS void LANGUAGE plpgsql AS $fn$ +DECLARE var INT; +BEGIN + WITH ins AS (INSERT INTO forcepushdown_schema.test_subquery VALUES (a) RETURNING data) + SELECT ins.data INTO var FROM ins; + RAISE NOTICE 'Result: %', var; +END; +$fn$; +SELECT create_distributed_function( + 'insert_select_data_cte1(int)', 'a', + colocate_with := 'test_subquery', + force_pushdown := true +); +DEBUG: switching to sequential query execution mode +DETAIL: A distributed function is created. To make sure subsequent commands see the type correctly we need to make sure to use only one connection for all future commands + create_distributed_function +--------------------------------------------------------------------- + +(1 row) + +CREATE FUNCTION insert_select_data_cte2(a integer) +RETURNS void LANGUAGE plpgsql AS $fn$ +DECLARE var INT; +BEGIN + WITH ins AS (INSERT INTO forcepushdown_schema.test_subquery VALUES (a) RETURNING data) + SELECT ins.data INTO var FROM forcepushdown_schema.test_subquery, ins WHERE forcepushdown_schema.test_subquery.data = a; + RAISE NOTICE 'Result: %', var; +END; +$fn$; +SELECT create_distributed_function( + 'insert_select_data_cte2(int)', 'a', + colocate_with := 'test_subquery', + force_pushdown := true +); +DEBUG: switching to sequential query execution mode +DETAIL: A distributed function is created. To make sure subsequent commands see the type correctly we need to make sure to use only one connection for all future commands + create_distributed_function +--------------------------------------------------------------------- + +(1 row) + +CREATE FUNCTION insert_data_cte_nondist(a integer) +RETURNS void LANGUAGE plpgsql AS $fn$ +DECLARE var INT; +BEGIN + -- Inserting a non-distribution argument (a+1) + WITH ins AS (INSERT INTO forcepushdown_schema.test_subquery VALUES (a+1) RETURNING data) + SELECT ins.data INTO var FROM forcepushdown_schema.test_subquery, ins WHERE forcepushdown_schema.test_subquery.data = a; + RAISE NOTICE 'Result: %', var; +END; +$fn$; +SELECT create_distributed_function( + 'insert_data_cte_nondist(int)', 'a', + colocate_with := 'test_subquery', + force_pushdown := true +); +DEBUG: switching to sequential query execution mode +DETAIL: A distributed function is created. To make sure subsequent commands see the type correctly we need to make sure to use only one connection for all future commands + create_distributed_function +--------------------------------------------------------------------- + +(1 row) + +INSERT INTO forcepushdown_schema.test_subquery VALUES(100, -1); +-- This should pass +SELECT select_data(100); +DEBUG: pushing down the function call +NOTICE: Result: -1 +DETAIL: from localhost:xxxxx + select_data +--------------------------------------------------------------------- + +(1 row) + +BEGIN; +SELECT select_data(100); +DEBUG: pushing down function call in a multi-statement transaction +DEBUG: pushing down the function call +ERROR: cannot execute a distributed query from a query on a shard +CONTEXT: SQL statement "SELECT result FROM forcepushdown_schema.test_subquery WHERE data = + (SELECT data FROM forcepushdown_schema.test_subquery WHERE data = a)" +PL/pgSQL function forcepushdown_schema.select_data(integer) line XX at SQL statement +while executing command on localhost:xxxxx +END; +-- This should fail +SELECT select_data_noncolocate(100); +DEBUG: pushing down the function call +ERROR: queries must filter by the distribution argument in the same colocation group when using the forced function pushdown +HINT: consider turning off the flag force_pushdown instead +CONTEXT: SQL statement "SELECT data FROM forcepushdown_schema.test_subquery WHERE data = + (SELECT id FROM forcepushdown_schema.test_non_colocated WHERE id = a)" +PL/pgSQL function forcepushdown_schema.select_data_noncolocate(integer) line XX at SQL statement +while executing command on localhost:xxxxx +BEGIN; +SELECT select_data_noncolocate(100); +DEBUG: pushing down function call in a multi-statement transaction +DEBUG: pushing down the function call +ERROR: queries must filter by the distribution argument in the same colocation group when using the forced function pushdown +HINT: consider turning off the flag force_pushdown instead +CONTEXT: SQL statement "SELECT data FROM forcepushdown_schema.test_subquery WHERE data = + (SELECT id FROM forcepushdown_schema.test_non_colocated WHERE id = a)" +PL/pgSQL function forcepushdown_schema.select_data_noncolocate(integer) line XX at SQL statement +while executing command on localhost:xxxxx +END; +-- This should pass +SELECT insert_select_data_cte1(200); +DEBUG: pushing down the function call +NOTICE: Result: 200 +DETAIL: from localhost:xxxxx + insert_select_data_cte1 +--------------------------------------------------------------------- + +(1 row) + +BEGIN; +SELECT insert_select_data_cte1(200); +DEBUG: pushing down function call in a multi-statement transaction +DEBUG: pushing down the function call +NOTICE: Result: 200 +DETAIL: from localhost:xxxxx + insert_select_data_cte1 +--------------------------------------------------------------------- + +(1 row) + +COMMIT; +-- This should pass +SELECT insert_select_data_cte2(300); +DEBUG: pushing down the function call +NOTICE: Result: +DETAIL: from localhost:xxxxx + insert_select_data_cte2 +--------------------------------------------------------------------- + +(1 row) + +BEGIN; +SELECT insert_select_data_cte2(300); +DEBUG: pushing down function call in a multi-statement transaction +DEBUG: pushing down the function call +NOTICE: Result: 300 +DETAIL: from localhost:xxxxx + insert_select_data_cte2 +--------------------------------------------------------------------- + +(1 row) + +COMMIT; +-- This should pass +SELECT insert_data_cte_nondist(400); +DEBUG: pushing down the function call +ERROR: queries must filter by the distribution argument in the same colocation group when using the forced function pushdown +HINT: consider turning off the flag force_pushdown instead +CONTEXT: SQL statement "WITH ins AS (INSERT INTO forcepushdown_schema.test_subquery VALUES (a+1) RETURNING data) + SELECT ins.data FROM forcepushdown_schema.test_subquery, ins WHERE forcepushdown_schema.test_subquery.data = a" +PL/pgSQL function forcepushdown_schema.insert_data_cte_nondist(integer) line XX at SQL statement +while executing command on localhost:xxxxx +BEGIN; +SELECT insert_data_cte_nondist(400); +DEBUG: pushing down function call in a multi-statement transaction +DEBUG: pushing down the function call +ERROR: queries must filter by the distribution argument in the same colocation group when using the forced function pushdown +HINT: consider turning off the flag force_pushdown instead +CONTEXT: SQL statement "WITH ins AS (INSERT INTO forcepushdown_schema.test_subquery VALUES (a+1) RETURNING data) + SELECT ins.data FROM forcepushdown_schema.test_subquery, ins WHERE forcepushdown_schema.test_subquery.data = a" +PL/pgSQL function forcepushdown_schema.insert_data_cte_nondist(integer) line XX at SQL statement +while executing command on localhost:xxxxx +COMMIT; +-- Rows 100, 200, 300 should be seen +SELECT * FROM forcepushdown_schema.test_subquery ORDER BY 1; + data | result +--------------------------------------------------------------------- + 100 | -1 + 200 | + 200 | + 300 | + 300 | +(5 rows) + +RESET client_min_messages; +SET citus.log_remote_commands TO off; +DROP SCHEMA forcepushdown_schema CASCADE; +NOTICE: drop cascades to 32 other objects +DETAIL: drop cascades to table test_forcepushdown +drop cascades to table test_forcepushdown_noncolocate +drop cascades to function insert_data(integer) +drop cascades to function insert_data_non_distarg(integer) +drop cascades to function update_data_nonlocal(integer) +drop cascades to function insert_data_noncolocation(integer) +drop cascades to table test_nested +drop cascades to function inner_force_pushdown_function(integer) +drop cascades to function func_calls_forcepush_func() +drop cascades to function get_val() +drop cascades to function func_calls_forcepush_func_infrom() +drop cascades to function func_calls_forcepush_func_intarget() +drop cascades to function test_recursive(integer) +drop cascades to table emp +drop cascades to table emp_audit +drop cascades to function inner_emp(text) +drop cascades to function outer_emp() +drop cascades to function insert_select_data(integer) +drop cascades to function insert_select_data_nonlocal(integer) +drop cascades to table test_forcepushdown_char +drop cascades to table test_forcepushdown_varchar +drop cascades to table test_forcepushdown_text +drop cascades to function insert_data_char(character) +drop cascades to function insert_data_varchar(character varying) +drop cascades to function insert_data_text(text) +drop cascades to table test_subquery +drop cascades to table test_non_colocated +drop cascades to function select_data(integer) +drop cascades to function select_data_noncolocate(integer) +drop cascades to function insert_select_data_cte1(integer) +drop cascades to function insert_select_data_cte2(integer) +drop cascades to function insert_data_cte_nondist(integer) diff --git a/src/test/regress/expected/foreign_tables_mx.out b/src/test/regress/expected/foreign_tables_mx.out index a3de72259..17b1c99b8 100644 --- a/src/test/regress/expected/foreign_tables_mx.out +++ b/src/test/regress/expected/foreign_tables_mx.out @@ -66,16 +66,10 @@ ALTER FOREIGN TABLE public.foreign_table_newname ADD CONSTRAINT check_c_2 check( ALTER FOREIGN TABLE public.foreign_table_newname VALIDATE CONSTRAINT check_c_2; ALTER FOREIGN TABLE public.foreign_table_newname DROP constraint IF EXISTS check_c_2; -- trigger test -CREATE TABLE distributed_table(value int); -SELECT create_distributed_table('distributed_table', 'value'); - create_distributed_table ---------------------------------------------------------------------- - -(1 row) - +CREATE TABLE table42(value int); CREATE FUNCTION insert_42() RETURNS trigger AS $insert_42$ BEGIN - INSERT INTO distributed_table VALUES (42); + INSERT INTO table42 VALUES (42); RETURN NEW; END; $insert_42$ LANGUAGE plpgsql; @@ -85,7 +79,7 @@ FOR EACH ROW EXECUTE FUNCTION insert_42(); -- do the same pattern from the workers as well INSERT INTO public.foreign_table_newname VALUES (99, 'test_2'); delete from public.foreign_table_newname where id_test = 99; -select * from distributed_table ORDER BY value; +select * from table42 ORDER BY value; value --------------------------------------------------------------------- 42 @@ -96,7 +90,7 @@ alter foreign table public.foreign_table_newname disable trigger insert_42_trigg INSERT INTO public.foreign_table_newname VALUES (99, 'test_2'); delete from public.foreign_table_newname where id_test = 99; -- should not insert again as trigger disabled -select * from distributed_table ORDER BY value; +select * from table42 ORDER BY value; value --------------------------------------------------------------------- 42 diff --git a/src/test/regress/expected/metadata_sync_helpers.out b/src/test/regress/expected/metadata_sync_helpers.out index 61584503b..0d9c183f8 100644 --- a/src/test/regress/expected/metadata_sync_helpers.out +++ b/src/test/regress/expected/metadata_sync_helpers.out @@ -556,9 +556,9 @@ BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED; SET application_name to 'citus'; \set VERBOSITY terse - WITH distributed_object_data(typetext, objnames, objargs, distargumentindex, colocationid) - AS (VALUES ('non_existing_type', ARRAY['non_existing_user']::text[], ARRAY[]::text[], -1, 0)) - SELECT citus_internal_add_object_metadata(typetext, objnames, objargs, distargumentindex, colocationid) FROM distributed_object_data; + WITH distributed_object_data(typetext, objnames, objargs, distargumentindex, colocationid, force_pushdown) + AS (VALUES ('non_existing_type', ARRAY['non_existing_user']::text[], ARRAY[]::text[], -1, 0, false)) + SELECT citus_internal_add_object_metadata(typetext, objnames, objargs, distargumentindex, colocationid, force_pushdown) FROM distributed_object_data; ERROR: unrecognized object type "non_existing_type" ROLLBACK; -- check the sanity of distributionArgumentIndex and colocationId @@ -571,9 +571,9 @@ BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED; SET application_name to 'citus'; \set VERBOSITY terse - WITH distributed_object_data(typetext, objnames, objargs, distargumentindex, colocationid) - AS (VALUES ('role', ARRAY['metadata_sync_helper_role']::text[], ARRAY[]::text[], -100, 0)) - SELECT citus_internal_add_object_metadata(typetext, objnames, objargs, distargumentindex, colocationid) FROM distributed_object_data; + WITH distributed_object_data(typetext, objnames, objargs, distargumentindex, colocationid, force_pushdown) + AS (VALUES ('role', ARRAY['metadata_sync_helper_role']::text[], ARRAY[]::text[], -100, 0, false)) + SELECT citus_internal_add_object_metadata(typetext, objnames, objargs, distargumentindex, colocationid, force_pushdown) FROM distributed_object_data; ERROR: distribution_argument_index must be between 0 and 100 ROLLBACK; BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED; @@ -585,9 +585,9 @@ BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED; SET application_name to 'citus'; \set VERBOSITY terse - WITH distributed_object_data(typetext, objnames, objargs, distargumentindex, colocationid) - AS (VALUES ('role', ARRAY['metadata_sync_helper_role']::text[], ARRAY[]::text[], -1, -1)) - SELECT citus_internal_add_object_metadata(typetext, objnames, objargs, distargumentindex, colocationid) FROM distributed_object_data; + WITH distributed_object_data(typetext, objnames, objargs, distargumentindex, colocationid, force_pushdown) + AS (VALUES ('role', ARRAY['metadata_sync_helper_role']::text[], ARRAY[]::text[], -1, -1, false)) + SELECT citus_internal_add_object_metadata(typetext, objnames, objargs, distargumentindex, colocationid, force_pushdown) FROM distributed_object_data; ERROR: colocationId must be a positive number ROLLBACK; -- check with non-existing object @@ -600,9 +600,9 @@ BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED; SET application_name to 'citus'; \set VERBOSITY terse - WITH distributed_object_data(typetext, objnames, objargs, distargumentindex, colocationid) - AS (VALUES ('role', ARRAY['non_existing_user']::text[], ARRAY[]::text[], -1, 0)) - SELECT citus_internal_add_object_metadata(typetext, objnames, objargs, distargumentindex, colocationid) FROM distributed_object_data; + WITH distributed_object_data(typetext, objnames, objargs, distargumentindex, colocationid, force_pushdown) + AS (VALUES ('role', ARRAY['non_existing_user']::text[], ARRAY[]::text[], -1, 0, false)) + SELECT citus_internal_add_object_metadata(typetext, objnames, objargs, distargumentindex, colocationid, force_pushdown) FROM distributed_object_data; ERROR: role "non_existing_user" does not exist ROLLBACK; -- since citus_internal_add_object_metadata is strict function returns NULL @@ -616,9 +616,9 @@ BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED; SET application_name to 'citus'; \set VERBOSITY terse - WITH distributed_object_data(typetext, objnames, objargs, distargumentindex, colocationid) - AS (VALUES ('role', ARRAY['metadata_sync_helper_role']::text[], ARRAY[]::text[], 0, NULL::int)) - SELECT citus_internal_add_object_metadata(typetext, objnames, objargs, distargumentindex, colocationid) FROM distributed_object_data; + WITH distributed_object_data(typetext, objnames, objargs, distargumentindex, colocationid, force_pushdown) + AS (VALUES ('role', ARRAY['metadata_sync_helper_role']::text[], ARRAY[]::text[], 0, NULL::int, false)) + SELECT citus_internal_add_object_metadata(typetext, objnames, objargs, distargumentindex, colocationid, force_pushdown) FROM distributed_object_data; citus_internal_add_object_metadata --------------------------------------------------------------------- @@ -640,9 +640,9 @@ BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED; CREATE TABLE publication_test_table(id int); CREATE PUBLICATION publication_test FOR TABLE publication_test_table; SET ROLE metadata_sync_helper_role; - WITH distributed_object_data(typetext, objnames, objargs, distargumentindex, colocationid) - AS (VALUES ('publication', ARRAY['publication_test']::text[], ARRAY[]::text[], -1, 0)) - SELECT citus_internal_add_object_metadata(typetext, objnames, objargs, distargumentindex, colocationid) FROM distributed_object_data; + WITH distributed_object_data(typetext, objnames, objargs, distargumentindex, colocationid, force_pushdown) + AS (VALUES ('publication', ARRAY['publication_test']::text[], ARRAY[]::text[], -1, 0, false)) + SELECT citus_internal_add_object_metadata(typetext, objnames, objargs, distargumentindex, colocationid, force_pushdown) FROM distributed_object_data; ERROR: Object type 29 can not be distributed by Citus ROLLBACK; -- Show that citus_internal_add_object_metadata checks the priviliges @@ -659,9 +659,9 @@ BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED; AS $$ SELECT $1 $$ LANGUAGE SQL; SET ROLE metadata_sync_helper_role; - WITH distributed_object_data(typetext, objnames, objargs, distargumentindex, colocationid) - AS (VALUES ('function', ARRAY['distribution_test_function']::text[], ARRAY['integer']::text[], -1, 0)) - SELECT citus_internal_add_object_metadata(typetext, objnames, objargs, distargumentindex, colocationid) FROM distributed_object_data; + WITH distributed_object_data(typetext, objnames, objargs, distargumentindex, colocationid, force_pushdown) + AS (VALUES ('function', ARRAY['distribution_test_function']::text[], ARRAY['integer']::text[], -1, 0, false)) + SELECT citus_internal_add_object_metadata(typetext, objnames, objargs, distargumentindex, colocationid, force_pushdown) FROM distributed_object_data; ERROR: must be owner of function distribution_test_function ROLLBACK; -- we do not allow wrong partmethod diff --git a/src/test/regress/expected/multi_extension.out b/src/test/regress/expected/multi_extension.out index 701b5e26f..6aa2d3b29 100644 --- a/src/test/regress/expected/multi_extension.out +++ b/src/test/regress/expected/multi_extension.out @@ -997,16 +997,18 @@ SELECT * FROM multi_extension.print_extension_changes(); previous_object | current_object --------------------------------------------------------------------- function citus_disable_node(text,integer) void | + function create_distributed_function(regprocedure,text,text) void | function master_append_table_to_shard(bigint,text,text,integer) real | function master_apply_delete_command(text) integer | function master_get_table_metadata(text) record | | function citus_check_cluster_node_health() SETOF record | function citus_check_connection_to_node(text,integer) boolean | function citus_disable_node(text,integer,boolean) void - | function citus_internal_add_object_metadata(text,text[],text[],integer,integer) void + | function citus_internal_add_object_metadata(text,text[],text[],integer,integer,boolean) void | function citus_run_local_command(text) void + | function create_distributed_function(regprocedure,text,text,boolean) void | function worker_drop_sequence_dependency(text) void -(10 rows) +(12 rows) DROP TABLE multi_extension.prev_objects, multi_extension.extension_diff; -- show running version diff --git a/src/test/regress/expected/multi_metadata_attributes.out b/src/test/regress/expected/multi_metadata_attributes.out index 67389f51a..bf75e23ef 100644 --- a/src/test/regress/expected/multi_metadata_attributes.out +++ b/src/test/regress/expected/multi_metadata_attributes.out @@ -8,7 +8,8 @@ SELECT attrelid::regclass, attname, atthasmissing, attmissingval FROM pg_attribute WHERE atthasmissing AND attrelid NOT IN ('pg_dist_node'::regclass, 'pg_dist_rebalance_strategy'::regclass, - 'pg_dist_partition'::regclass) + 'pg_dist_partition'::regclass, + 'citus.pg_dist_object'::regclass) ORDER BY attrelid, attname; attrelid | attname | atthasmissing | attmissingval --------------------------------------------------------------------- diff --git a/src/test/regress/expected/multi_metadata_sync.out b/src/test/regress/expected/multi_metadata_sync.out index f1cedaf1c..1936440c5 100644 --- a/src/test/regress/expected/multi_metadata_sync.out +++ b/src/test/regress/expected/multi_metadata_sync.out @@ -48,7 +48,7 @@ SELECT unnest(master_metadata_snapshot()) order by 1; SELECT worker_drop_distributed_table(logicalrelid::regclass::text) FROM pg_dist_partition TRUNCATE citus.pg_dist_object TRUNCATE pg_dist_node CASCADE - WITH distributed_object_data(typetext, objnames, objargs, distargumentindex, colocationid) AS (VALUES ('role', ARRAY['postgres']::text[], ARRAY[]::text[], -1, 0), ('database', ARRAY['regression']::text[], ARRAY[]::text[], -1, 0), ('schema', ARRAY['public']::text[], ARRAY[]::text[], -1, 0)) SELECT citus_internal_add_object_metadata(typetext, objnames, objargs, distargumentindex::int, colocationid::int) FROM distributed_object_data; + WITH distributed_object_data(typetext, objnames, objargs, distargumentindex, colocationid, force_pushdown) AS (VALUES ('role', ARRAY['postgres']::text[], ARRAY[]::text[], -1, 0, false), ('database', ARRAY['regression']::text[], ARRAY[]::text[], -1, 0, false), ('schema', ARRAY['public']::text[], ARRAY[]::text[], -1, 0, false)) SELECT citus_internal_add_object_metadata(typetext, objnames, objargs, distargumentindex::int, colocationid::int, force_pushdown::bool) FROM distributed_object_data; (6 rows) -- this function is dropped in Citus10, added here for tests @@ -106,7 +106,7 @@ SELECT unnest(master_metadata_snapshot()) order by 1; SELECT worker_drop_distributed_table(logicalrelid::regclass::text) FROM pg_dist_partition TRUNCATE citus.pg_dist_object TRUNCATE pg_dist_node CASCADE - WITH distributed_object_data(typetext, objnames, objargs, distargumentindex, colocationid) AS (VALUES ('sequence', ARRAY['public', 'user_defined_seq']::text[], ARRAY[]::text[], -1, 0), ('sequence', ARRAY['public', 'mx_test_table_col_3_seq']::text[], ARRAY[]::text[], -1, 0), ('role', ARRAY['postgres']::text[], ARRAY[]::text[], -1, 0), ('database', ARRAY['regression']::text[], ARRAY[]::text[], -1, 0), ('schema', ARRAY['public']::text[], ARRAY[]::text[], -1, 0)) SELECT citus_internal_add_object_metadata(typetext, objnames, objargs, distargumentindex::int, colocationid::int) FROM distributed_object_data; + WITH distributed_object_data(typetext, objnames, objargs, distargumentindex, colocationid, force_pushdown) AS (VALUES ('sequence', ARRAY['public', 'user_defined_seq']::text[], ARRAY[]::text[], -1, 0, false), ('sequence', ARRAY['public', 'mx_test_table_col_3_seq']::text[], ARRAY[]::text[], -1, 0, false), ('role', ARRAY['postgres']::text[], ARRAY[]::text[], -1, 0, false), ('database', ARRAY['regression']::text[], ARRAY[]::text[], -1, 0, false), ('schema', ARRAY['public']::text[], ARRAY[]::text[], -1, 0, false)) SELECT citus_internal_add_object_metadata(typetext, objnames, objargs, distargumentindex::int, colocationid::int, force_pushdown::bool) FROM distributed_object_data; WITH placement_data(shardid, shardstate, shardlength, groupid, placementid) AS (VALUES (1310000, 1, 0, 1, 100000), (1310001, 1, 0, 2, 100001), (1310002, 1, 0, 1, 100002), (1310003, 1, 0, 2, 100003), (1310004, 1, 0, 1, 100004), (1310005, 1, 0, 2, 100005), (1310006, 1, 0, 1, 100006), (1310007, 1, 0, 2, 100007)) SELECT citus_internal_add_placement_metadata(shardid, shardstate, shardlength, groupid, placementid) FROM placement_data; WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) AS (VALUES ('public.mx_test_table'::regclass, 1310000, 't'::"char", '-2147483648', '-1610612737'), ('public.mx_test_table'::regclass, 1310001, 't'::"char", '-1610612736', '-1073741825'), ('public.mx_test_table'::regclass, 1310002, 't'::"char", '-1073741824', '-536870913'), ('public.mx_test_table'::regclass, 1310003, 't'::"char", '-536870912', '-1'), ('public.mx_test_table'::regclass, 1310004, 't'::"char", '0', '536870911'), ('public.mx_test_table'::regclass, 1310005, 't'::"char", '536870912', '1073741823'), ('public.mx_test_table'::regclass, 1310006, 't'::"char", '1073741824', '1610612735'), ('public.mx_test_table'::regclass, 1310007, 't'::"char", '1610612736', '2147483647')) SELECT citus_internal_add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data; (19 rows) @@ -133,7 +133,7 @@ SELECT unnest(master_metadata_snapshot()) order by 1; SELECT worker_drop_distributed_table(logicalrelid::regclass::text) FROM pg_dist_partition TRUNCATE citus.pg_dist_object TRUNCATE pg_dist_node CASCADE - WITH distributed_object_data(typetext, objnames, objargs, distargumentindex, colocationid) AS (VALUES ('sequence', ARRAY['public', 'user_defined_seq']::text[], ARRAY[]::text[], -1, 0), ('sequence', ARRAY['public', 'mx_test_table_col_3_seq']::text[], ARRAY[]::text[], -1, 0), ('role', ARRAY['postgres']::text[], ARRAY[]::text[], -1, 0), ('database', ARRAY['regression']::text[], ARRAY[]::text[], -1, 0), ('schema', ARRAY['public']::text[], ARRAY[]::text[], -1, 0)) SELECT citus_internal_add_object_metadata(typetext, objnames, objargs, distargumentindex::int, colocationid::int) FROM distributed_object_data; + WITH distributed_object_data(typetext, objnames, objargs, distargumentindex, colocationid, force_pushdown) AS (VALUES ('sequence', ARRAY['public', 'user_defined_seq']::text[], ARRAY[]::text[], -1, 0, false), ('sequence', ARRAY['public', 'mx_test_table_col_3_seq']::text[], ARRAY[]::text[], -1, 0, false), ('role', ARRAY['postgres']::text[], ARRAY[]::text[], -1, 0, false), ('database', ARRAY['regression']::text[], ARRAY[]::text[], -1, 0, false), ('schema', ARRAY['public']::text[], ARRAY[]::text[], -1, 0, false)) SELECT citus_internal_add_object_metadata(typetext, objnames, objargs, distargumentindex::int, colocationid::int, force_pushdown::bool) FROM distributed_object_data; WITH placement_data(shardid, shardstate, shardlength, groupid, placementid) AS (VALUES (1310000, 1, 0, 1, 100000), (1310001, 1, 0, 2, 100001), (1310002, 1, 0, 1, 100002), (1310003, 1, 0, 2, 100003), (1310004, 1, 0, 1, 100004), (1310005, 1, 0, 2, 100005), (1310006, 1, 0, 1, 100006), (1310007, 1, 0, 2, 100007)) SELECT citus_internal_add_placement_metadata(shardid, shardstate, shardlength, groupid, placementid) FROM placement_data; WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) AS (VALUES ('public.mx_test_table'::regclass, 1310000, 't'::"char", '-2147483648', '-1610612737'), ('public.mx_test_table'::regclass, 1310001, 't'::"char", '-1610612736', '-1073741825'), ('public.mx_test_table'::regclass, 1310002, 't'::"char", '-1073741824', '-536870913'), ('public.mx_test_table'::regclass, 1310003, 't'::"char", '-536870912', '-1'), ('public.mx_test_table'::regclass, 1310004, 't'::"char", '0', '536870911'), ('public.mx_test_table'::regclass, 1310005, 't'::"char", '536870912', '1073741823'), ('public.mx_test_table'::regclass, 1310006, 't'::"char", '1073741824', '1610612735'), ('public.mx_test_table'::regclass, 1310007, 't'::"char", '1610612736', '2147483647')) SELECT citus_internal_add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data; (20 rows) @@ -161,7 +161,7 @@ SELECT unnest(master_metadata_snapshot()) order by 1; SELECT worker_drop_distributed_table(logicalrelid::regclass::text) FROM pg_dist_partition TRUNCATE citus.pg_dist_object TRUNCATE pg_dist_node CASCADE - WITH distributed_object_data(typetext, objnames, objargs, distargumentindex, colocationid) AS (VALUES ('sequence', ARRAY['public', 'user_defined_seq']::text[], ARRAY[]::text[], -1, 0), ('sequence', ARRAY['mx_testing_schema', 'mx_test_table_col_3_seq']::text[], ARRAY[]::text[], -1, 0), ('role', ARRAY['postgres']::text[], ARRAY[]::text[], -1, 0), ('database', ARRAY['regression']::text[], ARRAY[]::text[], -1, 0), ('schema', ARRAY['public']::text[], ARRAY[]::text[], -1, 0), ('schema', ARRAY['mx_testing_schema']::text[], ARRAY[]::text[], -1, 0)) SELECT citus_internal_add_object_metadata(typetext, objnames, objargs, distargumentindex::int, colocationid::int) FROM distributed_object_data; + WITH distributed_object_data(typetext, objnames, objargs, distargumentindex, colocationid, force_pushdown) AS (VALUES ('sequence', ARRAY['public', 'user_defined_seq']::text[], ARRAY[]::text[], -1, 0, false), ('sequence', ARRAY['mx_testing_schema', 'mx_test_table_col_3_seq']::text[], ARRAY[]::text[], -1, 0, false), ('role', ARRAY['postgres']::text[], ARRAY[]::text[], -1, 0, false), ('database', ARRAY['regression']::text[], ARRAY[]::text[], -1, 0, false), ('schema', ARRAY['public']::text[], ARRAY[]::text[], -1, 0, false), ('schema', ARRAY['mx_testing_schema']::text[], ARRAY[]::text[], -1, 0, false)) SELECT citus_internal_add_object_metadata(typetext, objnames, objargs, distargumentindex::int, colocationid::int, force_pushdown::bool) FROM distributed_object_data; WITH placement_data(shardid, shardstate, shardlength, groupid, placementid) AS (VALUES (1310000, 1, 0, 1, 100000), (1310001, 1, 0, 2, 100001), (1310002, 1, 0, 1, 100002), (1310003, 1, 0, 2, 100003), (1310004, 1, 0, 1, 100004), (1310005, 1, 0, 2, 100005), (1310006, 1, 0, 1, 100006), (1310007, 1, 0, 2, 100007)) SELECT citus_internal_add_placement_metadata(shardid, shardstate, shardlength, groupid, placementid) FROM placement_data; WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) AS (VALUES ('mx_testing_schema.mx_test_table'::regclass, 1310000, 't'::"char", '-2147483648', '-1610612737'), ('mx_testing_schema.mx_test_table'::regclass, 1310001, 't'::"char", '-1610612736', '-1073741825'), ('mx_testing_schema.mx_test_table'::regclass, 1310002, 't'::"char", '-1073741824', '-536870913'), ('mx_testing_schema.mx_test_table'::regclass, 1310003, 't'::"char", '-536870912', '-1'), ('mx_testing_schema.mx_test_table'::regclass, 1310004, 't'::"char", '0', '536870911'), ('mx_testing_schema.mx_test_table'::regclass, 1310005, 't'::"char", '536870912', '1073741823'), ('mx_testing_schema.mx_test_table'::regclass, 1310006, 't'::"char", '1073741824', '1610612735'), ('mx_testing_schema.mx_test_table'::regclass, 1310007, 't'::"char", '1610612736', '2147483647')) SELECT citus_internal_add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data; (20 rows) @@ -195,7 +195,7 @@ SELECT unnest(master_metadata_snapshot()) order by 1; SELECT worker_drop_distributed_table(logicalrelid::regclass::text) FROM pg_dist_partition TRUNCATE citus.pg_dist_object TRUNCATE pg_dist_node CASCADE - WITH distributed_object_data(typetext, objnames, objargs, distargumentindex, colocationid) AS (VALUES ('sequence', ARRAY['public', 'user_defined_seq']::text[], ARRAY[]::text[], -1, 0), ('sequence', ARRAY['mx_testing_schema', 'mx_test_table_col_3_seq']::text[], ARRAY[]::text[], -1, 0), ('role', ARRAY['postgres']::text[], ARRAY[]::text[], -1, 0), ('database', ARRAY['regression']::text[], ARRAY[]::text[], -1, 0), ('schema', ARRAY['public']::text[], ARRAY[]::text[], -1, 0), ('schema', ARRAY['mx_testing_schema']::text[], ARRAY[]::text[], -1, 0)) SELECT citus_internal_add_object_metadata(typetext, objnames, objargs, distargumentindex::int, colocationid::int) FROM distributed_object_data; + WITH distributed_object_data(typetext, objnames, objargs, distargumentindex, colocationid, force_pushdown) AS (VALUES ('sequence', ARRAY['public', 'user_defined_seq']::text[], ARRAY[]::text[], -1, 0, false), ('sequence', ARRAY['mx_testing_schema', 'mx_test_table_col_3_seq']::text[], ARRAY[]::text[], -1, 0, false), ('role', ARRAY['postgres']::text[], ARRAY[]::text[], -1, 0, false), ('database', ARRAY['regression']::text[], ARRAY[]::text[], -1, 0, false), ('schema', ARRAY['public']::text[], ARRAY[]::text[], -1, 0, false), ('schema', ARRAY['mx_testing_schema']::text[], ARRAY[]::text[], -1, 0, false)) SELECT citus_internal_add_object_metadata(typetext, objnames, objargs, distargumentindex::int, colocationid::int, force_pushdown::bool) FROM distributed_object_data; WITH placement_data(shardid, shardstate, shardlength, groupid, placementid) AS (VALUES (1310000, 1, 0, 1, 100000), (1310001, 1, 0, 2, 100001), (1310002, 1, 0, 1, 100002), (1310003, 1, 0, 2, 100003), (1310004, 1, 0, 1, 100004), (1310005, 1, 0, 2, 100005), (1310006, 1, 0, 1, 100006), (1310007, 1, 0, 2, 100007)) SELECT citus_internal_add_placement_metadata(shardid, shardstate, shardlength, groupid, placementid) FROM placement_data; WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) AS (VALUES ('mx_testing_schema.mx_test_table'::regclass, 1310000, 't'::"char", '-2147483648', '-1610612737'), ('mx_testing_schema.mx_test_table'::regclass, 1310001, 't'::"char", '-1610612736', '-1073741825'), ('mx_testing_schema.mx_test_table'::regclass, 1310002, 't'::"char", '-1073741824', '-536870913'), ('mx_testing_schema.mx_test_table'::regclass, 1310003, 't'::"char", '-536870912', '-1'), ('mx_testing_schema.mx_test_table'::regclass, 1310004, 't'::"char", '0', '536870911'), ('mx_testing_schema.mx_test_table'::regclass, 1310005, 't'::"char", '536870912', '1073741823'), ('mx_testing_schema.mx_test_table'::regclass, 1310006, 't'::"char", '1073741824', '1610612735'), ('mx_testing_schema.mx_test_table'::regclass, 1310007, 't'::"char", '1610612736', '2147483647')) SELECT citus_internal_add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data; (20 rows) @@ -222,7 +222,7 @@ SELECT unnest(master_metadata_snapshot()) order by 1; SELECT worker_drop_distributed_table(logicalrelid::regclass::text) FROM pg_dist_partition TRUNCATE citus.pg_dist_object TRUNCATE pg_dist_node CASCADE - WITH distributed_object_data(typetext, objnames, objargs, distargumentindex, colocationid) AS (VALUES ('sequence', ARRAY['public', 'user_defined_seq']::text[], ARRAY[]::text[], -1, 0), ('sequence', ARRAY['mx_testing_schema', 'mx_test_table_col_3_seq']::text[], ARRAY[]::text[], -1, 0), ('role', ARRAY['postgres']::text[], ARRAY[]::text[], -1, 0), ('database', ARRAY['regression']::text[], ARRAY[]::text[], -1, 0), ('schema', ARRAY['public']::text[], ARRAY[]::text[], -1, 0), ('schema', ARRAY['mx_testing_schema']::text[], ARRAY[]::text[], -1, 0)) SELECT citus_internal_add_object_metadata(typetext, objnames, objargs, distargumentindex::int, colocationid::int) FROM distributed_object_data; + WITH distributed_object_data(typetext, objnames, objargs, distargumentindex, colocationid, force_pushdown) AS (VALUES ('sequence', ARRAY['public', 'user_defined_seq']::text[], ARRAY[]::text[], -1, 0, false), ('sequence', ARRAY['mx_testing_schema', 'mx_test_table_col_3_seq']::text[], ARRAY[]::text[], -1, 0, false), ('role', ARRAY['postgres']::text[], ARRAY[]::text[], -1, 0, false), ('database', ARRAY['regression']::text[], ARRAY[]::text[], -1, 0, false), ('schema', ARRAY['public']::text[], ARRAY[]::text[], -1, 0, false), ('schema', ARRAY['mx_testing_schema']::text[], ARRAY[]::text[], -1, 0, false)) SELECT citus_internal_add_object_metadata(typetext, objnames, objargs, distargumentindex::int, colocationid::int, force_pushdown::bool) FROM distributed_object_data; WITH placement_data(shardid, shardstate, shardlength, groupid, placementid) AS (VALUES (1310000, 1, 0, 1, 100000), (1310001, 1, 0, 2, 100001), (1310002, 1, 0, 1, 100002), (1310003, 1, 0, 2, 100003), (1310004, 1, 0, 1, 100004), (1310005, 1, 0, 2, 100005), (1310006, 1, 0, 1, 100006), (1310007, 1, 0, 2, 100007)) SELECT citus_internal_add_placement_metadata(shardid, shardstate, shardlength, groupid, placementid) FROM placement_data; WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) AS (VALUES ('mx_testing_schema.mx_test_table'::regclass, 1310000, 't'::"char", '-2147483648', '-1610612737'), ('mx_testing_schema.mx_test_table'::regclass, 1310001, 't'::"char", '-1610612736', '-1073741825'), ('mx_testing_schema.mx_test_table'::regclass, 1310002, 't'::"char", '-1073741824', '-536870913'), ('mx_testing_schema.mx_test_table'::regclass, 1310003, 't'::"char", '-536870912', '-1'), ('mx_testing_schema.mx_test_table'::regclass, 1310004, 't'::"char", '0', '536870911'), ('mx_testing_schema.mx_test_table'::regclass, 1310005, 't'::"char", '536870912', '1073741823'), ('mx_testing_schema.mx_test_table'::regclass, 1310006, 't'::"char", '1073741824', '1610612735'), ('mx_testing_schema.mx_test_table'::regclass, 1310007, 't'::"char", '1610612736', '2147483647')) SELECT citus_internal_add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data; (20 rows) @@ -1752,7 +1752,7 @@ SELECT unnest(master_metadata_snapshot()) order by 1; SELECT worker_drop_distributed_table(logicalrelid::regclass::text) FROM pg_dist_partition TRUNCATE citus.pg_dist_object TRUNCATE pg_dist_node CASCADE - WITH distributed_object_data(typetext, objnames, objargs, distargumentindex, colocationid) AS (VALUES ('sequence', ARRAY['public', 'user_defined_seq']::text[], ARRAY[]::text[], -1, 0), ('sequence', ARRAY['mx_testing_schema', 'mx_test_table_col_3_seq']::text[], ARRAY[]::text[], -1, 0), ('sequence', ARRAY['public', 'mx_test_sequence_0']::text[], ARRAY[]::text[], -1, 0), ('sequence', ARRAY['public', 'mx_test_sequence_1']::text[], ARRAY[]::text[], -1, 0), ('role', ARRAY['postgres']::text[], ARRAY[]::text[], -1, 0), ('database', ARRAY['regression']::text[], ARRAY[]::text[], -1, 0), ('schema', ARRAY['public']::text[], ARRAY[]::text[], -1, 0), ('schema', ARRAY['mx_testing_schema']::text[], ARRAY[]::text[], -1, 0), ('schema', ARRAY['mx_testing_schema_2']::text[], ARRAY[]::text[], -1, 0), ('schema', ARRAY['mx_test_schema_1']::text[], ARRAY[]::text[], -1, 0), ('schema', ARRAY['mx_test_schema_2']::text[], ARRAY[]::text[], -1, 0)) SELECT citus_internal_add_object_metadata(typetext, objnames, objargs, distargumentindex::int, colocationid::int) FROM distributed_object_data; + WITH distributed_object_data(typetext, objnames, objargs, distargumentindex, colocationid, force_pushdown) AS (VALUES ('sequence', ARRAY['public', 'user_defined_seq']::text[], ARRAY[]::text[], -1, 0, false), ('sequence', ARRAY['mx_testing_schema', 'mx_test_table_col_3_seq']::text[], ARRAY[]::text[], -1, 0, false), ('sequence', ARRAY['public', 'mx_test_sequence_0']::text[], ARRAY[]::text[], -1, 0, false), ('sequence', ARRAY['public', 'mx_test_sequence_1']::text[], ARRAY[]::text[], -1, 0, false), ('role', ARRAY['postgres']::text[], ARRAY[]::text[], -1, 0, false), ('database', ARRAY['regression']::text[], ARRAY[]::text[], -1, 0, false), ('schema', ARRAY['public']::text[], ARRAY[]::text[], -1, 0, false), ('schema', ARRAY['mx_testing_schema']::text[], ARRAY[]::text[], -1, 0, false), ('schema', ARRAY['mx_testing_schema_2']::text[], ARRAY[]::text[], -1, 0, false), ('schema', ARRAY['mx_test_schema_1']::text[], ARRAY[]::text[], -1, 0, false), ('schema', ARRAY['mx_test_schema_2']::text[], ARRAY[]::text[], -1, 0, false)) SELECT citus_internal_add_object_metadata(typetext, objnames, objargs, distargumentindex::int, colocationid::int, force_pushdown::bool) FROM distributed_object_data; WITH placement_data(shardid, shardstate, shardlength, groupid, placementid) AS (VALUES (1310000, 1, 0, 1, 100000), (1310001, 1, 0, 5, 100001), (1310002, 1, 0, 1, 100002), (1310003, 1, 0, 5, 100003), (1310004, 1, 0, 1, 100004), (1310005, 1, 0, 5, 100005), (1310006, 1, 0, 1, 100006), (1310007, 1, 0, 5, 100007)) SELECT citus_internal_add_placement_metadata(shardid, shardstate, shardlength, groupid, placementid) FROM placement_data; WITH placement_data(shardid, shardstate, shardlength, groupid, placementid) AS (VALUES (1310020, 1, 0, 1, 100020), (1310021, 1, 0, 5, 100021), (1310022, 1, 0, 1, 100022), (1310023, 1, 0, 5, 100023), (1310024, 1, 0, 1, 100024)) SELECT citus_internal_add_placement_metadata(shardid, shardstate, shardlength, groupid, placementid) FROM placement_data; WITH placement_data(shardid, shardstate, shardlength, groupid, placementid) AS (VALUES (1310025, 1, 0, 1, 100025), (1310026, 1, 0, 5, 100026), (1310027, 1, 0, 1, 100027), (1310028, 1, 0, 5, 100028), (1310029, 1, 0, 1, 100029)) SELECT citus_internal_add_placement_metadata(shardid, shardstate, shardlength, groupid, placementid) FROM placement_data; diff --git a/src/test/regress/expected/multi_mx_call.out b/src/test/regress/expected/multi_mx_call.out index 7f077c77a..64b033d41 100644 --- a/src/test/regress/expected/multi_mx_call.out +++ b/src/test/regress/expected/multi_mx_call.out @@ -131,6 +131,12 @@ BEGIN y := x; x := (select case groupid when 0 then 'F' else 'S' end from pg_dist_local_group); END;$$; +CREATE PROCEDURE mx_call_proc_copy(x int) +LANGUAGE plpgsql AS $$ +BEGIN + INSERT INTO multi_mx_call.mx_call_dist_table_1 + SELECT s,s FROM generate_series(100, 110) s; +END;$$; -- Test that undistributed procedures have no issue executing call multi_mx_call.mx_call_proc(2, 0); y @@ -144,6 +150,7 @@ call multi_mx_call.mx_call_proc_custom_types('S', 'A'); F | S (1 row) +call multi_mx_call.mx_call_proc_copy(2); -- Same for unqualified names call mx_call_proc(2, 0); y @@ -176,6 +183,12 @@ select create_distributed_function('mx_call_proc_custom_types(mx_call_enum,mx_ca (1 row) +select create_distributed_function('mx_call_proc_copy(int)'); + create_distributed_function +--------------------------------------------------------------------- + +(1 row) + -- We still don't route them to the workers, because they aren't -- colocated with any distributed tables. SET client_min_messages TO DEBUG1; @@ -206,6 +219,12 @@ DEBUG: stored procedure does not have co-located tables F | S (1 row) +call multi_mx_call.mx_call_proc_copy(2); +DEBUG: stored procedure does not have co-located tables +DEBUG: Collecting INSERT ... SELECT results on coordinator +CONTEXT: SQL statement "INSERT INTO multi_mx_call.mx_call_dist_table_1 + SELECT s,s FROM generate_series(100, 110) s" +PL/pgSQL function mx_call_proc_copy(integer) line XX at SQL statement -- Mark them as colocated with a table. Now we should route them to workers. select colocate_proc_with_table('mx_call_proc', 'mx_call_dist_table_1'::regclass, 1); colocate_proc_with_table @@ -225,6 +244,12 @@ select colocate_proc_with_table('mx_call_proc_custom_types', 'mx_call_dist_table (1 row) +select colocate_proc_with_table('mx_call_proc_copy', 'mx_call_dist_table_1'::regclass, 0); + colocate_proc_with_table +--------------------------------------------------------------------- + +(1 row) + call multi_mx_call.mx_call_proc(2, 0); DEBUG: pushing down the procedure y @@ -253,6 +278,8 @@ DEBUG: pushing down the procedure S | S (1 row) +call mx_call_proc_copy(2); +DEBUG: pushing down the procedure -- Test implicit cast of int to bigint call mx_call_proc_bigint(4, 2); DEBUG: pushing down the procedure @@ -398,18 +425,51 @@ DETAIL: A distributed function is created. To make sure subsequent commands see CALL multi_mx_call.mx_call_proc_tx(20); DEBUG: pushing down the procedure SELECT id, val FROM mx_call_dist_table_1 ORDER BY id, val; - id | val + id | val --------------------------------------------------------------------- - 3 | 1 - 3 | 5 - 4 | 5 - 6 | 5 - 9 | 2 - 10 | -2 - 11 | 3 - 20 | -2 - 21 | 3 -(9 rows) + 3 | 1 + 3 | 5 + 4 | 5 + 6 | 5 + 9 | 2 + 10 | -2 + 11 | 3 + 20 | -2 + 21 | 3 + 100 | 98 + 100 | 98 + 100 | 98 + 101 | 99 + 101 | 99 + 101 | 99 + 102 | 100 + 102 | 100 + 102 | 100 + 103 | 101 + 103 | 101 + 103 | 101 + 104 | 102 + 104 | 102 + 104 | 102 + 105 | 103 + 105 | 103 + 105 | 103 + 106 | 104 + 106 | 104 + 106 | 104 + 107 | 105 + 107 | 105 + 107 | 105 + 108 | 106 + 108 | 106 + 108 | 106 + 109 | 107 + 109 | 107 + 109 | 107 + 110 | 108 + 110 | 108 + 110 | 108 +(42 rows) -- Show that function delegation works from worker nodes as well \c - - - :worker_1_port @@ -539,4 +599,4 @@ PL/pgSQL function mx_call_proc(integer,integer) line XX at assignment reset client_min_messages; \set VERBOSITY terse drop schema multi_mx_call cascade; -NOTICE: drop cascades to 13 other objects +NOTICE: drop cascades to 14 other objects diff --git a/src/test/regress/expected/multi_mx_call_0.out b/src/test/regress/expected/multi_mx_call_0.out index 77667f75b..496e735c9 100644 --- a/src/test/regress/expected/multi_mx_call_0.out +++ b/src/test/regress/expected/multi_mx_call_0.out @@ -131,6 +131,12 @@ BEGIN y := x; x := (select case groupid when 0 then 'F' else 'S' end from pg_dist_local_group); END;$$; +CREATE PROCEDURE mx_call_proc_copy(x int) +LANGUAGE plpgsql AS $$ +BEGIN + INSERT INTO multi_mx_call.mx_call_dist_table_1 + SELECT s,s FROM generate_series(100, 110) s; +END;$$; -- Test that undistributed procedures have no issue executing call multi_mx_call.mx_call_proc(2, 0); y @@ -144,6 +150,7 @@ call multi_mx_call.mx_call_proc_custom_types('S', 'A'); F | S (1 row) +call multi_mx_call.mx_call_proc_copy(2); -- Same for unqualified names call mx_call_proc(2, 0); y @@ -176,6 +183,12 @@ select create_distributed_function('mx_call_proc_custom_types(mx_call_enum,mx_ca (1 row) +select create_distributed_function('mx_call_proc_copy(int)'); + create_distributed_function +--------------------------------------------------------------------- + +(1 row) + -- We still don't route them to the workers, because they aren't -- colocated with any distributed tables. SET client_min_messages TO DEBUG1; @@ -206,6 +219,12 @@ DEBUG: stored procedure does not have co-located tables F | S (1 row) +call multi_mx_call.mx_call_proc_copy(2); +DEBUG: stored procedure does not have co-located tables +DEBUG: Collecting INSERT ... SELECT results on coordinator +CONTEXT: SQL statement "INSERT INTO multi_mx_call.mx_call_dist_table_1 + SELECT s,s FROM generate_series(100, 110) s" +PL/pgSQL function mx_call_proc_copy(integer) line XX at SQL statement -- Mark them as colocated with a table. Now we should route them to workers. select colocate_proc_with_table('mx_call_proc', 'mx_call_dist_table_1'::regclass, 1); colocate_proc_with_table @@ -225,6 +244,12 @@ select colocate_proc_with_table('mx_call_proc_custom_types', 'mx_call_dist_table (1 row) +select colocate_proc_with_table('mx_call_proc_copy', 'mx_call_dist_table_1'::regclass, 0); + colocate_proc_with_table +--------------------------------------------------------------------- + +(1 row) + call multi_mx_call.mx_call_proc(2, 0); DEBUG: pushing down the procedure y @@ -253,6 +278,8 @@ DEBUG: pushing down the procedure S | S (1 row) +call mx_call_proc_copy(2); +DEBUG: pushing down the procedure -- Test implicit cast of int to bigint call mx_call_proc_bigint(4, 2); DEBUG: pushing down the procedure @@ -398,18 +425,51 @@ DETAIL: A distributed function is created. To make sure subsequent commands see CALL multi_mx_call.mx_call_proc_tx(20); DEBUG: pushing down the procedure SELECT id, val FROM mx_call_dist_table_1 ORDER BY id, val; - id | val + id | val --------------------------------------------------------------------- - 3 | 1 - 3 | 5 - 4 | 5 - 6 | 5 - 9 | 2 - 10 | -2 - 11 | 3 - 20 | -2 - 21 | 3 -(9 rows) + 3 | 1 + 3 | 5 + 4 | 5 + 6 | 5 + 9 | 2 + 10 | -2 + 11 | 3 + 20 | -2 + 21 | 3 + 100 | 98 + 100 | 98 + 100 | 98 + 101 | 99 + 101 | 99 + 101 | 99 + 102 | 100 + 102 | 100 + 102 | 100 + 103 | 101 + 103 | 101 + 103 | 101 + 104 | 102 + 104 | 102 + 104 | 102 + 105 | 103 + 105 | 103 + 105 | 103 + 106 | 104 + 106 | 104 + 106 | 104 + 107 | 105 + 107 | 105 + 107 | 105 + 108 | 106 + 108 | 106 + 108 | 106 + 109 | 107 + 109 | 107 + 109 | 107 + 110 | 108 + 110 | 108 + 110 | 108 +(42 rows) -- Show that function delegation works from worker nodes as well \c - - - :worker_1_port @@ -539,4 +599,4 @@ PL/pgSQL function mx_call_proc(integer,integer) line XX at assignment reset client_min_messages; \set VERBOSITY terse drop schema multi_mx_call cascade; -NOTICE: drop cascades to 13 other objects +NOTICE: drop cascades to 14 other objects diff --git a/src/test/regress/expected/multi_mx_function_call_delegation.out b/src/test/regress/expected/multi_mx_function_call_delegation.out index 4fd13cee8..e77e0c3b5 100644 --- a/src/test/regress/expected/multi_mx_function_call_delegation.out +++ b/src/test/regress/expected/multi_mx_function_call_delegation.out @@ -83,6 +83,16 @@ BEGIN y := x; x := (select case groupid when 0 then 'F' else 'S' end from pg_dist_local_group); END;$$; +-- function which internally uses COPY protocol without remote execution +CREATE FUNCTION mx_call_func_copy(x int) +RETURNS bool +LANGUAGE plpgsql AS $$ +BEGIN + INSERT INTO multi_mx_function_call_delegation.mx_call_dist_table_1 + SELECT s,s FROM generate_series(100, 110) s; + + RETURN true; +END;$$; -- Test that undistributed functions have no issue executing select multi_mx_function_call_delegation.mx_call_func(2, 0); mx_call_func @@ -96,6 +106,9 @@ select multi_mx_function_call_delegation.mx_call_func_custom_types('S', 'A'); (F,S) (1 row) +select multi_mx_function_call_delegation.mx_call_copy(2); +ERROR: function multi_mx_function_call_delegation.mx_call_copy(integer) does not exist +HINT: No function matches the given name and argument types. You might need to add explicit type casts. select squares(4); squares --------------------------------------------------------------------- @@ -131,6 +144,12 @@ select create_distributed_function('mx_call_func_custom_types(mx_call_enum,mx_ca (1 row) +select create_distributed_function('mx_call_func_copy(int)'); + create_distributed_function +--------------------------------------------------------------------- + +(1 row) + select create_distributed_function('squares(int)'); create_distributed_function --------------------------------------------------------------------- @@ -604,20 +623,6 @@ PL/pgSQL function mx_call_func(integer,integer) line XX at assignment 29 (1 row) -select mx_call_func(2, 0) from mx_call_dist_table_1; - mx_call_func ---------------------------------------------------------------------- - 28 - 28 - 28 - 28 - 28 - 28 - 28 - 28 - 28 -(9 rows) - select mx_call_func(2, 0) where mx_call_func(0, 2) = 0; DEBUG: generating subplan XXX_1 for subquery SELECT sum((t1.val OPERATOR(pg_catalog.+) t2.val)) AS sum FROM (multi_mx_function_call_delegation.mx_call_dist_table_1 t1 JOIN multi_mx_function_call_delegation.mx_call_dist_table_2 t2 ON ((t1.id OPERATOR(pg_catalog.=) t2.id))) CONTEXT: PL/pgSQL assignment "y := y + (select sum(t1.val + t2.val) from multi_mx_function_call_delegation.mx_call_dist_table_1 t1 join multi_mx_function_call_delegation.mx_call_dist_table_2 t2 on t1.id = t2.id)" @@ -647,6 +652,24 @@ PL/pgSQL function mx_call_func(integer,integer) line XX at assignment 29 | 27 (1 row) +-- we do not delegate the call, but do push down the query +-- that result in remote execution from workers +select mx_call_func(id, 0) from mx_call_dist_table_1; +ERROR: cannot execute a distributed query from a query on a shard +CONTEXT: PL/pgSQL assignment "y := y + (select sum(t1.val + t2.val) from multi_mx_function_call_delegation.mx_call_dist_table_1 t1 join multi_mx_function_call_delegation.mx_call_dist_table_2 t2 on t1.id = t2.id)" +PL/pgSQL function multi_mx_function_call_delegation.mx_call_func(integer,integer) line XX at assignment +while executing command on localhost:xxxxx +select mx_call_func(2, 0) from mx_call_dist_table_1 where id = 3; +ERROR: cannot execute a distributed query from a query on a shard +CONTEXT: PL/pgSQL assignment "y := y + (select sum(t1.val + t2.val) from multi_mx_function_call_delegation.mx_call_dist_table_1 t1 join multi_mx_function_call_delegation.mx_call_dist_table_2 t2 on t1.id = t2.id)" +PL/pgSQL function multi_mx_function_call_delegation.mx_call_func(integer,integer) line XX at assignment +while executing command on localhost:xxxxx +select mx_call_func_copy(2) from mx_call_dist_table_1 where id = 3; +ERROR: cannot execute a distributed query from a query on a shard +CONTEXT: SQL statement "INSERT INTO multi_mx_function_call_delegation.mx_call_dist_table_1 + SELECT s,s FROM generate_series(100, 110) s" +PL/pgSQL function multi_mx_function_call_delegation.mx_call_func_copy(integer) line XX at SQL statement +while executing command on localhost:xxxxx DO $$ BEGIN perform mx_call_func_tbl(40); END; $$; DEBUG: not pushing down function calls in a multi-statement transaction CONTEXT: SQL statement "SELECT mx_call_func_tbl(40)" @@ -725,7 +748,12 @@ HINT: Connect to the coordinator and run it again. -- show that functions can be delegated from worker nodes SET client_min_messages TO DEBUG1; SELECT mx_call_func(2, 0); -DEBUG: pushing down the function call +DEBUG: generating subplan XXX_1 for subquery SELECT sum((t1.val OPERATOR(pg_catalog.+) t2.val)) AS sum FROM (multi_mx_function_call_delegation.mx_call_dist_table_1 t1 JOIN multi_mx_function_call_delegation.mx_call_dist_table_2 t2 ON ((t1.id OPERATOR(pg_catalog.=) t2.id))) +CONTEXT: PL/pgSQL assignment "y := y + (select sum(t1.val + t2.val) from multi_mx_function_call_delegation.mx_call_dist_table_1 t1 join multi_mx_function_call_delegation.mx_call_dist_table_2 t2 on t1.id = t2.id)" +PL/pgSQL function mx_call_func(integer,integer) line XX at assignment +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT ((2 OPERATOR(pg_catalog.+) (SELECT intermediate_result.sum FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(sum bigint))))::integer +CONTEXT: PL/pgSQL assignment "y := y + (select sum(t1.val + t2.val) from multi_mx_function_call_delegation.mx_call_dist_table_1 t1 join multi_mx_function_call_delegation.mx_call_dist_table_2 t2 on t1.id = t2.id)" +PL/pgSQL function mx_call_func(integer,integer) line XX at assignment mx_call_func --------------------------------------------------------------------- 28 @@ -736,4 +764,4 @@ SET search_path TO multi_mx_function_call_delegation, public; RESET client_min_messages; \set VERBOSITY terse DROP SCHEMA multi_mx_function_call_delegation CASCADE; -NOTICE: drop cascades to 14 other objects +NOTICE: drop cascades to 15 other objects diff --git a/src/test/regress/expected/multi_mx_function_call_delegation_0.out b/src/test/regress/expected/multi_mx_function_call_delegation_0.out index 70672b455..657183bc2 100644 --- a/src/test/regress/expected/multi_mx_function_call_delegation_0.out +++ b/src/test/regress/expected/multi_mx_function_call_delegation_0.out @@ -83,6 +83,16 @@ BEGIN y := x; x := (select case groupid when 0 then 'F' else 'S' end from pg_dist_local_group); END;$$; +-- function which internally uses COPY protocol without remote execution +CREATE FUNCTION mx_call_func_copy(x int) +RETURNS bool +LANGUAGE plpgsql AS $$ +BEGIN + INSERT INTO multi_mx_function_call_delegation.mx_call_dist_table_1 + SELECT s,s FROM generate_series(100, 110) s; + + RETURN true; +END;$$; -- Test that undistributed functions have no issue executing select multi_mx_function_call_delegation.mx_call_func(2, 0); mx_call_func @@ -96,6 +106,9 @@ select multi_mx_function_call_delegation.mx_call_func_custom_types('S', 'A'); (F,S) (1 row) +select multi_mx_function_call_delegation.mx_call_copy(2); +ERROR: function multi_mx_function_call_delegation.mx_call_copy(integer) does not exist +HINT: No function matches the given name and argument types. You might need to add explicit type casts. select squares(4); squares --------------------------------------------------------------------- @@ -131,6 +144,12 @@ select create_distributed_function('mx_call_func_custom_types(mx_call_enum,mx_ca (1 row) +select create_distributed_function('mx_call_func_copy(int)'); + create_distributed_function +--------------------------------------------------------------------- + +(1 row) + select create_distributed_function('squares(int)'); create_distributed_function --------------------------------------------------------------------- @@ -604,20 +623,6 @@ PL/pgSQL function mx_call_func(integer,integer) line XX at assignment 29 (1 row) -select mx_call_func(2, 0) from mx_call_dist_table_1; - mx_call_func ---------------------------------------------------------------------- - 28 - 28 - 28 - 28 - 28 - 28 - 28 - 28 - 28 -(9 rows) - select mx_call_func(2, 0) where mx_call_func(0, 2) = 0; DEBUG: generating subplan XXX_1 for subquery SELECT sum((t1.val OPERATOR(pg_catalog.+) t2.val)) AS sum FROM (multi_mx_function_call_delegation.mx_call_dist_table_1 t1 JOIN multi_mx_function_call_delegation.mx_call_dist_table_2 t2 ON ((t1.id OPERATOR(pg_catalog.=) t2.id))) CONTEXT: SQL statement "SELECT y + (select sum(t1.val + t2.val) from multi_mx_function_call_delegation.mx_call_dist_table_1 t1 join multi_mx_function_call_delegation.mx_call_dist_table_2 t2 on t1.id = t2.id)" @@ -647,6 +652,24 @@ PL/pgSQL function mx_call_func(integer,integer) line XX at assignment 29 | 27 (1 row) +-- we do not delegate the call, but do push down the query +-- that result in remote execution from workers +select mx_call_func(id, 0) from mx_call_dist_table_1; +ERROR: cannot execute a distributed query from a query on a shard +CONTEXT: SQL statement "SELECT y + (select sum(t1.val + t2.val) from multi_mx_function_call_delegation.mx_call_dist_table_1 t1 join multi_mx_function_call_delegation.mx_call_dist_table_2 t2 on t1.id = t2.id)" +PL/pgSQL function multi_mx_function_call_delegation.mx_call_func(integer,integer) line XX at assignment +while executing command on localhost:xxxxx +select mx_call_func(2, 0) from mx_call_dist_table_1 where id = 3; +ERROR: cannot execute a distributed query from a query on a shard +CONTEXT: SQL statement "SELECT y + (select sum(t1.val + t2.val) from multi_mx_function_call_delegation.mx_call_dist_table_1 t1 join multi_mx_function_call_delegation.mx_call_dist_table_2 t2 on t1.id = t2.id)" +PL/pgSQL function multi_mx_function_call_delegation.mx_call_func(integer,integer) line XX at assignment +while executing command on localhost:xxxxx +select mx_call_func_copy(2) from mx_call_dist_table_1 where id = 3; +ERROR: cannot execute a distributed query from a query on a shard +CONTEXT: SQL statement "INSERT INTO multi_mx_function_call_delegation.mx_call_dist_table_1 + SELECT s,s FROM generate_series(100, 110) s" +PL/pgSQL function multi_mx_function_call_delegation.mx_call_func_copy(integer) line XX at SQL statement +while executing command on localhost:xxxxx DO $$ BEGIN perform mx_call_func_tbl(40); END; $$; DEBUG: not pushing down function calls in a multi-statement transaction CONTEXT: SQL statement "SELECT mx_call_func_tbl(40)" @@ -725,7 +748,12 @@ HINT: Connect to the coordinator and run it again. -- show that functions can be delegated from worker nodes SET client_min_messages TO DEBUG1; SELECT mx_call_func(2, 0); -DEBUG: pushing down the function call +DEBUG: generating subplan XXX_1 for subquery SELECT sum((t1.val OPERATOR(pg_catalog.+) t2.val)) AS sum FROM (multi_mx_function_call_delegation.mx_call_dist_table_1 t1 JOIN multi_mx_function_call_delegation.mx_call_dist_table_2 t2 ON ((t1.id OPERATOR(pg_catalog.=) t2.id))) +CONTEXT: SQL statement "SELECT y + (select sum(t1.val + t2.val) from multi_mx_function_call_delegation.mx_call_dist_table_1 t1 join multi_mx_function_call_delegation.mx_call_dist_table_2 t2 on t1.id = t2.id)" +PL/pgSQL function mx_call_func(integer,integer) line XX at assignment +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT (2 OPERATOR(pg_catalog.+) (SELECT intermediate_result.sum FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(sum bigint))) +CONTEXT: SQL statement "SELECT y + (select sum(t1.val + t2.val) from multi_mx_function_call_delegation.mx_call_dist_table_1 t1 join multi_mx_function_call_delegation.mx_call_dist_table_2 t2 on t1.id = t2.id)" +PL/pgSQL function mx_call_func(integer,integer) line XX at assignment mx_call_func --------------------------------------------------------------------- 28 @@ -736,4 +764,4 @@ SET search_path TO multi_mx_function_call_delegation, public; RESET client_min_messages; \set VERBOSITY terse DROP SCHEMA multi_mx_function_call_delegation CASCADE; -NOTICE: drop cascades to 14 other objects +NOTICE: drop cascades to 15 other objects diff --git a/src/test/regress/expected/upgrade_list_citus_objects.out b/src/test/regress/expected/upgrade_list_citus_objects.out index e38818594..c4106f849 100644 --- a/src/test/regress/expected/upgrade_list_citus_objects.out +++ b/src/test/regress/expected/upgrade_list_citus_objects.out @@ -69,7 +69,7 @@ ORDER BY 1; function citus_internal.replace_isolation_tester_func() function citus_internal.restore_isolation_tester_func() function citus_internal.upgrade_columnar_storage(regclass) - function citus_internal_add_object_metadata(text,text[],text[],integer,integer) + function citus_internal_add_object_metadata(text,text[],text[],integer,integer,boolean) function citus_internal_add_partition_metadata(regclass,"char",text,integer,"char") function citus_internal_add_placement_metadata(bigint,integer,bigint,integer,bigint) function citus_internal_add_shard_metadata(regclass,bigint,"char",text,text) @@ -118,7 +118,7 @@ ORDER BY 1; function coord_combine_agg(oid,cstring,anyelement) function coord_combine_agg_ffunc(internal,oid,cstring,anyelement) function coord_combine_agg_sfunc(internal,oid,cstring,anyelement) - function create_distributed_function(regprocedure,text,text) + function create_distributed_function(regprocedure,text,text,boolean) function create_distributed_table(regclass,text,citus.distribution_type,text,integer) function create_intermediate_result(text,text) function create_reference_table(regclass) diff --git a/src/test/regress/multi_schedule b/src/test/regress/multi_schedule index c31c68df5..6f2cab332 100644 --- a/src/test/regress/multi_schedule +++ b/src/test/regress/multi_schedule @@ -68,6 +68,7 @@ test: multi_basic_queries cross_join multi_complex_expressions multi_subquery mu test: multi_subquery_complex_reference_clause multi_subquery_window_functions multi_view multi_sql_function multi_prepare_sql test: sql_procedure multi_function_in_join row_types materialized_view test: multi_subquery_in_where_reference_clause adaptive_executor propagate_set_commands geqo +test: forcepushdown_functions # this should be run alone as it gets too many clients test: join_pushdown test: multi_subquery_union multi_subquery_in_where_clause multi_subquery_misc statement_cancel_error_message diff --git a/src/test/regress/sql/citus_local_table_triggers.sql b/src/test/regress/sql/citus_local_table_triggers.sql index d091c498b..76b192388 100644 --- a/src/test/regress/sql/citus_local_table_triggers.sql +++ b/src/test/regress/sql/citus_local_table_triggers.sql @@ -268,6 +268,15 @@ BEGIN END; $insert_100$ LANGUAGE plpgsql; +CREATE TABLE local_table (value int); + +CREATE FUNCTION insert_100_local() RETURNS trigger AS $insert_100$ +BEGIN + INSERT INTO local_table VALUES (100); + RETURN NEW; +END; +$insert_100$ LANGUAGE plpgsql; + BEGIN; CREATE TRIGGER insert_100_trigger AFTER TRUNCATE ON another_citus_local_table @@ -282,7 +291,7 @@ BEGIN; SELECT * FROM reference_table; ROLLBACK; - +-- cannot perform remote execution from a trigger on a Citus local table BEGIN; -- update should actually update something to test ON UPDATE CASCADE logic INSERT INTO another_citus_local_table VALUES (600); @@ -296,9 +305,47 @@ BEGIN; AFTER UPDATE ON citus_local_table FOR EACH STATEMENT EXECUTE FUNCTION insert_100(); + UPDATE another_citus_local_table SET value=value-1;; +ROLLBACK; + +-- can perform regular execution from a trigger on a Citus local table +BEGIN; + -- update should actually update something to test ON UPDATE CASCADE logic + INSERT INTO another_citus_local_table VALUES (600); + INSERT INTO citus_local_table VALUES (600); + + CREATE TRIGGER insert_100_trigger + AFTER UPDATE ON another_citus_local_table + FOR EACH STATEMENT EXECUTE FUNCTION insert_100_local(); + + CREATE TRIGGER insert_100_trigger + AFTER UPDATE ON citus_local_table + FOR EACH STATEMENT EXECUTE FUNCTION insert_100_local(); + UPDATE another_citus_local_table SET value=value-1;; -- we should see two rows with "100" - SELECT * FROM reference_table; + SELECT * FROM local_table; +ROLLBACK; + +-- can perform local execution from a trigger on a Citus local table +BEGIN; + SELECT citus_add_local_table_to_metadata('local_table'); + + -- update should actually update something to test ON UPDATE CASCADE logic + INSERT INTO another_citus_local_table VALUES (600); + INSERT INTO citus_local_table VALUES (600); + + CREATE TRIGGER insert_100_trigger + AFTER UPDATE ON another_citus_local_table + FOR EACH STATEMENT EXECUTE FUNCTION insert_100_local(); + + CREATE TRIGGER insert_100_trigger + AFTER UPDATE ON citus_local_table + FOR EACH STATEMENT EXECUTE FUNCTION insert_100_local(); + + UPDATE another_citus_local_table SET value=value-1;; + -- we should see two rows with "100" + SELECT * FROM local_table; ROLLBACK; -- test on partitioned citus local tables diff --git a/src/test/regress/sql/forcepushdown_functions.sql b/src/test/regress/sql/forcepushdown_functions.sql new file mode 100644 index 000000000..b85fa732e --- /dev/null +++ b/src/test/regress/sql/forcepushdown_functions.sql @@ -0,0 +1,586 @@ +SET citus.log_remote_commands TO OFF; +DROP SCHEMA IF EXISTS forcepushdown_schema CASCADE; +CREATE SCHEMA forcepushdown_schema; +SET search_path TO 'forcepushdown_schema'; +SET citus.shard_replication_factor = 1; +SET citus.shard_count = 32; +SET citus.next_shard_id TO 900000; + +CREATE TABLE test_forcepushdown(intcol int PRIMARY KEY, data char(50) default 'default'); +SELECT create_distributed_table('test_forcepushdown', 'intcol', colocate_with := 'none'); + +-- +--Table in a different colocation group +-- +CREATE TABLE test_forcepushdown_noncolocate(intcol int PRIMARY KEY); +SELECT create_distributed_table('test_forcepushdown_noncolocate', 'intcol', colocate_with := 'none'); + +CREATE FUNCTION insert_data(a integer) +RETURNS void LANGUAGE plpgsql AS $fn$ +BEGIN + INSERT INTO forcepushdown_schema.test_forcepushdown VALUES (a); +END; +$fn$; + +CREATE FUNCTION insert_data_non_distarg(a integer) +RETURNS void LANGUAGE plpgsql AS $fn$ +BEGIN + INSERT INTO forcepushdown_schema.test_forcepushdown VALUES (a+1); +END; +$fn$; + +CREATE FUNCTION update_data_nonlocal(a integer) +RETURNS void LANGUAGE plpgsql AS $fn$ +BEGIN + UPDATE forcepushdown_schema.test_forcepushdown SET data = 'non-default'; +END; +$fn$; + +CREATE FUNCTION insert_data_noncolocation(a int) +RETURNS void LANGUAGE plpgsql AS $fn$ +BEGIN + -- Insert into a different table than the function is colocated with + INSERT INTO forcepushdown_schema.test_forcepushdown_noncolocate VALUES (a); +END; +$fn$; + +SELECT create_distributed_function( + 'insert_data(int)', 'a', + colocate_with := 'test_forcepushdown', + force_pushdown := true +); + +SELECT create_distributed_function( + 'insert_data_non_distarg(int)', 'a', + colocate_with := 'test_forcepushdown', + force_pushdown := true +); + +SELECT create_distributed_function( + 'update_data_nonlocal(int)', 'a', + colocate_with := 'test_forcepushdown', + force_pushdown := true +); + +SELECT create_distributed_function( + 'insert_data_noncolocation(int)', 'a', + colocate_with := 'test_forcepushdown', + force_pushdown := true +); + +SET client_min_messages TO DEBUG1; +--SET citus.log_remote_commands TO on; + +SELECT public.wait_until_metadata_sync(30000); + +SELECT 'Transaction with no errors' Testing; +BEGIN; +INSERT INTO forcepushdown_schema.test_forcepushdown VALUES (1); +-- This call will insert both the rows locally on the remote worker +SELECT insert_data(2); +INSERT INTO forcepushdown_schema.test_forcepushdown VALUES (3); +COMMIT; + +SELECT 'Transaction with duplicate error in the remote function' Testing; +BEGIN; +INSERT INTO forcepushdown_schema.test_forcepushdown VALUES (4); +-- This call will fail with duplicate error on the remote worker +SELECT insert_data(3); +INSERT INTO forcepushdown_schema.test_forcepushdown VALUES (5); +COMMIT; + +SELECT 'Transaction with duplicate error in the local statement' Testing; +BEGIN; +INSERT INTO forcepushdown_schema.test_forcepushdown VALUES (6); +-- This call will insert both the rows locally on the remote worker +SELECT insert_data(7); +INSERT INTO forcepushdown_schema.test_forcepushdown VALUES (8); +-- This will fail +INSERT INTO forcepushdown_schema.test_forcepushdown VALUES (8); +COMMIT; + +SELECT 'Transaction with function using non-distribution argument' Testing; +BEGIN; +-- This should fail +SELECT insert_data_non_distarg(9); +COMMIT; + +SELECT 'Transaction with function doing remote connection' Testing; +BEGIN; +-- This statement will pass +INSERT INTO forcepushdown_schema.test_forcepushdown VALUES (11); +-- This call will try to update rows locally and on remote node(s) +SELECT update_data_nonlocal(12); +INSERT INTO forcepushdown_schema.test_forcepushdown VALUES (13); +COMMIT; + +SELECT 'Transaction with no errors but with a rollback' Testing; +BEGIN; +INSERT INTO forcepushdown_schema.test_forcepushdown VALUES (14); +-- This call will insert both the rows locally on the remote worker +SELECT insert_data(15); +INSERT INTO forcepushdown_schema.test_forcepushdown VALUES (16); +ROLLBACK; + +-- +-- Add function with pushdown=true in the targetList of a query +-- +BEGIN; +-- Query gets delegated to the node of the shard xx_900001 for the key=1, +-- and the function inserts value (1+17) locally on the shard xx_900031 +SELECT insert_data(intcol+17) from test_forcepushdown where intcol = 1; + +-- This will fail with duplicate error as the function already inserted +-- the value(1+17) +SELECT insert_data(18); +COMMIT; + +-- +-- Access a table with the same shard key as distribution argument but in a +-- different colocation group. +-- +BEGIN; +SELECT insert_data_noncolocation(19); +COMMIT; + +SELECT insert_data_noncolocation(19); + +-- This should have only the first 3 rows as all other transactions were rolled back. +SELECT * FROM forcepushdown_schema.test_forcepushdown ORDER BY 1; + +-- +-- Nested call, function with pushdown=false calling function with pushdown=true +-- +CREATE TABLE test_nested (id int, name text); +SELECT create_distributed_table('test_nested','id'); +INSERT INTO test_nested VALUES (100,'hundred'); +INSERT INTO test_nested VALUES (200,'twohundred'); +INSERT INTO test_nested VALUES (300,'threehundred'); +INSERT INTO test_nested VALUES (400,'fourhundred'); + +CREATE OR REPLACE FUNCTION inner_force_pushdown_function(int) +RETURNS NUMERIC AS $$ +DECLARE ret_val NUMERIC; +BEGIN + SELECT max(id)::numeric+1 INTO ret_val FROM forcepushdown_schema.test_nested WHERE id = $1; + RAISE NOTICE 'inner_force_pushdown_function():%', ret_val; + RETURN ret_val; +END; +$$ LANGUAGE plpgsql; + +CREATE OR REPLACE FUNCTION func_calls_forcepush_func() +RETURNS NUMERIC AS $$ +DECLARE incremented_val NUMERIC; +BEGIN + -- Constant distribution argument + SELECT inner_force_pushdown_function INTO incremented_val FROM inner_force_pushdown_function(100); + RETURN incremented_val; +END; +$$ LANGUAGE plpgsql; + +SELECT create_distributed_function('func_calls_forcepush_func()'); +SELECT create_distributed_function('inner_force_pushdown_function(int)', '$1', colocate_with := 'test_nested', force_pushdown := true); +SELECT public.wait_until_metadata_sync(30000); + +BEGIN; +SELECT func_calls_forcepush_func(); +COMMIT; + +SELECT func_calls_forcepush_func(); + +CREATE OR REPLACE FUNCTION get_val() +RETURNS INT AS $$ +BEGIN + RETURN 100::INT; +END; +$$ LANGUAGE plpgsql; + +-- +-- UDF calling another UDF in a FROM clause +-- fn() +-- { +-- select res into var from fn(); +-- } +-- +CREATE OR REPLACE FUNCTION func_calls_forcepush_func_infrom() +RETURNS NUMERIC AS $$ +DECLARE incremented_val NUMERIC; +DECLARE add_val INT; +BEGIN + add_val := get_val(); + SELECT inner_force_pushdown_function INTO incremented_val FROM inner_force_pushdown_function(add_val + 100); + RETURN incremented_val; +END; +$$ LANGUAGE plpgsql; + +SELECT func_calls_forcepush_func_infrom(); + +BEGIN; +SELECT func_calls_forcepush_func_infrom(); +COMMIT; + +-- +-- UDF calling another UDF in the SELECT targetList +-- fn() +-- { +-- select fn() into var; +-- } +-- +CREATE OR REPLACE FUNCTION func_calls_forcepush_func_intarget() +RETURNS NUMERIC AS $$ +DECLARE incremented_val NUMERIC; +DECLARE add_val INT; +BEGIN + add_val := get_val(); + SELECT inner_force_pushdown_function(100 + 100) INTO incremented_val OFFSET 0; + RETURN incremented_val; +END; +$$ LANGUAGE plpgsql; + +SELECT func_calls_forcepush_func_intarget(); + +BEGIN; +SELECT func_calls_forcepush_func_intarget(); +COMMIT; + +-- +-- Recursive function call with pushdown=true +-- +CREATE OR REPLACE FUNCTION test_recursive(inp integer) +RETURNS INT AS $$ +DECLARE var INT; +BEGIN + RAISE NOTICE 'input:%', inp; + if (inp > 1) then + inp := inp - 1; + var := forcepushdown_schema.test_recursive(inp); + RETURN var; + else + RETURN inp; + END if; +END; +$$ LANGUAGE plpgsql; + +SELECT create_distributed_function('test_recursive(int)', '$1', colocate_with := 'test_nested', force_pushdown := true); + +BEGIN; +SELECT test_recursive(5); +END; + +-- +-- Distributed function gets delegated indirectly (as part of a query) +-- +BEGIN; + +-- Query lands on the shard with key = 300(shard __900089) and the function inserts locally +SELECT inner_force_pushdown_function(id) FROM test_nested WHERE id = 300; + +-- Query lands on the shard with key = 300(shard __900089) and the function inserts remotely +SELECT insert_data_non_distarg(id) FROM test_nested WHERE id = 300; + +END; + +-- +-- Non constant distribution arguments +-- + +-- Param(PARAM_EXEC) node e.g. SELECT fn((SELECT col from test_nested where col=val)) +BEGIN; +SELECT inner_force_pushdown_function((SELECT id FROM test_nested WHERE id=400)); +END; + +CREATE TABLE emp ( + empname text NOT NULL, + salary integer +); + +CREATE TABLE emp_audit( + operation char(1) NOT NULL, + stamp timestamp NOT NULL, + userid text NOT NULL, + empname text NOT NULL, + salary integer +); + +SELECT create_distributed_table('emp','empname'); +SELECT create_distributed_table('emp_audit','empname'); + +CREATE OR REPLACE FUNCTION inner_emp(empname text) +RETURNS void +AS $$ +DECLARE +BEGIN + INSERT INTO emp VALUES (empname, 33); +END; +$$ LANGUAGE plpgsql; +CREATE OR REPLACE FUNCTION outer_emp() +RETURNS void +AS $$ +DECLARE +BEGIN + PERFORM inner_emp('hello'); +END; +$$ LANGUAGE plpgsql; + +SELECT create_distributed_function('inner_emp(text)','empname', force_pushdown := true); +SELECT outer_emp(); +SELECT * from emp; + +-- +-- INSERT..SELECT +-- +CREATE FUNCTION insert_select_data(a integer) +RETURNS void LANGUAGE plpgsql AS $fn$ +BEGIN + INSERT INTO forcepushdown_schema.test_forcepushdown SELECT(a+1); +END; +$fn$; + +SELECT create_distributed_function( + 'insert_select_data(int)', 'a', + colocate_with := 'test_forcepushdown', + force_pushdown := true +); + +-- Function lands on worker1 and issues COPY ... INSERT on the worker2 into the shard_900021 +BEGIN; +INSERT INTO forcepushdown_schema.test_forcepushdown VALUES (30); +-- This will fail +SELECT insert_select_data(20); +COMMIT; + +-- Function lands on worker2 and issues COPY ... INSERT on the same node into the shard_900029 +BEGIN; +-- This will pass +SELECT insert_select_data(21); +END; + +-- Function lands on worker2 and issues COPY ... INSERT on the worker1 into the shard_900028 +BEGIN; +INSERT INTO forcepushdown_schema.test_forcepushdown VALUES (30); +-- This will fail +SELECT insert_select_data(22); +END; + +-- Functions lands on worker1 and issues COPY ... INSERT on the worker2 into the shard_900021 +-- This will pass as there is no surrounding transaction +SELECT insert_select_data(20); + +-- (21+1) and (20+1) should appear +SELECT * FROM forcepushdown_schema.test_forcepushdown ORDER BY 1; + +CREATE FUNCTION insert_select_data_nonlocal(a integer) +RETURNS void LANGUAGE plpgsql AS $fn$ +BEGIN + INSERT INTO forcepushdown_schema.test_forcepushdown(intcol) + SELECT intcol FROM forcepushdown_schema.test_forcepushdown_noncolocate; +END; +$fn$; + +SELECT create_distributed_function( + 'insert_select_data_nonlocal(int)', 'a', + colocate_with := 'test_forcepushdown', + force_pushdown := true +); + +INSERT INTO forcepushdown_schema.test_forcepushdown_noncolocate VALUES (30); +INSERT INTO forcepushdown_schema.test_forcepushdown_noncolocate VALUES (31); +INSERT INTO forcepushdown_schema.test_forcepushdown_noncolocate VALUES (32); + +BEGIN; +INSERT INTO forcepushdown_schema.test_forcepushdown VALUES (40); +-- This will fail +SELECT insert_select_data_nonlocal(41); +COMMIT; + +-- Above 3 rows (30, 31, 32) should appear now +SELECT insert_select_data_nonlocal(40); + +SELECT * FROM forcepushdown_schema.test_forcepushdown ORDER BY 1; + +CREATE TABLE test_forcepushdown_char(data char(50) PRIMARY KEY); +SELECT create_distributed_table('test_forcepushdown_char', 'data', colocate_with := 'none'); +CREATE TABLE test_forcepushdown_varchar(data varchar PRIMARY KEY); +SELECT create_distributed_table('test_forcepushdown_varchar', 'data', colocate_with := 'none'); +CREATE TABLE test_forcepushdown_text(data text PRIMARY KEY); +SELECT create_distributed_table('test_forcepushdown_text', 'data', colocate_with := 'none'); + +CREATE FUNCTION insert_data_char(a char(50)) +RETURNS void LANGUAGE plpgsql AS $fn$ +BEGIN + INSERT INTO forcepushdown_schema.test_forcepushdown_char VALUES (a); +END; +$fn$; +SELECT create_distributed_function( + 'insert_data_char(char)', 'a', + colocate_with := 'test_forcepushdown_char', + force_pushdown := true +); + +CREATE FUNCTION insert_data_varchar(a varchar) +RETURNS void LANGUAGE plpgsql AS $fn$ +BEGIN + INSERT INTO forcepushdown_schema.test_forcepushdown_varchar VALUES (a); +END; +$fn$; +SELECT create_distributed_function( + 'insert_data_varchar(varchar)', 'a', + colocate_with := 'test_forcepushdown_varchar', + force_pushdown := true +); + +CREATE FUNCTION insert_data_text(a text) +RETURNS void LANGUAGE plpgsql AS $fn$ +BEGIN + INSERT INTO forcepushdown_schema.test_forcepushdown_text VALUES (a); +END; +$fn$; +SELECT create_distributed_function( + 'insert_data_text(text)', 'a', + colocate_with := 'test_forcepushdown_text', + force_pushdown := true +); + +SELECT insert_data_varchar('VARCHAR'); +BEGIN; +SELECT insert_data_varchar('VARCHAR2'); +COMMIT; + +SELECT insert_data_text('TEXT'); +BEGIN; +SELECT insert_data_text('TEXT2'); +COMMIT; + +-- Char is failing as the datatype is represented differently in the +-- PL/PgSQL and the exec engine. +SELECT insert_data_char('CHAR'); +BEGIN; +SELECT insert_data_char('CHAR'); +COMMIT; + +SELECT * from test_forcepushdown_char; +SELECT * from test_forcepushdown_varchar; +SELECT * from test_forcepushdown_text; + +-- Test sub query +CREATE TABLE test_subquery(data int, result int); +SELECT create_distributed_table('test_subquery', 'data', colocate_with := 'none'); + +CREATE TABLE test_non_colocated(id int); +SELECT create_distributed_table('test_non_colocated', 'id', colocate_with := 'none'); + +CREATE FUNCTION select_data(a integer) +RETURNS void LANGUAGE plpgsql AS $fn$ +DECLARE var INT; +BEGIN + SELECT result INTO var FROM forcepushdown_schema.test_subquery WHERE data = + (SELECT data FROM forcepushdown_schema.test_subquery WHERE data = a); + RAISE NOTICE 'Result: %', var; +END; +$fn$; +SELECT create_distributed_function( + 'select_data(int)', 'a', + colocate_with := 'test_subquery', + force_pushdown := true +); + +CREATE FUNCTION select_data_noncolocate(a integer) +RETURNS void LANGUAGE plpgsql AS $fn$ +DECLARE var INT; +BEGIN + -- Key is the same but colocation ID is different + SELECT data INTO var FROM forcepushdown_schema.test_subquery WHERE data = + (SELECT id FROM forcepushdown_schema.test_non_colocated WHERE id = a); + RAISE NOTICE 'Result: %', var; +END; +$fn$; +SELECT create_distributed_function( + 'select_data_noncolocate(int)', 'a', + colocate_with := 'test_subquery', + force_pushdown := true +); + +CREATE FUNCTION insert_select_data_cte1(a integer) +RETURNS void LANGUAGE plpgsql AS $fn$ +DECLARE var INT; +BEGIN + WITH ins AS (INSERT INTO forcepushdown_schema.test_subquery VALUES (a) RETURNING data) + SELECT ins.data INTO var FROM ins; + RAISE NOTICE 'Result: %', var; +END; +$fn$; +SELECT create_distributed_function( + 'insert_select_data_cte1(int)', 'a', + colocate_with := 'test_subquery', + force_pushdown := true +); + +CREATE FUNCTION insert_select_data_cte2(a integer) +RETURNS void LANGUAGE plpgsql AS $fn$ +DECLARE var INT; +BEGIN + WITH ins AS (INSERT INTO forcepushdown_schema.test_subquery VALUES (a) RETURNING data) + SELECT ins.data INTO var FROM forcepushdown_schema.test_subquery, ins WHERE forcepushdown_schema.test_subquery.data = a; + RAISE NOTICE 'Result: %', var; +END; +$fn$; +SELECT create_distributed_function( + 'insert_select_data_cte2(int)', 'a', + colocate_with := 'test_subquery', + force_pushdown := true +); + +CREATE FUNCTION insert_data_cte_nondist(a integer) +RETURNS void LANGUAGE plpgsql AS $fn$ +DECLARE var INT; +BEGIN + -- Inserting a non-distribution argument (a+1) + WITH ins AS (INSERT INTO forcepushdown_schema.test_subquery VALUES (a+1) RETURNING data) + SELECT ins.data INTO var FROM forcepushdown_schema.test_subquery, ins WHERE forcepushdown_schema.test_subquery.data = a; + RAISE NOTICE 'Result: %', var; +END; +$fn$; +SELECT create_distributed_function( + 'insert_data_cte_nondist(int)', 'a', + colocate_with := 'test_subquery', + force_pushdown := true +); + +INSERT INTO forcepushdown_schema.test_subquery VALUES(100, -1); + +-- This should pass +SELECT select_data(100); +BEGIN; +SELECT select_data(100); +END; + +-- This should fail +SELECT select_data_noncolocate(100); +BEGIN; +SELECT select_data_noncolocate(100); +END; + +-- This should pass +SELECT insert_select_data_cte1(200); +BEGIN; +SELECT insert_select_data_cte1(200); +COMMIT; + +-- This should pass +SELECT insert_select_data_cte2(300); +BEGIN; +SELECT insert_select_data_cte2(300); +COMMIT; + +-- This should pass +SELECT insert_data_cte_nondist(400); +BEGIN; +SELECT insert_data_cte_nondist(400); +COMMIT; + +-- Rows 100, 200, 300 should be seen +SELECT * FROM forcepushdown_schema.test_subquery ORDER BY 1; + +RESET client_min_messages; +SET citus.log_remote_commands TO off; +DROP SCHEMA forcepushdown_schema CASCADE; diff --git a/src/test/regress/sql/foreign_tables_mx.sql b/src/test/regress/sql/foreign_tables_mx.sql index a7c3138df..fdd391d10 100644 --- a/src/test/regress/sql/foreign_tables_mx.sql +++ b/src/test/regress/sql/foreign_tables_mx.sql @@ -65,17 +65,15 @@ ALTER FOREIGN TABLE public.foreign_table_newname VALIDATE CONSTRAINT check_c_2; ALTER FOREIGN TABLE public.foreign_table_newname DROP constraint IF EXISTS check_c_2; -- trigger test -CREATE TABLE distributed_table(value int); -SELECT create_distributed_table('distributed_table', 'value'); +CREATE TABLE table42(value int); CREATE FUNCTION insert_42() RETURNS trigger AS $insert_42$ BEGIN - INSERT INTO distributed_table VALUES (42); + INSERT INTO table42 VALUES (42); RETURN NEW; END; $insert_42$ LANGUAGE plpgsql; - CREATE TRIGGER insert_42_trigger AFTER DELETE ON public.foreign_table_newname FOR EACH ROW EXECUTE FUNCTION insert_42(); @@ -83,14 +81,14 @@ FOR EACH ROW EXECUTE FUNCTION insert_42(); -- do the same pattern from the workers as well INSERT INTO public.foreign_table_newname VALUES (99, 'test_2'); delete from public.foreign_table_newname where id_test = 99; -select * from distributed_table ORDER BY value; +select * from table42 ORDER BY value; -- disable trigger alter foreign table public.foreign_table_newname disable trigger insert_42_trigger; INSERT INTO public.foreign_table_newname VALUES (99, 'test_2'); delete from public.foreign_table_newname where id_test = 99; -- should not insert again as trigger disabled -select * from distributed_table ORDER BY value; +select * from table42 ORDER BY value; DROP TRIGGER insert_42_trigger ON public.foreign_table_newname; diff --git a/src/test/regress/sql/metadata_sync_helpers.sql b/src/test/regress/sql/metadata_sync_helpers.sql index e84fa55aa..dfa997079 100644 --- a/src/test/regress/sql/metadata_sync_helpers.sql +++ b/src/test/regress/sql/metadata_sync_helpers.sql @@ -346,9 +346,9 @@ BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED; SELECT assign_distributed_transaction_id(0, 8, '2021-07-09 15:41:55.542377+02'); SET application_name to 'citus'; \set VERBOSITY terse - WITH distributed_object_data(typetext, objnames, objargs, distargumentindex, colocationid) - AS (VALUES ('non_existing_type', ARRAY['non_existing_user']::text[], ARRAY[]::text[], -1, 0)) - SELECT citus_internal_add_object_metadata(typetext, objnames, objargs, distargumentindex, colocationid) FROM distributed_object_data; + WITH distributed_object_data(typetext, objnames, objargs, distargumentindex, colocationid, force_pushdown) + AS (VALUES ('non_existing_type', ARRAY['non_existing_user']::text[], ARRAY[]::text[], -1, 0, false)) + SELECT citus_internal_add_object_metadata(typetext, objnames, objargs, distargumentindex, colocationid, force_pushdown) FROM distributed_object_data; ROLLBACK; -- check the sanity of distributionArgumentIndex and colocationId @@ -356,18 +356,18 @@ BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED; SELECT assign_distributed_transaction_id(0, 8, '2021-07-09 15:41:55.542377+02'); SET application_name to 'citus'; \set VERBOSITY terse - WITH distributed_object_data(typetext, objnames, objargs, distargumentindex, colocationid) - AS (VALUES ('role', ARRAY['metadata_sync_helper_role']::text[], ARRAY[]::text[], -100, 0)) - SELECT citus_internal_add_object_metadata(typetext, objnames, objargs, distargumentindex, colocationid) FROM distributed_object_data; + WITH distributed_object_data(typetext, objnames, objargs, distargumentindex, colocationid, force_pushdown) + AS (VALUES ('role', ARRAY['metadata_sync_helper_role']::text[], ARRAY[]::text[], -100, 0, false)) + SELECT citus_internal_add_object_metadata(typetext, objnames, objargs, distargumentindex, colocationid, force_pushdown) FROM distributed_object_data; ROLLBACK; BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED; SELECT assign_distributed_transaction_id(0, 8, '2021-07-09 15:41:55.542377+02'); SET application_name to 'citus'; \set VERBOSITY terse - WITH distributed_object_data(typetext, objnames, objargs, distargumentindex, colocationid) - AS (VALUES ('role', ARRAY['metadata_sync_helper_role']::text[], ARRAY[]::text[], -1, -1)) - SELECT citus_internal_add_object_metadata(typetext, objnames, objargs, distargumentindex, colocationid) FROM distributed_object_data; + WITH distributed_object_data(typetext, objnames, objargs, distargumentindex, colocationid, force_pushdown) + AS (VALUES ('role', ARRAY['metadata_sync_helper_role']::text[], ARRAY[]::text[], -1, -1, false)) + SELECT citus_internal_add_object_metadata(typetext, objnames, objargs, distargumentindex, colocationid, force_pushdown) FROM distributed_object_data; ROLLBACK; -- check with non-existing object @@ -375,9 +375,9 @@ BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED; SELECT assign_distributed_transaction_id(0, 8, '2021-07-09 15:41:55.542377+02'); SET application_name to 'citus'; \set VERBOSITY terse - WITH distributed_object_data(typetext, objnames, objargs, distargumentindex, colocationid) - AS (VALUES ('role', ARRAY['non_existing_user']::text[], ARRAY[]::text[], -1, 0)) - SELECT citus_internal_add_object_metadata(typetext, objnames, objargs, distargumentindex, colocationid) FROM distributed_object_data; + WITH distributed_object_data(typetext, objnames, objargs, distargumentindex, colocationid, force_pushdown) + AS (VALUES ('role', ARRAY['non_existing_user']::text[], ARRAY[]::text[], -1, 0, false)) + SELECT citus_internal_add_object_metadata(typetext, objnames, objargs, distargumentindex, colocationid, force_pushdown) FROM distributed_object_data; ROLLBACK; -- since citus_internal_add_object_metadata is strict function returns NULL @@ -386,9 +386,9 @@ BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED; SELECT assign_distributed_transaction_id(0, 8, '2021-07-09 15:41:55.542377+02'); SET application_name to 'citus'; \set VERBOSITY terse - WITH distributed_object_data(typetext, objnames, objargs, distargumentindex, colocationid) - AS (VALUES ('role', ARRAY['metadata_sync_helper_role']::text[], ARRAY[]::text[], 0, NULL::int)) - SELECT citus_internal_add_object_metadata(typetext, objnames, objargs, distargumentindex, colocationid) FROM distributed_object_data; + WITH distributed_object_data(typetext, objnames, objargs, distargumentindex, colocationid, force_pushdown) + AS (VALUES ('role', ARRAY['metadata_sync_helper_role']::text[], ARRAY[]::text[], 0, NULL::int, false)) + SELECT citus_internal_add_object_metadata(typetext, objnames, objargs, distargumentindex, colocationid, force_pushdown) FROM distributed_object_data; ROLLBACK; \c - postgres - :worker_1_port @@ -404,9 +404,9 @@ BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED; CREATE PUBLICATION publication_test FOR TABLE publication_test_table; SET ROLE metadata_sync_helper_role; - WITH distributed_object_data(typetext, objnames, objargs, distargumentindex, colocationid) - AS (VALUES ('publication', ARRAY['publication_test']::text[], ARRAY[]::text[], -1, 0)) - SELECT citus_internal_add_object_metadata(typetext, objnames, objargs, distargumentindex, colocationid) FROM distributed_object_data; + WITH distributed_object_data(typetext, objnames, objargs, distargumentindex, colocationid, force_pushdown) + AS (VALUES ('publication', ARRAY['publication_test']::text[], ARRAY[]::text[], -1, 0, false)) + SELECT citus_internal_add_object_metadata(typetext, objnames, objargs, distargumentindex, colocationid, force_pushdown) FROM distributed_object_data; ROLLBACK; -- Show that citus_internal_add_object_metadata checks the priviliges @@ -420,9 +420,9 @@ BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED; LANGUAGE SQL; SET ROLE metadata_sync_helper_role; - WITH distributed_object_data(typetext, objnames, objargs, distargumentindex, colocationid) - AS (VALUES ('function', ARRAY['distribution_test_function']::text[], ARRAY['integer']::text[], -1, 0)) - SELECT citus_internal_add_object_metadata(typetext, objnames, objargs, distargumentindex, colocationid) FROM distributed_object_data; + WITH distributed_object_data(typetext, objnames, objargs, distargumentindex, colocationid, force_pushdown) + AS (VALUES ('function', ARRAY['distribution_test_function']::text[], ARRAY['integer']::text[], -1, 0, false)) + SELECT citus_internal_add_object_metadata(typetext, objnames, objargs, distargumentindex, colocationid, force_pushdown) FROM distributed_object_data; ROLLBACK; -- we do not allow wrong partmethod diff --git a/src/test/regress/sql/multi_metadata_attributes.sql b/src/test/regress/sql/multi_metadata_attributes.sql index e285db192..5cb1a89a9 100644 --- a/src/test/regress/sql/multi_metadata_attributes.sql +++ b/src/test/regress/sql/multi_metadata_attributes.sql @@ -9,5 +9,6 @@ SELECT attrelid::regclass, attname, atthasmissing, attmissingval FROM pg_attribute WHERE atthasmissing AND attrelid NOT IN ('pg_dist_node'::regclass, 'pg_dist_rebalance_strategy'::regclass, - 'pg_dist_partition'::regclass) + 'pg_dist_partition'::regclass, + 'citus.pg_dist_object'::regclass) ORDER BY attrelid, attname; diff --git a/src/test/regress/sql/multi_mx_call.sql b/src/test/regress/sql/multi_mx_call.sql index 7df194ea7..4728b8948 100644 --- a/src/test/regress/sql/multi_mx_call.sql +++ b/src/test/regress/sql/multi_mx_call.sql @@ -100,9 +100,18 @@ BEGIN x := (select case groupid when 0 then 'F' else 'S' end from pg_dist_local_group); END;$$; +CREATE PROCEDURE mx_call_proc_copy(x int) +LANGUAGE plpgsql AS $$ +BEGIN + INSERT INTO multi_mx_call.mx_call_dist_table_1 + SELECT s,s FROM generate_series(100, 110) s; +END;$$; + + -- Test that undistributed procedures have no issue executing call multi_mx_call.mx_call_proc(2, 0); call multi_mx_call.mx_call_proc_custom_types('S', 'A'); +call multi_mx_call.mx_call_proc_copy(2); -- Same for unqualified names call mx_call_proc(2, 0); @@ -112,6 +121,7 @@ call mx_call_proc_custom_types('S', 'A'); select create_distributed_function('mx_call_proc(int,int)'); select create_distributed_function('mx_call_proc_bigint(bigint,bigint)'); select create_distributed_function('mx_call_proc_custom_types(mx_call_enum,mx_call_enum)'); +select create_distributed_function('mx_call_proc_copy(int)'); -- We still don't route them to the workers, because they aren't -- colocated with any distributed tables. @@ -119,16 +129,19 @@ SET client_min_messages TO DEBUG1; call multi_mx_call.mx_call_proc(2, 0); call mx_call_proc_bigint(4, 2); call multi_mx_call.mx_call_proc_custom_types('S', 'A'); +call multi_mx_call.mx_call_proc_copy(2); -- Mark them as colocated with a table. Now we should route them to workers. select colocate_proc_with_table('mx_call_proc', 'mx_call_dist_table_1'::regclass, 1); select colocate_proc_with_table('mx_call_proc_bigint', 'mx_call_dist_table_bigint'::regclass, 1); select colocate_proc_with_table('mx_call_proc_custom_types', 'mx_call_dist_table_enum'::regclass, 1); +select colocate_proc_with_table('mx_call_proc_copy', 'mx_call_dist_table_1'::regclass, 0); call multi_mx_call.mx_call_proc(2, 0); call multi_mx_call.mx_call_proc_custom_types('S', 'A'); call mx_call_proc(2, 0); call mx_call_proc_custom_types('S', 'A'); +call mx_call_proc_copy(2); -- Test implicit cast of int to bigint call mx_call_proc_bigint(4, 2); diff --git a/src/test/regress/sql/multi_mx_function_call_delegation.sql b/src/test/regress/sql/multi_mx_function_call_delegation.sql index b2d26e853..4dfe91322 100644 --- a/src/test/regress/sql/multi_mx_function_call_delegation.sql +++ b/src/test/regress/sql/multi_mx_function_call_delegation.sql @@ -67,9 +67,21 @@ BEGIN x := (select case groupid when 0 then 'F' else 'S' end from pg_dist_local_group); END;$$; +-- function which internally uses COPY protocol without remote execution +CREATE FUNCTION mx_call_func_copy(x int) +RETURNS bool +LANGUAGE plpgsql AS $$ +BEGIN + INSERT INTO multi_mx_function_call_delegation.mx_call_dist_table_1 + SELECT s,s FROM generate_series(100, 110) s; + + RETURN true; +END;$$; + -- Test that undistributed functions have no issue executing select multi_mx_function_call_delegation.mx_call_func(2, 0); select multi_mx_function_call_delegation.mx_call_func_custom_types('S', 'A'); +select multi_mx_function_call_delegation.mx_call_copy(2); select squares(4); -- Same for unqualified name @@ -79,6 +91,7 @@ select mx_call_func(2, 0); select create_distributed_function('mx_call_func(int,int)'); select create_distributed_function('mx_call_func_bigint(bigint,bigint)'); select create_distributed_function('mx_call_func_custom_types(mx_call_enum,mx_call_enum)'); +select create_distributed_function('mx_call_func_copy(int)'); select create_distributed_function('squares(int)'); @@ -249,10 +262,15 @@ select mx_call_func(floor(random())::int, 2); -- test forms we don't distribute select * from mx_call_func(2, 0); -select mx_call_func(2, 0) from mx_call_dist_table_1; select mx_call_func(2, 0) where mx_call_func(0, 2) = 0; select mx_call_func(2, 0), mx_call_func(0, 2); +-- we do not delegate the call, but do push down the query +-- that result in remote execution from workers +select mx_call_func(id, 0) from mx_call_dist_table_1; +select mx_call_func(2, 0) from mx_call_dist_table_1 where id = 3; +select mx_call_func_copy(2) from mx_call_dist_table_1 where id = 3; + DO $$ BEGIN perform mx_call_func_tbl(40); END; $$; SELECT * FROM mx_call_dist_table_1 WHERE id >= 40 ORDER BY id, val;