diff --git a/src/backend/distributed/commands/alter_table.c b/src/backend/distributed/commands/alter_table.c index d574e997a..5fa670ab2 100644 --- a/src/backend/distributed/commands/alter_table.c +++ b/src/backend/distributed/commands/alter_table.c @@ -248,7 +248,8 @@ undistribute_table(PG_FUNCTION_ARGS) TableConversionParameters params = { .relationId = relationId, - .cascadeViaForeignKeys = cascadeViaForeignKeys + .cascadeViaForeignKeys = cascadeViaForeignKeys, + .bypassTenantCheck = false }; UndistributeTable(¶ms); @@ -429,6 +430,55 @@ UndistributeTables(List *relationIdList) } +/* + * EnsureUndistributeTenantTableSafe ensures that it is safe to undistribute a tenant table. + */ +void +EnsureUndistributeTenantTableSafe(Oid relationId, const char *operationName) +{ + Oid schemaId = get_rel_namespace(relationId); + Assert(IsTenantSchema(schemaId)); + + /* We only allow undistribute while altering schema */ + if (strcmp(operationName, TenantOperationNames[TENANT_SET_SCHEMA]) != 0) + { + ErrorIfTenantTable(relationId, operationName); + } + + char *tableName = get_rel_name(relationId); + char *schemaName = get_namespace_name(schemaId); + + /* + * Partition table cannot be undistributed. Otherwise, its parent table would still + * be a tenant table whereas partition table would be a local table. + */ + if (PartitionTable(relationId)) + { + ereport(ERROR, (errmsg("%s is not allowed for partition table %s in distributed " + "schema %s", operationName, tableName, schemaName), + errdetail("partition table should be under the same distributed " + "schema as its parent and be a tenant table."))); + } + + /* + * When table is referenced by or referencing to a table in the same tenant + * schema, we should disallow undistributing the table since we do not allow + * foreign keys from/to Citus local or Postgres local table to/from distributed + * schema. + */ + List *fkeyCommandsWithSingleShardTables = + GetFKeyCreationCommandsRelationInvolvedWithTableType( + relationId, INCLUDE_SINGLE_SHARD_TABLES); + if (fkeyCommandsWithSingleShardTables != NIL) + { + ereport(ERROR, (errmsg("%s is not allowed for table %s in distributed schema %s", + operationName, tableName, schemaName), + errdetail("distributed schemas cannot have foreign keys from/to " + "local tables or different schema"))); + } +} + + /* * UndistributeTable undistributes the given table. It uses ConvertTable function to * create a new local table and move everything to that table. @@ -449,7 +499,13 @@ UndistributeTable(TableConversionParameters *params) "because the table is not distributed"))); } - ErrorIfTenantTable(params->relationId, "undistribute_table"); + Oid schemaId = get_rel_namespace(params->relationId); + if (!params->bypassTenantCheck && IsTenantSchema(schemaId) && + IsCitusTableType(params->relationId, SINGLE_SHARD_DISTRIBUTED)) + { + EnsureUndistributeTenantTableSafe(params->relationId, + TenantOperationNames[TENANT_UNDISTRIBUTE_TABLE]); + } if (!params->cascadeViaForeignKeys) { @@ -506,7 +562,7 @@ AlterDistributedTable(TableConversionParameters *params) "is not distributed"))); } - ErrorIfTenantTable(params->relationId, "alter_distributed_table"); + ErrorIfTenantTable(params->relationId, TenantOperationNames[TENANT_ALTER_TABLE]); ErrorIfColocateWithTenantTable(params->colocateWith); EnsureTableNotForeign(params->relationId); @@ -1267,7 +1323,8 @@ ErrorIfColocateWithTenantTable(char *colocateWith) { text *colocateWithTableNameText = cstring_to_text(colocateWith); Oid colocateWithTableId = ResolveRelationId(colocateWithTableNameText, false); - ErrorIfTenantTable(colocateWithTableId, "colocate_with"); + ErrorIfTenantTable(colocateWithTableId, + TenantOperationNames[TENANT_COLOCATE_WITH]); } } diff --git a/src/backend/distributed/commands/cascade_table_operation_for_connected_relations.c b/src/backend/distributed/commands/cascade_table_operation_for_connected_relations.c index 1c01028d3..1102a3a51 100644 --- a/src/backend/distributed/commands/cascade_table_operation_for_connected_relations.c +++ b/src/backend/distributed/commands/cascade_table_operation_for_connected_relations.c @@ -468,7 +468,8 @@ ExecuteCascadeOperationForRelationIdList(List *relationIdList, { TableConversionParameters params = { .relationId = relationId, - .cascadeViaForeignKeys = cascadeViaForeignKeys + .cascadeViaForeignKeys = cascadeViaForeignKeys, + .bypassTenantCheck = false }; UndistributeTable(¶ms); } diff --git a/src/backend/distributed/commands/foreign_constraint.c b/src/backend/distributed/commands/foreign_constraint.c index b48b6c54a..40ccb0ddf 100644 --- a/src/backend/distributed/commands/foreign_constraint.c +++ b/src/backend/distributed/commands/foreign_constraint.c @@ -1356,6 +1356,10 @@ IsTableTypeIncluded(Oid relationId, int flags) { return (flags & INCLUDE_LOCAL_TABLES) != 0; } + else if (IsCitusTableType(relationId, SINGLE_SHARD_DISTRIBUTED)) + { + return (flags & INCLUDE_SINGLE_SHARD_TABLES) != 0; + } else if (IsCitusTableType(relationId, DISTRIBUTED_TABLE)) { return (flags & INCLUDE_DISTRIBUTED_TABLES) != 0; diff --git a/src/backend/distributed/commands/multi_copy.c b/src/backend/distributed/commands/multi_copy.c index 8e92fd7a8..f8e6378d4 100644 --- a/src/backend/distributed/commands/multi_copy.c +++ b/src/backend/distributed/commands/multi_copy.c @@ -2128,12 +2128,36 @@ CitusCopyDestReceiverStartup(DestReceiver *dest, int operation, int columnCount = inputTupleDescriptor->natts; Oid *finalTypeArray = palloc0(columnCount * sizeof(Oid)); - copyDest->columnCoercionPaths = - ColumnCoercionPaths(destTupleDescriptor, inputTupleDescriptor, - tableId, columnNameList, finalTypeArray); - - copyDest->columnOutputFunctions = - TypeOutputFunctions(columnCount, finalTypeArray, copyOutState->binary); + /* + * To ensure the proper co-location and distribution of the target table, + * the entire process of repartitioning intermediate files requires the + * destReceiver to be created on the target rather than the source. + * + * Within this specific code path, it is assumed that the employed model + * is for insert-select. Consequently, it validates the column types of + * destTupleDescriptor(target) during the intermediate result generation + * process. However, this approach varies significantly for MERGE operations, + * where the source tuple(s) can have arbitrary types and are not required to + * align with the target column names. + * + * Despite this minor setback, a significant portion of the code responsible + * for repartitioning intermediate files can be reused for the MERGE + * operation. By leveraging the ability to perform actual coercion during + * the writing process to the target table, we can bypass this specific route. + */ + if (copyDest->skipCoercions) + { + copyDest->columnOutputFunctions = + ColumnOutputFunctions(inputTupleDescriptor, copyOutState->binary); + } + else + { + copyDest->columnCoercionPaths = + ColumnCoercionPaths(destTupleDescriptor, inputTupleDescriptor, + tableId, columnNameList, finalTypeArray); + copyDest->columnOutputFunctions = + TypeOutputFunctions(columnCount, finalTypeArray, copyOutState->binary); + } } /* wrap the column names as Values */ @@ -2597,9 +2621,11 @@ ShardIdForTuple(CitusCopyDestReceiver *copyDest, Datum *columnValues, bool *colu /* find the partition column value */ partitionColumnValue = columnValues[partitionColumnIndex]; - - /* annoyingly this is evaluated twice, but at least we don't crash! */ - partitionColumnValue = CoerceColumnValue(partitionColumnValue, coercePath); + if (!copyDest->skipCoercions) + { + /* annoyingly this is evaluated twice, but at least we don't crash! */ + partitionColumnValue = CoerceColumnValue(partitionColumnValue, coercePath); + } } /* diff --git a/src/backend/distributed/commands/schema_based_sharding.c b/src/backend/distributed/commands/schema_based_sharding.c index 91bae0943..b717cb5ae 100644 --- a/src/backend/distributed/commands/schema_based_sharding.c +++ b/src/backend/distributed/commands/schema_based_sharding.c @@ -40,6 +40,14 @@ static void EnsureSchemaExist(Oid schemaId); /* controlled via citus.enable_schema_based_sharding GUC */ bool EnableSchemaBasedSharding = false; +const char *TenantOperationNames[TOTAL_TENANT_OPERATION] = { + "undistribute_table", + "alter_distributed_table", + "colocate_with", + "update_distributed_table_colocation", + "set schema", +}; + PG_FUNCTION_INFO_V1(citus_internal_unregister_tenant_schema_globally); PG_FUNCTION_INFO_V1(citus_schema_distribute); @@ -374,12 +382,7 @@ SchemaGetNonShardTableIdList(Oid schemaId) * - Schema name is in the allowed-list, * - Schema does not depend on an extension (created by extension), * - No extension depends on the schema (CREATE EXTENSION SCHEMA ), - * - Current user should be the owner of tables under the schema, - * - Table kinds are supported, - * - Referencing and referenced foreign keys for the tables under the schema are - * supported, - * - Tables under the schema are not owned by an extension, - * - Only Citus local and Postgres local tables exist under the schema. + * - Some checks for the table for being a valid tenant table. */ static void EnsureSchemaCanBeDistributed(Oid schemaId, List *schemaTableIdList) @@ -409,39 +412,55 @@ EnsureSchemaCanBeDistributed(Oid schemaId, List *schemaTableIdList) Oid relationId = InvalidOid; foreach_oid(relationId, schemaTableIdList) { - /* Ensure table owner */ - EnsureTableOwner(relationId); + EnsureTenantTable(relationId, "citus_schema_distribute"); + } +} - /* Check relation kind */ - EnsureTableKindSupportedForTenantSchema(relationId); - /* Check foreign keys */ - EnsureFKeysForTenantTable(relationId); +/* + * EnsureTenantTable ensures the table can be a valid tenant table. + * - Current user should be the owner of table, + * - Table kind is supported, + * - Referencing and referenced foreign keys for the table are supported, + * - Table is not owned by an extension, + * - Table should be Citus local or Postgres local table. + */ +void +EnsureTenantTable(Oid relationId, char *operationName) +{ + /* Ensure table owner */ + EnsureTableOwner(relationId); - /* Check table not owned by an extension */ - ObjectAddress *tableAddress = palloc0(sizeof(ObjectAddress)); - ObjectAddressSet(*tableAddress, RelationRelationId, relationId); - if (IsAnyObjectAddressOwnedByExtension(list_make1(tableAddress), NULL)) - { - char *tableName = get_namespace_name(schemaId); - ereport(ERROR, (errmsg("schema cannot be distributed since it has " - "table %s which is owned by an extension", - tableName))); - } + /* Check relation kind */ + EnsureTableKindSupportedForTenantSchema(relationId); - /* Postgres local tables are allowed */ - if (!IsCitusTable(relationId)) - { - continue; - } + /* Check foreign keys */ + EnsureFKeysForTenantTable(relationId); - /* Only Citus local tables, amongst Citus table types, are allowed */ - if (!IsCitusTableType(relationId, CITUS_LOCAL_TABLE)) - { - ereport(ERROR, (errmsg("schema already has distributed tables"), - errhint("Undistribute distributed tables under " - "the schema before distributing the schema."))); - } + /* Check table not owned by an extension */ + ObjectAddress *tableAddress = palloc0(sizeof(ObjectAddress)); + ObjectAddressSet(*tableAddress, RelationRelationId, relationId); + if (IsAnyObjectAddressOwnedByExtension(list_make1(tableAddress), NULL)) + { + Oid schemaId = get_rel_namespace(relationId); + char *tableName = get_namespace_name(schemaId); + ereport(ERROR, (errmsg("schema cannot be distributed since it has " + "table %s which is owned by an extension", + tableName))); + } + + /* Postgres local tables are allowed */ + if (!IsCitusTable(relationId)) + { + return; + } + + /* Only Citus local tables, amongst Citus table types, are allowed */ + if (!IsCitusTableType(relationId, CITUS_LOCAL_TABLE)) + { + ereport(ERROR, (errmsg("distributed schema cannot have distributed tables"), + errhint("Undistribute distributed tables before " + "'%s'.", operationName))); } } @@ -743,7 +762,7 @@ citus_schema_undistribute(PG_FUNCTION_ARGS) * if the given relation is a tenant table. */ void -ErrorIfTenantTable(Oid relationId, char *operationName) +ErrorIfTenantTable(Oid relationId, const char *operationName) { if (IsTenantSchema(get_rel_namespace(relationId))) { @@ -753,20 +772,3 @@ ErrorIfTenantTable(Oid relationId, char *operationName) operationName))); } } - - -/* - * ErrorIfTenantSchema errors out with the given operation name, - * if the given schema is a tenant schema. - */ -void -ErrorIfTenantSchema(Oid nspOid, char *operationName) -{ - if (IsTenantSchema(nspOid)) - { - ereport(ERROR, (errmsg( - "%s is not allowed for %s because it is a distributed schema", - get_namespace_name(nspOid), - operationName))); - } -} diff --git a/src/backend/distributed/commands/table.c b/src/backend/distributed/commands/table.c index a5e997969..4ea28c71d 100644 --- a/src/backend/distributed/commands/table.c +++ b/src/backend/distributed/commands/table.c @@ -41,6 +41,7 @@ #include "distributed/resource_lock.h" #include "distributed/version_compat.h" #include "distributed/worker_shard_visibility.h" +#include "distributed/tenant_schema_metadata.h" #include "foreign/foreign.h" #include "lib/stringinfo.h" #include "nodes/parsenodes.h" @@ -2310,9 +2311,52 @@ PreprocessAlterTableSchemaStmt(Node *node, const char *queryString, return NIL; } - ErrorIfTenantTable(relationId, "ALTER TABLE SET SCHEMA"); - ErrorIfTenantSchema(get_namespace_oid(stmt->newschema, false), - "ALTER TABLE SET SCHEMA"); + Oid oldSchemaId = get_rel_namespace(relationId); + Oid newSchemaId = get_namespace_oid(stmt->newschema, stmt->missing_ok); + if (!OidIsValid(oldSchemaId) || !OidIsValid(newSchemaId)) + { + return NIL; + } + + /* Do nothing if new schema is the same as old schema */ + if (newSchemaId == oldSchemaId) + { + return NIL; + } + + /* Undistribute table if its old schema is a tenant schema */ + if (IsTenantSchema(oldSchemaId) && IsCoordinator()) + { + EnsureUndistributeTenantTableSafe(relationId, + TenantOperationNames[TENANT_SET_SCHEMA]); + + char *oldSchemaName = get_namespace_name(oldSchemaId); + char *tableName = stmt->relation->relname; + ereport(NOTICE, (errmsg("undistributing table %s in distributed schema %s " + "before altering its schema", tableName, oldSchemaName))); + + /* Undistribute tenant table by suppressing weird notices */ + TableConversionParameters params = { + .relationId = relationId, + .cascadeViaForeignKeys = false, + .bypassTenantCheck = true, + .suppressNoticeMessages = true, + }; + UndistributeTable(¶ms); + + /* relation id changes after undistribute_table */ + relationId = get_relname_relid(tableName, oldSchemaId); + + /* + * After undistribution, the table could be Citus table or Postgres table. + * If it is Postgres table, do not propagate the `ALTER TABLE SET SCHEMA` + * command to workers. + */ + if (!IsCitusTable(relationId)) + { + return NIL; + } + } DDLJob *ddlJob = palloc0(sizeof(DDLJob)); QualifyTreeNode((Node *) stmt); @@ -4166,3 +4210,61 @@ ConvertNewTableIfNecessary(Node *createStmt) CreateCitusLocalTable(createdRelationId, cascade, autoConverted); } } + + +/* + * ConvertToTenantTableIfNecessary converts given relation to a tenant table if its + * schema changed to a distributed schema. + */ +void +ConvertToTenantTableIfNecessary(AlterObjectSchemaStmt *stmt) +{ + Assert(stmt->objectType == OBJECT_TABLE || stmt->objectType == OBJECT_FOREIGN_TABLE); + + if (!IsCoordinator()) + { + return; + } + + /* + * We will let Postgres deal with missing_ok + */ + List *tableAddresses = GetObjectAddressListFromParseTree((Node *) stmt, true, true); + + /* the code-path only supports a single object */ + Assert(list_length(tableAddresses) == 1); + + /* We have already asserted that we have exactly 1 address in the addresses. */ + ObjectAddress *tableAddress = linitial(tableAddresses); + char relKind = get_rel_relkind(tableAddress->objectId); + if (relKind == RELKIND_SEQUENCE || relKind == RELKIND_VIEW) + { + return; + } + + Oid relationId = tableAddress->objectId; + Oid schemaId = get_namespace_oid(stmt->newschema, stmt->missing_ok); + if (!OidIsValid(schemaId)) + { + return; + } + + /* + * Make table a tenant table when its schema actually changed. When its schema + * is not changed as in `ALTER TABLE SET SCHEMA `, we detect + * that by seeing the table is still a single shard table. (i.e. not undistributed + * at `preprocess` step) + */ + if (!IsCitusTableType(relationId, SINGLE_SHARD_DISTRIBUTED) && + IsTenantSchema(schemaId)) + { + EnsureTenantTable(relationId, "ALTER TABLE SET SCHEMA"); + + char *schemaName = get_namespace_name(schemaId); + char *tableName = stmt->relation->relname; + ereport(NOTICE, (errmsg("converting table %s to a tenant table in distributed " + "schema %s", tableName, schemaName))); + + CreateTenantSchemaTable(relationId); + } +} diff --git a/src/backend/distributed/commands/utility_hook.c b/src/backend/distributed/commands/utility_hook.c index e274731b8..3afe405b7 100644 --- a/src/backend/distributed/commands/utility_hook.c +++ b/src/backend/distributed/commands/utility_hook.c @@ -186,7 +186,9 @@ multi_ProcessUtility(PlannedStmt *pstmt, IsA(parsetree, ExecuteStmt) || IsA(parsetree, PrepareStmt) || IsA(parsetree, DiscardStmt) || - IsA(parsetree, DeallocateStmt)) + IsA(parsetree, DeallocateStmt) || + IsA(parsetree, DeclareCursorStmt) || + IsA(parsetree, FetchStmt)) { /* * Skip additional checks for common commands that do not have any @@ -370,6 +372,18 @@ multi_ProcessUtility(PlannedStmt *pstmt, ConvertNewTableIfNecessary(createStmt); } + + if (context == PROCESS_UTILITY_TOPLEVEL && + IsA(parsetree, AlterObjectSchemaStmt)) + { + AlterObjectSchemaStmt *alterSchemaStmt = castNode(AlterObjectSchemaStmt, + parsetree); + if (alterSchemaStmt->objectType == OBJECT_TABLE || + alterSchemaStmt->objectType == OBJECT_FOREIGN_TABLE) + { + ConvertToTenantTableIfNecessary(alterSchemaStmt); + } + } } UtilityHookLevel--; @@ -999,7 +1013,8 @@ UndistributeDisconnectedCitusLocalTables(void) TableConversionParameters params = { .relationId = citusLocalTableId, .cascadeViaForeignKeys = true, - .suppressNoticeMessages = true + .suppressNoticeMessages = true, + .bypassTenantCheck = false }; UndistributeTable(¶ms); } diff --git a/src/backend/distributed/executor/adaptive_executor.c b/src/backend/distributed/executor/adaptive_executor.c index d88cdbea8..0c710909b 100644 --- a/src/backend/distributed/executor/adaptive_executor.c +++ b/src/backend/distributed/executor/adaptive_executor.c @@ -1015,6 +1015,32 @@ ExecuteTaskListOutsideTransaction(RowModifyLevel modLevel, List *taskList, } +/* + * ExecuteTaskListIntoTupleDestWithParam is a proxy to ExecuteTaskListExtended() which uses + * bind params from executor state, and with defaults for some of the arguments. + */ +uint64 +ExecuteTaskListIntoTupleDestWithParam(RowModifyLevel modLevel, List *taskList, + TupleDestination *tupleDest, + bool expectResults, + ParamListInfo paramListInfo) +{ + int targetPoolSize = MaxAdaptiveExecutorPoolSize; + bool localExecutionSupported = true; + ExecutionParams *executionParams = CreateBasicExecutionParams( + modLevel, taskList, targetPoolSize, localExecutionSupported + ); + + executionParams->xactProperties = DecideTransactionPropertiesForTaskList( + modLevel, taskList, false); + executionParams->expectResults = expectResults; + executionParams->tupleDestination = tupleDest; + executionParams->paramListInfo = paramListInfo; + + return ExecuteTaskListExtended(executionParams); +} + + /* * ExecuteTaskListIntoTupleDest is a proxy to ExecuteTaskListExtended() with defaults * for some of the arguments. @@ -1052,7 +1078,6 @@ ExecuteTaskListExtended(ExecutionParams *executionParams) return 0; } - ParamListInfo paramListInfo = NULL; uint64 locallyProcessedRows = 0; TupleDestination *defaultTupleDest = executionParams->tupleDestination; @@ -1065,7 +1090,7 @@ ExecuteTaskListExtended(ExecutionParams *executionParams) DistributedExecution *execution = CreateDistributedExecution( executionParams->modLevel, executionParams->taskList, - paramListInfo, executionParams->targetPoolSize, + executionParams->paramListInfo, executionParams->targetPoolSize, defaultTupleDest, &executionParams->xactProperties, executionParams->jobIdList, executionParams->localExecutionSupported); @@ -1117,6 +1142,7 @@ CreateBasicExecutionParams(RowModifyLevel modLevel, executionParams->expectResults = false; executionParams->isUtilityCommand = false; executionParams->jobIdList = NIL; + executionParams->paramListInfo = NULL; return executionParams; } diff --git a/src/backend/distributed/executor/citus_custom_scan.c b/src/backend/distributed/executor/citus_custom_scan.c index 3dc1f5068..a2a2ff6cb 100644 --- a/src/backend/distributed/executor/citus_custom_scan.c +++ b/src/backend/distributed/executor/citus_custom_scan.c @@ -27,6 +27,8 @@ #include "distributed/listutils.h" #include "distributed/local_executor.h" #include "distributed/local_plan_cache.h" +#include "distributed/merge_executor.h" +#include "distributed/merge_planner.h" #include "distributed/multi_executor.h" #include "distributed/multi_server_executor.h" #include "distributed/multi_router_planner.h" @@ -53,6 +55,7 @@ extern AllowedDistributionColumn AllowedDistributionColumnValue; static Node * AdaptiveExecutorCreateScan(CustomScan *scan); static Node * NonPushableInsertSelectCreateScan(CustomScan *scan); static Node * DelayedErrorCreateScan(CustomScan *scan); +static Node * NonPushableMergeCommandCreateScan(CustomScan *scan); /* functions that are common to different scans */ static void CitusBeginScan(CustomScanState *node, EState *estate, int eflags); @@ -88,6 +91,11 @@ CustomScanMethods DelayedErrorCustomScanMethods = { DelayedErrorCreateScan }; +CustomScanMethods NonPushableMergeCommandCustomScanMethods = { + "Citus MERGE INTO ...", + NonPushableMergeCommandCreateScan +}; + /* * Define executor methods for the different executor types. @@ -111,6 +119,16 @@ static CustomExecMethods NonPushableInsertSelectCustomExecMethods = { }; +static CustomExecMethods NonPushableMergeCommandCustomExecMethods = { + .CustomName = "NonPushableMergeCommandScan", + .BeginCustomScan = CitusBeginScan, + .ExecCustomScan = NonPushableMergeCommandExecScan, + .EndCustomScan = CitusEndScan, + .ReScanCustomScan = CitusReScan, + .ExplainCustomScan = NonPushableMergeCommandExplainScan +}; + + /* * IsCitusCustomState returns if a given PlanState node is a CitusCustomState node. */ @@ -124,7 +142,8 @@ IsCitusCustomState(PlanState *planState) CustomScanState *css = castNode(CustomScanState, planState); if (css->methods == &AdaptiveExecutorCustomExecMethods || - css->methods == &NonPushableInsertSelectCustomExecMethods) + css->methods == &NonPushableInsertSelectCustomExecMethods || + css->methods == &NonPushableMergeCommandCustomExecMethods) { return true; } @@ -142,6 +161,7 @@ RegisterCitusCustomScanMethods(void) RegisterCustomScanMethods(&AdaptiveExecutorCustomScanMethods); RegisterCustomScanMethods(&NonPushableInsertSelectCustomScanMethods); RegisterCustomScanMethods(&DelayedErrorCustomScanMethods); + RegisterCustomScanMethods(&NonPushableMergeCommandCustomScanMethods); } @@ -723,6 +743,26 @@ DelayedErrorCreateScan(CustomScan *scan) } +/* + * NonPushableMergeCommandCreateScan creates the scan state for executing + * MERGE INTO ... into a distributed table with repartition of source rows. + */ +static Node * +NonPushableMergeCommandCreateScan(CustomScan *scan) +{ + CitusScanState *scanState = palloc0(sizeof(CitusScanState)); + + scanState->executorType = MULTI_EXECUTOR_NON_PUSHABLE_MERGE_QUERY; + scanState->customScanState.ss.ps.type = T_CustomScanState; + scanState->distributedPlan = GetDistributedPlan(scan); + scanState->customScanState.methods = &NonPushableMergeCommandCustomExecMethods; + scanState->finishedPreScan = false; + scanState->finishedRemoteScan = false; + + return (Node *) scanState; +} + + /* * CitusEndScan is used to clean up tuple store of the given custom scan state. */ @@ -780,7 +820,19 @@ CitusEndScan(CustomScanState *node) */ static void CitusReScan(CustomScanState *node) -{ } +{ + if (node->ss.ps.ps_ResultTupleSlot) + { + ExecClearTuple(node->ss.ps.ps_ResultTupleSlot); + } + ExecScanReScan(&node->ss); + + CitusScanState *scanState = (CitusScanState *) node; + if (scanState->tuplestorestate) + { + tuplestore_rescan(scanState->tuplestorestate); + } +} /* diff --git a/src/backend/distributed/executor/distributed_intermediate_results.c b/src/backend/distributed/executor/distributed_intermediate_results.c index e7a5830e6..c10303e18 100644 --- a/src/backend/distributed/executor/distributed_intermediate_results.c +++ b/src/backend/distributed/executor/distributed_intermediate_results.c @@ -610,6 +610,18 @@ QueryStringForFragmentsTransfer(NodeToNodeFragmentsTransfer *fragmentsTransfer) StringInfo fragmentNamesArrayString = makeStringInfo(); int fragmentCount = 0; NodePair *nodePair = &fragmentsTransfer->nodes; + uint32 sourceNodeId = nodePair->sourceNodeId; + + /* + * If the placement is dummy, for example, queries that generate + * intermediate results at the coordinator that need to be redistributed + * to worker nodes, we need the local id. + */ + if (sourceNodeId == LOCAL_NODE_ID) + { + nodePair->sourceNodeId = GetLocalNodeId(); + } + WorkerNode *sourceNode = LookupNodeByNodeIdOrError(nodePair->sourceNodeId); appendStringInfoString(fragmentNamesArrayString, "ARRAY["); diff --git a/src/backend/distributed/executor/insert_select_executor.c b/src/backend/distributed/executor/insert_select_executor.c index 58312ba19..4a15289e6 100644 --- a/src/backend/distributed/executor/insert_select_executor.c +++ b/src/backend/distributed/executor/insert_select_executor.c @@ -20,6 +20,7 @@ #include "distributed/insert_select_planner.h" #include "distributed/intermediate_results.h" #include "distributed/local_executor.h" +#include "distributed/merge_planner.h" #include "distributed/multi_executor.h" #include "distributed/multi_partitioning_utils.h" #include "distributed/multi_physical_planner.h" @@ -63,8 +64,6 @@ static HTAB * ExecutePlanIntoColocatedIntermediateResults(Oid targetRelationId, PlannedStmt *selectPlan, EState *executorState, char *intermediateResultIdPrefix); -static List * BuildColumnNameListFromTargetList(Oid targetRelationId, - List *insertTargetList); static int PartitionColumnIndexFromColumnList(Oid relationId, List *columnNameList); static void WrapTaskListForProjection(List *taskList, List *projectedTargetEntries); @@ -374,7 +373,7 @@ ExecutePlanIntoRelation(Oid targetRelationId, List *insertTargetList, * BuildColumnNameListForCopyStatement build the column name list given the insert * target list. */ -static List * +List * BuildColumnNameListFromTargetList(Oid targetRelationId, List *insertTargetList) { List *columnNameList = NIL; @@ -424,13 +423,13 @@ PartitionColumnIndexFromColumnList(Oid relationId, List *columnNameList) * given target list. */ int -DistributionColumnIndex(List *insertTargetList, Var *partitionColumn) +DistributionColumnIndex(List *insertTargetList, Var *distributionColumn) { TargetEntry *insertTargetEntry = NULL; int targetEntryIndex = 0; foreach_ptr(insertTargetEntry, insertTargetList) { - if (insertTargetEntry->resno == partitionColumn->varattno) + if (insertTargetEntry->resno == distributionColumn->varattno) { return targetEntryIndex; } diff --git a/src/backend/distributed/executor/merge_executor.c b/src/backend/distributed/executor/merge_executor.c new file mode 100644 index 000000000..f501497c0 --- /dev/null +++ b/src/backend/distributed/executor/merge_executor.c @@ -0,0 +1,337 @@ +/*------------------------------------------------------------------------- + * + * merge_executor.c + * + * Executor logic for MERGE SQL statement. + * + * Copyright (c) Citus Data, Inc. + *------------------------------------------------------------------------- + */ + +#include "postgres.h" +#include "miscadmin.h" + +#include "distributed/distributed_execution_locks.h" +#include "distributed/insert_select_executor.h" +#include "distributed/intermediate_results.h" +#include "distributed/listutils.h" +#include "distributed/merge_executor.h" +#include "distributed/merge_planner.h" +#include "distributed/multi_executor.h" +#include "distributed/multi_partitioning_utils.h" +#include "distributed/multi_router_planner.h" +#include "distributed/repartition_executor.h" +#include "distributed/subplan_execution.h" + +#include "nodes/execnodes.h" +#include "nodes/makefuncs.h" +#include "nodes/nodeFuncs.h" + +static void ExecuteSourceAtWorkerAndRepartition(CitusScanState *scanState); +static void ExecuteSourceAtCoordAndRedistribution(CitusScanState *scanState); +static HTAB * ExecuteMergeSourcePlanIntoColocatedIntermediateResults(Oid targetRelationId, + Query *mergeQuery, + List * + sourceTargetList, + PlannedStmt * + sourcePlan, + EState *executorState, + char * + intermediateResultIdPrefix, + int + partitionColumnIndex); + + +/* + * NonPushableMergeCommandExecScan performs an MERGE INTO distributed_table + * USING (source-query) ... command. This can be done either by aggregating + * task results at the coordinator and repartitioning the results, or by + * repartitioning task results and directly transferring data between nodes. + */ +TupleTableSlot * +NonPushableMergeCommandExecScan(CustomScanState *node) +{ + CitusScanState *scanState = (CitusScanState *) node; + DistributedPlan *distributedPlan = scanState->distributedPlan; + + if (!scanState->finishedRemoteScan) + { + switch (distributedPlan->modifyWithSelectMethod) + { + case MODIFY_WITH_SELECT_REPARTITION: + { + ExecuteSourceAtWorkerAndRepartition(scanState); + break; + } + + case MODIFY_WITH_SELECT_VIA_COORDINATOR: + { + ExecuteSourceAtCoordAndRedistribution(scanState); + break; + } + + default: + { + ereport(ERROR, (errmsg("Unexpected MERGE execution method(%d)", + distributedPlan->modifyWithSelectMethod))); + } + } + + scanState->finishedRemoteScan = true; + } + + TupleTableSlot *resultSlot = ReturnTupleFromTuplestore(scanState); + + return resultSlot; +} + + +/* + * ExecuteSourceAtWorkerAndRepartition Executes the Citus distributed plan, including any + * sub-plans, and captures the results in intermediate files. Subsequently, redistributes + * the result files to ensure colocation with the target, and directs the MERGE SQL + * operation to the target shards on the worker nodes, utilizing the colocated + * intermediate files as the data source. + */ +static void +ExecuteSourceAtWorkerAndRepartition(CitusScanState *scanState) +{ + DistributedPlan *distributedPlan = scanState->distributedPlan; + Query *mergeQuery = + copyObject(distributedPlan->modifyQueryViaCoordinatorOrRepartition); + RangeTblEntry *targetRte = ExtractResultRelationRTE(mergeQuery); + RangeTblEntry *sourceRte = ExtractMergeSourceRangeTableEntry(mergeQuery); + Oid targetRelationId = targetRte->relid; + bool hasReturning = distributedPlan->expectResults; + Query *sourceQuery = sourceRte->subquery; + PlannedStmt *sourcePlan = + copyObject(distributedPlan->selectPlanForModifyViaCoordinatorOrRepartition); + EState *executorState = ScanStateGetExecutorState(scanState); + + /* + * If we are dealing with partitioned table, we also need to lock its + * partitions. Here we only lock targetRelation, we acquire necessary + * locks on source tables during execution of those source queries. + */ + if (PartitionedTable(targetRelationId)) + { + LockPartitionRelations(targetRelationId, RowExclusiveLock); + } + + bool randomAccess = true; + bool interTransactions = false; + DistributedPlan *distSourcePlan = + GetDistributedPlan((CustomScan *) sourcePlan->planTree); + Job *distSourceJob = distSourcePlan->workerJob; + List *distSourceTaskList = distSourceJob->taskList; + bool binaryFormat = + CanUseBinaryCopyFormatForTargetList(sourceQuery->targetList); + + ereport(DEBUG1, (errmsg("Executing subplans of the source query and " + "storing the results at the respective node(s)"))); + + ExecuteSubPlans(distSourcePlan); + + /* + * We have a separate directory for each transaction, so choosing + * the same result prefix won't cause filename conflicts. Results + * directory name also includes node id and database id, so we don't + * need to include them in the filename. We include job id here for + * the case "MERGE USING " is executed recursively. + */ + StringInfo distResultPrefixString = makeStringInfo(); + appendStringInfo(distResultPrefixString, + "repartitioned_results_" UINT64_FORMAT, + distSourceJob->jobId); + char *distResultPrefix = distResultPrefixString->data; + CitusTableCacheEntry *targetRelation = GetCitusTableCacheEntry(targetRelationId); + + ereport(DEBUG1, (errmsg("Redistributing source result rows across nodes"))); + + /* + * partitionColumnIndex determines the column in the selectTaskList to + * use for (re)partitioning of the source result, which will colocate + * the result data with the target. + */ + int partitionColumnIndex = distributedPlan->sourceResultRepartitionColumnIndex; + + /* + * Below call partitions the results using shard ranges and partition method of + * targetRelation, and then colocates the result files with shards. These + * transfers are done by calls to fetch_intermediate_results() between nodes. + */ + List **redistributedResults = + RedistributeTaskListResults(distResultPrefix, + distSourceTaskList, partitionColumnIndex, + targetRelation, binaryFormat); + + ereport(DEBUG1, (errmsg("Executing final MERGE on workers using " + "intermediate results"))); + + /* + * At this point source query has been executed on workers and results + * have been fetched in such a way that they are colocated with corresponding + * target shard(s). Create and execute a list of tasks of form + * MERGE INTO ... USING SELECT * FROM read_intermediate_results(...); + */ + List *taskList = + GenerateTaskListWithRedistributedResults(mergeQuery, + targetRelation, + redistributedResults, + binaryFormat); + + scanState->tuplestorestate = + tuplestore_begin_heap(randomAccess, interTransactions, work_mem); + ParamListInfo paramListInfo = executorState->es_param_list_info; + TupleDesc tupleDescriptor = ScanStateGetTupleDescriptor(scanState); + TupleDestination *tupleDest = + CreateTupleStoreTupleDest(scanState->tuplestorestate, + tupleDescriptor); + uint64 rowsMerged = + ExecuteTaskListIntoTupleDestWithParam(ROW_MODIFY_NONCOMMUTATIVE, taskList, + tupleDest, + hasReturning, + paramListInfo); + executorState->es_processed = rowsMerged; +} + + +/* + * ExecuteSourceAtCoordAndRedistribution Executes the plan that necessitates evaluation + * at the coordinator and redistributes the resulting rows to intermediate files, + * ensuring colocation with the target shards. Directs the MERGE SQL operation to the + * target shards on the worker nodes, utilizing the colocated intermediate files as the + * data source. + */ +void +ExecuteSourceAtCoordAndRedistribution(CitusScanState *scanState) +{ + EState *executorState = ScanStateGetExecutorState(scanState); + DistributedPlan *distributedPlan = scanState->distributedPlan; + Query *mergeQuery = + copyObject(distributedPlan->modifyQueryViaCoordinatorOrRepartition); + RangeTblEntry *targetRte = ExtractResultRelationRTE(mergeQuery); + RangeTblEntry *sourceRte = ExtractMergeSourceRangeTableEntry(mergeQuery); + Query *sourceQuery = sourceRte->subquery; + Oid targetRelationId = targetRte->relid; + PlannedStmt *sourcePlan = + copyObject(distributedPlan->selectPlanForModifyViaCoordinatorOrRepartition); + char *intermediateResultIdPrefix = distributedPlan->intermediateResultIdPrefix; + bool hasReturning = distributedPlan->expectResults; + int partitionColumnIndex = distributedPlan->sourceResultRepartitionColumnIndex; + + /* + * If we are dealing with partitioned table, we also need to lock its + * partitions. Here we only lock targetRelation, we acquire necessary + * locks on source tables during execution of those source queries. + */ + if (PartitionedTable(targetRelationId)) + { + LockPartitionRelations(targetRelationId, RowExclusiveLock); + } + + ereport(DEBUG1, (errmsg("Collect source query results on coordinator"))); + + List *prunedTaskList = NIL; + HTAB *shardStateHash = + ExecuteMergeSourcePlanIntoColocatedIntermediateResults( + targetRelationId, + mergeQuery, + sourceQuery->targetList, + sourcePlan, + executorState, + intermediateResultIdPrefix, + partitionColumnIndex); + + ereport(DEBUG1, (errmsg("Create a MERGE task list that needs to be routed"))); + + /* generate tasks for the .. phase */ + List *taskList = + GenerateTaskListWithColocatedIntermediateResults(targetRelationId, mergeQuery, + intermediateResultIdPrefix); + + /* + * We cannot actually execute MERGE INTO ... tasks that read from + * intermediate results that weren't created because no rows were + * written to them. Prune those tasks out by only including tasks + * on shards with connections. + */ + Task *task = NULL; + foreach_ptr(task, taskList) + { + uint64 shardId = task->anchorShardId; + bool shardModified = false; + + hash_search(shardStateHash, &shardId, HASH_FIND, &shardModified); + if (shardModified) + { + prunedTaskList = lappend(prunedTaskList, task); + } + } + + if (prunedTaskList == NIL) + { + /* No task to execute */ + return; + } + + ereport(DEBUG1, (errmsg("Execute MERGE task list"))); + bool randomAccess = true; + bool interTransactions = false; + Assert(scanState->tuplestorestate == NULL); + scanState->tuplestorestate = tuplestore_begin_heap(randomAccess, interTransactions, + work_mem); + TupleDesc tupleDescriptor = ScanStateGetTupleDescriptor(scanState); + ParamListInfo paramListInfo = executorState->es_param_list_info; + TupleDestination *tupleDest = + CreateTupleStoreTupleDest(scanState->tuplestorestate, tupleDescriptor); + uint64 rowsMerged = + ExecuteTaskListIntoTupleDestWithParam(ROW_MODIFY_NONCOMMUTATIVE, + prunedTaskList, + tupleDest, + hasReturning, + paramListInfo); + executorState->es_processed = rowsMerged; +} + + +/* + * ExecuteMergeSourcePlanIntoColocatedIntermediateResults Executes the given PlannedStmt + * and inserts tuples into a set of intermediate results that are colocated with the + * target table for further processing MERGE INTO. It also returns the hash of shard + * states that were used to insert tuplesinto the target relation. + */ +static HTAB * +ExecuteMergeSourcePlanIntoColocatedIntermediateResults(Oid targetRelationId, + Query *mergeQuery, + List *sourceTargetList, + PlannedStmt *sourcePlan, + EState *executorState, + char *intermediateResultIdPrefix, + int partitionColumnIndex) +{ + ParamListInfo paramListInfo = executorState->es_param_list_info; + + /* Get column name list and partition column index for the target table */ + List *columnNameList = + BuildColumnNameListFromTargetList(targetRelationId, sourceTargetList); + + /* set up a DestReceiver that copies into the intermediate file */ + const bool publishableData = false; + CitusCopyDestReceiver *copyDest = CreateCitusCopyDestReceiver(targetRelationId, + columnNameList, + partitionColumnIndex, + executorState, + intermediateResultIdPrefix, + publishableData); + + /* We can skip when writing to intermediate files */ + copyDest->skipCoercions = true; + + ExecutePlanIntoDestReceiver(sourcePlan, paramListInfo, (DestReceiver *) copyDest); + + executorState->es_processed = copyDest->tuplesSent; + XactModificationLevel = XACT_MODIFICATION_DATA; + + return copyDest->shardStateHash; +} diff --git a/src/backend/distributed/executor/multi_server_executor.c b/src/backend/distributed/executor/multi_server_executor.c index 07f3d6856..ac144c350 100644 --- a/src/backend/distributed/executor/multi_server_executor.c +++ b/src/backend/distributed/executor/multi_server_executor.c @@ -24,6 +24,7 @@ #include "distributed/multi_executor.h" #include "distributed/multi_physical_planner.h" #include "distributed/multi_server_executor.h" +#include "distributed/multi_router_planner.h" #include "distributed/coordinator_protocol.h" #include "distributed/subplan_execution.h" #include "distributed/tuple_destination.h" @@ -49,6 +50,11 @@ JobExecutorType(DistributedPlan *distributedPlan) if (distributedPlan->modifyQueryViaCoordinatorOrRepartition != NULL) { + if (IsMergeQuery(distributedPlan->modifyQueryViaCoordinatorOrRepartition)) + { + return MULTI_EXECUTOR_NON_PUSHABLE_MERGE_QUERY; + } + /* * We go through * MULTI_EXECUTOR_NON_PUSHABLE_INSERT_SELECT because diff --git a/src/backend/distributed/executor/repartition_executor.c b/src/backend/distributed/executor/repartition_executor.c index b35527b99..af4f0ac7e 100644 --- a/src/backend/distributed/executor/repartition_executor.c +++ b/src/backend/distributed/executor/repartition_executor.c @@ -120,7 +120,7 @@ GenerateTaskListWithColocatedIntermediateResults(Oid targetRelationId, */ Query *modifyWithResultQuery = copyObject(modifyQueryViaCoordinatorOrRepartition); RangeTblEntry *insertRte = ExtractResultRelationRTE(modifyWithResultQuery); - RangeTblEntry *selectRte = ExtractSelectRangeTableEntry(modifyWithResultQuery); + RangeTblEntry *selectRte = ExtractSourceResultRangeTableEntry(modifyWithResultQuery); CitusTableCacheEntry *targetCacheEntry = GetCitusTableCacheEntry(targetRelationId); int shardCount = targetCacheEntry->shardIntervalArrayLength; @@ -139,11 +139,18 @@ GenerateTaskListWithColocatedIntermediateResults(Oid targetRelationId, /* during COPY, the shard ID is appended to the result name */ appendStringInfo(resultId, "%s_" UINT64_FORMAT, resultIdPrefix, shardId); + /* + * For MERGE SQL, use the USING clause list, the main query target list + * is NULL + */ + List *targetList = IsMergeQuery(modifyQueryViaCoordinatorOrRepartition) ? + selectRte->subquery->targetList : + modifyQueryViaCoordinatorOrRepartition->targetList; + /* generate the query on the intermediate result */ - Query *resultSelectQuery = BuildSubPlanResultQuery( - modifyQueryViaCoordinatorOrRepartition->targetList, - columnAliasList, - resultId->data); + Query *resultSelectQuery = BuildSubPlanResultQuery(targetList, + columnAliasList, + resultId->data); /* put the intermediate result query in the INSERT..SELECT */ selectRte->subquery = resultSelectQuery; @@ -214,8 +221,6 @@ GenerateTaskListWithRedistributedResults(Query *modifyQueryViaCoordinatorOrRepar */ Query *modifyResultQuery = copyObject(modifyQueryViaCoordinatorOrRepartition); RangeTblEntry *insertRte = ExtractResultRelationRTE(modifyResultQuery); - RangeTblEntry *selectRte = ExtractSelectRangeTableEntry(modifyResultQuery); - List *selectTargetList = selectRte->subquery->targetList; Oid targetRelationId = targetRelation->relationId; int shardCount = targetRelation->shardIntervalArrayLength; @@ -223,6 +228,10 @@ GenerateTaskListWithRedistributedResults(Query *modifyQueryViaCoordinatorOrRepar uint32 taskIdIndex = 1; uint64 jobId = INVALID_JOB_ID; + RangeTblEntry *selectRte = + ExtractSourceResultRangeTableEntry(modifyResultQuery); + List *selectTargetList = selectRte->subquery->targetList; + for (shardOffset = 0; shardOffset < shardCount; shardOffset++) { ShardInterval *targetShardInterval = diff --git a/src/backend/distributed/planner/distributed_planner.c b/src/backend/distributed/planner/distributed_planner.c index e53259b77..3b6a8f9f7 100644 --- a/src/backend/distributed/planner/distributed_planner.c +++ b/src/backend/distributed/planner/distributed_planner.c @@ -925,6 +925,10 @@ GetRouterPlanType(Query *query, Query *originalQuery, bool hasUnresolvedParams) } else if (IsMergeQuery(originalQuery)) { + if (hasUnresolvedParams) + { + return REPLAN_WITH_BOUND_PARAMETERS; + } return MERGE_QUERY; } else @@ -990,7 +994,8 @@ CreateDistributedPlan(uint64 planId, bool allowRecursivePlanning, Query *origina case MERGE_QUERY: { distributedPlan = - CreateMergePlan(originalQuery, query, plannerRestrictionContext); + CreateMergePlan(planId, originalQuery, query, plannerRestrictionContext, + boundParams); break; } @@ -1377,6 +1382,12 @@ FinalizePlan(PlannedStmt *localPlan, DistributedPlan *distributedPlan) break; } + case MULTI_EXECUTOR_NON_PUSHABLE_MERGE_QUERY: + { + customScan->methods = &NonPushableMergeCommandCustomScanMethods; + break; + } + default: { customScan->methods = &DelayedErrorCustomScanMethods; diff --git a/src/backend/distributed/planner/insert_select_planner.c b/src/backend/distributed/planner/insert_select_planner.c index a44db5c28..84e76c6d4 100644 --- a/src/backend/distributed/planner/insert_select_planner.c +++ b/src/backend/distributed/planner/insert_select_planner.c @@ -86,7 +86,6 @@ static DeferredErrorMessage * InsertPartitionColumnMatchesSelect(Query *query, static DistributedPlan * CreateNonPushableInsertSelectPlan(uint64 planId, Query *parse, ParamListInfo boundParams); static DeferredErrorMessage * NonPushableInsertSelectSupported(Query *insertSelectQuery); -static Query * WrapSubquery(Query *subquery); static void RelabelTargetEntryList(List *selectTargetList, List *insertTargetList); static List * AddInsertSelectCasts(List *insertTargetList, List *selectTargetList, Oid targetRelationId); @@ -1477,7 +1476,7 @@ InsertSelectResultIdPrefix(uint64 planId) * WrapSubquery wraps the given query as a subquery in a newly constructed * "SELECT * FROM (...subquery...) citus_insert_select_subquery" query. */ -static Query * +Query * WrapSubquery(Query *subquery) { ParseState *pstate = make_parsestate(NULL); diff --git a/src/backend/distributed/planner/local_distributed_join_planner.c b/src/backend/distributed/planner/local_distributed_join_planner.c index 449b96195..2c6a63de1 100644 --- a/src/backend/distributed/planner/local_distributed_join_planner.c +++ b/src/backend/distributed/planner/local_distributed_join_planner.c @@ -485,6 +485,8 @@ RequiredAttrNumbersForRelation(RangeTblEntry *rangeTableEntry, PlannerInfo *plannerInfo = relationRestriction->plannerInfo; + int rteIndex = relationRestriction->index; + /* * Here we used the query from plannerInfo because it has the optimizations * so that it doesn't have unnecessary columns. The original query doesn't have @@ -492,8 +494,18 @@ RequiredAttrNumbersForRelation(RangeTblEntry *rangeTableEntry, * 'required' attributes. */ Query *queryToProcess = plannerInfo->parse; - int rteIndex = relationRestriction->index; + return RequiredAttrNumbersForRelationInternal(queryToProcess, rteIndex); +} + + +/* + * RequiredAttrNumbersForRelationInternal returns the required attribute numbers + * for the input range-table-index in the query parameter. + */ +List * +RequiredAttrNumbersForRelationInternal(Query *queryToProcess, int rteIndex) +{ List *allVarsInQuery = pull_vars_of_level((Node *) queryToProcess, 0); List *requiredAttrNumbers = NIL; diff --git a/src/backend/distributed/planner/merge_planner.c b/src/backend/distributed/planner/merge_planner.c index 86163e131..13d0b84d6 100644 --- a/src/backend/distributed/planner/merge_planner.c +++ b/src/backend/distributed/planner/merge_planner.c @@ -13,92 +13,350 @@ #include "postgres.h" #include "nodes/makefuncs.h" +#include "nodes/nodeFuncs.h" #include "optimizer/optimizer.h" #include "parser/parsetree.h" +#include "tcop/tcopprot.h" #include "utils/lsyscache.h" #include "distributed/citus_clauses.h" +#include "distributed/citus_custom_scan.h" +#include "distributed/insert_select_planner.h" #include "distributed/listutils.h" +#include "distributed/local_distributed_join_planner.h" #include "distributed/merge_planner.h" #include "distributed/multi_logical_optimizer.h" #include "distributed/multi_router_planner.h" #include "distributed/pg_dist_node_metadata.h" #include "distributed/pg_version_constants.h" #include "distributed/query_pushdown_planning.h" +#include "distributed/query_colocation_checker.h" +#include "distributed/repartition_executor.h" +#include "distributed/shared_library_init.h" #if PG_VERSION_NUM >= PG_VERSION_15 -static DeferredErrorMessage * ErrorIfDistTablesNotColocated(Query *parse, - List * - distTablesList, - PlannerRestrictionContext - * - plannerRestrictionContext); -static DeferredErrorMessage * ErrorIfMergeHasUnsupportedTables(Oid targetRelationId, - Query *parse, - List *rangeTableList, - PlannerRestrictionContext * - restrictionContext); +static int SourceResultPartitionColumnIndex(Query *mergeQuery, + List *sourceTargetList, + CitusTableCacheEntry *targetRelation); +static Var * ValidateAndReturnVarIfSupported(Node *entryExpr); +static void ErrorIfMergeQueryQualAndTargetListNotSupported(Oid targetRelationId, + Query *originalQuery); +static void ErrorIfMergeNotSupported(Query *query, Oid targetRelationId, + List *rangeTableList); +static void ErrorIfMergeHasUnsupportedTables(Oid targetRelationId, List *rangeTableList); static bool IsDistributionColumnInMergeSource(Expr *columnExpression, Query *query, bool skipOuterVars); -static DeferredErrorMessage * InsertDistributionColumnMatchesSource(Oid targetRelationId, - Query *query); +static DeferredErrorMessage * DeferErrorIfRoutableMergeNotSupported(Query *query, + List *rangeTableList, + PlannerRestrictionContext + * + plannerRestrictionContext); static DeferredErrorMessage * MergeQualAndTargetListFunctionsSupported(Oid resultRelationId, - FromExpr *joinTree, + Query *query, Node *quals, List *targetList, CmdType commandType); +static DistributedPlan * CreateRouterMergePlan(Oid targetRelationId, Query *originalQuery, + Query *query, + List *rangeTableList, + PlannerRestrictionContext * + plannerRestrictionContext); +static void ErrorIfRepartitionMergeNotSupported(Oid targetRelationId, Query *mergeQuery, + Query *sourceQuery); +static void ConvertSourceRTEIntoSubquery(Query *mergeQuery, + RangeTblEntry *sourceRte, + PlannerRestrictionContext * + plannerRestrictionContext); +static void ConvertSubqueryRTEIntoSubquery(Query *mergeQuery, RangeTblEntry *sourceRte); +static void ConvertCteRTEIntoSubquery(Query *mergeQuery, RangeTblEntry *sourceRte); +static void ConvertRelationRTEIntoSubquery(Query *mergeQuery, + RangeTblEntry *sourceRte, + PlannerRestrictionContext * + plannerRestrictionContext); +static void ErrorIfUnsupportedCTEs(Query *query); +static void ContainsUnsupportedCTEs(Query *query); +static bool MergeQueryCTEWalker(Node *node, void *context); +static DistributedPlan * CreateNonPushableMergePlan(Oid targetRelationId, uint64 planId, + Query *originalQuery, + Query *query, + PlannerRestrictionContext * + plannerRestrictionContext, + ParamListInfo boundParams); +static char * MergeCommandResultIdPrefix(uint64 planId); + +#endif + /* - * ErrorIfDistTablesNotColocated Checks to see if - * - * - There are a minimum of two distributed tables (source and a target). - * - All the distributed tables are indeed colocated. - * - * If any of the conditions are not met, it raises an exception. + * CreateMergePlan + * 1) Check for conditions that are not supported in MERGE command. + * 2) Try to create a pushable plan + * - Check for conditions suitable for a routable plan, if not found, + * raise deferred error + * 3) Try to create repartition and redistribution plan + * - Check for conditions that prevent repartition strategy, if found, + * raise an exception and quit. */ -static DeferredErrorMessage * -ErrorIfDistTablesNotColocated(Query *parse, List *distTablesList, - PlannerRestrictionContext * - plannerRestrictionContext) +DistributedPlan * +CreateMergePlan(uint64 planId, Query *originalQuery, Query *query, + PlannerRestrictionContext *plannerRestrictionContext, + ParamListInfo boundParams) { - /* All MERGE tables must be distributed */ - if (list_length(distTablesList) < 2) + /* function is void for pre-15 versions of Postgres */ + #if PG_VERSION_NUM < PG_VERSION_15 + + ereport(ERROR, (errmsg("MERGE is not supported in pre-15 Postgres versions"))); + + #else + + Oid targetRelationId = ModifyQueryResultRelationId(originalQuery); + + /* + * Step 1: Look for definitive error conditions applicable to both Routable + * and Repartition strategies. + */ + List *rangeTableList = ExtractRangeTableEntryList(originalQuery); + ErrorIfMergeNotSupported(originalQuery, targetRelationId, rangeTableList); + + /* Step 2: Try pushable merge plan */ + DistributedPlan *distributedPlan = + CreateRouterMergePlan(targetRelationId, originalQuery, query, + rangeTableList, plannerRestrictionContext); + + /* Step 3: If the routing plan failed, try for repartition strategy */ + if (distributedPlan->planningError != NULL) { - return DeferredError(ERRCODE_FEATURE_NOT_SUPPORTED, - "For MERGE command, both the source and target " - "must be distributed", NULL, NULL); + RaiseDeferredError(distributedPlan->planningError, DEBUG1); + + /* If MERGE is not routable, try repartitioning */ + distributedPlan = + CreateNonPushableMergePlan(targetRelationId, planId, + originalQuery, query, + plannerRestrictionContext, + boundParams); } - /* All distributed tables must be colocated */ - if (!AllDistributedRelationsInRTEListColocated(distTablesList)) + return distributedPlan; + + #endif +} + + +#if PG_VERSION_NUM >= PG_VERSION_15 + +/* + * CreateRouterMergePlan attempts to create a pushable plan for the given MERGE + * SQL statement. If the planning fails, the ->planningError is set to a description + * of the failure. + */ +static DistributedPlan * +CreateRouterMergePlan(Oid targetRelationId, Query *originalQuery, Query *query, + List *rangeTableList, + PlannerRestrictionContext *plannerRestrictionContext) +{ + DistributedPlan *distributedPlan = CitusMakeNode(DistributedPlan); + + Assert(originalQuery->commandType == CMD_MERGE); + Assert(OidIsValid(targetRelationId)); + + distributedPlan->planningError = DeferErrorIfRoutableMergeNotSupported(originalQuery, + rangeTableList, + plannerRestrictionContext); + if (distributedPlan->planningError != NULL) { - return DeferredError(ERRCODE_FEATURE_NOT_SUPPORTED, - "For MERGE command, all the distributed tables " - "must be colocated", NULL, NULL); + return distributedPlan; } - return NULL; + Var *insertVar = + FetchAndValidateInsertVarIfExists(targetRelationId, originalQuery); + if (insertVar && + !IsDistributionColumnInMergeSource((Expr *) insertVar, originalQuery, true)) + { + ereport(ERROR, (errmsg("MERGE INSERT must use the source table " + "distribution column value"))); + } + + Job *job = RouterJob(originalQuery, plannerRestrictionContext, + &distributedPlan->planningError); + + if (distributedPlan->planningError != NULL) + { + return distributedPlan; + } + + ereport(DEBUG1, (errmsg("Creating MERGE router plan"))); + + distributedPlan->workerJob = job; + distributedPlan->targetRelationId = targetRelationId; + distributedPlan->modLevel = RowModifyLevelForQuery(query); + + /* There is no coordinator query for MERGE */ + distributedPlan->combineQuery = NULL; + + /* MERGE doesn't support RETURNING clause */ + distributedPlan->expectResults = false; + distributedPlan->fastPathRouterPlan = + plannerRestrictionContext->fastPathRestrictionContext->fastPathRouterQuery; + + return distributedPlan; +} + + +/* + * CreateNonPushableMergePlan comes into effect if the router planning fails + * and incorporates two planning strategies. + * + * ExecuteSourceAtWorkerAndRepartition(): Plan the source query independently, + * execute the results into intermediate files, and repartition the files to + * co-locate them with the merge-target table. Subsequently, compile a final + * merge query on the target table using the intermediate results as the data + * source. + * + * ExecuteSourceAtCoordAndRedistribution(): Execute the plan that requires + * evaluation at the coordinator, run the query on the coordinator, and + * redistribute the resulting rows to ensure colocation with the target shards. + * Direct the MERGE SQL operation to the worker nodes' target shards, using the + * intermediate files colocated with the data as the data source. + */ +static DistributedPlan * +CreateNonPushableMergePlan(Oid targetRelationId, uint64 planId, Query *originalQuery, + Query *query, + PlannerRestrictionContext *plannerRestrictionContext, + ParamListInfo boundParams) +{ + Query *mergeQuery = copyObject(originalQuery); + RangeTblEntry *sourceRte = ExtractMergeSourceRangeTableEntry(mergeQuery); + DistributedPlan *distributedPlan = CitusMakeNode(DistributedPlan); + + ereport(DEBUG1, (errmsg("Creating MERGE repartition plan"))); + ConvertSourceRTEIntoSubquery(mergeQuery, sourceRte, plannerRestrictionContext); + Query *sourceQuery = sourceRte->subquery; + + ErrorIfRepartitionMergeNotSupported(targetRelationId, mergeQuery, sourceQuery); + + CitusTableCacheEntry *targetRelation = GetCitusTableCacheEntry(targetRelationId); + + /* + * Get the index of the column in the source query that will be utilized + * to repartition the source rows, ensuring colocation with the target + */ + distributedPlan->sourceResultRepartitionColumnIndex = + SourceResultPartitionColumnIndex(mergeQuery, + sourceQuery->targetList, + targetRelation); + + /* + * Make a copy of the source query, since following code scribbles it + * but we need to keep the original for EXPLAIN. + */ + Query *sourceQueryCopy = copyObject(sourceQuery); + + /* plan the subquery, this may be another distributed query */ + int cursorOptions = CURSOR_OPT_PARALLEL_OK; + PlannedStmt *sourceRowsPlan = pg_plan_query(sourceQueryCopy, NULL, cursorOptions, + boundParams); + bool repartitioned = IsRedistributablePlan(sourceRowsPlan->planTree) && + IsSupportedRedistributionTarget(targetRelationId); + + /* If plan is distributed, no work at the coordinator */ + if (repartitioned) + { + distributedPlan->modifyWithSelectMethod = MODIFY_WITH_SELECT_REPARTITION; + } + else + { + distributedPlan->modifyWithSelectMethod = MODIFY_WITH_SELECT_VIA_COORDINATOR; + } + + /* There is no coordinator query for MERGE */ + distributedPlan->combineQuery = NULL; + + /* MERGE doesn't support RETURNING clause */ + distributedPlan->expectResults = false; + + distributedPlan->modLevel = RowModifyLevelForQuery(mergeQuery); + distributedPlan->targetRelationId = targetRelationId; + distributedPlan->intermediateResultIdPrefix = MergeCommandResultIdPrefix(planId); + distributedPlan->modifyQueryViaCoordinatorOrRepartition = mergeQuery; + distributedPlan->selectPlanForModifyViaCoordinatorOrRepartition = sourceRowsPlan; + distributedPlan->fastPathRouterPlan = + plannerRestrictionContext->fastPathRestrictionContext->fastPathRouterQuery; + + return distributedPlan; +} + + +/* + * ContainsUnsupportedCTEs checks the CTE if it's modifying or recursive CTE, if true, + * raises an exception. + */ +static void +ContainsUnsupportedCTEs(Query *query) +{ + if (query->hasModifyingCTE) + { + ereport(ERROR, (errmsg("CTEs with modifying actions are not yet " + "supported in MERGE"))); + } + + if (query->hasRecursive) + { + ereport(ERROR, (errmsg("Recursive CTEs are not yet " + "supported in MERGE"))); + } +} + + +/* + * MergeQueryCTEWalker descends into the MERGE query to check for any subqueries + */ +static bool +MergeQueryCTEWalker(Node *node, void *context) +{ + if (node == NULL) + { + return false; + } + + if (IsA(node, Query)) + { + Query *query = (Query *) node; + + ContainsUnsupportedCTEs(query); + + query_tree_walker(query, MergeQueryCTEWalker, NULL, 0); + + /* we're done, no need to recurse anymore for this query */ + return false; + } + + return expression_tree_walker(node, MergeQueryCTEWalker, context); +} + + +/* + * ErrorIfUnsupportedCTEs checks for unsupported CTEs, such as, modifying and recursive + */ +static void +ErrorIfUnsupportedCTEs(Query *query) +{ + ContainsUnsupportedCTEs(query); + query_tree_walker(query, MergeQueryCTEWalker, NULL, 0); } /* * ErrorIfMergeHasUnsupportedTables checks if all the tables(target, source or any CTE * present) in the MERGE command are local i.e. a combination of Citus local and Non-Citus - * tables (regular Postgres tables), or distributed tables with some restrictions, please - * see header of routine ErrorIfDistTablesNotColocated for details, raises an exception - * for all other combinations. + * tables (regular Postgres tables), or distributed tables with some restrictions + * raises an exception for all other combinations. */ -static DeferredErrorMessage * -ErrorIfMergeHasUnsupportedTables(Oid targetRelationId, Query *parse, List *rangeTableList, - PlannerRestrictionContext *restrictionContext) +static void +ErrorIfMergeHasUnsupportedTables(Oid targetRelationId, List *rangeTableList) { - List *distTablesList = NIL; - bool foundLocalTables = false; - bool foundReferenceTables = false; - RangeTblEntry *rangeTableEntry = NULL; foreach_ptr(rangeTableEntry, rangeTableList) { @@ -133,17 +391,18 @@ ErrorIfMergeHasUnsupportedTables(Oid targetRelationId, Query *parse, List *range case RTE_NAMEDTUPLESTORE: case RTE_RESULT: { - return DeferredError(ERRCODE_FEATURE_NOT_SUPPORTED, - "MERGE command is not supported with " - "Tuplestores and results", - NULL, NULL); + ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED), + errmsg("MERGE command is not supported with " + "Tuplestores and results"))); + break; } default: { - return DeferredError(ERRCODE_FEATURE_NOT_SUPPORTED, - "MERGE command: Unrecognized range table entry.", - NULL, NULL); + ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED), + errmsg( + "MERGE command: Unrecognized range table entry(%d) ", + rangeTableEntry->rtekind))); } } @@ -163,12 +422,10 @@ ErrorIfMergeHasUnsupportedTables(Oid targetRelationId, Query *parse, List *range if (relationId == targetRelationId) { /* Usually we don't reach this exception as the Postgres parser catches it */ - StringInfo errorMessage = makeStringInfo(); - appendStringInfo(errorMessage, "MERGE command is not allowed on " - "relation type(relkind:%c)", - rangeTableEntry->relkind); - return DeferredError(ERRCODE_FEATURE_NOT_SUPPORTED, - errorMessage->data, NULL, NULL); + ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED), + errmsg("MERGE command is not allowed on " + "relation type(relkind:%c)", + rangeTableEntry->relkind))); } break; } @@ -183,100 +440,58 @@ ErrorIfMergeHasUnsupportedTables(Oid targetRelationId, Query *parse, List *range default: { - StringInfo errorMessage = makeStringInfo(); - appendStringInfo(errorMessage, "Unexpected table type(relkind:%c) " - "in MERGE command", - rangeTableEntry->relkind); - return DeferredError(ERRCODE_FEATURE_NOT_SUPPORTED, - errorMessage->data, NULL, NULL); + ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED), + errmsg("Unexpected table type(relkind:%c) " + "in MERGE command", rangeTableEntry->relkind))); } } /* - * For now, save all distributed tables, later (below) we will - * check for supported combination(s). + * Check for unsupported distributed tables */ - if (IsCitusTableType(relationId, DISTRIBUTED_TABLE)) + if (extern_IsColumnarTableAmTable(relationId) && + relationId == targetRelationId) + { + /* Columnar tables are not supported */ + ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED), + errmsg("Columnar table as target is " + "not allowed in MERGE command"))); + } + else if (IsCitusTableType(relationId, DISTRIBUTED_TABLE)) { /* Append/Range distributed tables are not supported */ if (IsCitusTableType(relationId, APPEND_DISTRIBUTED) || IsCitusTableType(relationId, RANGE_DISTRIBUTED)) { - return DeferredError(ERRCODE_FEATURE_NOT_SUPPORTED, - "For MERGE command, all the distributed tables " - "must be colocated, for append/range distribution, " - "colocation is not supported", NULL, - "Consider using hash distribution instead"); + ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED), + errmsg("For MERGE command, append/range distribution " + "table is not supported yet"))); } - - distTablesList = lappend(distTablesList, rangeTableEntry); } - else if (IsCitusTableType(relationId, REFERENCE_TABLE)) + else if (IsCitusTableType(relationId, REFERENCE_TABLE) && + relationId == targetRelationId) { /* Reference table as a target is not allowed */ - if (relationId == targetRelationId) - { - return DeferredError(ERRCODE_FEATURE_NOT_SUPPORTED, - "Reference table as target " - "is not allowed in " - "MERGE command", NULL, NULL); - } - - foundReferenceTables = true; + ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED), + errmsg("Reference table as target is " + "not allowed in MERGE command"))); } else if (IsCitusTableType(relationId, CITUS_LOCAL_TABLE)) { - /* Citus local tables */ - foundLocalTables = true; + /* + * All the tables are local/reference, supported as long as + * coordinator is in the metadata. + */ + if (FindCoordinatorNodeId() == -1) + { + ereport(ERROR, (errmsg("Coordinator node is not in " + "the metadata"), + errhint("To ensure that the distributed planner " + "planner the Citus table, please consider " + "configuring a coordinator node"))); + } } - else if (!IsCitusTable(relationId)) - { - /* Regular Postgres table */ - foundLocalTables = true; - } - - /* Any other Citus table type missing ? */ } - - /* Ensure all tables are indeed local (or a combination of reference and local) */ - if (list_length(distTablesList) == 0) - { - /* - * All the tables are local/reference, supported as long as - * coordinator is in the metadata. - */ - if (FindCoordinatorNodeId() == -1) - { - elog(ERROR, "Coordinator node is not in the metadata. TODO better meesage"); - } - - /* All the tables are local/reference, supported */ - return NULL; - } - - if (foundLocalTables) - { - return DeferredError(ERRCODE_FEATURE_NOT_SUPPORTED, - "MERGE command is not supported with " - "combination of distributed/local tables yet", - NULL, NULL); - } - - if (foundReferenceTables) - { - return DeferredError(ERRCODE_FEATURE_NOT_SUPPORTED, - "MERGE command is not supported with " - "combination of distributed/reference yet", - NULL, - "If target is distributed, source " - "must be distributed and co-located"); - } - - - /* Ensure all distributed tables are indeed co-located */ - return ErrorIfDistTablesNotColocated(parse, - distTablesList, - restrictionContext); } @@ -320,118 +535,21 @@ IsDistributionColumnInMergeSource(Expr *columnExpression, Query *query, bool } -/* - * InsertDistributionColumnMatchesSource check to see if MERGE is inserting a - * value into the target which is not from the source table, if so, it - * raises an exception. - * Note: Inserting random values other than the joined column values will - * result in unexpected behaviour of rows ending up in incorrect shards, to - * prevent such mishaps, we disallow such inserts here. - */ -static DeferredErrorMessage * -InsertDistributionColumnMatchesSource(Oid targetRelationId, Query *query) -{ - Assert(IsMergeQuery(query)); - - if (!IsCitusTableType(targetRelationId, DISTRIBUTED_TABLE)) - { - return NULL; - } - - if (!HasDistributionKey(targetRelationId)) - { - return NULL; - } - - bool foundDistributionColumn = false; - MergeAction *action = NULL; - foreach_ptr(action, query->mergeActionList) - { - /* Skip MATCHED clause as INSERTS are not allowed in it*/ - if (action->matched) - { - continue; - } - - /* NOT MATCHED can have either INSERT or DO NOTHING */ - if (action->commandType == CMD_NOTHING) - { - return NULL; - } - - if (action->targetList == NIL) - { - /* INSERT DEFAULT VALUES is not allowed */ - return DeferredError(ERRCODE_FEATURE_NOT_SUPPORTED, - "cannot perform MERGE INSERT with DEFAULTS", - NULL, NULL); - } - - Assert(action->commandType == CMD_INSERT); - Var *targetKey = PartitionColumn(targetRelationId, 1); - - TargetEntry *targetEntry = NULL; - foreach_ptr(targetEntry, action->targetList) - { - AttrNumber originalAttrNo = targetEntry->resno; - - /* skip processing of target table non-partition columns */ - if (originalAttrNo != targetKey->varattno) - { - continue; - } - - foundDistributionColumn = true; - - if (IsA(targetEntry->expr, Var)) - { - if (IsDistributionColumnInMergeSource(targetEntry->expr, query, true)) - { - return NULL; - } - else - { - return DeferredError(ERRCODE_FEATURE_NOT_SUPPORTED, - "MERGE INSERT must use the source table " - "distribution column value", - NULL, NULL); - } - } - else - { - return DeferredError(ERRCODE_FEATURE_NOT_SUPPORTED, - "MERGE INSERT must refer a source column " - "for distribution column ", - NULL, NULL); - } - } - - if (!foundDistributionColumn) - { - return DeferredError(ERRCODE_FEATURE_NOT_SUPPORTED, - "MERGE INSERT must have distribution column as value", - NULL, NULL); - } - } - - return NULL; -} - - /* * MergeQualAndTargetListFunctionsSupported Checks WHEN/ON clause actions to see what functions * are allowed, if we are updating distribution column, etc. */ static DeferredErrorMessage * -MergeQualAndTargetListFunctionsSupported(Oid resultRelationId, FromExpr *joinTree, +MergeQualAndTargetListFunctionsSupported(Oid resultRelationId, Query *query, Node *quals, List *targetList, CmdType commandType) { - uint32 rangeTableId = 1; + uint32 targetRangeTableIndex = query->resultRelation; + FromExpr *joinTree = query->jointree; Var *distributionColumn = NULL; if (IsCitusTable(resultRelationId) && HasDistributionKey(resultRelationId)) { - distributionColumn = PartitionColumn(resultRelationId, rangeTableId); + distributionColumn = PartitionColumn(resultRelationId, targetRangeTableIndex); } ListCell *targetEntryCell = NULL; @@ -554,27 +672,367 @@ MergeQualAndTargetListFunctionsSupported(Oid resultRelationId, FromExpr *joinTre } -#endif +/* + * RepartitionMergeSupported checks if certain conditions cannot accommodate the + * strategy of repartition and redistribution of source rows, the routine will verify + * them and subsequently raises an exception. + */ +static void +ErrorIfRepartitionMergeNotSupported(Oid targetRelationId, Query *mergeQuery, + Query *sourceQuery) +{ + if (!IsCitusTableType(targetRelationId, DISTRIBUTED_TABLE)) + { + ereport(ERROR, + (errmsg("MERGE involving repartition of rows " + "is supported only if the target is distributed"))); + } + + RTEListProperties *queryRteListProperties = GetRTEListPropertiesForQuery(mergeQuery); + if (queryRteListProperties->hasPostgresLocalTable) + { + ereport(ERROR, (errmsg("MERGE INTO an distributed table from " + "Postgres table is not yet supported"))); + } + + queryRteListProperties = GetRTEListPropertiesForQuery(sourceQuery); + if (!queryRteListProperties->hasCitusTable) + { + ereport(ERROR, (errmsg("To MERGE into a distributed table, source must " + "be Citus table(s)"))); + } + + /* + * Sub-queries and CTEs are not allowed in actions and ON clause + */ + if (FindNodeMatchingCheckFunction((Node *) mergeQuery->jointree->quals, + IsNodeSubquery)) + { + ereport(ERROR, + (errmsg("Sub-queries and CTEs are not allowed in ON clause for MERGE " + "with repartitioning"), + errhint("Consider making the source and target colocated " + "and joined on the distribution column to make it a " + "routable query"))); + } + + MergeAction *action = NULL; + foreach_ptr(action, mergeQuery->mergeActionList) + { + if (FindNodeMatchingCheckFunction((Node *) action, IsNodeSubquery)) + { + ereport(ERROR, + (errmsg("Sub-queries and CTEs are not allowed in actions for MERGE " + "with repartitioning"), + errhint("Consider making the source and target colocated " + "and joined on the distribution column to make it a " + "routable query"))); + } + } +} /* - * MergeQuerySupported does check for a MERGE command in the query, if it finds - * one, it will verify the below criteria - * - Supported tables and combinations in ErrorIfMergeHasUnsupportedTables - * - Distributed tables requirements in ErrorIfDistTablesNotColocated - * - Checks target-lists and functions-in-quals in TargetlistAndFunctionsSupported + * ConvertCteRTEIntoSubquery takes a RTE_CTE and converts it into a RTE_SUBQUERY. */ -DeferredErrorMessage * -MergeQuerySupported(Oid resultRelationId, Query *originalQuery, bool multiShardQuery, - PlannerRestrictionContext *plannerRestrictionContext) +static void +ConvertCteRTEIntoSubquery(Query *mergeQuery, RangeTblEntry *sourceRte) { - /* function is void for pre-15 versions of Postgres */ - #if PG_VERSION_NUM < PG_VERSION_15 + CommonTableExpr *sourceCte = NULL; + CommonTableExpr *candidateCte = NULL; + List *cteList = NIL; + + /* + * Presently, CTEs are only permitted within the USING clause, and thus, + * we search for the corresponding one + */ + foreach_ptr(candidateCte, mergeQuery->cteList) + { + if (strcmp(candidateCte->ctename, sourceRte->ctename) == 0) + { + /* The source CTE that will be converted to a subquery */ + sourceCte = candidateCte; + } + else + { + /* + * Save any other CTEs that are referenced, either directly + * or indirectly, in the source CTE. + */ + cteList = lappend(cteList, candidateCte); + } + } + + Assert(sourceCte); + + Query *cteQuery = (Query *) copyObject(sourceCte->ctequery); + + sourceRte->rtekind = RTE_SUBQUERY; + + /* + * As we are delinking the CTE from main query, we have to walk through the + * tree and decrement the ctelevelsup, but by wrapping a subquery, we avoid + * adjusting the ctelevelsup in RTE's + */ + sourceRte->subquery = WrapSubquery(cteQuery); + + /* Copy the rest of the CTEs(if any) and remove them from main query */ + sourceRte->subquery->cteList = copyObject(cteList); + mergeQuery->cteList = NIL; + + /* Zero out CTE-specific fields */ + sourceRte->security_barrier = false; + sourceRte->ctename = NULL; + sourceRte->ctelevelsup = 0; + sourceRte->self_reference = false; + sourceRte->coltypes = NIL; + sourceRte->coltypmods = NIL; + sourceRte->colcollations = NIL; +} + + +/* + * ConvertRelationRTEIntoSubquery takes a RTE_RELATION and converts it into a RTE_SUBQUERY, + * which is basically a SELECT * FROM the relation. + */ +static void +ConvertRelationRTEIntoSubquery(Query *mergeQuery, RangeTblEntry *sourceRte, + PlannerRestrictionContext *plannerRestrictionContext) +{ + Query *sourceResultsQuery = makeNode(Query); + RangeTblRef *newRangeTableRef = makeNode(RangeTblRef); + List *requiredAttributes = NIL; + + RelationRestriction *relationRestriction = + RelationRestrictionForRelation(sourceRte, plannerRestrictionContext); + if (relationRestriction) + { + requiredAttributes = + RequiredAttrNumbersForRelationInternal(mergeQuery, + relationRestriction->index); + } + + sourceResultsQuery->commandType = CMD_SELECT; + + /* we copy the input rteRelation to preserve the rteIdentity */ + RangeTblEntry *newRangeTableEntry = copyObject(sourceRte); + sourceResultsQuery->rtable = list_make1(newRangeTableEntry); + + /* set the FROM expression to the subquery */ + newRangeTableRef->rtindex = SINGLE_RTE_INDEX; + sourceResultsQuery->jointree = makeFromExpr(list_make1(newRangeTableRef), NULL); + sourceResultsQuery->targetList = + CreateAllTargetListForRelation(sourceRte->relid, requiredAttributes); + List *restrictionList = + GetRestrictInfoListForRelation(sourceRte, plannerRestrictionContext); + List *copyRestrictionList = copyObject(restrictionList); + Expr *andedBoundExpressions = make_ands_explicit(copyRestrictionList); + sourceResultsQuery->jointree->quals = (Node *) andedBoundExpressions; + + /* + * Originally the quals were pointing to the RTE and its varno + * was pointing to its index in rtable. However now we converted the RTE + * to a subquery and the quals should be pointing to that subquery, which + * is the only RTE in its rtable, hence we update the varnos so that they + * point to the subquery RTE. + * Originally: rtable: [rte1, current_rte, rte3...] + * Now: rtable: [rte1, subquery[current_rte], rte3...] --subquery[current_rte] refers to its rtable. + */ + Node *quals = sourceResultsQuery->jointree->quals; + UpdateVarNosInNode(quals, SINGLE_RTE_INDEX); + + /* replace the function with the constructed subquery */ + sourceRte->rtekind = RTE_SUBQUERY; + sourceRte->subquery = sourceResultsQuery; + sourceRte->inh = false; +} + + +/* + * ConvertSubqueryRTEIntoSubquery takes a RTE_SUBQUERY and wraps it into a new + * subquery, which eliminates any resjunk columns and adjusts the CTE levelsup. + * In addition, if the subquery happens to be a SET operation, such as, + * (SELECT * from a UNION SELECT * FROM b), it reorders, adds casts and + * prepares a single taget list + */ +static void +ConvertSubqueryRTEIntoSubquery(Query *mergeQuery, RangeTblEntry *sourceRte) +{ + sourceRte->subquery = WrapSubquery(sourceRte->subquery); + + if (list_length(mergeQuery->cteList) > 0) + { + /* copy CTEs from the MERGE ... INTO statement into source subquery */ + sourceRte->subquery->cteList = copyObject(mergeQuery->cteList); + sourceRte->subquery->hasModifyingCTE = mergeQuery->hasModifyingCTE; + mergeQuery->cteList = NIL; + } +} + + +/* + * ConvertSourceRTEIntoSubquery converts MERGE's source RTE into a subquery, + * whose result rows are repartitioned during runtime. + */ +static void +ConvertSourceRTEIntoSubquery(Query *mergeQuery, RangeTblEntry *sourceRte, + PlannerRestrictionContext *plannerRestrictionContext) +{ + switch (sourceRte->rtekind) + { + case RTE_SUBQUERY: + { + ConvertSubqueryRTEIntoSubquery(mergeQuery, sourceRte); + return; + } + + case RTE_RELATION: + { + ConvertRelationRTEIntoSubquery(mergeQuery, + sourceRte, plannerRestrictionContext); + return; + } + + case RTE_CTE: + { + ConvertCteRTEIntoSubquery(mergeQuery, sourceRte); + return; + } + + default: + { + ereport(ERROR, (errmsg("Currently, Citus only supports " + "table, subquery, and CTEs as " + "valid sources for the MERGE " + "operation"))); + } + } +} + + +/* + * ErrorIfMergeNotSupported Checks for conditions that are not supported in either + * the routable or repartition strategies. It checks for + * - Supported table types and their combinations + * - Check the target lists and quals of both the query and merge actions + * - Supported CTEs + */ +static void +ErrorIfMergeNotSupported(Query *query, Oid targetRelationId, List *rangeTableList) +{ + ErrorIfMergeHasUnsupportedTables(targetRelationId, rangeTableList); + ErrorIfMergeQueryQualAndTargetListNotSupported(targetRelationId, query); + ErrorIfUnsupportedCTEs(query); +} + + +/* + * DeferErrorIfRoutableMergeNotSupported Checks for conditions that prevent pushable planning, if + * found, raises a deferred error, which then continues to try repartitioning strategy. + */ +static DeferredErrorMessage * +DeferErrorIfRoutableMergeNotSupported(Query *query, List *rangeTableList, + PlannerRestrictionContext *plannerRestrictionContext) +{ + List *distTablesList = NIL; + List *refTablesList = NIL; + List *localTablesList = NIL; + RangeTblEntry *rangeTableEntry = NULL; + + foreach_ptr(rangeTableEntry, rangeTableList) + { + Oid relationId = rangeTableEntry->relid; + + if (IsCitusTableType(relationId, DISTRIBUTED_TABLE)) + { + distTablesList = lappend(distTablesList, rangeTableEntry); + } + else if (IsCitusTableType(relationId, REFERENCE_TABLE)) + { + refTablesList = lappend(refTablesList, rangeTableEntry); + } + else if (IsCitusTableType(relationId, CITUS_LOCAL_TABLE)) + { + localTablesList = lappend(localTablesList, rangeTableEntry); + } + } + + if (list_length(distTablesList) > 0 && list_length(refTablesList) > 0) + { + ereport(DEBUG1, (errmsg( + "A mix of distributed and reference table, try repartitioning"))); + return DeferredError(ERRCODE_FEATURE_NOT_SUPPORTED, + "A mix of distributed and reference table, " + "routable query is not possible", NULL, NULL); + } + + if (list_length(distTablesList) > 0 && list_length(localTablesList) > 0) + { + ereport(DEBUG1, (errmsg( + "A mix of distributed and local table, try repartitioning"))); + return DeferredError(ERRCODE_FEATURE_NOT_SUPPORTED, + "A mix of distributed and citus-local table, " + "routable query is not possible", NULL, NULL); + } + + /* + * If all tables are either local or reference tables, no need to proceed further down + * as the below checks are applicable for distributed tables only + */ + if (list_length(distTablesList) == 0) + { + return NULL; + } + + /* Only one distributed table is involved in the MERGE */ + if (list_length(distTablesList) == 1) + { + return DeferredError(ERRCODE_FEATURE_NOT_SUPPORTED, + "There is only one distributed table, merge is not " + "pushable, try repartitioning", NULL, NULL); + } + + /* Ensure all distributed tables are indeed co-located */ + if (!AllDistributedRelationsInRTEListColocated(distTablesList)) + { + ereport(DEBUG1, (errmsg("Distributed tables are not co-located, try " + "repartitioning"))); + return DeferredError(ERRCODE_FEATURE_NOT_SUPPORTED, + "For MERGE command, all the distributed tables " + "must be colocated", NULL, NULL); + } + + DeferredErrorMessage *deferredError = + DeferErrorIfUnsupportedSubqueryPushdown(query, + plannerRestrictionContext); + if (deferredError) + { + ereport(DEBUG1, (errmsg("Sub-query is not pushable, try repartitioning"))); + return deferredError; + } + + if (HasDangerousJoinUsing(query->rtable, (Node *) query->jointree)) + { + ereport(DEBUG1, (errmsg( + "Query has ambigious joins, merge is not pushable, try repartitioning"))); + return DeferredError(ERRCODE_FEATURE_NOT_SUPPORTED, + "a join with USING causes an internal naming " + "conflict, use ON instead", NULL, NULL); + } return NULL; +} - #else +/* + * ErrorIfMergeQueryQualAndTargetListNotSupported does check for a MERGE command in the query, if it finds + * one, it will verify the below criteria + * - Distributed tables co-location requirements + * - Checks target-lists and functions-in-quals in TargetlistAndFunctionsSupported + */ +static void +ErrorIfMergeQueryQualAndTargetListNotSupported(Oid targetRelationId, Query *originalQuery) +{ /* * TODO: For now, we are adding an exception where any volatile or stable * functions are not allowed in the MERGE query, but this will become too @@ -585,42 +1043,20 @@ MergeQuerySupported(Oid resultRelationId, Query *originalQuery, bool multiShardQ */ if (contain_mutable_functions((Node *) originalQuery)) { - return DeferredError(ERRCODE_FEATURE_NOT_SUPPORTED, - "non-IMMUTABLE functions are not yet supported " - "in MERGE sql with distributed tables ", - NULL, NULL); + ereport(ERROR, (errmsg("non-IMMUTABLE functions are not yet " + "supported in MERGE sql with distributed tables"))); } - List *rangeTableList = ExtractRangeTableEntryList(originalQuery); - - /* - * Fast path queries cannot have merge command, and we prevent the remaining here. - * In Citus we have limited support for MERGE, it's allowed only if all - * the tables(target, source or any CTE) tables are are local i.e. a - * combination of Citus local and Non-Citus tables (regular Postgres tables) - * or distributed tables with some restrictions, please see header of routine - * ErrorIfDistTablesNotColocated for details. - */ DeferredErrorMessage *deferredError = - ErrorIfMergeHasUnsupportedTables(resultRelationId, - originalQuery, - rangeTableList, - plannerRestrictionContext); - if (deferredError) - { - /* MERGE's unsupported combination, raise the exception */ - RaiseDeferredError(deferredError, ERROR); - } + MergeQualAndTargetListFunctionsSupported(targetRelationId, + originalQuery, + originalQuery->jointree->quals, + originalQuery->targetList, + originalQuery->commandType); - deferredError = MergeQualAndTargetListFunctionsSupported(resultRelationId, - originalQuery->jointree, - originalQuery->jointree-> - quals, - originalQuery->targetList, - originalQuery->commandType); if (deferredError) { - return deferredError; + RaiseDeferredError(deferredError, ERROR); } /* @@ -631,8 +1067,8 @@ MergeQuerySupported(Oid resultRelationId, Query *originalQuery, bool multiShardQ foreach_ptr(action, originalQuery->mergeActionList) { Assert(originalQuery->returningList == NULL); - deferredError = MergeQualAndTargetListFunctionsSupported(resultRelationId, - originalQuery->jointree, + deferredError = MergeQualAndTargetListFunctionsSupported(targetRelationId, + originalQuery, action->qual, action->targetList, action->commandType); @@ -642,86 +1078,284 @@ MergeQuerySupported(Oid resultRelationId, Query *originalQuery, bool multiShardQ RaiseDeferredError(deferredError, ERROR); } } +} - deferredError = - InsertDistributionColumnMatchesSource(resultRelationId, originalQuery); - if (deferredError) + +/* + * MergeCommandResultIdPrefix returns the prefix to use for intermediate results of + * an MERGE INTO ... USING source-query results via the coordinator. + */ +static char * +MergeCommandResultIdPrefix(uint64 planId) +{ + StringInfo resultIdPrefix = makeStringInfo(); + appendStringInfo(resultIdPrefix, "merge_into_" UINT64_FORMAT, planId); + return resultIdPrefix->data; +} + + +/* + * ValidateAndReturnVarIfSupported Checks for valid expressions of type Var, and + * returns the Var if it finds one, for everything else, raises an exception. + */ +static Var * +ValidateAndReturnVarIfSupported(Node *entryExpr) +{ + if (!IsA(entryExpr, Var)) { - /* MERGE's unsupported scenario, raise the exception */ - RaiseDeferredError(deferredError, ERROR); + ereport(ERROR, (errmsg("MERGE INSERT is using unsupported expression type " + "for distribution column"), + errdetail("Inserting arbitrary values that don't correspond " + "to the joined column values can lead to unpredictable " + "outcomes where rows are incorrectly distributed " + "among different shards"))); } - if (multiShardQuery) + /* Found a Var inserting into target's distribution column */ + return (Var *) entryExpr; +} + + +/* + * SourceResultPartitionColumnIndex collects all Join conditions from the + * ON clause and verifies if there is a join, either left or right, with + * the distribution column of the given target. Once a match is found, it + * returns the index of that match in the source's target list. + */ +static int +SourceResultPartitionColumnIndex(Query *mergeQuery, List *sourceTargetList, + CitusTableCacheEntry *targetRelation) +{ + if (IsCitusTableType(targetRelation->relationId, SINGLE_SHARD_DISTRIBUTED)) { - deferredError = - DeferErrorIfUnsupportedSubqueryPushdown(originalQuery, - plannerRestrictionContext); - if (deferredError) + ereport(ERROR, (errmsg("MERGE operation on non-colocated " + "distributed table(s) without a shard " + "key is not yet supported"))); + } + + /* Get all the Join conditions from the ON clause */ + List *mergeJoinConditionList = WhereClauseList(mergeQuery->jointree); + Var *targetColumn = targetRelation->partitionColumn; + Var *sourceRepartitionVar = NULL; + + OpExpr *validJoinClause = + SinglePartitionJoinClause(list_make1(targetColumn), mergeJoinConditionList); + if (!validJoinClause) + { + ereport(ERROR, (errmsg("The required join operation is missing between " + "the target's distribution column and any " + "expression originating from the source. The " + "issue may arise from either a non-equi-join or " + "a mismatch in the datatypes of the columns being " + "joined."), + errdetail("Without a equi-join condition on the target's " + "distribution column, the source rows " + "cannot be efficiently redistributed, and " + "the NOT-MATCHED condition cannot be evaluated " + "unambiguously. This can result in incorrect or " + "unexpected results when attempting to merge " + "tables in a distributed setting"))); + } + + /* both are verified in SinglePartitionJoinClause to not be NULL, assert is to guard */ + Var *leftColumn = LeftColumnOrNULL(validJoinClause); + Var *rightColumn = RightColumnOrNULL(validJoinClause); + + Assert(leftColumn != NULL); + Assert(rightColumn != NULL); + + if (equal(targetColumn, leftColumn)) + { + sourceRepartitionVar = rightColumn; + } + else if (equal(targetColumn, rightColumn)) + { + sourceRepartitionVar = leftColumn; + } + + /* Either we find an insert-action or it's not relevant for certain class of tables */ + Var *insertVar = + FetchAndValidateInsertVarIfExists(targetRelation->relationId, mergeQuery); + if (insertVar) + { + /* INSERT action, must choose joining column for inserted value */ + bool joinedOnInsertColumn = + JoinOnColumns(list_make1(targetColumn), insertVar, mergeJoinConditionList); + if (joinedOnInsertColumn) { - return deferredError; + sourceRepartitionVar = insertVar; + } + else + { + ereport(ERROR, (errmsg("MERGE INSERT must use the " + "source's joining column for " + "target's distribution column"))); } } - if (HasDangerousJoinUsing(originalQuery->rtable, (Node *) originalQuery->jointree)) + Assert(sourceRepartitionVar); + + int sourceResultRepartitionColumnIndex = + DistributionColumnIndex(sourceTargetList, sourceRepartitionVar); + + if (sourceResultRepartitionColumnIndex == -1) { - return DeferredError(ERRCODE_FEATURE_NOT_SUPPORTED, - "a join with USING causes an internal naming " - "conflict, use ON instead", NULL, NULL); + ereport(ERROR, + (errmsg("Unexpected column index of the source list"))); + } + else + { + ereport(DEBUG1, (errmsg("Using column - index:%d from the source list " + "to redistribute", sourceResultRepartitionColumnIndex))); } - return NULL; + return sourceResultRepartitionColumnIndex; +} + + +#endif + + +/* + * ExtractMergeSourceRangeTableEntry returns the range table entry of source + * table or source query in USING clause. + */ +RangeTblEntry * +ExtractMergeSourceRangeTableEntry(Query *query) +{ + /* function is void for pre-15 versions of Postgres */ + #if PG_VERSION_NUM < PG_VERSION_15 + + ereport(ERROR, (errmsg("MERGE is not supported in pre-15 Postgres versions"))); + + #else + + Assert(IsMergeQuery(query)); + + List *fromList = query->jointree->fromlist; + + /* We should have only one RTE(MergeStmt->sourceRelation) in the from-list */ + if (list_length(fromList) != 1) + { + ereport(ERROR, (errmsg("Unexpected source list in MERGE sql USING clause"))); + } + + RangeTblRef *reference = linitial(fromList); + + /* + * The planner sometimes generates JoinExprs internally; these can + * have rtindex = 0 if there are no join alias variables referencing + * such joins. + */ + if (reference->rtindex == 0) + { + ereport(ERROR, (errmsg("Source is not an explicit query"), + errhint("Source query is a Join expression, " + "try converting into a query as SELECT * " + "FROM (..Join..)"))); + } + + Assert(reference->rtindex >= 1); + RangeTblEntry *subqueryRte = rt_fetch(reference->rtindex, query->rtable); + + return subqueryRte; #endif } /* - * CreateMergePlan attempts to create a plan for the given MERGE SQL - * statement. If planning fails ->planningError is set to a description - * of the failure. + * FetchAndValidateInsertVarIfExists checks to see if MERGE is inserting a + * value into the target which is not from the source table, if so, it + * raises an exception. The return value is the Var that's being inserted + * into the target's distribution column, If no INSERT action exist, it + * simply returns a NULL. + * Note: Inserting random values other than the joined column values will + * result in unexpected behaviour of rows ending up in incorrect shards, to + * prevent such mishaps, we disallow such inserts here. */ -DistributedPlan * -CreateMergePlan(Query *originalQuery, Query *query, - PlannerRestrictionContext *plannerRestrictionContext) +Var * +FetchAndValidateInsertVarIfExists(Oid targetRelationId, Query *query) { - DistributedPlan *distributedPlan = CitusMakeNode(DistributedPlan); - bool multiShardQuery = false; - Oid targetRelationId = ModifyQueryResultRelationId(originalQuery); + /* function is void for pre-15 versions of Postgres */ + #if PG_VERSION_NUM < PG_VERSION_15 - Assert(originalQuery->commandType == CMD_MERGE); - Assert(OidIsValid(targetRelationId)); + ereport(ERROR, (errmsg("MERGE is not supported in pre-15 Postgres versions"))); - distributedPlan->targetRelationId = targetRelationId; - distributedPlan->modLevel = RowModifyLevelForQuery(query); - distributedPlan->planningError = MergeQuerySupported(targetRelationId, - originalQuery, - multiShardQuery, - plannerRestrictionContext); + #else - if (distributedPlan->planningError != NULL) + Assert(IsMergeQuery(query)); + + if (!IsCitusTableType(targetRelationId, DISTRIBUTED_TABLE)) { - return distributedPlan; + return NULL; } - Job *job = RouterJob(originalQuery, plannerRestrictionContext, - &distributedPlan->planningError); - - if (distributedPlan->planningError != NULL) + if (!HasDistributionKey(targetRelationId)) { - return distributedPlan; + return NULL; } - ereport(DEBUG1, (errmsg("Creating MERGE router plan"))); + bool foundDistributionColumn = false; + MergeAction *action = NULL; + uint32 targetRangeTableIndex = query->resultRelation; + foreach_ptr(action, query->mergeActionList) + { + /* Skip MATCHED clause as INSERTS are not allowed in it */ + if (action->matched) + { + continue; + } - distributedPlan->workerJob = job; - distributedPlan->combineQuery = NULL; + /* NOT MATCHED can have either INSERT or DO NOTHING */ + if (action->commandType == CMD_NOTHING) + { + return NULL; + } - /* MERGE doesn't support RETURNING clause */ - distributedPlan->expectResults = false; - distributedPlan->fastPathRouterPlan = - plannerRestrictionContext->fastPathRestrictionContext->fastPathRouterQuery; + if (action->targetList == NIL) + { + /* INSERT DEFAULT VALUES is not allowed */ + ereport(ERROR, (errmsg("cannot perform MERGE INSERT with DEFAULTS"), + errdetail("Inserting arbitrary values that don't correspond " + "to the joined column values can lead to " + "unpredictable outcomes where rows are " + "incorrectly distributed among different " + "shards"))); + } - return distributedPlan; + Assert(action->commandType == CMD_INSERT); + Var *targetDistributionKey = + PartitionColumn(targetRelationId, targetRangeTableIndex); + + TargetEntry *targetEntry = NULL; + foreach_ptr(targetEntry, action->targetList) + { + AttrNumber originalAttrNo = targetEntry->resno; + + /* skip processing of target table non-distribution columns */ + if (originalAttrNo != targetDistributionKey->varattno) + { + continue; + } + + foundDistributionColumn = true; + + Node *insertExpr = + strip_implicit_coercions((Node *) copyObject(targetEntry->expr)); + return ValidateAndReturnVarIfSupported(insertExpr); + } + + if (!foundDistributionColumn) + { + ereport(ERROR, + (errmsg("MERGE INSERT must have distribution column as value"))); + } + } + + return NULL; + + #endif } diff --git a/src/backend/distributed/planner/multi_explain.c b/src/backend/distributed/planner/multi_explain.c index 248117904..1cc3d4102 100644 --- a/src/backend/distributed/planner/multi_explain.c +++ b/src/backend/distributed/planner/multi_explain.c @@ -33,6 +33,7 @@ #include "distributed/insert_select_planner.h" #include "distributed/insert_select_executor.h" #include "distributed/listutils.h" +#include "distributed/merge_planner.h" #include "distributed/multi_executor.h" #include "distributed/multi_explain.h" #include "distributed/multi_logical_optimizer.h" @@ -244,9 +245,8 @@ NonPushableInsertSelectExplainScan(CustomScanState *node, List *ancestors, */ Query *queryCopy = copyObject(selectRte->subquery); - bool repartition = distributedPlan->modifyWithSelectMethod == - MODIFY_WITH_SELECT_REPARTITION; - + bool repartition = + distributedPlan->modifyWithSelectMethod == MODIFY_WITH_SELECT_REPARTITION; if (es->analyze) { @@ -282,6 +282,67 @@ NonPushableInsertSelectExplainScan(CustomScanState *node, List *ancestors, } +/* + * NonPushableMergeSqlExplainScan is a custom scan explain callback function + * which is used to print explain information of a Citus plan for MERGE INTO + * distributed_table USING (source query/table), where source can be any query + * whose results are repartitioned to colocated with the target table. + */ +void +NonPushableMergeCommandExplainScan(CustomScanState *node, List *ancestors, + struct ExplainState *es) +{ + CitusScanState *scanState = (CitusScanState *) node; + DistributedPlan *distributedPlan = scanState->distributedPlan; + Query *mergeQuery = distributedPlan->modifyQueryViaCoordinatorOrRepartition; + RangeTblEntry *sourceRte = ExtractMergeSourceRangeTableEntry(mergeQuery); + + /* + * Create a copy because ExplainOneQuery can modify the query, and later + * executions of prepared statements might require it. See + * https://github.com/citusdata/citus/issues/3947 for what can happen. + */ + Query *sourceQueryCopy = copyObject(sourceRte->subquery); + bool repartition = + distributedPlan->modifyWithSelectMethod == MODIFY_WITH_SELECT_REPARTITION; + + if (es->analyze) + { + ereport(ERROR, (errmsg("EXPLAIN ANALYZE is currently not supported for " + "MERGE INTO ... commands with repartitioning"))); + } + + Oid targetRelationId = ModifyQueryResultRelationId(mergeQuery); + StringInfo mergeMethodMessage = makeStringInfo(); + appendStringInfo(mergeMethodMessage, + "MERGE INTO %s method", get_rel_name(targetRelationId)); + + if (repartition) + { + ExplainPropertyText(mergeMethodMessage->data, "repartition", es); + } + else + { + ExplainPropertyText(mergeMethodMessage->data, "pull to coordinator", es); + } + + ExplainOpenGroup("Source Query", "Source Query", false, es); + + /* explain the MERGE source query */ + IntoClause *into = NULL; + ParamListInfo params = NULL; + + /* + * With PG14, we need to provide a string here, for now we put an empty + * string, which is valid according to postgres. + */ + char *queryString = pstrdup(""); + ExplainOneQuery(sourceQueryCopy, 0, into, es, queryString, params, NULL); + + ExplainCloseGroup("Source Query", "Source Query", false, es); +} + + /* * ExplainSubPlans generates EXPLAIN output for subplans for CTEs * and complex subqueries. Because the planning for these queries diff --git a/src/backend/distributed/planner/multi_join_order.c b/src/backend/distributed/planner/multi_join_order.c index 0fff79ed8..79007b70d 100644 --- a/src/backend/distributed/planner/multi_join_order.c +++ b/src/backend/distributed/planner/multi_join_order.c @@ -81,8 +81,6 @@ static JoinOrderNode * CartesianProductReferenceJoin(JoinOrderNode *joinNode, JoinType joinType); static JoinOrderNode * LocalJoin(JoinOrderNode *joinNode, TableEntry *candidateTable, List *applicableJoinClauses, JoinType joinType); -static bool JoinOnColumns(List *currentPartitionColumnList, Var *candidatePartitionColumn, - List *joinClauseList); static JoinOrderNode * SinglePartitionJoin(JoinOrderNode *joinNode, TableEntry *candidateTable, List *applicableJoinClauses, @@ -212,7 +210,7 @@ ExtractLeftMostRangeTableIndex(Node *node, int *rangeTableIndex) /* * JoinOnColumns determines whether two columns are joined by a given join clause list. */ -static bool +bool JoinOnColumns(List *currentPartitionColumnList, Var *candidateColumn, List *joinClauseList) { diff --git a/src/backend/distributed/planner/multi_router_planner.c b/src/backend/distributed/planner/multi_router_planner.c index 87ab1277f..41ae916ad 100644 --- a/src/backend/distributed/planner/multi_router_planner.c +++ b/src/backend/distributed/planner/multi_router_planner.c @@ -388,6 +388,26 @@ AddPartitionKeyNotNullFilterToSelect(Query *subqery) } +/* + * ExtractSourceResultRangeTableEntry Generic wrapper for modification commands that + * utilizes results as input, based on an source query. + */ +RangeTblEntry * +ExtractSourceResultRangeTableEntry(Query *query) +{ + if (IsMergeQuery(query)) + { + return ExtractMergeSourceRangeTableEntry(query); + } + else if (CheckInsertSelectQuery(query)) + { + return ExtractSelectRangeTableEntry(query); + } + + return NULL; +} + + /* * ExtractSelectRangeTableEntry returns the range table entry of the subquery. * Note that the function expects and asserts that the input query be @@ -1863,19 +1883,7 @@ RouterJob(Query *originalQuery, PlannerRestrictionContext *plannerRestrictionCon if (*planningError) { - /* - * For MERGE, we do _not_ plan any other router job than the MERGE job itself, - * let's not continue further down the lane in distributed planning, simply - * bail out. - */ - if (IsMergeQuery(originalQuery)) - { - RaiseDeferredError(*planningError, ERROR); - } - else - { - return NULL; - } + return NULL; } Job *job = CreateJob(originalQuery); @@ -2366,14 +2374,7 @@ PlanRouterQuery(Query *originalQuery, Assert(UpdateOrDeleteOrMergeQuery(originalQuery)); - if (IsMergeQuery(originalQuery)) - { - targetRelationId = ModifyQueryResultRelationId(originalQuery); - planningError = MergeQuerySupported(targetRelationId, originalQuery, - isMultiShardQuery, - plannerRestrictionContext); - } - else + if (!IsMergeQuery(originalQuery)) { planningError = ModifyQuerySupported(originalQuery, originalQuery, isMultiShardQuery, diff --git a/src/backend/distributed/planner/recursive_planning.c b/src/backend/distributed/planner/recursive_planning.c index 936b17364..f582fd9df 100644 --- a/src/backend/distributed/planner/recursive_planning.c +++ b/src/backend/distributed/planner/recursive_planning.c @@ -188,7 +188,6 @@ static Query * BuildReadIntermediateResultsQuery(List *targetEntryList, List *columnAliasList, Const *resultIdConst, Oid functionOid, bool useBinaryCopyFormat); -static void UpdateVarNosInNode(Node *node, Index newVarNo); static Query * CreateOuterSubquery(RangeTblEntry *rangeTableEntry, List *outerSubqueryTargetList); static List * GenerateRequiredColNamesFromTargetList(List *targetList); @@ -1891,7 +1890,7 @@ GenerateRequiredColNamesFromTargetList(List *targetList) * UpdateVarNosInNode iterates the Vars in the * given node and updates the varno's as the newVarNo. */ -static void +void UpdateVarNosInNode(Node *node, Index newVarNo) { List *varList = pull_var_clause(node, PVC_RECURSE_AGGREGATES | diff --git a/src/backend/distributed/sql/citus--11.3-1--12.0-1.sql b/src/backend/distributed/sql/citus--11.3-1--12.0-1.sql index 688b79fe8..998ffc2be 100644 --- a/src/backend/distributed/sql/citus--11.3-1--12.0-1.sql +++ b/src/backend/distributed/sql/citus--11.3-1--12.0-1.sql @@ -39,3 +39,6 @@ DROP FUNCTION citus_shard_sizes; -- udfs to convert a regular/tenant schema to a tenant/regular schema #include "udfs/citus_schema_distribute/12.0-1.sql" #include "udfs/citus_schema_undistribute/12.0-1.sql" + +#include "udfs/drop_old_time_partitions/12.0-1.sql" +#include "udfs/get_missing_time_partition_ranges/12.0-1.sql" diff --git a/src/backend/distributed/sql/downgrades/citus--12.0-1--11.3-1.sql b/src/backend/distributed/sql/downgrades/citus--12.0-1--11.3-1.sql index 3acd60311..1adb4cb72 100644 --- a/src/backend/distributed/sql/downgrades/citus--12.0-1--11.3-1.sql +++ b/src/backend/distributed/sql/downgrades/citus--12.0-1--11.3-1.sql @@ -73,3 +73,6 @@ DROP FUNCTION pg_catalog.citus_stat_tenants_local_internal( OUT DOUBLE PRECISION, OUT BIGINT); #include "../udfs/citus_stat_tenants_local/11.3-1.sql" + +#include "../udfs/drop_old_time_partitions/10.2-1.sql" +#include "../udfs/get_missing_time_partition_ranges/10.2-1.sql" diff --git a/src/backend/distributed/sql/udfs/drop_old_time_partitions/12.0-1.sql b/src/backend/distributed/sql/udfs/drop_old_time_partitions/12.0-1.sql new file mode 100644 index 000000000..a4bcd45a0 --- /dev/null +++ b/src/backend/distributed/sql/udfs/drop_old_time_partitions/12.0-1.sql @@ -0,0 +1,68 @@ +CREATE OR REPLACE PROCEDURE pg_catalog.drop_old_time_partitions( + table_name regclass, + older_than timestamptz) +LANGUAGE plpgsql +AS $$ +DECLARE + -- properties of the partitioned table + number_of_partition_columns int; + partition_column_index int; + partition_column_type regtype; + + -- used to support dynamic type casting between the partition column type and timestamptz + custom_cast text; + is_partition_column_castable boolean; + older_partitions_query text; + + r record; +BEGIN + -- check whether the table is time partitioned table, if not error out + SELECT partnatts, partattrs[0] + INTO number_of_partition_columns, partition_column_index + FROM pg_catalog.pg_partitioned_table + WHERE partrelid = table_name; + + IF NOT FOUND THEN + RAISE '% is not partitioned', table_name::text; + ELSIF number_of_partition_columns <> 1 THEN + RAISE 'partitioned tables with multiple partition columns are not supported'; + END IF; + + -- get datatype here to check interval-table type + SELECT atttypid + INTO partition_column_type + FROM pg_attribute + WHERE attrelid = table_name::oid + AND attnum = partition_column_index; + + -- we currently only support partitioning by date, timestamp, and timestamptz + custom_cast = ''; + IF partition_column_type <> 'date'::regtype + AND partition_column_type <> 'timestamp'::regtype + AND partition_column_type <> 'timestamptz'::regtype THEN + SELECT EXISTS(SELECT OID FROM pg_cast WHERE castsource = partition_column_type AND casttarget = 'timestamptz'::regtype) AND + EXISTS(SELECT OID FROM pg_cast WHERE castsource = 'timestamptz'::regtype AND casttarget = partition_column_type) + INTO is_partition_column_castable; + IF not is_partition_column_castable THEN + RAISE 'type of the partition column of the table % must be date, timestamp or timestamptz', table_name; + END IF; + custom_cast = format('::%s', partition_column_type); + END IF; + + older_partitions_query = format('SELECT partition, nspname AS schema_name, relname AS table_name, from_value, to_value + FROM pg_catalog.time_partitions, pg_catalog.pg_class c, pg_catalog.pg_namespace n + WHERE parent_table = $1 AND partition = c.oid AND c.relnamespace = n.oid + AND to_value IS NOT NULL + AND to_value%1$s::timestamptz <= $2 + ORDER BY to_value%1$s::timestamptz', custom_cast); + FOR r IN EXECUTE older_partitions_query USING table_name, older_than + LOOP + RAISE NOTICE 'dropping % with start time % and end time %', r.partition, r.from_value, r.to_value; + EXECUTE format('DROP TABLE %I.%I', r.schema_name, r.table_name); + END LOOP; +END; +$$; +COMMENT ON PROCEDURE pg_catalog.drop_old_time_partitions( + table_name regclass, + older_than timestamptz) +IS 'drop old partitions of a time-partitioned table'; diff --git a/src/backend/distributed/sql/udfs/drop_old_time_partitions/latest.sql b/src/backend/distributed/sql/udfs/drop_old_time_partitions/latest.sql index c735d9f67..a4bcd45a0 100644 --- a/src/backend/distributed/sql/udfs/drop_old_time_partitions/latest.sql +++ b/src/backend/distributed/sql/udfs/drop_old_time_partitions/latest.sql @@ -1,6 +1,6 @@ CREATE OR REPLACE PROCEDURE pg_catalog.drop_old_time_partitions( - table_name regclass, - older_than timestamptz) + table_name regclass, + older_than timestamptz) LANGUAGE plpgsql AS $$ DECLARE @@ -9,6 +9,11 @@ DECLARE partition_column_index int; partition_column_type regtype; + -- used to support dynamic type casting between the partition column type and timestamptz + custom_cast text; + is_partition_column_castable boolean; + older_partitions_query text; + r record; BEGIN -- check whether the table is time partitioned table, if not error out @@ -31,19 +36,26 @@ BEGIN AND attnum = partition_column_index; -- we currently only support partitioning by date, timestamp, and timestamptz + custom_cast = ''; IF partition_column_type <> 'date'::regtype AND partition_column_type <> 'timestamp'::regtype AND partition_column_type <> 'timestamptz'::regtype THEN + SELECT EXISTS(SELECT OID FROM pg_cast WHERE castsource = partition_column_type AND casttarget = 'timestamptz'::regtype) AND + EXISTS(SELECT OID FROM pg_cast WHERE castsource = 'timestamptz'::regtype AND casttarget = partition_column_type) + INTO is_partition_column_castable; + IF not is_partition_column_castable THEN RAISE 'type of the partition column of the table % must be date, timestamp or timestamptz', table_name; + END IF; + custom_cast = format('::%s', partition_column_type); END IF; - FOR r IN - SELECT partition, nspname AS schema_name, relname AS table_name, from_value, to_value - FROM pg_catalog.time_partitions, pg_catalog.pg_class c, pg_catalog.pg_namespace n - WHERE parent_table = table_name AND partition = c.oid AND c.relnamespace = n.oid - AND to_value IS NOT NULL - AND to_value::timestamptz <= older_than - ORDER BY to_value::timestamptz + older_partitions_query = format('SELECT partition, nspname AS schema_name, relname AS table_name, from_value, to_value + FROM pg_catalog.time_partitions, pg_catalog.pg_class c, pg_catalog.pg_namespace n + WHERE parent_table = $1 AND partition = c.oid AND c.relnamespace = n.oid + AND to_value IS NOT NULL + AND to_value%1$s::timestamptz <= $2 + ORDER BY to_value%1$s::timestamptz', custom_cast); + FOR r IN EXECUTE older_partitions_query USING table_name, older_than LOOP RAISE NOTICE 'dropping % with start time % and end time %', r.partition, r.from_value, r.to_value; EXECUTE format('DROP TABLE %I.%I', r.schema_name, r.table_name); @@ -51,6 +63,6 @@ BEGIN END; $$; COMMENT ON PROCEDURE pg_catalog.drop_old_time_partitions( - table_name regclass, - older_than timestamptz) + table_name regclass, + older_than timestamptz) IS 'drop old partitions of a time-partitioned table'; diff --git a/src/backend/distributed/sql/udfs/get_missing_time_partition_ranges/12.0-1.sql b/src/backend/distributed/sql/udfs/get_missing_time_partition_ranges/12.0-1.sql new file mode 100644 index 000000000..4cdca999a --- /dev/null +++ b/src/backend/distributed/sql/udfs/get_missing_time_partition_ranges/12.0-1.sql @@ -0,0 +1,239 @@ +CREATE OR REPLACE FUNCTION pg_catalog.get_missing_time_partition_ranges( + table_name regclass, + partition_interval INTERVAL, + to_value timestamptz, + from_value timestamptz DEFAULT now()) +returns table( + partition_name text, + range_from_value text, + range_to_value text) +LANGUAGE plpgsql +AS $$ +DECLARE + -- properties of the partitioned table + table_name_text text; + table_schema_text text; + number_of_partition_columns int; + partition_column_index int; + partition_column_type regtype; + + -- used for generating time ranges + current_range_from_value timestamptz := NULL; + current_range_to_value timestamptz := NULL; + current_range_from_value_text text; + current_range_to_value_text text; + + -- used to check whether there are misaligned (manually created) partitions + manual_partition regclass; + manual_partition_from_value_text text; + manual_partition_to_value_text text; + + -- used for partition naming + partition_name_format text; + max_table_name_length int := current_setting('max_identifier_length'); + + -- used to determine whether the partition_interval is a day multiple + is_day_multiple boolean; + + -- used to support dynamic type casting between the partition column type and timestamptz + custom_cast text; + is_partition_column_castable boolean; + partition regclass; + partition_covers_query text; + partition_exist_query text; +BEGIN + -- check whether the table is time partitioned table, if not error out + SELECT relname, nspname, partnatts, partattrs[0] + INTO table_name_text, table_schema_text, number_of_partition_columns, partition_column_index + FROM pg_catalog.pg_partitioned_table, pg_catalog.pg_class c, pg_catalog.pg_namespace n + WHERE partrelid = c.oid AND c.oid = table_name + AND c.relnamespace = n.oid; + IF NOT FOUND THEN + RAISE '% is not partitioned', table_name; + ELSIF number_of_partition_columns <> 1 THEN + RAISE 'partitioned tables with multiple partition columns are not supported'; + END IF; + + -- to not to have partitions to be created in parallel + EXECUTE format('LOCK TABLE %I.%I IN SHARE UPDATE EXCLUSIVE MODE', table_schema_text, table_name_text); + + -- get datatype here to check interval-table type alignment and generate range values in the right data format + SELECT atttypid + INTO partition_column_type + FROM pg_attribute + WHERE attrelid = table_name::oid + AND attnum = partition_column_index; + + -- we currently only support partitioning by date, timestamp, and timestamptz + custom_cast = ''; + IF partition_column_type <> 'date'::regtype + AND partition_column_type <> 'timestamp'::regtype + AND partition_column_type <> 'timestamptz'::regtype THEN + SELECT EXISTS(SELECT OID FROM pg_cast WHERE castsource = partition_column_type AND casttarget = 'timestamptz'::regtype) AND + EXISTS(SELECT OID FROM pg_cast WHERE castsource = 'timestamptz'::regtype AND casttarget = partition_column_type) + INTO is_partition_column_castable; + IF not is_partition_column_castable THEN + RAISE 'type of the partition column of the table % must be date, timestamp or timestamptz', table_name; + END IF; + custom_cast = format('::%s', partition_column_type); + END IF; + + IF partition_column_type = 'date'::regtype AND partition_interval IS NOT NULL THEN + SELECT date_trunc('day', partition_interval) = partition_interval + INTO is_day_multiple; + + IF NOT is_day_multiple THEN + RAISE 'partition interval of date partitioned table must be day or multiple days'; + END IF; + END IF; + + -- If no partition exists, truncate from_value to find intuitive initial value. + -- If any partition exist, use the initial partition as the pivot partition. + -- tp.to_value and tp.from_value are equal to '', if default partition exists. + EXECUTE format('SELECT tp.from_value%1$s::timestamptz, tp.to_value%1$s::timestamptz + FROM pg_catalog.time_partitions tp + WHERE parent_table = $1 AND tp.to_value <> '' AND tp.from_value <> '' + ORDER BY tp.from_value%1$s::timestamptz ASC + LIMIT 1', custom_cast) + INTO current_range_from_value, current_range_to_value + USING table_name; + + IF current_range_from_value is NULL THEN + -- Decide on the current_range_from_value of the initial partition according to interval of the table. + -- Since we will create all other partitions by adding intervals, truncating given start time will provide + -- more intuitive interval ranges, instead of starting from from_value directly. + IF partition_interval < INTERVAL '1 hour' THEN + current_range_from_value = date_trunc('minute', from_value); + ELSIF partition_interval < INTERVAL '1 day' THEN + current_range_from_value = date_trunc('hour', from_value); + ELSIF partition_interval < INTERVAL '1 week' THEN + current_range_from_value = date_trunc('day', from_value); + ELSIF partition_interval < INTERVAL '1 month' THEN + current_range_from_value = date_trunc('week', from_value); + ELSIF partition_interval = INTERVAL '3 months' THEN + current_range_from_value = date_trunc('quarter', from_value); + ELSIF partition_interval < INTERVAL '1 year' THEN + current_range_from_value = date_trunc('month', from_value); + ELSE + current_range_from_value = date_trunc('year', from_value); + END IF; + + current_range_to_value := current_range_from_value + partition_interval; + ELSE + -- if from_value is newer than pivot's from value, go forward, else go backward + IF from_value >= current_range_from_value THEN + WHILE current_range_from_value < from_value LOOP + current_range_from_value := current_range_from_value + partition_interval; + END LOOP; + ELSE + WHILE current_range_from_value > from_value LOOP + current_range_from_value := current_range_from_value - partition_interval; + END LOOP; + END IF; + current_range_to_value := current_range_from_value + partition_interval; + END IF; + + -- reuse pg_partman naming scheme for back-and-forth migration + IF partition_interval = INTERVAL '3 months' THEN + -- include quarter in partition name + partition_name_format = 'YYYY"q"Q'; + ELSIF partition_interval = INTERVAL '1 week' THEN + -- include week number in partition name + partition_name_format := 'IYYY"w"IW'; + ELSE + -- always start with the year + partition_name_format := 'YYYY'; + + IF partition_interval < INTERVAL '1 year' THEN + -- include month in partition name + partition_name_format := partition_name_format || '_MM'; + END IF; + + IF partition_interval < INTERVAL '1 month' THEN + -- include day of month in partition name + partition_name_format := partition_name_format || '_DD'; + END IF; + + IF partition_interval < INTERVAL '1 day' THEN + -- include time of day in partition name + partition_name_format := partition_name_format || '_HH24MI'; + END IF; + + IF partition_interval < INTERVAL '1 minute' THEN + -- include seconds in time of day in partition name + partition_name_format := partition_name_format || 'SS'; + END IF; + END IF; + + partition_exist_query = format('SELECT partition FROM pg_catalog.time_partitions tp + WHERE tp.from_value%1$s::timestamptz = $1 AND tp.to_value%1$s::timestamptz = $2 AND parent_table = $3', + custom_cast); + partition_covers_query = format('SELECT partition, tp.from_value, tp.to_value + FROM pg_catalog.time_partitions tp + WHERE + (($1 >= tp.from_value%1$s::timestamptz AND $1 < tp.to_value%1$s::timestamptz) OR + ($2 > tp.from_value%1$s::timestamptz AND $2 < tp.to_value%1$s::timestamptz)) AND + parent_table = $3', + custom_cast); + + WHILE current_range_from_value < to_value LOOP + -- Check whether partition with given range has already been created + -- Since partition interval can be given with different types, we are converting + -- all variables to timestamptz to make sure that we are comparing same type of parameters + EXECUTE partition_exist_query into partition using current_range_from_value, current_range_to_value, table_name; + + IF partition is not NULL THEN + current_range_from_value := current_range_to_value; + current_range_to_value := current_range_to_value + partition_interval; + CONTINUE; + END IF; + + -- Check whether any other partition covers from_value or to_value + -- That means some partitions doesn't align with the initial partition. + -- In other words, gap(s) exist between partitions which is not multiple of intervals. + EXECUTE partition_covers_query + INTO manual_partition, manual_partition_from_value_text, manual_partition_to_value_text + using current_range_from_value, current_range_to_value, table_name; + + IF manual_partition is not NULL THEN + RAISE 'partition % with the range from % to % does not align with the initial partition given the partition interval', + manual_partition::text, + manual_partition_from_value_text, + manual_partition_to_value_text + USING HINT = 'Only use partitions of the same size, without gaps between partitions.'; + END IF; + + IF partition_column_type = 'date'::regtype THEN + SELECT current_range_from_value::date::text INTO current_range_from_value_text; + SELECT current_range_to_value::date::text INTO current_range_to_value_text; + ELSIF partition_column_type = 'timestamp without time zone'::regtype THEN + SELECT current_range_from_value::timestamp::text INTO current_range_from_value_text; + SELECT current_range_to_value::timestamp::text INTO current_range_to_value_text; + ELSIF partition_column_type = 'timestamp with time zone'::regtype THEN + SELECT current_range_from_value::timestamptz::text INTO current_range_from_value_text; + SELECT current_range_to_value::timestamptz::text INTO current_range_to_value_text; + ELSE + EXECUTE format('SELECT $1%s::text', custom_cast) INTO current_range_from_value_text using current_range_from_value; + EXECUTE format('SELECT $1%s::text', custom_cast) INTO current_range_to_value_text using current_range_to_value; + END IF; + + -- use range values within the name of partition to have unique partition names + RETURN QUERY + SELECT + substring(table_name_text, 0, max_table_name_length - length(to_char(current_range_from_value, partition_name_format)) - 1) || '_p' || + to_char(current_range_from_value, partition_name_format), + current_range_from_value_text, + current_range_to_value_text; + + current_range_from_value := current_range_to_value; + current_range_to_value := current_range_to_value + partition_interval; + END LOOP; + RETURN; +END; +$$; +COMMENT ON FUNCTION pg_catalog.get_missing_time_partition_ranges( + table_name regclass, + partition_interval INTERVAL, + to_value timestamptz, + from_value timestamptz) +IS 'get missing partitions ranges for table within the range using the given interval'; diff --git a/src/backend/distributed/sql/udfs/get_missing_time_partition_ranges/latest.sql b/src/backend/distributed/sql/udfs/get_missing_time_partition_ranges/latest.sql index 214f03383..4cdca999a 100644 --- a/src/backend/distributed/sql/udfs/get_missing_time_partition_ranges/latest.sql +++ b/src/backend/distributed/sql/udfs/get_missing_time_partition_ranges/latest.sql @@ -34,6 +34,13 @@ DECLARE -- used to determine whether the partition_interval is a day multiple is_day_multiple boolean; + + -- used to support dynamic type casting between the partition column type and timestamptz + custom_cast text; + is_partition_column_castable boolean; + partition regclass; + partition_covers_query text; + partition_exist_query text; BEGIN -- check whether the table is time partitioned table, if not error out SELECT relname, nspname, partnatts, partattrs[0] @@ -58,10 +65,17 @@ BEGIN AND attnum = partition_column_index; -- we currently only support partitioning by date, timestamp, and timestamptz + custom_cast = ''; IF partition_column_type <> 'date'::regtype AND partition_column_type <> 'timestamp'::regtype - AND partition_column_type <> 'timestamptz'::regtype THEN + AND partition_column_type <> 'timestamptz'::regtype THEN + SELECT EXISTS(SELECT OID FROM pg_cast WHERE castsource = partition_column_type AND casttarget = 'timestamptz'::regtype) AND + EXISTS(SELECT OID FROM pg_cast WHERE castsource = 'timestamptz'::regtype AND casttarget = partition_column_type) + INTO is_partition_column_castable; + IF not is_partition_column_castable THEN RAISE 'type of the partition column of the table % must be date, timestamp or timestamptz', table_name; + END IF; + custom_cast = format('::%s', partition_column_type); END IF; IF partition_column_type = 'date'::regtype AND partition_interval IS NOT NULL THEN @@ -76,14 +90,15 @@ BEGIN -- If no partition exists, truncate from_value to find intuitive initial value. -- If any partition exist, use the initial partition as the pivot partition. -- tp.to_value and tp.from_value are equal to '', if default partition exists. - SELECT tp.from_value::timestamptz, tp.to_value::timestamptz + EXECUTE format('SELECT tp.from_value%1$s::timestamptz, tp.to_value%1$s::timestamptz + FROM pg_catalog.time_partitions tp + WHERE parent_table = $1 AND tp.to_value <> '' AND tp.from_value <> '' + ORDER BY tp.from_value%1$s::timestamptz ASC + LIMIT 1', custom_cast) INTO current_range_from_value, current_range_to_value - FROM pg_catalog.time_partitions tp - WHERE parent_table = table_name AND tp.to_value <> '' AND tp.from_value <> '' - ORDER BY tp.from_value::timestamptz ASC - LIMIT 1; + USING table_name; - IF NOT FOUND THEN + IF current_range_from_value is NULL THEN -- Decide on the current_range_from_value of the initial partition according to interval of the table. -- Since we will create all other partitions by adding intervals, truncating given start time will provide -- more intuitive interval ranges, instead of starting from from_value directly. @@ -150,16 +165,24 @@ BEGIN END IF; END IF; + partition_exist_query = format('SELECT partition FROM pg_catalog.time_partitions tp + WHERE tp.from_value%1$s::timestamptz = $1 AND tp.to_value%1$s::timestamptz = $2 AND parent_table = $3', + custom_cast); + partition_covers_query = format('SELECT partition, tp.from_value, tp.to_value + FROM pg_catalog.time_partitions tp + WHERE + (($1 >= tp.from_value%1$s::timestamptz AND $1 < tp.to_value%1$s::timestamptz) OR + ($2 > tp.from_value%1$s::timestamptz AND $2 < tp.to_value%1$s::timestamptz)) AND + parent_table = $3', + custom_cast); + WHILE current_range_from_value < to_value LOOP -- Check whether partition with given range has already been created -- Since partition interval can be given with different types, we are converting -- all variables to timestamptz to make sure that we are comparing same type of parameters - PERFORM * FROM pg_catalog.time_partitions tp - WHERE - tp.from_value::timestamptz = current_range_from_value::timestamptz AND - tp.to_value::timestamptz = current_range_to_value::timestamptz AND - parent_table = table_name; - IF found THEN + EXECUTE partition_exist_query into partition using current_range_from_value, current_range_to_value, table_name; + + IF partition is not NULL THEN current_range_from_value := current_range_to_value; current_range_to_value := current_range_to_value + partition_interval; CONTINUE; @@ -168,20 +191,16 @@ BEGIN -- Check whether any other partition covers from_value or to_value -- That means some partitions doesn't align with the initial partition. -- In other words, gap(s) exist between partitions which is not multiple of intervals. - SELECT partition, tp.from_value::text, tp.to_value::text + EXECUTE partition_covers_query INTO manual_partition, manual_partition_from_value_text, manual_partition_to_value_text - FROM pg_catalog.time_partitions tp - WHERE - ((current_range_from_value::timestamptz >= tp.from_value::timestamptz AND current_range_from_value < tp.to_value::timestamptz) OR - (current_range_to_value::timestamptz > tp.from_value::timestamptz AND current_range_to_value::timestamptz < tp.to_value::timestamptz)) AND - parent_table = table_name; + using current_range_from_value, current_range_to_value, table_name; - IF found THEN + IF manual_partition is not NULL THEN RAISE 'partition % with the range from % to % does not align with the initial partition given the partition interval', manual_partition::text, manual_partition_from_value_text, manual_partition_to_value_text - USING HINT = 'Only use partitions of the same size, without gaps between partitions.'; + USING HINT = 'Only use partitions of the same size, without gaps between partitions.'; END IF; IF partition_column_type = 'date'::regtype THEN @@ -194,7 +213,8 @@ BEGIN SELECT current_range_from_value::timestamptz::text INTO current_range_from_value_text; SELECT current_range_to_value::timestamptz::text INTO current_range_to_value_text; ELSE - RAISE 'type of the partition column of the table % must be date, timestamp or timestamptz', table_name; + EXECUTE format('SELECT $1%s::text', custom_cast) INTO current_range_from_value_text using current_range_from_value; + EXECUTE format('SELECT $1%s::text', custom_cast) INTO current_range_to_value_text using current_range_to_value; END IF; -- use range values within the name of partition to have unique partition names @@ -212,7 +232,7 @@ BEGIN END; $$; COMMENT ON FUNCTION pg_catalog.get_missing_time_partition_ranges( - table_name regclass, + table_name regclass, partition_interval INTERVAL, to_value timestamptz, from_value timestamptz) diff --git a/src/backend/distributed/utils/colocation_utils.c b/src/backend/distributed/utils/colocation_utils.c index dba791ba4..62a13af3a 100644 --- a/src/backend/distributed/utils/colocation_utils.c +++ b/src/backend/distributed/utils/colocation_utils.c @@ -116,7 +116,7 @@ update_distributed_table_colocation(PG_FUNCTION_ARGS) text *colocateWithTableNameText = PG_GETARG_TEXT_P(1); EnsureTableOwner(targetRelationId); - ErrorIfTenantTable(targetRelationId, "update_distributed_table_colocation"); + ErrorIfTenantTable(targetRelationId, TenantOperationNames[TENANT_UPDATE_COLOCATION]); char *colocateWithTableName = text_to_cstring(colocateWithTableNameText); if (IsColocateWithNone(colocateWithTableName)) @@ -127,7 +127,8 @@ update_distributed_table_colocation(PG_FUNCTION_ARGS) else { Oid colocateWithTableId = ResolveRelationId(colocateWithTableNameText, false); - ErrorIfTenantTable(colocateWithTableId, "colocate_with"); + ErrorIfTenantTable(colocateWithTableId, + TenantOperationNames[TENANT_COLOCATE_WITH]); EnsureTableOwner(colocateWithTableId); MarkTablesColocated(colocateWithTableId, targetRelationId); } diff --git a/src/include/distributed/citus_custom_scan.h b/src/include/distributed/citus_custom_scan.h index f31138ac2..a3da4958c 100644 --- a/src/include/distributed/citus_custom_scan.h +++ b/src/include/distributed/citus_custom_scan.h @@ -34,6 +34,7 @@ typedef struct CitusScanState extern CustomScanMethods AdaptiveExecutorCustomScanMethods; extern CustomScanMethods NonPushableInsertSelectCustomScanMethods; extern CustomScanMethods DelayedErrorCustomScanMethods; +extern CustomScanMethods NonPushableMergeCommandCustomScanMethods; extern void RegisterCitusCustomScanMethods(void); diff --git a/src/include/distributed/commands.h b/src/include/distributed/commands.h index 442e58058..a013f3977 100644 --- a/src/include/distributed/commands.h +++ b/src/include/distributed/commands.h @@ -118,7 +118,7 @@ typedef enum ExtractForeignKeyConstraintsMode /* exclude the self-referencing foreign keys */ EXCLUDE_SELF_REFERENCES = 1 << 2, - /* any combination of the 4 flags below is supported */ + /* any combination of the 5 flags below is supported */ /* include foreign keys when the other table is a distributed table*/ INCLUDE_DISTRIBUTED_TABLES = 1 << 3, @@ -131,9 +131,13 @@ typedef enum ExtractForeignKeyConstraintsMode /* include foreign keys when the other table is a Postgres local table*/ INCLUDE_LOCAL_TABLES = 1 << 6, + /* include foreign keys when the other table is a single shard table*/ + INCLUDE_SINGLE_SHARD_TABLES = 1 << 7, + /* include foreign keys regardless of the other table's type */ INCLUDE_ALL_TABLE_TYPES = INCLUDE_DISTRIBUTED_TABLES | INCLUDE_REFERENCE_TABLES | - INCLUDE_CITUS_LOCAL_TABLES | INCLUDE_LOCAL_TABLES + INCLUDE_CITUS_LOCAL_TABLES | INCLUDE_LOCAL_TABLES | + INCLUDE_SINGLE_SHARD_TABLES } ExtractForeignKeyConstraintMode; @@ -155,6 +159,19 @@ typedef enum SearchForeignKeyColumnFlags /* callers can also pass union of above flags */ } SearchForeignKeyColumnFlags; + +typedef enum TenantOperation +{ + TENANT_UNDISTRIBUTE_TABLE = 0, + TENANT_ALTER_TABLE, + TENANT_COLOCATE_WITH, + TENANT_UPDATE_COLOCATION, + TENANT_SET_SCHEMA, +} TenantOperation; + +#define TOTAL_TENANT_OPERATION 5 +extern const char *TenantOperationNames[TOTAL_TENANT_OPERATION]; + /* begin.c - forward declarations */ extern void SaveBeginCommandProperties(TransactionStmt *transactionStmt); @@ -593,6 +610,7 @@ extern char * GetAlterColumnWithNextvalDefaultCmd(Oid sequenceOid, Oid relationI extern void ErrorIfTableHasIdentityColumn(Oid relationId); extern void ConvertNewTableIfNecessary(Node *createStmt); +extern void ConvertToTenantTableIfNecessary(AlterObjectSchemaStmt *alterObjectSchemaStmt); /* text_search.c - forward declarations */ extern List * GetCreateTextSearchConfigStatements(const ObjectAddress *address); @@ -792,11 +810,11 @@ extern void UpdateAutoConvertedForConnectedRelations(List *relationId, bool extern bool ShouldUseSchemaBasedSharding(char *schemaName); extern bool ShouldCreateTenantSchemaTable(Oid relationId); extern bool IsTenantSchema(Oid schemaId); +extern void EnsureTenantTable(Oid relationId, char *operationName); extern void ErrorIfIllegalPartitioningInTenantSchema(Oid parentRelationId, Oid partitionRelationId); extern void CreateTenantSchemaTable(Oid relationId); -extern void ErrorIfTenantTable(Oid relationId, char *operationName); -extern void ErrorIfTenantSchema(Oid nspOid, char *operationName); +extern void ErrorIfTenantTable(Oid relationId, const char *operationName); extern uint32 CreateTenantSchemaColocationId(void); #endif /*CITUS_COMMANDS_H */ diff --git a/src/include/distributed/commands/multi_copy.h b/src/include/distributed/commands/multi_copy.h index 689725e70..70f93cfb9 100644 --- a/src/include/distributed/commands/multi_copy.h +++ b/src/include/distributed/commands/multi_copy.h @@ -152,6 +152,12 @@ typedef struct CitusCopyDestReceiver * upfront. */ uint64 appendShardId; + + /* + * When copying to intermediate files, we can skip coercions and run them + * when merging into the target tables. + */ + bool skipCoercions; } CitusCopyDestReceiver; diff --git a/src/include/distributed/insert_select_executor.h b/src/include/distributed/insert_select_executor.h index 1b08f5a94..fd8282014 100644 --- a/src/include/distributed/insert_select_executor.h +++ b/src/include/distributed/insert_select_executor.h @@ -18,6 +18,7 @@ extern TupleTableSlot * NonPushableInsertSelectExecScan(CustomScanState *node); - +extern List * BuildColumnNameListFromTargetList(Oid targetRelationId, + List *insertTargetList); #endif /* INSERT_SELECT_EXECUTOR_H */ diff --git a/src/include/distributed/insert_select_planner.h b/src/include/distributed/insert_select_planner.h index 74b8a0708..771d1d60f 100644 --- a/src/include/distributed/insert_select_planner.h +++ b/src/include/distributed/insert_select_planner.h @@ -44,6 +44,7 @@ extern DistributedPlan * CreateInsertSelectIntoLocalTablePlan(uint64 planId, plannerRestrictionContext); extern char * InsertSelectResultIdPrefix(uint64 planId); extern bool PlanningInsertSelect(void); +extern Query * WrapSubquery(Query *subquery); #endif /* INSERT_SELECT_PLANNER_H */ diff --git a/src/include/distributed/local_distributed_join_planner.h b/src/include/distributed/local_distributed_join_planner.h index f2108f603..dfb45f149 100644 --- a/src/include/distributed/local_distributed_join_planner.h +++ b/src/include/distributed/local_distributed_join_planner.h @@ -33,5 +33,6 @@ extern void RecursivelyPlanLocalTableJoins(Query *query, extern List * RequiredAttrNumbersForRelation(RangeTblEntry *relationRte, PlannerRestrictionContext * plannerRestrictionContext); +extern List * RequiredAttrNumbersForRelationInternal(Query *queryToProcess, int rteIndex); #endif /* LOCAL_DISTRIBUTED_JOIN_PLANNER_H */ diff --git a/src/include/distributed/merge_executor.h b/src/include/distributed/merge_executor.h new file mode 100644 index 000000000..0bc31ab74 --- /dev/null +++ b/src/include/distributed/merge_executor.h @@ -0,0 +1,17 @@ +/*------------------------------------------------------------------------- + * + * merge_executor.h + * + * Declarations for public functions and types related to executing + * MERGE INTO ... SQL commands. + * + * Copyright (c) Citus Data, Inc. + * + *------------------------------------------------------------------------- + */ +#ifndef MERGE_EXECUTOR_H +#define MERGE_EXECUTOR_H + +extern TupleTableSlot * NonPushableMergeCommandExecScan(CustomScanState *node); + +#endif /* MERGE_EXECUTOR_H */ diff --git a/src/include/distributed/merge_planner.h b/src/include/distributed/merge_planner.h index b4ec1852f..1548dae6a 100644 --- a/src/include/distributed/merge_planner.h +++ b/src/include/distributed/merge_planner.h @@ -19,16 +19,18 @@ #include "distributed/errormessage.h" #include "distributed/multi_physical_planner.h" -extern DeferredErrorMessage * MergeQuerySupported(Oid resultRelationId, - Query *originalQuery, - bool multiShardQuery, - PlannerRestrictionContext * - plannerRestrictionContext); -extern DistributedPlan * CreateMergePlan(Query *originalQuery, Query *query, +extern DistributedPlan * CreateMergePlan(uint64 planId, Query *originalQuery, + Query *query, PlannerRestrictionContext * - plannerRestrictionContext); + plannerRestrictionContext, + ParamListInfo boundParams); extern bool IsLocalTableModification(Oid targetRelationId, Query *query, uint64 shardId, RTEListProperties *rteProperties); +extern void NonPushableMergeCommandExplainScan(CustomScanState *node, List *ancestors, + struct ExplainState *es); +extern Var * FetchAndValidateInsertVarIfExists(Oid targetRelationId, Query *query); +extern RangeTblEntry * ExtractMergeSourceRangeTableEntry(Query *query); + #endif /* MERGE_PLANNER_H */ diff --git a/src/include/distributed/metadata_utility.h b/src/include/distributed/metadata_utility.h index 473d105e8..6536e89bc 100644 --- a/src/include/distributed/metadata_utility.h +++ b/src/include/distributed/metadata_utility.h @@ -172,6 +172,12 @@ typedef struct TableConversionParameters * messages that we explicitly issue */ bool suppressNoticeMessages; + + /* + * bypassTenantCheck skips tenant table checks to allow some internal + * operations which are normally disallowed + */ + bool bypassTenantCheck; } TableConversionParameters; typedef struct TableConversionReturn @@ -363,6 +369,7 @@ extern void CreateDistributedTable(Oid relationId, char *distributionColumnName, bool shardCountIsStrict, char *colocateWithTableName); extern void CreateReferenceTable(Oid relationId); extern void CreateTruncateTrigger(Oid relationId); +extern void EnsureUndistributeTenantTableSafe(Oid relationId, const char *operationName); extern TableConversionReturn * UndistributeTable(TableConversionParameters *params); extern void UndistributeTables(List *relationIdList); diff --git a/src/include/distributed/multi_executor.h b/src/include/distributed/multi_executor.h index 4e7f13601..5ae010d87 100644 --- a/src/include/distributed/multi_executor.h +++ b/src/include/distributed/multi_executor.h @@ -114,6 +114,9 @@ typedef struct ExecutionParams /* isUtilityCommand is true if the current execution is for a utility * command such as a DDL command.*/ bool isUtilityCommand; + + /* pass bind parameters to the distributed executor for parameterized plans */ + ParamListInfo paramListInfo; } ExecutionParams; ExecutionParams * CreateBasicExecutionParams(RowModifyLevel modLevel, @@ -122,6 +125,11 @@ ExecutionParams * CreateBasicExecutionParams(RowModifyLevel modLevel, bool localExecutionSupported); extern uint64 ExecuteTaskListExtended(ExecutionParams *executionParams); +extern uint64 ExecuteTaskListIntoTupleDestWithParam(RowModifyLevel modLevel, + List *taskList, + TupleDestination *tupleDest, + bool expectResults, + ParamListInfo paramListInfo); extern uint64 ExecuteTaskListIntoTupleDest(RowModifyLevel modLevel, List *taskList, TupleDestination *tupleDest, bool expectResults); diff --git a/src/include/distributed/multi_join_order.h b/src/include/distributed/multi_join_order.h index 92d1edaf2..4e4ba1dd2 100644 --- a/src/include/distributed/multi_join_order.h +++ b/src/include/distributed/multi_join_order.h @@ -108,6 +108,8 @@ extern Var * DistPartitionKey(Oid relationId); extern Var * DistPartitionKeyOrError(Oid relationId); extern char PartitionMethod(Oid relationId); extern char TableReplicationModel(Oid relationId); +extern bool JoinOnColumns(List *currentPartitionColumnList, Var *candidatePartitionColumn, + List *joinClauseList); #endif /* MULTI_JOIN_ORDER_H */ diff --git a/src/include/distributed/multi_physical_planner.h b/src/include/distributed/multi_physical_planner.h index c457918db..b7acc0574 100644 --- a/src/include/distributed/multi_physical_planner.h +++ b/src/include/distributed/multi_physical_planner.h @@ -463,6 +463,13 @@ typedef struct DistributedPlan * or if prepared statement parameters prevented successful planning. */ DeferredErrorMessage *planningError; + + /* + * When performing query execution scenarios that require repartitioning + * the source rows, this field stores the index of the column in the list + * of source rows to be repartitioned for colocation with the target. + */ + int sourceResultRepartitionColumnIndex; } DistributedPlan; diff --git a/src/include/distributed/multi_router_planner.h b/src/include/distributed/multi_router_planner.h index a255fd520..160cf6605 100644 --- a/src/include/distributed/multi_router_planner.h +++ b/src/include/distributed/multi_router_planner.h @@ -118,5 +118,6 @@ extern Job * RouterJob(Query *originalQuery, PlannerRestrictionContext *plannerRestrictionContext, DeferredErrorMessage **planningError); extern bool ContainsOnlyLocalTables(RTEListProperties *rteProperties); +extern RangeTblEntry * ExtractSourceResultRangeTableEntry(Query *query); #endif /* MULTI_ROUTER_PLANNER_H */ diff --git a/src/include/distributed/multi_server_executor.h b/src/include/distributed/multi_server_executor.h index 56dd6d808..f49ef60c2 100644 --- a/src/include/distributed/multi_server_executor.h +++ b/src/include/distributed/multi_server_executor.h @@ -29,7 +29,8 @@ typedef enum { MULTI_EXECUTOR_INVALID_FIRST = 0, MULTI_EXECUTOR_ADAPTIVE = 1, - MULTI_EXECUTOR_NON_PUSHABLE_INSERT_SELECT = 2 + MULTI_EXECUTOR_NON_PUSHABLE_INSERT_SELECT = 2, + MULTI_EXECUTOR_NON_PUSHABLE_MERGE_QUERY = 3 } MultiExecutorType; diff --git a/src/include/distributed/recursive_planning.h b/src/include/distributed/recursive_planning.h index e849d7158..8943443aa 100644 --- a/src/include/distributed/recursive_planning.h +++ b/src/include/distributed/recursive_planning.h @@ -46,6 +46,7 @@ extern void ReplaceRTERelationWithRteSubquery(RangeTblEntry *rangeTableEntry, extern bool IsRecursivelyPlannableRelation(RangeTblEntry *rangeTableEntry); extern bool IsRelationLocalTableOrMatView(Oid relationId); extern bool ContainsReferencesToOuterQuery(Query *query); +extern void UpdateVarNosInNode(Node *node, Index newVarNo); #endif /* RECURSIVE_PLANNING_H */ diff --git a/src/include/distributed/repartition_executor.h b/src/include/distributed/repartition_executor.h index 98173b828..de4ad122a 100644 --- a/src/include/distributed/repartition_executor.h +++ b/src/include/distributed/repartition_executor.h @@ -15,7 +15,7 @@ extern bool EnableRepartitionedInsertSelect; -extern int DistributionColumnIndex(List *insertTargetList, Var *partitionColumn); +extern int DistributionColumnIndex(List *insertTargetList, Var *distributionColumn); extern List * GenerateTaskListWithColocatedIntermediateResults(Oid targetRelationId, Query * modifyQueryViaCoordinatorOrRepartition, diff --git a/src/test/regress/bin/normalize.sed b/src/test/regress/bin/normalize.sed index 5b958b636..0370f4e98 100644 --- a/src/test/regress/bin/normalize.sed +++ b/src/test/regress/bin/normalize.sed @@ -159,6 +159,8 @@ s/Subplan [0-9]+\_/Subplan XXX\_/g # Plan numbers in insert select s/read_intermediate_result\('insert_select_[0-9]+_/read_intermediate_result('insert_select_XXX_/g +# Plan numbers in merge into +s/read_intermediate_result\('merge_into_[0-9]+_/read_intermediate_result('merge_into_XXX_/g # ignore job id in repartitioned insert/select s/repartitioned_results_[0-9]+/repartitioned_results_xxxxx/g diff --git a/src/test/regress/citus_tests/config.py b/src/test/regress/citus_tests/config.py index 16b18d1e7..560806962 100644 --- a/src/test/regress/citus_tests/config.py +++ b/src/test/regress/citus_tests/config.py @@ -220,6 +220,7 @@ class AllSingleShardTableDefaultConfig(CitusDefaultClusterConfig): # "dist_query_single_shard" table acts differently when the table # has a single shard. This is explained with a comment in the test. "nested_execution", + "merge_arbitrary", ] diff --git a/src/test/regress/expected/citus_schema_distribute_undistribute.out b/src/test/regress/expected/citus_schema_distribute_undistribute.out index 8e6803b6f..ae08b6c6a 100644 --- a/src/test/regress/expected/citus_schema_distribute_undistribute.out +++ b/src/test/regress/expected/citus_schema_distribute_undistribute.out @@ -482,8 +482,8 @@ SELECT create_distributed_table('tenant1.dist', 'id'); (1 row) SELECT citus_schema_distribute('tenant1'); -ERROR: schema already has distributed tables -HINT: Undistribute distributed tables under the schema before distributing the schema. +ERROR: distributed schema cannot have distributed tables +HINT: Undistribute distributed tables before 'citus_schema_distribute'. SELECT undistribute_table('tenant1.dist'); undistribute_table --------------------------------------------------------------------- @@ -510,8 +510,8 @@ SELECT create_reference_table('tenant1.ref2'); (1 row) SELECT citus_schema_distribute('tenant1'); -ERROR: schema already has distributed tables -HINT: Undistribute distributed tables under the schema before distributing the schema. +ERROR: distributed schema cannot have distributed tables +HINT: Undistribute distributed tables before 'citus_schema_distribute'. SELECT undistribute_table('tenant1.ref2'); undistribute_table --------------------------------------------------------------------- @@ -766,8 +766,8 @@ SELECT create_distributed_table('tenant1.new_dist', 'id'); (1 row) SELECT citus_schema_distribute('tenant1'); -ERROR: schema already has distributed tables -HINT: Undistribute distributed tables under the schema before distributing the schema. +ERROR: distributed schema cannot have distributed tables +HINT: Undistribute distributed tables before 'citus_schema_distribute'. SELECT undistribute_table('tenant1.new_dist'); undistribute_table --------------------------------------------------------------------- @@ -795,8 +795,8 @@ SELECT create_distributed_table('tenant1.single_shard_t', NULL); (1 row) SELECT citus_schema_distribute('tenant1'); -ERROR: schema already has distributed tables -HINT: Undistribute distributed tables under the schema before distributing the schema. +ERROR: distributed schema cannot have distributed tables +HINT: Undistribute distributed tables before 'citus_schema_distribute'. SELECT undistribute_table('tenant1.single_shard_t'); undistribute_table --------------------------------------------------------------------- diff --git a/src/test/regress/expected/merge.out b/src/test/regress/expected/merge.out index 85d6daab6..882a22091 100644 --- a/src/test/regress/expected/merge.out +++ b/src/test/regress/expected/merge.out @@ -20,6 +20,14 @@ SET citus.next_shard_id TO 4000000; SET citus.explain_all_tasks TO true; SET citus.shard_replication_factor TO 1; SET citus.max_adaptive_executor_pool_size TO 1; +SET client_min_messages = warning; +SELECT 1 FROM master_add_node('localhost', :master_port, groupid => 0); + ?column? +--------------------------------------------------------------------- + 1 +(1 row) + +RESET client_min_messages; CREATE TABLE source ( order_id INT, @@ -199,7 +207,7 @@ HINT: To remove the local data, run: SELECT truncate_local_data_after_distribut (1 row) -SELECT create_distributed_table('source', 'customer_id'); +SELECT create_distributed_table('source', 'customer_id', colocate_with=>'target'); NOTICE: Copying data from local table... NOTICE: copying the data has completed DETAIL: The local data in the table is no longer visible, but is still on disk. @@ -433,7 +441,7 @@ HINT: To remove the local data, run: SELECT truncate_local_data_after_distribut (1 row) -SELECT create_distributed_table('s1', 'id'); +SELECT create_distributed_table('s1', 'id', colocate_with=>'t1'); NOTICE: Copying data from local table... NOTICE: copying the data has completed DETAIL: The local data in the table is no longer visible, but is still on disk. @@ -643,7 +651,7 @@ HINT: To remove the local data, run: SELECT truncate_local_data_after_distribut (1 row) -SELECT create_distributed_table('s2', 'id'); +SELECT create_distributed_table('s2', 'id', colocate_with => 't2'); NOTICE: Copying data from local table... NOTICE: copying the data has completed DETAIL: The local data in the table is no longer visible, but is still on disk. @@ -1397,7 +1405,7 @@ HINT: To remove the local data, run: SELECT truncate_local_data_after_distribut (1 row) -SELECT create_distributed_table('source_cj1', 'sid1'); +SELECT create_distributed_table('source_cj1', 'sid1', colocate_with => 'target_cj'); NOTICE: Copying data from local table... NOTICE: copying the data has completed DETAIL: The local data in the table is no longer visible, but is still on disk. @@ -1407,7 +1415,7 @@ HINT: To remove the local data, run: SELECT truncate_local_data_after_distribut (1 row) -SELECT create_distributed_table('source_cj2', 'sid2'); +SELECT create_distributed_table('source_cj2', 'sid2', colocate_with => 'target_cj'); NOTICE: Copying data from local table... NOTICE: copying the data has completed DETAIL: The local data in the table is no longer visible, but is still on disk. @@ -1418,19 +1426,13 @@ HINT: To remove the local data, run: SELECT truncate_local_data_after_distribut (1 row) BEGIN; -SET citus.log_remote_commands to true; MERGE INTO target_cj t -USING source_cj1 s1 INNER JOIN source_cj2 s2 ON sid1 = sid2 +USING (SELECT * FROM source_cj1 s1 INNER JOIN source_cj2 s2 ON sid1 = sid2) s ON t.tid = sid1 AND t.tid = 2 WHEN MATCHED THEN UPDATE SET src = src2 WHEN NOT MATCHED THEN DO NOTHING; -NOTICE: issuing BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;SELECT assign_distributed_transaction_id(xx, xx, 'xxxxxxx'); -DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx -NOTICE: issuing MERGE INTO merge_schema.target_cj_xxxxxxx t USING (merge_schema.source_cj1_xxxxxxx s1 JOIN merge_schema.source_cj2_xxxxxxx s2 ON ((s1.sid1 OPERATOR(pg_catalog.=) s2.sid2))) ON ((t.tid OPERATOR(pg_catalog.=) s1.sid1) AND (t.tid OPERATOR(pg_catalog.=) 2)) WHEN MATCHED THEN UPDATE SET src = s2.src2 WHEN NOT MATCHED THEN DO NOTHING -DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx -SET citus.log_remote_commands to false; SELECT * FROM target_cj ORDER BY 1; tid | src | val --------------------------------------------------------------------- @@ -1444,8 +1446,8 @@ ROLLBACK; BEGIN; -- try accessing columns from either side of the source join MERGE INTO target_cj t -USING source_cj1 s2 - INNER JOIN source_cj2 s1 ON sid1 = sid2 AND val1 = 10 +USING (SELECT * FROM source_cj1 s2 + INNER JOIN source_cj2 s1 ON sid1 = sid2 AND val1 = 10) s ON t.tid = sid1 AND t.tid = 2 WHEN MATCHED THEN UPDATE SET src = src1, val = val2 @@ -1513,7 +1515,7 @@ SELECT * FROM target_cj ORDER BY 1; ROLLBACK; -- Test PREPARE -PREPARE foo(int) AS +PREPARE merge_prepare(int) AS MERGE INTO target_cj target USING (SELECT * FROM source_cj1) sub ON target.tid = sub.sid1 AND target.tid = $1 @@ -1531,11 +1533,11 @@ SELECT * FROM target_cj ORDER BY 1; (4 rows) BEGIN; -EXECUTE foo(2); -EXECUTE foo(2); -EXECUTE foo(2); -EXECUTE foo(2); -EXECUTE foo(2); +EXECUTE merge_prepare(2); +EXECUTE merge_prepare(2); +EXECUTE merge_prepare(2); +EXECUTE merge_prepare(2); +EXECUTE merge_prepare(2); SELECT * FROM target_cj ORDER BY 1; tid | src | val --------------------------------------------------------------------- @@ -1549,12 +1551,7 @@ ROLLBACK; BEGIN; SET citus.log_remote_commands to true; SET client_min_messages TO DEBUG1; -EXECUTE foo(2); -DEBUG: -DEBUG: -DEBUG: -DEBUG: -DEBUG: Creating MERGE router plan +EXECUTE merge_prepare(2); DEBUG: DEBUG: Creating MERGE router plan NOTICE: issuing BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;SELECT assign_distributed_transaction_id(xx, xx, 'xxxxxxx'); @@ -1562,7 +1559,7 @@ DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx NOTICE: issuing MERGE INTO merge_schema.target_cj_xxxxxxx target USING (SELECT source_cj1.sid1, source_cj1.src1, source_cj1.val1 FROM merge_schema.source_cj1_xxxxxxx source_cj1) sub ON ((target.tid OPERATOR(pg_catalog.=) sub.sid1) AND (target.tid OPERATOR(pg_catalog.=) $1)) WHEN MATCHED THEN UPDATE SET val = sub.val1 WHEN NOT MATCHED THEN DO NOTHING DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx RESET client_min_messages; -EXECUTE foo(2); +EXECUTE merge_prepare(2); NOTICE: issuing MERGE INTO merge_schema.target_cj_xxxxxxx target USING (SELECT source_cj1.sid1, source_cj1.src1, source_cj1.val1 FROM merge_schema.source_cj1_xxxxxxx source_cj1) sub ON ((target.tid OPERATOR(pg_catalog.=) sub.sid1) AND (target.tid OPERATOR(pg_catalog.=) $1)) WHEN MATCHED THEN UPDATE SET val = sub.val1 WHEN NOT MATCHED THEN DO NOTHING DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx SET citus.log_remote_commands to false; @@ -1602,7 +1599,7 @@ HINT: To remove the local data, run: SELECT truncate_local_data_after_distribut (1 row) -SELECT create_distributed_table('citus_source', 'id'); +SELECT create_distributed_table('citus_source', 'id', colocate_with => 'citus_target'); NOTICE: Copying data from local table... NOTICE: copying the data has completed DETAIL: The local data in the table is no longer visible, but is still on disk. @@ -2519,9 +2516,495 @@ WHERE pg_result.t1 IS NULL OR local_ref.t1 IS NULL; 0 (1 row) +-- Now make target as distributed, keep reference as source +TRUNCATE reftarget_local; +TRUNCATE refsource_ref; +INSERT INTO reftarget_local VALUES(1, 0); +INSERT INTO reftarget_local VALUES(3, 100); +INSERT INTO refsource_ref VALUES(1, 1); +INSERT INTO refsource_ref VALUES(2, 2); +INSERT INTO refsource_ref VALUES(3, 3); +SELECT create_distributed_table('reftarget_local', 't1'); +NOTICE: Copying data from local table... +NOTICE: copying the data has completed +DETAIL: The local data in the table is no longer visible, but is still on disk. +HINT: To remove the local data, run: SELECT truncate_local_data_after_distributing_table($$merge_schema.reftarget_local$$) + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +MERGE INTO reftarget_local +USING (SELECT * FROM refsource_ref UNION SELECT * FROM refsource_ref) AS foo ON reftarget_local.t1 = foo.s1 +WHEN MATCHED AND reftarget_local.t2 = 100 THEN + DELETE +WHEN MATCHED THEN + UPDATE SET t2 = t2 + 100 +WHEN NOT MATCHED THEN + INSERT VALUES(foo.s1); +SELECT * INTO dist_reftarget FROM reftarget_local ORDER BY 1, 2; +-- Should be equal +SELECT c.*, p.* +FROM dist_reftarget c, pg_result p +WHERE c.t1 = p.t1 +ORDER BY 1,2; + t1 | t2 | t1 | t2 +--------------------------------------------------------------------- + 1 | 100 | 1 | 100 + 2 | | 2 | +(2 rows) + +-- Must return zero rows +SELECT count(*) +FROM pg_result FULL OUTER JOIN dist_reftarget ON pg_result.t1 = dist_reftarget.t1 +WHERE pg_result.t1 IS NULL OR dist_reftarget.t1 IS NULL; + count +--------------------------------------------------------------------- + 0 +(1 row) + +-- +-- Distributed (target), Reference(source) +-- +CREATE TABLE demo_distributed(id1 int, val1 int); +CREATE TABLE demo_source_table(id2 int, val2 int); +CREATE FUNCTION setup_demo_data() RETURNS VOID AS $$ +INSERT INTO demo_distributed VALUES(1, 100); +INSERT INTO demo_distributed VALUES(7, 100); +INSERT INTO demo_distributed VALUES(15, 100); +INSERT INTO demo_distributed VALUES(100, 0); +INSERT INTO demo_distributed VALUES(300, 100); +INSERT INTO demo_distributed VALUES(400, 0); + +INSERT INTO demo_source_table VALUES(1, 77); +INSERT INTO demo_source_table VALUES(15, 77); +INSERT INTO demo_source_table VALUES(75, 77); +INSERT INTO demo_source_table VALUES(100, 77); +INSERT INTO demo_source_table VALUES(300, 77); +INSERT INTO demo_source_table VALUES(400, 77); +INSERT INTO demo_source_table VALUES(500, 77); +$$ +LANGUAGE SQL; +CREATE FUNCTION merge_demo_data() RETURNS VOID AS $$ +MERGE INTO demo_distributed t +USING demo_source_table s ON s.id2 = t.id1 +WHEN MATCHED AND t.val1= 0 THEN + DELETE +WHEN MATCHED THEN + UPDATE SET val1 = val1 + s.val2 +WHEN NOT MATCHED THEN + INSERT VALUES(s.id2, s.val2); +$$ +LANGUAGE SQL; +SELECT setup_demo_data(); + setup_demo_data +--------------------------------------------------------------------- + +(1 row) + +SELECT merge_demo_data(); + merge_demo_data +--------------------------------------------------------------------- + +(1 row) + +SELECT * INTO pg_demo_result FROM demo_distributed ORDER BY 1, 2; +TRUNCATE demo_distributed; +TRUNCATE demo_source_table; +SELECT create_distributed_table('demo_distributed', 'id1'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +SELECT create_reference_table('demo_source_table'); + create_reference_table +--------------------------------------------------------------------- + +(1 row) + +SELECT setup_demo_data(); + setup_demo_data +--------------------------------------------------------------------- + +(1 row) + +SELECT merge_demo_data(); + merge_demo_data +--------------------------------------------------------------------- + +(1 row) + +SELECT * INTO dist_demo_result FROM demo_distributed ORDER BY 1, 2; +-- Should be equal +SELECT c.*, p.* +FROM dist_demo_result c, pg_demo_result p +WHERE c.id1 = p.id1 +ORDER BY 1,2; + id1 | val1 | id1 | val1 +--------------------------------------------------------------------- + 1 | 177 | 1 | 177 + 7 | 100 | 7 | 100 + 15 | 177 | 15 | 177 + 75 | 77 | 75 | 77 + 300 | 177 | 300 | 177 + 500 | 77 | 500 | 77 +(6 rows) + +-- Must return zero rows +SELECT count(*) +FROM pg_demo_result p FULL OUTER JOIN dist_demo_result d ON p.id1 = d.id1 +WHERE p.id1 IS NULL OR d.id1 IS NULL; + count +--------------------------------------------------------------------- + 0 +(1 row) + +-- Now convert source as distributed, but non-colocated with target +DROP TABLE pg_demo_result, dist_demo_result; +SELECT undistribute_table('demo_distributed'); +NOTICE: creating a new table for merge_schema.demo_distributed +NOTICE: moving the data of merge_schema.demo_distributed +NOTICE: dropping the old merge_schema.demo_distributed +NOTICE: renaming the new table to merge_schema.demo_distributed + undistribute_table +--------------------------------------------------------------------- + +(1 row) + +SELECT undistribute_table('demo_source_table'); +NOTICE: creating a new table for merge_schema.demo_source_table +NOTICE: moving the data of merge_schema.demo_source_table +NOTICE: dropping the old merge_schema.demo_source_table +NOTICE: renaming the new table to merge_schema.demo_source_table + undistribute_table +--------------------------------------------------------------------- + +(1 row) + +CREATE OR REPLACE FUNCTION merge_demo_data() RETURNS VOID AS $$ +MERGE INTO demo_distributed t +USING (SELECT id2,val2 FROM demo_source_table UNION SELECT val2,id2 FROM demo_source_table) AS s +ON t.id1 = s.id2 +WHEN MATCHED THEN + UPDATE SET val1 = val1 + 1; +$$ +LANGUAGE SQL; +TRUNCATE demo_distributed; +TRUNCATE demo_source_table; +SELECT setup_demo_data(); + setup_demo_data +--------------------------------------------------------------------- + +(1 row) + +SELECT merge_demo_data(); + merge_demo_data +--------------------------------------------------------------------- + +(1 row) + +SELECT * INTO pg_demo_result FROM demo_distributed ORDER BY 1, 2; +SELECT create_distributed_table('demo_distributed', 'id1'); +NOTICE: Copying data from local table... +NOTICE: copying the data has completed +DETAIL: The local data in the table is no longer visible, but is still on disk. +HINT: To remove the local data, run: SELECT truncate_local_data_after_distributing_table($$merge_schema.demo_distributed$$) + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +SELECT create_distributed_table('demo_source_table', 'id2', colocate_with=>'none'); +NOTICE: Copying data from local table... +NOTICE: copying the data has completed +DETAIL: The local data in the table is no longer visible, but is still on disk. +HINT: To remove the local data, run: SELECT truncate_local_data_after_distributing_table($$merge_schema.demo_source_table$$) + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +TRUNCATE demo_distributed; +TRUNCATE demo_source_table; +SELECT setup_demo_data(); + setup_demo_data +--------------------------------------------------------------------- + +(1 row) + +SELECT merge_demo_data(); + merge_demo_data +--------------------------------------------------------------------- + +(1 row) + +SELECT * INTO dist_demo_result FROM demo_distributed ORDER BY 1, 2; +-- Should be equal +SELECT c.*, p.* +FROM dist_demo_result c, pg_demo_result p +WHERE c.id1 = p.id1 +ORDER BY 1,2; + id1 | val1 | id1 | val1 +--------------------------------------------------------------------- + 1 | 101 | 1 | 101 + 7 | 100 | 7 | 100 + 15 | 101 | 15 | 101 + 100 | 1 | 100 | 1 + 300 | 101 | 300 | 101 + 400 | 1 | 400 | 1 +(6 rows) + +-- Must return zero rows +SELECT count(*) +FROM pg_demo_result p FULL OUTER JOIN dist_demo_result d ON p.id1 = d.id1 +WHERE p.id1 IS NULL OR d.id1 IS NULL; + count +--------------------------------------------------------------------- + 0 +(1 row) + +-- Test with LIMIT +CREATE OR REPLACE FUNCTION merge_demo_data() RETURNS VOID AS $$ +MERGE INTO demo_distributed t +USING (SELECT 999 as s3, demo_source_table.* FROM (SELECT * FROM demo_source_table ORDER BY 1 LIMIT 3) as foo LEFT JOIN demo_source_table USING(id2)) AS s +ON t.id1 = s.id2 +WHEN MATCHED THEN + UPDATE SET val1 = s3 +WHEN NOT MATCHED THEN + INSERT VALUES(id2, s3); +$$ +LANGUAGE SQL; +DROP TABLE pg_demo_result, dist_demo_result; +SELECT undistribute_table('demo_distributed'); +NOTICE: creating a new table for merge_schema.demo_distributed +NOTICE: moving the data of merge_schema.demo_distributed +NOTICE: dropping the old merge_schema.demo_distributed +NOTICE: renaming the new table to merge_schema.demo_distributed + undistribute_table +--------------------------------------------------------------------- + +(1 row) + +SELECT undistribute_table('demo_source_table'); +NOTICE: creating a new table for merge_schema.demo_source_table +NOTICE: moving the data of merge_schema.demo_source_table +NOTICE: dropping the old merge_schema.demo_source_table +NOTICE: renaming the new table to merge_schema.demo_source_table + undistribute_table +--------------------------------------------------------------------- + +(1 row) + +TRUNCATE demo_distributed; +TRUNCATE demo_source_table; +SELECT setup_demo_data(); + setup_demo_data +--------------------------------------------------------------------- + +(1 row) + +SELECT merge_demo_data(); + merge_demo_data +--------------------------------------------------------------------- + +(1 row) + +SELECT * INTO pg_demo_result FROM demo_distributed ORDER BY 1, 2; +SELECT create_distributed_table('demo_distributed', 'id1'); +NOTICE: Copying data from local table... +NOTICE: copying the data has completed +DETAIL: The local data in the table is no longer visible, but is still on disk. +HINT: To remove the local data, run: SELECT truncate_local_data_after_distributing_table($$merge_schema.demo_distributed$$) + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +SELECT create_distributed_table('demo_source_table', 'id2', colocate_with=>'none'); +NOTICE: Copying data from local table... +NOTICE: copying the data has completed +DETAIL: The local data in the table is no longer visible, but is still on disk. +HINT: To remove the local data, run: SELECT truncate_local_data_after_distributing_table($$merge_schema.demo_source_table$$) + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +TRUNCATE demo_distributed; +TRUNCATE demo_source_table; +SELECT setup_demo_data(); + setup_demo_data +--------------------------------------------------------------------- + +(1 row) + +SELECT merge_demo_data(); + merge_demo_data +--------------------------------------------------------------------- + +(1 row) + +SELECT * INTO dist_demo_result FROM demo_distributed ORDER BY 1, 2; +-- Should be equal +SELECT c.*, p.* +FROM dist_demo_result c, pg_demo_result p +WHERE c.id1 = p.id1 +ORDER BY 1,2; + id1 | val1 | id1 | val1 +--------------------------------------------------------------------- + 1 | 999 | 1 | 999 + 7 | 100 | 7 | 100 + 15 | 999 | 15 | 999 + 75 | 999 | 75 | 999 + 100 | 0 | 100 | 0 + 300 | 100 | 300 | 100 + 400 | 0 | 400 | 0 +(7 rows) + +-- Must return zero rows +SELECT count(*) +FROM pg_demo_result p FULL OUTER JOIN dist_demo_result d ON p.id1 = d.id1 +WHERE p.id1 IS NULL OR d.id1 IS NULL; + count +--------------------------------------------------------------------- + 0 +(1 row) + +-- Test explain with repartition +SET citus.explain_all_tasks TO false; +EXPLAIN (COSTS OFF) +MERGE INTO demo_distributed t +USING (SELECT 999 as s3, demo_source_table.* FROM (SELECT * FROM demo_source_table ORDER BY 1 LIMIT 3) as foo LEFT JOIN demo_source_table USING(id2)) AS s +ON t.id1 = s.id2 +WHEN MATCHED THEN + UPDATE SET val1 = s3 +WHEN NOT MATCHED THEN + INSERT VALUES(id2, s3); + QUERY PLAN +--------------------------------------------------------------------- + Custom Scan (Citus MERGE INTO ...) + MERGE INTO demo_distributed method: pull to coordinator + -> Custom Scan (Citus Adaptive) + -> Distributed Subplan XXX_1 + -> Limit + -> Sort + Sort Key: remote_scan.id2 + -> Custom Scan (Citus Adaptive) + Task Count: 4 + Tasks Shown: One of 4 + -> Task + Node: host=localhost port=xxxxx dbname=regression + -> Limit + -> Sort + Sort Key: id2 + -> Seq Scan on demo_source_table_4000135 demo_source_table + -> Distributed Subplan XXX_2 + -> Custom Scan (Citus Adaptive) + Task Count: 4 + Tasks Shown: One of 4 + -> Task + Node: host=localhost port=xxxxx dbname=regression + -> Seq Scan on demo_source_table_4000135 demo_source_table + Task Count: 1 + Tasks Shown: All + -> Task + Node: host=localhost port=xxxxx dbname=regression + -> Merge Left Join + Merge Cond: (intermediate_result.id2 = intermediate_result_1.id2) + -> Sort + Sort Key: intermediate_result.id2 + -> Function Scan on read_intermediate_result intermediate_result + -> Sort + Sort Key: intermediate_result_1.id2 + -> Function Scan on read_intermediate_result intermediate_result_1 +(35 rows) + +-- Test multiple join conditions on distribution column +MERGE INTO demo_distributed t +USING (SELECT id2+1 as key, id2+3 as key2 FROM demo_source_table) s +ON t.id1 = s.key2 ANd t.id1 = s.key +WHEN NOT MATCHED THEN + INSERT VALUES(s.key2, 333); +MERGE INTO demo_distributed t +USING (SELECT id2+1 as key, id2+2 as key2 FROM demo_source_table) s +ON t.id1 = s.key2 AND t.id1 = s.key +WHEN NOT MATCHED THEN + DO NOTHING; +MERGE INTO demo_distributed t +USING (SELECT id2+1 as key, id2+3 as key2 FROM demo_source_table) s +ON t.val1 = s.key2 AND t.id1 = s.key AND t.id1 = s.key2 +WHEN NOT MATCHED THEN + INSERT VALUES(s.key2, 444); +-- Test aggregate functions in source-query +SELECT COUNT(*) FROM demo_distributed where val1 = 150; + count +--------------------------------------------------------------------- + 0 +(1 row) + +SELECT COUNT(*) FROM demo_distributed where id1 = 2; + count +--------------------------------------------------------------------- + 0 +(1 row) + +-- One row with Key=7 updated in demo_distributed to 150 +MERGE INTO demo_distributed t +USING (SELECT count(DISTINCT id2)::int4 as key FROM demo_source_table GROUP BY val2) s +ON t.id1 = s.key +WHEN NOT MATCHED THEN INSERT VALUES(s.key, 1) +WHEN MATCHED THEN UPDATE SET val1 = 150; +-- Seven rows with Key=2 inserted in demo_distributed +MERGE INTO demo_distributed t +USING (SELECT (count(DISTINCT val2) + 1)::int4 as key FROM demo_source_table GROUP BY id2) s +ON t.id1 = s.key +WHEN NOT MATCHED THEN INSERT VALUES(s.key, 1) +WHEN MATCHED THEN UPDATE SET val1 = 150; +SELECT COUNT(*) FROM demo_distributed where val1 = 150; + count +--------------------------------------------------------------------- + 1 +(1 row) + +SELECT COUNT(*) FROM demo_distributed where id1 = 2; + count +--------------------------------------------------------------------- + 7 +(1 row) + -- -- Error and Unsupported scenarios -- +-- Test explain analyze with repartition +EXPLAIN ANALYZE +MERGE INTO demo_distributed t +USING (SELECT 999 as s3, demo_source_table.* FROM (SELECT * FROM demo_source_table ORDER BY 1 LIMIT 3) as foo LEFT JOIN demo_source_table USING(id2)) AS s +ON t.id1 = s.id2 +WHEN MATCHED THEN + UPDATE SET val1 = s3 +WHEN NOT MATCHED THEN + INSERT VALUES(id2, s3); +ERROR: EXPLAIN ANALYZE is currently not supported for MERGE INTO ... commands with repartitioning +-- Source without a table +MERGE INTO target_cj t +USING (VALUES (1, 1), (2, 1), (3, 3)) as s (sid, val) +ON t.tid = s.sid AND t.tid = 2 +WHEN MATCHED THEN + UPDATE SET val = s.val +WHEN NOT MATCHED THEN + DO NOTHING; +ERROR: To MERGE into a distributed table, source must be Citus table(s) +-- Incomplete source +MERGE INTO target_cj t +USING (source_cj1 s1 INNER JOIN source_cj2 s2 ON sid1 = val2) s +ON t.tid = s.sid1 AND t.tid = 2 +WHEN MATCHED THEN + UPDATE SET src = src2 +WHEN NOT MATCHED THEN + DO NOTHING; +ERROR: Source is not an explicit query +HINT: Source query is a Join expression, try converting into a query as SELECT * FROM (..Join..) -- Reference as a target and local as source MERGE INTO refsource_ref USING (SELECT * FROM reftarget_local UNION SELECT * FROM reftarget_local) AS foo ON refsource_ref.s1 = foo.t1 @@ -2530,40 +3013,20 @@ WHEN MATCHED THEN WHEN NOT MATCHED THEN INSERT VALUES(foo.t1); ERROR: Reference table as target is not allowed in MERGE command --- Reference as a source and distributed as target -MERGE INTO target_set t -USING refsource_ref AS s ON t.t1 = s.s1 -WHEN MATCHED THEN - DO NOTHING; -ERROR: MERGE command is not supported with combination of distributed/reference yet -HINT: If target is distributed, source must be distributed and co-located MERGE INTO target_set USING source_set AS foo ON target_set.t1 = foo.s1 WHEN MATCHED THEN UPDATE SET ctid = '(0,100)'; ERROR: cannot assign to system column "ctid" -MERGE INTO target_set -USING (SELECT s1,s2 FROM source_set UNION SELECT s2,s1 FROM source_set) AS foo ON target_set.t1 = foo.s1 -WHEN MATCHED THEN - UPDATE SET t2 = t2 + 1; -ERROR: cannot pushdown the subquery since not all subqueries in the UNION have the partition column in the same position -DETAIL: Each leaf query of the UNION should return the partition column in the same position and all joins must be on the partition column -MERGE INTO target_set -USING (SELECT 2 as s3, source_set.* FROM (SELECT * FROM source_set LIMIT 1) as foo LEFT JOIN source_set USING( s1)) AS foo -ON target_set.t1 = foo.s1 -WHEN MATCHED THEN UPDATE SET t2 = t2 + 1 -WHEN NOT MATCHED THEN INSERT VALUES(s1, s3); -ERROR: cannot push down this subquery -DETAIL: Limit clause is currently unsupported when a subquery references a column from another query -- modifying CTE not supported EXPLAIN -WITH cte_1 AS (DELETE FROM target_json) +WITH cte_1 AS (DELETE FROM target_json RETURNING *) MERGE INTO target_json sda -USING source_json sdn +USING cte_1 sdn ON sda.id = sdn.id WHEN NOT matched THEN INSERT (id, z) VALUES (sdn.id, 5); -ERROR: MERGE command is only supported when all distributed tables are co-located and joined on their distribution columns +ERROR: CTEs with modifying actions are not yet supported in MERGE -- Grouping sets not supported MERGE INTO citus_target t USING (SELECT count(*), id FROM citus_source GROUP BY GROUPING SETS (id, val)) subq @@ -2574,8 +3037,8 @@ WHEN NOT MATCHED THEN INSERT VALUES (subq.id, 99) WHEN MATCHED AND t.id < 350 THEN DELETE; -ERROR: cannot push down this subquery -DETAIL: could not run distributed query with GROUPING SETS, CUBE, or ROLLUP +ERROR: could not run distributed query with GROUPING SETS, CUBE, or ROLLUP +HINT: Consider using an equality filter on the distributed table's partition column. WITH subq AS ( SELECT count(*), id FROM citus_source GROUP BY GROUPING SETS (id, val) @@ -2589,8 +3052,8 @@ WHEN NOT MATCHED THEN INSERT VALUES (subq.id, 99) WHEN MATCHED AND t.id < 350 THEN DELETE; -ERROR: cannot push down this subquery -DETAIL: could not run distributed query with GROUPING SETS, CUBE, or ROLLUP +ERROR: could not run distributed query with GROUPING SETS, CUBE, or ROLLUP +HINT: Consider using an equality filter on the distributed table's partition column. -- try inserting unmatched distribution column value MERGE INTO citus_target t USING citus_source s @@ -2598,18 +3061,22 @@ ON t.id = s.id WHEN NOT MATCHED THEN INSERT DEFAULT VALUES; ERROR: cannot perform MERGE INSERT with DEFAULTS +DETAIL: Inserting arbitrary values that don't correspond to the joined column values can lead to unpredictable outcomes where rows are incorrectly distributed among different shards MERGE INTO citus_target t USING citus_source s ON t.id = s.id WHEN NOT MATCHED THEN INSERT VALUES(10000); -ERROR: MERGE INSERT must refer a source column for distribution column +ERROR: MERGE INSERT is using unsupported expression type for distribution column +DETAIL: Inserting arbitrary values that don't correspond to the joined column values can lead to unpredictable outcomes where rows are incorrectly distributed among different shards MERGE INTO citus_target t USING citus_source s ON t.id = s.id WHEN NOT MATCHED THEN INSERT (id) VALUES(1000); -ERROR: MERGE INSERT must refer a source column for distribution column +ERROR: MERGE INSERT is using unsupported expression type for distribution column +DETAIL: Inserting arbitrary values that don't correspond to the joined column values can lead to unpredictable outcomes where rows are incorrectly distributed among different shards +-- Colocated merge MERGE INTO t1 t USING s1 s ON t.id = s.id @@ -2622,6 +3089,13 @@ ON t.id = s.id WHEN NOT MATCHED THEN INSERT (val) VALUES(s.val); ERROR: MERGE INSERT must have distribution column as value +-- Non-colocated merge +MERGE INTO t1 t +USING s1 s +ON t.id = s.val +WHEN NOT MATCHED THEN + INSERT (id) VALUES(s.id); +ERROR: MERGE INSERT must use the source's joining column for target's distribution column -- try updating the distribution key column BEGIN; MERGE INTO target_cj t @@ -2691,7 +3165,7 @@ MERGE INTO t1 UPDATE SET val = t1.val + 1 WHEN NOT MATCHED THEN INSERT (id, val) VALUES (s1.id, s1.val); -ERROR: MERGE command is not supported with combination of distributed/local tables yet +ERROR: MERGE involving repartition of rows is supported only if the target is distributed -- Now both s1 and t1 are distributed tables SELECT undistribute_table('t1'); NOTICE: creating a new table for merge_schema.t1 @@ -2757,17 +3231,7 @@ WHEN MATCHED AND (merge_when_and_write()) THEN UPDATE SET val = t1.val + s1.val; ERROR: non-IMMUTABLE functions are not yet supported in MERGE sql with distributed tables ROLLBACK; --- Joining on partition columns with sub-query -MERGE INTO t1 - USING (SELECT * FROM s1) sub ON (sub.val = t1.id) -- sub.val is not a distribution column - WHEN MATCHED AND sub.val = 0 THEN - DELETE - WHEN MATCHED THEN - UPDATE SET val = t1.val + 1 - WHEN NOT MATCHED THEN - INSERT (id, val) VALUES (sub.id, sub.val); -ERROR: MERGE command is only supported when all distributed tables are co-located and joined on their distribution columns --- Joining on partition columns with CTE +-- Joining on non-partition columns with CTE source, but INSERT incorrect column WITH s1_res AS ( SELECT * FROM s1 ) @@ -2779,7 +3243,7 @@ MERGE INTO t1 UPDATE SET val = t1.val + 1 WHEN NOT MATCHED THEN INSERT (id, val) VALUES (s1_res.id, s1_res.val); -ERROR: MERGE command is only supported when all distributed tables are co-located and joined on their distribution columns +ERROR: MERGE INSERT must use the source's joining column for target's distribution column -- Constant Join condition WITH s1_res AS ( SELECT * FROM s1 @@ -2792,8 +3256,9 @@ MERGE INTO t1 UPDATE SET val = t1.val + 1 WHEN NOT MATCHED THEN INSERT (id, val) VALUES (s1_res.id, s1_res.val); -ERROR: MERGE command is only supported when all distributed tables are co-located and joined on their distribution columns --- With a single WHEN clause, which causes a non-left join +ERROR: The required join operation is missing between the target's distribution column and any expression originating from the source. The issue may arise from either a non-equi-join or a mismatch in the datatypes of the columns being joined. +DETAIL: Without a equi-join condition on the target's distribution column, the source rows cannot be efficiently redistributed, and the NOT-MATCHED condition cannot be evaluated unambiguously. This can result in incorrect or unexpected results when attempting to merge tables in a distributed setting +-- Join condition without target distribution column WITH s1_res AS ( SELECT * FROM s1 ) @@ -2801,7 +3266,8 @@ WITH s1_res AS ( WHEN MATCHED THEN DELETE WHEN NOT MATCHED THEN INSERT (id, val) VALUES (s1_res.id, s1_res.val); -ERROR: MERGE command is only supported when all distributed tables are co-located and joined on their distribution columns +ERROR: The required join operation is missing between the target's distribution column and any expression originating from the source. The issue may arise from either a non-equi-join or a mismatch in the datatypes of the columns being joined. +DETAIL: Without a equi-join condition on the target's distribution column, the source rows cannot be efficiently redistributed, and the NOT-MATCHED condition cannot be evaluated unambiguously. This can result in incorrect or unexpected results when attempting to merge tables in a distributed setting -- -- Reference tables -- @@ -2895,7 +3361,7 @@ MERGE INTO t1 UPDATE SET val = t1.val + 1 WHEN NOT MATCHED THEN INSERT (id, val) VALUES (s1.id, s1.val); -ERROR: MERGE command is not supported with combination of distributed/local tables yet +ERROR: MERGE INTO an distributed table from Postgres table is not yet supported MERGE INTO t1 USING (SELECT * FROM s1) sub ON (sub.id = t1.id) WHEN MATCHED AND sub.val = 0 THEN @@ -2904,7 +3370,7 @@ MERGE INTO t1 UPDATE SET val = t1.val + 1 WHEN NOT MATCHED THEN INSERT (id, val) VALUES (sub.id, sub.val); -ERROR: MERGE command is not supported with combination of distributed/local tables yet +ERROR: MERGE INTO an distributed table from Postgres table is not yet supported CREATE TABLE pg(val int); SELECT create_distributed_table('s1', 'id'); NOTICE: Copying data from local table... @@ -2925,7 +3391,7 @@ MERGE INTO t1 UPDATE SET val = t1.val + 1 WHEN NOT MATCHED THEN INSERT (id, val) VALUES (sub.id, sub.val); -ERROR: MERGE command is not supported with combination of distributed/local tables yet +ERROR: MERGE INTO an distributed table from Postgres table is not yet supported -- Mix Postgres table in CTE WITH pg_res AS ( SELECT * FROM pg @@ -2938,7 +3404,7 @@ MERGE INTO t1 UPDATE SET val = t1.val + 1 WHEN NOT MATCHED THEN INSERT (id, val) VALUES (sub.id, sub.val); -ERROR: MERGE command is not supported with combination of distributed/local tables yet +ERROR: MERGE INTO an distributed table from Postgres table is not yet supported -- Match more than one source row should fail same as Postgres behavior SELECT undistribute_table('t1'); NOTICE: creating a new table for merge_schema.t1 @@ -2993,7 +3459,7 @@ WHEN NOT MATCHED THEN INSERT VALUES(mv_source.id, mv_source.val); ERROR: cannot execute MERGE on relation "mv_source" DETAIL: This operation is not supported for materialized views. --- Distributed tables *must* be colocated +-- Do not allow constant values into the distribution column CREATE TABLE dist_target(id int, val varchar); SELECT create_distributed_table('dist_target', 'id'); create_distributed_table @@ -3008,31 +3474,6 @@ SELECT create_distributed_table('dist_source', 'id', colocate_with => 'none'); (1 row) -MERGE INTO dist_target -USING dist_source -ON dist_target.id = dist_source.id -WHEN MATCHED THEN -UPDATE SET val = dist_source.val -WHEN NOT MATCHED THEN -INSERT VALUES(dist_source.id, dist_source.val); -ERROR: For MERGE command, all the distributed tables must be colocated --- Distributed tables *must* be joined on distribution column -CREATE TABLE dist_colocated(id int, val int); -SELECT create_distributed_table('dist_colocated', 'id', colocate_with => 'dist_target'); - create_distributed_table ---------------------------------------------------------------------- - -(1 row) - -MERGE INTO dist_target -USING dist_colocated -ON dist_target.id = dist_colocated.val -- val is not the distribution column -WHEN MATCHED THEN -UPDATE SET val = dist_colocated.val -WHEN NOT MATCHED THEN -INSERT VALUES(dist_colocated.id, dist_colocated.val); -ERROR: MERGE command is only supported when all distributed tables are co-located and joined on their distribution columns --- Both the source and target must be distributed MERGE INTO dist_target USING (SELECT 100 id) AS source ON dist_target.id = source.id AND dist_target.val = 'const' @@ -3040,7 +3481,7 @@ WHEN MATCHED THEN UPDATE SET val = 'source' WHEN NOT MATCHED THEN INSERT VALUES(source.id, 'source'); -ERROR: For MERGE command, both the source and target must be distributed +ERROR: To MERGE into a distributed table, source must be Citus table(s) -- Non-hash distributed tables (append/range). CREATE VIEW show_tables AS SELECT logicalrelid, partmethod @@ -3079,8 +3520,7 @@ WHEN MATCHED THEN UPDATE SET val = dist_source.val WHEN NOT MATCHED THEN INSERT VALUES(dist_source.id, dist_source.val); -ERROR: For MERGE command, all the distributed tables must be colocated, for append/range distribution, colocation is not supported -HINT: Consider using hash distribution instead +ERROR: For MERGE command, append/range distribution table is not supported yet SELECT undistribute_table('dist_source'); NOTICE: creating a new table for merge_schema.dist_source NOTICE: moving the data of merge_schema.dist_source @@ -3113,8 +3553,7 @@ WHEN MATCHED THEN UPDATE SET val = dist_source.val WHEN NOT MATCHED THEN INSERT VALUES(dist_source.id, dist_source.val); -ERROR: For MERGE command, all the distributed tables must be colocated, for append/range distribution, colocation is not supported -HINT: Consider using hash distribution instead +ERROR: For MERGE command, append/range distribution table is not supported yet -- Both are append tables SELECT undistribute_table('dist_target'); NOTICE: creating a new table for merge_schema.dist_target @@ -3166,8 +3605,7 @@ WHEN MATCHED THEN UPDATE SET val = dist_source.val WHEN NOT MATCHED THEN INSERT VALUES(dist_source.id, dist_source.val); -ERROR: For MERGE command, all the distributed tables must be colocated, for append/range distribution, colocation is not supported -HINT: Consider using hash distribution instead +ERROR: For MERGE command, append/range distribution table is not supported yet -- Both are range tables SELECT undistribute_table('dist_target'); NOTICE: creating a new table for merge_schema.dist_target @@ -3219,12 +3657,10 @@ WHEN MATCHED THEN UPDATE SET val = dist_source.val WHEN NOT MATCHED THEN INSERT VALUES(dist_source.id, dist_source.val); -ERROR: For MERGE command, all the distributed tables must be colocated, for append/range distribution, colocation is not supported -HINT: Consider using hash distribution instead +ERROR: For MERGE command, append/range distribution table is not supported yet -- test merge with single-shard tables CREATE SCHEMA query_single_shard_table; SET search_path TO query_single_shard_table; -SET client_min_messages TO DEBUG2; CREATE TABLE nullkey_c1_t1(a int, b int); CREATE TABLE nullkey_c1_t2(a int, b int); SELECT create_distributed_table('nullkey_c1_t1', null, colocate_with=>'none'); @@ -3254,32 +3690,33 @@ SELECT create_distributed_table('nullkey_c2_t2', null, colocate_with=>'nullkey_c (1 row) CREATE TABLE reference_table(a int, b int); +CREATE TABLE distributed_table(a int, b int); +CREATE TABLE citus_local_table(a int, b int); SELECT create_reference_table('reference_table'); create_reference_table --------------------------------------------------------------------- (1 row) -INSERT INTO reference_table SELECT i, i FROM generate_series(0, 5) i; -DEBUG: distributed INSERT ... SELECT can only select from distributed tables -DEBUG: Collecting INSERT ... SELECT results on coordinator -CREATE TABLE distributed_table(a int, b int); SELECT create_distributed_table('distributed_table', 'a'); create_distributed_table --------------------------------------------------------------------- (1 row) -INSERT INTO distributed_table SELECT i, i FROM generate_series(3, 8) i; -DEBUG: distributed INSERT ... SELECT can only select from distributed tables -DEBUG: Collecting INSERT ... SELECT results on coordinator -CREATE TABLE citus_local_table(a int, b int); SELECT citus_add_local_table_to_metadata('citus_local_table'); citus_add_local_table_to_metadata --------------------------------------------------------------------- (1 row) +SET client_min_messages TO DEBUG2; +INSERT INTO reference_table SELECT i, i FROM generate_series(0, 5) i; +DEBUG: distributed INSERT ... SELECT can only select from distributed tables +DEBUG: Collecting INSERT ... SELECT results on coordinator +INSERT INTO distributed_table SELECT i, i FROM generate_series(3, 8) i; +DEBUG: distributed INSERT ... SELECT can only select from distributed tables +DEBUG: Collecting INSERT ... SELECT results on coordinator INSERT INTO citus_local_table SELECT i, i FROM generate_series(0, 10) i; DEBUG: distributed INSERT ... SELECT can only select from distributed tables DEBUG: Collecting INSERT ... SELECT results on coordinator @@ -3288,44 +3725,70 @@ INSERT INTO postgres_local_table SELECT i, i FROM generate_series(5, 10) i; -- with a colocated table MERGE INTO nullkey_c1_t1 USING nullkey_c1_t2 ON (nullkey_c1_t1.a = nullkey_c1_t2.a) WHEN MATCHED THEN UPDATE SET b = nullkey_c1_t2.b; -DEBUG: +DEBUG: DEBUG: Creating MERGE router plan MERGE INTO nullkey_c1_t1 USING nullkey_c1_t2 ON (nullkey_c1_t1.a = nullkey_c1_t2.a) WHEN MATCHED THEN DELETE; -DEBUG: +DEBUG: DEBUG: Creating MERGE router plan MERGE INTO nullkey_c1_t1 USING nullkey_c1_t2 ON (nullkey_c1_t1.a = nullkey_c1_t2.a) WHEN MATCHED THEN UPDATE SET b = nullkey_c1_t2.b WHEN NOT MATCHED THEN INSERT VALUES (nullkey_c1_t2.a, nullkey_c1_t2.b); -DEBUG: +DEBUG: DEBUG: Creating MERGE router plan MERGE INTO nullkey_c1_t1 USING nullkey_c1_t2 ON (nullkey_c1_t1.a = nullkey_c1_t2.a) WHEN MATCHED THEN DELETE WHEN NOT MATCHED THEN INSERT VALUES (nullkey_c1_t2.a, nullkey_c1_t2.b); -DEBUG: +DEBUG: DEBUG: Creating MERGE router plan -- with non-colocated single-shard table MERGE INTO nullkey_c1_t1 USING nullkey_c2_t1 ON (nullkey_c1_t1.a = nullkey_c2_t1.a) WHEN MATCHED THEN UPDATE SET b = nullkey_c2_t1.b; -ERROR: For MERGE command, all the distributed tables must be colocated +DEBUG: Distributed tables are not co-located, try repartitioning +DEBUG: For MERGE command, all the distributed tables must be colocated +DEBUG: Creating MERGE repartition plan +ERROR: MERGE operation on non-colocated distributed table(s) without a shard key is not yet supported MERGE INTO nullkey_c1_t1 USING nullkey_c2_t1 ON (nullkey_c1_t1.a = nullkey_c2_t1.a) WHEN MATCHED THEN UPDATE SET b = nullkey_c2_t1.b WHEN NOT MATCHED THEN INSERT VALUES (nullkey_c2_t1.a, nullkey_c2_t1.b); -ERROR: For MERGE command, all the distributed tables must be colocated +DEBUG: Distributed tables are not co-located, try repartitioning +DEBUG: For MERGE command, all the distributed tables must be colocated +DEBUG: Creating MERGE repartition plan +ERROR: MERGE operation on non-colocated distributed table(s) without a shard key is not yet supported -- with a distributed table MERGE INTO nullkey_c1_t1 USING distributed_table ON (nullkey_c1_t1.a = distributed_table.a) WHEN MATCHED THEN UPDATE SET b = distributed_table.b WHEN NOT MATCHED THEN INSERT VALUES (distributed_table.a, distributed_table.b); -ERROR: For MERGE command, all the distributed tables must be colocated +DEBUG: Distributed tables are not co-located, try repartitioning +DEBUG: For MERGE command, all the distributed tables must be colocated +DEBUG: Creating MERGE repartition plan +ERROR: MERGE operation on non-colocated distributed table(s) without a shard key is not yet supported MERGE INTO distributed_table USING nullkey_c1_t1 ON (nullkey_c1_t1.a = distributed_table.a) WHEN MATCHED THEN DELETE WHEN NOT MATCHED THEN INSERT VALUES (nullkey_c1_t1.a, nullkey_c1_t1.b); -ERROR: For MERGE command, all the distributed tables must be colocated +DEBUG: Distributed tables are not co-located, try repartitioning +DEBUG: For MERGE command, all the distributed tables must be colocated +DEBUG: Creating MERGE repartition plan +DEBUG: Using column - index:0 from the source list to redistribute +DEBUG: Distributed planning for a fast-path router query +DEBUG: Creating router plan +DEBUG: Collect source query results on coordinator +DEBUG: Create a MERGE task list that needs to be routed +DEBUG: +DEBUG: distributed statement: MERGE INTO query_single_shard_table.distributed_table_4000173 citus_table_alias USING (SELECT intermediate_result.a, intermediate_result.b FROM read_intermediate_result('merge_into_XXX_4000173'::text, 'binary'::citus_copy_format) intermediate_result(a integer, b integer)) nullkey_c1_t1 ON (nullkey_c1_t1.a OPERATOR(pg_catalog.=) citus_table_alias.a) WHEN MATCHED THEN DELETE WHEN NOT MATCHED THEN INSERT (a, b) VALUES (nullkey_c1_t1.a, nullkey_c1_t1.b) +DEBUG: +DEBUG: distributed statement: MERGE INTO query_single_shard_table.distributed_table_4000174 citus_table_alias USING (SELECT intermediate_result.a, intermediate_result.b FROM read_intermediate_result('merge_into_XXX_4000174'::text, 'binary'::citus_copy_format) intermediate_result(a integer, b integer)) nullkey_c1_t1 ON (nullkey_c1_t1.a OPERATOR(pg_catalog.=) citus_table_alias.a) WHEN MATCHED THEN DELETE WHEN NOT MATCHED THEN INSERT (a, b) VALUES (nullkey_c1_t1.a, nullkey_c1_t1.b) +DEBUG: +DEBUG: distributed statement: MERGE INTO query_single_shard_table.distributed_table_4000175 citus_table_alias USING (SELECT intermediate_result.a, intermediate_result.b FROM read_intermediate_result('merge_into_XXX_4000175'::text, 'binary'::citus_copy_format) intermediate_result(a integer, b integer)) nullkey_c1_t1 ON (nullkey_c1_t1.a OPERATOR(pg_catalog.=) citus_table_alias.a) WHEN MATCHED THEN DELETE WHEN NOT MATCHED THEN INSERT (a, b) VALUES (nullkey_c1_t1.a, nullkey_c1_t1.b) +DEBUG: +DEBUG: distributed statement: MERGE INTO query_single_shard_table.distributed_table_4000176 citus_table_alias USING (SELECT intermediate_result.a, intermediate_result.b FROM read_intermediate_result('merge_into_XXX_4000176'::text, 'binary'::citus_copy_format) intermediate_result(a integer, b integer)) nullkey_c1_t1 ON (nullkey_c1_t1.a OPERATOR(pg_catalog.=) citus_table_alias.a) WHEN MATCHED THEN DELETE WHEN NOT MATCHED THEN INSERT (a, b) VALUES (nullkey_c1_t1.a, nullkey_c1_t1.b) -- with a reference table MERGE INTO nullkey_c1_t1 USING reference_table ON (nullkey_c1_t1.a = reference_table.a) WHEN MATCHED THEN UPDATE SET b = reference_table.b; -ERROR: MERGE command is not supported with combination of distributed/reference yet -HINT: If target is distributed, source must be distributed and co-located +DEBUG: A mix of distributed and reference table, try repartitioning +DEBUG: A mix of distributed and reference table, routable query is not possible +DEBUG: Creating MERGE repartition plan +ERROR: MERGE operation on non-colocated distributed table(s) without a shard key is not yet supported MERGE INTO reference_table USING nullkey_c1_t1 ON (nullkey_c1_t1.a = reference_table.a) WHEN MATCHED THEN UPDATE SET b = nullkey_c1_t1.b WHEN NOT MATCHED THEN INSERT VALUES (nullkey_c1_t1.a, nullkey_c1_t1.b); @@ -3333,42 +3796,189 @@ ERROR: Reference table as target is not allowed in MERGE command -- with a citus local table MERGE INTO nullkey_c1_t1 USING citus_local_table ON (nullkey_c1_t1.a = citus_local_table.a) WHEN MATCHED THEN UPDATE SET b = citus_local_table.b; -ERROR: MERGE command is not supported with combination of distributed/local tables yet +DEBUG: A mix of distributed and local table, try repartitioning +DEBUG: A mix of distributed and citus-local table, routable query is not possible +DEBUG: Creating MERGE repartition plan +ERROR: MERGE operation on non-colocated distributed table(s) without a shard key is not yet supported MERGE INTO citus_local_table USING nullkey_c1_t1 ON (nullkey_c1_t1.a = citus_local_table.a) WHEN MATCHED THEN DELETE; -ERROR: MERGE command is not supported with combination of distributed/local tables yet +DEBUG: A mix of distributed and local table, try repartitioning +DEBUG: A mix of distributed and citus-local table, routable query is not possible +DEBUG: Creating MERGE repartition plan +ERROR: MERGE involving repartition of rows is supported only if the target is distributed -- with a postgres table MERGE INTO nullkey_c1_t1 USING postgres_local_table ON (nullkey_c1_t1.a = postgres_local_table.a) WHEN MATCHED THEN UPDATE SET b = postgres_local_table.b; -ERROR: MERGE command is not supported with combination of distributed/local tables yet +DEBUG: There is only one distributed table, merge is not pushable, try repartitioning +DEBUG: Creating MERGE repartition plan +ERROR: MERGE INTO an distributed table from Postgres table is not yet supported MERGE INTO postgres_local_table USING nullkey_c1_t1 ON (nullkey_c1_t1.a = postgres_local_table.a) WHEN MATCHED THEN UPDATE SET b = nullkey_c1_t1.b WHEN NOT MATCHED THEN INSERT VALUES (nullkey_c1_t1.a, nullkey_c1_t1.b); -ERROR: MERGE command is not supported with combination of distributed/local tables yet +DEBUG: There is only one distributed table, merge is not pushable, try repartitioning +DEBUG: Creating MERGE repartition plan +ERROR: MERGE involving repartition of rows is supported only if the target is distributed -- using ctes WITH cte AS ( SELECT * FROM nullkey_c1_t1 ) MERGE INTO nullkey_c1_t1 USING cte ON (nullkey_c1_t1.a = cte.a) WHEN MATCHED THEN UPDATE SET b = cte.b; -DEBUG: +DEBUG: DEBUG: Creating MERGE router plan WITH cte AS ( SELECT * FROM distributed_table ) MERGE INTO nullkey_c1_t1 USING cte ON (nullkey_c1_t1.a = cte.a) WHEN MATCHED THEN UPDATE SET b = cte.b; -ERROR: For MERGE command, all the distributed tables must be colocated +DEBUG: Distributed tables are not co-located, try repartitioning +DEBUG: For MERGE command, all the distributed tables must be colocated +DEBUG: Creating MERGE repartition plan +ERROR: MERGE operation on non-colocated distributed table(s) without a shard key is not yet supported WITH cte AS materialized ( SELECT * FROM distributed_table ) MERGE INTO nullkey_c1_t1 USING cte ON (nullkey_c1_t1.a = cte.a) WHEN MATCHED THEN UPDATE SET b = cte.b; -ERROR: For MERGE command, all the distributed tables must be colocated +DEBUG: Distributed tables are not co-located, try repartitioning +DEBUG: For MERGE command, all the distributed tables must be colocated +DEBUG: Creating MERGE repartition plan +ERROR: MERGE operation on non-colocated distributed table(s) without a shard key is not yet supported SET client_min_messages TO WARNING; DROP SCHEMA query_single_shard_table CASCADE; -RESET client_min_messages; SET search_path TO merge_schema; +-- Test Columnar table +CREATE TABLE target_columnar(cid int, name text) USING columnar; +SELECT create_distributed_table('target_columnar', 'cid'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +MERGE INTO target_columnar t +USING demo_source_table s +ON t.cid = s.id2 +WHEN MATCHED THEN + UPDATE SET name = 'Columnar table updated by MERGE' +WHEN NOT MATCHED THEN + DO NOTHING; +ERROR: Columnar table as target is not allowed in MERGE command +MERGE INTO demo_distributed t +USING generate_series(0,100) as source(key) +ON (source.key + 1 = t.id1) + WHEN MATCHED THEN UPDATE SET val1 = 15; +ERROR: Currently, Citus only supports table, subquery, and CTEs as valid sources for the MERGE operation +-- This should fail in planning stage itself +EXPLAIN MERGE INTO demo_distributed t +USING demo_source_table s +ON (s.id2 + 1 = t.id1) + WHEN MATCHED THEN UPDATE SET val1 = 15; +ERROR: The required join operation is missing between the target's distribution column and any expression originating from the source. The issue may arise from either a non-equi-join or a mismatch in the datatypes of the columns being joined. +DETAIL: Without a equi-join condition on the target's distribution column, the source rows cannot be efficiently redistributed, and the NOT-MATCHED condition cannot be evaluated unambiguously. This can result in incorrect or unexpected results when attempting to merge tables in a distributed setting +-- Sub-queries and CTEs are not allowed in actions and ON clause +CREATE TABLE target_1 (a int, b int, c int); +SELECT create_distributed_table('target_1', 'a'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +CREATE TABLE source_2 (a int, b int, c int); +SELECT create_distributed_table('source_2', 'a'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +INSERT INTO target_1 VALUES(1, 2, 3); +INSERT INTO target_1 VALUES(4, 5, 6); +INSERT INTO target_1 VALUES(11, 12, 13); +INSERT INTO source_2 VALUES(1, 2, 3); +WITH cte_1 as (SELECT max(a) as max_a, max(b) as b FROM source_2) +MERGE INTO target_1 +USING cte_1 +ON (target_1.a = cte_1.b) +WHEN NOT MATCHED AND (SELECT max_a > 10 FROM cte_1) THEN + INSERT VALUES (cte_1.b, 100); +ERROR: Sub-queries and CTEs are not allowed in actions for MERGE with repartitioning +HINT: Consider making the source and target colocated and joined on the distribution column to make it a routable query +WITH cte_1 as (SELECT a, b FROM source_2) +MERGE INTO target_1 +USING cte_1 +ON (target_1.a = cte_1.b) +WHEN NOT MATCHED AND (SELECT a > 10 FROM cte_1) THEN + INSERT VALUES (cte_1.b, 100); +ERROR: Sub-queries and CTEs are not allowed in actions for MERGE with repartitioning +HINT: Consider making the source and target colocated and joined on the distribution column to make it a routable query +MERGE INTO target_1 +USING source_2 +ON (target_1.a = source_2.b) +WHEN NOT MATCHED AND (SELECT max_a > 10 FROM (SELECT max(a) as max_a, max(b) as b FROM target_1) as foo) THEN + INSERT VALUES (source_2.b, 100); +ERROR: Sub-queries and CTEs are not allowed in actions for MERGE with repartitioning +HINT: Consider making the source and target colocated and joined on the distribution column to make it a routable query +-- or same with CTEs +WITH cte_1 as (SELECT max(a) as max_a, max(b) as b FROM target_1) +MERGE INTO target_1 +USING source_2 +ON (target_1.a = source_2.b) +WHEN NOT MATCHED AND (SELECT max_a > 10 FROM (SELECT max(a) as max_a, max(b) as b FROM target_1) as foo) THEN + INSERT VALUES (source_2.b, 100); +ERROR: Sub-queries and CTEs are not allowed in actions for MERGE with repartitioning +HINT: Consider making the source and target colocated and joined on the distribution column to make it a routable query +WITH cte_1 as (SELECT a, b FROM target_1), cte_2 as (select b,a from target_1) +MERGE INTO target_1 +USING (SELECT * FROM source_2) as subq +ON (target_1.a = subq.b) +WHEN NOT MATCHED AND (SELECT a > 10 FROM cte_2) THEN + INSERT VALUES (subq.b, 100); +ERROR: Sub-queries and CTEs are not allowed in actions for MERGE with repartitioning +HINT: Consider making the source and target colocated and joined on the distribution column to make it a routable query +MERGE INTO source_2 +USING target_1 +ON (target_1.a = source_2.a) +WHEN MATCHED THEN + UPDATE SET b = (SELECT max(a) FROM source_2); +ERROR: Sub-queries and CTEs are not allowed in actions for MERGE with repartitioning +HINT: Consider making the source and target colocated and joined on the distribution column to make it a routable query +MERGE INTO source_2 +USING target_1 +ON (target_1.a = source_2.a) +WHEN NOT MATCHED THEN + INSERT VALUES (target_1.a,(select max(a) from target_1)); +ERROR: Sub-queries and CTEs are not allowed in actions for MERGE with repartitioning +HINT: Consider making the source and target colocated and joined on the distribution column to make it a routable query +MERGE INTO target_1 +USING source_2 +ON (target_1.a = source_2.b) +WHEN NOT MATCHED AND (SELECT max(c) > 10 FROM source_2) THEN + INSERT VALUES (source_2.b, 100); +ERROR: Sub-queries and CTEs are not allowed in actions for MERGE with repartitioning +HINT: Consider making the source and target colocated and joined on the distribution column to make it a routable query +-- Test in ON clause +MERGE INTO target_1 t2 +USING (SELECT * FROM source_2) AS t1 +ON (t1.a = t2.a AND (SELECT 1=1 FROM target_1)) +WHEN MATCHED THEN + DELETE; +ERROR: Sub-queries and CTEs are not allowed in ON clause for MERGE with repartitioning +HINT: Consider making the source and target colocated and joined on the distribution column to make it a routable query +MERGE INTO target_1 t2 +USING (SELECT * FROM source_2) AS t1 +ON (t1.a = t2.a AND (SELECT max(a) > 55 FROM target_1)) +WHEN MATCHED THEN + DELETE; +ERROR: Sub-queries and CTEs are not allowed in ON clause for MERGE with repartitioning +HINT: Consider making the source and target colocated and joined on the distribution column to make it a routable query +WITH cte_1 as (SELECT a, b FROM target_1), cte_2 as (select b,a from target_1) +MERGE INTO target_1 t2 +USING (SELECT * FROM cte_1) AS t1 +ON (t1.a = t2.a AND (SELECT max(a) > 55 FROM cte_2)) +WHEN MATCHED THEN + DELETE; +ERROR: Sub-queries and CTEs are not allowed in ON clause for MERGE with repartitioning +HINT: Consider making the source and target colocated and joined on the distribution column to make it a routable query +RESET client_min_messages; DROP SERVER foreign_server CASCADE; NOTICE: drop cascades to 3 other objects DETAIL: drop cascades to user mapping for postgres on server foreign_server @@ -3379,7 +3989,7 @@ CONTEXT: SQL statement "SELECT citus_drop_all_shards(v_obj.objid, v_obj.schema_ PL/pgSQL function citus_drop_trigger() line XX at PERFORM DROP FUNCTION merge_when_and_write(); DROP SCHEMA merge_schema CASCADE; -NOTICE: drop cascades to 90 other objects +NOTICE: drop cascades to 98 other objects DETAIL: drop cascades to function insert_data() drop cascades to table local_local drop cascades to table target @@ -3453,20 +4063,28 @@ drop cascades to table source_serial drop cascades to table target_serial drop cascades to table target_set drop cascades to table source_set -drop cascades to table reftarget_local_4000113 drop cascades to table refsource_ref drop cascades to table pg_result drop cascades to table refsource_ref_4000112 drop cascades to table pg_ref -drop cascades to table reftarget_local drop cascades to table local_ref +drop cascades to table reftarget_local +drop cascades to table dist_reftarget +drop cascades to function setup_demo_data() +drop cascades to function merge_demo_data() +drop cascades to table demo_distributed +drop cascades to table demo_source_table +drop cascades to table pg_demo_result +drop cascades to table dist_demo_result drop cascades to function add_s(integer,integer) drop cascades to table pg -drop cascades to table t1_4000133 -drop cascades to table s1_4000134 +drop cascades to table t1_4000158 +drop cascades to table s1_4000159 drop cascades to table t1 drop cascades to table s1 -drop cascades to table dist_colocated drop cascades to table dist_target drop cascades to table dist_source drop cascades to view show_tables +drop cascades to table target_columnar +drop cascades to table target_1 +drop cascades to table source_2 diff --git a/src/test/regress/expected/merge_arbitrary.out b/src/test/regress/expected/merge_arbitrary.out index 345ac1410..b55306b44 100644 --- a/src/test/regress/expected/merge_arbitrary.out +++ b/src/test/regress/expected/merge_arbitrary.out @@ -148,3 +148,51 @@ SELECT * FROM t1 order by id; (5 rows) ROLLBACK; +-- Test prepared statements with repartition +PREPARE merge_repartition_pg(int,int,int,int) as + MERGE INTO pg_target target + USING (SELECT id+1+$1 as key, val FROM (SELECT * FROM pg_source UNION SELECT * FROM pg_source WHERE id = $2) as foo) as source + ON (source.key = target.id AND $3 < 10000) + WHEN MATCHED THEN UPDATE SET val = (source.key::int+$4) + WHEN NOT MATCHED THEN INSERT VALUES (source.key, source.val); +PREPARE merge_repartition_citus(int,int,int,int) as + MERGE INTO citus_target target + USING (SELECT id+1+$1 as key, val FROM (SELECT * FROM citus_source UNION SELECT * FROM citus_source WHERE id = $2) as foo) as source + ON (source.key = target.id AND $3 < 10000) + WHEN MATCHED THEN UPDATE SET val = (source.key::int+$4) + WHEN NOT MATCHED THEN INSERT VALUES (source.key, source.val); +EXECUTE merge_repartition_pg(1,1,1,1); +EXECUTE merge_repartition_citus(1,1,1,1); +SET client_min_messages = NOTICE; +SELECT compare_data(); +NOTICE: The average of pg_target.id is equal to citus_target.id +NOTICE: The average of pg_target.val is equal to citus_target.val + compare_data +--------------------------------------------------------------------- + +(1 row) + +RESET client_min_messages; +EXECUTE merge_repartition_pg(1,100,1,1); +EXECUTE merge_repartition_citus(1,100,1,1); +EXECUTE merge_repartition_pg(2,200,1,1); +EXECUTE merge_repartition_citus(2,200,1,1); +EXECUTE merge_repartition_pg(3,300,1,1); +EXECUTE merge_repartition_citus(3,300,1,1); +EXECUTE merge_repartition_pg(4,400,1,1); +EXECUTE merge_repartition_citus(4,400,1,1); +EXECUTE merge_repartition_pg(5,500,1,1); +EXECUTE merge_repartition_citus(5,500,1,1); +-- Sixth time +EXECUTE merge_repartition_pg(6,600,1,6); +EXECUTE merge_repartition_citus(6,600,1,6); +SET client_min_messages = NOTICE; +SELECT compare_data(); +NOTICE: The average of pg_target.id is equal to citus_target.id +NOTICE: The average of pg_target.val is equal to citus_target.val + compare_data +--------------------------------------------------------------------- + +(1 row) + +RESET client_min_messages; diff --git a/src/test/regress/expected/merge_arbitrary_create.out b/src/test/regress/expected/merge_arbitrary_create.out index 9b2444f17..aff9ecd97 100644 --- a/src/test/regress/expected/merge_arbitrary_create.out +++ b/src/test/regress/expected/merge_arbitrary_create.out @@ -70,3 +70,77 @@ SELECT citus_add_local_table_to_metadata('s1'); (1 row) +-- Test prepared statements with repartition +CREATE TABLE pg_target(id int, val int); +CREATE TABLE pg_source(id int, val int, const int); +CREATE TABLE citus_target(id int, val int); +CREATE TABLE citus_source(id int, val int, const int); +SELECT citus_add_local_table_to_metadata('pg_target'); + citus_add_local_table_to_metadata +--------------------------------------------------------------------- + +(1 row) + +SELECT citus_add_local_table_to_metadata('pg_source'); + citus_add_local_table_to_metadata +--------------------------------------------------------------------- + +(1 row) + +-- +-- Load same set of data to both Postgres and Citus tables +-- +CREATE OR REPLACE FUNCTION setup_data() RETURNS VOID AS $$ + INSERT INTO pg_source SELECT i, i+1, 1 FROM generate_series(1, 10000) i; + INSERT INTO pg_target SELECT i, 1 FROM generate_series(5001, 10000) i; + INSERT INTO citus_source SELECT i, i+1, 1 FROM generate_series(1, 10000) i; + INSERT INTO citus_target SELECT i, 1 FROM generate_series(5001, 10000) i; +$$ +LANGUAGE SQL; +-- +-- Compares the final target tables, merge-modified data, of both Postgres and Citus tables +-- +CREATE OR REPLACE FUNCTION check_data(table1_name text, column1_name text, table2_name text, column2_name text) +RETURNS VOID AS $$ +DECLARE + table1_avg numeric; + table2_avg numeric; +BEGIN + EXECUTE format('SELECT COALESCE(AVG(%I), 0) FROM %I', column1_name, table1_name) INTO table1_avg; + EXECUTE format('SELECT COALESCE(AVG(%I), 0) FROM %I', column2_name, table2_name) INTO table2_avg; + + IF table1_avg > table2_avg THEN + RAISE EXCEPTION 'The average of %.% is greater than %.%', table1_name, column1_name, table2_name, column2_name; + ELSIF table1_avg < table2_avg THEN + RAISE EXCEPTION 'The average of %.% is less than %.%', table1_name, column1_name, table2_name, column2_name; + ELSE + RAISE NOTICE 'The average of %.% is equal to %.%', table1_name, column1_name, table2_name, column2_name; + END IF; +END; +$$ LANGUAGE plpgsql; +CREATE OR REPLACE FUNCTION compare_data() RETURNS VOID AS $$ + SELECT check_data('pg_target', 'id', 'citus_target', 'id'); + SELECT check_data('pg_target', 'val', 'citus_target', 'val'); +$$ +LANGUAGE SQL; +-- +-- Target and source are distributed, and non-colocated +-- +SELECT setup_data(); + setup_data +--------------------------------------------------------------------- + +(1 row) + +SELECT create_distributed_table('citus_target', 'id'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +SELECT create_distributed_table('citus_source', 'id', colocate_with=>'none'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + diff --git a/src/test/regress/expected/merge_partition_tables.out b/src/test/regress/expected/merge_partition_tables.out new file mode 100644 index 000000000..5ac375817 --- /dev/null +++ b/src/test/regress/expected/merge_partition_tables.out @@ -0,0 +1,230 @@ +SHOW server_version \gset +SELECT substring(:'server_version', '\d+')::int >= 15 AS server_version_ge_15 +\gset +\if :server_version_ge_15 +\else +\q +\endif +-- We create two sets of source and target tables, one set in Postgres and +-- the other in Citus distributed. We run the _exact_ MERGE SQL on both sets +-- and compare the final results of the target tables in Postgres and Citus. +-- The results should match. This process is repeated for various combinations +-- of MERGE SQL. +DROP SCHEMA IF EXISTS merge_partition_tables CASCADE; +NOTICE: schema "merge_partition_tables" does not exist, skipping +CREATE SCHEMA merge_partition_tables; +SET search_path TO merge_partition_tables; +SET citus.shard_count TO 4; +SET citus.next_shard_id TO 7000000; +SET citus.explain_all_tasks TO true; +SET citus.shard_replication_factor TO 1; +SET citus.max_adaptive_executor_pool_size TO 1; +SET client_min_messages = warning; +SELECT 1 FROM master_add_node('localhost', :master_port, groupid => 0); + ?column? +--------------------------------------------------------------------- + 1 +(1 row) + +RESET client_min_messages; +CREATE TABLE pg_target(id int, val int) PARTITION BY RANGE(id); +CREATE TABLE pg_source(id int, val int, const int) PARTITION BY RANGE(val); +CREATE TABLE citus_target(id int, val int) PARTITION BY RANGE(id); +CREATE TABLE citus_source(id int, val int, const int) PARTITION BY RANGE(val); +SELECT citus_add_local_table_to_metadata('citus_target'); + citus_add_local_table_to_metadata +--------------------------------------------------------------------- + +(1 row) + +SELECT citus_add_local_table_to_metadata('citus_source'); + citus_add_local_table_to_metadata +--------------------------------------------------------------------- + +(1 row) + +CREATE TABLE part1 PARTITION OF pg_target FOR VALUES FROM (1) TO (2500) WITH (autovacuum_enabled=off); +CREATE TABLE part2 PARTITION OF pg_target FOR VALUES FROM (2501) TO (5000) WITH (autovacuum_enabled=off); +CREATE TABLE part3 PARTITION OF pg_target FOR VALUES FROM (5001) TO (7500) WITH (autovacuum_enabled=off); +CREATE TABLE part4 PARTITION OF pg_target DEFAULT WITH (autovacuum_enabled=off); +CREATE TABLE part5 PARTITION OF citus_target FOR VALUES FROM (1) TO (2500) WITH (autovacuum_enabled=off); +CREATE TABLE part6 PARTITION OF citus_target FOR VALUES FROM (2501) TO (5000) WITH (autovacuum_enabled=off); +CREATE TABLE part7 PARTITION OF citus_target FOR VALUES FROM (5001) TO (7500) WITH (autovacuum_enabled=off); +CREATE TABLE part8 PARTITION OF citus_target DEFAULT WITH (autovacuum_enabled=off); +CREATE TABLE part9 PARTITION OF pg_source FOR VALUES FROM (1) TO (2500) WITH (autovacuum_enabled=off); +CREATE TABLE part10 PARTITION OF pg_source FOR VALUES FROM (2501) TO (5000) WITH (autovacuum_enabled=off); +CREATE TABLE part11 PARTITION OF pg_source FOR VALUES FROM (5001) TO (7500) WITH (autovacuum_enabled=off); +CREATE TABLE part12 PARTITION OF pg_source DEFAULT WITH (autovacuum_enabled=off); +CREATE TABLE part13 PARTITION OF citus_source FOR VALUES FROM (1) TO (2500) WITH (autovacuum_enabled=off); +CREATE TABLE part14 PARTITION OF citus_source FOR VALUES FROM (2501) TO (5000) WITH (autovacuum_enabled=off); +CREATE TABLE part15 PARTITION OF citus_source FOR VALUES FROM (5001) TO (7500) WITH (autovacuum_enabled=off); +CREATE TABLE part16 PARTITION OF citus_source DEFAULT WITH (autovacuum_enabled=off); +CREATE OR REPLACE FUNCTION cleanup_data() RETURNS VOID SET search_path TO merge_partition_tables AS $$ + TRUNCATE pg_target; + TRUNCATE pg_source; + TRUNCATE citus_target; + TRUNCATE citus_source; + SELECT undistribute_table('citus_target'); + SELECT undistribute_table('citus_source'); +$$ +LANGUAGE SQL; +-- +-- Load same set of data to both Postgres and Citus tables +-- +CREATE OR REPLACE FUNCTION setup_data() RETURNS VOID SET search_path TO merge_partition_tables AS $$ + INSERT INTO pg_source SELECT i, i+1, 1 FROM generate_series(1, 10000) i; + INSERT INTO pg_target SELECT i, 1 FROM generate_series(5001, 10000) i; + INSERT INTO citus_source SELECT i, i+1, 1 FROM generate_series(1, 10000) i; + INSERT INTO citus_target SELECT i, 1 FROM generate_series(5001, 10000) i; +$$ +LANGUAGE SQL; +-- +-- Compares the final target tables, merge-modified data, of both Postgres and Citus tables +-- +CREATE OR REPLACE FUNCTION check_data(table1_name text, column1_name text, table2_name text, column2_name text) +RETURNS VOID SET search_path TO merge_partition_tables AS $$ +DECLARE + table1_avg numeric; + table2_avg numeric; +BEGIN + EXECUTE format('SELECT COALESCE(AVG(%I), 0) FROM %I', column1_name, table1_name) INTO table1_avg; + EXECUTE format('SELECT COALESCE(AVG(%I), 0) FROM %I', column2_name, table2_name) INTO table2_avg; + + IF table1_avg > table2_avg THEN + RAISE EXCEPTION 'The average of %.% is greater than %.%', table1_name, column1_name, table2_name, column2_name; + ELSIF table1_avg < table2_avg THEN + RAISE EXCEPTION 'The average of %.% is less than %.%', table1_name, column1_name, table2_name, column2_name; + ELSE + RAISE NOTICE 'The average of %.% is equal to %.%', table1_name, column1_name, table2_name, column2_name; + END IF; +END; +$$ LANGUAGE plpgsql; +CREATE OR REPLACE FUNCTION compare_data() RETURNS VOID SET search_path TO merge_partition_tables AS $$ + SELECT check_data('pg_target', 'id', 'citus_target', 'id'); + SELECT check_data('pg_target', 'val', 'citus_target', 'val'); +$$ +LANGUAGE SQL; +-- Test colocated partition tables +SET client_min_messages = ERROR; +SELECT cleanup_data(); + cleanup_data +--------------------------------------------------------------------- + +(1 row) + +SELECT setup_data(); + setup_data +--------------------------------------------------------------------- + +(1 row) + +SELECT create_distributed_table('citus_target', 'id'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +SELECT create_distributed_table('citus_source', 'id', colocate_with=>'citus_target'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +RESET client_min_messages; +MERGE INTO pg_target t +USING pg_source s +ON t.id = s.id +WHEN MATCHED AND t.id <= 7500 THEN + UPDATE SET val = s.val + 1 +WHEN MATCHED THEN + DELETE +WHEN NOT MATCHED THEN + INSERT VALUES(s.id, s.val); +MERGE INTO citus_target t +USING citus_source s +ON t.id = s.id +WHEN MATCHED AND t.id <= 7500 THEN + UPDATE SET val = s.val + 1 +WHEN MATCHED THEN + DELETE +WHEN NOT MATCHED THEN + INSERT VALUES(s.id, s.val); +SELECT compare_data(); +NOTICE: The average of pg_target.id is equal to citus_target.id +CONTEXT: PL/pgSQL function check_data(text,text,text,text) line XX at RAISE +SQL function "compare_data" statement 1 +NOTICE: The average of pg_target.val is equal to citus_target.val +CONTEXT: PL/pgSQL function check_data(text,text,text,text) line XX at RAISE +SQL function "compare_data" statement 2 + compare_data +--------------------------------------------------------------------- + +(1 row) + +-- Test non-colocated partition tables +SET client_min_messages = ERROR; +SELECT cleanup_data(); + cleanup_data +--------------------------------------------------------------------- + +(1 row) + +SELECT setup_data(); + setup_data +--------------------------------------------------------------------- + +(1 row) + +SELECT create_distributed_table('citus_target', 'id'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +SELECT create_distributed_table('citus_source', 'id', colocate_with=>'none'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +RESET client_min_messages; +MERGE INTO pg_target t +USING pg_source s +ON t.id = s.id +WHEN MATCHED AND t.id <= 7500 THEN + UPDATE SET val = s.val + 1 +WHEN MATCHED THEN + DELETE +WHEN NOT MATCHED THEN + INSERT VALUES(s.id, s.val); +MERGE INTO citus_target t +USING citus_source s +ON t.id = s.id +WHEN MATCHED AND t.id <= 7500 THEN + UPDATE SET val = s.val + 1 +WHEN MATCHED THEN + DELETE +WHEN NOT MATCHED THEN + INSERT VALUES(s.id, s.val); +SELECT compare_data(); +NOTICE: The average of pg_target.id is equal to citus_target.id +CONTEXT: PL/pgSQL function check_data(text,text,text,text) line XX at RAISE +SQL function "compare_data" statement 1 +NOTICE: The average of pg_target.val is equal to citus_target.val +CONTEXT: PL/pgSQL function check_data(text,text,text,text) line XX at RAISE +SQL function "compare_data" statement 2 + compare_data +--------------------------------------------------------------------- + +(1 row) + +DROP SCHEMA merge_partition_tables CASCADE; +NOTICE: drop cascades to 8 other objects +DETAIL: drop cascades to table pg_target +drop cascades to table pg_source +drop cascades to function cleanup_data() +drop cascades to function setup_data() +drop cascades to function check_data(text,text,text,text) +drop cascades to function compare_data() +drop cascades to table citus_target +drop cascades to table citus_source diff --git a/src/test/regress/expected/merge_partition_tables_0.out b/src/test/regress/expected/merge_partition_tables_0.out new file mode 100644 index 000000000..a7e3fbf20 --- /dev/null +++ b/src/test/regress/expected/merge_partition_tables_0.out @@ -0,0 +1,6 @@ +SHOW server_version \gset +SELECT substring(:'server_version', '\d+')::int >= 15 AS server_version_ge_15 +\gset +\if :server_version_ge_15 +\else +\q diff --git a/src/test/regress/expected/merge_repartition1.out b/src/test/regress/expected/merge_repartition1.out new file mode 100644 index 000000000..0c3c47389 --- /dev/null +++ b/src/test/regress/expected/merge_repartition1.out @@ -0,0 +1,1245 @@ +SHOW server_version \gset +SELECT substring(:'server_version', '\d+')::int >= 15 AS server_version_ge_15 +\gset +\if :server_version_ge_15 +\else +\q +\endif +-- We create two sets of source and target tables, one set in Postgres and +-- the other in Citus distributed. We run the _exact_ MERGE SQL on both sets +-- and compare the final results of the target tables in Postgres and Citus. +-- The results should match. This process is repeated for various combinations +-- of MERGE SQL. +DROP SCHEMA IF EXISTS merge_repartition1_schema CASCADE; +NOTICE: schema "merge_repartition1_schema" does not exist, skipping +CREATE SCHEMA merge_repartition1_schema; +SET search_path TO merge_repartition1_schema; +SET citus.shard_count TO 4; +SET citus.next_shard_id TO 5000000; +SET citus.explain_all_tasks TO true; +SET citus.shard_replication_factor TO 1; +SET citus.max_adaptive_executor_pool_size TO 1; +SET client_min_messages = warning; +SELECT 1 FROM master_add_node('localhost', :master_port, groupid => 0); + ?column? +--------------------------------------------------------------------- + 1 +(1 row) + +RESET client_min_messages; +CREATE TABLE pg_target(id int, val int); +CREATE TABLE pg_source(id int, val int, const int); +CREATE TABLE citus_target(id int, val int); +CREATE TABLE citus_source(id int, val int, const int); +SELECT citus_add_local_table_to_metadata('citus_target'); + citus_add_local_table_to_metadata +--------------------------------------------------------------------- + +(1 row) + +SELECT citus_add_local_table_to_metadata('citus_source'); + citus_add_local_table_to_metadata +--------------------------------------------------------------------- + +(1 row) + +CREATE OR REPLACE FUNCTION cleanup_data() RETURNS VOID SET search_path TO merge_repartition1_schema AS $$ + TRUNCATE pg_target; + TRUNCATE pg_source; + TRUNCATE citus_target; + TRUNCATE citus_source; + SELECT undistribute_table('citus_target'); + SELECT undistribute_table('citus_source'); +$$ +LANGUAGE SQL; +-- +-- Load same set of data to both Postgres and Citus tables +-- +CREATE OR REPLACE FUNCTION setup_data() RETURNS VOID SET search_path TO merge_repartition1_schema AS $$ + INSERT INTO pg_source SELECT i, i+1, 1 FROM generate_series(1, 10000) i; + INSERT INTO pg_target SELECT i, 1 FROM generate_series(5001, 10000) i; + INSERT INTO citus_source SELECT i, i+1, 1 FROM generate_series(1, 10000) i; + INSERT INTO citus_target SELECT i, 1 FROM generate_series(5001, 10000) i; +$$ +LANGUAGE SQL; +-- +-- Compares the final target tables, merge-modified data, of both Postgres and Citus tables +-- +CREATE OR REPLACE FUNCTION check_data(table1_name text, column1_name text, table2_name text, column2_name text) +RETURNS VOID SET search_path TO merge_repartition1_schema AS $$ +DECLARE + table1_avg numeric; + table2_avg numeric; +BEGIN + EXECUTE format('SELECT COALESCE(AVG(%I), 0) FROM %I', column1_name, table1_name) INTO table1_avg; + EXECUTE format('SELECT COALESCE(AVG(%I), 0) FROM %I', column2_name, table2_name) INTO table2_avg; + + IF table1_avg > table2_avg THEN + RAISE EXCEPTION 'The average of %.% is greater than %.%', table1_name, column1_name, table2_name, column2_name; + ELSIF table1_avg < table2_avg THEN + RAISE EXCEPTION 'The average of %.% is less than %.%', table1_name, column1_name, table2_name, column2_name; + ELSE + RAISE NOTICE 'The average of %.% is equal to %.%', table1_name, column1_name, table2_name, column2_name; + END IF; +END; +$$ LANGUAGE plpgsql; +CREATE OR REPLACE FUNCTION compare_data() RETURNS VOID SET search_path TO merge_repartition1_schema AS $$ + SELECT check_data('pg_target', 'id', 'citus_target', 'id'); + SELECT check_data('pg_target', 'val', 'citus_target', 'val'); +$$ +LANGUAGE SQL; +-- +-- Target and source are distributed, and non-colocated +-- +SELECT cleanup_data(); +NOTICE: creating a new table for merge_repartition1_schema.citus_target +CONTEXT: SQL function "cleanup_data" statement 5 +NOTICE: moving the data of merge_repartition1_schema.citus_target +CONTEXT: SQL function "cleanup_data" statement 5 +NOTICE: dropping the old merge_repartition1_schema.citus_target +CONTEXT: SQL function "cleanup_data" statement 5 +NOTICE: renaming the new table to merge_repartition1_schema.citus_target +CONTEXT: SQL function "cleanup_data" statement 5 +NOTICE: creating a new table for merge_repartition1_schema.citus_source +CONTEXT: SQL function "cleanup_data" statement 6 +NOTICE: moving the data of merge_repartition1_schema.citus_source +CONTEXT: SQL function "cleanup_data" statement 6 +NOTICE: dropping the old merge_repartition1_schema.citus_source +CONTEXT: SQL function "cleanup_data" statement 6 +NOTICE: renaming the new table to merge_repartition1_schema.citus_source +CONTEXT: SQL function "cleanup_data" statement 6 + cleanup_data +--------------------------------------------------------------------- + +(1 row) + +SELECT setup_data(); + setup_data +--------------------------------------------------------------------- + +(1 row) + +SELECT create_distributed_table('citus_target', 'id'); +NOTICE: Copying data from local table... +NOTICE: copying the data has completed +DETAIL: The local data in the table is no longer visible, but is still on disk. +HINT: To remove the local data, run: SELECT truncate_local_data_after_distributing_table($$merge_repartition1_schema.citus_target$$) + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +SELECT create_distributed_table('citus_source', 'id', colocate_with=>'none'); +NOTICE: Copying data from local table... +NOTICE: copying the data has completed +DETAIL: The local data in the table is no longer visible, but is still on disk. +HINT: To remove the local data, run: SELECT truncate_local_data_after_distributing_table($$merge_repartition1_schema.citus_source$$) + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +MERGE INTO pg_target t +USING pg_source s +ON t.id = s.id +WHEN MATCHED AND t.id <= 7500 THEN + UPDATE SET val = s.val + 1 +WHEN MATCHED THEN + DELETE +WHEN NOT MATCHED THEN + INSERT VALUES(s.id, s.val); +MERGE INTO citus_target t +USING citus_source s +ON t.id = s.id +WHEN MATCHED AND t.id <= 7500 THEN + UPDATE SET val = s.val + 1 +WHEN MATCHED THEN + DELETE +WHEN NOT MATCHED THEN + INSERT VALUES(s.id, s.val); +SELECT compare_data(); +NOTICE: The average of pg_target.id is equal to citus_target.id +CONTEXT: PL/pgSQL function check_data(text,text,text,text) line XX at RAISE +SQL function "compare_data" statement 1 +NOTICE: The average of pg_target.val is equal to citus_target.val +CONTEXT: PL/pgSQL function check_data(text,text,text,text) line XX at RAISE +SQL function "compare_data" statement 2 + compare_data +--------------------------------------------------------------------- + +(1 row) + +-- +-- Target and source are distributed, and colocated but not joined on distribution column +-- +SELECT cleanup_data(); +NOTICE: creating a new table for merge_repartition1_schema.citus_target +CONTEXT: SQL function "cleanup_data" statement 5 +NOTICE: moving the data of merge_repartition1_schema.citus_target +CONTEXT: SQL function "cleanup_data" statement 5 +NOTICE: dropping the old merge_repartition1_schema.citus_target +CONTEXT: SQL function "cleanup_data" statement 5 +NOTICE: renaming the new table to merge_repartition1_schema.citus_target +CONTEXT: SQL function "cleanup_data" statement 5 +NOTICE: creating a new table for merge_repartition1_schema.citus_source +CONTEXT: SQL function "cleanup_data" statement 6 +NOTICE: moving the data of merge_repartition1_schema.citus_source +CONTEXT: SQL function "cleanup_data" statement 6 +NOTICE: dropping the old merge_repartition1_schema.citus_source +CONTEXT: SQL function "cleanup_data" statement 6 +NOTICE: renaming the new table to merge_repartition1_schema.citus_source +CONTEXT: SQL function "cleanup_data" statement 6 + cleanup_data +--------------------------------------------------------------------- + +(1 row) + +SELECT setup_data(); + setup_data +--------------------------------------------------------------------- + +(1 row) + +SELECT create_distributed_table('citus_target', 'id'); +NOTICE: Copying data from local table... +NOTICE: copying the data has completed +DETAIL: The local data in the table is no longer visible, but is still on disk. +HINT: To remove the local data, run: SELECT truncate_local_data_after_distributing_table($$merge_repartition1_schema.citus_target$$) + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +SELECT create_distributed_table('citus_source', 'id', colocate_with=>'citus_target'); +NOTICE: Copying data from local table... +NOTICE: copying the data has completed +DETAIL: The local data in the table is no longer visible, but is still on disk. +HINT: To remove the local data, run: SELECT truncate_local_data_after_distributing_table($$merge_repartition1_schema.citus_source$$) + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +MERGE INTO pg_target t +USING (SELECT * FROM pg_source) subq +ON (subq.val = t.id) +WHEN MATCHED AND t.id <= 7500 THEN + UPDATE SET val = subq.val + 1 +WHEN MATCHED THEN + DELETE +WHEN NOT MATCHED THEN + INSERT VALUES(subq.val, subq.id); +MERGE INTO citus_target t +USING (SELECT * FROM citus_source) subq +ON (subq.val = t.id) +WHEN MATCHED AND t.id <= 7500 THEN + UPDATE SET val = subq.val + 1 +WHEN MATCHED THEN + DELETE +WHEN NOT MATCHED THEN + INSERT VALUES(subq.val, subq.id); +SELECT compare_data(); +NOTICE: The average of pg_target.id is equal to citus_target.id +CONTEXT: PL/pgSQL function check_data(text,text,text,text) line XX at RAISE +SQL function "compare_data" statement 1 +NOTICE: The average of pg_target.val is equal to citus_target.val +CONTEXT: PL/pgSQL function check_data(text,text,text,text) line XX at RAISE +SQL function "compare_data" statement 2 + compare_data +--------------------------------------------------------------------- + +(1 row) + +-- +-- Target and source are distributed, colocated, joined on distribution column +-- but with nondistribution values +-- +SELECT cleanup_data(); +NOTICE: creating a new table for merge_repartition1_schema.citus_target +CONTEXT: SQL function "cleanup_data" statement 5 +NOTICE: moving the data of merge_repartition1_schema.citus_target +CONTEXT: SQL function "cleanup_data" statement 5 +NOTICE: dropping the old merge_repartition1_schema.citus_target +CONTEXT: SQL function "cleanup_data" statement 5 +NOTICE: renaming the new table to merge_repartition1_schema.citus_target +CONTEXT: SQL function "cleanup_data" statement 5 +NOTICE: creating a new table for merge_repartition1_schema.citus_source +CONTEXT: SQL function "cleanup_data" statement 6 +NOTICE: moving the data of merge_repartition1_schema.citus_source +CONTEXT: SQL function "cleanup_data" statement 6 +NOTICE: dropping the old merge_repartition1_schema.citus_source +CONTEXT: SQL function "cleanup_data" statement 6 +NOTICE: renaming the new table to merge_repartition1_schema.citus_source +CONTEXT: SQL function "cleanup_data" statement 6 + cleanup_data +--------------------------------------------------------------------- + +(1 row) + +SELECT setup_data(); + setup_data +--------------------------------------------------------------------- + +(1 row) + +SELECT create_distributed_table('citus_target', 'id'); +NOTICE: Copying data from local table... +NOTICE: copying the data has completed +DETAIL: The local data in the table is no longer visible, but is still on disk. +HINT: To remove the local data, run: SELECT truncate_local_data_after_distributing_table($$merge_repartition1_schema.citus_target$$) + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +SELECT create_distributed_table('citus_source', 'id', colocate_with=>'citus_target'); +NOTICE: Copying data from local table... +NOTICE: copying the data has completed +DETAIL: The local data in the table is no longer visible, but is still on disk. +HINT: To remove the local data, run: SELECT truncate_local_data_after_distributing_table($$merge_repartition1_schema.citus_source$$) + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +MERGE INTO pg_target t +USING (SELECT id,const FROM pg_source UNION SELECT const,id FROM pg_source ) AS s +ON t.id = s.id +WHEN MATCHED THEN + UPDATE SET val = s.const + 1 +WHEN NOT MATCHED THEN + INSERT VALUES(id, const); +MERGE INTO citus_target t +USING (SELECT id,const FROM citus_source UNION SELECT const,id FROM citus_source) AS s +ON t.id = s.id +WHEN MATCHED THEN + UPDATE SET val = s.const + 1 +WHEN NOT MATCHED THEN + INSERT VALUES(id, const); +SELECT compare_data(); +NOTICE: The average of pg_target.id is equal to citus_target.id +CONTEXT: PL/pgSQL function check_data(text,text,text,text) line XX at RAISE +SQL function "compare_data" statement 1 +NOTICE: The average of pg_target.val is equal to citus_target.val +CONTEXT: PL/pgSQL function check_data(text,text,text,text) line XX at RAISE +SQL function "compare_data" statement 2 + compare_data +--------------------------------------------------------------------- + +(1 row) + +-- +-- Repartition with a predicate on target_table_name rows in ON clause +-- +SELECT cleanup_data(); +NOTICE: creating a new table for merge_repartition1_schema.citus_target +CONTEXT: SQL function "cleanup_data" statement 5 +NOTICE: moving the data of merge_repartition1_schema.citus_target +CONTEXT: SQL function "cleanup_data" statement 5 +NOTICE: dropping the old merge_repartition1_schema.citus_target +CONTEXT: SQL function "cleanup_data" statement 5 +NOTICE: renaming the new table to merge_repartition1_schema.citus_target +CONTEXT: SQL function "cleanup_data" statement 5 +NOTICE: creating a new table for merge_repartition1_schema.citus_source +CONTEXT: SQL function "cleanup_data" statement 6 +NOTICE: moving the data of merge_repartition1_schema.citus_source +CONTEXT: SQL function "cleanup_data" statement 6 +NOTICE: dropping the old merge_repartition1_schema.citus_source +CONTEXT: SQL function "cleanup_data" statement 6 +NOTICE: renaming the new table to merge_repartition1_schema.citus_source +CONTEXT: SQL function "cleanup_data" statement 6 + cleanup_data +--------------------------------------------------------------------- + +(1 row) + +SELECT setup_data(); + setup_data +--------------------------------------------------------------------- + +(1 row) + +SELECT create_distributed_table('citus_target', 'id'); +NOTICE: Copying data from local table... +NOTICE: copying the data has completed +DETAIL: The local data in the table is no longer visible, but is still on disk. +HINT: To remove the local data, run: SELECT truncate_local_data_after_distributing_table($$merge_repartition1_schema.citus_target$$) + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +SELECT create_distributed_table('citus_source', 'id', colocate_with=>'none'); +NOTICE: Copying data from local table... +NOTICE: copying the data has completed +DETAIL: The local data in the table is no longer visible, but is still on disk. +HINT: To remove the local data, run: SELECT truncate_local_data_after_distributing_table($$merge_repartition1_schema.citus_source$$) + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +MERGE INTO pg_target t +USING (SELECT * FROM pg_source WHERE id < 9500) s +ON t.id = s.id AND t.id < 9000 +WHEN MATCHED AND t.id <= 7500 THEN + UPDATE SET val = s.val + 1 +WHEN MATCHED THEN + DELETE +WHEN NOT MATCHED THEN + INSERT VALUES(s.id, s.val); +MERGE INTO citus_target t +USING (SELECT * FROM citus_source WHERE id < 9500) s +ON t.id = s.id AND t.id < 9000 +WHEN MATCHED AND t.id <= 7500 THEN + UPDATE SET val = s.val + 1 +WHEN MATCHED THEN + DELETE +WHEN NOT MATCHED THEN + INSERT VALUES(s.id, s.val); +SELECT compare_data(); +NOTICE: The average of pg_target.id is equal to citus_target.id +CONTEXT: PL/pgSQL function check_data(text,text,text,text) line XX at RAISE +SQL function "compare_data" statement 1 +NOTICE: The average of pg_target.val is equal to citus_target.val +CONTEXT: PL/pgSQL function check_data(text,text,text,text) line XX at RAISE +SQL function "compare_data" statement 2 + compare_data +--------------------------------------------------------------------- + +(1 row) + +-- +-- Test CTE and non-colocated tables +-- +SELECT cleanup_data(); +NOTICE: creating a new table for merge_repartition1_schema.citus_target +CONTEXT: SQL function "cleanup_data" statement 5 +NOTICE: moving the data of merge_repartition1_schema.citus_target +CONTEXT: SQL function "cleanup_data" statement 5 +NOTICE: dropping the old merge_repartition1_schema.citus_target +CONTEXT: SQL function "cleanup_data" statement 5 +NOTICE: renaming the new table to merge_repartition1_schema.citus_target +CONTEXT: SQL function "cleanup_data" statement 5 +NOTICE: creating a new table for merge_repartition1_schema.citus_source +CONTEXT: SQL function "cleanup_data" statement 6 +NOTICE: moving the data of merge_repartition1_schema.citus_source +CONTEXT: SQL function "cleanup_data" statement 6 +NOTICE: dropping the old merge_repartition1_schema.citus_source +CONTEXT: SQL function "cleanup_data" statement 6 +NOTICE: renaming the new table to merge_repartition1_schema.citus_source +CONTEXT: SQL function "cleanup_data" statement 6 + cleanup_data +--------------------------------------------------------------------- + +(1 row) + +SELECT setup_data(); + setup_data +--------------------------------------------------------------------- + +(1 row) + +SELECT create_distributed_table('citus_target', 'id'); +NOTICE: Copying data from local table... +NOTICE: copying the data has completed +DETAIL: The local data in the table is no longer visible, but is still on disk. +HINT: To remove the local data, run: SELECT truncate_local_data_after_distributing_table($$merge_repartition1_schema.citus_target$$) + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +SELECT create_distributed_table('citus_source', 'id', colocate_with=>'none'); +NOTICE: Copying data from local table... +NOTICE: copying the data has completed +DETAIL: The local data in the table is no longer visible, but is still on disk. +HINT: To remove the local data, run: SELECT truncate_local_data_after_distributing_table($$merge_repartition1_schema.citus_source$$) + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +WITH cte AS ( + SELECT * FROM pg_source +) +MERGE INTO pg_target t +USING cte s +ON s.id = t.id +WHEN MATCHED AND t.id > 7500 THEN + UPDATE SET val = s.val + 1 +WHEN MATCHED THEN + DELETE +WHEN NOT MATCHED THEN + INSERT VALUES (s.id, s.val); +WITH cte AS ( + SELECT * FROM citus_source +) +MERGE INTO citus_target t +USING cte s +ON s.id = t.id +WHEN MATCHED AND t.id > 7500 THEN + UPDATE SET val = s.val + 1 +WHEN MATCHED THEN + DELETE +WHEN NOT MATCHED THEN + INSERT VALUES (s.id, s.val); +SELECT compare_data(); +NOTICE: The average of pg_target.id is equal to citus_target.id +CONTEXT: PL/pgSQL function check_data(text,text,text,text) line XX at RAISE +SQL function "compare_data" statement 1 +NOTICE: The average of pg_target.val is equal to citus_target.val +CONTEXT: PL/pgSQL function check_data(text,text,text,text) line XX at RAISE +SQL function "compare_data" statement 2 + compare_data +--------------------------------------------------------------------- + +(1 row) + +-- +-- Test nested CTEs +-- +SELECT cleanup_data(); +NOTICE: creating a new table for merge_repartition1_schema.citus_target +CONTEXT: SQL function "cleanup_data" statement 5 +NOTICE: moving the data of merge_repartition1_schema.citus_target +CONTEXT: SQL function "cleanup_data" statement 5 +NOTICE: dropping the old merge_repartition1_schema.citus_target +CONTEXT: SQL function "cleanup_data" statement 5 +NOTICE: renaming the new table to merge_repartition1_schema.citus_target +CONTEXT: SQL function "cleanup_data" statement 5 +NOTICE: creating a new table for merge_repartition1_schema.citus_source +CONTEXT: SQL function "cleanup_data" statement 6 +NOTICE: moving the data of merge_repartition1_schema.citus_source +CONTEXT: SQL function "cleanup_data" statement 6 +NOTICE: dropping the old merge_repartition1_schema.citus_source +CONTEXT: SQL function "cleanup_data" statement 6 +NOTICE: renaming the new table to merge_repartition1_schema.citus_source +CONTEXT: SQL function "cleanup_data" statement 6 + cleanup_data +--------------------------------------------------------------------- + +(1 row) + +SELECT setup_data(); + setup_data +--------------------------------------------------------------------- + +(1 row) + +SELECT create_distributed_table('citus_target', 'id'); +NOTICE: Copying data from local table... +NOTICE: copying the data has completed +DETAIL: The local data in the table is no longer visible, but is still on disk. +HINT: To remove the local data, run: SELECT truncate_local_data_after_distributing_table($$merge_repartition1_schema.citus_target$$) + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +SELECT create_distributed_table('citus_source', 'id', colocate_with=>'none'); +NOTICE: Copying data from local table... +NOTICE: copying the data has completed +DETAIL: The local data in the table is no longer visible, but is still on disk. +HINT: To remove the local data, run: SELECT truncate_local_data_after_distributing_table($$merge_repartition1_schema.citus_source$$) + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +WITH cte1 AS ( + SELECT * FROM pg_source ORDER BY 1 LIMIT 9000 +), +cte2 AS( + SELECT * FROM cte1 +), +cte3 AS( + SELECT * FROM cte2 +) +MERGE INTO pg_target t +USING cte3 s +ON (s.id=t.id) +WHEN MATCHED AND t.id > 7500 THEN + UPDATE SET val = s.val + 1 +WHEN MATCHED THEN + DELETE +WHEN NOT MATCHED THEN + INSERT VALUES (s.id, s.val); +WITH cte1 AS ( + SELECT * FROM citus_source ORDER BY 1 LIMIT 9000 +), +cte2 AS( + SELECT * FROM cte1 +), +cte3 AS( + SELECT * FROM cte2 +) +MERGE INTO citus_target t +USING cte3 s +ON (s.id=t.id) +WHEN MATCHED AND t.id > 7500 THEN + UPDATE SET val = s.val + 1 +WHEN MATCHED THEN + DELETE +WHEN NOT MATCHED THEN + INSERT VALUES (s.id, s.val); +SELECT compare_data(); +NOTICE: The average of pg_target.id is equal to citus_target.id +CONTEXT: PL/pgSQL function check_data(text,text,text,text) line XX at RAISE +SQL function "compare_data" statement 1 +NOTICE: The average of pg_target.val is equal to citus_target.val +CONTEXT: PL/pgSQL function check_data(text,text,text,text) line XX at RAISE +SQL function "compare_data" statement 2 + compare_data +--------------------------------------------------------------------- + +(1 row) + +-- +-- Target and source are distributed and colocated +-- +SELECT cleanup_data(); +NOTICE: creating a new table for merge_repartition1_schema.citus_target +CONTEXT: SQL function "cleanup_data" statement 5 +NOTICE: moving the data of merge_repartition1_schema.citus_target +CONTEXT: SQL function "cleanup_data" statement 5 +NOTICE: dropping the old merge_repartition1_schema.citus_target +CONTEXT: SQL function "cleanup_data" statement 5 +NOTICE: renaming the new table to merge_repartition1_schema.citus_target +CONTEXT: SQL function "cleanup_data" statement 5 +NOTICE: creating a new table for merge_repartition1_schema.citus_source +CONTEXT: SQL function "cleanup_data" statement 6 +NOTICE: moving the data of merge_repartition1_schema.citus_source +CONTEXT: SQL function "cleanup_data" statement 6 +NOTICE: dropping the old merge_repartition1_schema.citus_source +CONTEXT: SQL function "cleanup_data" statement 6 +NOTICE: renaming the new table to merge_repartition1_schema.citus_source +CONTEXT: SQL function "cleanup_data" statement 6 + cleanup_data +--------------------------------------------------------------------- + +(1 row) + +SELECT setup_data(); + setup_data +--------------------------------------------------------------------- + +(1 row) + +SELECT create_distributed_table('citus_target', 'id'); +NOTICE: Copying data from local table... +NOTICE: copying the data has completed +DETAIL: The local data in the table is no longer visible, but is still on disk. +HINT: To remove the local data, run: SELECT truncate_local_data_after_distributing_table($$merge_repartition1_schema.citus_target$$) + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +SELECT create_distributed_table('citus_source', 'id', colocate_with => 'citus_target'); +NOTICE: Copying data from local table... +NOTICE: copying the data has completed +DETAIL: The local data in the table is no longer visible, but is still on disk. +HINT: To remove the local data, run: SELECT truncate_local_data_after_distributing_table($$merge_repartition1_schema.citus_source$$) + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +MERGE INTO pg_target t +USING (SELECT 999 as newval, pg_source.* FROM (SELECT * FROM pg_source ORDER BY 1 LIMIT 6000) as src LEFT JOIN pg_source USING(id)) AS s +ON t.id = s.id +WHEN MATCHED AND t.id <= 5500 THEN + UPDATE SET val = newval +WHEN MATCHED THEN + DELETE +WHEN NOT MATCHED THEN + INSERT VALUES(id, newval); +MERGE INTO citus_target t +USING (SELECT 999 as newval, citus_source.* FROM (SELECT * FROM citus_source ORDER BY 1 LIMIT 6000) as src LEFT JOIN citus_source USING(id)) AS s +ON t.id = s.id +WHEN MATCHED AND t.id <= 5500 THEN + UPDATE SET val = newval +WHEN MATCHED THEN + DELETE +WHEN NOT MATCHED THEN + INSERT VALUES(id, newval); +SELECT compare_data(); +NOTICE: The average of pg_target.id is equal to citus_target.id +CONTEXT: PL/pgSQL function check_data(text,text,text,text) line XX at RAISE +SQL function "compare_data" statement 1 +NOTICE: The average of pg_target.val is equal to citus_target.val +CONTEXT: PL/pgSQL function check_data(text,text,text,text) line XX at RAISE +SQL function "compare_data" statement 2 + compare_data +--------------------------------------------------------------------- + +(1 row) + +-- +-- Target is distributed and source is reference +-- +SELECT cleanup_data(); +NOTICE: creating a new table for merge_repartition1_schema.citus_target +CONTEXT: SQL function "cleanup_data" statement 5 +NOTICE: moving the data of merge_repartition1_schema.citus_target +CONTEXT: SQL function "cleanup_data" statement 5 +NOTICE: dropping the old merge_repartition1_schema.citus_target +CONTEXT: SQL function "cleanup_data" statement 5 +NOTICE: renaming the new table to merge_repartition1_schema.citus_target +CONTEXT: SQL function "cleanup_data" statement 5 +NOTICE: creating a new table for merge_repartition1_schema.citus_source +CONTEXT: SQL function "cleanup_data" statement 6 +NOTICE: moving the data of merge_repartition1_schema.citus_source +CONTEXT: SQL function "cleanup_data" statement 6 +NOTICE: dropping the old merge_repartition1_schema.citus_source +CONTEXT: SQL function "cleanup_data" statement 6 +NOTICE: renaming the new table to merge_repartition1_schema.citus_source +CONTEXT: SQL function "cleanup_data" statement 6 + cleanup_data +--------------------------------------------------------------------- + +(1 row) + +SELECT setup_data(); + setup_data +--------------------------------------------------------------------- + +(1 row) + +SELECT create_distributed_table('citus_target', 'id'); +NOTICE: Copying data from local table... +NOTICE: copying the data has completed +DETAIL: The local data in the table is no longer visible, but is still on disk. +HINT: To remove the local data, run: SELECT truncate_local_data_after_distributing_table($$merge_repartition1_schema.citus_target$$) + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +SELECT create_reference_table('citus_source'); +NOTICE: Copying data from local table... +NOTICE: copying the data has completed +DETAIL: The local data in the table is no longer visible, but is still on disk. +HINT: To remove the local data, run: SELECT truncate_local_data_after_distributing_table($$merge_repartition1_schema.citus_source$$) + create_reference_table +--------------------------------------------------------------------- + +(1 row) + +MERGE INTO pg_target t +USING pg_source s +ON t.id = s.id +WHEN MATCHED AND t.id <= 7500 THEN + UPDATE SET val = s.val + 1 +WHEN MATCHED THEN + DELETE +WHEN NOT MATCHED THEN + INSERT VALUES(s.id, s.val); +MERGE INTO citus_target t +USING citus_source s +ON t.id = s.id +WHEN MATCHED AND t.id <= 7500 THEN + UPDATE SET val = s.val + 1 +WHEN MATCHED THEN + DELETE +WHEN NOT MATCHED THEN + INSERT VALUES(s.id, s.val); +SELECT compare_data(); +NOTICE: The average of pg_target.id is equal to citus_target.id +CONTEXT: PL/pgSQL function check_data(text,text,text,text) line XX at RAISE +SQL function "compare_data" statement 1 +NOTICE: The average of pg_target.val is equal to citus_target.val +CONTEXT: PL/pgSQL function check_data(text,text,text,text) line XX at RAISE +SQL function "compare_data" statement 2 + compare_data +--------------------------------------------------------------------- + +(1 row) + +-- +-- Target is distributed and reference as source in a sub-query +-- +SELECT cleanup_data(); +NOTICE: creating a new table for merge_repartition1_schema.citus_target +CONTEXT: SQL function "cleanup_data" statement 5 +NOTICE: moving the data of merge_repartition1_schema.citus_target +CONTEXT: SQL function "cleanup_data" statement 5 +NOTICE: dropping the old merge_repartition1_schema.citus_target +CONTEXT: SQL function "cleanup_data" statement 5 +NOTICE: renaming the new table to merge_repartition1_schema.citus_target +CONTEXT: SQL function "cleanup_data" statement 5 +NOTICE: creating a new table for merge_repartition1_schema.citus_source +CONTEXT: SQL function "cleanup_data" statement 6 +NOTICE: moving the data of merge_repartition1_schema.citus_source +CONTEXT: SQL function "cleanup_data" statement 6 +NOTICE: dropping the old merge_repartition1_schema.citus_source +CONTEXT: SQL function "cleanup_data" statement 6 +NOTICE: renaming the new table to merge_repartition1_schema.citus_source +CONTEXT: SQL function "cleanup_data" statement 6 + cleanup_data +--------------------------------------------------------------------- + +(1 row) + +SELECT setup_data(); + setup_data +--------------------------------------------------------------------- + +(1 row) + +SELECT create_distributed_table('citus_target', 'id'); +NOTICE: Copying data from local table... +NOTICE: copying the data has completed +DETAIL: The local data in the table is no longer visible, but is still on disk. +HINT: To remove the local data, run: SELECT truncate_local_data_after_distributing_table($$merge_repartition1_schema.citus_target$$) + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +SELECT create_reference_table('citus_source'); +NOTICE: Copying data from local table... +NOTICE: copying the data has completed +DETAIL: The local data in the table is no longer visible, but is still on disk. +HINT: To remove the local data, run: SELECT truncate_local_data_after_distributing_table($$merge_repartition1_schema.citus_source$$) + create_reference_table +--------------------------------------------------------------------- + +(1 row) + +MERGE INTO pg_target t +USING (SELECT * FROM pg_source UNION SELECT * FROM pg_source) AS s ON t.id = s.id +WHEN MATCHED AND t.id <= 7500 THEN + UPDATE SET val = s.val + t.val +WHEN MATCHED THEN + DELETE +WHEN NOT MATCHED THEN + INSERT VALUES(s.id, s.val); +MERGE INTO citus_target t +USING (SELECT * FROM citus_source UNION SELECT * FROM citus_source) AS s ON t.id = s.id +WHEN MATCHED AND t.id <= 7500 THEN + UPDATE SET val = s.val + t.val +WHEN MATCHED THEN + DELETE +WHEN NOT MATCHED THEN + INSERT VALUES(s.id, s.val); +SELECT compare_data(); +NOTICE: The average of pg_target.id is equal to citus_target.id +CONTEXT: PL/pgSQL function check_data(text,text,text,text) line XX at RAISE +SQL function "compare_data" statement 1 +NOTICE: The average of pg_target.val is equal to citus_target.val +CONTEXT: PL/pgSQL function check_data(text,text,text,text) line XX at RAISE +SQL function "compare_data" statement 2 + compare_data +--------------------------------------------------------------------- + +(1 row) + +-- +-- Target is distributed and citus-local as source +-- +SELECT cleanup_data(); +NOTICE: creating a new table for merge_repartition1_schema.citus_target +CONTEXT: SQL function "cleanup_data" statement 5 +NOTICE: moving the data of merge_repartition1_schema.citus_target +CONTEXT: SQL function "cleanup_data" statement 5 +NOTICE: dropping the old merge_repartition1_schema.citus_target +CONTEXT: SQL function "cleanup_data" statement 5 +NOTICE: renaming the new table to merge_repartition1_schema.citus_target +CONTEXT: SQL function "cleanup_data" statement 5 +NOTICE: creating a new table for merge_repartition1_schema.citus_source +CONTEXT: SQL function "cleanup_data" statement 6 +NOTICE: moving the data of merge_repartition1_schema.citus_source +CONTEXT: SQL function "cleanup_data" statement 6 +NOTICE: dropping the old merge_repartition1_schema.citus_source +CONTEXT: SQL function "cleanup_data" statement 6 +NOTICE: renaming the new table to merge_repartition1_schema.citus_source +CONTEXT: SQL function "cleanup_data" statement 6 + cleanup_data +--------------------------------------------------------------------- + +(1 row) + +SELECT setup_data(); + setup_data +--------------------------------------------------------------------- + +(1 row) + +SELECT create_distributed_table('citus_target', 'id'); +NOTICE: Copying data from local table... +NOTICE: copying the data has completed +DETAIL: The local data in the table is no longer visible, but is still on disk. +HINT: To remove the local data, run: SELECT truncate_local_data_after_distributing_table($$merge_repartition1_schema.citus_target$$) + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +SELECT citus_add_local_table_to_metadata('citus_source'); + citus_add_local_table_to_metadata +--------------------------------------------------------------------- + +(1 row) + +MERGE INTO pg_target t +USING pg_source s +ON t.id = s.id +WHEN MATCHED AND t.id <= 7500 THEN + UPDATE SET val = s.val + 1 +WHEN MATCHED THEN + DELETE +WHEN NOT MATCHED THEN + INSERT VALUES(s.id, s.val); +MERGE INTO citus_target t +USING citus_source s +ON t.id = s.id +WHEN MATCHED AND t.id <= 7500 THEN + UPDATE SET val = s.val + 1 +WHEN MATCHED THEN + DELETE +WHEN NOT MATCHED THEN + INSERT VALUES(s.id, s.val); +SELECT compare_data(); +NOTICE: The average of pg_target.id is equal to citus_target.id +CONTEXT: PL/pgSQL function check_data(text,text,text,text) line XX at RAISE +SQL function "compare_data" statement 1 +NOTICE: The average of pg_target.val is equal to citus_target.val +CONTEXT: PL/pgSQL function check_data(text,text,text,text) line XX at RAISE +SQL function "compare_data" statement 2 + compare_data +--------------------------------------------------------------------- + +(1 row) + +-- +-- Target and source distributed and non-colocated. The source query requires evaluation +-- at the coordinator +-- +SELECT cleanup_data(); +NOTICE: creating a new table for merge_repartition1_schema.citus_target +CONTEXT: SQL function "cleanup_data" statement 5 +NOTICE: moving the data of merge_repartition1_schema.citus_target +CONTEXT: SQL function "cleanup_data" statement 5 +NOTICE: dropping the old merge_repartition1_schema.citus_target +CONTEXT: SQL function "cleanup_data" statement 5 +NOTICE: renaming the new table to merge_repartition1_schema.citus_target +CONTEXT: SQL function "cleanup_data" statement 5 +NOTICE: creating a new table for merge_repartition1_schema.citus_source +CONTEXT: SQL function "cleanup_data" statement 6 +NOTICE: moving the data of merge_repartition1_schema.citus_source +CONTEXT: SQL function "cleanup_data" statement 6 +NOTICE: dropping the old merge_repartition1_schema.citus_source +CONTEXT: SQL function "cleanup_data" statement 6 +NOTICE: renaming the new table to merge_repartition1_schema.citus_source +CONTEXT: SQL function "cleanup_data" statement 6 + cleanup_data +--------------------------------------------------------------------- + +(1 row) + +SELECT setup_data(); + setup_data +--------------------------------------------------------------------- + +(1 row) + +SELECT create_distributed_table('citus_target', 'id'); +NOTICE: Copying data from local table... +NOTICE: copying the data has completed +DETAIL: The local data in the table is no longer visible, but is still on disk. +HINT: To remove the local data, run: SELECT truncate_local_data_after_distributing_table($$merge_repartition1_schema.citus_target$$) + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +SELECT create_distributed_table('citus_source', 'id', colocate_with=>'none'); +NOTICE: Copying data from local table... +NOTICE: copying the data has completed +DETAIL: The local data in the table is no longer visible, but is still on disk. +HINT: To remove the local data, run: SELECT truncate_local_data_after_distributing_table($$merge_repartition1_schema.citus_source$$) + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +MERGE INTO pg_target t +USING (SELECT 100 AS insval, MAX(const) AS updval, val, MAX(id) AS sid + FROM pg_source + GROUP BY val ORDER BY sid LIMIT 6000) AS s +ON t.id = s.sid +WHEN MATCHED AND t.id <= 5500 THEN + UPDATE SET val = updval + 1 +WHEN MATCHED THEN + DELETE +WHEN NOT MATCHED THEN + INSERT VALUES(sid, insval); +MERGE INTO citus_target t +USING (SELECT 100 AS insval, MAX(const) AS updval, val, MAX(id) AS sid + FROM citus_source + GROUP BY val ORDER BY sid LIMIT 6000) AS s +ON t.id = s.sid +WHEN MATCHED AND t.id <= 5500 THEN + UPDATE SET val = updval + 1 +WHEN MATCHED THEN + DELETE +WHEN NOT MATCHED THEN + INSERT VALUES(sid, insval); +SELECT compare_data(); +NOTICE: The average of pg_target.id is equal to citus_target.id +CONTEXT: PL/pgSQL function check_data(text,text,text,text) line XX at RAISE +SQL function "compare_data" statement 1 +NOTICE: The average of pg_target.val is equal to citus_target.val +CONTEXT: PL/pgSQL function check_data(text,text,text,text) line XX at RAISE +SQL function "compare_data" statement 2 + compare_data +--------------------------------------------------------------------- + +(1 row) + +-- Test source-query that requires repartitioning on top of MERGE repartitioning +SELECT cleanup_data(); +NOTICE: creating a new table for merge_repartition1_schema.citus_target +CONTEXT: SQL function "cleanup_data" statement 5 +NOTICE: moving the data of merge_repartition1_schema.citus_target +CONTEXT: SQL function "cleanup_data" statement 5 +NOTICE: dropping the old merge_repartition1_schema.citus_target +CONTEXT: SQL function "cleanup_data" statement 5 +NOTICE: renaming the new table to merge_repartition1_schema.citus_target +CONTEXT: SQL function "cleanup_data" statement 5 +NOTICE: creating a new table for merge_repartition1_schema.citus_source +CONTEXT: SQL function "cleanup_data" statement 6 +NOTICE: moving the data of merge_repartition1_schema.citus_source +CONTEXT: SQL function "cleanup_data" statement 6 +NOTICE: dropping the old merge_repartition1_schema.citus_source +CONTEXT: SQL function "cleanup_data" statement 6 +NOTICE: renaming the new table to merge_repartition1_schema.citus_source +CONTEXT: SQL function "cleanup_data" statement 6 + cleanup_data +--------------------------------------------------------------------- + +(1 row) + +SELECT setup_data(); + setup_data +--------------------------------------------------------------------- + +(1 row) + +SELECT create_distributed_table('citus_target', 'id'); +NOTICE: Copying data from local table... +NOTICE: copying the data has completed +DETAIL: The local data in the table is no longer visible, but is still on disk. +HINT: To remove the local data, run: SELECT truncate_local_data_after_distributing_table($$merge_repartition1_schema.citus_target$$) + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +SELECT create_distributed_table('citus_source', 'id', colocate_with=>'none'); +NOTICE: Copying data from local table... +NOTICE: copying the data has completed +DETAIL: The local data in the table is no longer visible, but is still on disk. +HINT: To remove the local data, run: SELECT truncate_local_data_after_distributing_table($$merge_repartition1_schema.citus_source$$) + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +MERGE INTO pg_target t +USING (SELECT s1.val FROM pg_source s1 JOIN pg_source s2 USING (val)) AS s +ON t.id = s.val +WHEN MATCHED THEN + UPDATE SET val = t.val + 1; +SET citus.enable_repartition_joins TO true; +MERGE INTO citus_target t +USING (SELECT s1.val FROM citus_source s1 JOIN citus_source s2 USING (val)) AS s +ON t.id = s.val +WHEN MATCHED THEN + UPDATE SET val = t.val + 1; +SELECT compare_data(); +NOTICE: The average of pg_target.id is equal to citus_target.id +CONTEXT: PL/pgSQL function check_data(text,text,text,text) line XX at RAISE +SQL function "compare_data" statement 1 +NOTICE: The average of pg_target.val is equal to citus_target.val +CONTEXT: PL/pgSQL function check_data(text,text,text,text) line XX at RAISE +SQL function "compare_data" statement 2 + compare_data +--------------------------------------------------------------------- + +(1 row) + +-- +-- Test columnar as source table +-- +SET client_min_messages TO WARNING; +SELECT cleanup_data(); + cleanup_data +--------------------------------------------------------------------- + +(1 row) + +RESET client_min_messages; +SELECT setup_data(); + setup_data +--------------------------------------------------------------------- + +(1 row) + +SELECT create_distributed_table('citus_target', 'id'); +NOTICE: Copying data from local table... +NOTICE: copying the data has completed +DETAIL: The local data in the table is no longer visible, but is still on disk. +HINT: To remove the local data, run: SELECT truncate_local_data_after_distributing_table($$merge_repartition1_schema.citus_target$$) + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +SELECT create_distributed_table('citus_source', 'id', colocate_with=>'none'); +NOTICE: Copying data from local table... +NOTICE: copying the data has completed +DETAIL: The local data in the table is no longer visible, but is still on disk. +HINT: To remove the local data, run: SELECT truncate_local_data_after_distributing_table($$merge_repartition1_schema.citus_source$$) + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +SELECT alter_table_set_access_method('citus_source', 'columnar'); +NOTICE: creating a new table for merge_repartition1_schema.citus_source +NOTICE: moving the data of merge_repartition1_schema.citus_source +NOTICE: dropping the old merge_repartition1_schema.citus_source +NOTICE: renaming the new table to merge_repartition1_schema.citus_source + alter_table_set_access_method +--------------------------------------------------------------------- + +(1 row) + +MERGE INTO pg_target t +USING pg_source s +ON t.id = s.id +WHEN MATCHED AND t.id <= 7500 THEN + UPDATE SET val = s.val + 1 +WHEN MATCHED THEN + DELETE +WHEN NOT MATCHED THEN + INSERT VALUES(s.id, s.val); +MERGE INTO citus_target t +USING citus_source s +ON t.id = s.id +WHEN MATCHED AND t.id <= 7500 THEN + UPDATE SET val = s.val + 1 +WHEN MATCHED THEN + DELETE +WHEN NOT MATCHED THEN + INSERT VALUES(s.id, s.val); +SELECT compare_data(); +NOTICE: The average of pg_target.id is equal to citus_target.id +CONTEXT: PL/pgSQL function check_data(text,text,text,text) line XX at RAISE +SQL function "compare_data" statement 1 +NOTICE: The average of pg_target.val is equal to citus_target.val +CONTEXT: PL/pgSQL function check_data(text,text,text,text) line XX at RAISE +SQL function "compare_data" statement 2 + compare_data +--------------------------------------------------------------------- + +(1 row) + +SELECT alter_table_set_access_method('citus_source', 'heap'); +NOTICE: creating a new table for merge_repartition1_schema.citus_source +NOTICE: moving the data of merge_repartition1_schema.citus_source +NOTICE: dropping the old merge_repartition1_schema.citus_source +NOTICE: renaming the new table to merge_repartition1_schema.citus_source + alter_table_set_access_method +--------------------------------------------------------------------- + +(1 row) + +-- Test CTE/Subquery in merge-actions (works only for router query) +SELECT cleanup_data(); +NOTICE: creating a new table for merge_repartition1_schema.citus_target +CONTEXT: SQL function "cleanup_data" statement 5 +NOTICE: moving the data of merge_repartition1_schema.citus_target +CONTEXT: SQL function "cleanup_data" statement 5 +NOTICE: dropping the old merge_repartition1_schema.citus_target +CONTEXT: SQL function "cleanup_data" statement 5 +NOTICE: renaming the new table to merge_repartition1_schema.citus_target +CONTEXT: SQL function "cleanup_data" statement 5 +NOTICE: creating a new table for merge_repartition1_schema.citus_source +CONTEXT: SQL function "cleanup_data" statement 6 +NOTICE: moving the data of merge_repartition1_schema.citus_source +CONTEXT: SQL function "cleanup_data" statement 6 +NOTICE: dropping the old merge_repartition1_schema.citus_source +CONTEXT: SQL function "cleanup_data" statement 6 +NOTICE: renaming the new table to merge_repartition1_schema.citus_source +CONTEXT: SQL function "cleanup_data" statement 6 + cleanup_data +--------------------------------------------------------------------- + +(1 row) + +SELECT setup_data(); + setup_data +--------------------------------------------------------------------- + +(1 row) + +SELECT create_distributed_table('citus_target', 'id'); +NOTICE: Copying data from local table... +NOTICE: copying the data has completed +DETAIL: The local data in the table is no longer visible, but is still on disk. +HINT: To remove the local data, run: SELECT truncate_local_data_after_distributing_table($$merge_repartition1_schema.citus_target$$) + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +SELECT create_distributed_table('citus_source', 'id', colocate_with=>'citus_target'); +NOTICE: Copying data from local table... +NOTICE: copying the data has completed +DETAIL: The local data in the table is no longer visible, but is still on disk. +HINT: To remove the local data, run: SELECT truncate_local_data_after_distributing_table($$merge_repartition1_schema.citus_source$$) + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +MERGE INTO pg_target +USING pg_source +ON (pg_target.id = pg_source.id) +WHEN MATCHED AND (SELECT max_a > 5001 FROM (SELECT max(id) as max_a, max(val) as b FROM pg_target WHERE id = pg_source.id) AS foo) THEN + DELETE +WHEN NOT MATCHED AND (SELECT max_a < 5001 FROM (SELECT max(id) as max_a, max(val) as b FROM pg_target WHERE id = pg_source.id) AS foo) THEN + INSERT VALUES (pg_source.id, 100); +MERGE INTO citus_target +USING citus_source +ON (citus_target.id = citus_source.id) +WHEN MATCHED AND (SELECT max_a > 5001 FROM (SELECT max(id) as max_a, max(val) as b FROM citus_target WHERE id = citus_source.id) AS foo) THEN + DELETE +WHEN NOT MATCHED AND (SELECT max_a < 5001 FROM (SELECT max(id) as max_a, max(val) as b FROM citus_target WHERE id = citus_source.id) AS foo) THEN + INSERT VALUES (citus_source.id, 100); +SELECT compare_data(); +NOTICE: The average of pg_target.id is equal to citus_target.id +CONTEXT: PL/pgSQL function check_data(text,text,text,text) line XX at RAISE +SQL function "compare_data" statement 1 +NOTICE: The average of pg_target.val is equal to citus_target.val +CONTEXT: PL/pgSQL function check_data(text,text,text,text) line XX at RAISE +SQL function "compare_data" statement 2 + compare_data +--------------------------------------------------------------------- + +(1 row) + +DROP SCHEMA merge_repartition1_schema CASCADE; +NOTICE: drop cascades to 8 other objects +DETAIL: drop cascades to table pg_target +drop cascades to table pg_source +drop cascades to function cleanup_data() +drop cascades to function setup_data() +drop cascades to function check_data(text,text,text,text) +drop cascades to function compare_data() +drop cascades to table citus_target +drop cascades to table citus_source diff --git a/src/test/regress/expected/merge_repartition1_0.out b/src/test/regress/expected/merge_repartition1_0.out new file mode 100644 index 000000000..a7e3fbf20 --- /dev/null +++ b/src/test/regress/expected/merge_repartition1_0.out @@ -0,0 +1,6 @@ +SHOW server_version \gset +SELECT substring(:'server_version', '\d+')::int >= 15 AS server_version_ge_15 +\gset +\if :server_version_ge_15 +\else +\q diff --git a/src/test/regress/expected/merge_repartition2.out b/src/test/regress/expected/merge_repartition2.out new file mode 100644 index 000000000..898b7c77a --- /dev/null +++ b/src/test/regress/expected/merge_repartition2.out @@ -0,0 +1,212 @@ +SHOW server_version \gset +SELECT substring(:'server_version', '\d+')::int >= 15 AS server_version_ge_15 +\gset +\if :server_version_ge_15 +\else +\q +\endif +-- We create two sets of source and target tables, one set in Postgres and +-- the other in Citus distributed. We run the _exact_ MERGE SQL on both sets +-- and compare the final results of the target tables in Postgres and Citus. +-- The results should match. This process is repeated for various combinations +-- of MERGE SQL. +DROP SCHEMA IF EXISTS merge_repartition2_schema CASCADE; +NOTICE: schema "merge_repartition2_schema" does not exist, skipping +CREATE SCHEMA merge_repartition2_schema; +SET search_path TO merge_repartition2_schema; +SET citus.shard_count TO 4; +SET citus.next_shard_id TO 6000000; +SET citus.explain_all_tasks TO true; +SET citus.shard_replication_factor TO 1; +SET citus.max_adaptive_executor_pool_size TO 1; +SET client_min_messages = warning; +SELECT 1 FROM master_add_node('localhost', :master_port, groupid => 0); + ?column? +--------------------------------------------------------------------- + 1 +(1 row) + +RESET client_min_messages; +CREATE TABLE pg_target(id int, val int); +CREATE TABLE pg_source(id int, val int, const int); +CREATE TABLE citus_target(id int, val int); +CREATE TABLE citus_source(id int, val int, const int); +SELECT citus_add_local_table_to_metadata('citus_target'); + citus_add_local_table_to_metadata +--------------------------------------------------------------------- + +(1 row) + +SELECT citus_add_local_table_to_metadata('citus_source'); + citus_add_local_table_to_metadata +--------------------------------------------------------------------- + +(1 row) + +CREATE OR REPLACE FUNCTION cleanup_data() RETURNS VOID SET search_path TO merge_repartition2_schema AS $$ + TRUNCATE pg_target; + TRUNCATE pg_source; + TRUNCATE citus_target; + TRUNCATE citus_source; + SELECT undistribute_table('citus_target'); + SELECT undistribute_table('citus_source'); +$$ +LANGUAGE SQL; +-- +-- Load same set of data to both Postgres and Citus tables +-- +CREATE OR REPLACE FUNCTION setup_data() RETURNS VOID SET search_path TO merge_repartition2_schema AS $$ + INSERT INTO pg_source SELECT i, i+1, 1 FROM generate_series(1, 100000) i; + INSERT INTO pg_target SELECT i, 1 FROM generate_series(50001, 100000) i; + INSERT INTO citus_source SELECT i, i+1, 1 FROM generate_series(1, 100000) i; + INSERT INTO citus_target SELECT i, 1 FROM generate_series(50001, 100000) i; +$$ +LANGUAGE SQL; +-- +-- Compares the final target tables, merge-modified data, of both Postgres and Citus tables +-- +CREATE OR REPLACE FUNCTION check_data(table1_name text, column1_name text, table2_name text, column2_name text) +RETURNS VOID SET search_path TO merge_repartition2_schema AS $$ +DECLARE + table1_avg numeric; + table2_avg numeric; +BEGIN + EXECUTE format('SELECT COALESCE(AVG(%I), 0) FROM %I', column1_name, table1_name) INTO table1_avg; + EXECUTE format('SELECT COALESCE(AVG(%I), 0) FROM %I', column2_name, table2_name) INTO table2_avg; + + IF table1_avg > table2_avg THEN + RAISE EXCEPTION 'The average of %.% is greater than %.%', table1_name, column1_name, table2_name, column2_name; + ELSIF table1_avg < table2_avg THEN + RAISE EXCEPTION 'The average of %.% is less than %.%', table1_name, column1_name, table2_name, column2_name; + ELSE + RAISE NOTICE 'The average of %.% is equal to %.%', table1_name, column1_name, table2_name, column2_name; + END IF; +END; +$$ LANGUAGE plpgsql; +CREATE OR REPLACE FUNCTION compare_data() RETURNS VOID SET search_path TO merge_repartition2_schema AS $$ + SELECT check_data('pg_target', 'id', 'citus_target', 'id'); + SELECT check_data('pg_target', 'val', 'citus_target', 'val'); +$$ +LANGUAGE SQL; +-- Test nested cte +SELECT cleanup_data(); +NOTICE: creating a new table for merge_repartition2_schema.citus_target +CONTEXT: SQL function "cleanup_data" statement 5 +NOTICE: moving the data of merge_repartition2_schema.citus_target +CONTEXT: SQL function "cleanup_data" statement 5 +NOTICE: dropping the old merge_repartition2_schema.citus_target +CONTEXT: SQL function "cleanup_data" statement 5 +NOTICE: renaming the new table to merge_repartition2_schema.citus_target +CONTEXT: SQL function "cleanup_data" statement 5 +NOTICE: creating a new table for merge_repartition2_schema.citus_source +CONTEXT: SQL function "cleanup_data" statement 6 +NOTICE: moving the data of merge_repartition2_schema.citus_source +CONTEXT: SQL function "cleanup_data" statement 6 +NOTICE: dropping the old merge_repartition2_schema.citus_source +CONTEXT: SQL function "cleanup_data" statement 6 +NOTICE: renaming the new table to merge_repartition2_schema.citus_source +CONTEXT: SQL function "cleanup_data" statement 6 + cleanup_data +--------------------------------------------------------------------- + +(1 row) + +SELECT setup_data(); + setup_data +--------------------------------------------------------------------- + +(1 row) + +SELECT create_distributed_table('citus_target', 'id'); +NOTICE: Copying data from local table... +NOTICE: copying the data has completed +DETAIL: The local data in the table is no longer visible, but is still on disk. +HINT: To remove the local data, run: SELECT truncate_local_data_after_distributing_table($$merge_repartition2_schema.citus_target$$) + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +SELECT create_distributed_table('citus_source', 'id', colocate_with=>'none'); +NOTICE: Copying data from local table... +NOTICE: copying the data has completed +DETAIL: The local data in the table is no longer visible, but is still on disk. +HINT: To remove the local data, run: SELECT truncate_local_data_after_distributing_table($$merge_repartition2_schema.citus_source$$) + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +WITH cte_top AS(WITH cte_1 AS (WITH cte_2 AS (SELECT id, val FROM pg_source) SELECT * FROM cte_2) SELECT * FROM cte_1) +MERGE INTO pg_target t +USING (SELECT const, val, id FROM pg_source WHERE id IN (SELECT id FROM cte_top)) as s +ON (s.id = t.id) +WHEN MATCHED AND t.id <= 75000 THEN + UPDATE SET val = (s.val::int8+1) +WHEN MATCHED THEN + DELETE +WHEN NOT MATCHED THEN + INSERT VALUES (s.id, s.val); +WITH cte_top AS(WITH cte_1 AS (WITH cte_2 AS (SELECT id, val FROM citus_source) SELECT * FROM cte_2) SELECT * FROM cte_1) +MERGE INTO citus_target t +USING (SELECT const, val, id FROM citus_source WHERE id IN (SELECT id FROM cte_top)) as s +ON (s.id = t.id) +WHEN MATCHED AND t.id <= 75000 THEN + UPDATE SET val = (s.val::int8+1) +WHEN MATCHED THEN + DELETE +WHEN NOT MATCHED THEN + INSERT VALUES (s.id, s.val); +SELECT compare_data(); +NOTICE: The average of pg_target.id is equal to citus_target.id +CONTEXT: PL/pgSQL function check_data(text,text,text,text) line XX at RAISE +SQL function "compare_data" statement 1 +NOTICE: The average of pg_target.val is equal to citus_target.val +CONTEXT: PL/pgSQL function check_data(text,text,text,text) line XX at RAISE +SQL function "compare_data" statement 2 + compare_data +--------------------------------------------------------------------- + +(1 row) + +-- Test aggregate function in source query +MERGE INTO pg_target t +USING (SELECT count(id+1)::text as value, val as key FROM pg_source group by key) s +ON t.id = s.key +WHEN MATCHED AND t.id <= 75000 THEN + UPDATE SET val = (s.value::int8+1) +WHEN MATCHED THEN + DELETE +WHEN NOT MATCHED THEN + INSERT VALUES(s.key, value::int4+10); +MERGE INTO citus_target t +USING (SELECT count(id+1)::text as value, val as key FROM citus_source group by key) s +ON t.id = s.key +WHEN MATCHED AND t.id <= 75000 THEN + UPDATE SET val = (s.value::int8+1) +WHEN MATCHED THEN + DELETE +WHEN NOT MATCHED THEN + INSERT VALUES(s.key, value::int4+10); +SELECT compare_data(); +NOTICE: The average of pg_target.id is equal to citus_target.id +CONTEXT: PL/pgSQL function check_data(text,text,text,text) line XX at RAISE +SQL function "compare_data" statement 1 +NOTICE: The average of pg_target.val is equal to citus_target.val +CONTEXT: PL/pgSQL function check_data(text,text,text,text) line XX at RAISE +SQL function "compare_data" statement 2 + compare_data +--------------------------------------------------------------------- + +(1 row) + +DROP SCHEMA merge_repartition2_schema CASCADE; +NOTICE: drop cascades to 8 other objects +DETAIL: drop cascades to table pg_target +drop cascades to table pg_source +drop cascades to function cleanup_data() +drop cascades to function setup_data() +drop cascades to function check_data(text,text,text,text) +drop cascades to function compare_data() +drop cascades to table citus_target +drop cascades to table citus_source diff --git a/src/test/regress/expected/merge_repartition2_0.out b/src/test/regress/expected/merge_repartition2_0.out new file mode 100644 index 000000000..a7e3fbf20 --- /dev/null +++ b/src/test/regress/expected/merge_repartition2_0.out @@ -0,0 +1,6 @@ +SHOW server_version \gset +SELECT substring(:'server_version', '\d+')::int >= 15 AS server_version_ge_15 +\gset +\if :server_version_ge_15 +\else +\q diff --git a/src/test/regress/expected/multi_partitioning.out b/src/test/regress/expected/multi_partitioning.out index 47139614d..c0a21d4d5 100644 --- a/src/test/regress/expected/multi_partitioning.out +++ b/src/test/regress/expected/multi_partitioning.out @@ -4059,6 +4059,47 @@ BEGIN; ROLLBACK; set client_min_messages to notice; +-- 7) test with bigint partition column +CREATE FUNCTION nanos_to_timestamptz(nanos bigint) RETURNS timestamptz LANGUAGE plpgsql AS +$$ +DECLARE + value timestamptz; +BEGIN + select to_timestamp(nanos * 1.0 / 1000000000) into value; + return value; +END; +$$; +CREATE CAST (bigint AS timestamptz) WITH FUNCTION nanos_to_timestamptz(bigint); +CREATE FUNCTION timestamptz_to_nanos(ts timestamptz) RETURNS bigint LANGUAGE plpgsql AS +$$ +DECLARE + value bigint; +BEGIN + select extract(epoch from ts) * 1000000000 into value; + return value; +END; +$$; +CREATE CAST (timestamptz AS bigint) WITH FUNCTION timestamptz_to_nanos(timestamptz); +CREATE TABLE bigint_partitioned_table (timestamp bigint, description text) partition by range (timestamp); +BEGIN; + SELECT create_time_partitions('bigint_partitioned_table', INTERVAL '1 month', '2023-05-01', '2023-01-1'); + create_time_partitions +--------------------------------------------------------------------- + t +(1 row) + + SELECT * FROM time_partitions WHERE parent_table = 'bigint_partitioned_table'::regclass ORDER BY 3; + parent_table | partition_column | partition | from_value | to_value | access_method +--------------------------------------------------------------------- + bigint_partitioned_table | timestamp | bigint_partitioned_table_p2023_01 | 1672560000000000000 | 1675238400000000000 | heap + bigint_partitioned_table | timestamp | bigint_partitioned_table_p2023_02 | 1675238400000000000 | 1677657600000000000 | heap + bigint_partitioned_table | timestamp | bigint_partitioned_table_p2023_03 | 1677657600000000000 | 1680332400000000000 | heap + bigint_partitioned_table | timestamp | bigint_partitioned_table_p2023_04 | 1680332400000000000 | 1682924400000000000 | heap +(4 rows) + +ROLLBACK; +DROP CAST (bigint AS timestamptz); +DROP CAST (timestamptz AS bigint); -- c) test drop_old_time_partitions -- 1) test with date partitioned table CREATE TABLE date_partitioned_table_to_exp (event_date date, event int) partition by range (event_date); @@ -4359,10 +4400,13 @@ SELECT a, b FROM stxdinp GROUP BY 1, 2; (10 rows) DROP SCHEMA partitioning_schema CASCADE; -NOTICE: drop cascades to 5 other objects +NOTICE: drop cascades to 8 other objects DETAIL: drop cascades to table "schema-test" drop cascades to table another_distributed_table drop cascades to table distributed_parent_table +drop cascades to function nanos_to_timestamptz(bigint) +drop cascades to function timestamptz_to_nanos(timestamp with time zone) +drop cascades to table bigint_partitioned_table drop cascades to table part_table_with_very_long_name drop cascades to table stxdinp RESET search_path; diff --git a/src/test/regress/expected/multi_utility_statements.out b/src/test/regress/expected/multi_utility_statements.out index ad97dd267..ccfe3a333 100644 --- a/src/test/regress/expected/multi_utility_statements.out +++ b/src/test/regress/expected/multi_utility_statements.out @@ -254,6 +254,76 @@ FETCH FORWARD 3 FROM holdCursor; 1 | 19 (3 rows) +CLOSE holdCursor; +-- Test DECLARE CURSOR .. WITH HOLD inside transaction block +BEGIN; +DECLARE holdCursor CURSOR WITH HOLD FOR + SELECT * FROM cursor_me WHERE x = 1 ORDER BY y; +FETCH 3 FROM holdCursor; + x | y +--------------------------------------------------------------------- + 1 | 10 + 1 | 11 + 1 | 12 +(3 rows) + +FETCH BACKWARD 3 FROM holdCursor; + x | y +--------------------------------------------------------------------- + 1 | 11 + 1 | 10 +(2 rows) + +FETCH FORWARD 3 FROM holdCursor; + x | y +--------------------------------------------------------------------- + 1 | 10 + 1 | 11 + 1 | 12 +(3 rows) + +COMMIT; +FETCH 3 FROM holdCursor; + x | y +--------------------------------------------------------------------- + 1 | 13 + 1 | 14 + 1 | 15 +(3 rows) + +CLOSE holdCursor; +-- Test DECLARE NO SCROLL CURSOR .. WITH HOLD inside transaction block +BEGIN; +DECLARE holdCursor NO SCROLL CURSOR WITH HOLD FOR + SELECT * FROM cursor_me WHERE x = 1 ORDER BY y; +FETCH 3 FROM holdCursor; + x | y +--------------------------------------------------------------------- + 1 | 10 + 1 | 11 + 1 | 12 +(3 rows) + +FETCH FORWARD 3 FROM holdCursor; + x | y +--------------------------------------------------------------------- + 1 | 13 + 1 | 14 + 1 | 15 +(3 rows) + +COMMIT; +FETCH 3 FROM holdCursor; + x | y +--------------------------------------------------------------------- + 1 | 16 + 1 | 17 + 1 | 18 +(3 rows) + +FETCH BACKWARD 3 FROM holdCursor; +ERROR: cursor can only scan forward +HINT: Declare it with SCROLL option to enable backward scan. CLOSE holdCursor; -- Test DECLARE CURSOR .. WITH HOLD with parameter CREATE OR REPLACE FUNCTION declares_cursor(p int) diff --git a/src/test/regress/expected/pg15.out b/src/test/regress/expected/pg15.out index 68c4c4466..667305225 100644 --- a/src/test/regress/expected/pg15.out +++ b/src/test/regress/expected/pg15.out @@ -406,14 +406,16 @@ SELECT create_distributed_table('tbl2', 'x'); MERGE INTO tbl1 USING tbl2 ON (true) WHEN MATCHED THEN DELETE; -ERROR: MERGE command is only supported when all distributed tables are co-located and joined on their distribution columns --- also, not inside subqueries & ctes +ERROR: The required join operation is missing between the target's distribution column and any expression originating from the source. The issue may arise from either a non-equi-join or a mismatch in the datatypes of the columns being joined. +DETAIL: Without a equi-join condition on the target's distribution column, the source rows cannot be efficiently redistributed, and the NOT-MATCHED condition cannot be evaluated unambiguously. This can result in incorrect or unexpected results when attempting to merge tables in a distributed setting +-- also, inside subqueries & ctes WITH targq AS ( SELECT * FROM tbl2 ) MERGE INTO tbl1 USING targq ON (true) WHEN MATCHED THEN DELETE; -ERROR: MERGE command is only supported when all distributed tables are co-located and joined on their distribution columns +ERROR: The required join operation is missing between the target's distribution column and any expression originating from the source. The issue may arise from either a non-equi-join or a mismatch in the datatypes of the columns being joined. +DETAIL: Without a equi-join condition on the target's distribution column, the source rows cannot be efficiently redistributed, and the NOT-MATCHED condition cannot be evaluated unambiguously. This can result in incorrect or unexpected results when attempting to merge tables in a distributed setting WITH foo AS ( MERGE INTO tbl1 USING tbl2 ON (true) WHEN MATCHED THEN DELETE @@ -429,7 +431,8 @@ USING tbl2 ON (true) WHEN MATCHED THEN DO NOTHING; -ERROR: MERGE command is only supported when all distributed tables are co-located and joined on their distribution columns +ERROR: The required join operation is missing between the target's distribution column and any expression originating from the source. The issue may arise from either a non-equi-join or a mismatch in the datatypes of the columns being joined. +DETAIL: Without a equi-join condition on the target's distribution column, the source rows cannot be efficiently redistributed, and the NOT-MATCHED condition cannot be evaluated unambiguously. This can result in incorrect or unexpected results when attempting to merge tables in a distributed setting MERGE INTO tbl1 t USING tbl2 ON (true) diff --git a/src/test/regress/expected/pgmerge.out b/src/test/regress/expected/pgmerge.out index 7742610f4..895bf0680 100644 --- a/src/test/regress/expected/pgmerge.out +++ b/src/test/regress/expected/pgmerge.out @@ -15,6 +15,14 @@ SET search_path TO pgmerge_schema; SET citus.use_citus_managed_tables to true; \set SHOW_CONTEXT errors SET citus.next_shard_id TO 4001000; +SET client_min_messages = warning; +SELECT 1 FROM master_add_node('localhost', :master_port, groupid => 0); + ?column? +--------------------------------------------------------------------- + 1 +(1 row) + +RESET client_min_messages; CREATE USER regress_merge_privs; CREATE USER regress_merge_no_privs; DROP TABLE IF EXISTS target; diff --git a/src/test/regress/expected/publication.out b/src/test/regress/expected/publication.out index 7fc75637d..702d23f1f 100644 --- a/src/test/regress/expected/publication.out +++ b/src/test/regress/expected/publication.out @@ -90,13 +90,18 @@ SELECT DISTINCT c FROM ( SELECT worker_create_or_replace_object('CREATE PUBLICATION pubtables_orig WITH (publish_via_partition_root = ''false'', publish = ''insert, truncate'')'); (1 row) --- distribute a table, creating a mixed publication +-- distribute a table and create a tenant schema, creating a mixed publication SELECT create_distributed_table('test','x', colocate_with := 'none'); create_distributed_table --------------------------------------------------------------------- (1 row) +SET citus.enable_schema_based_sharding TO ON; +CREATE SCHEMA citus_schema_1; +CREATE TABLE citus_schema_1.test (x int primary key, y int, "column-1" int, doc xml); +SET citus.enable_schema_based_sharding TO OFF; +ALTER PUBLICATION pubtables_orig ADD TABLE citus_schema_1.test; -- some generic operations ALTER PUBLICATION pubtables_orig RENAME TO pubtables; ALTER PUBLICATION pubtables SET (publish = 'insert, update, delete'); @@ -108,7 +113,11 @@ ERROR: relation "notexist" does not exist -- operations with a distributed table ALTER PUBLICATION pubtables DROP TABLE test; ALTER PUBLICATION pubtables ADD TABLE test; -ALTER PUBLICATION pubtables SET TABLE test, "test-pubs", "publication-1"."test-pubs"; +ALTER PUBLICATION pubtables SET TABLE test, "test-pubs", "publication-1"."test-pubs", citus_schema_1.test; +-- operations with a tenant schema table +ALTER PUBLICATION pubtables DROP TABLE citus_schema_1.test; +ALTER PUBLICATION pubtables ADD TABLE citus_schema_1.test; +ALTER PUBLICATION pubtables SET TABLE test, "test-pubs", "publication-1"."test-pubs", citus_schema_1.test; -- operations with a local table in a mixed publication ALTER PUBLICATION pubtables DROP TABLE "test-pubs"; ALTER PUBLICATION pubtables ADD TABLE "test-pubs"; @@ -124,9 +133,9 @@ SELECT DISTINCT c FROM ( FROM run_command_on_workers($$ SELECT array_agg(c) FROM (SELECT c FROM unnest(activate_node_snapshot()) c WHERE c LIKE '%CREATE PUBLICATION%' AND c LIKE '%pubtables%' ORDER BY 1) s$$) ORDER BY c) s; - c + c --------------------------------------------------------------------- - SELECT worker_create_or_replace_object('CREATE PUBLICATION pubtables FOR TABLE publication.test, TABLE publication."test-pubs" WITH (publish_via_partition_root = ''false'', publish = ''insert, update, delete'')'); + SELECT worker_create_or_replace_object('CREATE PUBLICATION pubtables FOR TABLE publication.test, TABLE citus_schema_1.test, TABLE publication."test-pubs" WITH (publish_via_partition_root = ''false'', publish = ''insert, update, delete'')'); (1 row) -- operations with a strangely named distributed table in a mixed publication @@ -134,7 +143,7 @@ ALTER PUBLICATION pubtables DROP TABLE "test-pubs"; ALTER PUBLICATION pubtables ADD TABLE "test-pubs"; -- create a publication with distributed and local tables DROP PUBLICATION pubtables; -CREATE PUBLICATION pubtables FOR TABLE test, "test-pubs", "publication-1"."test-pubs"; +CREATE PUBLICATION pubtables FOR TABLE test, "test-pubs", "publication-1"."test-pubs", citus_schema_1.test; -- change distributed tables SELECT alter_distributed_table('test', shard_count := 5, cascade_to_colocated := true); NOTICE: creating a new table for publication.test @@ -194,9 +203,9 @@ SELECT DISTINCT c FROM ( FROM run_command_on_workers($$ SELECT array_agg(c) FROM (SELECT c FROM unnest(activate_node_snapshot()) c WHERE c LIKE '%CREATE PUBLICATION%' AND c LIKE '%pubtables%' ORDER BY 1) s$$) ORDER BY c) s; - c + c --------------------------------------------------------------------- - SELECT worker_create_or_replace_object('CREATE PUBLICATION pubtables FOR TABLE publication.test, TABLE publication."test-pubs" WITH (publish_via_partition_root = ''false'', publish = ''insert, update, delete, truncate'')'); + SELECT worker_create_or_replace_object('CREATE PUBLICATION pubtables FOR TABLE citus_schema_1.test, TABLE publication.test, TABLE publication."test-pubs" WITH (publish_via_partition_root = ''false'', publish = ''insert, update, delete, truncate'')'); (1 row) -- partitioned table @@ -257,10 +266,11 @@ SELECT substring(:'server_version', '\d+')::int >= 15 AS server_version_ge_15 SET client_min_messages TO ERROR; DROP SCHEMA publication CASCADE; DROP SCHEMA "publication-1" CASCADE; +DROP SCHEMA citus_schema_1 CASCADE; \q \endif -- recreate a mixed publication -CREATE PUBLICATION pubtables FOR TABLE test, "publication-1"."test-pubs"; +CREATE PUBLICATION pubtables FOR TABLE test, "publication-1"."test-pubs", citus_schema_1.test; -- operations on an existing distributed table ALTER PUBLICATION pubtables DROP TABLE test; ALTER PUBLICATION pubtables ADD TABLE test (y); @@ -277,6 +287,22 @@ SELECT DISTINCT c FROM ( SELECT worker_create_or_replace_object('CREATE PUBLICATION pubtables FOR TABLE publication.test WHERE (CASE test.x WHEN 5 THEN true ELSE false END) WITH (publish_via_partition_root = ''false'', publish = ''insert, update, delete, truncate'')'); (1 row) +-- operations on an existing tenant schema table +ALTER PUBLICATION pubtables ADD TABLE citus_schema_1.test (y); +ALTER PUBLICATION pubtables DROP TABLE citus_schema_1.test; +ALTER PUBLICATION pubtables SET TABLE citus_schema_1.test WHERE (doc IS DOCUMENT); +ALTER PUBLICATION pubtables SET TABLE citus_schema_1.test WHERE (xmlexists('//foo[text() = ''bar'']' PASSING BY VALUE doc)); +ALTER PUBLICATION pubtables SET TABLE citus_schema_1.test WHERE (CASE x WHEN 5 THEN true ELSE false END); +SELECT DISTINCT c FROM ( + SELECT unnest(result::text[]) c + FROM run_command_on_workers($$ + SELECT array_agg(c) FROM (SELECT c FROM unnest(activate_node_snapshot()) c WHERE c LIKE '%CREATE PUBLICATION%' AND c LIKE '%pubtables%' ORDER BY 1) s$$) + ORDER BY c) s; + c +--------------------------------------------------------------------- + SELECT worker_create_or_replace_object('CREATE PUBLICATION pubtables FOR TABLE citus_schema_1.test WHERE (CASE test.x WHEN 5 THEN true ELSE false END) WITH (publish_via_partition_root = ''false'', publish = ''insert, update, delete, truncate'')'); +(1 row) + ALTER PUBLICATION pubtables SET TABLE test ("column-1", x) WHERE (x > "column-1"), "publication-1"."test-pubs"; -- operations on a local table ALTER PUBLICATION pubtables DROP TABLE "publication-1"."test-pubs"; @@ -363,3 +389,4 @@ DROP PUBLICATION pubpartitioned; SET client_min_messages TO ERROR; DROP SCHEMA publication CASCADE; DROP SCHEMA "publication-1" CASCADE; +DROP SCHEMA citus_schema_1 CASCADE; diff --git a/src/test/regress/expected/publication_0.out b/src/test/regress/expected/publication_0.out index 02978ff65..14fa94d17 100644 --- a/src/test/regress/expected/publication_0.out +++ b/src/test/regress/expected/publication_0.out @@ -90,13 +90,18 @@ SELECT DISTINCT c FROM ( SELECT worker_create_or_replace_object('CREATE PUBLICATION pubtables_orig WITH (publish_via_partition_root = ''false'', publish = ''insert, truncate'')'); (1 row) --- distribute a table, creating a mixed publication +-- distribute a table and create a tenant schema, creating a mixed publication SELECT create_distributed_table('test','x', colocate_with := 'none'); create_distributed_table --------------------------------------------------------------------- (1 row) +SET citus.enable_schema_based_sharding TO ON; +CREATE SCHEMA citus_schema_1; +CREATE TABLE citus_schema_1.test (x int primary key, y int, "column-1" int, doc xml); +SET citus.enable_schema_based_sharding TO OFF; +ALTER PUBLICATION pubtables_orig ADD TABLE citus_schema_1.test; -- some generic operations ALTER PUBLICATION pubtables_orig RENAME TO pubtables; ALTER PUBLICATION pubtables SET (publish = 'insert, update, delete'); @@ -108,7 +113,11 @@ ERROR: relation "notexist" does not exist -- operations with a distributed table ALTER PUBLICATION pubtables DROP TABLE test; ALTER PUBLICATION pubtables ADD TABLE test; -ALTER PUBLICATION pubtables SET TABLE test, "test-pubs", "publication-1"."test-pubs"; +ALTER PUBLICATION pubtables SET TABLE test, "test-pubs", "publication-1"."test-pubs", citus_schema_1.test; +-- operations with a tenant schema table +ALTER PUBLICATION pubtables DROP TABLE citus_schema_1.test; +ALTER PUBLICATION pubtables ADD TABLE citus_schema_1.test; +ALTER PUBLICATION pubtables SET TABLE test, "test-pubs", "publication-1"."test-pubs", citus_schema_1.test; -- operations with a local table in a mixed publication ALTER PUBLICATION pubtables DROP TABLE "test-pubs"; ALTER PUBLICATION pubtables ADD TABLE "test-pubs"; @@ -124,9 +133,9 @@ SELECT DISTINCT c FROM ( FROM run_command_on_workers($$ SELECT array_agg(c) FROM (SELECT c FROM unnest(activate_node_snapshot()) c WHERE c LIKE '%CREATE PUBLICATION%' AND c LIKE '%pubtables%' ORDER BY 1) s$$) ORDER BY c) s; - c + c --------------------------------------------------------------------- - SELECT worker_create_or_replace_object('CREATE PUBLICATION pubtables FOR TABLE publication.test, publication."test-pubs" WITH (publish_via_partition_root = ''false'', publish = ''insert, update, delete'')'); + SELECT worker_create_or_replace_object('CREATE PUBLICATION pubtables FOR TABLE publication.test, citus_schema_1.test, publication."test-pubs" WITH (publish_via_partition_root = ''false'', publish = ''insert, update, delete'')'); (1 row) -- operations with a strangely named distributed table in a mixed publication @@ -134,7 +143,7 @@ ALTER PUBLICATION pubtables DROP TABLE "test-pubs"; ALTER PUBLICATION pubtables ADD TABLE "test-pubs"; -- create a publication with distributed and local tables DROP PUBLICATION pubtables; -CREATE PUBLICATION pubtables FOR TABLE test, "test-pubs", "publication-1"."test-pubs"; +CREATE PUBLICATION pubtables FOR TABLE test, "test-pubs", "publication-1"."test-pubs", citus_schema_1.test; -- change distributed tables SELECT alter_distributed_table('test', shard_count := 5, cascade_to_colocated := true); NOTICE: creating a new table for publication.test @@ -194,9 +203,9 @@ SELECT DISTINCT c FROM ( FROM run_command_on_workers($$ SELECT array_agg(c) FROM (SELECT c FROM unnest(activate_node_snapshot()) c WHERE c LIKE '%CREATE PUBLICATION%' AND c LIKE '%pubtables%' ORDER BY 1) s$$) ORDER BY c) s; - c + c --------------------------------------------------------------------- - SELECT worker_create_or_replace_object('CREATE PUBLICATION pubtables FOR TABLE publication.test, publication."test-pubs" WITH (publish_via_partition_root = ''false'', publish = ''insert, update, delete, truncate'')'); + SELECT worker_create_or_replace_object('CREATE PUBLICATION pubtables FOR TABLE citus_schema_1.test, publication.test, publication."test-pubs" WITH (publish_via_partition_root = ''false'', publish = ''insert, update, delete, truncate'')'); (1 row) -- partitioned table @@ -257,4 +266,5 @@ SELECT substring(:'server_version', '\d+')::int >= 15 AS server_version_ge_15 SET client_min_messages TO ERROR; DROP SCHEMA publication CASCADE; DROP SCHEMA "publication-1" CASCADE; +DROP SCHEMA citus_schema_1 CASCADE; \q diff --git a/src/test/regress/expected/schema_based_sharding.out b/src/test/regress/expected/schema_based_sharding.out index 951fc05e7..b041708cf 100644 --- a/src/test/regress/expected/schema_based_sharding.out +++ b/src/test/regress/expected/schema_based_sharding.out @@ -80,24 +80,148 @@ ERROR: tenant_2.test_table is not allowed for update_distributed_table_colocati -- verify we also don't allow colocate_with a tenant table SELECT update_distributed_table_colocation('regular_schema.test_table', colocate_with => 'tenant_2.test_table'); ERROR: tenant_2.test_table is not allowed for colocate_with because it belongs to a distributed schema --- verify we don't allow undistribute_table for tenant tables -SELECT undistribute_table('tenant_2.test_table'); -ERROR: tenant_2.test_table is not allowed for undistribute_table because it belongs to a distributed schema +-- verify we do not allow undistribute_table for tenant tables +CREATE TABLE tenant_2.undist_table(id int); +SELECT undistribute_table('tenant_2.undist_table'); +ERROR: tenant_2.undist_table is not allowed for undistribute_table because it belongs to a distributed schema -- verify we don't allow alter_distributed_table for tenant tables SELECT alter_distributed_table('tenant_2.test_table', colocate_with => 'none'); ERROR: tenant_2.test_table is not allowed for alter_distributed_table because it belongs to a distributed schema -- verify we also don't allow colocate_with a tenant table SELECT alter_distributed_table('regular_schema.test_table', colocate_with => 'tenant_2.test_table'); ERROR: tenant_2.test_table is not allowed for colocate_with because it belongs to a distributed schema --- verify we don't allow ALTER TABLE SET SCHEMA for tenant tables -ALTER TABLE tenant_2.test_table SET SCHEMA regular_schema; -ERROR: tenant_2.test_table is not allowed for ALTER TABLE SET SCHEMA because it belongs to a distributed schema --- verify we don't allow ALTER TABLE SET SCHEMA for tenant schemas -ALTER TABLE regular_schema.test_table SET SCHEMA tenant_2; -ERROR: tenant_2 is not allowed for ALTER TABLE SET SCHEMA because it is a distributed schema --- the same, from tenant schema to tenant schema -ALTER TABLE tenant_2.test_table SET SCHEMA tenant_3; -ERROR: tenant_2.test_table is not allowed for ALTER TABLE SET SCHEMA because it belongs to a distributed schema +-- verify we can set tenant table's schema to regular schema +CREATE TABLE tenant_2.test_table2(id int); +ALTER TABLE tenant_2.test_table2 SET SCHEMA regular_schema; +NOTICE: undistributing table test_table2 in distributed schema tenant_2 before altering its schema +-- verify that regular_schema.test_table2 does not exist in pg_dist_partition +SELECT COUNT(*)=0 FROM pg_dist_partition +WHERE logicalrelid = 'regular_schema.test_table2'::regclass AND + partmethod = 'n' AND repmodel = 's' AND colocationid > 0; + ?column? +--------------------------------------------------------------------- + t +(1 row) + +-- verify that tenant_2.test_table2 does not exist +SELECT * FROM tenant_2.test_table2; +ERROR: relation "tenant_2.test_table2" does not exist +-- verify we can set regular table's schema to distributed schema +CREATE TABLE regular_schema.test_table3(id int); +ALTER TABLE regular_schema.test_table3 SET SCHEMA tenant_2; +NOTICE: converting table test_table3 to a tenant table in distributed schema tenant_2 +-- verify that tenant_2.test_table3 is recorded in pg_dist_partition as a single-shard table. +SELECT COUNT(*)=1 FROM pg_dist_partition +WHERE logicalrelid = 'tenant_2.test_table3'::regclass AND + partmethod = 'n' AND repmodel = 's' AND colocationid > 0; + ?column? +--------------------------------------------------------------------- + t +(1 row) + +-- verify that regular_schema.test_table3 does not exist +SELECT * FROM regular_schema.test_table3; +ERROR: relation "regular_schema.test_table3" does not exist +-- verify we can set tenant table's schema to another distributed schema +CREATE TABLE tenant_2.test_table4(id int); +ALTER TABLE tenant_2.test_table4 SET SCHEMA tenant_3; +NOTICE: undistributing table test_table4 in distributed schema tenant_2 before altering its schema +NOTICE: converting table test_table4 to a tenant table in distributed schema tenant_3 +-- verify that tenant_3.test_table4 is recorded in pg_dist_partition as a single-shard table. +SELECT COUNT(*)=1 FROM pg_dist_partition +WHERE logicalrelid = 'tenant_3.test_table4'::regclass AND + partmethod = 'n' AND repmodel = 's' AND colocationid > 0; + ?column? +--------------------------------------------------------------------- + t +(1 row) + +-- verify that tenant_2.test_table4 does not exist +SELECT * FROM tenant_2.test_table4; +ERROR: relation "tenant_2.test_table4" does not exist +-- verify that we can put a local table in regular schema into distributed schema +CREATE TABLE regular_schema.pg_local_tbl(id int); +ALTER TABLE regular_schema.pg_local_tbl SET SCHEMA tenant_2; +NOTICE: converting table pg_local_tbl to a tenant table in distributed schema tenant_2 +-- verify that we can put a Citus local table in regular schema into distributed schema +CREATE TABLE regular_schema.citus_local_tbl(id int); +SELECT citus_add_local_table_to_metadata('regular_schema.citus_local_tbl'); + citus_add_local_table_to_metadata +--------------------------------------------------------------------- + +(1 row) + +ALTER TABLE regular_schema.citus_local_tbl SET SCHEMA tenant_2; +NOTICE: converting table citus_local_tbl to a tenant table in distributed schema tenant_2 +-- verify that we do not allow a hash distributed table in regular schema into distributed schema +CREATE TABLE regular_schema.hash_dist_tbl(id int); +SELECT create_distributed_table('regular_schema.hash_dist_tbl', 'id'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +ALTER TABLE regular_schema.hash_dist_tbl SET SCHEMA tenant_2; +ERROR: distributed schema cannot have distributed tables +HINT: Undistribute distributed tables before 'ALTER TABLE SET SCHEMA'. +-- verify that we do not allow a reference table in regular schema into distributed schema +CREATE TABLE regular_schema.ref_tbl(id int PRIMARY KEY); +SELECT create_reference_table('regular_schema.ref_tbl'); + create_reference_table +--------------------------------------------------------------------- + +(1 row) + +ALTER TABLE regular_schema.ref_tbl SET SCHEMA tenant_2; +ERROR: distributed schema cannot have distributed tables +HINT: Undistribute distributed tables before 'ALTER TABLE SET SCHEMA'. +-- verify that we can put a table in tenant schema into regular schema +CREATE TABLE tenant_2.tenant_tbl(id int); +ALTER TABLE tenant_2.tenant_tbl SET SCHEMA regular_schema; +NOTICE: undistributing table tenant_tbl in distributed schema tenant_2 before altering its schema +-- verify that we can put a table in tenant schema into another tenant schema +CREATE TABLE tenant_2.tenant_tbl2(id int); +ALTER TABLE tenant_2.tenant_tbl2 SET SCHEMA tenant_3; +NOTICE: undistributing table tenant_tbl2 in distributed schema tenant_2 before altering its schema +NOTICE: converting table tenant_tbl2 to a tenant table in distributed schema tenant_3 +-- verify that we do not allow a local table in regular schema into distributed schema if it has foreign key to a non-reference table in another schema +CREATE TABLE regular_schema.pg_local_tbl1(id int PRIMARY KEY); +CREATE TABLE regular_schema.pg_local_tbl2(id int REFERENCES regular_schema.pg_local_tbl1(id)); +ALTER TABLE regular_schema.pg_local_tbl2 SET SCHEMA tenant_2; +ERROR: foreign keys from distributed schemas can only point to the same distributed schema or reference tables in regular schemas +DETAIL: "tenant_2.pg_local_tbl2" references "regular_schema.pg_local_tbl1" via foreign key constraint "pg_local_tbl2_id_fkey" +-- verify that we allow a local table in regular schema into distributed schema if it has foreign key to a reference table in another schema +CREATE TABLE regular_schema.pg_local_tbl3(id int REFERENCES regular_schema.ref_tbl(id)); +ALTER TABLE regular_schema.pg_local_tbl3 SET SCHEMA tenant_2; +NOTICE: converting table pg_local_tbl3 to a tenant table in distributed schema tenant_2 +-- verify that we do not allow a table in tenant schema into regular schema if it has foreign key to/from another table in the same schema +CREATE TABLE tenant_2.tenant_tbl1(id int PRIMARY KEY); +CREATE TABLE tenant_2.tenant_tbl2(id int REFERENCES tenant_2.tenant_tbl1(id)); +ALTER TABLE tenant_2.tenant_tbl1 SET SCHEMA regular_schema; +ERROR: set schema is not allowed for table tenant_tbl1 in distributed schema tenant_2 +DETAIL: distributed schemas cannot have foreign keys from/to local tables or different schema +ALTER TABLE tenant_2.tenant_tbl2 SET SCHEMA regular_schema; +ERROR: set schema is not allowed for table tenant_tbl2 in distributed schema tenant_2 +DETAIL: distributed schemas cannot have foreign keys from/to local tables or different schema +-- verify that we do not allow a table in distributed schema into another distributed schema if it has foreign key to/from another table in the same schema +CREATE TABLE tenant_2.tenant_tbl3(id int PRIMARY KEY); +CREATE TABLE tenant_2.tenant_tbl4(id int REFERENCES tenant_2.tenant_tbl3(id)); +ALTER TABLE tenant_2.tenant_tbl3 SET SCHEMA tenant_3; +ERROR: set schema is not allowed for table tenant_tbl3 in distributed schema tenant_2 +DETAIL: distributed schemas cannot have foreign keys from/to local tables or different schema +ALTER TABLE tenant_2.tenant_tbl4 SET SCHEMA tenant_3; +ERROR: set schema is not allowed for table tenant_tbl4 in distributed schema tenant_2 +DETAIL: distributed schemas cannot have foreign keys from/to local tables or different schema +-- alter set non-existent schema +ALTER TABLE tenant_2.test_table SET SCHEMA ghost_schema; +ERROR: schema "ghost_schema" does not exist +ALTER TABLE IF EXISTS tenant_2.test_table SET SCHEMA ghost_schema; +ERROR: schema "ghost_schema" does not exist +-- alter set non-existent table +ALTER TABLE tenant_2.ghost_table SET SCHEMA ghost_schema; +ERROR: relation "tenant_2.ghost_table" does not exist +ALTER TABLE IF EXISTS tenant_2.ghost_table SET SCHEMA ghost_schema; +NOTICE: relation "ghost_table" does not exist, skipping -- (on coordinator) verify that colocation id is set for empty tenants too SELECT colocationid > 0 FROM pg_dist_schema WHERE schemaid::regnamespace::text IN ('tenant_1', 'tenant_3'); @@ -265,8 +389,8 @@ SELECT EXISTS( (1 row) INSERT INTO tenant_4.another_partitioned_table VALUES (1, 'a'); -ERROR: insert or update on table "another_partitioned_table_child_1920040" violates foreign key constraint "another_partitioned_table_a_fkey_1920039" -DETAIL: Key (a)=(1) is not present in table "partitioned_table_1920037". +ERROR: insert or update on table "another_partitioned_table_child_1920090" violates foreign key constraint "another_partitioned_table_a_fkey_1920089" +DETAIL: Key (a)=(1) is not present in table "partitioned_table_1920087". CONTEXT: while executing command on localhost:xxxxx INSERT INTO tenant_4.partitioned_table VALUES (1, 'a'); INSERT INTO tenant_4.another_partitioned_table VALUES (1, 'a'); diff --git a/src/test/regress/multi_schedule b/src/test/regress/multi_schedule index 527dec8f7..4d42dbc78 100644 --- a/src/test/regress/multi_schedule +++ b/src/test/regress/multi_schedule @@ -111,8 +111,9 @@ test: background_task_queue_monitor test: clock # MERGE tests -test: merge -test: pgmerge +test: merge pgmerge merge_repartition2 +test: merge_repartition1 +test: merge_partition_tables # --------- # test that no tests leaked intermediate results. This should always be last diff --git a/src/test/regress/sql/merge.sql b/src/test/regress/sql/merge.sql index 911642812..db3a76fb6 100644 --- a/src/test/regress/sql/merge.sql +++ b/src/test/regress/sql/merge.sql @@ -21,6 +21,9 @@ SET citus.next_shard_id TO 4000000; SET citus.explain_all_tasks TO true; SET citus.shard_replication_factor TO 1; SET citus.max_adaptive_executor_pool_size TO 1; +SET client_min_messages = warning; +SELECT 1 FROM master_add_node('localhost', :master_port, groupid => 0); +RESET client_min_messages; CREATE TABLE source ( @@ -142,7 +145,7 @@ SELECT insert_data(); SELECT undistribute_table('target'); SELECT undistribute_table('source'); SELECT create_distributed_table('target', 'customer_id'); -SELECT create_distributed_table('source', 'customer_id'); +SELECT create_distributed_table('source', 'customer_id', colocate_with=>'target'); -- Updates one of the row with customer_id = 30002 SELECT * from target t WHERE t.customer_id = 30002; @@ -280,7 +283,7 @@ TRUNCATE t1; TRUNCATE s1; SELECT load(); SELECT create_distributed_table('t1', 'id'); -SELECT create_distributed_table('s1', 'id'); +SELECT create_distributed_table('s1', 'id', colocate_with=>'t1'); SELECT * FROM t1 order by id; @@ -368,7 +371,7 @@ SELECT insert_data(); SELECT undistribute_table('t2'); SELECT undistribute_table('s2'); SELECT create_distributed_table('t2', 'id'); -SELECT create_distributed_table('s2', 'id'); +SELECT create_distributed_table('s2', 'id', colocate_with => 't2'); SELECT * FROM t2 ORDER BY 1; SET citus.log_remote_commands to true; @@ -924,27 +927,25 @@ ROLLBACK; -- Test the same scenarios with distributed tables SELECT create_distributed_table('target_cj', 'tid'); -SELECT create_distributed_table('source_cj1', 'sid1'); -SELECT create_distributed_table('source_cj2', 'sid2'); +SELECT create_distributed_table('source_cj1', 'sid1', colocate_with => 'target_cj'); +SELECT create_distributed_table('source_cj2', 'sid2', colocate_with => 'target_cj'); BEGIN; -SET citus.log_remote_commands to true; MERGE INTO target_cj t -USING source_cj1 s1 INNER JOIN source_cj2 s2 ON sid1 = sid2 +USING (SELECT * FROM source_cj1 s1 INNER JOIN source_cj2 s2 ON sid1 = sid2) s ON t.tid = sid1 AND t.tid = 2 WHEN MATCHED THEN UPDATE SET src = src2 WHEN NOT MATCHED THEN DO NOTHING; -SET citus.log_remote_commands to false; SELECT * FROM target_cj ORDER BY 1; ROLLBACK; BEGIN; -- try accessing columns from either side of the source join MERGE INTO target_cj t -USING source_cj1 s2 - INNER JOIN source_cj2 s1 ON sid1 = sid2 AND val1 = 10 +USING (SELECT * FROM source_cj1 s2 + INNER JOIN source_cj2 s1 ON sid1 = sid2 AND val1 = 10) s ON t.tid = sid1 AND t.tid = 2 WHEN MATCHED THEN UPDATE SET src = src1, val = val2 @@ -982,7 +983,7 @@ ROLLBACK; -- Test PREPARE -PREPARE foo(int) AS +PREPARE merge_prepare(int) AS MERGE INTO target_cj target USING (SELECT * FROM source_cj1) sub ON target.tid = sub.sid1 AND target.tid = $1 @@ -994,11 +995,11 @@ WHEN NOT MATCHED THEN SELECT * FROM target_cj ORDER BY 1; BEGIN; -EXECUTE foo(2); -EXECUTE foo(2); -EXECUTE foo(2); -EXECUTE foo(2); -EXECUTE foo(2); +EXECUTE merge_prepare(2); +EXECUTE merge_prepare(2); +EXECUTE merge_prepare(2); +EXECUTE merge_prepare(2); +EXECUTE merge_prepare(2); SELECT * FROM target_cj ORDER BY 1; ROLLBACK; @@ -1006,10 +1007,10 @@ BEGIN; SET citus.log_remote_commands to true; SET client_min_messages TO DEBUG1; -EXECUTE foo(2); +EXECUTE merge_prepare(2); RESET client_min_messages; -EXECUTE foo(2); +EXECUTE merge_prepare(2); SET citus.log_remote_commands to false; SELECT * FROM target_cj ORDER BY 1; @@ -1036,7 +1037,7 @@ INSERT INTO citus_target SELECT i, 'target' FROM generate_series(250, 500) i; INSERT INTO citus_source SELECT i, 'source' FROM generate_series(1, 500) i; SELECT create_distributed_table('citus_target', 'id'); -SELECT create_distributed_table('citus_source', 'id'); +SELECT create_distributed_table('citus_source', 'id', colocate_with => 'citus_target'); -- -- This routine compares the target tables of Postgres and Citus and @@ -1622,10 +1623,271 @@ SELECT count(*) FROM pg_result FULL OUTER JOIN local_ref ON pg_result.t1 = local_ref.t1 WHERE pg_result.t1 IS NULL OR local_ref.t1 IS NULL; +-- Now make target as distributed, keep reference as source +TRUNCATE reftarget_local; +TRUNCATE refsource_ref; +INSERT INTO reftarget_local VALUES(1, 0); +INSERT INTO reftarget_local VALUES(3, 100); +INSERT INTO refsource_ref VALUES(1, 1); +INSERT INTO refsource_ref VALUES(2, 2); +INSERT INTO refsource_ref VALUES(3, 3); + +SELECT create_distributed_table('reftarget_local', 't1'); + +MERGE INTO reftarget_local +USING (SELECT * FROM refsource_ref UNION SELECT * FROM refsource_ref) AS foo ON reftarget_local.t1 = foo.s1 +WHEN MATCHED AND reftarget_local.t2 = 100 THEN + DELETE +WHEN MATCHED THEN + UPDATE SET t2 = t2 + 100 +WHEN NOT MATCHED THEN + INSERT VALUES(foo.s1); +SELECT * INTO dist_reftarget FROM reftarget_local ORDER BY 1, 2; + +-- Should be equal +SELECT c.*, p.* +FROM dist_reftarget c, pg_result p +WHERE c.t1 = p.t1 +ORDER BY 1,2; + +-- Must return zero rows +SELECT count(*) +FROM pg_result FULL OUTER JOIN dist_reftarget ON pg_result.t1 = dist_reftarget.t1 +WHERE pg_result.t1 IS NULL OR dist_reftarget.t1 IS NULL; + +-- +-- Distributed (target), Reference(source) +-- +CREATE TABLE demo_distributed(id1 int, val1 int); +CREATE TABLE demo_source_table(id2 int, val2 int); + +CREATE FUNCTION setup_demo_data() RETURNS VOID AS $$ +INSERT INTO demo_distributed VALUES(1, 100); +INSERT INTO demo_distributed VALUES(7, 100); +INSERT INTO demo_distributed VALUES(15, 100); +INSERT INTO demo_distributed VALUES(100, 0); +INSERT INTO demo_distributed VALUES(300, 100); +INSERT INTO demo_distributed VALUES(400, 0); + +INSERT INTO demo_source_table VALUES(1, 77); +INSERT INTO demo_source_table VALUES(15, 77); +INSERT INTO demo_source_table VALUES(75, 77); +INSERT INTO demo_source_table VALUES(100, 77); +INSERT INTO demo_source_table VALUES(300, 77); +INSERT INTO demo_source_table VALUES(400, 77); +INSERT INTO demo_source_table VALUES(500, 77); +$$ +LANGUAGE SQL; + +CREATE FUNCTION merge_demo_data() RETURNS VOID AS $$ +MERGE INTO demo_distributed t +USING demo_source_table s ON s.id2 = t.id1 +WHEN MATCHED AND t.val1= 0 THEN + DELETE +WHEN MATCHED THEN + UPDATE SET val1 = val1 + s.val2 +WHEN NOT MATCHED THEN + INSERT VALUES(s.id2, s.val2); +$$ +LANGUAGE SQL; + +SELECT setup_demo_data(); +SELECT merge_demo_data(); +SELECT * INTO pg_demo_result FROM demo_distributed ORDER BY 1, 2; + +TRUNCATE demo_distributed; +TRUNCATE demo_source_table; + +SELECT create_distributed_table('demo_distributed', 'id1'); +SELECT create_reference_table('demo_source_table'); + +SELECT setup_demo_data(); +SELECT merge_demo_data(); + +SELECT * INTO dist_demo_result FROM demo_distributed ORDER BY 1, 2; + +-- Should be equal +SELECT c.*, p.* +FROM dist_demo_result c, pg_demo_result p +WHERE c.id1 = p.id1 +ORDER BY 1,2; + +-- Must return zero rows +SELECT count(*) +FROM pg_demo_result p FULL OUTER JOIN dist_demo_result d ON p.id1 = d.id1 +WHERE p.id1 IS NULL OR d.id1 IS NULL; + +-- Now convert source as distributed, but non-colocated with target +DROP TABLE pg_demo_result, dist_demo_result; +SELECT undistribute_table('demo_distributed'); +SELECT undistribute_table('demo_source_table'); + +CREATE OR REPLACE FUNCTION merge_demo_data() RETURNS VOID AS $$ +MERGE INTO demo_distributed t +USING (SELECT id2,val2 FROM demo_source_table UNION SELECT val2,id2 FROM demo_source_table) AS s +ON t.id1 = s.id2 +WHEN MATCHED THEN + UPDATE SET val1 = val1 + 1; +$$ +LANGUAGE SQL; + +TRUNCATE demo_distributed; +TRUNCATE demo_source_table; + +SELECT setup_demo_data(); +SELECT merge_demo_data(); +SELECT * INTO pg_demo_result FROM demo_distributed ORDER BY 1, 2; + +SELECT create_distributed_table('demo_distributed', 'id1'); +SELECT create_distributed_table('demo_source_table', 'id2', colocate_with=>'none'); + +TRUNCATE demo_distributed; +TRUNCATE demo_source_table; + +SELECT setup_demo_data(); +SELECT merge_demo_data(); +SELECT * INTO dist_demo_result FROM demo_distributed ORDER BY 1, 2; + +-- Should be equal +SELECT c.*, p.* +FROM dist_demo_result c, pg_demo_result p +WHERE c.id1 = p.id1 +ORDER BY 1,2; + +-- Must return zero rows +SELECT count(*) +FROM pg_demo_result p FULL OUTER JOIN dist_demo_result d ON p.id1 = d.id1 +WHERE p.id1 IS NULL OR d.id1 IS NULL; + +-- Test with LIMIT + +CREATE OR REPLACE FUNCTION merge_demo_data() RETURNS VOID AS $$ +MERGE INTO demo_distributed t +USING (SELECT 999 as s3, demo_source_table.* FROM (SELECT * FROM demo_source_table ORDER BY 1 LIMIT 3) as foo LEFT JOIN demo_source_table USING(id2)) AS s +ON t.id1 = s.id2 +WHEN MATCHED THEN + UPDATE SET val1 = s3 +WHEN NOT MATCHED THEN + INSERT VALUES(id2, s3); +$$ +LANGUAGE SQL; + +DROP TABLE pg_demo_result, dist_demo_result; +SELECT undistribute_table('demo_distributed'); +SELECT undistribute_table('demo_source_table'); + +TRUNCATE demo_distributed; +TRUNCATE demo_source_table; + +SELECT setup_demo_data(); +SELECT merge_demo_data(); +SELECT * INTO pg_demo_result FROM demo_distributed ORDER BY 1, 2; + +SELECT create_distributed_table('demo_distributed', 'id1'); +SELECT create_distributed_table('demo_source_table', 'id2', colocate_with=>'none'); + +TRUNCATE demo_distributed; +TRUNCATE demo_source_table; + +SELECT setup_demo_data(); +SELECT merge_demo_data(); +SELECT * INTO dist_demo_result FROM demo_distributed ORDER BY 1, 2; + +-- Should be equal +SELECT c.*, p.* +FROM dist_demo_result c, pg_demo_result p +WHERE c.id1 = p.id1 +ORDER BY 1,2; + +-- Must return zero rows +SELECT count(*) +FROM pg_demo_result p FULL OUTER JOIN dist_demo_result d ON p.id1 = d.id1 +WHERE p.id1 IS NULL OR d.id1 IS NULL; + +-- Test explain with repartition +SET citus.explain_all_tasks TO false; +EXPLAIN (COSTS OFF) +MERGE INTO demo_distributed t +USING (SELECT 999 as s3, demo_source_table.* FROM (SELECT * FROM demo_source_table ORDER BY 1 LIMIT 3) as foo LEFT JOIN demo_source_table USING(id2)) AS s +ON t.id1 = s.id2 +WHEN MATCHED THEN + UPDATE SET val1 = s3 +WHEN NOT MATCHED THEN + INSERT VALUES(id2, s3); + +-- Test multiple join conditions on distribution column +MERGE INTO demo_distributed t +USING (SELECT id2+1 as key, id2+3 as key2 FROM demo_source_table) s +ON t.id1 = s.key2 ANd t.id1 = s.key +WHEN NOT MATCHED THEN + INSERT VALUES(s.key2, 333); + +MERGE INTO demo_distributed t +USING (SELECT id2+1 as key, id2+2 as key2 FROM demo_source_table) s +ON t.id1 = s.key2 AND t.id1 = s.key +WHEN NOT MATCHED THEN + DO NOTHING; + +MERGE INTO demo_distributed t +USING (SELECT id2+1 as key, id2+3 as key2 FROM demo_source_table) s +ON t.val1 = s.key2 AND t.id1 = s.key AND t.id1 = s.key2 +WHEN NOT MATCHED THEN + INSERT VALUES(s.key2, 444); + +-- Test aggregate functions in source-query +SELECT COUNT(*) FROM demo_distributed where val1 = 150; +SELECT COUNT(*) FROM demo_distributed where id1 = 2; + +-- One row with Key=7 updated in demo_distributed to 150 +MERGE INTO demo_distributed t +USING (SELECT count(DISTINCT id2)::int4 as key FROM demo_source_table GROUP BY val2) s +ON t.id1 = s.key +WHEN NOT MATCHED THEN INSERT VALUES(s.key, 1) +WHEN MATCHED THEN UPDATE SET val1 = 150; + +-- Seven rows with Key=2 inserted in demo_distributed +MERGE INTO demo_distributed t +USING (SELECT (count(DISTINCT val2) + 1)::int4 as key FROM demo_source_table GROUP BY id2) s +ON t.id1 = s.key +WHEN NOT MATCHED THEN INSERT VALUES(s.key, 1) +WHEN MATCHED THEN UPDATE SET val1 = 150; + +SELECT COUNT(*) FROM demo_distributed where val1 = 150; +SELECT COUNT(*) FROM demo_distributed where id1 = 2; + -- -- Error and Unsupported scenarios -- + +-- Test explain analyze with repartition +EXPLAIN ANALYZE +MERGE INTO demo_distributed t +USING (SELECT 999 as s3, demo_source_table.* FROM (SELECT * FROM demo_source_table ORDER BY 1 LIMIT 3) as foo LEFT JOIN demo_source_table USING(id2)) AS s +ON t.id1 = s.id2 +WHEN MATCHED THEN + UPDATE SET val1 = s3 +WHEN NOT MATCHED THEN + INSERT VALUES(id2, s3); + +-- Source without a table +MERGE INTO target_cj t +USING (VALUES (1, 1), (2, 1), (3, 3)) as s (sid, val) +ON t.tid = s.sid AND t.tid = 2 +WHEN MATCHED THEN + UPDATE SET val = s.val +WHEN NOT MATCHED THEN + DO NOTHING; + +-- Incomplete source +MERGE INTO target_cj t +USING (source_cj1 s1 INNER JOIN source_cj2 s2 ON sid1 = val2) s +ON t.tid = s.sid1 AND t.tid = 2 +WHEN MATCHED THEN + UPDATE SET src = src2 +WHEN NOT MATCHED THEN + DO NOTHING; + -- Reference as a target and local as source MERGE INTO refsource_ref USING (SELECT * FROM reftarget_local UNION SELECT * FROM reftarget_local) AS foo ON refsource_ref.s1 = foo.t1 @@ -1634,34 +1896,16 @@ WHEN MATCHED THEN WHEN NOT MATCHED THEN INSERT VALUES(foo.t1); --- Reference as a source and distributed as target -MERGE INTO target_set t -USING refsource_ref AS s ON t.t1 = s.s1 -WHEN MATCHED THEN - DO NOTHING; - MERGE INTO target_set USING source_set AS foo ON target_set.t1 = foo.s1 WHEN MATCHED THEN UPDATE SET ctid = '(0,100)'; -MERGE INTO target_set -USING (SELECT s1,s2 FROM source_set UNION SELECT s2,s1 FROM source_set) AS foo ON target_set.t1 = foo.s1 -WHEN MATCHED THEN - UPDATE SET t2 = t2 + 1; - -MERGE INTO target_set -USING (SELECT 2 as s3, source_set.* FROM (SELECT * FROM source_set LIMIT 1) as foo LEFT JOIN source_set USING( s1)) AS foo -ON target_set.t1 = foo.s1 -WHEN MATCHED THEN UPDATE SET t2 = t2 + 1 -WHEN NOT MATCHED THEN INSERT VALUES(s1, s3); - - -- modifying CTE not supported EXPLAIN -WITH cte_1 AS (DELETE FROM target_json) +WITH cte_1 AS (DELETE FROM target_json RETURNING *) MERGE INTO target_json sda -USING source_json sdn +USING cte_1 sdn ON sda.id = sdn.id WHEN NOT matched THEN INSERT (id, z) VALUES (sdn.id, 5); @@ -1710,6 +1954,7 @@ ON t.id = s.id WHEN NOT MATCHED THEN INSERT (id) VALUES(1000); +-- Colocated merge MERGE INTO t1 t USING s1 s ON t.id = s.id @@ -1722,6 +1967,13 @@ ON t.id = s.id WHEN NOT MATCHED THEN INSERT (val) VALUES(s.val); +-- Non-colocated merge +MERGE INTO t1 t +USING s1 s +ON t.id = s.val +WHEN NOT MATCHED THEN + INSERT (id) VALUES(s.id); + -- try updating the distribution key column BEGIN; MERGE INTO target_cj t @@ -1810,17 +2062,7 @@ WHEN MATCHED AND (merge_when_and_write()) THEN ROLLBACK; --- Joining on partition columns with sub-query -MERGE INTO t1 - USING (SELECT * FROM s1) sub ON (sub.val = t1.id) -- sub.val is not a distribution column - WHEN MATCHED AND sub.val = 0 THEN - DELETE - WHEN MATCHED THEN - UPDATE SET val = t1.val + 1 - WHEN NOT MATCHED THEN - INSERT (id, val) VALUES (sub.id, sub.val); - --- Joining on partition columns with CTE +-- Joining on non-partition columns with CTE source, but INSERT incorrect column WITH s1_res AS ( SELECT * FROM s1 ) @@ -1846,7 +2088,7 @@ MERGE INTO t1 WHEN NOT MATCHED THEN INSERT (id, val) VALUES (s1_res.id, s1_res.val); --- With a single WHEN clause, which causes a non-left join +-- Join condition without target distribution column WITH s1_res AS ( SELECT * FROM s1 ) @@ -1953,34 +2195,12 @@ WHEN MATCHED THEN WHEN NOT MATCHED THEN INSERT VALUES(mv_source.id, mv_source.val); --- Distributed tables *must* be colocated +-- Do not allow constant values into the distribution column CREATE TABLE dist_target(id int, val varchar); SELECT create_distributed_table('dist_target', 'id'); CREATE TABLE dist_source(id int, val varchar); SELECT create_distributed_table('dist_source', 'id', colocate_with => 'none'); -MERGE INTO dist_target -USING dist_source -ON dist_target.id = dist_source.id -WHEN MATCHED THEN -UPDATE SET val = dist_source.val -WHEN NOT MATCHED THEN -INSERT VALUES(dist_source.id, dist_source.val); - --- Distributed tables *must* be joined on distribution column -CREATE TABLE dist_colocated(id int, val int); -SELECT create_distributed_table('dist_colocated', 'id', colocate_with => 'dist_target'); - -MERGE INTO dist_target -USING dist_colocated -ON dist_target.id = dist_colocated.val -- val is not the distribution column -WHEN MATCHED THEN -UPDATE SET val = dist_colocated.val -WHEN NOT MATCHED THEN -INSERT VALUES(dist_colocated.id, dist_colocated.val); - - --- Both the source and target must be distributed MERGE INTO dist_target USING (SELECT 100 id) AS source ON dist_target.id = source.id AND dist_target.val = 'const' @@ -2055,7 +2275,6 @@ INSERT VALUES(dist_source.id, dist_source.val); CREATE SCHEMA query_single_shard_table; SET search_path TO query_single_shard_table; -SET client_min_messages TO DEBUG2; CREATE TABLE nullkey_c1_t1(a int, b int); CREATE TABLE nullkey_c1_t2(a int, b int); @@ -2068,15 +2287,17 @@ SELECT create_distributed_table('nullkey_c2_t1', null, colocate_with=>'none'); SELECT create_distributed_table('nullkey_c2_t2', null, colocate_with=>'nullkey_c2_t1', distribution_type=>null); CREATE TABLE reference_table(a int, b int); +CREATE TABLE distributed_table(a int, b int); +CREATE TABLE citus_local_table(a int, b int); SELECT create_reference_table('reference_table'); +SELECT create_distributed_table('distributed_table', 'a'); +SELECT citus_add_local_table_to_metadata('citus_local_table'); + +SET client_min_messages TO DEBUG2; INSERT INTO reference_table SELECT i, i FROM generate_series(0, 5) i; -CREATE TABLE distributed_table(a int, b int); -SELECT create_distributed_table('distributed_table', 'a'); INSERT INTO distributed_table SELECT i, i FROM generate_series(3, 8) i; -CREATE TABLE citus_local_table(a int, b int); -SELECT citus_add_local_table_to_metadata('citus_local_table'); INSERT INTO citus_local_table SELECT i, i FROM generate_series(0, 10) i; CREATE TABLE postgres_local_table(a int, b int); @@ -2159,9 +2380,117 @@ WHEN MATCHED THEN UPDATE SET b = cte.b; SET client_min_messages TO WARNING; DROP SCHEMA query_single_shard_table CASCADE; -RESET client_min_messages; SET search_path TO merge_schema; +-- Test Columnar table +CREATE TABLE target_columnar(cid int, name text) USING columnar; +SELECT create_distributed_table('target_columnar', 'cid'); +MERGE INTO target_columnar t +USING demo_source_table s +ON t.cid = s.id2 +WHEN MATCHED THEN + UPDATE SET name = 'Columnar table updated by MERGE' +WHEN NOT MATCHED THEN + DO NOTHING; + +MERGE INTO demo_distributed t +USING generate_series(0,100) as source(key) +ON (source.key + 1 = t.id1) + WHEN MATCHED THEN UPDATE SET val1 = 15; + +-- This should fail in planning stage itself +EXPLAIN MERGE INTO demo_distributed t +USING demo_source_table s +ON (s.id2 + 1 = t.id1) + WHEN MATCHED THEN UPDATE SET val1 = 15; + +-- Sub-queries and CTEs are not allowed in actions and ON clause +CREATE TABLE target_1 (a int, b int, c int); +SELECT create_distributed_table('target_1', 'a'); + +CREATE TABLE source_2 (a int, b int, c int); +SELECT create_distributed_table('source_2', 'a'); + +INSERT INTO target_1 VALUES(1, 2, 3); +INSERT INTO target_1 VALUES(4, 5, 6); +INSERT INTO target_1 VALUES(11, 12, 13); + +INSERT INTO source_2 VALUES(1, 2, 3); + +WITH cte_1 as (SELECT max(a) as max_a, max(b) as b FROM source_2) +MERGE INTO target_1 +USING cte_1 +ON (target_1.a = cte_1.b) +WHEN NOT MATCHED AND (SELECT max_a > 10 FROM cte_1) THEN + INSERT VALUES (cte_1.b, 100); + +WITH cte_1 as (SELECT a, b FROM source_2) +MERGE INTO target_1 +USING cte_1 +ON (target_1.a = cte_1.b) +WHEN NOT MATCHED AND (SELECT a > 10 FROM cte_1) THEN + INSERT VALUES (cte_1.b, 100); + +MERGE INTO target_1 +USING source_2 +ON (target_1.a = source_2.b) +WHEN NOT MATCHED AND (SELECT max_a > 10 FROM (SELECT max(a) as max_a, max(b) as b FROM target_1) as foo) THEN + INSERT VALUES (source_2.b, 100); + +-- or same with CTEs +WITH cte_1 as (SELECT max(a) as max_a, max(b) as b FROM target_1) +MERGE INTO target_1 +USING source_2 +ON (target_1.a = source_2.b) +WHEN NOT MATCHED AND (SELECT max_a > 10 FROM (SELECT max(a) as max_a, max(b) as b FROM target_1) as foo) THEN + INSERT VALUES (source_2.b, 100); + +WITH cte_1 as (SELECT a, b FROM target_1), cte_2 as (select b,a from target_1) +MERGE INTO target_1 +USING (SELECT * FROM source_2) as subq +ON (target_1.a = subq.b) +WHEN NOT MATCHED AND (SELECT a > 10 FROM cte_2) THEN + INSERT VALUES (subq.b, 100); + +MERGE INTO source_2 +USING target_1 +ON (target_1.a = source_2.a) +WHEN MATCHED THEN + UPDATE SET b = (SELECT max(a) FROM source_2); + +MERGE INTO source_2 +USING target_1 +ON (target_1.a = source_2.a) +WHEN NOT MATCHED THEN + INSERT VALUES (target_1.a,(select max(a) from target_1)); + +MERGE INTO target_1 +USING source_2 +ON (target_1.a = source_2.b) +WHEN NOT MATCHED AND (SELECT max(c) > 10 FROM source_2) THEN + INSERT VALUES (source_2.b, 100); + +-- Test in ON clause +MERGE INTO target_1 t2 +USING (SELECT * FROM source_2) AS t1 +ON (t1.a = t2.a AND (SELECT 1=1 FROM target_1)) +WHEN MATCHED THEN + DELETE; + +MERGE INTO target_1 t2 +USING (SELECT * FROM source_2) AS t1 +ON (t1.a = t2.a AND (SELECT max(a) > 55 FROM target_1)) +WHEN MATCHED THEN + DELETE; + +WITH cte_1 as (SELECT a, b FROM target_1), cte_2 as (select b,a from target_1) +MERGE INTO target_1 t2 +USING (SELECT * FROM cte_1) AS t1 +ON (t1.a = t2.a AND (SELECT max(a) > 55 FROM cte_2)) +WHEN MATCHED THEN + DELETE; + +RESET client_min_messages; DROP SERVER foreign_server CASCADE; DROP FUNCTION merge_when_and_write(); DROP SCHEMA merge_schema CASCADE; diff --git a/src/test/regress/sql/merge_arbitrary.sql b/src/test/regress/sql/merge_arbitrary.sql index 17b7d4f90..6c0a931dc 100644 --- a/src/test/regress/sql/merge_arbitrary.sql +++ b/src/test/regress/sql/merge_arbitrary.sql @@ -131,3 +131,48 @@ BEGIN; EXECUTE local(0, 1); SELECT * FROM t1 order by id; ROLLBACK; + +-- Test prepared statements with repartition +PREPARE merge_repartition_pg(int,int,int,int) as + MERGE INTO pg_target target + USING (SELECT id+1+$1 as key, val FROM (SELECT * FROM pg_source UNION SELECT * FROM pg_source WHERE id = $2) as foo) as source + ON (source.key = target.id AND $3 < 10000) + WHEN MATCHED THEN UPDATE SET val = (source.key::int+$4) + WHEN NOT MATCHED THEN INSERT VALUES (source.key, source.val); + +PREPARE merge_repartition_citus(int,int,int,int) as + MERGE INTO citus_target target + USING (SELECT id+1+$1 as key, val FROM (SELECT * FROM citus_source UNION SELECT * FROM citus_source WHERE id = $2) as foo) as source + ON (source.key = target.id AND $3 < 10000) + WHEN MATCHED THEN UPDATE SET val = (source.key::int+$4) + WHEN NOT MATCHED THEN INSERT VALUES (source.key, source.val); + +EXECUTE merge_repartition_pg(1,1,1,1); +EXECUTE merge_repartition_citus(1,1,1,1); + +SET client_min_messages = NOTICE; +SELECT compare_data(); +RESET client_min_messages; + +EXECUTE merge_repartition_pg(1,100,1,1); +EXECUTE merge_repartition_citus(1,100,1,1); + +EXECUTE merge_repartition_pg(2,200,1,1); +EXECUTE merge_repartition_citus(2,200,1,1); + +EXECUTE merge_repartition_pg(3,300,1,1); +EXECUTE merge_repartition_citus(3,300,1,1); + +EXECUTE merge_repartition_pg(4,400,1,1); +EXECUTE merge_repartition_citus(4,400,1,1); + +EXECUTE merge_repartition_pg(5,500,1,1); +EXECUTE merge_repartition_citus(5,500,1,1); + +-- Sixth time +EXECUTE merge_repartition_pg(6,600,1,6); +EXECUTE merge_repartition_citus(6,600,1,6); + +SET client_min_messages = NOTICE; +SELECT compare_data(); +RESET client_min_messages; diff --git a/src/test/regress/sql/merge_arbitrary_create.sql b/src/test/regress/sql/merge_arbitrary_create.sql index edf9b0d9d..efa3185da 100644 --- a/src/test/regress/sql/merge_arbitrary_create.sql +++ b/src/test/regress/sql/merge_arbitrary_create.sql @@ -48,3 +48,57 @@ CREATE TABLE s1(id int, val int); SELECT citus_add_local_table_to_metadata('t1'); SELECT citus_add_local_table_to_metadata('s1'); + +-- Test prepared statements with repartition +CREATE TABLE pg_target(id int, val int); +CREATE TABLE pg_source(id int, val int, const int); +CREATE TABLE citus_target(id int, val int); +CREATE TABLE citus_source(id int, val int, const int); +SELECT citus_add_local_table_to_metadata('pg_target'); +SELECT citus_add_local_table_to_metadata('pg_source'); + +-- +-- Load same set of data to both Postgres and Citus tables +-- +CREATE OR REPLACE FUNCTION setup_data() RETURNS VOID AS $$ + INSERT INTO pg_source SELECT i, i+1, 1 FROM generate_series(1, 10000) i; + INSERT INTO pg_target SELECT i, 1 FROM generate_series(5001, 10000) i; + INSERT INTO citus_source SELECT i, i+1, 1 FROM generate_series(1, 10000) i; + INSERT INTO citus_target SELECT i, 1 FROM generate_series(5001, 10000) i; +$$ +LANGUAGE SQL; + +-- +-- Compares the final target tables, merge-modified data, of both Postgres and Citus tables +-- +CREATE OR REPLACE FUNCTION check_data(table1_name text, column1_name text, table2_name text, column2_name text) +RETURNS VOID AS $$ +DECLARE + table1_avg numeric; + table2_avg numeric; +BEGIN + EXECUTE format('SELECT COALESCE(AVG(%I), 0) FROM %I', column1_name, table1_name) INTO table1_avg; + EXECUTE format('SELECT COALESCE(AVG(%I), 0) FROM %I', column2_name, table2_name) INTO table2_avg; + + IF table1_avg > table2_avg THEN + RAISE EXCEPTION 'The average of %.% is greater than %.%', table1_name, column1_name, table2_name, column2_name; + ELSIF table1_avg < table2_avg THEN + RAISE EXCEPTION 'The average of %.% is less than %.%', table1_name, column1_name, table2_name, column2_name; + ELSE + RAISE NOTICE 'The average of %.% is equal to %.%', table1_name, column1_name, table2_name, column2_name; + END IF; +END; +$$ LANGUAGE plpgsql; + +CREATE OR REPLACE FUNCTION compare_data() RETURNS VOID AS $$ + SELECT check_data('pg_target', 'id', 'citus_target', 'id'); + SELECT check_data('pg_target', 'val', 'citus_target', 'val'); +$$ +LANGUAGE SQL; + +-- +-- Target and source are distributed, and non-colocated +-- +SELECT setup_data(); +SELECT create_distributed_table('citus_target', 'id'); +SELECT create_distributed_table('citus_source', 'id', colocate_with=>'none'); diff --git a/src/test/regress/sql/merge_partition_tables.sql b/src/test/regress/sql/merge_partition_tables.sql new file mode 100644 index 000000000..ab40fd23e --- /dev/null +++ b/src/test/regress/sql/merge_partition_tables.sql @@ -0,0 +1,164 @@ + +SHOW server_version \gset +SELECT substring(:'server_version', '\d+')::int >= 15 AS server_version_ge_15 +\gset +\if :server_version_ge_15 +\else +\q +\endif + +-- We create two sets of source and target tables, one set in Postgres and +-- the other in Citus distributed. We run the _exact_ MERGE SQL on both sets +-- and compare the final results of the target tables in Postgres and Citus. +-- The results should match. This process is repeated for various combinations +-- of MERGE SQL. + +DROP SCHEMA IF EXISTS merge_partition_tables CASCADE; +CREATE SCHEMA merge_partition_tables; +SET search_path TO merge_partition_tables; +SET citus.shard_count TO 4; +SET citus.next_shard_id TO 7000000; +SET citus.explain_all_tasks TO true; +SET citus.shard_replication_factor TO 1; +SET citus.max_adaptive_executor_pool_size TO 1; +SET client_min_messages = warning; +SELECT 1 FROM master_add_node('localhost', :master_port, groupid => 0); +RESET client_min_messages; + + +CREATE TABLE pg_target(id int, val int) PARTITION BY RANGE(id); +CREATE TABLE pg_source(id int, val int, const int) PARTITION BY RANGE(val); +CREATE TABLE citus_target(id int, val int) PARTITION BY RANGE(id); +CREATE TABLE citus_source(id int, val int, const int) PARTITION BY RANGE(val); +SELECT citus_add_local_table_to_metadata('citus_target'); +SELECT citus_add_local_table_to_metadata('citus_source'); + +CREATE TABLE part1 PARTITION OF pg_target FOR VALUES FROM (1) TO (2500) WITH (autovacuum_enabled=off); +CREATE TABLE part2 PARTITION OF pg_target FOR VALUES FROM (2501) TO (5000) WITH (autovacuum_enabled=off); +CREATE TABLE part3 PARTITION OF pg_target FOR VALUES FROM (5001) TO (7500) WITH (autovacuum_enabled=off); +CREATE TABLE part4 PARTITION OF pg_target DEFAULT WITH (autovacuum_enabled=off); +CREATE TABLE part5 PARTITION OF citus_target FOR VALUES FROM (1) TO (2500) WITH (autovacuum_enabled=off); +CREATE TABLE part6 PARTITION OF citus_target FOR VALUES FROM (2501) TO (5000) WITH (autovacuum_enabled=off); +CREATE TABLE part7 PARTITION OF citus_target FOR VALUES FROM (5001) TO (7500) WITH (autovacuum_enabled=off); +CREATE TABLE part8 PARTITION OF citus_target DEFAULT WITH (autovacuum_enabled=off); + +CREATE TABLE part9 PARTITION OF pg_source FOR VALUES FROM (1) TO (2500) WITH (autovacuum_enabled=off); +CREATE TABLE part10 PARTITION OF pg_source FOR VALUES FROM (2501) TO (5000) WITH (autovacuum_enabled=off); +CREATE TABLE part11 PARTITION OF pg_source FOR VALUES FROM (5001) TO (7500) WITH (autovacuum_enabled=off); +CREATE TABLE part12 PARTITION OF pg_source DEFAULT WITH (autovacuum_enabled=off); +CREATE TABLE part13 PARTITION OF citus_source FOR VALUES FROM (1) TO (2500) WITH (autovacuum_enabled=off); +CREATE TABLE part14 PARTITION OF citus_source FOR VALUES FROM (2501) TO (5000) WITH (autovacuum_enabled=off); +CREATE TABLE part15 PARTITION OF citus_source FOR VALUES FROM (5001) TO (7500) WITH (autovacuum_enabled=off); +CREATE TABLE part16 PARTITION OF citus_source DEFAULT WITH (autovacuum_enabled=off); + +CREATE OR REPLACE FUNCTION cleanup_data() RETURNS VOID SET search_path TO merge_partition_tables AS $$ + TRUNCATE pg_target; + TRUNCATE pg_source; + TRUNCATE citus_target; + TRUNCATE citus_source; + SELECT undistribute_table('citus_target'); + SELECT undistribute_table('citus_source'); +$$ +LANGUAGE SQL; + +-- +-- Load same set of data to both Postgres and Citus tables +-- +CREATE OR REPLACE FUNCTION setup_data() RETURNS VOID SET search_path TO merge_partition_tables AS $$ + INSERT INTO pg_source SELECT i, i+1, 1 FROM generate_series(1, 10000) i; + INSERT INTO pg_target SELECT i, 1 FROM generate_series(5001, 10000) i; + INSERT INTO citus_source SELECT i, i+1, 1 FROM generate_series(1, 10000) i; + INSERT INTO citus_target SELECT i, 1 FROM generate_series(5001, 10000) i; +$$ +LANGUAGE SQL; + +-- +-- Compares the final target tables, merge-modified data, of both Postgres and Citus tables +-- +CREATE OR REPLACE FUNCTION check_data(table1_name text, column1_name text, table2_name text, column2_name text) +RETURNS VOID SET search_path TO merge_partition_tables AS $$ +DECLARE + table1_avg numeric; + table2_avg numeric; +BEGIN + EXECUTE format('SELECT COALESCE(AVG(%I), 0) FROM %I', column1_name, table1_name) INTO table1_avg; + EXECUTE format('SELECT COALESCE(AVG(%I), 0) FROM %I', column2_name, table2_name) INTO table2_avg; + + IF table1_avg > table2_avg THEN + RAISE EXCEPTION 'The average of %.% is greater than %.%', table1_name, column1_name, table2_name, column2_name; + ELSIF table1_avg < table2_avg THEN + RAISE EXCEPTION 'The average of %.% is less than %.%', table1_name, column1_name, table2_name, column2_name; + ELSE + RAISE NOTICE 'The average of %.% is equal to %.%', table1_name, column1_name, table2_name, column2_name; + END IF; +END; +$$ LANGUAGE plpgsql; + +CREATE OR REPLACE FUNCTION compare_data() RETURNS VOID SET search_path TO merge_partition_tables AS $$ + SELECT check_data('pg_target', 'id', 'citus_target', 'id'); + SELECT check_data('pg_target', 'val', 'citus_target', 'val'); +$$ +LANGUAGE SQL; + +-- Test colocated partition tables + +SET client_min_messages = ERROR; +SELECT cleanup_data(); +SELECT setup_data(); +SELECT create_distributed_table('citus_target', 'id'); +SELECT create_distributed_table('citus_source', 'id', colocate_with=>'citus_target'); +RESET client_min_messages; + +MERGE INTO pg_target t +USING pg_source s +ON t.id = s.id +WHEN MATCHED AND t.id <= 7500 THEN + UPDATE SET val = s.val + 1 +WHEN MATCHED THEN + DELETE +WHEN NOT MATCHED THEN + INSERT VALUES(s.id, s.val); + +MERGE INTO citus_target t +USING citus_source s +ON t.id = s.id +WHEN MATCHED AND t.id <= 7500 THEN + UPDATE SET val = s.val + 1 +WHEN MATCHED THEN + DELETE +WHEN NOT MATCHED THEN + INSERT VALUES(s.id, s.val); + +SELECT compare_data(); + +-- Test non-colocated partition tables + +SET client_min_messages = ERROR; +SELECT cleanup_data(); +SELECT setup_data(); +SELECT create_distributed_table('citus_target', 'id'); +SELECT create_distributed_table('citus_source', 'id', colocate_with=>'none'); +RESET client_min_messages; + +MERGE INTO pg_target t +USING pg_source s +ON t.id = s.id +WHEN MATCHED AND t.id <= 7500 THEN + UPDATE SET val = s.val + 1 +WHEN MATCHED THEN + DELETE +WHEN NOT MATCHED THEN + INSERT VALUES(s.id, s.val); + +MERGE INTO citus_target t +USING citus_source s +ON t.id = s.id +WHEN MATCHED AND t.id <= 7500 THEN + UPDATE SET val = s.val + 1 +WHEN MATCHED THEN + DELETE +WHEN NOT MATCHED THEN + INSERT VALUES(s.id, s.val); + +SELECT compare_data(); +DROP SCHEMA merge_partition_tables CASCADE; diff --git a/src/test/regress/sql/merge_repartition1.sql b/src/test/regress/sql/merge_repartition1.sql new file mode 100644 index 000000000..4d73e999d --- /dev/null +++ b/src/test/regress/sql/merge_repartition1.sql @@ -0,0 +1,515 @@ +SHOW server_version \gset +SELECT substring(:'server_version', '\d+')::int >= 15 AS server_version_ge_15 +\gset +\if :server_version_ge_15 +\else +\q +\endif + +-- We create two sets of source and target tables, one set in Postgres and +-- the other in Citus distributed. We run the _exact_ MERGE SQL on both sets +-- and compare the final results of the target tables in Postgres and Citus. +-- The results should match. This process is repeated for various combinations +-- of MERGE SQL. + +DROP SCHEMA IF EXISTS merge_repartition1_schema CASCADE; +CREATE SCHEMA merge_repartition1_schema; +SET search_path TO merge_repartition1_schema; +SET citus.shard_count TO 4; +SET citus.next_shard_id TO 5000000; +SET citus.explain_all_tasks TO true; +SET citus.shard_replication_factor TO 1; +SET citus.max_adaptive_executor_pool_size TO 1; +SET client_min_messages = warning; +SELECT 1 FROM master_add_node('localhost', :master_port, groupid => 0); +RESET client_min_messages; + + +CREATE TABLE pg_target(id int, val int); +CREATE TABLE pg_source(id int, val int, const int); +CREATE TABLE citus_target(id int, val int); +CREATE TABLE citus_source(id int, val int, const int); +SELECT citus_add_local_table_to_metadata('citus_target'); +SELECT citus_add_local_table_to_metadata('citus_source'); + +CREATE OR REPLACE FUNCTION cleanup_data() RETURNS VOID SET search_path TO merge_repartition1_schema AS $$ + TRUNCATE pg_target; + TRUNCATE pg_source; + TRUNCATE citus_target; + TRUNCATE citus_source; + SELECT undistribute_table('citus_target'); + SELECT undistribute_table('citus_source'); +$$ +LANGUAGE SQL; +-- +-- Load same set of data to both Postgres and Citus tables +-- +CREATE OR REPLACE FUNCTION setup_data() RETURNS VOID SET search_path TO merge_repartition1_schema AS $$ + INSERT INTO pg_source SELECT i, i+1, 1 FROM generate_series(1, 10000) i; + INSERT INTO pg_target SELECT i, 1 FROM generate_series(5001, 10000) i; + INSERT INTO citus_source SELECT i, i+1, 1 FROM generate_series(1, 10000) i; + INSERT INTO citus_target SELECT i, 1 FROM generate_series(5001, 10000) i; +$$ +LANGUAGE SQL; + +-- +-- Compares the final target tables, merge-modified data, of both Postgres and Citus tables +-- +CREATE OR REPLACE FUNCTION check_data(table1_name text, column1_name text, table2_name text, column2_name text) +RETURNS VOID SET search_path TO merge_repartition1_schema AS $$ +DECLARE + table1_avg numeric; + table2_avg numeric; +BEGIN + EXECUTE format('SELECT COALESCE(AVG(%I), 0) FROM %I', column1_name, table1_name) INTO table1_avg; + EXECUTE format('SELECT COALESCE(AVG(%I), 0) FROM %I', column2_name, table2_name) INTO table2_avg; + + IF table1_avg > table2_avg THEN + RAISE EXCEPTION 'The average of %.% is greater than %.%', table1_name, column1_name, table2_name, column2_name; + ELSIF table1_avg < table2_avg THEN + RAISE EXCEPTION 'The average of %.% is less than %.%', table1_name, column1_name, table2_name, column2_name; + ELSE + RAISE NOTICE 'The average of %.% is equal to %.%', table1_name, column1_name, table2_name, column2_name; + END IF; +END; +$$ LANGUAGE plpgsql; + +CREATE OR REPLACE FUNCTION compare_data() RETURNS VOID SET search_path TO merge_repartition1_schema AS $$ + SELECT check_data('pg_target', 'id', 'citus_target', 'id'); + SELECT check_data('pg_target', 'val', 'citus_target', 'val'); +$$ +LANGUAGE SQL; + +-- +-- Target and source are distributed, and non-colocated +-- +SELECT cleanup_data(); +SELECT setup_data(); +SELECT create_distributed_table('citus_target', 'id'); +SELECT create_distributed_table('citus_source', 'id', colocate_with=>'none'); + +MERGE INTO pg_target t +USING pg_source s +ON t.id = s.id +WHEN MATCHED AND t.id <= 7500 THEN + UPDATE SET val = s.val + 1 +WHEN MATCHED THEN + DELETE +WHEN NOT MATCHED THEN + INSERT VALUES(s.id, s.val); + +MERGE INTO citus_target t +USING citus_source s +ON t.id = s.id +WHEN MATCHED AND t.id <= 7500 THEN + UPDATE SET val = s.val + 1 +WHEN MATCHED THEN + DELETE +WHEN NOT MATCHED THEN + INSERT VALUES(s.id, s.val); + +SELECT compare_data(); + +-- +-- Target and source are distributed, and colocated but not joined on distribution column +-- +SELECT cleanup_data(); +SELECT setup_data(); +SELECT create_distributed_table('citus_target', 'id'); +SELECT create_distributed_table('citus_source', 'id', colocate_with=>'citus_target'); + +MERGE INTO pg_target t +USING (SELECT * FROM pg_source) subq +ON (subq.val = t.id) +WHEN MATCHED AND t.id <= 7500 THEN + UPDATE SET val = subq.val + 1 +WHEN MATCHED THEN + DELETE +WHEN NOT MATCHED THEN + INSERT VALUES(subq.val, subq.id); + +MERGE INTO citus_target t +USING (SELECT * FROM citus_source) subq +ON (subq.val = t.id) +WHEN MATCHED AND t.id <= 7500 THEN + UPDATE SET val = subq.val + 1 +WHEN MATCHED THEN + DELETE +WHEN NOT MATCHED THEN + INSERT VALUES(subq.val, subq.id); + +SELECT compare_data(); + +-- +-- Target and source are distributed, colocated, joined on distribution column +-- but with nondistribution values +-- +SELECT cleanup_data(); +SELECT setup_data(); +SELECT create_distributed_table('citus_target', 'id'); +SELECT create_distributed_table('citus_source', 'id', colocate_with=>'citus_target'); + +MERGE INTO pg_target t +USING (SELECT id,const FROM pg_source UNION SELECT const,id FROM pg_source ) AS s +ON t.id = s.id +WHEN MATCHED THEN + UPDATE SET val = s.const + 1 +WHEN NOT MATCHED THEN + INSERT VALUES(id, const); + +MERGE INTO citus_target t +USING (SELECT id,const FROM citus_source UNION SELECT const,id FROM citus_source) AS s +ON t.id = s.id +WHEN MATCHED THEN + UPDATE SET val = s.const + 1 +WHEN NOT MATCHED THEN + INSERT VALUES(id, const); + +SELECT compare_data(); + +-- +-- Repartition with a predicate on target_table_name rows in ON clause +-- +SELECT cleanup_data(); +SELECT setup_data(); +SELECT create_distributed_table('citus_target', 'id'); +SELECT create_distributed_table('citus_source', 'id', colocate_with=>'none'); + +MERGE INTO pg_target t +USING (SELECT * FROM pg_source WHERE id < 9500) s +ON t.id = s.id AND t.id < 9000 +WHEN MATCHED AND t.id <= 7500 THEN + UPDATE SET val = s.val + 1 +WHEN MATCHED THEN + DELETE +WHEN NOT MATCHED THEN + INSERT VALUES(s.id, s.val); + +MERGE INTO citus_target t +USING (SELECT * FROM citus_source WHERE id < 9500) s +ON t.id = s.id AND t.id < 9000 +WHEN MATCHED AND t.id <= 7500 THEN + UPDATE SET val = s.val + 1 +WHEN MATCHED THEN + DELETE +WHEN NOT MATCHED THEN + INSERT VALUES(s.id, s.val); + +SELECT compare_data(); + +-- +-- Test CTE and non-colocated tables +-- +SELECT cleanup_data(); +SELECT setup_data(); +SELECT create_distributed_table('citus_target', 'id'); +SELECT create_distributed_table('citus_source', 'id', colocate_with=>'none'); + +WITH cte AS ( + SELECT * FROM pg_source +) +MERGE INTO pg_target t +USING cte s +ON s.id = t.id +WHEN MATCHED AND t.id > 7500 THEN + UPDATE SET val = s.val + 1 +WHEN MATCHED THEN + DELETE +WHEN NOT MATCHED THEN + INSERT VALUES (s.id, s.val); + +WITH cte AS ( + SELECT * FROM citus_source +) +MERGE INTO citus_target t +USING cte s +ON s.id = t.id +WHEN MATCHED AND t.id > 7500 THEN + UPDATE SET val = s.val + 1 +WHEN MATCHED THEN + DELETE +WHEN NOT MATCHED THEN + INSERT VALUES (s.id, s.val); + +SELECT compare_data(); + +-- +-- Test nested CTEs +-- +SELECT cleanup_data(); +SELECT setup_data(); +SELECT create_distributed_table('citus_target', 'id'); +SELECT create_distributed_table('citus_source', 'id', colocate_with=>'none'); + +WITH cte1 AS ( + SELECT * FROM pg_source ORDER BY 1 LIMIT 9000 +), +cte2 AS( + SELECT * FROM cte1 +), +cte3 AS( + SELECT * FROM cte2 +) +MERGE INTO pg_target t +USING cte3 s +ON (s.id=t.id) +WHEN MATCHED AND t.id > 7500 THEN + UPDATE SET val = s.val + 1 +WHEN MATCHED THEN + DELETE +WHEN NOT MATCHED THEN + INSERT VALUES (s.id, s.val); + +WITH cte1 AS ( + SELECT * FROM citus_source ORDER BY 1 LIMIT 9000 +), +cte2 AS( + SELECT * FROM cte1 +), +cte3 AS( + SELECT * FROM cte2 +) +MERGE INTO citus_target t +USING cte3 s +ON (s.id=t.id) +WHEN MATCHED AND t.id > 7500 THEN + UPDATE SET val = s.val + 1 +WHEN MATCHED THEN + DELETE +WHEN NOT MATCHED THEN + INSERT VALUES (s.id, s.val); + +SELECT compare_data(); + +-- +-- Target and source are distributed and colocated +-- +SELECT cleanup_data(); +SELECT setup_data(); +SELECT create_distributed_table('citus_target', 'id'); +SELECT create_distributed_table('citus_source', 'id', colocate_with => 'citus_target'); + +MERGE INTO pg_target t +USING (SELECT 999 as newval, pg_source.* FROM (SELECT * FROM pg_source ORDER BY 1 LIMIT 6000) as src LEFT JOIN pg_source USING(id)) AS s +ON t.id = s.id +WHEN MATCHED AND t.id <= 5500 THEN + UPDATE SET val = newval +WHEN MATCHED THEN + DELETE +WHEN NOT MATCHED THEN + INSERT VALUES(id, newval); + +MERGE INTO citus_target t +USING (SELECT 999 as newval, citus_source.* FROM (SELECT * FROM citus_source ORDER BY 1 LIMIT 6000) as src LEFT JOIN citus_source USING(id)) AS s +ON t.id = s.id +WHEN MATCHED AND t.id <= 5500 THEN + UPDATE SET val = newval +WHEN MATCHED THEN + DELETE +WHEN NOT MATCHED THEN + INSERT VALUES(id, newval); + +SELECT compare_data(); + +-- +-- Target is distributed and source is reference +-- +SELECT cleanup_data(); +SELECT setup_data(); +SELECT create_distributed_table('citus_target', 'id'); +SELECT create_reference_table('citus_source'); + +MERGE INTO pg_target t +USING pg_source s +ON t.id = s.id +WHEN MATCHED AND t.id <= 7500 THEN + UPDATE SET val = s.val + 1 +WHEN MATCHED THEN + DELETE +WHEN NOT MATCHED THEN + INSERT VALUES(s.id, s.val); + +MERGE INTO citus_target t +USING citus_source s +ON t.id = s.id +WHEN MATCHED AND t.id <= 7500 THEN + UPDATE SET val = s.val + 1 +WHEN MATCHED THEN + DELETE +WHEN NOT MATCHED THEN + INSERT VALUES(s.id, s.val); + +SELECT compare_data(); + +-- +-- Target is distributed and reference as source in a sub-query +-- +SELECT cleanup_data(); +SELECT setup_data(); +SELECT create_distributed_table('citus_target', 'id'); +SELECT create_reference_table('citus_source'); + +MERGE INTO pg_target t +USING (SELECT * FROM pg_source UNION SELECT * FROM pg_source) AS s ON t.id = s.id +WHEN MATCHED AND t.id <= 7500 THEN + UPDATE SET val = s.val + t.val +WHEN MATCHED THEN + DELETE +WHEN NOT MATCHED THEN + INSERT VALUES(s.id, s.val); + +MERGE INTO citus_target t +USING (SELECT * FROM citus_source UNION SELECT * FROM citus_source) AS s ON t.id = s.id +WHEN MATCHED AND t.id <= 7500 THEN + UPDATE SET val = s.val + t.val +WHEN MATCHED THEN + DELETE +WHEN NOT MATCHED THEN + INSERT VALUES(s.id, s.val); +SELECT compare_data(); + +-- +-- Target is distributed and citus-local as source +-- +SELECT cleanup_data(); +SELECT setup_data(); +SELECT create_distributed_table('citus_target', 'id'); +SELECT citus_add_local_table_to_metadata('citus_source'); + +MERGE INTO pg_target t +USING pg_source s +ON t.id = s.id +WHEN MATCHED AND t.id <= 7500 THEN + UPDATE SET val = s.val + 1 +WHEN MATCHED THEN + DELETE +WHEN NOT MATCHED THEN + INSERT VALUES(s.id, s.val); + +MERGE INTO citus_target t +USING citus_source s +ON t.id = s.id +WHEN MATCHED AND t.id <= 7500 THEN + UPDATE SET val = s.val + 1 +WHEN MATCHED THEN + DELETE +WHEN NOT MATCHED THEN + INSERT VALUES(s.id, s.val); + +SELECT compare_data(); + +-- +-- Target and source distributed and non-colocated. The source query requires evaluation +-- at the coordinator +-- +SELECT cleanup_data(); +SELECT setup_data(); +SELECT create_distributed_table('citus_target', 'id'); +SELECT create_distributed_table('citus_source', 'id', colocate_with=>'none'); + +MERGE INTO pg_target t +USING (SELECT 100 AS insval, MAX(const) AS updval, val, MAX(id) AS sid + FROM pg_source + GROUP BY val ORDER BY sid LIMIT 6000) AS s +ON t.id = s.sid +WHEN MATCHED AND t.id <= 5500 THEN + UPDATE SET val = updval + 1 +WHEN MATCHED THEN + DELETE +WHEN NOT MATCHED THEN + INSERT VALUES(sid, insval); + +MERGE INTO citus_target t +USING (SELECT 100 AS insval, MAX(const) AS updval, val, MAX(id) AS sid + FROM citus_source + GROUP BY val ORDER BY sid LIMIT 6000) AS s +ON t.id = s.sid +WHEN MATCHED AND t.id <= 5500 THEN + UPDATE SET val = updval + 1 +WHEN MATCHED THEN + DELETE +WHEN NOT MATCHED THEN + INSERT VALUES(sid, insval); + +SELECT compare_data(); + +-- Test source-query that requires repartitioning on top of MERGE repartitioning +SELECT cleanup_data(); +SELECT setup_data(); +SELECT create_distributed_table('citus_target', 'id'); +SELECT create_distributed_table('citus_source', 'id', colocate_with=>'none'); + +MERGE INTO pg_target t +USING (SELECT s1.val FROM pg_source s1 JOIN pg_source s2 USING (val)) AS s +ON t.id = s.val +WHEN MATCHED THEN + UPDATE SET val = t.val + 1; + +SET citus.enable_repartition_joins TO true; +MERGE INTO citus_target t +USING (SELECT s1.val FROM citus_source s1 JOIN citus_source s2 USING (val)) AS s +ON t.id = s.val +WHEN MATCHED THEN + UPDATE SET val = t.val + 1; + +SELECT compare_data(); + +-- +-- Test columnar as source table +-- +SET client_min_messages TO WARNING; +SELECT cleanup_data(); +RESET client_min_messages; +SELECT setup_data(); +SELECT create_distributed_table('citus_target', 'id'); +SELECT create_distributed_table('citus_source', 'id', colocate_with=>'none'); +SELECT alter_table_set_access_method('citus_source', 'columnar'); + +MERGE INTO pg_target t +USING pg_source s +ON t.id = s.id +WHEN MATCHED AND t.id <= 7500 THEN + UPDATE SET val = s.val + 1 +WHEN MATCHED THEN + DELETE +WHEN NOT MATCHED THEN + INSERT VALUES(s.id, s.val); + +MERGE INTO citus_target t +USING citus_source s +ON t.id = s.id +WHEN MATCHED AND t.id <= 7500 THEN + UPDATE SET val = s.val + 1 +WHEN MATCHED THEN + DELETE +WHEN NOT MATCHED THEN + INSERT VALUES(s.id, s.val); + +SELECT compare_data(); +SELECT alter_table_set_access_method('citus_source', 'heap'); + +-- Test CTE/Subquery in merge-actions (works only for router query) +SELECT cleanup_data(); +SELECT setup_data(); +SELECT create_distributed_table('citus_target', 'id'); +SELECT create_distributed_table('citus_source', 'id', colocate_with=>'citus_target'); + +MERGE INTO pg_target +USING pg_source +ON (pg_target.id = pg_source.id) +WHEN MATCHED AND (SELECT max_a > 5001 FROM (SELECT max(id) as max_a, max(val) as b FROM pg_target WHERE id = pg_source.id) AS foo) THEN + DELETE +WHEN NOT MATCHED AND (SELECT max_a < 5001 FROM (SELECT max(id) as max_a, max(val) as b FROM pg_target WHERE id = pg_source.id) AS foo) THEN + INSERT VALUES (pg_source.id, 100); + +MERGE INTO citus_target +USING citus_source +ON (citus_target.id = citus_source.id) +WHEN MATCHED AND (SELECT max_a > 5001 FROM (SELECT max(id) as max_a, max(val) as b FROM citus_target WHERE id = citus_source.id) AS foo) THEN + DELETE +WHEN NOT MATCHED AND (SELECT max_a < 5001 FROM (SELECT max(id) as max_a, max(val) as b FROM citus_target WHERE id = citus_source.id) AS foo) THEN + INSERT VALUES (citus_source.id, 100); + +SELECT compare_data(); + +DROP SCHEMA merge_repartition1_schema CASCADE; diff --git a/src/test/regress/sql/merge_repartition2.sql b/src/test/regress/sql/merge_repartition2.sql new file mode 100644 index 000000000..7a4812274 --- /dev/null +++ b/src/test/regress/sql/merge_repartition2.sql @@ -0,0 +1,139 @@ + +SHOW server_version \gset +SELECT substring(:'server_version', '\d+')::int >= 15 AS server_version_ge_15 +\gset +\if :server_version_ge_15 +\else +\q +\endif + +-- We create two sets of source and target tables, one set in Postgres and +-- the other in Citus distributed. We run the _exact_ MERGE SQL on both sets +-- and compare the final results of the target tables in Postgres and Citus. +-- The results should match. This process is repeated for various combinations +-- of MERGE SQL. + +DROP SCHEMA IF EXISTS merge_repartition2_schema CASCADE; +CREATE SCHEMA merge_repartition2_schema; +SET search_path TO merge_repartition2_schema; +SET citus.shard_count TO 4; +SET citus.next_shard_id TO 6000000; +SET citus.explain_all_tasks TO true; +SET citus.shard_replication_factor TO 1; +SET citus.max_adaptive_executor_pool_size TO 1; +SET client_min_messages = warning; +SELECT 1 FROM master_add_node('localhost', :master_port, groupid => 0); +RESET client_min_messages; + + +CREATE TABLE pg_target(id int, val int); +CREATE TABLE pg_source(id int, val int, const int); +CREATE TABLE citus_target(id int, val int); +CREATE TABLE citus_source(id int, val int, const int); +SELECT citus_add_local_table_to_metadata('citus_target'); +SELECT citus_add_local_table_to_metadata('citus_source'); + +CREATE OR REPLACE FUNCTION cleanup_data() RETURNS VOID SET search_path TO merge_repartition2_schema AS $$ + TRUNCATE pg_target; + TRUNCATE pg_source; + TRUNCATE citus_target; + TRUNCATE citus_source; + SELECT undistribute_table('citus_target'); + SELECT undistribute_table('citus_source'); +$$ +LANGUAGE SQL; +-- +-- Load same set of data to both Postgres and Citus tables +-- +CREATE OR REPLACE FUNCTION setup_data() RETURNS VOID SET search_path TO merge_repartition2_schema AS $$ + INSERT INTO pg_source SELECT i, i+1, 1 FROM generate_series(1, 100000) i; + INSERT INTO pg_target SELECT i, 1 FROM generate_series(50001, 100000) i; + INSERT INTO citus_source SELECT i, i+1, 1 FROM generate_series(1, 100000) i; + INSERT INTO citus_target SELECT i, 1 FROM generate_series(50001, 100000) i; +$$ +LANGUAGE SQL; + +-- +-- Compares the final target tables, merge-modified data, of both Postgres and Citus tables +-- +CREATE OR REPLACE FUNCTION check_data(table1_name text, column1_name text, table2_name text, column2_name text) +RETURNS VOID SET search_path TO merge_repartition2_schema AS $$ +DECLARE + table1_avg numeric; + table2_avg numeric; +BEGIN + EXECUTE format('SELECT COALESCE(AVG(%I), 0) FROM %I', column1_name, table1_name) INTO table1_avg; + EXECUTE format('SELECT COALESCE(AVG(%I), 0) FROM %I', column2_name, table2_name) INTO table2_avg; + + IF table1_avg > table2_avg THEN + RAISE EXCEPTION 'The average of %.% is greater than %.%', table1_name, column1_name, table2_name, column2_name; + ELSIF table1_avg < table2_avg THEN + RAISE EXCEPTION 'The average of %.% is less than %.%', table1_name, column1_name, table2_name, column2_name; + ELSE + RAISE NOTICE 'The average of %.% is equal to %.%', table1_name, column1_name, table2_name, column2_name; + END IF; +END; +$$ LANGUAGE plpgsql; + +CREATE OR REPLACE FUNCTION compare_data() RETURNS VOID SET search_path TO merge_repartition2_schema AS $$ + SELECT check_data('pg_target', 'id', 'citus_target', 'id'); + SELECT check_data('pg_target', 'val', 'citus_target', 'val'); +$$ +LANGUAGE SQL; + +-- Test nested cte +SELECT cleanup_data(); +SELECT setup_data(); +SELECT create_distributed_table('citus_target', 'id'); +SELECT create_distributed_table('citus_source', 'id', colocate_with=>'none'); + +WITH cte_top AS(WITH cte_1 AS (WITH cte_2 AS (SELECT id, val FROM pg_source) SELECT * FROM cte_2) SELECT * FROM cte_1) +MERGE INTO pg_target t +USING (SELECT const, val, id FROM pg_source WHERE id IN (SELECT id FROM cte_top)) as s +ON (s.id = t.id) +WHEN MATCHED AND t.id <= 75000 THEN + UPDATE SET val = (s.val::int8+1) +WHEN MATCHED THEN + DELETE +WHEN NOT MATCHED THEN + INSERT VALUES (s.id, s.val); + +WITH cte_top AS(WITH cte_1 AS (WITH cte_2 AS (SELECT id, val FROM citus_source) SELECT * FROM cte_2) SELECT * FROM cte_1) +MERGE INTO citus_target t +USING (SELECT const, val, id FROM citus_source WHERE id IN (SELECT id FROM cte_top)) as s +ON (s.id = t.id) +WHEN MATCHED AND t.id <= 75000 THEN + UPDATE SET val = (s.val::int8+1) +WHEN MATCHED THEN + DELETE +WHEN NOT MATCHED THEN + INSERT VALUES (s.id, s.val); + +SELECT compare_data(); + +-- Test aggregate function in source query + +MERGE INTO pg_target t +USING (SELECT count(id+1)::text as value, val as key FROM pg_source group by key) s +ON t.id = s.key +WHEN MATCHED AND t.id <= 75000 THEN + UPDATE SET val = (s.value::int8+1) +WHEN MATCHED THEN + DELETE +WHEN NOT MATCHED THEN + INSERT VALUES(s.key, value::int4+10); + +MERGE INTO citus_target t +USING (SELECT count(id+1)::text as value, val as key FROM citus_source group by key) s +ON t.id = s.key +WHEN MATCHED AND t.id <= 75000 THEN + UPDATE SET val = (s.value::int8+1) +WHEN MATCHED THEN + DELETE +WHEN NOT MATCHED THEN + INSERT VALUES(s.key, value::int4+10); + +SELECT compare_data(); + +DROP SCHEMA merge_repartition2_schema CASCADE; + diff --git a/src/test/regress/sql/multi_partitioning.sql b/src/test/regress/sql/multi_partitioning.sql index 6fbd92638..39f7e8316 100644 --- a/src/test/regress/sql/multi_partitioning.sql +++ b/src/test/regress/sql/multi_partitioning.sql @@ -1839,6 +1839,40 @@ BEGIN; SELECT * FROM time_partitions WHERE parent_table = 'date_partitioned_citus_local_table'::regclass ORDER BY 3; ROLLBACK; set client_min_messages to notice; + +-- 7) test with bigint partition column +CREATE FUNCTION nanos_to_timestamptz(nanos bigint) RETURNS timestamptz LANGUAGE plpgsql AS +$$ +DECLARE + value timestamptz; +BEGIN + select to_timestamp(nanos * 1.0 / 1000000000) into value; + return value; +END; +$$; +CREATE CAST (bigint AS timestamptz) WITH FUNCTION nanos_to_timestamptz(bigint); + +CREATE FUNCTION timestamptz_to_nanos(ts timestamptz) RETURNS bigint LANGUAGE plpgsql AS +$$ +DECLARE + value bigint; +BEGIN + select extract(epoch from ts) * 1000000000 into value; + return value; +END; +$$; +CREATE CAST (timestamptz AS bigint) WITH FUNCTION timestamptz_to_nanos(timestamptz); + +CREATE TABLE bigint_partitioned_table (timestamp bigint, description text) partition by range (timestamp); + +BEGIN; + SELECT create_time_partitions('bigint_partitioned_table', INTERVAL '1 month', '2023-05-01', '2023-01-1'); + SELECT * FROM time_partitions WHERE parent_table = 'bigint_partitioned_table'::regclass ORDER BY 3; +ROLLBACK; + +DROP CAST (bigint AS timestamptz); +DROP CAST (timestamptz AS bigint); + -- c) test drop_old_time_partitions -- 1) test with date partitioned table CREATE TABLE date_partitioned_table_to_exp (event_date date, event int) partition by range (event_date); diff --git a/src/test/regress/sql/multi_utility_statements.sql b/src/test/regress/sql/multi_utility_statements.sql index 36f1bf876..bec722aef 100644 --- a/src/test/regress/sql/multi_utility_statements.sql +++ b/src/test/regress/sql/multi_utility_statements.sql @@ -137,6 +137,30 @@ FETCH FORWARD 3 FROM holdCursor; CLOSE holdCursor; +-- Test DECLARE CURSOR .. WITH HOLD inside transaction block +BEGIN; +DECLARE holdCursor CURSOR WITH HOLD FOR + SELECT * FROM cursor_me WHERE x = 1 ORDER BY y; +FETCH 3 FROM holdCursor; +FETCH BACKWARD 3 FROM holdCursor; +FETCH FORWARD 3 FROM holdCursor; +COMMIT; + +FETCH 3 FROM holdCursor; +CLOSE holdCursor; + +-- Test DECLARE NO SCROLL CURSOR .. WITH HOLD inside transaction block +BEGIN; +DECLARE holdCursor NO SCROLL CURSOR WITH HOLD FOR + SELECT * FROM cursor_me WHERE x = 1 ORDER BY y; +FETCH 3 FROM holdCursor; +FETCH FORWARD 3 FROM holdCursor; +COMMIT; + +FETCH 3 FROM holdCursor; +FETCH BACKWARD 3 FROM holdCursor; +CLOSE holdCursor; + -- Test DECLARE CURSOR .. WITH HOLD with parameter CREATE OR REPLACE FUNCTION declares_cursor(p int) RETURNS void AS $$ diff --git a/src/test/regress/sql/pg15.sql b/src/test/regress/sql/pg15.sql index e29ceff28..a8ac91901 100644 --- a/src/test/regress/sql/pg15.sql +++ b/src/test/regress/sql/pg15.sql @@ -255,7 +255,7 @@ SELECT create_distributed_table('tbl2', 'x'); MERGE INTO tbl1 USING tbl2 ON (true) WHEN MATCHED THEN DELETE; --- also, not inside subqueries & ctes +-- also, inside subqueries & ctes WITH targq AS ( SELECT * FROM tbl2 ) diff --git a/src/test/regress/sql/pgmerge.sql b/src/test/regress/sql/pgmerge.sql index 86dc15040..ab1f4a40d 100644 --- a/src/test/regress/sql/pgmerge.sql +++ b/src/test/regress/sql/pgmerge.sql @@ -19,6 +19,10 @@ SET citus.use_citus_managed_tables to true; SET citus.next_shard_id TO 4001000; +SET client_min_messages = warning; +SELECT 1 FROM master_add_node('localhost', :master_port, groupid => 0); +RESET client_min_messages; + CREATE USER regress_merge_privs; CREATE USER regress_merge_no_privs; DROP TABLE IF EXISTS target; diff --git a/src/test/regress/sql/publication.sql b/src/test/regress/sql/publication.sql index 488c0408c..8bd2ea923 100644 --- a/src/test/regress/sql/publication.sql +++ b/src/test/regress/sql/publication.sql @@ -84,8 +84,13 @@ SELECT DISTINCT c FROM ( SELECT array_agg(c) FROM (SELECT c FROM unnest(activate_node_snapshot()) c WHERE c LIKE '%CREATE PUBLICATION%' AND c LIKE '%pubtables%' ORDER BY 1) s$$) ORDER BY c) s; --- distribute a table, creating a mixed publication +-- distribute a table and create a tenant schema, creating a mixed publication SELECT create_distributed_table('test','x', colocate_with := 'none'); +SET citus.enable_schema_based_sharding TO ON; +CREATE SCHEMA citus_schema_1; +CREATE TABLE citus_schema_1.test (x int primary key, y int, "column-1" int, doc xml); +SET citus.enable_schema_based_sharding TO OFF; +ALTER PUBLICATION pubtables_orig ADD TABLE citus_schema_1.test; -- some generic operations ALTER PUBLICATION pubtables_orig RENAME TO pubtables; @@ -97,7 +102,12 @@ ALTER PUBLICATION pubtables ADD TABLE notexist; -- operations with a distributed table ALTER PUBLICATION pubtables DROP TABLE test; ALTER PUBLICATION pubtables ADD TABLE test; -ALTER PUBLICATION pubtables SET TABLE test, "test-pubs", "publication-1"."test-pubs"; +ALTER PUBLICATION pubtables SET TABLE test, "test-pubs", "publication-1"."test-pubs", citus_schema_1.test; + +-- operations with a tenant schema table +ALTER PUBLICATION pubtables DROP TABLE citus_schema_1.test; +ALTER PUBLICATION pubtables ADD TABLE citus_schema_1.test; +ALTER PUBLICATION pubtables SET TABLE test, "test-pubs", "publication-1"."test-pubs", citus_schema_1.test; -- operations with a local table in a mixed publication ALTER PUBLICATION pubtables DROP TABLE "test-pubs"; @@ -118,7 +128,7 @@ ALTER PUBLICATION pubtables ADD TABLE "test-pubs"; -- create a publication with distributed and local tables DROP PUBLICATION pubtables; -CREATE PUBLICATION pubtables FOR TABLE test, "test-pubs", "publication-1"."test-pubs"; +CREATE PUBLICATION pubtables FOR TABLE test, "test-pubs", "publication-1"."test-pubs", citus_schema_1.test; -- change distributed tables SELECT alter_distributed_table('test', shard_count := 5, cascade_to_colocated := true); @@ -184,11 +194,12 @@ SELECT substring(:'server_version', '\d+')::int >= 15 AS server_version_ge_15 SET client_min_messages TO ERROR; DROP SCHEMA publication CASCADE; DROP SCHEMA "publication-1" CASCADE; +DROP SCHEMA citus_schema_1 CASCADE; \q \endif -- recreate a mixed publication -CREATE PUBLICATION pubtables FOR TABLE test, "publication-1"."test-pubs"; +CREATE PUBLICATION pubtables FOR TABLE test, "publication-1"."test-pubs", citus_schema_1.test; -- operations on an existing distributed table ALTER PUBLICATION pubtables DROP TABLE test; @@ -197,6 +208,19 @@ ALTER PUBLICATION pubtables SET TABLE test WHERE (doc IS DOCUMENT); ALTER PUBLICATION pubtables SET TABLE test WHERE (xmlexists('//foo[text() = ''bar'']' PASSING BY VALUE doc)); ALTER PUBLICATION pubtables SET TABLE test WHERE (CASE x WHEN 5 THEN true ELSE false END); +SELECT DISTINCT c FROM ( + SELECT unnest(result::text[]) c + FROM run_command_on_workers($$ + SELECT array_agg(c) FROM (SELECT c FROM unnest(activate_node_snapshot()) c WHERE c LIKE '%CREATE PUBLICATION%' AND c LIKE '%pubtables%' ORDER BY 1) s$$) + ORDER BY c) s; + +-- operations on an existing tenant schema table +ALTER PUBLICATION pubtables ADD TABLE citus_schema_1.test (y); +ALTER PUBLICATION pubtables DROP TABLE citus_schema_1.test; +ALTER PUBLICATION pubtables SET TABLE citus_schema_1.test WHERE (doc IS DOCUMENT); +ALTER PUBLICATION pubtables SET TABLE citus_schema_1.test WHERE (xmlexists('//foo[text() = ''bar'']' PASSING BY VALUE doc)); +ALTER PUBLICATION pubtables SET TABLE citus_schema_1.test WHERE (CASE x WHEN 5 THEN true ELSE false END); + SELECT DISTINCT c FROM ( SELECT unnest(result::text[]) c FROM run_command_on_workers($$ @@ -260,3 +284,4 @@ DROP PUBLICATION pubpartitioned; SET client_min_messages TO ERROR; DROP SCHEMA publication CASCADE; DROP SCHEMA "publication-1" CASCADE; +DROP SCHEMA citus_schema_1 CASCADE; diff --git a/src/test/regress/sql/schema_based_sharding.sql b/src/test/regress/sql/schema_based_sharding.sql index 88ca77a40..7b16ba904 100644 --- a/src/test/regress/sql/schema_based_sharding.sql +++ b/src/test/regress/sql/schema_based_sharding.sql @@ -59,18 +59,100 @@ SELECT citus_add_local_table_to_metadata('tenant_2.test_table'); SELECT update_distributed_table_colocation('tenant_2.test_table', colocate_with => 'none'); -- verify we also don't allow colocate_with a tenant table SELECT update_distributed_table_colocation('regular_schema.test_table', colocate_with => 'tenant_2.test_table'); --- verify we don't allow undistribute_table for tenant tables -SELECT undistribute_table('tenant_2.test_table'); + +-- verify we do not allow undistribute_table for tenant tables +CREATE TABLE tenant_2.undist_table(id int); +SELECT undistribute_table('tenant_2.undist_table'); + -- verify we don't allow alter_distributed_table for tenant tables SELECT alter_distributed_table('tenant_2.test_table', colocate_with => 'none'); -- verify we also don't allow colocate_with a tenant table SELECT alter_distributed_table('regular_schema.test_table', colocate_with => 'tenant_2.test_table'); --- verify we don't allow ALTER TABLE SET SCHEMA for tenant tables -ALTER TABLE tenant_2.test_table SET SCHEMA regular_schema; --- verify we don't allow ALTER TABLE SET SCHEMA for tenant schemas -ALTER TABLE regular_schema.test_table SET SCHEMA tenant_2; --- the same, from tenant schema to tenant schema -ALTER TABLE tenant_2.test_table SET SCHEMA tenant_3; + +-- verify we can set tenant table's schema to regular schema +CREATE TABLE tenant_2.test_table2(id int); +ALTER TABLE tenant_2.test_table2 SET SCHEMA regular_schema; +-- verify that regular_schema.test_table2 does not exist in pg_dist_partition +SELECT COUNT(*)=0 FROM pg_dist_partition +WHERE logicalrelid = 'regular_schema.test_table2'::regclass AND + partmethod = 'n' AND repmodel = 's' AND colocationid > 0; +-- verify that tenant_2.test_table2 does not exist +SELECT * FROM tenant_2.test_table2; + +-- verify we can set regular table's schema to distributed schema +CREATE TABLE regular_schema.test_table3(id int); +ALTER TABLE regular_schema.test_table3 SET SCHEMA tenant_2; +-- verify that tenant_2.test_table3 is recorded in pg_dist_partition as a single-shard table. +SELECT COUNT(*)=1 FROM pg_dist_partition +WHERE logicalrelid = 'tenant_2.test_table3'::regclass AND + partmethod = 'n' AND repmodel = 's' AND colocationid > 0; +-- verify that regular_schema.test_table3 does not exist +SELECT * FROM regular_schema.test_table3; + +-- verify we can set tenant table's schema to another distributed schema +CREATE TABLE tenant_2.test_table4(id int); +ALTER TABLE tenant_2.test_table4 SET SCHEMA tenant_3; +-- verify that tenant_3.test_table4 is recorded in pg_dist_partition as a single-shard table. +SELECT COUNT(*)=1 FROM pg_dist_partition +WHERE logicalrelid = 'tenant_3.test_table4'::regclass AND + partmethod = 'n' AND repmodel = 's' AND colocationid > 0; +-- verify that tenant_2.test_table4 does not exist +SELECT * FROM tenant_2.test_table4; + +-- verify that we can put a local table in regular schema into distributed schema +CREATE TABLE regular_schema.pg_local_tbl(id int); +ALTER TABLE regular_schema.pg_local_tbl SET SCHEMA tenant_2; + +-- verify that we can put a Citus local table in regular schema into distributed schema +CREATE TABLE regular_schema.citus_local_tbl(id int); +SELECT citus_add_local_table_to_metadata('regular_schema.citus_local_tbl'); +ALTER TABLE regular_schema.citus_local_tbl SET SCHEMA tenant_2; + +-- verify that we do not allow a hash distributed table in regular schema into distributed schema +CREATE TABLE regular_schema.hash_dist_tbl(id int); +SELECT create_distributed_table('regular_schema.hash_dist_tbl', 'id'); +ALTER TABLE regular_schema.hash_dist_tbl SET SCHEMA tenant_2; + +-- verify that we do not allow a reference table in regular schema into distributed schema +CREATE TABLE regular_schema.ref_tbl(id int PRIMARY KEY); +SELECT create_reference_table('regular_schema.ref_tbl'); +ALTER TABLE regular_schema.ref_tbl SET SCHEMA tenant_2; + +-- verify that we can put a table in tenant schema into regular schema +CREATE TABLE tenant_2.tenant_tbl(id int); +ALTER TABLE tenant_2.tenant_tbl SET SCHEMA regular_schema; + +-- verify that we can put a table in tenant schema into another tenant schema +CREATE TABLE tenant_2.tenant_tbl2(id int); +ALTER TABLE tenant_2.tenant_tbl2 SET SCHEMA tenant_3; + +-- verify that we do not allow a local table in regular schema into distributed schema if it has foreign key to a non-reference table in another schema +CREATE TABLE regular_schema.pg_local_tbl1(id int PRIMARY KEY); +CREATE TABLE regular_schema.pg_local_tbl2(id int REFERENCES regular_schema.pg_local_tbl1(id)); +ALTER TABLE regular_schema.pg_local_tbl2 SET SCHEMA tenant_2; + +-- verify that we allow a local table in regular schema into distributed schema if it has foreign key to a reference table in another schema +CREATE TABLE regular_schema.pg_local_tbl3(id int REFERENCES regular_schema.ref_tbl(id)); +ALTER TABLE regular_schema.pg_local_tbl3 SET SCHEMA tenant_2; + +-- verify that we do not allow a table in tenant schema into regular schema if it has foreign key to/from another table in the same schema +CREATE TABLE tenant_2.tenant_tbl1(id int PRIMARY KEY); +CREATE TABLE tenant_2.tenant_tbl2(id int REFERENCES tenant_2.tenant_tbl1(id)); +ALTER TABLE tenant_2.tenant_tbl1 SET SCHEMA regular_schema; +ALTER TABLE tenant_2.tenant_tbl2 SET SCHEMA regular_schema; + +-- verify that we do not allow a table in distributed schema into another distributed schema if it has foreign key to/from another table in the same schema +CREATE TABLE tenant_2.tenant_tbl3(id int PRIMARY KEY); +CREATE TABLE tenant_2.tenant_tbl4(id int REFERENCES tenant_2.tenant_tbl3(id)); +ALTER TABLE tenant_2.tenant_tbl3 SET SCHEMA tenant_3; +ALTER TABLE tenant_2.tenant_tbl4 SET SCHEMA tenant_3; + +-- alter set non-existent schema +ALTER TABLE tenant_2.test_table SET SCHEMA ghost_schema; +ALTER TABLE IF EXISTS tenant_2.test_table SET SCHEMA ghost_schema; +-- alter set non-existent table +ALTER TABLE tenant_2.ghost_table SET SCHEMA ghost_schema; +ALTER TABLE IF EXISTS tenant_2.ghost_table SET SCHEMA ghost_schema; -- (on coordinator) verify that colocation id is set for empty tenants too SELECT colocationid > 0 FROM pg_dist_schema