Merge branch 'master' into fix-dropping-mat-views-when-alter-table

pull/4584/head
Ahmet Gedemenli 2021-01-27 16:33:10 +03:00 committed by GitHub
commit b2c1bbddd4
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
37 changed files with 682 additions and 234 deletions

View File

@ -162,6 +162,12 @@ typedef struct TableConversionState
* ALTER_TABLE_SET_ACCESS_METHOD -> AlterTableSetAccessMethod
*/
TableConversionFunction function;
/*
* suppressNoticeMessages determines if we want to suppress NOTICE
* messages that we explicitly issue
*/
bool suppressNoticeMessages;
} TableConversionState;
@ -177,7 +183,8 @@ static TableConversionState * CreateTableConversion(TableConversionParameters *p
static void CreateDistributedTableLike(TableConversionState *con);
static void CreateCitusTableLike(TableConversionState *con);
static List * GetViewCreationCommandsOfTable(Oid relationId);
static void ReplaceTable(Oid sourceId, Oid targetId, List *justBeforeDropCommands);
static void ReplaceTable(Oid sourceId, Oid targetId, List *justBeforeDropCommands,
bool suppressNoticeMessages);
static void CheckAlterDistributedTableConversionParameters(TableConversionState *con);
static char * CreateWorkerChangeSequenceDependencyCommand(char *sequenceSchemaName,
char *sequenceName,
@ -522,15 +529,18 @@ ConvertTable(TableConversionState *con)
bool includeIndexes = true;
if (con->accessMethod && strcmp(con->accessMethod, "columnar") == 0)
{
List *explicitIndexesOnTable = GetExplicitIndexOidList(con->relationId);
Oid indexOid = InvalidOid;
foreach_oid(indexOid, explicitIndexesOnTable)
if (!con->suppressNoticeMessages)
{
ereport(NOTICE, (errmsg("the index %s on table %s will be dropped, "
"because columnar tables cannot have indexes",
get_rel_name(indexOid),
quote_qualified_identifier(con->schemaName,
con->relationName))));
List *explicitIndexesOnTable = GetExplicitIndexOidList(con->relationId);
Oid indexOid = InvalidOid;
foreach_oid(indexOid, explicitIndexesOnTable)
{
ereport(NOTICE, (errmsg("the index %s on table %s will be dropped, "
"because columnar tables cannot have indexes",
get_rel_name(indexOid),
quote_qualified_identifier(con->schemaName,
con->relationName))));
}
}
includeIndexes = false;
@ -580,9 +590,12 @@ ConvertTable(TableConversionState *con)
if (PartitionedTable(con->relationId))
{
ereport(NOTICE, (errmsg("converting the partitions of %s",
quote_qualified_identifier(con->schemaName,
con->relationName))));
if (!con->suppressNoticeMessages)
{
ereport(NOTICE, (errmsg("converting the partitions of %s",
quote_qualified_identifier(con->schemaName,
con->relationName))));
}
List *partitionList = PartitionList(con->relationId);
@ -617,6 +630,7 @@ ConvertTable(TableConversionState *con)
.shardCount = con->shardCount,
.cascadeToColocated = cascadeOption,
.colocateWith = con->colocateWith,
.suppressNoticeMessages = con->suppressNoticeMessages,
/*
* Even if we called UndistributeTable with cascade option, we
@ -636,9 +650,12 @@ ConvertTable(TableConversionState *con)
}
}
ereport(NOTICE, (errmsg("creating a new table for %s",
quote_qualified_identifier(con->schemaName,
con->relationName))));
if (!con->suppressNoticeMessages)
{
ereport(NOTICE, (errmsg("creating a new table for %s",
quote_qualified_identifier(con->schemaName,
con->relationName))));
}
TableDDLCommand *tableCreationCommand = NULL;
foreach_ptr(tableCreationCommand, preLoadCommands)
@ -687,7 +704,8 @@ ConvertTable(TableConversionState *con)
CreateCitusTableLike(con);
}
ReplaceTable(con->relationId, con->newRelationId, justBeforeDropCommands);
ReplaceTable(con->relationId, con->newRelationId, justBeforeDropCommands,
con->suppressNoticeMessages);
TableDDLCommand *tableConstructionCommand = NULL;
foreach_ptr(tableConstructionCommand, postLoadCommands)
@ -732,7 +750,8 @@ ConvertTable(TableConversionState *con)
.shardCountIsNull = con->shardCountIsNull,
.shardCount = con->shardCount,
.colocateWith = qualifiedRelationName,
.cascadeToColocated = CASCADE_TO_COLOCATED_NO_ALREADY_CASCADED
.cascadeToColocated = CASCADE_TO_COLOCATED_NO_ALREADY_CASCADED,
.suppressNoticeMessages = con->suppressNoticeMessages
};
TableConversionReturn *colocatedReturn = con->function(&cascadeParam);
foreignKeyCommands = list_concat(foreignKeyCommands,
@ -875,6 +894,7 @@ CreateTableConversion(TableConversionParameters *params)
con->accessMethod = params->accessMethod;
con->cascadeToColocated = params->cascadeToColocated;
con->cascadeViaForeignKeys = params->cascadeViaForeignKeys;
con->suppressNoticeMessages = params->suppressNoticeMessages;
Relation relation = try_relation_open(con->relationId, ExclusiveLock);
if (relation == NULL)
@ -1079,7 +1099,8 @@ GetViewCreationCommandsOfTable(Oid relationId)
* Source and target tables need to be in the same schema and have the same columns.
*/
void
ReplaceTable(Oid sourceId, Oid targetId, List *justBeforeDropCommands)
ReplaceTable(Oid sourceId, Oid targetId, List *justBeforeDropCommands,
bool suppressNoticeMessages)
{
char *sourceName = get_rel_name(sourceId);
char *targetName = get_rel_name(targetId);
@ -1090,8 +1111,11 @@ ReplaceTable(Oid sourceId, Oid targetId, List *justBeforeDropCommands)
if (!PartitionedTable(sourceId))
{
ereport(NOTICE, (errmsg("Moving the data of %s",
quote_qualified_identifier(schemaName, sourceName))));
if (!suppressNoticeMessages)
{
ereport(NOTICE, (errmsg("Moving the data of %s",
quote_qualified_identifier(schemaName, sourceName))));
}
appendStringInfo(query, "INSERT INTO %s SELECT * FROM %s",
quote_qualified_identifier(schemaName, targetName),
@ -1129,16 +1153,22 @@ ReplaceTable(Oid sourceId, Oid targetId, List *justBeforeDropCommands)
ExecuteQueryViaSPI(justBeforeDropCommand, SPI_OK_UTILITY);
}
ereport(NOTICE, (errmsg("Dropping the old %s",
quote_qualified_identifier(schemaName, sourceName))));
if (!suppressNoticeMessages)
{
ereport(NOTICE, (errmsg("Dropping the old %s",
quote_qualified_identifier(schemaName, sourceName))));
}
resetStringInfo(query);
appendStringInfo(query, "DROP TABLE %s CASCADE",
quote_qualified_identifier(schemaName, sourceName));
ExecuteQueryViaSPI(query->data, SPI_OK_UTILITY);
ereport(NOTICE, (errmsg("Renaming the new table to %s",
quote_qualified_identifier(schemaName, sourceName))));
if (!suppressNoticeMessages)
{
ereport(NOTICE, (errmsg("Renaming the new table to %s",
quote_qualified_identifier(schemaName, sourceName))));
}
resetStringInfo(query);
appendStringInfo(query, "ALTER TABLE %s RENAME TO %s",
@ -1312,23 +1342,26 @@ CheckAlterDistributedTableConversionParameters(TableConversionState *con)
}
}
/* Notices for no operation UDF calls */
if (sameDistColumn)
if (!con->suppressNoticeMessages)
{
ereport(NOTICE, (errmsg("table is already distributed by %s",
con->distributionColumn)));
}
/* Notices for no operation UDF calls */
if (sameDistColumn)
{
ereport(NOTICE, (errmsg("table is already distributed by %s",
con->distributionColumn)));
}
if (sameShardCount)
{
ereport(NOTICE, (errmsg("shard count of the table is already %d",
con->shardCount)));
}
if (sameShardCount)
{
ereport(NOTICE, (errmsg("shard count of the table is already %d",
con->shardCount)));
}
if (sameColocateWith)
{
ereport(NOTICE, (errmsg("table is already colocated with %s",
con->colocateWith)));
if (sameColocateWith)
{
ereport(NOTICE, (errmsg("table is already colocated with %s",
con->colocateWith)));
}
}
}

View File

@ -23,6 +23,7 @@
#include "distributed/commands.h"
#include "distributed/foreign_key_relationship.h"
#include "distributed/listutils.h"
#include "distributed/local_executor.h"
#include "distributed/multi_executor.h"
#include "distributed/multi_partitioning_utils.h"
#include "distributed/reference_table_utils.h"
@ -292,7 +293,7 @@ DropRelationForeignKeys(Oid relationId, int fKeyFlags)
SetLocalEnableLocalReferenceForeignKeys(false);
List *dropFkeyCascadeCommandList = GetRelationDropFkeyCommands(relationId, fKeyFlags);
ExecuteAndLogDDLCommandList(dropFkeyCascadeCommandList);
ExecuteAndLogUtilityCommandList(dropFkeyCascadeCommandList);
SetLocalEnableLocalReferenceForeignKeys(oldEnableLocalReferenceForeignKeys);
}
@ -428,32 +429,30 @@ ExecuteCascadeOperationForRelationIdList(List *relationIdList,
/*
* ExecuteAndLogDDLCommandList takes a list of ddl commands and calls
* ExecuteAndLogDDLCommand function for each of them.
* ExecuteAndLogUtilityCommandList takes a list of utility commands and calls
* ExecuteAndLogUtilityCommand function for each of them.
*/
void
ExecuteAndLogDDLCommandList(List *ddlCommandList)
ExecuteAndLogUtilityCommandList(List *utilityCommandList)
{
char *ddlCommand = NULL;
foreach_ptr(ddlCommand, ddlCommandList)
char *utilityCommand = NULL;
foreach_ptr(utilityCommand, utilityCommandList)
{
ExecuteAndLogDDLCommand(ddlCommand);
ExecuteAndLogUtilityCommand(utilityCommand);
}
}
/*
* ExecuteAndLogDDLCommand takes a ddl command and logs it in DEBUG4 log level.
* ExecuteAndLogUtilityCommand takes a utility command and logs it in DEBUG4 log level.
* Then, parses and executes it via CitusProcessUtility.
*/
void
ExecuteAndLogDDLCommand(const char *commandString)
ExecuteAndLogUtilityCommand(const char *commandString)
{
ereport(DEBUG4, (errmsg("executing \"%s\"", commandString)));
Node *parseTree = ParseTreeNode(commandString);
ProcessUtilityParseTree(parseTree, commandString, PROCESS_UTILITY_TOPLEVEL,
NULL, None_Receiver, NULL);
ExecuteUtilityCommand(commandString);
}

View File

@ -92,13 +92,12 @@ create_citus_local_table(PG_FUNCTION_ARGS)
* not chained with any reference tables back to postgres tables.
* So give a warning to user for that.
*/
ereport(WARNING, (errmsg("citus local tables that are not chained with "
"reference tables via foreign keys might be "
"automatically converted back to postgres tables"),
ereport(WARNING, (errmsg("local tables that are added to metadata but not "
"chained with reference tables via foreign keys might "
"be automatically converted back to postgres tables"),
errhint("Consider setting "
"citus.enable_local_reference_table_foreign_keys "
"to 'off' to disable automatically undistributing "
"citus local tables")));
"to 'off' to disable this behavior")));
}
Oid relationId = PG_GETARG_OID(0);
@ -154,7 +153,7 @@ CreateCitusLocalTable(Oid relationId, bool cascadeViaForeignKeys)
/*
* Creating Citus local tables relies on functions that accesses
* shards locally (e.g., ExecuteAndLogDDLCommand()). As long as
* shards locally (e.g., ExecuteAndLogUtilityCommand()). As long as
* we don't teach those functions to access shards remotely, we
* cannot relax this check.
*/
@ -209,9 +208,9 @@ CreateCitusLocalTable(Oid relationId, bool cascadeViaForeignKeys)
ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
errmsg("relation %s is involved in a foreign key "
"relationship with another table", qualifiedRelationName),
errhint("Use cascade_via_foreign_keys option to convert "
errhint("Use cascade_via_foreign_keys option to add "
"all the relations involved in a foreign key "
"relationship with %s to a citus local table by "
"relationship with %s to citus metadata by "
"executing SELECT create_citus_local_table($$%s$$, "
"cascade_via_foreign_keys=>true)",
qualifiedRelationName, qualifiedRelationName)));
@ -246,7 +245,7 @@ CreateCitusLocalTable(Oid relationId, bool cascadeViaForeignKeys)
* from scratch, below we simply recreate the shell table executing them
* via process utility.
*/
ExecuteAndLogDDLCommandList(shellTableDDLEvents);
ExecuteAndLogUtilityCommandList(shellTableDDLEvents);
/*
* Set shellRelationId as the relation with relationId now points
@ -280,7 +279,7 @@ ErrorIfUnsupportedCreateCitusLocalTable(Relation relation)
{
if (!RelationIsValid(relation))
{
ereport(ERROR, (errmsg("cannot create citus local table, relation does "
ereport(ERROR, (errmsg("cannot add local table to metadata, relation does "
"not exist")));
}
@ -319,16 +318,16 @@ ErrorIfUnsupportedCitusLocalTableKind(Oid relationId)
if (IsChildTable(relationId) || IsParentTable(relationId))
{
ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
errmsg("cannot create citus local table \"%s\", citus local "
"tables cannot be involved in inheritance relationships",
relationName)));
errmsg("cannot add local table \"%s\" to metadata, local tables "
"added to metadata cannot be involved in inheritance "
"relationships", relationName)));
}
if (PartitionTable(relationId))
{
ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
errmsg("cannot create citus local table \"%s\", citus local "
"tables cannot be partition of other tables",
errmsg("cannot add local table \"%s\" to metadata, local tables "
"added to metadata cannot be partition of other tables ",
relationName)));
}
@ -336,9 +335,9 @@ ErrorIfUnsupportedCitusLocalTableKind(Oid relationId)
if (!(relationKind == RELKIND_RELATION || relationKind == RELKIND_FOREIGN_TABLE))
{
ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
errmsg("cannot create citus local table \"%s\", only regular "
"tables and foreign tables are supported for citus local "
"table creation", relationName)));
errmsg("cannot add local table \"%s\" to metadata, only regular "
"tables and foreign tables can be added to citus metadata ",
relationName)));
}
}
@ -436,7 +435,7 @@ RenameRelationToShardRelation(Oid shellRelationId, uint64 shardId)
appendStringInfo(renameCommand, "ALTER TABLE %s RENAME TO %s;",
qualifiedShellRelationName, quotedShardRelationName);
ExecuteAndLogDDLCommand(renameCommand->data);
ExecuteAndLogUtilityCommand(renameCommand->data);
}
@ -456,7 +455,7 @@ RenameShardRelationConstraints(Oid shardRelationId, uint64 shardId)
{
const char *commandString =
GetRenameShardConstraintCommand(shardRelationId, constraintName, shardId);
ExecuteAndLogDDLCommand(commandString);
ExecuteAndLogUtilityCommand(commandString);
}
}
@ -550,7 +549,7 @@ RenameShardRelationIndexes(Oid shardRelationId, uint64 shardId)
foreach_oid(indexOid, indexOidList)
{
const char *commandString = GetRenameShardIndexCommand(indexOid, shardId);
ExecuteAndLogDDLCommand(commandString);
ExecuteAndLogUtilityCommand(commandString);
}
}
@ -591,7 +590,7 @@ RenameShardRelationStatistics(Oid shardRelationId, uint64 shardId)
char *command = NULL;
foreach_ptr(command, statsCommandList)
{
ExecuteAndLogDDLCommand(command);
ExecuteAndLogUtilityCommand(command);
}
}
@ -636,7 +635,7 @@ RenameShardRelationNonTruncateTriggers(Oid shardRelationId, uint64 shardId)
char *triggerName = NameStr(triggerForm->tgname);
char *commandString =
GetRenameShardTriggerCommand(shardRelationId, triggerName, shardId);
ExecuteAndLogDDLCommand(commandString);
ExecuteAndLogUtilityCommand(commandString);
}
heap_freetuple(triggerTuple);
@ -688,7 +687,7 @@ DropRelationTruncateTriggers(Oid relationId)
{
char *triggerName = NameStr(triggerForm->tgname);
char *commandString = GetDropTriggerCommand(relationId, triggerName);
ExecuteAndLogDDLCommand(commandString);
ExecuteAndLogUtilityCommand(commandString);
}
heap_freetuple(triggerTuple);
@ -868,7 +867,7 @@ DropDefaultColumnDefinition(Oid relationId, char *columnName)
"ALTER TABLE %s ALTER COLUMN %s DROP DEFAULT",
qualifiedRelationName, quotedColumnName);
ExecuteAndLogDDLCommand(sequenceDropCommand->data);
ExecuteAndLogUtilityCommand(sequenceDropCommand->data);
}
@ -891,7 +890,7 @@ TransferSequenceOwnership(Oid sequenceId, Oid targetRelationId, char *targetColu
qualifiedSequenceName, qualifiedTargetRelationName,
quotedTargetColumnName);
ExecuteAndLogDDLCommand(sequenceOwnershipCommand->data);
ExecuteAndLogUtilityCommand(sequenceOwnershipCommand->data);
}

View File

@ -558,9 +558,11 @@ DropFKeysAndUndistributeTable(Oid relationId)
char *relationName = get_rel_name(relationId);
Oid schemaId = get_rel_namespace(relationId);
/* suppress notices messages not to be too verbose */
TableConversionParameters params = {
.relationId = relationId,
.cascadeViaForeignKeys = false
.cascadeViaForeignKeys = false,
.suppressNoticeMessages = true
};
UndistributeTable(&params);

View File

@ -222,11 +222,11 @@ ErrorIfUnsupportedForeignConstraintExists(Relation relation, char referencingDis
ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
errmsg("cannot create foreign key constraint "
"since foreign keys from reference tables "
"and citus local tables to distributed tables "
"and local tables to distributed tables "
"are not supported"),
errdetail("Reference tables and citus local tables "
errdetail("Reference tables and local tables "
"can only have foreign keys to reference "
"tables and citus local tables")));
"tables and local tables")));
}
/*
@ -335,7 +335,7 @@ EnsureSupportedFKeyBetweenCitusLocalAndRefTable(Form_pg_constraint fKeyConstrain
ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
errmsg("cannot define foreign key constraint, "
"foreign keys from reference tables to "
"citus local tables can only be defined "
"local tables can only be defined "
"with NO ACTION or RESTRICT behaviors"),
errhint(USE_CREATE_REFERENCE_TABLE_HINT,
referencedTableName)));
@ -435,10 +435,10 @@ ErrorOutForFKeyBetweenPostgresAndCitusLocalTable(Oid localTableId)
ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
errmsg("cannot create foreign key constraint as \"%s\" is "
"a postgres local table", localTableName),
errhint("first create a citus local table from the postgres "
"local table using SELECT create_citus_local_table('%s') "
errhint("first add local table to citus metadata "
"by using SELECT create_citus_local_table('%s') "
"and execute the ALTER TABLE command to create the "
"foreign key to citus local table", localTableName)));
"foreign key to local table", localTableName)));
}

View File

@ -1799,8 +1799,8 @@ ErrorIfCitusLocalTablePartitionCommand(AlterTableCmd *alterTableCmd, Oid parentR
ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
errmsg("cannot execute ATTACH/DETACH PARTITION command as "
"citus local tables cannot be involved in partition "
"relationships with other tables")));
"local tables added to metadata cannot be involved in "
"partition relationships with other tables")));
}

View File

@ -553,7 +553,8 @@ ErrorOutForTriggerIfNotCitusLocalTable(Oid relationId)
return;
}
ereport(ERROR, (errmsg("triggers are only supported for citus local tables")));
ereport(ERROR, (errmsg("triggers are only supported for local tables added "
"to metadata")));
}

View File

@ -729,10 +729,18 @@ UndistributeDisconnectedCitusLocalTables(void)
* undistribute it via cascade. Here, instead of first dropping foreing
* keys then undistributing the table, we just set cascadeViaForeignKeys
* to true for simplicity.
*
* We suppress notices messages not to be too verbose. On the other hand,
* as UndistributeTable moves data to a new table, we want to inform user
* as it might take some time.
*/
ereport(NOTICE, (errmsg("removing table %s from metadata as it is not "
"connected to any reference tables via foreign keys",
generate_qualified_relation_name(citusLocalTableId))));
TableConversionParameters params = {
.relationId = citusLocalTableId,
.cascadeViaForeignKeys = true
.cascadeViaForeignKeys = true,
.suppressNoticeMessages = true
};
UndistributeTable(&params);
}

