Merge branch 'master' into avoid-exceptional-control-flow-in-fluent-py

pull/5722/head
Philip Dubé 2022-02-18 16:10:45 +00:00 committed by GitHub
commit 3d044dc543
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
121 changed files with 2139 additions and 757 deletions

View File

@ -37,7 +37,7 @@ static char * CreateCollationDDLInternal(Oid collationId, Oid *collowner,
char **quotedCollationName);
static List * FilterNameListForDistributedCollations(List *objects, bool missing_ok,
List **addresses);
static bool ShouldPropagateDefineCollationStmt(void);
/*
* GetCreateCollationDDLInternal returns a CREATE COLLATE sql string for the
@ -519,6 +519,26 @@ DefineCollationStmtObjectAddress(Node *node, bool missing_ok)
}
/*
* PreprocessDefineCollationStmt executed before the collation has been
* created locally to ensure that if the collation create statement will
* be propagated, the node is a coordinator node
*/
List *
PreprocessDefineCollationStmt(Node *node, const char *queryString,
ProcessUtilityContext processUtilityContext)
{
Assert(castNode(DefineStmt, node)->kind == OBJECT_COLLATION);
if (ShouldPropagateDefineCollationStmt())
{
EnsureCoordinator();
}
return NIL;
}
/*
* PostprocessDefineCollationStmt executed after the collation has been
* created locally and before we create it on the worker nodes.
@ -531,16 +551,7 @@ PostprocessDefineCollationStmt(Node *node, const char *queryString)
{
Assert(castNode(DefineStmt, node)->kind == OBJECT_COLLATION);
if (!ShouldPropagate())
{
return NIL;
}
/*
* If the create collation command is a part of a multi-statement transaction,
* do not propagate it
*/
if (IsMultiStatementTransaction())
if (!ShouldPropagateDefineCollationStmt())
{
return NIL;
}
@ -548,13 +559,38 @@ PostprocessDefineCollationStmt(Node *node, const char *queryString)
ObjectAddress collationAddress =
DefineCollationStmtObjectAddress(node, false);
if (IsObjectDistributed(&collationAddress))
{
EnsureCoordinator();
}
EnsureDependenciesExistOnAllNodes(&collationAddress);
return NodeDDLTaskList(NON_COORDINATOR_NODES, CreateCollationDDLsIdempotent(
/* to prevent recursion with mx we disable ddl propagation */
List *commands = list_make1(DISABLE_DDL_PROPAGATION);
commands = list_concat(commands, CreateCollationDDLsIdempotent(
collationAddress.objectId));
commands = lappend(commands, ENABLE_DDL_PROPAGATION);
return NodeDDLTaskList(NON_COORDINATOR_NODES, commands);
}
/*
* ShouldPropagateDefineCollationStmt checks if collation define
* statement should be propagated. Don't propagate if:
* - metadata syncing if off
* - statement is part of a multi stmt transaction and the multi shard connection
* type is not sequential
*/
static bool
ShouldPropagateDefineCollationStmt()
{
if (!ShouldPropagate())
{
return false;
}
if (IsMultiStatementTransaction() &&
MultiShardConnectionType != SEQUENTIAL_CONNECTION)
{
return false;
}
return true;
}

View File

@ -175,7 +175,7 @@ static DistributeObjectOps Any_CreateFunction = {
.preprocess = PreprocessCreateFunctionStmt,
.postprocess = PostprocessCreateFunctionStmt,
.address = CreateFunctionStmtObjectAddress,
.markDistributed = false,
.markDistributed = true,
};
static DistributeObjectOps Any_CreatePolicy = {
.deparse = NULL,
@ -276,7 +276,7 @@ static DistributeObjectOps Collation_AlterOwner = {
static DistributeObjectOps Collation_Define = {
.deparse = NULL,
.qualify = NULL,
.preprocess = NULL,
.preprocess = PreprocessDefineCollationStmt,
.postprocess = PostprocessDefineCollationStmt,
.address = DefineCollationStmtObjectAddress,
.markDistributed = true,

View File

@ -25,6 +25,7 @@
#include "access/htup_details.h"
#include "access/xact.h"
#include "catalog/pg_aggregate.h"
#include "catalog/dependency.h"
#include "catalog/namespace.h"
#include "catalog/pg_proc.h"
#include "catalog/pg_type.h"
@ -38,6 +39,7 @@
#include "distributed/listutils.h"
#include "distributed/maintenanced.h"
#include "distributed/metadata_utility.h"
#include "distributed/metadata/dependency.h"
#include "distributed/coordinator_protocol.h"
#include "distributed/metadata/distobject.h"
#include "distributed/metadata/pg_dist_object.h"
@ -80,6 +82,7 @@ static void EnsureFunctionCanBeColocatedWithTable(Oid functionOid, Oid
static bool ShouldPropagateCreateFunction(CreateFunctionStmt *stmt);
static bool ShouldPropagateAlterFunction(const ObjectAddress *address);
static bool ShouldAddFunctionSignature(FunctionParameterMode mode);
static ObjectAddress * GetUndistributableDependency(ObjectAddress *functionAddress);
static ObjectAddress FunctionToObjectAddress(ObjectType objectType,
ObjectWithArgs *objectWithArgs,
bool missing_ok);
@ -759,7 +762,7 @@ UpdateFunctionDistributionInfo(const ObjectAddress *distAddress,
/*
* GetFunctionDDLCommand returns the complete "CREATE OR REPLACE FUNCTION ..." statement for
* the specified function followed by "ALTER FUNCTION .. SET OWNER ..".
* the specified function.
*
* useCreateOrReplace is ignored for non-aggregate functions.
*/
@ -1170,46 +1173,23 @@ GetAggregateDDLCommand(const RegProcedure funcOid, bool useCreateOrReplace)
/*
* ShouldPropagateCreateFunction tests if we need to propagate a CREATE FUNCTION
* statement. We only propagate replace's of distributed functions to keep the function on
* the workers in sync with the one on the coordinator.
* statement.
*/
static bool
ShouldPropagateCreateFunction(CreateFunctionStmt *stmt)
{
if (creating_extension)
if (!ShouldPropagate())
{
/*
* extensions should be created separately on the workers, functions cascading
* from an extension should therefore not be propagated.
*/
return false;
}
if (!EnableMetadataSync)
{
/*
* we are configured to disable object propagation, should not propagate anything
*/
return false;
}
if (!stmt->replace)
{
/*
* Since we only care for a replace of distributed functions if the statement is
* not a replace we are going to ignore.
*/
return false;
}
/*
* Even though its a replace we should accept an non-existing function, it will just
* not be distributed
* If the create command is a part of a multi-statement transaction that is not in
* sequential mode, don't propagate.
*/
ObjectAddress address = GetObjectAddressFromParseTree((Node *) stmt, true);
if (!IsObjectDistributed(&address))
if (IsMultiStatementTransaction() &&
MultiShardConnectionType != SEQUENTIAL_CONNECTION)
{
/* do not propagate alter function for non-distributed functions */
return false;
}
@ -1253,12 +1233,10 @@ ShouldPropagateAlterFunction(const ObjectAddress *address)
/*
* PreprocessCreateFunctionStmt is called during the planning phase for CREATE [OR REPLACE]
* FUNCTION. We primarily care for the replace variant of this statement to keep
* distributed functions in sync. We bail via a check on ShouldPropagateCreateFunction
* which checks for the OR REPLACE modifier.
* FUNCTION before it is created on the local node internally.
*
* Since we use pg_get_functiondef to get the ddl command we actually do not do any
* planning here, instead we defer the plan creation to the processing step.
* planning here, instead we defer the plan creation to the postprocessing step.
*
* Instead we do our basic housekeeping where we make sure we are on the coordinator and
* can propagate the function in sequential mode.
@ -1279,7 +1257,7 @@ PreprocessCreateFunctionStmt(Node *node, const char *queryString,
EnsureSequentialMode(OBJECT_FUNCTION);
/*
* ddl jobs will be generated during the Processing phase as we need the function to
* ddl jobs will be generated during the postprocessing phase as we need the function to
* be updated in the catalog to get its sql representation
*/
return NIL;
@ -1290,6 +1268,11 @@ PreprocessCreateFunctionStmt(Node *node, const char *queryString,
* PostprocessCreateFunctionStmt actually creates the plan we need to execute for function
* propagation. This is the downside of using pg_get_functiondef to get the sql statement.
*
* If function depends on any non-distributed relation (except sequence and composite type),
* Citus can not distribute it. In order to not to prevent users from creating local
* functions on the coordinator WARNING message will be sent to the customer about the case
* instead of erroring out.
*
* Besides creating the plan we also make sure all (new) dependencies of the function are
* created on all nodes.
*/
@ -1303,18 +1286,113 @@ PostprocessCreateFunctionStmt(Node *node, const char *queryString)
return NIL;
}
ObjectAddress address = GetObjectAddressFromParseTree((Node *) stmt, false);
EnsureDependenciesExistOnAllNodes(&address);
ObjectAddress functionAddress = GetObjectAddressFromParseTree((Node *) stmt, false);
List *commands = list_make4(DISABLE_DDL_PROPAGATION,
GetFunctionDDLCommand(address.objectId, true),
GetFunctionAlterOwnerCommand(address.objectId),
ENABLE_DDL_PROPAGATION);
if (IsObjectAddressOwnedByExtension(&functionAddress, NULL))
{
return NIL;
}
/*
* This check should have been valid for all objects not only for functions. Though,
* we do this limited check for now as functions are more likely to be used with
* such dependencies, and we want to scope it for now.
*/
ObjectAddress *undistributableDependency = GetUndistributableDependency(
&functionAddress);
if (undistributableDependency != NULL)
{
if (SupportedDependencyByCitus(undistributableDependency))
{
/*
* Citus can't distribute some relations as dependency, although those
* types as supported by Citus. So we can use get_rel_name directly
*/
RangeVar *functionRangeVar = makeRangeVarFromNameList(stmt->funcname);
char *functionName = functionRangeVar->relname;
char *dependentRelationName =
get_rel_name(undistributableDependency->objectId);
ereport(WARNING, (errmsg("Citus can't distribute function \"%s\" having "
"dependency on non-distributed relation \"%s\"",
functionName, dependentRelationName),
errdetail("Function will be created only locally"),
errhint("To distribute function, distribute dependent "
"relations first. Then, re-create the function")));
}
else
{
char *objectType = NULL;
#if PG_VERSION_NUM >= PG_VERSION_14
objectType = getObjectTypeDescription(undistributableDependency, false);
#else
objectType = getObjectTypeDescription(undistributableDependency);
#endif
ereport(WARNING, (errmsg("Citus can't distribute functions having "
"dependency on unsupported object of type \"%s\"",
objectType),
errdetail("Function will be created only locally")));
}
return NIL;
}
EnsureDependenciesExistOnAllNodes(&functionAddress);
List *commands = list_make1(DISABLE_DDL_PROPAGATION);
commands = list_concat(commands, CreateFunctionDDLCommandsIdempotent(
&functionAddress));
commands = list_concat(commands, list_make1(ENABLE_DDL_PROPAGATION));
return NodeDDLTaskList(NON_COORDINATOR_NODES, commands);
}
/*
* GetUndistributableDependency checks whether object has any non-distributable
* dependency. If any one found, it will be returned.
*/
static ObjectAddress *
GetUndistributableDependency(ObjectAddress *objectAddress)
{
List *dependencies = GetAllDependenciesForObject(objectAddress);
ObjectAddress *dependency = NULL;
foreach_ptr(dependency, dependencies)
{
if (IsObjectDistributed(dependency))
{
continue;
}
if (!SupportedDependencyByCitus(dependency))
{
/*
* Since roles should be handled manually with Citus community, skip them.
*/
if (getObjectClass(dependency) != OCLASS_ROLE)
{
return dependency;
}
}
if (getObjectClass(dependency) == OCLASS_CLASS)
{
/*
* Citus can only distribute dependent non-distributed sequence
* and composite types.
*/
char relKind = get_rel_relkind(dependency->objectId);
if (relKind != RELKIND_SEQUENCE && relKind != RELKIND_COMPOSITE_TYPE)
{
return dependency;
}
}
}
return NULL;
}
/*
* CreateFunctionStmtObjectAddress returns the ObjectAddress for the subject of the
* CREATE [OR REPLACE] FUNCTION statement. If missing_ok is false it will error with the

View File

@ -957,6 +957,20 @@ CreateTypeDDLCommandsIdempotent(const ObjectAddress *typeAddress)
return NIL;
}
HeapTuple tup = SearchSysCacheCopy1(TYPEOID, ObjectIdGetDatum(typeAddress->objectId));
if (!HeapTupleIsValid(tup))
{
elog(ERROR, "cache lookup failed for type %u", typeAddress->objectId);
}
/* Don't send any command if the type is a table's row type */
Form_pg_type typTup = (Form_pg_type) GETSTRUCT(tup);
if (typTup->typtype == TYPTYPE_COMPOSITE &&
get_rel_relkind(typTup->typrelid) != RELKIND_COMPOSITE_TYPE)
{
return NIL;
}
Node *stmt = CreateTypeStmtByObjectAddress(typeAddress);
/* capture ddl command for recreation and wrap in create if not exists construct */

View File

@ -237,16 +237,17 @@ 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();
InTopLevelDelegatedFunctionCall = false;
}
/*
* 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.
*/
CheckAndResetAllowedShardKeyValueIfNeeded();
}
PG_CATCH();
{
@ -260,13 +261,15 @@ CitusExecutorRun(QueryDesc *queryDesc,
if (ExecutorLevel == 0 && PlannerLevel == 0)
{
/*
* In case of an exception, reset the pinned shard-key, for more
* details see the function header.
*/
ResetAllowedShardKeyValue();
InTopLevelDelegatedFunctionCall = false;
}
/*
* In case of an exception, reset the pinned shard-key, for more
* details see the function header.
*/
CheckAndResetAllowedShardKeyValueIfNeeded();
PG_RE_THROW();
}
PG_END_TRY();

View File

@ -156,6 +156,8 @@ static bool FollowAllSupportedDependencies(ObjectAddressCollector *collector,
DependencyDefinition *definition);
static bool FollowNewSupportedDependencies(ObjectAddressCollector *collector,
DependencyDefinition *definition);
static bool FollowAllDependencies(ObjectAddressCollector *collector,
DependencyDefinition *definition);
static void ApplyAddToDependencyList(ObjectAddressCollector *collector,
DependencyDefinition *definition);
static List * ExpandCitusSupportedTypes(ObjectAddressCollector *collector,
@ -212,15 +214,42 @@ GetDependenciesForObject(const ObjectAddress *target)
/*
* GetAllDependenciesForObject returns a list of all the ObjectAddresses to be
* created in order before the target object could safely be created on a
* worker. As a caller, you probably need GetDependenciesForObject() which
* eliminates already distributed objects from the returned list.
* GetAllSupportedDependenciesForObject returns a list of all the ObjectAddresses to be
* created in order before the target object could safely be created on a worker, if all
* dependent objects are distributable. As a caller, you probably need to use
* GetDependenciesForObject() which eliminates already distributed objects from the returned
* list.
*
* Some of the object might already be created on a worker. It should be created
* in an idempotent way.
*/
List *
GetAllSupportedDependenciesForObject(const ObjectAddress *target)
{
ObjectAddressCollector collector = { 0 };
InitObjectAddressCollector(&collector);
RecurseObjectDependencies(*target,
&ExpandCitusSupportedTypes,
&FollowAllSupportedDependencies,
&ApplyAddToDependencyList,
&collector);
return collector.dependencyList;
}
/*
* GetAllDependenciesForObject returns a list of all the dependent objects of the given
* object irrespective of whether the dependent object is supported by Citus or not, if
* the object can be found as dependency with RecurseObjectDependencies and
* ExpandCitusSupportedTypes.
*
* This function will be used to provide meaningful error messages if any dependent
* object for a given object is not supported. If you want to create dependencies for
* an object, you probably need to use GetDependenciesForObject().
*/
List *
GetAllDependenciesForObject(const ObjectAddress *target)
{
ObjectAddressCollector collector = { 0 };
@ -228,7 +257,7 @@ GetAllDependenciesForObject(const ObjectAddress *target)
RecurseObjectDependencies(*target,
&ExpandCitusSupportedTypes,
&FollowAllSupportedDependencies,
&FollowAllDependencies,
&ApplyAddToDependencyList,
&collector);
@ -903,10 +932,61 @@ FollowAllSupportedDependencies(ObjectAddressCollector *collector,
/*
* ApplyAddToDependencyList is an apply function for RecurseObjectDependencies that will collect
* all the ObjectAddresses for pg_depend entries to the context. The context here is
* assumed to be a (ObjectAddressCollector *) to the location where all ObjectAddresses
* will be collected.
* FollowAllDependencies applies filters on pg_depend entries to follow the dependency
* tree of objects in depth first order. We will visit all objects irrespective of it is
* supported by Citus or not.
*/
static bool
FollowAllDependencies(ObjectAddressCollector *collector,
DependencyDefinition *definition)
{
if (definition->mode == DependencyPgDepend)
{
/*
* For dependencies found in pg_depend:
*
* Follow only normal and extension dependencies. The latter is used to reach the
* extensions, the objects that directly depend on the extension are eliminated
* during the "apply" phase.
*
* Other dependencies are internal dependencies and managed by postgres.
*/
if (definition->data.pg_depend.deptype != DEPENDENCY_NORMAL &&
definition->data.pg_depend.deptype != DEPENDENCY_EXTENSION)
{
return false;
}
}
/* rest of the tests are to see if we want to follow the actual dependency */
ObjectAddress address = DependencyDefinitionObjectAddress(definition);
/*
* If the object is already in our dependency list we do not have to follow any
* further
*/
if (IsObjectAddressCollected(address, collector))
{
return false;
}
if (CitusExtensionObject(&address))
{
/* following citus extension could complicate role management */
return false;
}
return true;
}
/*
* ApplyAddToDependencyList is an apply function for RecurseObjectDependencies that will
* collect all the ObjectAddresses for pg_depend entries to the context, except it is
* extension owned one.
*
* The context here is assumed to be a (ObjectAddressCollector *) to the location where
* all ObjectAddresses will be collected.
*/
static void
ApplyAddToDependencyList(ObjectAddressCollector *collector,

View File

@ -723,6 +723,16 @@ FunctionInFromClause(List *fromlist, Query *query)
static void
EnableInForceDelegatedFuncExecution(Const *distArgument, uint32 colocationId)
{
/*
* If the distribution key is already set, the key is fixed until
* the force-delegation function returns. All nested force-delegation
* functions must use the same key.
*/
if (AllowedDistributionColumnValue.isActive)
{
return;
}
/*
* The saved distribution argument need to persist through the life
* of the query, both during the planning (where we save) and execution
@ -734,6 +744,7 @@ EnableInForceDelegatedFuncExecution(Const *distArgument, uint32 colocationId)
colocationId));
AllowedDistributionColumnValue.distributionColumnValue = copyObject(distArgument);
AllowedDistributionColumnValue.colocationId = colocationId;
AllowedDistributionColumnValue.executorLevel = ExecutorLevel;
AllowedDistributionColumnValue.isActive = true;
MemoryContextSwitchTo(oldcontext);
}
@ -747,15 +758,22 @@ EnableInForceDelegatedFuncExecution(Const *distArgument, uint32 colocationId)
* the 2PC. Reset the distribution argument value once the function ends.
*/
void
ResetAllowedShardKeyValue(void)
CheckAndResetAllowedShardKeyValueIfNeeded(void)
{
if (AllowedDistributionColumnValue.isActive)
/*
* If no distribution argument is pinned or the pinned argument was
* set by a nested-executor from upper level, nothing to reset.
*/
if (!AllowedDistributionColumnValue.isActive ||
ExecutorLevel > AllowedDistributionColumnValue.executorLevel)
{
pfree(AllowedDistributionColumnValue.distributionColumnValue);
AllowedDistributionColumnValue.isActive = false;
return;
}
InTopLevelDelegatedFunctionCall = false;
Assert(ExecutorLevel == AllowedDistributionColumnValue.executorLevel);
pfree(AllowedDistributionColumnValue.distributionColumnValue);
AllowedDistributionColumnValue.isActive = false;
AllowedDistributionColumnValue.executorLevel = 0;
}
@ -767,6 +785,7 @@ bool
IsShardKeyValueAllowed(Const *shardKey, uint32 colocationId)
{
Assert(AllowedDistributionColumnValue.isActive);
Assert(ExecutorLevel > AllowedDistributionColumnValue.executorLevel);
ereport(DEBUG4, errmsg("Comparing saved:%s with Shard key: %s colocationid:%d:%d",
pretty_format_node_dump(

View File

@ -47,7 +47,7 @@ citus_get_all_dependencies_for_object(PG_FUNCTION_ARGS)
ObjectAddress address = { 0 };
ObjectAddressSubSet(address, classid, objid, objsubid);
List *dependencies = GetAllDependenciesForObject(&address);
List *dependencies = GetAllSupportedDependenciesForObject(&address);
ObjectAddress *dependency = NULL;
foreach_ptr(dependency, dependencies)
{

View File

@ -557,7 +557,8 @@ ResetGlobalVariables()
MetadataSyncOnCommit = false;
InTopLevelDelegatedFunctionCall = false;
ResetWorkerErrorIndication();
AllowedDistributionColumnValue.isActive = false;
memset(&AllowedDistributionColumnValue, 0,
sizeof(AllowedDistributionColumn));
}

View File

@ -151,6 +151,8 @@ extern ObjectAddress AlterCollationSchemaStmtObjectAddress(Node *stmt,
extern List * PostprocessAlterCollationSchemaStmt(Node *stmt, const char *queryString);
extern char * GenerateBackupNameForCollationCollision(const ObjectAddress *address);
extern ObjectAddress DefineCollationStmtObjectAddress(Node *stmt, bool missing_ok);
extern List * PreprocessDefineCollationStmt(Node *stmt, const char *queryString,
ProcessUtilityContext processUtilityContext);
extern List * PostprocessDefineCollationStmt(Node *stmt, const char *queryString);
/* database.c - forward declarations */

View File

@ -23,7 +23,7 @@ extern bool InTopLevelDelegatedFunctionCall;
extern bool InDelegatedProcedureCall;
PlannedStmt * TryToDelegateFunctionCall(DistributedPlanningContext *planContext);
extern void ResetAllowedShardKeyValue(void);
extern void CheckAndResetAllowedShardKeyValueIfNeeded(void);
extern bool IsShardKeyValueAllowed(Const *shardKey, uint32 colocationId);
#endif /* FUNCTION_CALL_DELEGATION_H */

View File

@ -19,6 +19,7 @@
extern List * GetUniqueDependenciesList(List *objectAddressesList);
extern List * GetDependenciesForObject(const ObjectAddress *target);
extern List * GetAllSupportedDependenciesForObject(const ObjectAddress *target);
extern List * GetAllDependenciesForObject(const ObjectAddress *target);
extern List * OrderObjectAddressListInDependencyOrder(List *objectAddressList);
extern bool SupportedDependencyByCitus(const ObjectAddress *address);

View File

@ -70,6 +70,9 @@ typedef struct AllowedDistributionColumn
Const *distributionColumnValue;
uint32 colocationId;
bool isActive;
/* In nested executor, track the level at which value is set */
int executorLevel;
} AllowedDistributionColumn;
/*

View File

@ -875,6 +875,7 @@ BEGIN
RETURN $1 * $1;
END;
$function$;
SET citus.enable_metadata_sync TO OFF;
CREATE OR REPLACE FUNCTION square_func(int)
RETURNS int
LANGUAGE plpgsql
@ -883,6 +884,7 @@ BEGIN
RETURN $1 * $1;
END;
$function$;
RESET citus.enable_metadata_sync;
SELECT const_function(1), string_agg(a::character, ',') FROM t1;
NOTICE: stable_fn called
CONTEXT: PL/pgSQL function const_function(integer) line XX at RAISE

View File

@ -58,6 +58,7 @@ CREATE TABLE postgres_local_table(a int, b int);
-- We shouldn't use LIMIT in INSERT SELECT queries to make the test faster as
-- LIMIT would force planner to wrap SELECT query in an intermediate result and
-- this might reduce the coverage of the test cases.
SET citus.enable_metadata_sync TO OFF;
CREATE FUNCTION clear_and_init_test_tables() RETURNS void AS $$
BEGIN
SET client_min_messages to ERROR;
@ -74,6 +75,7 @@ CREATE FUNCTION clear_and_init_test_tables() RETURNS void AS $$
RESET client_min_messages;
END;
$$ LANGUAGE plpgsql;
RESET citus.enable_metadata_sync;
---------------------------------------------------------------------
---- SELECT ----
---------------------------------------------------------------------

View File

@ -523,6 +523,7 @@ BEGIN
RETURN trunc(random() * (end_int-start_int) + start_int);
END;
$$ LANGUAGE 'plpgsql' STRICT;
SET citus.enable_metadata_sync TO OFF;
CREATE OR REPLACE PROCEDURE coordinator_evaluation.test_procedure(int)
LANGUAGE plpgsql
AS $procedure$
@ -532,6 +533,7 @@ BEGIN
PERFORM DISTINCT value FROM coordinator_evaluation_table_2 WHERE key = filterKey;
END;
$procedure$;
RESET citus.enable_metadata_sync;
-- we couldn't find a meaningful query to write for this
-- however this query fails before https://github.com/citusdata/citus/pull/3454
SET client_min_messages TO DEBUG2;
@ -564,6 +566,7 @@ BEGIN
INSERT INTO coordinator_evaluation_table_2 VALUES (filterKey, filterKey);
END;
$procedure$;
DEBUG: switching to sequential query execution mode
RESET citus.log_remote_commands ;
RESET client_min_messages;
-- these calls would INSERT key = 101, so test if insert succeeded

View File

@ -833,11 +833,13 @@ EXECUTE router_with_only_function;
SET citus.log_local_commands TO ON;
SET search_path TO coordinator_evaluation_combinations_modify;
-- returns 2 on the worker
SET citus.enable_metadata_sync TO OFF;
CREATE OR REPLACE FUNCTION get_constant_stable()
RETURNS INT AS $$
BEGIN
RETURN 2;
END; $$ language plpgsql STABLE;
RESET citus.enable_metadata_sync;
-- all local values
INSERT INTO user_info_data (user_id, u_data) VALUES
(3, '(''test3'', 3)'), (4, '(''test4'', 4)'), (7, '(''test7'', 7)'),

View File

@ -898,9 +898,11 @@ NOTICE: executing the command locally: SELECT count(*) AS count FROM (SELECT in
-- a helper function which return true if the coordinated
-- trannsaction uses 2PC
SET citus.enable_metadata_sync TO OFF;
CREATE OR REPLACE FUNCTION coordinated_transaction_should_use_2PC()
RETURNS BOOL LANGUAGE C STRICT VOLATILE AS 'citus',
$$coordinated_transaction_should_use_2PC$$;
RESET citus.enable_metadata_sync;
-- a local SELECT followed by remote SELECTs
-- does not trigger 2PC
BEGIN;

View File

@ -163,3 +163,19 @@ SELECT run_command_on_workers($$DROP USER collationuser;$$);
(localhost,57638,t,"DROP ROLE")
(2 rows)
\c - - - :worker_1_port
-- test creating a collation on a worker
CREATE COLLATION another_german_phonebook (provider = icu, locale = 'de-u-co-phonebk');
ERROR: operation is not allowed on this node
HINT: Connect to the coordinator and run it again.
-- test if creating a collation on a worker on a local
-- schema raises the right error
SET citus.enable_ddl_propagation TO off;
CREATE SCHEMA collation_creation_on_worker;
SET citus.enable_ddl_propagation TO on;
CREATE COLLATION collation_creation_on_worker.another_german_phonebook (provider = icu, locale = 'de-u-co-phonebk');
ERROR: operation is not allowed on this node
HINT: Connect to the coordinator and run it again.
SET citus.enable_ddl_propagation TO off;
DROP SCHEMA collation_creation_on_worker;
SET citus.enable_ddl_propagation TO on;

View File

@ -3,6 +3,7 @@
CREATE SCHEMA proc_conflict;
\c - - - :worker_1_port
SET search_path TO proc_conflict;
SET citus.enable_metadata_sync TO OFF;
CREATE FUNCTION existing_func(state int, i int) RETURNS int AS $$
BEGIN
RETURN state * 2 + i;
@ -12,6 +13,7 @@ CREATE AGGREGATE existing_agg(int) (
SFUNC = existing_func,
STYPE = int
);
RESET citus.enable_metadata_sync;
\c - - - :master_port
SET search_path TO proc_conflict;
CREATE FUNCTION existing_func(state int, i int) RETURNS int AS $$
@ -62,6 +64,7 @@ DROP AGGREGATE existing_agg(int) CASCADE;
DROP FUNCTION existing_func(int, int) CASCADE;
\c - - - :worker_1_port
SET search_path TO proc_conflict;
SET citus.enable_metadata_sync TO OFF;
CREATE FUNCTION existing_func(state int, i int) RETURNS int AS $$
BEGIN
RETURN state * 3 + i;
@ -71,6 +74,7 @@ CREATE AGGREGATE existing_agg(int) (
SFUNC = existing_func,
STYPE = int
);
RESET citus.enable_metadata_sync;
\c - - - :master_port
SET search_path TO proc_conflict;
CREATE FUNCTION existing_func(state int, i int) RETURNS int AS $$

View File

@ -228,7 +228,10 @@ BEGIN
DELETE FROM modify_fast_path WHERE key = $1 AND value_1 = $2;
END;
$$ LANGUAGE plpgsql;
DEBUG: switching to sequential query execution mode
DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands
SELECT modify_fast_path_plpsql(1,1);
DEBUG: function does not have co-located tables
DEBUG: Deferred pruning for a fast-path router query
CONTEXT: SQL statement "DELETE FROM modify_fast_path WHERE key = $1 AND value_1 = $2"
PL/pgSQL function modify_fast_path_plpsql(integer,integer) line XX at SQL statement
@ -241,6 +244,7 @@ PL/pgSQL function modify_fast_path_plpsql(integer,integer) line XX at SQL statem
(1 row)
SELECT modify_fast_path_plpsql(2,2);
DEBUG: function does not have co-located tables
DEBUG: Deferred pruning for a fast-path router query
CONTEXT: SQL statement "DELETE FROM modify_fast_path WHERE key = $1 AND value_1 = $2"
PL/pgSQL function modify_fast_path_plpsql(integer,integer) line XX at SQL statement
@ -253,6 +257,7 @@ PL/pgSQL function modify_fast_path_plpsql(integer,integer) line XX at SQL statem
(1 row)
SELECT modify_fast_path_plpsql(3,3);
DEBUG: function does not have co-located tables
DEBUG: Deferred pruning for a fast-path router query
CONTEXT: SQL statement "DELETE FROM modify_fast_path WHERE key = $1 AND value_1 = $2"
PL/pgSQL function modify_fast_path_plpsql(integer,integer) line XX at SQL statement
@ -265,6 +270,7 @@ PL/pgSQL function modify_fast_path_plpsql(integer,integer) line XX at SQL statem
(1 row)
SELECT modify_fast_path_plpsql(4,4);
DEBUG: function does not have co-located tables
DEBUG: Deferred pruning for a fast-path router query
CONTEXT: SQL statement "DELETE FROM modify_fast_path WHERE key = $1 AND value_1 = $2"
PL/pgSQL function modify_fast_path_plpsql(integer,integer) line XX at SQL statement
@ -277,6 +283,7 @@ PL/pgSQL function modify_fast_path_plpsql(integer,integer) line XX at SQL statem
(1 row)
SELECT modify_fast_path_plpsql(5,5);
DEBUG: function does not have co-located tables
DEBUG: Deferred pruning for a fast-path router query
CONTEXT: SQL statement "DELETE FROM modify_fast_path WHERE key = $1 AND value_1 = $2"
PL/pgSQL function modify_fast_path_plpsql(integer,integer) line XX at SQL statement
@ -289,6 +296,7 @@ PL/pgSQL function modify_fast_path_plpsql(integer,integer) line XX at SQL statem
(1 row)
SELECT modify_fast_path_plpsql(6,6);
DEBUG: function does not have co-located tables
DEBUG: Deferred pruning for a fast-path router query
CONTEXT: SQL statement "DELETE FROM modify_fast_path WHERE key = $1 AND value_1 = $2"
PL/pgSQL function modify_fast_path_plpsql(integer,integer) line XX at SQL statement
@ -301,6 +309,7 @@ PL/pgSQL function modify_fast_path_plpsql(integer,integer) line XX at SQL statem
(1 row)
SELECT modify_fast_path_plpsql(6,6);
DEBUG: function does not have co-located tables
modify_fast_path_plpsql
---------------------------------------------------------------------

View File

@ -293,6 +293,8 @@ BEGIN
RETURN ret_val;
END;
$$ LANGUAGE plpgsql;
DEBUG: switching to sequential query execution mode
DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands
CREATE OR REPLACE FUNCTION func_calls_forcepush_func()
RETURNS NUMERIC AS $$
DECLARE incremented_val NUMERIC;
@ -302,6 +304,8 @@ BEGIN
RETURN incremented_val;
END;
$$ LANGUAGE plpgsql;
DEBUG: switching to sequential query execution mode
DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands
SELECT create_distributed_function('func_calls_forcepush_func()');
DEBUG: switching to sequential query execution mode
DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands
@ -354,12 +358,16 @@ PL/pgSQL function func_calls_forcepush_func() line XX at SQL statement
101
(1 row)
-- Block distributing that function as distributing it causes
-- different test output on PG 14.
SET citus.enable_metadata_sync TO OFF;
CREATE OR REPLACE FUNCTION get_val()
RETURNS INT AS $$
BEGIN
RETURN 100::INT;
END;
$$ LANGUAGE plpgsql;
RESET citus.enable_metadata_sync;
--
-- UDF calling another UDF in a FROM clause
-- fn()
@ -377,7 +385,10 @@ BEGIN
RETURN incremented_val;
END;
$$ LANGUAGE plpgsql;
DEBUG: switching to sequential query execution mode
DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands
SELECT func_calls_forcepush_func_infrom();
DEBUG: function does not have co-located tables
DEBUG: pushing down function call in a multi-statement transaction
CONTEXT: SQL statement "SELECT inner_force_delegation_function FROM inner_force_delegation_function(add_val + 100)"
PL/pgSQL function func_calls_forcepush_func_infrom() line XX at SQL statement
@ -395,6 +406,7 @@ PL/pgSQL function func_calls_forcepush_func_infrom() line XX at SQL statement
BEGIN;
SELECT func_calls_forcepush_func_infrom();
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_delegation_function FROM inner_force_delegation_function(add_val + 100)"
PL/pgSQL function func_calls_forcepush_func_infrom() line XX at SQL statement
@ -428,7 +440,10 @@ BEGIN
RETURN incremented_val;
END;
$$ LANGUAGE plpgsql;
DEBUG: switching to sequential query execution mode
DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands
SELECT func_calls_forcepush_func_intarget();
DEBUG: function does not have co-located tables
DEBUG: pushing down function call in a multi-statement transaction
CONTEXT: SQL statement "SELECT inner_force_delegation_function(100 + 100) OFFSET 0"
PL/pgSQL function func_calls_forcepush_func_intarget() line XX at SQL statement
@ -446,6 +461,7 @@ PL/pgSQL function func_calls_forcepush_func_intarget() line XX at SQL statement
BEGIN;
SELECT func_calls_forcepush_func_intarget();
DEBUG: not pushing down function calls in a multi-statement transaction
NOTICE: inner_force_delegation_function():201
DETAIL: from localhost:xxxxx
CONTEXT: SQL statement "SELECT inner_force_delegation_function(100 + 100) OFFSET 0"
@ -473,6 +489,8 @@ BEGIN
END if;
END;
$$ LANGUAGE plpgsql;
DEBUG: switching to sequential query execution mode
DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands
SELECT create_distributed_function('test_recursive(int)', '$1', colocate_with := 'test_nested', force_delegation := true);
DEBUG: switching to sequential query execution mode
DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands
@ -544,6 +562,8 @@ BEGIN
RETURN x + y;
END;
$$ LANGUAGE plpgsql;
DEBUG: switching to sequential query execution mode
DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands
SELECT create_distributed_function(
'test_non_constant(int,bigint)',
'$1',
@ -610,6 +630,8 @@ BEGIN
INSERT INTO emp VALUES (empname, 33);
END;
$$ LANGUAGE plpgsql;
DEBUG: switching to sequential query execution mode
DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands
CREATE OR REPLACE FUNCTION outer_emp()
RETURNS void
AS $$
@ -618,6 +640,8 @@ BEGIN
PERFORM inner_emp('hello');
END;
$$ LANGUAGE plpgsql;
DEBUG: switching to sequential query execution mode
DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands
SELECT create_distributed_function('inner_emp(text)','empname', force_delegation := true);
DEBUG: switching to sequential query execution mode
DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands
@ -627,6 +651,7 @@ DETAIL: A command for a distributed function is run. To make sure subsequent co
(1 row)
SELECT outer_emp();
DEBUG: function does not have co-located tables
DEBUG: Skipping pushdown of function from a PL/PgSQL simple expression
CONTEXT: SQL statement "SELECT inner_emp('hello')"
PL/pgSQL function outer_emp() line XX at PERFORM
@ -650,6 +675,8 @@ BEGIN
INSERT INTO forcepushdown_schema.test_forcepushdown SELECT(a+1);
END;
$fn$;
DEBUG: switching to sequential query execution mode
DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands
SELECT create_distributed_function(
'insert_select_data(int)', 'a',
colocate_with := 'test_forcepushdown',
@ -725,6 +752,8 @@ BEGIN
SELECT intcol FROM forcepushdown_schema.test_forcepushdown_noncolocate;
END;
$fn$;
DEBUG: switching to sequential query execution mode
DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands
SELECT create_distributed_function(
'insert_select_data_nonlocal(int)', 'a',
colocate_with := 'test_forcepushdown',
@ -803,6 +832,8 @@ BEGIN
INSERT INTO forcepushdown_schema.test_forcepushdown_char VALUES (a);
END;
$fn$;
DEBUG: switching to sequential query execution mode
DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands
SELECT create_distributed_function(
'insert_data_char(char)', 'a',
colocate_with := 'test_forcepushdown_char',
@ -821,6 +852,8 @@ BEGIN
INSERT INTO forcepushdown_schema.test_forcepushdown_varchar VALUES (a);
END;
$fn$;
DEBUG: switching to sequential query execution mode
DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands
SELECT create_distributed_function(
'insert_data_varchar(varchar)', 'a',
colocate_with := 'test_forcepushdown_varchar',
@ -839,6 +872,8 @@ BEGIN
INSERT INTO forcepushdown_schema.test_forcepushdown_text VALUES (a);
END;
$fn$;
DEBUG: switching to sequential query execution mode
DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands
SELECT create_distributed_function(
'insert_data_text(text)', 'a',
colocate_with := 'test_forcepushdown_text',
@ -947,6 +982,8 @@ BEGIN
RAISE NOTICE 'Result: %', var;
END;
$fn$;
DEBUG: switching to sequential query execution mode
DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands
SELECT create_distributed_function(
'select_data(int)', 'a',
colocate_with := 'test_subquery',
@ -969,6 +1006,8 @@ BEGIN
RAISE NOTICE 'Result: %', var;
END;
$fn$;
DEBUG: switching to sequential query execution mode
DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands
SELECT create_distributed_function(
'select_data_noncolocate(int)', 'a',
colocate_with := 'test_subquery',
@ -990,6 +1029,8 @@ BEGIN
RAISE NOTICE 'Result: %', var;
END;
$fn$;
DEBUG: switching to sequential query execution mode
DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands
SELECT create_distributed_function(
'insert_select_data_cte1(int)', 'a',
colocate_with := 'test_subquery',
@ -1011,6 +1052,8 @@ BEGIN
RAISE NOTICE 'Result: %', var;
END;
$fn$;
DEBUG: switching to sequential query execution mode
DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands
SELECT create_distributed_function(
'insert_select_data_cte2(int)', 'a',
colocate_with := 'test_subquery',
@ -1033,6 +1076,8 @@ BEGIN
RAISE NOTICE 'Result: %', var;
END;
$fn$;
DEBUG: switching to sequential query execution mode
DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands
SELECT create_distributed_function(
'insert_data_cte_nondist(int)', 'a',
colocate_with := 'test_subquery',
@ -1203,6 +1248,8 @@ BEGIN
RETURN x + y;
END;
$$ LANGUAGE plpgsql;
DEBUG: switching to sequential query execution mode
DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands
SELECT create_distributed_function('test_prepare(int,int)','x',force_delegation :=true, colocate_with := 'table_test_prepare');
DEBUG: switching to sequential query execution mode
DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands
@ -1223,9 +1270,12 @@ BEGIN
PERFORM 1, 1 + a FROM test_prepare(x + 1, y + 1) a;
END;
$$ LANGUAGE plpgsql;
DEBUG: switching to sequential query execution mode
DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands
-- First 5 get delegated and succeeds
BEGIN;
SELECT outer_test_prepare(1,1);
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 FROM test_prepare(x, y)"
PL/pgSQL function outer_test_prepare(integer,integer) line XX at PERFORM
@ -1244,6 +1294,7 @@ PL/pgSQL function outer_test_prepare(integer,integer) line XX at PERFORM
(1 row)
SELECT outer_test_prepare(1,1);
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 FROM test_prepare(x, y)"
PL/pgSQL function outer_test_prepare(integer,integer) line XX at PERFORM
@ -1262,6 +1313,7 @@ PL/pgSQL function outer_test_prepare(integer,integer) line XX at PERFORM
(1 row)
SELECT outer_test_prepare(1,1);
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 FROM test_prepare(x, y)"
PL/pgSQL function outer_test_prepare(integer,integer) line XX at PERFORM
@ -1280,6 +1332,7 @@ PL/pgSQL function outer_test_prepare(integer,integer) line XX at PERFORM
(1 row)
SELECT outer_test_prepare(1,1);
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 FROM test_prepare(x, y)"
PL/pgSQL function outer_test_prepare(integer,integer) line XX at PERFORM
@ -1298,6 +1351,7 @@ PL/pgSQL function outer_test_prepare(integer,integer) line XX at PERFORM
(1 row)
SELECT outer_test_prepare(1,1);
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 FROM test_prepare(x, y)"
PL/pgSQL function outer_test_prepare(integer,integer) line XX at PERFORM
@ -1324,6 +1378,7 @@ SELECT COUNT(*) FROM table_test_prepare;
-- 6th execution will be generic plan and should get delegated
SELECT outer_test_prepare(1,1);
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 FROM test_prepare(x, y)"
PL/pgSQL function outer_test_prepare(integer,integer) line XX at PERFORM
@ -1342,6 +1397,7 @@ PL/pgSQL function outer_test_prepare(integer,integer) line XX at PERFORM
(1 row)
SELECT outer_test_prepare(1,1);
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 FROM test_prepare(x, y)"
PL/pgSQL function outer_test_prepare(integer,integer) line XX at PERFORM
@ -1362,6 +1418,7 @@ PL/pgSQL function outer_test_prepare(integer,integer) line XX at PERFORM
END;
-- Fails as expected
SELECT outer_test_prepare(1,2);
DEBUG: function does not have co-located tables
DEBUG: pushing down function call in a multi-statement transaction
CONTEXT: SQL statement "SELECT FROM test_prepare(x, y)"
PL/pgSQL function outer_test_prepare(integer,integer) line XX at PERFORM
@ -1397,6 +1454,8 @@ BEGIN
RETURN x;
END;
$$ LANGUAGE plpgsql;
DEBUG: switching to sequential query execution mode
DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands
SELECT create_distributed_function('test(int)', 'x',
colocate_with := 'test_perform', force_delegation := true);
DEBUG: switching to sequential query execution mode
@ -1418,10 +1477,471 @@ NOTICE: INPUT 3
CONTEXT: PL/pgSQL function test(integer) line XX at RAISE
SQL statement "SELECT test(3)"
PL/pgSQL function inline_code_block line XX at PERFORM
CREATE TABLE testnested_table (x int, y int);
SELECT create_distributed_table('testnested_table','x');
create_distributed_table
---------------------------------------------------------------------
(1 row)
CREATE OR REPLACE FUNCTION inner_fn(x int)
RETURNS void
AS $$
DECLARE
BEGIN
INSERT INTO forcepushdown_schema.testnested_table VALUES (x,x);
END;
$$ LANGUAGE plpgsql;
DEBUG: switching to sequential query execution mode
DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands
-- Non-force function calling force-delegation function
CREATE OR REPLACE FUNCTION outer_local_fn()
RETURNS void
AS $$
DECLARE
BEGIN
PERFORM 1 FROM inner_fn(1);
INSERT INTO forcepushdown_schema.testnested_table VALUES (2,3);
PERFORM 1 FROM inner_fn(4);
INSERT INTO forcepushdown_schema.testnested_table VALUES (5,6);
END;
$$ LANGUAGE plpgsql;
DEBUG: switching to sequential query execution mode
DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands
SELECT create_distributed_function('inner_fn(int)','x',
colocate_with:='testnested_table', force_delegation := true);
DEBUG: switching to sequential query execution mode
DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands
create_distributed_function
---------------------------------------------------------------------
(1 row)
SELECT outer_local_fn();
DEBUG: function does not have co-located tables
DEBUG: pushing down function call in a multi-statement transaction
CONTEXT: SQL statement "SELECT 1 FROM inner_fn(1)"
PL/pgSQL function outer_local_fn() line XX at PERFORM
DEBUG: pushing down the function call
CONTEXT: SQL statement "SELECT 1 FROM inner_fn(1)"
PL/pgSQL function outer_local_fn() line XX at PERFORM
DEBUG: pushing down function call in a multi-statement transaction
CONTEXT: SQL statement "SELECT 1 FROM inner_fn(4)"
PL/pgSQL function outer_local_fn() line XX at PERFORM
DEBUG: pushing down the function call
CONTEXT: SQL statement "SELECT 1 FROM inner_fn(4)"
PL/pgSQL function outer_local_fn() line XX at PERFORM
outer_local_fn
---------------------------------------------------------------------
(1 row)
-- Rows from 1-6 should appear
SELECT * FROM testnested_table ORDER BY 1;
x | y
---------------------------------------------------------------------
1 | 1
2 | 3
4 | 4
5 | 6
(4 rows)
BEGIN;
SELECT outer_local_fn();
DEBUG: not pushing down function calls in a multi-statement transaction
outer_local_fn
---------------------------------------------------------------------
(1 row)
END;
SELECT * FROM testnested_table ORDER BY 1;
x | y
---------------------------------------------------------------------
1 | 1
1 | 1
2 | 3
2 | 3
4 | 4
4 | 4
5 | 6
5 | 6
(8 rows)
DROP FUNCTION inner_fn(int);
DEBUG: switching to sequential query execution mode
DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands
DROP FUNCTION outer_local_fn();
DEBUG: switching to sequential query execution mode
DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands
TRUNCATE TABLE testnested_table;
CREATE OR REPLACE FUNCTION inner_fn(x int)
RETURNS void
AS $$
DECLARE
BEGIN
INSERT INTO forcepushdown_schema.testnested_table VALUES (x,x);
END;
$$ LANGUAGE plpgsql;
DEBUG: switching to sequential query execution mode
DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands
-- Force-delegation function calling non-force function
CREATE OR REPLACE FUNCTION outer_fn(y int, z int)
RETURNS void
AS $$
DECLARE
BEGIN
PERFORM 1 FROM forcepushdown_schema.inner_fn(y);
INSERT INTO forcepushdown_schema.testnested_table VALUES (y,y);
PERFORM 1 FROM forcepushdown_schema.inner_fn(z);
INSERT INTO forcepushdown_schema.testnested_table VALUES (z,z);
END;
$$ LANGUAGE plpgsql;
DEBUG: switching to sequential query execution mode
DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands
SELECT create_distributed_function('inner_fn(int)','x',
colocate_with:='testnested_table', force_delegation := false);
DEBUG: switching to sequential query execution mode
DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands
create_distributed_function
---------------------------------------------------------------------
(1 row)
SELECT create_distributed_function('outer_fn(int, int)','y',
colocate_with:='testnested_table', force_delegation := true);
DEBUG: switching to sequential query execution mode
DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands
create_distributed_function
---------------------------------------------------------------------
(1 row)
SELECT outer_fn(1, 2);
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 disabling forced delegation through create_distributed_table(..., force_delegation := false)
CONTEXT: SQL statement "INSERT INTO forcepushdown_schema.testnested_table VALUES (x,x)"
PL/pgSQL function forcepushdown_schema.inner_fn(integer) line XX at SQL statement
SQL statement "SELECT 1 FROM forcepushdown_schema.inner_fn(z)"
PL/pgSQL function forcepushdown_schema.outer_fn(integer,integer) line XX at PERFORM
while executing command on localhost:xxxxx
BEGIN;
SELECT outer_fn(1, 2);
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 disabling forced delegation through create_distributed_table(..., force_delegation := false)
CONTEXT: SQL statement "INSERT INTO forcepushdown_schema.testnested_table VALUES (x,x)"
PL/pgSQL function forcepushdown_schema.inner_fn(integer) line XX at SQL statement
SQL statement "SELECT 1 FROM forcepushdown_schema.inner_fn(z)"
PL/pgSQL function forcepushdown_schema.outer_fn(integer,integer) line XX at PERFORM
while executing command on localhost:xxxxx
END;
-- No rows
SELECT * FROM testnested_table ORDER BY 1;
x | y
---------------------------------------------------------------------
(0 rows)
-- Force-delegation function calling force-delegation function
CREATE OR REPLACE FUNCTION force_push_inner(y int)
RETURNS void
AS $$
DECLARE
BEGIN
INSERT INTO forcepushdown_schema.testnested_table VALUES (y,y);
END;
$$ LANGUAGE plpgsql;
DEBUG: switching to sequential query execution mode
DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands
CREATE OR REPLACE FUNCTION force_push_outer(x int)
RETURNS void
AS $$
DECLARE
BEGIN
INSERT INTO forcepushdown_schema.testnested_table VALUES (x,x);
PERFORM forcepushdown_schema.force_push_inner(x+1) LIMIT 1;
END;
$$ LANGUAGE plpgsql;
DEBUG: switching to sequential query execution mode
DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands
SELECT create_distributed_function(
'force_push_outer(int)', 'x',
colocate_with := 'testnested_table',
force_delegation := true
);
DEBUG: switching to sequential query execution mode
DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands
create_distributed_function
---------------------------------------------------------------------
(1 row)
SELECT create_distributed_function(
'force_push_inner(int)', 'y',
colocate_with := 'testnested_table',
force_delegation := true
);
DEBUG: switching to sequential query execution mode
DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands
create_distributed_function
---------------------------------------------------------------------
(1 row)
-- Keys 7,8,9,14 fall on one node and 15 on a different node
-- Function gets delegated to node with shard-key = 7 and inner function
-- will not be delegated but inserts shard-key = 8 locally
SELECT force_push_outer(7);
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 disabling forced delegation through create_distributed_table(..., force_delegation := false)
CONTEXT: SQL statement "INSERT INTO forcepushdown_schema.testnested_table VALUES (y,y)"
PL/pgSQL function forcepushdown_schema.force_push_inner(integer) line XX at SQL statement
SQL statement "SELECT forcepushdown_schema.force_push_inner(x+1) LIMIT 1"
PL/pgSQL function forcepushdown_schema.force_push_outer(integer) line XX at PERFORM
while executing command on localhost:xxxxx
BEGIN;
-- Function gets delegated to node with shard-key = 8 and inner function
-- will not be delegated but inserts shard-key = 9 locally
SELECT force_push_outer(8);
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 disabling forced delegation through create_distributed_table(..., force_delegation := false)
CONTEXT: SQL statement "INSERT INTO forcepushdown_schema.testnested_table VALUES (y,y)"
PL/pgSQL function forcepushdown_schema.force_push_inner(integer) line XX at SQL statement
SQL statement "SELECT forcepushdown_schema.force_push_inner(x+1) LIMIT 1"
PL/pgSQL function forcepushdown_schema.force_push_outer(integer) line XX at PERFORM
while executing command on localhost:xxxxx
END;
BEGIN;
-- Function gets delegated to node with shard-key = 14 and inner function
-- will not be delegated but fails to insert shard-key = 15 remotely
SELECT force_push_outer(14);
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 disabling forced delegation through create_distributed_table(..., force_delegation := false)
CONTEXT: SQL statement "INSERT INTO forcepushdown_schema.testnested_table VALUES (y,y)"
PL/pgSQL function forcepushdown_schema.force_push_inner(integer) line XX at SQL statement
SQL statement "SELECT forcepushdown_schema.force_push_inner(x+1) LIMIT 1"
PL/pgSQL function forcepushdown_schema.force_push_outer(integer) line XX at PERFORM
while executing command on localhost:xxxxx
END;
SELECT * FROM testnested_table ORDER BY 1;
x | y
---------------------------------------------------------------------
(0 rows)
--
-- Function-1() --> function-2() --> function-3()
--
CREATE OR REPLACE FUNCTION force_push_1(x int)
RETURNS void
AS $$
DECLARE
BEGIN
INSERT INTO forcepushdown_schema.testnested_table VALUES (x,x);
PERFORM forcepushdown_schema.force_push_2(x+1) LIMIT 1;
END;
$$ LANGUAGE plpgsql;
DEBUG: switching to sequential query execution mode
DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands
CREATE OR REPLACE FUNCTION force_push_2(y int)
RETURNS void
AS $$
DECLARE
BEGIN
INSERT INTO forcepushdown_schema.testnested_table VALUES (y,y);
PERFORM forcepushdown_schema.force_push_3(y+1) LIMIT 1;
END;
$$ LANGUAGE plpgsql;
DEBUG: switching to sequential query execution mode
DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands
CREATE OR REPLACE FUNCTION force_push_3(z int)
RETURNS void
AS $$
DECLARE
BEGIN
INSERT INTO forcepushdown_schema.testnested_table VALUES (z,z);
END;
$$ LANGUAGE plpgsql;
DEBUG: switching to sequential query execution mode
DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands
SELECT create_distributed_function(
'force_push_1(int)', 'x',
colocate_with := 'testnested_table',
force_delegation := true
);
DEBUG: switching to sequential query execution mode
DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands
create_distributed_function
---------------------------------------------------------------------
(1 row)
SELECT create_distributed_function(
'force_push_2(int)', 'y',
colocate_with := 'testnested_table',
force_delegation := true
);
DEBUG: switching to sequential query execution mode
DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands
create_distributed_function
---------------------------------------------------------------------
(1 row)
SELECT create_distributed_function(
'force_push_3(int)', 'z',
colocate_with := 'testnested_table',
force_delegation := true
);
DEBUG: switching to sequential query execution mode
DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands
create_distributed_function
---------------------------------------------------------------------
(1 row)
TRUNCATE TABLE testnested_table;
BEGIN;
-- All local inserts
SELECT force_push_1(7);
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 disabling forced delegation through create_distributed_table(..., force_delegation := false)
CONTEXT: SQL statement "INSERT INTO forcepushdown_schema.testnested_table VALUES (y,y)"
PL/pgSQL function forcepushdown_schema.force_push_2(integer) line XX at SQL statement
SQL statement "SELECT forcepushdown_schema.force_push_2(x+1) LIMIT 1"
PL/pgSQL function forcepushdown_schema.force_push_1(integer) line XX at PERFORM
while executing command on localhost:xxxxx
END;
BEGIN;
-- Local(shard-keys 13, 15) + remote insert (shard-key 14)
SELECT force_push_1(13);
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 disabling forced delegation through create_distributed_table(..., force_delegation := false)
CONTEXT: SQL statement "INSERT INTO forcepushdown_schema.testnested_table VALUES (y,y)"
PL/pgSQL function forcepushdown_schema.force_push_2(integer) line XX at SQL statement
SQL statement "SELECT forcepushdown_schema.force_push_2(x+1) LIMIT 1"
PL/pgSQL function forcepushdown_schema.force_push_1(integer) line XX at PERFORM
while executing command on localhost:xxxxx
END;
SELECT * FROM testnested_table ORDER BY 1;
x | y
---------------------------------------------------------------------
(0 rows)
TRUNCATE TABLE testnested_table;
CREATE OR REPLACE FUNCTION force_push_inner(y int)
RETURNS void
AS $$
DECLARE
BEGIN
INSERT INTO forcepushdown_schema.testnested_table VALUES (y,y);
END;
$$ LANGUAGE plpgsql;
DEBUG: switching to sequential query execution mode
DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands
CREATE OR REPLACE FUNCTION force_push_outer(x int)
RETURNS void
AS $$
DECLARE
BEGIN
PERFORM FROM forcepushdown_schema.force_push_inner(x);
INSERT INTO forcepushdown_schema.testnested_table VALUES (x+1,x+1);
END;
$$ LANGUAGE plpgsql;
DEBUG: switching to sequential query execution mode
DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands
SELECT create_distributed_function(
'force_push_inner(int)', 'y',
colocate_with := 'testnested_table',
force_delegation := true
);
DEBUG: switching to sequential query execution mode
DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands
create_distributed_function
---------------------------------------------------------------------
(1 row)
SELECT create_distributed_function(
'force_push_outer(int)', 'x',
colocate_with := 'testnested_table',
force_delegation := true
);
DEBUG: switching to sequential query execution mode
DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands
create_distributed_function
---------------------------------------------------------------------
(1 row)
BEGIN;
SELECT force_push_outer(7);
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 disabling forced delegation through create_distributed_table(..., force_delegation := false)
CONTEXT: SQL statement "INSERT INTO forcepushdown_schema.testnested_table VALUES (x+1,x+1)"
PL/pgSQL function forcepushdown_schema.force_push_outer(integer) line XX at SQL statement
while executing command on localhost:xxxxx
END;
TABLE testnested_table ORDER BY 1;
x | y
---------------------------------------------------------------------
(0 rows)
CREATE OR REPLACE FUNCTION force_push_inner(y int)
RETURNS void
AS $$
DECLARE
BEGIN
RAISE NOTICE '%', y;
END;
$$ LANGUAGE plpgsql;
DEBUG: switching to sequential query execution mode
DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands
CREATE OR REPLACE FUNCTION force_push_outer(x int)
RETURNS void
AS $$
DECLARE
BEGIN
PERFORM FROM forcepushdown_schema.force_push_inner(x+1);
INSERT INTO forcepushdown_schema.testnested_table VALUES (x,x);
END;
$$ LANGUAGE plpgsql;
DEBUG: switching to sequential query execution mode
DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands
BEGIN;
SELECT force_push_outer(9);
DEBUG: pushing down function call in a multi-statement transaction
DEBUG: pushing down the function call
NOTICE: 10
DETAIL: from localhost:xxxxx
force_push_outer
---------------------------------------------------------------------
(1 row)
END;
TABLE testnested_table ORDER BY 1;
x | y
---------------------------------------------------------------------
9 | 9
(1 row)
RESET client_min_messages;
SET citus.log_remote_commands TO off;
DROP SCHEMA forcepushdown_schema CASCADE;
NOTICE: drop cascades to 38 other objects
NOTICE: drop cascades to 46 other objects
DETAIL: drop cascades to table test_forcepushdown
drop cascades to table test_forcepushdown_noncolocate
drop cascades to function insert_data(integer)
@ -1460,3 +1980,11 @@ drop cascades to function test_prepare(integer,integer)
drop cascades to function outer_test_prepare(integer,integer)
drop cascades to table test_perform
drop cascades to function test(integer)
drop cascades to table testnested_table
drop cascades to function inner_fn(integer)
drop cascades to function outer_fn(integer,integer)
drop cascades to function force_push_inner(integer)
drop cascades to function force_push_outer(integer)
drop cascades to function force_push_1(integer)
drop cascades to function force_push_2(integer)
drop cascades to function force_push_3(integer)

View File

@ -0,0 +1,321 @@
CREATE SCHEMA function_propagation_schema;
SET search_path TO 'function_propagation_schema';
-- Check whether supported dependencies can be distributed while propagating functions
-- Check types
SET citus.enable_metadata_sync TO OFF;
CREATE TYPE function_prop_type AS (a int, b int);
RESET citus.enable_metadata_sync;
CREATE OR REPLACE FUNCTION func_1(param_1 function_prop_type)
RETURNS int
LANGUAGE plpgsql AS
$$
BEGIN
return 1;
END;
$$;
-- Check all dependent objects and function depends on all nodes
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema'::regnamespace::oid;
pg_identify_object_as_address
---------------------------------------------------------------------
(schema,{function_propagation_schema},{})
(1 row)
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.function_prop_type'::regtype::oid;
pg_identify_object_as_address
---------------------------------------------------------------------
(type,{function_propagation_schema.function_prop_type},{})
(1 row)
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_1'::regproc::oid;
pg_identify_object_as_address
---------------------------------------------------------------------
(function,"{function_propagation_schema,func_1}",{function_propagation_schema.function_prop_type})
(1 row)
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema'::regnamespace::oid;$$) ORDER BY 1,2;
nodename | nodeport | success | result
---------------------------------------------------------------------
localhost | 57637 | t | (schema,{function_propagation_schema},{})
localhost | 57638 | t | (schema,{function_propagation_schema},{})
(2 rows)
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.function_prop_type'::regtype::oid;$$) ORDER BY 1,2;
nodename | nodeport | success | result
---------------------------------------------------------------------
localhost | 57637 | t | (type,{function_propagation_schema.function_prop_type},{})
localhost | 57638 | t | (type,{function_propagation_schema.function_prop_type},{})
(2 rows)
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_1'::regproc::oid;$$) ORDER BY 1,2;
nodename | nodeport | success | result
---------------------------------------------------------------------
localhost | 57637 | t | (function,"{function_propagation_schema,func_1}",{function_propagation_schema.function_prop_type})
localhost | 57638 | t | (function,"{function_propagation_schema,func_1}",{function_propagation_schema.function_prop_type})
(2 rows)
SET citus.enable_metadata_sync TO OFF;
CREATE TYPE function_prop_type_2 AS (a int, b int);
RESET citus.enable_metadata_sync;
CREATE OR REPLACE FUNCTION func_2(param_1 int)
RETURNS function_prop_type_2
LANGUAGE plpgsql AS
$$
BEGIN
return 1;
END;
$$;
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.function_prop_type_2'::regtype::oid;
pg_identify_object_as_address
---------------------------------------------------------------------
(type,{function_propagation_schema.function_prop_type_2},{})
(1 row)
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_2'::regproc::oid;
pg_identify_object_as_address
---------------------------------------------------------------------
(function,"{function_propagation_schema,func_2}",{integer})
(1 row)
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.function_prop_type_2'::regtype::oid;$$) ORDER BY 1,2;
nodename | nodeport | success | result
---------------------------------------------------------------------
localhost | 57637 | t | (type,{function_propagation_schema.function_prop_type_2},{})
localhost | 57638 | t | (type,{function_propagation_schema.function_prop_type_2},{})
(2 rows)
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_2'::regproc::oid;$$) ORDER BY 1,2;
nodename | nodeport | success | result
---------------------------------------------------------------------
localhost | 57637 | t | (function,"{function_propagation_schema,func_2}",{integer})
localhost | 57638 | t | (function,"{function_propagation_schema,func_2}",{integer})
(2 rows)
-- Have a separate check for type created in transaction
BEGIN;
CREATE TYPE function_prop_type_3 AS (a int, b int);
COMMIT;
-- Objects in the body part is not found as dependency
CREATE OR REPLACE FUNCTION func_3(param_1 int)
RETURNS int
LANGUAGE plpgsql AS
$$
DECLARE
internal_param1 function_prop_type_3;
BEGIN
return 1;
END;
$$;
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.function_prop_type_3'::regtype::oid;
pg_identify_object_as_address
---------------------------------------------------------------------
(0 rows)
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_3'::regproc::oid;
pg_identify_object_as_address
---------------------------------------------------------------------
(function,"{function_propagation_schema,func_3}",{integer})
(1 row)
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_3'::regproc::oid;$$) ORDER BY 1,2;
nodename | nodeport | success | result
---------------------------------------------------------------------
localhost | 57637 | t | (function,"{function_propagation_schema,func_3}",{integer})
localhost | 57638 | t | (function,"{function_propagation_schema,func_3}",{integer})
(2 rows)
-- Check table
CREATE TABLE function_prop_table(a int, b int);
-- Non-distributed table is not distributed as dependency
CREATE OR REPLACE FUNCTION func_4(param_1 function_prop_table)
RETURNS int
LANGUAGE plpgsql AS
$$
BEGIN
return 1;
END;
$$;
WARNING: Citus can't distribute function "func_4" having dependency on non-distributed relation "function_prop_table"
DETAIL: Function will be created only locally
HINT: To distribute function, distribute dependent relations first. Then, re-create the function
CREATE OR REPLACE FUNCTION func_5(param_1 int)
RETURNS function_prop_table
LANGUAGE plpgsql AS
$$
BEGIN
return 1;
END;
$$;
WARNING: Citus can't distribute function "func_5" having dependency on non-distributed relation "function_prop_table"
DETAIL: Function will be created only locally
HINT: To distribute function, distribute dependent relations first. Then, re-create the function
-- Functions can be created with distributed table dependency
SELECT create_distributed_table('function_prop_table', 'a');
create_distributed_table
---------------------------------------------------------------------
(1 row)
CREATE OR REPLACE FUNCTION func_6(param_1 function_prop_table)
RETURNS int
LANGUAGE plpgsql AS
$$
BEGIN
return 1;
END;
$$;
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_6'::regproc::oid;
pg_identify_object_as_address
---------------------------------------------------------------------
(function,"{function_propagation_schema,func_6}",{function_propagation_schema.function_prop_table})
(1 row)
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_6'::regproc::oid;$$) ORDER BY 1,2;
nodename | nodeport | success | result
---------------------------------------------------------------------
localhost | 57637 | t | (function,"{function_propagation_schema,func_6}",{function_propagation_schema.function_prop_table})
localhost | 57638 | t | (function,"{function_propagation_schema,func_6}",{function_propagation_schema.function_prop_table})
(2 rows)
-- Views are not supported
CREATE VIEW function_prop_view AS SELECT * FROM function_prop_table;
CREATE OR REPLACE FUNCTION func_7(param_1 function_prop_view)
RETURNS int
LANGUAGE plpgsql AS
$$
BEGIN
return 1;
END;
$$;
WARNING: Citus can't distribute functions having dependency on unsupported object of type "view"
DETAIL: Function will be created only locally
CREATE OR REPLACE FUNCTION func_8(param_1 int)
RETURNS function_prop_view
LANGUAGE plpgsql AS
$$
BEGIN
return 1;
END;
$$;
WARNING: Citus can't distribute functions having dependency on unsupported object of type "view"
DETAIL: Function will be created only locally
-- Check within transaction
BEGIN;
CREATE TYPE type_in_transaction AS (a int, b int);
CREATE OR REPLACE FUNCTION func_in_transaction(param_1 type_in_transaction)
RETURNS int
LANGUAGE plpgsql AS
$$
BEGIN
return 1;
END;
$$;
-- Within transaction functions are not distributed
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.type_in_transaction'::regtype::oid;
pg_identify_object_as_address
---------------------------------------------------------------------
(0 rows)
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction'::regproc::oid;
pg_identify_object_as_address
---------------------------------------------------------------------
(0 rows)
COMMIT;
-- Show that recreating it outside transaction distributes the function and dependencies
CREATE OR REPLACE FUNCTION func_in_transaction(param_1 type_in_transaction)
RETURNS int
LANGUAGE plpgsql AS
$$
BEGIN
return 1;
END;
$$;
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.type_in_transaction'::regtype::oid;
pg_identify_object_as_address
---------------------------------------------------------------------
(type,{function_propagation_schema.type_in_transaction},{})
(1 row)
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction'::regproc::oid;
pg_identify_object_as_address
---------------------------------------------------------------------
(function,"{function_propagation_schema,func_in_transaction}",{function_propagation_schema.type_in_transaction})
(1 row)
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.type_in_transaction'::regtype::oid;$$) ORDER BY 1,2;
nodename | nodeport | success | result
---------------------------------------------------------------------
localhost | 57637 | t | (type,{function_propagation_schema.type_in_transaction},{})
localhost | 57638 | t | (type,{function_propagation_schema.type_in_transaction},{})
(2 rows)
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction'::regproc::oid;$$) ORDER BY 1,2;
nodename | nodeport | success | result
---------------------------------------------------------------------
localhost | 57637 | t | (function,"{function_propagation_schema,func_in_transaction}",{function_propagation_schema.type_in_transaction})
localhost | 57638 | t | (function,"{function_propagation_schema,func_in_transaction}",{function_propagation_schema.type_in_transaction})
(2 rows)
-- Test for SQL function with unsupported object in function body
CREATE TABLE table_in_sql_body(id int);
CREATE FUNCTION max_of_table()
RETURNS int
LANGUAGE SQL AS
$$
SELECT max(id) FROM table_in_sql_body
$$;
-- Show that only function has propagated, since the table is not resolved as dependency
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.type_in_transaction'::regclass::oid;
pg_identify_object_as_address
---------------------------------------------------------------------
(0 rows)
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.max_of_table'::regproc::oid;
pg_identify_object_as_address
---------------------------------------------------------------------
(function,"{function_propagation_schema,max_of_table}",{})
(1 row)
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.max_of_table'::regproc::oid;$$) ORDER BY 1,2;
nodename | nodeport | success | result
---------------------------------------------------------------------
localhost | 57637 | t | (function,"{function_propagation_schema,max_of_table}",{})
localhost | 57638 | t | (function,"{function_propagation_schema,max_of_table}",{})
(2 rows)
-- Check extension owned table
CREATE TABLE extension_owned_table(a int);
SELECT run_command_on_workers($$
CREATE TABLE function_propagation_schema.extension_owned_table(a int);
$$
);
run_command_on_workers
---------------------------------------------------------------------
(localhost,57637,t,"CREATE TABLE")
(localhost,57638,t,"CREATE TABLE")
(2 rows)
CREATE EXTENSION seg;
ALTER EXTENSION seg ADD TABLE extension_owned_table;
NOTICE: Citus does not propagate adding/dropping member objects
HINT: You can add/drop the member objects on the workers as well.
SELECT run_command_on_workers($$
ALTER EXTENSION seg ADD TABLE function_propagation_schema.extension_owned_table;
$$);
run_command_on_workers
---------------------------------------------------------------------
(localhost,57637,t,"ALTER EXTENSION")
(localhost,57638,t,"ALTER EXTENSION")
(2 rows)
CREATE OR REPLACE FUNCTION func_for_ext_check(param_1 extension_owned_table)
RETURNS int
LANGUAGE plpgsql AS
$$
BEGIN
return 1;
END;
$$;
RESET search_path;
SET client_min_messages TO WARNING;
DROP SCHEMA function_propagation_schema CASCADE;

View File

@ -62,6 +62,7 @@ SET citus.log_local_commands TO ON;
-- returns true of the distribution key filter
-- on the distributed tables (e.g., WHERE key = 1), we'll hit a shard
-- placement which is local to this not
SET citus.enable_metadata_sync TO OFF;
CREATE OR REPLACE FUNCTION shard_of_distribution_column_is_local(dist_key int) RETURNS bool AS $$
DECLARE shard_is_local BOOLEAN := FALSE;
@ -84,6 +85,7 @@ CREATE OR REPLACE FUNCTION shard_of_distribution_column_is_local(dist_key int) R
RETURN shard_is_local;
END;
$$ LANGUAGE plpgsql;
RESET citus.enable_metadata_sync;
-- pick some example values that reside on the shards locally and remote
-- distribution key values of 1,6, 500 and 701 are LOCAL to shards,
-- we'll use these values in the tests

View File

@ -94,6 +94,7 @@ SET search_path TO local_shard_execution;
-- returns true of the distribution key filter
-- on the distributed tables (e.g., WHERE key = 1), we'll hit a shard
-- placement which is local to this not
SET citus.enable_metadata_sync TO OFF;
CREATE OR REPLACE FUNCTION shard_of_distribution_column_is_local(dist_key int) RETURNS bool AS $$
DECLARE shard_is_local BOOLEAN := FALSE;
@ -116,6 +117,7 @@ CREATE OR REPLACE FUNCTION shard_of_distribution_column_is_local(dist_key int) R
RETURN shard_is_local;
END;
$$ LANGUAGE plpgsql;
RESET citus.enable_metadata_sync;
-- test case for issue #3556
SET citus.log_intermediate_results TO TRUE;
SET client_min_messages TO DEBUG1;
@ -801,6 +803,7 @@ BEGIN;
ERROR: VACUUM cannot run inside a transaction block
ROLLBACK;
-- make sure that functions can use local execution
SET citus.enable_metadata_sync TO OFF;
CREATE OR REPLACE PROCEDURE only_local_execution() AS $$
DECLARE cnt INT;
BEGIN
@ -896,6 +899,7 @@ CREATE OR REPLACE PROCEDURE local_execution_followed_by_dist() AS $$
SELECT count(*) INTO cnt FROM distributed_table;
END;
$$ LANGUAGE plpgsql;
RESET citus.enable_metadata_sync;
CALL local_execution_followed_by_dist();
NOTICE: executing the command locally: INSERT INTO local_shard_execution.distributed_table_1470001 AS citus_table_alias (key, value, age) VALUES (1, '11'::text, 21) ON CONFLICT(key) DO UPDATE SET value = '29'::text
CONTEXT: SQL statement "INSERT INTO distributed_table VALUES (1, '11',21) ON CONFLICT(key) DO UPDATE SET value = '29'"

View File

@ -770,6 +770,7 @@ NOTICE: executing the command locally: INSERT INTO local_shard_execution_replic
ERROR: VACUUM cannot run inside a transaction block
ROLLBACK;
-- make sure that functions can use local execution
SET citus.enable_metadata_sync TO OFF;
CREATE OR REPLACE PROCEDURE only_local_execution() AS $$
DECLARE cnt INT;
BEGIN
@ -865,6 +866,7 @@ CREATE OR REPLACE PROCEDURE local_execution_followed_by_dist() AS $$
SELECT count(*) INTO cnt FROM distributed_table;
END;
$$ LANGUAGE plpgsql;
RESET citus.enable_metadata_sync;
CALL local_execution_followed_by_dist();
NOTICE: executing the command locally: INSERT INTO local_shard_execution_replicated.distributed_table_1500001 AS citus_table_alias (key, value, age) VALUES (1, '11'::text, 21) ON CONFLICT(key) DO UPDATE SET value = '29'::text
CONTEXT: SQL statement "INSERT INTO distributed_table VALUES (1, '11',21) ON CONFLICT(key) DO UPDATE SET value = '29'"

View File

@ -987,6 +987,7 @@ BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
ERROR: Node with group id 123123123 for shard placement xxxxx does not exist
ROLLBACK;
-- create a volatile function that returns the local node id
SET citus.enable_metadata_sync TO OFF;
CREATE OR REPLACE FUNCTION get_node_id()
RETURNS INT AS $$
DECLARE localGroupId int;
@ -999,6 +1000,7 @@ BEGIN
nodeport = 57637 AND nodename = 'localhost' AND isactive AND nodecluster = 'default';
RETURN localGroupId;
END; $$ language plpgsql;
RESET citus.enable_metadata_sync;
-- fails because we ingest more placements for the same shards to the same worker node
BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
SELECT assign_distributed_transaction_id(0, 8, '2021-07-09 15:41:55.542377+02');

View File

@ -7,6 +7,7 @@
-- reference and hash-distributed version of orders, customer and part tables.
SET citus.next_shard_id TO 360000;
-- this function is dropped in Citus10, added here for tests
SET citus.enable_metadata_sync TO OFF;
CREATE OR REPLACE FUNCTION pg_catalog.master_create_distributed_table(table_name regclass,
distribution_column text,
distribution_method citus.distribution_type)
@ -23,6 +24,7 @@ CREATE OR REPLACE FUNCTION pg_catalog.master_create_worker_shards(table_name tex
RETURNS void
AS 'citus', $$master_create_worker_shards$$
LANGUAGE C STRICT;
RESET citus.enable_metadata_sync;
CREATE TABLE lineitem (
l_orderkey bigint not null,
l_partkey integer not null,

View File

@ -689,17 +689,7 @@ SELECT create_distributed_function('func_custom_param(intpair)');
(1 row)
RESET citus.enable_metadata_sync;
SELECT deparse_and_run_on_workers($cmd$
ALTER FUNCTION func_custom_param RENAME TO func_with_custom_param;
$cmd$);
INFO: Propagating deparsed query: ALTER FUNCTION function_tests.func_custom_param(function_tests.intpair) RENAME TO func_with_custom_param;
CONTEXT: PL/pgSQL function deparse_and_run_on_workers(text) line XX at RAISE
deparse_and_run_on_workers
---------------------------------------------------------------------
(localhost,57637,t,"ALTER FUNCTION")
(localhost,57638,t,"ALTER FUNCTION")
(2 rows)
-- a function that returns TABLE
CREATE FUNCTION func_returns_table(IN count INT)
RETURNS TABLE (x INT, y INT)
@ -713,17 +703,7 @@ SELECT create_distributed_function('func_returns_table(INT)');
(1 row)
RESET citus.enable_metadata_sync;
SELECT deparse_and_run_on_workers($cmd$
ALTER FUNCTION func_returns_table ROWS 100;
$cmd$);
INFO: Propagating deparsed query: ALTER FUNCTION function_tests.func_returns_table(integer) ROWS 100.000000;
CONTEXT: PL/pgSQL function deparse_and_run_on_workers(text) line XX at RAISE
deparse_and_run_on_workers
---------------------------------------------------------------------
(localhost,57637,t,"ALTER FUNCTION")
(localhost,57638,t,"ALTER FUNCTION")
(2 rows)
-- clear objects
SET client_min_messages TO WARNING; -- suppress cascading objects dropping
DROP SCHEMA "CiTuS.TeeN" CASCADE;

View File

@ -77,6 +77,8 @@ END
$func$ LANGUAGE plpgsql;
CREATE SCHEMA test;
:create_function_test_maintenance_worker
WARNING: Citus can't distribute functions having dependency on unsupported object of type "view"
DETAIL: Function will be created only locally
-- check maintenance daemon is started
SELECT datname, current_database(),
usename, (SELECT extowner::regrole::text FROM pg_extension WHERE extname = 'citus')
@ -1198,6 +1200,8 @@ HINT: You can manually create a database and its extensions on workers.
CREATE EXTENSION citus;
CREATE SCHEMA test;
:create_function_test_maintenance_worker
WARNING: Citus can't distribute functions having dependency on unsupported object of type "view"
DETAIL: Function will be created only locally
-- see that the daemon started
SELECT datname, current_database(),
usename, (SELECT extowner::regrole::text FROM pg_extension WHERE extname = 'citus')

View File

@ -39,14 +39,8 @@ DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT table1.id, ta
CREATE FUNCTION add(integer, integer) RETURNS integer
AS 'SELECT $1 + $2;'
LANGUAGE SQL;
SELECT create_distributed_function('add(integer,integer)');
DEBUG: switching to sequential query execution mode
DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands
create_distributed_function
---------------------------------------------------------------------
(1 row)
SELECT * FROM table1 JOIN add(3,5) sum ON (id = sum) ORDER BY id ASC;
id | data | sum
---------------------------------------------------------------------
@ -60,7 +54,10 @@ BEGIN
RETURN i + 1;
END;
$$ LANGUAGE plpgsql;
DEBUG: switching to sequential query execution mode
DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands
SELECT * FROM table1 JOIN increment(2) val ON (id = val) ORDER BY id ASC;
DEBUG: function does not have co-located tables
DEBUG: generating subplan XXX_1 for subquery SELECT val FROM functions_in_joins.increment(2) val(val)
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT table1.id, table1.data, val.val FROM (functions_in_joins.table1 JOIN (SELECT intermediate_result.val FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(val integer)) val ON ((table1.id OPERATOR(pg_catalog.=) val.val))) ORDER BY table1.id
id | data | val
@ -69,6 +66,8 @@ DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT table1.id, ta
(1 row)
-- a function that returns a set of integers
-- Block distributing function as we have tests below to test it locally
SET citus.enable_metadata_sync TO OFF;
CREATE OR REPLACE FUNCTION next_k_integers(IN first_value INTEGER,
IN k INTEGER DEFAULT 3,
OUT result INTEGER)
@ -77,6 +76,7 @@ BEGIN
RETURN QUERY SELECT x FROM generate_series(first_value, first_value+k-1) f(x);
END;
$$ LANGUAGE plpgsql;
RESET citus.enable_metadata_sync;
SELECT *
FROM table1 JOIN next_k_integers(3,2) next_integers ON (id = next_integers.result)
ORDER BY id ASC;
@ -93,7 +93,10 @@ CREATE FUNCTION get_set_of_records() RETURNS SETOF RECORD AS $cmd$
SELECT x, x+1 FROM generate_series(0,4) f(x)
$cmd$
LANGUAGE SQL;
DEBUG: switching to sequential query execution mode
DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands
SELECT * FROM table1 JOIN get_set_of_records() AS t2(x int, y int) ON (id = x) ORDER BY id ASC;
DEBUG: function does not have co-located tables
DEBUG: generating subplan XXX_1 for subquery SELECT x, y FROM functions_in_joins.get_set_of_records() t2(x integer, y integer)
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT table1.id, table1.data, t2.x, t2.y FROM (functions_in_joins.table1 JOIN (SELECT intermediate_result.x, intermediate_result.y FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(x integer, y integer)) t2 ON ((table1.id OPERATOR(pg_catalog.=) t2.x))) ORDER BY table1.id
id | data | x | y
@ -108,7 +111,10 @@ DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT table1.id, ta
CREATE FUNCTION dup(int) RETURNS TABLE(f1 int, f2 text)
AS $$ SELECT $1, CAST($1 AS text) || ' is text' $$
LANGUAGE SQL;
DEBUG: switching to sequential query execution mode
DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands
SELECT f.* FROM table1 t JOIN dup(32) f ON (f1 = id);
DEBUG: function does not have co-located tables
DEBUG: generating subplan XXX_1 for subquery SELECT f1, f2 FROM functions_in_joins.dup(32) f(f1, f2)
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT f.f1, f.f2 FROM (functions_in_joins.table1 t JOIN (SELECT intermediate_result.f1, intermediate_result.f2 FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(f1 integer, f2 text)) f ON ((f.f1 OPERATOR(pg_catalog.=) t.id)))
f1 | f2
@ -119,7 +125,10 @@ DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT f.f1, f.f2 FR
-- a stable function
CREATE OR REPLACE FUNCTION the_minimum_id()
RETURNS INTEGER STABLE AS 'SELECT min(id) FROM table1' LANGUAGE SQL;
DEBUG: switching to sequential query execution mode
DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands
SELECT * FROM table1 JOIN the_minimum_id() min_id ON (id = min_id);
DEBUG: function does not have co-located tables
DEBUG: generating subplan XXX_1 for subquery SELECT min_id FROM functions_in_joins.the_minimum_id() min_id(min_id)
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT table1.id, table1.data, min_id.min_id FROM (functions_in_joins.table1 JOIN (SELECT intermediate_result.min_id FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(min_id integer)) min_id ON ((table1.id OPERATOR(pg_catalog.=) min_id.min_id)))
id | data | min_id
@ -180,7 +189,10 @@ begin
return result;
end;
$$ language plpgsql;
DEBUG: switching to sequential query execution mode
DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands
SELECT * FROM table1 JOIN max_and_min() m ON (m.maximum = data OR m.minimum = data) ORDER BY 1,2,3,4;
DEBUG: function does not have co-located tables
DEBUG: generating subplan XXX_1 for subquery SELECT minimum, maximum FROM functions_in_joins.max_and_min() m(minimum, maximum)
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT table1.id, table1.data, m.minimum, m.maximum FROM (functions_in_joins.table1 JOIN (SELECT intermediate_result.minimum, intermediate_result.maximum FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(minimum integer, maximum integer)) m ON (((m.maximum OPERATOR(pg_catalog.=) table1.data) OR (m.minimum OPERATOR(pg_catalog.=) table1.data)))) ORDER BY table1.id, table1.data, m.minimum, m.maximum
id | data | minimum | maximum
@ -198,6 +210,7 @@ SET client_min_messages TO ERROR;
-- function joins in CTE results can create lateral joins that are not supported
-- we execute the query within a function to consolidate the error messages
-- between different executors
SET citus.enable_metadata_sync TO OFF;
CREATE FUNCTION raise_failed_execution_func_join(query text) RETURNS void AS $$
BEGIN
EXECUTE query;
@ -209,6 +222,7 @@ BEGIN
END IF;
END;
$$LANGUAGE plpgsql;
RESET citus.enable_metadata_sync;
SELECT raise_failed_execution_func_join($$
WITH one_row AS (
SELECT * FROM table1 WHERE id=52
@ -219,8 +233,10 @@ SELECT raise_failed_execution_func_join($$
$$);
ERROR: Task failed to execute
-- a user-defined immutable function
SET citus.enable_metadata_sync TO OFF;
CREATE OR REPLACE FUNCTION the_answer_to_life()
RETURNS INTEGER IMMUTABLE AS 'SELECT 42' LANGUAGE SQL;
RESET citus.enable_metadata_sync;
SELECT raise_failed_execution_func_join($$
SELECT * FROM table1 JOIN the_answer_to_life() the_answer ON (id = the_answer);
$$);

View File

@ -39,14 +39,8 @@ DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT table1.id, ta
CREATE FUNCTION add(integer, integer) RETURNS integer
AS 'SELECT $1 + $2;'
LANGUAGE SQL;
SELECT create_distributed_function('add(integer,integer)');
DEBUG: switching to sequential query execution mode
DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands
create_distributed_function
---------------------------------------------------------------------
(1 row)
SELECT * FROM table1 JOIN add(3,5) sum ON (id = sum) ORDER BY id ASC;
DEBUG: generating subplan XXX_1 for subquery SELECT sum FROM functions_in_joins.add(3, 5) sum(sum)
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT table1.id, table1.data, sum.sum FROM (functions_in_joins.table1 JOIN (SELECT intermediate_result.sum FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(sum integer)) sum ON ((table1.id OPERATOR(pg_catalog.=) sum.sum))) ORDER BY table1.id
@ -62,7 +56,10 @@ BEGIN
RETURN i + 1;
END;
$$ LANGUAGE plpgsql;
DEBUG: switching to sequential query execution mode
DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands
SELECT * FROM table1 JOIN increment(2) val ON (id = val) ORDER BY id ASC;
DEBUG: function does not have co-located tables
DEBUG: generating subplan XXX_1 for subquery SELECT val FROM functions_in_joins.increment(2) val(val)
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT table1.id, table1.data, val.val FROM (functions_in_joins.table1 JOIN (SELECT intermediate_result.val FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(val integer)) val ON ((table1.id OPERATOR(pg_catalog.=) val.val))) ORDER BY table1.id
id | data | val
@ -71,6 +68,8 @@ DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT table1.id, ta
(1 row)
-- a function that returns a set of integers
-- Block distributing function as we have tests below to test it locally
SET citus.enable_metadata_sync TO OFF;
CREATE OR REPLACE FUNCTION next_k_integers(IN first_value INTEGER,
IN k INTEGER DEFAULT 3,
OUT result INTEGER)
@ -79,6 +78,7 @@ BEGIN
RETURN QUERY SELECT x FROM generate_series(first_value, first_value+k-1) f(x);
END;
$$ LANGUAGE plpgsql;
RESET citus.enable_metadata_sync;
SELECT *
FROM table1 JOIN next_k_integers(3,2) next_integers ON (id = next_integers.result)
ORDER BY id ASC;
@ -95,7 +95,10 @@ CREATE FUNCTION get_set_of_records() RETURNS SETOF RECORD AS $cmd$
SELECT x, x+1 FROM generate_series(0,4) f(x)
$cmd$
LANGUAGE SQL;
DEBUG: switching to sequential query execution mode
DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands
SELECT * FROM table1 JOIN get_set_of_records() AS t2(x int, y int) ON (id = x) ORDER BY id ASC;
DEBUG: function does not have co-located tables
DEBUG: generating subplan XXX_1 for subquery SELECT x, y FROM functions_in_joins.get_set_of_records() t2(x integer, y integer)
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT table1.id, table1.data, t2.x, t2.y FROM (functions_in_joins.table1 JOIN (SELECT intermediate_result.x, intermediate_result.y FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(x integer, y integer)) t2 ON ((table1.id OPERATOR(pg_catalog.=) t2.x))) ORDER BY table1.id
id | data | x | y
@ -110,7 +113,10 @@ DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT table1.id, ta
CREATE FUNCTION dup(int) RETURNS TABLE(f1 int, f2 text)
AS $$ SELECT $1, CAST($1 AS text) || ' is text' $$
LANGUAGE SQL;
DEBUG: switching to sequential query execution mode
DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands
SELECT f.* FROM table1 t JOIN dup(32) f ON (f1 = id);
DEBUG: function does not have co-located tables
DEBUG: generating subplan XXX_1 for subquery SELECT f1, f2 FROM functions_in_joins.dup(32) f(f1, f2)
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT f.f1, f.f2 FROM (functions_in_joins.table1 t JOIN (SELECT intermediate_result.f1, intermediate_result.f2 FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(f1 integer, f2 text)) f ON ((f.f1 OPERATOR(pg_catalog.=) t.id)))
f1 | f2
@ -121,7 +127,10 @@ DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT f.f1, f.f2 FR
-- a stable function
CREATE OR REPLACE FUNCTION the_minimum_id()
RETURNS INTEGER STABLE AS 'SELECT min(id) FROM table1' LANGUAGE SQL;
DEBUG: switching to sequential query execution mode
DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands
SELECT * FROM table1 JOIN the_minimum_id() min_id ON (id = min_id);
DEBUG: function does not have co-located tables
DEBUG: generating subplan XXX_1 for subquery SELECT min_id FROM functions_in_joins.the_minimum_id() min_id(min_id)
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT table1.id, table1.data, min_id.min_id FROM (functions_in_joins.table1 JOIN (SELECT intermediate_result.min_id FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(min_id integer)) min_id ON ((table1.id OPERATOR(pg_catalog.=) min_id.min_id)))
id | data | min_id
@ -182,7 +191,10 @@ begin
return result;
end;
$$ language plpgsql;
DEBUG: switching to sequential query execution mode
DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands
SELECT * FROM table1 JOIN max_and_min() m ON (m.maximum = data OR m.minimum = data) ORDER BY 1,2,3,4;
DEBUG: function does not have co-located tables
DEBUG: generating subplan XXX_1 for subquery SELECT minimum, maximum FROM functions_in_joins.max_and_min() m(minimum, maximum)
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT table1.id, table1.data, m.minimum, m.maximum FROM (functions_in_joins.table1 JOIN (SELECT intermediate_result.minimum, intermediate_result.maximum FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(minimum integer, maximum integer)) m ON (((m.maximum OPERATOR(pg_catalog.=) table1.data) OR (m.minimum OPERATOR(pg_catalog.=) table1.data)))) ORDER BY table1.id, table1.data, m.minimum, m.maximum
id | data | minimum | maximum
@ -200,6 +212,7 @@ SET client_min_messages TO ERROR;
-- function joins in CTE results can create lateral joins that are not supported
-- we execute the query within a function to consolidate the error messages
-- between different executors
SET citus.enable_metadata_sync TO OFF;
CREATE FUNCTION raise_failed_execution_func_join(query text) RETURNS void AS $$
BEGIN
EXECUTE query;
@ -211,6 +224,7 @@ BEGIN
END IF;
END;
$$LANGUAGE plpgsql;
RESET citus.enable_metadata_sync;
SELECT raise_failed_execution_func_join($$
WITH one_row AS (
SELECT * FROM table1 WHERE id=52
@ -221,8 +235,10 @@ SELECT raise_failed_execution_func_join($$
$$);
ERROR: Task failed to execute
-- a user-defined immutable function
SET citus.enable_metadata_sync TO OFF;
CREATE OR REPLACE FUNCTION the_answer_to_life()
RETURNS INTEGER IMMUTABLE AS 'SELECT 42' LANGUAGE SQL;
RESET citus.enable_metadata_sync;
SELECT raise_failed_execution_func_join($$
SELECT * FROM table1 JOIN the_answer_to_life() the_answer ON (id = the_answer);
$$);

View File

@ -127,6 +127,7 @@ WHERE
user_id < 0;
NOTICE: evaluating on master
-- make sure we don't evaluate stable functions with column arguments
SET citus.enable_metadata_sync TO OFF;
CREATE OR REPLACE FUNCTION evaluate_on_master(x int)
RETURNS int LANGUAGE plpgsql STABLE
AS $function$
@ -135,6 +136,7 @@ BEGIN
RETURN x;
END;
$function$;
RESET citus.enable_metadata_sync;
INSERT INTO raw_events_second (user_id, value_1)
SELECT
user_id, evaluate_on_master(value_1)

View File

@ -26,10 +26,12 @@ ALTER SEQUENCE pg_catalog.pg_dist_placement_placementid_seq RESTART 100000;
SELECT nextval('pg_catalog.pg_dist_groupid_seq') AS last_group_id \gset
SELECT nextval('pg_catalog.pg_dist_node_nodeid_seq') AS last_node_id \gset
-- Create the necessary test utility function
SET citus.enable_metadata_sync TO OFF;
CREATE FUNCTION activate_node_snapshot()
RETURNS text[]
LANGUAGE C STRICT
AS 'citus';
RESET citus.enable_metadata_sync;
COMMENT ON FUNCTION activate_node_snapshot()
IS 'commands to activate node snapshot';
-- Show that none of the existing tables are qualified to be MX tables
@ -89,12 +91,14 @@ SELECT unnest(activate_node_snapshot()) order by 1;
(27 rows)
-- this function is dropped in Citus10, added here for tests
SET citus.enable_metadata_sync TO OFF;
CREATE OR REPLACE FUNCTION pg_catalog.master_create_distributed_table(table_name regclass,
distribution_column text,
distribution_method citus.distribution_type)
RETURNS void
LANGUAGE C STRICT
AS 'citus', $$master_create_distributed_table$$;
RESET citus.enable_metadata_sync;
COMMENT ON FUNCTION pg_catalog.master_create_distributed_table(table_name regclass,
distribution_column text,
distribution_method citus.distribution_type)

View File

@ -467,13 +467,6 @@ UPDATE limit_orders SET placed_at = LEAST(placed_at, now()::timestamp) WHERE id
UPDATE limit_orders SET array_of_values = 1 || array_of_values WHERE id = 246;
CREATE FUNCTION immutable_append(old_values int[], new_value int)
RETURNS int[] AS $$ SELECT old_values || new_value $$ LANGUAGE SQL IMMUTABLE;
\c - - - :worker_1_port
CREATE FUNCTION immutable_append(old_values int[], new_value int)
RETURNS int[] AS $$ SELECT old_values || new_value $$ LANGUAGE SQL IMMUTABLE;
\c - - - :worker_2_port
CREATE FUNCTION immutable_append(old_values int[], new_value int)
RETURNS int[] AS $$ SELECT old_values || new_value $$ LANGUAGE SQL IMMUTABLE;
\c - - - :master_port
-- immutable function calls with vars are also allowed
UPDATE limit_orders
SET array_of_values = immutable_append(array_of_values, 2) WHERE id = 246;

View File

@ -329,22 +329,15 @@ SELECT count(*) FROM pg_dist_transaction;
(1 row)
-- create a check function
SELECT * from run_command_on_workers('CREATE FUNCTION reject_large_id() RETURNS trigger AS $rli$
CREATE FUNCTION reject_large_id() RETURNS trigger AS $rli$
BEGIN
IF (NEW.id > 30) THEN
RAISE ''illegal value'';
RAISE 'illegal value';
END IF;
RETURN NEW;
END;
$rli$ LANGUAGE plpgsql;')
ORDER BY nodeport;
nodename | nodeport | success | result
---------------------------------------------------------------------
localhost | 57637 | t | CREATE FUNCTION
localhost | 57638 | t | CREATE FUNCTION
(2 rows)
$rli$ LANGUAGE plpgsql;
-- register after insert trigger
SELECT * FROM run_command_on_placements('researchers', 'CREATE CONSTRAINT TRIGGER reject_large_researcher_id AFTER INSERT ON %s DEFERRABLE INITIALLY DEFERRED FOR EACH ROW EXECUTE PROCEDURE reject_large_id()')
ORDER BY nodeport, shardid;
@ -418,14 +411,7 @@ ORDER BY nodeport, shardid;
localhost | 57638 | 1200001 | t | DROP TRIGGER
(4 rows)
SELECT * FROM run_command_on_workers('drop function reject_large_id()')
ORDER BY nodeport;
nodename | nodeport | success | result
---------------------------------------------------------------------
localhost | 57637 | t | DROP FUNCTION
localhost | 57638 | t | DROP FUNCTION
(2 rows)
DROP FUNCTION reject_large_id();
-- ALTER and copy are compatible
BEGIN;
ALTER TABLE labs ADD COLUMN motto text;
@ -522,6 +508,7 @@ AND s.logicalrelid = 'objects'::regclass;
-- create trigger on one worker to reject certain values
\c - - - :worker_2_port
SET citus.enable_metadata_sync TO OFF;
CREATE FUNCTION reject_bad() RETURNS trigger AS $rb$
BEGIN
IF (NEW.name = 'BAD') THEN
@ -531,6 +518,7 @@ CREATE FUNCTION reject_bad() RETURNS trigger AS $rb$
RETURN NEW;
END;
$rb$ LANGUAGE plpgsql;
RESET citus.enable_metadata_sync;
CREATE CONSTRAINT TRIGGER reject_bad
AFTER INSERT ON objects_1200003
DEFERRABLE INITIALLY IMMEDIATE
@ -573,6 +561,7 @@ DELETE FROM objects;
-- there cannot be errors on different shards at different times
-- because the first failure will fail the whole transaction
\c - - - :worker_1_port
SET citus.enable_metadata_sync TO OFF;
CREATE FUNCTION reject_bad() RETURNS trigger AS $rb$
BEGIN
IF (NEW.name = 'BAD') THEN
@ -582,6 +571,7 @@ CREATE FUNCTION reject_bad() RETURNS trigger AS $rb$
RETURN NEW;
END;
$rb$ LANGUAGE plpgsql;
RESET citus.enable_metadata_sync;
CREATE CONSTRAINT TRIGGER reject_bad
AFTER INSERT ON labs_1200002
DEFERRABLE INITIALLY IMMEDIATE
@ -850,6 +840,7 @@ SELECT * FROM reference_modifying_xacts;
-- lets fail on of the workers at before the commit time
\c - - - :worker_1_port
SET citus.enable_metadata_sync TO OFF;
CREATE FUNCTION reject_bad_reference() RETURNS trigger AS $rb$
BEGIN
IF (NEW.key = 999) THEN
@ -859,6 +850,7 @@ CREATE FUNCTION reject_bad_reference() RETURNS trigger AS $rb$
RETURN NEW;
END;
$rb$ LANGUAGE plpgsql;
RESET citus.enable_metadata_sync;
CREATE CONSTRAINT TRIGGER reject_bad_reference
AFTER INSERT ON reference_modifying_xacts_1200006
DEFERRABLE INITIALLY IMMEDIATE
@ -938,6 +930,7 @@ INSERT INTO hash_modifying_xacts VALUES (2, 2);
ABORT;
-- lets fail one of the workers before COMMIT time for the hash table
\c - - - :worker_1_port
SET citus.enable_metadata_sync TO OFF;
CREATE FUNCTION reject_bad_hash() RETURNS trigger AS $rb$
BEGIN
IF (NEW.key = 997) THEN
@ -947,6 +940,7 @@ CREATE FUNCTION reject_bad_hash() RETURNS trigger AS $rb$
RETURN NEW;
END;
$rb$ LANGUAGE plpgsql;
RESET citus.enable_metadata_sync;
CREATE CONSTRAINT TRIGGER reject_bad_hash
AFTER INSERT ON hash_modifying_xacts_1200007
DEFERRABLE INITIALLY IMMEDIATE

View File

@ -622,10 +622,12 @@ ERROR: permission denied for function worker_cleanup_job_schema_cache
RESET ROLE;
-- to test access to files created during repartition we will create some on worker 1
\c - - - :worker_1_port
SET citus.enable_metadata_sync TO OFF;
CREATE OR REPLACE FUNCTION citus_rm_job_directory(bigint)
RETURNS void
AS 'citus'
LANGUAGE C STRICT;
RESET citus.enable_metadata_sync;
SET ROLE full_access;
SELECT worker_hash_partition_table(42,1,'SELECT a FROM generate_series(1,100) AS a', 'a', 23, ARRAY[-2147483648, -1073741824, 0, 1073741824]::int4[]);
worker_hash_partition_table
@ -636,10 +638,12 @@ SELECT worker_hash_partition_table(42,1,'SELECT a FROM generate_series(1,100) AS
RESET ROLE;
-- all attempts for transfer are initiated from other workers
\c - - - :worker_2_port
SET citus.enable_metadata_sync TO OFF;
CREATE OR REPLACE FUNCTION citus_rm_job_directory(bigint)
RETURNS void
AS 'citus'
LANGUAGE C STRICT;
RESET citus.enable_metadata_sync;
-- super user should not be able to copy files created by a user
SELECT worker_fetch_partition_file(42, 1, 1, 1, 'localhost', :worker_1_port);
WARNING: could not open file "base/pgsql_job_cache/job_0042/task_000001/p_00001.xxxx": No such file or directory

View File

@ -139,6 +139,8 @@ BEGIN
INSERT INTO mx_add_coordinator.ref(groupid) VALUES (gid);
END;
$$;
DEBUG: switching to sequential query execution mode
DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands
SELECT create_distributed_function('my_group_id()', colocate_with := 'ref');
DEBUG: switching to sequential query execution mode
DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands

View File

@ -402,6 +402,7 @@ select colocate_proc_with_table('mx_call_proc', 'mx_call_dist_table_1'::regclass
-- Test that we handle transactional constructs correctly inside a procedure
-- that is routed to the workers.
SET citus.enable_metadata_sync TO OFF;
CREATE PROCEDURE mx_call_proc_tx(x int) LANGUAGE plpgsql AS $$
BEGIN
INSERT INTO multi_mx_call.mx_call_dist_table_1 VALUES (x, -1), (x+1, 4);
@ -411,6 +412,7 @@ BEGIN
-- Now do the final update!
UPDATE multi_mx_call.mx_call_dist_table_1 SET val = val-1 WHERE id >= x;
END;$$;
RESET citus.enable_metadata_sync;
-- before distribution ...
CALL multi_mx_call.mx_call_proc_tx(10);
-- after distribution ...
@ -486,6 +488,8 @@ BEGIN
RAISE WARNING 'warning';
RAISE EXCEPTION 'error';
END;$$;
DEBUG: switching to sequential query execution mode
DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands
select create_distributed_function('mx_call_proc_raise(int)', '$1', 'mx_call_dist_table_1');
DEBUG: switching to sequential query execution mode
DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands
@ -552,6 +556,8 @@ SET client_min_messages TO DEBUG1;
--
CREATE FUNCTION mx_call_add(int, int) RETURNS int
AS 'select $1 + $2;' LANGUAGE SQL IMMUTABLE;
DEBUG: switching to sequential query execution mode
DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands
SELECT create_distributed_function('mx_call_add(int,int)');
DEBUG: switching to sequential query execution mode
DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands

View File

@ -402,6 +402,7 @@ select colocate_proc_with_table('mx_call_proc', 'mx_call_dist_table_1'::regclass
-- Test that we handle transactional constructs correctly inside a procedure
-- that is routed to the workers.
SET citus.enable_metadata_sync TO OFF;
CREATE PROCEDURE mx_call_proc_tx(x int) LANGUAGE plpgsql AS $$
BEGIN
INSERT INTO multi_mx_call.mx_call_dist_table_1 VALUES (x, -1), (x+1, 4);
@ -411,6 +412,7 @@ BEGIN
-- Now do the final update!
UPDATE multi_mx_call.mx_call_dist_table_1 SET val = val-1 WHERE id >= x;
END;$$;
RESET citus.enable_metadata_sync;
-- before distribution ...
CALL multi_mx_call.mx_call_proc_tx(10);
-- after distribution ...
@ -486,6 +488,8 @@ BEGIN
RAISE WARNING 'warning';
RAISE EXCEPTION 'error';
END;$$;
DEBUG: switching to sequential query execution mode
DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands
select create_distributed_function('mx_call_proc_raise(int)', '$1', 'mx_call_dist_table_1');
DEBUG: switching to sequential query execution mode
DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands
@ -552,6 +556,8 @@ SET client_min_messages TO DEBUG1;
--
CREATE FUNCTION mx_call_add(int, int) RETURNS int
AS 'select $1 + $2;' LANGUAGE SQL IMMUTABLE;
DEBUG: switching to sequential query execution mode
DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands
SELECT create_distributed_function('mx_call_add(int,int)');
DEBUG: switching to sequential query execution mode
DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands

View File

@ -63,30 +63,7 @@ CREATE TABLE citus_local_table(a int);
SELECT citus_add_local_table_to_metadata('citus_local_table');
ERROR: operation is not allowed on this node
HINT: Connect to the coordinator and run it again.
-- create UDFs in worker node
CREATE OR REPLACE FUNCTION simpleTestFunction(theValue integer)
RETURNS text AS
$$
DECLARE
strresult text;
BEGIN
RETURN theValue * 3 / 2 + 1;
END;
$$
LANGUAGE 'plpgsql' IMMUTABLE;
SET search_path TO citus_mx_test_schema;
CREATE OR REPLACE FUNCTION simpleTestFunction2(theValue integer)
RETURNS text AS
$$
DECLARE
strresult text;
BEGIN
RETURN theValue * 3 / 2 + 1;
END;
$$
LANGUAGE 'plpgsql' IMMUTABLE;
CREATE FUNCTION public.immutable_append_mx(old_values int[], new_value int)
RETURNS int[] AS $$ SELECT old_values || new_value $$ LANGUAGE SQL IMMUTABLE;
-- create operator
CREATE OPERATOR citus_mx_test_schema.=== (
LEFTARG = int,
@ -98,30 +75,7 @@ CREATE OPERATOR citus_mx_test_schema.=== (
);
-- now create required stuff in the worker 2
\c - - - :worker_2_port
-- create UDF
CREATE OR REPLACE FUNCTION simpleTestFunction(theValue integer)
RETURNS text AS
$$
DECLARE
strresult text;
BEGIN
RETURN theValue * 3 / 2 + 1;
END;
$$
LANGUAGE 'plpgsql' IMMUTABLE;
SET search_path TO citus_mx_test_schema;
CREATE OR REPLACE FUNCTION simpleTestFunction2(theValue integer)
RETURNS text AS
$$
DECLARE
strresult text;
BEGIN
RETURN theValue * 3 / 2 + 1;
END;
$$
LANGUAGE 'plpgsql' IMMUTABLE;
CREATE FUNCTION public.immutable_append_mx(old_values int[], new_value int)
RETURNS int[] AS $$ SELECT old_values || new_value $$ LANGUAGE SQL IMMUTABLE;
-- create operator
CREATE OPERATOR citus_mx_test_schema.=== (
LEFTARG = int,

View File

@ -13,6 +13,7 @@ VACUUM ANALYZE customer_mx;
VACUUM ANALYZE supplier_mx;
\c - - - :worker_1_port
-- Function that parses explain output as JSON
SET citus.enable_metadata_sync TO OFF;
CREATE FUNCTION explain_json(query text)
RETURNS jsonb
AS $BODY$
@ -36,6 +37,7 @@ END;
$BODY$ LANGUAGE plpgsql;
\c - - - :worker_2_port
-- Function that parses explain output as JSON
SET citus.enable_metadata_sync TO OFF;
CREATE FUNCTION explain_json(query text)
RETURNS jsonb
AS $BODY$
@ -57,6 +59,7 @@ BEGIN
RETURN result;
END;
$BODY$ LANGUAGE plpgsql;
RESET citus.enable_metadata_sync;
-- Test Text format
EXPLAIN (COSTS FALSE, FORMAT TEXT)
SELECT l_quantity, count(*) count_quantity FROM lineitem_mx

View File

@ -409,8 +409,11 @@ BEGIN
WHERE id >= x
ORDER BY 1, 2;
END;$$;
DEBUG: switching to sequential query execution mode
DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands
-- before distribution ...
select mx_call_func_tbl(10);
DEBUG: function does not have co-located tables
mx_call_func_tbl
---------------------------------------------------------------------
(10,-1)
@ -441,6 +444,8 @@ BEGIN
RAISE WARNING 'warning';
RAISE EXCEPTION 'error';
END;$$;
DEBUG: switching to sequential query execution mode
DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands
select create_distributed_function('mx_call_func_raise(int)', '$1', 'mx_call_dist_table_1');
DEBUG: switching to sequential query execution mode
DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands
@ -593,6 +598,8 @@ SET citus.shard_replication_factor = 1;
--
CREATE FUNCTION mx_call_add(int, int) RETURNS int
AS 'select $1 + $2;' LANGUAGE SQL IMMUTABLE;
DEBUG: switching to sequential query execution mode
DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands
SELECT create_distributed_function('mx_call_add(int,int)', '$1');
DEBUG: switching to sequential query execution mode
DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands

View File

@ -409,8 +409,11 @@ BEGIN
WHERE id >= x
ORDER BY 1, 2;
END;$$;
DEBUG: switching to sequential query execution mode
DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands
-- before distribution ...
select mx_call_func_tbl(10);
DEBUG: function does not have co-located tables
mx_call_func_tbl
---------------------------------------------------------------------
(10,-1)
@ -441,6 +444,8 @@ BEGIN
RAISE WARNING 'warning';
RAISE EXCEPTION 'error';
END;$$;
DEBUG: switching to sequential query execution mode
DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands
select create_distributed_function('mx_call_func_raise(int)', '$1', 'mx_call_dist_table_1');
DEBUG: switching to sequential query execution mode
DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands
@ -593,6 +598,8 @@ SET citus.shard_replication_factor = 1;
--
CREATE FUNCTION mx_call_add(int, int) RETURNS int
AS 'select $1 + $2;' LANGUAGE SQL IMMUTABLE;
DEBUG: switching to sequential query execution mode
DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands
SELECT create_distributed_function('mx_call_add(int,int)', '$1');
DEBUG: switching to sequential query execution mode
DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands

View File

@ -344,16 +344,6 @@ SELECT "Column", "Type", "Modifiers" FROM table_desc WHERE relid='distributed_mx
(3 rows)
\c - no_access_mx - :worker_1_port
-- see the comment in the top of the file
CREATE OR REPLACE FUNCTION raise_failed_aclcheck(query text) RETURNS void AS $$
BEGIN
EXECUTE query;
EXCEPTION WHEN OTHERS THEN
IF SQLERRM LIKE 'must be owner of%' THEN
RAISE 'must be owner of the object';
END IF;
END;
$$LANGUAGE plpgsql;
SELECT raise_failed_aclcheck($$
DROP TABLE distributed_mx_table;
$$);

View File

@ -266,6 +266,7 @@ UPDATE limit_orders_mx SET array_of_values = 1 || array_of_values WHERE id = 246
-- immutable function calls with vars are also allowed
UPDATE limit_orders_mx
SET array_of_values = immutable_append_mx(array_of_values, 2) WHERE id = 246;
SET citus.enable_metadata_sync TO OFF;
CREATE FUNCTION stable_append_mx(old_values int[], new_value int)
RETURNS int[] AS $$ BEGIN RETURN old_values || new_value; END; $$
LANGUAGE plpgsql STABLE;
@ -282,6 +283,7 @@ SELECT array_of_values FROM limit_orders_mx WHERE id = 246;
-- STRICT functions work as expected
CREATE FUNCTION temp_strict_func(integer,integer) RETURNS integer AS
'SELECT COALESCE($1, 2) + COALESCE($1, 3);' LANGUAGE SQL STABLE STRICT;
RESET citus.enable_metadata_sync;
UPDATE limit_orders_mx SET bidder_id = temp_strict_func(1, null) WHERE id = 246;
ERROR: null value in column "bidder_id" violates not-null constraint
SELECT array_of_values FROM limit_orders_mx WHERE id = 246;

View File

@ -227,6 +227,7 @@ SELECT * FROM objects_mx WHERE id = 1;
-- create trigger on one worker to reject certain values
\c - - - :worker_1_port
SET citus.enable_metadata_sync TO OFF;
CREATE FUNCTION reject_bad_mx() RETURNS trigger AS $rb$
BEGIN
IF (NEW.name = 'BAD') THEN
@ -236,6 +237,7 @@ CREATE FUNCTION reject_bad_mx() RETURNS trigger AS $rb$
RETURN NEW;
END;
$rb$ LANGUAGE plpgsql;
RESET citus.enable_metadata_sync;
CREATE CONSTRAINT TRIGGER reject_bad_mx
AFTER INSERT ON objects_mx_1220103
DEFERRABLE INITIALLY IMMEDIATE

View File

@ -829,10 +829,12 @@ SELECT master_update_node(nodeid, 'localhost', 12345) FROM pg_dist_node;
(1 row)
SET citus.enable_metadata_sync TO OFF;
CREATE OR REPLACE FUNCTION trigger_metadata_sync()
RETURNS void
LANGUAGE C STRICT
AS 'citus';
RESET citus.enable_metadata_sync;
SELECT trigger_metadata_sync();
trigger_metadata_sync
---------------------------------------------------------------------

View File

@ -52,13 +52,7 @@ CREATE TABLE repartition_udt_other (
-- so that the OID is off.
\c - - - :worker_1_port
-- START type creation
-- ... as well as a function to use as its comparator...
CREATE FUNCTION equal_test_udt_function(test_udt, test_udt) RETURNS boolean
AS 'select $1.i = $2.i AND $1.i2 = $2.i2;'
LANGUAGE SQL
IMMUTABLE
RETURNS NULL ON NULL INPUT;
-- ... use that function to create a custom equality operator...
-- Use function to create a custom equality operator...
CREATE OPERATOR = (
LEFTARG = test_udt,
RIGHTARG = test_udt,
@ -68,13 +62,6 @@ CREATE OPERATOR = (
);
-- ... and create a custom operator family for hash indexes...
CREATE OPERATOR FAMILY tudt_op_fam USING hash;
-- ... create a test HASH function. Though it is a poor hash function,
-- it is acceptable for our tests
CREATE FUNCTION test_udt_hash(test_udt) RETURNS int
AS 'SELECT hashtext( ($1.i + $1.i2)::text);'
LANGUAGE SQL
IMMUTABLE
RETURNS NULL ON NULL INPUT;
-- We need to define two different operator classes for the composite types
-- One uses BTREE the other uses HASH
CREATE OPERATOR CLASS tudt_op_fam_clas3
@ -87,13 +74,7 @@ FUNCTION 1 test_udt_hash(test_udt);
-- END type creation
\c - - - :worker_2_port
-- START type creation
-- ... as well as a function to use as its comparator...
CREATE FUNCTION equal_test_udt_function(test_udt, test_udt) RETURNS boolean
AS 'select $1.i = $2.i AND $1.i2 = $2.i2;'
LANGUAGE SQL
IMMUTABLE
RETURNS NULL ON NULL INPUT;
-- ... use that function to create a custom equality operator...
-- Use function to create a custom equality operator...
CREATE OPERATOR = (
LEFTARG = test_udt,
RIGHTARG = test_udt,
@ -103,13 +84,6 @@ CREATE OPERATOR = (
);
-- ... and create a custom operator family for hash indexes...
CREATE OPERATOR FAMILY tudt_op_fam USING hash;
-- ... create a test HASH function. Though it is a poor hash function,
-- it is acceptable for our tests
CREATE FUNCTION test_udt_hash(test_udt) RETURNS int
AS 'SELECT hashtext( ($1.i + $1.i2)::text);'
LANGUAGE SQL
IMMUTABLE
RETURNS NULL ON NULL INPUT;
-- We need to define two different operator classes for the composite types
-- One uses BTREE the other uses HASH
CREATE OPERATOR CLASS tudt_op_fam_clas3

View File

@ -1363,6 +1363,7 @@ DEBUG: query has a single distribution column value: 1
(5 rows)
-- queries inside plpgsql functions could be router plannable
SET citus.enable_metadata_sync TO OFF;
CREATE OR REPLACE FUNCTION author_articles_max_id() RETURNS int AS $$
DECLARE
max_id integer;
@ -1418,6 +1419,7 @@ PL/pgSQL function author_articles_id_word_count() line XX at RETURN QUERY
41 | 11814
(5 rows)
RESET citus.enable_metadata_sync;
-- materialized views can be created for router plannable queries
CREATE MATERIALIZED VIEW mv_articles_hash_mx AS
SELECT * FROM articles_hash_mx WHERE author_id = 1;

View File

@ -4,6 +4,7 @@
ALTER SEQUENCE pg_catalog.pg_dist_shardid_seq RESTART 225000;
SET citus.shard_count TO 2;
-- this function is dropped in Citus10, added here for tests
SET citus.enable_metadata_sync TO OFF;
CREATE OR REPLACE FUNCTION pg_catalog.master_create_distributed_table(table_name regclass,
distribution_column text,
distribution_method citus.distribution_type)
@ -20,6 +21,7 @@ CREATE OR REPLACE FUNCTION pg_catalog.master_create_worker_shards(table_name tex
RETURNS void
AS 'citus', $$master_create_worker_shards$$
LANGUAGE C STRICT;
RESET citus.enable_metadata_sync;
-- Verify that a table name > 56 characters gets hashed properly.
CREATE TABLE too_long_12345678901234567890123456789012345678901234567890 (
col1 integer not null,

View File

@ -1132,10 +1132,6 @@ DROP TABLE http_request;
-- (re-)planning, but not when executing.
-- first create helper function
CREATE OR REPLACE FUNCTION immutable_bleat(text) RETURNS int LANGUAGE plpgsql IMMUTABLE AS $$BEGIN RAISE NOTICE '%', $1;RETURN 1;END$$;
\c - - - :worker_1_port
CREATE OR REPLACE FUNCTION immutable_bleat(text) RETURNS int LANGUAGE plpgsql IMMUTABLE AS $$BEGIN RAISE NOTICE '%', $1;RETURN 1;END$$;
\c - - - :worker_2_port
CREATE OR REPLACE FUNCTION immutable_bleat(text) RETURNS int LANGUAGE plpgsql IMMUTABLE AS $$BEGIN RAISE NOTICE '%', $1;RETURN 1;END$$;
\c - - - :master_port
-- test table
CREATE TABLE test_table (test_id integer NOT NULL, data text);

View File

@ -53,13 +53,7 @@ CREATE TABLE repartition_udt_other (
-- so that the OID is off.
\c - - :public_worker_1_host :worker_1_port
-- START type creation
-- ... as well as a function to use as its comparator...
CREATE FUNCTION equal_test_udt_function(test_udt, test_udt) RETURNS boolean
AS 'select $1.i = $2.i AND $1.i2 = $2.i2;'
LANGUAGE SQL
IMMUTABLE
RETURNS NULL ON NULL INPUT;
-- ... use that function to create a custom equality operator...
-- Use function to create a custom equality operator...
CREATE OPERATOR = (
LEFTARG = test_udt,
RIGHTARG = test_udt,
@ -69,13 +63,6 @@ CREATE OPERATOR = (
);
-- ... and create a custom operator family for hash indexes...
CREATE OPERATOR FAMILY tudt_op_fam USING hash;
-- ... create a test HASH function. Though it is a poor hash function,
-- it is acceptable for our tests
CREATE FUNCTION test_udt_hash(test_udt) RETURNS int
AS 'SELECT hashtext( ($1.i + $1.i2)::text);'
LANGUAGE SQL
IMMUTABLE
RETURNS NULL ON NULL INPUT;
-- We need to define two different operator classes for the composite types
-- One uses BTREE the other uses HASH
CREATE OPERATOR CLASS tudt_op_fam_clas3
@ -88,13 +75,7 @@ FUNCTION 1 test_udt_hash(test_udt);
-- END type creation
\c - - :public_worker_2_host :worker_2_port
-- START type creation
-- ... as well as a function to use as its comparator...
CREATE FUNCTION equal_test_udt_function(test_udt, test_udt) RETURNS boolean
AS 'select $1.i = $2.i AND $1.i2 = $2.i2;'
LANGUAGE SQL
IMMUTABLE
RETURNS NULL ON NULL INPUT;
-- ... use that function to create a custom equality operator...
-- Use function to create a custom equality operator...
CREATE OPERATOR = (
LEFTARG = test_udt,
RIGHTARG = test_udt,
@ -104,13 +85,6 @@ CREATE OPERATOR = (
);
-- ... and create a custom operator family for hash indexes...
CREATE OPERATOR FAMILY tudt_op_fam USING hash;
-- ... create a test HASH function. Though it is a poor hash function,
-- it is acceptable for our tests
CREATE FUNCTION test_udt_hash(test_udt) RETURNS int
AS 'SELECT hashtext( ($1.i + $1.i2)::text);'
LANGUAGE SQL
IMMUTABLE
RETURNS NULL ON NULL INPUT;
-- We need to define two different operator classes for the composite types
-- One uses BTREE the other uses HASH
CREATE OPERATOR CLASS tudt_op_fam_clas3

View File

@ -13,28 +13,6 @@ LANGUAGE sql IMMUTABLE AS $_$
ORDER BY 1 LIMIT 2 - MOD(array_upper($1, 1), 2)
OFFSET CEIL(array_upper($1, 1) / 2.0) - 1) sub;
$_$;
\c - - :public_worker_1_host :worker_1_port
DROP FUNCTION IF EXISTS median(double precision[]);
NOTICE: function median(pg_catalog.float8[]) does not exist, skipping
CREATE FUNCTION median(double precision[]) RETURNS double precision
LANGUAGE sql IMMUTABLE AS $_$
SELECT AVG(val) FROM
(SELECT val FROM unnest($1) val
ORDER BY 1 LIMIT 2 - MOD(array_upper($1, 1), 2)
OFFSET CEIL(array_upper($1, 1) / 2.0) - 1) sub;
$_$;
\c - - :public_worker_2_host :worker_2_port
DROP FUNCTION IF EXISTS median(double precision[]);
NOTICE: function median(pg_catalog.float8[]) does not exist, skipping
CREATE FUNCTION median(double precision[]) RETURNS double precision
LANGUAGE sql IMMUTABLE AS $_$
SELECT AVG(val) FROM
(SELECT val FROM unnest($1) val
ORDER BY 1 LIMIT 2 - MOD(array_upper($1, 1), 2)
OFFSET CEIL(array_upper($1, 1) / 2.0) - 1) sub;
$_$;
-- Run query on master
\c - - :master_host :master_port
SELECT * FROM (SELECT median(ARRAY[1,2,sum(l_suppkey)]) as median, count(*)
FROM lineitem GROUP BY l_partkey) AS a
WHERE median > 2;

View File

@ -1592,10 +1592,13 @@ BEGIN
RETURN QUERY SELECT 1;
END;
$$ LANGUAGE plpgsql;
DEBUG: switching to sequential query execution mode
DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands
SELECT 1 FROM authors_reference r JOIN (
SELECT s.datid FROM number1() s LEFT JOIN pg_database d ON s.datid = d.oid
) num_db ON (r.id = num_db.datid) LIMIT 1;
DEBUG: found no worker with all shard placements
DEBUG: function does not have co-located tables
DEBUG: generating subplan XXX_1 for subquery SELECT datid FROM public.number1() s(datid)
DEBUG: Creating router plan
DEBUG: generating subplan XXX_2 for subquery SELECT s.datid FROM ((SELECT intermediate_result.datid FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(datid integer)) s LEFT JOIN pg_database d ON (((s.datid)::oid OPERATOR(pg_catalog.=) d.oid)))
@ -1610,6 +1613,7 @@ CREATE VIEW num_db AS
SELECT s.datid FROM number1() s LEFT JOIN pg_database d ON s.datid = d.oid;
SELECT 1 FROM authors_reference r JOIN num_db ON (r.id = num_db.datid) LIMIT 1;
DEBUG: found no worker with all shard placements
DEBUG: function does not have co-located tables
DEBUG: generating subplan XXX_1 for subquery SELECT datid FROM public.number1() s(datid)
DEBUG: Creating router plan
DEBUG: generating subplan XXX_2 for subquery SELECT s.datid FROM ((SELECT intermediate_result.datid FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(datid integer)) s LEFT JOIN pg_database d ON (((s.datid)::oid OPERATOR(pg_catalog.=) d.oid)))
@ -2014,6 +2018,8 @@ BEGIN
END IF;
END;
$$LANGUAGE plpgsql;
DEBUG: switching to sequential query execution mode
DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands
SET client_min_messages TO ERROR;
\set VERBOSITY terse
SELECT raise_failed_execution_router($$
@ -2307,7 +2313,10 @@ BEGIN
return max_id;
END;
$$ LANGUAGE plpgsql;
DEBUG: switching to sequential query execution mode
DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands
SELECT author_articles_max_id();
DEBUG: function does not have co-located tables
DEBUG: Creating router plan
CONTEXT: SQL statement "SELECT MAX(id) FROM articles_hash ah
WHERE author_id = 1"
@ -2332,7 +2341,10 @@ BEGIN
END;
$$ LANGUAGE plpgsql;
DEBUG: switching to sequential query execution mode
DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands
SELECT * FROM author_articles_id_word_count() ORDER BY 1;
DEBUG: function does not have co-located tables
DEBUG: Creating router plan
CONTEXT: SQL statement "SELECT ah.id, ah.word_count
FROM articles_hash ah

View File

@ -1225,6 +1225,7 @@ DEBUG: query has a single distribution column value: 1
(0 rows)
-- create a dummy function to be used in filtering
SET citus.enable_metadata_sync TO OFF;
CREATE OR REPLACE FUNCTION someDummyFunction(regclass)
RETURNS text AS
$$
@ -1247,6 +1248,7 @@ BEGIN
END IF;
END;
$$LANGUAGE plpgsql;
RESET citus.enable_metadata_sync;
SET client_min_messages TO ERROR;
\set VERBOSITY terse
-- fast path router plannable, but errors
@ -1611,11 +1613,14 @@ BEGIN
return max_id;
END;
$$ LANGUAGE plpgsql;
DEBUG: switching to sequential query execution mode
DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands
-- we don't want too many details. though we're omitting
-- "DETAIL: distribution column value:", we see it acceptable
-- since the query results verifies the correctness
\set VERBOSITY terse
SELECT author_articles_max_id();
DEBUG: function does not have co-located tables
DEBUG: Distributed planning for a fast-path router query
DEBUG: Creating router plan
DEBUG: query has a single distribution column value: 1
@ -1625,30 +1630,35 @@ DEBUG: query has a single distribution column value: 1
(1 row)
SELECT author_articles_max_id();
DEBUG: function does not have co-located tables
author_articles_max_id
---------------------------------------------------------------------
41
(1 row)
SELECT author_articles_max_id();
DEBUG: function does not have co-located tables
author_articles_max_id
---------------------------------------------------------------------
41
(1 row)
SELECT author_articles_max_id();
DEBUG: function does not have co-located tables
author_articles_max_id
---------------------------------------------------------------------
41
(1 row)
SELECT author_articles_max_id();
DEBUG: function does not have co-located tables
author_articles_max_id
---------------------------------------------------------------------
41
(1 row)
SELECT author_articles_max_id();
DEBUG: function does not have co-located tables
author_articles_max_id
---------------------------------------------------------------------
41
@ -1665,7 +1675,9 @@ BEGIN
return max_id;
END;
$$ LANGUAGE plpgsql;
DEBUG: switching to sequential query execution mode
SELECT author_articles_max_id(1);
DEBUG: function does not have co-located tables
DEBUG: Deferred pruning for a fast-path router query
DEBUG: Creating router plan
author_articles_max_id
@ -1674,6 +1686,7 @@ DEBUG: Creating router plan
(1 row)
SELECT author_articles_max_id(1);
DEBUG: function does not have co-located tables
DEBUG: Deferred pruning for a fast-path router query
DEBUG: Creating router plan
author_articles_max_id
@ -1682,6 +1695,7 @@ DEBUG: Creating router plan
(1 row)
SELECT author_articles_max_id(1);
DEBUG: function does not have co-located tables
DEBUG: Deferred pruning for a fast-path router query
DEBUG: Creating router plan
author_articles_max_id
@ -1690,6 +1704,7 @@ DEBUG: Creating router plan
(1 row)
SELECT author_articles_max_id(1);
DEBUG: function does not have co-located tables
DEBUG: Deferred pruning for a fast-path router query
DEBUG: Creating router plan
author_articles_max_id
@ -1698,6 +1713,7 @@ DEBUG: Creating router plan
(1 row)
SELECT author_articles_max_id(1);
DEBUG: function does not have co-located tables
DEBUG: Deferred pruning for a fast-path router query
DEBUG: Creating router plan
author_articles_max_id
@ -1706,6 +1722,7 @@ DEBUG: Creating router plan
(1 row)
SELECT author_articles_max_id(1);
DEBUG: function does not have co-located tables
DEBUG: Deferred pruning for a fast-path router query
DEBUG: Creating router plan
author_articles_max_id
@ -1724,7 +1741,9 @@ BEGIN
END;
$$ LANGUAGE plpgsql;
DEBUG: switching to sequential query execution mode
SELECT * FROM author_articles_id_word_count();
DEBUG: function does not have co-located tables
DEBUG: Distributed planning for a fast-path router query
DEBUG: Creating router plan
DEBUG: query has a single distribution column value: 1
@ -1738,6 +1757,7 @@ DEBUG: query has a single distribution column value: 1
(5 rows)
SELECT * FROM author_articles_id_word_count();
DEBUG: function does not have co-located tables
id | word_count
---------------------------------------------------------------------
1 | 9572
@ -1748,6 +1768,7 @@ SELECT * FROM author_articles_id_word_count();
(5 rows)
SELECT * FROM author_articles_id_word_count();
DEBUG: function does not have co-located tables
id | word_count
---------------------------------------------------------------------
1 | 9572
@ -1758,6 +1779,7 @@ SELECT * FROM author_articles_id_word_count();
(5 rows)
SELECT * FROM author_articles_id_word_count();
DEBUG: function does not have co-located tables
id | word_count
---------------------------------------------------------------------
1 | 9572
@ -1768,6 +1790,7 @@ SELECT * FROM author_articles_id_word_count();
(5 rows)
SELECT * FROM author_articles_id_word_count();
DEBUG: function does not have co-located tables
id | word_count
---------------------------------------------------------------------
1 | 9572
@ -1778,6 +1801,7 @@ SELECT * FROM author_articles_id_word_count();
(5 rows)
SELECT * FROM author_articles_id_word_count();
DEBUG: function does not have co-located tables
id | word_count
---------------------------------------------------------------------
1 | 9572
@ -1798,7 +1822,9 @@ BEGIN
END;
$$ LANGUAGE plpgsql;
DEBUG: switching to sequential query execution mode
SELECT * FROM author_articles_id_word_count(1);
DEBUG: function does not have co-located tables
DEBUG: Deferred pruning for a fast-path router query
DEBUG: Creating router plan
id | word_count
@ -1811,6 +1837,7 @@ DEBUG: Creating router plan
(5 rows)
SELECT * FROM author_articles_id_word_count(1);
DEBUG: function does not have co-located tables
DEBUG: Deferred pruning for a fast-path router query
DEBUG: Creating router plan
id | word_count
@ -1823,6 +1850,7 @@ DEBUG: Creating router plan
(5 rows)
SELECT * FROM author_articles_id_word_count(1);
DEBUG: function does not have co-located tables
DEBUG: Deferred pruning for a fast-path router query
DEBUG: Creating router plan
id | word_count
@ -1835,6 +1863,7 @@ DEBUG: Creating router plan
(5 rows)
SELECT * FROM author_articles_id_word_count(1);
DEBUG: function does not have co-located tables
DEBUG: Deferred pruning for a fast-path router query
DEBUG: Creating router plan
id | word_count
@ -1847,6 +1876,7 @@ DEBUG: Creating router plan
(5 rows)
SELECT * FROM author_articles_id_word_count(1);
DEBUG: function does not have co-located tables
DEBUG: Deferred pruning for a fast-path router query
DEBUG: Creating router plan
id | word_count
@ -1859,6 +1889,7 @@ DEBUG: Creating router plan
(5 rows)
SELECT * FROM author_articles_id_word_count(1);
DEBUG: function does not have co-located tables
DEBUG: Deferred pruning for a fast-path router query
DEBUG: Creating router plan
id | word_count

View File

@ -194,31 +194,6 @@ BEGIN
END;
$$
LANGUAGE 'plpgsql' IMMUTABLE;
-- create UDF in worker node 1
\c - - - :worker_1_port
CREATE OR REPLACE FUNCTION dummyFunction(theValue integer)
RETURNS text AS
$$
DECLARE
strresult text;
BEGIN
RETURN theValue * 3 / 2 + 1;
END;
$$
LANGUAGE 'plpgsql' IMMUTABLE;
-- create UDF in worker node 2
\c - - - :worker_2_port
CREATE OR REPLACE FUNCTION dummyFunction(theValue integer)
RETURNS text AS
$$
DECLARE
strresult text;
BEGIN
RETURN theValue * 3 / 2 + 1;
END;
$$
LANGUAGE 'plpgsql' IMMUTABLE;
\c - - - :master_port
-- UDF in public, table in a schema other than public, search_path is not set
SELECT dummyFunction(n_nationkey) FROM test_schema_support.nation_hash GROUP BY 1 ORDER BY 1;
dummyfunction
@ -260,33 +235,6 @@ BEGIN
END;
$$
LANGUAGE 'plpgsql' IMMUTABLE;
-- create UDF in worker node 1 in schema
\c - - - :worker_1_port
SET search_path TO test_schema_support;
CREATE OR REPLACE FUNCTION dummyFunction2(theValue integer)
RETURNS text AS
$$
DECLARE
strresult text;
BEGIN
RETURN theValue * 3 / 2 + 1;
END;
$$
LANGUAGE 'plpgsql' IMMUTABLE;
-- create UDF in worker node 2 in schema
\c - - - :worker_2_port
SET search_path TO test_schema_support;
CREATE OR REPLACE FUNCTION dummyFunction2(theValue integer)
RETURNS text AS
$$
DECLARE
strresult text;
BEGIN
RETURN theValue * 3 / 2 + 1;
END;
$$
LANGUAGE 'plpgsql' IMMUTABLE;
\c - - - :master_port
-- UDF in schema, table in a schema other than public, search_path is not set
SET search_path TO public;
SELECT test_schema_support.dummyFunction2(n_nationkey) FROM test_schema_support.nation_hash GROUP BY 1 ORDER BY 1;

View File

@ -1563,22 +1563,8 @@ LIMIT 10;
(10 rows)
-- this is one complex join query derived from a user's production query
-- first declare the function on workers on master
-- declare the function on workers and master
-- With array_index:
SELECT * FROM run_command_on_workers('CREATE OR REPLACE FUNCTION array_index(ANYARRAY, ANYELEMENT)
RETURNS INT AS $$
SELECT i
FROM (SELECT generate_series(array_lower($1, 1), array_upper($1, 1))) g(i)
WHERE $1 [i] = $2
LIMIT 1;
$$ LANGUAGE sql')
ORDER BY 1,2;
nodename | nodeport | success | result
---------------------------------------------------------------------
localhost | 57637 | t | CREATE FUNCTION
localhost | 57638 | t | CREATE FUNCTION
(2 rows)
CREATE OR REPLACE FUNCTION array_index(ANYARRAY, ANYELEMENT)
RETURNS INT AS $$
SELECT i
@ -1745,14 +1731,6 @@ LIMIT 10;
(10 rows)
-- drop created functions
SELECT * FROM run_command_on_workers('DROP FUNCTION array_index(ANYARRAY, ANYELEMENT)')
ORDER BY 1,2;
nodename | nodeport | success | result
---------------------------------------------------------------------
localhost | 57637 | t | DROP FUNCTION
localhost | 57638 | t | DROP FUNCTION
(2 rows)
DROP FUNCTION array_index(ANYARRAY, ANYELEMENT);
-- a query with a constant subquery
SELECT count(*) as subquery_count
@ -1883,6 +1861,8 @@ CREATE OR REPLACE FUNCTION volatile_func_test()
RETURNS INT AS $$
SELECT 1;
$$ LANGUAGE sql VOLATILE;
DEBUG: switching to sequential query execution mode
DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands
-- Citus should be able to evalute functions/row comparisons on the LIMIT/OFFSET
SELECT user_id, array_length(events_table, 1)
FROM (
@ -1987,21 +1967,6 @@ CREATE FUNCTION test_join_function_2(integer, integer) RETURNS bool
LANGUAGE SQL
IMMUTABLE
RETURNS NULL ON NULL INPUT;
SELECT run_command_on_workers($f$
CREATE FUNCTION test_join_function_2(integer, integer) RETURNS bool
AS 'select $1 > $2;'
LANGUAGE SQL
IMMUTABLE
RETURNS NULL ON NULL INPUT;
$f$);
run_command_on_workers
---------------------------------------------------------------------
(localhost,57637,t,"CREATE FUNCTION")
(localhost,57638,t,"CREATE FUNCTION")
(2 rows)
-- we don't support joins via functions
SELECT user_id, array_length(events_table, 1)
FROM (
@ -2285,16 +2250,5 @@ LIMIT 1;
(1 row)
DROP FUNCTION test_join_function_2(integer, integer);
SELECT run_command_on_workers($f$
DROP FUNCTION test_join_function_2(integer, integer);
$f$);
run_command_on_workers
---------------------------------------------------------------------
(localhost,57637,t,"DROP FUNCTION")
(localhost,57638,t,"DROP FUNCTION")
(2 rows)
SET citus.enable_router_execution TO TRUE;
SET citus.subquery_pushdown to OFF;

View File

@ -38,6 +38,7 @@ SELECT dist_node_check.matches AND dist_placement_check.matches
FROM dist_node_check CROSS JOIN dist_placement_check
$$;
-- partition_task_list_results tests the internal PartitionTasklistResults function
SET citus.enable_metadata_sync TO OFF;
CREATE OR REPLACE FUNCTION pg_catalog.partition_task_list_results(resultIdPrefix text,
query text,
target_table regclass,
@ -63,3 +64,4 @@ BEGIN
PERFORM public.wait_until_metadata_sync(30000);
END;
$function$;
RESET citus.enable_metadata_sync;

View File

@ -292,11 +292,13 @@ SET citus.override_table_visibility TO false;
(1 row)
-- test DROP table with failing worker
SET citus.enable_metadata_sync TO OFF;
CREATE FUNCTION fail_drop_table() RETURNS event_trigger AS $fdt$
BEGIN
RAISE 'illegal value';
END;
$fdt$ LANGUAGE plpgsql;
RESET citus.enable_metadata_sync;
CREATE EVENT TRIGGER fail_drop_table ON sql_drop EXECUTE PROCEDURE fail_drop_table();
\c - - - :master_port
\set VERBOSITY terse

View File

@ -62,6 +62,7 @@ SELECT * FROM mx_table ORDER BY col_1;
-- Try commands from metadata worker
\c - - - :worker_1_port
-- this function is dropped in Citus10, added here for tests
SET citus.enable_metadata_sync TO OFF;
CREATE OR REPLACE FUNCTION pg_catalog.master_create_distributed_table(table_name regclass,
distribution_column text,
distribution_method citus.distribution_type)
@ -78,6 +79,7 @@ CREATE OR REPLACE FUNCTION pg_catalog.master_create_worker_shards(table_name tex
RETURNS void
AS 'citus', $$master_create_worker_shards$$
LANGUAGE C STRICT;
RESET citus.enable_metadata_sync;
CREATE TABLE mx_table_worker(col_1 text);
-- master_create_distributed_table
SELECT master_create_distributed_table('mx_table_worker', 'col_1', 'hash');

View File

@ -22,6 +22,7 @@ BEGIN
RETURN result;
END;
$BODY$ LANGUAGE plpgsql;
DEBUG: switching to sequential query execution mode
SHOW log_error_verbosity;
log_error_verbosity
---------------------------------------------------------------------
@ -36,6 +37,7 @@ FROM
(SELECT users_table.user_id FROM users_table, events_table WHERE users_table.user_id = events_table.user_id AND event_type IN (5,6,7,8)) as bar
WHERE
foo.user_id = bar.user_id;$$);
DEBUG: function does not have co-located tables
DEBUG: generating subplan XXX_1 for subquery SELECT users_table.user_id, random() AS random FROM public.users_table, public.events_table WHERE ((users_table.user_id OPERATOR(pg_catalog.=) events_table.value_2) AND (events_table.event_type OPERATOR(pg_catalog.=) ANY (ARRAY[1, 2, 3, 4])))
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (SELECT intermediate_result.user_id, intermediate_result.random FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(user_id integer, random double precision)) foo, (SELECT users_table.user_id FROM public.users_table, public.events_table WHERE ((users_table.user_id OPERATOR(pg_catalog.=) events_table.user_id) AND (events_table.event_type OPERATOR(pg_catalog.=) ANY (ARRAY[5, 6, 7, 8])))) bar WHERE (foo.user_id OPERATOR(pg_catalog.=) bar.user_id)
valid
@ -51,6 +53,7 @@ FROM
(SELECT users_table.user_id FROM users_table, events_table WHERE users_table.user_id = events_table.value_2 AND event_type IN (5,6,7,8)) as bar
WHERE
foo.user_id = bar.user_id;$$);
DEBUG: function does not have co-located tables
DEBUG: generating subplan XXX_1 for subquery SELECT users_table.user_id, random() AS random FROM public.users_table, public.events_table WHERE ((users_table.user_id OPERATOR(pg_catalog.=) events_table.value_2) AND (events_table.event_type OPERATOR(pg_catalog.=) ANY (ARRAY[1, 2, 3, 4])))
DEBUG: generating subplan XXX_2 for subquery SELECT users_table.user_id FROM public.users_table, public.events_table WHERE ((users_table.user_id OPERATOR(pg_catalog.=) events_table.value_2) AND (events_table.event_type OPERATOR(pg_catalog.=) ANY (ARRAY[5, 6, 7, 8])))
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (SELECT intermediate_result.user_id, intermediate_result.random FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(user_id integer, random double precision)) foo, (SELECT intermediate_result.user_id FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(user_id integer)) bar WHERE (foo.user_id OPERATOR(pg_catalog.=) bar.user_id)
@ -73,6 +76,7 @@ WHERE
users_table, events_table
WHERE
users_table.user_id = events_table.value_2 AND event_type IN (5,6));$$);
DEBUG: function does not have co-located tables
DEBUG: generating subplan XXX_1 for subquery SELECT users_table.user_id FROM public.users_table, public.events_table WHERE ((users_table.user_id OPERATOR(pg_catalog.=) events_table.value_2) AND (events_table.event_type OPERATOR(pg_catalog.=) ANY (ARRAY[5, 6])))
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM public.users_table WHERE (value_1 OPERATOR(pg_catalog.=) ANY (SELECT intermediate_result.user_id FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(user_id integer)))
valid
@ -89,6 +93,7 @@ SELECT count(*) FROM q1, (SELECT
users_table, events_table
WHERE
users_table.user_id = events_table.value_2 AND event_type IN (1,2,3,4)) as bar WHERE bar.user_id = q1.user_id ;$$);
DEBUG: function does not have co-located tables
DEBUG: CTE q1 is going to be inlined via distributed planning
DEBUG: generating subplan XXX_1 for subquery SELECT users_table.user_id, random() AS random FROM public.users_table, public.events_table WHERE ((users_table.user_id OPERATOR(pg_catalog.=) events_table.value_2) AND (events_table.event_type OPERATOR(pg_catalog.=) ANY (ARRAY[1, 2, 3, 4])))
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (SELECT users_table.user_id FROM public.users_table) q1, (SELECT intermediate_result.user_id, intermediate_result.random FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(user_id integer, random double precision)) bar WHERE (bar.user_id OPERATOR(pg_catalog.=) q1.user_id)
@ -101,6 +106,7 @@ DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS c
SELECT true AS valid FROM explain_json($$
(SELECT users_table.user_id FROM users_table, events_table WHERE users_table.user_id = events_table.value_2 AND event_type IN (1,2,3,4)) UNION
(SELECT users_table.user_id FROM users_table, events_table WHERE users_table.user_id = events_table.user_id AND event_type IN (5,6,7,8));$$);
DEBUG: function does not have co-located tables
DEBUG: generating subplan XXX_1 for subquery SELECT users_table.user_id FROM public.users_table, public.events_table WHERE ((users_table.user_id OPERATOR(pg_catalog.=) events_table.value_2) AND (events_table.event_type OPERATOR(pg_catalog.=) ANY (ARRAY[1, 2, 3, 4])))
DEBUG: generating subplan XXX_2 for subquery SELECT users_table.user_id FROM public.users_table, public.events_table WHERE ((users_table.user_id OPERATOR(pg_catalog.=) events_table.user_id) AND (events_table.event_type OPERATOR(pg_catalog.=) ANY (ARRAY[5, 6, 7, 8])))
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT intermediate_result.user_id FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(user_id integer) UNION SELECT intermediate_result.user_id FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(user_id integer)
@ -137,6 +143,7 @@ FROM (
) q
ORDER BY 2 DESC, 1;
$$);
DEBUG: function does not have co-located tables
DEBUG: generating subplan XXX_1 for subquery SELECT users_table.user_id FROM public.users_table, public.events_table WHERE ((users_table.user_id OPERATOR(pg_catalog.=) events_table.value_2) AND (events_table.event_type OPERATOR(pg_catalog.=) ANY (ARRAY[1, 2, 3, 4])))
DEBUG: push down of limit count: 5
DEBUG: generating subplan XXX_2 for subquery SELECT user_id FROM public.users_table WHERE ((value_2 OPERATOR(pg_catalog.>=) 5) AND (EXISTS (SELECT intermediate_result.user_id FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(user_id integer)))) LIMIT 5
@ -158,6 +165,7 @@ FROM
(SELECT users_table.user_id, value_1 FROM users_table, events_table WHERE users_table.user_id = events_table.user_id AND event_type IN (5,6,7,8)) as bar
WHERE
foo.user_id = bar.value_1;$$);
DEBUG: function does not have co-located tables
DEBUG: generating subplan XXX_1 for subquery SELECT users_table.user_id, users_table.value_1 FROM public.users_table, public.events_table WHERE ((users_table.user_id OPERATOR(pg_catalog.=) events_table.user_id) AND (events_table.event_type OPERATOR(pg_catalog.=) ANY (ARRAY[5, 6, 7, 8])))
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (SELECT users_table.user_id, random() AS random FROM public.users_table, public.events_table WHERE ((users_table.user_id OPERATOR(pg_catalog.=) events_table.user_id) AND (events_table.event_type OPERATOR(pg_catalog.=) ANY (ARRAY[1, 2, 3, 4])))) foo, (SELECT intermediate_result.user_id, intermediate_result.value_1 FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(user_id integer, value_1 integer)) bar WHERE (foo.user_id OPERATOR(pg_catalog.=) bar.value_1)
valid

View File

@ -32,6 +32,7 @@ BEGIN
RETURN result;
END;
$BODY$ LANGUAGE plpgsql;
DEBUG: switching to sequential query execution mode
-- leaf queries contain colocated joins
-- but not the subquery
SELECT true AS valid FROM explain_json_2($$
@ -43,6 +44,7 @@ SELECT true AS valid FROM explain_json_2($$
WHERE
foo.value_2 = bar.value_2;
$$);
DEBUG: function does not have co-located tables
DEBUG: generating subplan XXX_1 for subquery SELECT users_table.value_2 FROM public.users_table, public.events_table WHERE ((users_table.user_id OPERATOR(pg_catalog.=) events_table.user_id) AND (events_table.event_type OPERATOR(pg_catalog.=) ANY (ARRAY[5, 6, 7, 8])))
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT foo.value_2 FROM (SELECT users_table.value_2 FROM public.users_table, public.events_table WHERE ((users_table.user_id OPERATOR(pg_catalog.=) events_table.user_id) AND (events_table.event_type OPERATOR(pg_catalog.=) ANY (ARRAY[1, 2, 3, 4])))) foo, (SELECT intermediate_result.value_2 FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(value_2 integer)) bar WHERE (foo.value_2 OPERATOR(pg_catalog.=) bar.value_2)
valid
@ -63,6 +65,7 @@ SELECT true AS valid FROM explain_json_2($$
(SELECT event_type FROM events_table WHERE user_id < 100);
$$);
DEBUG: function does not have co-located tables
DEBUG: generating subplan XXX_1 for subquery SELECT event_type FROM public.events_table WHERE (user_id OPERATOR(pg_catalog.<) 100)
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM public.events_table WHERE (event_type OPERATOR(pg_catalog.=) ANY (SELECT intermediate_result.event_type FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(event_type integer)))
valid
@ -82,6 +85,7 @@ SELECT true AS valid FROM explain_json_2($$
NOT IN
(SELECT user_id FROM events_table WHERE event_type = 2);
$$);
DEBUG: function does not have co-located tables
DEBUG: generating subplan XXX_1 for subquery SELECT user_id FROM public.events_table WHERE (event_type OPERATOR(pg_catalog.=) 2)
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM public.events_table WHERE (NOT (user_id OPERATOR(pg_catalog.=) ANY (SELECT intermediate_result.user_id FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(user_id integer))))
valid
@ -103,6 +107,7 @@ SELECT true AS valid FROM explain_json_2($$
foo.event_type IN (SELECT event_type FROM events_table WHERE user_id < 3);
$$);
DEBUG: function does not have co-located tables
DEBUG: generating subplan XXX_1 for subquery SELECT event_type FROM public.events_table WHERE (user_id OPERATOR(pg_catalog.<) 3)
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT foo.user_id FROM (SELECT users_table.user_id, events_table.event_type FROM public.users_table, public.events_table WHERE ((users_table.user_id OPERATOR(pg_catalog.=) events_table.user_id) AND (events_table.event_type OPERATOR(pg_catalog.=) ANY (ARRAY[1, 2, 3, 4])))) foo, (SELECT users_table.user_id FROM public.users_table, public.events_table WHERE ((users_table.user_id OPERATOR(pg_catalog.=) events_table.user_id) AND (events_table.event_type OPERATOR(pg_catalog.=) ANY (ARRAY[5, 6, 7, 8])))) bar WHERE ((foo.user_id OPERATOR(pg_catalog.=) bar.user_id) AND (foo.event_type OPERATOR(pg_catalog.=) ANY (SELECT intermediate_result.event_type FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(event_type integer))))
valid
@ -123,6 +128,7 @@ SELECT true AS valid FROM explain_json_2($$
foo.user_id = bar.user_id AND
foo.user_id IN (SELECT user_id FROM events_table WHERE user_id < 10);
$$);
DEBUG: function does not have co-located tables
DEBUG: generating subplan XXX_1 for subquery SELECT (users_table.user_id OPERATOR(pg_catalog./) 2) AS user_id FROM public.users_table, public.events_table WHERE ((users_table.user_id OPERATOR(pg_catalog.=) events_table.user_id) AND (events_table.event_type OPERATOR(pg_catalog.=) ANY (ARRAY[5, 6, 7, 8])))
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT foo.user_id FROM (SELECT users_table.user_id, events_table.event_type FROM public.users_table, public.events_table WHERE ((users_table.user_id OPERATOR(pg_catalog.=) events_table.user_id) AND (events_table.event_type OPERATOR(pg_catalog.=) ANY (ARRAY[1, 2, 3, 4])))) foo, (SELECT intermediate_result.user_id FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(user_id integer)) bar WHERE ((foo.user_id OPERATOR(pg_catalog.=) bar.user_id) AND (foo.user_id OPERATOR(pg_catalog.=) ANY (SELECT events_table.user_id FROM public.events_table WHERE (events_table.user_id OPERATOR(pg_catalog.<) 10))))
valid
@ -143,6 +149,7 @@ SELECT true AS valid FROM explain_json_2($$
foo.user_id = bar.user_id AND
foo.user_id NOT IN (SELECT user_id FROM events_table WHERE user_id < 10);
$$);
DEBUG: function does not have co-located tables
DEBUG: generating subplan XXX_1 for subquery SELECT (users_table.user_id OPERATOR(pg_catalog./) 2) AS user_id FROM public.users_table, public.events_table WHERE ((users_table.user_id OPERATOR(pg_catalog.=) events_table.user_id) AND (events_table.event_type OPERATOR(pg_catalog.=) ANY (ARRAY[5, 6, 7, 8])))
DEBUG: generating subplan XXX_2 for subquery SELECT user_id FROM public.events_table WHERE (user_id OPERATOR(pg_catalog.<) 10)
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT foo.user_id FROM (SELECT users_table.user_id, events_table.event_type FROM public.users_table, public.events_table WHERE ((users_table.user_id OPERATOR(pg_catalog.=) events_table.user_id) AND (events_table.event_type OPERATOR(pg_catalog.=) ANY (ARRAY[1, 2, 3, 4])))) foo, (SELECT intermediate_result.user_id FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(user_id integer)) bar WHERE ((foo.user_id OPERATOR(pg_catalog.=) bar.user_id) AND (NOT (foo.user_id OPERATOR(pg_catalog.=) ANY (SELECT intermediate_result.user_id FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(user_id integer)))))
@ -164,6 +171,7 @@ SELECT true AS valid FROM explain_json_2($$
foo.user_id = bar.user_id AND
foo.event_type IN (SELECT event_type FROM events_table WHERE user_id < 4);
$$);
DEBUG: function does not have co-located tables
DEBUG: generating subplan XXX_1 for subquery SELECT users_table.user_id, events_table.event_type FROM public.users_table, public.events_table WHERE ((users_table.user_id OPERATOR(pg_catalog.=) events_table.value_2) AND (events_table.event_type OPERATOR(pg_catalog.=) ANY (ARRAY[1, 2, 3, 4])))
DEBUG: generating subplan XXX_2 for subquery SELECT event_type FROM public.events_table WHERE (user_id OPERATOR(pg_catalog.<) 4)
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT foo.user_id FROM (SELECT intermediate_result.user_id, intermediate_result.event_type FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(user_id integer, event_type integer)) foo, (SELECT users_table.user_id FROM public.users_table, public.events_table WHERE ((users_table.user_id OPERATOR(pg_catalog.=) events_table.user_id) AND (events_table.event_type OPERATOR(pg_catalog.=) ANY (ARRAY[5, 6, 7, 8])))) bar WHERE ((foo.user_id OPERATOR(pg_catalog.=) bar.user_id) AND (foo.event_type OPERATOR(pg_catalog.=) ANY (SELECT intermediate_result.event_type FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(event_type integer))))
@ -188,6 +196,7 @@ SELECT true AS valid FROM explain_json_2($$
) as foo_top, events_table WHERE events_table.user_id = foo_top.user_id;
$$);
DEBUG: function does not have co-located tables
DEBUG: generating subplan XXX_1 for subquery SELECT users_table.user_id, events_table.event_type FROM public.users_table, public.events_table WHERE ((users_table.user_id OPERATOR(pg_catalog.=) events_table.value_2) AND (events_table.event_type OPERATOR(pg_catalog.=) ANY (ARRAY[1, 2, 3, 4])))
DEBUG: generating subplan XXX_2 for subquery SELECT users_table.user_id FROM public.users_table, public.events_table WHERE ((users_table.user_id OPERATOR(pg_catalog.=) events_table.event_type) AND (events_table.event_type OPERATOR(pg_catalog.=) ANY (ARRAY[5, 6, 7, 8])))
DEBUG: generating subplan XXX_3 for subquery SELECT event_type FROM public.events_table WHERE (user_id OPERATOR(pg_catalog.=) 5)
@ -222,6 +231,7 @@ SELECT true AS valid FROM explain_json_2($$
) as foo_top;
$$);
DEBUG: function does not have co-located tables
DEBUG: generating subplan XXX_1 for subquery SELECT users_table.user_id, users_table.value_1 FROM public.users_table, public.events_table WHERE ((users_table.user_id OPERATOR(pg_catalog.=) events_table.user_id) AND (events_table.event_type OPERATOR(pg_catalog.=) ANY (ARRAY[17, 18, 19, 20])))
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT user_id, random FROM (SELECT foo1.user_id, random() AS random FROM (SELECT users_table.user_id, users_table.value_1 FROM public.users_table, public.events_table WHERE ((users_table.user_id OPERATOR(pg_catalog.=) events_table.user_id) AND (events_table.event_type OPERATOR(pg_catalog.=) ANY (ARRAY[1, 2, 3, 4])))) foo1, (SELECT users_table.user_id, users_table.value_1 FROM public.users_table, public.events_table WHERE ((users_table.user_id OPERATOR(pg_catalog.=) events_table.user_id) AND (events_table.event_type OPERATOR(pg_catalog.=) ANY (ARRAY[5, 6, 7, 8])))) foo2, (SELECT users_table.user_id, users_table.value_1 FROM public.users_table, public.events_table WHERE ((users_table.user_id OPERATOR(pg_catalog.=) events_table.user_id) AND (events_table.event_type OPERATOR(pg_catalog.=) ANY (ARRAY[9, 10, 11, 12])))) foo3, (SELECT users_table.user_id, users_table.value_1 FROM public.users_table, public.events_table WHERE ((users_table.user_id OPERATOR(pg_catalog.=) events_table.user_id) AND (events_table.event_type OPERATOR(pg_catalog.=) ANY (ARRAY[13, 14, 15, 16])))) foo4, (SELECT intermediate_result.user_id, intermediate_result.value_1 FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(user_id integer, value_1 integer)) foo5 WHERE ((foo1.user_id OPERATOR(pg_catalog.=) foo4.user_id) AND (foo1.user_id OPERATOR(pg_catalog.=) foo2.user_id) AND (foo1.user_id OPERATOR(pg_catalog.=) foo3.user_id) AND (foo1.user_id OPERATOR(pg_catalog.=) foo4.user_id) AND (foo1.user_id OPERATOR(pg_catalog.=) foo5.value_1))) foo_top
valid
@ -253,6 +263,7 @@ SELECT true AS valid FROM explain_json_2($$
foo1.user_id = foo5.user_id
) as foo_top;
$$);
DEBUG: function does not have co-located tables
DEBUG: generating subplan XXX_1 for subquery SELECT users_table.user_id, users_table.value_1 FROM public.users_table, public.events_table WHERE ((users_table.user_id OPERATOR(pg_catalog.=) events_table.value_2) AND (events_table.event_type OPERATOR(pg_catalog.=) ANY (ARRAY[17, 18, 19, 20])))
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT user_id, random FROM (SELECT foo1.user_id, random() AS random FROM (SELECT users_table.user_id, users_table.value_1 FROM public.users_table, public.events_table WHERE ((users_table.user_id OPERATOR(pg_catalog.=) events_table.user_id) AND (events_table.event_type OPERATOR(pg_catalog.=) ANY (ARRAY[1, 2, 3, 4])))) foo1, (SELECT users_table.user_id, users_table.value_1 FROM public.users_table, public.events_table WHERE ((users_table.user_id OPERATOR(pg_catalog.=) events_table.user_id) AND (events_table.event_type OPERATOR(pg_catalog.=) ANY (ARRAY[5, 6, 7, 8])))) foo2, (SELECT users_table.user_id, users_table.value_1 FROM public.users_table, public.events_table WHERE ((users_table.user_id OPERATOR(pg_catalog.=) events_table.user_id) AND (events_table.event_type OPERATOR(pg_catalog.=) ANY (ARRAY[9, 10, 11, 12])))) foo3, (SELECT users_table.user_id, users_table.value_1 FROM public.users_table, public.events_table WHERE ((users_table.user_id OPERATOR(pg_catalog.=) events_table.user_id) AND (events_table.event_type OPERATOR(pg_catalog.=) ANY (ARRAY[13, 14, 15, 16])))) foo4, (SELECT intermediate_result.user_id, intermediate_result.value_1 FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(user_id integer, value_1 integer)) foo5 WHERE ((foo1.user_id OPERATOR(pg_catalog.=) foo4.user_id) AND (foo1.user_id OPERATOR(pg_catalog.=) foo2.user_id) AND (foo1.user_id OPERATOR(pg_catalog.=) foo3.user_id) AND (foo1.user_id OPERATOR(pg_catalog.=) foo4.user_id) AND (foo1.user_id OPERATOR(pg_catalog.=) foo5.user_id))) foo_top
valid
@ -282,6 +293,7 @@ SELECT true AS valid FROM explain_json_2($$
foo1.user_id = foo5.value_1
) as foo_top;
$$);
DEBUG: function does not have co-located tables
DEBUG: generating subplan XXX_1 for subquery SELECT users_table.user_id, users_table.value_1 FROM public.users_table, public.events_table WHERE ((users_table.user_id OPERATOR(pg_catalog.=) events_table.value_2) AND (events_table.event_type OPERATOR(pg_catalog.=) ANY (ARRAY[5, 6, 7, 8])))
DEBUG: generating subplan XXX_2 for subquery SELECT users_table.user_id, users_table.value_1 FROM public.users_table, public.events_table WHERE ((users_table.user_id OPERATOR(pg_catalog.=) events_table.user_id) AND (events_table.event_type OPERATOR(pg_catalog.=) ANY (ARRAY[17, 18, 19, 20])))
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT user_id, random FROM (SELECT foo1.user_id, random() AS random FROM (SELECT users_table.user_id, users_table.value_1 FROM public.users_table, public.events_table WHERE ((users_table.user_id OPERATOR(pg_catalog.=) events_table.user_id) AND (events_table.event_type OPERATOR(pg_catalog.=) ANY (ARRAY[1, 2, 3, 4])))) foo1, (SELECT intermediate_result.user_id, intermediate_result.value_1 FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(user_id integer, value_1 integer)) foo2, (SELECT users_table.user_id, users_table.value_1 FROM public.users_table, public.events_table WHERE ((users_table.user_id OPERATOR(pg_catalog.=) events_table.user_id) AND (events_table.event_type OPERATOR(pg_catalog.=) ANY (ARRAY[9, 10, 11, 12])))) foo3, (SELECT users_table.user_id, users_table.value_1 FROM public.users_table, public.events_table WHERE ((users_table.user_id OPERATOR(pg_catalog.=) events_table.user_id) AND (events_table.event_type OPERATOR(pg_catalog.=) ANY (ARRAY[13, 14, 15, 16])))) foo4, (SELECT intermediate_result.user_id, intermediate_result.value_1 FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(user_id integer, value_1 integer)) foo5 WHERE ((foo1.user_id OPERATOR(pg_catalog.=) foo4.user_id) AND (foo1.user_id OPERATOR(pg_catalog.=) foo2.user_id) AND (foo1.user_id OPERATOR(pg_catalog.=) foo3.user_id) AND (foo1.user_id OPERATOR(pg_catalog.=) foo4.user_id) AND (foo1.user_id OPERATOR(pg_catalog.=) foo5.value_1))) foo_top
@ -313,6 +325,7 @@ SELECT true AS valid FROM explain_json_2($$
foo2.user_id = foo5.value_1
) as foo_top;
$$);
DEBUG: function does not have co-located tables
DEBUG: generating subplan XXX_1 for subquery SELECT users_table.user_id, users_table.value_1 FROM public.users_table, public.events_table WHERE ((users_table.user_id OPERATOR(pg_catalog.=) events_table.value_2) AND (events_table.event_type OPERATOR(pg_catalog.=) ANY (ARRAY[5, 6, 7, 8])))
DEBUG: generating subplan XXX_2 for subquery SELECT users_table.user_id, users_table.value_1 FROM public.users_table, public.events_table WHERE ((users_table.user_id OPERATOR(pg_catalog.=) events_table.user_id) AND (events_table.event_type OPERATOR(pg_catalog.=) ANY (ARRAY[17, 18, 19, 20])))
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT user_id, random FROM (SELECT foo1.user_id, random() AS random FROM (SELECT users_table.user_id, users_table.value_1 FROM public.users_table, public.events_table WHERE ((users_table.user_id OPERATOR(pg_catalog.=) events_table.user_id) AND (events_table.event_type OPERATOR(pg_catalog.=) ANY (ARRAY[1, 2, 3, 4])))) foo1, (SELECT intermediate_result.user_id, intermediate_result.value_1 FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(user_id integer, value_1 integer)) foo2, (SELECT users_table.user_id, users_table.value_1 FROM public.users_table, public.events_table WHERE ((users_table.user_id OPERATOR(pg_catalog.=) events_table.user_id) AND (events_table.event_type OPERATOR(pg_catalog.=) ANY (ARRAY[9, 10, 11, 12])))) foo3, (SELECT users_table.user_id, users_table.value_1 FROM public.users_table, public.events_table WHERE ((users_table.user_id OPERATOR(pg_catalog.=) events_table.user_id) AND (events_table.event_type OPERATOR(pg_catalog.=) ANY (ARRAY[13, 14, 15, 16])))) foo4, (SELECT intermediate_result.user_id, intermediate_result.value_1 FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(user_id integer, value_1 integer)) foo5 WHERE ((foo1.user_id OPERATOR(pg_catalog.=) foo4.user_id) AND (foo1.user_id OPERATOR(pg_catalog.=) foo2.user_id) AND (foo1.user_id OPERATOR(pg_catalog.=) foo3.user_id) AND (foo1.user_id OPERATOR(pg_catalog.=) foo4.user_id) AND (foo2.user_id OPERATOR(pg_catalog.=) foo5.value_1))) foo_top
@ -346,6 +359,7 @@ SELECT true AS valid FROM explain_json_2($$
foo.user_id = bar.user_id) as bar_top
ON (foo_top.user_id = bar_top.user_id);
$$);
DEBUG: function does not have co-located tables
DEBUG: generating subplan XXX_1 for subquery SELECT users_table.user_id FROM public.users_table, public.events_table WHERE ((users_table.user_id OPERATOR(pg_catalog.=) events_table.value_2) AND (events_table.event_type OPERATOR(pg_catalog.=) ANY (ARRAY[1, 2, 3, 4])))
DEBUG: generating subplan XXX_2 for subquery SELECT users_table.user_id FROM public.users_table, public.events_table WHERE ((users_table.user_id OPERATOR(pg_catalog.=) events_table.value_2) AND (events_table.event_type OPERATOR(pg_catalog.=) ANY (ARRAY[1, 2, 3, 4])))
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM ((SELECT foo.user_id FROM (SELECT intermediate_result.user_id FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(user_id integer)) foo, (SELECT users_table.user_id FROM public.users_table, public.events_table WHERE ((users_table.user_id OPERATOR(pg_catalog.=) events_table.user_id) AND (events_table.event_type OPERATOR(pg_catalog.=) ANY (ARRAY[5, 6, 7, 8])))) bar WHERE (foo.user_id OPERATOR(pg_catalog.=) bar.user_id)) foo_top JOIN (SELECT foo.user_id FROM (SELECT intermediate_result.user_id FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(user_id integer)) foo, (SELECT users_table.user_id FROM public.users_table, public.events_table WHERE ((users_table.user_id OPERATOR(pg_catalog.=) events_table.user_id) AND (events_table.event_type OPERATOR(pg_catalog.=) ANY (ARRAY[5, 6, 7, 8])))) bar WHERE (foo.user_id OPERATOR(pg_catalog.=) bar.user_id)) bar_top ON ((foo_top.user_id OPERATOR(pg_catalog.=) bar_top.user_id)))
@ -380,6 +394,7 @@ SELECT true AS valid FROM explain_json_2($$
ON (foo_top.value_2 = bar_top.user_id);
$$);
DEBUG: function does not have co-located tables
DEBUG: generating subplan XXX_1 for subquery SELECT foo.user_id FROM (SELECT DISTINCT users_table.user_id FROM public.users_table, public.events_table WHERE ((users_table.user_id OPERATOR(pg_catalog.=) events_table.user_id) AND (events_table.event_type OPERATOR(pg_catalog.=) ANY (ARRAY[9, 10, 11, 12])))) foo, (SELECT DISTINCT users_table.user_id FROM public.users_table, public.events_table WHERE ((users_table.user_id OPERATOR(pg_catalog.=) events_table.user_id) AND (events_table.event_type OPERATOR(pg_catalog.=) ANY (ARRAY[13, 14, 15, 16])))) bar WHERE (foo.user_id OPERATOR(pg_catalog.=) bar.user_id)
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM ((SELECT foo.user_id, foo.value_2 FROM (SELECT DISTINCT users_table.user_id, users_table.value_2 FROM public.users_table, public.events_table WHERE ((users_table.user_id OPERATOR(pg_catalog.=) events_table.user_id) AND (events_table.event_type OPERATOR(pg_catalog.=) ANY (ARRAY[1, 2, 3, 4])))) foo, (SELECT DISTINCT users_table.user_id FROM public.users_table, public.events_table WHERE ((users_table.user_id OPERATOR(pg_catalog.=) events_table.user_id) AND (events_table.event_type OPERATOR(pg_catalog.=) ANY (ARRAY[5, 6, 7, 8])))) bar WHERE (foo.user_id OPERATOR(pg_catalog.=) bar.user_id)) foo_top JOIN (SELECT intermediate_result.user_id FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(user_id integer)) bar_top ON ((foo_top.value_2 OPERATOR(pg_catalog.=) bar_top.user_id)))
valid
@ -411,6 +426,7 @@ SELECT true AS valid FROM explain_json_2($$
foo.user_id = bar.user_id) as bar_top
ON (foo_top.value_2 = bar_top.user_id);
$$);
DEBUG: function does not have co-located tables
DEBUG: generating subplan XXX_1 for subquery SELECT DISTINCT users_table.user_id FROM public.users_table, public.events_table WHERE ((users_table.user_id OPERATOR(pg_catalog.=) events_table.value_2) AND (events_table.event_type OPERATOR(pg_catalog.=) ANY (ARRAY[13, 14, 15, 16])))
DEBUG: generating subplan XXX_2 for subquery SELECT foo.user_id FROM (SELECT DISTINCT users_table.user_id FROM public.users_table, public.events_table WHERE ((users_table.user_id OPERATOR(pg_catalog.=) events_table.user_id) AND (events_table.event_type OPERATOR(pg_catalog.=) ANY (ARRAY[9, 10, 11, 12])))) foo, (SELECT intermediate_result.user_id FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(user_id integer)) bar WHERE (foo.user_id OPERATOR(pg_catalog.=) bar.user_id)
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM ((SELECT foo.user_id, foo.value_2 FROM (SELECT DISTINCT users_table.user_id, users_table.value_2 FROM public.users_table, public.events_table WHERE ((users_table.user_id OPERATOR(pg_catalog.=) events_table.user_id) AND (events_table.event_type OPERATOR(pg_catalog.=) ANY (ARRAY[1, 2, 3, 4])))) foo, (SELECT DISTINCT users_table.user_id FROM public.users_table, public.events_table WHERE ((users_table.user_id OPERATOR(pg_catalog.=) events_table.user_id) AND (events_table.event_type OPERATOR(pg_catalog.=) ANY (ARRAY[5, 6, 7, 8])))) bar WHERE (foo.user_id OPERATOR(pg_catalog.=) bar.user_id)) foo_top JOIN (SELECT intermediate_result.user_id FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(user_id integer)) bar_top ON ((foo_top.value_2 OPERATOR(pg_catalog.=) bar_top.user_id)))
@ -432,6 +448,7 @@ SELECT true AS valid FROM explain_json_2($$
WHERE foo.my_users = users_table.user_id) as mid_level_query
) as bar;
$$);
DEBUG: function does not have co-located tables
DEBUG: generating subplan XXX_1 for subquery SELECT events_table.user_id AS my_users FROM public.events_table, public.users_table WHERE (events_table.event_type OPERATOR(pg_catalog.=) users_table.user_id)
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (SELECT mid_level_query.user_id FROM (SELECT DISTINCT users_table.user_id FROM public.users_table, (SELECT intermediate_result.my_users FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(my_users integer)) foo WHERE (foo.my_users OPERATOR(pg_catalog.=) users_table.user_id)) mid_level_query) bar
valid
@ -451,6 +468,7 @@ SELECT true AS valid FROM explain_json_2($$
WHERE foo.my_users = users_table.user_id) as mid_level_query ) as bar;
$$);
DEBUG: function does not have co-located tables
DEBUG: generating subplan XXX_1 for subquery SELECT events_table.event_type AS my_users, random() AS random FROM public.events_table, public.users_table WHERE (events_table.user_id OPERATOR(pg_catalog.=) users_table.user_id)
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (SELECT mid_level_query.user_id FROM (SELECT DISTINCT users_table.user_id FROM public.users_table, (SELECT intermediate_result.my_users, intermediate_result.random FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(my_users integer, random double precision)) foo WHERE (foo.my_users OPERATOR(pg_catalog.=) users_table.user_id)) mid_level_query) bar
valid
@ -474,6 +492,7 @@ DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS c
WHERE foo.my_users = users_table.user_id) as mid_level_query
) as bar;
$$);
DEBUG: function does not have co-located tables
DEBUG: generating subplan XXX_1 for subquery SELECT events_table.user_id, random() AS random FROM public.users_table, public.events_table WHERE (users_table.user_id OPERATOR(pg_catalog.=) events_table.user_id)
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (SELECT mid_level_query.user_id FROM (SELECT DISTINCT users_table.user_id FROM public.users_table, (SELECT events_table.user_id AS my_users FROM public.events_table, (SELECT intermediate_result.user_id, intermediate_result.random FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(user_id integer, random double precision)) selected_users WHERE (events_table.event_type OPERATOR(pg_catalog.=) selected_users.user_id)) foo WHERE (foo.my_users OPERATOR(pg_catalog.=) users_table.user_id)) mid_level_query) bar
valid
@ -507,6 +526,7 @@ SELECT true AS valid FROM explain_json_2($$
) as bar;
$$);
DEBUG: function does not have co-located tables
DEBUG: generating subplan XXX_1 for subquery SELECT value_2 FROM public.events_table
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (SELECT mid_level_query.user_id FROM (SELECT DISTINCT users_table.user_id FROM public.users_table, (SELECT events_table.user_id AS my_users FROM public.events_table, (SELECT events_table_1.user_id FROM public.users_table users_table_1, public.events_table events_table_1 WHERE ((users_table_1.user_id OPERATOR(pg_catalog.=) events_table_1.user_id) AND (users_table_1.user_id OPERATOR(pg_catalog.=) ANY (SELECT intermediate_result.value_2 FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(value_2 integer))))) selected_users WHERE (events_table.user_id OPERATOR(pg_catalog.=) selected_users.user_id)) foo WHERE (foo.my_users OPERATOR(pg_catalog.=) users_table.user_id)) mid_level_query) bar
valid
@ -528,6 +548,7 @@ WHERE
users_table, events_table
WHERE
users_table.user_id = events_table.value_2 AND event_type IN (5,6));$$);
DEBUG: function does not have co-located tables
DEBUG: generating subplan XXX_1 for subquery SELECT users_table.user_id FROM public.users_table, public.events_table WHERE ((users_table.user_id OPERATOR(pg_catalog.=) events_table.value_2) AND (events_table.event_type OPERATOR(pg_catalog.=) ANY (ARRAY[5, 6])))
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM public.users_table WHERE (value_1 OPERATOR(pg_catalog.=) ANY (SELECT intermediate_result.user_id FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(user_id integer)))
valid
@ -544,6 +565,7 @@ SELECT count(*) FROM q1, (SELECT
users_table, events_table
WHERE
users_table.user_id = events_table.value_2 AND event_type IN (1,2,3,4)) as bar WHERE bar.user_id = q1.user_id ;$$);
DEBUG: function does not have co-located tables
DEBUG: CTE q1 is going to be inlined via distributed planning
DEBUG: generating subplan XXX_1 for subquery SELECT users_table.user_id, random() AS random FROM public.users_table, public.events_table WHERE ((users_table.user_id OPERATOR(pg_catalog.=) events_table.value_2) AND (events_table.event_type OPERATOR(pg_catalog.=) ANY (ARRAY[1, 2, 3, 4])))
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (SELECT users_table.user_id FROM public.users_table) q1, (SELECT intermediate_result.user_id, intermediate_result.random FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(user_id integer, random double precision)) bar WHERE (bar.user_id OPERATOR(pg_catalog.=) q1.user_id)
@ -561,6 +583,7 @@ SELECT true AS valid FROM explain_json_2($$
users_table, events_table
WHERE
users_table.user_id = events_table.user_id AND event_type IN (1,2,3,4)) as bar WHERE bar.user_id = q1.user_id ;$$);
DEBUG: function does not have co-located tables
DEBUG: generating subplan XXX_1 for CTE q1: SELECT user_id FROM public.users_table
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (SELECT intermediate_result.user_id FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(user_id integer)) q1, (SELECT users_table.user_id, random() AS random FROM public.users_table, public.events_table WHERE ((users_table.user_id OPERATOR(pg_catalog.=) events_table.user_id) AND (events_table.event_type OPERATOR(pg_catalog.=) ANY (ARRAY[1, 2, 3, 4])))) bar WHERE (bar.user_id OPERATOR(pg_catalog.=) q1.user_id)
valid
@ -572,6 +595,7 @@ DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS c
SELECT true AS valid FROM explain_json_2($$
(SELECT users_table.user_id FROM users_table, events_table WHERE users_table.user_id = events_table.value_2 AND event_type IN (1,2,3,4)) UNION
(SELECT users_table.user_id FROM users_table, events_table WHERE users_table.user_id = events_table.user_id AND event_type IN (5,6,7,8));$$);
DEBUG: function does not have co-located tables
DEBUG: generating subplan XXX_1 for subquery SELECT users_table.user_id FROM public.users_table, public.events_table WHERE ((users_table.user_id OPERATOR(pg_catalog.=) events_table.value_2) AND (events_table.event_type OPERATOR(pg_catalog.=) ANY (ARRAY[1, 2, 3, 4])))
DEBUG: generating subplan XXX_2 for subquery SELECT users_table.user_id FROM public.users_table, public.events_table WHERE ((users_table.user_id OPERATOR(pg_catalog.=) events_table.user_id) AND (events_table.event_type OPERATOR(pg_catalog.=) ANY (ARRAY[5, 6, 7, 8])))
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT intermediate_result.user_id FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(user_id integer) UNION SELECT intermediate_result.user_id FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(user_id integer)
@ -608,6 +632,7 @@ FROM (
) q
ORDER BY 2 DESC, 1;
$$);
DEBUG: function does not have co-located tables
DEBUG: generating subplan XXX_1 for subquery SELECT users_table.user_id FROM public.users_table, public.events_table WHERE ((users_table.user_id OPERATOR(pg_catalog.=) events_table.value_2) AND (events_table.event_type OPERATOR(pg_catalog.=) ANY (ARRAY[1, 2, 3, 4])))
DEBUG: push down of limit count: 5
DEBUG: generating subplan XXX_2 for subquery SELECT user_id FROM public.users_table WHERE ((value_2 OPERATOR(pg_catalog.>=) 5) AND (EXISTS (SELECT intermediate_result.user_id FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(user_id integer)))) LIMIT 5
@ -629,6 +654,7 @@ SELECT true AS valid FROM explain_json_2($$
FROM
(users_table u1 JOIN users_table u2 using(value_1)) a JOIN (SELECT value_1, random() FROM users_table) as u3 USING (value_1);
$$);
DEBUG: function does not have co-located tables
DEBUG: generating subplan XXX_1 for subquery SELECT value_1, random() AS random FROM public.users_table
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM ((public.users_table u1 JOIN public.users_table u2 USING (value_1)) a(value_1, user_id, "time", value_2, value_3, value_4, user_id_1, time_1, value_2_1, value_3_1, value_4_1) JOIN (SELECT intermediate_result.value_1, intermediate_result.random FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(value_1 integer, random double precision)) u3 USING (value_1))
ERROR: complex joins are only supported when all distributed tables are co-located and joined on their distribution columns
@ -641,6 +667,7 @@ SELECT true AS valid FROM explain_json_2($$
FROM
(SELECT * FROM users_table u1 JOIN users_table u2 using(value_1)) a JOIN (SELECT value_1, random() FROM users_table) as u3 USING (value_1);
$$);
DEBUG: function does not have co-located tables
DEBUG: generating subplan XXX_1 for subquery SELECT u1.value_1, u1.user_id, u1."time", u1.value_2, u1.value_3, u1.value_4, u2.user_id, u2."time", u2.value_2, u2.value_3, u2.value_4 FROM (public.users_table u1 JOIN public.users_table u2 USING (value_1))
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM ((SELECT intermediate_result.value_1, intermediate_result.user_id, intermediate_result."time", intermediate_result.value_2, intermediate_result.value_3, intermediate_result.value_4, intermediate_result.user_id_1 AS user_id, intermediate_result.time_1 AS "time", intermediate_result.value_2_1 AS value_2, intermediate_result.value_3_1 AS value_3, intermediate_result.value_4_1 AS value_4 FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(value_1 integer, user_id integer, "time" timestamp without time zone, value_2 integer, value_3 double precision, value_4 bigint, user_id_1 integer, time_1 timestamp without time zone, value_2_1 integer, value_3_1 double precision, value_4_1 bigint)) a(value_1, user_id, "time", value_2, value_3, value_4, user_id_1, time_1, value_2_1, value_3_1, value_4_1) JOIN (SELECT users_table.value_1, random() AS random FROM public.users_table) u3 USING (value_1))
valid
@ -660,6 +687,7 @@ SELECT true AS valid FROM explain_json_2($$
events_table
using (value_2);
$$);
DEBUG: function does not have co-located tables
DEBUG: generating subplan XXX_1 for subquery SELECT value_2, random() AS random FROM public.users_table
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM ((SELECT intermediate_result.value_2, intermediate_result.random FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(value_2 integer, random double precision)) u1 JOIN public.events_table USING (value_2))
valid
@ -678,6 +706,7 @@ SELECT true AS valid FROM explain_json_2($$
(SELECT value_2, random() FROM users_table) as u2
USING(value_2);
$$);
DEBUG: function does not have co-located tables
DEBUG: generating subplan XXX_1 for subquery SELECT value_2, random() AS random FROM public.users_table
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM ((SELECT users_table.value_2, random() AS random FROM public.users_table) u1 LEFT JOIN (SELECT intermediate_result.value_2, intermediate_result.random FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(value_2 integer, random double precision)) u2 USING (value_2))
valid
@ -698,6 +727,7 @@ SELECT true AS valid FROM explain_json_2($$
(SELECT value_2, random() FROM users_table) as u2
USING(value_2);
$$);
DEBUG: function does not have co-located tables
DEBUG: generating subplan XXX_1 for subquery SELECT value_2, random() AS random FROM public.users_table
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM ((SELECT users_table.value_2, random() AS random FROM public.users_table) u1 RIGHT JOIN (SELECT intermediate_result.value_2, intermediate_result.random FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(value_2 integer, random double precision)) u2 USING (value_2))
ERROR: cannot pushdown the subquery
@ -717,6 +747,7 @@ SELECT true AS valid FROM explain_json_2($$
(SELECT value_1 FROM users_table) as foo ON (a.user_id = foo.value_1)
);
$$);
DEBUG: function does not have co-located tables
DEBUG: generating subplan XXX_1 for subquery SELECT user_id FROM public.users_table
DEBUG: generating subplan XXX_2 for subquery SELECT user_id FROM public.users_table
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT intermediate_result.user_id FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(user_id integer) UNION SELECT intermediate_result.user_id FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(user_id integer)
@ -741,6 +772,7 @@ SELECT true AS valid FROM explain_json_2($$
users_table as foo ON (a.user_id = foo.value_1)
);
$$);
DEBUG: function does not have co-located tables
DEBUG: generating subplan XXX_1 for subquery SELECT user_id FROM public.users_table
DEBUG: generating subplan XXX_2 for subquery SELECT user_id FROM public.users_table
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT intermediate_result.user_id FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(user_id integer) UNION SELECT intermediate_result.user_id FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(user_id integer)
@ -774,6 +806,7 @@ SELECT true AS valid FROM explain_json_2($$
ON(foo.user_id = bar.value_1)
);
$$);
DEBUG: function does not have co-located tables
DEBUG: generating subplan XXX_1 for subquery SELECT value_1 FROM public.users_table
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT foo.user_id, a.user_id, bar.value_1 FROM (((SELECT users_table.user_id FROM public.users_table) foo JOIN (SELECT users_table.user_id FROM public.users_table WHERE (users_table.user_id OPERATOR(pg_catalog.=) ANY (ARRAY[1, 2, 3, 4])) UNION SELECT users_table.user_id FROM public.users_table WHERE (users_table.user_id OPERATOR(pg_catalog.=) ANY (ARRAY[5, 6, 7, 8]))) a ON ((a.user_id OPERATOR(pg_catalog.=) foo.user_id))) JOIN (SELECT intermediate_result.value_1 FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(value_1 integer)) bar ON ((foo.user_id OPERATOR(pg_catalog.=) bar.value_1)))
valid
@ -813,6 +846,7 @@ SELECT true AS valid FROM explain_json_2($$
WHERE
non_colocated_subquery.value_2 != non_colocated_subquery_2.cnt
$$);
DEBUG: function does not have co-located tables
DEBUG: CTE non_colocated_subquery is going to be inlined via distributed planning
DEBUG: CTE non_colocated_subquery_2 is going to be inlined via distributed planning
DEBUG: generating subplan XXX_1 for subquery SELECT users_table.value_2 FROM public.users_table, public.events_table WHERE ((users_table.user_id OPERATOR(pg_catalog.=) events_table.user_id) AND (events_table.event_type OPERATOR(pg_catalog.=) ANY (ARRAY[5, 6, 7, 8])))
@ -837,6 +871,7 @@ SELECT true AS valid FROM explain_json_2($$
AND
foo.value_2 = baz.value_2
$$);
DEBUG: function does not have co-located tables
DEBUG: generating subplan XXX_1 for subquery SELECT users_table_local.value_2 FROM non_colocated_subquery.users_table_local, non_colocated_subquery.events_table_local WHERE ((users_table_local.user_id OPERATOR(pg_catalog.=) events_table_local.user_id) AND (events_table_local.event_type OPERATOR(pg_catalog.=) ANY (ARRAY[5, 6, 7, 8])))
DEBUG: generating subplan XXX_2 for subquery SELECT users_table.value_2 FROM public.users_table, public.events_table WHERE ((users_table.user_id OPERATOR(pg_catalog.=) events_table.user_id) AND (events_table.event_type OPERATOR(pg_catalog.=) ANY (ARRAY[9, 10, 11, 12])))
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (SELECT users_table.value_2 FROM public.users_table, public.events_table WHERE ((users_table.user_id OPERATOR(pg_catalog.=) events_table.user_id) AND (events_table.event_type OPERATOR(pg_catalog.=) ANY (ARRAY[1, 2, 3, 4])))) foo, (SELECT intermediate_result.value_2 FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(value_2 integer)) bar, (SELECT intermediate_result.value_2 FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(value_2 integer)) baz WHERE ((foo.value_2 OPERATOR(pg_catalog.=) bar.value_2) AND (foo.value_2 OPERATOR(pg_catalog.=) baz.value_2))
@ -872,6 +907,7 @@ SELECT true AS valid FROM explain_json_2($$
AND
foo.user_id IN (SELECT users_table.user_id FROM users_table, events_table WHERE users_table.user_id = events_table.user_id AND event_type IN (1,2))
$$);
DEBUG: function does not have co-located tables
DEBUG: generating subplan XXX_1 for subquery SELECT value_1, value_2 FROM public.users_table
DEBUG: generating subplan XXX_2 for subquery SELECT value_1 FROM public.users_table WHERE (value_2 OPERATOR(pg_catalog.<) 1)
DEBUG: generating subplan XXX_3 for subquery SELECT value_2 FROM public.users_table WHERE (value_1 OPERATOR(pg_catalog.<) 2)
@ -894,6 +930,7 @@ SELECT true AS valid FROM explain_json_2($$
users_table_ref.user_id = foo.user_id
AND foo.user_id = bar.value_2;
$$);
DEBUG: function does not have co-located tables
DEBUG: generating subplan XXX_1 for subquery SELECT user_id, value_2 FROM public.events_table
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM public.users_reference_table users_table_ref, (SELECT users_table.user_id FROM public.users_table) foo, (SELECT intermediate_result.user_id, intermediate_result.value_2 FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(user_id integer, value_2 integer)) bar WHERE ((users_table_ref.user_id OPERATOR(pg_catalog.=) foo.user_id) AND (foo.user_id OPERATOR(pg_catalog.=) bar.value_2))
valid
@ -942,6 +979,7 @@ JOIN LATERAL
WHERE user_id = users_table.user_id) AS bar
LEFT JOIN users_table u2 ON u2.user_id = bar.value_2) AS foo ON TRUE
$$);
DEBUG: function does not have co-located tables
DEBUG: Router planner cannot handle multi-shard select queries
DEBUG: skipping recursive planning for the subquery since it contains references to outer queries
DEBUG: skipping recursive planning for the subquery since it contains references to outer queries

View File

@ -145,9 +145,13 @@ CREATE TYPE myvarchar;
CREATE FUNCTION myvarcharin(cstring, oid, integer) RETURNS myvarchar
LANGUAGE internal IMMUTABLE PARALLEL SAFE STRICT AS 'varcharin';
NOTICE: return type myvarchar is only a shell
WARNING: Citus can't distribute functions having dependency on unsupported object of type "type"
DETAIL: Function will be created only locally
CREATE FUNCTION myvarcharout(myvarchar) RETURNS cstring
LANGUAGE internal IMMUTABLE PARALLEL SAFE STRICT AS 'varcharout';
NOTICE: argument type myvarchar is only a shell
WARNING: Citus can't distribute functions having dependency on unsupported object of type "type"
DETAIL: Function will be created only locally
CREATE TYPE myvarchar (
input = myvarcharin,
output = myvarcharout,

View File

@ -186,22 +186,6 @@ CREATE FUNCTION dintdict_lexize(internal, internal, internal, internal) RETURNS
CREATE TEXT SEARCH TEMPLATE intdict_template (LEXIZE = dintdict_lexize, INIT = dintdict_init );
CREATE TEXT SEARCH DICTIONARY intdict (TEMPLATE = intdict_template);
COMMENT ON TEXT SEARCH DICTIONARY intdict IS 'dictionary for integers';
SELECT run_command_on_workers($$
CREATE FUNCTION dintdict_init(internal) RETURNS internal AS 'dict_int.so' LANGUAGE C STRICT;
$$);
run_command_on_workers
---------------------------------------------------------------------
(localhost,57637,t,"CREATE FUNCTION")
(1 row)
SELECT run_command_on_workers($$
CREATE FUNCTION dintdict_lexize(internal, internal, internal, internal) RETURNS internal AS 'dict_int.so' LANGUAGE C STRICT;
$$);
run_command_on_workers
---------------------------------------------------------------------
(localhost,57637,t,"CREATE FUNCTION")
(1 row)
SELECT run_command_on_workers($$
CREATE TEXT SEARCH TEMPLATE intdict_template (LEXIZE = dintdict_lexize, INIT = dintdict_init );
$$);

View File

@ -186,22 +186,6 @@ CREATE FUNCTION dintdict_lexize(internal, internal, internal, internal) RETURNS
CREATE TEXT SEARCH TEMPLATE intdict_template (LEXIZE = dintdict_lexize, INIT = dintdict_init );
CREATE TEXT SEARCH DICTIONARY intdict (TEMPLATE = intdict_template);
COMMENT ON TEXT SEARCH DICTIONARY intdict IS 'dictionary for integers';
SELECT run_command_on_workers($$
CREATE FUNCTION dintdict_init(internal) RETURNS internal AS 'dict_int.so' LANGUAGE C STRICT;
$$);
run_command_on_workers
---------------------------------------------------------------------
(localhost,57637,t,"CREATE FUNCTION")
(1 row)
SELECT run_command_on_workers($$
CREATE FUNCTION dintdict_lexize(internal, internal, internal, internal) RETURNS internal AS 'dict_int.so' LANGUAGE C STRICT;
$$);
run_command_on_workers
---------------------------------------------------------------------
(localhost,57637,t,"CREATE FUNCTION")
(1 row)
SELECT run_command_on_workers($$
CREATE TEXT SEARCH TEMPLATE intdict_template (LEXIZE = dintdict_lexize, INIT = dintdict_init );
$$);

View File

@ -43,11 +43,13 @@ SELECT create_reference_table('numbers');
INSERT INTO numbers VALUES (20), (21);
NOTICE: executing the command locally: INSERT INTO replicate_ref_to_coordinator.numbers_8000001 AS citus_table_alias (a) VALUES (20), (21)
SET citus.enable_metadata_sync TO OFF;
CREATE OR REPLACE FUNCTION my_volatile_fn()
RETURNS INT AS $$
BEGIN
RETURN 1;
END; $$ language plpgsql VOLATILE;
RESET citus.enable_metadata_sync;
-- INSERT ... SELECT between reference tables
BEGIN;
EXPLAIN (COSTS OFF) INSERT INTO squares SELECT a, a*a FROM numbers;

View File

@ -142,22 +142,35 @@ SELECT master_move_shard_placement(20000000, 'localhost', :worker_2_port, 'local
(1 row)
SELECT run_command_on_workers($cmd$
-- override the function for testing purpose
create or replace function pg_catalog.citus_local_disk_space_stats(OUT available_disk_size bigint, OUT total_disk_size bigint)
as $BODY$
begin
select 20 into available_disk_size;
select 8500 into total_disk_size;
end
$BODY$ language plpgsql;
$cmd$);
run_command_on_workers
---------------------------------------------------------------------
(localhost,57637,t,"CREATE FUNCTION")
(localhost,57638,t,"CREATE FUNCTION")
(2 rows)
-- override the function for testing purpose
-- since it is extension owned function, propagate it to workers manually
create or replace function pg_catalog.citus_local_disk_space_stats(OUT available_disk_size bigint, OUT total_disk_size bigint)
as $BODY$
begin
select 20 into available_disk_size;
select 8500 into total_disk_size;
end
$BODY$ language plpgsql;
\c - - - :worker_1_port
SET citus.enable_metadata_sync TO OFF;
create or replace function pg_catalog.citus_local_disk_space_stats(OUT available_disk_size bigint, OUT total_disk_size bigint)
as $BODY$
begin
select 20 into available_disk_size;
select 8500 into total_disk_size;
end
$BODY$ language plpgsql;
\c - - - :worker_2_port
SET citus.enable_metadata_sync TO OFF;
create or replace function pg_catalog.citus_local_disk_space_stats(OUT available_disk_size bigint, OUT total_disk_size bigint)
as $BODY$
begin
select 20 into available_disk_size;
select 8500 into total_disk_size;
end
$BODY$ language plpgsql;
\c - - - :master_port
SET search_path TO shard_move_deferred_delete;
SELECT citus_shard_cost_by_disk_size(20000001);
citus_shard_cost_by_disk_size
---------------------------------------------------------------------
@ -187,22 +200,35 @@ $cmd$);
(localhost,57638,t,1)
(2 rows)
SELECT run_command_on_workers($cmd$
-- override the function for testing purpose
create or replace function pg_catalog.citus_local_disk_space_stats(OUT available_disk_size bigint, OUT total_disk_size bigint)
as $BODY$
begin
select 8300 into available_disk_size;
select 8500 into total_disk_size;
end
$BODY$ language plpgsql;
$cmd$);
run_command_on_workers
---------------------------------------------------------------------
(localhost,57637,t,"CREATE FUNCTION")
(localhost,57638,t,"CREATE FUNCTION")
(2 rows)
-- override the function for testing purpose
-- since it is extension owned function, propagate it to workers manually
create or replace function pg_catalog.citus_local_disk_space_stats(OUT available_disk_size bigint, OUT total_disk_size bigint)
as $BODY$
begin
select 8300 into available_disk_size;
select 8500 into total_disk_size;
end
$BODY$ language plpgsql;
\c - - - :worker_1_port
SET citus.enable_metadata_sync TO OFF;
create or replace function pg_catalog.citus_local_disk_space_stats(OUT available_disk_size bigint, OUT total_disk_size bigint)
as $BODY$
begin
select 8300 into available_disk_size;
select 8500 into total_disk_size;
end
$BODY$ language plpgsql;
\c - - - :worker_2_port
SET citus.enable_metadata_sync TO OFF;
create or replace function pg_catalog.citus_local_disk_space_stats(OUT available_disk_size bigint, OUT total_disk_size bigint)
as $BODY$
begin
select 8300 into available_disk_size;
select 8500 into total_disk_size;
end
$BODY$ language plpgsql;
\c - - - :master_port
SET search_path TO shard_move_deferred_delete;
-- When there would not be enough free space left after the move, the move should fail
SELECT master_move_shard_placement(20000001, 'localhost', :worker_2_port, 'localhost', :worker_1_port);
ERROR: not enough empty space on node if the shard is moved, actual available space after move will be 108 bytes, desired available space after move is 850 bytes,estimated size increase on node after move is 8192 bytes.

View File

@ -241,6 +241,7 @@ RETURNS boolean
AS 'citus'
LANGUAGE C STRICT VOLATILE;
-- this function is dropped in Citus10, added here for tests
SET citus.enable_metadata_sync TO OFF;
CREATE OR REPLACE FUNCTION pg_catalog.master_create_distributed_table(table_name regclass,
distribution_column text,
distribution_method citus.distribution_type)
@ -257,6 +258,7 @@ CREATE OR REPLACE FUNCTION pg_catalog.master_create_worker_shards(table_name tex
RETURNS void
AS 'citus', $$master_create_worker_shards$$
LANGUAGE C STRICT;
RESET citus.enable_metadata_sync;
SET citus.next_shard_id TO 123000;
SELECT worker_node_responsive(node_name, node_port::int)
FROM master_get_active_worker_nodes()

View File

@ -126,6 +126,7 @@ WITH cte1 AS MATERIALIZED (SELECT id, value FROM func())
UPDATE dist_table dt SET value = cte1.value
FROM cte1 WHERE dt.id = 1;
DEBUG: generating subplan XXX_1 for CTE cte1: SELECT id, value FROM subquery_and_ctes.func() func(id, value)
DEBUG: function does not have co-located tables
DEBUG: Plan XXX query after replacing subqueries and CTEs: UPDATE subquery_and_ctes.dist_table dt SET value = cte1.value FROM (SELECT intermediate_result.id, intermediate_result.value FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(id integer, value integer)) cte1 WHERE (dt.id OPERATOR(pg_catalog.=) 1)
-- CTEs are recursively planned, and subquery foo is also recursively planned
-- final plan becomes a real-time plan since we also have events_table in the

View File

@ -53,6 +53,7 @@ SELECT create_distributed_table('tr', 'pk');
INSERT INTO tr SELECT c, c FROM generate_series(1, 5) as c;
-- this function is dropped in Citus10, added here for tests
SET citus.enable_metadata_sync TO OFF;
CREATE OR REPLACE FUNCTION pg_catalog.master_create_distributed_table(table_name regclass,
distribution_column text,
distribution_method citus.distribution_type)
@ -69,6 +70,7 @@ CREATE OR REPLACE FUNCTION pg_catalog.master_create_worker_shards(table_name tex
RETURNS void
AS 'citus', $$master_create_worker_shards$$
LANGUAGE C STRICT;
RESET citus.enable_metadata_sync;
CREATE TABLE t_range(id int, value_1 int);
SELECT create_distributed_table('t_range', 'id', 'range');
create_distributed_table

View File

@ -587,6 +587,8 @@ CREATE OR REPLACE FUNCTION fixed_volatile_value() RETURNS integer VOLATILE AS $$
RETURN 1;
END;
$$ LANGUAGE plpgsql;
DEBUG: switching to sequential query execution mode
DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands
EXECUTE test_values_pushdown(fixed_volatile_value(),2,3);
NOTICE: evaluated on the coordinator
CONTEXT: PL/pgSQL function fixed_volatile_value() line XX at RAISE

View File

@ -294,11 +294,13 @@ ALTER TABLE lineitem_alter_220000 DROP COLUMN first;
CREATE TABLE ddl_commands (command text UNIQUE DEFERRABLE INITIALLY DEFERRED);
-- Use an event trigger to log all DDL event tags in it
SET citus.enable_metadata_sync TO OFF;
CREATE FUNCTION log_ddl_tag() RETURNS event_trigger AS $ldt$
BEGIN
INSERT INTO ddl_commands VALUES (tg_tag);
END;
$ldt$ LANGUAGE plpgsql;
RESET citus.enable_metadata_sync;
CREATE EVENT TRIGGER log_ddl_tag ON ddl_command_end EXECUTE PROCEDURE log_ddl_tag();
@ -499,13 +501,14 @@ CREATE TABLE trigger_table (
SELECT create_distributed_table('trigger_table', 'id');
-- first set a trigger on a shard
\c - - - :worker_1_port
SET citus.enable_metadata_sync TO OFF;
CREATE FUNCTION update_value() RETURNS trigger AS $up$
BEGIN
NEW.value := 'trigger enabled';
RETURN NEW;
END;
$up$ LANGUAGE plpgsql;
RESET citus.enable_metadata_sync;
CREATE TRIGGER update_value
BEFORE INSERT ON trigger_table_220017

View File

@ -312,6 +312,8 @@ test: ssl_by_default
# ---------
# object distribution tests
# TODO: After deprecating parameterless create_distributed_function combine
# distributed_functions and function_propagation tests
# ---------
test: distributed_types distributed_types_conflict disable_object_propagation distributed_types_xact_add_enum_value text_search
test: check_mx
@ -319,6 +321,7 @@ test: distributed_functions distributed_functions_conflict
test: distributed_collations
test: distributed_procedure
test: distributed_collations_conflict
test: function_propagation
test: check_mx
# ---------

View File

@ -642,11 +642,13 @@ ALTER TABLE lineitem_alter_220000 DROP COLUMN first;
-- Create table to trigger at-xact-end (deferred) failure
CREATE TABLE ddl_commands (command text UNIQUE DEFERRABLE INITIALLY DEFERRED);
-- Use an event trigger to log all DDL event tags in it
SET citus.enable_metadata_sync TO OFF;
CREATE FUNCTION log_ddl_tag() RETURNS event_trigger AS $ldt$
BEGIN
INSERT INTO ddl_commands VALUES (tg_tag);
END;
$ldt$ LANGUAGE plpgsql;
RESET citus.enable_metadata_sync;
CREATE EVENT TRIGGER log_ddl_tag ON ddl_command_end EXECUTE PROCEDURE log_ddl_tag();
\c - - - :master_port
-- The above trigger will cause failure at transaction end on one placement.
@ -985,12 +987,14 @@ SELECT create_distributed_table('trigger_table', 'id');
-- first set a trigger on a shard
\c - - - :worker_1_port
SET citus.enable_metadata_sync TO OFF;
CREATE FUNCTION update_value() RETURNS trigger AS $up$
BEGIN
NEW.value := 'trigger enabled';
RETURN NEW;
END;
$up$ LANGUAGE plpgsql;
RESET citus.enable_metadata_sync;
CREATE TRIGGER update_value
BEFORE INSERT ON trigger_table_220017
FOR EACH ROW EXECUTE PROCEDURE update_value();

View File

@ -635,6 +635,8 @@ INSERT INTO citus.pg_dist_object(classid, objid, objsubid) values('pg_class'::re
INSERT INTO citus.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'table_to_distribute'::regclass::oid, 0);
INSERT INTO citus.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'second_dustbunnies'::regclass::oid, 0);
SELECT 1 FROM master_activate_node('localhost', :worker_1_port);
NOTICE: Replicating postgres objects to node localhost:57637
DETAIL: There are 114 objects to replicate, depending on your environment this might take a while
?column?
---------------------------------------------------------------------
1

View File

@ -459,6 +459,7 @@ RETURN $1 * $1;
END;
$function$;
SET citus.enable_metadata_sync TO OFF;
CREATE OR REPLACE FUNCTION square_func(int)
RETURNS int
LANGUAGE plpgsql
@ -467,6 +468,7 @@ BEGIN
RETURN $1 * $1;
END;
$function$;
RESET citus.enable_metadata_sync;
SELECT const_function(1), string_agg(a::character, ',') FROM t1;
SELECT const_function(1), count(b) FROM t1;

View File

@ -43,6 +43,7 @@ CREATE TABLE postgres_local_table(a int, b int);
-- We shouldn't use LIMIT in INSERT SELECT queries to make the test faster as
-- LIMIT would force planner to wrap SELECT query in an intermediate result and
-- this might reduce the coverage of the test cases.
SET citus.enable_metadata_sync TO OFF;
CREATE FUNCTION clear_and_init_test_tables() RETURNS void AS $$
BEGIN
SET client_min_messages to ERROR;
@ -59,6 +60,7 @@ CREATE FUNCTION clear_and_init_test_tables() RETURNS void AS $$
RESET client_min_messages;
END;
$$ LANGUAGE plpgsql;
RESET citus.enable_metadata_sync;
----------------
---- SELECT ----

View File

@ -165,6 +165,7 @@ BEGIN
END;
$$ LANGUAGE 'plpgsql' STRICT;
SET citus.enable_metadata_sync TO OFF;
CREATE OR REPLACE PROCEDURE coordinator_evaluation.test_procedure(int)
LANGUAGE plpgsql
AS $procedure$
@ -174,6 +175,7 @@ BEGIN
PERFORM DISTINCT value FROM coordinator_evaluation_table_2 WHERE key = filterKey;
END;
$procedure$;
RESET citus.enable_metadata_sync;
-- we couldn't find a meaningful query to write for this
-- however this query fails before https://github.com/citusdata/citus/pull/3454

View File

@ -275,11 +275,13 @@ SET citus.log_local_commands TO ON;
SET search_path TO coordinator_evaluation_combinations_modify;
-- returns 2 on the worker
SET citus.enable_metadata_sync TO OFF;
CREATE OR REPLACE FUNCTION get_constant_stable()
RETURNS INT AS $$
BEGIN
RETURN 2;
END; $$ language plpgsql STABLE;
RESET citus.enable_metadata_sync;
-- all local values

View File

@ -373,9 +373,11 @@ inserts AS (
-- a helper function which return true if the coordinated
-- trannsaction uses 2PC
SET citus.enable_metadata_sync TO OFF;
CREATE OR REPLACE FUNCTION coordinated_transaction_should_use_2PC()
RETURNS BOOL LANGUAGE C STRICT VOLATILE AS 'citus',
$$coordinated_transaction_should_use_2PC$$;
RESET citus.enable_metadata_sync;
-- a local SELECT followed by remote SELECTs
-- does not trigger 2PC

View File

@ -93,3 +93,19 @@ DROP SCHEMA collation_tests CASCADE;
DROP SCHEMA collation_tests2 CASCADE;
DROP USER collationuser;
SELECT run_command_on_workers($$DROP USER collationuser;$$);
\c - - - :worker_1_port
-- test creating a collation on a worker
CREATE COLLATION another_german_phonebook (provider = icu, locale = 'de-u-co-phonebk');
-- test if creating a collation on a worker on a local
-- schema raises the right error
SET citus.enable_ddl_propagation TO off;
CREATE SCHEMA collation_creation_on_worker;
SET citus.enable_ddl_propagation TO on;
CREATE COLLATION collation_creation_on_worker.another_german_phonebook (provider = icu, locale = 'de-u-co-phonebk');
SET citus.enable_ddl_propagation TO off;
DROP SCHEMA collation_creation_on_worker;
SET citus.enable_ddl_propagation TO on;

View File

@ -5,6 +5,7 @@ CREATE SCHEMA proc_conflict;
\c - - - :worker_1_port
SET search_path TO proc_conflict;
SET citus.enable_metadata_sync TO OFF;
CREATE FUNCTION existing_func(state int, i int) RETURNS int AS $$
BEGIN
RETURN state * 2 + i;
@ -14,6 +15,7 @@ CREATE AGGREGATE existing_agg(int) (
SFUNC = existing_func,
STYPE = int
);
RESET citus.enable_metadata_sync;
\c - - - :master_port
SET search_path TO proc_conflict;
@ -58,7 +60,7 @@ DROP FUNCTION existing_func(int, int) CASCADE;
\c - - - :worker_1_port
SET search_path TO proc_conflict;
SET citus.enable_metadata_sync TO OFF;
CREATE FUNCTION existing_func(state int, i int) RETURNS int AS $$
BEGIN
RETURN state * 3 + i;
@ -68,6 +70,7 @@ CREATE AGGREGATE existing_agg(int) (
SFUNC = existing_func,
STYPE = int
);
RESET citus.enable_metadata_sync;
\c - - - :master_port
SET search_path TO proc_conflict;

View File

@ -189,12 +189,16 @@ COMMIT;
SELECT func_calls_forcepush_func();
-- Block distributing that function as distributing it causes
-- different test output on PG 14.
SET citus.enable_metadata_sync TO OFF;
CREATE OR REPLACE FUNCTION get_val()
RETURNS INT AS $$
BEGIN
RETURN 100::INT;
END;
$$ LANGUAGE plpgsql;
RESET citus.enable_metadata_sync;
--
-- UDF calling another UDF in a FROM clause
@ -683,6 +687,250 @@ BEGIN
END;
$$ LANGUAGE plpgsql;
CREATE TABLE testnested_table (x int, y int);
SELECT create_distributed_table('testnested_table','x');
CREATE OR REPLACE FUNCTION inner_fn(x int)
RETURNS void
AS $$
DECLARE
BEGIN
INSERT INTO forcepushdown_schema.testnested_table VALUES (x,x);
END;
$$ LANGUAGE plpgsql;
-- Non-force function calling force-delegation function
CREATE OR REPLACE FUNCTION outer_local_fn()
RETURNS void
AS $$
DECLARE
BEGIN
PERFORM 1 FROM inner_fn(1);
INSERT INTO forcepushdown_schema.testnested_table VALUES (2,3);
PERFORM 1 FROM inner_fn(4);
INSERT INTO forcepushdown_schema.testnested_table VALUES (5,6);
END;
$$ LANGUAGE plpgsql;
SELECT create_distributed_function('inner_fn(int)','x',
colocate_with:='testnested_table', force_delegation := true);
SELECT outer_local_fn();
-- Rows from 1-6 should appear
SELECT * FROM testnested_table ORDER BY 1;
BEGIN;
SELECT outer_local_fn();
END;
SELECT * FROM testnested_table ORDER BY 1;
DROP FUNCTION inner_fn(int);
DROP FUNCTION outer_local_fn();
TRUNCATE TABLE testnested_table;
CREATE OR REPLACE FUNCTION inner_fn(x int)
RETURNS void
AS $$
DECLARE
BEGIN
INSERT INTO forcepushdown_schema.testnested_table VALUES (x,x);
END;
$$ LANGUAGE plpgsql;
-- Force-delegation function calling non-force function
CREATE OR REPLACE FUNCTION outer_fn(y int, z int)
RETURNS void
AS $$
DECLARE
BEGIN
PERFORM 1 FROM forcepushdown_schema.inner_fn(y);
INSERT INTO forcepushdown_schema.testnested_table VALUES (y,y);
PERFORM 1 FROM forcepushdown_schema.inner_fn(z);
INSERT INTO forcepushdown_schema.testnested_table VALUES (z,z);
END;
$$ LANGUAGE plpgsql;
SELECT create_distributed_function('inner_fn(int)','x',
colocate_with:='testnested_table', force_delegation := false);
SELECT create_distributed_function('outer_fn(int, int)','y',
colocate_with:='testnested_table', force_delegation := true);
SELECT outer_fn(1, 2);
BEGIN;
SELECT outer_fn(1, 2);
END;
-- No rows
SELECT * FROM testnested_table ORDER BY 1;
-- Force-delegation function calling force-delegation function
CREATE OR REPLACE FUNCTION force_push_inner(y int)
RETURNS void
AS $$
DECLARE
BEGIN
INSERT INTO forcepushdown_schema.testnested_table VALUES (y,y);
END;
$$ LANGUAGE plpgsql;
CREATE OR REPLACE FUNCTION force_push_outer(x int)
RETURNS void
AS $$
DECLARE
BEGIN
INSERT INTO forcepushdown_schema.testnested_table VALUES (x,x);
PERFORM forcepushdown_schema.force_push_inner(x+1) LIMIT 1;
END;
$$ LANGUAGE plpgsql;
SELECT create_distributed_function(
'force_push_outer(int)', 'x',
colocate_with := 'testnested_table',
force_delegation := true
);
SELECT create_distributed_function(
'force_push_inner(int)', 'y',
colocate_with := 'testnested_table',
force_delegation := true
);
-- Keys 7,8,9,14 fall on one node and 15 on a different node
-- Function gets delegated to node with shard-key = 7 and inner function
-- will not be delegated but inserts shard-key = 8 locally
SELECT force_push_outer(7);
BEGIN;
-- Function gets delegated to node with shard-key = 8 and inner function
-- will not be delegated but inserts shard-key = 9 locally
SELECT force_push_outer(8);
END;
BEGIN;
-- Function gets delegated to node with shard-key = 14 and inner function
-- will not be delegated but fails to insert shard-key = 15 remotely
SELECT force_push_outer(14);
END;
SELECT * FROM testnested_table ORDER BY 1;
--
-- Function-1() --> function-2() --> function-3()
--
CREATE OR REPLACE FUNCTION force_push_1(x int)
RETURNS void
AS $$
DECLARE
BEGIN
INSERT INTO forcepushdown_schema.testnested_table VALUES (x,x);
PERFORM forcepushdown_schema.force_push_2(x+1) LIMIT 1;
END;
$$ LANGUAGE plpgsql;
CREATE OR REPLACE FUNCTION force_push_2(y int)
RETURNS void
AS $$
DECLARE
BEGIN
INSERT INTO forcepushdown_schema.testnested_table VALUES (y,y);
PERFORM forcepushdown_schema.force_push_3(y+1) LIMIT 1;
END;
$$ LANGUAGE plpgsql;
CREATE OR REPLACE FUNCTION force_push_3(z int)
RETURNS void
AS $$
DECLARE
BEGIN
INSERT INTO forcepushdown_schema.testnested_table VALUES (z,z);
END;
$$ LANGUAGE plpgsql;
SELECT create_distributed_function(
'force_push_1(int)', 'x',
colocate_with := 'testnested_table',
force_delegation := true
);
SELECT create_distributed_function(
'force_push_2(int)', 'y',
colocate_with := 'testnested_table',
force_delegation := true
);
SELECT create_distributed_function(
'force_push_3(int)', 'z',
colocate_with := 'testnested_table',
force_delegation := true
);
TRUNCATE TABLE testnested_table;
BEGIN;
-- All local inserts
SELECT force_push_1(7);
END;
BEGIN;
-- Local(shard-keys 13, 15) + remote insert (shard-key 14)
SELECT force_push_1(13);
END;
SELECT * FROM testnested_table ORDER BY 1;
TRUNCATE TABLE testnested_table;
CREATE OR REPLACE FUNCTION force_push_inner(y int)
RETURNS void
AS $$
DECLARE
BEGIN
INSERT INTO forcepushdown_schema.testnested_table VALUES (y,y);
END;
$$ LANGUAGE plpgsql;
CREATE OR REPLACE FUNCTION force_push_outer(x int)
RETURNS void
AS $$
DECLARE
BEGIN
PERFORM FROM forcepushdown_schema.force_push_inner(x);
INSERT INTO forcepushdown_schema.testnested_table VALUES (x+1,x+1);
END;
$$ LANGUAGE plpgsql;
SELECT create_distributed_function(
'force_push_inner(int)', 'y',
colocate_with := 'testnested_table',
force_delegation := true
);
SELECT create_distributed_function(
'force_push_outer(int)', 'x',
colocate_with := 'testnested_table',
force_delegation := true
);
BEGIN;
SELECT force_push_outer(7);
END;
TABLE testnested_table ORDER BY 1;
CREATE OR REPLACE FUNCTION force_push_inner(y int)
RETURNS void
AS $$
DECLARE
BEGIN
RAISE NOTICE '%', y;
END;
$$ LANGUAGE plpgsql;
CREATE OR REPLACE FUNCTION force_push_outer(x int)
RETURNS void
AS $$
DECLARE
BEGIN
PERFORM FROM forcepushdown_schema.force_push_inner(x+1);
INSERT INTO forcepushdown_schema.testnested_table VALUES (x,x);
END;
$$ LANGUAGE plpgsql;
BEGIN;
SELECT force_push_outer(9);
END;
TABLE testnested_table ORDER BY 1;
RESET client_min_messages;
SET citus.log_remote_commands TO off;
DROP SCHEMA forcepushdown_schema CASCADE;

View File

@ -0,0 +1,193 @@
CREATE SCHEMA function_propagation_schema;
SET search_path TO 'function_propagation_schema';
-- Check whether supported dependencies can be distributed while propagating functions
-- Check types
SET citus.enable_metadata_sync TO OFF;
CREATE TYPE function_prop_type AS (a int, b int);
RESET citus.enable_metadata_sync;
CREATE OR REPLACE FUNCTION func_1(param_1 function_prop_type)
RETURNS int
LANGUAGE plpgsql AS
$$
BEGIN
return 1;
END;
$$;
-- Check all dependent objects and function depends on all nodes
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema'::regnamespace::oid;
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.function_prop_type'::regtype::oid;
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_1'::regproc::oid;
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema'::regnamespace::oid;$$) ORDER BY 1,2;
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.function_prop_type'::regtype::oid;$$) ORDER BY 1,2;
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_1'::regproc::oid;$$) ORDER BY 1,2;
SET citus.enable_metadata_sync TO OFF;
CREATE TYPE function_prop_type_2 AS (a int, b int);
RESET citus.enable_metadata_sync;
CREATE OR REPLACE FUNCTION func_2(param_1 int)
RETURNS function_prop_type_2
LANGUAGE plpgsql AS
$$
BEGIN
return 1;
END;
$$;
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.function_prop_type_2'::regtype::oid;
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_2'::regproc::oid;
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.function_prop_type_2'::regtype::oid;$$) ORDER BY 1,2;
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_2'::regproc::oid;$$) ORDER BY 1,2;
-- Have a separate check for type created in transaction
BEGIN;
CREATE TYPE function_prop_type_3 AS (a int, b int);
COMMIT;
-- Objects in the body part is not found as dependency
CREATE OR REPLACE FUNCTION func_3(param_1 int)
RETURNS int
LANGUAGE plpgsql AS
$$
DECLARE
internal_param1 function_prop_type_3;
BEGIN
return 1;
END;
$$;
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.function_prop_type_3'::regtype::oid;
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_3'::regproc::oid;
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_3'::regproc::oid;$$) ORDER BY 1,2;
-- Check table
CREATE TABLE function_prop_table(a int, b int);
-- Non-distributed table is not distributed as dependency
CREATE OR REPLACE FUNCTION func_4(param_1 function_prop_table)
RETURNS int
LANGUAGE plpgsql AS
$$
BEGIN
return 1;
END;
$$;
CREATE OR REPLACE FUNCTION func_5(param_1 int)
RETURNS function_prop_table
LANGUAGE plpgsql AS
$$
BEGIN
return 1;
END;
$$;
-- Functions can be created with distributed table dependency
SELECT create_distributed_table('function_prop_table', 'a');
CREATE OR REPLACE FUNCTION func_6(param_1 function_prop_table)
RETURNS int
LANGUAGE plpgsql AS
$$
BEGIN
return 1;
END;
$$;
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_6'::regproc::oid;
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_6'::regproc::oid;$$) ORDER BY 1,2;
-- Views are not supported
CREATE VIEW function_prop_view AS SELECT * FROM function_prop_table;
CREATE OR REPLACE FUNCTION func_7(param_1 function_prop_view)
RETURNS int
LANGUAGE plpgsql AS
$$
BEGIN
return 1;
END;
$$;
CREATE OR REPLACE FUNCTION func_8(param_1 int)
RETURNS function_prop_view
LANGUAGE plpgsql AS
$$
BEGIN
return 1;
END;
$$;
-- Check within transaction
BEGIN;
CREATE TYPE type_in_transaction AS (a int, b int);
CREATE OR REPLACE FUNCTION func_in_transaction(param_1 type_in_transaction)
RETURNS int
LANGUAGE plpgsql AS
$$
BEGIN
return 1;
END;
$$;
-- Within transaction functions are not distributed
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.type_in_transaction'::regtype::oid;
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction'::regproc::oid;
COMMIT;
-- Show that recreating it outside transaction distributes the function and dependencies
CREATE OR REPLACE FUNCTION func_in_transaction(param_1 type_in_transaction)
RETURNS int
LANGUAGE plpgsql AS
$$
BEGIN
return 1;
END;
$$;
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.type_in_transaction'::regtype::oid;
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction'::regproc::oid;
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.type_in_transaction'::regtype::oid;$$) ORDER BY 1,2;
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction'::regproc::oid;$$) ORDER BY 1,2;
-- Test for SQL function with unsupported object in function body
CREATE TABLE table_in_sql_body(id int);
CREATE FUNCTION max_of_table()
RETURNS int
LANGUAGE SQL AS
$$
SELECT max(id) FROM table_in_sql_body
$$;
-- Show that only function has propagated, since the table is not resolved as dependency
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.type_in_transaction'::regclass::oid;
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.max_of_table'::regproc::oid;
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.max_of_table'::regproc::oid;$$) ORDER BY 1,2;
-- Check extension owned table
CREATE TABLE extension_owned_table(a int);
SELECT run_command_on_workers($$
CREATE TABLE function_propagation_schema.extension_owned_table(a int);
$$
);
CREATE EXTENSION seg;
ALTER EXTENSION seg ADD TABLE extension_owned_table;
SELECT run_command_on_workers($$
ALTER EXTENSION seg ADD TABLE function_propagation_schema.extension_owned_table;
$$);
CREATE OR REPLACE FUNCTION func_for_ext_check(param_1 extension_owned_table)
RETURNS int
LANGUAGE plpgsql AS
$$
BEGIN
return 1;
END;
$$;
RESET search_path;
SET client_min_messages TO WARNING;
DROP SCHEMA function_propagation_schema CASCADE;

View File

@ -48,6 +48,7 @@ SET citus.log_local_commands TO ON;
-- returns true of the distribution key filter
-- on the distributed tables (e.g., WHERE key = 1), we'll hit a shard
-- placement which is local to this not
SET citus.enable_metadata_sync TO OFF;
CREATE OR REPLACE FUNCTION shard_of_distribution_column_is_local(dist_key int) RETURNS bool AS $$
DECLARE shard_is_local BOOLEAN := FALSE;
@ -70,6 +71,7 @@ CREATE OR REPLACE FUNCTION shard_of_distribution_column_is_local(dist_key int) R
RETURN shard_is_local;
END;
$$ LANGUAGE plpgsql;
RESET citus.enable_metadata_sync;
-- pick some example values that reside on the shards locally and remote

View File

@ -71,6 +71,7 @@ SET search_path TO local_shard_execution;
-- returns true of the distribution key filter
-- on the distributed tables (e.g., WHERE key = 1), we'll hit a shard
-- placement which is local to this not
SET citus.enable_metadata_sync TO OFF;
CREATE OR REPLACE FUNCTION shard_of_distribution_column_is_local(dist_key int) RETURNS bool AS $$
DECLARE shard_is_local BOOLEAN := FALSE;
@ -93,6 +94,7 @@ CREATE OR REPLACE FUNCTION shard_of_distribution_column_is_local(dist_key int) R
RETURN shard_is_local;
END;
$$ LANGUAGE plpgsql;
RESET citus.enable_metadata_sync;
-- test case for issue #3556
SET citus.log_intermediate_results TO TRUE;
@ -418,6 +420,7 @@ BEGIN;
ROLLBACK;
-- make sure that functions can use local execution
SET citus.enable_metadata_sync TO OFF;
CREATE OR REPLACE PROCEDURE only_local_execution() AS $$
DECLARE cnt INT;
BEGIN
@ -491,6 +494,7 @@ CREATE OR REPLACE PROCEDURE local_execution_followed_by_dist() AS $$
SELECT count(*) INTO cnt FROM distributed_table;
END;
$$ LANGUAGE plpgsql;
RESET citus.enable_metadata_sync;
CALL local_execution_followed_by_dist();

View File

@ -384,6 +384,7 @@ BEGIN;
ROLLBACK;
-- make sure that functions can use local execution
SET citus.enable_metadata_sync TO OFF;
CREATE OR REPLACE PROCEDURE only_local_execution() AS $$
DECLARE cnt INT;
BEGIN
@ -457,6 +458,7 @@ CREATE OR REPLACE PROCEDURE local_execution_followed_by_dist() AS $$
SELECT count(*) INTO cnt FROM distributed_table;
END;
$$ LANGUAGE plpgsql;
RESET citus.enable_metadata_sync;
CALL local_execution_followed_by_dist();

View File

@ -618,6 +618,7 @@ BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
ROLLBACK;
-- create a volatile function that returns the local node id
SET citus.enable_metadata_sync TO OFF;
CREATE OR REPLACE FUNCTION get_node_id()
RETURNS INT AS $$
DECLARE localGroupId int;
@ -630,6 +631,7 @@ BEGIN
nodeport = 57637 AND nodename = 'localhost' AND isactive AND nodecluster = 'default';
RETURN localGroupId;
END; $$ language plpgsql;
RESET citus.enable_metadata_sync;
-- fails because we ingest more placements for the same shards to the same worker node
BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;

View File

@ -10,6 +10,7 @@
SET citus.next_shard_id TO 360000;
-- this function is dropped in Citus10, added here for tests
SET citus.enable_metadata_sync TO OFF;
CREATE OR REPLACE FUNCTION pg_catalog.master_create_distributed_table(table_name regclass,
distribution_column text,
distribution_method citus.distribution_type)
@ -27,6 +28,7 @@ CREATE OR REPLACE FUNCTION pg_catalog.master_create_worker_shards(table_name tex
RETURNS void
AS 'citus', $$master_create_worker_shards$$
LANGUAGE C STRICT;
RESET citus.enable_metadata_sync;
CREATE TABLE lineitem (
l_orderkey bigint not null,

View File

@ -359,10 +359,7 @@ SET citus.enable_metadata_sync TO OFF;
SELECT create_distributed_function('func_custom_param(intpair)');
RESET citus.enable_metadata_sync;
SELECT deparse_and_run_on_workers($cmd$
ALTER FUNCTION func_custom_param RENAME TO func_with_custom_param;
$cmd$);
-- a function that returns TABLE
CREATE FUNCTION func_returns_table(IN count INT)
@ -373,9 +370,7 @@ SET citus.enable_metadata_sync TO OFF;
SELECT create_distributed_function('func_returns_table(INT)');
RESET citus.enable_metadata_sync;
SELECT deparse_and_run_on_workers($cmd$
ALTER FUNCTION func_returns_table ROWS 100;
$cmd$);
-- clear objects
SET client_min_messages TO WARNING; -- suppress cascading objects dropping

View File

@ -33,7 +33,6 @@ SELECT * FROM table1 JOIN nextval('numbers') n ON (id = n) ORDER BY id ASC;
CREATE FUNCTION add(integer, integer) RETURNS integer
AS 'SELECT $1 + $2;'
LANGUAGE SQL;
SELECT create_distributed_function('add(integer,integer)');
SELECT * FROM table1 JOIN add(3,5) sum ON (id = sum) ORDER BY id ASC;
-- Check join of plpgsql functions
@ -46,6 +45,8 @@ $$ LANGUAGE plpgsql;
SELECT * FROM table1 JOIN increment(2) val ON (id = val) ORDER BY id ASC;
-- a function that returns a set of integers
-- Block distributing function as we have tests below to test it locally
SET citus.enable_metadata_sync TO OFF;
CREATE OR REPLACE FUNCTION next_k_integers(IN first_value INTEGER,
IN k INTEGER DEFAULT 3,
OUT result INTEGER)
@ -54,6 +55,7 @@ BEGIN
RETURN QUERY SELECT x FROM generate_series(first_value, first_value+k-1) f(x);
END;
$$ LANGUAGE plpgsql;
RESET citus.enable_metadata_sync;
SELECT *
FROM table1 JOIN next_k_integers(3,2) next_integers ON (id = next_integers.result)
ORDER BY id ASC;
@ -124,6 +126,7 @@ SET client_min_messages TO ERROR;
-- function joins in CTE results can create lateral joins that are not supported
-- we execute the query within a function to consolidate the error messages
-- between different executors
SET citus.enable_metadata_sync TO OFF;
CREATE FUNCTION raise_failed_execution_func_join(query text) RETURNS void AS $$
BEGIN
EXECUTE query;
@ -135,6 +138,7 @@ BEGIN
END IF;
END;
$$LANGUAGE plpgsql;
RESET citus.enable_metadata_sync;
SELECT raise_failed_execution_func_join($$
WITH one_row AS (
@ -146,8 +150,10 @@ SELECT raise_failed_execution_func_join($$
$$);
-- a user-defined immutable function
SET citus.enable_metadata_sync TO OFF;
CREATE OR REPLACE FUNCTION the_answer_to_life()
RETURNS INTEGER IMMUTABLE AS 'SELECT 42' LANGUAGE SQL;
RESET citus.enable_metadata_sync;
SELECT raise_failed_execution_func_join($$
SELECT * FROM table1 JOIN the_answer_to_life() the_answer ON (id = the_answer);

View File

@ -106,6 +106,7 @@ WHERE
user_id < 0;
-- make sure we don't evaluate stable functions with column arguments
SET citus.enable_metadata_sync TO OFF;
CREATE OR REPLACE FUNCTION evaluate_on_master(x int)
RETURNS int LANGUAGE plpgsql STABLE
AS $function$
@ -114,6 +115,7 @@ BEGIN
RETURN x;
END;
$function$;
RESET citus.enable_metadata_sync;
INSERT INTO raw_events_second (user_id, value_1)
SELECT

View File

@ -20,10 +20,12 @@ SELECT nextval('pg_catalog.pg_dist_groupid_seq') AS last_group_id \gset
SELECT nextval('pg_catalog.pg_dist_node_nodeid_seq') AS last_node_id \gset
-- Create the necessary test utility function
SET citus.enable_metadata_sync TO OFF;
CREATE FUNCTION activate_node_snapshot()
RETURNS text[]
LANGUAGE C STRICT
AS 'citus';
RESET citus.enable_metadata_sync;
COMMENT ON FUNCTION activate_node_snapshot()
IS 'commands to activate node snapshot';
@ -43,12 +45,14 @@ ALTER ROLE CURRENT_USER WITH PASSWORD 'dummypassword';
SELECT unnest(activate_node_snapshot()) order by 1;
-- this function is dropped in Citus10, added here for tests
SET citus.enable_metadata_sync TO OFF;
CREATE OR REPLACE FUNCTION pg_catalog.master_create_distributed_table(table_name regclass,
distribution_column text,
distribution_method citus.distribution_type)
RETURNS void
LANGUAGE C STRICT
AS 'citus', $$master_create_distributed_table$$;
RESET citus.enable_metadata_sync;
COMMENT ON FUNCTION pg_catalog.master_create_distributed_table(table_name regclass,
distribution_column text,
distribution_method citus.distribution_type)

View File

@ -363,16 +363,6 @@ UPDATE limit_orders SET array_of_values = 1 || array_of_values WHERE id = 246;
CREATE FUNCTION immutable_append(old_values int[], new_value int)
RETURNS int[] AS $$ SELECT old_values || new_value $$ LANGUAGE SQL IMMUTABLE;
\c - - - :worker_1_port
CREATE FUNCTION immutable_append(old_values int[], new_value int)
RETURNS int[] AS $$ SELECT old_values || new_value $$ LANGUAGE SQL IMMUTABLE;
\c - - - :worker_2_port
CREATE FUNCTION immutable_append(old_values int[], new_value int)
RETURNS int[] AS $$ SELECT old_values || new_value $$ LANGUAGE SQL IMMUTABLE;
\c - - - :master_port
-- immutable function calls with vars are also allowed
UPDATE limit_orders
SET array_of_values = immutable_append(array_of_values, 2) WHERE id = 246;

View File

@ -282,18 +282,16 @@ SELECT * FROM researchers WHERE lab_id = 6;
-- verify 2pc
SELECT count(*) FROM pg_dist_transaction;
-- create a check function
SELECT * from run_command_on_workers('CREATE FUNCTION reject_large_id() RETURNS trigger AS $rli$
CREATE FUNCTION reject_large_id() RETURNS trigger AS $rli$
BEGIN
IF (NEW.id > 30) THEN
RAISE ''illegal value'';
RAISE 'illegal value';
END IF;
RETURN NEW;
END;
$rli$ LANGUAGE plpgsql;')
ORDER BY nodeport;
$rli$ LANGUAGE plpgsql;
-- register after insert trigger
SELECT * FROM run_command_on_placements('researchers', 'CREATE CONSTRAINT TRIGGER reject_large_researcher_id AFTER INSERT ON %s DEFERRABLE INITIALLY DEFERRED FOR EACH ROW EXECUTE PROCEDURE reject_large_id()')
@ -344,9 +342,7 @@ SELECT * FROM researchers WHERE lab_id = 6;
SELECT * from run_command_on_placements('researchers', 'drop trigger reject_large_researcher_id on %s')
ORDER BY nodeport, shardid;
SELECT * FROM run_command_on_workers('drop function reject_large_id()')
ORDER BY nodeport;
DROP FUNCTION reject_large_id();
-- ALTER and copy are compatible
BEGIN;
ALTER TABLE labs ADD COLUMN motto text;
@ -416,6 +412,7 @@ AND s.logicalrelid = 'objects'::regclass;
-- create trigger on one worker to reject certain values
\c - - - :worker_2_port
SET citus.enable_metadata_sync TO OFF;
CREATE FUNCTION reject_bad() RETURNS trigger AS $rb$
BEGIN
IF (NEW.name = 'BAD') THEN
@ -425,6 +422,7 @@ CREATE FUNCTION reject_bad() RETURNS trigger AS $rb$
RETURN NEW;
END;
$rb$ LANGUAGE plpgsql;
RESET citus.enable_metadata_sync;
CREATE CONSTRAINT TRIGGER reject_bad
AFTER INSERT ON objects_1200003
@ -460,6 +458,7 @@ DELETE FROM objects;
-- there cannot be errors on different shards at different times
-- because the first failure will fail the whole transaction
\c - - - :worker_1_port
SET citus.enable_metadata_sync TO OFF;
CREATE FUNCTION reject_bad() RETURNS trigger AS $rb$
BEGIN
IF (NEW.name = 'BAD') THEN
@ -469,6 +468,7 @@ CREATE FUNCTION reject_bad() RETURNS trigger AS $rb$
RETURN NEW;
END;
$rb$ LANGUAGE plpgsql;
RESET citus.enable_metadata_sync;
CREATE CONSTRAINT TRIGGER reject_bad
AFTER INSERT ON labs_1200002
@ -668,7 +668,7 @@ SELECT * FROM reference_modifying_xacts;
-- lets fail on of the workers at before the commit time
\c - - - :worker_1_port
SET citus.enable_metadata_sync TO OFF;
CREATE FUNCTION reject_bad_reference() RETURNS trigger AS $rb$
BEGIN
IF (NEW.key = 999) THEN
@ -678,6 +678,7 @@ CREATE FUNCTION reject_bad_reference() RETURNS trigger AS $rb$
RETURN NEW;
END;
$rb$ LANGUAGE plpgsql;
RESET citus.enable_metadata_sync;
CREATE CONSTRAINT TRIGGER reject_bad_reference
AFTER INSERT ON reference_modifying_xacts_1200006
@ -762,7 +763,7 @@ ABORT;
-- lets fail one of the workers before COMMIT time for the hash table
\c - - - :worker_1_port
SET citus.enable_metadata_sync TO OFF;
CREATE FUNCTION reject_bad_hash() RETURNS trigger AS $rb$
BEGIN
IF (NEW.key = 997) THEN
@ -772,6 +773,7 @@ CREATE FUNCTION reject_bad_hash() RETURNS trigger AS $rb$
RETURN NEW;
END;
$rb$ LANGUAGE plpgsql;
RESET citus.enable_metadata_sync;
CREATE CONSTRAINT TRIGGER reject_bad_hash
AFTER INSERT ON hash_modifying_xacts_1200007

View File

@ -369,21 +369,24 @@ RESET ROLE;
-- to test access to files created during repartition we will create some on worker 1
\c - - - :worker_1_port
SET citus.enable_metadata_sync TO OFF;
CREATE OR REPLACE FUNCTION citus_rm_job_directory(bigint)
RETURNS void
AS 'citus'
LANGUAGE C STRICT;
RESET citus.enable_metadata_sync;
SET ROLE full_access;
SELECT worker_hash_partition_table(42,1,'SELECT a FROM generate_series(1,100) AS a', 'a', 23, ARRAY[-2147483648, -1073741824, 0, 1073741824]::int4[]);
RESET ROLE;
-- all attempts for transfer are initiated from other workers
\c - - - :worker_2_port
SET citus.enable_metadata_sync TO OFF;
CREATE OR REPLACE FUNCTION citus_rm_job_directory(bigint)
RETURNS void
AS 'citus'
LANGUAGE C STRICT;
RESET citus.enable_metadata_sync;
-- super user should not be able to copy files created by a user
SELECT worker_fetch_partition_file(42, 1, 1, 1, 'localhost', :worker_1_port);

View File

@ -180,6 +180,7 @@ select colocate_proc_with_table('mx_call_proc', 'mx_call_dist_table_1'::regclass
-- Test that we handle transactional constructs correctly inside a procedure
-- that is routed to the workers.
SET citus.enable_metadata_sync TO OFF;
CREATE PROCEDURE mx_call_proc_tx(x int) LANGUAGE plpgsql AS $$
BEGIN
INSERT INTO multi_mx_call.mx_call_dist_table_1 VALUES (x, -1), (x+1, 4);
@ -189,6 +190,7 @@ BEGIN
-- Now do the final update!
UPDATE multi_mx_call.mx_call_dist_table_1 SET val = val-1 WHERE id >= x;
END;$$;
RESET citus.enable_metadata_sync;
-- before distribution ...
CALL multi_mx_call.mx_call_proc_tx(10);

View File

@ -63,33 +63,7 @@ CREATE TYPE order_side_mx AS ENUM ('buy', 'sell');
CREATE TABLE citus_local_table(a int);
SELECT citus_add_local_table_to_metadata('citus_local_table');
-- create UDFs in worker node
CREATE OR REPLACE FUNCTION simpleTestFunction(theValue integer)
RETURNS text AS
$$
DECLARE
strresult text;
BEGIN
RETURN theValue * 3 / 2 + 1;
END;
$$
LANGUAGE 'plpgsql' IMMUTABLE;
SET search_path TO citus_mx_test_schema;
CREATE OR REPLACE FUNCTION simpleTestFunction2(theValue integer)
RETURNS text AS
$$
DECLARE
strresult text;
BEGIN
RETURN theValue * 3 / 2 + 1;
END;
$$
LANGUAGE 'plpgsql' IMMUTABLE;
CREATE FUNCTION public.immutable_append_mx(old_values int[], new_value int)
RETURNS int[] AS $$ SELECT old_values || new_value $$ LANGUAGE SQL IMMUTABLE;
-- create operator
CREATE OPERATOR citus_mx_test_schema.=== (
LEFTARG = int,
@ -103,32 +77,7 @@ CREATE OPERATOR citus_mx_test_schema.=== (
-- now create required stuff in the worker 2
\c - - - :worker_2_port
-- create UDF
CREATE OR REPLACE FUNCTION simpleTestFunction(theValue integer)
RETURNS text AS
$$
DECLARE
strresult text;
BEGIN
RETURN theValue * 3 / 2 + 1;
END;
$$
LANGUAGE 'plpgsql' IMMUTABLE;
SET search_path TO citus_mx_test_schema;
CREATE OR REPLACE FUNCTION simpleTestFunction2(theValue integer)
RETURNS text AS
$$
DECLARE
strresult text;
BEGIN
RETURN theValue * 3 / 2 + 1;
END;
$$
LANGUAGE 'plpgsql' IMMUTABLE;
CREATE FUNCTION public.immutable_append_mx(old_values int[], new_value int)
RETURNS int[] AS $$ SELECT old_values || new_value $$ LANGUAGE SQL IMMUTABLE;
-- create operator
CREATE OPERATOR citus_mx_test_schema.=== (

View File

@ -18,6 +18,7 @@ VACUUM ANALYZE supplier_mx;
\c - - - :worker_1_port
-- Function that parses explain output as JSON
SET citus.enable_metadata_sync TO OFF;
CREATE FUNCTION explain_json(query text)
RETURNS jsonb
AS $BODY$
@ -43,6 +44,7 @@ $BODY$ LANGUAGE plpgsql;
\c - - - :worker_2_port
-- Function that parses explain output as JSON
SET citus.enable_metadata_sync TO OFF;
CREATE FUNCTION explain_json(query text)
RETURNS jsonb
AS $BODY$
@ -65,6 +67,7 @@ BEGIN
RETURN result;
END;
$BODY$ LANGUAGE plpgsql;
RESET citus.enable_metadata_sync;
-- Test Text format

Some files were not shown because too many files have changed in this diff Show More