diff --git a/src/backend/distributed/commands/create_distributed_table.c b/src/backend/distributed/commands/create_distributed_table.c index 220a95894..d1fb9e832 100644 --- a/src/backend/distributed/commands/create_distributed_table.c +++ b/src/backend/distributed/commands/create_distributed_table.c @@ -21,7 +21,9 @@ #include "catalog/dependency.h" #include "catalog/index.h" #include "catalog/pg_am.h" +#if (PG_VERSION_NUM < 110000) #include "catalog/pg_constraint_fn.h" +#endif #include "catalog/pg_enum.h" #include "catalog/pg_extension.h" #include "catalog/pg_opclass.h" @@ -1103,8 +1105,9 @@ CreateTruncateTrigger(Oid relationId) trigger->whenClause = NULL; trigger->isconstraint = false; - CreateTrigger(trigger, NULL, relationId, InvalidOid, InvalidOid, InvalidOid, - internal); + CreateTriggerInternal(trigger, NULL, relationId, InvalidOid, InvalidOid, InvalidOid, + InvalidOid, InvalidOid, NULL, + internal, false); } diff --git a/src/backend/distributed/commands/multi_copy.c b/src/backend/distributed/commands/multi_copy.c index a74acefda..8f06d4fd7 100644 --- a/src/backend/distributed/commands/multi_copy.c +++ b/src/backend/distributed/commands/multi_copy.c @@ -799,12 +799,13 @@ OpenCopyConnections(CopyStmt *copyStatement, ShardConnections *shardConnections, List *connectionList = NULL; int64 shardId = shardConnections->shardId; bool raiseInterrupts = true; + MemoryContext localContext = + AllocSetContextCreateExtended(CurrentMemoryContext, + "OpenCopyConnections", + ALLOCSET_DEFAULT_MINSIZE, + ALLOCSET_DEFAULT_INITSIZE, + ALLOCSET_DEFAULT_MAXSIZE); - MemoryContext localContext = AllocSetContextCreate(CurrentMemoryContext, - "OpenCopyConnections", - ALLOCSET_DEFAULT_MINSIZE, - ALLOCSET_DEFAULT_INITSIZE, - ALLOCSET_DEFAULT_MAXSIZE); /* release finalized placement list at the end of this function */ MemoryContext oldContext = MemoryContextSwitchTo(localContext); @@ -1280,8 +1281,24 @@ ConversionPathForTypes(Oid inputType, Oid destType, CopyCoercionData *result) case COERCION_PATH_ARRAYCOERCE: { - ereport(ERROR, (errmsg("can not run query which uses an implicit coercion" - " between array types"))); + Oid inputBaseType = get_base_element_type(inputType); + Oid destBaseType = get_base_element_type(destType); + CoercionPathType baseCoercionType = COERCION_PATH_NONE; + + if (inputBaseType != InvalidOid && destBaseType != InvalidOid) + { + baseCoercionType = find_coercion_pathway(inputBaseType, destBaseType, + COERCION_EXPLICIT, + &coercionFuncId); + } + + if (baseCoercionType != COERCION_PATH_COERCEVIAIO) + { + ereport(ERROR, (errmsg("can not run query which uses an implicit coercion" + " between array types"))); + } + + /* fallthrough */ } case COERCION_PATH_COERCEVIAIO: diff --git a/src/backend/distributed/connection/connection_management.c b/src/backend/distributed/connection/connection_management.c index d345ceea4..0c55564ea 100644 --- a/src/backend/distributed/connection/connection_management.c +++ b/src/backend/distributed/connection/connection_management.c @@ -25,6 +25,7 @@ #include "distributed/metadata_cache.h" #include "distributed/hash_helpers.h" #include "distributed/placement_connection.h" +#include "distributed/version_compat.h" #include "mb/pg_wchar.h" #include "utils/hsearch.h" #include "utils/memutils.h" @@ -58,10 +59,11 @@ InitializeConnectionManagement(void) * management. Doing so, instead of allocating in TopMemoryContext, makes * it easier to associate used memory. */ - ConnectionContext = AllocSetContextCreate(TopMemoryContext, "Connection Context", - ALLOCSET_DEFAULT_MINSIZE, - ALLOCSET_DEFAULT_INITSIZE, - ALLOCSET_DEFAULT_MAXSIZE); + ConnectionContext = AllocSetContextCreateExtended(TopMemoryContext, + "Connection Context", + ALLOCSET_DEFAULT_MINSIZE, + ALLOCSET_DEFAULT_INITSIZE, + ALLOCSET_DEFAULT_MAXSIZE); /* create (host,port,user,database) -> [connection] hash */ memset(&info, 0, sizeof(info)); diff --git a/src/backend/distributed/executor/multi_router_executor.c b/src/backend/distributed/executor/multi_router_executor.c index 3c14822f1..5a55ffc6d 100644 --- a/src/backend/distributed/executor/multi_router_executor.c +++ b/src/backend/distributed/executor/multi_router_executor.c @@ -46,6 +46,7 @@ #include "distributed/remote_commands.h" #include "distributed/remote_transaction.h" #include "distributed/resource_lock.h" +#include "distributed/version_compat.h" #include "executor/execdesc.h" #include "executor/executor.h" #include "executor/instrument.h" @@ -1260,6 +1261,9 @@ SendQueryInSingleRowMode(MultiConnection *connection, char *query, Oid *parameterTypes = NULL; const char **parameterValues = NULL; + /* force evaluation of bound params */ + paramListInfo = copyParamList(paramListInfo); + ExtractParametersFromParamListInfo(paramListInfo, ¶meterTypes, ¶meterValues); @@ -1381,11 +1385,12 @@ StoreQueryResult(CitusScanState *scanState, MultiConnection *connection, bool randomAccess = true; bool interTransactions = false; bool commandFailed = false; - MemoryContext ioContext = AllocSetContextCreate(CurrentMemoryContext, - "StoreQueryResult", - ALLOCSET_DEFAULT_MINSIZE, - ALLOCSET_DEFAULT_INITSIZE, - ALLOCSET_DEFAULT_MAXSIZE); + MemoryContext ioContext = AllocSetContextCreateExtended(CurrentMemoryContext, + "StoreQueryResult", + ALLOCSET_DEFAULT_MINSIZE, + ALLOCSET_DEFAULT_INITSIZE, + ALLOCSET_DEFAULT_MAXSIZE); + *rows = 0; if (scanState->tuplestorestate == NULL) diff --git a/src/backend/distributed/executor/multi_utility.c b/src/backend/distributed/executor/multi_utility.c index a9a50f6cd..607698741 100644 --- a/src/backend/distributed/executor/multi_utility.c +++ b/src/backend/distributed/executor/multi_utility.c @@ -128,9 +128,10 @@ static Node * WorkerProcessAlterTableStmt(AlterTableStmt *alterTableStatement, static List * PlanAlterObjectSchemaStmt(AlterObjectSchemaStmt *alterObjectSchemaStmt, const char *alterObjectSchemaCommand); static void ProcessVacuumStmt(VacuumStmt *vacuumStmt, const char *vacuumCommand); -static bool IsSupportedDistributedVacuumStmt(Oid relationId, VacuumStmt *vacuumStmt); -static List * VacuumTaskList(Oid relationId, VacuumStmt *vacuumStmt); -static StringInfo DeparseVacuumStmtPrefix(VacuumStmt *vacuumStmt); +static bool IsSupportedDistributedVacuumStmt(VacuumStmt *vacuumStmt, + List *vacuumRelationIdList); +static List * VacuumTaskList(Oid relationId, int vacuumOptions, List *vacuumColumnList); +static StringInfo DeparseVacuumStmtPrefix(int vacuumFlags); static char * DeparseVacuumColumnNames(List *columnNameList); @@ -1162,8 +1163,7 @@ PlanDropIndexStmt(DropStmt *dropIndexStatement, const char *dropIndexCommand) Oid relationId = InvalidOid; bool isDistributedRelation = false; struct DropRelationCallbackState state; - bool missingOK = true; - bool noWait = false; + uint32 rvrFlags = RVR_MISSING_OK; LOCKMODE lockmode = AccessExclusiveLock; List *objectNameList = (List *) lfirst(dropObjectCell); @@ -1188,8 +1188,9 @@ PlanDropIndexStmt(DropStmt *dropIndexStatement, const char *dropIndexCommand) state.relkind = RELKIND_INDEX; state.heapOid = InvalidOid; state.concurrent = dropIndexStatement->concurrent; - indexId = RangeVarGetRelidExtended(rangeVar, lockmode, missingOK, - noWait, RangeVarCallbackForDropIndex, + + indexId = RangeVarGetRelidInternal(rangeVar, lockmode, rvrFlags, + RangeVarCallbackForDropIndex, (void *) &state); /* @@ -1568,17 +1569,15 @@ PlanAlterObjectSchemaStmt(AlterObjectSchemaStmt *alterObjectSchemaStmt, const char *alterObjectSchemaCommand) { Oid relationId = InvalidOid; - bool noWait = false; if (alterObjectSchemaStmt->relation == NULL) { return NIL; } - relationId = RangeVarGetRelidExtended(alterObjectSchemaStmt->relation, - AccessExclusiveLock, - alterObjectSchemaStmt->missing_ok, - noWait, NULL, NULL); + relationId = RangeVarGetRelid(alterObjectSchemaStmt->relation, + AccessExclusiveLock, + alterObjectSchemaStmt->missing_ok); /* first check whether a distributed relation is affected */ if (!OidIsValid(relationId) || !IsDistributedTable(relationId)) @@ -1609,73 +1608,107 @@ PlanAlterObjectSchemaStmt(AlterObjectSchemaStmt *alterObjectSchemaStmt, static void ProcessVacuumStmt(VacuumStmt *vacuumStmt, const char *vacuumCommand) { - Oid relationId = InvalidOid; - List *taskList = NIL; + int relationIndex = 0; bool supportedVacuumStmt = false; + List *vacuumRelationList = ExtractVacuumTargetRels(vacuumStmt); + ListCell *vacuumRelationCell = NULL; + List *relationIdList = NIL; + ListCell *relationIdCell = NULL; + LOCKMODE lockMode = (vacuumStmt->options & VACOPT_FULL) ? AccessExclusiveLock : + ShareUpdateExclusiveLock; + int executedVacuumCount = 0; - if (vacuumStmt->relation != NULL) + foreach(vacuumRelationCell, vacuumRelationList) { - LOCKMODE lockMode = (vacuumStmt->options & VACOPT_FULL) ? - AccessExclusiveLock : ShareUpdateExclusiveLock; - - relationId = RangeVarGetRelid(vacuumStmt->relation, lockMode, false); - - if (relationId == InvalidOid) - { - return; - } + RangeVar *vacuumRelation = (RangeVar *) lfirst(vacuumRelationCell); + Oid relationId = RangeVarGetRelid(vacuumRelation, lockMode, false); + relationIdList = lappend_oid(relationIdList, relationId); } - supportedVacuumStmt = IsSupportedDistributedVacuumStmt(relationId, vacuumStmt); + supportedVacuumStmt = IsSupportedDistributedVacuumStmt(vacuumStmt, relationIdList); if (!supportedVacuumStmt) { return; } - taskList = VacuumTaskList(relationId, vacuumStmt); - - /* - * VACUUM commands cannot run inside a transaction block, so we use - * the "bare" commit protocol without BEGIN/COMMIT. However, ANALYZE - * commands can run inside a transaction block. - */ - if ((vacuumStmt->options & VACOPT_VACUUM) != 0) + /* execute vacuum on distributed tables */ + foreach(relationIdCell, relationIdList) { - /* save old commit protocol to restore at xact end */ - Assert(SavedMultiShardCommitProtocol == COMMIT_PROTOCOL_BARE); - SavedMultiShardCommitProtocol = MultiShardCommitProtocol; - MultiShardCommitProtocol = COMMIT_PROTOCOL_BARE; - } + Oid relationId = lfirst_oid(relationIdCell); + if (IsDistributedTable(relationId)) + { + List *vacuumColumnList = NIL; + List *taskList = NIL; - ExecuteModifyTasksWithoutResults(taskList); + /* + * VACUUM commands cannot run inside a transaction block, so we use + * the "bare" commit protocol without BEGIN/COMMIT. However, ANALYZE + * commands can run inside a transaction block. Notice that we do this + * once even if there are multiple distributed tables to be vacuumed. + */ + if (executedVacuumCount == 0 && (vacuumStmt->options & VACOPT_VACUUM) != 0) + { + /* save old commit protocol to restore at xact end */ + Assert(SavedMultiShardCommitProtocol == COMMIT_PROTOCOL_BARE); + SavedMultiShardCommitProtocol = MultiShardCommitProtocol; + MultiShardCommitProtocol = COMMIT_PROTOCOL_BARE; + } + + vacuumColumnList = VacuumColumnList(vacuumStmt, relationIndex); + taskList = VacuumTaskList(relationId, vacuumStmt->options, vacuumColumnList); + + ExecuteModifyTasksWithoutResults(taskList); + + executedVacuumCount++; + } + relationIndex++; + } } /* * IsSupportedDistributedVacuumStmt returns whether distributed execution of a - * given VacuumStmt is supported. The provided relationId (if valid) represents - * the table targeted by the provided statement. + * given VacuumStmt is supported. The provided relationId list represents + * the list of tables targeted by the provided statement. * * Returns true if the statement requires distributed execution and returns * false otherwise; however, this function will raise errors if the provided * statement needs distributed execution but contains unsupported options. */ static bool -IsSupportedDistributedVacuumStmt(Oid relationId, VacuumStmt *vacuumStmt) +IsSupportedDistributedVacuumStmt(VacuumStmt *vacuumStmt, List *vacuumRelationIdList) { const char *stmtName = (vacuumStmt->options & VACOPT_VACUUM) ? "VACUUM" : "ANALYZE"; bool distributeStmt = false; + ListCell *relationIdCell = NULL; + int distributedRelationCount = 0; + int vacuumedRelationCount = 0; - if (vacuumStmt->relation == NULL) + /* + * No table in the vacuum statement means vacuuming all relations + * which is not supported by citus. + */ + vacuumedRelationCount = list_length(vacuumRelationIdList); + if (vacuumedRelationCount == 0) { /* WARN for unqualified VACUUM commands */ ereport(WARNING, (errmsg("not propagating %s command to worker nodes", stmtName), errhint("Provide a specific table in order to %s " "distributed tables.", stmtName))); } - else if (!OidIsValid(relationId) || !IsDistributedTable(relationId)) + + foreach(relationIdCell, vacuumRelationIdList) { - /* Nothing to do here; relation no longer exists or is not distributed */ + Oid relationId = lfirst_oid(relationIdCell); + if (OidIsValid(relationId) && IsDistributedTable(relationId)) + { + distributedRelationCount++; + } + } + + if (distributedRelationCount == 0) + { + /* nothing to do here */ } else if (!EnableDDLPropagation) { @@ -1705,20 +1738,22 @@ IsSupportedDistributedVacuumStmt(Oid relationId, VacuumStmt *vacuumStmt) * a VacuumStmt which targets a distributed relation. */ static List * -VacuumTaskList(Oid relationId, VacuumStmt *vacuumStmt) +VacuumTaskList(Oid relationId, int vacuumOptions, List *vacuumColumnList) { List *taskList = NIL; List *shardIntervalList = NIL; ListCell *shardIntervalCell = NULL; uint64 jobId = INVALID_JOB_ID; int taskId = 1; - StringInfo vacuumString = DeparseVacuumStmtPrefix(vacuumStmt); - const char *columnNames = DeparseVacuumColumnNames(vacuumStmt->va_cols); + StringInfo vacuumString = DeparseVacuumStmtPrefix(vacuumOptions); + const char *columnNames = NULL; const int vacuumPrefixLen = vacuumString->len; Oid schemaId = get_rel_namespace(relationId); char *schemaName = get_namespace_name(schemaId); char *tableName = get_rel_name(relationId); + columnNames = DeparseVacuumColumnNames(vacuumColumnList); + /* * We obtain ShareUpdateExclusiveLock here to not conflict with INSERT's * RowExclusiveLock. However if VACUUM FULL is used, we already obtain @@ -1770,10 +1805,9 @@ VacuumTaskList(Oid relationId, VacuumStmt *vacuumStmt) * statements. */ static StringInfo -DeparseVacuumStmtPrefix(VacuumStmt *vacuumStmt) +DeparseVacuumStmtPrefix(int vacuumFlags) { StringInfo vacuumPrefix = makeStringInfo(); - int vacuumFlags = vacuumStmt->options; const int unsupportedFlags PG_USED_FOR_ASSERTS_ONLY = ~( VACOPT_ANALYZE | VACOPT_DISABLE_PAGE_SKIPPING | @@ -2433,7 +2467,7 @@ ErrorIfUnsupportedConstraint(Relation relation, char distributionMethod, } attributeCount = indexInfo->ii_NumIndexAttrs; - attributeNumberArray = indexInfo->ii_KeyAttrNumbers; + attributeNumberArray = IndexInfoAttributeNumberArray(indexInfo); for (attributeIndex = 0; attributeIndex < attributeCount; attributeIndex++) { @@ -3390,8 +3424,9 @@ RangeVarCallbackForDropIndex(const RangeVar *rel, Oid relOid, Oid oldRelOid, voi /* Allow DROP to either table owner or schema owner */ if (!pg_class_ownercheck(relOid, GetUserId()) && !pg_namespace_ownercheck(classform->relnamespace, GetUserId())) - aclcheck_error(ACLCHECK_NOT_OWNER, ACL_KIND_CLASS, - rel->relname); + { + aclcheck_error(ACLCHECK_NOT_OWNER, ACLCHECK_OBJECT_INDEX, rel->relname); + } if (!allowSystemTableMods && IsSystemClass(relOid, classform)) ereport(ERROR, @@ -3657,7 +3692,7 @@ PlanGrantStmt(GrantStmt *grantStmt) * grants aren't interesting anyway. */ if (grantStmt->targtype != ACL_TARGET_OBJECT || - grantStmt->objtype != ACL_OBJECT_RELATION) + grantStmt->objtype != RELATION_OBJECT_TYPE) { return NIL; } diff --git a/src/backend/distributed/master/master_metadata_utility.c b/src/backend/distributed/master/master_metadata_utility.c index 385de9b31..4a166360c 100644 --- a/src/backend/distributed/master/master_metadata_utility.c +++ b/src/backend/distributed/master/master_metadata_utility.c @@ -41,6 +41,7 @@ #include "distributed/remote_commands.h" #include "distributed/worker_manager.h" #include "distributed/worker_protocol.h" +#include "distributed/version_compat.h" #include "nodes/makefuncs.h" #include "parser/scansup.h" #include "storage/lmgr.h" @@ -1278,8 +1279,7 @@ EnsureTablePermissions(Oid relationId, AclMode mode) if (aclresult != ACLCHECK_OK) { - aclcheck_error(aclresult, ACL_KIND_CLASS, - get_rel_name(relationId)); + aclcheck_error(aclresult, ACLCHECK_OBJECT_TABLE, get_rel_name(relationId)); } } @@ -1293,7 +1293,7 @@ EnsureTableOwner(Oid relationId) { if (!pg_class_ownercheck(relationId, GetUserId())) { - aclcheck_error(ACLCHECK_NOT_OWNER, ACL_KIND_CLASS, + aclcheck_error(ACLCHECK_NOT_OWNER, ACLCHECK_OBJECT_TABLE, get_rel_name(relationId)); } } diff --git a/src/backend/distributed/master/master_node_protocol.c b/src/backend/distributed/master/master_node_protocol.c index 6976317e9..c46f8fcfb 100644 --- a/src/backend/distributed/master/master_node_protocol.c +++ b/src/backend/distributed/master/master_node_protocol.c @@ -33,7 +33,9 @@ #include "catalog/namespace.h" #include "catalog/pg_class.h" #include "catalog/pg_constraint.h" +#if (PG_VERSION_NUM < 110000) #include "catalog/pg_constraint_fn.h" +#endif #include "catalog/pg_index.h" #include "catalog/pg_type.h" #include "catalog/pg_namespace.h" diff --git a/src/backend/distributed/master/master_stage_protocol.c b/src/backend/distributed/master/master_stage_protocol.c index d39b46446..dfde2a72d 100644 --- a/src/backend/distributed/master/master_stage_protocol.c +++ b/src/backend/distributed/master/master_stage_protocol.c @@ -44,6 +44,7 @@ #include "distributed/transaction_management.h" #include "distributed/worker_manager.h" #include "distributed/worker_protocol.h" +#include "distributed/version_compat.h" #include "storage/lmgr.h" #include "utils/builtins.h" #include "utils/fmgroids.h" @@ -860,8 +861,8 @@ WorkerShardStats(ShardPlacement *placement, Oid relationId, char *shardName, /* fill in the partition column name and shard name in the query. */ partitionColumn = PartitionColumn(relationId, unusedTableId); - partitionColumnName = get_attname(relationId, partitionColumn->varattno); - + partitionColumnName = get_attname_internal(relationId, partitionColumn->varattno, + false); appendStringInfo(partitionValueQuery, SHARD_RANGE_QUERY, partitionColumnName, partitionColumnName, shardName); diff --git a/src/backend/distributed/metadata/metadata_sync.c b/src/backend/distributed/metadata/metadata_sync.c index 915d49b81..97d95c782 100644 --- a/src/backend/distributed/metadata/metadata_sync.c +++ b/src/backend/distributed/metadata/metadata_sync.c @@ -92,10 +92,9 @@ start_metadata_sync_to_node(PG_FUNCTION_ARGS) EnsureModificationsCanRun(); CheckCitusVersion(ERROR); - PreventTransactionChain(true, "start_metadata_sync_to_node"); + PreventInTransactionBlock(true, "start_metadata_sync_to_node"); workerNode = FindWorkerNode(nodeNameString, nodePort); - if (workerNode == NULL) { ereport(ERROR, (errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE), diff --git a/src/backend/distributed/planner/distributed_planner.c b/src/backend/distributed/planner/distributed_planner.c index 17b25bf8a..ee51376bb 100644 --- a/src/backend/distributed/planner/distributed_planner.c +++ b/src/backend/distributed/planner/distributed_planner.c @@ -639,6 +639,9 @@ CreateDistributedSelectPlan(uint64 planId, Query *originalQuery, Query *query, return NULL; } + /* force evaluation of bound params */ + boundParams = copyParamList(boundParams); + /* * If there are parameters that do have a value in boundParams, replace * them in the original query. This allows us to more easily cut the @@ -1464,14 +1467,22 @@ HasUnresolvedExternParamsWalker(Node *expression, ParamListInfo boundParams) if (boundParams && paramId > 0 && paramId <= boundParams->numParams) { ParamExternData *externParam = &boundParams->params[paramId - 1]; + Oid paramType = externParam->ptype; /* give hook a chance in case parameter is dynamic */ - if (!OidIsValid(externParam->ptype) && boundParams->paramFetch != NULL) + if (!OidIsValid(paramType) && boundParams->paramFetch != NULL) { +#if (PG_VERSION_NUM >= 110000) + ParamExternData externParamPlaceholder; + externParam = (*boundParams->paramFetch)(boundParams, paramId, false, + &externParamPlaceholder); +#else (*boundParams->paramFetch)(boundParams, paramId); +#endif + paramType = externParam->ptype; } - if (OidIsValid(externParam->ptype)) + if (OidIsValid(paramType)) { return false; } diff --git a/src/backend/distributed/planner/insert_select_planner.c b/src/backend/distributed/planner/insert_select_planner.c index 9e1b6a5f9..9273262b9 100644 --- a/src/backend/distributed/planner/insert_select_planner.c +++ b/src/backend/distributed/planner/insert_select_planner.c @@ -1142,6 +1142,7 @@ CreateCoordinatorInsertSelectPlan(Query *parse) /* copy CTEs from the INSERT ... SELECT statement into outer SELECT */ selectQuery->cteList = copyObject(insertSelectQuery->cteList); + selectQuery->hasModifyingCTE = insertSelectQuery->hasModifyingCTE; } else if (selectQuery->setOperations != NULL) { diff --git a/src/backend/distributed/planner/multi_explain.c b/src/backend/distributed/planner/multi_explain.c index 052cc63d5..3ba1b569e 100644 --- a/src/backend/distributed/planner/multi_explain.c +++ b/src/backend/distributed/planner/multi_explain.c @@ -36,6 +36,7 @@ #include "distributed/recursive_planning.h" #include "distributed/placement_connection.h" #include "distributed/worker_protocol.h" +#include "distributed/version_compat.h" #include "lib/stringinfo.h" #include "nodes/plannodes.h" #include "nodes/primnodes.h" @@ -241,7 +242,7 @@ ExplainJob(Job *job, ExplainState *es) ExplainOpenGroup("Job", "Job", true, es); - ExplainPropertyInteger("Task Count", taskCount, es); + ExplainPropertyIntegerInternal("Task Count", NULL, taskCount, es); if (dependedJobCount > 0) { @@ -317,9 +318,8 @@ ExplainMapMergeJob(MapMergeJob *mapMergeJob, ExplainState *es) } ExplainOpenGroup("MapMergeJob", NULL, true, es); - - ExplainPropertyInteger("Map Task Count", mapTaskCount, es); - ExplainPropertyInteger("Merge Task Count", mergeTaskCount, es); + ExplainPropertyIntegerInternal("Map Task Count", NULL, mapTaskCount, es); + ExplainPropertyIntegerInternal("Merge Task Count", NULL, mergeTaskCount, es); if (dependedJobCount > 0) { @@ -662,7 +662,10 @@ ExplainOneQuery(Query *query, IntoClause *into, ExplainState *es, { /* if an advisor plugin is present, let it manage things */ if (ExplainOneQuery_hook) -#if (PG_VERSION_NUM >= 100000) +#if (PG_VERSION_NUM >= 110000) + (*ExplainOneQuery_hook) (query, cursorOptions, into, es, + queryString, params, queryEnv); +#elif (PG_VERSION_NUM >= 100000) (*ExplainOneQuery_hook) (query, cursorOptions, into, es, queryString, params); #else diff --git a/src/backend/distributed/planner/multi_logical_planner.c b/src/backend/distributed/planner/multi_logical_planner.c index f848df702..7bfa8aa97 100644 --- a/src/backend/distributed/planner/multi_logical_planner.c +++ b/src/backend/distributed/planner/multi_logical_planner.c @@ -29,6 +29,7 @@ #include "distributed/relation_restriction_equivalence.h" #include "distributed/multi_router_planner.h" #include "distributed/worker_protocol.h" +#include "distributed/version_compat.h" #include "nodes/makefuncs.h" #include "nodes/nodeFuncs.h" #include "nodes/relation.h" @@ -2706,7 +2707,8 @@ ExtractFromExpressionWalker(Node *node, QualifierWalkerContext *walkerContext) { /* this part of code only run for subqueries */ Node *joinClause = eval_const_expressions(NULL, joinQualifiersNode); - joinClause = (Node *) canonicalize_qual((Expr *) joinClause); + joinClause = (Node *) canonicalize_qual_compat((Expr *) joinClause, + false); joinQualifierList = make_ands_implicit((Expr *) joinClause); } } @@ -2739,7 +2741,8 @@ ExtractFromExpressionWalker(Node *node, QualifierWalkerContext *walkerContext) { /* this part of code only run for subqueries */ Node *fromClause = eval_const_expressions(NULL, fromQualifiersNode); - fromClause = (Node *) canonicalize_qual((Expr *) fromClause); + fromClause = (Node *) canonicalize_qual_compat((Expr *) fromClause, + false); fromQualifierList = make_ands_implicit((Expr *) fromClause); } diff --git a/src/backend/distributed/planner/multi_physical_planner.c b/src/backend/distributed/planner/multi_physical_planner.c index 708d371c6..f44376312 100644 --- a/src/backend/distributed/planner/multi_physical_planner.c +++ b/src/backend/distributed/planner/multi_physical_planner.c @@ -47,6 +47,7 @@ #include "distributed/task_tracker.h" #include "distributed/worker_manager.h" #include "distributed/worker_protocol.h" +#include "distributed/version_compat.h" #include "nodes/makefuncs.h" #include "nodes/nodeFuncs.h" #include "optimizer/clauses.h" @@ -4234,7 +4235,7 @@ ColumnName(Var *column, List *rangeTableList) else if (rangeTableKind == CITUS_RTE_RELATION) { Oid relationId = rangeTableEntry->relid; - columnName = get_attname(relationId, columnNumber); + columnName = get_attname_internal(relationId, columnNumber, false); } Assert(columnName != NULL); diff --git a/src/backend/distributed/relay/relay_event_utility.c b/src/backend/distributed/relay/relay_event_utility.c index 120e74999..1a5eb0d6c 100644 --- a/src/backend/distributed/relay/relay_event_utility.c +++ b/src/backend/distributed/relay/relay_event_utility.c @@ -31,6 +31,7 @@ #include "catalog/pg_constraint.h" #include "distributed/metadata_cache.h" #include "distributed/relay_utility.h" +#include "distributed/version_compat.h" #include "lib/stringinfo.h" #include "mb/pg_wchar.h" #include "nodes/nodes.h" @@ -289,9 +290,8 @@ RelayEventExtendNames(Node *parseTree, char *schemaName, uint64 shardId) case T_GrantStmt: { GrantStmt *grantStmt = (GrantStmt *) parseTree; - if (grantStmt->targtype == ACL_TARGET_OBJECT && - grantStmt->objtype == ACL_OBJECT_RELATION) + grantStmt->objtype == RELATION_OBJECT_TYPE) { ListCell *lc; diff --git a/src/backend/distributed/transaction/transaction_recovery.c b/src/backend/distributed/transaction/transaction_recovery.c index ffec8bb69..452250015 100644 --- a/src/backend/distributed/transaction/transaction_recovery.c +++ b/src/backend/distributed/transaction/transaction_recovery.c @@ -32,6 +32,7 @@ #include "distributed/remote_commands.h" #include "distributed/transaction_recovery.h" #include "distributed/worker_manager.h" +#include "distributed/version_compat.h" #include "lib/stringinfo.h" #include "storage/lmgr.h" #include "storage/lock.h" @@ -176,11 +177,12 @@ RecoverWorkerTransactions(WorkerNode *workerNode) return 0; } - localContext = AllocSetContextCreate(CurrentMemoryContext, - "RecoverWorkerTransactions", - ALLOCSET_DEFAULT_MINSIZE, - ALLOCSET_DEFAULT_INITSIZE, - ALLOCSET_DEFAULT_MAXSIZE); + localContext = AllocSetContextCreateExtended(CurrentMemoryContext, + "RecoverWorkerTransactions", + ALLOCSET_DEFAULT_MINSIZE, + ALLOCSET_DEFAULT_INITSIZE, + ALLOCSET_DEFAULT_MAXSIZE); + oldContext = MemoryContextSwitchTo(localContext); /* take table lock first to avoid running concurrently */ diff --git a/src/backend/distributed/utils/distribution_column.c b/src/backend/distributed/utils/distribution_column.c index e2a925a04..8d642cfb3 100644 --- a/src/backend/distributed/utils/distribution_column.c +++ b/src/backend/distributed/utils/distribution_column.c @@ -18,6 +18,7 @@ #include "access/htup_details.h" #include "distributed/distribution_column.h" #include "distributed/metadata_cache.h" +#include "distributed/version_compat.h" #include "nodes/makefuncs.h" #include "nodes/nodes.h" #include "nodes/primnodes.h" @@ -205,7 +206,7 @@ ColumnNameToColumn(Oid relationId, char *columnNodeString) columnNumber, relationName))); } - columnName = get_attname(relationId, column->varattno); + columnName = get_attname_internal(relationId, column->varattno, false); if (columnName == NULL) { char *relationName = get_rel_name(relationId); diff --git a/src/backend/distributed/utils/maintenanced.c b/src/backend/distributed/utils/maintenanced.c index 6d4f34612..605ec81e8 100644 --- a/src/backend/distributed/utils/maintenanced.c +++ b/src/backend/distributed/utils/maintenanced.c @@ -35,6 +35,7 @@ #include "distributed/metadata_cache.h" #include "distributed/statistics_collection.h" #include "distributed/transaction_recovery.h" +#include "distributed/version_compat.h" #include "nodes/makefuncs.h" #include "postmaster/bgworker.h" #include "nodes/makefuncs.h" @@ -276,7 +277,7 @@ CitusMaintenanceDaemonMain(Datum main_arg) databaseOid, myDbData->userOid); /* connect to database, after that we can actually access catalogs */ - BackgroundWorkerInitializeConnectionByOid(databaseOid, myDbData->userOid); + BackgroundWorkerInitializeConnectionByOid(databaseOid, myDbData->userOid, 0); /* make worker recognizable in pg_stat_activity */ pgstat_report_appname("Citus Maintenance Daemon"); diff --git a/src/backend/distributed/utils/multi_partitioning_utils.c b/src/backend/distributed/utils/multi_partitioning_utils.c index aa8f8da83..940afd184 100644 --- a/src/backend/distributed/utils/multi_partitioning_utils.c +++ b/src/backend/distributed/utils/multi_partitioning_utils.c @@ -15,7 +15,9 @@ #endif #include "catalog/pg_class.h" #include "catalog/pg_inherits.h" -#include "catalog/pg_inherits_fn.h" +#if (PG_VERSION_NUM < 110000) +#include "catalog/pg_constraint_fn.h" +#endif #include "distributed/citus_ruleutils.h" #include "distributed/multi_partitioning_utils.h" #include "lib/stringinfo.h" @@ -23,6 +25,7 @@ #include "utils/builtins.h" #include "utils/fmgroids.h" #include "utils/lsyscache.h" +#include "utils/rel.h" #include "utils/syscache.h" diff --git a/src/backend/distributed/utils/ruleutils_11.c b/src/backend/distributed/utils/ruleutils_11.c index 51715487c..4b5d3a97a 100644 --- a/src/backend/distributed/utils/ruleutils_11.c +++ b/src/backend/distributed/utils/ruleutils_11.c @@ -96,13 +96,13 @@ #define PRETTYINDENT_LIMIT 40 /* wrap limit */ /* Pretty flags */ -#define PRETTYFLAG_PAREN 1 -#define PRETTYFLAG_INDENT 2 +#define PRETTYFLAG_PAREN 0x0001 +#define PRETTYFLAG_INDENT 0x0002 /* Default line length for pretty-print wrapping: 0 means wrap always */ #define WRAP_COLUMN_DEFAULT 0 -/* macro to test if pretty action needed */ +/* macros to test if pretty action needed */ #define PRETTY_PAREN(context) ((context)->prettyFlags & PRETTYFLAG_PAREN) #define PRETTY_INDENT(context) ((context)->prettyFlags & PRETTYFLAG_INDENT) @@ -122,7 +122,7 @@ typedef struct int prettyFlags; /* enabling of pretty-print functions */ int wrapColumn; /* max line length, or -1 for no limit */ int indentLevel; /* current indent level for prettyprint */ - bool varprefix; /* TRUE to print prefixes on Vars */ + bool varprefix; /* true to print prefixes on Vars */ Oid distrelid; /* the distributed table being modified, if valid */ int64 shardid; /* a distributed table's shardid, if positive */ ParseExprKind special_exprkind; /* set only for exprkinds needing special @@ -142,7 +142,7 @@ typedef struct * rtable_columns holds the column alias names to be used for each RTE. * * In some cases we need to make names of merged JOIN USING columns unique - * across the whole query, not only per-RTE. If so, unique_using is TRUE + * across the whole query, not only per-RTE. If so, unique_using is true * and using_names is a list of C strings representing names already assigned * to USING columns. * @@ -317,7 +317,7 @@ static void set_relation_column_names(deparse_namespace *dpns, deparse_columns *colinfo); static void set_join_column_names(deparse_namespace *dpns, RangeTblEntry *rte, deparse_columns *colinfo); -static bool colname_is_unique(char *colname, deparse_namespace *dpns, +static bool colname_is_unique(const char *colname, deparse_namespace *dpns, deparse_columns *colinfo); static char *make_colname_unique(char *colname, deparse_namespace *dpns, deparse_columns *colinfo); @@ -824,7 +824,7 @@ set_using_names(deparse_namespace *dpns, Node *jtnode, List *parentUsing) * If there's a USING clause, select the USING column names and push * those names down to the children. We have two strategies: * - * If dpns->unique_using is TRUE, we force all USING names to be + * If dpns->unique_using is true, we force all USING names to be * unique across the whole query level. In principle we'd only need * the names of dangerous USING columns to be globally unique, but to * safely assign all USING names in a single pass, we have to enforce @@ -837,7 +837,7 @@ set_using_names(deparse_namespace *dpns, Node *jtnode, List *parentUsing) * this simplifies the logic and seems likely to lead to less aliasing * overall. * - * If dpns->unique_using is FALSE, we only need USING names to be + * If dpns->unique_using is false, we only need USING names to be * unique within their own join RTE. We still need to honor * pushed-down names, though. * @@ -1350,7 +1350,7 @@ set_join_column_names(deparse_namespace *dpns, RangeTblEntry *rte, * dpns is query-wide info, colinfo is for the column's RTE */ static bool -colname_is_unique(char *colname, deparse_namespace *dpns, +colname_is_unique(const char *colname, deparse_namespace *dpns, deparse_columns *colinfo) { int i; @@ -2194,7 +2194,7 @@ get_simple_values_rte(Query *query) ListCell *lc; /* - * We want to return TRUE even if the Query also contains OLD or NEW rule + * We want to return true even if the Query also contains OLD or NEW rule * RTEs. So the idea is to scan the rtable and see if there is only one * inFromCl RTE that is a VALUES RTE. */ @@ -2891,6 +2891,8 @@ get_rule_windowspec(WindowClause *wc, List *targetList, appendStringInfoString(buf, "RANGE "); else if (wc->frameOptions & FRAMEOPTION_ROWS) appendStringInfoString(buf, "ROWS "); + else if (wc->frameOptions & FRAMEOPTION_GROUPS) + appendStringInfoString(buf, "GROUPS "); else Assert(false); if (wc->frameOptions & FRAMEOPTION_BETWEEN) @@ -2899,12 +2901,12 @@ get_rule_windowspec(WindowClause *wc, List *targetList, appendStringInfoString(buf, "UNBOUNDED PRECEDING "); else if (wc->frameOptions & FRAMEOPTION_START_CURRENT_ROW) appendStringInfoString(buf, "CURRENT ROW "); - else if (wc->frameOptions & FRAMEOPTION_START_VALUE) + else if (wc->frameOptions & FRAMEOPTION_START_OFFSET) { get_rule_expr(wc->startOffset, context, false); - if (wc->frameOptions & FRAMEOPTION_START_VALUE_PRECEDING) + if (wc->frameOptions & FRAMEOPTION_START_OFFSET_PRECEDING) appendStringInfoString(buf, " PRECEDING "); - else if (wc->frameOptions & FRAMEOPTION_START_VALUE_FOLLOWING) + else if (wc->frameOptions & FRAMEOPTION_START_OFFSET_FOLLOWING) appendStringInfoString(buf, " FOLLOWING "); else Assert(false); @@ -2918,12 +2920,12 @@ get_rule_windowspec(WindowClause *wc, List *targetList, appendStringInfoString(buf, "UNBOUNDED FOLLOWING "); else if (wc->frameOptions & FRAMEOPTION_END_CURRENT_ROW) appendStringInfoString(buf, "CURRENT ROW "); - else if (wc->frameOptions & FRAMEOPTION_END_VALUE) + else if (wc->frameOptions & FRAMEOPTION_END_OFFSET) { get_rule_expr(wc->endOffset, context, false); - if (wc->frameOptions & FRAMEOPTION_END_VALUE_PRECEDING) + if (wc->frameOptions & FRAMEOPTION_END_OFFSET_PRECEDING) appendStringInfoString(buf, " PRECEDING "); - else if (wc->frameOptions & FRAMEOPTION_END_VALUE_FOLLOWING) + else if (wc->frameOptions & FRAMEOPTION_END_OFFSET_FOLLOWING) appendStringInfoString(buf, " FOLLOWING "); else Assert(false); @@ -2931,6 +2933,12 @@ get_rule_windowspec(WindowClause *wc, List *targetList, else Assert(false); } + if (wc->frameOptions & FRAMEOPTION_EXCLUDE_CURRENT_ROW) + appendStringInfoString(buf, "EXCLUDE CURRENT ROW "); + else if (wc->frameOptions & FRAMEOPTION_EXCLUDE_GROUP) + appendStringInfoString(buf, "EXCLUDE GROUP "); + else if (wc->frameOptions & FRAMEOPTION_EXCLUDE_TIES) + appendStringInfoString(buf, "EXCLUDE TIES "); /* we will now have a trailing space; remove it */ buf->len--; } @@ -3023,8 +3031,9 @@ get_insert_query_def(Query *query, deparse_context *context) * tle->resname, since resname will fail to track RENAME. */ appendStringInfoString(buf, - quote_identifier(get_relid_attribute_name(rte->relid, - tle->resno))); + quote_identifier(get_attname(rte->relid, + tle->resno, + false))); /* * Print any indirection needed (subfields or subscripts), and strip @@ -3358,8 +3367,9 @@ get_update_query_targetlist_def(Query *query, List *targetList, * tle->resname, since resname will fail to track RENAME. */ appendStringInfoString(buf, - quote_identifier(get_relid_attribute_name(rte->relid, - tle->resno))); + quote_identifier(get_attname(rte->relid, + tle->resno, + false))); /* * Print any indirection needed (subfields or subscripts), and strip @@ -3536,7 +3546,7 @@ get_utility_query_def(Query *query, deparse_context *context) * the Var's varlevelsup has to be interpreted with respect to a context * above the current one; levelsup indicates the offset. * - * If istoplevel is TRUE, the Var is at the top level of a SELECT's + * If istoplevel is true, the Var is at the top level of a SELECT's * targetlist, which means we need special treatment of whole-row Vars. * Instead of the normal "tab.*", we'll print "tab.*::typename", which is a * dirty hack to prevent "tab.*" from being expanded into multiple columns. @@ -3681,7 +3691,7 @@ get_variable(Var *var, int levelsup, bool istoplevel, deparse_context *context) else if (GetRangeTblKind(rte) == CITUS_RTE_SHARD) { /* System column on a Citus shard */ - attname = get_relid_attribute_name(rte->relid, attnum); + attname = get_attname(rte->relid, attnum, false); } else { @@ -3869,17 +3879,12 @@ get_name_for_var_field(Var *var, int fieldno, /* * If it's a Var of type RECORD, we have to find what the Var refers to; - * if not, we can use get_expr_result_type. If that fails, we try - * lookup_rowtype_tupdesc, which will probably fail too, but will ereport - * an acceptable message. + * if not, we can use get_expr_result_tupdesc(). */ if (!IsA(var, Var) || var->vartype != RECORDOID) { - if (get_expr_result_type((Node *) var, NULL, &tupleDesc) != TYPEFUNC_COMPOSITE) - tupleDesc = lookup_rowtype_tupdesc_copy(exprType((Node *) var), - exprTypmod((Node *) var)); - Assert(tupleDesc); + tupleDesc = get_expr_result_tupdesc((Node *) var, false); /* Got the tupdesc, so we can extract the field name */ Assert(fieldno >= 1 && fieldno <= tupleDesc->natts); return NameStr(TupleDescAttr(tupleDesc, fieldno - 1)->attname); @@ -4182,14 +4187,9 @@ get_name_for_var_field(Var *var, int fieldno, /* * We now have an expression we can't expand any more, so see if - * get_expr_result_type() can do anything with it. If not, pass to - * lookup_rowtype_tupdesc() which will probably fail, but will give an - * appropriate error message while failing. + * get_expr_result_tupdesc() can do anything with it. */ - if (get_expr_result_type(expr, NULL, &tupleDesc) != TYPEFUNC_COMPOSITE) - tupleDesc = lookup_rowtype_tupdesc_copy(exprType(expr), - exprTypmod(expr)); - Assert(tupleDesc); + tupleDesc = get_expr_result_tupdesc(expr, false); /* Got the tupdesc, so we can extract the field name */ Assert(fieldno >= 1 && fieldno <= tupleDesc->natts); return NameStr(TupleDescAttr(tupleDesc, fieldno - 1)->attname); @@ -5838,8 +5838,23 @@ get_rule_expr(Node *node, deparse_context *context, ListCell *cell; char *sep; + if (spec->is_default) + { + appendStringInfoString(buf, "DEFAULT"); + break; + } + switch (spec->strategy) { + case PARTITION_STRATEGY_HASH: + Assert(spec->modulus > 0 && spec->remainder >= 0); + Assert(spec->modulus > spec->remainder); + + appendStringInfoString(buf, "FOR VALUES"); + appendStringInfo(buf, " WITH (modulus %d, remainder %d)", + spec->modulus, spec->remainder); + break; + case PARTITION_STRATEGY_LIST: Assert(spec->listdatums != NIL); @@ -7461,8 +7476,8 @@ processIndirection(Node *node, deparse_context *context) * target lists, but this function cannot be used for that case. */ Assert(list_length(fstore->fieldnums) == 1); - fieldname = get_relid_attribute_name(typrelid, - linitial_int(fstore->fieldnums)); + fieldname = get_attname(typrelid, + linitial_int(fstore->fieldnums), false); appendStringInfo(buf, ".%s", quote_identifier(fieldname)); /* @@ -7688,7 +7703,7 @@ generate_fragment_name(char *schemaName, char *tableName) * means a FuncExpr or Aggref, not some other way of calling a function), then * has_variadic must specify whether variadic arguments have been merged, * and *use_variadic_p will be set to indicate whether to print VARIADIC in - * the output. For non-FuncExpr cases, has_variadic should be FALSE and + * the output. For non-FuncExpr cases, has_variadic should be false and * use_variadic_p can be NULL. * * The result includes all necessary quoting and schema-prefixing. diff --git a/src/backend/distributed/utils/statistics_collection.c b/src/backend/distributed/utils/statistics_collection.c index 8dfb951ac..80ca7aa38 100644 --- a/src/backend/distributed/utils/statistics_collection.c +++ b/src/backend/distributed/utils/statistics_collection.c @@ -43,6 +43,7 @@ typedef struct utsname #include "distributed/shardinterval_utils.h" #include "distributed/statistics_collection.h" #include "distributed/worker_manager.h" +#include "distributed/version_compat.h" #include "lib/stringinfo.h" #include "utils/builtins.h" #include "utils/json.h" @@ -249,7 +250,7 @@ CheckForUpdatesCallback(char *contents, size_t size, size_t count, void *userDat { Datum responseCStringDatum = CStringGetDatum(responseNullTerminated->data); Datum responseJasonbDatum = DirectFunctionCall1(jsonb_in, responseCStringDatum); - responseJsonb = DatumGetJsonb(responseJasonbDatum); + responseJsonb = DatumGetJsonbP(responseJasonbDatum); ReleaseCurrentSubTransaction(); } PG_CATCH(); diff --git a/src/backend/distributed/worker/task_tracker.c b/src/backend/distributed/worker/task_tracker.c index d5dfe89fe..d2543745a 100644 --- a/src/backend/distributed/worker/task_tracker.c +++ b/src/backend/distributed/worker/task_tracker.c @@ -35,6 +35,7 @@ #include "distributed/task_tracker.h" #include "distributed/transmit.h" #include "distributed/worker_protocol.h" +#include "distributed/version_compat.h" #include "libpq/hba.h" #include "libpq/pqsignal.h" #include "lib/stringinfo.h" @@ -145,10 +146,10 @@ TaskTrackerMain(Datum main_arg) * that we can reset the context during error recovery and thereby avoid * possible memory leaks. */ - TaskTrackerContext = AllocSetContextCreate(TopMemoryContext, "Task Tracker", - ALLOCSET_DEFAULT_MINSIZE, - ALLOCSET_DEFAULT_INITSIZE, - ALLOCSET_DEFAULT_MAXSIZE); + TaskTrackerContext = AllocSetContextCreateExtended(TopMemoryContext, "Task Tracker", + ALLOCSET_DEFAULT_MINSIZE, + ALLOCSET_DEFAULT_INITSIZE, + ALLOCSET_DEFAULT_MAXSIZE); MemoryContextSwitchTo(TaskTrackerContext); /* diff --git a/src/backend/distributed/worker/worker_merge_protocol.c b/src/backend/distributed/worker/worker_merge_protocol.c index 394e12236..f95d19d16 100644 --- a/src/backend/distributed/worker/worker_merge_protocol.c +++ b/src/backend/distributed/worker/worker_merge_protocol.c @@ -25,6 +25,7 @@ #include "commands/tablecmds.h" #include "distributed/metadata_cache.h" #include "distributed/worker_protocol.h" +#include "distributed/version_compat.h" #include "executor/spi.h" #include "nodes/makefuncs.h" #include "parser/parse_type.h" @@ -333,7 +334,7 @@ RemoveJobSchema(StringInfo schemaName) bool permissionsOK = pg_namespace_ownercheck(schemaId, GetUserId()); if (!permissionsOK) { - aclcheck_error(ACLCHECK_NOT_OWNER, ACL_KIND_NAMESPACE, schemaName->data); + aclcheck_error(ACLCHECK_NOT_OWNER, ACLCHECK_OBJECT_SCHEMA, schemaName->data); } schemaObject.classId = NamespaceRelationId; diff --git a/src/backend/distributed/worker/worker_partition_protocol.c b/src/backend/distributed/worker/worker_partition_protocol.c index c6a1d8fa9..a4be31b3b 100644 --- a/src/backend/distributed/worker/worker_partition_protocol.c +++ b/src/backend/distributed/worker/worker_partition_protocol.c @@ -1000,11 +1000,11 @@ InitRowOutputState(void) * recover palloc'd memory. This avoids any problems with leaks inside data * type output routines, and should be faster than retail pfree's anyway. */ - rowOutputState->rowcontext = AllocSetContextCreate(CurrentMemoryContext, - "WorkerRowOutputContext", - ALLOCSET_DEFAULT_MINSIZE, - ALLOCSET_DEFAULT_INITSIZE, - ALLOCSET_DEFAULT_MAXSIZE); + rowOutputState->rowcontext = AllocSetContextCreateExtended(CurrentMemoryContext, + "WorkerRowOutputContext", + ALLOCSET_DEFAULT_MINSIZE, + ALLOCSET_DEFAULT_INITSIZE, + ALLOCSET_DEFAULT_MAXSIZE); /* allocate the message buffer to use for serializing a row */ rowOutputState->fe_msgbuf = makeStringInfo(); diff --git a/src/include/distributed/version_compat.h b/src/include/distributed/version_compat.h index e9b41ed01..a638027f9 100644 --- a/src/include/distributed/version_compat.h +++ b/src/include/distributed/version_compat.h @@ -12,6 +12,9 @@ #define VERSION_COMPAT_H #include "postgres.h" +#include "commands/explain.h" +#include "catalog/namespace.h" +#include "nodes/parsenodes.h" #if (PG_VERSION_NUM >= 90600 && PG_VERSION_NUM < 90700) @@ -25,10 +28,37 @@ #include "access/hash.h" #include "storage/fd.h" +#include "optimizer/prep.h" +#include "utils/memutils.h" /* PostgreSQL 11 splits hash procs into "standard" and "extended" */ #define HASHSTANDARD_PROC HASHPROC +/* following functions are renamed in PG11 */ +#define PreventInTransactionBlock PreventTransactionChain +#define DatumGetJsonbP(d) DatumGetJsonb(d) + +/* following defines also exist for PG11 */ +#define RELATION_OBJECT_TYPE ACL_OBJECT_RELATION +#define IndexInfoAttributeNumberArray(indexinfo) (indexinfo->ii_KeyAttrNumbers) + +/* CreateTrigger api is changed in PG11 */ +#define CreateTriggerInternal(stmt, queryString, relOid, refRelOid, constraintOid, \ + indexOid, funcoid, parentTriggerOid, whenClause, isInternal, \ + in_partition) \ + CreateTrigger(stmt, queryString, relOid, refRelOid, constraintOid, indexOid, \ + isInternal) + +#define get_attname_internal(relationId, columnNumber, false) \ + get_attname(relationId, columnNumber) + +#define BackgroundWorkerInitializeConnectionByOid(dboid, useroid, flags) \ + BackgroundWorkerInitializeConnectionByOid(dboid, useroid) + + +#define ACLCHECK_OBJECT_TABLE ACL_KIND_CLASS +#define ACLCHECK_OBJECT_SCHEMA ACL_KIND_NAMESPACE +#define ACLCHECK_OBJECT_INDEX ACL_KIND_CLASS static inline int BasicOpenFilePerm(FileName fileName, int fileFlags, int fileMode) @@ -44,6 +74,142 @@ PathNameOpenFilePerm(FileName fileName, int fileFlags, int fileMode) } +static inline MemoryContext +AllocSetContextCreateExtended(MemoryContext parent, const char *name, Size minContextSize, + Size initBlockSize, Size maxBlockSize) +{ + return AllocSetContextCreate(parent, name, minContextSize, initBlockSize, + maxBlockSize); +} + + +static inline void +ExplainPropertyIntegerInternal(const char *qlabel, const char *unit, int64 value, + ExplainState *es) +{ + return ExplainPropertyInteger(qlabel, value, es); +} + + +static inline List * +ExtractVacuumTargetRels(VacuumStmt *vacuumStmt) +{ + List *vacuumList = NIL; + + if (vacuumStmt->relation != NULL) + { + vacuumList = list_make1(vacuumStmt->relation); + } + + return vacuumList; +} + + +static inline List * +VacuumColumnList(VacuumStmt *vacuumStmt, int relationIndex) +{ + Assert(relationIndex == 0); + + return vacuumStmt->va_cols; +} + + +#define RVR_MISSING_OK 1 +#define RVR_NOWAIT 2 + +static inline Oid +RangeVarGetRelidInternal(const RangeVar *relation, LOCKMODE lockmode, uint32 flags, + RangeVarGetRelidCallback callback, void *callback_arg) +{ + bool missingOK = ((flags & RVR_MISSING_OK) != 0); + bool noWait = ((flags & RVR_NOWAIT) != 0); + + return RangeVarGetRelidExtended(relation, lockmode, missingOK, noWait, + callback, callback_arg); +} + + +static inline Expr * +canonicalize_qual_compat(Expr *qual, bool is_check) +{ + return canonicalize_qual(qual); +} + + #endif +#if (PG_VERSION_NUM >= 110000) +#include "optimizer/prep.h" + +/* following macros should be removed when we drop support for PG10 and below */ +#define RELATION_OBJECT_TYPE OBJECT_TABLE +#define IndexInfoAttributeNumberArray(indexinfo) (indexinfo->ii_IndexAttrNumbers) +#define CreateTriggerInternal CreateTrigger +#define get_attname_internal get_attname + +#define ACLCHECK_OBJECT_TABLE OBJECT_TABLE +#define ACLCHECK_OBJECT_SCHEMA OBJECT_SCHEMA +#define ACLCHECK_OBJECT_INDEX OBJECT_INDEX + + +static inline void +ExplainPropertyIntegerInternal(const char *qlabel, const char *unit, int64 value, + ExplainState *es) +{ + return ExplainPropertyInteger(qlabel, unit, value, es); +} + + +static inline Expr * +canonicalize_qual_compat(Expr *qual, bool is_check) +{ + return canonicalize_qual(qual, is_check); +} + + +/* + * ExtractVacuumTargetRels returns list of target + * relations from vacuum statement. + */ +static inline List * +ExtractVacuumTargetRels(VacuumStmt *vacuumStmt) +{ + List *vacuumList = NIL; + + ListCell *vacuumRelationCell = NULL; + foreach(vacuumRelationCell, vacuumStmt->rels) + { + VacuumRelation *vacuumRelation = (VacuumRelation *) lfirst(vacuumRelationCell); + vacuumList = lappend(vacuumList, vacuumRelation->relation); + } + + return vacuumList; +} + + +/* + * VacuumColumnList returns list of columns from relation + * in the vacuum statement at specified relationIndex. + */ +static inline List * +VacuumColumnList(VacuumStmt *vacuumStmt, int relationIndex) +{ + VacuumRelation *vacuumRelation = (VacuumRelation *) list_nth(vacuumStmt->rels, + relationIndex); + + return vacuumRelation->va_cols; +} + + +static inline Oid +RangeVarGetRelidInternal(const RangeVar *relation, LOCKMODE lockmode, uint32 flags, + RangeVarGetRelidCallback callback, void *callback_arg) +{ + return RangeVarGetRelidExtended(relation, lockmode, flags, callback, callback_arg); +} + + +#endif + + #endif /* VERSION_COMPAT_H */ diff --git a/src/test/regress/expected/isolation_partitioned_copy_vs_all.out b/src/test/regress/expected/isolation_partitioned_copy_vs_all.out index ff62d1989..626d9a0cd 100644 --- a/src/test/regress/expected/isolation_partitioned_copy_vs_all.out +++ b/src/test/regress/expected/isolation_partitioned_copy_vs_all.out @@ -231,7 +231,7 @@ step s1-copy: COPY partitioned_copy FROM PROGRAM 'echo 5, f, 5 && echo 6, g, 6 & step s2-table-size: SELECT citus_total_relation_size('partitioned_copy'); citus_total_relation_size -32768 +0 step s1-commit: COMMIT; step s1-select-count: SELECT COUNT(*) FROM partitioned_copy; count @@ -511,7 +511,7 @@ step s1-begin: BEGIN; step s1-table-size: SELECT citus_total_relation_size('partitioned_copy'); citus_total_relation_size -32768 +0 step s2-copy: COPY partitioned_copy FROM PROGRAM 'echo 5, f, 5 && echo 6, g, 6 && echo 7, h, 7 && echo 8, i, 8 && echo 9, j, 9' WITH CSV; step s1-commit: COMMIT; step s1-select-count: SELECT COUNT(*) FROM partitioned_copy; diff --git a/src/test/regress/expected/isolation_partitioned_copy_vs_all_0.out b/src/test/regress/expected/isolation_partitioned_copy_vs_all_0.out index e24461b46..ff62d1989 100644 --- a/src/test/regress/expected/isolation_partitioned_copy_vs_all_0.out +++ b/src/test/regress/expected/isolation_partitioned_copy_vs_all_0.out @@ -1,6 +1,576 @@ Parsed test spec with 2 sessions starting permutation: s1-initialize s1-begin s1-copy s2-copy s1-commit s1-select-count -setup failed: ERROR: syntax error at or near "PARTITION" -LINE 3: ...itioned_copy(id integer, data text, int_data int) PARTITION ... - ^ +create_distributed_table + + +step s1-initialize: COPY partitioned_copy FROM PROGRAM 'echo 0, a, 0 && echo 1, b, 1 && echo 2, c, 2 && echo 3, d, 3 && echo 4, e, 4' WITH CSV; +step s1-begin: BEGIN; +step s1-copy: COPY partitioned_copy FROM PROGRAM 'echo 5, f, 5 && echo 6, g, 6 && echo 7, h, 7 && echo 8, i, 8 && echo 9, j, 9' WITH CSV; +step s2-copy: COPY partitioned_copy FROM PROGRAM 'echo 5, f, 5 && echo 6, g, 6 && echo 7, h, 7 && echo 8, i, 8 && echo 9, j, 9' WITH CSV; +step s1-commit: COMMIT; +step s1-select-count: SELECT COUNT(*) FROM partitioned_copy; +count + +15 + +starting permutation: s1-initialize s1-begin s1-copy s2-router-select s1-commit s1-select-count +create_distributed_table + + +step s1-initialize: COPY partitioned_copy FROM PROGRAM 'echo 0, a, 0 && echo 1, b, 1 && echo 2, c, 2 && echo 3, d, 3 && echo 4, e, 4' WITH CSV; +step s1-begin: BEGIN; +step s1-copy: COPY partitioned_copy FROM PROGRAM 'echo 5, f, 5 && echo 6, g, 6 && echo 7, h, 7 && echo 8, i, 8 && echo 9, j, 9' WITH CSV; +step s2-router-select: SELECT * FROM partitioned_copy WHERE id = 1; +id data int_data + +1 b 1 +step s1-commit: COMMIT; +step s1-select-count: SELECT COUNT(*) FROM partitioned_copy; +count + +10 + +starting permutation: s1-initialize s1-begin s1-copy s2-real-time-select s1-commit s1-select-count +create_distributed_table + + +step s1-initialize: COPY partitioned_copy FROM PROGRAM 'echo 0, a, 0 && echo 1, b, 1 && echo 2, c, 2 && echo 3, d, 3 && echo 4, e, 4' WITH CSV; +step s1-begin: BEGIN; +step s1-copy: COPY partitioned_copy FROM PROGRAM 'echo 5, f, 5 && echo 6, g, 6 && echo 7, h, 7 && echo 8, i, 8 && echo 9, j, 9' WITH CSV; +step s2-real-time-select: SELECT * FROM partitioned_copy ORDER BY 1, 2; +id data int_data + +0 a 0 +1 b 1 +2 c 2 +3 d 3 +4 e 4 +step s1-commit: COMMIT; +step s1-select-count: SELECT COUNT(*) FROM partitioned_copy; +count + +10 + +starting permutation: s1-initialize s1-begin s1-copy s2-task-tracker-select s1-commit s1-select-count +create_distributed_table + + +step s1-initialize: COPY partitioned_copy FROM PROGRAM 'echo 0, a, 0 && echo 1, b, 1 && echo 2, c, 2 && echo 3, d, 3 && echo 4, e, 4' WITH CSV; +step s1-begin: BEGIN; +step s1-copy: COPY partitioned_copy FROM PROGRAM 'echo 5, f, 5 && echo 6, g, 6 && echo 7, h, 7 && echo 8, i, 8 && echo 9, j, 9' WITH CSV; +step s2-task-tracker-select: + SET citus.task_executor_type TO "task-tracker"; + SELECT * FROM partitioned_copy AS t1 JOIN partitioned_copy AS t2 ON t1.id = t2.int_data ORDER BY 1, 2, 3, 4; + +id data int_data id data int_data + +0 a 0 0 a 0 +1 b 1 1 b 1 +2 c 2 2 c 2 +3 d 3 3 d 3 +4 e 4 4 e 4 +step s1-commit: COMMIT; +step s1-select-count: SELECT COUNT(*) FROM partitioned_copy; +count + +10 + +starting permutation: s1-initialize s1-begin s1-copy s2-insert s1-commit s1-select-count +create_distributed_table + + +step s1-initialize: COPY partitioned_copy FROM PROGRAM 'echo 0, a, 0 && echo 1, b, 1 && echo 2, c, 2 && echo 3, d, 3 && echo 4, e, 4' WITH CSV; +step s1-begin: BEGIN; +step s1-copy: COPY partitioned_copy FROM PROGRAM 'echo 5, f, 5 && echo 6, g, 6 && echo 7, h, 7 && echo 8, i, 8 && echo 9, j, 9' WITH CSV; +step s2-insert: INSERT INTO partitioned_copy VALUES(0, 'k', 0); +step s1-commit: COMMIT; +step s1-select-count: SELECT COUNT(*) FROM partitioned_copy; +count + +11 + +starting permutation: s1-initialize s1-begin s1-copy s2-insert-select s1-commit s1-select-count +create_distributed_table + + +step s1-initialize: COPY partitioned_copy FROM PROGRAM 'echo 0, a, 0 && echo 1, b, 1 && echo 2, c, 2 && echo 3, d, 3 && echo 4, e, 4' WITH CSV; +step s1-begin: BEGIN; +step s1-copy: COPY partitioned_copy FROM PROGRAM 'echo 5, f, 5 && echo 6, g, 6 && echo 7, h, 7 && echo 8, i, 8 && echo 9, j, 9' WITH CSV; +step s2-insert-select: INSERT INTO partitioned_copy SELECT * FROM partitioned_copy; +step s1-commit: COMMIT; +step s1-select-count: SELECT COUNT(*) FROM partitioned_copy; +count + +15 + +starting permutation: s1-initialize s1-begin s1-copy s2-update s1-commit s1-select-count +create_distributed_table + + +step s1-initialize: COPY partitioned_copy FROM PROGRAM 'echo 0, a, 0 && echo 1, b, 1 && echo 2, c, 2 && echo 3, d, 3 && echo 4, e, 4' WITH CSV; +step s1-begin: BEGIN; +step s1-copy: COPY partitioned_copy FROM PROGRAM 'echo 5, f, 5 && echo 6, g, 6 && echo 7, h, 7 && echo 8, i, 8 && echo 9, j, 9' WITH CSV; +step s2-update: UPDATE partitioned_copy SET data = 'l' WHERE id = 0; +step s1-commit: COMMIT; +step s1-select-count: SELECT COUNT(*) FROM partitioned_copy; +count + +10 + +starting permutation: s1-initialize s1-begin s1-copy s2-delete s1-commit s1-select-count +create_distributed_table + + +step s1-initialize: COPY partitioned_copy FROM PROGRAM 'echo 0, a, 0 && echo 1, b, 1 && echo 2, c, 2 && echo 3, d, 3 && echo 4, e, 4' WITH CSV; +step s1-begin: BEGIN; +step s1-copy: COPY partitioned_copy FROM PROGRAM 'echo 5, f, 5 && echo 6, g, 6 && echo 7, h, 7 && echo 8, i, 8 && echo 9, j, 9' WITH CSV; +step s2-delete: DELETE FROM partitioned_copy WHERE id = 1; +step s1-commit: COMMIT; +step s1-select-count: SELECT COUNT(*) FROM partitioned_copy; +count + +9 + +starting permutation: s1-initialize s1-begin s1-copy s2-truncate s1-commit s1-select-count +create_distributed_table + + +step s1-initialize: COPY partitioned_copy FROM PROGRAM 'echo 0, a, 0 && echo 1, b, 1 && echo 2, c, 2 && echo 3, d, 3 && echo 4, e, 4' WITH CSV; +step s1-begin: BEGIN; +step s1-copy: COPY partitioned_copy FROM PROGRAM 'echo 5, f, 5 && echo 6, g, 6 && echo 7, h, 7 && echo 8, i, 8 && echo 9, j, 9' WITH CSV; +step s2-truncate: TRUNCATE partitioned_copy; +step s1-commit: COMMIT; +step s2-truncate: <... completed> +step s1-select-count: SELECT COUNT(*) FROM partitioned_copy; +count + +0 + +starting permutation: s1-initialize s1-begin s1-copy s2-drop s1-commit s1-select-count +create_distributed_table + + +step s1-initialize: COPY partitioned_copy FROM PROGRAM 'echo 0, a, 0 && echo 1, b, 1 && echo 2, c, 2 && echo 3, d, 3 && echo 4, e, 4' WITH CSV; +step s1-begin: BEGIN; +step s1-copy: COPY partitioned_copy FROM PROGRAM 'echo 5, f, 5 && echo 6, g, 6 && echo 7, h, 7 && echo 8, i, 8 && echo 9, j, 9' WITH CSV; +step s2-drop: DROP TABLE partitioned_copy; +step s1-commit: COMMIT; +step s2-drop: <... completed> +step s1-select-count: SELECT COUNT(*) FROM partitioned_copy; +ERROR: relation "partitioned_copy" does not exist + +starting permutation: s1-initialize s1-begin s1-copy s2-ddl-add-column s1-commit s1-select-count s1-show-columns +create_distributed_table + + +step s1-initialize: COPY partitioned_copy FROM PROGRAM 'echo 0, a, 0 && echo 1, b, 1 && echo 2, c, 2 && echo 3, d, 3 && echo 4, e, 4' WITH CSV; +step s1-begin: BEGIN; +step s1-copy: COPY partitioned_copy FROM PROGRAM 'echo 5, f, 5 && echo 6, g, 6 && echo 7, h, 7 && echo 8, i, 8 && echo 9, j, 9' WITH CSV; +step s2-ddl-add-column: ALTER TABLE partitioned_copy ADD new_column int DEFAULT 0; +step s1-commit: COMMIT; +step s2-ddl-add-column: <... completed> +step s1-select-count: SELECT COUNT(*) FROM partitioned_copy; +count + +10 +step s1-show-columns: SELECT run_command_on_workers('SELECT column_name FROM information_schema.columns WHERE table_name LIKE ''partitioned_copy%'' AND column_name = ''new_column'' ORDER BY 1 LIMIT 1'); +run_command_on_workers + +(localhost,57637,t,new_column) +(localhost,57638,t,new_column) + +starting permutation: s1-initialize s1-ddl-add-column s1-begin s1-copy-additional-column s2-ddl-drop-column s1-commit s1-select-count s1-show-columns +create_distributed_table + + +step s1-initialize: COPY partitioned_copy FROM PROGRAM 'echo 0, a, 0 && echo 1, b, 1 && echo 2, c, 2 && echo 3, d, 3 && echo 4, e, 4' WITH CSV; +step s1-ddl-add-column: ALTER TABLE partitioned_copy ADD new_column int DEFAULT 0; +step s1-begin: BEGIN; +step s1-copy-additional-column: COPY partitioned_copy FROM PROGRAM 'echo 5, f, 5, 5 && echo 6, g, 6, 6 && echo 7, h, 7, 7 && echo 8, i, 8, 8 && echo 9, j, 9, 9' WITH CSV; +step s2-ddl-drop-column: ALTER TABLE partitioned_copy DROP new_column; +step s1-commit: COMMIT; +step s2-ddl-drop-column: <... completed> +step s1-select-count: SELECT COUNT(*) FROM partitioned_copy; +count + +10 +step s1-show-columns: SELECT run_command_on_workers('SELECT column_name FROM information_schema.columns WHERE table_name LIKE ''partitioned_copy%'' AND column_name = ''new_column'' ORDER BY 1 LIMIT 1'); +run_command_on_workers + +(localhost,57637,t,"") +(localhost,57638,t,"") + +starting permutation: s1-initialize s1-begin s1-copy s2-ddl-rename-column s1-commit s1-select-count s1-show-columns +create_distributed_table + + +step s1-initialize: COPY partitioned_copy FROM PROGRAM 'echo 0, a, 0 && echo 1, b, 1 && echo 2, c, 2 && echo 3, d, 3 && echo 4, e, 4' WITH CSV; +step s1-begin: BEGIN; +step s1-copy: COPY partitioned_copy FROM PROGRAM 'echo 5, f, 5 && echo 6, g, 6 && echo 7, h, 7 && echo 8, i, 8 && echo 9, j, 9' WITH CSV; +step s2-ddl-rename-column: ALTER TABLE partitioned_copy RENAME data TO new_column; +step s1-commit: COMMIT; +step s2-ddl-rename-column: <... completed> +step s1-select-count: SELECT COUNT(*) FROM partitioned_copy; +count + +10 +step s1-show-columns: SELECT run_command_on_workers('SELECT column_name FROM information_schema.columns WHERE table_name LIKE ''partitioned_copy%'' AND column_name = ''new_column'' ORDER BY 1 LIMIT 1'); +run_command_on_workers + +(localhost,57637,t,new_column) +(localhost,57638,t,new_column) + +starting permutation: s1-initialize s1-begin s1-copy s2-table-size s1-commit s1-select-count +create_distributed_table + + +step s1-initialize: COPY partitioned_copy FROM PROGRAM 'echo 0, a, 0 && echo 1, b, 1 && echo 2, c, 2 && echo 3, d, 3 && echo 4, e, 4' WITH CSV; +step s1-begin: BEGIN; +step s1-copy: COPY partitioned_copy FROM PROGRAM 'echo 5, f, 5 && echo 6, g, 6 && echo 7, h, 7 && echo 8, i, 8 && echo 9, j, 9' WITH CSV; +step s2-table-size: SELECT citus_total_relation_size('partitioned_copy'); +citus_total_relation_size + +32768 +step s1-commit: COMMIT; +step s1-select-count: SELECT COUNT(*) FROM partitioned_copy; +count + +10 + +starting permutation: s1-initialize s1-begin s1-copy s2-master-modify-multiple-shards s1-commit s1-select-count +create_distributed_table + + +step s1-initialize: COPY partitioned_copy FROM PROGRAM 'echo 0, a, 0 && echo 1, b, 1 && echo 2, c, 2 && echo 3, d, 3 && echo 4, e, 4' WITH CSV; +step s1-begin: BEGIN; +step s1-copy: COPY partitioned_copy FROM PROGRAM 'echo 5, f, 5 && echo 6, g, 6 && echo 7, h, 7 && echo 8, i, 8 && echo 9, j, 9' WITH CSV; +step s2-master-modify-multiple-shards: SELECT master_modify_multiple_shards('DELETE FROM partitioned_copy;'); +master_modify_multiple_shards + +5 +step s1-commit: COMMIT; +step s1-select-count: SELECT COUNT(*) FROM partitioned_copy; +count + +5 + +starting permutation: s1-initialize s1-begin s1-copy s2-master-drop-all-shards s1-commit s1-select-count +create_distributed_table + + +step s1-initialize: COPY partitioned_copy FROM PROGRAM 'echo 0, a, 0 && echo 1, b, 1 && echo 2, c, 2 && echo 3, d, 3 && echo 4, e, 4' WITH CSV; +step s1-begin: BEGIN; +step s1-copy: COPY partitioned_copy FROM PROGRAM 'echo 5, f, 5 && echo 6, g, 6 && echo 7, h, 7 && echo 8, i, 8 && echo 9, j, 9' WITH CSV; +step s2-master-drop-all-shards: SELECT master_drop_all_shards('partitioned_copy'::regclass, 'public', 'partitioned_copy'); +step s1-commit: COMMIT; +step s2-master-drop-all-shards: <... completed> +master_drop_all_shards + +4 +step s1-select-count: SELECT COUNT(*) FROM partitioned_copy; +count + +0 + +starting permutation: s1-drop s1-create-non-distributed-table s1-initialize s1-begin s1-copy s2-distribute-table s1-commit s1-select-count +create_distributed_table + + +step s1-drop: DROP TABLE partitioned_copy; +step s1-create-non-distributed-table: CREATE TABLE partitioned_copy(id integer, data text, int_data int); COPY partitioned_copy FROM PROGRAM 'echo 0, a, 0 && echo 1, b, 1 && echo 2, c, 2 && echo 3, d, 3 && echo 4, e, 4' WITH CSV; +step s1-initialize: COPY partitioned_copy FROM PROGRAM 'echo 0, a, 0 && echo 1, b, 1 && echo 2, c, 2 && echo 3, d, 3 && echo 4, e, 4' WITH CSV; +step s1-begin: BEGIN; +step s1-copy: COPY partitioned_copy FROM PROGRAM 'echo 5, f, 5 && echo 6, g, 6 && echo 7, h, 7 && echo 8, i, 8 && echo 9, j, 9' WITH CSV; +step s2-distribute-table: SELECT create_distributed_table('partitioned_copy', 'id'); +step s1-commit: COMMIT; +step s2-distribute-table: <... completed> +create_distributed_table + + +step s1-select-count: SELECT COUNT(*) FROM partitioned_copy; +count + +15 + +starting permutation: s1-initialize s1-begin s1-router-select s2-copy s1-commit s1-select-count +create_distributed_table + + +step s1-initialize: COPY partitioned_copy FROM PROGRAM 'echo 0, a, 0 && echo 1, b, 1 && echo 2, c, 2 && echo 3, d, 3 && echo 4, e, 4' WITH CSV; +step s1-begin: BEGIN; +step s1-router-select: SELECT * FROM partitioned_copy WHERE id = 1; +id data int_data + +1 b 1 +step s2-copy: COPY partitioned_copy FROM PROGRAM 'echo 5, f, 5 && echo 6, g, 6 && echo 7, h, 7 && echo 8, i, 8 && echo 9, j, 9' WITH CSV; +step s1-commit: COMMIT; +step s1-select-count: SELECT COUNT(*) FROM partitioned_copy; +count + +10 + +starting permutation: s1-initialize s1-begin s1-real-time-select s2-copy s1-commit s1-select-count +create_distributed_table + + +step s1-initialize: COPY partitioned_copy FROM PROGRAM 'echo 0, a, 0 && echo 1, b, 1 && echo 2, c, 2 && echo 3, d, 3 && echo 4, e, 4' WITH CSV; +step s1-begin: BEGIN; +step s1-real-time-select: SELECT * FROM partitioned_copy ORDER BY 1, 2; +id data int_data + +0 a 0 +1 b 1 +2 c 2 +3 d 3 +4 e 4 +step s2-copy: COPY partitioned_copy FROM PROGRAM 'echo 5, f, 5 && echo 6, g, 6 && echo 7, h, 7 && echo 8, i, 8 && echo 9, j, 9' WITH CSV; +step s1-commit: COMMIT; +step s1-select-count: SELECT COUNT(*) FROM partitioned_copy; +count + +10 + +starting permutation: s1-initialize s1-begin s1-task-tracker-select s2-copy s1-commit s1-select-count +create_distributed_table + + +step s1-initialize: COPY partitioned_copy FROM PROGRAM 'echo 0, a, 0 && echo 1, b, 1 && echo 2, c, 2 && echo 3, d, 3 && echo 4, e, 4' WITH CSV; +step s1-begin: BEGIN; +step s1-task-tracker-select: + SET citus.task_executor_type TO "task-tracker"; + SELECT * FROM partitioned_copy AS t1 JOIN partitioned_copy AS t2 ON t1.id = t2.int_data ORDER BY 1, 2, 3, 4; + +id data int_data id data int_data + +0 a 0 0 a 0 +1 b 1 1 b 1 +2 c 2 2 c 2 +3 d 3 3 d 3 +4 e 4 4 e 4 +step s2-copy: COPY partitioned_copy FROM PROGRAM 'echo 5, f, 5 && echo 6, g, 6 && echo 7, h, 7 && echo 8, i, 8 && echo 9, j, 9' WITH CSV; +step s1-commit: COMMIT; +step s1-select-count: SELECT COUNT(*) FROM partitioned_copy; +count + +10 + +starting permutation: s1-initialize s1-begin s1-insert s2-copy s1-commit s1-select-count +create_distributed_table + + +step s1-initialize: COPY partitioned_copy FROM PROGRAM 'echo 0, a, 0 && echo 1, b, 1 && echo 2, c, 2 && echo 3, d, 3 && echo 4, e, 4' WITH CSV; +step s1-begin: BEGIN; +step s1-insert: INSERT INTO partitioned_copy VALUES(0, 'k', 0); +step s2-copy: COPY partitioned_copy FROM PROGRAM 'echo 5, f, 5 && echo 6, g, 6 && echo 7, h, 7 && echo 8, i, 8 && echo 9, j, 9' WITH CSV; +step s1-commit: COMMIT; +step s1-select-count: SELECT COUNT(*) FROM partitioned_copy; +count + +11 + +starting permutation: s1-initialize s1-begin s1-insert-select s2-copy s1-commit s1-select-count +create_distributed_table + + +step s1-initialize: COPY partitioned_copy FROM PROGRAM 'echo 0, a, 0 && echo 1, b, 1 && echo 2, c, 2 && echo 3, d, 3 && echo 4, e, 4' WITH CSV; +step s1-begin: BEGIN; +step s1-insert-select: INSERT INTO partitioned_copy SELECT * FROM partitioned_copy; +step s2-copy: COPY partitioned_copy FROM PROGRAM 'echo 5, f, 5 && echo 6, g, 6 && echo 7, h, 7 && echo 8, i, 8 && echo 9, j, 9' WITH CSV; +step s1-commit: COMMIT; +step s1-select-count: SELECT COUNT(*) FROM partitioned_copy; +count + +15 + +starting permutation: s1-initialize s1-begin s1-update s2-copy s1-commit s1-select-count +create_distributed_table + + +step s1-initialize: COPY partitioned_copy FROM PROGRAM 'echo 0, a, 0 && echo 1, b, 1 && echo 2, c, 2 && echo 3, d, 3 && echo 4, e, 4' WITH CSV; +step s1-begin: BEGIN; +step s1-update: UPDATE partitioned_copy SET data = 'l' WHERE id = 0; +step s2-copy: COPY partitioned_copy FROM PROGRAM 'echo 5, f, 5 && echo 6, g, 6 && echo 7, h, 7 && echo 8, i, 8 && echo 9, j, 9' WITH CSV; +step s1-commit: COMMIT; +step s1-select-count: SELECT COUNT(*) FROM partitioned_copy; +count + +10 + +starting permutation: s1-initialize s1-begin s1-delete s2-copy s1-commit s1-select-count +create_distributed_table + + +step s1-initialize: COPY partitioned_copy FROM PROGRAM 'echo 0, a, 0 && echo 1, b, 1 && echo 2, c, 2 && echo 3, d, 3 && echo 4, e, 4' WITH CSV; +step s1-begin: BEGIN; +step s1-delete: DELETE FROM partitioned_copy WHERE id = 1; +step s2-copy: COPY partitioned_copy FROM PROGRAM 'echo 5, f, 5 && echo 6, g, 6 && echo 7, h, 7 && echo 8, i, 8 && echo 9, j, 9' WITH CSV; +step s1-commit: COMMIT; +step s1-select-count: SELECT COUNT(*) FROM partitioned_copy; +count + +9 + +starting permutation: s1-initialize s1-begin s1-truncate s2-copy s1-commit s1-select-count +create_distributed_table + + +step s1-initialize: COPY partitioned_copy FROM PROGRAM 'echo 0, a, 0 && echo 1, b, 1 && echo 2, c, 2 && echo 3, d, 3 && echo 4, e, 4' WITH CSV; +step s1-begin: BEGIN; +step s1-truncate: TRUNCATE partitioned_copy; +step s2-copy: COPY partitioned_copy FROM PROGRAM 'echo 5, f, 5 && echo 6, g, 6 && echo 7, h, 7 && echo 8, i, 8 && echo 9, j, 9' WITH CSV; +step s1-commit: COMMIT; +step s2-copy: <... completed> +step s1-select-count: SELECT COUNT(*) FROM partitioned_copy; +count + +5 + +starting permutation: s1-initialize s1-begin s1-drop s2-copy s1-commit s1-select-count +create_distributed_table + + +step s1-initialize: COPY partitioned_copy FROM PROGRAM 'echo 0, a, 0 && echo 1, b, 1 && echo 2, c, 2 && echo 3, d, 3 && echo 4, e, 4' WITH CSV; +step s1-begin: BEGIN; +step s1-drop: DROP TABLE partitioned_copy; +step s2-copy: COPY partitioned_copy FROM PROGRAM 'echo 5, f, 5 && echo 6, g, 6 && echo 7, h, 7 && echo 8, i, 8 && echo 9, j, 9' WITH CSV; +step s1-commit: COMMIT; +step s2-copy: <... completed> +error in steps s1-commit s2-copy: ERROR: relation "partitioned_copy" does not exist +step s1-select-count: SELECT COUNT(*) FROM partitioned_copy; +ERROR: relation "partitioned_copy" does not exist + +starting permutation: s1-initialize s1-begin s1-ddl-add-column s2-copy s1-commit s1-select-count s1-show-columns +create_distributed_table + + +step s1-initialize: COPY partitioned_copy FROM PROGRAM 'echo 0, a, 0 && echo 1, b, 1 && echo 2, c, 2 && echo 3, d, 3 && echo 4, e, 4' WITH CSV; +step s1-begin: BEGIN; +step s1-ddl-add-column: ALTER TABLE partitioned_copy ADD new_column int DEFAULT 0; +step s2-copy: COPY partitioned_copy FROM PROGRAM 'echo 5, f, 5 && echo 6, g, 6 && echo 7, h, 7 && echo 8, i, 8 && echo 9, j, 9' WITH CSV; +step s1-commit: COMMIT; +step s2-copy: <... completed> +error in steps s1-commit s2-copy: ERROR: missing data for column "new_column" +step s1-select-count: SELECT COUNT(*) FROM partitioned_copy; +count + +5 +step s1-show-columns: SELECT run_command_on_workers('SELECT column_name FROM information_schema.columns WHERE table_name LIKE ''partitioned_copy%'' AND column_name = ''new_column'' ORDER BY 1 LIMIT 1'); +run_command_on_workers + +(localhost,57637,t,new_column) +(localhost,57638,t,new_column) + +starting permutation: s1-initialize s1-ddl-add-column s1-begin s1-ddl-drop-column s2-copy s1-commit s1-select-count s1-show-columns +create_distributed_table + + +step s1-initialize: COPY partitioned_copy FROM PROGRAM 'echo 0, a, 0 && echo 1, b, 1 && echo 2, c, 2 && echo 3, d, 3 && echo 4, e, 4' WITH CSV; +step s1-ddl-add-column: ALTER TABLE partitioned_copy ADD new_column int DEFAULT 0; +step s1-begin: BEGIN; +step s1-ddl-drop-column: ALTER TABLE partitioned_copy DROP new_column; +step s2-copy: COPY partitioned_copy FROM PROGRAM 'echo 5, f, 5 && echo 6, g, 6 && echo 7, h, 7 && echo 8, i, 8 && echo 9, j, 9' WITH CSV; +step s1-commit: COMMIT; +step s2-copy: <... completed> +step s1-select-count: SELECT COUNT(*) FROM partitioned_copy; +count + +10 +step s1-show-columns: SELECT run_command_on_workers('SELECT column_name FROM information_schema.columns WHERE table_name LIKE ''partitioned_copy%'' AND column_name = ''new_column'' ORDER BY 1 LIMIT 1'); +run_command_on_workers + +(localhost,57637,t,"") +(localhost,57638,t,"") + +starting permutation: s1-initialize s1-begin s1-ddl-rename-column s2-copy s1-commit s1-select-count s1-show-columns +create_distributed_table + + +step s1-initialize: COPY partitioned_copy FROM PROGRAM 'echo 0, a, 0 && echo 1, b, 1 && echo 2, c, 2 && echo 3, d, 3 && echo 4, e, 4' WITH CSV; +step s1-begin: BEGIN; +step s1-ddl-rename-column: ALTER TABLE partitioned_copy RENAME data TO new_column; +step s2-copy: COPY partitioned_copy FROM PROGRAM 'echo 5, f, 5 && echo 6, g, 6 && echo 7, h, 7 && echo 8, i, 8 && echo 9, j, 9' WITH CSV; +step s1-commit: COMMIT; +step s2-copy: <... completed> +step s1-select-count: SELECT COUNT(*) FROM partitioned_copy; +count + +10 +step s1-show-columns: SELECT run_command_on_workers('SELECT column_name FROM information_schema.columns WHERE table_name LIKE ''partitioned_copy%'' AND column_name = ''new_column'' ORDER BY 1 LIMIT 1'); +run_command_on_workers + +(localhost,57637,t,new_column) +(localhost,57638,t,new_column) + +starting permutation: s1-initialize s1-begin s1-table-size s2-copy s1-commit s1-select-count +create_distributed_table + + +step s1-initialize: COPY partitioned_copy FROM PROGRAM 'echo 0, a, 0 && echo 1, b, 1 && echo 2, c, 2 && echo 3, d, 3 && echo 4, e, 4' WITH CSV; +step s1-begin: BEGIN; +step s1-table-size: SELECT citus_total_relation_size('partitioned_copy'); +citus_total_relation_size + +32768 +step s2-copy: COPY partitioned_copy FROM PROGRAM 'echo 5, f, 5 && echo 6, g, 6 && echo 7, h, 7 && echo 8, i, 8 && echo 9, j, 9' WITH CSV; +step s1-commit: COMMIT; +step s1-select-count: SELECT COUNT(*) FROM partitioned_copy; +count + +10 + +starting permutation: s1-initialize s1-begin s1-master-modify-multiple-shards s2-copy s1-commit s1-select-count +create_distributed_table + + +step s1-initialize: COPY partitioned_copy FROM PROGRAM 'echo 0, a, 0 && echo 1, b, 1 && echo 2, c, 2 && echo 3, d, 3 && echo 4, e, 4' WITH CSV; +step s1-begin: BEGIN; +step s1-master-modify-multiple-shards: SELECT master_modify_multiple_shards('DELETE FROM partitioned_copy;'); +master_modify_multiple_shards + +5 +step s2-copy: COPY partitioned_copy FROM PROGRAM 'echo 5, f, 5 && echo 6, g, 6 && echo 7, h, 7 && echo 8, i, 8 && echo 9, j, 9' WITH CSV; +step s1-commit: COMMIT; +step s1-select-count: SELECT COUNT(*) FROM partitioned_copy; +count + +5 + +starting permutation: s1-initialize s1-begin s1-master-drop-all-shards s2-copy s1-commit s1-select-count +create_distributed_table + + +step s1-initialize: COPY partitioned_copy FROM PROGRAM 'echo 0, a, 0 && echo 1, b, 1 && echo 2, c, 2 && echo 3, d, 3 && echo 4, e, 4' WITH CSV; +step s1-begin: BEGIN; +step s1-master-drop-all-shards: SELECT master_drop_all_shards('partitioned_copy'::regclass, 'public', 'partitioned_copy'); +master_drop_all_shards + +4 +step s2-copy: COPY partitioned_copy FROM PROGRAM 'echo 5, f, 5 && echo 6, g, 6 && echo 7, h, 7 && echo 8, i, 8 && echo 9, j, 9' WITH CSV; +step s1-commit: COMMIT; +step s2-copy: <... completed> +error in steps s1-commit s2-copy: ERROR: could not find any shards into which to copy +step s1-select-count: SELECT COUNT(*) FROM partitioned_copy; +count + +0 + +starting permutation: s1-drop s1-create-non-distributed-table s1-initialize s1-begin s1-distribute-table s2-copy s1-commit s1-select-count +create_distributed_table + + +step s1-drop: DROP TABLE partitioned_copy; +step s1-create-non-distributed-table: CREATE TABLE partitioned_copy(id integer, data text, int_data int); COPY partitioned_copy FROM PROGRAM 'echo 0, a, 0 && echo 1, b, 1 && echo 2, c, 2 && echo 3, d, 3 && echo 4, e, 4' WITH CSV; +step s1-initialize: COPY partitioned_copy FROM PROGRAM 'echo 0, a, 0 && echo 1, b, 1 && echo 2, c, 2 && echo 3, d, 3 && echo 4, e, 4' WITH CSV; +step s1-begin: BEGIN; +step s1-distribute-table: SELECT create_distributed_table('partitioned_copy', 'id'); +create_distributed_table + + +step s2-copy: COPY partitioned_copy FROM PROGRAM 'echo 5, f, 5 && echo 6, g, 6 && echo 7, h, 7 && echo 8, i, 8 && echo 9, j, 9' WITH CSV; +step s1-commit: COMMIT; +step s2-copy: <... completed> +step s1-select-count: SELECT COUNT(*) FROM partitioned_copy; +count + +15 diff --git a/src/test/regress/expected/isolation_partitioned_copy_vs_all_1.out b/src/test/regress/expected/isolation_partitioned_copy_vs_all_1.out new file mode 100644 index 000000000..e24461b46 --- /dev/null +++ b/src/test/regress/expected/isolation_partitioned_copy_vs_all_1.out @@ -0,0 +1,6 @@ +Parsed test spec with 2 sessions + +starting permutation: s1-initialize s1-begin s1-copy s2-copy s1-commit s1-select-count +setup failed: ERROR: syntax error at or near "PARTITION" +LINE 3: ...itioned_copy(id integer, data text, int_data int) PARTITION ... + ^ diff --git a/src/test/regress/expected/multi_complex_count_distinct_0.out b/src/test/regress/expected/multi_complex_count_distinct_0.out new file mode 100644 index 000000000..1109fbeb4 --- /dev/null +++ b/src/test/regress/expected/multi_complex_count_distinct_0.out @@ -0,0 +1,1049 @@ +-- +-- COMPLEX_COUNT_DISTINCT +-- +-- print whether we're using version > 10 to make version-specific tests clear +SHOW server_version \gset +SELECT substring(:'server_version', '\d+')::int > 10 AS version_above_ten; + version_above_ten +------------------- + f +(1 row) + +SET citus.next_shard_id TO 240000; +CREATE TABLE lineitem_hash ( + l_orderkey bigint not null, + l_partkey integer not null, + l_suppkey integer not null, + l_linenumber integer not null, + l_quantity decimal(15, 2) not null, + l_extendedprice decimal(15, 2) not null, + l_discount decimal(15, 2) not null, + l_tax decimal(15, 2) not null, + l_returnflag char(1) not null, + l_linestatus char(1) not null, + l_shipdate date not null, + l_commitdate date not null, + l_receiptdate date not null, + l_shipinstruct char(25) not null, + l_shipmode char(10) not null, + l_comment varchar(44) not null, + PRIMARY KEY(l_orderkey, l_linenumber) ); + +SELECT master_create_distributed_table('lineitem_hash', 'l_orderkey', 'hash'); + master_create_distributed_table +--------------------------------- + +(1 row) + +SELECT master_create_worker_shards('lineitem_hash', 8, 1); + master_create_worker_shards +----------------------------- + +(1 row) + +\copy lineitem_hash FROM '/Users/mtuncer/dev/citus/features/remove_poll/src/test/regress/data/lineitem.1.data' with delimiter '|' +\copy lineitem_hash FROM '/Users/mtuncer/dev/citus/features/remove_poll/src/test/regress/data/lineitem.2.data' with delimiter '|' +ANALYZE lineitem_hash; +SET citus.task_executor_type to "task-tracker"; +-- count(distinct) is supported on top level query if there +-- is a grouping on the partition key +SELECT + l_orderkey, count(DISTINCT l_partkey) + FROM lineitem_hash + GROUP BY l_orderkey + ORDER BY 2 DESC, 1 DESC + LIMIT 10; + l_orderkey | count +------------+------- + 14885 | 7 + 14884 | 7 + 14821 | 7 + 14790 | 7 + 14785 | 7 + 14755 | 7 + 14725 | 7 + 14694 | 7 + 14627 | 7 + 14624 | 7 +(10 rows) + +EXPLAIN (COSTS false, VERBOSE true) +SELECT + l_orderkey, count(DISTINCT l_partkey) + FROM lineitem_hash + GROUP BY l_orderkey + ORDER BY 2 DESC, 1 DESC + LIMIT 10; + QUERY PLAN +---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- + Limit + Output: remote_scan.l_orderkey, COALESCE((pg_catalog.sum((COALESCE((pg_catalog.sum((COALESCE((pg_catalog.sum(remote_scan.count))::bigint, '0'::bigint))))::bigint, '0'::bigint))))::bigint, '0'::bigint) + -> Sort + Output: remote_scan.l_orderkey, COALESCE((pg_catalog.sum((COALESCE((pg_catalog.sum(remote_scan.count))::bigint, '0'::bigint))))::bigint, '0'::bigint) + Sort Key: COALESCE((pg_catalog.sum((COALESCE((pg_catalog.sum(remote_scan.count))::bigint, '0'::bigint))))::bigint, '0'::bigint) DESC, remote_scan.l_orderkey DESC + -> HashAggregate + Output: remote_scan.l_orderkey, COALESCE((pg_catalog.sum(remote_scan.count))::bigint, '0'::bigint) + Group Key: remote_scan.l_orderkey + -> Custom Scan (Citus Task-Tracker) + Output: remote_scan.l_orderkey, remote_scan.count + Task Count: 8 + Tasks Shown: One of 8 + -> Task + Node: host=localhost port=57637 dbname=regression + -> Limit + Output: l_orderkey, (count(DISTINCT l_partkey)) + -> Sort + Output: l_orderkey, (count(DISTINCT l_partkey)) + Sort Key: (count(DISTINCT lineitem_hash.l_partkey)) DESC, lineitem_hash.l_orderkey DESC + -> GroupAggregate + Output: l_orderkey, count(DISTINCT l_partkey) + Group Key: lineitem_hash.l_orderkey + -> Index Scan Backward using lineitem_hash_pkey_240000 on public.lineitem_hash_240000 lineitem_hash + Output: l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment +(24 rows) + +-- it is also supported if there is no grouping or grouping is on non-partition field +SELECT + count(DISTINCT l_partkey) + FROM lineitem_hash + ORDER BY 1 DESC + LIMIT 10; + count +------- + 11661 +(1 row) + +EXPLAIN (COSTS false, VERBOSE true) +SELECT + count(DISTINCT l_partkey) + FROM lineitem_hash + ORDER BY 1 DESC + LIMIT 10; + QUERY PLAN +---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- + Limit + Output: count(DISTINCT (count(DISTINCT (count(DISTINCT remote_scan.count))))) + -> Sort + Output: count(DISTINCT (count(DISTINCT remote_scan.count))) + Sort Key: count(DISTINCT (count(DISTINCT remote_scan.count))) DESC + -> Aggregate + Output: count(DISTINCT remote_scan.count) + -> Custom Scan (Citus Task-Tracker) + Output: remote_scan.count + Task Count: 8 + Tasks Shown: One of 8 + -> Task + Node: host=localhost port=57637 dbname=regression + -> HashAggregate + Output: l_partkey + Group Key: lineitem_hash.l_partkey + -> Seq Scan on public.lineitem_hash_240000 lineitem_hash + Output: l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment +(18 rows) + +SELECT + l_shipmode, count(DISTINCT l_partkey) + FROM lineitem_hash + GROUP BY l_shipmode + ORDER BY 2 DESC, 1 DESC + LIMIT 10; + l_shipmode | count +------------+------- + TRUCK | 1757 + MAIL | 1730 + AIR | 1702 + FOB | 1700 + RAIL | 1696 + SHIP | 1684 + REG AIR | 1676 +(7 rows) + +EXPLAIN (COSTS false, VERBOSE true) +SELECT + l_shipmode, count(DISTINCT l_partkey) + FROM lineitem_hash + GROUP BY l_shipmode + ORDER BY 2 DESC, 1 DESC + LIMIT 10; + QUERY PLAN +---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- + Limit + Output: remote_scan.l_shipmode, count(DISTINCT (count(DISTINCT (count(DISTINCT remote_scan.count))))) + -> Sort + Output: remote_scan.l_shipmode, count(DISTINCT (count(DISTINCT remote_scan.count))) + Sort Key: count(DISTINCT (count(DISTINCT remote_scan.count))) DESC, remote_scan.l_shipmode DESC + -> GroupAggregate + Output: remote_scan.l_shipmode, count(DISTINCT remote_scan.count) + Group Key: remote_scan.l_shipmode + -> Sort + Output: remote_scan.l_shipmode, remote_scan.count + Sort Key: remote_scan.l_shipmode DESC + -> Custom Scan (Citus Task-Tracker) + Output: remote_scan.l_shipmode, remote_scan.count + Task Count: 8 + Tasks Shown: One of 8 + -> Task + Node: host=localhost port=57637 dbname=regression + -> HashAggregate + Output: l_shipmode, l_partkey + Group Key: lineitem_hash.l_shipmode, lineitem_hash.l_partkey + -> Seq Scan on public.lineitem_hash_240000 lineitem_hash + Output: l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment +(22 rows) + +-- mixed mode count distinct, grouped by partition column +SELECT + l_orderkey, count(distinct l_partkey), count(distinct l_shipmode) + FROM lineitem_hash + GROUP BY l_orderkey + ORDER BY 3 DESC, 2 DESC, 1 + LIMIT 10; + l_orderkey | count | count +------------+-------+------- + 226 | 7 | 7 + 1316 | 7 | 7 + 1477 | 7 | 7 + 3555 | 7 | 7 + 12258 | 7 | 7 + 12835 | 7 | 7 + 768 | 7 | 6 + 1121 | 7 | 6 + 1153 | 7 | 6 + 1281 | 7 | 6 +(10 rows) + +EXPLAIN (COSTS false, VERBOSE true) +SELECT + l_orderkey, count(distinct l_partkey), count(distinct l_shipmode) + FROM lineitem_hash + GROUP BY l_orderkey + ORDER BY 3 DESC, 2 DESC, 1 + LIMIT 10; + QUERY PLAN +---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- + Limit + Output: remote_scan.l_orderkey, COALESCE((pg_catalog.sum((COALESCE((pg_catalog.sum((COALESCE((pg_catalog.sum(remote_scan.count))::bigint, '0'::bigint))))::bigint, '0'::bigint))))::bigint, '0'::bigint), COALESCE((pg_catalog.sum((COALESCE((pg_catalog.sum((COALESCE((pg_catalog.sum(remote_scan.count_1))::bigint, '0'::bigint))))::bigint, '0'::bigint))))::bigint, '0'::bigint) + -> Sort + Output: remote_scan.l_orderkey, COALESCE((pg_catalog.sum((COALESCE((pg_catalog.sum(remote_scan.count))::bigint, '0'::bigint))))::bigint, '0'::bigint), COALESCE((pg_catalog.sum((COALESCE((pg_catalog.sum(remote_scan.count_1))::bigint, '0'::bigint))))::bigint, '0'::bigint) + Sort Key: COALESCE((pg_catalog.sum((COALESCE((pg_catalog.sum(remote_scan.count_1))::bigint, '0'::bigint))))::bigint, '0'::bigint) DESC, COALESCE((pg_catalog.sum((COALESCE((pg_catalog.sum(remote_scan.count))::bigint, '0'::bigint))))::bigint, '0'::bigint) DESC, remote_scan.l_orderkey + -> HashAggregate + Output: remote_scan.l_orderkey, COALESCE((pg_catalog.sum(remote_scan.count))::bigint, '0'::bigint), COALESCE((pg_catalog.sum(remote_scan.count_1))::bigint, '0'::bigint) + Group Key: remote_scan.l_orderkey + -> Custom Scan (Citus Task-Tracker) + Output: remote_scan.l_orderkey, remote_scan.count, remote_scan.count_1 + Task Count: 8 + Tasks Shown: One of 8 + -> Task + Node: host=localhost port=57637 dbname=regression + -> Limit + Output: l_orderkey, (count(DISTINCT l_partkey)), (count(DISTINCT l_shipmode)) + -> Sort + Output: l_orderkey, (count(DISTINCT l_partkey)), (count(DISTINCT l_shipmode)) + Sort Key: (count(DISTINCT lineitem_hash.l_shipmode)) DESC, (count(DISTINCT lineitem_hash.l_partkey)) DESC, lineitem_hash.l_orderkey + -> GroupAggregate + Output: l_orderkey, count(DISTINCT l_partkey), count(DISTINCT l_shipmode) + Group Key: lineitem_hash.l_orderkey + -> Index Scan using lineitem_hash_pkey_240000 on public.lineitem_hash_240000 lineitem_hash + Output: l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment +(24 rows) + +-- partition/non-partition column count distinct no grouping +SELECT + count(distinct l_orderkey), count(distinct l_partkey), count(distinct l_shipmode) + FROM lineitem_hash; + count | count | count +-------+-------+------- + 2985 | 11661 | 7 +(1 row) + +EXPLAIN (COSTS false, VERBOSE true) +SELECT + count(distinct l_orderkey), count(distinct l_partkey), count(distinct l_shipmode) + FROM lineitem_hash; + QUERY PLAN +---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- + Aggregate + Output: count(DISTINCT remote_scan.count), count(DISTINCT remote_scan.count_1), count(DISTINCT remote_scan.count_2) + -> Custom Scan (Citus Task-Tracker) + Output: remote_scan.count, remote_scan.count_1, remote_scan.count_2 + Task Count: 8 + Tasks Shown: One of 8 + -> Task + Node: host=localhost port=57637 dbname=regression + -> HashAggregate + Output: l_orderkey, l_partkey, l_shipmode + Group Key: lineitem_hash.l_orderkey, lineitem_hash.l_partkey, lineitem_hash.l_shipmode + -> Seq Scan on public.lineitem_hash_240000 lineitem_hash + Output: l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment +(13 rows) + +-- distinct/non-distinct on partition and non-partition columns +SELECT + count(distinct l_orderkey), count(l_orderkey), + count(distinct l_partkey), count(l_partkey), + count(distinct l_shipmode), count(l_shipmode) + FROM lineitem_hash; + count | count | count | count | count | count +-------+-------+-------+-------+-------+------- + 2985 | 12000 | 11661 | 12000 | 7 | 12000 +(1 row) + +-- mixed mode count distinct, grouped by non-partition column +SELECT + l_shipmode, count(distinct l_partkey), count(distinct l_orderkey) + FROM lineitem_hash + GROUP BY l_shipmode + ORDER BY 1, 2 DESC, 3 DESC; + l_shipmode | count | count +------------+-------+------- + AIR | 1702 | 1327 + FOB | 1700 | 1276 + MAIL | 1730 | 1299 + RAIL | 1696 | 1265 + REG AIR | 1676 | 1275 + SHIP | 1684 | 1289 + TRUCK | 1757 | 1333 +(7 rows) + +-- mixed mode count distinct, grouped by non-partition column +-- having on partition column +SELECT + l_shipmode, count(distinct l_partkey), count(distinct l_orderkey) + FROM lineitem_hash + GROUP BY l_shipmode + HAVING count(distinct l_orderkey) > 1300 + ORDER BY 1, 2 DESC; + l_shipmode | count | count +------------+-------+------- + AIR | 1702 | 1327 + TRUCK | 1757 | 1333 +(2 rows) + +-- same but having clause is not on target list +SELECT + l_shipmode, count(distinct l_partkey) + FROM lineitem_hash + GROUP BY l_shipmode + HAVING count(distinct l_orderkey) > 1300 + ORDER BY 1, 2 DESC; + l_shipmode | count +------------+------- + AIR | 1702 + TRUCK | 1757 +(2 rows) + +-- mixed mode count distinct, grouped by non-partition column +-- having on non-partition column +SELECT + l_shipmode, count(distinct l_partkey), count(distinct l_suppkey) + FROM lineitem_hash + GROUP BY l_shipmode + HAVING count(distinct l_suppkey) > 1550 + ORDER BY 1, 2 DESC; + l_shipmode | count | count +------------+-------+------- + AIR | 1702 | 1564 + FOB | 1700 | 1571 + MAIL | 1730 | 1573 + RAIL | 1696 | 1581 + REG AIR | 1676 | 1557 + SHIP | 1684 | 1554 + TRUCK | 1757 | 1602 +(7 rows) + +-- same but having clause is not on target list +SELECT + l_shipmode, count(distinct l_partkey) + FROM lineitem_hash + GROUP BY l_shipmode + HAVING count(distinct l_suppkey) > 1550 + ORDER BY 1, 2 DESC; + l_shipmode | count +------------+------- + AIR | 1702 + FOB | 1700 + MAIL | 1730 + RAIL | 1696 + REG AIR | 1676 + SHIP | 1684 + TRUCK | 1757 +(7 rows) + +EXPLAIN (COSTS false, VERBOSE true) +SELECT + l_shipmode, count(distinct l_partkey) + FROM lineitem_hash + GROUP BY l_shipmode + HAVING count(distinct l_suppkey) > 1550 + ORDER BY 1, 2 DESC; + QUERY PLAN +---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- + Sort + Output: remote_scan.l_shipmode, count(DISTINCT (count(DISTINCT remote_scan.count))) + Sort Key: remote_scan.l_shipmode, count(DISTINCT (count(DISTINCT remote_scan.count))) DESC + -> GroupAggregate + Output: remote_scan.l_shipmode, count(DISTINCT remote_scan.count) + Group Key: remote_scan.l_shipmode + Filter: (count(DISTINCT remote_scan.worker_column_3) > 1550) + -> Sort + Output: remote_scan.l_shipmode, remote_scan.count, remote_scan.worker_column_3 + Sort Key: remote_scan.l_shipmode + -> Custom Scan (Citus Task-Tracker) + Output: remote_scan.l_shipmode, remote_scan.count, remote_scan.worker_column_3 + Task Count: 8 + Tasks Shown: One of 8 + -> Task + Node: host=localhost port=57637 dbname=regression + -> HashAggregate + Output: l_shipmode, l_partkey, l_suppkey + Group Key: lineitem_hash.l_shipmode, lineitem_hash.l_partkey, lineitem_hash.l_suppkey + -> Seq Scan on public.lineitem_hash_240000 lineitem_hash + Output: l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment +(21 rows) + +-- count distinct is supported on single table subqueries +SELECT * + FROM ( + SELECT + l_orderkey, count(DISTINCT l_partkey) + FROM lineitem_hash + GROUP BY l_orderkey) sub + ORDER BY 2 DESC, 1 DESC + LIMIT 10; + l_orderkey | count +------------+------- + 14885 | 7 + 14884 | 7 + 14821 | 7 + 14790 | 7 + 14785 | 7 + 14755 | 7 + 14725 | 7 + 14694 | 7 + 14627 | 7 + 14624 | 7 +(10 rows) + +SELECT * + FROM ( + SELECT + l_partkey, count(DISTINCT l_orderkey) + FROM lineitem_hash + GROUP BY l_partkey) sub + ORDER BY 2 DESC, 1 DESC + LIMIT 10; + l_partkey | count +-----------+------- + 199146 | 3 + 188804 | 3 + 177771 | 3 + 160895 | 3 + 149926 | 3 + 136884 | 3 + 87761 | 3 + 15283 | 3 + 6983 | 3 + 1927 | 3 +(10 rows) + +EXPLAIN (COSTS false, VERBOSE true) +SELECT * + FROM ( + SELECT + l_partkey, count(DISTINCT l_orderkey) + FROM lineitem_hash + GROUP BY l_partkey) sub + ORDER BY 2 DESC, 1 DESC + LIMIT 10; + QUERY PLAN +------------------------------------------------------------------------- + Limit + Output: remote_scan.l_partkey, remote_scan.count + -> Sort + Output: remote_scan.l_partkey, remote_scan.count + Sort Key: remote_scan.count DESC, remote_scan.l_partkey DESC + -> Custom Scan (Citus Task-Tracker) + Output: remote_scan.l_partkey, remote_scan.count + Task Count: 4 + Tasks Shown: None, not supported for re-partition queries + -> MapMergeJob + Map Task Count: 8 + Merge Task Count: 4 +(12 rows) + +-- count distinct with filters +SELECT + l_orderkey, + count(DISTINCT l_suppkey) FILTER (WHERE l_shipmode = 'AIR'), + count(DISTINCT l_suppkey) + FROM lineitem_hash + GROUP BY l_orderkey + ORDER BY 2 DESC, 3 DESC, 1 + LIMIT 10; + l_orderkey | count | count +------------+-------+------- + 4964 | 4 | 7 + 12005 | 4 | 7 + 5409 | 4 | 6 + 164 | 3 | 7 + 322 | 3 | 7 + 871 | 3 | 7 + 1156 | 3 | 7 + 1574 | 3 | 7 + 2054 | 3 | 7 + 2309 | 3 | 7 +(10 rows) + +EXPLAIN (COSTS false, VERBOSE true) +SELECT + l_orderkey, + count(DISTINCT l_suppkey) FILTER (WHERE l_shipmode = 'AIR'), + count(DISTINCT l_suppkey) + FROM lineitem_hash + GROUP BY l_orderkey + ORDER BY 2 DESC, 3 DESC, 1 + LIMIT 10; + QUERY PLAN +---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- + Limit + Output: remote_scan.l_orderkey, COALESCE((pg_catalog.sum((COALESCE((pg_catalog.sum((COALESCE((pg_catalog.sum(remote_scan.count))::bigint, '0'::bigint))))::bigint, '0'::bigint))))::bigint, '0'::bigint), COALESCE((pg_catalog.sum((COALESCE((pg_catalog.sum((COALESCE((pg_catalog.sum(remote_scan.count_1))::bigint, '0'::bigint))))::bigint, '0'::bigint))))::bigint, '0'::bigint) + -> Sort + Output: remote_scan.l_orderkey, COALESCE((pg_catalog.sum((COALESCE((pg_catalog.sum(remote_scan.count))::bigint, '0'::bigint))))::bigint, '0'::bigint), COALESCE((pg_catalog.sum((COALESCE((pg_catalog.sum(remote_scan.count_1))::bigint, '0'::bigint))))::bigint, '0'::bigint) + Sort Key: COALESCE((pg_catalog.sum((COALESCE((pg_catalog.sum(remote_scan.count))::bigint, '0'::bigint))))::bigint, '0'::bigint) DESC, COALESCE((pg_catalog.sum((COALESCE((pg_catalog.sum(remote_scan.count_1))::bigint, '0'::bigint))))::bigint, '0'::bigint) DESC, remote_scan.l_orderkey + -> HashAggregate + Output: remote_scan.l_orderkey, COALESCE((pg_catalog.sum(remote_scan.count))::bigint, '0'::bigint), COALESCE((pg_catalog.sum(remote_scan.count_1))::bigint, '0'::bigint) + Group Key: remote_scan.l_orderkey + -> Custom Scan (Citus Task-Tracker) + Output: remote_scan.l_orderkey, remote_scan.count, remote_scan.count_1 + Task Count: 8 + Tasks Shown: One of 8 + -> Task + Node: host=localhost port=57637 dbname=regression + -> Limit + Output: l_orderkey, (count(DISTINCT l_suppkey) FILTER (WHERE (l_shipmode = 'AIR'::bpchar))), (count(DISTINCT l_suppkey)) + -> Sort + Output: l_orderkey, (count(DISTINCT l_suppkey) FILTER (WHERE (l_shipmode = 'AIR'::bpchar))), (count(DISTINCT l_suppkey)) + Sort Key: (count(DISTINCT lineitem_hash.l_suppkey) FILTER (WHERE (lineitem_hash.l_shipmode = 'AIR'::bpchar))) DESC, (count(DISTINCT lineitem_hash.l_suppkey)) DESC, lineitem_hash.l_orderkey + -> GroupAggregate + Output: l_orderkey, count(DISTINCT l_suppkey) FILTER (WHERE (l_shipmode = 'AIR'::bpchar)), count(DISTINCT l_suppkey) + Group Key: lineitem_hash.l_orderkey + -> Index Scan using lineitem_hash_pkey_240000 on public.lineitem_hash_240000 lineitem_hash + Output: l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment +(24 rows) + +-- group by on non-partition column +SELECT + l_suppkey, count(DISTINCT l_partkey) FILTER (WHERE l_shipmode = 'AIR') + FROM lineitem_hash + GROUP BY l_suppkey + ORDER BY 2 DESC, 1 DESC + LIMIT 10; + l_suppkey | count +-----------+------- + 7680 | 4 + 7703 | 3 + 7542 | 3 + 7072 | 3 + 6335 | 3 + 5873 | 3 + 1318 | 3 + 1042 | 3 + 160 | 3 + 9872 | 2 +(10 rows) + +-- explaining the same query fails +EXPLAIN (COSTS false, VERBOSE true) +SELECT + l_suppkey, count(DISTINCT l_partkey) FILTER (WHERE l_shipmode = 'AIR') + FROM lineitem_hash + GROUP BY l_suppkey + ORDER BY 2 DESC, 1 DESC + LIMIT 10; +ERROR: bogus varattno for OUTER_VAR var: 3 +-- without group by, on partition column +SELECT + count(DISTINCT l_orderkey) FILTER (WHERE l_shipmode = 'AIR') + FROM lineitem_hash; + count +------- + 1327 +(1 row) + +-- without group by, on non-partition column +SELECT + count(DISTINCT l_partkey) FILTER (WHERE l_shipmode = 'AIR') + FROM lineitem_hash; + count +------- + 1702 +(1 row) + +SELECT + count(DISTINCT l_partkey) FILTER (WHERE l_shipmode = 'AIR'), + count(DISTINCT l_partkey), + count(DISTINCT l_shipdate) + FROM lineitem_hash; + count | count | count +-------+-------+------- + 1702 | 11661 | 2470 +(1 row) + +-- filter column already exists in target list +SELECT * + FROM ( + SELECT + l_orderkey, count(DISTINCT l_partkey) FILTER (WHERE l_orderkey > 100) + FROM lineitem_hash + GROUP BY l_orderkey) sub + ORDER BY 2 DESC, 1 DESC + LIMIT 10; + l_orderkey | count +------------+------- + 14885 | 7 + 14884 | 7 + 14821 | 7 + 14790 | 7 + 14785 | 7 + 14755 | 7 + 14725 | 7 + 14694 | 7 + 14627 | 7 + 14624 | 7 +(10 rows) + +-- filter column does not exist in target list +SELECT * + FROM ( + SELECT + l_orderkey, count(DISTINCT l_partkey) FILTER (WHERE l_shipmode = 'AIR') + FROM lineitem_hash + GROUP BY l_orderkey) sub + ORDER BY 2 DESC, 1 DESC + LIMIT 10; + l_orderkey | count +------------+------- + 12005 | 4 + 5409 | 4 + 4964 | 4 + 14848 | 3 + 14496 | 3 + 13473 | 3 + 13122 | 3 + 12929 | 3 + 12645 | 3 + 12417 | 3 +(10 rows) + +-- case expr in count distinct is supported. +-- count orders partkeys if l_shipmode is air +SELECT * + FROM ( + SELECT + l_orderkey, count(DISTINCT CASE WHEN l_shipmode = 'AIR' THEN l_partkey ELSE NULL END) as count + FROM lineitem_hash + GROUP BY l_orderkey) sub + WHERE count > 0 + ORDER BY 2 DESC, 1 DESC + LIMIT 10; + l_orderkey | count +------------+------- + 12005 | 4 + 5409 | 4 + 4964 | 4 + 14848 | 3 + 14496 | 3 + 13473 | 3 + 13122 | 3 + 12929 | 3 + 12645 | 3 + 12417 | 3 +(10 rows) + +-- text like operator is also supported +SELECT * + FROM ( + SELECT + l_orderkey, count(DISTINCT CASE WHEN l_shipmode like '%A%' THEN l_partkey ELSE NULL END) as count + FROM lineitem_hash + GROUP BY l_orderkey) sub + WHERE count > 0 + ORDER BY 2 DESC, 1 DESC + LIMIT 10; + l_orderkey | count +------------+------- + 14275 | 7 + 14181 | 7 + 13605 | 7 + 12707 | 7 + 12384 | 7 + 11746 | 7 + 10727 | 7 + 10467 | 7 + 5636 | 7 + 4614 | 7 +(10 rows) + +-- count distinct is rejected if it does not reference any columns +SELECT * + FROM ( + SELECT + l_linenumber, count(DISTINCT 1) + FROM lineitem_hash + GROUP BY l_linenumber) sub + ORDER BY 2 DESC, 1 DESC + LIMIT 10; +ERROR: cannot compute aggregate (distinct) +DETAIL: aggregate (distinct) with no columns is unsupported +HINT: You can load the hll extension from contrib packages and enable distinct approximations. +-- count distinct is rejected if it does not reference any columns +SELECT * + FROM ( + SELECT + l_linenumber, count(DISTINCT (random() * 5)::int) + FROM lineitem_hash + GROUP BY l_linenumber) sub + ORDER BY 2 DESC, 1 DESC + LIMIT 10; +ERROR: cannot compute aggregate (distinct) +DETAIL: aggregate (distinct) with no columns is unsupported +HINT: You can load the hll extension from contrib packages and enable distinct approximations. +-- even non-const function calls are supported within count distinct +SELECT * + FROM ( + SELECT + l_orderkey, count(DISTINCT (random() * 5)::int = l_linenumber) + FROM lineitem_hash + GROUP BY l_orderkey) sub + ORDER BY 2 DESC, 1 DESC + LIMIT 0; + l_orderkey | count +------------+------- +(0 rows) + +-- multiple nested subquery +SELECT + total, + avg(avg_count) as total_avg_count + FROM ( + SELECT + number_sum, + count(DISTINCT l_suppkey) as total, + avg(total_count) avg_count + FROM ( + SELECT + l_suppkey, + sum(l_linenumber) as number_sum, + count(DISTINCT l_shipmode) as total_count + FROM + lineitem_hash + WHERE + l_partkey > 100 and + l_quantity > 2 and + l_orderkey < 10000 + GROUP BY + l_suppkey) as distributed_table + WHERE + number_sum >= 10 + GROUP BY + number_sum) as distributed_table_2 + GROUP BY + total + ORDER BY + total_avg_count DESC; + total | total_avg_count +-------+-------------------- + 1 | 3.6000000000000000 + 6 | 2.8333333333333333 + 10 | 2.6000000000000000 + 27 | 2.5555555555555556 + 32 | 2.4687500000000000 + 77 | 2.1948051948051948 + 57 | 2.1754385964912281 +(7 rows) + +-- multiple cases query +SELECT * + FROM ( + SELECT + count(DISTINCT + CASE + WHEN l_shipmode = 'TRUCK' THEN l_partkey + WHEN l_shipmode = 'AIR' THEN l_quantity + WHEN l_shipmode = 'SHIP' THEN l_discount + ELSE l_suppkey + END) as count, + l_shipdate + FROM + lineitem_hash + GROUP BY + l_shipdate) sub + WHERE + count > 0 + ORDER BY + 1 DESC, 2 DESC + LIMIT 10; + count | l_shipdate +-------+------------ + 14 | 07-30-1997 + 13 | 05-26-1998 + 13 | 08-08-1997 + 13 | 11-17-1995 + 13 | 01-09-1993 + 12 | 01-15-1998 + 12 | 10-15-1997 + 12 | 09-07-1997 + 12 | 06-02-1997 + 12 | 03-14-1997 +(10 rows) + +-- count DISTINCT expression +SELECT * + FROM ( + SELECT + l_quantity, count(DISTINCT ((l_orderkey / 1000) * 1000 )) as count + FROM + lineitem_hash + GROUP BY + l_quantity) sub + WHERE + count > 0 + ORDER BY + 2 DESC, 1 DESC + LIMIT 10; + l_quantity | count +------------+------- + 48.00 | 13 + 47.00 | 13 + 37.00 | 13 + 33.00 | 13 + 26.00 | 13 + 25.00 | 13 + 23.00 | 13 + 21.00 | 13 + 15.00 | 13 + 12.00 | 13 +(10 rows) + +-- count DISTINCT is part of an expression which inclues another aggregate +SELECT * + FROM ( + SELECT + sum(((l_partkey * l_tax) / 100)) / + count(DISTINCT + CASE + WHEN l_shipmode = 'TRUCK' THEN l_partkey + ELSE l_suppkey + END) as avg, + l_shipmode + FROM + lineitem_hash + GROUP BY + l_shipmode) sub + ORDER BY + 1 DESC, 2 DESC + LIMIT 10; + avg | l_shipmode +-------------------------+------------ + 44.82904609027336300064 | MAIL + 44.80704536679536679537 | SHIP + 44.68891732736572890026 | AIR + 44.34106724470134874759 | REG AIR + 43.12739987269255251432 | FOB + 43.07299253636938646426 | RAIL + 40.50298377916903813318 | TRUCK +(7 rows) + +--- count DISTINCT CASE WHEN expression +SELECT * + FROM ( + SELECT + count(DISTINCT + CASE + WHEN l_shipmode = 'TRUCK' THEN l_linenumber + WHEN l_shipmode = 'AIR' THEN l_linenumber + 10 + ELSE 2 + END) as avg + FROM + lineitem_hash + GROUP BY l_shipdate) sub + ORDER BY 1 DESC + LIMIT 10; + avg +----- + 7 + 6 + 6 + 6 + 6 + 6 + 6 + 6 + 5 + 5 +(10 rows) + +-- COUNT DISTINCT (c1, c2) +SELECT * + FROM + (SELECT + l_shipmode, + count(DISTINCT (l_shipdate, l_tax)) + FROM + lineitem_hash + GROUP BY + l_shipmode) t + ORDER BY + 2 DESC,1 DESC + LIMIT 10; + l_shipmode | count +------------+------- + TRUCK | 1689 + MAIL | 1683 + FOB | 1655 + AIR | 1650 + SHIP | 1644 + RAIL | 1636 + REG AIR | 1607 +(7 rows) + +-- distinct on non-var (type cast/field select) columns are also +-- supported if grouped on distribution column +-- random is added to prevent flattening by postgresql +SELECT + l_orderkey, count(a::int), count(distinct a::int) + FROM ( + SELECT l_orderkey, l_orderkey * 1.5 a, random() b + FROM lineitem_hash) sub + GROUP BY 1 + ORDER BY 1 DESC + LIMIT 5; + l_orderkey | count | count +------------+-------+------- + 14947 | 2 | 1 + 14946 | 2 | 1 + 14945 | 6 | 1 + 14944 | 2 | 1 + 14919 | 1 | 1 +(5 rows) + +SELECT user_id, + count(sub.a::int), + count(DISTINCT sub.a::int), + count(DISTINCT (sub).a) +FROM + (SELECT user_id, + unnest(ARRAY[user_id * 1.5])a, + random() b + FROM users_table + ) sub +GROUP BY 1 +ORDER BY 1 DESC +LIMIT 5; + user_id | count | count | count +---------+-------+-------+------- + 6 | 11 | 1 | 1 + 5 | 27 | 1 | 1 + 4 | 24 | 1 | 1 + 3 | 18 | 1 | 1 + 2 | 19 | 1 | 1 +(5 rows) + +CREATE TYPE test_item AS +( + id INTEGER, + duration INTEGER +); +SELECT * FROM run_command_on_workers($$CREATE TYPE test_item AS +( + id INTEGER, + duration INTEGER +)$$) ORDER BY nodeport; + nodename | nodeport | success | result +-----------+----------+---------+------------- + localhost | 57637 | t | CREATE TYPE + localhost | 57638 | t | CREATE TYPE +(2 rows) + +CREATE TABLE test_count_distinct_array (key int, value int , value_arr test_item[]); +SELECT create_distributed_table('test_count_distinct_array', 'key'); + create_distributed_table +-------------------------- + +(1 row) + +INSERT INTO test_count_distinct_array SELECT i, i, ARRAY[(i,i)::test_item] FROM generate_Series(0, 1000) i; +SELECT + key, + count(DISTINCT value), + count(DISTINCT (item)."id"), + count(DISTINCT (item)."id" * 3) +FROM + ( + SELECT key, unnest(value_arr) as item, value FROM test_count_distinct_array + ) as sub +GROUP BY 1 +ORDER BY 1 DESC +LIMIT 5; + key | count | count | count +------+-------+-------+------- + 1000 | 1 | 1 | 1 + 999 | 1 | 1 | 1 + 998 | 1 | 1 | 1 + 997 | 1 | 1 | 1 + 996 | 1 | 1 | 1 +(5 rows) + +DROP TABLE test_count_distinct_array; +DROP TYPE test_item; +SELECT * FROM run_command_on_workers($$DROP TYPE test_item$$) ORDER BY nodeport; + nodename | nodeport | success | result +-----------+----------+---------+----------- + localhost | 57637 | t | DROP TYPE + localhost | 57638 | t | DROP TYPE +(2 rows) + +-- other distinct aggregate are not supported +SELECT * + FROM ( + SELECT + l_linenumber, sum(DISTINCT l_partkey) + FROM lineitem_hash + GROUP BY l_linenumber) sub + ORDER BY 2 DESC, 1 DESC + LIMIT 10; +ERROR: cannot compute aggregate (distinct) +DETAIL: Only count(distinct) aggregate is supported in subqueries +SELECT * + FROM ( + SELECT + l_linenumber, avg(DISTINCT l_partkey) + FROM lineitem_hash + GROUP BY l_linenumber) sub + ORDER BY 2 DESC, 1 DESC + LIMIT 10; +ERROR: cannot compute aggregate (distinct) +DETAIL: Only count(distinct) aggregate is supported in subqueries +-- whole row references, oid, and ctid are not supported in count distinct +-- test table does not have oid or ctid enabled, so tests for them are skipped +SELECT * + FROM ( + SELECT + l_linenumber, count(DISTINCT lineitem_hash) + FROM lineitem_hash + GROUP BY l_linenumber) sub + ORDER BY 2 DESC, 1 DESC + LIMIT 10; +ERROR: cannot compute count (distinct) +DETAIL: Non-column references are not supported yet +SELECT * + FROM ( + SELECT + l_linenumber, count(DISTINCT lineitem_hash.*) + FROM lineitem_hash + GROUP BY l_linenumber) sub + ORDER BY 2 DESC, 1 DESC + LIMIT 10; +ERROR: cannot compute count (distinct) +DETAIL: Non-column references are not supported yet +DROP TABLE lineitem_hash; diff --git a/src/test/regress/expected/multi_join_order_additional.out b/src/test/regress/expected/multi_join_order_additional.out index 556ec639e..209a1f255 100644 --- a/src/test/regress/expected/multi_join_order_additional.out +++ b/src/test/regress/expected/multi_join_order_additional.out @@ -6,9 +6,9 @@ SET citus.next_shard_id TO 650000; SET citus.explain_distributed_queries TO off; SET citus.log_multi_join_order TO TRUE; SET citus.task_executor_type = 'task-tracker'; -- can't explain all queries otherwise -SET client_min_messages TO DEBUG2; SET citus.shard_count to 2; SET citus.shard_replication_factor to 1; +RESET client_min_messages; -- Create new table definitions for use in testing in distributed planning and -- execution functionality. Also create indexes to boost performance. CREATE TABLE lineitem_hash ( @@ -29,8 +29,6 @@ CREATE TABLE lineitem_hash ( l_shipmode char(10) not null, l_comment varchar(44) not null, PRIMARY KEY(l_orderkey, l_linenumber) ); -DEBUG: CREATE TABLE / PRIMARY KEY will create implicit index "lineitem_hash_pkey" for table "lineitem_hash" -DEBUG: building index "lineitem_hash_pkey" on table "lineitem_hash" SELECT create_distributed_table('lineitem_hash', 'l_orderkey'); create_distributed_table -------------------------- @@ -38,7 +36,6 @@ SELECT create_distributed_table('lineitem_hash', 'l_orderkey'); (1 row) CREATE INDEX lineitem_hash_time_index ON lineitem_hash (l_shipdate); -DEBUG: building index "lineitem_hash_time_index" on table "lineitem_hash" CREATE TABLE orders_hash ( o_orderkey bigint not null, o_custkey integer not null, @@ -50,8 +47,6 @@ CREATE TABLE orders_hash ( o_shippriority integer not null, o_comment varchar(79) not null, PRIMARY KEY(o_orderkey) ); -DEBUG: CREATE TABLE / PRIMARY KEY will create implicit index "orders_hash_pkey" for table "orders_hash" -DEBUG: building index "orders_hash_pkey" on table "orders_hash" SELECT create_distributed_table('orders_hash', 'o_orderkey'); create_distributed_table -------------------------- @@ -73,6 +68,7 @@ SELECT create_distributed_table('customer_hash', 'c_custkey'); (1 row) +SET client_min_messages TO DEBUG2; -- The following query checks that we can correctly handle self-joins EXPLAIN SELECT l1.l_quantity FROM lineitem l1, lineitem l2 WHERE l1.l_orderkey = l2.l_orderkey AND l1.l_quantity > 5; diff --git a/src/test/regress/expected/multi_multiuser.out b/src/test/regress/expected/multi_multiuser.out index d6ac26418..b8053a52b 100644 --- a/src/test/regress/expected/multi_multiuser.out +++ b/src/test/regress/expected/multi_multiuser.out @@ -3,6 +3,14 @@ -- -- Test user permissions. -- +-- print whether we're using version > 10 to make version-specific tests clear +SHOW server_version \gset +SELECT substring(:'server_version', '\d+')::int > 10 AS version_above_ten; + version_above_ten +------------------- + t +(1 row) + SET citus.next_shard_id TO 1420000; ALTER SEQUENCE pg_catalog.pg_dist_jobid_seq RESTART 1420000; SET citus.shard_replication_factor TO 1; @@ -110,7 +118,7 @@ HINT: Run the command with a superuser. -- check read permission SET ROLE read_access; EXECUTE prepare_insert(1); -ERROR: permission denied for relation test +ERROR: permission denied for table test EXECUTE prepare_select; count ------- @@ -118,7 +126,7 @@ EXECUTE prepare_select; (1 row) INSERT INTO test VALUES (2); -ERROR: permission denied for relation test +ERROR: permission denied for table test SELECT count(*) FROM test; count ------- @@ -153,21 +161,21 @@ SET citus.task_executor_type TO 'real-time'; -- check no permission SET ROLE no_access; EXECUTE prepare_insert(1); -ERROR: permission denied for relation test +ERROR: permission denied for table test EXECUTE prepare_select; -ERROR: permission denied for relation test +ERROR: permission denied for table test INSERT INTO test VALUES (2); -ERROR: permission denied for relation test +ERROR: permission denied for table test SELECT count(*) FROM test; -ERROR: permission denied for relation test +ERROR: permission denied for table test SELECT count(*) FROM test WHERE id = 1; -ERROR: permission denied for relation test +ERROR: permission denied for table test SET citus.task_executor_type TO 'task-tracker'; SELECT count(*) FROM test; -ERROR: permission denied for relation test +ERROR: permission denied for table test -- test re-partition query SELECT count(*) FROM test a JOIN test b ON (a.val = b.val) WHERE a.id = 1 AND b.id = 2; -ERROR: permission denied for relation test +ERROR: permission denied for table test -- should not be able to transmit directly COPY "postgresql.conf" TO STDOUT WITH (format transmit); ERROR: operation is not allowed @@ -200,7 +208,7 @@ END; -- as long as we don't read from a table BEGIN; SELECT create_intermediate_result('topten', 'SELECT count(*) FROM test'); -ERROR: permission denied for relation test +ERROR: permission denied for table test ABORT; RESET ROLE; DROP TABLE test; diff --git a/src/test/regress/expected/multi_multiuser_0.out b/src/test/regress/expected/multi_multiuser_0.out new file mode 100644 index 000000000..06613b2e0 --- /dev/null +++ b/src/test/regress/expected/multi_multiuser_0.out @@ -0,0 +1,217 @@ +-- +-- MULTI_MULTIUSERS +-- +-- Test user permissions. +-- +-- print whether we're using version > 10 to make version-specific tests clear +SHOW server_version \gset +SELECT substring(:'server_version', '\d+')::int > 10 AS version_above_ten; + version_above_ten +------------------- + f +(1 row) + +SET citus.next_shard_id TO 1420000; +ALTER SEQUENCE pg_catalog.pg_dist_jobid_seq RESTART 1420000; +SET citus.shard_replication_factor TO 1; +CREATE TABLE test (id integer, val integer); +SELECT create_distributed_table('test', 'id'); + create_distributed_table +-------------------------- + +(1 row) + +-- turn off propagation to avoid Enterprise processing the following section +SET citus.enable_ddl_propagation TO off; +CREATE USER full_access; +NOTICE: not propagating CREATE ROLE/USER commands to worker nodes +HINT: Connect to worker nodes directly to manually create all necessary users and roles. +CREATE USER read_access; +NOTICE: not propagating CREATE ROLE/USER commands to worker nodes +HINT: Connect to worker nodes directly to manually create all necessary users and roles. +CREATE USER no_access; +NOTICE: not propagating CREATE ROLE/USER commands to worker nodes +HINT: Connect to worker nodes directly to manually create all necessary users and roles. +GRANT ALL ON TABLE test TO full_access; +GRANT SELECT ON TABLE test TO read_access; +SET citus.enable_ddl_propagation TO DEFAULT; +\c - - - :worker_1_port +CREATE USER full_access; +NOTICE: not propagating CREATE ROLE/USER commands to worker nodes +HINT: Connect to worker nodes directly to manually create all necessary users and roles. +CREATE USER read_access; +NOTICE: not propagating CREATE ROLE/USER commands to worker nodes +HINT: Connect to worker nodes directly to manually create all necessary users and roles. +CREATE USER no_access; +NOTICE: not propagating CREATE ROLE/USER commands to worker nodes +HINT: Connect to worker nodes directly to manually create all necessary users and roles. +GRANT ALL ON TABLE test_1420000 TO full_access; +GRANT SELECT ON TABLE test_1420000 TO read_access; +GRANT ALL ON TABLE test_1420002 TO full_access; +GRANT SELECT ON TABLE test_1420002 TO read_access; +\c - - - :worker_2_port +CREATE USER full_access; +NOTICE: not propagating CREATE ROLE/USER commands to worker nodes +HINT: Connect to worker nodes directly to manually create all necessary users and roles. +CREATE USER read_access; +NOTICE: not propagating CREATE ROLE/USER commands to worker nodes +HINT: Connect to worker nodes directly to manually create all necessary users and roles. +CREATE USER no_access; +NOTICE: not propagating CREATE ROLE/USER commands to worker nodes +HINT: Connect to worker nodes directly to manually create all necessary users and roles. +GRANT ALL ON TABLE test_1420001 TO full_access; +GRANT SELECT ON TABLE test_1420001 TO read_access; +GRANT ALL ON TABLE test_1420003 TO full_access; +GRANT SELECT ON TABLE test_1420003 TO read_access; +\c - - - :master_port +-- create prepare tests +PREPARE prepare_insert AS INSERT INTO test VALUES ($1); +PREPARE prepare_select AS SELECT count(*) FROM test; +-- not allowed to read absolute paths, even as superuser +COPY "/etc/passwd" TO STDOUT WITH (format transmit); +ERROR: absolute path not allowed +-- check full permission +SET ROLE full_access; +EXECUTE prepare_insert(1); +EXECUTE prepare_select; + count +------- + 1 +(1 row) + +INSERT INTO test VALUES (2); +SELECT count(*) FROM test; + count +------- + 2 +(1 row) + +SELECT count(*) FROM test WHERE id = 1; + count +------- + 1 +(1 row) + +SET citus.task_executor_type TO 'task-tracker'; +SELECT count(*) FROM test; + count +------- + 2 +(1 row) + +-- test re-partition query (needs to transmit intermediate results) +SELECT count(*) FROM test a JOIN test b ON (a.val = b.val) WHERE a.id = 1 AND b.id = 2; + count +------- + 0 +(1 row) + +-- should not be able to transmit directly +COPY "postgresql.conf" TO STDOUT WITH (format transmit); +ERROR: operation is not allowed +HINT: Run the command with a superuser. +SET citus.task_executor_type TO 'real-time'; +-- should not be able to transmit directly +COPY "postgresql.conf" TO STDOUT WITH (format transmit); +ERROR: operation is not allowed +HINT: Run the command with a superuser. +-- check read permission +SET ROLE read_access; +EXECUTE prepare_insert(1); +ERROR: permission denied for relation test +EXECUTE prepare_select; + count +------- + 2 +(1 row) + +INSERT INTO test VALUES (2); +ERROR: permission denied for relation test +SELECT count(*) FROM test; + count +------- + 2 +(1 row) + +SELECT count(*) FROM test WHERE id = 1; + count +------- + 1 +(1 row) + +SET citus.task_executor_type TO 'task-tracker'; +SELECT count(*) FROM test; + count +------- + 2 +(1 row) + +-- test re-partition query (needs to transmit intermediate results) +SELECT count(*) FROM test a JOIN test b ON (a.val = b.val) WHERE a.id = 1 AND b.id = 2; + count +------- + 0 +(1 row) + +-- should not be able to transmit directly +COPY "postgresql.conf" TO STDOUT WITH (format transmit); +ERROR: operation is not allowed +HINT: Run the command with a superuser. +SET citus.task_executor_type TO 'real-time'; +-- check no permission +SET ROLE no_access; +EXECUTE prepare_insert(1); +ERROR: permission denied for relation test +EXECUTE prepare_select; +ERROR: permission denied for relation test +INSERT INTO test VALUES (2); +ERROR: permission denied for relation test +SELECT count(*) FROM test; +ERROR: permission denied for relation test +SELECT count(*) FROM test WHERE id = 1; +ERROR: permission denied for relation test +SET citus.task_executor_type TO 'task-tracker'; +SELECT count(*) FROM test; +ERROR: permission denied for relation test +-- test re-partition query +SELECT count(*) FROM test a JOIN test b ON (a.val = b.val) WHERE a.id = 1 AND b.id = 2; +ERROR: permission denied for relation test +-- should not be able to transmit directly +COPY "postgresql.conf" TO STDOUT WITH (format transmit); +ERROR: operation is not allowed +HINT: Run the command with a superuser. +SET citus.task_executor_type TO 'real-time'; +-- should be able to use intermediate results as any user +BEGIN; +SELECT create_intermediate_result('topten', 'SELECT s FROM generate_series(1,10) s'); + create_intermediate_result +---------------------------- + 10 +(1 row) + +SELECT * FROM read_intermediate_result('topten', 'binary'::citus_copy_format) AS res (s int) ORDER BY s; + s +---- + 1 + 2 + 3 + 4 + 5 + 6 + 7 + 8 + 9 + 10 +(10 rows) + +END; +-- as long as we don't read from a table +BEGIN; +SELECT create_intermediate_result('topten', 'SELECT count(*) FROM test'); +ERROR: permission denied for relation test +ABORT; +RESET ROLE; +DROP TABLE test; +DROP USER full_access; +DROP USER read_access; +DROP USER no_access; diff --git a/src/test/regress/expected/multi_mx_ddl.out b/src/test/regress/expected/multi_mx_ddl.out index 9f2249275..fbfb0c160 100644 --- a/src/test/regress/expected/multi_mx_ddl.out +++ b/src/test/regress/expected/multi_mx_ddl.out @@ -240,10 +240,7 @@ SELECT "Column", "Type", "Modifiers" FROM table_desc WHERE relid='mx_ddl_table_1 -- Show that DDL commands are done within a two-phase commit transaction \c - - - :master_port -SET client_min_messages TO debug2; CREATE INDEX ddl_test_index ON mx_ddl_table(value); -DEBUG: building index "ddl_test_index" on table "mx_ddl_table" -RESET client_min_messages; DROP INDEX ddl_test_index; -- show that sequences owned by mx tables result in unique values SET citus.shard_replication_factor TO 1; diff --git a/src/test/regress/expected/multi_null_minmax_value_pruning.out b/src/test/regress/expected/multi_null_minmax_value_pruning.out index 1b18c3803..a409530d8 100644 --- a/src/test/regress/expected/multi_null_minmax_value_pruning.out +++ b/src/test/regress/expected/multi_null_minmax_value_pruning.out @@ -4,12 +4,12 @@ -- This test checks that we can handle null min/max values in shard statistics -- and that we don't partition or join prune shards that have null values. SET citus.next_shard_id TO 760000; --- print whether we're using version > 9 to make version-specific tests clear +-- print major version number for version-specific tests SHOW server_version \gset -SELECT substring(:'server_version', '\d+')::int > 9 AS version_above_nine; - version_above_nine --------------------- - t +SELECT substring(:'server_version', '\d+')::int AS server_version; + server_version +---------------- + 11 (1 row) SET client_min_messages TO DEBUG2; diff --git a/src/test/regress/expected/multi_null_minmax_value_pruning_0.out b/src/test/regress/expected/multi_null_minmax_value_pruning_0.out index 7a2df60d0..ee23db50d 100644 --- a/src/test/regress/expected/multi_null_minmax_value_pruning_0.out +++ b/src/test/regress/expected/multi_null_minmax_value_pruning_0.out @@ -4,12 +4,12 @@ -- This test checks that we can handle null min/max values in shard statistics -- and that we don't partition or join prune shards that have null values. SET citus.next_shard_id TO 760000; --- print whether we're using version > 9 to make version-specific tests clear +-- print major version number for version-specific tests SHOW server_version \gset -SELECT substring(:'server_version', '\d+')::int > 9 AS version_above_nine; - version_above_nine --------------------- - f +SELECT substring(:'server_version', '\d+')::int AS server_version; + server_version +---------------- + 10 (1 row) SET client_min_messages TO DEBUG2; @@ -69,8 +69,8 @@ SELECT sum(l_linenumber), avg(l_linenumber) FROM lineitem, orders LOG: join order: [ "lineitem" ][ local partition join "orders" ] DEBUG: join prunable for intervals [1,5986] and [8997,14947] DEBUG: join prunable for intervals [8997,14947] and [1,5986] - QUERY PLAN ------------------------------------------------------------------------------------------------------- + QUERY PLAN +-------------------------------------------------------------------------------- Aggregate -> Custom Scan (Citus Real-Time) Task Count: 2 @@ -78,18 +78,20 @@ DEBUG: join prunable for intervals [8997,14947] and [1,5986] -> Task Node: host=localhost port=57637 dbname=regression -> Aggregate - -> Merge Join - Merge Cond: (orders.o_orderkey = lineitem.l_orderkey) - -> Index Only Scan using orders_pkey_290003 on orders_290003 orders - -> Index Only Scan using lineitem_pkey_290001 on lineitem_290001 lineitem + -> Hash Join + Hash Cond: (lineitem.l_orderkey = orders.o_orderkey) + -> Seq Scan on lineitem_290001 lineitem + -> Hash + -> Seq Scan on orders_290003 orders -> Task Node: host=localhost port=57638 dbname=regression -> Aggregate - -> Merge Join - Merge Cond: (orders.o_orderkey = lineitem.l_orderkey) - -> Index Only Scan using orders_pkey_290002 on orders_290002 orders - -> Index Only Scan using lineitem_pkey_290000 on lineitem_290000 lineitem -(18 rows) + -> Hash Join + Hash Cond: (lineitem.l_orderkey = orders.o_orderkey) + -> Seq Scan on lineitem_290000 lineitem + -> Hash + -> Seq Scan on orders_290002 orders +(20 rows) -- Now set the minimum value for a shard to null. Then check that we don't apply -- partition or join pruning for the shard with null min value. Since it is not diff --git a/src/test/regress/expected/multi_null_minmax_value_pruning_1.out b/src/test/regress/expected/multi_null_minmax_value_pruning_1.out new file mode 100644 index 000000000..5cf7c70c9 --- /dev/null +++ b/src/test/regress/expected/multi_null_minmax_value_pruning_1.out @@ -0,0 +1,304 @@ +-- +-- MULTI_NULL_MINMAX_VALUE_PRUNING +-- +-- This test checks that we can handle null min/max values in shard statistics +-- and that we don't partition or join prune shards that have null values. +SET citus.next_shard_id TO 760000; +-- print major version number for version-specific tests +SHOW server_version \gset +SELECT substring(:'server_version', '\d+')::int AS server_version; + server_version +---------------- + 9 +(1 row) + +SET client_min_messages TO DEBUG2; +SET citus.explain_all_tasks TO on; +-- to avoid differing explain output - executor doesn't matter, +-- because were testing pruning here. +SET citus.task_executor_type TO 'real-time'; +-- Change configuration to treat lineitem and orders tables as large +SET citus.large_table_shard_count TO 2; +SET citus.log_multi_join_order to true; +SET citus.enable_repartition_joins to ON; +SELECT shardminvalue, shardmaxvalue from pg_dist_shard WHERE shardid = 290000; + shardminvalue | shardmaxvalue +---------------+--------------- + 1 | 5986 +(1 row) + +SELECT shardminvalue, shardmaxvalue from pg_dist_shard WHERE shardid = 290001; + shardminvalue | shardmaxvalue +---------------+--------------- + 8997 | 14947 +(1 row) + +-- Check that partition and join pruning works when min/max values exist +-- Adding l_orderkey = 1 to make the query not router executable +EXPLAIN (COSTS FALSE) +SELECT l_orderkey, l_linenumber, l_shipdate FROM lineitem WHERE l_orderkey = 9030 or l_orderkey = 1; +LOG: join order: [ "lineitem" ] + QUERY PLAN +----------------------------------------------------------------------- + Custom Scan (Citus Real-Time) + Task Count: 2 + Tasks Shown: All + -> Task + Node: host=localhost port=57637 dbname=regression + -> Bitmap Heap Scan on lineitem_290001 lineitem + Recheck Cond: ((l_orderkey = 9030) OR (l_orderkey = 1)) + -> BitmapOr + -> Bitmap Index Scan on lineitem_pkey_290001 + Index Cond: (l_orderkey = 9030) + -> Bitmap Index Scan on lineitem_pkey_290001 + Index Cond: (l_orderkey = 1) + -> Task + Node: host=localhost port=57638 dbname=regression + -> Bitmap Heap Scan on lineitem_290000 lineitem + Recheck Cond: ((l_orderkey = 9030) OR (l_orderkey = 1)) + -> BitmapOr + -> Bitmap Index Scan on lineitem_pkey_290000 + Index Cond: (l_orderkey = 9030) + -> Bitmap Index Scan on lineitem_pkey_290000 + Index Cond: (l_orderkey = 1) +(21 rows) + +EXPLAIN (COSTS FALSE) +SELECT sum(l_linenumber), avg(l_linenumber) FROM lineitem, orders + WHERE l_orderkey = o_orderkey; +LOG: join order: [ "lineitem" ][ local partition join "orders" ] +DEBUG: join prunable for intervals [1,5986] and [8997,14947] +DEBUG: join prunable for intervals [8997,14947] and [1,5986] + QUERY PLAN +------------------------------------------------------------------------------------------------------ + Aggregate + -> Custom Scan (Citus Real-Time) + Task Count: 2 + Tasks Shown: All + -> Task + Node: host=localhost port=57637 dbname=regression + -> Aggregate + -> Merge Join + Merge Cond: (orders.o_orderkey = lineitem.l_orderkey) + -> Index Only Scan using orders_pkey_290003 on orders_290003 orders + -> Index Only Scan using lineitem_pkey_290001 on lineitem_290001 lineitem + -> Task + Node: host=localhost port=57638 dbname=regression + -> Aggregate + -> Merge Join + Merge Cond: (orders.o_orderkey = lineitem.l_orderkey) + -> Index Only Scan using orders_pkey_290002 on orders_290002 orders + -> Index Only Scan using lineitem_pkey_290000 on lineitem_290000 lineitem +(18 rows) + +-- Now set the minimum value for a shard to null. Then check that we don't apply +-- partition or join pruning for the shard with null min value. Since it is not +-- supported with single-repartition join, dual-repartition has been used. +UPDATE pg_dist_shard SET shardminvalue = NULL WHERE shardid = 290000; +EXPLAIN (COSTS FALSE) +SELECT l_orderkey, l_linenumber, l_shipdate FROM lineitem WHERE l_orderkey = 9030; +LOG: join order: [ "lineitem" ] + QUERY PLAN +------------------------------------------------------------------------------- + Custom Scan (Citus Real-Time) + Task Count: 2 + Tasks Shown: All + -> Task + Node: host=localhost port=57637 dbname=regression + -> Index Scan using lineitem_pkey_290001 on lineitem_290001 lineitem + Index Cond: (l_orderkey = 9030) + -> Task + Node: host=localhost port=57638 dbname=regression + -> Index Scan using lineitem_pkey_290000 on lineitem_290000 lineitem + Index Cond: (l_orderkey = 9030) +(11 rows) + +EXPLAIN (COSTS FALSE) +SELECT sum(l_linenumber), avg(l_linenumber) FROM lineitem, orders + WHERE l_partkey = o_custkey; +LOG: join order: [ "lineitem" ][ dual partition join "orders" ] +DEBUG: join prunable for task partitionId 0 and 1 +DEBUG: join prunable for task partitionId 0 and 2 +DEBUG: join prunable for task partitionId 0 and 3 +DEBUG: join prunable for task partitionId 1 and 0 +DEBUG: join prunable for task partitionId 1 and 2 +DEBUG: join prunable for task partitionId 1 and 3 +DEBUG: join prunable for task partitionId 2 and 0 +DEBUG: join prunable for task partitionId 2 and 1 +DEBUG: join prunable for task partitionId 2 and 3 +DEBUG: join prunable for task partitionId 3 and 0 +DEBUG: join prunable for task partitionId 3 and 1 +DEBUG: join prunable for task partitionId 3 and 2 +DEBUG: pruning merge fetch taskId 1 +DETAIL: Creating dependency on merge taskId 3 +DEBUG: pruning merge fetch taskId 2 +DETAIL: Creating dependency on merge taskId 3 +DEBUG: pruning merge fetch taskId 4 +DETAIL: Creating dependency on merge taskId 6 +DEBUG: pruning merge fetch taskId 5 +DETAIL: Creating dependency on merge taskId 6 +DEBUG: pruning merge fetch taskId 7 +DETAIL: Creating dependency on merge taskId 9 +DEBUG: pruning merge fetch taskId 8 +DETAIL: Creating dependency on merge taskId 9 +DEBUG: pruning merge fetch taskId 10 +DETAIL: Creating dependency on merge taskId 12 +DEBUG: pruning merge fetch taskId 11 +DETAIL: Creating dependency on merge taskId 12 +DEBUG: cannot use real time executor with repartition jobs +HINT: Since you enabled citus.enable_repartition_joins Citus chose to use task-tracker. + QUERY PLAN +------------------------------------------------------------------- + Aggregate + -> Custom Scan (Citus Task-Tracker) + Task Count: 4 + Tasks Shown: None, not supported for re-partition queries + -> MapMergeJob + Map Task Count: 2 + Merge Task Count: 4 + -> MapMergeJob + Map Task Count: 2 + Merge Task Count: 4 +(10 rows) + +-- Next, set the maximum value for another shard to null. Then check that we +-- don't apply partition or join pruning for this other shard either. Since it +-- is not supported with single-repartition join, dual-repartition has been used. +UPDATE pg_dist_shard SET shardmaxvalue = NULL WHERE shardid = 290001; +EXPLAIN (COSTS FALSE) +SELECT l_orderkey, l_linenumber, l_shipdate FROM lineitem WHERE l_orderkey = 9030; +LOG: join order: [ "lineitem" ] + QUERY PLAN +------------------------------------------------------------------------------- + Custom Scan (Citus Real-Time) + Task Count: 2 + Tasks Shown: All + -> Task + Node: host=localhost port=57637 dbname=regression + -> Index Scan using lineitem_pkey_290001 on lineitem_290001 lineitem + Index Cond: (l_orderkey = 9030) + -> Task + Node: host=localhost port=57638 dbname=regression + -> Index Scan using lineitem_pkey_290000 on lineitem_290000 lineitem + Index Cond: (l_orderkey = 9030) +(11 rows) + +EXPLAIN (COSTS FALSE) +SELECT sum(l_linenumber), avg(l_linenumber) FROM lineitem, orders + WHERE l_partkey = o_custkey; +LOG: join order: [ "lineitem" ][ dual partition join "orders" ] +DEBUG: join prunable for task partitionId 0 and 1 +DEBUG: join prunable for task partitionId 0 and 2 +DEBUG: join prunable for task partitionId 0 and 3 +DEBUG: join prunable for task partitionId 1 and 0 +DEBUG: join prunable for task partitionId 1 and 2 +DEBUG: join prunable for task partitionId 1 and 3 +DEBUG: join prunable for task partitionId 2 and 0 +DEBUG: join prunable for task partitionId 2 and 1 +DEBUG: join prunable for task partitionId 2 and 3 +DEBUG: join prunable for task partitionId 3 and 0 +DEBUG: join prunable for task partitionId 3 and 1 +DEBUG: join prunable for task partitionId 3 and 2 +DEBUG: pruning merge fetch taskId 1 +DETAIL: Creating dependency on merge taskId 3 +DEBUG: pruning merge fetch taskId 2 +DETAIL: Creating dependency on merge taskId 3 +DEBUG: pruning merge fetch taskId 4 +DETAIL: Creating dependency on merge taskId 6 +DEBUG: pruning merge fetch taskId 5 +DETAIL: Creating dependency on merge taskId 6 +DEBUG: pruning merge fetch taskId 7 +DETAIL: Creating dependency on merge taskId 9 +DEBUG: pruning merge fetch taskId 8 +DETAIL: Creating dependency on merge taskId 9 +DEBUG: pruning merge fetch taskId 10 +DETAIL: Creating dependency on merge taskId 12 +DEBUG: pruning merge fetch taskId 11 +DETAIL: Creating dependency on merge taskId 12 +DEBUG: cannot use real time executor with repartition jobs +HINT: Since you enabled citus.enable_repartition_joins Citus chose to use task-tracker. + QUERY PLAN +------------------------------------------------------------------- + Aggregate + -> Custom Scan (Citus Task-Tracker) + Task Count: 4 + Tasks Shown: None, not supported for re-partition queries + -> MapMergeJob + Map Task Count: 2 + Merge Task Count: 4 + -> MapMergeJob + Map Task Count: 2 + Merge Task Count: 4 +(10 rows) + +-- Last, set the minimum value to 0 and check that we don't treat it as null. We +-- should apply partition and join pruning for this shard now. Since it is not +-- supported with single-repartition join, dual-repartition has been used. +UPDATE pg_dist_shard SET shardminvalue = '0' WHERE shardid = 290000; +EXPLAIN (COSTS FALSE) +SELECT l_orderkey, l_linenumber, l_shipdate FROM lineitem WHERE l_orderkey = 9030; +LOG: join order: [ "lineitem" ] +DEBUG: Plan is router executable + QUERY PLAN +------------------------------------------------------------------------------- + Custom Scan (Citus Router) + Task Count: 1 + Tasks Shown: All + -> Task + Node: host=localhost port=57637 dbname=regression + -> Index Scan using lineitem_pkey_290001 on lineitem_290001 lineitem + Index Cond: (l_orderkey = 9030) +(7 rows) + +EXPLAIN (COSTS FALSE) +SELECT sum(l_linenumber), avg(l_linenumber) FROM lineitem, orders + WHERE l_partkey = o_custkey; +LOG: join order: [ "lineitem" ][ dual partition join "orders" ] +DEBUG: join prunable for task partitionId 0 and 1 +DEBUG: join prunable for task partitionId 0 and 2 +DEBUG: join prunable for task partitionId 0 and 3 +DEBUG: join prunable for task partitionId 1 and 0 +DEBUG: join prunable for task partitionId 1 and 2 +DEBUG: join prunable for task partitionId 1 and 3 +DEBUG: join prunable for task partitionId 2 and 0 +DEBUG: join prunable for task partitionId 2 and 1 +DEBUG: join prunable for task partitionId 2 and 3 +DEBUG: join prunable for task partitionId 3 and 0 +DEBUG: join prunable for task partitionId 3 and 1 +DEBUG: join prunable for task partitionId 3 and 2 +DEBUG: pruning merge fetch taskId 1 +DETAIL: Creating dependency on merge taskId 3 +DEBUG: pruning merge fetch taskId 2 +DETAIL: Creating dependency on merge taskId 3 +DEBUG: pruning merge fetch taskId 4 +DETAIL: Creating dependency on merge taskId 6 +DEBUG: pruning merge fetch taskId 5 +DETAIL: Creating dependency on merge taskId 6 +DEBUG: pruning merge fetch taskId 7 +DETAIL: Creating dependency on merge taskId 9 +DEBUG: pruning merge fetch taskId 8 +DETAIL: Creating dependency on merge taskId 9 +DEBUG: pruning merge fetch taskId 10 +DETAIL: Creating dependency on merge taskId 12 +DEBUG: pruning merge fetch taskId 11 +DETAIL: Creating dependency on merge taskId 12 +DEBUG: cannot use real time executor with repartition jobs +HINT: Since you enabled citus.enable_repartition_joins Citus chose to use task-tracker. + QUERY PLAN +------------------------------------------------------------------- + Aggregate + -> Custom Scan (Citus Task-Tracker) + Task Count: 4 + Tasks Shown: None, not supported for re-partition queries + -> MapMergeJob + Map Task Count: 2 + Merge Task Count: 4 + -> MapMergeJob + Map Task Count: 2 + Merge Task Count: 4 +(10 rows) + +-- Set minimum and maximum values for two shards back to their original values +UPDATE pg_dist_shard SET shardminvalue = '1' WHERE shardid = 290000; +UPDATE pg_dist_shard SET shardmaxvalue = '14947' WHERE shardid = 290001; +SET client_min_messages TO NOTICE; diff --git a/src/test/regress/expected/multi_orderby_limit_pushdown.out b/src/test/regress/expected/multi_orderby_limit_pushdown.out index c68bcd5d9..0e6060655 100644 --- a/src/test/regress/expected/multi_orderby_limit_pushdown.out +++ b/src/test/regress/expected/multi_orderby_limit_pushdown.out @@ -181,30 +181,30 @@ LIMIT 2; 1 (2 rows) -EXPLAIN +EXPLAIN (COSTS OFF) SELECT user_id FROM users_table GROUP BY user_id ORDER BY (10000 / (sum(value_1 + value_2))) DESC LIMIT 2; - QUERY PLAN ---------------------------------------------------------------------------------------------------------------------------------- - Limit (cost=0.00..0.00 rows=0 width=0) - -> Sort (cost=0.00..0.00 rows=0 width=0) + QUERY PLAN +------------------------------------------------------------------------------------------------------------------------------ + Limit + -> Sort Sort Key: (10000 / (pg_catalog.sum(((10000 / (pg_catalog.sum(remote_scan.worker_column_2))::bigint))))::bigint) DESC - -> HashAggregate (cost=0.00..0.00 rows=0 width=0) + -> HashAggregate Group Key: remote_scan.user_id - -> Custom Scan (Citus Real-Time) (cost=0.00..0.00 rows=0 width=0) + -> Custom Scan (Citus Real-Time) Task Count: 4 Tasks Shown: One of 4 -> Task Node: host=localhost port=57637 dbname=regression - -> Limit (cost=1.78..1.78 rows=2 width=20) - -> Sort (cost=1.78..1.78 rows=2 width=20) + -> Limit + -> Sort Sort Key: ((10000 / sum((value_1 + value_2)))) DESC - -> HashAggregate (cost=1.74..1.77 rows=2 width=20) + -> HashAggregate Group Key: user_id - -> Seq Scan on users_table_1400000 users_table (cost=0.00..1.33 rows=33 width=12) + -> Seq Scan on users_table_1400000 users_table (16 rows) SELECT 10000 / (sum(value_1 + value_2)) @@ -249,30 +249,30 @@ LIMIT 2; 4 (2 rows) -EXPLAIN +EXPLAIN (COSTS OFF) SELECT user_id FROM users_table GROUP BY user_id ORDER BY sum(value_1) DESC LIMIT 2; - QUERY PLAN --------------------------------------------------------------------------------------------------------------------------------- - Limit (cost=0.00..0.00 rows=0 width=0) - -> Sort (cost=0.00..0.00 rows=0 width=0) + QUERY PLAN +---------------------------------------------------------------------------------------------------------- + Limit + -> Sort Sort Key: (pg_catalog.sum(((pg_catalog.sum(remote_scan.worker_column_2))::bigint)))::bigint DESC - -> HashAggregate (cost=0.00..0.00 rows=0 width=0) + -> HashAggregate Group Key: remote_scan.user_id - -> Custom Scan (Citus Real-Time) (cost=0.00..0.00 rows=0 width=0) + -> Custom Scan (Citus Real-Time) Task Count: 4 Tasks Shown: One of 4 -> Task Node: host=localhost port=57637 dbname=regression - -> Limit (cost=1.53..1.53 rows=2 width=12) - -> Sort (cost=1.53..1.53 rows=2 width=12) + -> Limit + -> Sort Sort Key: (sum(value_1)) DESC - -> HashAggregate (cost=1.50..1.52 rows=2 width=12) + -> HashAggregate Group Key: user_id - -> Seq Scan on users_table_1400000 users_table (cost=0.00..1.33 rows=33 width=8) + -> Seq Scan on users_table_1400000 users_table (16 rows) SELECT ut.user_id, avg(ut.value_2) @@ -290,7 +290,7 @@ LIMIT 5; 4 | 2.1739130434782609 (5 rows) -EXPLAIN +EXPLAIN (COSTS OFF) SELECT ut.user_id, avg(ut.value_2) FROM users_table ut, events_table et WHERE ut.user_id = et.user_id and et.value_2 < 5 @@ -325,7 +325,7 @@ LIMIT 2; 6 | 5 (2 rows) -EXPLAIN +EXPLAIN (COSTS OFF) SELECT ut.user_id, count(DISTINCT ut.value_2) FROM users_table ut, events_table et WHERE ut.user_id = et.user_id and et.value_2 < 5 diff --git a/src/test/regress/expected/multi_partitioning.out b/src/test/regress/expected/multi_partitioning.out index b18d70a5c..173755136 100644 --- a/src/test/regress/expected/multi_partitioning.out +++ b/src/test/regress/expected/multi_partitioning.out @@ -4,6 +4,14 @@ SET citus.next_shard_id TO 1660000; SET citus.shard_count TO 4; SET citus.shard_replication_factor TO 1; +-- print major version number for version-specific tests +SHOW server_version \gset +SELECT substring(:'server_version', '\d+')::int AS server_version; + server_version +---------------- + 11 +(1 row) + -- -- Distributed Partitioned Table Creation Tests -- @@ -299,8 +307,8 @@ SELECT * FROM partitioning_test WHERE id = 7 OR id = 8 ORDER BY 1; -- UPDATE that tries to move a row to a non-existing partition (this should fail) UPDATE partitioning_test SET time = '2020-07-07' WHERE id = 7; -ERROR: new row for relation "partitioning_test_2013_1660021" violates partition constraint -DETAIL: Failing row contains (7, 2020-07-07). +ERROR: no partition of relation "partitioning_test_1660001" found for row +DETAIL: Partition key of the failing row contains ("time") = (2020-07-07). CONTEXT: while executing command on localhost:57638 -- UPDATE with subqueries on partitioned table UPDATE @@ -402,19 +410,24 @@ CONTEXT: while executing command on localhost:57638 -- -- test CREATE INDEX -- CREATE INDEX on partitioned table - this will error out +-- on earlier versions of postgres earlier than 11. CREATE INDEX partitioning_index ON partitioning_test(id); -ERROR: cannot create index on partitioned table "partitioning_test" -- CREATE INDEX on partition CREATE INDEX partitioning_2009_index ON partitioning_test_2009(id); -- CREATE INDEX CONCURRENTLY on partition CREATE INDEX CONCURRENTLY partitioned_2010_index ON partitioning_test_2010(id); -- see index is created SELECT tablename, indexname FROM pg_indexes WHERE tablename LIKE 'partitioning_test%' ORDER BY indexname; - tablename | indexname -------------------------+------------------------- + tablename | indexname +------------------------+------------------------------- partitioning_test_2010 | partitioned_2010_index partitioning_test_2009 | partitioning_2009_index -(2 rows) + partitioning_test_2009 | partitioning_test_2009_id_idx + partitioning_test_2010 | partitioning_test_2010_id_idx + partitioning_test_2011 | partitioning_test_2011_id_idx + partitioning_test_2012 | partitioning_test_2012_id_idx + partitioning_test_2013 | partitioning_test_2013_id_idx +(7 rows) -- test add COLUMN -- add COLUMN to partitioned table @@ -442,9 +455,8 @@ SELECT name, type FROM table_attrs WHERE relid = 'partitioning_test_2010'::regcl -- test add PRIMARY KEY -- add PRIMARY KEY to partitioned table - this will error out ALTER TABLE partitioning_test ADD CONSTRAINT partitioning_primary PRIMARY KEY (id); -ERROR: primary key constraints are not supported on partitioned tables -LINE 1: ALTER TABLE partitioning_test ADD CONSTRAINT partitioning_pr... - ^ +ERROR: insufficient columns in PRIMARY KEY constraint definition +DETAIL: PRIMARY KEY constraint on table "partitioning_test" lacks column "time" which is part of the partition key. -- ADD PRIMARY KEY to partition ALTER TABLE partitioning_test_2009 ADD CONSTRAINT partitioning_2009_primary PRIMARY KEY (id); -- see PRIMARY KEY is created @@ -465,9 +477,7 @@ WHERE -- test ADD FOREIGN CONSTRAINT -- add FOREIGN CONSTRAINT to partitioned table -- this will error out ALTER TABLE partitioning_test ADD CONSTRAINT partitioning_foreign FOREIGN KEY (id) REFERENCES partitioning_test_2009 (id); -ERROR: foreign key constraints are not supported on partitioned tables -LINE 1: ALTER TABLE partitioning_test ADD CONSTRAINT partitioning_fo... - ^ +ERROR: cannot ALTER TABLE "partitioning_test_2009" because it is being used by active queries in this session -- add FOREIGN CONSTRAINT to partition INSERT INTO partitioning_test_2009 VALUES (5, '2009-06-06'); INSERT INTO partitioning_test_2009 VALUES (6, '2009-07-07'); diff --git a/src/test/regress/expected/multi_partitioning_0.out b/src/test/regress/expected/multi_partitioning_0.out index 9f18191ea..7fc346729 100644 --- a/src/test/regress/expected/multi_partitioning_0.out +++ b/src/test/regress/expected/multi_partitioning_0.out @@ -4,52 +4,48 @@ SET citus.next_shard_id TO 1660000; SET citus.shard_count TO 4; SET citus.shard_replication_factor TO 1; +-- print major version number for version-specific tests +SHOW server_version \gset +SELECT substring(:'server_version', '\d+')::int AS server_version; + server_version +---------------- + 10 +(1 row) + -- -- Distributed Partitioned Table Creation Tests -- -- 1-) Distributing partitioned table -- create partitioned table CREATE TABLE partitioning_test(id int, time date) PARTITION BY RANGE (time); -ERROR: syntax error at or near "PARTITION" -LINE 1: CREATE TABLE partitioning_test(id int, time date) PARTITION ... - ^ -- create its partitions CREATE TABLE partitioning_test_2009 PARTITION OF partitioning_test FOR VALUES FROM ('2009-01-01') TO ('2010-01-01'); -ERROR: syntax error at or near "PARTITION" -LINE 1: CREATE TABLE partitioning_test_2009 PARTITION OF partitionin... - ^ CREATE TABLE partitioning_test_2010 PARTITION OF partitioning_test FOR VALUES FROM ('2010-01-01') TO ('2011-01-01'); -ERROR: syntax error at or near "PARTITION" -LINE 1: CREATE TABLE partitioning_test_2010 PARTITION OF partitionin... - ^ -- load some data and distribute tables INSERT INTO partitioning_test VALUES (1, '2009-06-06'); -ERROR: relation "partitioning_test" does not exist -LINE 1: INSERT INTO partitioning_test VALUES (1, '2009-06-06'); - ^ INSERT INTO partitioning_test VALUES (2, '2010-07-07'); -ERROR: relation "partitioning_test" does not exist -LINE 1: INSERT INTO partitioning_test VALUES (2, '2010-07-07'); - ^ INSERT INTO partitioning_test_2009 VALUES (3, '2009-09-09'); -ERROR: relation "partitioning_test_2009" does not exist -LINE 1: INSERT INTO partitioning_test_2009 VALUES (3, '2009-09-09'); - ^ INSERT INTO partitioning_test_2010 VALUES (4, '2010-03-03'); -ERROR: relation "partitioning_test_2010" does not exist -LINE 1: INSERT INTO partitioning_test_2010 VALUES (4, '2010-03-03'); - ^ -- distribute partitioned table SELECT create_distributed_table('partitioning_test', 'id'); -ERROR: relation "partitioning_test" does not exist -LINE 1: SELECT create_distributed_table('partitioning_test', 'id'); - ^ +NOTICE: Copying data from local table... +NOTICE: Copying data from local table... + create_distributed_table +-------------------------- + +(1 row) + -- see the data is loaded to shards SELECT * FROM partitioning_test ORDER BY 1; -ERROR: relation "partitioning_test" does not exist -LINE 1: SELECT * FROM partitioning_test ORDER BY 1; - ^ + id | time +----+------------ + 1 | 06-06-2009 + 2 | 07-07-2010 + 3 | 09-09-2009 + 4 | 03-03-2010 +(4 rows) + -- see partitioned table and its partitions are distributed SELECT logicalrelid @@ -58,9 +54,13 @@ FROM WHERE logicalrelid IN ('partitioning_test', 'partitioning_test_2009', 'partitioning_test_2010') ORDER BY 1; -ERROR: relation "partitioning_test" does not exist -LINE 6: logicalrelid IN ('partitioning_test', 'partitioning_test_20... - ^ + logicalrelid +------------------------ + partitioning_test + partitioning_test_2009 + partitioning_test_2010 +(3 rows) + SELECT logicalrelid, count(*) FROM pg_dist_shard @@ -69,14 +69,15 @@ GROUP BY logicalrelid ORDER BY 1,2; -ERROR: relation "partitioning_test" does not exist -LINE 4: WHERE logicalrelid IN ('partitioning_test', 'partitioning_t... - ^ + logicalrelid | count +------------------------+------- + partitioning_test | 4 + partitioning_test_2009 | 4 + partitioning_test_2010 | 4 +(3 rows) + -- 2-) Creating partition of a distributed table CREATE TABLE partitioning_test_2011 PARTITION OF partitioning_test FOR VALUES FROM ('2011-01-01') TO ('2012-01-01'); -ERROR: syntax error at or near "PARTITION" -LINE 1: CREATE TABLE partitioning_test_2011 PARTITION OF partitionin... - ^ -- new partition is automatically distributed as well SELECT logicalrelid @@ -85,9 +86,12 @@ FROM WHERE logicalrelid IN ('partitioning_test', 'partitioning_test_2011') ORDER BY 1; -ERROR: relation "partitioning_test" does not exist -LINE 6: logicalrelid IN ('partitioning_test', 'partitioning_test_20... - ^ + logicalrelid +------------------------ + partitioning_test + partitioning_test_2011 +(2 rows) + SELECT logicalrelid, count(*) FROM pg_dist_shard @@ -96,18 +100,19 @@ GROUP BY logicalrelid ORDER BY 1,2; -ERROR: relation "partitioning_test" does not exist -LINE 4: WHERE logicalrelid IN ('partitioning_test', 'partitioning_t... - ^ + logicalrelid | count +------------------------+------- + partitioning_test | 4 + partitioning_test_2011 | 4 +(2 rows) + -- 3-) Attaching non distributed table to a distributed table CREATE TABLE partitioning_test_2012(id int, time date); -- load some data INSERT INTO partitioning_test_2012 VALUES (5, '2012-06-06'); INSERT INTO partitioning_test_2012 VALUES (6, '2012-07-07'); ALTER TABLE partitioning_test ATTACH PARTITION partitioning_test_2012 FOR VALUES FROM ('2012-01-01') TO ('2013-01-01'); -ERROR: syntax error at or near "ATTACH" -LINE 1: ALTER TABLE partitioning_test ATTACH PARTITION partitioning_... - ^ +NOTICE: Copying data from local table... -- attached partition is distributed as well SELECT logicalrelid @@ -116,9 +121,12 @@ FROM WHERE logicalrelid IN ('partitioning_test', 'partitioning_test_2012') ORDER BY 1; -ERROR: relation "partitioning_test" does not exist -LINE 6: logicalrelid IN ('partitioning_test', 'partitioning_test_20... - ^ + logicalrelid +------------------------ + partitioning_test + partitioning_test_2012 +(2 rows) + SELECT logicalrelid, count(*) FROM pg_dist_shard @@ -127,14 +135,24 @@ GROUP BY logicalrelid ORDER BY 1,2; -ERROR: relation "partitioning_test" does not exist -LINE 4: WHERE logicalrelid IN ('partitioning_test', 'partitioning_t... - ^ + logicalrelid | count +------------------------+------- + partitioning_test | 4 + partitioning_test_2012 | 4 +(2 rows) + -- see the data is loaded to shards SELECT * FROM partitioning_test ORDER BY 1; -ERROR: relation "partitioning_test" does not exist -LINE 1: SELECT * FROM partitioning_test ORDER BY 1; - ^ + id | time +----+------------ + 1 | 06-06-2009 + 2 | 07-07-2010 + 3 | 09-09-2009 + 4 | 03-03-2010 + 5 | 06-06-2012 + 6 | 07-07-2012 +(6 rows) + -- 4-) Attaching distributed table to distributed table CREATE TABLE partitioning_test_2013(id int, time date); SELECT create_distributed_table('partitioning_test_2013', 'id'); @@ -147,51 +165,42 @@ SELECT create_distributed_table('partitioning_test_2013', 'id'); INSERT INTO partitioning_test_2013 VALUES (7, '2013-06-06'); INSERT INTO partitioning_test_2013 VALUES (8, '2013-07-07'); ALTER TABLE partitioning_test ATTACH PARTITION partitioning_test_2013 FOR VALUES FROM ('2013-01-01') TO ('2014-01-01'); -ERROR: syntax error at or near "ATTACH" -LINE 1: ALTER TABLE partitioning_test ATTACH PARTITION partitioning_... - ^ -- see the data is loaded to shards SELECT * FROM partitioning_test ORDER BY 1; -ERROR: relation "partitioning_test" does not exist -LINE 1: SELECT * FROM partitioning_test ORDER BY 1; - ^ + id | time +----+------------ + 1 | 06-06-2009 + 2 | 07-07-2010 + 3 | 09-09-2009 + 4 | 03-03-2010 + 5 | 06-06-2012 + 6 | 07-07-2012 + 7 | 06-06-2013 + 8 | 07-07-2013 +(8 rows) + -- 5-) Failure cases while creating distributed partitioned tables -- cannot distribute a partition if its parent is not distributed CREATE TABLE partitioning_test_failure(id int, time date) PARTITION BY RANGE (time); -ERROR: syntax error at or near "PARTITION" -LINE 1: ...ABLE partitioning_test_failure(id int, time date) PARTITION ... - ^ CREATE TABLE partitioning_test_failure_2009 PARTITION OF partitioning_test_failure FOR VALUES FROM ('2009-01-01') TO ('2010-01-01'); -ERROR: syntax error at or near "PARTITION" -LINE 1: CREATE TABLE partitioning_test_failure_2009 PARTITION OF par... - ^ SELECT create_distributed_table('partitioning_test_failure_2009', 'id'); -ERROR: relation "partitioning_test_failure_2009" does not exist -LINE 1: SELECT create_distributed_table('partitioning_test_failure_2... - ^ +ERROR: cannot distribute relation "partitioning_test_failure_2009" which is partition of "partitioning_test_failure" +DETAIL: Citus does not support distributing partitions if their parent is not distributed table. +HINT: Distribute the partitioned table "partitioning_test_failure" instead. -- only hash distributed tables can have partitions SELECT create_distributed_table('partitioning_test_failure', 'id', 'append'); -ERROR: relation "partitioning_test_failure" does not exist -LINE 1: SELECT create_distributed_table('partitioning_test_failure',... - ^ +ERROR: distributing partitioned tables in only supported for hash-distributed tables SELECT create_distributed_table('partitioning_test_failure', 'id', 'range'); -ERROR: relation "partitioning_test_failure" does not exist -LINE 1: SELECT create_distributed_table('partitioning_test_failure',... - ^ +ERROR: distributing partitioned tables in only supported for hash-distributed tables SELECT create_reference_table('partitioning_test_failure'); -ERROR: relation "partitioning_test_failure" does not exist -LINE 1: SELECT create_reference_table('partitioning_test_failure'); - ^ +ERROR: distributing partitioned tables in only supported for hash-distributed tables -- replication factor > 1 is not allowed in distributed partitioned tables SET citus.shard_replication_factor TO 2; SELECT create_distributed_table('partitioning_test_failure', 'id'); -ERROR: relation "partitioning_test_failure" does not exist -LINE 1: SELECT create_distributed_table('partitioning_test_failure',... - ^ +ERROR: distributing partitioned tables with replication factor greater than 1 is not supported SET citus.shard_replication_factor TO 1; -- non-distributed tables cannot have distributed partitions; DROP TABLE partitioning_test_failure_2009; -ERROR: table "partitioning_test_failure_2009" does not exist CREATE TABLE partitioning_test_failure_2009(id int, time date); SELECT create_distributed_table('partitioning_test_failure_2009', 'id'); create_distributed_table @@ -200,123 +209,107 @@ SELECT create_distributed_table('partitioning_test_failure_2009', 'id'); (1 row) ALTER TABLE partitioning_test_failure ATTACH PARTITION partitioning_test_failure_2009 FOR VALUES FROM ('2009-01-01') TO ('2010-01-01'); -ERROR: syntax error at or near "ATTACH" -LINE 1: ALTER TABLE partitioning_test_failure ATTACH PARTITION parti... - ^ +ERROR: non-distributed tables cannot have distributed partitions +HINT: Distribute the partitioned table "partitioning_test_failure_2009" instead -- multi-level partitioning is not allowed DROP TABLE partitioning_test_failure_2009; CREATE TABLE partitioning_test_failure_2009 PARTITION OF partitioning_test_failure FOR VALUES FROM ('2009-01-01') TO ('2010-01-01') PARTITION BY RANGE (time); -ERROR: syntax error at or near "PARTITION" -LINE 1: CREATE TABLE partitioning_test_failure_2009 PARTITION OF par... - ^ SELECT create_distributed_table('partitioning_test_failure', 'id'); -ERROR: relation "partitioning_test_failure" does not exist -LINE 1: SELECT create_distributed_table('partitioning_test_failure',... - ^ +ERROR: distributing multi-level partitioned tables is not supported +DETAIL: Relation "partitioning_test_failure_2009" is partitioned table itself and it is also partition of relation "partitioning_test_failure". -- multi-level partitioning is not allowed in different order DROP TABLE partitioning_test_failure_2009; -ERROR: table "partitioning_test_failure_2009" does not exist SELECT create_distributed_table('partitioning_test_failure', 'id'); -ERROR: relation "partitioning_test_failure" does not exist -LINE 1: SELECT create_distributed_table('partitioning_test_failure',... - ^ + create_distributed_table +-------------------------- + +(1 row) + CREATE TABLE partitioning_test_failure_2009 PARTITION OF partitioning_test_failure FOR VALUES FROM ('2009-01-01') TO ('2010-01-01') PARTITION BY RANGE (time); -ERROR: syntax error at or near "PARTITION" -LINE 1: CREATE TABLE partitioning_test_failure_2009 PARTITION OF par... - ^ +ERROR: distributing multi-level partitioned tables is not supported +DETAIL: Relation "partitioning_test_failure_2009" is partitioned table itself and it is also partition of relation "partitioning_test_failure". -- -- DMLs in distributed partitioned tables -- -- test COPY -- COPY data to partitioned table COPY partitioning_test FROM STDIN WITH CSV; -ERROR: relation "partitioning_test" does not exist -9,2009-01-01 -10,2010-01-01 -11,2011-01-01 -12,2012-01-01 -\. -invalid command \. -- COPY data to partition directly COPY partitioning_test_2009 FROM STDIN WITH CSV; -ERROR: syntax error at or near "9" -LINE 1: 9,2009-01-01 - ^ -13,2009-01-02 -14,2009-01-03 -\. -invalid command \. -- see the data is loaded to shards SELECT * FROM partitioning_test WHERE id >= 9 ORDER BY 1; -ERROR: syntax error at or near "13" -LINE 1: 13,2009-01-02 - ^ + id | time +----+------------ + 9 | 01-01-2009 + 10 | 01-01-2010 + 11 | 01-01-2011 + 12 | 01-01-2012 + 13 | 01-02-2009 + 14 | 01-03-2009 +(6 rows) + -- test INSERT -- INSERT INTO the partitioned table INSERT INTO partitioning_test VALUES(15, '2009-02-01'); -ERROR: relation "partitioning_test" does not exist -LINE 1: INSERT INTO partitioning_test VALUES(15, '2009-02-01'); - ^ INSERT INTO partitioning_test VALUES(16, '2010-02-01'); -ERROR: relation "partitioning_test" does not exist -LINE 1: INSERT INTO partitioning_test VALUES(16, '2010-02-01'); - ^ INSERT INTO partitioning_test VALUES(17, '2011-02-01'); -ERROR: relation "partitioning_test" does not exist -LINE 1: INSERT INTO partitioning_test VALUES(17, '2011-02-01'); - ^ INSERT INTO partitioning_test VALUES(18, '2012-02-01'); -ERROR: relation "partitioning_test" does not exist -LINE 1: INSERT INTO partitioning_test VALUES(18, '2012-02-01'); - ^ -- INSERT INTO the partitions directly table INSERT INTO partitioning_test VALUES(19, '2009-02-02'); -ERROR: relation "partitioning_test" does not exist -LINE 1: INSERT INTO partitioning_test VALUES(19, '2009-02-02'); - ^ INSERT INTO partitioning_test VALUES(20, '2010-02-02'); -ERROR: relation "partitioning_test" does not exist -LINE 1: INSERT INTO partitioning_test VALUES(20, '2010-02-02'); - ^ -- see the data is loaded to shards SELECT * FROM partitioning_test WHERE id >= 15 ORDER BY 1; -ERROR: relation "partitioning_test" does not exist -LINE 1: SELECT * FROM partitioning_test WHERE id >= 15 ORDER BY 1; - ^ + id | time +----+------------ + 15 | 02-01-2009 + 16 | 02-01-2010 + 17 | 02-01-2011 + 18 | 02-01-2012 + 19 | 02-02-2009 + 20 | 02-02-2010 +(6 rows) + -- test INSERT/SELECT -- INSERT/SELECT from partition to partitioned table INSERT INTO partitioning_test SELECT * FROM partitioning_test_2011; -ERROR: relation "partitioning_test" does not exist -LINE 1: INSERT INTO partitioning_test SELECT * FROM partitioning_tes... - ^ -- INSERT/SELECT from partitioned table to partition INSERT INTO partitioning_test_2012 SELECT * FROM partitioning_test WHERE time >= '2012-01-01' AND time < '2013-01-01'; -ERROR: relation "partitioning_test" does not exist -LINE 1: INSERT INTO partitioning_test_2012 SELECT * FROM partitionin... - ^ -- see the data is loaded to shards (rows in the given range should be duplicated) SELECT * FROM partitioning_test WHERE time >= '2011-01-01' AND time < '2013-01-01' ORDER BY 1; -ERROR: relation "partitioning_test" does not exist -LINE 1: SELECT * FROM partitioning_test WHERE time >= '2011-01-01' A... - ^ + id | time +----+------------ + 5 | 06-06-2012 + 5 | 06-06-2012 + 6 | 07-07-2012 + 6 | 07-07-2012 + 11 | 01-01-2011 + 11 | 01-01-2011 + 12 | 01-01-2012 + 12 | 01-01-2012 + 17 | 02-01-2011 + 17 | 02-01-2011 + 18 | 02-01-2012 + 18 | 02-01-2012 +(12 rows) + -- test UPDATE -- UPDATE partitioned table UPDATE partitioning_test SET time = '2013-07-07' WHERE id = 7; -ERROR: relation "partitioning_test" does not exist -LINE 1: UPDATE partitioning_test SET time = '2013-07-07' WHERE id = ... - ^ -- UPDATE partition directly UPDATE partitioning_test_2013 SET time = '2013-08-08' WHERE id = 8; -- see the data is updated SELECT * FROM partitioning_test WHERE id = 7 OR id = 8 ORDER BY 1; -ERROR: relation "partitioning_test" does not exist -LINE 1: SELECT * FROM partitioning_test WHERE id = 7 OR id = 8 ORDER... - ^ + id | time +----+------------ + 7 | 07-07-2013 + 8 | 08-08-2013 +(2 rows) + -- UPDATE that tries to move a row to a non-existing partition (this should fail) UPDATE partitioning_test SET time = '2020-07-07' WHERE id = 7; -ERROR: relation "partitioning_test" does not exist -LINE 1: UPDATE partitioning_test SET time = '2020-07-07' WHERE id = ... - ^ +ERROR: new row for relation "partitioning_test_2013_1660021" violates partition constraint +DETAIL: Failing row contains (7, 2020-07-07). +CONTEXT: while executing command on localhost:57638 -- UPDATE with subqueries on partitioned table UPDATE partitioning_test @@ -324,9 +317,6 @@ SET time = time + INTERVAL '1 day' WHERE id IN (SELECT id FROM partitioning_test WHERE id = 1); -ERROR: relation "partitioning_test" does not exist -LINE 2: partitioning_test - ^ -- UPDATE with subqueries on partition UPDATE partitioning_test_2009 @@ -334,92 +324,138 @@ SET time = time + INTERVAL '1 month' WHERE id IN (SELECT id FROM partitioning_test WHERE id = 2); -ERROR: relation "partitioning_test_2009" does not exist -LINE 2: partitioning_test_2009 - ^ -- see the data is updated SELECT * FROM partitioning_test WHERE id = 1 OR id = 2 ORDER BY 1; -ERROR: relation "partitioning_test" does not exist -LINE 1: SELECT * FROM partitioning_test WHERE id = 1 OR id = 2 ORDER... - ^ + id | time +----+------------ + 1 | 06-07-2009 + 2 | 07-07-2010 +(2 rows) + -- test DELETE -- DELETE from partitioned table DELETE FROM partitioning_test WHERE id = 9; -ERROR: relation "partitioning_test" does not exist -LINE 1: DELETE FROM partitioning_test WHERE id = 9; - ^ -- DELETE from partition directly DELETE FROM partitioning_test_2010 WHERE id = 10; -ERROR: relation "partitioning_test_2010" does not exist -LINE 1: DELETE FROM partitioning_test_2010 WHERE id = 10; - ^ -- see the data is deleted SELECT * FROM partitioning_test WHERE id = 9 OR id = 10 ORDER BY 1; -ERROR: relation "partitioning_test" does not exist -LINE 1: SELECT * FROM partitioning_test WHERE id = 9 OR id = 10 ORDE... - ^ + id | time +----+------ +(0 rows) + -- test master_modify_multiple_shards -- master_modify_multiple_shards on partitioned table SELECT master_modify_multiple_shards('UPDATE partitioning_test SET time = time + INTERVAL ''1 day'''); -ERROR: relation "partitioning_test" does not exist + master_modify_multiple_shards +------------------------------- + 24 +(1 row) + -- see rows are UPDATED SELECT * FROM partitioning_test ORDER BY 1; -ERROR: relation "partitioning_test" does not exist -LINE 1: SELECT * FROM partitioning_test ORDER BY 1; - ^ + id | time +----+------------ + 1 | 06-08-2009 + 2 | 07-08-2010 + 3 | 09-10-2009 + 4 | 03-04-2010 + 5 | 06-07-2012 + 5 | 06-07-2012 + 6 | 07-08-2012 + 6 | 07-08-2012 + 7 | 07-08-2013 + 8 | 08-09-2013 + 11 | 01-02-2011 + 11 | 01-02-2011 + 12 | 01-02-2012 + 12 | 01-02-2012 + 13 | 01-03-2009 + 14 | 01-04-2009 + 15 | 02-02-2009 + 16 | 02-02-2010 + 17 | 02-02-2011 + 17 | 02-02-2011 + 18 | 02-02-2012 + 18 | 02-02-2012 + 19 | 02-03-2009 + 20 | 02-03-2010 +(24 rows) + -- master_modify_multiple_shards on partition directly SELECT master_modify_multiple_shards('UPDATE partitioning_test_2009 SET time = time + INTERVAL ''1 day'''); -ERROR: relation "partitioning_test_2009" does not exist + master_modify_multiple_shards +------------------------------- + 6 +(1 row) + -- see rows are UPDATED SELECT * FROM partitioning_test_2009 ORDER BY 1; -ERROR: relation "partitioning_test_2009" does not exist -LINE 1: SELECT * FROM partitioning_test_2009 ORDER BY 1; - ^ + id | time +----+------------ + 1 | 06-09-2009 + 3 | 09-11-2009 + 13 | 01-04-2009 + 14 | 01-05-2009 + 15 | 02-03-2009 + 19 | 02-04-2009 +(6 rows) + -- test master_modify_multiple_shards which fails in workers (updated value is outside of partition bounds) SELECT master_modify_multiple_shards('UPDATE partitioning_test_2009 SET time = time + INTERVAL ''6 month'''); -ERROR: relation "partitioning_test_2009" does not exist +ERROR: new row for relation "partitioning_test_2009_1660005" violates partition constraint +DETAIL: Failing row contains (3, 2010-03-11). +CONTEXT: while executing command on localhost:57638 -- -- DDL in distributed partitioned tables -- -- test CREATE INDEX -- CREATE INDEX on partitioned table - this will error out +-- on earlier versions of postgres earlier than 11. CREATE INDEX partitioning_index ON partitioning_test(id); -ERROR: relation "partitioning_test" does not exist +ERROR: cannot create index on partitioned table "partitioning_test" -- CREATE INDEX on partition CREATE INDEX partitioning_2009_index ON partitioning_test_2009(id); -ERROR: relation "partitioning_test_2009" does not exist -- CREATE INDEX CONCURRENTLY on partition CREATE INDEX CONCURRENTLY partitioned_2010_index ON partitioning_test_2010(id); -ERROR: relation "partitioning_test_2010" does not exist -- see index is created SELECT tablename, indexname FROM pg_indexes WHERE tablename LIKE 'partitioning_test%' ORDER BY indexname; - tablename | indexname ------------+----------- -(0 rows) + tablename | indexname +------------------------+------------------------- + partitioning_test_2010 | partitioned_2010_index + partitioning_test_2009 | partitioning_2009_index +(2 rows) -- test add COLUMN -- add COLUMN to partitioned table ALTER TABLE partitioning_test ADD new_column int; -ERROR: relation "partitioning_test" does not exist -- add COLUMN to partition - this will error out ALTER TABLE partitioning_test_2010 ADD new_column_2 int; -ERROR: relation "partitioning_test_2010" does not exist +ERROR: cannot add column to a partition -- see additional column is created SELECT name, type FROM table_attrs WHERE relid = 'partitioning_test'::regclass ORDER BY 1; -ERROR: relation "partitioning_test" does not exist -LINE 1: SELECT name, type FROM table_attrs WHERE relid = 'partitioni... - ^ + name | type +------------+--------- + id | integer + new_column | integer + time | date +(3 rows) + SELECT name, type FROM table_attrs WHERE relid = 'partitioning_test_2010'::regclass ORDER BY 1; -ERROR: relation "partitioning_test_2010" does not exist -LINE 1: SELECT name, type FROM table_attrs WHERE relid = 'partitioni... - ^ + name | type +------------+--------- + id | integer + new_column | integer + time | date +(3 rows) + -- test add PRIMARY KEY -- add PRIMARY KEY to partitioned table - this will error out ALTER TABLE partitioning_test ADD CONSTRAINT partitioning_primary PRIMARY KEY (id); -ERROR: relation "partitioning_test" does not exist +ERROR: primary key constraints are not supported on partitioned tables +LINE 1: ALTER TABLE partitioning_test ADD CONSTRAINT partitioning_pr... + ^ -- ADD PRIMARY KEY to partition ALTER TABLE partitioning_test_2009 ADD CONSTRAINT partitioning_2009_primary PRIMARY KEY (id); -ERROR: relation "partitioning_test_2009" does not exist -- see PRIMARY KEY is created SELECT table_name, @@ -430,169 +466,170 @@ FROM WHERE table_name = 'partitioning_test_2009' AND constraint_name = 'partitioning_2009_primary'; - table_name | constraint_name | constraint_type -------------+-----------------+----------------- -(0 rows) + table_name | constraint_name | constraint_type +------------------------+---------------------------+----------------- + partitioning_test_2009 | partitioning_2009_primary | PRIMARY KEY +(1 row) -- test ADD FOREIGN CONSTRAINT -- add FOREIGN CONSTRAINT to partitioned table -- this will error out ALTER TABLE partitioning_test ADD CONSTRAINT partitioning_foreign FOREIGN KEY (id) REFERENCES partitioning_test_2009 (id); -ERROR: relation "partitioning_test" does not exist +ERROR: foreign key constraints are not supported on partitioned tables +LINE 1: ALTER TABLE partitioning_test ADD CONSTRAINT partitioning_fo... + ^ -- add FOREIGN CONSTRAINT to partition INSERT INTO partitioning_test_2009 VALUES (5, '2009-06-06'); -ERROR: relation "partitioning_test_2009" does not exist -LINE 1: INSERT INTO partitioning_test_2009 VALUES (5, '2009-06-06'); - ^ INSERT INTO partitioning_test_2009 VALUES (6, '2009-07-07'); -ERROR: relation "partitioning_test_2009" does not exist -LINE 1: INSERT INTO partitioning_test_2009 VALUES (6, '2009-07-07'); - ^ INSERT INTO partitioning_test_2009 VALUES(12, '2009-02-01'); -ERROR: relation "partitioning_test_2009" does not exist -LINE 1: INSERT INTO partitioning_test_2009 VALUES(12, '2009-02-01'); - ^ INSERT INTO partitioning_test_2009 VALUES(18, '2009-02-01'); -ERROR: relation "partitioning_test_2009" does not exist -LINE 1: INSERT INTO partitioning_test_2009 VALUES(18, '2009-02-01'); - ^ ALTER TABLE partitioning_test_2012 ADD CONSTRAINT partitioning_2012_foreign FOREIGN KEY (id) REFERENCES partitioning_test_2009 (id) ON DELETE CASCADE; -ERROR: relation "partitioning_test_2009" does not exist -- see FOREIGN KEY is created SELECT "Constraint" FROM table_fkeys WHERE relid = 'partitioning_test_2012'::regclass ORDER BY 1; - Constraint ------------- -(0 rows) + Constraint +--------------------------- + partitioning_2012_foreign +(1 row) -- test ON DELETE CASCADE works DELETE FROM partitioning_test_2009 WHERE id = 5; -ERROR: relation "partitioning_test_2009" does not exist -LINE 1: DELETE FROM partitioning_test_2009 WHERE id = 5; - ^ -- see that element is deleted from both partitions SELECT * FROM partitioning_test_2009 WHERE id = 5 ORDER BY 1; -ERROR: relation "partitioning_test_2009" does not exist -LINE 1: SELECT * FROM partitioning_test_2009 WHERE id = 5 ORDER BY 1... - ^ + id | time | new_column +----+------+------------ +(0 rows) + SELECT * FROM partitioning_test_2012 WHERE id = 5 ORDER BY 1; - id | time -----+------------ - 5 | 06-06-2012 -(1 row) + id | time | new_column +----+------+------------ +(0 rows) -- test DETACH partition ALTER TABLE partitioning_test DETACH PARTITION partitioning_test_2009; -ERROR: syntax error at or near "DETACH" -LINE 1: ALTER TABLE partitioning_test DETACH PARTITION partitioning_... - ^ -- see DETACHed partitions content is not accessible from partitioning_test; SELECT * FROM partitioning_test WHERE time >= '2009-01-01' AND time < '2010-01-01' ORDER BY 1; -ERROR: relation "partitioning_test" does not exist -LINE 1: SELECT * FROM partitioning_test WHERE time >= '2009-01-01' A... - ^ + id | time | new_column +----+------+------------ +(0 rows) + -- -- Transaction tests -- -- DDL in transaction BEGIN; ALTER TABLE partitioning_test ADD newer_column int; -ERROR: relation "partitioning_test" does not exist -- see additional column is created SELECT name, type FROM table_attrs WHERE relid = 'partitioning_test'::regclass ORDER BY 1; -ERROR: current transaction is aborted, commands ignored until end of transaction block + name | type +--------------+--------- + id | integer + new_column | integer + newer_column | integer + time | date +(4 rows) + ROLLBACK; -- see rollback is successful SELECT name, type FROM table_attrs WHERE relid = 'partitioning_test'::regclass ORDER BY 1; -ERROR: relation "partitioning_test" does not exist -LINE 1: SELECT name, type FROM table_attrs WHERE relid = 'partitioni... - ^ + name | type +------------+--------- + id | integer + new_column | integer + time | date +(3 rows) + -- COPY in transaction BEGIN; COPY partitioning_test FROM STDIN WITH CSV; -ERROR: relation "partitioning_test" does not exist -22,2010-01-01,22 -23,2011-01-01,23 -24,2013-01-01,24 -\. -invalid command \. -- see the data is loaded to shards SELECT * FROM partitioning_test WHERE id = 22 ORDER BY 1; -ERROR: syntax error at or near "22" -LINE 1: 22,2010-01-01,22 - ^ + id | time | new_column +----+------------+------------ + 22 | 01-01-2010 | 22 +(1 row) + SELECT * FROM partitioning_test WHERE id = 23 ORDER BY 1; -ERROR: current transaction is aborted, commands ignored until end of transaction block + id | time | new_column +----+------------+------------ + 23 | 01-01-2011 | 23 +(1 row) + SELECT * FROM partitioning_test WHERE id = 24 ORDER BY 1; -ERROR: current transaction is aborted, commands ignored until end of transaction block + id | time | new_column +----+------------+------------ + 24 | 01-01-2013 | 24 +(1 row) + ROLLBACK; -- see rollback is successful SELECT * FROM partitioning_test WHERE id >= 22 ORDER BY 1; -ERROR: relation "partitioning_test" does not exist -LINE 1: SELECT * FROM partitioning_test WHERE id >= 22 ORDER BY 1; - ^ + id | time | new_column +----+------+------------ +(0 rows) + -- DML in transaction BEGIN; -- INSERT in transaction INSERT INTO partitioning_test VALUES(25, '2010-02-02'); -ERROR: relation "partitioning_test" does not exist -LINE 1: INSERT INTO partitioning_test VALUES(25, '2010-02-02'); - ^ -- see the data is loaded to shards SELECT * FROM partitioning_test WHERE id = 25 ORDER BY 1; -ERROR: current transaction is aborted, commands ignored until end of transaction block + id | time | new_column +----+------------+------------ + 25 | 02-02-2010 | +(1 row) + -- INSERT/SELECT in transaction INSERT INTO partitioning_test SELECT * FROM partitioning_test WHERE id = 25; -ERROR: current transaction is aborted, commands ignored until end of transaction block -- see the data is loaded to shards SELECT * FROM partitioning_test WHERE id = 25 ORDER BY 1; -ERROR: current transaction is aborted, commands ignored until end of transaction block + id | time | new_column +----+------------+------------ + 25 | 02-02-2010 | + 25 | 02-02-2010 | +(2 rows) + -- UPDATE in transaction UPDATE partitioning_test SET time = '2010-10-10' WHERE id = 25; -ERROR: current transaction is aborted, commands ignored until end of transaction block -- see the data is updated SELECT * FROM partitioning_test WHERE id = 25 ORDER BY 1; -ERROR: current transaction is aborted, commands ignored until end of transaction block + id | time | new_column +----+------------+------------ + 25 | 10-10-2010 | + 25 | 10-10-2010 | +(2 rows) + -- perform operations on partition and partioned tables together INSERT INTO partitioning_test VALUES(26, '2010-02-02', 26); -ERROR: current transaction is aborted, commands ignored until end of transaction block INSERT INTO partitioning_test_2010 VALUES(26, '2010-02-02', 26); -ERROR: current transaction is aborted, commands ignored until end of transaction block COPY partitioning_test FROM STDIN WITH CSV; -ERROR: current transaction is aborted, commands ignored until end of transaction block -26,2010-02-02,26 -\. -invalid command \. COPY partitioning_test_2010 FROM STDIN WITH CSV; -ERROR: syntax error at or near "26" -LINE 1: 26,2010-02-02,26 - ^ -26,2010-02-02,26 -\. -invalid command \. -- see the data is loaded to shards (we should see 4 rows with same content) SELECT * FROM partitioning_test WHERE id = 26 ORDER BY 1; -ERROR: syntax error at or near "26" -LINE 1: 26,2010-02-02,26 - ^ + id | time | new_column +----+------------+------------ + 26 | 02-02-2010 | 26 + 26 | 02-02-2010 | 26 + 26 | 02-02-2010 | 26 + 26 | 02-02-2010 | 26 +(4 rows) + ROLLBACK; -- see rollback is successful SELECT * FROM partitioning_test WHERE id = 26 ORDER BY 1; -ERROR: relation "partitioning_test" does not exist -LINE 1: SELECT * FROM partitioning_test WHERE id = 26 ORDER BY 1; - ^ + id | time | new_column +----+------+------------ +(0 rows) + -- DETACH and DROP in a transaction BEGIN; ALTER TABLE partitioning_test DETACH PARTITION partitioning_test_2011; -ERROR: syntax error at or near "DETACH" -LINE 1: ALTER TABLE partitioning_test DETACH PARTITION partitioning_... - ^ DROP TABLE partitioning_test_2011; -ERROR: current transaction is aborted, commands ignored until end of transaction block COMMIT; -- see DROPed partitions content is not accessible SELECT * FROM partitioning_test WHERE time >= '2011-01-01' AND time < '2012-01-01' ORDER BY 1; -ERROR: relation "partitioning_test" does not exist -LINE 1: SELECT * FROM partitioning_test WHERE time >= '2011-01-01' A... - ^ + id | time | new_column +----+------+------------ +(0 rows) + -- -- Misc tests -- @@ -601,76 +638,58 @@ LINE 1: SELECT * FROM partitioning_test WHERE time >= '2011-01-01' A... TRUNCATE partitioning_test_2012; -- see partition is TRUNCATEd SELECT * FROM partitioning_test_2012 ORDER BY 1; - id | time -----+------ + id | time | new_column +----+------+------------ (0 rows) -- test TRUNCATE partitioned table TRUNCATE partitioning_test; -ERROR: relation "partitioning_test" does not exist -- see partitioned table is TRUNCATEd SELECT * FROM partitioning_test ORDER BY 1; -ERROR: relation "partitioning_test" does not exist -LINE 1: SELECT * FROM partitioning_test ORDER BY 1; - ^ + id | time | new_column +----+------+------------ +(0 rows) + -- test DROP -- test DROP partition INSERT INTO partitioning_test_2010 VALUES(27, '2010-02-01'); -ERROR: relation "partitioning_test_2010" does not exist -LINE 1: INSERT INTO partitioning_test_2010 VALUES(27, '2010-02-01'); - ^ DROP TABLE partitioning_test_2010; -ERROR: table "partitioning_test_2010" does not exist -- see DROPped partitions content is not accessible from partitioning_test; SELECT * FROM partitioning_test WHERE time >= '2010-01-01' AND time < '2011-01-01' ORDER BY 1; -ERROR: relation "partitioning_test" does not exist -LINE 1: SELECT * FROM partitioning_test WHERE time >= '2010-01-01' A... - ^ + id | time | new_column +----+------+------------ +(0 rows) + -- test DROP partitioned table DROP TABLE partitioning_test; -ERROR: table "partitioning_test" does not exist -- dropping the parent should CASCADE to the children as well SELECT table_name FROM information_schema.tables WHERE table_name LIKE 'partitioning_test%' ORDER BY 1; - table_name ------------------------- - partitioning_test_2012 - partitioning_test_2013 + table_name +--------------------------- + partitioning_test_2009 + partitioning_test_failure (2 rows) -- test distributing partitioned table colocated with non-partitioned table CREATE TABLE partitioned_users_table (user_id int, time timestamp, value_1 int, value_2 int, value_3 float, value_4 bigint) PARTITION BY RANGE (time); -ERROR: syntax error at or near "PARTITION" -LINE 1: ... int, value_2 int, value_3 float, value_4 bigint) PARTITION ... - ^ CREATE TABLE partitioned_events_table (user_id int, time timestamp, event_type int, value_2 int, value_3 float, value_4 bigint) PARTITION BY RANGE (time); -ERROR: syntax error at or near "PARTITION" -LINE 1: ... int, value_2 int, value_3 float, value_4 bigint) PARTITION ... - ^ SELECT create_distributed_table('partitioned_users_table', 'user_id', colocate_with => 'users_table'); -ERROR: relation "partitioned_users_table" does not exist -LINE 1: SELECT create_distributed_table('partitioned_users_table', '... - ^ + create_distributed_table +-------------------------- + +(1 row) + SELECT create_distributed_table('partitioned_events_table', 'user_id', colocate_with => 'events_table'); -ERROR: relation "partitioned_events_table" does not exist -LINE 1: SELECT create_distributed_table('partitioned_events_table', ... - ^ + create_distributed_table +-------------------------- + +(1 row) + -- INSERT/SELECT from regular table to partitioned table CREATE TABLE partitioned_users_table_2009 PARTITION OF partitioned_users_table FOR VALUES FROM ('2017-01-01') TO ('2018-01-01'); -ERROR: syntax error at or near "PARTITION" -LINE 1: CREATE TABLE partitioned_users_table_2009 PARTITION OF parti... - ^ CREATE TABLE partitioned_events_table_2009 PARTITION OF partitioned_events_table FOR VALUES FROM ('2017-01-01') TO ('2018-01-01'); -ERROR: syntax error at or near "PARTITION" -LINE 1: CREATE TABLE partitioned_events_table_2009 PARTITION OF part... - ^ INSERT INTO partitioned_events_table SELECT * FROM events_table; -ERROR: relation "partitioned_events_table" does not exist -LINE 1: INSERT INTO partitioned_events_table SELECT * FROM events_ta... - ^ INSERT INTO partitioned_users_table_2009 SELECT * FROM users_table; -ERROR: relation "partitioned_users_table_2009" does not exist -LINE 1: INSERT INTO partitioned_users_table_2009 SELECT * FROM users... - ^ -- -- Complex JOINs, subqueries, UNIONs etc... -- @@ -716,9 +735,14 @@ FROM ) AS final_query GROUP BY types ORDER BY types; -ERROR: relation "partitioned_events_table" does not exist -LINE 14: partitioned_events_table as "events" - ^ + types | sumofeventtype +-------+---------------- + 0 | 43 + 1 | 44 + 2 | 8 + 3 | 25 +(4 rows) + -- UNION and JOIN on both partitioned and regular tables SELECT ("final_query"."event_types") as types, count(*) AS sumOfEventType FROM @@ -790,31 +814,25 @@ GROUP BY types ORDER BY types; -ERROR: relation "partitioned_events_table" does not exist -LINE 18: partitioned_events_table as "events" - ^ + types | sumofeventtype +-------+---------------- + 0 | 367 + 2 | 360 + 3 | 57 +(3 rows) + -- test LIST partitioning CREATE TABLE list_partitioned_events_table (user_id int, time date, event_type int, value_2 int, value_3 float, value_4 bigint) PARTITION BY LIST (time); -ERROR: syntax error at or near "PARTITION" -LINE 1: ... int, value_2 int, value_3 float, value_4 bigint) PARTITION ... - ^ CREATE TABLE list_partitioned_events_table_2014_01_01_05 PARTITION OF list_partitioned_events_table FOR VALUES IN ('2017-11-21', '2017-11-22', '2017-11-23', '2017-11-24', '2017-11-25'); -ERROR: syntax error at or near "PARTITION" -LINE 1: ...TABLE list_partitioned_events_table_2014_01_01_05 PARTITION ... - ^ CREATE TABLE list_partitioned_events_table_2014_01_06_10 PARTITION OF list_partitioned_events_table FOR VALUES IN ('2017-11-26', '2017-11-27', '2017-11-28', '2017-11-29', '2017-11-30'); -ERROR: syntax error at or near "PARTITION" -LINE 1: ...TABLE list_partitioned_events_table_2014_01_06_10 PARTITION ... - ^ CREATE TABLE list_partitioned_events_table_2014_01_11_15 PARTITION OF list_partitioned_events_table FOR VALUES IN ('2017-12-01', '2017-12-02', '2017-12-03', '2017-12-04', '2017-12-05'); -ERROR: syntax error at or near "PARTITION" -LINE 1: ...TABLE list_partitioned_events_table_2014_01_11_15 PARTITION ... - ^ -- test distributing partitioned table colocated with another partitioned table SELECT create_distributed_table('list_partitioned_events_table', 'user_id', colocate_with => 'partitioned_events_table'); -ERROR: relation "list_partitioned_events_table" does not exist -LINE 1: SELECT create_distributed_table('list_partitioned_events_tab... - ^ + create_distributed_table +-------------------------- + +(1 row) + -- INSERT/SELECT from partitioned table to partitioned table INSERT INTO list_partitioned_events_table @@ -830,9 +848,6 @@ FROM WHERE time >= '2017-11-21' AND time <= '2017-12-01'; -ERROR: relation "list_partitioned_events_table" does not exist -LINE 2: list_partitioned_events_table - ^ -- LEFT JOINs used with INNER JOINs on range partitioned table, list partitioned table and non-partitioned table SELECT count(*) AS cnt, "generated_group_field" @@ -870,139 +885,164 @@ count(*) AS cnt, "generated_group_field" ORDER BY cnt DESC, generated_group_field ASC LIMIT 10; -ERROR: relation "list_partitioned_events_table" does not exist -LINE 15: list_partitioned_events_table as "list_partitio... - ^ + cnt | generated_group_field +------+----------------------- + 1851 | 1 + 1077 | 4 + 963 | 2 + 955 | 3 + 768 | 5 + 639 | 0 +(6 rows) + -- -- Additional partitioning features -- -- test multi column partitioning CREATE TABLE multi_column_partitioning(c1 int, c2 int) PARTITION BY RANGE (c1, c2); -ERROR: syntax error at or near "PARTITION" -LINE 1: ...E TABLE multi_column_partitioning(c1 int, c2 int) PARTITION ... - ^ CREATE TABLE multi_column_partitioning_0_0_10_0 PARTITION OF multi_column_partitioning FOR VALUES FROM (0, 0) TO (10, 0); -ERROR: syntax error at or near "PARTITION" -LINE 1: CREATE TABLE multi_column_partitioning_0_0_10_0 PARTITION OF... - ^ SELECT create_distributed_table('multi_column_partitioning', 'c1'); -ERROR: relation "multi_column_partitioning" does not exist -LINE 1: SELECT create_distributed_table('multi_column_partitioning',... - ^ + create_distributed_table +-------------------------- + +(1 row) + -- test INSERT to multi-column partitioned table INSERT INTO multi_column_partitioning VALUES(1, 1); -ERROR: relation "multi_column_partitioning" does not exist -LINE 1: INSERT INTO multi_column_partitioning VALUES(1, 1); - ^ INSERT INTO multi_column_partitioning_0_0_10_0 VALUES(5, -5); -ERROR: relation "multi_column_partitioning_0_0_10_0" does not exist -LINE 1: INSERT INTO multi_column_partitioning_0_0_10_0 VALUES(5, -5)... - ^ -- test INSERT to multi-column partitioned table where no suitable partition exists INSERT INTO multi_column_partitioning VALUES(10, 1); -ERROR: relation "multi_column_partitioning" does not exist -LINE 1: INSERT INTO multi_column_partitioning VALUES(10, 1); - ^ +ERROR: no partition of relation "multi_column_partitioning_1660068" found for row +DETAIL: Partition key of the failing row contains (c1, c2) = (10, 1). +CONTEXT: while executing command on localhost:57637 -- test with MINVALUE/MAXVALUE CREATE TABLE multi_column_partitioning_10_max_20_min PARTITION OF multi_column_partitioning FOR VALUES FROM (10, MAXVALUE) TO (20, MINVALUE); -ERROR: syntax error at or near "PARTITION" -LINE 1: ...ATE TABLE multi_column_partitioning_10_max_20_min PARTITION ... - ^ -- test INSERT to partition with MINVALUE/MAXVALUE bounds INSERT INTO multi_column_partitioning VALUES(11, -11); -ERROR: relation "multi_column_partitioning" does not exist -LINE 1: INSERT INTO multi_column_partitioning VALUES(11, -11); - ^ INSERT INTO multi_column_partitioning_10_max_20_min VALUES(19, -19); -ERROR: relation "multi_column_partitioning_10_max_20_min" does not exist -LINE 1: INSERT INTO multi_column_partitioning_10_max_20_min VALUES(1... - ^ -- test INSERT to multi-column partitioned table where no suitable partition exists INSERT INTO multi_column_partitioning VALUES(20, -20); -ERROR: relation "multi_column_partitioning" does not exist -LINE 1: INSERT INTO multi_column_partitioning VALUES(20, -20); - ^ +ERROR: no partition of relation "multi_column_partitioning_1660068" found for row +DETAIL: Partition key of the failing row contains (c1, c2) = (20, -20). +CONTEXT: while executing command on localhost:57637 -- see data is loaded to multi-column partitioned table SELECT * FROM multi_column_partitioning ORDER BY 1, 2; -ERROR: relation "multi_column_partitioning" does not exist -LINE 1: SELECT * FROM multi_column_partitioning ORDER BY 1, 2; - ^ + c1 | c2 +----+----- + 1 | 1 + 5 | -5 + 11 | -11 + 19 | -19 +(4 rows) + -- -- Tests for locks on partitioned tables -- CREATE TABLE partitioning_locks(id int, ref_id int, time date) PARTITION BY RANGE (time); -ERROR: syntax error at or near "PARTITION" -LINE 1: ...partitioning_locks(id int, ref_id int, time date) PARTITION ... - ^ -- create its partitions CREATE TABLE partitioning_locks_2009 PARTITION OF partitioning_locks FOR VALUES FROM ('2009-01-01') TO ('2010-01-01'); -ERROR: syntax error at or near "PARTITION" -LINE 1: CREATE TABLE partitioning_locks_2009 PARTITION OF partitioni... - ^ CREATE TABLE partitioning_locks_2010 PARTITION OF partitioning_locks FOR VALUES FROM ('2010-01-01') TO ('2011-01-01'); -ERROR: syntax error at or near "PARTITION" -LINE 1: CREATE TABLE partitioning_locks_2010 PARTITION OF partitioni... - ^ -- distribute partitioned table SELECT create_distributed_table('partitioning_locks', 'id'); -ERROR: relation "partitioning_locks" does not exist -LINE 1: SELECT create_distributed_table('partitioning_locks', 'id'); - ^ + create_distributed_table +-------------------------- + +(1 row) + -- test locks on router SELECT BEGIN; SELECT * FROM partitioning_locks WHERE id = 1 ORDER BY 1, 2; -ERROR: relation "partitioning_locks" does not exist -LINE 1: SELECT * FROM partitioning_locks WHERE id = 1 ORDER BY 1, 2; - ^ + id | ref_id | time +----+--------+------ +(0 rows) + SELECT relation::regclass, locktype, mode FROM pg_locks WHERE relation::regclass::text LIKE 'partitioning_locks%' AND pid = pg_backend_pid() ORDER BY 1, 2, 3; -ERROR: current transaction is aborted, commands ignored until end of transaction block + relation | locktype | mode +-------------------------+----------+----------------- + partitioning_locks | relation | AccessShareLock + partitioning_locks_2009 | relation | AccessShareLock + partitioning_locks_2010 | relation | AccessShareLock +(3 rows) + COMMIT; -- test locks on real-time SELECT BEGIN; SELECT * FROM partitioning_locks ORDER BY 1, 2; -ERROR: relation "partitioning_locks" does not exist -LINE 1: SELECT * FROM partitioning_locks ORDER BY 1, 2; - ^ + id | ref_id | time +----+--------+------ +(0 rows) + SELECT relation::regclass, locktype, mode FROM pg_locks WHERE relation::regclass::text LIKE 'partitioning_locks%' AND pid = pg_backend_pid() ORDER BY 1, 2, 3; -ERROR: current transaction is aborted, commands ignored until end of transaction block + relation | locktype | mode +-------------------------+----------+----------------- + partitioning_locks | relation | AccessShareLock + partitioning_locks_2009 | relation | AccessShareLock + partitioning_locks_2010 | relation | AccessShareLock +(3 rows) + COMMIT; -- test locks on task-tracker SELECT SET citus.task_executor_type TO 'task-tracker'; BEGIN; SELECT * FROM partitioning_locks AS pl1 JOIN partitioning_locks AS pl2 ON pl1.id = pl2.ref_id ORDER BY 1, 2; -ERROR: relation "partitioning_locks" does not exist -LINE 1: SELECT * FROM partitioning_locks AS pl1 JOIN partitioning_lo... - ^ + id | ref_id | time | id | ref_id | time +----+--------+------+----+--------+------ +(0 rows) + SELECT relation::regclass, locktype, mode FROM pg_locks WHERE relation::regclass::text LIKE 'partitioning_locks%' AND pid = pg_backend_pid() ORDER BY 1, 2, 3; -ERROR: current transaction is aborted, commands ignored until end of transaction block + relation | locktype | mode +-------------------------+----------+----------------- + partitioning_locks | relation | AccessShareLock + partitioning_locks_2009 | relation | AccessShareLock + partitioning_locks_2010 | relation | AccessShareLock +(3 rows) + COMMIT; SET citus.task_executor_type TO 'real-time'; -- test locks on INSERT BEGIN; INSERT INTO partitioning_locks VALUES(1, 1, '2009-01-01'); -ERROR: relation "partitioning_locks" does not exist -LINE 1: INSERT INTO partitioning_locks VALUES(1, 1, '2009-01-01'); - ^ SELECT relation::regclass, locktype, mode FROM pg_locks WHERE relation::regclass::text LIKE 'partitioning_locks%' AND pid = pg_backend_pid() ORDER BY 1, 2, 3; -ERROR: current transaction is aborted, commands ignored until end of transaction block + relation | locktype | mode +-------------------------+----------+------------------ + partitioning_locks | relation | AccessShareLock + partitioning_locks | relation | RowExclusiveLock + partitioning_locks_2009 | relation | AccessShareLock + partitioning_locks_2009 | relation | RowExclusiveLock + partitioning_locks_2010 | relation | AccessShareLock + partitioning_locks_2010 | relation | RowExclusiveLock +(6 rows) + COMMIT; -- test locks on UPDATE BEGIN; UPDATE partitioning_locks SET time = '2009-02-01' WHERE id = 1; -ERROR: relation "partitioning_locks" does not exist -LINE 1: UPDATE partitioning_locks SET time = '2009-02-01' WHERE id =... - ^ SELECT relation::regclass, locktype, mode FROM pg_locks WHERE relation::regclass::text LIKE 'partitioning_locks%' AND pid = pg_backend_pid() ORDER BY 1, 2, 3; -ERROR: current transaction is aborted, commands ignored until end of transaction block + relation | locktype | mode +-------------------------+----------+------------------ + partitioning_locks | relation | AccessShareLock + partitioning_locks | relation | RowExclusiveLock + partitioning_locks_2009 | relation | AccessShareLock + partitioning_locks_2009 | relation | RowExclusiveLock + partitioning_locks_2010 | relation | AccessShareLock + partitioning_locks_2010 | relation | RowExclusiveLock +(6 rows) + COMMIT; -- test locks on DELETE BEGIN; DELETE FROM partitioning_locks WHERE id = 1; -ERROR: relation "partitioning_locks" does not exist -LINE 1: DELETE FROM partitioning_locks WHERE id = 1; - ^ SELECT relation::regclass, locktype, mode FROM pg_locks WHERE relation::regclass::text LIKE 'partitioning_locks%' AND pid = pg_backend_pid() ORDER BY 1, 2, 3; -ERROR: current transaction is aborted, commands ignored until end of transaction block + relation | locktype | mode +-------------------------+----------+------------------ + partitioning_locks | relation | AccessShareLock + partitioning_locks | relation | RowExclusiveLock + partitioning_locks_2009 | relation | AccessShareLock + partitioning_locks_2009 | relation | RowExclusiveLock + partitioning_locks_2010 | relation | AccessShareLock + partitioning_locks_2010 | relation | RowExclusiveLock +(6 rows) + COMMIT; -- test locks on INSERT/SELECT CREATE TABLE partitioning_locks_for_select(id int, ref_id int, time date); @@ -1014,46 +1054,91 @@ SELECT create_distributed_table('partitioning_locks_for_select', 'id'); BEGIN; INSERT INTO partitioning_locks SELECT * FROM partitioning_locks_for_select; -ERROR: relation "partitioning_locks" does not exist -LINE 1: INSERT INTO partitioning_locks SELECT * FROM partitioning_lo... - ^ SELECT relation::regclass, locktype, mode FROM pg_locks WHERE relation::regclass::text LIKE 'partitioning_locks%' AND pid = pg_backend_pid() ORDER BY 1, 2, 3; -ERROR: current transaction is aborted, commands ignored until end of transaction block + relation | locktype | mode +-------------------------------+----------+------------------ + partitioning_locks | relation | AccessShareLock + partitioning_locks | relation | RowExclusiveLock + partitioning_locks_2009 | relation | AccessShareLock + partitioning_locks_2009 | relation | RowExclusiveLock + partitioning_locks_2010 | relation | AccessShareLock + partitioning_locks_2010 | relation | RowExclusiveLock + partitioning_locks_for_select | relation | AccessShareLock +(7 rows) + COMMIT; -- test locks on coordinator INSERT/SELECT BEGIN; INSERT INTO partitioning_locks SELECT * FROM partitioning_locks_for_select LIMIT 5; -ERROR: relation "partitioning_locks" does not exist -LINE 1: INSERT INTO partitioning_locks SELECT * FROM partitioning_lo... - ^ SELECT relation::regclass, locktype, mode FROM pg_locks WHERE relation::regclass::text LIKE 'partitioning_locks%' AND pid = pg_backend_pid() ORDER BY 1, 2, 3; -ERROR: current transaction is aborted, commands ignored until end of transaction block + relation | locktype | mode +-------------------------------+----------+------------------ + partitioning_locks | relation | AccessShareLock + partitioning_locks | relation | RowExclusiveLock + partitioning_locks_2009 | relation | RowExclusiveLock + partitioning_locks_2010 | relation | RowExclusiveLock + partitioning_locks_for_select | relation | AccessShareLock +(5 rows) + COMMIT; -- test locks on master_modify_multiple_shards BEGIN; SELECT master_modify_multiple_shards('UPDATE partitioning_locks SET time = ''2009-03-01'''); -ERROR: relation "partitioning_locks" does not exist + master_modify_multiple_shards +------------------------------- + 0 +(1 row) + SELECT relation::regclass, locktype, mode FROM pg_locks WHERE relation::regclass::text LIKE 'partitioning_locks%' AND pid = pg_backend_pid() ORDER BY 1, 2, 3; -ERROR: current transaction is aborted, commands ignored until end of transaction block + relation | locktype | mode +-------------------------+----------+------------------ + partitioning_locks | relation | AccessShareLock + partitioning_locks | relation | RowExclusiveLock + partitioning_locks_2009 | relation | RowExclusiveLock + partitioning_locks_2010 | relation | RowExclusiveLock +(4 rows) + COMMIT; -- test locks on DDL BEGIN; ALTER TABLE partitioning_locks ADD COLUMN new_column int; -ERROR: relation "partitioning_locks" does not exist SELECT relation::regclass, locktype, mode FROM pg_locks WHERE relation::regclass::text LIKE 'partitioning_locks%' AND pid = pg_backend_pid() ORDER BY 1, 2, 3; -ERROR: current transaction is aborted, commands ignored until end of transaction block + relation | locktype | mode +-------------------------+----------+--------------------- + partitioning_locks | relation | AccessExclusiveLock + partitioning_locks | relation | AccessShareLock + partitioning_locks_2009 | relation | AccessExclusiveLock + partitioning_locks_2010 | relation | AccessExclusiveLock +(4 rows) + COMMIT; -- test locks on TRUNCATE BEGIN; TRUNCATE partitioning_locks; -ERROR: relation "partitioning_locks" does not exist SELECT relation::regclass, locktype, mode FROM pg_locks WHERE relation::regclass::text LIKE 'partitioning_locks%' AND pid = pg_backend_pid() ORDER BY 1, 2, 3; -ERROR: current transaction is aborted, commands ignored until end of transaction block + relation | locktype | mode +-------------------------+----------+--------------------- + partitioning_locks | relation | AccessExclusiveLock + partitioning_locks | relation | AccessShareLock + partitioning_locks_2009 | relation | AccessExclusiveLock + partitioning_locks_2009 | relation | AccessShareLock + partitioning_locks_2009 | relation | RowExclusiveLock + partitioning_locks_2009 | relation | ShareLock + partitioning_locks_2010 | relation | AccessExclusiveLock + partitioning_locks_2010 | relation | AccessShareLock + partitioning_locks_2010 | relation | RowExclusiveLock + partitioning_locks_2010 | relation | ShareLock +(10 rows) + COMMIT; -- test shard resource locks with master_modify_multiple_shards BEGIN; SELECT master_modify_multiple_shards('UPDATE partitioning_locks_2009 SET time = ''2009-03-01'''); -ERROR: relation "partitioning_locks_2009" does not exist + master_modify_multiple_shards +------------------------------- + 0 +(1 row) + -- see the locks on parent table SELECT logicalrelid, @@ -1068,12 +1153,26 @@ WHERE pid = pg_backend_pid() ORDER BY 1, 2, 3; -ERROR: current transaction is aborted, commands ignored until end of transaction block + logicalrelid | locktype | mode +-------------------------+----------+-------------------------- + partitioning_locks | advisory | ShareUpdateExclusiveLock + partitioning_locks | advisory | ShareUpdateExclusiveLock + partitioning_locks | advisory | ShareUpdateExclusiveLock + partitioning_locks | advisory | ShareUpdateExclusiveLock + partitioning_locks_2009 | advisory | ShareLock + partitioning_locks_2009 | advisory | ShareLock + partitioning_locks_2009 | advisory | ShareLock + partitioning_locks_2009 | advisory | ShareLock + partitioning_locks_2009 | advisory | ShareUpdateExclusiveLock + partitioning_locks_2009 | advisory | ShareUpdateExclusiveLock + partitioning_locks_2009 | advisory | ShareUpdateExclusiveLock + partitioning_locks_2009 | advisory | ShareUpdateExclusiveLock +(12 rows) + COMMIT; -- test shard resource locks with TRUNCATE BEGIN; TRUNCATE partitioning_locks_2009; -ERROR: relation "partitioning_locks_2009" does not exist -- see the locks on parent table SELECT logicalrelid, @@ -1088,14 +1187,26 @@ WHERE pid = pg_backend_pid() ORDER BY 1, 2, 3; -ERROR: current transaction is aborted, commands ignored until end of transaction block + logicalrelid | locktype | mode +-------------------------+----------+-------------------------- + partitioning_locks | advisory | ShareUpdateExclusiveLock + partitioning_locks | advisory | ShareUpdateExclusiveLock + partitioning_locks | advisory | ShareUpdateExclusiveLock + partitioning_locks | advisory | ShareUpdateExclusiveLock + partitioning_locks_2009 | advisory | ShareLock + partitioning_locks_2009 | advisory | ShareLock + partitioning_locks_2009 | advisory | ShareLock + partitioning_locks_2009 | advisory | ShareLock + partitioning_locks_2009 | advisory | ShareUpdateExclusiveLock + partitioning_locks_2009 | advisory | ShareUpdateExclusiveLock + partitioning_locks_2009 | advisory | ShareUpdateExclusiveLock + partitioning_locks_2009 | advisory | ShareUpdateExclusiveLock +(12 rows) + COMMIT; -- test shard resource locks with INSERT/SELECT BEGIN; INSERT INTO partitioning_locks_2009 SELECT * FROM partitioning_locks WHERE time >= '2009-01-01' AND time < '2010-01-01'; -ERROR: relation "partitioning_locks_2009" does not exist -LINE 1: INSERT INTO partitioning_locks_2009 SELECT * FROM partitioni... - ^ -- see the locks on parent table SELECT logicalrelid, @@ -1110,7 +1221,22 @@ WHERE pid = pg_backend_pid() ORDER BY 1, 2, 3; -ERROR: current transaction is aborted, commands ignored until end of transaction block + logicalrelid | locktype | mode +-------------------------+----------+-------------------------- + partitioning_locks | advisory | ShareUpdateExclusiveLock + partitioning_locks | advisory | ShareUpdateExclusiveLock + partitioning_locks | advisory | ShareUpdateExclusiveLock + partitioning_locks | advisory | ShareUpdateExclusiveLock + partitioning_locks_2009 | advisory | ShareLock + partitioning_locks_2009 | advisory | ShareLock + partitioning_locks_2009 | advisory | ShareLock + partitioning_locks_2009 | advisory | ShareLock + partitioning_locks_2009 | advisory | ShareUpdateExclusiveLock + partitioning_locks_2009 | advisory | ShareUpdateExclusiveLock + partitioning_locks_2009 | advisory | ShareUpdateExclusiveLock + partitioning_locks_2009 | advisory | ShareUpdateExclusiveLock +(12 rows) + COMMIT; DROP TABLE IF EXISTS @@ -1121,43 +1247,28 @@ IF EXISTS multi_column_partitioning, partitioning_locks, partitioning_locks_for_select; -NOTICE: table "partitioning_test_2009" does not exist, skipping -NOTICE: table "partitioned_events_table" does not exist, skipping -NOTICE: table "partitioned_users_table" does not exist, skipping -NOTICE: table "list_partitioned_events_table" does not exist, skipping -NOTICE: table "multi_column_partitioning" does not exist, skipping -NOTICE: table "partitioning_locks" does not exist, skipping -- make sure we can create a partitioned table with streaming replication SET citus.replication_model TO 'streaming'; CREATE TABLE partitioning_test(id int, time date) PARTITION BY RANGE (time); -ERROR: syntax error at or near "PARTITION" -LINE 1: CREATE TABLE partitioning_test(id int, time date) PARTITION ... - ^ CREATE TABLE partitioning_test_2009 PARTITION OF partitioning_test FOR VALUES FROM ('2009-01-01') TO ('2010-01-01'); -ERROR: syntax error at or near "PARTITION" -LINE 1: CREATE TABLE partitioning_test_2009 PARTITION OF partitionin... - ^ SELECT create_distributed_table('partitioning_test', 'id'); -ERROR: relation "partitioning_test" does not exist -LINE 1: SELECT create_distributed_table('partitioning_test', 'id'); - ^ + create_distributed_table +-------------------------- + +(1 row) + DROP TABLE partitioning_test; -ERROR: table "partitioning_test" does not exist -- make sure we can attach partitions to a distributed table in a schema CREATE SCHEMA partitioning_schema; CREATE TABLE partitioning_schema."schema-test"(id int, time date) PARTITION BY RANGE (time); -ERROR: syntax error at or near "PARTITION" -LINE 1: ...titioning_schema."schema-test"(id int, time date) PARTITION ... - ^ SELECT create_distributed_table('partitioning_schema."schema-test"', 'id'); -ERROR: relation "partitioning_schema.schema-test" does not exist -LINE 1: SELECT create_distributed_table('partitioning_schema."schema... - ^ + create_distributed_table +-------------------------- + +(1 row) + CREATE TABLE partitioning_schema."schema-test_2009"(id int, time date); ALTER TABLE partitioning_schema."schema-test" ATTACH PARTITION partitioning_schema."schema-test_2009" FOR VALUES FROM ('2009-01-01') TO ('2010-01-01'); -ERROR: syntax error at or near "ATTACH" -LINE 1: ALTER TABLE partitioning_schema."schema-test" ATTACH PARTITI... - ^ -- attached partition is distributed as well SELECT logicalrelid @@ -1166,9 +1277,12 @@ FROM WHERE logicalrelid IN ('partitioning_schema."schema-test"'::regclass, 'partitioning_schema."schema-test_2009"'::regclass) ORDER BY 1; -ERROR: relation "partitioning_schema.schema-test" does not exist -LINE 6: logicalrelid IN ('partitioning_schema."schema-test"'::regcl... - ^ + logicalrelid +---------------------------------------- + partitioning_schema."schema-test" + partitioning_schema."schema-test_2009" +(2 rows) + SELECT logicalrelid, count(*) FROM @@ -1179,24 +1293,22 @@ GROUP BY logicalrelid ORDER BY 1,2; -ERROR: relation "partitioning_schema.schema-test" does not exist -LINE 6: logicalrelid IN ('partitioning_schema."schema-test"'::re... - ^ + logicalrelid | count +----------------------------------------+------- + partitioning_schema."schema-test" | 4 + partitioning_schema."schema-test_2009" | 4 +(2 rows) + DROP TABLE partitioning_schema."schema-test"; -ERROR: table "schema-test" does not exist -- make sure we can create partition of a distributed table in a schema CREATE TABLE partitioning_schema."schema-test"(id int, time date) PARTITION BY RANGE (time); -ERROR: syntax error at or near "PARTITION" -LINE 1: ...titioning_schema."schema-test"(id int, time date) PARTITION ... - ^ SELECT create_distributed_table('partitioning_schema."schema-test"', 'id'); -ERROR: relation "partitioning_schema.schema-test" does not exist -LINE 1: SELECT create_distributed_table('partitioning_schema."schema... - ^ + create_distributed_table +-------------------------- + +(1 row) + CREATE TABLE partitioning_schema."schema-test_2009" PARTITION OF partitioning_schema."schema-test" FOR VALUES FROM ('2009-01-01') TO ('2010-01-01'); -ERROR: syntax error at or near "PARTITION" -LINE 1: ...EATE TABLE partitioning_schema."schema-test_2009" PARTITION ... - ^ -- newly created partition is distributed as well SELECT logicalrelid @@ -1205,9 +1317,12 @@ FROM WHERE logicalrelid IN ('partitioning_schema."schema-test"'::regclass, 'partitioning_schema."schema-test_2009"'::regclass) ORDER BY 1; -ERROR: relation "partitioning_schema.schema-test" does not exist -LINE 6: logicalrelid IN ('partitioning_schema."schema-test"'::regcl... - ^ + logicalrelid +---------------------------------------- + partitioning_schema."schema-test" + partitioning_schema."schema-test_2009" +(2 rows) + SELECT logicalrelid, count(*) FROM @@ -1218,25 +1333,23 @@ GROUP BY logicalrelid ORDER BY 1,2; -ERROR: relation "partitioning_schema.schema-test" does not exist -LINE 6: logicalrelid IN ('partitioning_schema."schema-test"'::re... - ^ + logicalrelid | count +----------------------------------------+------- + partitioning_schema."schema-test" | 4 + partitioning_schema."schema-test_2009" | 4 +(2 rows) + DROP TABLE partitioning_schema."schema-test"; -ERROR: table "schema-test" does not exist -- make sure creating partitioned tables works while search_path is set CREATE TABLE partitioning_schema."schema-test"(id int, time date) PARTITION BY RANGE (time); -ERROR: syntax error at or near "PARTITION" -LINE 1: ...titioning_schema."schema-test"(id int, time date) PARTITION ... - ^ SET search_path = partitioning_schema; SELECT create_distributed_table('"schema-test"', 'id'); -ERROR: relation "schema-test" does not exist -LINE 1: SELECT create_distributed_table('"schema-test"', 'id'); - ^ + create_distributed_table +-------------------------- + +(1 row) + CREATE TABLE partitioning_schema."schema-test_2009" PARTITION OF "schema-test" FOR VALUES FROM ('2009-01-01') TO ('2010-01-01'); -ERROR: syntax error at or near "PARTITION" -LINE 1: ...EATE TABLE partitioning_schema."schema-test_2009" PARTITION ... - ^ -- newly created partition is distributed as well SELECT logicalrelid @@ -1245,9 +1358,12 @@ FROM WHERE logicalrelid IN ('partitioning_schema."schema-test"'::regclass, 'partitioning_schema."schema-test_2009"'::regclass) ORDER BY 1; -ERROR: relation "partitioning_schema.schema-test" does not exist -LINE 6: logicalrelid IN ('partitioning_schema."schema-test"'::regcl... - ^ + logicalrelid +-------------------- + "schema-test" + "schema-test_2009" +(2 rows) + SELECT logicalrelid, count(*) FROM @@ -1258,8 +1374,11 @@ GROUP BY logicalrelid ORDER BY 1,2; -ERROR: relation "partitioning_schema.schema-test" does not exist -LINE 6: logicalrelid IN ('partitioning_schema."schema-test"'::re... - ^ + logicalrelid | count +--------------------+------- + "schema-test" | 4 + "schema-test_2009" | 4 +(2 rows) + DROP SCHEMA partitioning_schema CASCADE; -NOTICE: drop cascades to table "schema-test_2009" +NOTICE: drop cascades to table "schema-test" diff --git a/src/test/regress/expected/multi_partitioning_1.out b/src/test/regress/expected/multi_partitioning_1.out new file mode 100644 index 000000000..48909e0f8 --- /dev/null +++ b/src/test/regress/expected/multi_partitioning_1.out @@ -0,0 +1,1274 @@ +-- +-- Distributed Partitioned Table Tests +-- +SET citus.next_shard_id TO 1660000; +SET citus.shard_count TO 4; +SET citus.shard_replication_factor TO 1; +-- print major version number for version-specific tests +SHOW server_version \gset +SELECT substring(:'server_version', '\d+')::int AS server_version; + server_version +---------------- + 9 +(1 row) + +-- +-- Distributed Partitioned Table Creation Tests +-- +-- 1-) Distributing partitioned table +-- create partitioned table +CREATE TABLE partitioning_test(id int, time date) PARTITION BY RANGE (time); +ERROR: syntax error at or near "PARTITION" +LINE 1: CREATE TABLE partitioning_test(id int, time date) PARTITION ... + ^ + +-- create its partitions +CREATE TABLE partitioning_test_2009 PARTITION OF partitioning_test FOR VALUES FROM ('2009-01-01') TO ('2010-01-01'); +ERROR: syntax error at or near "PARTITION" +LINE 1: CREATE TABLE partitioning_test_2009 PARTITION OF partitionin... + ^ +CREATE TABLE partitioning_test_2010 PARTITION OF partitioning_test FOR VALUES FROM ('2010-01-01') TO ('2011-01-01'); +ERROR: syntax error at or near "PARTITION" +LINE 1: CREATE TABLE partitioning_test_2010 PARTITION OF partitionin... + ^ +-- load some data and distribute tables +INSERT INTO partitioning_test VALUES (1, '2009-06-06'); +ERROR: relation "partitioning_test" does not exist +LINE 1: INSERT INTO partitioning_test VALUES (1, '2009-06-06'); + ^ +INSERT INTO partitioning_test VALUES (2, '2010-07-07'); +ERROR: relation "partitioning_test" does not exist +LINE 1: INSERT INTO partitioning_test VALUES (2, '2010-07-07'); + ^ +INSERT INTO partitioning_test_2009 VALUES (3, '2009-09-09'); +ERROR: relation "partitioning_test_2009" does not exist +LINE 1: INSERT INTO partitioning_test_2009 VALUES (3, '2009-09-09'); + ^ +INSERT INTO partitioning_test_2010 VALUES (4, '2010-03-03'); +ERROR: relation "partitioning_test_2010" does not exist +LINE 1: INSERT INTO partitioning_test_2010 VALUES (4, '2010-03-03'); + ^ +-- distribute partitioned table +SELECT create_distributed_table('partitioning_test', 'id'); +ERROR: relation "partitioning_test" does not exist +LINE 1: SELECT create_distributed_table('partitioning_test', 'id'); + ^ +-- see the data is loaded to shards +SELECT * FROM partitioning_test ORDER BY 1; +ERROR: relation "partitioning_test" does not exist +LINE 1: SELECT * FROM partitioning_test ORDER BY 1; + ^ +-- see partitioned table and its partitions are distributed +SELECT + logicalrelid +FROM + pg_dist_partition +WHERE + logicalrelid IN ('partitioning_test', 'partitioning_test_2009', 'partitioning_test_2010') +ORDER BY 1; +ERROR: relation "partitioning_test" does not exist +LINE 6: logicalrelid IN ('partitioning_test', 'partitioning_test_20... + ^ +SELECT + logicalrelid, count(*) +FROM pg_dist_shard + WHERE logicalrelid IN ('partitioning_test', 'partitioning_test_2009', 'partitioning_test_2010') +GROUP BY + logicalrelid +ORDER BY + 1,2; +ERROR: relation "partitioning_test" does not exist +LINE 4: WHERE logicalrelid IN ('partitioning_test', 'partitioning_t... + ^ +-- 2-) Creating partition of a distributed table +CREATE TABLE partitioning_test_2011 PARTITION OF partitioning_test FOR VALUES FROM ('2011-01-01') TO ('2012-01-01'); +ERROR: syntax error at or near "PARTITION" +LINE 1: CREATE TABLE partitioning_test_2011 PARTITION OF partitionin... + ^ +-- new partition is automatically distributed as well +SELECT + logicalrelid +FROM + pg_dist_partition +WHERE + logicalrelid IN ('partitioning_test', 'partitioning_test_2011') +ORDER BY 1; +ERROR: relation "partitioning_test" does not exist +LINE 6: logicalrelid IN ('partitioning_test', 'partitioning_test_20... + ^ +SELECT + logicalrelid, count(*) +FROM pg_dist_shard + WHERE logicalrelid IN ('partitioning_test', 'partitioning_test_2011') +GROUP BY + logicalrelid +ORDER BY + 1,2; +ERROR: relation "partitioning_test" does not exist +LINE 4: WHERE logicalrelid IN ('partitioning_test', 'partitioning_t... + ^ +-- 3-) Attaching non distributed table to a distributed table +CREATE TABLE partitioning_test_2012(id int, time date); +-- load some data +INSERT INTO partitioning_test_2012 VALUES (5, '2012-06-06'); +INSERT INTO partitioning_test_2012 VALUES (6, '2012-07-07'); +ALTER TABLE partitioning_test ATTACH PARTITION partitioning_test_2012 FOR VALUES FROM ('2012-01-01') TO ('2013-01-01'); +ERROR: syntax error at or near "ATTACH" +LINE 1: ALTER TABLE partitioning_test ATTACH PARTITION partitioning_... + ^ +-- attached partition is distributed as well +SELECT + logicalrelid +FROM + pg_dist_partition +WHERE + logicalrelid IN ('partitioning_test', 'partitioning_test_2012') +ORDER BY 1; +ERROR: relation "partitioning_test" does not exist +LINE 6: logicalrelid IN ('partitioning_test', 'partitioning_test_20... + ^ +SELECT + logicalrelid, count(*) +FROM pg_dist_shard + WHERE logicalrelid IN ('partitioning_test', 'partitioning_test_2012') +GROUP BY + logicalrelid +ORDER BY + 1,2; +ERROR: relation "partitioning_test" does not exist +LINE 4: WHERE logicalrelid IN ('partitioning_test', 'partitioning_t... + ^ +-- see the data is loaded to shards +SELECT * FROM partitioning_test ORDER BY 1; +ERROR: relation "partitioning_test" does not exist +LINE 1: SELECT * FROM partitioning_test ORDER BY 1; + ^ +-- 4-) Attaching distributed table to distributed table +CREATE TABLE partitioning_test_2013(id int, time date); +SELECT create_distributed_table('partitioning_test_2013', 'id'); + create_distributed_table +-------------------------- + +(1 row) + +-- load some data +INSERT INTO partitioning_test_2013 VALUES (7, '2013-06-06'); +INSERT INTO partitioning_test_2013 VALUES (8, '2013-07-07'); +ALTER TABLE partitioning_test ATTACH PARTITION partitioning_test_2013 FOR VALUES FROM ('2013-01-01') TO ('2014-01-01'); +ERROR: syntax error at or near "ATTACH" +LINE 1: ALTER TABLE partitioning_test ATTACH PARTITION partitioning_... + ^ +-- see the data is loaded to shards +SELECT * FROM partitioning_test ORDER BY 1; +ERROR: relation "partitioning_test" does not exist +LINE 1: SELECT * FROM partitioning_test ORDER BY 1; + ^ +-- 5-) Failure cases while creating distributed partitioned tables +-- cannot distribute a partition if its parent is not distributed +CREATE TABLE partitioning_test_failure(id int, time date) PARTITION BY RANGE (time); +ERROR: syntax error at or near "PARTITION" +LINE 1: ...ABLE partitioning_test_failure(id int, time date) PARTITION ... + ^ +CREATE TABLE partitioning_test_failure_2009 PARTITION OF partitioning_test_failure FOR VALUES FROM ('2009-01-01') TO ('2010-01-01'); +ERROR: syntax error at or near "PARTITION" +LINE 1: CREATE TABLE partitioning_test_failure_2009 PARTITION OF par... + ^ +SELECT create_distributed_table('partitioning_test_failure_2009', 'id'); +ERROR: relation "partitioning_test_failure_2009" does not exist +LINE 1: SELECT create_distributed_table('partitioning_test_failure_2... + ^ +-- only hash distributed tables can have partitions +SELECT create_distributed_table('partitioning_test_failure', 'id', 'append'); +ERROR: relation "partitioning_test_failure" does not exist +LINE 1: SELECT create_distributed_table('partitioning_test_failure',... + ^ +SELECT create_distributed_table('partitioning_test_failure', 'id', 'range'); +ERROR: relation "partitioning_test_failure" does not exist +LINE 1: SELECT create_distributed_table('partitioning_test_failure',... + ^ +SELECT create_reference_table('partitioning_test_failure'); +ERROR: relation "partitioning_test_failure" does not exist +LINE 1: SELECT create_reference_table('partitioning_test_failure'); + ^ +-- replication factor > 1 is not allowed in distributed partitioned tables +SET citus.shard_replication_factor TO 2; +SELECT create_distributed_table('partitioning_test_failure', 'id'); +ERROR: relation "partitioning_test_failure" does not exist +LINE 1: SELECT create_distributed_table('partitioning_test_failure',... + ^ +SET citus.shard_replication_factor TO 1; +-- non-distributed tables cannot have distributed partitions; +DROP TABLE partitioning_test_failure_2009; +ERROR: table "partitioning_test_failure_2009" does not exist +CREATE TABLE partitioning_test_failure_2009(id int, time date); +SELECT create_distributed_table('partitioning_test_failure_2009', 'id'); + create_distributed_table +-------------------------- + +(1 row) + +ALTER TABLE partitioning_test_failure ATTACH PARTITION partitioning_test_failure_2009 FOR VALUES FROM ('2009-01-01') TO ('2010-01-01'); +ERROR: syntax error at or near "ATTACH" +LINE 1: ALTER TABLE partitioning_test_failure ATTACH PARTITION parti... + ^ +-- multi-level partitioning is not allowed +DROP TABLE partitioning_test_failure_2009; +CREATE TABLE partitioning_test_failure_2009 PARTITION OF partitioning_test_failure FOR VALUES FROM ('2009-01-01') TO ('2010-01-01') PARTITION BY RANGE (time); +ERROR: syntax error at or near "PARTITION" +LINE 1: CREATE TABLE partitioning_test_failure_2009 PARTITION OF par... + ^ +SELECT create_distributed_table('partitioning_test_failure', 'id'); +ERROR: relation "partitioning_test_failure" does not exist +LINE 1: SELECT create_distributed_table('partitioning_test_failure',... + ^ +-- multi-level partitioning is not allowed in different order +DROP TABLE partitioning_test_failure_2009; +ERROR: table "partitioning_test_failure_2009" does not exist +SELECT create_distributed_table('partitioning_test_failure', 'id'); +ERROR: relation "partitioning_test_failure" does not exist +LINE 1: SELECT create_distributed_table('partitioning_test_failure',... + ^ +CREATE TABLE partitioning_test_failure_2009 PARTITION OF partitioning_test_failure FOR VALUES FROM ('2009-01-01') TO ('2010-01-01') PARTITION BY RANGE (time); +ERROR: syntax error at or near "PARTITION" +LINE 1: CREATE TABLE partitioning_test_failure_2009 PARTITION OF par... + ^ +-- +-- DMLs in distributed partitioned tables +-- +-- test COPY +-- COPY data to partitioned table +COPY partitioning_test FROM STDIN WITH CSV; +ERROR: relation "partitioning_test" does not exist +9,2009-01-01 +10,2010-01-01 +11,2011-01-01 +12,2012-01-01 +\. +invalid command \. +-- COPY data to partition directly +COPY partitioning_test_2009 FROM STDIN WITH CSV; +ERROR: syntax error at or near "9" +LINE 1: 9,2009-01-01 + ^ +13,2009-01-02 +14,2009-01-03 +\. +invalid command \. +-- see the data is loaded to shards +SELECT * FROM partitioning_test WHERE id >= 9 ORDER BY 1; +ERROR: syntax error at or near "13" +LINE 1: 13,2009-01-02 + ^ +-- test INSERT +-- INSERT INTO the partitioned table +INSERT INTO partitioning_test VALUES(15, '2009-02-01'); +ERROR: relation "partitioning_test" does not exist +LINE 1: INSERT INTO partitioning_test VALUES(15, '2009-02-01'); + ^ +INSERT INTO partitioning_test VALUES(16, '2010-02-01'); +ERROR: relation "partitioning_test" does not exist +LINE 1: INSERT INTO partitioning_test VALUES(16, '2010-02-01'); + ^ +INSERT INTO partitioning_test VALUES(17, '2011-02-01'); +ERROR: relation "partitioning_test" does not exist +LINE 1: INSERT INTO partitioning_test VALUES(17, '2011-02-01'); + ^ +INSERT INTO partitioning_test VALUES(18, '2012-02-01'); +ERROR: relation "partitioning_test" does not exist +LINE 1: INSERT INTO partitioning_test VALUES(18, '2012-02-01'); + ^ +-- INSERT INTO the partitions directly table +INSERT INTO partitioning_test VALUES(19, '2009-02-02'); +ERROR: relation "partitioning_test" does not exist +LINE 1: INSERT INTO partitioning_test VALUES(19, '2009-02-02'); + ^ +INSERT INTO partitioning_test VALUES(20, '2010-02-02'); +ERROR: relation "partitioning_test" does not exist +LINE 1: INSERT INTO partitioning_test VALUES(20, '2010-02-02'); + ^ +-- see the data is loaded to shards +SELECT * FROM partitioning_test WHERE id >= 15 ORDER BY 1; +ERROR: relation "partitioning_test" does not exist +LINE 1: SELECT * FROM partitioning_test WHERE id >= 15 ORDER BY 1; + ^ +-- test INSERT/SELECT +-- INSERT/SELECT from partition to partitioned table +INSERT INTO partitioning_test SELECT * FROM partitioning_test_2011; +ERROR: relation "partitioning_test" does not exist +LINE 1: INSERT INTO partitioning_test SELECT * FROM partitioning_tes... + ^ +-- INSERT/SELECT from partitioned table to partition +INSERT INTO partitioning_test_2012 SELECT * FROM partitioning_test WHERE time >= '2012-01-01' AND time < '2013-01-01'; +ERROR: relation "partitioning_test" does not exist +LINE 1: INSERT INTO partitioning_test_2012 SELECT * FROM partitionin... + ^ +-- see the data is loaded to shards (rows in the given range should be duplicated) +SELECT * FROM partitioning_test WHERE time >= '2011-01-01' AND time < '2013-01-01' ORDER BY 1; +ERROR: relation "partitioning_test" does not exist +LINE 1: SELECT * FROM partitioning_test WHERE time >= '2011-01-01' A... + ^ +-- test UPDATE +-- UPDATE partitioned table +UPDATE partitioning_test SET time = '2013-07-07' WHERE id = 7; +ERROR: relation "partitioning_test" does not exist +LINE 1: UPDATE partitioning_test SET time = '2013-07-07' WHERE id = ... + ^ +-- UPDATE partition directly +UPDATE partitioning_test_2013 SET time = '2013-08-08' WHERE id = 8; +-- see the data is updated +SELECT * FROM partitioning_test WHERE id = 7 OR id = 8 ORDER BY 1; +ERROR: relation "partitioning_test" does not exist +LINE 1: SELECT * FROM partitioning_test WHERE id = 7 OR id = 8 ORDER... + ^ +-- UPDATE that tries to move a row to a non-existing partition (this should fail) +UPDATE partitioning_test SET time = '2020-07-07' WHERE id = 7; +ERROR: relation "partitioning_test" does not exist +LINE 1: UPDATE partitioning_test SET time = '2020-07-07' WHERE id = ... + ^ +-- UPDATE with subqueries on partitioned table +UPDATE + partitioning_test +SET + time = time + INTERVAL '1 day' +WHERE + id IN (SELECT id FROM partitioning_test WHERE id = 1); +ERROR: relation "partitioning_test" does not exist +LINE 2: partitioning_test + ^ +-- UPDATE with subqueries on partition +UPDATE + partitioning_test_2009 +SET + time = time + INTERVAL '1 month' +WHERE + id IN (SELECT id FROM partitioning_test WHERE id = 2); +ERROR: relation "partitioning_test_2009" does not exist +LINE 2: partitioning_test_2009 + ^ +-- see the data is updated +SELECT * FROM partitioning_test WHERE id = 1 OR id = 2 ORDER BY 1; +ERROR: relation "partitioning_test" does not exist +LINE 1: SELECT * FROM partitioning_test WHERE id = 1 OR id = 2 ORDER... + ^ +-- test DELETE +-- DELETE from partitioned table +DELETE FROM partitioning_test WHERE id = 9; +ERROR: relation "partitioning_test" does not exist +LINE 1: DELETE FROM partitioning_test WHERE id = 9; + ^ +-- DELETE from partition directly +DELETE FROM partitioning_test_2010 WHERE id = 10; +ERROR: relation "partitioning_test_2010" does not exist +LINE 1: DELETE FROM partitioning_test_2010 WHERE id = 10; + ^ +-- see the data is deleted +SELECT * FROM partitioning_test WHERE id = 9 OR id = 10 ORDER BY 1; +ERROR: relation "partitioning_test" does not exist +LINE 1: SELECT * FROM partitioning_test WHERE id = 9 OR id = 10 ORDE... + ^ +-- test master_modify_multiple_shards +-- master_modify_multiple_shards on partitioned table +SELECT master_modify_multiple_shards('UPDATE partitioning_test SET time = time + INTERVAL ''1 day'''); +ERROR: relation "partitioning_test" does not exist +-- see rows are UPDATED +SELECT * FROM partitioning_test ORDER BY 1; +ERROR: relation "partitioning_test" does not exist +LINE 1: SELECT * FROM partitioning_test ORDER BY 1; + ^ +-- master_modify_multiple_shards on partition directly +SELECT master_modify_multiple_shards('UPDATE partitioning_test_2009 SET time = time + INTERVAL ''1 day'''); +ERROR: relation "partitioning_test_2009" does not exist +-- see rows are UPDATED +SELECT * FROM partitioning_test_2009 ORDER BY 1; +ERROR: relation "partitioning_test_2009" does not exist +LINE 1: SELECT * FROM partitioning_test_2009 ORDER BY 1; + ^ +-- test master_modify_multiple_shards which fails in workers (updated value is outside of partition bounds) +SELECT master_modify_multiple_shards('UPDATE partitioning_test_2009 SET time = time + INTERVAL ''6 month'''); +ERROR: relation "partitioning_test_2009" does not exist +-- +-- DDL in distributed partitioned tables +-- +-- test CREATE INDEX +-- CREATE INDEX on partitioned table - this will error out +-- on earlier versions of postgres earlier than 11. +CREATE INDEX partitioning_index ON partitioning_test(id); +ERROR: relation "partitioning_test" does not exist +-- CREATE INDEX on partition +CREATE INDEX partitioning_2009_index ON partitioning_test_2009(id); +ERROR: relation "partitioning_test_2009" does not exist +-- CREATE INDEX CONCURRENTLY on partition +CREATE INDEX CONCURRENTLY partitioned_2010_index ON partitioning_test_2010(id); +ERROR: relation "partitioning_test_2010" does not exist +-- see index is created +SELECT tablename, indexname FROM pg_indexes WHERE tablename LIKE 'partitioning_test%' ORDER BY indexname; + tablename | indexname +-----------+----------- +(0 rows) + +-- test add COLUMN +-- add COLUMN to partitioned table +ALTER TABLE partitioning_test ADD new_column int; +ERROR: relation "partitioning_test" does not exist +-- add COLUMN to partition - this will error out +ALTER TABLE partitioning_test_2010 ADD new_column_2 int; +ERROR: relation "partitioning_test_2010" does not exist +-- see additional column is created +SELECT name, type FROM table_attrs WHERE relid = 'partitioning_test'::regclass ORDER BY 1; +ERROR: relation "partitioning_test" does not exist +LINE 1: SELECT name, type FROM table_attrs WHERE relid = 'partitioni... + ^ +SELECT name, type FROM table_attrs WHERE relid = 'partitioning_test_2010'::regclass ORDER BY 1; +ERROR: relation "partitioning_test_2010" does not exist +LINE 1: SELECT name, type FROM table_attrs WHERE relid = 'partitioni... + ^ +-- test add PRIMARY KEY +-- add PRIMARY KEY to partitioned table - this will error out +ALTER TABLE partitioning_test ADD CONSTRAINT partitioning_primary PRIMARY KEY (id); +ERROR: relation "partitioning_test" does not exist +-- ADD PRIMARY KEY to partition +ALTER TABLE partitioning_test_2009 ADD CONSTRAINT partitioning_2009_primary PRIMARY KEY (id); +ERROR: relation "partitioning_test_2009" does not exist +-- see PRIMARY KEY is created +SELECT + table_name, + constraint_name, + constraint_type +FROM + information_schema.table_constraints +WHERE + table_name = 'partitioning_test_2009' AND + constraint_name = 'partitioning_2009_primary'; + table_name | constraint_name | constraint_type +------------+-----------------+----------------- +(0 rows) + +-- test ADD FOREIGN CONSTRAINT +-- add FOREIGN CONSTRAINT to partitioned table -- this will error out +ALTER TABLE partitioning_test ADD CONSTRAINT partitioning_foreign FOREIGN KEY (id) REFERENCES partitioning_test_2009 (id); +ERROR: relation "partitioning_test" does not exist +-- add FOREIGN CONSTRAINT to partition +INSERT INTO partitioning_test_2009 VALUES (5, '2009-06-06'); +ERROR: relation "partitioning_test_2009" does not exist +LINE 1: INSERT INTO partitioning_test_2009 VALUES (5, '2009-06-06'); + ^ +INSERT INTO partitioning_test_2009 VALUES (6, '2009-07-07'); +ERROR: relation "partitioning_test_2009" does not exist +LINE 1: INSERT INTO partitioning_test_2009 VALUES (6, '2009-07-07'); + ^ +INSERT INTO partitioning_test_2009 VALUES(12, '2009-02-01'); +ERROR: relation "partitioning_test_2009" does not exist +LINE 1: INSERT INTO partitioning_test_2009 VALUES(12, '2009-02-01'); + ^ +INSERT INTO partitioning_test_2009 VALUES(18, '2009-02-01'); +ERROR: relation "partitioning_test_2009" does not exist +LINE 1: INSERT INTO partitioning_test_2009 VALUES(18, '2009-02-01'); + ^ +ALTER TABLE partitioning_test_2012 ADD CONSTRAINT partitioning_2012_foreign FOREIGN KEY (id) REFERENCES partitioning_test_2009 (id) ON DELETE CASCADE; +ERROR: relation "partitioning_test_2009" does not exist +-- see FOREIGN KEY is created +SELECT "Constraint" FROM table_fkeys WHERE relid = 'partitioning_test_2012'::regclass ORDER BY 1; + Constraint +------------ +(0 rows) + +-- test ON DELETE CASCADE works +DELETE FROM partitioning_test_2009 WHERE id = 5; +ERROR: relation "partitioning_test_2009" does not exist +LINE 1: DELETE FROM partitioning_test_2009 WHERE id = 5; + ^ +-- see that element is deleted from both partitions +SELECT * FROM partitioning_test_2009 WHERE id = 5 ORDER BY 1; +ERROR: relation "partitioning_test_2009" does not exist +LINE 1: SELECT * FROM partitioning_test_2009 WHERE id = 5 ORDER BY 1... + ^ +SELECT * FROM partitioning_test_2012 WHERE id = 5 ORDER BY 1; + id | time +----+------------ + 5 | 06-06-2012 +(1 row) + +-- test DETACH partition +ALTER TABLE partitioning_test DETACH PARTITION partitioning_test_2009; +ERROR: syntax error at or near "DETACH" +LINE 1: ALTER TABLE partitioning_test DETACH PARTITION partitioning_... + ^ +-- see DETACHed partitions content is not accessible from partitioning_test; +SELECT * FROM partitioning_test WHERE time >= '2009-01-01' AND time < '2010-01-01' ORDER BY 1; +ERROR: relation "partitioning_test" does not exist +LINE 1: SELECT * FROM partitioning_test WHERE time >= '2009-01-01' A... + ^ +-- +-- Transaction tests +-- +-- DDL in transaction +BEGIN; +ALTER TABLE partitioning_test ADD newer_column int; +ERROR: relation "partitioning_test" does not exist +-- see additional column is created +SELECT name, type FROM table_attrs WHERE relid = 'partitioning_test'::regclass ORDER BY 1; +ERROR: current transaction is aborted, commands ignored until end of transaction block +ROLLBACK; +-- see rollback is successful +SELECT name, type FROM table_attrs WHERE relid = 'partitioning_test'::regclass ORDER BY 1; +ERROR: relation "partitioning_test" does not exist +LINE 1: SELECT name, type FROM table_attrs WHERE relid = 'partitioni... + ^ +-- COPY in transaction +BEGIN; +COPY partitioning_test FROM STDIN WITH CSV; +ERROR: relation "partitioning_test" does not exist +22,2010-01-01,22 +23,2011-01-01,23 +24,2013-01-01,24 +\. +invalid command \. +-- see the data is loaded to shards +SELECT * FROM partitioning_test WHERE id = 22 ORDER BY 1; +ERROR: syntax error at or near "22" +LINE 1: 22,2010-01-01,22 + ^ +SELECT * FROM partitioning_test WHERE id = 23 ORDER BY 1; +ERROR: current transaction is aborted, commands ignored until end of transaction block +SELECT * FROM partitioning_test WHERE id = 24 ORDER BY 1; +ERROR: current transaction is aborted, commands ignored until end of transaction block +ROLLBACK; +-- see rollback is successful +SELECT * FROM partitioning_test WHERE id >= 22 ORDER BY 1; +ERROR: relation "partitioning_test" does not exist +LINE 1: SELECT * FROM partitioning_test WHERE id >= 22 ORDER BY 1; + ^ +-- DML in transaction +BEGIN; +-- INSERT in transaction +INSERT INTO partitioning_test VALUES(25, '2010-02-02'); +ERROR: relation "partitioning_test" does not exist +LINE 1: INSERT INTO partitioning_test VALUES(25, '2010-02-02'); + ^ +-- see the data is loaded to shards +SELECT * FROM partitioning_test WHERE id = 25 ORDER BY 1; +ERROR: current transaction is aborted, commands ignored until end of transaction block +-- INSERT/SELECT in transaction +INSERT INTO partitioning_test SELECT * FROM partitioning_test WHERE id = 25; +ERROR: current transaction is aborted, commands ignored until end of transaction block +-- see the data is loaded to shards +SELECT * FROM partitioning_test WHERE id = 25 ORDER BY 1; +ERROR: current transaction is aborted, commands ignored until end of transaction block +-- UPDATE in transaction +UPDATE partitioning_test SET time = '2010-10-10' WHERE id = 25; +ERROR: current transaction is aborted, commands ignored until end of transaction block +-- see the data is updated +SELECT * FROM partitioning_test WHERE id = 25 ORDER BY 1; +ERROR: current transaction is aborted, commands ignored until end of transaction block +-- perform operations on partition and partioned tables together +INSERT INTO partitioning_test VALUES(26, '2010-02-02', 26); +ERROR: current transaction is aborted, commands ignored until end of transaction block +INSERT INTO partitioning_test_2010 VALUES(26, '2010-02-02', 26); +ERROR: current transaction is aborted, commands ignored until end of transaction block +COPY partitioning_test FROM STDIN WITH CSV; +ERROR: current transaction is aborted, commands ignored until end of transaction block +26,2010-02-02,26 +\. +invalid command \. +COPY partitioning_test_2010 FROM STDIN WITH CSV; +ERROR: syntax error at or near "26" +LINE 1: 26,2010-02-02,26 + ^ +26,2010-02-02,26 +\. +invalid command \. +-- see the data is loaded to shards (we should see 4 rows with same content) +SELECT * FROM partitioning_test WHERE id = 26 ORDER BY 1; +ERROR: syntax error at or near "26" +LINE 1: 26,2010-02-02,26 + ^ +ROLLBACK; +-- see rollback is successful +SELECT * FROM partitioning_test WHERE id = 26 ORDER BY 1; +ERROR: relation "partitioning_test" does not exist +LINE 1: SELECT * FROM partitioning_test WHERE id = 26 ORDER BY 1; + ^ +-- DETACH and DROP in a transaction +BEGIN; +ALTER TABLE partitioning_test DETACH PARTITION partitioning_test_2011; +ERROR: syntax error at or near "DETACH" +LINE 1: ALTER TABLE partitioning_test DETACH PARTITION partitioning_... + ^ +DROP TABLE partitioning_test_2011; +ERROR: current transaction is aborted, commands ignored until end of transaction block +COMMIT; +-- see DROPed partitions content is not accessible +SELECT * FROM partitioning_test WHERE time >= '2011-01-01' AND time < '2012-01-01' ORDER BY 1; +ERROR: relation "partitioning_test" does not exist +LINE 1: SELECT * FROM partitioning_test WHERE time >= '2011-01-01' A... + ^ +-- +-- Misc tests +-- +-- test TRUNCATE +-- test TRUNCATE partition +TRUNCATE partitioning_test_2012; +-- see partition is TRUNCATEd +SELECT * FROM partitioning_test_2012 ORDER BY 1; + id | time +----+------ +(0 rows) + +-- test TRUNCATE partitioned table +TRUNCATE partitioning_test; +ERROR: relation "partitioning_test" does not exist +-- see partitioned table is TRUNCATEd +SELECT * FROM partitioning_test ORDER BY 1; +ERROR: relation "partitioning_test" does not exist +LINE 1: SELECT * FROM partitioning_test ORDER BY 1; + ^ +-- test DROP +-- test DROP partition +INSERT INTO partitioning_test_2010 VALUES(27, '2010-02-01'); +ERROR: relation "partitioning_test_2010" does not exist +LINE 1: INSERT INTO partitioning_test_2010 VALUES(27, '2010-02-01'); + ^ +DROP TABLE partitioning_test_2010; +ERROR: table "partitioning_test_2010" does not exist +-- see DROPped partitions content is not accessible from partitioning_test; +SELECT * FROM partitioning_test WHERE time >= '2010-01-01' AND time < '2011-01-01' ORDER BY 1; +ERROR: relation "partitioning_test" does not exist +LINE 1: SELECT * FROM partitioning_test WHERE time >= '2010-01-01' A... + ^ +-- test DROP partitioned table +DROP TABLE partitioning_test; +ERROR: table "partitioning_test" does not exist +-- dropping the parent should CASCADE to the children as well +SELECT table_name FROM information_schema.tables WHERE table_name LIKE 'partitioning_test%' ORDER BY 1; + table_name +------------------------ + partitioning_test_2012 + partitioning_test_2013 +(2 rows) + +-- test distributing partitioned table colocated with non-partitioned table +CREATE TABLE partitioned_users_table (user_id int, time timestamp, value_1 int, value_2 int, value_3 float, value_4 bigint) PARTITION BY RANGE (time); +ERROR: syntax error at or near "PARTITION" +LINE 1: ... int, value_2 int, value_3 float, value_4 bigint) PARTITION ... + ^ +CREATE TABLE partitioned_events_table (user_id int, time timestamp, event_type int, value_2 int, value_3 float, value_4 bigint) PARTITION BY RANGE (time); +ERROR: syntax error at or near "PARTITION" +LINE 1: ... int, value_2 int, value_3 float, value_4 bigint) PARTITION ... + ^ +SELECT create_distributed_table('partitioned_users_table', 'user_id', colocate_with => 'users_table'); +ERROR: relation "partitioned_users_table" does not exist +LINE 1: SELECT create_distributed_table('partitioned_users_table', '... + ^ +SELECT create_distributed_table('partitioned_events_table', 'user_id', colocate_with => 'events_table'); +ERROR: relation "partitioned_events_table" does not exist +LINE 1: SELECT create_distributed_table('partitioned_events_table', ... + ^ +-- INSERT/SELECT from regular table to partitioned table +CREATE TABLE partitioned_users_table_2009 PARTITION OF partitioned_users_table FOR VALUES FROM ('2017-01-01') TO ('2018-01-01'); +ERROR: syntax error at or near "PARTITION" +LINE 1: CREATE TABLE partitioned_users_table_2009 PARTITION OF parti... + ^ +CREATE TABLE partitioned_events_table_2009 PARTITION OF partitioned_events_table FOR VALUES FROM ('2017-01-01') TO ('2018-01-01'); +ERROR: syntax error at or near "PARTITION" +LINE 1: CREATE TABLE partitioned_events_table_2009 PARTITION OF part... + ^ +INSERT INTO partitioned_events_table SELECT * FROM events_table; +ERROR: relation "partitioned_events_table" does not exist +LINE 1: INSERT INTO partitioned_events_table SELECT * FROM events_ta... + ^ +INSERT INTO partitioned_users_table_2009 SELECT * FROM users_table; +ERROR: relation "partitioned_users_table_2009" does not exist +LINE 1: INSERT INTO partitioned_users_table_2009 SELECT * FROM users... + ^ +-- +-- Complex JOINs, subqueries, UNIONs etc... +-- +-- subquery with UNIONs on partitioned table +SELECT ("final_query"."event_types") as types, count(*) AS sumOfEventType +FROM + (SELECT *, random() + FROM + (SELECT + "t"."user_id", "t"."time", unnest("t"."collected_events") AS "event_types" + FROM + (SELECT + "t1"."user_id", min("t1"."time") AS "time", array_agg(("t1"."event") ORDER BY TIME ASC, event DESC) AS collected_events + FROM( + (SELECT + "events"."user_id", "events"."time", 0 AS event + FROM + partitioned_events_table as "events" + WHERE + event_type IN (1, 2) ) + UNION + (SELECT + "events"."user_id", "events"."time", 1 AS event + FROM + partitioned_events_table as "events" + WHERE + event_type IN (3, 4) ) + UNION + (SELECT + "events"."user_id", "events"."time", 2 AS event + FROM + partitioned_events_table as "events" + WHERE + event_type IN (5, 6) ) + UNION + (SELECT + "events"."user_id", "events"."time", 3 AS event + FROM + partitioned_events_table as "events" + WHERE + event_type IN (1, 6))) t1 + GROUP BY "t1"."user_id") AS t) "q" +) AS final_query +GROUP BY types +ORDER BY types; +ERROR: relation "partitioned_events_table" does not exist +LINE 14: partitioned_events_table as "events" + ^ +-- UNION and JOIN on both partitioned and regular tables +SELECT ("final_query"."event_types") as types, count(*) AS sumOfEventType +FROM + (SELECT + *, random() + FROM + (SELECT + "t"."user_id", "t"."time", unnest("t"."collected_events") AS "event_types" + FROM + (SELECT + "t1"."user_id", min("t1"."time") AS "time", array_agg(("t1"."event") ORDER BY TIME ASC, event DESC) AS collected_events + FROM ( + (SELECT + * + FROM + (SELECT + "events"."time", 0 AS event, "events"."user_id" + FROM + partitioned_events_table as "events" + WHERE + event_type IN (1, 2)) events_subquery_1) + UNION + (SELECT * + FROM + ( + SELECT * FROM + ( + SELECT + max("events"."time"), + 0 AS event, + "events"."user_id" + FROM + events_table as "events", users_table as "users" + WHERE + events.user_id = users.user_id AND + event_type IN (1, 2) + GROUP BY "events"."user_id" + ) as events_subquery_5 + ) events_subquery_2) + UNION + (SELECT * + FROM + (SELECT + "events"."time", 2 AS event, "events"."user_id" + FROM + partitioned_events_table as "events" + WHERE + event_type IN (3, 4)) events_subquery_3) + UNION + (SELECT * + FROM + (SELECT + "events"."time", 3 AS event, "events"."user_id" + FROM + events_table as "events" + WHERE + event_type IN (5, 6)) events_subquery_4) + ) t1 + GROUP BY "t1"."user_id") AS t) "q" +INNER JOIN + (SELECT + "users"."user_id" + FROM + partitioned_users_table as "users" + WHERE + value_1 > 2 and value_1 < 5) AS t + ON (t.user_id = q.user_id)) as final_query +GROUP BY + types +ORDER BY + types; +ERROR: relation "partitioned_events_table" does not exist +LINE 18: partitioned_events_table as "events" + ^ +-- test LIST partitioning +CREATE TABLE list_partitioned_events_table (user_id int, time date, event_type int, value_2 int, value_3 float, value_4 bigint) PARTITION BY LIST (time); +ERROR: syntax error at or near "PARTITION" +LINE 1: ... int, value_2 int, value_3 float, value_4 bigint) PARTITION ... + ^ +CREATE TABLE list_partitioned_events_table_2014_01_01_05 PARTITION OF list_partitioned_events_table FOR VALUES IN ('2017-11-21', '2017-11-22', '2017-11-23', '2017-11-24', '2017-11-25'); +ERROR: syntax error at or near "PARTITION" +LINE 1: ...TABLE list_partitioned_events_table_2014_01_01_05 PARTITION ... + ^ +CREATE TABLE list_partitioned_events_table_2014_01_06_10 PARTITION OF list_partitioned_events_table FOR VALUES IN ('2017-11-26', '2017-11-27', '2017-11-28', '2017-11-29', '2017-11-30'); +ERROR: syntax error at or near "PARTITION" +LINE 1: ...TABLE list_partitioned_events_table_2014_01_06_10 PARTITION ... + ^ +CREATE TABLE list_partitioned_events_table_2014_01_11_15 PARTITION OF list_partitioned_events_table FOR VALUES IN ('2017-12-01', '2017-12-02', '2017-12-03', '2017-12-04', '2017-12-05'); +ERROR: syntax error at or near "PARTITION" +LINE 1: ...TABLE list_partitioned_events_table_2014_01_11_15 PARTITION ... + ^ +-- test distributing partitioned table colocated with another partitioned table +SELECT create_distributed_table('list_partitioned_events_table', 'user_id', colocate_with => 'partitioned_events_table'); +ERROR: relation "list_partitioned_events_table" does not exist +LINE 1: SELECT create_distributed_table('list_partitioned_events_tab... + ^ +-- INSERT/SELECT from partitioned table to partitioned table +INSERT INTO + list_partitioned_events_table +SELECT + user_id, + date_trunc('day', time) as time, + event_type, + value_2, + value_3, + value_4 +FROM + events_table +WHERE + time >= '2017-11-21' AND + time <= '2017-12-01'; +ERROR: relation "list_partitioned_events_table" does not exist +LINE 2: list_partitioned_events_table + ^ +-- LEFT JOINs used with INNER JOINs on range partitioned table, list partitioned table and non-partitioned table +SELECT +count(*) AS cnt, "generated_group_field" + FROM + (SELECT + "eventQuery"."user_id", random(), generated_group_field + FROM + (SELECT + "multi_group_wrapper_1".*, generated_group_field, random() + FROM + (SELECT * + FROM + (SELECT + "list_partitioned_events_table"."time", "list_partitioned_events_table"."user_id" as event_user_id + FROM + list_partitioned_events_table as "list_partitioned_events_table" + WHERE + user_id > 2) "temp_data_queries" + INNER JOIN + (SELECT + "users"."user_id" + FROM + partitioned_users_table as "users" + WHERE + user_id > 2 and value_2 = 1) "user_filters_1" + ON ("temp_data_queries".event_user_id = "user_filters_1".user_id)) AS "multi_group_wrapper_1" + LEFT JOIN + (SELECT + "users"."user_id" AS "user_id", value_2 AS "generated_group_field" + FROM + partitioned_users_table as "users") "left_group_by_1" + ON ("left_group_by_1".user_id = "multi_group_wrapper_1".event_user_id)) "eventQuery") "pushedDownQuery" + GROUP BY + "generated_group_field" + ORDER BY + cnt DESC, generated_group_field ASC + LIMIT 10; +ERROR: relation "list_partitioned_events_table" does not exist +LINE 15: list_partitioned_events_table as "list_partitio... + ^ +-- +-- Additional partitioning features +-- +-- test multi column partitioning +CREATE TABLE multi_column_partitioning(c1 int, c2 int) PARTITION BY RANGE (c1, c2); +ERROR: syntax error at or near "PARTITION" +LINE 1: ...E TABLE multi_column_partitioning(c1 int, c2 int) PARTITION ... + ^ +CREATE TABLE multi_column_partitioning_0_0_10_0 PARTITION OF multi_column_partitioning FOR VALUES FROM (0, 0) TO (10, 0); +ERROR: syntax error at or near "PARTITION" +LINE 1: CREATE TABLE multi_column_partitioning_0_0_10_0 PARTITION OF... + ^ +SELECT create_distributed_table('multi_column_partitioning', 'c1'); +ERROR: relation "multi_column_partitioning" does not exist +LINE 1: SELECT create_distributed_table('multi_column_partitioning',... + ^ +-- test INSERT to multi-column partitioned table +INSERT INTO multi_column_partitioning VALUES(1, 1); +ERROR: relation "multi_column_partitioning" does not exist +LINE 1: INSERT INTO multi_column_partitioning VALUES(1, 1); + ^ +INSERT INTO multi_column_partitioning_0_0_10_0 VALUES(5, -5); +ERROR: relation "multi_column_partitioning_0_0_10_0" does not exist +LINE 1: INSERT INTO multi_column_partitioning_0_0_10_0 VALUES(5, -5)... + ^ +-- test INSERT to multi-column partitioned table where no suitable partition exists +INSERT INTO multi_column_partitioning VALUES(10, 1); +ERROR: relation "multi_column_partitioning" does not exist +LINE 1: INSERT INTO multi_column_partitioning VALUES(10, 1); + ^ +-- test with MINVALUE/MAXVALUE +CREATE TABLE multi_column_partitioning_10_max_20_min PARTITION OF multi_column_partitioning FOR VALUES FROM (10, MAXVALUE) TO (20, MINVALUE); +ERROR: syntax error at or near "PARTITION" +LINE 1: ...ATE TABLE multi_column_partitioning_10_max_20_min PARTITION ... + ^ +-- test INSERT to partition with MINVALUE/MAXVALUE bounds +INSERT INTO multi_column_partitioning VALUES(11, -11); +ERROR: relation "multi_column_partitioning" does not exist +LINE 1: INSERT INTO multi_column_partitioning VALUES(11, -11); + ^ +INSERT INTO multi_column_partitioning_10_max_20_min VALUES(19, -19); +ERROR: relation "multi_column_partitioning_10_max_20_min" does not exist +LINE 1: INSERT INTO multi_column_partitioning_10_max_20_min VALUES(1... + ^ +-- test INSERT to multi-column partitioned table where no suitable partition exists +INSERT INTO multi_column_partitioning VALUES(20, -20); +ERROR: relation "multi_column_partitioning" does not exist +LINE 1: INSERT INTO multi_column_partitioning VALUES(20, -20); + ^ +-- see data is loaded to multi-column partitioned table +SELECT * FROM multi_column_partitioning ORDER BY 1, 2; +ERROR: relation "multi_column_partitioning" does not exist +LINE 1: SELECT * FROM multi_column_partitioning ORDER BY 1, 2; + ^ +-- +-- Tests for locks on partitioned tables +-- +CREATE TABLE partitioning_locks(id int, ref_id int, time date) PARTITION BY RANGE (time); +ERROR: syntax error at or near "PARTITION" +LINE 1: ...partitioning_locks(id int, ref_id int, time date) PARTITION ... + ^ +-- create its partitions +CREATE TABLE partitioning_locks_2009 PARTITION OF partitioning_locks FOR VALUES FROM ('2009-01-01') TO ('2010-01-01'); +ERROR: syntax error at or near "PARTITION" +LINE 1: CREATE TABLE partitioning_locks_2009 PARTITION OF partitioni... + ^ +CREATE TABLE partitioning_locks_2010 PARTITION OF partitioning_locks FOR VALUES FROM ('2010-01-01') TO ('2011-01-01'); +ERROR: syntax error at or near "PARTITION" +LINE 1: CREATE TABLE partitioning_locks_2010 PARTITION OF partitioni... + ^ +-- distribute partitioned table +SELECT create_distributed_table('partitioning_locks', 'id'); +ERROR: relation "partitioning_locks" does not exist +LINE 1: SELECT create_distributed_table('partitioning_locks', 'id'); + ^ +-- test locks on router SELECT +BEGIN; +SELECT * FROM partitioning_locks WHERE id = 1 ORDER BY 1, 2; +ERROR: relation "partitioning_locks" does not exist +LINE 1: SELECT * FROM partitioning_locks WHERE id = 1 ORDER BY 1, 2; + ^ +SELECT relation::regclass, locktype, mode FROM pg_locks WHERE relation::regclass::text LIKE 'partitioning_locks%' AND pid = pg_backend_pid() ORDER BY 1, 2, 3; +ERROR: current transaction is aborted, commands ignored until end of transaction block +COMMIT; +-- test locks on real-time SELECT +BEGIN; +SELECT * FROM partitioning_locks ORDER BY 1, 2; +ERROR: relation "partitioning_locks" does not exist +LINE 1: SELECT * FROM partitioning_locks ORDER BY 1, 2; + ^ +SELECT relation::regclass, locktype, mode FROM pg_locks WHERE relation::regclass::text LIKE 'partitioning_locks%' AND pid = pg_backend_pid() ORDER BY 1, 2, 3; +ERROR: current transaction is aborted, commands ignored until end of transaction block +COMMIT; +-- test locks on task-tracker SELECT +SET citus.task_executor_type TO 'task-tracker'; +BEGIN; +SELECT * FROM partitioning_locks AS pl1 JOIN partitioning_locks AS pl2 ON pl1.id = pl2.ref_id ORDER BY 1, 2; +ERROR: relation "partitioning_locks" does not exist +LINE 1: SELECT * FROM partitioning_locks AS pl1 JOIN partitioning_lo... + ^ +SELECT relation::regclass, locktype, mode FROM pg_locks WHERE relation::regclass::text LIKE 'partitioning_locks%' AND pid = pg_backend_pid() ORDER BY 1, 2, 3; +ERROR: current transaction is aborted, commands ignored until end of transaction block +COMMIT; +SET citus.task_executor_type TO 'real-time'; +-- test locks on INSERT +BEGIN; +INSERT INTO partitioning_locks VALUES(1, 1, '2009-01-01'); +ERROR: relation "partitioning_locks" does not exist +LINE 1: INSERT INTO partitioning_locks VALUES(1, 1, '2009-01-01'); + ^ +SELECT relation::regclass, locktype, mode FROM pg_locks WHERE relation::regclass::text LIKE 'partitioning_locks%' AND pid = pg_backend_pid() ORDER BY 1, 2, 3; +ERROR: current transaction is aborted, commands ignored until end of transaction block +COMMIT; +-- test locks on UPDATE +BEGIN; +UPDATE partitioning_locks SET time = '2009-02-01' WHERE id = 1; +ERROR: relation "partitioning_locks" does not exist +LINE 1: UPDATE partitioning_locks SET time = '2009-02-01' WHERE id =... + ^ +SELECT relation::regclass, locktype, mode FROM pg_locks WHERE relation::regclass::text LIKE 'partitioning_locks%' AND pid = pg_backend_pid() ORDER BY 1, 2, 3; +ERROR: current transaction is aborted, commands ignored until end of transaction block +COMMIT; +-- test locks on DELETE +BEGIN; +DELETE FROM partitioning_locks WHERE id = 1; +ERROR: relation "partitioning_locks" does not exist +LINE 1: DELETE FROM partitioning_locks WHERE id = 1; + ^ +SELECT relation::regclass, locktype, mode FROM pg_locks WHERE relation::regclass::text LIKE 'partitioning_locks%' AND pid = pg_backend_pid() ORDER BY 1, 2, 3; +ERROR: current transaction is aborted, commands ignored until end of transaction block +COMMIT; +-- test locks on INSERT/SELECT +CREATE TABLE partitioning_locks_for_select(id int, ref_id int, time date); +SELECT create_distributed_table('partitioning_locks_for_select', 'id'); + create_distributed_table +-------------------------- + +(1 row) + +BEGIN; +INSERT INTO partitioning_locks SELECT * FROM partitioning_locks_for_select; +ERROR: relation "partitioning_locks" does not exist +LINE 1: INSERT INTO partitioning_locks SELECT * FROM partitioning_lo... + ^ +SELECT relation::regclass, locktype, mode FROM pg_locks WHERE relation::regclass::text LIKE 'partitioning_locks%' AND pid = pg_backend_pid() ORDER BY 1, 2, 3; +ERROR: current transaction is aborted, commands ignored until end of transaction block +COMMIT; +-- test locks on coordinator INSERT/SELECT +BEGIN; +INSERT INTO partitioning_locks SELECT * FROM partitioning_locks_for_select LIMIT 5; +ERROR: relation "partitioning_locks" does not exist +LINE 1: INSERT INTO partitioning_locks SELECT * FROM partitioning_lo... + ^ +SELECT relation::regclass, locktype, mode FROM pg_locks WHERE relation::regclass::text LIKE 'partitioning_locks%' AND pid = pg_backend_pid() ORDER BY 1, 2, 3; +ERROR: current transaction is aborted, commands ignored until end of transaction block +COMMIT; +-- test locks on master_modify_multiple_shards +BEGIN; +SELECT master_modify_multiple_shards('UPDATE partitioning_locks SET time = ''2009-03-01'''); +ERROR: relation "partitioning_locks" does not exist +SELECT relation::regclass, locktype, mode FROM pg_locks WHERE relation::regclass::text LIKE 'partitioning_locks%' AND pid = pg_backend_pid() ORDER BY 1, 2, 3; +ERROR: current transaction is aborted, commands ignored until end of transaction block +COMMIT; +-- test locks on DDL +BEGIN; +ALTER TABLE partitioning_locks ADD COLUMN new_column int; +ERROR: relation "partitioning_locks" does not exist +SELECT relation::regclass, locktype, mode FROM pg_locks WHERE relation::regclass::text LIKE 'partitioning_locks%' AND pid = pg_backend_pid() ORDER BY 1, 2, 3; +ERROR: current transaction is aborted, commands ignored until end of transaction block +COMMIT; +-- test locks on TRUNCATE +BEGIN; +TRUNCATE partitioning_locks; +ERROR: relation "partitioning_locks" does not exist +SELECT relation::regclass, locktype, mode FROM pg_locks WHERE relation::regclass::text LIKE 'partitioning_locks%' AND pid = pg_backend_pid() ORDER BY 1, 2, 3; +ERROR: current transaction is aborted, commands ignored until end of transaction block +COMMIT; +-- test shard resource locks with master_modify_multiple_shards +BEGIN; +SELECT master_modify_multiple_shards('UPDATE partitioning_locks_2009 SET time = ''2009-03-01'''); +ERROR: relation "partitioning_locks_2009" does not exist +-- see the locks on parent table +SELECT + logicalrelid, + locktype, + mode +FROM + pg_locks AS l JOIN pg_dist_shard AS s +ON + l.objid = s.shardid +WHERE + logicalrelid IN ('partitioning_locks', 'partitioning_locks_2009', 'partitioning_locks_2010') AND + pid = pg_backend_pid() +ORDER BY + 1, 2, 3; +ERROR: current transaction is aborted, commands ignored until end of transaction block +COMMIT; +-- test shard resource locks with TRUNCATE +BEGIN; +TRUNCATE partitioning_locks_2009; +ERROR: relation "partitioning_locks_2009" does not exist +-- see the locks on parent table +SELECT + logicalrelid, + locktype, + mode +FROM + pg_locks AS l JOIN pg_dist_shard AS s +ON + l.objid = s.shardid +WHERE + logicalrelid IN ('partitioning_locks', 'partitioning_locks_2009', 'partitioning_locks_2010') AND + pid = pg_backend_pid() +ORDER BY + 1, 2, 3; +ERROR: current transaction is aborted, commands ignored until end of transaction block +COMMIT; +-- test shard resource locks with INSERT/SELECT +BEGIN; +INSERT INTO partitioning_locks_2009 SELECT * FROM partitioning_locks WHERE time >= '2009-01-01' AND time < '2010-01-01'; +ERROR: relation "partitioning_locks_2009" does not exist +LINE 1: INSERT INTO partitioning_locks_2009 SELECT * FROM partitioni... + ^ +-- see the locks on parent table +SELECT + logicalrelid, + locktype, + mode +FROM + pg_locks AS l JOIN pg_dist_shard AS s +ON + l.objid = s.shardid +WHERE + logicalrelid IN ('partitioning_locks', 'partitioning_locks_2009', 'partitioning_locks_2010') AND + pid = pg_backend_pid() +ORDER BY + 1, 2, 3; +ERROR: current transaction is aborted, commands ignored until end of transaction block +COMMIT; +DROP TABLE +IF EXISTS + partitioning_test_2009, + partitioned_events_table, + partitioned_users_table, + list_partitioned_events_table, + multi_column_partitioning, + partitioning_locks, + partitioning_locks_for_select; +NOTICE: table "partitioning_test_2009" does not exist, skipping +NOTICE: table "partitioned_events_table" does not exist, skipping +NOTICE: table "partitioned_users_table" does not exist, skipping +NOTICE: table "list_partitioned_events_table" does not exist, skipping +NOTICE: table "multi_column_partitioning" does not exist, skipping +NOTICE: table "partitioning_locks" does not exist, skipping +-- make sure we can create a partitioned table with streaming replication +SET citus.replication_model TO 'streaming'; +CREATE TABLE partitioning_test(id int, time date) PARTITION BY RANGE (time); +ERROR: syntax error at or near "PARTITION" +LINE 1: CREATE TABLE partitioning_test(id int, time date) PARTITION ... + ^ +CREATE TABLE partitioning_test_2009 PARTITION OF partitioning_test FOR VALUES FROM ('2009-01-01') TO ('2010-01-01'); +ERROR: syntax error at or near "PARTITION" +LINE 1: CREATE TABLE partitioning_test_2009 PARTITION OF partitionin... + ^ +SELECT create_distributed_table('partitioning_test', 'id'); +ERROR: relation "partitioning_test" does not exist +LINE 1: SELECT create_distributed_table('partitioning_test', 'id'); + ^ +DROP TABLE partitioning_test; +ERROR: table "partitioning_test" does not exist +-- make sure we can attach partitions to a distributed table in a schema +CREATE SCHEMA partitioning_schema; +CREATE TABLE partitioning_schema."schema-test"(id int, time date) PARTITION BY RANGE (time); +ERROR: syntax error at or near "PARTITION" +LINE 1: ...titioning_schema."schema-test"(id int, time date) PARTITION ... + ^ +SELECT create_distributed_table('partitioning_schema."schema-test"', 'id'); +ERROR: relation "partitioning_schema.schema-test" does not exist +LINE 1: SELECT create_distributed_table('partitioning_schema."schema... + ^ +CREATE TABLE partitioning_schema."schema-test_2009"(id int, time date); +ALTER TABLE partitioning_schema."schema-test" ATTACH PARTITION partitioning_schema."schema-test_2009" FOR VALUES FROM ('2009-01-01') TO ('2010-01-01'); +ERROR: syntax error at or near "ATTACH" +LINE 1: ALTER TABLE partitioning_schema."schema-test" ATTACH PARTITI... + ^ +-- attached partition is distributed as well +SELECT + logicalrelid +FROM + pg_dist_partition +WHERE + logicalrelid IN ('partitioning_schema."schema-test"'::regclass, 'partitioning_schema."schema-test_2009"'::regclass) +ORDER BY 1; +ERROR: relation "partitioning_schema.schema-test" does not exist +LINE 6: logicalrelid IN ('partitioning_schema."schema-test"'::regcl... + ^ +SELECT + logicalrelid, count(*) +FROM + pg_dist_shard +WHERE + logicalrelid IN ('partitioning_schema."schema-test"'::regclass, 'partitioning_schema."schema-test_2009"'::regclass) +GROUP BY + logicalrelid +ORDER BY + 1,2; +ERROR: relation "partitioning_schema.schema-test" does not exist +LINE 6: logicalrelid IN ('partitioning_schema."schema-test"'::re... + ^ +DROP TABLE partitioning_schema."schema-test"; +ERROR: table "schema-test" does not exist +-- make sure we can create partition of a distributed table in a schema +CREATE TABLE partitioning_schema."schema-test"(id int, time date) PARTITION BY RANGE (time); +ERROR: syntax error at or near "PARTITION" +LINE 1: ...titioning_schema."schema-test"(id int, time date) PARTITION ... + ^ +SELECT create_distributed_table('partitioning_schema."schema-test"', 'id'); +ERROR: relation "partitioning_schema.schema-test" does not exist +LINE 1: SELECT create_distributed_table('partitioning_schema."schema... + ^ +CREATE TABLE partitioning_schema."schema-test_2009" PARTITION OF partitioning_schema."schema-test" FOR VALUES FROM ('2009-01-01') TO ('2010-01-01'); +ERROR: syntax error at or near "PARTITION" +LINE 1: ...EATE TABLE partitioning_schema."schema-test_2009" PARTITION ... + ^ +-- newly created partition is distributed as well +SELECT + logicalrelid +FROM + pg_dist_partition +WHERE + logicalrelid IN ('partitioning_schema."schema-test"'::regclass, 'partitioning_schema."schema-test_2009"'::regclass) +ORDER BY 1; +ERROR: relation "partitioning_schema.schema-test" does not exist +LINE 6: logicalrelid IN ('partitioning_schema."schema-test"'::regcl... + ^ +SELECT + logicalrelid, count(*) +FROM + pg_dist_shard +WHERE + logicalrelid IN ('partitioning_schema."schema-test"'::regclass, 'partitioning_schema."schema-test_2009"'::regclass) +GROUP BY + logicalrelid +ORDER BY + 1,2; +ERROR: relation "partitioning_schema.schema-test" does not exist +LINE 6: logicalrelid IN ('partitioning_schema."schema-test"'::re... + ^ +DROP TABLE partitioning_schema."schema-test"; +ERROR: table "schema-test" does not exist +-- make sure creating partitioned tables works while search_path is set +CREATE TABLE partitioning_schema."schema-test"(id int, time date) PARTITION BY RANGE (time); +ERROR: syntax error at or near "PARTITION" +LINE 1: ...titioning_schema."schema-test"(id int, time date) PARTITION ... + ^ +SET search_path = partitioning_schema; +SELECT create_distributed_table('"schema-test"', 'id'); +ERROR: relation "schema-test" does not exist +LINE 1: SELECT create_distributed_table('"schema-test"', 'id'); + ^ +CREATE TABLE partitioning_schema."schema-test_2009" PARTITION OF "schema-test" FOR VALUES FROM ('2009-01-01') TO ('2010-01-01'); +ERROR: syntax error at or near "PARTITION" +LINE 1: ...EATE TABLE partitioning_schema."schema-test_2009" PARTITION ... + ^ +-- newly created partition is distributed as well +SELECT + logicalrelid +FROM + pg_dist_partition +WHERE + logicalrelid IN ('partitioning_schema."schema-test"'::regclass, 'partitioning_schema."schema-test_2009"'::regclass) +ORDER BY 1; +ERROR: relation "partitioning_schema.schema-test" does not exist +LINE 6: logicalrelid IN ('partitioning_schema."schema-test"'::regcl... + ^ +SELECT + logicalrelid, count(*) +FROM + pg_dist_shard +WHERE + logicalrelid IN ('partitioning_schema."schema-test"'::regclass, 'partitioning_schema."schema-test_2009"'::regclass) +GROUP BY + logicalrelid +ORDER BY + 1,2; +ERROR: relation "partitioning_schema.schema-test" does not exist +LINE 6: logicalrelid IN ('partitioning_schema."schema-test"'::re... + ^ +DROP SCHEMA partitioning_schema CASCADE; +NOTICE: drop cascades to table "schema-test_2009" diff --git a/src/test/regress/expected/multi_partitioning_utils.out b/src/test/regress/expected/multi_partitioning_utils.out index a1443ee20..25973029c 100644 --- a/src/test/regress/expected/multi_partitioning_utils.out +++ b/src/test/regress/expected/multi_partitioning_utils.out @@ -1,3 +1,11 @@ +-- This test has different output per major version +SHOW server_version \gset +SELECT substring(:'server_version', '\d+')::int as server_major_version; + server_major_version +---------------------- + 11 +(1 row) + -- =================================================================== -- create test functions -- =================================================================== @@ -191,6 +199,7 @@ SELECT worker_apply_inter_shard_ddl_command(referencing_shard:=100, referencing_ id | integer | | | | plain | | time | date | | | | plain | | Partition key: RANGE ("time") +Number of partitions: 0 -- now lets have some more complex partitioning hierarcies with -- tables on different schemas and constraints on the tables diff --git a/src/test/regress/expected/multi_partitioning_utils_0.out b/src/test/regress/expected/multi_partitioning_utils_0.out index 8f4906f48..d1b9b0639 100644 --- a/src/test/regress/expected/multi_partitioning_utils_0.out +++ b/src/test/regress/expected/multi_partitioning_utils_0.out @@ -1,3 +1,11 @@ +-- This test has different output per major version +SHOW server_version \gset +SELECT substring(:'server_version', '\d+')::int as server_major_version; + server_major_version +---------------------- + 10 +(1 row) + -- =================================================================== -- create test functions -- =================================================================== @@ -72,69 +80,101 @@ END; $function$; -- create a partitioned table CREATE TABLE date_partitioned_table(id int, time date) PARTITION BY RANGE (time); -ERROR: syntax error at or near "PARTITION" -LINE 1: ...E TABLE date_partitioned_table(id int, time date) PARTITION ... - ^ -- we should be able to get the partitioning information even if there are no partitions SELECT generate_partition_information('date_partitioned_table'); -ERROR: relation "date_partitioned_table" does not exist -LINE 1: SELECT generate_partition_information('date_partitioned_tabl... - ^ + generate_partition_information +-------------------------------- + RANGE ("time") +(1 row) + -- we should be able to drop and re-create the partitioned table using the command that Citus generate SELECT drop_and_recreate_partitioned_table('date_partitioned_table'); -ERROR: relation "date_partitioned_table" does not exist -LINE 1: SELECT drop_and_recreate_partitioned_table('date_partitioned... - ^ + drop_and_recreate_partitioned_table +------------------------------------- + +(1 row) + -- we should also be able to see the PARTITION BY ... for the parent table SELECT master_get_table_ddl_events('date_partitioned_table'); -ERROR: relation "date_partitioned_table" does not exist + master_get_table_ddl_events +--------------------------------------------------------------------------------------------------- + CREATE TABLE public.date_partitioned_table (id integer, "time" date) PARTITION BY RANGE ("time") +(1 row) + -- now create the partitions CREATE TABLE date_partition_2006 PARTITION OF date_partitioned_table FOR VALUES FROM ('2006-01-01') TO ('2007-01-01'); -ERROR: syntax error at or near "PARTITION" -LINE 1: CREATE TABLE date_partition_2006 PARTITION OF date_partition... - ^ CREATE TABLE date_partition_2007 PARTITION OF date_partitioned_table FOR VALUES FROM ('2007-01-01') TO ('2008-01-01'); -ERROR: syntax error at or near "PARTITION" -LINE 1: CREATE TABLE date_partition_2007 PARTITION OF date_partition... - ^ -- we should be able to get the partitioning information after the partitions are created SELECT generate_partition_information('date_partitioned_table'); -ERROR: relation "date_partitioned_table" does not exist -LINE 1: SELECT generate_partition_information('date_partitioned_tabl... - ^ + generate_partition_information +-------------------------------- + RANGE ("time") +(1 row) + -- lets get the attach partition commands SELECT generate_alter_table_attach_partition_command('date_partition_2006'); -ERROR: relation "date_partition_2006" does not exist -LINE 1: ...ECT generate_alter_table_attach_partition_command('date_part... - ^ + generate_alter_table_attach_partition_command +----------------------------------------------------------------------------------------------------------------------------------------- + ALTER TABLE public.date_partitioned_table ATTACH PARTITION public.date_partition_2006 FOR VALUES FROM ('01-01-2006') TO ('01-01-2007'); +(1 row) + SELECT generate_alter_table_attach_partition_command('date_partition_2007'); -ERROR: relation "date_partition_2007" does not exist -LINE 1: ...ECT generate_alter_table_attach_partition_command('date_part... - ^ + generate_alter_table_attach_partition_command +----------------------------------------------------------------------------------------------------------------------------------------- + ALTER TABLE public.date_partitioned_table ATTACH PARTITION public.date_partition_2007 FOR VALUES FROM ('01-01-2007') TO ('01-01-2008'); +(1 row) + -- detach and attach the partition by the command generated by us \d+ date_partitioned_table + Table "public.date_partitioned_table" + Column | Type | Collation | Nullable | Default | Storage | Stats target | Description +--------+---------+-----------+----------+---------+---------+--------------+------------- + id | integer | | | | plain | | + time | date | | | | plain | | +Partition key: RANGE ("time") +Partitions: date_partition_2006 FOR VALUES FROM ('01-01-2006') TO ('01-01-2007'), + date_partition_2007 FOR VALUES FROM ('01-01-2007') TO ('01-01-2008') + SELECT detach_and_attach_partition('date_partition_2007', 'date_partitioned_table'); -ERROR: relation "date_partition_2007" does not exist -LINE 1: SELECT detach_and_attach_partition('date_partition_2007', 'd... - ^ + detach_and_attach_partition +----------------------------- + +(1 row) + -- check that both partitions are visiable \d+ date_partitioned_table + Table "public.date_partitioned_table" + Column | Type | Collation | Nullable | Default | Storage | Stats target | Description +--------+---------+-----------+----------+---------+---------+--------------+------------- + id | integer | | | | plain | | + time | date | | | | plain | | +Partition key: RANGE ("time") +Partitions: date_partition_2006 FOR VALUES FROM ('01-01-2006') TO ('01-01-2007'), + date_partition_2007 FOR VALUES FROM ('01-01-2007') TO ('01-01-2008') + -- make sure that inter shard commands work as expected -- assume that the shardId is 100 CREATE TABLE date_partitioned_table_100 (id int, time date) PARTITION BY RANGE (time); -ERROR: syntax error at or near "PARTITION" -LINE 1: ...LE date_partitioned_table_100 (id int, time date) PARTITION ... - ^ CREATE TABLE date_partition_2007_100 (id int, time date ); -- now create the partitioning hierarcy SELECT worker_apply_inter_shard_ddl_command(referencing_shard:=100, referencing_schema_name:='public', referenced_shard:=100, referenced_schema_name:='public', command:='ALTER TABLE date_partitioned_table ATTACH PARTITION date_partition_2007 FOR VALUES FROM (''2007-01-01'') TO (''2008-01-02'')' ); -ERROR: syntax error at or near "ATTACH" -LINE 1: SELECT worker_apply_inter_shard_ddl_command(referencing_shar... - ^ + worker_apply_inter_shard_ddl_command +-------------------------------------- + +(1 row) + -- the hierarcy is successfully created \d+ date_partitioned_table_100 + Table "public.date_partitioned_table_100" + Column | Type | Collation | Nullable | Default | Storage | Stats target | Description +--------+---------+-----------+----------+---------+---------+--------------+------------- + id | integer | | | | plain | | + time | date | | | | plain | | +Partition key: RANGE ("time") +Partitions: date_partition_2007_100 FOR VALUES FROM ('01-01-2007') TO ('01-02-2008') + -- Citus can also get the DDL events for the partitions as regular tables SELECT master_get_table_ddl_events('date_partition_2007_100'); master_get_table_ddl_events @@ -146,90 +186,111 @@ SELECT master_get_table_ddl_events('date_partition_2007_100'); SELECT worker_apply_inter_shard_ddl_command(referencing_shard:=100, referencing_schema_name:='public', referenced_shard:=100, referenced_schema_name:='public', command:='ALTER TABLE date_partitioned_table DETACH PARTITION date_partition_2007' ); -ERROR: syntax error at or near "DETACH" -LINE 1: SELECT worker_apply_inter_shard_ddl_command(referencing_shar... - ^ + worker_apply_inter_shard_ddl_command +-------------------------------------- + +(1 row) + -- the hierarcy is successfully broken \d+ date_partitioned_table_100 + Table "public.date_partitioned_table_100" + Column | Type | Collation | Nullable | Default | Storage | Stats target | Description +--------+---------+-----------+----------+---------+---------+--------------+------------- + id | integer | | | | plain | | + time | date | | | | plain | | +Partition key: RANGE ("time") + -- now lets have some more complex partitioning hierarcies with -- tables on different schemas and constraints on the tables CREATE SCHEMA partition_parent_schema; CREATE TABLE partition_parent_schema.parent_table (id int NOT NULL, time date DEFAULT now()) PARTITION BY RANGE (time); -ERROR: syntax error at or near "PARTITION" -LINE 1: ..._table (id int NOT NULL, time date DEFAULT now()) PARTITION ... - ^ CREATE SCHEMA partition_child_1_schema; CREATE TABLE partition_child_1_schema.child_1 (id int NOT NULL, time date ); CREATE SCHEMA partition_child_2_schema; CREATE TABLE partition_child_2_schema.child_2 (id int NOT NULL, time date ); -- we should be able to get the partitioning information even if there are no partitions SELECT generate_partition_information('partition_parent_schema.parent_table'); -ERROR: relation "partition_parent_schema.parent_table" does not exist -LINE 1: SELECT generate_partition_information('partition_parent_sche... - ^ + generate_partition_information +-------------------------------- + RANGE ("time") +(1 row) + -- we should be able to drop and re-create the partitioned table using the command that Citus generate SELECT drop_and_recreate_partitioned_table('partition_parent_schema.parent_table'); -ERROR: relation "partition_parent_schema.parent_table" does not exist -LINE 1: SELECT drop_and_recreate_partitioned_table('partition_parent... - ^ +NOTICE: schema "partition_parent_schema" already exists, skipping +CONTEXT: SQL statement "CREATE SCHEMA IF NOT EXISTS partition_parent_schema AUTHORIZATION postgres" +PL/pgSQL function drop_and_recreate_partitioned_table(regclass) line 15 at EXECUTE + drop_and_recreate_partitioned_table +------------------------------------- + +(1 row) + ALTER TABLE partition_parent_schema.parent_table ATTACH PARTITION partition_child_1_schema.child_1 FOR VALUES FROM ('2009-01-01') TO ('2010-01-02'); -ERROR: syntax error at or near "ATTACH" -LINE 1: ALTER TABLE partition_parent_schema.parent_table ATTACH PART... - ^ SET search_path = 'partition_parent_schema'; ALTER TABLE parent_table ATTACH PARTITION partition_child_2_schema.child_2 FOR VALUES FROM ('2006-01-01') TO ('2007-01-01'); -ERROR: syntax error at or near "ATTACH" -LINE 1: ALTER TABLE parent_table ATTACH PARTITION partition_child_2... - ^ SELECT public.generate_partition_information('parent_table'); -ERROR: relation "parent_table" does not exist -LINE 1: SELECT public.generate_partition_information('parent_table')... - ^ + generate_partition_information +-------------------------------- + RANGE ("time") +(1 row) + -- lets get the attach partition commands SELECT public.generate_alter_table_attach_partition_command('partition_child_1_schema.child_1'); - generate_alter_table_attach_partition_command ------------------------------------------------ - + generate_alter_table_attach_partition_command +------------------------------------------------------------------------------------------------------------------------------------------------------ + ALTER TABLE partition_parent_schema.parent_table ATTACH PARTITION partition_child_1_schema.child_1 FOR VALUES FROM ('01-01-2009') TO ('01-02-2010'); (1 row) SET search_path = 'partition_child_2_schema'; SELECT public.generate_alter_table_attach_partition_command('child_2'); - generate_alter_table_attach_partition_command ------------------------------------------------ - + generate_alter_table_attach_partition_command +------------------------------------------------------------------------------------------------------------------------------------------------------ + ALTER TABLE partition_parent_schema.parent_table ATTACH PARTITION partition_child_2_schema.child_2 FOR VALUES FROM ('01-01-2006') TO ('01-01-2007'); (1 row) SET search_path = 'partition_parent_schema'; -- detach and attach the partition by the command generated by us \d+ parent_table + Table "partition_parent_schema.parent_table" + Column | Type | Collation | Nullable | Default | Storage | Stats target | Description +--------+---------+-----------+----------+---------+---------+--------------+------------- + id | integer | | not null | | plain | | + time | date | | | now() | plain | | +Partition key: RANGE ("time") +Partitions: partition_child_1_schema.child_1 FOR VALUES FROM ('01-01-2009') TO ('01-02-2010'), + partition_child_2_schema.child_2 FOR VALUES FROM ('01-01-2006') TO ('01-01-2007') + SELECT public.detach_and_attach_partition('partition_child_1_schema.child_1', 'parent_table'); -ERROR: relation "parent_table" does not exist -LINE 1: ...ach_partition('partition_child_1_schema.child_1', 'parent_ta... - ^ + detach_and_attach_partition +----------------------------- + +(1 row) + -- check that both partitions are visiable \d+ parent_table + Table "partition_parent_schema.parent_table" + Column | Type | Collation | Nullable | Default | Storage | Stats target | Description +--------+---------+-----------+----------+---------+---------+--------------+------------- + id | integer | | not null | | plain | | + time | date | | | now() | plain | | +Partition key: RANGE ("time") +Partitions: partition_child_1_schema.child_1 FOR VALUES FROM ('01-01-2009') TO ('01-02-2010'), + partition_child_2_schema.child_2 FOR VALUES FROM ('01-01-2006') TO ('01-01-2007') + -- some very simple checks that should error out SELECT public.generate_alter_table_attach_partition_command('parent_table'); -ERROR: relation "parent_table" does not exist -LINE 1: ...lic.generate_alter_table_attach_partition_command('parent_ta... - ^ +ERROR: "parent_table" is not a partition SELECT public.generate_partition_information('partition_child_1_schema.child_1'); - generate_partition_information --------------------------------- - -(1 row) - +ERROR: "child_1" is not a parent table SELECT public.print_partitions('partition_child_1_schema.child_1'); - print_partitions ------------------- - -(1 row) - +ERROR: "child_1" is not a parent table -- now pring the partitions SELECT public.print_partitions('parent_table'); -ERROR: relation "parent_table" does not exist -LINE 1: SELECT public.print_partitions('parent_table'); - ^ + print_partitions +------------------ + child_1,child_2 +(1 row) + SET search_path = 'public'; -- test multi column / expression partitioning with UNBOUNDED ranges CREATE OR REPLACE FUNCTION some_function(input_val text) @@ -244,9 +305,6 @@ CREATE TABLE multi_column_partitioned ( b int, c text ) PARTITION BY RANGE (a, (a+b+1), some_function(upper(c))); -ERROR: syntax error at or near "PARTITION" -LINE 5: ) PARTITION BY RANGE (a, (a+b+1), some_function(upper(c))); - ^ CREATE TABLE multi_column_partition_1( a int, b int, @@ -259,65 +317,71 @@ CREATE TABLE multi_column_partition_2( ); -- partitioning information SELECT generate_partition_information('multi_column_partitioned'); -ERROR: relation "multi_column_partitioned" does not exist -LINE 1: SELECT generate_partition_information('multi_column_partitio... - ^ + generate_partition_information +----------------------------------------------------- + RANGE (a, (((a + b) + 1)), some_function(upper(c))) +(1 row) + SELECT master_get_table_ddl_events('multi_column_partitioned'); -ERROR: relation "multi_column_partitioned" does not exist + master_get_table_ddl_events +------------------------------------------------------------------------------------------------------------------------------------------------------ + CREATE TABLE public.multi_column_partitioned (a integer, b integer, c text) PARTITION BY RANGE (a, (((a + b) + 1)), public.some_function(upper(c))) +(1 row) + SELECT drop_and_recreate_partitioned_table('multi_column_partitioned'); -ERROR: relation "multi_column_partitioned" does not exist -LINE 1: SELECT drop_and_recreate_partitioned_table('multi_column_par... - ^ + drop_and_recreate_partitioned_table +------------------------------------- + +(1 row) + -- partitions and their ranges ALTER TABLE multi_column_partitioned ATTACH PARTITION multi_column_partition_1 FOR VALUES FROM (1, 10, '250') TO (1, 20, '250'); -ERROR: syntax error at or near "ATTACH" -LINE 1: ALTER TABLE multi_column_partitioned ATTACH PARTITION multi_... - ^ SELECT generate_alter_table_attach_partition_command('multi_column_partition_1'); - generate_alter_table_attach_partition_command ------------------------------------------------ - + generate_alter_table_attach_partition_command +------------------------------------------------------------------------------------------------------------------------------------------------ + ALTER TABLE public.multi_column_partitioned ATTACH PARTITION public.multi_column_partition_1 FOR VALUES FROM (1, 10, '250') TO (1, 20, '250'); (1 row) ALTER TABLE multi_column_partitioned ATTACH PARTITION multi_column_partition_2 FOR VALUES FROM (10, 1000, '2500') TO (MAXVALUE, MAXVALUE, MAXVALUE); -ERROR: syntax error at or near "ATTACH" -LINE 1: ALTER TABLE multi_column_partitioned ATTACH PARTITION multi_... - ^ SELECT generate_alter_table_attach_partition_command('multi_column_partition_2'); - generate_alter_table_attach_partition_command ------------------------------------------------ - + generate_alter_table_attach_partition_command +-------------------------------------------------------------------------------------------------------------------------------------------------------------------- + ALTER TABLE public.multi_column_partitioned ATTACH PARTITION public.multi_column_partition_2 FOR VALUES FROM (10, 1000, '2500') TO (MAXVALUE, MAXVALUE, MAXVALUE); (1 row) SELECT generate_alter_table_detach_partition_command('multi_column_partition_2'); - generate_alter_table_detach_partition_command ------------------------------------------------ - + generate_alter_table_detach_partition_command +--------------------------------------------------------------------------------------------------------- + ALTER TABLE IF EXISTS public.multi_column_partitioned DETACH PARTITION public.multi_column_partition_2; (1 row) -- finally a test with LIST partitioning CREATE TABLE list_partitioned (col1 NUMERIC, col2 NUMERIC, col3 VARCHAR(10)) PARTITION BY LIST (col1) ; -ERROR: syntax error at or near "PARTITION" -LINE 1: ...ed (col1 NUMERIC, col2 NUMERIC, col3 VARCHAR(10)) PARTITION ... - ^ SELECT generate_partition_information('list_partitioned'); -ERROR: relation "list_partitioned" does not exist -LINE 1: SELECT generate_partition_information('list_partitioned'); - ^ + generate_partition_information +-------------------------------- + LIST (col1) +(1 row) + SELECT master_get_table_ddl_events('list_partitioned'); -ERROR: relation "list_partitioned" does not exist + master_get_table_ddl_events +------------------------------------------------------------------------------------------------------------------------- + CREATE TABLE public.list_partitioned (col1 numeric, col2 numeric, col3 character varying(10)) PARTITION BY LIST (col1) +(1 row) + SELECT drop_and_recreate_partitioned_table('list_partitioned'); -ERROR: relation "list_partitioned" does not exist -LINE 1: SELECT drop_and_recreate_partitioned_table('list_partitioned... - ^ + drop_and_recreate_partitioned_table +------------------------------------- + +(1 row) + CREATE TABLE list_partitioned_1 PARTITION OF list_partitioned FOR VALUES IN (100, 101, 102, 103, 104); -ERROR: syntax error at or near "PARTITION" -LINE 1: CREATE TABLE list_partitioned_1 PARTITION OF list_partitione... - ^ SELECT generate_alter_table_attach_partition_command('list_partitioned_1'); -ERROR: relation "list_partitioned_1" does not exist -LINE 1: ...ECT generate_alter_table_attach_partition_command('list_part... - ^ + generate_alter_table_attach_partition_command +----------------------------------------------------------------------------------------------------------------------------------- + ALTER TABLE public.list_partitioned ATTACH PARTITION public.list_partitioned_1 FOR VALUES IN ('100', '101', '102', '103', '104'); +(1 row) + -- also differentiate partitions and inhereted tables CREATE TABLE cities ( name text, @@ -337,9 +401,11 @@ SELECT table_inherits('capitals'); -- although date_partition_2006 inherits from its parent -- returns false since the hierarcy is formed via partitioning SELECT table_inherits('date_partition_2006'); -ERROR: relation "date_partition_2006" does not exist -LINE 1: SELECT table_inherits('date_partition_2006'); - ^ + table_inherits +---------------- + f +(1 row) + -- returns true since cities inherited by capitals SELECT table_inherited('cities'); table_inherited @@ -350,9 +416,11 @@ SELECT table_inherited('cities'); -- although date_partitioned_table inherited by its partitions -- returns false since the hierarcy is formed via partitioning SELECT table_inherited('date_partitioned_table'); -ERROR: relation "date_partitioned_table" does not exist -LINE 1: SELECT table_inherited('date_partitioned_table'); - ^ + table_inherited +----------------- + f +(1 row) + -- also these are not supported SELECT master_get_table_ddl_events('capitals'); ERROR: capitals is not a regular, foreign or partitioned table @@ -360,4 +428,3 @@ SELECT master_get_table_ddl_events('cities'); ERROR: cities is not a regular, foreign or partitioned table -- dropping parents frop the partitions DROP TABLE date_partitioned_table, multi_column_partitioned, list_partitioned, partition_parent_schema.parent_table, cities, capitals; -ERROR: table "date_partitioned_table" does not exist diff --git a/src/test/regress/expected/multi_partitioning_utils_1.out b/src/test/regress/expected/multi_partitioning_utils_1.out new file mode 100644 index 000000000..e7dd17914 --- /dev/null +++ b/src/test/regress/expected/multi_partitioning_utils_1.out @@ -0,0 +1,371 @@ +-- This test has different output per major version +SHOW server_version \gset +SELECT substring(:'server_version', '\d+')::int as server_major_version; + server_major_version +---------------------- + 9 +(1 row) + +-- =================================================================== +-- create test functions +-- =================================================================== +CREATE FUNCTION generate_alter_table_detach_partition_command(regclass) + RETURNS text + AS 'citus' + LANGUAGE C STRICT; +CREATE FUNCTION generate_alter_table_attach_partition_command(regclass) + RETURNS text + AS 'citus' + LANGUAGE C STRICT; +CREATE FUNCTION generate_partition_information(regclass) + RETURNS text + AS 'citus' + LANGUAGE C STRICT; +CREATE FUNCTION print_partitions(regclass) + RETURNS text + AS 'citus' + LANGUAGE C STRICT; +CREATE FUNCTION table_inherits(regclass) + RETURNS bool + AS 'citus' + LANGUAGE C STRICT; +CREATE FUNCTION table_inherited(regclass) + RETURNS bool + AS 'citus' + LANGUAGE C STRICT; +CREATE OR REPLACE FUNCTION detach_and_attach_partition(partition_name regclass, parent_table_name regclass) +RETURNS void LANGUAGE plpgsql VOLATILE +AS $function$ +DECLARE + detach_partition_command text := ''; + attach_partition_command text := ''; + command_result text := ''; + +BEGIN + -- first generate the command + SELECT public.generate_alter_table_attach_partition_command(partition_name) INTO attach_partition_command; + + -- now genereate the detach command + SELECT public.generate_alter_table_detach_partition_command(partition_name) INTO detach_partition_command; + + -- later detach the same partition + EXECUTE detach_partition_command; + + -- not attach it again + EXECUTE attach_partition_command; +END; +$function$; +CREATE OR REPLACE FUNCTION drop_and_recreate_partitioned_table(parent_table_name regclass) +RETURNS void LANGUAGE plpgsql VOLATILE +AS $function$ +DECLARE + command text := ''; + +BEGIN + -- first generate the command + CREATE TABLE partitioned_table_create_commands AS SELECT master_get_table_ddl_events(parent_table_name::text); + + -- later detach the same partition + EXECUTE 'DROP TABLE ' || parent_table_name::text || ';'; + + FOR command IN SELECT * FROM partitioned_table_create_commands + LOOP + -- can do some processing here + EXECUTE command; + END LOOP; + + DROP TABLE partitioned_table_create_commands; + +END; +$function$; +-- create a partitioned table +CREATE TABLE date_partitioned_table(id int, time date) PARTITION BY RANGE (time); +ERROR: syntax error at or near "PARTITION" +LINE 1: ...E TABLE date_partitioned_table(id int, time date) PARTITION ... + ^ +-- we should be able to get the partitioning information even if there are no partitions +SELECT generate_partition_information('date_partitioned_table'); +ERROR: relation "date_partitioned_table" does not exist +LINE 1: SELECT generate_partition_information('date_partitioned_tabl... + ^ +-- we should be able to drop and re-create the partitioned table using the command that Citus generate +SELECT drop_and_recreate_partitioned_table('date_partitioned_table'); +ERROR: relation "date_partitioned_table" does not exist +LINE 1: SELECT drop_and_recreate_partitioned_table('date_partitioned... + ^ +-- we should also be able to see the PARTITION BY ... for the parent table +SELECT master_get_table_ddl_events('date_partitioned_table'); +ERROR: relation "date_partitioned_table" does not exist +-- now create the partitions +CREATE TABLE date_partition_2006 PARTITION OF date_partitioned_table FOR VALUES FROM ('2006-01-01') TO ('2007-01-01'); +ERROR: syntax error at or near "PARTITION" +LINE 1: CREATE TABLE date_partition_2006 PARTITION OF date_partition... + ^ +CREATE TABLE date_partition_2007 PARTITION OF date_partitioned_table FOR VALUES FROM ('2007-01-01') TO ('2008-01-01'); +ERROR: syntax error at or near "PARTITION" +LINE 1: CREATE TABLE date_partition_2007 PARTITION OF date_partition... + ^ +-- we should be able to get the partitioning information after the partitions are created +SELECT generate_partition_information('date_partitioned_table'); +ERROR: relation "date_partitioned_table" does not exist +LINE 1: SELECT generate_partition_information('date_partitioned_tabl... + ^ +-- lets get the attach partition commands +SELECT generate_alter_table_attach_partition_command('date_partition_2006'); +ERROR: relation "date_partition_2006" does not exist +LINE 1: ...ECT generate_alter_table_attach_partition_command('date_part... + ^ +SELECT generate_alter_table_attach_partition_command('date_partition_2007'); +ERROR: relation "date_partition_2007" does not exist +LINE 1: ...ECT generate_alter_table_attach_partition_command('date_part... + ^ +-- detach and attach the partition by the command generated by us +\d+ date_partitioned_table +SELECT detach_and_attach_partition('date_partition_2007', 'date_partitioned_table'); +ERROR: relation "date_partition_2007" does not exist +LINE 1: SELECT detach_and_attach_partition('date_partition_2007', 'd... + ^ +-- check that both partitions are visiable +\d+ date_partitioned_table +-- make sure that inter shard commands work as expected +-- assume that the shardId is 100 +CREATE TABLE date_partitioned_table_100 (id int, time date) PARTITION BY RANGE (time); +ERROR: syntax error at or near "PARTITION" +LINE 1: ...LE date_partitioned_table_100 (id int, time date) PARTITION ... + ^ +CREATE TABLE date_partition_2007_100 (id int, time date ); +-- now create the partitioning hierarcy +SELECT worker_apply_inter_shard_ddl_command(referencing_shard:=100, referencing_schema_name:='public', + referenced_shard:=100, referenced_schema_name:='public', + command:='ALTER TABLE date_partitioned_table ATTACH PARTITION date_partition_2007 FOR VALUES FROM (''2007-01-01'') TO (''2008-01-02'')' ); +ERROR: syntax error at or near "ATTACH" +LINE 1: SELECT worker_apply_inter_shard_ddl_command(referencing_shar... + ^ +-- the hierarcy is successfully created +\d+ date_partitioned_table_100 +-- Citus can also get the DDL events for the partitions as regular tables +SELECT master_get_table_ddl_events('date_partition_2007_100'); + master_get_table_ddl_events +----------------------------------------------------------------------- + CREATE TABLE public.date_partition_2007_100 (id integer, "time" date) +(1 row) + +-- now break the partitioning hierarcy +SELECT worker_apply_inter_shard_ddl_command(referencing_shard:=100, referencing_schema_name:='public', + referenced_shard:=100, referenced_schema_name:='public', + command:='ALTER TABLE date_partitioned_table DETACH PARTITION date_partition_2007' ); +ERROR: syntax error at or near "DETACH" +LINE 1: SELECT worker_apply_inter_shard_ddl_command(referencing_shar... + ^ +-- the hierarcy is successfully broken +\d+ date_partitioned_table_100 +-- now lets have some more complex partitioning hierarcies with +-- tables on different schemas and constraints on the tables +CREATE SCHEMA partition_parent_schema; +CREATE TABLE partition_parent_schema.parent_table (id int NOT NULL, time date DEFAULT now()) PARTITION BY RANGE (time); +ERROR: syntax error at or near "PARTITION" +LINE 1: ..._table (id int NOT NULL, time date DEFAULT now()) PARTITION ... + ^ +CREATE SCHEMA partition_child_1_schema; +CREATE TABLE partition_child_1_schema.child_1 (id int NOT NULL, time date ); +CREATE SCHEMA partition_child_2_schema; +CREATE TABLE partition_child_2_schema.child_2 (id int NOT NULL, time date ); +-- we should be able to get the partitioning information even if there are no partitions +SELECT generate_partition_information('partition_parent_schema.parent_table'); +ERROR: relation "partition_parent_schema.parent_table" does not exist +LINE 1: SELECT generate_partition_information('partition_parent_sche... + ^ +-- we should be able to drop and re-create the partitioned table using the command that Citus generate +SELECT drop_and_recreate_partitioned_table('partition_parent_schema.parent_table'); +ERROR: relation "partition_parent_schema.parent_table" does not exist +LINE 1: SELECT drop_and_recreate_partitioned_table('partition_parent... + ^ +ALTER TABLE partition_parent_schema.parent_table ATTACH PARTITION partition_child_1_schema.child_1 FOR VALUES FROM ('2009-01-01') TO ('2010-01-02'); +ERROR: syntax error at or near "ATTACH" +LINE 1: ALTER TABLE partition_parent_schema.parent_table ATTACH PART... + ^ +SET search_path = 'partition_parent_schema'; +ALTER TABLE parent_table ATTACH PARTITION partition_child_2_schema.child_2 FOR VALUES FROM ('2006-01-01') TO ('2007-01-01'); +ERROR: syntax error at or near "ATTACH" +LINE 1: ALTER TABLE parent_table ATTACH PARTITION partition_child_2... + ^ +SELECT public.generate_partition_information('parent_table'); +ERROR: relation "parent_table" does not exist +LINE 1: SELECT public.generate_partition_information('parent_table')... + ^ +-- lets get the attach partition commands +SELECT public.generate_alter_table_attach_partition_command('partition_child_1_schema.child_1'); + generate_alter_table_attach_partition_command +----------------------------------------------- + +(1 row) + +SET search_path = 'partition_child_2_schema'; +SELECT public.generate_alter_table_attach_partition_command('child_2'); + generate_alter_table_attach_partition_command +----------------------------------------------- + +(1 row) + +SET search_path = 'partition_parent_schema'; +-- detach and attach the partition by the command generated by us +\d+ parent_table +SELECT public.detach_and_attach_partition('partition_child_1_schema.child_1', 'parent_table'); +ERROR: relation "parent_table" does not exist +LINE 1: ...ach_partition('partition_child_1_schema.child_1', 'parent_ta... + ^ +-- check that both partitions are visiable +\d+ parent_table +-- some very simple checks that should error out +SELECT public.generate_alter_table_attach_partition_command('parent_table'); +ERROR: relation "parent_table" does not exist +LINE 1: ...lic.generate_alter_table_attach_partition_command('parent_ta... + ^ +SELECT public.generate_partition_information('partition_child_1_schema.child_1'); + generate_partition_information +-------------------------------- + +(1 row) + +SELECT public.print_partitions('partition_child_1_schema.child_1'); + print_partitions +------------------ + +(1 row) + +-- now pring the partitions +SELECT public.print_partitions('parent_table'); +ERROR: relation "parent_table" does not exist +LINE 1: SELECT public.print_partitions('parent_table'); + ^ +SET search_path = 'public'; +-- test multi column / expression partitioning with UNBOUNDED ranges +CREATE OR REPLACE FUNCTION some_function(input_val text) +RETURNS text LANGUAGE plpgsql IMMUTABLE +AS $function$ +BEGIN + return reverse(input_val); +END; +$function$; +CREATE TABLE multi_column_partitioned ( + a int, + b int, + c text + ) PARTITION BY RANGE (a, (a+b+1), some_function(upper(c))); +ERROR: syntax error at or near "PARTITION" +LINE 5: ) PARTITION BY RANGE (a, (a+b+1), some_function(upper(c))); + ^ +CREATE TABLE multi_column_partition_1( + a int, + b int, + c text +); +CREATE TABLE multi_column_partition_2( + a int, + b int, + c text +); +-- partitioning information +SELECT generate_partition_information('multi_column_partitioned'); +ERROR: relation "multi_column_partitioned" does not exist +LINE 1: SELECT generate_partition_information('multi_column_partitio... + ^ +SELECT master_get_table_ddl_events('multi_column_partitioned'); +ERROR: relation "multi_column_partitioned" does not exist +SELECT drop_and_recreate_partitioned_table('multi_column_partitioned'); +ERROR: relation "multi_column_partitioned" does not exist +LINE 1: SELECT drop_and_recreate_partitioned_table('multi_column_par... + ^ +-- partitions and their ranges +ALTER TABLE multi_column_partitioned ATTACH PARTITION multi_column_partition_1 FOR VALUES FROM (1, 10, '250') TO (1, 20, '250'); +ERROR: syntax error at or near "ATTACH" +LINE 1: ALTER TABLE multi_column_partitioned ATTACH PARTITION multi_... + ^ +SELECT generate_alter_table_attach_partition_command('multi_column_partition_1'); + generate_alter_table_attach_partition_command +----------------------------------------------- + +(1 row) + +ALTER TABLE multi_column_partitioned ATTACH PARTITION multi_column_partition_2 FOR VALUES FROM (10, 1000, '2500') TO (MAXVALUE, MAXVALUE, MAXVALUE); +ERROR: syntax error at or near "ATTACH" +LINE 1: ALTER TABLE multi_column_partitioned ATTACH PARTITION multi_... + ^ +SELECT generate_alter_table_attach_partition_command('multi_column_partition_2'); + generate_alter_table_attach_partition_command +----------------------------------------------- + +(1 row) + +SELECT generate_alter_table_detach_partition_command('multi_column_partition_2'); + generate_alter_table_detach_partition_command +----------------------------------------------- + +(1 row) + +-- finally a test with LIST partitioning +CREATE TABLE list_partitioned (col1 NUMERIC, col2 NUMERIC, col3 VARCHAR(10)) PARTITION BY LIST (col1) ; +ERROR: syntax error at or near "PARTITION" +LINE 1: ...ed (col1 NUMERIC, col2 NUMERIC, col3 VARCHAR(10)) PARTITION ... + ^ +SELECT generate_partition_information('list_partitioned'); +ERROR: relation "list_partitioned" does not exist +LINE 1: SELECT generate_partition_information('list_partitioned'); + ^ +SELECT master_get_table_ddl_events('list_partitioned'); +ERROR: relation "list_partitioned" does not exist +SELECT drop_and_recreate_partitioned_table('list_partitioned'); +ERROR: relation "list_partitioned" does not exist +LINE 1: SELECT drop_and_recreate_partitioned_table('list_partitioned... + ^ +CREATE TABLE list_partitioned_1 PARTITION OF list_partitioned FOR VALUES IN (100, 101, 102, 103, 104); +ERROR: syntax error at or near "PARTITION" +LINE 1: CREATE TABLE list_partitioned_1 PARTITION OF list_partitione... + ^ +SELECT generate_alter_table_attach_partition_command('list_partitioned_1'); +ERROR: relation "list_partitioned_1" does not exist +LINE 1: ...ECT generate_alter_table_attach_partition_command('list_part... + ^ +-- also differentiate partitions and inhereted tables +CREATE TABLE cities ( + name text, + population float, + altitude int -- in feet +); +CREATE TABLE capitals ( + state char(2) +) INHERITS (cities); +-- returns true since capitals inherits from cities +SELECT table_inherits('capitals'); + table_inherits +---------------- + t +(1 row) + +-- although date_partition_2006 inherits from its parent +-- returns false since the hierarcy is formed via partitioning +SELECT table_inherits('date_partition_2006'); +ERROR: relation "date_partition_2006" does not exist +LINE 1: SELECT table_inherits('date_partition_2006'); + ^ +-- returns true since cities inherited by capitals +SELECT table_inherited('cities'); + table_inherited +----------------- + t +(1 row) + +-- although date_partitioned_table inherited by its partitions +-- returns false since the hierarcy is formed via partitioning +SELECT table_inherited('date_partitioned_table'); +ERROR: relation "date_partitioned_table" does not exist +LINE 1: SELECT table_inherited('date_partitioned_table'); + ^ +-- also these are not supported +SELECT master_get_table_ddl_events('capitals'); +ERROR: capitals is not a regular, foreign or partitioned table +SELECT master_get_table_ddl_events('cities'); +ERROR: cities is not a regular, foreign or partitioned table +-- dropping parents frop the partitions +DROP TABLE date_partitioned_table, multi_column_partitioned, list_partitioned, partition_parent_schema.parent_table, cities, capitals; +ERROR: table "date_partitioned_table" does not exist diff --git a/src/test/regress/expected/multi_subtransactions.out b/src/test/regress/expected/multi_subtransactions.out index 8a34a7a6d..45dedde93 100644 --- a/src/test/regress/expected/multi_subtransactions.out +++ b/src/test/regress/expected/multi_subtransactions.out @@ -1,3 +1,11 @@ +-- print whether we're using version > 10 to make version-specific tests clear +SHOW server_version \gset +SELECT substring(:'server_version', '\d+')::int > 10 AS version_above_nine; + version_above_nine +-------------------- + t +(1 row) + CREATE TABLE artists ( id bigint NOT NULL, name text NOT NULL @@ -119,7 +127,7 @@ ERROR: null value in column "name" violates not-null constraint SAVEPOINT s3; ERROR: current transaction is aborted, commands ignored until end of transaction block ROLLBACK TO SAVEPOINT s3; -ERROR: no such savepoint +ERROR: savepoint "s3" does not exist COMMIT; -- =================================================================== -- Tests for replication factor > 1 diff --git a/src/test/regress/expected/multi_subtransactions_0.out b/src/test/regress/expected/multi_subtransactions_0.out new file mode 100644 index 000000000..b21697b05 --- /dev/null +++ b/src/test/regress/expected/multi_subtransactions_0.out @@ -0,0 +1,285 @@ +-- print whether we're using version > 10 to make version-specific tests clear +SHOW server_version \gset +SELECT substring(:'server_version', '\d+')::int > 10 AS version_above_nine; + version_above_nine +-------------------- + f +(1 row) + +CREATE TABLE artists ( + id bigint NOT NULL, + name text NOT NULL +); +SELECT create_distributed_table('artists', 'id'); + create_distributed_table +-------------------------- + +(1 row) + +-- add some data +INSERT INTO artists VALUES (1, 'Pablo Picasso'); +INSERT INTO artists VALUES (2, 'Vincent van Gogh'); +INSERT INTO artists VALUES (3, 'Claude Monet'); +INSERT INTO artists VALUES (4, 'William Kurelek'); +-- RELEASE SAVEPOINT +BEGIN; +INSERT INTO artists VALUES (5, 'Asher Lev'); +SAVEPOINT s1; +DELETE FROM artists WHERE id=5; +RELEASE SAVEPOINT s1; +COMMIT; +SELECT * FROM artists WHERE id=5; + id | name +----+------ +(0 rows) + +-- ROLLBACK TO SAVEPOINT +BEGIN; +INSERT INTO artists VALUES (5, 'Asher Lev'); +SAVEPOINT s1; +DELETE FROM artists WHERE id=5; +ROLLBACK TO SAVEPOINT s1; +COMMIT; +SELECT * FROM artists WHERE id=5; + id | name +----+----------- + 5 | Asher Lev +(1 row) + +-- Serial sub-transaction releases +BEGIN; +SAVEPOINT s1; +DELETE FROM artists WHERE id=5; +RELEASE SAVEPOINT s1; +SAVEPOINT s2; +INSERT INTO artists VALUES (5, 'Jacob Kahn'); +RELEASE SAVEPOINT s2; +COMMIT; +SELECT * FROM artists WHERE id=5; + id | name +----+------------ + 5 | Jacob Kahn +(1 row) + +-- Serial sub-transaction rollbacks +BEGIN; +SAVEPOINT s1; +UPDATE artists SET name='A' WHERE id=5; +ROLLBACK TO SAVEPOINT s1; +SAVEPOINT s2; +DELETE FROM artists WHERE id=5; +ROLLBACK TO SAVEPOINT s2; +COMMIT; +SELECT * FROM artists WHERE id=5; + id | name +----+------------ + 5 | Jacob Kahn +(1 row) + +-- Multiple sub-transaction activity before first query +BEGIN; +SAVEPOINT s0; +SAVEPOINT s1; +SAVEPOINT s2; +SAVEPOINT s3; +ROLLBACK TO SAVEPOINT s2; +RELEASE SAVEPOINT s1; +INSERT INTO artists VALUES (6, 'John J. Audubon'); +ROLLBACK TO SAVEPOINT s0; +INSERT INTO artists VALUES (6, 'Emily Carr'); +COMMIT; +SELECT * FROM artists WHERE id=6; + id | name +----+------------ + 6 | Emily Carr +(1 row) + +-- Release after rollback +BEGIN; +SAVEPOINT s1; +ROLLBACK TO s1; +RELEASE SAVEPOINT s1; +SAVEPOINT s2; +INSERT INTO artists VALUES (7, 'John J. Audubon'); +ROLLBACK TO s2; +RELEASE SAVEPOINT s2; +COMMIT; +SELECT * FROM artists WHERE id=7; + id | name +----+------ +(0 rows) + +-- Recover from errors +\set VERBOSITY terse +BEGIN; +SAVEPOINT s1; +SAVEPOINT s2; +INSERT INTO artists VALUES (7, NULL); +ERROR: null value in column "name" violates not-null constraint +ROLLBACK TO SAVEPOINT s1; +COMMIT; +-- Don't recover from errors +BEGIN; +SAVEPOINT s1; +SAVEPOINT s2; +INSERT INTO artists VALUES (7, NULL); +ERROR: null value in column "name" violates not-null constraint +SAVEPOINT s3; +ERROR: current transaction is aborted, commands ignored until end of transaction block +ROLLBACK TO SAVEPOINT s3; +ERROR: no such savepoint +COMMIT; +-- =================================================================== +-- Tests for replication factor > 1 +-- =================================================================== +CREATE TABLE researchers ( + id bigint NOT NULL, + lab_id int NOT NULL, + name text NOT NULL +); +SELECT master_create_distributed_table('researchers', 'lab_id', 'hash'); + master_create_distributed_table +--------------------------------- + +(1 row) + +SELECT master_create_worker_shards('researchers', 2, 2); + master_create_worker_shards +----------------------------- + +(1 row) + +-- Basic rollback and release +BEGIN; +INSERT INTO researchers VALUES (7, 4, 'Jan Plaza'); +SAVEPOINT s1; +INSERT INTO researchers VALUES (8, 4, 'Alonzo Church'); +ROLLBACK TO s1; +RELEASE SAVEPOINT s1; +COMMIT; +SELECT * FROM researchers WHERE id in (7, 8); + id | lab_id | name +----+--------+----------- + 7 | 4 | Jan Plaza +(1 row) + +-- Recover from failure on one of nodes +BEGIN; +SAVEPOINT s1; +INSERT INTO researchers VALUES (11, 11, 'Dana Scott'); +INSERT INTO researchers VALUES (NULL, 10, 'Stephen Kleene'); +ERROR: null value in column "id" violates not-null constraint +ROLLBACK TO SAVEPOINT s1; +INSERT INTO researchers VALUES (12, 10, 'Stephen Kleene'); +COMMIT; +SELECT * FROM researchers WHERE lab_id=10; + id | lab_id | name +----+--------+---------------- + 12 | 10 | Stephen Kleene +(1 row) + +-- Don't recover, but rollback +BEGIN; +SAVEPOINT s1; +INSERT INTO researchers VALUES (NULL, 10, 'Raymond Smullyan'); +ERROR: null value in column "id" violates not-null constraint +RELEASE SAVEPOINT s1; +ERROR: current transaction is aborted, commands ignored until end of transaction block +SAVEPOINT s2; +ERROR: current transaction is aborted, commands ignored until end of transaction block +ROLLBACK; +SELECT * FROM researchers WHERE lab_id=10; + id | lab_id | name +----+--------+---------------- + 12 | 10 | Stephen Kleene +(1 row) + +-- Don't recover, and commit +BEGIN; +SAVEPOINT s1; +INSERT INTO researchers VALUES (NULL, 10, 'Raymond Smullyan'); +ERROR: null value in column "id" violates not-null constraint +RELEASE SAVEPOINT s1; +ERROR: current transaction is aborted, commands ignored until end of transaction block +SAVEPOINT s2; +ERROR: current transaction is aborted, commands ignored until end of transaction block +COMMIT; +SELECT * FROM researchers WHERE lab_id=10; + id | lab_id | name +----+--------+---------------- + 12 | 10 | Stephen Kleene +(1 row) + +-- Implicit savepoints via pl/pgsql exceptions +BEGIN; +DO $$ +BEGIN + INSERT INTO researchers VALUES (15, 10, 'Melvin Fitting'); + INSERT INTO researchers VALUES (NULL, 10, 'Raymond Smullyan'); +EXCEPTION + WHEN not_null_violation THEN + RAISE NOTICE 'caught not_null_violation'; +END $$; +NOTICE: caught not_null_violation +COMMIT; +SELECT * FROM researchers WHERE lab_id=10; + id | lab_id | name +----+--------+---------------- + 12 | 10 | Stephen Kleene +(1 row) + +BEGIN; +DO $$ +BEGIN + INSERT INTO researchers VALUES (15, 10, 'Melvin Fitting'); + RAISE EXCEPTION plpgsql_error; +EXCEPTION + WHEN plpgsql_error THEN + RAISE NOTICE 'caught manual plpgsql_error'; +END $$; +NOTICE: caught manual plpgsql_error +COMMIT; +SELECT * FROM researchers WHERE lab_id=10; + id | lab_id | name +----+--------+---------------- + 12 | 10 | Stephen Kleene +(1 row) + +BEGIN; +DO $$ +BEGIN + INSERT INTO researchers VALUES (15, 10, 'Melvin Fitting'); + INSERT INTO researchers VALUES (NULL, 10, 'Raymond Smullyan'); +EXCEPTION + WHEN not_null_violation THEN + RAISE EXCEPTION not_null_violation; -- rethrow it +END $$; +ERROR: not_null_violation +COMMIT; +SELECT * FROM researchers WHERE lab_id=10; + id | lab_id | name +----+--------+---------------- + 12 | 10 | Stephen Kleene +(1 row) + +-- Insert something after catching error. +BEGIN; +DO $$ +BEGIN + INSERT INTO researchers VALUES (15, 10, 'Melvin Fitting'); + INSERT INTO researchers VALUES (NULL, 10, 'Raymond Smullyan'); +EXCEPTION + WHEN not_null_violation THEN + INSERT INTO researchers VALUES (32, 10, 'Raymond Smullyan'); +END $$; +COMMIT; +SELECT * FROM researchers WHERE lab_id=10; + id | lab_id | name +----+--------+------------------ + 12 | 10 | Stephen Kleene + 32 | 10 | Raymond Smullyan +(2 rows) + +-- Clean-up +DROP TABLE artists; +DROP TABLE researchers; diff --git a/src/test/regress/expected/multi_utilities.out b/src/test/regress/expected/multi_utilities.out index 85b3fe432..75e18764b 100644 --- a/src/test/regress/expected/multi_utilities.out +++ b/src/test/regress/expected/multi_utilities.out @@ -1,4 +1,12 @@ SET citus.next_shard_id TO 990000; +-- print server version > 10 to make version-specific tests clear +SHOW server_version \gset +SELECT substring(:'server_version', '\d+')::int > 10 as version_above_ten; + version_above_ten +------------------- + t +(1 row) + -- =================================================================== -- test utility statement functionality -- =================================================================== @@ -174,6 +182,19 @@ SELECT master_create_worker_shards('dustbunnies', 1, 2); -- add some data to the distributed table \copy dustbunnies (id, name) from stdin with csv +CREATE TABLE second_dustbunnies(id integer, name text, age integer); +SELECT master_create_distributed_table('second_dustbunnies', 'id', 'hash'); + master_create_distributed_table +--------------------------------- + +(1 row) + +SELECT master_create_worker_shards('second_dustbunnies', 1, 2); + master_create_worker_shards +----------------------------- + +(1 row) + -- following approach adapted from PostgreSQL's stats.sql file -- save relevant stat counter values in refreshable view \c - - - :worker_1_port @@ -361,6 +382,8 @@ WHERE tablename = 'dustbunnies_990002' ORDER BY attname; VACUUM; WARNING: not propagating VACUUM command to worker nodes HINT: Provide a specific table in order to VACUUM distributed tables. +-- check for multiple table vacuum +VACUUM dustbunnies, second_dustbunnies; -- check the current number of vacuum and analyze run on dustbunnies SELECT run_command_on_workers($$SELECT wait_for_stats()$$); run_command_on_workers @@ -372,8 +395,8 @@ SELECT run_command_on_workers($$SELECT wait_for_stats()$$); SELECT run_command_on_workers($$SELECT pg_stat_get_vacuum_count(tablename::regclass) from pg_tables where tablename LIKE 'dustbunnies_%' limit 1$$); run_command_on_workers ------------------------ - (localhost,57637,t,3) - (localhost,57638,t,3) + (localhost,57637,t,4) + (localhost,57638,t,4) (2 rows) SELECT run_command_on_workers($$SELECT pg_stat_get_analyze_count(tablename::regclass) from pg_tables where tablename LIKE 'dustbunnies_%' limit 1$$); @@ -403,8 +426,8 @@ SELECT run_command_on_workers($$SELECT wait_for_stats()$$); SELECT run_command_on_workers($$SELECT pg_stat_get_vacuum_count(tablename::regclass) from pg_tables where tablename LIKE 'dustbunnies_%' limit 1$$); run_command_on_workers ------------------------ - (localhost,57637,t,3) - (localhost,57638,t,3) + (localhost,57637,t,4) + (localhost,57638,t,4) (2 rows) SELECT run_command_on_workers($$SELECT pg_stat_get_analyze_count(tablename::regclass) from pg_tables where tablename LIKE 'dustbunnies_%' limit 1$$); diff --git a/src/test/regress/expected/multi_utilities_0.out b/src/test/regress/expected/multi_utilities_0.out new file mode 100644 index 000000000..c2a8c0191 --- /dev/null +++ b/src/test/regress/expected/multi_utilities_0.out @@ -0,0 +1,478 @@ +SET citus.next_shard_id TO 990000; +-- print server version > 10 to make version-specific tests clear +SHOW server_version \gset +SELECT substring(:'server_version', '\d+')::int > 10 as version_above_ten; + version_above_ten +------------------- + f +(1 row) + +-- =================================================================== +-- test utility statement functionality +-- =================================================================== +CREATE TABLE sharded_table ( name text, id bigint ); +SELECT master_create_distributed_table('sharded_table', 'id', 'hash'); + master_create_distributed_table +--------------------------------- + +(1 row) + +SELECT master_create_worker_shards('sharded_table', 2, 1); + master_create_worker_shards +----------------------------- + +(1 row) + +-- COPY out is supported with distributed tables +COPY sharded_table TO STDOUT; +COPY (SELECT COUNT(*) FROM sharded_table) TO STDOUT; +0 +BEGIN; +SET TRANSACTION READ ONLY; +COPY sharded_table TO STDOUT; +COPY (SELECT COUNT(*) FROM sharded_table) TO STDOUT; +0 +COMMIT; +-- ANALYZE is supported in a transaction block +BEGIN; +ANALYZE sharded_table; +ANALYZE sharded_table; +END; +-- cursors may not involve distributed tables +DECLARE all_sharded_rows CURSOR FOR SELECT * FROM sharded_table; +ERROR: DECLARE CURSOR can only be used in transaction blocks +-- verify PREPARE functionality +PREPARE sharded_insert AS INSERT INTO sharded_table VALUES ('adam', 1); +PREPARE sharded_update AS UPDATE sharded_table SET name = 'bob' WHERE id = 1; +PREPARE sharded_delete AS DELETE FROM sharded_table WHERE id = 1; +PREPARE sharded_query AS SELECT name FROM sharded_table WHERE id = 1; +EXECUTE sharded_query; + name +------ +(0 rows) + +EXECUTE sharded_insert; +EXECUTE sharded_query; + name +------ + adam +(1 row) + +EXECUTE sharded_update; +EXECUTE sharded_query; + name +------ + bob +(1 row) + +EXECUTE sharded_delete; +EXECUTE sharded_query; + name +------ +(0 rows) + +-- try to drop shards with where clause +SELECT master_apply_delete_command('DELETE FROM sharded_table WHERE id > 0'); +ERROR: cannot delete from hash distributed table with this command +DETAIL: Delete statements on hash-partitioned tables are not supported with master_apply_delete_command. +HINT: Use master_modify_multiple_shards command instead. +-- drop all shards +SELECT master_apply_delete_command('DELETE FROM sharded_table'); +ERROR: cannot delete from hash distributed table with this command +DETAIL: Delete statements on hash-partitioned tables are not supported with master_apply_delete_command. +HINT: Use master_modify_multiple_shards command instead. +-- lock shard metadata: take some share locks and exclusive locks +BEGIN; +SELECT lock_shard_metadata(5, ARRAY[999001, 999002, 999002]); + lock_shard_metadata +--------------------- + +(1 row) + +SELECT lock_shard_metadata(7, ARRAY[999001, 999003, 999004]); + lock_shard_metadata +--------------------- + +(1 row) + +SELECT locktype, objid, mode, granted +FROM pg_locks +WHERE objid IN (999001, 999002, 999003, 999004) +ORDER BY objid, mode; + locktype | objid | mode | granted +----------+--------+---------------+--------- + advisory | 999001 | ExclusiveLock | t + advisory | 999001 | ShareLock | t + advisory | 999002 | ShareLock | t + advisory | 999003 | ExclusiveLock | t + advisory | 999004 | ExclusiveLock | t +(5 rows) + +END; +-- lock shard metadata: unsupported lock type +SELECT lock_shard_metadata(0, ARRAY[990001, 999002]); +ERROR: unsupported lockmode 0 +-- lock shard metadata: invalid shard ID +SELECT lock_shard_metadata(5, ARRAY[0]); + lock_shard_metadata +--------------------- + +(1 row) + +-- lock shard metadata: lock nothing +SELECT lock_shard_metadata(5, ARRAY[]::bigint[]); +ERROR: no locks specified +-- lock shard resources: take some share locks and exclusive locks +BEGIN; +SELECT lock_shard_resources(5, ARRAY[999001, 999002, 999002]); + lock_shard_resources +---------------------- + +(1 row) + +SELECT lock_shard_resources(7, ARRAY[999001, 999003, 999004]); + lock_shard_resources +---------------------- + +(1 row) + +SELECT locktype, objid, mode, granted +FROM pg_locks +WHERE objid IN (999001, 999002, 999003, 999004) +ORDER BY objid, mode; + locktype | objid | mode | granted +----------+--------+---------------+--------- + advisory | 999001 | ExclusiveLock | t + advisory | 999001 | ShareLock | t + advisory | 999002 | ShareLock | t + advisory | 999003 | ExclusiveLock | t + advisory | 999004 | ExclusiveLock | t +(5 rows) + +END; +-- lock shard metadata: unsupported lock type +SELECT lock_shard_resources(0, ARRAY[990001, 999002]); +ERROR: unsupported lockmode 0 +-- lock shard metadata: invalid shard ID +SELECT lock_shard_resources(5, ARRAY[-1]); + lock_shard_resources +---------------------- + +(1 row) + +-- lock shard metadata: lock nothing +SELECT lock_shard_resources(5, ARRAY[]::bigint[]); +ERROR: no locks specified +-- drop table +DROP TABLE sharded_table; +-- VACUUM tests +-- create a table with a single shard (for convenience) +CREATE TABLE dustbunnies (id integer, name text, age integer); +SELECT master_create_distributed_table('dustbunnies', 'id', 'hash'); + master_create_distributed_table +--------------------------------- + +(1 row) + +SELECT master_create_worker_shards('dustbunnies', 1, 2); + master_create_worker_shards +----------------------------- + +(1 row) + +-- add some data to the distributed table +\copy dustbunnies (id, name) from stdin with csv +CREATE TABLE second_dustbunnies(id integer, name text, age integer); +SELECT master_create_distributed_table('second_dustbunnies', 'id', 'hash'); + master_create_distributed_table +--------------------------------- + +(1 row) + +SELECT master_create_worker_shards('second_dustbunnies', 1, 2); + master_create_worker_shards +----------------------------- + +(1 row) + +-- following approach adapted from PostgreSQL's stats.sql file +-- save relevant stat counter values in refreshable view +\c - - - :worker_1_port +CREATE MATERIALIZED VIEW prevcounts AS +SELECT analyze_count, vacuum_count FROM pg_stat_user_tables +WHERE relname='dustbunnies_990002'; +-- create function that sleeps until those counters increment +create function wait_for_stats() returns void as $$ +declare + start_time timestamptz := clock_timestamp(); + analyze_updated bool; + vacuum_updated bool; +begin + -- we don't want to wait forever; loop will exit after 10 seconds + for i in 1 .. 100 loop + + -- check to see if analyze has been updated + SELECT (st.analyze_count >= pc.analyze_count + 1) INTO analyze_updated + FROM pg_stat_user_tables AS st, pg_class AS cl, prevcounts AS pc + WHERE st.relname='dustbunnies_990002' AND cl.relname='dustbunnies_990002'; + + -- check to see if vacuum has been updated + SELECT (st.vacuum_count >= pc.vacuum_count + 1) INTO vacuum_updated + FROM pg_stat_user_tables AS st, pg_class AS cl, prevcounts AS pc + WHERE st.relname='dustbunnies_990002' AND cl.relname='dustbunnies_990002'; + + exit when analyze_updated or vacuum_updated; + + -- wait a little + perform pg_sleep(0.1); + + -- reset stats snapshot so we can test again + perform pg_stat_clear_snapshot(); + + end loop; + + -- report time waited in postmaster log (where it won't change test output) + raise log 'wait_for_stats delayed % seconds', + extract(epoch from clock_timestamp() - start_time); +end +$$ language plpgsql; +\c - - - :worker_2_port +CREATE MATERIALIZED VIEW prevcounts AS +SELECT analyze_count, vacuum_count FROM pg_stat_user_tables +WHERE relname='dustbunnies_990001'; +-- create function that sleeps until those counters increment +create function wait_for_stats() returns void as $$ +declare + start_time timestamptz := clock_timestamp(); + analyze_updated bool; + vacuum_updated bool; +begin + -- we don't want to wait forever; loop will exit after 10 seconds + for i in 1 .. 100 loop + + -- check to see if analyze has been updated + SELECT (st.analyze_count >= pc.analyze_count + 1) INTO analyze_updated + FROM pg_stat_user_tables AS st, pg_class AS cl, prevcounts AS pc + WHERE st.relname='dustbunnies_990001' AND cl.relname='dustbunnies_990001'; + + -- check to see if vacuum has been updated + SELECT (st.vacuum_count >= pc.vacuum_count + 1) INTO vacuum_updated + FROM pg_stat_user_tables AS st, pg_class AS cl, prevcounts AS pc + WHERE st.relname='dustbunnies_990001' AND cl.relname='dustbunnies_990001'; + + exit when analyze_updated or vacuum_updated; + + -- wait a little + perform pg_sleep(0.1); + + -- reset stats snapshot so we can test again + perform pg_stat_clear_snapshot(); + + end loop; + + -- report time waited in postmaster log (where it won't change test output) + raise log 'wait_for_stats delayed % seconds', + extract(epoch from clock_timestamp() - start_time); +end +$$ language plpgsql; +-- run VACUUM and ANALYZE against the table on the master +\c - - - :master_port +VACUUM dustbunnies; +ANALYZE dustbunnies; +-- verify that the VACUUM and ANALYZE ran +\c - - - :worker_1_port +SELECT wait_for_stats(); + wait_for_stats +---------------- + +(1 row) + +REFRESH MATERIALIZED VIEW prevcounts; +SELECT pg_stat_get_vacuum_count('dustbunnies_990002'::regclass); + pg_stat_get_vacuum_count +-------------------------- + 1 +(1 row) + +SELECT pg_stat_get_analyze_count('dustbunnies_990002'::regclass); + pg_stat_get_analyze_count +--------------------------- + 1 +(1 row) + +-- get file node to verify VACUUM FULL +SELECT relfilenode AS oldnode FROM pg_class WHERE oid='dustbunnies_990002'::regclass +\gset +-- send a VACUUM FULL and a VACUUM ANALYZE +\c - - - :master_port +VACUUM (FULL) dustbunnies; +VACUUM ANALYZE dustbunnies; +-- verify that relfilenode changed +\c - - - :worker_1_port +SELECT relfilenode != :oldnode AS table_rewritten FROM pg_class +WHERE oid='dustbunnies_990002'::regclass; + table_rewritten +----------------- + t +(1 row) + +-- verify the VACUUM ANALYZE incremented both vacuum and analyze counts +SELECT wait_for_stats(); + wait_for_stats +---------------- + +(1 row) + +SELECT pg_stat_get_vacuum_count('dustbunnies_990002'::regclass); + pg_stat_get_vacuum_count +-------------------------- + 2 +(1 row) + +SELECT pg_stat_get_analyze_count('dustbunnies_990002'::regclass); + pg_stat_get_analyze_count +--------------------------- + 2 +(1 row) + +-- disable auto-VACUUM for next test +ALTER TABLE dustbunnies_990002 SET (autovacuum_enabled = false); +SELECT relfrozenxid AS frozenxid FROM pg_class WHERE oid='dustbunnies_990002'::regclass +\gset +-- send a VACUUM FREEZE after adding a new row +\c - - - :master_port +INSERT INTO dustbunnies VALUES (5, 'peter'); +VACUUM (FREEZE) dustbunnies; +-- verify that relfrozenxid increased +\c - - - :worker_1_port +SELECT relfrozenxid::text::integer > :frozenxid AS frozen_performed FROM pg_class +WHERE oid='dustbunnies_990002'::regclass; + frozen_performed +------------------ + t +(1 row) + +-- check there are no nulls in either column +SELECT attname, null_frac FROM pg_stats +WHERE tablename = 'dustbunnies_990002' ORDER BY attname; + attname | null_frac +---------+----------- + age | 1 + id | 0 + name | 0 +(3 rows) + +-- add NULL values, then perform column-specific ANALYZE +\c - - - :master_port +INSERT INTO dustbunnies VALUES (6, NULL, NULL); +ANALYZE dustbunnies (name); +-- verify that name's NULL ratio is updated but age's is not +\c - - - :worker_1_port +SELECT attname, null_frac FROM pg_stats +WHERE tablename = 'dustbunnies_990002' ORDER BY attname; + attname | null_frac +---------+----------- + age | 1 + id | 0 + name | 0.166667 +(3 rows) + +\c - - - :master_port +-- verify warning for unqualified VACUUM +VACUUM; +WARNING: not propagating VACUUM command to worker nodes +HINT: Provide a specific table in order to VACUUM distributed tables. +-- check for multiple table vacuum +VACUUM dustbunnies, second_dustbunnies; +ERROR: syntax error at or near "," +LINE 1: VACUUM dustbunnies, second_dustbunnies; + ^ +-- check the current number of vacuum and analyze run on dustbunnies +SELECT run_command_on_workers($$SELECT wait_for_stats()$$); + run_command_on_workers +------------------------ + (localhost,57637,t,"") + (localhost,57638,t,"") +(2 rows) + +SELECT run_command_on_workers($$SELECT pg_stat_get_vacuum_count(tablename::regclass) from pg_tables where tablename LIKE 'dustbunnies_%' limit 1$$); + run_command_on_workers +------------------------ + (localhost,57637,t,3) + (localhost,57638,t,3) +(2 rows) + +SELECT run_command_on_workers($$SELECT pg_stat_get_analyze_count(tablename::regclass) from pg_tables where tablename LIKE 'dustbunnies_%' limit 1$$); + run_command_on_workers +------------------------ + (localhost,57637,t,3) + (localhost,57638,t,3) +(2 rows) + +-- and warning when using targeted VACUUM without DDL propagation +SET citus.enable_ddl_propagation to false; +VACUUM dustbunnies; +WARNING: not propagating VACUUM command to worker nodes +HINT: Set citus.enable_ddl_propagation to true in order to send targeted VACUUM commands to worker nodes. +ANALYZE dustbunnies; +WARNING: not propagating ANALYZE command to worker nodes +HINT: Set citus.enable_ddl_propagation to true in order to send targeted ANALYZE commands to worker nodes. +SET citus.enable_ddl_propagation to DEFAULT; +-- should not propagate the vacuum and analyze +SELECT run_command_on_workers($$SELECT wait_for_stats()$$); + run_command_on_workers +------------------------ + (localhost,57637,t,"") + (localhost,57638,t,"") +(2 rows) + +SELECT run_command_on_workers($$SELECT pg_stat_get_vacuum_count(tablename::regclass) from pg_tables where tablename LIKE 'dustbunnies_%' limit 1$$); + run_command_on_workers +------------------------ + (localhost,57637,t,3) + (localhost,57638,t,3) +(2 rows) + +SELECT run_command_on_workers($$SELECT pg_stat_get_analyze_count(tablename::regclass) from pg_tables where tablename LIKE 'dustbunnies_%' limit 1$$); + run_command_on_workers +------------------------ + (localhost,57637,t,3) + (localhost,57638,t,3) +(2 rows) + +-- test worker_hash +SELECT worker_hash(123); + worker_hash +------------- + -205084363 +(1 row) + +SELECT worker_hash('1997-08-08'::date); + worker_hash +------------- + -499701663 +(1 row) + +-- test a custom type (this test should run after multi_data_types) +SELECT worker_hash('(1, 2)'); +ERROR: cannot find a hash function for the input type +HINT: Cast input to a data type with a hash function. +SELECT worker_hash('(1, 2)'::test_composite_type); + worker_hash +------------- + -1895345704 +(1 row) + +SELECT citus_truncate_trigger(); +ERROR: must be called as trigger +-- confirm that citus_create_restore_point works +SELECT 1 FROM citus_create_restore_point('regression-test'); + ?column? +---------- + 1 +(1 row) + +-- TODO: support VERBOSE +-- VACUUM VERBOSE dustbunnies; +-- VACUUM (FULL, VERBOSE) dustbunnies; +-- ANALYZE VERBOSE dustbunnies; diff --git a/src/test/regress/expected/multi_view.out b/src/test/regress/expected/multi_view.out index d8c546af9..035ebc31d 100644 --- a/src/test/regress/expected/multi_view.out +++ b/src/test/regress/expected/multi_view.out @@ -5,6 +5,14 @@ -- Citus features: simple selects, aggregates, joins, outer joins -- router queries, single row inserts, multi row inserts via insert -- into select, multi row insert via copy commands. +-- print whether we're using version > 10 to make version-specific tests clear +SHOW server_version \gset +SELECT substring(:'server_version', '\d+')::int > 10 AS version_above_ten; + version_above_ten +------------------- + t +(1 row) + SELECT count(*) FROM lineitem_hash_part; count ------- @@ -789,8 +797,8 @@ RESET citus.subquery_pushdown; VACUUM ANALYZE users_table; -- explain tests EXPLAIN (COSTS FALSE) SELECT user_id FROM recent_selected_users GROUP BY 1 ORDER BY 1; - QUERY PLAN ---------------------------------------------------------------------------------------------------------------------------------------------------------- + QUERY PLAN +--------------------------------------------------------------------------------------------------------------------------------------------- Sort Sort Key: remote_scan.user_id -> HashAggregate @@ -802,19 +810,17 @@ EXPLAIN (COSTS FALSE) SELECT user_id FROM recent_selected_users GROUP BY 1 ORDER Node: host=localhost port=57637 dbname=regression -> HashAggregate Group Key: users_table.user_id - -> Hash Join - Hash Cond: (users_table.user_id = ru.user_id) + -> Nested Loop + Join Filter: (users_table.user_id = users_table_1.user_id) + -> Sort + Sort Key: (max(users_table_1."time")) DESC + -> HashAggregate + Group Key: users_table_1.user_id + Filter: (max(users_table_1."time") > '2017-11-23 16:20:33.264457'::timestamp without time zone) + -> Seq Scan on users_table_1400000 users_table_1 -> Seq Scan on users_table_1400000 users_table Filter: ((value_1 >= 1) AND (value_1 < 3)) - -> Hash - -> Subquery Scan on ru - -> Sort - Sort Key: (max(users_table_1."time")) DESC - -> HashAggregate - Group Key: users_table_1.user_id - Filter: (max(users_table_1."time") > '2017-11-23 16:20:33.264457'::timestamp without time zone) - -> Seq Scan on users_table_1400000 users_table_1 -(23 rows) +(21 rows) EXPLAIN (COSTS FALSE) SELECT * FROM ( diff --git a/src/test/regress/expected/multi_view_0.out b/src/test/regress/expected/multi_view_0.out new file mode 100644 index 000000000..1cc59c768 --- /dev/null +++ b/src/test/regress/expected/multi_view_0.out @@ -0,0 +1,937 @@ +-- +-- MULTI_VIEW +-- +-- This file contains test cases for view support. It verifies various +-- Citus features: simple selects, aggregates, joins, outer joins +-- router queries, single row inserts, multi row inserts via insert +-- into select, multi row insert via copy commands. +-- print whether we're using version > 10 to make version-specific tests clear +SHOW server_version \gset +SELECT substring(:'server_version', '\d+')::int > 10 AS version_above_ten; + version_above_ten +------------------- + f +(1 row) + +SELECT count(*) FROM lineitem_hash_part; + count +------- + 12000 +(1 row) + +SELECT count(*) FROM orders_hash_part; + count +------- + 2985 +(1 row) + +-- create a view for priority orders +CREATE VIEW priority_orders AS SELECT * FROM orders_hash_part WHERE o_orderpriority < '3-MEDIUM'; +-- aggregate pushdown +SELECT o_orderpriority, count(*) FROM priority_orders GROUP BY 1 ORDER BY 2, 1; + o_orderpriority | count +-----------------+------- + 2-HIGH | 593 + 1-URGENT | 604 +(2 rows) + +SELECT o_orderpriority, count(*) FROM orders_hash_part WHERE o_orderpriority < '3-MEDIUM' GROUP BY 1 ORDER BY 2,1; + o_orderpriority | count +-----------------+------- + 2-HIGH | 593 + 1-URGENT | 604 +(2 rows) + +-- filters +SELECT o_orderpriority, count(*) as all, count(*) FILTER (WHERE o_orderstatus ='F') as fullfilled FROM priority_orders GROUP BY 1 ORDER BY 2, 1; + o_orderpriority | all | fullfilled +-----------------+-----+------------ + 2-HIGH | 593 | 271 + 1-URGENT | 604 | 280 +(2 rows) + +-- having +SELECT o_orderdate, count(*) from priority_orders group by 1 having (count(*) > 3) order by 2 desc, 1 desc; + o_orderdate | count +-------------+------- + 08-20-1996 | 5 + 10-10-1994 | 4 + 05-05-1994 | 4 + 04-07-1994 | 4 + 03-17-1993 | 4 +(5 rows) + +-- having with filters +SELECT o_orderdate, count(*) as all, count(*) FILTER(WHERE o_orderstatus = 'F') from priority_orders group by 1 having (count(*) > 3) order by 2 desc, 1 desc; + o_orderdate | all | count +-------------+-----+------- + 08-20-1996 | 5 | 0 + 10-10-1994 | 4 | 4 + 05-05-1994 | 4 | 4 + 04-07-1994 | 4 | 4 + 03-17-1993 | 4 | 4 +(5 rows) + +-- limit +SELECT o_orderkey, o_totalprice from orders_hash_part order by 2 desc, 1 asc limit 5 ; + o_orderkey | o_totalprice +------------+-------------- + 4421 | 401055.62 + 10209 | 400191.77 + 11142 | 395039.05 + 14179 | 384265.43 + 11296 | 378166.33 +(5 rows) + +SELECT o_orderkey, o_totalprice from priority_orders order by 2 desc, 1 asc limit 1 ; + o_orderkey | o_totalprice +------------+-------------- + 14179 | 384265.43 +(1 row) + +CREATE VIEW priority_lineitem AS SELECT li.* FROM lineitem_hash_part li JOIN priority_orders ON (l_orderkey = o_orderkey); +SELECT l_orderkey, count(*) FROM priority_lineitem GROUP BY 1 ORDER BY 2 DESC, 1 LIMIT 5; + l_orderkey | count +------------+------- + 7 | 7 + 225 | 7 + 226 | 7 + 322 | 7 + 326 | 7 +(5 rows) + +CREATE VIEW air_shipped_lineitems AS SELECT * FROM lineitem_hash_part WHERE l_shipmode = 'AIR'; +-- join between view and table +SELECT count(*) FROM orders_hash_part join air_shipped_lineitems ON (o_orderkey = l_orderkey); + count +------- + 1706 +(1 row) + +-- join between views +SELECT count(*) FROM priority_orders join air_shipped_lineitems ON (o_orderkey = l_orderkey); + count +------- + 700 +(1 row) + +-- count distinct on partition column is supported +SELECT count(distinct o_orderkey) FROM priority_orders join air_shipped_lineitems ON (o_orderkey = l_orderkey); + count +------- + 551 +(1 row) + +-- count distinct on non-partition column is supported +SELECT count(distinct o_orderpriority) FROM priority_orders join air_shipped_lineitems ON (o_orderkey = l_orderkey); + count +------- + 2 +(1 row) + +-- count distinct on partition column is supported on router queries +SELECT count(distinct o_orderkey) FROM priority_orders join air_shipped_lineitems + ON (o_orderkey = l_orderkey) + WHERE (o_orderkey = 231); + count +------- + 1 +(1 row) + +-- select distinct on router joins of views also works +SELECT distinct(o_orderkey) FROM priority_orders join air_shipped_lineitems + ON (o_orderkey = l_orderkey) + WHERE (o_orderkey = 231); + o_orderkey +------------ + 231 +(1 row) + +-- left join support depends on flattening of the query +-- following query fails since the inner part is kept as subquery +SELECT * FROM priority_orders left join air_shipped_lineitems ON (o_orderkey = l_orderkey); +ERROR: cannot perform distributed planning on this query +DETAIL: Subqueries in outer joins are not supported +-- however, this works +SELECT count(*) FROM priority_orders left join lineitem_hash_part ON (o_orderkey = l_orderkey) WHERE l_shipmode ='AIR'; + count +------- + 700 +(1 row) + +-- view at the inner side of is not supported +SELECT count(*) FROM priority_orders right join lineitem_hash_part ON (o_orderkey = l_orderkey) WHERE l_shipmode ='AIR'; +ERROR: cannot perform distributed planning on this query +DETAIL: Subqueries in outer joins are not supported +-- but view at the outer side is. This is essentially the same as a left join with arguments reversed. +SELECT count(*) FROM lineitem_hash_part right join priority_orders ON (o_orderkey = l_orderkey) WHERE l_shipmode ='AIR'; + count +------- + 700 +(1 row) + +-- left join on router query is supported +SELECT o_orderkey, l_linenumber FROM priority_orders left join air_shipped_lineitems ON (o_orderkey = l_orderkey) + WHERE o_orderkey = 2; + o_orderkey | l_linenumber +------------+-------------- + 2 | +(1 row) + +-- repartition query on view join +-- it passes planning, fails at execution stage +SET client_min_messages TO DEBUG1; +SELECT * FROM priority_orders JOIN air_shipped_lineitems ON (o_custkey = l_suppkey) ORDER BY o_orderkey DESC, o_custkey DESC, o_orderpriority DESC LIMIT 5; +DEBUG: generating subplan 22_1 for subquery SELECT lineitem_hash_part.l_orderkey, lineitem_hash_part.l_partkey, lineitem_hash_part.l_suppkey, lineitem_hash_part.l_linenumber, lineitem_hash_part.l_quantity, lineitem_hash_part.l_extendedprice, lineitem_hash_part.l_discount, lineitem_hash_part.l_tax, lineitem_hash_part.l_returnflag, lineitem_hash_part.l_linestatus, lineitem_hash_part.l_shipdate, lineitem_hash_part.l_commitdate, lineitem_hash_part.l_receiptdate, lineitem_hash_part.l_shipinstruct, lineitem_hash_part.l_shipmode, lineitem_hash_part.l_comment FROM public.lineitem_hash_part WHERE (lineitem_hash_part.l_shipmode = 'AIR'::bpchar) +DEBUG: Plan 22 query after replacing subqueries and CTEs: SELECT priority_orders.o_orderkey, priority_orders.o_custkey, priority_orders.o_orderstatus, priority_orders.o_totalprice, priority_orders.o_orderdate, priority_orders.o_orderpriority, priority_orders.o_clerk, priority_orders.o_shippriority, priority_orders.o_comment, air_shipped_lineitems.l_orderkey, air_shipped_lineitems.l_partkey, air_shipped_lineitems.l_suppkey, air_shipped_lineitems.l_linenumber, air_shipped_lineitems.l_quantity, air_shipped_lineitems.l_extendedprice, air_shipped_lineitems.l_discount, air_shipped_lineitems.l_tax, air_shipped_lineitems.l_returnflag, air_shipped_lineitems.l_linestatus, air_shipped_lineitems.l_shipdate, air_shipped_lineitems.l_commitdate, air_shipped_lineitems.l_receiptdate, air_shipped_lineitems.l_shipinstruct, air_shipped_lineitems.l_shipmode, air_shipped_lineitems.l_comment FROM ((SELECT orders_hash_part.o_orderkey, orders_hash_part.o_custkey, orders_hash_part.o_orderstatus, orders_hash_part.o_totalprice, orders_hash_part.o_orderdate, orders_hash_part.o_orderpriority, orders_hash_part.o_clerk, orders_hash_part.o_shippriority, orders_hash_part.o_comment FROM public.orders_hash_part WHERE (orders_hash_part.o_orderpriority < '3-MEDIUM'::bpchar)) priority_orders JOIN (SELECT intermediate_result.l_orderkey, intermediate_result.l_partkey, intermediate_result.l_suppkey, intermediate_result.l_linenumber, intermediate_result.l_quantity, intermediate_result.l_extendedprice, intermediate_result.l_discount, intermediate_result.l_tax, intermediate_result.l_returnflag, intermediate_result.l_linestatus, intermediate_result.l_shipdate, intermediate_result.l_commitdate, intermediate_result.l_receiptdate, intermediate_result.l_shipinstruct, intermediate_result.l_shipmode, intermediate_result.l_comment FROM read_intermediate_result('22_1'::text, 'binary'::citus_copy_format) intermediate_result(l_orderkey bigint, l_partkey integer, l_suppkey integer, l_linenumber integer, l_quantity numeric(15,2), l_extendedprice numeric(15,2), l_discount numeric(15,2), l_tax numeric(15,2), l_returnflag character(1), l_linestatus character(1), l_shipdate date, l_commitdate date, l_receiptdate date, l_shipinstruct character(25), l_shipmode character(10), l_comment character varying(44))) air_shipped_lineitems ON ((priority_orders.o_custkey = air_shipped_lineitems.l_suppkey))) ORDER BY priority_orders.o_orderkey DESC, priority_orders.o_custkey DESC, priority_orders.o_orderpriority DESC LIMIT 5 +DEBUG: push down of limit count: 5 + o_orderkey | o_custkey | o_orderstatus | o_totalprice | o_orderdate | o_orderpriority | o_clerk | o_shippriority | o_comment | l_orderkey | l_partkey | l_suppkey | l_linenumber | l_quantity | l_extendedprice | l_discount | l_tax | l_returnflag | l_linestatus | l_shipdate | l_commitdate | l_receiptdate | l_shipinstruct | l_shipmode | l_comment +------------+-----------+---------------+--------------+-------------+-----------------+-----------------+----------------+-------------------------------------------------------+------------+-----------+-----------+--------------+------------+-----------------+------------+-------+--------------+--------------+------------+--------------+---------------+---------------------------+------------+------------------------------------------- + 14821 | 1435 | O | 322002.95 | 06-12-1998 | 2-HIGH | Clerk#000000630 | 0 | n packages are furiously ironic ideas. d | 1607 | 118923 | 1435 | 2 | 37.00 | 71851.04 | 0.05 | 0.02 | N | O | 02-27-1996 | 02-18-1996 | 03-16-1996 | NONE | AIR | alongside + 14790 | 613 | O | 270163.54 | 08-21-1996 | 2-HIGH | Clerk#000000347 | 0 | p. regular deposits wake. final n | 2629 | 123076 | 613 | 2 | 31.00 | 34071.17 | 0.08 | 0.03 | N | O | 05-24-1998 | 05-26-1998 | 06-10-1998 | COLLECT COD | AIR | ate blithely bold, regular deposits. bold + 14758 | 1225 | F | 37812.49 | 10-27-1993 | 2-HIGH | Clerk#000000687 | 0 | ages nag about the furio | 9156 | 176190 | 1225 | 2 | 22.00 | 27856.18 | 0.03 | 0.00 | R | F | 02-08-1994 | 04-01-1994 | 02-24-1994 | DELIVER IN PERSON | AIR | equests dete + 14725 | 569 | O | 261801.45 | 06-17-1995 | 2-HIGH | Clerk#000000177 | 0 | ng asymptotes. final, ironic accounts cajole after | 14688 | 173017 | 569 | 3 | 10.00 | 10900.10 | 0.02 | 0.08 | N | O | 03-14-1997 | 04-22-1997 | 04-05-1997 | COLLECT COD | AIR | riously even packages sleep a + 14657 | 370 | F | 116160.53 | 02-28-1994 | 1-URGENT | Clerk#000000756 | 0 | ly across the ironic, ironic instructions. bold ideas | 5153 | 67863 | 370 | 3 | 30.00 | 54925.80 | 0.09 | 0.01 | N | O | 11-10-1995 | 11-14-1995 | 11-16-1995 | DELIVER IN PERSON | AIR | beans sleep bl +(5 rows) + +RESET client_min_messages; +SELECT count(*) FROM priority_orders JOIN air_shipped_lineitems ON (o_custkey = l_suppkey); + count +------- + 192 +(1 row) + +-- materialized views work +-- insert into... select works with views +CREATE TABLE temp_lineitem(LIKE lineitem_hash_part); +SELECT create_distributed_table('temp_lineitem', 'l_orderkey', 'hash', 'lineitem_hash_part'); + create_distributed_table +-------------------------- + +(1 row) + +INSERT INTO temp_lineitem SELECT * FROM air_shipped_lineitems; +SELECT count(*) FROM temp_lineitem; + count +------- + 1706 +(1 row) + +-- following is a where false query, should not be inserting anything +INSERT INTO temp_lineitem SELECT * FROM air_shipped_lineitems WHERE l_shipmode = 'MAIL'; +SELECT count(*) FROM temp_lineitem; + count +------- + 1706 +(1 row) + +-- can create and query materialized views +CREATE MATERIALIZED VIEW mode_counts +AS SELECT l_shipmode, count(*) FROM temp_lineitem GROUP BY l_shipmode; +SELECT * FROM mode_counts WHERE l_shipmode = 'AIR' ORDER BY 2 DESC, 1 LIMIT 10; + l_shipmode | count +------------+------- + AIR | 1706 +(1 row) + +-- materialized views are local, cannot join with distributed tables +SELECT count(*) FROM mode_counts JOIN temp_lineitem USING (l_shipmode); +ERROR: relation mode_counts is not distributed +-- new data is not immediately reflected in the view +INSERT INTO temp_lineitem SELECT * FROM air_shipped_lineitems; +SELECT * FROM mode_counts WHERE l_shipmode = 'AIR' ORDER BY 2 DESC, 1 LIMIT 10; + l_shipmode | count +------------+------- + AIR | 1706 +(1 row) + +-- refresh updates the materialised view with new data +REFRESH MATERIALIZED VIEW mode_counts; +SELECT * FROM mode_counts WHERE l_shipmode = 'AIR' ORDER BY 2 DESC, 1 LIMIT 10; + l_shipmode | count +------------+------- + AIR | 3412 +(1 row) + +DROP MATERIALIZED VIEW mode_counts; +SET citus.task_executor_type to "task-tracker"; +-- single view repartition subqueries are not supported +SELECT l_suppkey, count(*) FROM + (SELECT l_suppkey, l_shipdate, count(*) + FROM air_shipped_lineitems GROUP BY l_suppkey, l_shipdate) supps + GROUP BY l_suppkey ORDER BY 2 DESC, 1 LIMIT 5; +ERROR: cannot perform distributed planning on this query +DETAIL: Subqueries without group by clause are not supported yet +-- logically same query without a view works fine +SELECT l_suppkey, count(*) FROM + (SELECT l_suppkey, l_shipdate, count(*) + FROM lineitem_hash_part WHERE l_shipmode = 'AIR' GROUP BY l_suppkey, l_shipdate) supps + GROUP BY l_suppkey ORDER BY 2 DESC, 1 LIMIT 5; + l_suppkey | count +-----------+------- + 7680 | 4 + 160 | 3 + 1042 | 3 + 1318 | 3 + 5873 | 3 +(5 rows) + +-- when a view is replaced by actual query it still fails +SELECT l_suppkey, count(*) FROM + (SELECT l_suppkey, l_shipdate, count(*) + FROM (SELECT * FROM lineitem_hash_part WHERE l_shipmode = 'AIR') asi + GROUP BY l_suppkey, l_shipdate) supps + GROUP BY l_suppkey ORDER BY 2 DESC, 1 LIMIT 5; +ERROR: cannot perform distributed planning on this query +DETAIL: Subqueries without group by clause are not supported yet +-- repartition query on view with single table subquery +CREATE VIEW supp_count_view AS SELECT * FROM (SELECT l_suppkey, count(*) FROM lineitem_hash_part GROUP BY 1) s1; +SELECT * FROM supp_count_view ORDER BY 2 DESC, 1 LIMIT 10; + l_suppkey | count +-----------+------- + 6104 | 8 + 1868 | 6 + 5532 | 6 + 5849 | 6 + 6169 | 6 + 6669 | 6 + 6692 | 6 + 7703 | 6 + 7869 | 6 + 8426 | 6 +(10 rows) + +SET citus.task_executor_type to DEFAULT; +-- create a view with aggregate +CREATE VIEW lineitems_by_shipping_method AS + SELECT l_shipmode, count(*) as cnt FROM lineitem_hash_part GROUP BY 1; +-- following will be supported via recursive planning +SELECT * FROM lineitems_by_shipping_method ORDER BY 1,2 LIMIT 5; + l_shipmode | cnt +------------+------ + AIR | 1706 + FOB | 1709 + MAIL | 1739 + RAIL | 1706 + REG AIR | 1679 +(5 rows) + +-- create a view with group by on partition column +CREATE VIEW lineitems_by_orderkey AS + SELECT + l_orderkey, count(*) + FROM + lineitem_hash_part + GROUP BY 1; +-- this should work since we're able to push down this query +SELECT * FROM lineitems_by_orderkey ORDER BY 2 DESC, 1 ASC LIMIT 10; + l_orderkey | count +------------+------- + 7 | 7 + 68 | 7 + 129 | 7 + 164 | 7 + 194 | 7 + 225 | 7 + 226 | 7 + 322 | 7 + 326 | 7 + 354 | 7 +(10 rows) + +-- it would also work since it is made router plannable +SELECT * FROM lineitems_by_orderkey WHERE l_orderkey = 100; + l_orderkey | count +------------+------- + 100 | 5 +(1 row) + +DROP TABLE temp_lineitem CASCADE; +DROP VIEW supp_count_view; +DROP VIEW lineitems_by_orderkey; +DROP VIEW lineitems_by_shipping_method; +DROP VIEW air_shipped_lineitems; +DROP VIEW priority_lineitem; +DROP VIEW priority_orders; +-- new tests for real time use case including views and subqueries +-- create view to display recent user who has an activity after a timestamp +CREATE VIEW recent_users AS + SELECT user_id, max(time) as lastseen FROM users_table + GROUP BY user_id + HAVING max(time) > '2017-11-23 16:20:33.264457'::timestamp order by 2 DESC; +SELECT * FROM recent_users; + user_id | lastseen +---------+--------------------------------- + 1 | Thu Nov 23 17:30:34.635085 2017 + 5 | Thu Nov 23 16:48:32.08896 2017 + 3 | Thu Nov 23 17:18:51.048758 2017 +(3 rows) + +-- create a view for recent_events +CREATE VIEW recent_events AS + SELECT user_id, time FROM events_table + WHERE time > '2017-11-23 16:20:33.264457'::timestamp; +SELECT count(*) FROM recent_events; + count +------- + 6 +(1 row) + +-- count number of events of recent_users +SELECT count(*) FROM recent_users ru JOIN events_table et ON (ru.user_id = et.user_id); + count +------- + 50 +(1 row) + +-- count number of events of per recent users order by count +SELECT ru.user_id, count(*) + FROM recent_users ru + JOIN events_table et + ON (ru.user_id = et.user_id) + GROUP BY ru.user_id + ORDER BY 2 DESC, 1; + user_id | count +---------+------- + 3 | 21 + 1 | 15 + 5 | 14 +(3 rows) + +-- the same query with a left join however, it would still generate the same result +SELECT ru.user_id, count(*) + FROM recent_users ru + LEFT JOIN events_table et + ON (ru.user_id = et.user_id) + GROUP BY ru.user_id + ORDER BY 2 DESC, 1; + user_id | count +---------+------- + 3 | 21 + 1 | 15 + 5 | 14 +(3 rows) + +-- query wrapped inside a subquery, it needs another top level order by +SELECT * FROM + (SELECT ru.user_id, count(*) + FROM recent_users ru + JOIN events_table et + ON (ru.user_id = et.user_id) + GROUP BY ru.user_id + ORDER BY 2 DESC, 1) s1 +ORDER BY 2 DESC, 1; + user_id | count +---------+------- + 3 | 21 + 1 | 15 + 5 | 14 +(3 rows) + +-- non-partition key joins are not supported inside subquery +-- since the join with a table +SELECT * FROM + (SELECT ru.user_id, count(*) + FROM recent_users ru + JOIN events_table et + ON (ru.user_id = et.event_type) + GROUP BY ru.user_id + ORDER BY 2 DESC, 1) s1 +ORDER BY 2 DESC, 1; +ERROR: bogus varno: 3 +-- join between views +-- recent users who has an event in recent events +SELECT ru.user_id FROM recent_users ru JOIN recent_events re USING(user_id) GROUP BY ru.user_id ORDER BY ru.user_id; + user_id +--------- + 1 + 3 +(2 rows) + +-- outer join inside a subquery +-- recent_events who are not done by recent users +SELECT count(*) FROM ( + SELECT re.*, ru.user_id AS recent_user + FROM recent_events re LEFT JOIN recent_users ru USING(user_id)) reu + WHERE recent_user IS NULL; + count +------- + 2 +(1 row) + +-- same query with anti-join +SELECT count(*) + FROM recent_events re LEFT JOIN recent_users ru ON(ru.user_id = re.user_id) + WHERE ru.user_id IS NULL; + count +------- + 2 +(1 row) + +-- join between view and table +-- users who has recent activity and they have an entry with value_1 is less than 3 +SELECT ut.* FROM recent_users ru JOIN users_table ut USING (user_id) WHERE ut.value_1 < 3 ORDER BY 1,2; + user_id | time | value_1 | value_2 | value_3 | value_4 +---------+---------------------------------+---------+---------+---------+--------- + 1 | Thu Nov 23 09:26:42.145043 2017 | 1 | 3 | 3 | + 3 | Wed Nov 22 18:43:51.450263 2017 | 1 | 1 | 4 | + 3 | Wed Nov 22 20:43:31.008625 2017 | 1 | 3 | 2 | + 3 | Thu Nov 23 00:15:45.610845 2017 | 1 | 1 | 4 | + 3 | Thu Nov 23 03:23:24.702501 2017 | 1 | 2 | 5 | + 3 | Thu Nov 23 06:20:05.854857 2017 | 1 | 4 | 2 | + 3 | Thu Nov 23 09:57:41.540228 2017 | 2 | 2 | 3 | + 3 | Thu Nov 23 11:18:53.114408 2017 | 2 | 2 | 0 | + 3 | Thu Nov 23 12:56:49.29191 2017 | 0 | 5 | 1 | + 3 | Thu Nov 23 17:18:51.048758 2017 | 1 | 5 | 5 | + 5 | Wed Nov 22 20:43:18.667473 2017 | 0 | 3 | 2 | + 5 | Wed Nov 22 21:02:07.575129 2017 | 2 | 0 | 2 | + 5 | Wed Nov 22 22:10:24.315371 2017 | 1 | 2 | 1 | + 5 | Thu Nov 23 00:54:44.192608 2017 | 1 | 3 | 2 | + 5 | Thu Nov 23 07:47:09.542999 2017 | 1 | 4 | 3 | + 5 | Thu Nov 23 09:05:08.53142 2017 | 2 | 2 | 2 | + 5 | Thu Nov 23 09:17:47.706703 2017 | 2 | 5 | 3 | + 5 | Thu Nov 23 10:15:31.764558 2017 | 2 | 2 | 2 | + 5 | Thu Nov 23 14:29:02.557934 2017 | 2 | 1 | 2 | + 5 | Thu Nov 23 15:55:08.493462 2017 | 0 | 3 | 3 | + 5 | Thu Nov 23 16:28:38.455322 2017 | 2 | 5 | 4 | +(21 rows) + +-- determine if a recent user has done a given event type or not +SELECT ru.user_id, CASE WHEN et.user_id IS NULL THEN 'NO' ELSE 'YES' END as done_event + FROM recent_users ru + LEFT JOIN events_table et + ON(ru.user_id = et.user_id AND et.event_type = 6) + ORDER BY 2 DESC, 1; + user_id | done_event +---------+------------ + 1 | YES + 3 | NO + 5 | NO +(3 rows) + +-- view vs table join wrapped inside a subquery +SELECT * FROM + (SELECT ru.user_id, CASE WHEN et.user_id IS NULL THEN 'NO' ELSE 'YES' END as done_event + FROM recent_users ru + LEFT JOIN events_table et + ON(ru.user_id = et.user_id AND et.event_type = 6) + ) s1 +ORDER BY 2 DESC, 1; + user_id | done_event +---------+------------ + 1 | YES + 3 | NO + 5 | NO +(3 rows) + +-- event vs table non-partition-key join is not supported +-- given that we cannot recursively plan tables yet +SELECT * FROM + (SELECT ru.user_id, CASE WHEN et.user_id IS NULL THEN 'NO' ELSE 'YES' END as done_event + FROM recent_users ru + LEFT JOIN events_table et + ON(ru.user_id = et.event_type) + ) s1 +ORDER BY 2 DESC, 1; +ERROR: bogus varno: 3 +-- create a select only view +CREATE VIEW selected_users AS SELECT * FROM users_table WHERE value_1 >= 1 and value_1 <3; +CREATE VIEW recent_selected_users AS SELECT su.* FROM selected_users su JOIN recent_users ru USING(user_id); +SELECT user_id FROM recent_selected_users GROUP BY 1 ORDER BY 1; + user_id +--------- + 1 + 3 + 5 +(3 rows) + +-- this would be supported when we implement where partition_key in (subquery) support +SELECT et.user_id, et.time FROM events_table et WHERE et.user_id IN (SELECT user_id FROM recent_selected_users) GROUP BY 1,2 ORDER BY 1 DESC,2 DESC LIMIT 5; + user_id | time +---------+--------------------------------- + 5 | Thu Nov 23 16:11:02.929469 2017 + 5 | Thu Nov 23 14:40:40.467511 2017 + 5 | Thu Nov 23 14:28:51.833214 2017 + 5 | Thu Nov 23 14:23:09.889786 2017 + 5 | Thu Nov 23 13:26:45.571108 2017 +(5 rows) + +-- it is supported when it is a router query +SELECT count(*) FROM events_table et WHERE et.user_id IN (SELECT user_id FROM recent_selected_users WHERE user_id = 1); + count +------- + 15 +(1 row) + +-- union between views is supported through recursive planning +(SELECT user_id FROM recent_users) +UNION +(SELECT user_id FROM selected_users) +ORDER BY 1; + user_id +--------- + 1 + 2 + 3 + 4 + 5 + 6 +(6 rows) + +-- wrapping it inside a SELECT * works +SELECT * + FROM ( + (SELECT user_id FROM recent_users) + UNION + (SELECT user_id FROM selected_users) ) u + WHERE user_id < 2 AND user_id > 0 + ORDER BY user_id; + user_id +--------- + 1 +(1 row) + +-- union all also works for views +SELECT * + FROM ( + (SELECT user_id FROM recent_users) + UNION ALL + (SELECT user_id FROM selected_users) ) u + WHERE user_id < 2 AND user_id > 0 + ORDER BY user_id; + user_id +--------- + 1 + 1 +(2 rows) + +SELECT count(*) + FROM ( + (SELECT user_id FROM recent_users) + UNION + (SELECT user_id FROM selected_users) ) u + WHERE user_id < 2 AND user_id > 0; + count +------- + 1 +(1 row) + +-- UNION ALL between views is supported through recursive planning +SELECT count(*) + FROM ( + (SELECT user_id FROM recent_users) + UNION ALL + (SELECT user_id FROM selected_users) ) u + WHERE user_id < 2 AND user_id > 0; + count +------- + 2 +(1 row) + +-- expand view definitions and re-run last 2 queries +SELECT count(*) + FROM ( + (SELECT user_id FROM (SELECT user_id, max(time) as lastseen FROM users_table + GROUP BY user_id + HAVING max(time) > '2017-11-22 05:45:49.978738'::timestamp order by 2 DESC) aa + ) + UNION + (SELECT user_id FROM (SELECT * FROM users_table WHERE value_1 >= 1 and value_1 < 3) bb) ) u + WHERE user_id < 2 AND user_id > 0; + count +------- + 1 +(1 row) + +SELECT count(*) + FROM ( + (SELECT user_id FROM (SELECT user_id, max(time) as lastseen FROM users_table + GROUP BY user_id + HAVING max(time) > '2017-11-22 05:45:49.978738'::timestamp order by 2 DESC) aa + ) + UNION ALL + (SELECT user_id FROM (SELECT * FROM users_table WHERE value_1 >= 1 and value_1 < 3) bb) ) u + WHERE user_id < 2 AND user_id > 0; + count +------- + 2 +(1 row) + +-- test distinct +-- distinct is supported if it is on a partition key +CREATE VIEW distinct_user_with_value_1_3 AS SELECT DISTINCT user_id FROM users_table WHERE value_1 = 3; +SELECT * FROM distinct_user_with_value_1_3 ORDER BY user_id; + user_id +--------- + 1 + 2 + 3 + 4 + 5 + 6 +(6 rows) + +-- distinct is not supported if it is on a non-partition key +-- but will be supported via recursive planning +CREATE VIEW distinct_value_1 AS SELECT DISTINCT value_1 FROM users_table WHERE value_2 = 3; +SELECT * FROM distinct_value_1 ORDER BY 1 DESC LIMIT 5; + value_1 +--------- + 5 + 4 + 3 + 2 + 1 +(5 rows) + +-- CTEs are supported even if they are on views +CREATE VIEW cte_view_1 AS +WITH c1 AS (SELECT * FROM users_table WHERE value_1 = 3) SELECT * FROM c1 WHERE value_2 < 4; +SELECT * FROM cte_view_1 ORDER BY 1,2,3,4,5 LIMIT 5; + user_id | time | value_1 | value_2 | value_3 | value_4 +---------+---------------------------------+---------+---------+---------+--------- + 1 | Thu Nov 23 03:32:50.803031 2017 | 3 | 2 | 1 | + 2 | Thu Nov 23 13:52:54.83829 2017 | 3 | 1 | 4 | + 3 | Wed Nov 22 23:24:32.080584 2017 | 3 | 2 | 5 | + 4 | Wed Nov 22 23:59:46.493416 2017 | 3 | 1 | 3 | + 4 | Thu Nov 23 01:55:21.824618 2017 | 3 | 1 | 4 | +(5 rows) + +-- this is single shard query and still not supported since it has view + cte +-- router planner can't detect it +SELECT * FROM cte_view_1 WHERE user_id = 2 ORDER BY 1,2,3,4,5; + user_id | time | value_1 | value_2 | value_3 | value_4 +---------+--------------------------------+---------+---------+---------+--------- + 2 | Thu Nov 23 13:52:54.83829 2017 | 3 | 1 | 4 | +(1 row) + +-- if CTE itself prunes down to a single shard than the view is supported (router plannable) +CREATE VIEW cte_view_2 AS +WITH c1 AS (SELECT * FROM users_table WHERE user_id = 2) SELECT * FROM c1 WHERE value_1 = 3; +SELECT * FROM cte_view_2; + user_id | time | value_1 | value_2 | value_3 | value_4 +---------+---------------------------------+---------+---------+---------+--------- + 2 | Thu Nov 23 00:19:14.138058 2017 | 3 | 4 | 0 | + 2 | Thu Nov 23 13:52:54.83829 2017 | 3 | 1 | 4 | + 2 | Wed Nov 22 18:19:49.944985 2017 | 3 | 5 | 1 | + 2 | Thu Nov 23 11:41:04.042936 2017 | 3 | 4 | 1 | +(4 rows) + +CREATE VIEW router_view AS SELECT * FROM users_table WHERE user_id = 2; +-- router plannable +SELECT user_id FROM router_view GROUP BY 1; + user_id +--------- + 2 +(1 row) + +-- join a router view + SELECT * FROM (SELECT user_id FROM router_view GROUP BY 1) rv JOIN recent_events USING (user_id) ORDER BY 2 LIMIT 3; + user_id | time +---------+--------------------------------- + 2 | Thu Nov 23 17:26:14.563216 2017 +(1 row) + + SELECT * FROM (SELECT user_id FROM router_view GROUP BY 1) rv JOIN (SELECT * FROM recent_events) re USING (user_id) ORDER BY 2 LIMIT 3; + user_id | time +---------+--------------------------------- + 2 | Thu Nov 23 17:26:14.563216 2017 +(1 row) + +-- views with limits +CREATE VIEW recent_10_users AS + SELECT user_id, max(time) as lastseen FROM users_table + GROUP BY user_id + ORDER BY lastseen DESC + LIMIT 10; +-- this is not supported since it has limit in it and subquery_pushdown is not set +SELECT * FROM recent_10_users; + user_id | lastseen +---------+--------------------------------- + 1 | Thu Nov 23 17:30:34.635085 2017 + 3 | Thu Nov 23 17:18:51.048758 2017 + 5 | Thu Nov 23 16:48:32.08896 2017 + 4 | Thu Nov 23 15:32:02.360969 2017 + 6 | Thu Nov 23 14:43:18.024104 2017 + 2 | Thu Nov 23 13:52:54.83829 2017 +(6 rows) + +SET citus.subquery_pushdown to ON; +-- still not supported since outer query does not have limit +-- it shows a different (subquery with single relation) error message +SELECT * FROM recent_10_users; +ERROR: cannot perform distributed planning on this query +DETAIL: Subqueries with limit are not supported yet +-- now it displays more correct error message +SELECT et.* FROM recent_10_users JOIN events_table et USING(user_id); +ERROR: cannot push down this subquery +DETAIL: Limit in subquery without limit in the outermost query is unsupported +-- now both are supported when there is a limit on the outer most query +SELECT * FROM recent_10_users ORDER BY lastseen DESC LIMIT 10; + user_id | lastseen +---------+--------------------------------- + 1 | Thu Nov 23 17:30:34.635085 2017 + 3 | Thu Nov 23 17:18:51.048758 2017 + 5 | Thu Nov 23 16:48:32.08896 2017 + 4 | Thu Nov 23 15:32:02.360969 2017 + 6 | Thu Nov 23 14:43:18.024104 2017 + 2 | Thu Nov 23 13:52:54.83829 2017 +(6 rows) + +SELECT et.* FROM recent_10_users JOIN events_table et USING(user_id) ORDER BY et.time DESC LIMIT 10; + user_id | time | event_type | value_2 | value_3 | value_4 +---------+---------------------------------+------------+---------+---------+--------- + 1 | Thu Nov 23 21:54:46.924477 2017 | 6 | 4 | 5 | + 4 | Thu Nov 23 18:10:21.338399 2017 | 1 | 2 | 4 | + 3 | Thu Nov 23 18:08:26.550729 2017 | 2 | 4 | 3 | + 2 | Thu Nov 23 17:26:14.563216 2017 | 1 | 5 | 3 | + 3 | Thu Nov 23 16:44:41.903713 2017 | 4 | 2 | 2 | + 3 | Thu Nov 23 16:31:56.219594 2017 | 5 | 1 | 2 | + 4 | Thu Nov 23 16:20:33.264457 2017 | 0 | 0 | 3 | + 5 | Thu Nov 23 16:11:02.929469 2017 | 4 | 2 | 0 | + 2 | Thu Nov 23 15:58:49.273421 2017 | 5 | 1 | 2 | + 5 | Thu Nov 23 14:40:40.467511 2017 | 1 | 4 | 1 | +(10 rows) + +RESET citus.subquery_pushdown; +VACUUM ANALYZE users_table; +-- explain tests +EXPLAIN (COSTS FALSE) SELECT user_id FROM recent_selected_users GROUP BY 1 ORDER BY 1; + QUERY PLAN +--------------------------------------------------------------------------------------------------------------------------------------------------------- + Sort + Sort Key: remote_scan.user_id + -> HashAggregate + Group Key: remote_scan.user_id + -> Custom Scan (Citus Real-Time) + Task Count: 4 + Tasks Shown: One of 4 + -> Task + Node: host=localhost port=57637 dbname=regression + -> HashAggregate + Group Key: users_table.user_id + -> Hash Join + Hash Cond: (users_table.user_id = ru.user_id) + -> Seq Scan on users_table_1400000 users_table + Filter: ((value_1 >= 1) AND (value_1 < 3)) + -> Hash + -> Subquery Scan on ru + -> Sort + Sort Key: (max(users_table_1."time")) DESC + -> HashAggregate + Group Key: users_table_1.user_id + Filter: (max(users_table_1."time") > '2017-11-23 16:20:33.264457'::timestamp without time zone) + -> Seq Scan on users_table_1400000 users_table_1 +(23 rows) + +EXPLAIN (COSTS FALSE) SELECT * + FROM ( + (SELECT user_id FROM recent_users) + UNION + (SELECT user_id FROM selected_users) ) u + WHERE user_id < 4 AND user_id > 1 + ORDER BY user_id; + QUERY PLAN +------------------------------------------------------------------------------------------------------------------------------------------------- + Sort + Sort Key: remote_scan.user_id + -> Custom Scan (Citus Real-Time) + Task Count: 4 + Tasks Shown: One of 4 + -> Task + Node: host=localhost port=57637 dbname=regression + -> Unique + -> Sort + Sort Key: recent_users.user_id + -> Append + -> Subquery Scan on recent_users + -> Sort + Sort Key: (max(users_table."time")) DESC + -> GroupAggregate + Group Key: users_table.user_id + Filter: (max(users_table."time") > '2017-11-23 16:20:33.264457'::timestamp without time zone) + -> Sort + Sort Key: users_table.user_id + -> Seq Scan on users_table_1400000 users_table + Filter: ((user_id < 4) AND (user_id > 1)) + -> Seq Scan on users_table_1400000 users_table_1 + Filter: ((value_1 >= 1) AND (value_1 < 3) AND (user_id < 4) AND (user_id > 1)) +(23 rows) + +EXPLAIN (COSTS FALSE) SELECT et.* FROM recent_10_users JOIN events_table et USING(user_id) ORDER BY et.time DESC LIMIT 10; + QUERY PLAN +--------------------------------------------------------------------------------------------------------------------- + Limit + -> Sort + Sort Key: remote_scan."time" DESC + -> Custom Scan (Citus Real-Time) + -> Distributed Subplan 95_1 + -> Limit + -> Sort + Sort Key: max((max(remote_scan.lastseen))) DESC + -> HashAggregate + Group Key: remote_scan.user_id + -> Custom Scan (Citus Real-Time) + Task Count: 4 + Tasks Shown: One of 4 + -> Task + Node: host=localhost port=57637 dbname=regression + -> Limit + -> Sort + Sort Key: (max("time")) DESC + -> HashAggregate + Group Key: user_id + -> Seq Scan on users_table_1400000 users_table + Task Count: 4 + Tasks Shown: One of 4 + -> Task + Node: host=localhost port=57637 dbname=regression + -> Limit + -> Sort + Sort Key: et."time" DESC + -> Hash Join + Hash Cond: (intermediate_result.user_id = et.user_id) + -> Function Scan on read_intermediate_result intermediate_result + -> Hash + -> Seq Scan on events_table_1400004 et +(33 rows) + +SET citus.subquery_pushdown to ON; +EXPLAIN (COSTS FALSE) SELECT et.* FROM recent_10_users JOIN events_table et USING(user_id) ORDER BY et.time DESC LIMIT 10; + QUERY PLAN +--------------------------------------------------------------------------------------------------------------------- + Limit + -> Sort + Sort Key: remote_scan."time" DESC + -> Custom Scan (Citus Real-Time) + Task Count: 4 + Tasks Shown: One of 4 + -> Task + Node: host=localhost port=57637 dbname=regression + -> Limit + -> Sort + Sort Key: et."time" DESC + -> Hash Join + Hash Cond: (et.user_id = recent_10_users.user_id) + -> Seq Scan on events_table_1400004 et + -> Hash + -> Subquery Scan on recent_10_users + -> Limit + -> Sort + Sort Key: (max(users_table."time")) DESC + -> HashAggregate + Group Key: users_table.user_id + -> Seq Scan on users_table_1400000 users_table +(22 rows) + +RESET citus.subquery_pushdown; +DROP VIEW recent_10_users; +DROP VIEW router_view; +DROP VIEW cte_view_2; +DROP VIEW cte_view_1; +DROP VIEW distinct_value_1; +DROP VIEW distinct_user_with_value_1_3; +DROP VIEW recent_selected_users; +DROP VIEW selected_users; +DROP VIEW recent_events; +DROP VIEW recent_users; diff --git a/src/test/regress/expected/subqueries_not_supported.out b/src/test/regress/expected/subqueries_not_supported.out index e223d6b1f..de3a97b25 100644 --- a/src/test/regress/expected/subqueries_not_supported.out +++ b/src/test/regress/expected/subqueries_not_supported.out @@ -17,6 +17,7 @@ FROM WHERE users_table_local.user_id = evs.user_id ) as foo; ERROR: relation users_table_local is not distributed +RESET client_min_messages; -- we don't support subqueries with local tables when they are not leaf queries SELECT user_id FROM users_table WHERE user_id IN (SELECT @@ -28,6 +29,7 @@ SELECT user_id FROM users_table WHERE user_id IN ERROR: relation "events_table_local" does not exist LINE 5: users_table_local JOIN (SELECT user_id FROM events_table_... ^ +SET client_min_messages TO DEBUG1; -- we don't support aggregate distinct if the group by is not on partition key, expect for count distinct -- thus baz and bar are recursively planned but not foo SELECT diff --git a/src/test/regress/expected/subquery_in_where.out b/src/test/regress/expected/subquery_in_where.out index 93c10f97c..337a0d7dc 100644 --- a/src/test/regress/expected/subquery_in_where.out +++ b/src/test/regress/expected/subquery_in_where.out @@ -684,8 +684,7 @@ DEBUG: Plan 69 query after replacing subqueries and CTEs: SELECT count(*) AS co 10 (1 row) -DROP TABLE local_table; -DEBUG: EventTriggerInvoke 19977 SET client_min_messages TO DEFAULT; +DROP TABLE local_table; DROP SCHEMA subquery_in_where CASCADE; SET search_path TO public; diff --git a/src/test/regress/multi_schedule b/src/test/regress/multi_schedule index 25afd63df..711cff155 100644 --- a/src/test/regress/multi_schedule +++ b/src/test/regress/multi_schedule @@ -99,7 +99,8 @@ test: multi_large_table_join_planning multi_large_table_pruning multi_large_tabl # --------- test: with_nested with_where with_basics with_prepare with_set_operations -test: with_modifying with_executors with_join with_partitioning +test: with_modifying +test: with_executors with_join with_partitioning # ---------- # Tests to check our large record loading and shard deletion behavior diff --git a/src/test/regress/sql/multi_join_order_additional.sql b/src/test/regress/sql/multi_join_order_additional.sql index 0cf87ed7b..d5cedf2be 100644 --- a/src/test/regress/sql/multi_join_order_additional.sql +++ b/src/test/regress/sql/multi_join_order_additional.sql @@ -10,9 +10,9 @@ SET citus.next_shard_id TO 650000; SET citus.explain_distributed_queries TO off; SET citus.log_multi_join_order TO TRUE; SET citus.task_executor_type = 'task-tracker'; -- can't explain all queries otherwise -SET client_min_messages TO DEBUG2; SET citus.shard_count to 2; SET citus.shard_replication_factor to 1; +RESET client_min_messages; -- Create new table definitions for use in testing in distributed planning and -- execution functionality. Also create indexes to boost performance. @@ -63,6 +63,7 @@ CREATE TABLE customer_hash ( c_comment varchar(117) not null); SELECT create_distributed_table('customer_hash', 'c_custkey'); +SET client_min_messages TO DEBUG2; -- The following query checks that we can correctly handle self-joins EXPLAIN SELECT l1.l_quantity FROM lineitem l1, lineitem l2 diff --git a/src/test/regress/sql/multi_multiuser.sql b/src/test/regress/sql/multi_multiuser.sql index c2137febd..9ce6c628b 100644 --- a/src/test/regress/sql/multi_multiuser.sql +++ b/src/test/regress/sql/multi_multiuser.sql @@ -4,6 +4,10 @@ -- Test user permissions. -- +-- print whether we're using version > 10 to make version-specific tests clear +SHOW server_version \gset +SELECT substring(:'server_version', '\d+')::int > 10 AS version_above_ten; + SET citus.next_shard_id TO 1420000; ALTER SEQUENCE pg_catalog.pg_dist_jobid_seq RESTART 1420000; diff --git a/src/test/regress/sql/multi_mx_ddl.sql b/src/test/regress/sql/multi_mx_ddl.sql index 4a6e803d5..93afbbf0c 100644 --- a/src/test/regress/sql/multi_mx_ddl.sql +++ b/src/test/regress/sql/multi_mx_ddl.sql @@ -91,12 +91,8 @@ SELECT "Column", "Type", "Modifiers" FROM table_desc WHERE relid='mx_ddl_table_1 -- Show that DDL commands are done within a two-phase commit transaction \c - - - :master_port -SET client_min_messages TO debug2; - CREATE INDEX ddl_test_index ON mx_ddl_table(value); -RESET client_min_messages; - DROP INDEX ddl_test_index; -- show that sequences owned by mx tables result in unique values diff --git a/src/test/regress/sql/multi_null_minmax_value_pruning.sql b/src/test/regress/sql/multi_null_minmax_value_pruning.sql index 156cd6c26..3b01a36f5 100644 --- a/src/test/regress/sql/multi_null_minmax_value_pruning.sql +++ b/src/test/regress/sql/multi_null_minmax_value_pruning.sql @@ -8,9 +8,9 @@ SET citus.next_shard_id TO 760000; --- print whether we're using version > 9 to make version-specific tests clear +-- print major version number for version-specific tests SHOW server_version \gset -SELECT substring(:'server_version', '\d+')::int > 9 AS version_above_nine; +SELECT substring(:'server_version', '\d+')::int AS server_version; SET client_min_messages TO DEBUG2; SET citus.explain_all_tasks TO on; diff --git a/src/test/regress/sql/multi_orderby_limit_pushdown.sql b/src/test/regress/sql/multi_orderby_limit_pushdown.sql index 0381511d5..e30661863 100644 --- a/src/test/regress/sql/multi_orderby_limit_pushdown.sql +++ b/src/test/regress/sql/multi_orderby_limit_pushdown.sql @@ -109,7 +109,7 @@ GROUP BY user_id ORDER BY (10000 / (sum(value_1 + value_2))) DESC LIMIT 2; -EXPLAIN +EXPLAIN (COSTS OFF) SELECT user_id FROM users_table GROUP BY user_id @@ -139,7 +139,7 @@ GROUP BY user_id ORDER BY sum(value_1) DESC LIMIT 2; -EXPLAIN +EXPLAIN (COSTS OFF) SELECT user_id FROM users_table GROUP BY user_id @@ -153,7 +153,7 @@ GROUP BY ut.user_id ORDER BY MAX(et.time), AVG(ut.value_1) LIMIT 5; -EXPLAIN +EXPLAIN (COSTS OFF) SELECT ut.user_id, avg(ut.value_2) FROM users_table ut, events_table et WHERE ut.user_id = et.user_id and et.value_2 < 5 @@ -176,7 +176,7 @@ GROUP BY ut.user_id ORDER BY 2, AVG(ut.value_1), 1 DESC LIMIT 2; -EXPLAIN +EXPLAIN (COSTS OFF) SELECT ut.user_id, count(DISTINCT ut.value_2) FROM users_table ut, events_table et WHERE ut.user_id = et.user_id and et.value_2 < 5 diff --git a/src/test/regress/sql/multi_partitioning.sql b/src/test/regress/sql/multi_partitioning.sql index 04ddf9fd5..48d35ffc4 100644 --- a/src/test/regress/sql/multi_partitioning.sql +++ b/src/test/regress/sql/multi_partitioning.sql @@ -6,6 +6,10 @@ SET citus.next_shard_id TO 1660000; SET citus.shard_count TO 4; SET citus.shard_replication_factor TO 1; +-- print major version number for version-specific tests +SHOW server_version \gset +SELECT substring(:'server_version', '\d+')::int AS server_version; + -- -- Distributed Partitioned Table Creation Tests -- @@ -257,6 +261,7 @@ SELECT master_modify_multiple_shards('UPDATE partitioning_test_2009 SET time = t -- test CREATE INDEX -- CREATE INDEX on partitioned table - this will error out +-- on earlier versions of postgres earlier than 11. CREATE INDEX partitioning_index ON partitioning_test(id); -- CREATE INDEX on partition diff --git a/src/test/regress/sql/multi_partitioning_utils.sql b/src/test/regress/sql/multi_partitioning_utils.sql index faf3f8f7d..df19ff569 100644 --- a/src/test/regress/sql/multi_partitioning_utils.sql +++ b/src/test/regress/sql/multi_partitioning_utils.sql @@ -1,3 +1,7 @@ +-- This test has different output per major version +SHOW server_version \gset +SELECT substring(:'server_version', '\d+')::int as server_major_version; + -- =================================================================== -- create test functions -- =================================================================== diff --git a/src/test/regress/sql/multi_subtransactions.sql b/src/test/regress/sql/multi_subtransactions.sql index 0998cb014..54f553192 100644 --- a/src/test/regress/sql/multi_subtransactions.sql +++ b/src/test/regress/sql/multi_subtransactions.sql @@ -1,3 +1,7 @@ +-- print whether we're using version > 10 to make version-specific tests clear +SHOW server_version \gset +SELECT substring(:'server_version', '\d+')::int > 10 AS version_above_nine; + CREATE TABLE artists ( id bigint NOT NULL, diff --git a/src/test/regress/sql/multi_utilities.sql b/src/test/regress/sql/multi_utilities.sql index f3558c513..2069bd6ca 100644 --- a/src/test/regress/sql/multi_utilities.sql +++ b/src/test/regress/sql/multi_utilities.sql @@ -1,6 +1,8 @@ SET citus.next_shard_id TO 990000; - +-- print server version > 10 to make version-specific tests clear +SHOW server_version \gset +SELECT substring(:'server_version', '\d+')::int > 10 as version_above_ten; -- =================================================================== -- test utility statement functionality @@ -107,6 +109,10 @@ SELECT master_create_worker_shards('dustbunnies', 1, 2); 4,roger \. +CREATE TABLE second_dustbunnies(id integer, name text, age integer); +SELECT master_create_distributed_table('second_dustbunnies', 'id', 'hash'); +SELECT master_create_worker_shards('second_dustbunnies', 1, 2); + -- following approach adapted from PostgreSQL's stats.sql file -- save relevant stat counter values in refreshable view @@ -255,6 +261,9 @@ WHERE tablename = 'dustbunnies_990002' ORDER BY attname; -- verify warning for unqualified VACUUM VACUUM; +-- check for multiple table vacuum +VACUUM dustbunnies, second_dustbunnies; + -- check the current number of vacuum and analyze run on dustbunnies SELECT run_command_on_workers($$SELECT wait_for_stats()$$); SELECT run_command_on_workers($$SELECT pg_stat_get_vacuum_count(tablename::regclass) from pg_tables where tablename LIKE 'dustbunnies_%' limit 1$$); diff --git a/src/test/regress/sql/multi_view.sql b/src/test/regress/sql/multi_view.sql index cd89cdd83..5f2f2ef1f 100644 --- a/src/test/regress/sql/multi_view.sql +++ b/src/test/regress/sql/multi_view.sql @@ -7,6 +7,10 @@ -- router queries, single row inserts, multi row inserts via insert -- into select, multi row insert via copy commands. +-- print whether we're using version > 10 to make version-specific tests clear +SHOW server_version \gset +SELECT substring(:'server_version', '\d+')::int > 10 AS version_above_ten; + SELECT count(*) FROM lineitem_hash_part; SELECT count(*) FROM orders_hash_part; diff --git a/src/test/regress/sql/subqueries_not_supported.sql b/src/test/regress/sql/subqueries_not_supported.sql index f6d868f2b..efd5c351f 100644 --- a/src/test/regress/sql/subqueries_not_supported.sql +++ b/src/test/regress/sql/subqueries_not_supported.sql @@ -20,6 +20,7 @@ FROM WHERE users_table_local.user_id = evs.user_id ) as foo; +RESET client_min_messages; -- we don't support subqueries with local tables when they are not leaf queries SELECT user_id FROM users_table WHERE user_id IN (SELECT @@ -29,6 +30,8 @@ SELECT user_id FROM users_table WHERE user_id IN USING (user_id) ); +SET client_min_messages TO DEBUG1; + -- we don't support aggregate distinct if the group by is not on partition key, expect for count distinct -- thus baz and bar are recursively planned but not foo SELECT diff --git a/src/test/regress/sql/subquery_in_where.sql b/src/test/regress/sql/subquery_in_where.sql index f4b12f1f8..afcf6fccc 100644 --- a/src/test/regress/sql/subquery_in_where.sql +++ b/src/test/regress/sql/subquery_in_where.sql @@ -509,9 +509,8 @@ IN FROM local_table); -DROP TABLE local_table; - SET client_min_messages TO DEFAULT; +DROP TABLE local_table; DROP SCHEMA subquery_in_where CASCADE; SET search_path TO public;