View File

@ -128,8 +128,7 @@ static uint64 LocallyPlanAndExecuteMultipleQueries(List *queryStrings,
static void ExtractParametersForLocalExecution(ParamListInfo paramListInfo,
Oid **parameterTypes,
const char ***parameterValues);
static void LocallyExecuteUtilityTask(const char *utilityCommand);
static void LocallyExecuteUdfTaskQuery(Query *localUdfCommandQuery);
static void ExecuteUdfTaskQuery(Query *localUdfCommandQuery);
static void EnsureTransitionPossible(LocalExecutionStatus from,
LocalExecutionStatus to);
@ -241,7 +240,7 @@ ExecuteLocalTaskListExtended(List *taskList,
if (isUtilityCommand)
{
LocallyExecuteUtilityTask(TaskQueryString(task));
ExecuteUtilityCommand(TaskQueryString(task));
continue;
}
@ -373,43 +372,40 @@ ExtractParametersForLocalExecution(ParamListInfo paramListInfo, Oid **parameterT
/*
* LocallyExecuteUtilityTask executes the given local task query in the current
* ExecuteUtilityCommand executes the given task query in the current
* session.
*/
static void
LocallyExecuteUtilityTask(const char *localTaskQueryCommand)
void
ExecuteUtilityCommand(const char *taskQueryCommand)
{
List *parseTreeList = pg_parse_query(localTaskQueryCommand);
RawStmt *localTaskRawStmt = NULL;
List *parseTreeList = pg_parse_query(taskQueryCommand);
RawStmt *taskRawStmt = NULL;
foreach_ptr(localTaskRawStmt, parseTreeList)
foreach_ptr(taskRawStmt, parseTreeList)
{
Node *localTaskRawParseTree = localTaskRawStmt->stmt;
Node *taskRawParseTree = taskRawStmt->stmt;
/*
* Actually, the query passed to this function would mostly be a
* utility command to be executed locally. However, some utility
* commands do trigger udf calls (e.g worker_apply_shard_ddl_command)
* to execute commands in a generic way. But as we support local
* execution of utility commands, we should also process those udf
* calls locally as well. In that case, we simply execute the query
* implying the udf call in below conditional block.
* The query passed to this function would mostly be a utility
* command. However, some utility commands trigger udf calls
* (e.g alter_columnar_table_set()). In that case, we execute
* the query with the udf call in below conditional block.
*/
if (IsA(localTaskRawParseTree, SelectStmt))
if (IsA(taskRawParseTree, SelectStmt))
{
/* we have no external parameters to rewrite the UDF call RawStmt */
Query *localUdfTaskQuery =
RewriteRawQueryStmt(localTaskRawStmt, localTaskQueryCommand, NULL, 0);
Query *udfTaskQuery =
RewriteRawQueryStmt(taskRawStmt, taskQueryCommand, NULL, 0);
LocallyExecuteUdfTaskQuery(localUdfTaskQuery);
ExecuteUdfTaskQuery(udfTaskQuery);
}
else
{
/*
* It is a regular utility command we should execute it locally via
* It is a regular utility command we should execute it via
* process utility.
*/
ProcessUtilityParseTree(localTaskRawParseTree, localTaskQueryCommand,
ProcessUtilityParseTree(taskRawParseTree, taskQueryCommand,
PROCESS_UTILITY_TOPLEVEL, NULL, None_Receiver,
NULL);
}
@ -418,15 +414,15 @@ LocallyExecuteUtilityTask(const char *localTaskQueryCommand)
/*
* LocallyExecuteUdfTaskQuery executes the given udf command locally. Local udf
* command is simply a "SELECT udf_call()" query and so it cannot be executed
* ExecuteUdfTaskQuery executes the given udf command. A udf command
* is simply a "SELECT udf_call()" query and so it cannot be executed
* via process utility.
*/
static void
LocallyExecuteUdfTaskQuery(Query *localUdfTaskQuery)
ExecuteUdfTaskQuery(Query *udfTaskQuery)
{
/* we do not expect any results */
ExecuteQueryIntoDestReceiver(localUdfTaskQuery, NULL, None_Receiver);
ExecuteQueryIntoDestReceiver(udfTaskQuery, NULL, None_Receiver);
}

View File

@ -177,7 +177,7 @@ master_apply_delete_command(PG_FUNCTION_ARGS)
{
ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
errmsg("cannot delete from table"),
errdetail("Delete statements on reference and citus "
errdetail("Delete statements on reference and "
"local tables are not supported.")));
}

View File

@ -311,9 +311,9 @@ ErrorIfMoveCitusLocalTable(Oid relationId)
char *qualifiedRelationName = generate_qualified_relation_name(relationId);
ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
errmsg("table %s is a citus local table, moving shard of "
"a citus local table is currently not supported",
qualifiedRelationName)));
errmsg("table %s is a local table, moving shard of "
"a local table added to metadata is currently "
"not supported", qualifiedRelationName)));
}
@ -385,9 +385,10 @@ ErrorIfTableCannotBeReplicated(Oid relationId)
if (IsCitusTableTypeCacheEntry(tableEntry, CITUS_LOCAL_TABLE))
{
ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
(errmsg("Table %s is a citus local table. Replicating "
"shard of a citus local table currently is not "
"supported", quote_literal_cstr(relationName)))));
(errmsg("Table %s is a local table. Replicating "
"shard of a local table added to metadata "
"currently is not supported",
quote_literal_cstr(relationName)))));
}
/*

View File

@ -153,10 +153,10 @@ master_create_empty_shard(PG_FUNCTION_ARGS)
}
else if (IsCitusTableType(relationId, CITUS_LOCAL_TABLE))
{
ereport(ERROR, (errmsg("relation \"%s\" is a citus local table",
ereport(ERROR, (errmsg("relation \"%s\" is a local table",
relationName),
errdetail("We currently don't support creating shards "
"on citus local tables")));
"on local tables")));
}
char replicationModel = TableReplicationModel(relationId);
@ -265,8 +265,7 @@ master_append_table_to_shard(PG_FUNCTION_ARGS)
{
ereport(ERROR, (errmsg("cannot append to shardId " UINT64_FORMAT, shardId),
errdetail("We currently don't support appending to shards "
"in hash-partitioned, reference and citus local "
"tables")));
"in hash-partitioned, reference and local tables")));
}
/* ensure that the shard placement metadata does not change during the append */

