PG11 compatibility update

- changes in ruleutils_11.c is reflected
- vacuum statement api change is handled. We now allow
  multi-table vacuum commands.
- some other function header changes are reflected
- api conflicts between PG11 and earlier versions
  are handled by adding shims in version_compat.h
- various regression tests are fixed due output and
  functionality in PG1
- no change is made to support new features in PG11
  they need to be handled by new commit
pull/2118/head
Murat Tuncer 2018-03-30 15:38:35 +03:00
parent 49255213d4
commit a6fe5ca183
64 changed files with 6943 additions and 902 deletions

View File

@ -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);
}

View File

@ -799,13 +799,14 @@ OpenCopyConnections(CopyStmt *copyStatement, ShardConnections *shardConnections,
List *connectionList = NULL;
int64 shardId = shardConnections->shardId;
bool raiseInterrupts = true;
MemoryContext localContext = AllocSetContextCreate(CurrentMemoryContext,
MemoryContext localContext =
AllocSetContextCreateExtended(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);
@ -1279,11 +1280,27 @@ ConversionPathForTypes(Oid inputType, Oid destType, CopyCoercionData *result)
}
case COERCION_PATH_ARRAYCOERCE:
{
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:
{
result->coercionType = COERCION_PATH_COERCEVIAIO;

View File

@ -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,7 +59,8 @@ InitializeConnectionManagement(void)
* management. Doing so, instead of allocating in TopMemoryContext, makes
* it easier to associate used memory.
*/
ConnectionContext = AllocSetContextCreate(TopMemoryContext, "Connection Context",
ConnectionContext = AllocSetContextCreateExtended(TopMemoryContext,
"Connection Context",
ALLOCSET_DEFAULT_MINSIZE,
ALLOCSET_DEFAULT_INITSIZE,
ALLOCSET_DEFAULT_MAXSIZE);

View File

@ -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, &parameterTypes,
&parameterValues);
@ -1381,11 +1385,12 @@ StoreQueryResult(CitusScanState *scanState, MultiConnection *connection,
bool randomAccess = true;
bool interTransactions = false;
bool commandFailed = false;
MemoryContext ioContext = AllocSetContextCreate(CurrentMemoryContext,
MemoryContext ioContext = AllocSetContextCreateExtended(CurrentMemoryContext,
"StoreQueryResult",
ALLOCSET_DEFAULT_MINSIZE,
ALLOCSET_DEFAULT_INITSIZE,
ALLOCSET_DEFAULT_MAXSIZE);
*rows = 0;
if (scanState->tuplestorestate == NULL)

View File

@ -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,
relationId = RangeVarGetRelid(alterObjectSchemaStmt->relation,
AccessExclusiveLock,
alterObjectSchemaStmt->missing_ok,
noWait, NULL, NULL);
alterObjectSchemaStmt->missing_ok);
/* first check whether a distributed relation is affected */
if (!OidIsValid(relationId) || !IsDistributedTable(relationId))
@ -1609,37 +1608,45 @@ 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);
/* execute vacuum on distributed tables */
foreach(relationIdCell, relationIdList)
{
Oid relationId = lfirst_oid(relationIdCell);
if (IsDistributedTable(relationId))
{
List *vacuumColumnList = NIL;
List *taskList = NIL;
/*
* 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.
* commands can run inside a transaction block. Notice that we do this
* once even if there are multiple distributed tables to be vacuumed.
*/
if ((vacuumStmt->options & VACOPT_VACUUM) != 0)
if (executedVacuumCount == 0 && (vacuumStmt->options & VACOPT_VACUUM) != 0)
{
/* save old commit protocol to restore at xact end */
Assert(SavedMultiShardCommitProtocol == COMMIT_PROTOCOL_BARE);
@ -1647,35 +1654,61 @@ ProcessVacuumStmt(VacuumStmt *vacuumStmt, const char *vacuumCommand)
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;
}

View File

@ -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));
}
}

View File

@ -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"

View File

@ -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);

View File

@ -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),

View File

@ -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;
}

View File

@ -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)
{

View File

@ -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

View File

@ -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);
}

View File

@ -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);

View File

@ -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;

View File