View File

@ -572,7 +572,7 @@ DistributedInsertSelectSupported(Query *queryTree, RangeTblEntry *insertRte,
{
return DeferredError(ERRCODE_FEATURE_NOT_SUPPORTED,
"distributed INSERT ... SELECT cannot insert into a "
"citus local table",
"local table that is added to metadata",
NULL, NULL);
}

View File

@ -771,10 +771,10 @@ DeferErrorIfUnsupportedLocalTableJoin(List *rangeTableList)
if (ModifiesLocalTableWithRemoteCitusLocalTable(rangeTableList))
{
return DeferredError(ERRCODE_FEATURE_NOT_SUPPORTED,
"Modifying local tables with citus local tables is "
"supported only from the coordinator.",
"Modifying local tables with remote local tables is "
"not supported.",
NULL,
"Consider wrapping citus local table to a CTE, or subquery");
"Consider wrapping remote local table to a CTE, or subquery");
}
return NULL;
}
@ -953,7 +953,7 @@ ModifyQuerySupported(Query *queryTree, Query *originalQuery, bool multiShardQuer
if (IsCitusTable(rangeTableEntry->relid))
{
appendStringInfo(errorMessage,
"citus local table %s cannot be joined with these distributed tables",
"local table %s cannot be joined with these distributed tables",
relationName);
}
else
@ -2879,7 +2879,7 @@ BuildRoutesForInsert(Query *query, DeferredErrorMessage **planningError)
}
else if (IsCitusTableTypeCacheEntry(cacheEntry, CITUS_LOCAL_TABLE))
{
ereport(ERROR, (errmsg("citus local table cannot have %d shards",
ereport(ERROR, (errmsg("local table cannot have %d shards",
shardCount)));
}
}
@ -3545,7 +3545,7 @@ DeferErrorIfUnsupportedRouterPlannableSelectQuery(Query *query)
else if (IsCitusTableType(distributedTableId, CITUS_LOCAL_TABLE))
{
hasPostgresOrCitusLocalTable = true;
elog(DEBUG4, "Router planner finds a citus local table");
elog(DEBUG4, "Router planner finds a local table added to metadata");
continue;
}

View File

@ -772,7 +772,7 @@ CheckConflictingRelationAccesses(Oid relationId, ShardPlacementAccessType access
"a parallel operation on a distributed table",
relationName),
errdetail("When there is a foreign key to a reference "
"table or to a citus local table, Citus needs "
"table or to a local table, Citus needs "
"to perform all operations over a single "
"connection per node to ensure consistency."),
errhint("Try re-running the transaction with "

View File

@ -244,7 +244,7 @@ MarkTablesColocated(Oid sourceRelationId, Oid targetRelationId)
if (IsCitusTableType(sourceRelationId, CITUS_LOCAL_TABLE) ||
IsCitusTableType(targetRelationId, CITUS_LOCAL_TABLE))
{
ereport(ERROR, (errmsg("citus local tables cannot be colocated with "
ereport(ERROR, (errmsg("local tables cannot be colocated with "
"other tables")));
}

View File

@ -228,8 +228,8 @@ ShardIndex(ShardInterval *shardInterval)
{
ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
errmsg("finding index of a given shard is only supported for "
"hash distributed tables, reference tables and citus "
"local tables")));
"hash distributed tables, reference tables and local "
"tables that are added to citus metadata")));
}
/* short-circuit for reference tables */

View File

@ -486,8 +486,8 @@ extern void ErrorIfAnyPartitionRelationInvolvedInNonInheritedFKey(List *relation
extern bool RelationIdListHasReferenceTable(List *relationIdList);
extern void DropRelationForeignKeys(Oid relationId, int flags);
extern void SetLocalEnableLocalReferenceForeignKeys(bool state);
extern void ExecuteAndLogDDLCommandList(List *ddlCommandList);
extern void ExecuteAndLogDDLCommand(const char *commandString);
extern void ExecuteAndLogUtilityCommandList(List *ddlCommandList);
extern void ExecuteAndLogUtilityCommand(const char *commandString);
extern void ExecuteForeignKeyCreateCommandList(List *ddlCommandList,
bool skip_validation);

View File

@ -37,6 +37,7 @@ extern uint64 ExecuteLocalTaskListExtended(List *taskList, ParamListInfo
bool isUtilityCommand);
extern void ExtractLocalAndRemoteTasks(bool readOnlyPlan, List *taskList,
List **localTaskList, List **remoteTaskList);
extern void ExecuteUtilityCommand(const char *utilityCommand);
extern bool ShouldExecuteTasksLocally(List *taskList);
extern bool AnyTaskAccessesLocalNode(List *taskList);
extern bool TaskAccessesLocalNode(Task *task);

View File

@ -153,6 +153,12 @@ typedef struct TableConversionParameters
* to the table
*/
bool cascadeViaForeignKeys;
/*
* suppressNoticeMessages determines if we want to suppress NOTICE
* messages that we explicitly issue
*/
bool suppressNoticeMessages;
} TableConversionParameters;
typedef struct TableConversionReturn

View File

@ -202,5 +202,5 @@ s/citus_local_table_4_[0-9]+/citus_local_table_4_xxxxxx/g
s/ERROR: cannot append to shardId [0-9]+/ERROR: cannot append to shardId xxxxxx/g
# hide warning/hint message that we get when executing create_citus_local_table
/citus local tables that are not chained with reference tables via foreign keys might be automatically converted back to postgres tables$/d
/Consider setting citus.enable_local_reference_table_foreign_keys to 'off' to disable automatically undistributing citus local tables$/d
/local tables that are added to metadata but not chained with reference tables via foreign keys might be automatically converted back to postgres tables$/d
/Consider setting citus.enable_local_reference_table_foreign_keys to 'off' to disable this behavior$/d

View File

@ -111,12 +111,12 @@ CREATE TABLE partitioned_table_1 PARTITION OF partitioned_table FOR VALUES FROM
CREATE TABLE partitioned_table_2 PARTITION OF partitioned_table FOR VALUES FROM (10) TO (20);
-- cannot create partitioned citus local tables
SELECT create_citus_local_table('partitioned_table');
ERROR: cannot create citus local table "partitioned_table", only regular tables and foreign tables are supported for citus local table creation
ERROR: cannot add local table "partitioned_table" to metadata, only regular tables and foreign tables can be added to citus metadata
BEGIN;
CREATE TABLE citus_local_table PARTITION OF partitioned_table FOR VALUES FROM (20) TO (30);
-- cannot create citus local table as a partition of a local table
SELECT create_citus_local_table('citus_local_table');
ERROR: cannot create citus local table "citus_local_table", citus local tables cannot be partition of other tables
ERROR: cannot add local table "citus_local_table" to metadata, local tables added to metadata cannot be partition of other tables
ROLLBACK;
BEGIN;
CREATE TABLE citus_local_table (a int, b int);
@ -147,16 +147,16 @@ BEGIN;
-- cannot attach citus local table to a partitioned distributed table
ALTER TABLE partitioned_table ATTACH PARTITION citus_local_table FOR VALUES FROM (20) TO (30);
ERROR: cannot execute ATTACH/DETACH PARTITION command as citus local tables cannot be involved in partition relationships with other tables
ERROR: cannot execute ATTACH/DETACH PARTITION command as local tables added to metadata cannot be involved in partition relationships with other tables
ROLLBACK;
-- show that we do not support inheritance relationships --
CREATE TABLE parent_table (a int, b text);
CREATE TABLE child_table () INHERITS (parent_table);
-- both of below should error out
SELECT create_citus_local_table('parent_table');
ERROR: cannot create citus local table "parent_table", citus local tables cannot be involved in inheritance relationships
ERROR: cannot add local table "parent_table" to metadata, local tables added to metadata cannot be involved in inheritance relationships
SELECT create_citus_local_table('child_table');
ERROR: cannot create citus local table "child_table", citus local tables cannot be involved in inheritance relationships
ERROR: cannot add local table "child_table" to metadata, local tables added to metadata cannot be involved in inheritance relationships
-- show that we support UNLOGGED tables --
CREATE UNLOGGED TABLE unlogged_table (a int primary key);
SELECT create_citus_local_table('unlogged_table');
@ -298,20 +298,20 @@ SELECT create_reference_table('reference_table');
-- show that colociation of citus local tables are not supported for now
-- between citus local tables
SELECT update_distributed_table_colocation('citus_local_table_1', colocate_with => 'citus_local_table_2');
ERROR: citus local tables cannot be colocated with other tables
ERROR: local tables cannot be colocated with other tables
-- between citus local tables and reference tables
SELECT update_distributed_table_colocation('citus_local_table_1', colocate_with => 'reference_table');
ERROR: citus local tables cannot be colocated with other tables
ERROR: local tables cannot be colocated with other tables
SELECT update_distributed_table_colocation('reference_table', colocate_with => 'citus_local_table_1');
ERROR: citus local tables cannot be colocated with other tables
ERROR: local tables cannot be colocated with other tables
-- between citus local tables and distributed tables
SELECT update_distributed_table_colocation('citus_local_table_1', colocate_with => 'distributed_table');
ERROR: citus local tables cannot be colocated with other tables
ERROR: local tables cannot be colocated with other tables
SELECT update_distributed_table_colocation('distributed_table', colocate_with => 'citus_local_table_1');
ERROR: citus local tables cannot be colocated with other tables
ERROR: local tables cannot be colocated with other tables
-- master_create_empty_shard is not supported
SELECT master_create_empty_shard('citus_local_table_1');
ERROR: relation "citus_local_table_1" is a citus local table
ERROR: relation "citus_local_table_1" is a local table
-- get_shard_id_for_distribution_column is supported
SELECT get_shard_id_for_distribution_column('citus_local_table_1', 'not_checking_this_arg_for_non_dist_tables');
get_shard_id_for_distribution_column
@ -328,7 +328,7 @@ SELECT get_shard_id_for_distribution_column('citus_local_table_1');
-- master_copy_shard_placement is not supported
SELECT master_copy_shard_placement(shardid, 'localhost', :master_port, 'localhost', :worker_1_port, true)
FROM (SELECT shardid FROM pg_dist_shard WHERE logicalrelid='citus_local_table_1'::regclass) as shardid;
ERROR: Table 'citus_local_table_1' is a citus local table. Replicating shard of a citus local table currently is not supported
ERROR: Table 'citus_local_table_1' is a local table. Replicating shard of a local table added to metadata currently is not supported
-- undistribute_table is supported
BEGIN;
SELECT undistribute_table('citus_local_table_1');
@ -478,7 +478,7 @@ BEGIN;
ROLLBACK;
-- between citus local tables and distributed tables
ALTER TABLE citus_local_table_1 ADD CONSTRAINT fkey_c_to_dist FOREIGN KEY(a) references distributed_table(a);
ERROR: cannot create foreign key constraint since foreign keys from reference tables and citus local tables to distributed tables are not supported
ERROR: cannot create foreign key constraint since foreign keys from reference tables and local tables to distributed tables are not supported
ALTER TABLE distributed_table ADD CONSTRAINT fkey_dist_to_c FOREIGN KEY(a) references citus_local_table_1(a);
ERROR: cannot create foreign key constraint since relations are not colocated or not referencing a reference table
-- between citus local tables and local tables
@ -699,7 +699,7 @@ ROLLBACK;
SELECT update_distributed_table_colocation('citus_local_table_4', colocate_with => 'none');
ERROR: relation citus_local_table_4 should be a hash distributed table
SELECT master_create_empty_shard('citus_local_table_4');
ERROR: relation "citus_local_table_4" is a citus local table
ERROR: relation "citus_local_table_4" is a local table
SELECT master_apply_delete_command('DELETE FROM citus_local_table_4');
ERROR: cannot delete from table
CREATE TABLE postgres_local_table (a int);

View File

@ -893,7 +893,7 @@ UPDATE citus_local_table
SET a=5
FROM (SELECT b FROM distributed_table) AS foo
WHERE foo.b = citus_local_table.b;
ERROR: citus local table citus_local_table cannot be joined with these distributed tables
ERROR: local table citus_local_table cannot be joined with these distributed tables
---------------------------------------------------------------------
-- test different execution paths --
---------------------------------------------------------------------

View File

@ -591,21 +591,21 @@ SELECT clear_and_init_test_tables();
DELETE FROM citus_local_table
USING postgres_local_table
WHERE citus_local_table.b = postgres_local_table.b;
ERROR: Modifying local tables with citus local tables is supported only from the coordinator.
ERROR: Modifying local tables with remote local tables is not supported.
UPDATE citus_local_table
SET b = 5
FROM postgres_local_table
WHERE citus_local_table.a = 3 AND citus_local_table.b = postgres_local_table.b;
ERROR: Modifying local tables with citus local tables is supported only from the coordinator.
ERROR: Modifying local tables with remote local tables is not supported.
DELETE FROM postgres_local_table
USING citus_local_table
WHERE citus_local_table.b = postgres_local_table.b;
ERROR: Modifying local tables with citus local tables is supported only from the coordinator.
ERROR: Modifying local tables with remote local tables is not supported.
UPDATE postgres_local_table
SET b = 5
FROM citus_local_table
WHERE citus_local_table.a = 3 AND citus_local_table.b = postgres_local_table.b;
ERROR: Modifying local tables with citus local tables is supported only from the coordinator.
ERROR: Modifying local tables with remote local tables is not supported.
-- no direct joins supported
UPDATE distributed_table
SET b = 6
@ -776,7 +776,7 @@ UPDATE citus_local_table
SET a=5
FROM (SELECT b FROM distributed_table) AS foo
WHERE foo.b = citus_local_table.b;
ERROR: citus local table citus_local_table cannot be joined with these distributed tables
ERROR: local table citus_local_table cannot be joined with these distributed tables
---------------------------------------------------------------------
-- test different execution paths --
---------------------------------------------------------------------

View File

@ -34,11 +34,11 @@ SELECT create_reference_table('reference_table');
CREATE TRIGGER update_value_dist
AFTER INSERT ON distributed_table
FOR EACH ROW EXECUTE FUNCTION update_value();
ERROR: triggers are only supported for citus local tables
ERROR: triggers are only supported for local tables added to metadata
CREATE TRIGGER update_value_ref
AFTER INSERT ON reference_table
FOR EACH ROW EXECUTE FUNCTION update_value();
ERROR: triggers are only supported for citus local tables
ERROR: triggers are only supported for local tables added to metadata
---------------------------------------------------------------------
-- show that we error out for trigger commands on distributed & reference tables
---------------------------------------------------------------------
@ -56,42 +56,42 @@ SET citus.enable_ddl_propagation to ON;
CREATE EXTENSION seg;
-- below all should error out
ALTER TRIGGER update_value_dist ON distributed_table RENAME TO update_value_dist1;
ERROR: triggers are only supported for citus local tables
ERROR: triggers are only supported for local tables added to metadata
ALTER TRIGGER update_value_dist ON distributed_table DEPENDS ON EXTENSION seg;
ERROR: triggers are only supported for citus local tables
ERROR: triggers are only supported for local tables added to metadata
DROP TRIGGER update_value_dist ON distributed_table;
ERROR: triggers are only supported for citus local tables
ERROR: triggers are only supported for local tables added to metadata
ALTER TABLE distributed_table DISABLE TRIGGER ALL;
ERROR: triggers are only supported for citus local tables
ERROR: triggers are only supported for local tables added to metadata
ALTER TABLE distributed_table DISABLE TRIGGER USER;
ERROR: triggers are only supported for citus local tables
ERROR: triggers are only supported for local tables added to metadata
ALTER TABLE distributed_table DISABLE TRIGGER update_value_dist;
ERROR: triggers are only supported for citus local tables
ERROR: triggers are only supported for local tables added to metadata
ALTER TABLE distributed_table ENABLE TRIGGER ALL;
ERROR: triggers are only supported for citus local tables
ERROR: triggers are only supported for local tables added to metadata
ALTER TABLE distributed_table ENABLE TRIGGER USER;
ERROR: triggers are only supported for citus local tables
ERROR: triggers are only supported for local tables added to metadata
ALTER TABLE distributed_table ENABLE TRIGGER update_value_dist;
ERROR: triggers are only supported for citus local tables
ERROR: triggers are only supported for local tables added to metadata
-- below all should error out
ALTER TRIGGER update_value_ref ON reference_table RENAME TO update_value_ref1;
ERROR: triggers are only supported for citus local tables
ERROR: triggers are only supported for local tables added to metadata
ALTER TRIGGER update_value_ref ON reference_table DEPENDS ON EXTENSION seg;
ERROR: triggers are only supported for citus local tables
ERROR: triggers are only supported for local tables added to metadata
DROP TRIGGER update_value_ref ON reference_table;
ERROR: triggers are only supported for citus local tables
ERROR: triggers are only supported for local tables added to metadata
ALTER TABLE reference_table DISABLE TRIGGER ALL;
ERROR: triggers are only supported for citus local tables
ERROR: triggers are only supported for local tables added to metadata
ALTER TABLE reference_table DISABLE TRIGGER USER;
ERROR: triggers are only supported for citus local tables
ERROR: triggers are only supported for local tables added to metadata
ALTER TABLE reference_table DISABLE TRIGGER update_value_ref;
ERROR: triggers are only supported for citus local tables
ERROR: triggers are only supported for local tables added to metadata
ALTER TABLE reference_table ENABLE TRIGGER ALL;
ERROR: triggers are only supported for citus local tables
ERROR: triggers are only supported for local tables added to metadata
ALTER TABLE reference_table ENABLE TRIGGER USER;
ERROR: triggers are only supported for citus local tables
ERROR: triggers are only supported for local tables added to metadata
ALTER TABLE reference_table ENABLE TRIGGER update_value_ref;
ERROR: triggers are only supported for citus local tables
ERROR: triggers are only supported for local tables added to metadata
---------------------------------------------------------------------
-- show that we do not allow creating citus tables if the
-- table has already triggers

View File

@ -247,6 +247,28 @@ $cmd$);
(localhost,57638,20090007,t,"(100,1000,pglz,15)")
(4 rows)
-- verify undistribute works
SELECT undistribute_table('table_option');
NOTICE: creating a new table for columnar_citus_integration.table_option
NOTICE: Moving the data of columnar_citus_integration.table_option
NOTICE: Dropping the old columnar_citus_integration.table_option
NOTICE: Renaming the new table to columnar_citus_integration.table_option
undistribute_table
---------------------------------------------------------------------
(1 row)
SELECT * FROM pg_dist_partition WHERE logicalrelid = 'table_option'::regclass;
logicalrelid | partmethod | partkey | colocationid | repmodel
---------------------------------------------------------------------
(0 rows)
SELECT compression FROM columnar.options WHERE regclass = 'table_option'::regclass;
compression
---------------------------------------------------------------------
none
(1 row)
DROP TABLE table_option, table_option_2;
-- verify settings get to all placements when there are multiple replica's
SET citus.shard_replication_factor TO 2;
@ -544,6 +566,28 @@ $cmd$);
(localhost,57638,20090015,t,"(100,1000,pglz,19)")
(8 rows)
-- verify undistribute works
SELECT undistribute_table('table_option');
NOTICE: creating a new table for columnar_citus_integration.table_option
NOTICE: Moving the data of columnar_citus_integration.table_option
NOTICE: Dropping the old columnar_citus_integration.table_option
NOTICE: Renaming the new table to columnar_citus_integration.table_option
undistribute_table
---------------------------------------------------------------------
(1 row)
SELECT * FROM pg_dist_partition WHERE logicalrelid = 'table_option'::regclass;
logicalrelid | partmethod | partkey | colocationid | repmodel
---------------------------------------------------------------------
(0 rows)
SELECT compression FROM columnar.options WHERE regclass = 'table_option'::regclass;
compression
---------------------------------------------------------------------
none
(1 row)
DROP TABLE table_option, table_option_2;
-- test options on a reference table
CREATE TABLE table_option_reference (a int, b text) USING columnar;
@ -761,6 +805,267 @@ $cmd$);
(localhost,57638,20090017,t,"(100,1000,pglz,9)")
(2 rows)
-- verify undistribute works
SELECT undistribute_table('table_option_reference');
NOTICE: creating a new table for columnar_citus_integration.table_option_reference
NOTICE: Moving the data of columnar_citus_integration.table_option_reference
NOTICE: Dropping the old columnar_citus_integration.table_option_reference
NOTICE: Renaming the new table to columnar_citus_integration.table_option_reference
undistribute_table
---------------------------------------------------------------------
(1 row)
SELECT * FROM pg_dist_partition WHERE logicalrelid = 'table_option_reference'::regclass;
logicalrelid | partmethod | partkey | colocationid | repmodel
---------------------------------------------------------------------
(0 rows)
SELECT compression FROM columnar.options WHERE regclass = 'table_option_reference'::regclass;
compression
---------------------------------------------------------------------
none
(1 row)
DROP TABLE table_option_reference, table_option_reference_2;
SET citus.shard_replication_factor TO 1;
-- test options on a citus local table
SELECT 1 FROM master_add_node('localhost', :master_port, groupId => 0);
?column?
---------------------------------------------------------------------
1
(1 row)
CREATE TABLE table_option_citus_local (a int, b text) USING columnar;
SELECT create_citus_local_table('table_option_citus_local');
create_citus_local_table
---------------------------------------------------------------------
(1 row)
-- setting: compression
-- get baseline for setting
SELECT run_command_on_placements('table_option_citus_local',$cmd$
SELECT compression FROM columnar.options WHERE regclass = '%s'::regclass;
$cmd$);
run_command_on_placements
---------------------------------------------------------------------
(localhost,57636,20090018,t,none)
(1 row)
-- change setting
SELECT alter_columnar_table_set('table_option_citus_local', compression => 'pglz');
alter_columnar_table_set
---------------------------------------------------------------------
(1 row)
-- verify setting
SELECT run_command_on_placements('table_option_citus_local',$cmd$
SELECT compression FROM columnar.options WHERE regclass = '%s'::regclass;
$cmd$);
run_command_on_placements
---------------------------------------------------------------------
(localhost,57636,20090018,t,pglz)
(1 row)
-- reset setting
SELECT alter_columnar_table_reset('table_option_citus_local', compression => true);
alter_columnar_table_reset
---------------------------------------------------------------------
(1 row)
-- verify setting
SELECT run_command_on_placements('table_option_citus_local',$cmd$
SELECT compression FROM columnar.options WHERE regclass = '%s'::regclass;
$cmd$);
run_command_on_placements
---------------------------------------------------------------------
(localhost,57636,20090018,t,none)
(1 row)
-- setting: compression_level
-- get baseline for setting
SELECT run_command_on_placements('table_option_citus_local',$cmd$
SELECT compression_level FROM columnar.options WHERE regclass = '%s'::regclass;
$cmd$);
run_command_on_placements
---------------------------------------------------------------------
(localhost,57636,20090018,t,3)
(1 row)
-- change setting
SELECT alter_columnar_table_set('table_option_citus_local', compression_level => 11);
alter_columnar_table_set
---------------------------------------------------------------------
(1 row)
-- verify setting
SELECT run_command_on_placements('table_option_citus_local',$cmd$
SELECT compression_level FROM columnar.options WHERE regclass = '%s'::regclass;
$cmd$);
run_command_on_placements
---------------------------------------------------------------------
(localhost,57636,20090018,t,11)
(1 row)
-- reset setting
SELECT alter_columnar_table_reset('table_option_citus_local', compression_level => true);
alter_columnar_table_reset
---------------------------------------------------------------------
(1 row)
-- verify setting
SELECT run_command_on_placements('table_option_citus_local',$cmd$
SELECT compression_level FROM columnar.options WHERE regclass = '%s'::regclass;
$cmd$);
run_command_on_placements
---------------------------------------------------------------------
(localhost,57636,20090018,t,3)
(1 row)
-- setting: chunk_row_count
-- get baseline for setting
SELECT run_command_on_placements('table_option_citus_local',$cmd$
SELECT chunk_row_count FROM columnar.options WHERE regclass = '%s'::regclass;
$cmd$);
run_command_on_placements
---------------------------------------------------------------------
(localhost,57636,20090018,t,10000)
(1 row)
-- change setting
SELECT alter_columnar_table_set('table_option_citus_local', chunk_row_count => 100);
alter_columnar_table_set
---------------------------------------------------------------------
(1 row)
-- verify setting
SELECT run_command_on_placements('table_option_citus_local',$cmd$
SELECT chunk_row_count FROM columnar.options WHERE regclass = '%s'::regclass;
$cmd$);
run_command_on_placements
---------------------------------------------------------------------
(localhost,57636,20090018,t,100)
(1 row)
-- reset setting
SELECT alter_columnar_table_reset('table_option_citus_local', chunk_row_count => true);
alter_columnar_table_reset
---------------------------------------------------------------------
(1 row)
-- verify setting
SELECT run_command_on_placements('table_option_citus_local',$cmd$
SELECT chunk_row_count FROM columnar.options WHERE regclass = '%s'::regclass;
$cmd$);
run_command_on_placements
---------------------------------------------------------------------
(localhost,57636,20090018,t,10000)
(1 row)
-- setting: stripe_row_count
-- get baseline for setting
SELECT run_command_on_placements('table_option_citus_local',$cmd$
SELECT stripe_row_count FROM columnar.options WHERE regclass = '%s'::regclass;
$cmd$);
run_command_on_placements
---------------------------------------------------------------------
(localhost,57636,20090018,t,150000)
(1 row)
-- change setting
SELECT alter_columnar_table_set('table_option_citus_local', stripe_row_count => 100);
alter_columnar_table_set
---------------------------------------------------------------------
(1 row)
-- verify setting
SELECT run_command_on_placements('table_option_citus_local',$cmd$
SELECT stripe_row_count FROM columnar.options WHERE regclass = '%s'::regclass;
$cmd$);
run_command_on_placements
---------------------------------------------------------------------
(localhost,57636,20090018,t,100)
(1 row)
-- reset setting
SELECT alter_columnar_table_reset('table_option_citus_local', stripe_row_count => true);
alter_columnar_table_reset
---------------------------------------------------------------------
(1 row)
-- verify setting
SELECT run_command_on_placements('table_option_citus_local',$cmd$
SELECT stripe_row_count FROM columnar.options WHERE regclass = '%s'::regclass;
$cmd$);
run_command_on_placements
---------------------------------------------------------------------
(localhost,57636,20090018,t,150000)
(1 row)
-- verify settings are propagated when creating a table
CREATE TABLE table_option_citus_local_2 (a int, b text) USING columnar;
SELECT alter_columnar_table_set('table_option_citus_local_2',
chunk_row_count => 100,
stripe_row_count => 1000,
compression => 'pglz',
compression_level => 9);
alter_columnar_table_set
---------------------------------------------------------------------
(1 row)
SELECT create_citus_local_table('table_option_citus_local_2');
create_citus_local_table
---------------------------------------------------------------------
(1 row)
-- verify settings on placements
SELECT run_command_on_placements('table_option_citus_local_2',$cmd$
SELECT ROW(chunk_row_count, stripe_row_count, compression, compression_level) FROM columnar.options WHERE regclass = '%s'::regclass;
$cmd$);
run_command_on_placements
---------------------------------------------------------------------
(localhost,57636,20090019,t,"(100,1000,pglz,9)")
(1 row)
-- verify undistribute works
SELECT undistribute_table('table_option_citus_local');
NOTICE: creating a new table for columnar_citus_integration.table_option_citus_local
NOTICE: Moving the data of columnar_citus_integration.table_option_citus_local
NOTICE: Dropping the old columnar_citus_integration.table_option_citus_local
NOTICE: Renaming the new table to columnar_citus_integration.table_option_citus_local
undistribute_table
---------------------------------------------------------------------
(1 row)
SELECT * FROM pg_dist_partition WHERE logicalrelid = 'table_option_citus_local'::regclass;
logicalrelid | partmethod | partkey | colocationid | repmodel
---------------------------------------------------------------------
(0 rows)
SELECT compression FROM columnar.options WHERE regclass = 'table_option_citus_local'::regclass;
compression
---------------------------------------------------------------------
none
(1 row)
DROP TABLE table_option_citus_local, table_option_citus_local_2;
SELECT 1 FROM master_remove_node('localhost', :master_port);
?column?
---------------------------------------------------------------------
1
(1 row)
SET client_min_messages TO WARNING;
DROP SCHEMA columnar_citus_integration CASCADE;

View File

@ -120,7 +120,7 @@ BEGIN;
-- create_citus_local_table(cascade_via_foreign_keys) would fail for
-- partitioned_table as create_citus_local_table doesn't support partitioned tables
SELECT create_citus_local_table('local_table_2', cascade_via_foreign_keys=>true);
ERROR: cannot create citus local table "partitioned_table", only regular tables and foreign tables are supported for citus local table creation
ERROR: cannot add local table "partitioned_table" to metadata, only regular tables and foreign tables can be added to citus metadata
ROLLBACK;
BEGIN;
DROP TABLE local_table_2;

View File

@ -311,7 +311,7 @@ BEGIN;
-- tables are not supported
ALTER TABLE citus_local_table_5 ADD CONSTRAINT fkey_12 FOREIGN KEY (col_1) REFERENCES citus_local_table_6(col_1);
SELECT create_distributed_table('citus_local_table_6', 'col_1');
ERROR: cannot create foreign key constraint since foreign keys from reference tables and citus local tables to distributed tables are not supported
ERROR: cannot create foreign key constraint since foreign keys from reference tables and local tables to distributed tables are not supported
ROLLBACK;
-- have some more tests with foreign keys between citus local
-- and reference tables

View File

@ -102,7 +102,7 @@ ALTER TABLE partitioned_table_1 ADD CONSTRAINT fkey_8 FOREIGN KEY (col_1) REFERE
-- now that we attached partitioned table to graph below errors out
-- since we cannot create citus local table from partitioned tables
ALTER TABLE reference_table_1 ADD CONSTRAINT fkey_9 FOREIGN KEY (col_1) REFERENCES local_table_1(col_1);
ERROR: cannot create citus local table "partitioned_table_1", only regular tables and foreign tables are supported for citus local table creation
ERROR: cannot add local table "partitioned_table_1" to metadata, only regular tables and foreign tables can be added to citus metadata
ALTER TABLE partitioned_table_1 DROP CONSTRAINT fkey_8;
BEGIN;
-- now that we detached partitioned table from graph, succeeds
@ -214,7 +214,7 @@ BEGIN;
(1 row)
ALTER TABLE reference_table_2 ADD CONSTRAINT fkey_11 FOREIGN KEY (col_1) REFERENCES local_table_2(col_1) ON DELETE CASCADE;
ERROR: cannot define foreign key constraint, foreign keys from reference tables to citus local tables can only be defined with NO ACTION or RESTRICT behaviors
ERROR: cannot define foreign key constraint, foreign keys from reference tables to local tables can only be defined with NO ACTION or RESTRICT behaviors
ROLLBACK;
BEGIN;
SELECT create_reference_table('reference_table_2');
@ -224,7 +224,7 @@ BEGIN;
(1 row)
ALTER TABLE reference_table_2 ADD CONSTRAINT fkey_local_to_ref FOREIGN KEY (col_1) REFERENCES local_table_2(col_1) ON UPDATE CASCADE;
ERROR: cannot define foreign key constraint, foreign keys from reference tables to citus local tables can only be defined with NO ACTION or RESTRICT behaviors
ERROR: cannot define foreign key constraint, foreign keys from reference tables to local tables can only be defined with NO ACTION or RESTRICT behaviors
ROLLBACK;
-- but we support such foreign key behaviors when foreign key is from
-- citus local to reference table
@ -304,7 +304,7 @@ SELECT create_distributed_table('distributed_table', 'col_1');
CREATE TABLE local_table_5 (
col_1 INT UNIQUE REFERENCES distributed_table(col_1),
FOREIGN KEY (col_1) REFERENCES reference_table_1(col_1));
ERROR: cannot create foreign key constraint since foreign keys from reference tables and citus local tables to distributed tables are not supported
ERROR: cannot create foreign key constraint since foreign keys from reference tables and local tables to distributed tables are not supported
BEGIN;
ALTER TABLE distributed_table ADD CONSTRAINT fkey_11 FOREIGN KEY (col_1) REFERENCES reference_table_1(col_1);
CREATE TABLE local_table_5 (
@ -577,7 +577,7 @@ BEGIN;
-- errors out as foreign keys from reference tables to citus local tables
-- cannot have CASCADE behavior
SELECT create_reference_table('local_table_1');
ERROR: cannot define foreign key constraint, foreign keys from reference tables to citus local tables can only be defined with NO ACTION or RESTRICT behaviors
ERROR: cannot define foreign key constraint, foreign keys from reference tables to local tables can only be defined with NO ACTION or RESTRICT behaviors
ROLLBACK;
SET citus.enable_local_execution TO OFF;
-- show that this errors out as it tries to convert connected relations to citus
@ -623,7 +623,7 @@ ORDER BY tablename;
-- this errors out as we don't support creating citus local
-- tables from partitioned tables
CREATE TABLE part_local_table (col_1 INT REFERENCES reference_table_1(col_1)) PARTITION BY RANGE (col_1);
ERROR: cannot create citus local table "part_local_table", only regular tables and foreign tables are supported for citus local table creation
ERROR: cannot add local table "part_local_table" to metadata, only regular tables and foreign tables can be added to citus metadata
-- they fail as col_99 does not exist
CREATE TABLE local_table_5 (col_1 INT, FOREIGN KEY (col_99) REFERENCES reference_table_1(col_1));
ERROR: column "col_99" referenced in foreign key constraint does not exist

View File

@ -722,7 +722,7 @@ BEGIN;
UPDATE reference_table SET id = 101 WHERE id = 99;
ERROR: cannot modify table "reference_table" because there was a parallel operation on a distributed table
DETAIL: When there is a foreign key to a reference table or to a citus local table, Citus needs to perform all operations over a single connection per node to ensure consistency.
DETAIL: When there is a foreign key to a reference table or to a local table, Citus needs to perform all operations over a single connection per node to ensure consistency.
HINT: Try re-running the transaction with "SET LOCAL citus.multi_shard_modify_mode TO 'sequential';"
ROLLBACK;
BEGIN;
@ -734,7 +734,7 @@ BEGIN;
UPDATE transitive_reference_table SET id = 101 WHERE id = 99;
ERROR: cannot modify table "transitive_reference_table" because there was a parallel operation on a distributed table
DETAIL: When there is a foreign key to a reference table or to a citus local table, Citus needs to perform all operations over a single connection per node to ensure consistency.
DETAIL: When there is a foreign key to a reference table or to a local table, Citus needs to perform all operations over a single connection per node to ensure consistency.
HINT: Try re-running the transaction with "SET LOCAL citus.multi_shard_modify_mode TO 'sequential';"
ROLLBACK;
-- case 4.3: SELECT to a dist table is follwed by an unrelated DDL to a reference table
@ -1074,7 +1074,7 @@ BEGIN;
UPDATE unrelated_dist_table SET value_1 = 15;
UPDATE reference_table SET id = 101 WHERE id = 99;
ERROR: cannot modify table "reference_table" because there was a parallel operation on a distributed table
DETAIL: When there is a foreign key to a reference table or to a citus local table, Citus needs to perform all operations over a single connection per node to ensure consistency.
DETAIL: When there is a foreign key to a reference table or to a local table, Citus needs to perform all operations over a single connection per node to ensure consistency.
HINT: Try re-running the transaction with "SET LOCAL citus.multi_shard_modify_mode TO 'sequential';"
UPDATE on_update_fkey_table SET value_1 = 5 WHERE id != 11;
ERROR: current transaction is aborted, commands ignored until end of transaction block

View File

@ -139,7 +139,7 @@ step s1-drop-table: DROP TABLE citus_local_table_1;
step s2-create-citus-local-table-1: SELECT create_citus_local_table('citus_local_table_1'); <waiting ...>
step s1-commit: COMMIT;
step s2-create-citus-local-table-1: <... completed>
error in steps s1-commit s2-create-citus-local-table-1: ERROR: cannot create citus local table, relation does not exist
error in steps s1-commit s2-create-citus-local-table-1: ERROR: cannot add local table to metadata, relation does not exist
step s2-commit: COMMIT;
master_remove_node

View File

@ -835,8 +835,8 @@ SELECT create_distributed_table('referenced_by_reference_table', 'id');
CREATE TABLE reference_table(id int, referencing_column int REFERENCES referenced_by_reference_table(id));
SELECT create_reference_table('reference_table');
ERROR: cannot create foreign key constraint since foreign keys from reference tables and citus local tables to distributed tables are not supported
DETAIL: Reference tables and citus local tables can only have foreign keys to reference tables and citus local tables
ERROR: cannot create foreign key constraint since foreign keys from reference tables and local tables to distributed tables are not supported
DETAIL: Reference tables and local tables can only have foreign keys to reference tables and local tables
-- test foreign key creation on CREATE TABLE from + to reference table
DROP TABLE reference_table;
CREATE TABLE reference_table(id int PRIMARY KEY, referencing_column int);
@ -886,8 +886,8 @@ SELECT create_reference_table('reference_table');
(1 row)
ALTER TABLE reference_table ADD CONSTRAINT fk FOREIGN KEY(referencing_column) REFERENCES referenced_by_reference_table(id);
ERROR: cannot create foreign key constraint since foreign keys from reference tables and citus local tables to distributed tables are not supported
DETAIL: Reference tables and citus local tables can only have foreign keys to reference tables and citus local tables
ERROR: cannot create foreign key constraint since foreign keys from reference tables and local tables to distributed tables are not supported
DETAIL: Reference tables and local tables can only have foreign keys to reference tables and local tables
-- test foreign key creation on ALTER TABLE to reference table
CREATE TABLE references_to_reference_table(id int, referencing_column int);
SELECT create_distributed_table('references_to_reference_table', 'referencing_column');

View File

@ -1196,8 +1196,8 @@ DEBUG: cannot perform distributed INSERT INTO ... SELECT because the partition
DETAIL: The target table's partition column should correspond to a partition column in the subquery.
DEBUG: performing repartitioned INSERT ... SELECT
RESET client_min_messages;
-- some tests for mark_tables_colocated
-- should error out
-- some tests for mark_tables_colocated
-- should error out
SELECT update_distributed_table_colocation('colocated_table_test_2', colocate_with => 'reference_table_test');
ERROR: relation reference_table_test should be a hash distributed table
SELECT update_distributed_table_colocation('reference_table_test', colocate_with => 'reference_table_test_fifth');
@ -1404,7 +1404,7 @@ ALTER TABLE reference_schema.reference_table_ddl_test RENAME TO reference_table_
-- cannot delete / drop shards from a reference table
SELECT master_apply_delete_command('DELETE FROM reference_schema.reference_table_ddl');
ERROR: cannot delete from table
DETAIL: Delete statements on reference and citus local tables are not supported.
DETAIL: Delete statements on reference and local tables are not supported.
-- cannot add shards
SELECT master_create_empty_shard('reference_schema.reference_table_ddl');
ERROR: relation "reference_schema.reference_table_ddl" is a reference table
@ -1428,7 +1428,7 @@ SELECT master_update_shard_statistics(:a_shard_id);
CREATE TABLE append_reference_tmp_table (id INT);
SELECT master_append_table_to_shard(:a_shard_id, 'append_reference_tmp_table', 'localhost', :master_port);
ERROR: cannot append to shardId xxxxxx
DETAIL: We currently don't support appending to shards in hash-partitioned, reference and citus local tables
DETAIL: We currently don't support appending to shards in hash-partitioned, reference and local tables
SELECT master_get_table_ddl_events('reference_schema.reference_table_ddl');
master_get_table_ddl_events
---------------------------------------------------------------------

View File

@ -52,12 +52,9 @@ NOTICE: executing the command locally: SELECT l1 FROM ref_citus_local_fkeys.cit
-- show that we support drop constraint
ALTER TABLE citus_local_table DROP CONSTRAINT fkey_local_to_ref;
NOTICE: executing the command locally: SELECT worker_apply_inter_shard_ddl_command (1506000, 'ref_citus_local_fkeys', 1506001, 'ref_citus_local_fkeys', 'ALTER TABLE citus_local_table DROP CONSTRAINT fkey_local_to_ref;')
NOTICE: creating a new table for ref_citus_local_fkeys.citus_local_table
NOTICE: Moving the data of ref_citus_local_fkeys.citus_local_table
NOTICE: removing table ref_citus_local_fkeys.citus_local_table from metadata as it is not connected to any reference tables via foreign keys
NOTICE: executing the command locally: SELECT l1 FROM ref_citus_local_fkeys.citus_local_table_1506000 citus_local_table
NOTICE: Dropping the old ref_citus_local_fkeys.citus_local_table
NOTICE: executing the command locally: DROP TABLE IF EXISTS ref_citus_local_fkeys.citus_local_table_xxxxx CASCADE
NOTICE: Renaming the new table to ref_citus_local_fkeys.citus_local_table
-- we support ON UPDATE CASCADE behaviour in "ALTER TABLE ADD fkey citus_local_table (to reference table)" commands
ALTER TABLE citus_local_table ADD CONSTRAINT fkey_local_to_ref FOREIGN KEY(l1) REFERENCES reference_table(r1) ON UPDATE CASCADE;
NOTICE: executing the command locally: SELECT worker_apply_inter_shard_ddl_command (1506002, 'ref_citus_local_fkeys', 1506001, 'ref_citus_local_fkeys', 'ALTER TABLE citus_local_table ADD CONSTRAINT fkey_local_to_ref FOREIGN KEY(l1) REFERENCES reference_table(r1) ON UPDATE CASCADE;')
@ -79,12 +76,9 @@ NOTICE: executing the command locally: SELECT l1 FROM ref_citus_local_fkeys.cit
-- drop constraint for next commands
ALTER TABLE citus_local_table DROP CONSTRAINT fkey_local_to_ref;
NOTICE: executing the command locally: SELECT worker_apply_inter_shard_ddl_command (1506002, 'ref_citus_local_fkeys', 1506001, 'ref_citus_local_fkeys', 'ALTER TABLE citus_local_table DROP CONSTRAINT fkey_local_to_ref;')
NOTICE: creating a new table for ref_citus_local_fkeys.citus_local_table
NOTICE: Moving the data of ref_citus_local_fkeys.citus_local_table
NOTICE: removing table ref_citus_local_fkeys.citus_local_table from metadata as it is not connected to any reference tables via foreign keys
NOTICE: executing the command locally: SELECT l1 FROM ref_citus_local_fkeys.citus_local_table_1506002 citus_local_table
NOTICE: Dropping the old ref_citus_local_fkeys.citus_local_table
NOTICE: executing the command locally: DROP TABLE IF EXISTS ref_citus_local_fkeys.citus_local_table_xxxxx CASCADE
NOTICE: Renaming the new table to ref_citus_local_fkeys.citus_local_table
INSERT INTO citus_local_table VALUES (2);
-- show that we are checking for foreign key constraint while defining, below should fail
ALTER TABLE citus_local_table ADD CONSTRAINT fkey_local_to_ref FOREIGN KEY(l1) REFERENCES reference_table(r1);
@ -107,12 +101,9 @@ NOTICE: executing the command locally: INSERT INTO ref_citus_local_fkeys.citus_
-- drop and add constraint for next commands
ALTER TABLE citus_local_table DROP CONSTRAINT fkey_local_to_ref;
NOTICE: executing the command locally: SELECT worker_apply_inter_shard_ddl_command (1506004, 'ref_citus_local_fkeys', 1506001, 'ref_citus_local_fkeys', 'ALTER TABLE citus_local_table DROP CONSTRAINT fkey_local_to_ref;')
NOTICE: creating a new table for ref_citus_local_fkeys.citus_local_table
NOTICE: Moving the data of ref_citus_local_fkeys.citus_local_table
NOTICE: removing table ref_citus_local_fkeys.citus_local_table from metadata as it is not connected to any reference tables via foreign keys
NOTICE: executing the command locally: SELECT l1 FROM ref_citus_local_fkeys.citus_local_table_1506004 citus_local_table
NOTICE: Dropping the old ref_citus_local_fkeys.citus_local_table
NOTICE: executing the command locally: DROP TABLE IF EXISTS ref_citus_local_fkeys.citus_local_table_xxxxx CASCADE
NOTICE: Renaming the new table to ref_citus_local_fkeys.citus_local_table
ALTER TABLE citus_local_table ADD CONSTRAINT fkey_local_to_ref FOREIGN KEY(l1) REFERENCES reference_table(r1);
NOTICE: executing the command locally: SELECT worker_apply_inter_shard_ddl_command (1506005, 'ref_citus_local_fkeys', 1506001, 'ref_citus_local_fkeys', 'ALTER TABLE citus_local_table ADD CONSTRAINT fkey_local_to_ref FOREIGN KEY(l1) REFERENCES reference_table(r1);')
-- show that drop table without CASCADE errors out
@ -124,12 +115,9 @@ BEGIN;
NOTICE: drop cascades to constraint fkey_local_to_ref on table citus_local_table
NOTICE: executing the command locally: DROP TABLE IF EXISTS ref_citus_local_fkeys.reference_table_xxxxx CASCADE
NOTICE: drop cascades to constraint fkey_local_to_ref_1506005 on table ref_citus_local_fkeys.citus_local_table_1506005
NOTICE: creating a new table for ref_citus_local_fkeys.citus_local_table
NOTICE: Moving the data of ref_citus_local_fkeys.citus_local_table
NOTICE: removing table ref_citus_local_fkeys.citus_local_table from metadata as it is not connected to any reference tables via foreign keys
NOTICE: executing the command locally: SELECT l1 FROM ref_citus_local_fkeys.citus_local_table_1506005 citus_local_table
NOTICE: Dropping the old ref_citus_local_fkeys.citus_local_table
NOTICE: executing the command locally: DROP TABLE IF EXISTS ref_citus_local_fkeys.citus_local_table_xxxxx CASCADE
NOTICE: Renaming the new table to ref_citus_local_fkeys.citus_local_table
ROLLBACK;
-- drop tables finally
DROP TABLE citus_local_table, reference_table;
@ -171,17 +159,17 @@ NOTICE: executing the command locally: SELECT worker_apply_inter_shard_ddl_comm
ERROR: insert or update on table "reference_table_1506007" violates foreign key constraint "fkey_ref_to_local_1506007"
-- we do not support CASCADE / SET NULL / SET DEFAULT behavior in "ALTER TABLE ADD fkey reference_table (to citus_local_table)" commands
ALTER TABLE reference_table ADD CONSTRAINT fkey_ref_to_local FOREIGN KEY(r1) REFERENCES citus_local_table(l1) ON DELETE CASCADE;
ERROR: cannot define foreign key constraint, foreign keys from reference tables to citus local tables can only be defined with NO ACTION or RESTRICT behaviors
ERROR: cannot define foreign key constraint, foreign keys from reference tables to local tables can only be defined with NO ACTION or RESTRICT behaviors
ALTER TABLE reference_table ADD CONSTRAINT fkey_ref_to_local FOREIGN KEY(r1) REFERENCES citus_local_table(l1) ON DELETE SET NULL;
ERROR: cannot define foreign key constraint, foreign keys from reference tables to citus local tables can only be defined with NO ACTION or RESTRICT behaviors
ERROR: cannot define foreign key constraint, foreign keys from reference tables to local tables can only be defined with NO ACTION or RESTRICT behaviors
ALTER TABLE reference_table ADD CONSTRAINT fkey_ref_to_local FOREIGN KEY(r1) REFERENCES citus_local_table(l1) ON DELETE SET DEFAULT;
ERROR: cannot define foreign key constraint, foreign keys from reference tables to citus local tables can only be defined with NO ACTION or RESTRICT behaviors
ERROR: cannot define foreign key constraint, foreign keys from reference tables to local tables can only be defined with NO ACTION or RESTRICT behaviors
ALTER TABLE reference_table ADD CONSTRAINT fkey_ref_to_local FOREIGN KEY(r1) REFERENCES citus_local_table(l1) ON UPDATE CASCADE;
ERROR: cannot define foreign key constraint, foreign keys from reference tables to citus local tables can only be defined with NO ACTION or RESTRICT behaviors
ERROR: cannot define foreign key constraint, foreign keys from reference tables to local tables can only be defined with NO ACTION or RESTRICT behaviors
ALTER TABLE reference_table ADD CONSTRAINT fkey_ref_to_local FOREIGN KEY(r1) REFERENCES citus_local_table(l1) ON UPDATE SET NULL;
ERROR: cannot define foreign key constraint, foreign keys from reference tables to citus local tables can only be defined with NO ACTION or RESTRICT behaviors
ERROR: cannot define foreign key constraint, foreign keys from reference tables to local tables can only be defined with NO ACTION or RESTRICT behaviors
ALTER TABLE reference_table ADD CONSTRAINT fkey_ref_to_local FOREIGN KEY(r1) REFERENCES citus_local_table(l1) ON UPDATE SET DEFAULT;
ERROR: cannot define foreign key constraint, foreign keys from reference tables to citus local tables can only be defined with NO ACTION or RESTRICT behaviors
ERROR: cannot define foreign key constraint, foreign keys from reference tables to local tables can only be defined with NO ACTION or RESTRICT behaviors
INSERT INTO citus_local_table VALUES (3);
NOTICE: executing the command locally: INSERT INTO ref_citus_local_fkeys.citus_local_table_1506006 (l1) VALUES (3)
-- .. but we allow such foreign keys with RESTRICT behavior
@ -196,10 +184,7 @@ NOTICE: executing the command locally: SELECT worker_apply_inter_shard_ddl_comm
-- tables works fine with remote execution too
SET citus.enable_local_execution TO OFF;
ALTER TABLE reference_table DROP CONSTRAINT fkey_ref_to_local;
NOTICE: creating a new table for ref_citus_local_fkeys.citus_local_table
NOTICE: Moving the data of ref_citus_local_fkeys.citus_local_table
NOTICE: Dropping the old ref_citus_local_fkeys.citus_local_table
NOTICE: Renaming the new table to ref_citus_local_fkeys.citus_local_table
NOTICE: removing table ref_citus_local_fkeys.citus_local_table from metadata as it is not connected to any reference tables via foreign keys
ALTER TABLE reference_table ADD CONSTRAINT fkey_ref_to_local FOREIGN KEY(r1) REFERENCES citus_local_table(l1) ON DELETE NO ACTION;
ERROR: cannot execute command because a local execution has accessed a placement in the transaction
SET citus.enable_local_execution TO ON;
@ -223,12 +208,9 @@ NOTICE: Replicating reference table "reference_table" to the node localhost:xxx
BEGIN;
ALTER TABLE reference_table DROP CONSTRAINT fkey_ref_to_local;
NOTICE: executing the command locally: SELECT worker_apply_inter_shard_ddl_command (1506007, 'ref_citus_local_fkeys', 1506009, 'ref_citus_local_fkeys', 'ALTER TABLE reference_table DROP CONSTRAINT fkey_ref_to_local;')
NOTICE: creating a new table for ref_citus_local_fkeys.citus_local_table
NOTICE: Moving the data of ref_citus_local_fkeys.citus_local_table
NOTICE: removing table ref_citus_local_fkeys.citus_local_table from metadata as it is not connected to any reference tables via foreign keys
NOTICE: executing the command locally: SELECT l1 FROM ref_citus_local_fkeys.citus_local_table_1506009 citus_local_table
NOTICE: Dropping the old ref_citus_local_fkeys.citus_local_table
NOTICE: executing the command locally: DROP TABLE IF EXISTS ref_citus_local_fkeys.citus_local_table_xxxxx CASCADE
NOTICE: Renaming the new table to ref_citus_local_fkeys.citus_local_table
ROLLBACK;
-- show that drop table errors as expected
DROP TABLE citus_local_table;

View File

@ -999,7 +999,7 @@ SELECT value, count(*) FROM trigger_table GROUP BY value ORDER BY value;
(1 row)
ALTER TABLE trigger_table DISABLE TRIGGER ALL;
ERROR: triggers are only supported for citus local tables
ERROR: triggers are only supported for local tables added to metadata
INSERT INTO trigger_table VALUES (1, 'trigger disabled');
SELECT value, count(*) FROM trigger_table GROUP BY value ORDER BY value;
value | count
@ -1008,7 +1008,7 @@ SELECT value, count(*) FROM trigger_table GROUP BY value ORDER BY value;
(1 row)
ALTER TABLE trigger_table ENABLE TRIGGER ALL;
ERROR: triggers are only supported for citus local tables
ERROR: triggers are only supported for local tables added to metadata
INSERT INTO trigger_table VALUES (1, 'trigger disabled');
SELECT value, count(*) FROM trigger_table GROUP BY value ORDER BY value;
value | count

View File

@ -82,7 +82,7 @@ FROM
pg_dist_shard
WHERE 'multi_append_table_to_shard_right_reference_hash'::regclass::oid = logicalrelid;
ERROR: cannot append to shardId 230001
DETAIL: We currently don't support appending to shards in hash-partitioned, reference and citus local tables
DETAIL: We currently don't support appending to shards in hash-partitioned, reference and local tables
-- Clean up after test
SELECT master_apply_delete_command('DELETE FROM multi_append_table_to_shard_left');
master_apply_delete_command

View File

@ -98,6 +98,11 @@ SELECT run_command_on_placements('table_option_2',$cmd$
SELECT ROW(chunk_row_count, stripe_row_count, compression, compression_level) FROM columnar.options WHERE regclass = '%s'::regclass;
$cmd$);
-- verify undistribute works
SELECT undistribute_table('table_option');
SELECT * FROM pg_dist_partition WHERE logicalrelid = 'table_option'::regclass;
SELECT compression FROM columnar.options WHERE regclass = 'table_option'::regclass;
DROP TABLE table_option, table_option_2;
-- verify settings get to all placements when there are multiple replica's
@ -193,6 +198,11 @@ SELECT run_command_on_placements('table_option_2',$cmd$
SELECT ROW(chunk_row_count, stripe_row_count, compression, compression_level) FROM columnar.options WHERE regclass = '%s'::regclass;
$cmd$);
-- verify undistribute works
SELECT undistribute_table('table_option');
SELECT * FROM pg_dist_partition WHERE logicalrelid = 'table_option'::regclass;
SELECT compression FROM columnar.options WHERE regclass = 'table_option'::regclass;
DROP TABLE table_option, table_option_2;
-- test options on a reference table
@ -285,7 +295,113 @@ SELECT run_command_on_placements('table_option_reference_2',$cmd$
SELECT ROW(chunk_row_count, stripe_row_count, compression, compression_level) FROM columnar.options WHERE regclass = '%s'::regclass;
$cmd$);
-- verify undistribute works
SELECT undistribute_table('table_option_reference');
SELECT * FROM pg_dist_partition WHERE logicalrelid = 'table_option_reference'::regclass;
SELECT compression FROM columnar.options WHERE regclass = 'table_option_reference'::regclass;
DROP TABLE table_option_reference, table_option_reference_2;
SET citus.shard_replication_factor TO 1;
-- test options on a citus local table
SELECT 1 FROM master_add_node('localhost', :master_port, groupId => 0);
CREATE TABLE table_option_citus_local (a int, b text) USING columnar;
SELECT create_citus_local_table('table_option_citus_local');
-- setting: compression
-- get baseline for setting
SELECT run_command_on_placements('table_option_citus_local',$cmd$
SELECT compression FROM columnar.options WHERE regclass = '%s'::regclass;
$cmd$);
-- change setting
SELECT alter_columnar_table_set('table_option_citus_local', compression => 'pglz');
-- verify setting
SELECT run_command_on_placements('table_option_citus_local',$cmd$
SELECT compression FROM columnar.options WHERE regclass = '%s'::regclass;
$cmd$);
-- reset setting
SELECT alter_columnar_table_reset('table_option_citus_local', compression => true);
-- verify setting
SELECT run_command_on_placements('table_option_citus_local',$cmd$
SELECT compression FROM columnar.options WHERE regclass = '%s'::regclass;
$cmd$);
-- setting: compression_level
-- get baseline for setting
SELECT run_command_on_placements('table_option_citus_local',$cmd$
SELECT compression_level FROM columnar.options WHERE regclass = '%s'::regclass;
$cmd$);
-- change setting
SELECT alter_columnar_table_set('table_option_citus_local', compression_level => 11);
-- verify setting
SELECT run_command_on_placements('table_option_citus_local',$cmd$
SELECT compression_level FROM columnar.options WHERE regclass = '%s'::regclass;
$cmd$);
-- reset setting
SELECT alter_columnar_table_reset('table_option_citus_local', compression_level => true);
-- verify setting
SELECT run_command_on_placements('table_option_citus_local',$cmd$
SELECT compression_level FROM columnar.options WHERE regclass = '%s'::regclass;
$cmd$);
-- setting: chunk_row_count
-- get baseline for setting
SELECT run_command_on_placements('table_option_citus_local',$cmd$
SELECT chunk_row_count FROM columnar.options WHERE regclass = '%s'::regclass;
$cmd$);
-- change setting
SELECT alter_columnar_table_set('table_option_citus_local', chunk_row_count => 100);
-- verify setting
SELECT run_command_on_placements('table_option_citus_local',$cmd$
SELECT chunk_row_count FROM columnar.options WHERE regclass = '%s'::regclass;
$cmd$);
-- reset setting
SELECT alter_columnar_table_reset('table_option_citus_local', chunk_row_count => true);
-- verify setting
SELECT run_command_on_placements('table_option_citus_local',$cmd$
SELECT chunk_row_count FROM columnar.options WHERE regclass = '%s'::regclass;
$cmd$);
-- setting: stripe_row_count
-- get baseline for setting
SELECT run_command_on_placements('table_option_citus_local',$cmd$
SELECT stripe_row_count FROM columnar.options WHERE regclass = '%s'::regclass;
$cmd$);
-- change setting
SELECT alter_columnar_table_set('table_option_citus_local', stripe_row_count => 100);
-- verify setting
SELECT run_command_on_placements('table_option_citus_local',$cmd$
SELECT stripe_row_count FROM columnar.options WHERE regclass = '%s'::regclass;
$cmd$);
-- reset setting
SELECT alter_columnar_table_reset('table_option_citus_local', stripe_row_count => true);
-- verify setting
SELECT run_command_on_placements('table_option_citus_local',$cmd$
SELECT stripe_row_count FROM columnar.options WHERE regclass = '%s'::regclass;
$cmd$);
-- verify settings are propagated when creating a table
CREATE TABLE table_option_citus_local_2 (a int, b text) USING columnar;
SELECT alter_columnar_table_set('table_option_citus_local_2',
chunk_row_count => 100,
stripe_row_count => 1000,
compression => 'pglz',
compression_level => 9);
SELECT create_citus_local_table('table_option_citus_local_2');
-- verify settings on placements
SELECT run_command_on_placements('table_option_citus_local_2',$cmd$
SELECT ROW(chunk_row_count, stripe_row_count, compression, compression_level) FROM columnar.options WHERE regclass = '%s'::regclass;
$cmd$);
-- verify undistribute works
SELECT undistribute_table('table_option_citus_local');
SELECT * FROM pg_dist_partition WHERE logicalrelid = 'table_option_citus_local'::regclass;
SELECT compression FROM columnar.options WHERE regclass = 'table_option_citus_local'::regclass;
DROP TABLE table_option_citus_local, table_option_citus_local_2;
SELECT 1 FROM master_remove_node('localhost', :master_port);
SET client_min_messages TO WARNING;
DROP SCHEMA columnar_citus_integration CASCADE;