@ -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,
localContext = AllocSetContextCreateExtended(CurrentMemoryContext,
"RecoverWorkerTransactions",
ALLOCSET_DEFAULT_MINSIZE,
ALLOCSET_DEFAULT_INITSIZE,
ALLOCSET_DEFAULT_MAXSIZE);
oldContext = MemoryContextSwitchTo(localContext);
/* take table lock first to avoid running concurrently */

View File

@ -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);

View File

@ -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");

View File

@ -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"

View File

@ -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.

View File

@ -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();

View File

@ -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,7 +146,7 @@ TaskTrackerMain(Datum main_arg)
* that we can reset the context during error recovery and thereby avoid
* possible memory leaks.
*/
TaskTrackerContext = AllocSetContextCreate(TopMemoryContext, "Task Tracker",
TaskTrackerContext = AllocSetContextCreateExtended(TopMemoryContext, "Task Tracker",
ALLOCSET_DEFAULT_MINSIZE,
ALLOCSET_DEFAULT_INITSIZE,
ALLOCSET_DEFAULT_MAXSIZE);

View File

@ -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;

View File

@ -1000,7 +1000,7 @@ 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,
rowOutputState->rowcontext = AllocSetContextCreateExtended(CurrentMemoryContext,
"WorkerRowOutputContext",
ALLOCSET_DEFAULT_MINSIZE,
ALLOCSET_DEFAULT_INITSIZE,

View File

@ -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 */

View File

@ -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;

View File

@ -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; <waiting ...>
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; <waiting ...>
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; <waiting ...>
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; <waiting ...>
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; <waiting ...>
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'); <waiting ...>
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'); <waiting ...>
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; <waiting ...>
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; <waiting ...>
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; <waiting ...>
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; <waiting ...>
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; <waiting ...>
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; <waiting ...>
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; <waiting ...>
step s1-commit: COMMIT;
step s2-copy: <... completed>
step s1-select-count: SELECT COUNT(*) FROM partitioned_copy;
count
15

View File

@ -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 ...
^

File diff suppressed because it is too large Load Diff

View File

@ -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;

View File

@ -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;

View File

@ -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;

View File

@ -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;

View File

@ -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;

View File

@ -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;
@ -70,7 +70,7 @@ 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
@ -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

View File

@ -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;

View File

@ -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)
------------------------------------------------------------------------------------------------------------------------------
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)
----------------------------------------------------------------------------------------------------------
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

View File

@ -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,8 +410,8 @@ 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
@ -411,10 +419,15 @@ 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
------------------------+-------------------------
------------------------+-------------------------------
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');

File diff suppressed because it is too large Load Diff

File diff suppressed because it is too large Load Diff

View File

@ -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

View File

@ -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
-----------------------------------------------
------------------------------------------------------------------------------------------------------------------------------------------------------
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
-----------------------------------------------
------------------------------------------------------------------------------------------------------------------------------------------------------
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...
^
-- 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
-----------------------------------------------
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');
SELECT generate_alter_table_attach_partition_command('multi_column_partition_1');
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
-----------------------------------------------
--------------------------------------------------------------------------------------------------------------------------------------------------------------------
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
-----------------------------------------------
---------------------------------------------------------------------------------------------------------
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

View File

@ -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

View File

@ -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

View File

@ -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;

View File

@ -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$$);

View File

@ -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;

View File

@ -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
-------
@ -790,7 +798,7 @@ 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
@ -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)
-> Seq Scan on users_table_1400000 users_table
Filter: ((value_1 >= 1) AND (value_1 < 3))
-> Hash
-> Subquery Scan on ru
-> 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
(23 rows)
-> Seq Scan on users_table_1400000 users_table
Filter: ((value_1 >= 1) AND (value_1 < 3))
(21 rows)
EXPLAIN (COSTS FALSE) SELECT *
FROM (

View File

@ -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;

View File

@ -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

View File

@ -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;

View File

@ -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

View File

@ -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

View File

@ -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;

View File

@ -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

View File

@ -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;

View File

@ -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

View File

@ -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

View File

@ -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
-- ===================================================================

View File

@ -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,

View File

@ -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$$);

View File

@ -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;

View File

@ -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

View File

@ -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;