mirror of https://github.com/citusdata/citus.git
Use current user to sync metadata
Before this commit, we always synced the metadata with superuser. However, that creates various edge cases such as visibility errors or self distributed deadlocks or complicates user access checks. Instead, with this commit, we use the current user to sync the metadata. Note that, `start_metadata_sync_to_node` still requires super user because accessing certain metadata (like pg_dist_node) always require superuser (e.g., the current user should be a superuser). However, metadata syncing operations regarding the distributed tables can now be done with regular users, as long as the user is the owner of the table. A table owner can still insert non-sense metadata, however it'd only affect its own table. So, we cannot do anything about that.pull/5105/head
parent
b3cc9d63cb
commit
2c349e6dfd
|
@ -207,6 +207,7 @@ static ScanKeyData DistObjectScanKey[3];
|
||||||
|
|
||||||
/* local function forward declarations */
|
/* local function forward declarations */
|
||||||
static bool IsCitusTableViaCatalog(Oid relationId);
|
static bool IsCitusTableViaCatalog(Oid relationId);
|
||||||
|
static HeapTuple PgDistPartitionTupleViaCatalog(Oid relationId);
|
||||||
static ShardIdCacheEntry * LookupShardIdCacheEntry(int64 shardId);
|
static ShardIdCacheEntry * LookupShardIdCacheEntry(int64 shardId);
|
||||||
static CitusTableCacheEntry * BuildCitusTableCacheEntry(Oid relationId);
|
static CitusTableCacheEntry * BuildCitusTableCacheEntry(Oid relationId);
|
||||||
static void BuildCachedShardList(CitusTableCacheEntry *cacheEntry);
|
static void BuildCachedShardList(CitusTableCacheEntry *cacheEntry);
|
||||||
|
@ -236,13 +237,9 @@ static void InvalidateLocalGroupIdRelationCacheCallback(Datum argument, Oid rela
|
||||||
static void CitusTableCacheEntryReleaseCallback(ResourceReleasePhase phase, bool isCommit,
|
static void CitusTableCacheEntryReleaseCallback(ResourceReleasePhase phase, bool isCommit,
|
||||||
bool isTopLevel, void *arg);
|
bool isTopLevel, void *arg);
|
||||||
static HeapTuple LookupDistPartitionTuple(Relation pgDistPartition, Oid relationId);
|
static HeapTuple LookupDistPartitionTuple(Relation pgDistPartition, Oid relationId);
|
||||||
static List * LookupDistShardTuples(Oid relationId);
|
|
||||||
static void GetPartitionTypeInputInfo(char *partitionKeyString, char partitionMethod,
|
static void GetPartitionTypeInputInfo(char *partitionKeyString, char partitionMethod,
|
||||||
Oid *columnTypeId, int32 *columnTypeMod,
|
Oid *columnTypeId, int32 *columnTypeMod,
|
||||||
Oid *intervalTypeId, int32 *intervalTypeMod);
|
Oid *intervalTypeId, int32 *intervalTypeMod);
|
||||||
static ShardInterval * TupleToShardInterval(HeapTuple heapTuple,
|
|
||||||
TupleDesc tupleDescriptor, Oid intervalTypeId,
|
|
||||||
int32 intervalTypeMod);
|
|
||||||
static void CachedNamespaceLookup(const char *nspname, Oid *cachedOid);
|
static void CachedNamespaceLookup(const char *nspname, Oid *cachedOid);
|
||||||
static void CachedRelationLookup(const char *relationName, Oid *cachedOid);
|
static void CachedRelationLookup(const char *relationName, Oid *cachedOid);
|
||||||
static void CachedRelationNamespaceLookup(const char *relationName, Oid relnamespace,
|
static void CachedRelationNamespaceLookup(const char *relationName, Oid relnamespace,
|
||||||
|
@ -427,6 +424,66 @@ IsCitusTable(Oid relationId)
|
||||||
*/
|
*/
|
||||||
static bool
|
static bool
|
||||||
IsCitusTableViaCatalog(Oid relationId)
|
IsCitusTableViaCatalog(Oid relationId)
|
||||||
|
{
|
||||||
|
HeapTuple partitionTuple = PgDistPartitionTupleViaCatalog(relationId);
|
||||||
|
|
||||||
|
bool heapTupleIsValid = HeapTupleIsValid(partitionTuple);
|
||||||
|
|
||||||
|
if (heapTupleIsValid)
|
||||||
|
{
|
||||||
|
heap_freetuple(partitionTuple);
|
||||||
|
}
|
||||||
|
return heapTupleIsValid;
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/*
|
||||||
|
* PartitionMethodViaCatalog gets a relationId and returns the partition
|
||||||
|
* method column from pg_dist_partition via reading from catalog.
|
||||||
|
*/
|
||||||
|
char
|
||||||
|
PartitionMethodViaCatalog(Oid relationId)
|
||||||
|
{
|
||||||
|
HeapTuple partitionTuple = PgDistPartitionTupleViaCatalog(relationId);
|
||||||
|
if (!HeapTupleIsValid(partitionTuple))
|
||||||
|
{
|
||||||
|
return DISTRIBUTE_BY_INVALID;
|
||||||
|
}
|
||||||
|
|
||||||
|
Datum datumArray[Natts_pg_dist_partition];
|
||||||
|
bool isNullArray[Natts_pg_dist_partition];
|
||||||
|
|
||||||
|
Relation pgDistPartition = table_open(DistPartitionRelationId(), AccessShareLock);
|
||||||
|
|
||||||
|
TupleDesc tupleDescriptor = RelationGetDescr(pgDistPartition);
|
||||||
|
heap_deform_tuple(partitionTuple, tupleDescriptor, datumArray, isNullArray);
|
||||||
|
|
||||||
|
if (isNullArray[Anum_pg_dist_partition_partmethod - 1])
|
||||||
|
{
|
||||||
|
/* partition method cannot be NULL, still let's make sure */
|
||||||
|
heap_freetuple(partitionTuple);
|
||||||
|
table_close(pgDistPartition, NoLock);
|
||||||
|
return DISTRIBUTE_BY_INVALID;
|
||||||
|
}
|
||||||
|
|
||||||
|
Datum partitionMethodDatum = datumArray[Anum_pg_dist_partition_partmethod - 1];
|
||||||
|
char partitionMethodChar = DatumGetChar(partitionMethodDatum);
|
||||||
|
|
||||||
|
heap_freetuple(partitionTuple);
|
||||||
|
table_close(pgDistPartition, NoLock);
|
||||||
|
|
||||||
|
return partitionMethodChar;
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/*
|
||||||
|
* PgDistPartitionTupleViaCatalog is a helper function that searches
|
||||||
|
* pg_dist_partition for the given relationId. The caller is responsible
|
||||||
|
* for ensuring that the returned heap tuple is valid before accessing
|
||||||
|
* its fields.
|
||||||
|
*/
|
||||||
|
static HeapTuple
|
||||||
|
PgDistPartitionTupleViaCatalog(Oid relationId)
|
||||||
{
|
{
|
||||||
const int scanKeyCount = 1;
|
const int scanKeyCount = 1;
|
||||||
ScanKeyData scanKey[1];
|
ScanKeyData scanKey[1];
|
||||||
|
@ -442,10 +499,17 @@ IsCitusTableViaCatalog(Oid relationId)
|
||||||
indexOK, NULL, scanKeyCount, scanKey);
|
indexOK, NULL, scanKeyCount, scanKey);
|
||||||
|
|
||||||
HeapTuple partitionTuple = systable_getnext(scanDescriptor);
|
HeapTuple partitionTuple = systable_getnext(scanDescriptor);
|
||||||
|
|
||||||
|
if (HeapTupleIsValid(partitionTuple))
|
||||||
|
{
|
||||||
|
/* callers should have the tuple in their memory contexts */
|
||||||
|
partitionTuple = heap_copytuple(partitionTuple);
|
||||||
|
}
|
||||||
|
|
||||||
systable_endscan(scanDescriptor);
|
systable_endscan(scanDescriptor);
|
||||||
table_close(pgDistPartition, AccessShareLock);
|
table_close(pgDistPartition, AccessShareLock);
|
||||||
|
|
||||||
return HeapTupleIsValid(partitionTuple);
|
return partitionTuple;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
|
@ -1504,7 +1568,7 @@ BuildCachedShardList(CitusTableCacheEntry *cacheEntry)
|
||||||
cacheEntry->shardIntervalArrayLength++;
|
cacheEntry->shardIntervalArrayLength++;
|
||||||
|
|
||||||
/* build list of shard placements */
|
/* build list of shard placements */
|
||||||
List *placementList = BuildShardPlacementList(shardInterval);
|
List *placementList = BuildShardPlacementList(shardId);
|
||||||
int numberOfPlacements = list_length(placementList);
|
int numberOfPlacements = list_length(placementList);
|
||||||
|
|
||||||
/* and copy that list into the cache entry */
|
/* and copy that list into the cache entry */
|
||||||
|
@ -3997,7 +4061,7 @@ LookupDistPartitionTuple(Relation pgDistPartition, Oid relationId)
|
||||||
* LookupDistShardTuples returns a list of all dist_shard tuples for the
|
* LookupDistShardTuples returns a list of all dist_shard tuples for the
|
||||||
* specified relation.
|
* specified relation.
|
||||||
*/
|
*/
|
||||||
static List *
|
List *
|
||||||
LookupDistShardTuples(Oid relationId)
|
LookupDistShardTuples(Oid relationId)
|
||||||
{
|
{
|
||||||
List *distShardTupleList = NIL;
|
List *distShardTupleList = NIL;
|
||||||
|
@ -4208,7 +4272,7 @@ GetIntervalTypeInfo(char partitionMethod, Var *partitionColumn,
|
||||||
* TupleToShardInterval transforms the specified dist_shard tuple into a new
|
* TupleToShardInterval transforms the specified dist_shard tuple into a new
|
||||||
* ShardInterval using the provided descriptor and partition type information.
|
* ShardInterval using the provided descriptor and partition type information.
|
||||||
*/
|
*/
|
||||||
static ShardInterval *
|
ShardInterval *
|
||||||
TupleToShardInterval(HeapTuple heapTuple, TupleDesc tupleDescriptor, Oid
|
TupleToShardInterval(HeapTuple heapTuple, TupleDesc tupleDescriptor, Oid
|
||||||
intervalTypeId,
|
intervalTypeId,
|
||||||
int32 intervalTypeMod)
|
int32 intervalTypeMod)
|
||||||
|
|
|
@ -21,17 +21,22 @@
|
||||||
#include "access/genam.h"
|
#include "access/genam.h"
|
||||||
#include "access/heapam.h"
|
#include "access/heapam.h"
|
||||||
#include "access/htup_details.h"
|
#include "access/htup_details.h"
|
||||||
|
#include "access/nbtree.h"
|
||||||
#include "access/sysattr.h"
|
#include "access/sysattr.h"
|
||||||
#include "access/xact.h"
|
#include "access/xact.h"
|
||||||
#include "catalog/dependency.h"
|
#include "catalog/dependency.h"
|
||||||
#include "catalog/indexing.h"
|
#include "catalog/indexing.h"
|
||||||
|
#include "catalog/pg_am.h"
|
||||||
#include "catalog/pg_attrdef.h"
|
#include "catalog/pg_attrdef.h"
|
||||||
#include "catalog/pg_depend.h"
|
#include "catalog/pg_depend.h"
|
||||||
#include "catalog/pg_foreign_server.h"
|
#include "catalog/pg_foreign_server.h"
|
||||||
#include "catalog/pg_namespace.h"
|
#include "catalog/pg_namespace.h"
|
||||||
#include "catalog/pg_type.h"
|
#include "catalog/pg_type.h"
|
||||||
#include "commands/async.h"
|
#include "commands/async.h"
|
||||||
|
#include "distributed/argutils.h"
|
||||||
|
#include "distributed/backend_data.h"
|
||||||
#include "distributed/citus_ruleutils.h"
|
#include "distributed/citus_ruleutils.h"
|
||||||
|
#include "distributed/colocation_utils.h"
|
||||||
#include "distributed/commands.h"
|
#include "distributed/commands.h"
|
||||||
#include "distributed/deparser.h"
|
#include "distributed/deparser.h"
|
||||||
#include "distributed/distribution_column.h"
|
#include "distributed/distribution_column.h"
|
||||||
|
@ -44,9 +49,13 @@
|
||||||
#include "distributed/metadata/distobject.h"
|
#include "distributed/metadata/distobject.h"
|
||||||
#include "distributed/multi_join_order.h"
|
#include "distributed/multi_join_order.h"
|
||||||
#include "distributed/multi_partitioning_utils.h"
|
#include "distributed/multi_partitioning_utils.h"
|
||||||
|
#include "distributed/multi_physical_planner.h"
|
||||||
#include "distributed/pg_dist_node.h"
|
#include "distributed/pg_dist_node.h"
|
||||||
|
#include "distributed/pg_dist_shard.h"
|
||||||
#include "distributed/remote_commands.h"
|
#include "distributed/remote_commands.h"
|
||||||
|
#include "distributed/resource_lock.h"
|
||||||
#include "distributed/worker_manager.h"
|
#include "distributed/worker_manager.h"
|
||||||
|
#include "distributed/worker_protocol.h"
|
||||||
#include "distributed/worker_transaction.h"
|
#include "distributed/worker_transaction.h"
|
||||||
#include "distributed/version_compat.h"
|
#include "distributed/version_compat.h"
|
||||||
#include "executor/spi.h"
|
#include "executor/spi.h"
|
||||||
|
@ -65,6 +74,10 @@
|
||||||
#include "utils/syscache.h"
|
#include "utils/syscache.h"
|
||||||
|
|
||||||
|
|
||||||
|
/* managed via a GUC */
|
||||||
|
char *EnableManualMetadataChangesForUser = "";
|
||||||
|
|
||||||
|
|
||||||
static List * GetDistributedTableDDLEvents(Oid relationId);
|
static List * GetDistributedTableDDLEvents(Oid relationId);
|
||||||
static char * LocalGroupIdUpdateCommand(int32 groupId);
|
static char * LocalGroupIdUpdateCommand(int32 groupId);
|
||||||
static void UpdateDistNodeBoolAttr(const char *nodeName, int32 nodePort,
|
static void UpdateDistNodeBoolAttr(const char *nodeName, int32 nodePort,
|
||||||
|
@ -88,10 +101,34 @@ static char * GenerateSetRoleQuery(Oid roleOid);
|
||||||
static void MetadataSyncSigTermHandler(SIGNAL_ARGS);
|
static void MetadataSyncSigTermHandler(SIGNAL_ARGS);
|
||||||
static void MetadataSyncSigAlrmHandler(SIGNAL_ARGS);
|
static void MetadataSyncSigAlrmHandler(SIGNAL_ARGS);
|
||||||
|
|
||||||
|
|
||||||
|
static bool ShouldSkipMetadataChecks(void);
|
||||||
|
static void EnsurePartitionMetadataIsSane(char distributionMethod, int colocationId,
|
||||||
|
char replicationModel);
|
||||||
|
static void EnsureCoordinatorInitiatedOperation(void);
|
||||||
|
static void EnsureShardMetadataIsSane(Oid relationId, int64 shardId, char storageType,
|
||||||
|
text *shardMinValue,
|
||||||
|
text *shardMaxValue);
|
||||||
|
static void EnsureShardPlacementMetadataIsSane(Oid relationId, int64 shardId,
|
||||||
|
int64 placementId, int32 shardState,
|
||||||
|
int64 shardLength, int32 groupId);
|
||||||
|
|
||||||
PG_FUNCTION_INFO_V1(start_metadata_sync_to_node);
|
PG_FUNCTION_INFO_V1(start_metadata_sync_to_node);
|
||||||
PG_FUNCTION_INFO_V1(stop_metadata_sync_to_node);
|
PG_FUNCTION_INFO_V1(stop_metadata_sync_to_node);
|
||||||
PG_FUNCTION_INFO_V1(worker_record_sequence_dependency);
|
PG_FUNCTION_INFO_V1(worker_record_sequence_dependency);
|
||||||
|
|
||||||
|
|
||||||
|
/*
|
||||||
|
* Functions to modify metadata. Normally modifying metadata requires
|
||||||
|
* superuser. However, these functions can be called with superusers
|
||||||
|
* or regular users as long as the regular user owns the input object.
|
||||||
|
*/
|
||||||
|
PG_FUNCTION_INFO_V1(citus_internal_add_partition_metadata);
|
||||||
|
PG_FUNCTION_INFO_V1(citus_internal_add_shard_metadata);
|
||||||
|
PG_FUNCTION_INFO_V1(citus_internal_add_placement_metadata);
|
||||||
|
PG_FUNCTION_INFO_V1(citus_internal_update_placement_metadata);
|
||||||
|
|
||||||
|
|
||||||
static bool got_SIGTERM = false;
|
static bool got_SIGTERM = false;
|
||||||
static bool got_SIGALRM = false;
|
static bool got_SIGALRM = false;
|
||||||
|
|
||||||
|
@ -165,6 +202,7 @@ StartMetadataSyncToNode(const char *nodeNameString, int32 nodePort)
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
UseCoordinatedTransaction();
|
||||||
MarkNodeHasMetadata(nodeNameString, nodePort, true);
|
MarkNodeHasMetadata(nodeNameString, nodePort, true);
|
||||||
|
|
||||||
if (!NodeIsPrimary(workerNode))
|
if (!NodeIsPrimary(workerNode))
|
||||||
|
@ -310,7 +348,7 @@ ShouldSyncTableMetadata(Oid relationId)
|
||||||
static bool
|
static bool
|
||||||
SyncMetadataSnapshotToNode(WorkerNode *workerNode, bool raiseOnError)
|
SyncMetadataSnapshotToNode(WorkerNode *workerNode, bool raiseOnError)
|
||||||
{
|
{
|
||||||
char *extensionOwner = CitusExtensionOwnerName();
|
char *currentUser = CurrentUserName();
|
||||||
|
|
||||||
/* generate and add the local group id's update query */
|
/* generate and add the local group id's update query */
|
||||||
char *localGroupIdUpdateCommand = LocalGroupIdUpdateCommand(workerNode->groupId);
|
char *localGroupIdUpdateCommand = LocalGroupIdUpdateCommand(workerNode->groupId);
|
||||||
|
@ -337,7 +375,7 @@ SyncMetadataSnapshotToNode(WorkerNode *workerNode, bool raiseOnError)
|
||||||
{
|
{
|
||||||
SendCommandListToWorkerInSingleTransaction(workerNode->workerName,
|
SendCommandListToWorkerInSingleTransaction(workerNode->workerName,
|
||||||
workerNode->workerPort,
|
workerNode->workerPort,
|
||||||
extensionOwner,
|
currentUser,
|
||||||
recreateMetadataSnapshotCommandList);
|
recreateMetadataSnapshotCommandList);
|
||||||
return true;
|
return true;
|
||||||
}
|
}
|
||||||
|
@ -346,7 +384,7 @@ SyncMetadataSnapshotToNode(WorkerNode *workerNode, bool raiseOnError)
|
||||||
bool success =
|
bool success =
|
||||||
SendOptionalCommandListToWorkerInTransaction(workerNode->workerName,
|
SendOptionalCommandListToWorkerInTransaction(workerNode->workerName,
|
||||||
workerNode->workerPort,
|
workerNode->workerPort,
|
||||||
extensionOwner,
|
currentUser,
|
||||||
recreateMetadataSnapshotCommandList);
|
recreateMetadataSnapshotCommandList);
|
||||||
return success;
|
return success;
|
||||||
}
|
}
|
||||||
|
@ -360,7 +398,7 @@ SyncMetadataSnapshotToNode(WorkerNode *workerNode, bool raiseOnError)
|
||||||
static void
|
static void
|
||||||
DropMetadataSnapshotOnNode(WorkerNode *workerNode)
|
DropMetadataSnapshotOnNode(WorkerNode *workerNode)
|
||||||
{
|
{
|
||||||
char *extensionOwner = CitusExtensionOwnerName();
|
char *userName = CurrentUserName();
|
||||||
|
|
||||||
/* generate the queries which drop the metadata */
|
/* generate the queries which drop the metadata */
|
||||||
List *dropMetadataCommandList = MetadataDropCommands();
|
List *dropMetadataCommandList = MetadataDropCommands();
|
||||||
|
@ -370,7 +408,7 @@ DropMetadataSnapshotOnNode(WorkerNode *workerNode)
|
||||||
|
|
||||||
SendOptionalCommandListToWorkerInTransaction(workerNode->workerName,
|
SendOptionalCommandListToWorkerInTransaction(workerNode->workerName,
|
||||||
workerNode->workerPort,
|
workerNode->workerPort,
|
||||||
extensionOwner,
|
userName,
|
||||||
dropMetadataCommandList);
|
dropMetadataCommandList);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -728,28 +766,26 @@ DistributionCreateCommand(CitusTableCacheEntry *cacheEntry)
|
||||||
generate_qualified_relation_name(relationId);
|
generate_qualified_relation_name(relationId);
|
||||||
uint32 colocationId = cacheEntry->colocationId;
|
uint32 colocationId = cacheEntry->colocationId;
|
||||||
char replicationModel = cacheEntry->replicationModel;
|
char replicationModel = cacheEntry->replicationModel;
|
||||||
StringInfo tablePartitionKeyString = makeStringInfo();
|
StringInfo tablePartitionKeyNameString = makeStringInfo();
|
||||||
|
|
||||||
if (IsCitusTableTypeCacheEntry(cacheEntry, CITUS_TABLE_WITH_NO_DIST_KEY))
|
if (IsCitusTableTypeCacheEntry(cacheEntry, CITUS_TABLE_WITH_NO_DIST_KEY))
|
||||||
{
|
{
|
||||||
appendStringInfo(tablePartitionKeyString, "NULL");
|
appendStringInfo(tablePartitionKeyNameString, "NULL");
|
||||||
}
|
}
|
||||||
else
|
else
|
||||||
{
|
{
|
||||||
char *partitionKeyColumnName = ColumnToColumnName(relationId, partitionKeyString);
|
char *partitionKeyColumnName =
|
||||||
appendStringInfo(tablePartitionKeyString, "column_name_to_column(%s,%s)",
|
ColumnToColumnName(relationId, partitionKeyString);
|
||||||
quote_literal_cstr(qualifiedRelationName),
|
appendStringInfo(tablePartitionKeyNameString, "%s",
|
||||||
quote_literal_cstr(partitionKeyColumnName));
|
quote_literal_cstr(partitionKeyColumnName));
|
||||||
}
|
}
|
||||||
|
|
||||||
appendStringInfo(insertDistributionCommand,
|
appendStringInfo(insertDistributionCommand,
|
||||||
"INSERT INTO pg_dist_partition "
|
"SELECT citus_internal_add_partition_metadata "
|
||||||
"(logicalrelid, partmethod, partkey, colocationid, repmodel) "
|
|
||||||
"VALUES "
|
|
||||||
"(%s::regclass, '%c', %s, %d, '%c')",
|
"(%s::regclass, '%c', %s, %d, '%c')",
|
||||||
quote_literal_cstr(qualifiedRelationName),
|
quote_literal_cstr(qualifiedRelationName),
|
||||||
distributionMethod,
|
distributionMethod,
|
||||||
tablePartitionKeyString->data,
|
tablePartitionKeyNameString->data,
|
||||||
colocationId,
|
colocationId,
|
||||||
replicationModel);
|
replicationModel);
|
||||||
|
|
||||||
|
@ -806,10 +842,7 @@ List *
|
||||||
ShardListInsertCommand(List *shardIntervalList)
|
ShardListInsertCommand(List *shardIntervalList)
|
||||||
{
|
{
|
||||||
List *commandList = NIL;
|
List *commandList = NIL;
|
||||||
StringInfo insertPlacementCommand = makeStringInfo();
|
|
||||||
StringInfo insertShardCommand = makeStringInfo();
|
|
||||||
int shardCount = list_length(shardIntervalList);
|
int shardCount = list_length(shardIntervalList);
|
||||||
int processedShardCount = 0;
|
|
||||||
|
|
||||||
/* if there are no shards, return empty list */
|
/* if there are no shards, return empty list */
|
||||||
if (shardCount == 0)
|
if (shardCount == 0)
|
||||||
|
@ -818,6 +851,11 @@ ShardListInsertCommand(List *shardIntervalList)
|
||||||
}
|
}
|
||||||
|
|
||||||
/* add placements to insertPlacementCommand */
|
/* add placements to insertPlacementCommand */
|
||||||
|
StringInfo insertPlacementCommand = makeStringInfo();
|
||||||
|
appendStringInfo(insertPlacementCommand,
|
||||||
|
"WITH placement_data(shardid, shardstate, "
|
||||||
|
"shardlength, groupid, placementid) AS (VALUES ");
|
||||||
|
|
||||||
ShardInterval *shardInterval = NULL;
|
ShardInterval *shardInterval = NULL;
|
||||||
foreach_ptr(shardInterval, shardIntervalList)
|
foreach_ptr(shardInterval, shardIntervalList)
|
||||||
{
|
{
|
||||||
|
@ -827,41 +865,39 @@ ShardListInsertCommand(List *shardIntervalList)
|
||||||
ShardPlacement *placement = NULL;
|
ShardPlacement *placement = NULL;
|
||||||
foreach_ptr(placement, shardPlacementList)
|
foreach_ptr(placement, shardPlacementList)
|
||||||
{
|
{
|
||||||
if (insertPlacementCommand->len == 0)
|
|
||||||
{
|
|
||||||
/* generate the shard placement query without any values yet */
|
|
||||||
appendStringInfo(insertPlacementCommand,
|
appendStringInfo(insertPlacementCommand,
|
||||||
"INSERT INTO pg_dist_placement "
|
"(%ld, %d, %ld, %d, %ld)",
|
||||||
"(shardid, shardstate, shardlength,"
|
|
||||||
" groupid, placementid) "
|
|
||||||
"VALUES ");
|
|
||||||
}
|
|
||||||
else
|
|
||||||
{
|
|
||||||
appendStringInfo(insertPlacementCommand, ",");
|
|
||||||
}
|
|
||||||
|
|
||||||
appendStringInfo(insertPlacementCommand,
|
|
||||||
"(" UINT64_FORMAT ", 1, " UINT64_FORMAT ", %d, "
|
|
||||||
UINT64_FORMAT ")",
|
|
||||||
shardId,
|
shardId,
|
||||||
|
placement->shardState,
|
||||||
placement->shardLength,
|
placement->shardLength,
|
||||||
placement->groupId,
|
placement->groupId,
|
||||||
placement->placementId);
|
placement->placementId);
|
||||||
|
|
||||||
|
if (!(llast(shardPlacementList) == placement &&
|
||||||
|
llast(shardIntervalList) == shardInterval))
|
||||||
|
{
|
||||||
|
/*
|
||||||
|
* As long as this is not the last placement of the last shard,
|
||||||
|
* append the comma.
|
||||||
|
*/
|
||||||
|
appendStringInfo(insertPlacementCommand, ", ");
|
||||||
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
/* add the command to the list that we'll return */
|
appendStringInfo(insertPlacementCommand, ") ");
|
||||||
commandList = lappend(commandList, insertPlacementCommand->data);
|
|
||||||
|
|
||||||
/* now, generate the shard query without any values yet */
|
appendStringInfo(insertPlacementCommand,
|
||||||
appendStringInfo(insertShardCommand,
|
"SELECT citus_internal_add_placement_metadata("
|
||||||
"INSERT INTO pg_dist_shard "
|
"shardid, shardstate, shardlength, groupid, placementid) "
|
||||||
"(logicalrelid, shardid, shardstorage,"
|
"FROM placement_data;");
|
||||||
" shardminvalue, shardmaxvalue) "
|
|
||||||
"VALUES ");
|
|
||||||
|
|
||||||
/* now add shards to insertShardCommand */
|
/* now add shards to insertShardCommand */
|
||||||
|
StringInfo insertShardCommand = makeStringInfo();
|
||||||
|
appendStringInfo(insertShardCommand,
|
||||||
|
"WITH shard_data(relationname, shardid, storagetype, "
|
||||||
|
"shardminvalue, shardmaxvalue) AS (VALUES ");
|
||||||
|
|
||||||
foreach_ptr(shardInterval, shardIntervalList)
|
foreach_ptr(shardInterval, shardIntervalList)
|
||||||
{
|
{
|
||||||
uint64 shardId = shardInterval->shardId;
|
uint64 shardId = shardInterval->shardId;
|
||||||
|
@ -892,22 +928,29 @@ ShardListInsertCommand(List *shardIntervalList)
|
||||||
}
|
}
|
||||||
|
|
||||||
appendStringInfo(insertShardCommand,
|
appendStringInfo(insertShardCommand,
|
||||||
"(%s::regclass, " UINT64_FORMAT ", '%c', %s, %s)",
|
"(%s::regclass, %ld, '%c'::\"char\", %s, %s)",
|
||||||
quote_literal_cstr(qualifiedRelationName),
|
quote_literal_cstr(qualifiedRelationName),
|
||||||
shardId,
|
shardId,
|
||||||
shardInterval->storageType,
|
shardInterval->storageType,
|
||||||
minHashToken->data,
|
minHashToken->data,
|
||||||
maxHashToken->data);
|
maxHashToken->data);
|
||||||
|
|
||||||
processedShardCount++;
|
if (llast(shardIntervalList) != shardInterval)
|
||||||
if (processedShardCount != shardCount)
|
|
||||||
{
|
{
|
||||||
appendStringInfo(insertShardCommand, ", ");
|
appendStringInfo(insertShardCommand, ", ");
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
/* finally add the command to the list that we'll return */
|
appendStringInfo(insertShardCommand, ") ");
|
||||||
|
|
||||||
|
appendStringInfo(insertShardCommand,
|
||||||
|
"SELECT citus_internal_add_shard_metadata(relationname, shardid, "
|
||||||
|
"storagetype, shardminvalue, shardmaxvalue) "
|
||||||
|
"FROM shard_data;");
|
||||||
|
|
||||||
|
/* first insert shards, than the placements */
|
||||||
commandList = lappend(commandList, insertShardCommand->data);
|
commandList = lappend(commandList, insertShardCommand->data);
|
||||||
|
commandList = lappend(commandList, insertPlacementCommand->data);
|
||||||
|
|
||||||
return commandList;
|
return commandList;
|
||||||
}
|
}
|
||||||
|
@ -1429,6 +1472,10 @@ worker_record_sequence_dependency(PG_FUNCTION_ARGS)
|
||||||
.objectSubId = columnForm->attnum
|
.objectSubId = columnForm->attnum
|
||||||
};
|
};
|
||||||
|
|
||||||
|
|
||||||
|
EnsureTableOwner(sequenceOid);
|
||||||
|
EnsureTableOwner(relationOid);
|
||||||
|
|
||||||
/* dependency from sequence to table */
|
/* dependency from sequence to table */
|
||||||
recordDependencyOn(&sequenceAddr, &relationAddr, DEPENDENCY_AUTO);
|
recordDependencyOn(&sequenceAddr, &relationAddr, DEPENDENCY_AUTO);
|
||||||
|
|
||||||
|
@ -2006,3 +2053,523 @@ ShouldInitiateMetadataSync(bool *lockFailure)
|
||||||
*lockFailure = false;
|
*lockFailure = false;
|
||||||
return shouldSyncMetadata;
|
return shouldSyncMetadata;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/*
|
||||||
|
* citus_internal_add_partition_metadata is an internal UDF to
|
||||||
|
* add a row to pg_dist_partition.
|
||||||
|
*/
|
||||||
|
Datum
|
||||||
|
citus_internal_add_partition_metadata(PG_FUNCTION_ARGS)
|
||||||
|
{
|
||||||
|
PG_ENSURE_ARGNOTNULL(0, "relation");
|
||||||
|
Oid relationId = PG_GETARG_OID(0);
|
||||||
|
|
||||||
|
PG_ENSURE_ARGNOTNULL(1, "distribution method");
|
||||||
|
char distributionMethod = PG_GETARG_CHAR(1);
|
||||||
|
|
||||||
|
PG_ENSURE_ARGNOTNULL(3, "Colocation ID");
|
||||||
|
int colocationId = PG_GETARG_INT32(3);
|
||||||
|
|
||||||
|
PG_ENSURE_ARGNOTNULL(4, "replication model");
|
||||||
|
char replicationModel = PG_GETARG_CHAR(4);
|
||||||
|
|
||||||
|
text *distributionColumnText = NULL;
|
||||||
|
char *distributionColumnString = NULL;
|
||||||
|
Var *distributionColumnVar = NULL;
|
||||||
|
|
||||||
|
/* only owner of the table (or superuser) is allowed to add the Citus metadata */
|
||||||
|
EnsureTableOwner(relationId);
|
||||||
|
|
||||||
|
/* we want to serialize all the metadata changes to this table */
|
||||||
|
LockRelationOid(relationId, ShareUpdateExclusiveLock);
|
||||||
|
|
||||||
|
if (!PG_ARGISNULL(2))
|
||||||
|
{
|
||||||
|
distributionColumnText = PG_GETARG_TEXT_P(2);
|
||||||
|
distributionColumnString = text_to_cstring(distributionColumnText);
|
||||||
|
|
||||||
|
Relation relation = relation_open(relationId, AccessShareLock);
|
||||||
|
distributionColumnVar =
|
||||||
|
BuildDistributionKeyFromColumnName(relation, distributionColumnString);
|
||||||
|
Assert(distributionColumnVar != NULL);
|
||||||
|
|
||||||
|
relation_close(relation, NoLock);
|
||||||
|
}
|
||||||
|
|
||||||
|
if (!ShouldSkipMetadataChecks())
|
||||||
|
{
|
||||||
|
/* this UDF is not allowed allowed for executing as a separate command */
|
||||||
|
EnsureCoordinatorInitiatedOperation();
|
||||||
|
|
||||||
|
if (distributionMethod == DISTRIBUTE_BY_NONE && distributionColumnVar != NULL)
|
||||||
|
{
|
||||||
|
ereport(ERROR, (errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
|
||||||
|
errmsg("Reference or local tables cannot have "
|
||||||
|
"distribution columns")));
|
||||||
|
}
|
||||||
|
else if (distributionMethod != DISTRIBUTE_BY_NONE &&
|
||||||
|
distributionColumnVar == NULL)
|
||||||
|
{
|
||||||
|
ereport(ERROR, (errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
|
||||||
|
errmsg("Distribution column cannot be NULL for "
|
||||||
|
"relation \"%s\"", get_rel_name(relationId))));
|
||||||
|
}
|
||||||
|
|
||||||
|
/*
|
||||||
|
* Even if the table owner is a malicious user and the partition
|
||||||
|
* metadata is not sane, the user can only affect its own tables.
|
||||||
|
* Given that the user is owner of the table, we should allow.
|
||||||
|
*/
|
||||||
|
EnsurePartitionMetadataIsSane(distributionMethod, colocationId,
|
||||||
|
replicationModel);
|
||||||
|
}
|
||||||
|
|
||||||
|
InsertIntoPgDistPartition(relationId, distributionMethod, distributionColumnVar,
|
||||||
|
colocationId, replicationModel);
|
||||||
|
|
||||||
|
PG_RETURN_VOID();
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/*
|
||||||
|
* EnsurePartitionMetadataIsSane ensures that the input values are safe
|
||||||
|
* for inserting into pg_dist_partition metadata.
|
||||||
|
*/
|
||||||
|
static void
|
||||||
|
EnsurePartitionMetadataIsSane(char distributionMethod, int colocationId,
|
||||||
|
char replicationModel)
|
||||||
|
{
|
||||||
|
if (!(distributionMethod == DISTRIBUTE_BY_HASH ||
|
||||||
|
distributionMethod == DISTRIBUTE_BY_NONE))
|
||||||
|
{
|
||||||
|
ereport(ERROR, (errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
|
||||||
|
errmsg("Metadata syncing is only allowed for hash, reference "
|
||||||
|
"and local tables:%c", distributionMethod)));
|
||||||
|
}
|
||||||
|
|
||||||
|
if (colocationId < INVALID_COLOCATION_ID)
|
||||||
|
{
|
||||||
|
ereport(ERROR, (errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
|
||||||
|
errmsg("Metadata syncing is only allowed for valid "
|
||||||
|
"colocation id values.")));
|
||||||
|
}
|
||||||
|
|
||||||
|
if (!(replicationModel == REPLICATION_MODEL_2PC ||
|
||||||
|
replicationModel == REPLICATION_MODEL_STREAMING ||
|
||||||
|
replicationModel == REPLICATION_MODEL_COORDINATOR))
|
||||||
|
{
|
||||||
|
ereport(ERROR, (errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
|
||||||
|
errmsg("Metadata syncing is only allowed for "
|
||||||
|
"known replication models.")));
|
||||||
|
}
|
||||||
|
|
||||||
|
if (distributionMethod == DISTRIBUTE_BY_HASH &&
|
||||||
|
replicationModel != REPLICATION_MODEL_STREAMING)
|
||||||
|
{
|
||||||
|
ereport(ERROR, (errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
|
||||||
|
errmsg("Hash distributed tables can only have '%c' "
|
||||||
|
"as the replication model.",
|
||||||
|
REPLICATION_MODEL_STREAMING)));
|
||||||
|
}
|
||||||
|
|
||||||
|
if (distributionMethod == DISTRIBUTE_BY_NONE &&
|
||||||
|
!(replicationModel == REPLICATION_MODEL_STREAMING ||
|
||||||
|
replicationModel == REPLICATION_MODEL_2PC))
|
||||||
|
{
|
||||||
|
ereport(ERROR, (errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
|
||||||
|
errmsg("Local or references tables can only have '%c' or '%c' "
|
||||||
|
"as the replication model.",
|
||||||
|
REPLICATION_MODEL_STREAMING, REPLICATION_MODEL_2PC)));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/*
|
||||||
|
* citus_internal_add_shard_metadata is an internal UDF to
|
||||||
|
* add a row to pg_dist_shard.
|
||||||
|
*/
|
||||||
|
Datum
|
||||||
|
citus_internal_add_shard_metadata(PG_FUNCTION_ARGS)
|
||||||
|
{
|
||||||
|
PG_ENSURE_ARGNOTNULL(0, "relation");
|
||||||
|
Oid relationId = PG_GETARG_OID(0);
|
||||||
|
|
||||||
|
PG_ENSURE_ARGNOTNULL(1, "shard id");
|
||||||
|
int64 shardId = PG_GETARG_INT64(1);
|
||||||
|
|
||||||
|
PG_ENSURE_ARGNOTNULL(2, "storage type");
|
||||||
|
char storageType = PG_GETARG_CHAR(2);
|
||||||
|
|
||||||
|
text *shardMinValue = NULL;
|
||||||
|
if (!PG_ARGISNULL(3))
|
||||||
|
{
|
||||||
|
shardMinValue = PG_GETARG_TEXT_P(3);
|
||||||
|
}
|
||||||
|
|
||||||
|
text *shardMaxValue = NULL;
|
||||||
|
if (!PG_ARGISNULL(4))
|
||||||
|
{
|
||||||
|
shardMaxValue = PG_GETARG_TEXT_P(4);
|
||||||
|
}
|
||||||
|
|
||||||
|
/* only owner of the table (or superuser) is allowed to add the Citus metadata */
|
||||||
|
EnsureTableOwner(relationId);
|
||||||
|
|
||||||
|
/* we want to serialize all the metadata changes to this table */
|
||||||
|
LockRelationOid(relationId, ShareUpdateExclusiveLock);
|
||||||
|
|
||||||
|
if (!ShouldSkipMetadataChecks())
|
||||||
|
{
|
||||||
|
/* this UDF is not allowed allowed for executing as a separate command */
|
||||||
|
EnsureCoordinatorInitiatedOperation();
|
||||||
|
|
||||||
|
/*
|
||||||
|
* Even if the table owner is a malicious user and the shard metadata is
|
||||||
|
* not sane, the user can only affect its own tables. Given that the
|
||||||
|
* user is owner of the table, we should allow.
|
||||||
|
*/
|
||||||
|
EnsureShardMetadataIsSane(relationId, shardId, storageType, shardMinValue,
|
||||||
|
shardMaxValue);
|
||||||
|
}
|
||||||
|
|
||||||
|
InsertShardRow(relationId, shardId, storageType, shardMinValue, shardMaxValue);
|
||||||
|
|
||||||
|
PG_RETURN_VOID();
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/*
|
||||||
|
* EnsureCoordinatorInitiatedOperation is a helper function which ensures that
|
||||||
|
* the execution is initiated by the coordinator on a worker node.
|
||||||
|
*/
|
||||||
|
static void
|
||||||
|
EnsureCoordinatorInitiatedOperation(void)
|
||||||
|
{
|
||||||
|
/*
|
||||||
|
* We are restricting the operation to only MX workers with the local group id
|
||||||
|
* check. The other two checks are to ensure that the operation is initiated
|
||||||
|
* by the coordinator.
|
||||||
|
*/
|
||||||
|
if (!IsCitusInitiatedRemoteBackend() || !MyBackendIsInDisributedTransaction() ||
|
||||||
|
GetLocalGroupId() == COORDINATOR_GROUP_ID)
|
||||||
|
{
|
||||||
|
ereport(ERROR, (errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
|
||||||
|
errmsg("This is an internal function that only Citus "
|
||||||
|
"requires to use in a distributed transaction")));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/*
|
||||||
|
* EnsureShardMetadataIsSane ensures that the input values are safe
|
||||||
|
* for inserting into pg_dist_shard metadata.
|
||||||
|
*/
|
||||||
|
static void
|
||||||
|
EnsureShardMetadataIsSane(Oid relationId, int64 shardId, char storageType,
|
||||||
|
text *shardMinValue, text *shardMaxValue)
|
||||||
|
{
|
||||||
|
if (shardId <= INVALID_SHARD_ID)
|
||||||
|
{
|
||||||
|
ereport(ERROR, (errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
|
||||||
|
errmsg("Invalid shard id: %ld", shardId)));
|
||||||
|
}
|
||||||
|
|
||||||
|
if (!(storageType == SHARD_STORAGE_TABLE ||
|
||||||
|
storageType == SHARD_STORAGE_FOREIGN ||
|
||||||
|
storageType == SHARD_STORAGE_COLUMNAR))
|
||||||
|
{
|
||||||
|
ereport(ERROR, (errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
|
||||||
|
errmsg("Invalid shard storage type: %c", storageType)));
|
||||||
|
}
|
||||||
|
|
||||||
|
char partitionMethod = PartitionMethodViaCatalog(relationId);
|
||||||
|
if (partitionMethod == DISTRIBUTE_BY_INVALID)
|
||||||
|
{
|
||||||
|
/* connection from the coordinator operating on a shard */
|
||||||
|
ereport(ERROR, (errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
|
||||||
|
errmsg("The relation \"%s\" does not have a valid "
|
||||||
|
"entry in pg_dist_partition.",
|
||||||
|
get_rel_name(relationId))));
|
||||||
|
}
|
||||||
|
else if (!(partitionMethod == DISTRIBUTE_BY_HASH ||
|
||||||
|
partitionMethod == DISTRIBUTE_BY_NONE))
|
||||||
|
{
|
||||||
|
/* connection from the coordinator operating on a shard */
|
||||||
|
ereport(ERROR, (errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
|
||||||
|
errmsg("Metadata syncing is only allowed for hash, "
|
||||||
|
"reference and local tables: %c", partitionMethod)));
|
||||||
|
}
|
||||||
|
|
||||||
|
List *distShardTupleList = LookupDistShardTuples(relationId);
|
||||||
|
if (partitionMethod == DISTRIBUTE_BY_NONE)
|
||||||
|
{
|
||||||
|
if (shardMinValue != NULL || shardMaxValue != NULL)
|
||||||
|
{
|
||||||
|
char *relationName = get_rel_name(relationId);
|
||||||
|
ereport(ERROR, (errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
|
||||||
|
errmsg("Shards of reference or local table \"%s\" should "
|
||||||
|
"have NULL shard ranges", relationName)));
|
||||||
|
}
|
||||||
|
else if (list_length(distShardTupleList) != 0)
|
||||||
|
{
|
||||||
|
char *relationName = get_rel_name(relationId);
|
||||||
|
ereport(ERROR, (errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
|
||||||
|
errmsg("relation \"%s\" has already at least one shard, "
|
||||||
|
"adding more is not allowed", relationName)));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
else if (partitionMethod == DISTRIBUTE_BY_HASH)
|
||||||
|
{
|
||||||
|
if (shardMinValue == NULL || shardMaxValue == NULL)
|
||||||
|
{
|
||||||
|
char *relationName = get_rel_name(relationId);
|
||||||
|
ereport(ERROR, (errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
|
||||||
|
errmsg("Shards of has distributed table \"%s\" "
|
||||||
|
"cannot have NULL shard ranges", relationName)));
|
||||||
|
}
|
||||||
|
|
||||||
|
char *shardMinValueString = text_to_cstring(shardMinValue);
|
||||||
|
char *shardMaxValueString = text_to_cstring(shardMaxValue);
|
||||||
|
|
||||||
|
/* pg_strtoint32 does the syntax and out of bound checks for us */
|
||||||
|
int32 shardMinValueInt = pg_strtoint32(shardMinValueString);
|
||||||
|
int32 shardMaxValueInt = pg_strtoint32(shardMaxValueString);
|
||||||
|
|
||||||
|
if (shardMinValueInt > shardMaxValueInt)
|
||||||
|
{
|
||||||
|
ereport(ERROR, (errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
|
||||||
|
errmsg("shardMinValue=%d is greater than "
|
||||||
|
"shardMaxValue=%d for table \"%s\", which is "
|
||||||
|
"not allowed", shardMinValueInt,
|
||||||
|
shardMaxValueInt, get_rel_name(relationId))));
|
||||||
|
}
|
||||||
|
|
||||||
|
/*
|
||||||
|
* We are only dealing with hash distributed tables, that's why we
|
||||||
|
* can hard code data type and typemod.
|
||||||
|
*/
|
||||||
|
const int intervalTypeId = INT4OID;
|
||||||
|
const int intervalTypeMod = -1;
|
||||||
|
|
||||||
|
Relation distShardRelation = table_open(DistShardRelationId(), AccessShareLock);
|
||||||
|
TupleDesc distShardTupleDesc = RelationGetDescr(distShardRelation);
|
||||||
|
|
||||||
|
FmgrInfo *shardIntervalCompareFunction =
|
||||||
|
GetFunctionInfo(intervalTypeId, BTREE_AM_OID, BTORDER_PROC);
|
||||||
|
|
||||||
|
HeapTuple shardTuple = NULL;
|
||||||
|
foreach_ptr(shardTuple, distShardTupleList)
|
||||||
|
{
|
||||||
|
ShardInterval *shardInterval =
|
||||||
|
TupleToShardInterval(shardTuple, distShardTupleDesc,
|
||||||
|
intervalTypeId, intervalTypeMod);
|
||||||
|
|
||||||
|
Datum firstMin = Int32GetDatum(shardMinValueInt);
|
||||||
|
Datum firstMax = Int32GetDatum(shardMaxValueInt);
|
||||||
|
Datum secondMin = shardInterval->minValue;
|
||||||
|
Datum secondMax = shardInterval->maxValue;
|
||||||
|
Oid collationId = InvalidOid;
|
||||||
|
|
||||||
|
/*
|
||||||
|
* This is an unexpected case as we are reading the metadata, which has
|
||||||
|
* already been verified for being not NULL. Still, lets be extra
|
||||||
|
* cautious to avoid any crashes.
|
||||||
|
*/
|
||||||
|
if (!shardInterval->minValueExists || !shardInterval->maxValueExists)
|
||||||
|
{
|
||||||
|
char *relationName = get_rel_name(relationId);
|
||||||
|
ereport(ERROR, (errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
|
||||||
|
errmsg("Shards of has distributed table \"%s\" "
|
||||||
|
"cannot have NULL shard ranges", relationName)));
|
||||||
|
}
|
||||||
|
|
||||||
|
if (ShardIntervalsOverlapWithParams(firstMin, firstMax, secondMin, secondMax,
|
||||||
|
shardIntervalCompareFunction,
|
||||||
|
collationId))
|
||||||
|
{
|
||||||
|
ereport(ERROR, (errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
|
||||||
|
errmsg("Shard intervals overlap for table \"%s\": "
|
||||||
|
"%ld and %ld", get_rel_name(relationId),
|
||||||
|
shardId, shardInterval->shardId)));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
table_close(distShardRelation, NoLock);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/*
|
||||||
|
* citus_internal_add_placement_metadata is an internal UDF to
|
||||||
|
* add a row to pg_dist_placement.
|
||||||
|
*/
|
||||||
|
Datum
|
||||||
|
citus_internal_add_placement_metadata(PG_FUNCTION_ARGS)
|
||||||
|
{
|
||||||
|
int64 shardId = PG_GETARG_INT64(0);
|
||||||
|
int32 shardState = PG_GETARG_INT32(1);
|
||||||
|
int64 shardLength = PG_GETARG_INT64(2);
|
||||||
|
int32 groupId = PG_GETARG_INT32(3);
|
||||||
|
int64 placementId = PG_GETARG_INT64(4);
|
||||||
|
|
||||||
|
bool missingOk = false;
|
||||||
|
Oid relationId = LookupShardRelationFromCatalog(shardId, missingOk);
|
||||||
|
|
||||||
|
/* only owner of the table is allowed to modify the metadata */
|
||||||
|
EnsureTableOwner(relationId);
|
||||||
|
|
||||||
|
/* we want to serialize all the metadata changes to this table */
|
||||||
|
LockRelationOid(relationId, ShareUpdateExclusiveLock);
|
||||||
|
|
||||||
|
if (!ShouldSkipMetadataChecks())
|
||||||
|
{
|
||||||
|
/* this UDF is not allowed allowed for executing as a separate command */
|
||||||
|
EnsureCoordinatorInitiatedOperation();
|
||||||
|
|
||||||
|
/*
|
||||||
|
* Even if the table owner is a malicious user, as long as the shard placements
|
||||||
|
* fit into basic requirements of Citus metadata, the user can only affect its
|
||||||
|
* own tables. Given that the user is owner of the table, we should allow.
|
||||||
|
*/
|
||||||
|
EnsureShardPlacementMetadataIsSane(relationId, shardId, placementId, shardState,
|
||||||
|
shardLength, groupId);
|
||||||
|
}
|
||||||
|
|
||||||
|
InsertShardPlacementRow(shardId, placementId, shardState, shardLength, groupId);
|
||||||
|
|
||||||
|
PG_RETURN_VOID();
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/*
|
||||||
|
* EnsureShardPlacementMetadataIsSane ensures if the input parameters for
|
||||||
|
* the shard placement metadata is sane.
|
||||||
|
*/
|
||||||
|
static void
|
||||||
|
EnsureShardPlacementMetadataIsSane(Oid relationId, int64 shardId, int64 placementId,
|
||||||
|
int32 shardState, int64 shardLength, int32 groupId)
|
||||||
|
{
|
||||||
|
/* we have just read the metadata, so we are sure that the shard exists */
|
||||||
|
Assert(ShardExists(shardId));
|
||||||
|
|
||||||
|
if (placementId <= INVALID_PLACEMENT_ID)
|
||||||
|
{
|
||||||
|
ereport(ERROR, (errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
|
||||||
|
errmsg("Shard placement has invalid placement id "
|
||||||
|
"(%ld) for shard(%ld)", placementId, shardId)));
|
||||||
|
}
|
||||||
|
|
||||||
|
if (shardState != SHARD_STATE_ACTIVE)
|
||||||
|
{
|
||||||
|
ereport(ERROR, (errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
|
||||||
|
errmsg("Invalid shard state: %d", shardState)));
|
||||||
|
}
|
||||||
|
|
||||||
|
bool nodeIsInMetadata = false;
|
||||||
|
WorkerNode *workerNode =
|
||||||
|
PrimaryNodeForGroup(groupId, &nodeIsInMetadata);
|
||||||
|
if (!workerNode)
|
||||||
|
{
|
||||||
|
ereport(ERROR, (errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
|
||||||
|
errmsg("Node with group id %d for shard placement "
|
||||||
|
"%ld does not exist", groupId, shardId)));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/*
|
||||||
|
* ShouldSkipMetadataChecks returns true if the current user is allowed to
|
||||||
|
* make any
|
||||||
|
*/
|
||||||
|
static bool
|
||||||
|
ShouldSkipMetadataChecks(void)
|
||||||
|
{
|
||||||
|
if (strcmp(EnableManualMetadataChangesForUser, "") != 0)
|
||||||
|
{
|
||||||
|
/*
|
||||||
|
* EnableManualMetadataChangesForUser is a GUC which
|
||||||
|
* can be changed by a super user. We use this GUC as
|
||||||
|
* a safety belt in case the current metadata checks are
|
||||||
|
* too restrictive and the operator can allow users to skip
|
||||||
|
* the checks.
|
||||||
|
*/
|
||||||
|
|
||||||
|
/*
|
||||||
|
* Make sure that the user exists, and print it to prevent any
|
||||||
|
* optimization skipping the get_role_oid call.
|
||||||
|
*/
|
||||||
|
bool missingOK = false;
|
||||||
|
Oid allowedUserId = get_role_oid(EnableManualMetadataChangesForUser, missingOK);
|
||||||
|
if (allowedUserId == GetUserId())
|
||||||
|
{
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/*
|
||||||
|
* citus_internal_update_placement_metadata is an internal UDF to
|
||||||
|
* update a row in pg_dist_placement.
|
||||||
|
*/
|
||||||
|
Datum
|
||||||
|
citus_internal_update_placement_metadata(PG_FUNCTION_ARGS)
|
||||||
|
{
|
||||||
|
int64 shardId = PG_GETARG_INT64(0);
|
||||||
|
int32 sourceGroupId = PG_GETARG_INT32(1);
|
||||||
|
int32 targetGroupId = PG_GETARG_INT32(2);
|
||||||
|
|
||||||
|
ShardPlacement *placement = NULL;
|
||||||
|
if (!ShouldSkipMetadataChecks())
|
||||||
|
{
|
||||||
|
/* this UDF is not allowed allowed for executing as a separate command */
|
||||||
|
EnsureCoordinatorInitiatedOperation();
|
||||||
|
|
||||||
|
if (!ShardExists(shardId))
|
||||||
|
{
|
||||||
|
ereport(ERROR, (errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
|
||||||
|
errmsg("Shard id does not exists: %ld", shardId)));
|
||||||
|
}
|
||||||
|
|
||||||
|
bool missingOk = false;
|
||||||
|
EnsureShardOwner(shardId, missingOk);
|
||||||
|
|
||||||
|
/*
|
||||||
|
* This function ensures that the source group exists hence we
|
||||||
|
* call it from this code-block.
|
||||||
|
*/
|
||||||
|
placement = ActiveShardPlacementOnGroup(sourceGroupId, shardId);
|
||||||
|
|
||||||
|
bool nodeIsInMetadata = false;
|
||||||
|
WorkerNode *workerNode =
|
||||||
|
PrimaryNodeForGroup(targetGroupId, &nodeIsInMetadata);
|
||||||
|
if (!workerNode)
|
||||||
|
{
|
||||||
|
ereport(ERROR, (errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
|
||||||
|
errmsg("Node with group id %d for shard placement "
|
||||||
|
"%ld does not exist", targetGroupId, shardId)));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
else
|
||||||
|
{
|
||||||
|
placement = ActiveShardPlacementOnGroup(sourceGroupId, shardId);
|
||||||
|
}
|
||||||
|
|
||||||
|
/*
|
||||||
|
* Updating pg_dist_placement ensures that the node with targetGroupId
|
||||||
|
* exists and this is the only placement on that group.
|
||||||
|
*/
|
||||||
|
if (placement == NULL)
|
||||||
|
{
|
||||||
|
ereport(ERROR, (errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
|
||||||
|
errmsg("Active placement for shard %ld is not "
|
||||||
|
"found on group:%d", shardId, targetGroupId)));
|
||||||
|
}
|
||||||
|
|
||||||
|
UpdatePlacementGroupId(placement->placementId, targetGroupId);
|
||||||
|
|
||||||
|
PG_RETURN_VOID();
|
||||||
|
}
|
||||||
|
|
|
@ -1472,9 +1472,8 @@ ActiveShardPlacement(uint64 shardId, bool missingOk)
|
||||||
* because it shares code with other routines in this file.
|
* because it shares code with other routines in this file.
|
||||||
*/
|
*/
|
||||||
List *
|
List *
|
||||||
BuildShardPlacementList(ShardInterval *shardInterval)
|
BuildShardPlacementList(int64 shardId)
|
||||||
{
|
{
|
||||||
int64 shardId = shardInterval->shardId;
|
|
||||||
List *shardPlacementList = NIL;
|
List *shardPlacementList = NIL;
|
||||||
ScanKeyData scanKey[1];
|
ScanKeyData scanKey[1];
|
||||||
int scanKeyCount = 1;
|
int scanKeyCount = 1;
|
||||||
|
@ -2060,6 +2059,62 @@ UpdateShardPlacementState(uint64 placementId, char shardState)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/*
|
||||||
|
* UpdatePlacementGroupId sets the groupId for the placement identified
|
||||||
|
* by placementId.
|
||||||
|
*/
|
||||||
|
void
|
||||||
|
UpdatePlacementGroupId(uint64 placementId, int groupId)
|
||||||
|
{
|
||||||
|
ScanKeyData scanKey[1];
|
||||||
|
int scanKeyCount = 1;
|
||||||
|
bool indexOK = true;
|
||||||
|
Datum values[Natts_pg_dist_placement];
|
||||||
|
bool isnull[Natts_pg_dist_placement];
|
||||||
|
bool replace[Natts_pg_dist_placement];
|
||||||
|
bool colIsNull = false;
|
||||||
|
|
||||||
|
Relation pgDistPlacement = table_open(DistPlacementRelationId(), RowExclusiveLock);
|
||||||
|
TupleDesc tupleDescriptor = RelationGetDescr(pgDistPlacement);
|
||||||
|
ScanKeyInit(&scanKey[0], Anum_pg_dist_placement_placementid,
|
||||||
|
BTEqualStrategyNumber, F_INT8EQ, Int64GetDatum(placementId));
|
||||||
|
|
||||||
|
SysScanDesc scanDescriptor = systable_beginscan(pgDistPlacement,
|
||||||
|
DistPlacementPlacementidIndexId(),
|
||||||
|
indexOK,
|
||||||
|
NULL, scanKeyCount, scanKey);
|
||||||
|
|
||||||
|
HeapTuple heapTuple = systable_getnext(scanDescriptor);
|
||||||
|
if (!HeapTupleIsValid(heapTuple))
|
||||||
|
{
|
||||||
|
ereport(ERROR, (errmsg("could not find valid entry for shard placement "
|
||||||
|
UINT64_FORMAT,
|
||||||
|
placementId)));
|
||||||
|
}
|
||||||
|
|
||||||
|
memset(replace, 0, sizeof(replace));
|
||||||
|
|
||||||
|
values[Anum_pg_dist_placement_groupid - 1] = Int32GetDatum(groupId);
|
||||||
|
isnull[Anum_pg_dist_placement_groupid - 1] = false;
|
||||||
|
replace[Anum_pg_dist_placement_groupid - 1] = true;
|
||||||
|
|
||||||
|
heapTuple = heap_modify_tuple(heapTuple, tupleDescriptor, values, isnull, replace);
|
||||||
|
|
||||||
|
CatalogTupleUpdate(pgDistPlacement, &heapTuple->t_self, heapTuple);
|
||||||
|
|
||||||
|
uint64 shardId = DatumGetInt64(heap_getattr(heapTuple,
|
||||||
|
Anum_pg_dist_placement_shardid,
|
||||||
|
tupleDescriptor, &colIsNull));
|
||||||
|
Assert(!colIsNull);
|
||||||
|
CitusInvalidateRelcacheByShardId(shardId);
|
||||||
|
|
||||||
|
CommandCounterIncrement();
|
||||||
|
|
||||||
|
systable_endscan(scanDescriptor);
|
||||||
|
table_close(pgDistPlacement, NoLock);
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
/*
|
/*
|
||||||
* Check that the current user has `mode` permissions on relationId, error out
|
* Check that the current user has `mode` permissions on relationId, error out
|
||||||
* if not. Superusers always have such permissions.
|
* if not. Superusers always have such permissions.
|
||||||
|
|
|
@ -1490,6 +1490,9 @@ UpdateColocatedShardPlacementMetadataOnWorkers(int64 shardId,
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
uint32 sourceGroupId = GroupForNode(sourceNodeName, sourceNodePort);
|
||||||
|
uint32 targetGroupId = GroupForNode(targetNodeName, targetNodePort);
|
||||||
|
|
||||||
List *colocatedShardList = ColocatedShardIntervalList(shardInterval);
|
List *colocatedShardList = ColocatedShardIntervalList(shardInterval);
|
||||||
|
|
||||||
/* iterate through the colocated shards and copy each */
|
/* iterate through the colocated shards and copy each */
|
||||||
|
@ -1498,15 +1501,10 @@ UpdateColocatedShardPlacementMetadataOnWorkers(int64 shardId,
|
||||||
ShardInterval *colocatedShard = (ShardInterval *) lfirst(colocatedShardCell);
|
ShardInterval *colocatedShard = (ShardInterval *) lfirst(colocatedShardCell);
|
||||||
StringInfo updateCommand = makeStringInfo();
|
StringInfo updateCommand = makeStringInfo();
|
||||||
|
|
||||||
appendStringInfo(updateCommand, "UPDATE pg_dist_shard_placement "
|
appendStringInfo(updateCommand,
|
||||||
"SET nodename=%s, nodeport=%d WHERE "
|
"SELECT citus_internal_update_placement_metadata(%ld, %d, %d)",
|
||||||
"shardid=%lu AND nodename=%s AND nodeport=%d",
|
|
||||||
quote_literal_cstr(targetNodeName),
|
|
||||||
targetNodePort,
|
|
||||||
colocatedShard->shardId,
|
colocatedShard->shardId,
|
||||||
quote_literal_cstr(sourceNodeName),
|
sourceGroupId, targetGroupId);
|
||||||
sourceNodePort);
|
|
||||||
|
|
||||||
SendCommandToWorkersWithMetadata(updateCommand->data);
|
SendCommandToWorkersWithMetadata(updateCommand->data);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -4148,24 +4148,41 @@ ShardIntervalsOverlap(ShardInterval *firstInterval, ShardInterval *secondInterva
|
||||||
|
|
||||||
Assert(IsCitusTableTypeCacheEntry(intervalRelation, DISTRIBUTED_TABLE));
|
Assert(IsCitusTableTypeCacheEntry(intervalRelation, DISTRIBUTED_TABLE));
|
||||||
|
|
||||||
FmgrInfo *comparisonFunction = intervalRelation->shardIntervalCompareFunction;
|
if (!(firstInterval->minValueExists && firstInterval->maxValueExists &&
|
||||||
Oid collation = intervalRelation->partitionColumn->varcollid;
|
secondInterval->minValueExists && secondInterval->maxValueExists))
|
||||||
|
{
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
|
||||||
Datum firstMin = firstInterval->minValue;
|
Datum firstMin = firstInterval->minValue;
|
||||||
Datum firstMax = firstInterval->maxValue;
|
Datum firstMax = firstInterval->maxValue;
|
||||||
Datum secondMin = secondInterval->minValue;
|
Datum secondMin = secondInterval->minValue;
|
||||||
Datum secondMax = secondInterval->maxValue;
|
Datum secondMax = secondInterval->maxValue;
|
||||||
|
|
||||||
|
FmgrInfo *comparisonFunction = intervalRelation->shardIntervalCompareFunction;
|
||||||
|
Oid collation = intervalRelation->partitionColumn->varcollid;
|
||||||
|
|
||||||
|
return ShardIntervalsOverlapWithParams(firstMin, firstMax, secondMin, secondMax,
|
||||||
|
comparisonFunction, collation);
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/*
|
||||||
|
* ShardIntervalsOverlapWithParams is a helper function which compares the input
|
||||||
|
* shard min/max values, and returns true if the shards overlap.
|
||||||
|
* The caller is responsible to ensure the input shard min/max values are not NULL.
|
||||||
|
*/
|
||||||
|
bool
|
||||||
|
ShardIntervalsOverlapWithParams(Datum firstMin, Datum firstMax, Datum secondMin,
|
||||||
|
Datum secondMax, FmgrInfo *comparisonFunction,
|
||||||
|
Oid collation)
|
||||||
|
{
|
||||||
/*
|
/*
|
||||||
* We need to have min/max values for both intervals first. Then, we assume
|
* We need to have min/max values for both intervals first. Then, we assume
|
||||||
* two intervals i1 = [min1, max1] and i2 = [min2, max2] do not overlap if
|
* two intervals i1 = [min1, max1] and i2 = [min2, max2] do not overlap if
|
||||||
* (max1 < min2) or (max2 < min1). For details, please see the explanation
|
* (max1 < min2) or (max2 < min1). For details, please see the explanation
|
||||||
* on overlapping intervals at http://www.rgrjr.com/emacs/overlap.html.
|
* on overlapping intervals at http://www.rgrjr.com/emacs/overlap.html.
|
||||||
*/
|
*/
|
||||||
if (firstInterval->minValueExists && firstInterval->maxValueExists &&
|
|
||||||
secondInterval->minValueExists && secondInterval->maxValueExists)
|
|
||||||
{
|
|
||||||
Datum firstDatum = FunctionCall2Coll(comparisonFunction, collation, firstMax,
|
Datum firstDatum = FunctionCall2Coll(comparisonFunction, collation, firstMax,
|
||||||
secondMin);
|
secondMin);
|
||||||
Datum secondDatum = FunctionCall2Coll(comparisonFunction, collation, secondMax,
|
Datum secondDatum = FunctionCall2Coll(comparisonFunction, collation, secondMax,
|
||||||
|
@ -4177,7 +4194,6 @@ ShardIntervalsOverlap(ShardInterval *firstInterval, ShardInterval *secondInterva
|
||||||
{
|
{
|
||||||
return false;
|
return false;
|
||||||
}
|
}
|
||||||
}
|
|
||||||
|
|
||||||
return true;
|
return true;
|
||||||
}
|
}
|
||||||
|
|
|
@ -845,6 +845,17 @@ RegisterCitusConfigVariables(void)
|
||||||
GUC_STANDARD,
|
GUC_STANDARD,
|
||||||
NULL, NULL, NULL);
|
NULL, NULL, NULL);
|
||||||
|
|
||||||
|
DefineCustomStringVariable(
|
||||||
|
"citus.enable_manual_metadata_changes_for_user",
|
||||||
|
gettext_noop("Enables some helper UDFs to modify metadata "
|
||||||
|
"for the given user"),
|
||||||
|
NULL,
|
||||||
|
&EnableManualMetadataChangesForUser,
|
||||||
|
"",
|
||||||
|
PGC_SIGHUP,
|
||||||
|
GUC_SUPERUSER_ONLY | GUC_NO_SHOW_ALL,
|
||||||
|
NULL, NULL, NULL);
|
||||||
|
|
||||||
DefineCustomBoolVariable(
|
DefineCustomBoolVariable(
|
||||||
"citus.enable_object_propagation",
|
"citus.enable_object_propagation",
|
||||||
gettext_noop("Enables propagating object creation for more complex objects, "
|
gettext_noop("Enables propagating object creation for more complex objects, "
|
||||||
|
|
|
@ -3,6 +3,14 @@
|
||||||
-- bump version to 10.2-1
|
-- bump version to 10.2-1
|
||||||
|
|
||||||
DROP FUNCTION IF EXISTS pg_catalog.stop_metadata_sync_to_node(text, integer);
|
DROP FUNCTION IF EXISTS pg_catalog.stop_metadata_sync_to_node(text, integer);
|
||||||
|
GRANT ALL ON FUNCTION pg_catalog.worker_record_sequence_dependency(regclass,regclass,name) TO PUBLIC;
|
||||||
|
|
||||||
|
-- the same shard cannot have placements on different nodes
|
||||||
|
ALTER TABLE pg_catalog.pg_dist_placement ADD CONSTRAINT placement_shardid_groupid_unique_index UNIQUE (shardid, groupid);
|
||||||
|
|
||||||
#include "udfs/stop_metadata_sync_to_node/10.2-1.sql"
|
#include "udfs/stop_metadata_sync_to_node/10.2-1.sql"
|
||||||
#include "../../columnar/sql/columnar--10.1-1--10.2-1.sql"
|
#include "../../columnar/sql/columnar--10.1-1--10.2-1.sql"
|
||||||
|
#include "udfs/citus_internal_add_partition_metadata/10.2-1.sql";
|
||||||
|
#include "udfs/citus_internal_add_shard_metadata/10.2-1.sql";
|
||||||
|
#include "udfs/citus_internal_add_placement_metadata/10.2-1.sql";
|
||||||
|
#include "udfs/citus_internal_update_placement_metadata/10.2-1.sql";
|
||||||
|
|
|
@ -10,3 +10,11 @@ CREATE FUNCTION pg_catalog.stop_metadata_sync_to_node(nodename text, nodeport in
|
||||||
AS 'MODULE_PATHNAME', $$stop_metadata_sync_to_node$$;
|
AS 'MODULE_PATHNAME', $$stop_metadata_sync_to_node$$;
|
||||||
COMMENT ON FUNCTION pg_catalog.stop_metadata_sync_to_node(nodename text, nodeport integer)
|
COMMENT ON FUNCTION pg_catalog.stop_metadata_sync_to_node(nodename text, nodeport integer)
|
||||||
IS 'stop metadata sync to node';
|
IS 'stop metadata sync to node';
|
||||||
|
|
||||||
|
DROP FUNCTION pg_catalog.citus_internal_add_partition_metadata(regclass, "char", text, integer, "char");
|
||||||
|
DROP FUNCTION pg_catalog.citus_internal_add_shard_metadata(regclass, bigint, "char", text, text);
|
||||||
|
DROP FUNCTION pg_catalog.citus_internal_add_placement_metadata(bigint, integer, bigint, integer, bigint);
|
||||||
|
DROP FUNCTION pg_catalog.citus_internal_update_placement_metadata(bigint, integer, integer);
|
||||||
|
|
||||||
|
REVOKE ALL ON FUNCTION pg_catalog.worker_record_sequence_dependency(regclass,regclass,name) FROM PUBLIC;
|
||||||
|
ALTER TABLE pg_catalog.pg_dist_placement DROP CONSTRAINT placement_shardid_groupid_unique_index;
|
||||||
|
|
10
src/backend/distributed/sql/udfs/citus_internal_add_partition_metadata/10.2-1.sql
generated
Normal file
10
src/backend/distributed/sql/udfs/citus_internal_add_partition_metadata/10.2-1.sql
generated
Normal file
|
@ -0,0 +1,10 @@
|
||||||
|
CREATE OR REPLACE FUNCTION pg_catalog.citus_internal_add_partition_metadata(
|
||||||
|
relation_id regclass, distribution_method "char",
|
||||||
|
distribution_column text, colocation_id integer,
|
||||||
|
replication_model "char")
|
||||||
|
RETURNS void
|
||||||
|
LANGUAGE C
|
||||||
|
AS 'MODULE_PATHNAME';
|
||||||
|
|
||||||
|
COMMENT ON FUNCTION pg_catalog.citus_internal_add_partition_metadata(regclass, "char", text, integer, "char") IS
|
||||||
|
'Inserts into pg_dist_partition with user checks';
|
|
@ -0,0 +1,10 @@
|
||||||
|
CREATE OR REPLACE FUNCTION pg_catalog.citus_internal_add_partition_metadata(
|
||||||
|
relation_id regclass, distribution_method "char",
|
||||||
|
distribution_column text, colocation_id integer,
|
||||||
|
replication_model "char")
|
||||||
|
RETURNS void
|
||||||
|
LANGUAGE C
|
||||||
|
AS 'MODULE_PATHNAME';
|
||||||
|
|
||||||
|
COMMENT ON FUNCTION pg_catalog.citus_internal_add_partition_metadata(regclass, "char", text, integer, "char") IS
|
||||||
|
'Inserts into pg_dist_partition with user checks';
|
10
src/backend/distributed/sql/udfs/citus_internal_add_placement_metadata/10.2-1.sql
generated
Normal file
10
src/backend/distributed/sql/udfs/citus_internal_add_placement_metadata/10.2-1.sql
generated
Normal file
|
@ -0,0 +1,10 @@
|
||||||
|
CREATE OR REPLACE FUNCTION pg_catalog.citus_internal_add_placement_metadata(
|
||||||
|
shard_id bigint, shard_state integer,
|
||||||
|
shard_length bigint, group_id integer,
|
||||||
|
placement_id bigint)
|
||||||
|
RETURNS void
|
||||||
|
LANGUAGE C STRICT
|
||||||
|
AS 'MODULE_PATHNAME';
|
||||||
|
|
||||||
|
COMMENT ON FUNCTION pg_catalog.citus_internal_add_placement_metadata(bigint, integer, bigint, integer, bigint) IS
|
||||||
|
'Inserts into pg_dist_shard_placement with user checks';
|
|
@ -0,0 +1,10 @@
|
||||||
|
CREATE OR REPLACE FUNCTION pg_catalog.citus_internal_add_placement_metadata(
|
||||||
|
shard_id bigint, shard_state integer,
|
||||||
|
shard_length bigint, group_id integer,
|
||||||
|
placement_id bigint)
|
||||||
|
RETURNS void
|
||||||
|
LANGUAGE C STRICT
|
||||||
|
AS 'MODULE_PATHNAME';
|
||||||
|
|
||||||
|
COMMENT ON FUNCTION pg_catalog.citus_internal_add_placement_metadata(bigint, integer, bigint, integer, bigint) IS
|
||||||
|
'Inserts into pg_dist_shard_placement with user checks';
|
10
src/backend/distributed/sql/udfs/citus_internal_add_shard_metadata/10.2-1.sql
generated
Normal file
10
src/backend/distributed/sql/udfs/citus_internal_add_shard_metadata/10.2-1.sql
generated
Normal file
|
@ -0,0 +1,10 @@
|
||||||
|
CREATE OR REPLACE FUNCTION pg_catalog.citus_internal_add_shard_metadata(
|
||||||
|
relation_id regclass, shard_id bigint,
|
||||||
|
storage_type "char", shard_min_value text,
|
||||||
|
shard_max_value text
|
||||||
|
)
|
||||||
|
RETURNS void
|
||||||
|
LANGUAGE C
|
||||||
|
AS 'MODULE_PATHNAME';
|
||||||
|
COMMENT ON FUNCTION pg_catalog.citus_internal_add_shard_metadata(regclass, bigint, "char", text, text) IS
|
||||||
|
'Inserts into pg_dist_shard with user checks';
|
|
@ -0,0 +1,10 @@
|
||||||
|
CREATE OR REPLACE FUNCTION pg_catalog.citus_internal_add_shard_metadata(
|
||||||
|
relation_id regclass, shard_id bigint,
|
||||||
|
storage_type "char", shard_min_value text,
|
||||||
|
shard_max_value text
|
||||||
|
)
|
||||||
|
RETURNS void
|
||||||
|
LANGUAGE C
|
||||||
|
AS 'MODULE_PATHNAME';
|
||||||
|
COMMENT ON FUNCTION pg_catalog.citus_internal_add_shard_metadata(regclass, bigint, "char", text, text) IS
|
||||||
|
'Inserts into pg_dist_shard with user checks';
|
9
src/backend/distributed/sql/udfs/citus_internal_update_placement_metadata/10.2-1.sql
generated
Normal file
9
src/backend/distributed/sql/udfs/citus_internal_update_placement_metadata/10.2-1.sql
generated
Normal file
|
@ -0,0 +1,9 @@
|
||||||
|
CREATE OR REPLACE FUNCTION pg_catalog.citus_internal_update_placement_metadata(
|
||||||
|
shard_id bigint, source_group_id integer,
|
||||||
|
target_group_id integer)
|
||||||
|
RETURNS void
|
||||||
|
LANGUAGE C STRICT
|
||||||
|
AS 'MODULE_PATHNAME';
|
||||||
|
|
||||||
|
COMMENT ON FUNCTION pg_catalog.citus_internal_update_placement_metadata(bigint, integer, integer) IS
|
||||||
|
'Updates into pg_dist_placement with user checks';
|
|
@ -0,0 +1,9 @@
|
||||||
|
CREATE OR REPLACE FUNCTION pg_catalog.citus_internal_update_placement_metadata(
|
||||||
|
shard_id bigint, source_group_id integer,
|
||||||
|
target_group_id integer)
|
||||||
|
RETURNS void
|
||||||
|
LANGUAGE C STRICT
|
||||||
|
AS 'MODULE_PATHNAME';
|
||||||
|
|
||||||
|
COMMENT ON FUNCTION pg_catalog.citus_internal_update_placement_metadata(bigint, integer, integer) IS
|
||||||
|
'Updates into pg_dist_placement with user checks';
|
|
@ -900,6 +900,23 @@ MyBackendGotCancelledDueToDeadlock(bool clearState)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/*
|
||||||
|
* MyBackendIsInDisributedTransaction returns true if MyBackendData
|
||||||
|
* is in a distributed transaction.
|
||||||
|
*/
|
||||||
|
bool
|
||||||
|
MyBackendIsInDisributedTransaction(void)
|
||||||
|
{
|
||||||
|
/* backend might not have used citus yet and thus not initialized backend data */
|
||||||
|
if (!MyBackendData)
|
||||||
|
{
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
|
||||||
|
return IsInDistributedTransaction(MyBackendData);
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
/*
|
/*
|
||||||
* ActiveDistributedTransactionNumbers returns a list of pointers to
|
* ActiveDistributedTransactionNumbers returns a list of pointers to
|
||||||
* transaction numbers of distributed transactions that are in progress
|
* transaction numbers of distributed transactions that are in progress
|
||||||
|
|
|
@ -58,7 +58,7 @@ static void SendCommandToWorkersOutsideTransaction(TargetWorkerSet targetWorkerS
|
||||||
void
|
void
|
||||||
SendCommandToWorker(const char *nodeName, int32 nodePort, const char *command)
|
SendCommandToWorker(const char *nodeName, int32 nodePort, const char *command)
|
||||||
{
|
{
|
||||||
const char *nodeUser = CitusExtensionOwnerName();
|
const char *nodeUser = CurrentUserName();
|
||||||
SendCommandToWorkerAsUser(nodeName, nodePort, nodeUser, command);
|
SendCommandToWorkerAsUser(nodeName, nodePort, nodeUser, command);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -118,7 +118,7 @@ SendCommandToWorkerAsUser(const char *nodeName, int32 nodePort, const char *node
|
||||||
void
|
void
|
||||||
SendCommandToWorkersWithMetadata(const char *command)
|
SendCommandToWorkersWithMetadata(const char *command)
|
||||||
{
|
{
|
||||||
SendCommandToMetadataWorkersParams(command, CitusExtensionOwnerName(),
|
SendCommandToMetadataWorkersParams(command, CurrentUserName(),
|
||||||
0, NULL, NULL);
|
0, NULL, NULL);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -168,7 +168,7 @@ SendBareCommandListToMetadataWorkers(List *commandList)
|
||||||
{
|
{
|
||||||
TargetWorkerSet targetWorkerSet = NON_COORDINATOR_METADATA_NODES;
|
TargetWorkerSet targetWorkerSet = NON_COORDINATOR_METADATA_NODES;
|
||||||
List *workerNodeList = TargetWorkerSetNodeList(targetWorkerSet, ShareLock);
|
List *workerNodeList = TargetWorkerSetNodeList(targetWorkerSet, ShareLock);
|
||||||
char *nodeUser = CitusExtensionOwnerName();
|
char *nodeUser = CurrentUserName();
|
||||||
|
|
||||||
ErrorIfAnyMetadataNodeOutOfSync(workerNodeList);
|
ErrorIfAnyMetadataNodeOutOfSync(workerNodeList);
|
||||||
|
|
||||||
|
@ -197,51 +197,6 @@ SendBareCommandListToMetadataWorkers(List *commandList)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
/*
|
|
||||||
* SendBareOptionalCommandListToAllWorkersAsUser sends a list of commands
|
|
||||||
* to all workers in serial. Commands are committed immediately: new
|
|
||||||
* connections are always used and no transaction block is used (hence "bare").
|
|
||||||
*/
|
|
||||||
int
|
|
||||||
SendBareOptionalCommandListToAllWorkersAsUser(List *commandList, const char *user)
|
|
||||||
{
|
|
||||||
TargetWorkerSet targetWorkerSet = NON_COORDINATOR_NODES;
|
|
||||||
List *workerNodeList = TargetWorkerSetNodeList(targetWorkerSet, ShareLock);
|
|
||||||
int maxError = RESPONSE_OKAY;
|
|
||||||
|
|
||||||
/* run commands serially */
|
|
||||||
WorkerNode *workerNode = NULL;
|
|
||||||
foreach_ptr(workerNode, workerNodeList)
|
|
||||||
{
|
|
||||||
const char *nodeName = workerNode->workerName;
|
|
||||||
int nodePort = workerNode->workerPort;
|
|
||||||
int connectionFlags = FORCE_NEW_CONNECTION;
|
|
||||||
|
|
||||||
MultiConnection *workerConnection = GetNodeUserDatabaseConnection(connectionFlags,
|
|
||||||
nodeName,
|
|
||||||
nodePort, user,
|
|
||||||
NULL);
|
|
||||||
|
|
||||||
/* iterate over the commands and execute them in the same connection */
|
|
||||||
const char *commandString = NULL;
|
|
||||||
foreach_ptr(commandString, commandList)
|
|
||||||
{
|
|
||||||
int result = ExecuteOptionalRemoteCommand(workerConnection, commandString,
|
|
||||||
NULL);
|
|
||||||
if (result != RESPONSE_OKAY)
|
|
||||||
{
|
|
||||||
maxError = Max(maxError, result);
|
|
||||||
break;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
CloseConnection(workerConnection);
|
|
||||||
}
|
|
||||||
|
|
||||||
return maxError;
|
|
||||||
}
|
|
||||||
|
|
||||||
|
|
||||||
/*
|
/*
|
||||||
* SendCommandToMetadataWorkersParams is a wrapper around
|
* SendCommandToMetadataWorkersParams is a wrapper around
|
||||||
* SendCommandToWorkersParamsInternal() enforcing some extra checks.
|
* SendCommandToWorkersParamsInternal() enforcing some extra checks.
|
||||||
|
|
|
@ -107,9 +107,6 @@ lock_shard_metadata(PG_FUNCTION_ARGS)
|
||||||
ereport(ERROR, (errmsg("no locks specified")));
|
ereport(ERROR, (errmsg("no locks specified")));
|
||||||
}
|
}
|
||||||
|
|
||||||
/* we don't want random users to block writes */
|
|
||||||
EnsureSuperUser();
|
|
||||||
|
|
||||||
int shardIdCount = ArrayObjectCount(shardIdArrayObject);
|
int shardIdCount = ArrayObjectCount(shardIdArrayObject);
|
||||||
Datum *shardIdArrayDatum = DeconstructArrayObject(shardIdArrayObject);
|
Datum *shardIdArrayDatum = DeconstructArrayObject(shardIdArrayObject);
|
||||||
|
|
||||||
|
@ -117,6 +114,17 @@ lock_shard_metadata(PG_FUNCTION_ARGS)
|
||||||
{
|
{
|
||||||
int64 shardId = DatumGetInt64(shardIdArrayDatum[shardIdIndex]);
|
int64 shardId = DatumGetInt64(shardIdArrayDatum[shardIdIndex]);
|
||||||
|
|
||||||
|
/*
|
||||||
|
* We don't want random users to block writes. The callers of this
|
||||||
|
* function either operates on all the colocated placements, such
|
||||||
|
* as shard moves, or requires superuser such as adding node.
|
||||||
|
* In other words, the coordinator initiated operations has already
|
||||||
|
* ensured table owner, we are preventing any malicious attempt to
|
||||||
|
* use this function.
|
||||||
|
*/
|
||||||
|
bool missingOk = true;
|
||||||
|
EnsureShardOwner(shardId, missingOk);
|
||||||
|
|
||||||
LockShardDistributionMetadata(shardId, lockMode);
|
LockShardDistributionMetadata(shardId, lockMode);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -124,6 +132,34 @@ lock_shard_metadata(PG_FUNCTION_ARGS)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/*
|
||||||
|
* EnsureShardOwner gets the shardId and reads pg_dist_partition to find
|
||||||
|
* the corresponding relationId. If the relation does not exist, the function
|
||||||
|
* returns. If the relation exists, the function ensures if the current
|
||||||
|
* user is the owner of the table.
|
||||||
|
*
|
||||||
|
*/
|
||||||
|
void
|
||||||
|
EnsureShardOwner(uint64 shardId, bool missingOk)
|
||||||
|
{
|
||||||
|
Oid relationId = LookupShardRelationFromCatalog(shardId, missingOk);
|
||||||
|
|
||||||
|
if (!OidIsValid(relationId) && missingOk)
|
||||||
|
{
|
||||||
|
/*
|
||||||
|
* This could happen in two ways. First, a malicious user is trying
|
||||||
|
* to acquire locks on non-existing shards. Second, the metadata has
|
||||||
|
* not been synced (or not yet visible) to this node. In the second
|
||||||
|
* case, there is no point in locking the shards because no other
|
||||||
|
* transaction can be accessing the table.
|
||||||
|
*/
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
|
||||||
|
EnsureTableOwner(relationId);
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
/*
|
/*
|
||||||
* lock_shard_resources allows shard resources to be locked
|
* lock_shard_resources allows shard resources to be locked
|
||||||
* remotely to serialise non-commutative writes on shards.
|
* remotely to serialise non-commutative writes on shards.
|
||||||
|
@ -144,9 +180,6 @@ lock_shard_resources(PG_FUNCTION_ARGS)
|
||||||
ereport(ERROR, (errmsg("no locks specified")));
|
ereport(ERROR, (errmsg("no locks specified")));
|
||||||
}
|
}
|
||||||
|
|
||||||
/* we don't want random users to block writes */
|
|
||||||
EnsureSuperUser();
|
|
||||||
|
|
||||||
int shardIdCount = ArrayObjectCount(shardIdArrayObject);
|
int shardIdCount = ArrayObjectCount(shardIdArrayObject);
|
||||||
Datum *shardIdArrayDatum = DeconstructArrayObject(shardIdArrayObject);
|
Datum *shardIdArrayDatum = DeconstructArrayObject(shardIdArrayObject);
|
||||||
|
|
||||||
|
@ -154,6 +187,17 @@ lock_shard_resources(PG_FUNCTION_ARGS)
|
||||||
{
|
{
|
||||||
int64 shardId = DatumGetInt64(shardIdArrayDatum[shardIdIndex]);
|
int64 shardId = DatumGetInt64(shardIdArrayDatum[shardIdIndex]);
|
||||||
|
|
||||||
|
/*
|
||||||
|
* We don't want random users to block writes. The callers of this
|
||||||
|
* function either operates on all the colocated placements, such
|
||||||
|
* as shard moves, or requires superuser such as adding node.
|
||||||
|
* In other words, the coordinator initiated operations has already
|
||||||
|
* ensured table owner, we are preventing any malicious attempt to
|
||||||
|
* use this function.
|
||||||
|
*/
|
||||||
|
bool missingOk = true;
|
||||||
|
EnsureShardOwner(shardId, missingOk);
|
||||||
|
|
||||||
LockShardResource(shardId, lockMode);
|
LockShardResource(shardId, lockMode);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -176,7 +220,7 @@ LockShardListResourcesOnFirstWorker(LOCKMODE lockmode, List *shardIntervalList)
|
||||||
int totalShardIntervalCount = list_length(shardIntervalList);
|
int totalShardIntervalCount = list_length(shardIntervalList);
|
||||||
WorkerNode *firstWorkerNode = GetFirstPrimaryWorkerNode();
|
WorkerNode *firstWorkerNode = GetFirstPrimaryWorkerNode();
|
||||||
int connectionFlags = 0;
|
int connectionFlags = 0;
|
||||||
const char *superuser = CitusExtensionOwnerName();
|
const char *superuser = CurrentUserName();
|
||||||
|
|
||||||
appendStringInfo(lockCommand, "SELECT lock_shard_resources(%d, ARRAY[", lockmode);
|
appendStringInfo(lockCommand, "SELECT lock_shard_resources(%d, ARRAY[", lockmode);
|
||||||
|
|
||||||
|
@ -831,7 +875,6 @@ lock_relation_if_exists(PG_FUNCTION_ARGS)
|
||||||
CitusRangeVarCallbackForLockTable,
|
CitusRangeVarCallbackForLockTable,
|
||||||
(void *) &lockMode);
|
(void *) &lockMode);
|
||||||
bool relationExists = OidIsValid(relationId);
|
bool relationExists = OidIsValid(relationId);
|
||||||
|
|
||||||
PG_RETURN_BOOL(relationExists);
|
PG_RETURN_BOOL(relationExists);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -50,7 +50,6 @@ Datum
|
||||||
worker_drop_distributed_table(PG_FUNCTION_ARGS)
|
worker_drop_distributed_table(PG_FUNCTION_ARGS)
|
||||||
{
|
{
|
||||||
CheckCitusVersion(ERROR);
|
CheckCitusVersion(ERROR);
|
||||||
EnsureSuperUser();
|
|
||||||
|
|
||||||
text *relationName = PG_GETARG_TEXT_P(0);
|
text *relationName = PG_GETARG_TEXT_P(0);
|
||||||
Oid relationId = ResolveRelationId(relationName, true);
|
Oid relationId = ResolveRelationId(relationName, true);
|
||||||
|
@ -65,6 +64,8 @@ worker_drop_distributed_table(PG_FUNCTION_ARGS)
|
||||||
PG_RETURN_VOID();
|
PG_RETURN_VOID();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
EnsureTableOwner(relationId);
|
||||||
|
|
||||||
List *shardList = LoadShardList(relationId);
|
List *shardList = LoadShardList(relationId);
|
||||||
|
|
||||||
/* first check the relation type */
|
/* first check the relation type */
|
||||||
|
|
|
@ -36,9 +36,9 @@ Datum
|
||||||
worker_create_truncate_trigger(PG_FUNCTION_ARGS)
|
worker_create_truncate_trigger(PG_FUNCTION_ARGS)
|
||||||
{
|
{
|
||||||
CheckCitusVersion(ERROR);
|
CheckCitusVersion(ERROR);
|
||||||
EnsureSuperUser();
|
|
||||||
|
|
||||||
Oid relationId = PG_GETARG_OID(0);
|
Oid relationId = PG_GETARG_OID(0);
|
||||||
|
EnsureTableOwner(relationId);
|
||||||
|
|
||||||
/* Create the truncate trigger */
|
/* Create the truncate trigger */
|
||||||
CreateTruncateTrigger(relationId);
|
CreateTruncateTrigger(relationId);
|
||||||
|
|
|
@ -66,6 +66,7 @@ extern void MarkCitusInitiatedCoordinatorBackend(void);
|
||||||
extern void GetBackendDataForProc(PGPROC *proc, BackendData *result);
|
extern void GetBackendDataForProc(PGPROC *proc, BackendData *result);
|
||||||
extern void CancelTransactionDueToDeadlock(PGPROC *proc);
|
extern void CancelTransactionDueToDeadlock(PGPROC *proc);
|
||||||
extern bool MyBackendGotCancelledDueToDeadlock(bool clearState);
|
extern bool MyBackendGotCancelledDueToDeadlock(bool clearState);
|
||||||
|
extern bool MyBackendIsInDisributedTransaction(void);
|
||||||
extern List * ActiveDistributedTransactionNumbers(void);
|
extern List * ActiveDistributedTransactionNumbers(void);
|
||||||
extern LocalTransactionId GetMyProcLocalTransactionId(void);
|
extern LocalTransactionId GetMyProcLocalTransactionId(void);
|
||||||
extern int GetAllActiveClientBackendCount(void);
|
extern int GetAllActiveClientBackendCount(void);
|
||||||
|
|
|
@ -143,6 +143,9 @@ extern bool IsCitusTableTypeCacheEntry(CitusTableCacheEntry *tableEtnry,
|
||||||
CitusTableType tableType);
|
CitusTableType tableType);
|
||||||
|
|
||||||
extern bool IsCitusTable(Oid relationId);
|
extern bool IsCitusTable(Oid relationId);
|
||||||
|
extern char PgDistPartitionViaCatalog(Oid relationId);
|
||||||
|
extern List * LookupDistShardTuples(Oid relationId);
|
||||||
|
extern char PartitionMethodViaCatalog(Oid relationId);
|
||||||
extern bool IsCitusLocalTableByDistParams(char partitionMethod, char replicationModel);
|
extern bool IsCitusLocalTableByDistParams(char partitionMethod, char replicationModel);
|
||||||
extern List * CitusTableList(void);
|
extern List * CitusTableList(void);
|
||||||
extern ShardInterval * LoadShardInterval(uint64 shardId);
|
extern ShardInterval * LoadShardInterval(uint64 shardId);
|
||||||
|
@ -196,6 +199,9 @@ extern void ErrorIfInconsistentShardIntervals(CitusTableCacheEntry *cacheEntry);
|
||||||
extern void EnsureModificationsCanRun(void);
|
extern void EnsureModificationsCanRun(void);
|
||||||
extern char LookupDistributionMethod(Oid distributionMethodOid);
|
extern char LookupDistributionMethod(Oid distributionMethodOid);
|
||||||
extern bool RelationExists(Oid relationId);
|
extern bool RelationExists(Oid relationId);
|
||||||
|
extern ShardInterval * TupleToShardInterval(HeapTuple heapTuple,
|
||||||
|
TupleDesc tupleDescriptor, Oid intervalTypeId,
|
||||||
|
int32 intervalTypeMod);
|
||||||
|
|
||||||
/* access WorkerNodeHash */
|
/* access WorkerNodeHash */
|
||||||
extern bool HasAnyNodes(void);
|
extern bool HasAnyNodes(void);
|
||||||
|
|
|
@ -82,4 +82,8 @@ extern Oid GetAttributeTypeOid(Oid relationId, AttrNumber attnum);
|
||||||
"groupid = EXCLUDED.groupid"
|
"groupid = EXCLUDED.groupid"
|
||||||
#define METADATA_SYNC_CHANNEL "metadata_sync"
|
#define METADATA_SYNC_CHANNEL "metadata_sync"
|
||||||
|
|
||||||
|
|
||||||
|
/* controlled via GUC */
|
||||||
|
extern char *EnableManualMetadataChangesForUser;
|
||||||
|
|
||||||
#endif /* METADATA_SYNC_H */
|
#endif /* METADATA_SYNC_H */
|
||||||
|
|
|
@ -216,7 +216,7 @@ extern List * ActiveShardPlacementListOnGroup(uint64 shardId, int32 groupId);
|
||||||
extern List * ActiveShardPlacementList(uint64 shardId);
|
extern List * ActiveShardPlacementList(uint64 shardId);
|
||||||
extern List * ShardPlacementListWithoutOrphanedPlacements(uint64 shardId);
|
extern List * ShardPlacementListWithoutOrphanedPlacements(uint64 shardId);
|
||||||
extern ShardPlacement * ActiveShardPlacement(uint64 shardId, bool missingOk);
|
extern ShardPlacement * ActiveShardPlacement(uint64 shardId, bool missingOk);
|
||||||
extern List * BuildShardPlacementList(ShardInterval *shardInterval);
|
extern List * BuildShardPlacementList(int64 shardId);
|
||||||
extern List * AllShardPlacementsOnNodeGroup(int32 groupId);
|
extern List * AllShardPlacementsOnNodeGroup(int32 groupId);
|
||||||
extern List * AllShardPlacementsWithShardPlacementState(ShardState shardState);
|
extern List * AllShardPlacementsWithShardPlacementState(ShardState shardState);
|
||||||
extern List * GroupShardPlacementsForTableOnGroup(Oid relationId, int32 groupId);
|
extern List * GroupShardPlacementsForTableOnGroup(Oid relationId, int32 groupId);
|
||||||
|
@ -241,6 +241,7 @@ extern void UpdatePartitionShardPlacementStates(ShardPlacement *parentShardPlace
|
||||||
char shardState);
|
char shardState);
|
||||||
extern void MarkShardPlacementInactive(ShardPlacement *shardPlacement);
|
extern void MarkShardPlacementInactive(ShardPlacement *shardPlacement);
|
||||||
extern void UpdateShardPlacementState(uint64 placementId, char shardState);
|
extern void UpdateShardPlacementState(uint64 placementId, char shardState);
|
||||||
|
extern void UpdatePlacementGroupId(uint64 placementId, int groupId);
|
||||||
extern void DeleteShardPlacementRow(uint64 placementId);
|
extern void DeleteShardPlacementRow(uint64 placementId);
|
||||||
extern void CreateDistributedTable(Oid relationId, Var *distributionColumn,
|
extern void CreateDistributedTable(Oid relationId, Var *distributionColumn,
|
||||||
char distributionMethod, int shardCount,
|
char distributionMethod, int shardCount,
|
||||||
|
|
|
@ -557,6 +557,10 @@ extern Var * MakeInt4Column(void);
|
||||||
extern int CompareShardPlacements(const void *leftElement, const void *rightElement);
|
extern int CompareShardPlacements(const void *leftElement, const void *rightElement);
|
||||||
extern bool ShardIntervalsOverlap(ShardInterval *firstInterval,
|
extern bool ShardIntervalsOverlap(ShardInterval *firstInterval,
|
||||||
ShardInterval *secondInterval);
|
ShardInterval *secondInterval);
|
||||||
|
extern bool ShardIntervalsOverlapWithParams(Datum firstMin, Datum firstMax,
|
||||||
|
Datum secondMin, Datum secondMax,
|
||||||
|
FmgrInfo *comparisonFunction,
|
||||||
|
Oid collation);
|
||||||
extern bool CoPartitionedTables(Oid firstRelationId, Oid secondRelationId);
|
extern bool CoPartitionedTables(Oid firstRelationId, Oid secondRelationId);
|
||||||
extern ShardInterval ** GenerateSyntheticShardIntervalArray(int partitionCount);
|
extern ShardInterval ** GenerateSyntheticShardIntervalArray(int partitionCount);
|
||||||
extern RowModifyLevel RowModifyLevelForQuery(Query *query);
|
extern RowModifyLevel RowModifyLevelForQuery(Query *query);
|
||||||
|
|
|
@ -54,6 +54,7 @@ typedef FormData_pg_dist_partition *Form_pg_dist_partition;
|
||||||
#define DISTRIBUTE_BY_RANGE 'r'
|
#define DISTRIBUTE_BY_RANGE 'r'
|
||||||
#define DISTRIBUTE_BY_NONE 'n'
|
#define DISTRIBUTE_BY_NONE 'n'
|
||||||
#define REDISTRIBUTE_BY_HASH 'x'
|
#define REDISTRIBUTE_BY_HASH 'x'
|
||||||
|
#define DISTRIBUTE_BY_INVALID '\0'
|
||||||
|
|
||||||
/*
|
/*
|
||||||
* Valid values for repmodel are 'c' for coordinator, 's' for streaming
|
* Valid values for repmodel are 'c' for coordinator, 's' for streaming
|
||||||
|
|
|
@ -114,6 +114,7 @@ typedef enum CitusOperations
|
||||||
extern void LockShardDistributionMetadata(int64 shardId, LOCKMODE lockMode);
|
extern void LockShardDistributionMetadata(int64 shardId, LOCKMODE lockMode);
|
||||||
extern void LockPlacementCleanup(void);
|
extern void LockPlacementCleanup(void);
|
||||||
extern bool TryLockPlacementCleanup(void);
|
extern bool TryLockPlacementCleanup(void);
|
||||||
|
extern void EnsureShardOwner(uint64 shardId, bool missingOk);
|
||||||
extern void LockShardListMetadataOnWorkers(LOCKMODE lockmode, List *shardIntervalList);
|
extern void LockShardListMetadataOnWorkers(LOCKMODE lockmode, List *shardIntervalList);
|
||||||
extern void BlockWritesToShardList(List *shardList);
|
extern void BlockWritesToShardList(List *shardList);
|
||||||
|
|
||||||
|
|
|
@ -43,8 +43,6 @@ extern bool SendOptionalCommandListToWorkerInTransaction(const char *nodeName, i
|
||||||
List *commandList);
|
List *commandList);
|
||||||
extern void SendCommandToWorkersWithMetadata(const char *command);
|
extern void SendCommandToWorkersWithMetadata(const char *command);
|
||||||
extern void SendBareCommandListToMetadataWorkers(List *commandList);
|
extern void SendBareCommandListToMetadataWorkers(List *commandList);
|
||||||
extern int SendBareOptionalCommandListToAllWorkersAsUser(List *commandList,
|
|
||||||
const char *user);
|
|
||||||
extern void EnsureNoModificationsHaveBeenDone(void);
|
extern void EnsureNoModificationsHaveBeenDone(void);
|
||||||
extern void SendCommandListToWorkerInSingleTransaction(const char *nodeName,
|
extern void SendCommandListToWorkerInSingleTransaction(const char *nodeName,
|
||||||
int32 nodePort,
|
int32 nodePort,
|
||||||
|
|
|
@ -136,7 +136,7 @@ SELECT hasmetadata FROM pg_dist_node WHERE nodeport=:worker_2_proxy_port;
|
||||||
|
|
||||||
-- Check failures on DDL command propagation
|
-- Check failures on DDL command propagation
|
||||||
CREATE TABLE t2 (id int PRIMARY KEY);
|
CREATE TABLE t2 (id int PRIMARY KEY);
|
||||||
SELECT citus.mitmproxy('conn.onParse(query="^INSERT INTO pg_dist_placement").kill()');
|
SELECT citus.mitmproxy('conn.onParse(query="citus_internal_add_placement_metadata").kill()');
|
||||||
mitmproxy
|
mitmproxy
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
|
|
||||||
|
@ -147,7 +147,7 @@ ERROR: server closed the connection unexpectedly
|
||||||
This probably means the server terminated abnormally
|
This probably means the server terminated abnormally
|
||||||
before or while processing the request.
|
before or while processing the request.
|
||||||
CONTEXT: while executing command on localhost:xxxxx
|
CONTEXT: while executing command on localhost:xxxxx
|
||||||
SELECT citus.mitmproxy('conn.onParse(query="^INSERT INTO pg_dist_shard").cancel(' || :pid || ')');
|
SELECT citus.mitmproxy('conn.onParse(query="citus_internal_add_shard_metadata").cancel(' || :pid || ')');
|
||||||
mitmproxy
|
mitmproxy
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
|
|
||||||
|
|
|
@ -28,11 +28,11 @@ step detector-dump-wait-edges:
|
||||||
|
|
||||||
waiting_transaction_numblocking_transaction_numblocking_transaction_waiting
|
waiting_transaction_numblocking_transaction_numblocking_transaction_waiting
|
||||||
|
|
||||||
400 399 f
|
401 400 f
|
||||||
transactionnumberwaitingtransactionnumbers
|
transactionnumberwaitingtransactionnumbers
|
||||||
|
|
||||||
399
|
400
|
||||||
400 399
|
401 400
|
||||||
step s1-abort:
|
step s1-abort:
|
||||||
ABORT;
|
ABORT;
|
||||||
|
|
||||||
|
@ -75,14 +75,14 @@ step detector-dump-wait-edges:
|
||||||
|
|
||||||
waiting_transaction_numblocking_transaction_numblocking_transaction_waiting
|
waiting_transaction_numblocking_transaction_numblocking_transaction_waiting
|
||||||
|
|
||||||
404 403 f
|
405 404 f
|
||||||
405 403 f
|
406 404 f
|
||||||
405 404 t
|
406 405 t
|
||||||
transactionnumberwaitingtransactionnumbers
|
transactionnumberwaitingtransactionnumbers
|
||||||
|
|
||||||
403
|
404
|
||||||
404 403
|
405 404
|
||||||
405 403,404
|
406 404,405
|
||||||
step s1-abort:
|
step s1-abort:
|
||||||
ABORT;
|
ABORT;
|
||||||
|
|
||||||
|
|
|
@ -0,0 +1,901 @@
|
||||||
|
CREATE SCHEMA metadata_sync_helpers;
|
||||||
|
SET search_path TO metadata_sync_helpers;
|
||||||
|
SET citus.next_shard_id TO 1420000;
|
||||||
|
SET citus.next_placement_id TO 1500000;
|
||||||
|
-- supress notice messages to make sure that the tests
|
||||||
|
-- do not diverge with enterprise
|
||||||
|
SET client_min_messages TO WARNING;
|
||||||
|
CREATE ROLE metadata_sync_helper_role WITH LOGIN;
|
||||||
|
GRANT ALL ON SCHEMA metadata_sync_helpers TO metadata_sync_helper_role;
|
||||||
|
RESET client_min_messages;
|
||||||
|
\c - metadata_sync_helper_role -
|
||||||
|
SET search_path TO metadata_sync_helpers;
|
||||||
|
CREATE TABLE test(col_1 int);
|
||||||
|
-- not in a distributed transaction
|
||||||
|
SELECT citus_internal_add_partition_metadata ('test'::regclass, 'h', 'col_1', 0, 's');
|
||||||
|
ERROR: This is an internal function that only Citus requires to use in a distributed transaction
|
||||||
|
-- in a distributed transaction, but the application name is not Citus
|
||||||
|
BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
|
||||||
|
SELECT assign_distributed_transaction_id(0, 8, '2021-07-09 15:41:55.542377+02');
|
||||||
|
assign_distributed_transaction_id
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
SELECT citus_internal_add_partition_metadata ('test'::regclass, 'h', 'col_1', 0, 's');
|
||||||
|
ERROR: This is an internal function that only Citus requires to use in a distributed transaction
|
||||||
|
ROLLBACK;
|
||||||
|
-- in a distributed transaction and the application name is Citus
|
||||||
|
-- but we are on the coordinator, so still not allowed
|
||||||
|
BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
|
||||||
|
SELECT assign_distributed_transaction_id(0, 8, '2021-07-09 15:41:55.542377+02');
|
||||||
|
assign_distributed_transaction_id
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
SET application_name to 'citus';
|
||||||
|
SELECT citus_internal_add_partition_metadata ('test'::regclass, 'h', 'col_1', 0, 's');
|
||||||
|
ERROR: This is an internal function that only Citus requires to use in a distributed transaction
|
||||||
|
ROLLBACK;
|
||||||
|
-- connect back as super user, and then connect to the worker
|
||||||
|
-- with the superuser to make sure we can ingest metadata with
|
||||||
|
-- a regular user under the certain conditions
|
||||||
|
\c - postgres -
|
||||||
|
-- we don't need the table/schema anymore
|
||||||
|
SET client_min_messages TO ERROR;
|
||||||
|
DROP SCHEMA metadata_sync_helpers CASCADE;
|
||||||
|
DROP ROLE metadata_sync_helper_role;
|
||||||
|
\c - - - :worker_1_port
|
||||||
|
CREATE SCHEMA metadata_sync_helpers;
|
||||||
|
SET search_path TO metadata_sync_helpers;
|
||||||
|
CREATE TABLE test(col_1 int, col_2 int);
|
||||||
|
-- supress notice messages to make sure that the tests
|
||||||
|
-- do not diverge with enterprise
|
||||||
|
SET client_min_messages TO WARNING;
|
||||||
|
SET citus.enable_ddl_propagation TO OFF;
|
||||||
|
CREATE ROLE metadata_sync_helper_role WITH LOGIN;
|
||||||
|
GRANT ALL ON SCHEMA metadata_sync_helpers TO metadata_sync_helper_role;
|
||||||
|
RESET client_min_messages;
|
||||||
|
RESET citus.enable_ddl_propagation;
|
||||||
|
-- connect back with the regular user
|
||||||
|
\c - metadata_sync_helper_role - :worker_1_port
|
||||||
|
SET search_path TO metadata_sync_helpers;
|
||||||
|
-- in a distributed transaction and the application name is Citus
|
||||||
|
-- and we are on the worker, still not allowed because the user is not
|
||||||
|
-- owner of the table test
|
||||||
|
BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
|
||||||
|
SELECT assign_distributed_transaction_id(0, 8, '2021-07-09 15:41:55.542377+02');
|
||||||
|
assign_distributed_transaction_id
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
SET application_name to 'citus';
|
||||||
|
SELECT citus_internal_add_partition_metadata ('test'::regclass, 'h', 'col_1', 0, 's');
|
||||||
|
ERROR: must be owner of table test
|
||||||
|
ROLLBACK;
|
||||||
|
-- finally, a user can only add its own tables to the metadata
|
||||||
|
CREATE TABLE test_2(col_1 int, col_2 int);
|
||||||
|
BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
|
||||||
|
SELECT assign_distributed_transaction_id(0, 8, '2021-07-09 15:41:55.542377+02');
|
||||||
|
assign_distributed_transaction_id
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
SET application_name to 'citus';
|
||||||
|
SELECT citus_internal_add_partition_metadata ('test_2'::regclass, 'h', 'col_1', 0, 's');
|
||||||
|
citus_internal_add_partition_metadata
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
SELECT count(*) FROM pg_dist_partition WHERE logicalrelid = 'metadata_sync_helpers.test_2'::regclass;
|
||||||
|
count
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
1
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
ROLLBACK;
|
||||||
|
-- fails because there is no X distribution method
|
||||||
|
BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
|
||||||
|
SELECT assign_distributed_transaction_id(0, 8, '2021-07-09 15:41:55.542377+02');
|
||||||
|
assign_distributed_transaction_id
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
SET application_name to 'citus';
|
||||||
|
SELECT citus_internal_add_partition_metadata ('test_2'::regclass, 'X', 'col_1', 0, 's');
|
||||||
|
ERROR: Metadata syncing is only allowed for hash, reference and local tables:X
|
||||||
|
ROLLBACK;
|
||||||
|
-- fails because there is the column does not exist
|
||||||
|
BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
|
||||||
|
SELECT assign_distributed_transaction_id(0, 8, '2021-07-09 15:41:55.542377+02');
|
||||||
|
assign_distributed_transaction_id
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
SET application_name to 'citus';
|
||||||
|
SELECT citus_internal_add_partition_metadata ('test_2'::regclass, 'h', 'non_existing_col', 0, 's');
|
||||||
|
ERROR: column "non_existing_col" of relation "test_2" does not exist
|
||||||
|
ROLLBACK;
|
||||||
|
--- fails because we do not allow NULL parameters
|
||||||
|
BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
|
||||||
|
SELECT assign_distributed_transaction_id(0, 8, '2021-07-09 15:41:55.542377+02');
|
||||||
|
assign_distributed_transaction_id
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
SET application_name to 'citus';
|
||||||
|
SELECT citus_internal_add_partition_metadata (NULL, 'h', 'non_existing_col', 0, 's');
|
||||||
|
ERROR: relation cannot be NULL
|
||||||
|
ROLLBACK;
|
||||||
|
-- fails because colocationId cannot be negative
|
||||||
|
BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
|
||||||
|
SELECT assign_distributed_transaction_id(0, 8, '2021-07-09 15:41:55.542377+02');
|
||||||
|
assign_distributed_transaction_id
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
SET application_name to 'citus';
|
||||||
|
SELECT citus_internal_add_partition_metadata ('test_2'::regclass, 'h', 'col_1', -1, 's');
|
||||||
|
ERROR: Metadata syncing is only allowed for valid colocation id values.
|
||||||
|
ROLLBACK;
|
||||||
|
-- fails because there is no X replication model
|
||||||
|
BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
|
||||||
|
SELECT assign_distributed_transaction_id(0, 8, '2021-07-09 15:41:55.542377+02');
|
||||||
|
assign_distributed_transaction_id
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
SET application_name to 'citus';
|
||||||
|
SELECT citus_internal_add_partition_metadata ('test_2'::regclass, 'X', 'col_1', 0, 'X');
|
||||||
|
ERROR: Metadata syncing is only allowed for hash, reference and local tables:X
|
||||||
|
ROLLBACK;
|
||||||
|
-- the same table cannot be added twice, that is enforced by a primary key
|
||||||
|
BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
|
||||||
|
SELECT assign_distributed_transaction_id(0, 8, '2021-07-09 15:41:55.542377+02');
|
||||||
|
assign_distributed_transaction_id
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
SET application_name to 'citus';
|
||||||
|
\set VERBOSITY terse
|
||||||
|
SELECT citus_internal_add_partition_metadata ('test_2'::regclass, 'h', 'col_1', 0, 's');
|
||||||
|
citus_internal_add_partition_metadata
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
SELECT citus_internal_add_partition_metadata ('test_2'::regclass, 'h', 'col_1', 0, 's');
|
||||||
|
ERROR: duplicate key value violates unique constraint "pg_dist_partition_logical_relid_index"
|
||||||
|
ROLLBACK;
|
||||||
|
-- the same table cannot be added twice, that is enforced by a primary key even if distribution key changes
|
||||||
|
BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
|
||||||
|
SELECT assign_distributed_transaction_id(0, 8, '2021-07-09 15:41:55.542377+02');
|
||||||
|
assign_distributed_transaction_id
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
SET application_name to 'citus';
|
||||||
|
\set VERBOSITY terse
|
||||||
|
SELECT citus_internal_add_partition_metadata ('test_2'::regclass, 'h', 'col_1', 0, 's');
|
||||||
|
citus_internal_add_partition_metadata
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
SELECT citus_internal_add_partition_metadata ('test_2'::regclass, 'h', 'col_2', 0, 's');
|
||||||
|
ERROR: duplicate key value violates unique constraint "pg_dist_partition_logical_relid_index"
|
||||||
|
ROLLBACK;
|
||||||
|
-- hash distributed table cannot have NULL distribution key
|
||||||
|
BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
|
||||||
|
SELECT assign_distributed_transaction_id(0, 8, '2021-07-09 15:41:55.542377+02');
|
||||||
|
assign_distributed_transaction_id
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
SET application_name to 'citus';
|
||||||
|
\set VERBOSITY terse
|
||||||
|
SELECT citus_internal_add_partition_metadata ('test_2'::regclass, 'h', NULL, 0, 's');
|
||||||
|
ERROR: Distribution column cannot be NULL for relation "test_2"
|
||||||
|
ROLLBACK;
|
||||||
|
-- even if metadata_sync_helper_role is not owner of the table test
|
||||||
|
-- the operator allowed metadata_sync_helper_role user to skip
|
||||||
|
-- checks, such as wrong partition method can be set
|
||||||
|
-- connect back with the regular user
|
||||||
|
\c - postgres - :worker_1_port
|
||||||
|
SET search_path TO metadata_sync_helpers;
|
||||||
|
ALTER SYSTEM SET citus.enable_manual_metadata_changes_for_user TO 'metadata_sync_helper_role';
|
||||||
|
SELECT pg_reload_conf();
|
||||||
|
pg_reload_conf
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
t
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
SELECT pg_sleep(0.1);
|
||||||
|
pg_sleep
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
\c - metadata_sync_helper_role - :worker_1_port
|
||||||
|
SET search_path TO metadata_sync_helpers;
|
||||||
|
BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
|
||||||
|
SELECT assign_distributed_transaction_id(0, 8, '2021-07-09 15:41:55.542377+02');
|
||||||
|
assign_distributed_transaction_id
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
SET application_name to 'citus';
|
||||||
|
SELECT citus_internal_add_partition_metadata ('test_2'::regclass, 'X', 'col_1', 0, 's');
|
||||||
|
citus_internal_add_partition_metadata
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
ROLLBACK;
|
||||||
|
-- should throw error even if we skip the checks, there are no such nodes
|
||||||
|
BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
|
||||||
|
SELECT assign_distributed_transaction_id(0, 8, '2021-07-09 15:41:55.542377+02');
|
||||||
|
assign_distributed_transaction_id
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
SET application_name to 'citus';
|
||||||
|
\set VERBOSITY terse
|
||||||
|
SELECT citus_internal_update_placement_metadata(1420007, 10000, 11111);
|
||||||
|
ERROR: could not find valid entry for shard xxxxx
|
||||||
|
ROLLBACK;
|
||||||
|
-- non-existing users should fail to pass the checks
|
||||||
|
\c - postgres - :worker_1_port
|
||||||
|
SET search_path TO metadata_sync_helpers;
|
||||||
|
ALTER SYSTEM SET citus.enable_manual_metadata_changes_for_user TO 'non_existing_user';
|
||||||
|
SELECT pg_reload_conf();
|
||||||
|
pg_reload_conf
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
t
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
SELECT pg_sleep(0.1);
|
||||||
|
pg_sleep
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
\c - metadata_sync_helper_role - :worker_1_port
|
||||||
|
SET search_path TO metadata_sync_helpers;
|
||||||
|
BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
|
||||||
|
SELECT assign_distributed_transaction_id(0, 8, '2021-07-09 15:41:55.542377+02');
|
||||||
|
assign_distributed_transaction_id
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
SET application_name to 'citus';
|
||||||
|
SELECT citus_internal_add_partition_metadata ('test_2'::regclass, 'X', 'col_1', 0, 's');
|
||||||
|
ERROR: role "non_existing_user" does not exist
|
||||||
|
ROLLBACK;
|
||||||
|
\c - postgres - :worker_1_port
|
||||||
|
SET search_path TO metadata_sync_helpers;
|
||||||
|
ALTER SYSTEM RESET citus.enable_manual_metadata_changes_for_user;
|
||||||
|
SELECT pg_reload_conf();
|
||||||
|
pg_reload_conf
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
t
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
SELECT pg_sleep(0.1);
|
||||||
|
pg_sleep
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
\c - metadata_sync_helper_role - :worker_1_port
|
||||||
|
SET search_path TO metadata_sync_helpers;
|
||||||
|
-- we also enforce some checks on the reference tables
|
||||||
|
-- cannot add because reference tables cannot have distribution column
|
||||||
|
CREATE TABLE test_ref(col_1 int, col_2 int);
|
||||||
|
BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
|
||||||
|
SELECT assign_distributed_transaction_id(0, 8, '2021-07-09 15:41:55.542377+02');
|
||||||
|
assign_distributed_transaction_id
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
SET application_name to 'citus';
|
||||||
|
SELECT citus_internal_add_partition_metadata ('test_ref'::regclass, 'n', 'col_1', 0, 's');
|
||||||
|
ERROR: Reference or local tables cannot have distribution columns
|
||||||
|
ROLLBACK;
|
||||||
|
-- non-valid replication model
|
||||||
|
BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
|
||||||
|
SELECT assign_distributed_transaction_id(0, 8, '2021-07-09 15:41:55.542377+02');
|
||||||
|
assign_distributed_transaction_id
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
SET application_name to 'citus';
|
||||||
|
SELECT citus_internal_add_partition_metadata ('test_ref'::regclass, 'n', NULL, 0, 'A');
|
||||||
|
ERROR: Metadata syncing is only allowed for known replication models.
|
||||||
|
ROLLBACK;
|
||||||
|
-- not-matching replication model for reference table
|
||||||
|
BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
|
||||||
|
SELECT assign_distributed_transaction_id(0, 8, '2021-07-09 15:41:55.542377+02');
|
||||||
|
assign_distributed_transaction_id
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
SET application_name to 'citus';
|
||||||
|
SELECT citus_internal_add_partition_metadata ('test_ref'::regclass, 'n', NULL, 0, 'c');
|
||||||
|
ERROR: Local or references tables can only have 's' or 't' as the replication model.
|
||||||
|
ROLLBACK;
|
||||||
|
-- not-matching replication model for hash table
|
||||||
|
BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
|
||||||
|
SELECT assign_distributed_transaction_id(0, 8, '2021-07-09 15:41:55.542377+02');
|
||||||
|
assign_distributed_transaction_id
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
SET application_name to 'citus';
|
||||||
|
SELECT citus_internal_add_partition_metadata ('test_2'::regclass, 'h', 'col_1', 0, 't');
|
||||||
|
ERROR: Hash distributed tables can only have 's' as the replication model.
|
||||||
|
ROLLBACK;
|
||||||
|
-- add entry for super user table
|
||||||
|
\c - postgres - :worker_1_port
|
||||||
|
SET search_path TO metadata_sync_helpers;
|
||||||
|
CREATE TABLE super_user_table(col_1 int);
|
||||||
|
BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
|
||||||
|
SELECT assign_distributed_transaction_id(0, 8, '2021-07-09 15:41:55.542377+02');
|
||||||
|
assign_distributed_transaction_id
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
SET application_name to 'citus';
|
||||||
|
SELECT citus_internal_add_partition_metadata ('super_user_table'::regclass, 'h', 'col_1', 0, 's');
|
||||||
|
citus_internal_add_partition_metadata
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
COMMIT;
|
||||||
|
-- now, lets check shard metadata
|
||||||
|
-- the user is only allowed to add a shard for add a table which they do not own
|
||||||
|
\c - metadata_sync_helper_role - :worker_1_port
|
||||||
|
SET search_path TO metadata_sync_helpers;
|
||||||
|
BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
|
||||||
|
SELECT assign_distributed_transaction_id(0, 8, '2021-07-09 15:41:55.542377+02');
|
||||||
|
assign_distributed_transaction_id
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
SET application_name to 'citus';
|
||||||
|
\set VERBOSITY terse
|
||||||
|
WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue)
|
||||||
|
AS (VALUES ('super_user_table'::regclass, 1420000::bigint, 't'::"char", '-2147483648'::text, '-1610612737'::text))
|
||||||
|
SELECT citus_internal_add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data;
|
||||||
|
ERROR: must be owner of table super_user_table
|
||||||
|
ROLLBACK;
|
||||||
|
-- the user is only allowed to add a shard for add a table which is in pg_dist_partition
|
||||||
|
BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
|
||||||
|
SELECT assign_distributed_transaction_id(0, 8, '2021-07-09 15:41:55.542377+02');
|
||||||
|
assign_distributed_transaction_id
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
SET application_name to 'citus';
|
||||||
|
\set VERBOSITY terse
|
||||||
|
WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue)
|
||||||
|
AS (VALUES ('test_2'::regclass, 1420000::bigint, 't'::"char", '-2147483648'::text, '-1610612737'::text))
|
||||||
|
SELECT citus_internal_add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data;
|
||||||
|
ERROR: The relation "test_2" does not have a valid entry in pg_dist_partition.
|
||||||
|
ROLLBACK;
|
||||||
|
-- ok, now add the table to the pg_dist_partition
|
||||||
|
BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
|
||||||
|
SELECT assign_distributed_transaction_id(0, 8, '2021-07-09 15:41:55.542377+02');
|
||||||
|
assign_distributed_transaction_id
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
SET application_name to 'citus';
|
||||||
|
SELECT citus_internal_add_partition_metadata ('test_2'::regclass, 'h', 'col_1', 0, 's');
|
||||||
|
citus_internal_add_partition_metadata
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
SELECT citus_internal_add_partition_metadata ('test_ref'::regclass, 'n', NULL, 0, 't');
|
||||||
|
citus_internal_add_partition_metadata
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
COMMIT;
|
||||||
|
-- invalid shard ids are not allowed
|
||||||
|
BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
|
||||||
|
SELECT assign_distributed_transaction_id(0, 8, '2021-07-09 15:41:55.542377+02');
|
||||||
|
assign_distributed_transaction_id
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
SET application_name to 'citus';
|
||||||
|
\set VERBOSITY terse
|
||||||
|
WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue)
|
||||||
|
AS (VALUES ('test_2'::regclass, -1, 't'::"char", '-2147483648'::text, '-1610612737'::text))
|
||||||
|
SELECT citus_internal_add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data;
|
||||||
|
ERROR: Invalid shard id: -1
|
||||||
|
ROLLBACK;
|
||||||
|
-- invalid storage types are not allowed
|
||||||
|
BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
|
||||||
|
SELECT assign_distributed_transaction_id(0, 8, '2021-07-09 15:41:55.542377+02');
|
||||||
|
assign_distributed_transaction_id
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
SET application_name to 'citus';
|
||||||
|
\set VERBOSITY terse
|
||||||
|
WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue)
|
||||||
|
AS (VALUES ('test_2'::regclass, 1420000, 'X'::"char", '-2147483648'::text, '-1610612737'::text))
|
||||||
|
SELECT citus_internal_add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data;
|
||||||
|
ERROR: Invalid shard storage type: X
|
||||||
|
ROLLBACK;
|
||||||
|
-- NULL shard ranges are not allowed for hash distributed tables
|
||||||
|
BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
|
||||||
|
SELECT assign_distributed_transaction_id(0, 8, '2021-07-09 15:41:55.542377+02');
|
||||||
|
assign_distributed_transaction_id
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
SET application_name to 'citus';
|
||||||
|
\set VERBOSITY terse
|
||||||
|
WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue)
|
||||||
|
AS (VALUES ('test_2'::regclass, 1420000, 't'::"char", NULL, '-1610612737'::text))
|
||||||
|
SELECT citus_internal_add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data;
|
||||||
|
ERROR: Shards of has distributed table "test_2" cannot have NULL shard ranges
|
||||||
|
ROLLBACK;
|
||||||
|
-- non-integer shard ranges are not allowed
|
||||||
|
BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
|
||||||
|
SELECT assign_distributed_transaction_id(0, 8, '2021-07-09 15:41:55.542377+02');
|
||||||
|
assign_distributed_transaction_id
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
SET application_name to 'citus';
|
||||||
|
\set VERBOSITY terse
|
||||||
|
WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue)
|
||||||
|
AS (VALUES ('test_2'::regclass, 1420000::bigint, 't'::"char", 'non-int'::text, '-1610612737'::text))
|
||||||
|
SELECT citus_internal_add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data;
|
||||||
|
ERROR: invalid input syntax for integer: "non-int"
|
||||||
|
ROLLBACK;
|
||||||
|
-- shardMinValue should be smaller than shardMaxValue
|
||||||
|
BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
|
||||||
|
SELECT assign_distributed_transaction_id(0, 8, '2021-07-09 15:41:55.542377+02');
|
||||||
|
assign_distributed_transaction_id
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
SET application_name to 'citus';
|
||||||
|
\set VERBOSITY terse
|
||||||
|
WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue)
|
||||||
|
AS (VALUES ('test_2'::regclass, 1420000::bigint, 't'::"char", '-1610612737'::text, '-2147483648'::text))
|
||||||
|
SELECT citus_internal_add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data;
|
||||||
|
ERROR: shardMinValue=-1610612737 is greater than shardMaxValue=-2147483648 for table "test_2", which is not allowed
|
||||||
|
ROLLBACK;
|
||||||
|
-- we do not allow overlapping shards for the same table
|
||||||
|
BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
|
||||||
|
SELECT assign_distributed_transaction_id(0, 8, '2021-07-09 15:41:55.542377+02');
|
||||||
|
assign_distributed_transaction_id
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
SET application_name to 'citus';
|
||||||
|
\set VERBOSITY terse
|
||||||
|
WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue)
|
||||||
|
AS (VALUES ('test_2'::regclass, 1420000::bigint, 't'::"char", '10'::text, '20'::text),
|
||||||
|
('test_2'::regclass, 1420001::bigint, 't'::"char", '20'::text, '30'::text),
|
||||||
|
('test_2'::regclass, 1420002::bigint, 't'::"char", '10'::text, '50'::text))
|
||||||
|
SELECT citus_internal_add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data;
|
||||||
|
ERROR: Shard intervals overlap for table "test_2": 1420001 and 1420000
|
||||||
|
ROLLBACK;
|
||||||
|
\c - postgres - :worker_1_port
|
||||||
|
-- we do not allow wrong partmethod
|
||||||
|
-- so manually insert wrong partmethod for the sake of the test
|
||||||
|
SET search_path TO metadata_sync_helpers;
|
||||||
|
BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
|
||||||
|
SELECT assign_distributed_transaction_id(0, 8, '2021-07-09 15:41:55.542377+02');
|
||||||
|
assign_distributed_transaction_id
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
SET application_name to 'citus';
|
||||||
|
\set VERBOSITY terse
|
||||||
|
UPDATE pg_dist_partition SET partmethod = 'X';
|
||||||
|
WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue)
|
||||||
|
AS (VALUES ('test_2'::regclass, 1420000::bigint, 't'::"char", '10'::text, '20'::text),
|
||||||
|
('test_2'::regclass, 1420001::bigint, 't'::"char", '20'::text, '30'::text))
|
||||||
|
SELECT citus_internal_add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data;
|
||||||
|
ERROR: Metadata syncing is only allowed for hash, reference and local tables: X
|
||||||
|
ROLLBACK;
|
||||||
|
-- we do not allow NULL shardMinMax values
|
||||||
|
-- so manually insert wrong partmethod for the sake of the test
|
||||||
|
SET search_path TO metadata_sync_helpers;
|
||||||
|
BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
|
||||||
|
SELECT assign_distributed_transaction_id(0, 8, '2021-07-09 15:41:55.542377+02');
|
||||||
|
assign_distributed_transaction_id
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
SET application_name to 'citus';
|
||||||
|
\set VERBOSITY terse
|
||||||
|
WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue)
|
||||||
|
AS (VALUES ('test_2'::regclass, 1420000::bigint, 't'::"char", '10'::text, '20'::text))
|
||||||
|
SELECT citus_internal_add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data;
|
||||||
|
citus_internal_add_shard_metadata
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
-- manually ingest NULL values, otherwise not likely unless metadata is corrupted
|
||||||
|
UPDATE pg_dist_shard SET shardminvalue = NULL WHERE shardid = 1420000;
|
||||||
|
WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue)
|
||||||
|
AS (VALUES ('test_2'::regclass, 1420001::bigint, 't'::"char", '20'::text, '30'::text))
|
||||||
|
SELECT citus_internal_add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data;
|
||||||
|
ERROR: Shards of has distributed table "test_2" cannot have NULL shard ranges
|
||||||
|
ROLLBACK;
|
||||||
|
\c - metadata_sync_helper_role - :worker_1_port
|
||||||
|
SET search_path TO metadata_sync_helpers;
|
||||||
|
-- now, add few shards
|
||||||
|
BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
|
||||||
|
SELECT assign_distributed_transaction_id(0, 8, '2021-07-09 15:41:55.542377+02');
|
||||||
|
assign_distributed_transaction_id
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
SET application_name to 'citus';
|
||||||
|
\set VERBOSITY terse
|
||||||
|
WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue)
|
||||||
|
AS (VALUES ('test_2'::regclass, 1420000::bigint, 't'::"char", '11'::text, '20'::text),
|
||||||
|
('test_2'::regclass, 1420001::bigint, 't'::"char", '21'::text, '30'::text),
|
||||||
|
('test_2'::regclass, 1420002::bigint, 't'::"char", '31'::text, '40'::text),
|
||||||
|
('test_2'::regclass, 1420003::bigint, 't'::"char", '41'::text, '50'::text),
|
||||||
|
('test_2'::regclass, 1420004::bigint, 't'::"char", '51'::text, '60'::text),
|
||||||
|
('test_2'::regclass, 1420005::bigint, 't'::"char", '61'::text, '70'::text))
|
||||||
|
SELECT citus_internal_add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data;
|
||||||
|
citus_internal_add_shard_metadata
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
(6 rows)
|
||||||
|
|
||||||
|
COMMIT;
|
||||||
|
-- shardMin/MaxValues should be NULL for reference tables
|
||||||
|
BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
|
||||||
|
SELECT assign_distributed_transaction_id(0, 8, '2021-07-09 15:41:55.542377+02');
|
||||||
|
assign_distributed_transaction_id
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
SET application_name to 'citus';
|
||||||
|
\set VERBOSITY terse
|
||||||
|
WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue)
|
||||||
|
AS (VALUES ('test_ref'::regclass, 1420003::bigint, 't'::"char", '-1610612737'::text, NULL))
|
||||||
|
SELECT citus_internal_add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data;
|
||||||
|
ERROR: Shards of reference or local table "test_ref" should have NULL shard ranges
|
||||||
|
ROLLBACK;
|
||||||
|
-- reference tables cannot have multiple shards
|
||||||
|
BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
|
||||||
|
SELECT assign_distributed_transaction_id(0, 8, '2021-07-09 15:41:55.542377+02');
|
||||||
|
assign_distributed_transaction_id
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
SET application_name to 'citus';
|
||||||
|
\set VERBOSITY terse
|
||||||
|
WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue)
|
||||||
|
AS (VALUES ('test_ref'::regclass, 1420006::bigint, 't'::"char", NULL, NULL),
|
||||||
|
('test_ref'::regclass, 1420007::bigint, 't'::"char", NULL, NULL))
|
||||||
|
SELECT citus_internal_add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data;
|
||||||
|
ERROR: relation "test_ref" has already at least one shard, adding more is not allowed
|
||||||
|
ROLLBACK;
|
||||||
|
-- finally, add a shard for reference tables
|
||||||
|
BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
|
||||||
|
SELECT assign_distributed_transaction_id(0, 8, '2021-07-09 15:41:55.542377+02');
|
||||||
|
assign_distributed_transaction_id
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
SET application_name to 'citus';
|
||||||
|
\set VERBOSITY terse
|
||||||
|
WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue)
|
||||||
|
AS (VALUES ('test_ref'::regclass, 1420006::bigint, 't'::"char", NULL, NULL))
|
||||||
|
SELECT citus_internal_add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data;
|
||||||
|
citus_internal_add_shard_metadata
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
COMMIT;
|
||||||
|
\c - postgres - :worker_1_port
|
||||||
|
SET search_path TO metadata_sync_helpers;
|
||||||
|
-- and a shard for the superuser table
|
||||||
|
BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
|
||||||
|
SELECT assign_distributed_transaction_id(0, 8, '2021-07-09 15:41:55.542377+02');
|
||||||
|
assign_distributed_transaction_id
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
SET application_name to 'citus';
|
||||||
|
\set VERBOSITY terse
|
||||||
|
WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue)
|
||||||
|
AS (VALUES ('super_user_table'::regclass, 1420007::bigint, 't'::"char", '11'::text, '20'::text))
|
||||||
|
SELECT citus_internal_add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data;
|
||||||
|
citus_internal_add_shard_metadata
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
COMMIT;
|
||||||
|
\c - metadata_sync_helper_role - :worker_1_port
|
||||||
|
SET search_path TO metadata_sync_helpers;
|
||||||
|
-- now, check placement metadata
|
||||||
|
-- shard does not exist
|
||||||
|
BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
|
||||||
|
SELECT assign_distributed_transaction_id(0, 8, '2021-07-09 15:41:55.542377+02');
|
||||||
|
assign_distributed_transaction_id
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
SET application_name to 'citus';
|
||||||
|
\set VERBOSITY terse
|
||||||
|
WITH placement_data(shardid, shardstate, shardlength, groupid, placementid) AS
|
||||||
|
(VALUES (-10, 1, 0::bigint, 1::int, 1500000::bigint))
|
||||||
|
SELECT citus_internal_add_placement_metadata(shardid, shardstate, shardlength, groupid, placementid) FROM placement_data;
|
||||||
|
ERROR: could not find valid entry for shard xxxxx
|
||||||
|
ROLLBACK;
|
||||||
|
-- invalid placementid
|
||||||
|
BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
|
||||||
|
SELECT assign_distributed_transaction_id(0, 8, '2021-07-09 15:41:55.542377+02');
|
||||||
|
assign_distributed_transaction_id
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
SET application_name to 'citus';
|
||||||
|
\set VERBOSITY terse
|
||||||
|
WITH placement_data(shardid, shardstate, shardlength, groupid, placementid) AS
|
||||||
|
(VALUES (1420000, 1, 0::bigint, 1::int, -10))
|
||||||
|
SELECT citus_internal_add_placement_metadata(shardid, shardstate, shardlength, groupid, placementid) FROM placement_data;
|
||||||
|
ERROR: Shard placement has invalid placement id (-10) for shard(1420000)
|
||||||
|
ROLLBACK;
|
||||||
|
-- non-existing shard
|
||||||
|
BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
|
||||||
|
SELECT assign_distributed_transaction_id(0, 8, '2021-07-09 15:41:55.542377+02');
|
||||||
|
assign_distributed_transaction_id
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
SET application_name to 'citus';
|
||||||
|
\set VERBOSITY terse
|
||||||
|
WITH placement_data(shardid, shardstate, shardlength, groupid, placementid) AS
|
||||||
|
(VALUES (1430100, 1, 0::bigint, 1::int, 10))
|
||||||
|
SELECT citus_internal_add_placement_metadata(shardid, shardstate, shardlength, groupid, placementid) FROM placement_data;
|
||||||
|
ERROR: could not find valid entry for shard xxxxx
|
||||||
|
ROLLBACK;
|
||||||
|
-- invalid shard state
|
||||||
|
BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
|
||||||
|
SELECT assign_distributed_transaction_id(0, 8, '2021-07-09 15:41:55.542377+02');
|
||||||
|
assign_distributed_transaction_id
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
SET application_name to 'citus';
|
||||||
|
\set VERBOSITY terse
|
||||||
|
WITH placement_data(shardid, shardstate, shardlength, groupid, placementid) AS
|
||||||
|
(VALUES (1420000, 10, 0::bigint, 1::int, 1500000))
|
||||||
|
SELECT citus_internal_add_placement_metadata(shardid, shardstate, shardlength, groupid, placementid) FROM placement_data;
|
||||||
|
ERROR: Invalid shard state: 10
|
||||||
|
ROLLBACK;
|
||||||
|
-- non-existing node with non-existing node-id 123123123
|
||||||
|
BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
|
||||||
|
SELECT assign_distributed_transaction_id(0, 8, '2021-07-09 15:41:55.542377+02');
|
||||||
|
assign_distributed_transaction_id
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
SET application_name to 'citus';
|
||||||
|
\set VERBOSITY terse
|
||||||
|
WITH placement_data(shardid, shardstate, shardlength, groupid, placementid) AS
|
||||||
|
(VALUES ( 1420000, 1, 0::bigint, 123123123::int, 1500000))
|
||||||
|
SELECT citus_internal_add_placement_metadata(shardid, shardstate, shardlength, groupid, placementid) FROM placement_data;
|
||||||
|
ERROR: Node with group id 123123123 for shard placement xxxxx does not exist
|
||||||
|
ROLLBACK;
|
||||||
|
-- create a volatile function that returns the local node id
|
||||||
|
CREATE OR REPLACE FUNCTION get_node_id()
|
||||||
|
RETURNS INT AS $$
|
||||||
|
DECLARE localGroupId int;
|
||||||
|
BEGIN
|
||||||
|
SELECT
|
||||||
|
groupid into localGroupId
|
||||||
|
FROM
|
||||||
|
pg_dist_node
|
||||||
|
WHERE
|
||||||
|
nodeport = 57637 AND nodename = 'localhost' AND isactive AND nodecluster = 'default';
|
||||||
|
RETURN localGroupId;
|
||||||
|
END; $$ language plpgsql;
|
||||||
|
-- fails because we ingest more placements for the same shards to the same worker node
|
||||||
|
BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
|
||||||
|
SELECT assign_distributed_transaction_id(0, 8, '2021-07-09 15:41:55.542377+02');
|
||||||
|
assign_distributed_transaction_id
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
SET application_name to 'citus';
|
||||||
|
\set VERBOSITY terse
|
||||||
|
WITH placement_data(shardid, shardstate, shardlength, groupid, placementid) AS
|
||||||
|
(VALUES (1420000, 1, 0::bigint, get_node_id(), 1500000),
|
||||||
|
(1420000, 1, 0::bigint, get_node_id(), 1500001))
|
||||||
|
SELECT citus_internal_add_placement_metadata(shardid, shardstate, shardlength, groupid, placementid) FROM placement_data;
|
||||||
|
ERROR: duplicate key value violates unique constraint "placement_shardid_groupid_unique_index"
|
||||||
|
ROLLBACK;
|
||||||
|
-- shard is not owned by us
|
||||||
|
BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
|
||||||
|
SELECT assign_distributed_transaction_id(0, 8, '2021-07-09 15:41:55.542377+02');
|
||||||
|
assign_distributed_transaction_id
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
SET application_name to 'citus';
|
||||||
|
\set VERBOSITY terse
|
||||||
|
WITH placement_data(shardid, shardstate, shardlength, groupid, placementid) AS
|
||||||
|
(VALUES (1420007, 1, 0::bigint, get_node_id(), 1500000))
|
||||||
|
SELECT citus_internal_add_placement_metadata(shardid, shardstate, shardlength, groupid, placementid) FROM placement_data;
|
||||||
|
ERROR: must be owner of table super_user_table
|
||||||
|
ROLLBACK;
|
||||||
|
-- sucessfully add placements
|
||||||
|
BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
|
||||||
|
SELECT assign_distributed_transaction_id(0, 8, '2021-07-09 15:41:55.542377+02');
|
||||||
|
assign_distributed_transaction_id
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
SET application_name to 'citus';
|
||||||
|
\set VERBOSITY terse
|
||||||
|
WITH placement_data(shardid, shardstate, shardlength, groupid, placementid) AS
|
||||||
|
(VALUES (1420000, 1, 0::bigint, get_node_id(), 1500000),
|
||||||
|
(1420001, 1, 0::bigint, get_node_id(), 1500001),
|
||||||
|
(1420002, 1, 0::bigint, get_node_id(), 1500002),
|
||||||
|
(1420003, 1, 0::bigint, get_node_id(), 1500003),
|
||||||
|
(1420004, 1, 0::bigint, get_node_id(), 1500004),
|
||||||
|
(1420005, 1, 0::bigint, get_node_id(), 1500005))
|
||||||
|
SELECT citus_internal_add_placement_metadata(shardid, shardstate, shardlength, groupid, placementid) FROM placement_data;
|
||||||
|
citus_internal_add_placement_metadata
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
(6 rows)
|
||||||
|
|
||||||
|
COMMIT;
|
||||||
|
-- try to update placements
|
||||||
|
-- fails because we are trying to update it to non-existing node
|
||||||
|
BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
|
||||||
|
SELECT assign_distributed_transaction_id(0, 8, '2021-07-09 15:41:55.542377+02');
|
||||||
|
assign_distributed_transaction_id
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
SET application_name to 'citus';
|
||||||
|
\set VERBOSITY terse
|
||||||
|
SELECT citus_internal_update_placement_metadata(1420000, get_node_id(), get_node_id()+1000);
|
||||||
|
ERROR: Node with group id 1014 for shard placement xxxxx does not exist
|
||||||
|
COMMIT;
|
||||||
|
-- fails because the source node doesn't contain the shard
|
||||||
|
BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
|
||||||
|
SELECT assign_distributed_transaction_id(0, 8, '2021-07-09 15:41:55.542377+02');
|
||||||
|
assign_distributed_transaction_id
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
SET application_name to 'citus';
|
||||||
|
\set VERBOSITY terse
|
||||||
|
SELECT citus_internal_update_placement_metadata(1420000, get_node_id()+10000, get_node_id());
|
||||||
|
ERROR: Active placement for shard xxxxx is not found on group:14
|
||||||
|
COMMIT;
|
||||||
|
-- fails because shard does not exist
|
||||||
|
BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
|
||||||
|
SELECT assign_distributed_transaction_id(0, 8, '2021-07-09 15:41:55.542377+02');
|
||||||
|
assign_distributed_transaction_id
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
SET application_name to 'citus';
|
||||||
|
\set VERBOSITY terse
|
||||||
|
SELECT citus_internal_update_placement_metadata(0, get_node_id(), get_node_id()+1);
|
||||||
|
ERROR: Shard id does not exists: 0
|
||||||
|
COMMIT;
|
||||||
|
-- fails because none-existing shard
|
||||||
|
BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
|
||||||
|
SELECT assign_distributed_transaction_id(0, 8, '2021-07-09 15:41:55.542377+02');
|
||||||
|
assign_distributed_transaction_id
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
SET application_name to 'citus';
|
||||||
|
\set VERBOSITY terse
|
||||||
|
SELECT citus_internal_update_placement_metadata(213123123123, get_node_id(), get_node_id()+1);
|
||||||
|
ERROR: Shard id does not exists: 213123123123
|
||||||
|
COMMIT;
|
||||||
|
-- fails because we do not own the shard
|
||||||
|
BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
|
||||||
|
SELECT assign_distributed_transaction_id(0, 8, '2021-07-09 15:41:55.542377+02');
|
||||||
|
assign_distributed_transaction_id
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
SET application_name to 'citus';
|
||||||
|
\set VERBOSITY terse
|
||||||
|
SELECT citus_internal_update_placement_metadata(1420007, get_node_id(), get_node_id()+1);
|
||||||
|
ERROR: must be owner of table super_user_table
|
||||||
|
COMMIT;
|
||||||
|
-- we don't need the table/schema anymore
|
||||||
|
-- connect back as super user to drop everything
|
||||||
|
\c - postgres - :worker_1_port
|
||||||
|
SET search_path TO metadata_sync_helpers;
|
||||||
|
-- remove the manually generated metadata
|
||||||
|
DELETE FROM pg_dist_placement WHERE shardid IN (SELECT shardid FROM pg_dist_shard WHERE logicalrelid IN ('test_ref'::regclass, 'test_2'::regclass));
|
||||||
|
DELETE FROM pg_dist_shard WHERE logicalrelid IN ('test_ref'::regclass, 'test_2'::regclass);
|
||||||
|
DELETE FROM pg_dist_partition WHERE logicalrelid IN ('test_ref'::regclass, 'test_2'::regclass);
|
||||||
|
SET client_min_messages TO ERROR;
|
||||||
|
SET citus.enable_ddl_propagation TO OFF;
|
||||||
|
DROP OWNED BY metadata_sync_helper_role;
|
||||||
|
DROP ROLE metadata_sync_helper_role;
|
||||||
|
DROP SCHEMA metadata_sync_helpers CASCADE;
|
|
@ -79,15 +79,15 @@ SELECT unnest(master_metadata_snapshot()) order by 1;
|
||||||
ALTER TABLE public.mx_test_table OWNER TO postgres
|
ALTER TABLE public.mx_test_table OWNER TO postgres
|
||||||
CREATE TABLE public.mx_test_table (col_1 integer, col_2 text NOT NULL, col_3 bigint DEFAULT nextval('public.mx_test_table_col_3_seq'::regclass) NOT NULL, col_4 bigint DEFAULT nextval('public.user_defined_seq'::regclass))
|
CREATE TABLE public.mx_test_table (col_1 integer, col_2 text NOT NULL, col_3 bigint DEFAULT nextval('public.mx_test_table_col_3_seq'::regclass) NOT NULL, col_4 bigint DEFAULT nextval('public.user_defined_seq'::regclass))
|
||||||
INSERT INTO pg_dist_node (nodeid, groupid, nodename, nodeport, noderack, hasmetadata, metadatasynced, isactive, noderole, nodecluster) VALUES (1, 1, 'localhost', 57637, 'default', FALSE, FALSE, TRUE, 'primary'::noderole, 'default'),(2, 2, 'localhost', 57638, 'default', FALSE, FALSE, TRUE, 'primary'::noderole, 'default')
|
INSERT INTO pg_dist_node (nodeid, groupid, nodename, nodeport, noderack, hasmetadata, metadatasynced, isactive, noderole, nodecluster) VALUES (1, 1, 'localhost', 57637, 'default', FALSE, FALSE, TRUE, 'primary'::noderole, 'default'),(2, 2, 'localhost', 57638, 'default', FALSE, FALSE, TRUE, 'primary'::noderole, 'default')
|
||||||
INSERT INTO pg_dist_partition (logicalrelid, partmethod, partkey, colocationid, repmodel) VALUES ('public.mx_test_table'::regclass, 'h', column_name_to_column('public.mx_test_table','col_1'), 0, 's')
|
SELECT citus_internal_add_partition_metadata ('public.mx_test_table'::regclass, 'h', 'col_1', 0, 's')
|
||||||
INSERT INTO pg_dist_placement (shardid, shardstate, shardlength, groupid, placementid) VALUES (1310000, 1, 0, 1, 100000),(1310001, 1, 0, 2, 100001),(1310002, 1, 0, 1, 100002),(1310003, 1, 0, 2, 100003),(1310004, 1, 0, 1, 100004),(1310005, 1, 0, 2, 100005),(1310006, 1, 0, 1, 100006),(1310007, 1, 0, 2, 100007)
|
|
||||||
INSERT INTO pg_dist_shard (logicalrelid, shardid, shardstorage, shardminvalue, shardmaxvalue) VALUES ('public.mx_test_table'::regclass, 1310000, 't', '-2147483648', '-1610612737'),('public.mx_test_table'::regclass, 1310001, 't', '-1610612736', '-1073741825'),('public.mx_test_table'::regclass, 1310002, 't', '-1073741824', '-536870913'),('public.mx_test_table'::regclass, 1310003, 't', '-536870912', '-1'),('public.mx_test_table'::regclass, 1310004, 't', '0', '536870911'),('public.mx_test_table'::regclass, 1310005, 't', '536870912', '1073741823'),('public.mx_test_table'::regclass, 1310006, 't', '1073741824', '1610612735'),('public.mx_test_table'::regclass, 1310007, 't', '1610612736', '2147483647')
|
|
||||||
SELECT pg_catalog.worker_record_sequence_dependency('public.mx_test_table_col_3_seq'::regclass,'public.mx_test_table'::regclass,'col_3')
|
SELECT pg_catalog.worker_record_sequence_dependency('public.mx_test_table_col_3_seq'::regclass,'public.mx_test_table'::regclass,'col_3')
|
||||||
SELECT worker_apply_sequence_command ('CREATE SEQUENCE IF NOT EXISTS public.mx_test_table_col_3_seq INCREMENT BY 1 MINVALUE 1 MAXVALUE 9223372036854775807 START WITH 1 NO CYCLE','bigint')
|
SELECT worker_apply_sequence_command ('CREATE SEQUENCE IF NOT EXISTS public.mx_test_table_col_3_seq INCREMENT BY 1 MINVALUE 1 MAXVALUE 9223372036854775807 START WITH 1 NO CYCLE','bigint')
|
||||||
SELECT worker_apply_sequence_command ('CREATE SEQUENCE IF NOT EXISTS public.user_defined_seq INCREMENT BY 1 MINVALUE 1 MAXVALUE 9223372036854775807 START WITH 1 NO CYCLE','bigint')
|
SELECT worker_apply_sequence_command ('CREATE SEQUENCE IF NOT EXISTS public.user_defined_seq INCREMENT BY 1 MINVALUE 1 MAXVALUE 9223372036854775807 START WITH 1 NO CYCLE','bigint')
|
||||||
SELECT worker_create_truncate_trigger('public.mx_test_table')
|
SELECT worker_create_truncate_trigger('public.mx_test_table')
|
||||||
SELECT worker_drop_distributed_table(logicalrelid::regclass::text) FROM pg_dist_partition
|
SELECT worker_drop_distributed_table(logicalrelid::regclass::text) FROM pg_dist_partition
|
||||||
TRUNCATE pg_dist_node CASCADE
|
TRUNCATE pg_dist_node CASCADE
|
||||||
|
WITH placement_data(shardid, shardstate, shardlength, groupid, placementid) AS (VALUES (1310000, 1, 0, 1, 100000), (1310001, 1, 0, 2, 100001), (1310002, 1, 0, 1, 100002), (1310003, 1, 0, 2, 100003), (1310004, 1, 0, 1, 100004), (1310005, 1, 0, 2, 100005), (1310006, 1, 0, 1, 100006), (1310007, 1, 0, 2, 100007)) SELECT citus_internal_add_placement_metadata(shardid, shardstate, shardlength, groupid, placementid) FROM placement_data;
|
||||||
|
WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) AS (VALUES ('public.mx_test_table'::regclass, 1310000, 't'::"char", '-2147483648', '-1610612737'), ('public.mx_test_table'::regclass, 1310001, 't'::"char", '-1610612736', '-1073741825'), ('public.mx_test_table'::regclass, 1310002, 't'::"char", '-1073741824', '-536870913'), ('public.mx_test_table'::regclass, 1310003, 't'::"char", '-536870912', '-1'), ('public.mx_test_table'::regclass, 1310004, 't'::"char", '0', '536870911'), ('public.mx_test_table'::regclass, 1310005, 't'::"char", '536870912', '1073741823'), ('public.mx_test_table'::regclass, 1310006, 't'::"char", '1073741824', '1610612735'), ('public.mx_test_table'::regclass, 1310007, 't'::"char", '1610612736', '2147483647')) SELECT citus_internal_add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data;
|
||||||
(16 rows)
|
(16 rows)
|
||||||
|
|
||||||
-- Show that CREATE INDEX commands are included in the metadata snapshot
|
-- Show that CREATE INDEX commands are included in the metadata snapshot
|
||||||
|
@ -103,15 +103,15 @@ SELECT unnest(master_metadata_snapshot()) order by 1;
|
||||||
CREATE INDEX mx_index ON public.mx_test_table USING btree (col_2)
|
CREATE INDEX mx_index ON public.mx_test_table USING btree (col_2)
|
||||||
CREATE TABLE public.mx_test_table (col_1 integer, col_2 text NOT NULL, col_3 bigint DEFAULT nextval('public.mx_test_table_col_3_seq'::regclass) NOT NULL, col_4 bigint DEFAULT nextval('public.user_defined_seq'::regclass))
|
CREATE TABLE public.mx_test_table (col_1 integer, col_2 text NOT NULL, col_3 bigint DEFAULT nextval('public.mx_test_table_col_3_seq'::regclass) NOT NULL, col_4 bigint DEFAULT nextval('public.user_defined_seq'::regclass))
|
||||||
INSERT INTO pg_dist_node (nodeid, groupid, nodename, nodeport, noderack, hasmetadata, metadatasynced, isactive, noderole, nodecluster) VALUES (1, 1, 'localhost', 57637, 'default', FALSE, FALSE, TRUE, 'primary'::noderole, 'default'),(2, 2, 'localhost', 57638, 'default', FALSE, FALSE, TRUE, 'primary'::noderole, 'default')
|
INSERT INTO pg_dist_node (nodeid, groupid, nodename, nodeport, noderack, hasmetadata, metadatasynced, isactive, noderole, nodecluster) VALUES (1, 1, 'localhost', 57637, 'default', FALSE, FALSE, TRUE, 'primary'::noderole, 'default'),(2, 2, 'localhost', 57638, 'default', FALSE, FALSE, TRUE, 'primary'::noderole, 'default')
|
||||||
INSERT INTO pg_dist_partition (logicalrelid, partmethod, partkey, colocationid, repmodel) VALUES ('public.mx_test_table'::regclass, 'h', column_name_to_column('public.mx_test_table','col_1'), 0, 's')
|
SELECT citus_internal_add_partition_metadata ('public.mx_test_table'::regclass, 'h', 'col_1', 0, 's')
|
||||||
INSERT INTO pg_dist_placement (shardid, shardstate, shardlength, groupid, placementid) VALUES (1310000, 1, 0, 1, 100000),(1310001, 1, 0, 2, 100001),(1310002, 1, 0, 1, 100002),(1310003, 1, 0, 2, 100003),(1310004, 1, 0, 1, 100004),(1310005, 1, 0, 2, 100005),(1310006, 1, 0, 1, 100006),(1310007, 1, 0, 2, 100007)
|
|
||||||
INSERT INTO pg_dist_shard (logicalrelid, shardid, shardstorage, shardminvalue, shardmaxvalue) VALUES ('public.mx_test_table'::regclass, 1310000, 't', '-2147483648', '-1610612737'),('public.mx_test_table'::regclass, 1310001, 't', '-1610612736', '-1073741825'),('public.mx_test_table'::regclass, 1310002, 't', '-1073741824', '-536870913'),('public.mx_test_table'::regclass, 1310003, 't', '-536870912', '-1'),('public.mx_test_table'::regclass, 1310004, 't', '0', '536870911'),('public.mx_test_table'::regclass, 1310005, 't', '536870912', '1073741823'),('public.mx_test_table'::regclass, 1310006, 't', '1073741824', '1610612735'),('public.mx_test_table'::regclass, 1310007, 't', '1610612736', '2147483647')
|
|
||||||
SELECT pg_catalog.worker_record_sequence_dependency('public.mx_test_table_col_3_seq'::regclass,'public.mx_test_table'::regclass,'col_3')
|
SELECT pg_catalog.worker_record_sequence_dependency('public.mx_test_table_col_3_seq'::regclass,'public.mx_test_table'::regclass,'col_3')
|
||||||
SELECT worker_apply_sequence_command ('CREATE SEQUENCE IF NOT EXISTS public.mx_test_table_col_3_seq INCREMENT BY 1 MINVALUE 1 MAXVALUE 9223372036854775807 START WITH 1 NO CYCLE','bigint')
|
SELECT worker_apply_sequence_command ('CREATE SEQUENCE IF NOT EXISTS public.mx_test_table_col_3_seq INCREMENT BY 1 MINVALUE 1 MAXVALUE 9223372036854775807 START WITH 1 NO CYCLE','bigint')
|
||||||
SELECT worker_apply_sequence_command ('CREATE SEQUENCE IF NOT EXISTS public.user_defined_seq INCREMENT BY 1 MINVALUE 1 MAXVALUE 9223372036854775807 START WITH 1 NO CYCLE','bigint')
|
SELECT worker_apply_sequence_command ('CREATE SEQUENCE IF NOT EXISTS public.user_defined_seq INCREMENT BY 1 MINVALUE 1 MAXVALUE 9223372036854775807 START WITH 1 NO CYCLE','bigint')
|
||||||
SELECT worker_create_truncate_trigger('public.mx_test_table')
|
SELECT worker_create_truncate_trigger('public.mx_test_table')
|
||||||
SELECT worker_drop_distributed_table(logicalrelid::regclass::text) FROM pg_dist_partition
|
SELECT worker_drop_distributed_table(logicalrelid::regclass::text) FROM pg_dist_partition
|
||||||
TRUNCATE pg_dist_node CASCADE
|
TRUNCATE pg_dist_node CASCADE
|
||||||
|
WITH placement_data(shardid, shardstate, shardlength, groupid, placementid) AS (VALUES (1310000, 1, 0, 1, 100000), (1310001, 1, 0, 2, 100001), (1310002, 1, 0, 1, 100002), (1310003, 1, 0, 2, 100003), (1310004, 1, 0, 1, 100004), (1310005, 1, 0, 2, 100005), (1310006, 1, 0, 1, 100006), (1310007, 1, 0, 2, 100007)) SELECT citus_internal_add_placement_metadata(shardid, shardstate, shardlength, groupid, placementid) FROM placement_data;
|
||||||
|
WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) AS (VALUES ('public.mx_test_table'::regclass, 1310000, 't'::"char", '-2147483648', '-1610612737'), ('public.mx_test_table'::regclass, 1310001, 't'::"char", '-1610612736', '-1073741825'), ('public.mx_test_table'::regclass, 1310002, 't'::"char", '-1073741824', '-536870913'), ('public.mx_test_table'::regclass, 1310003, 't'::"char", '-536870912', '-1'), ('public.mx_test_table'::regclass, 1310004, 't'::"char", '0', '536870911'), ('public.mx_test_table'::regclass, 1310005, 't'::"char", '536870912', '1073741823'), ('public.mx_test_table'::regclass, 1310006, 't'::"char", '1073741824', '1610612735'), ('public.mx_test_table'::regclass, 1310007, 't'::"char", '1610612736', '2147483647')) SELECT citus_internal_add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data;
|
||||||
(17 rows)
|
(17 rows)
|
||||||
|
|
||||||
-- Show that schema changes are included in the metadata snapshot
|
-- Show that schema changes are included in the metadata snapshot
|
||||||
|
@ -128,15 +128,15 @@ SELECT unnest(master_metadata_snapshot()) order by 1;
|
||||||
CREATE INDEX mx_index ON mx_testing_schema.mx_test_table USING btree (col_2)
|
CREATE INDEX mx_index ON mx_testing_schema.mx_test_table USING btree (col_2)
|
||||||
CREATE TABLE mx_testing_schema.mx_test_table (col_1 integer, col_2 text NOT NULL, col_3 bigint DEFAULT nextval('mx_testing_schema.mx_test_table_col_3_seq'::regclass) NOT NULL, col_4 bigint DEFAULT nextval('public.user_defined_seq'::regclass))
|
CREATE TABLE mx_testing_schema.mx_test_table (col_1 integer, col_2 text NOT NULL, col_3 bigint DEFAULT nextval('mx_testing_schema.mx_test_table_col_3_seq'::regclass) NOT NULL, col_4 bigint DEFAULT nextval('public.user_defined_seq'::regclass))
|
||||||
INSERT INTO pg_dist_node (nodeid, groupid, nodename, nodeport, noderack, hasmetadata, metadatasynced, isactive, noderole, nodecluster) VALUES (1, 1, 'localhost', 57637, 'default', FALSE, FALSE, TRUE, 'primary'::noderole, 'default'),(2, 2, 'localhost', 57638, 'default', FALSE, FALSE, TRUE, 'primary'::noderole, 'default')
|
INSERT INTO pg_dist_node (nodeid, groupid, nodename, nodeport, noderack, hasmetadata, metadatasynced, isactive, noderole, nodecluster) VALUES (1, 1, 'localhost', 57637, 'default', FALSE, FALSE, TRUE, 'primary'::noderole, 'default'),(2, 2, 'localhost', 57638, 'default', FALSE, FALSE, TRUE, 'primary'::noderole, 'default')
|
||||||
INSERT INTO pg_dist_partition (logicalrelid, partmethod, partkey, colocationid, repmodel) VALUES ('mx_testing_schema.mx_test_table'::regclass, 'h', column_name_to_column('mx_testing_schema.mx_test_table','col_1'), 0, 's')
|
SELECT citus_internal_add_partition_metadata ('mx_testing_schema.mx_test_table'::regclass, 'h', 'col_1', 0, 's')
|
||||||
INSERT INTO pg_dist_placement (shardid, shardstate, shardlength, groupid, placementid) VALUES (1310000, 1, 0, 1, 100000),(1310001, 1, 0, 2, 100001),(1310002, 1, 0, 1, 100002),(1310003, 1, 0, 2, 100003),(1310004, 1, 0, 1, 100004),(1310005, 1, 0, 2, 100005),(1310006, 1, 0, 1, 100006),(1310007, 1, 0, 2, 100007)
|
|
||||||
INSERT INTO pg_dist_shard (logicalrelid, shardid, shardstorage, shardminvalue, shardmaxvalue) VALUES ('mx_testing_schema.mx_test_table'::regclass, 1310000, 't', '-2147483648', '-1610612737'),('mx_testing_schema.mx_test_table'::regclass, 1310001, 't', '-1610612736', '-1073741825'),('mx_testing_schema.mx_test_table'::regclass, 1310002, 't', '-1073741824', '-536870913'),('mx_testing_schema.mx_test_table'::regclass, 1310003, 't', '-536870912', '-1'),('mx_testing_schema.mx_test_table'::regclass, 1310004, 't', '0', '536870911'),('mx_testing_schema.mx_test_table'::regclass, 1310005, 't', '536870912', '1073741823'),('mx_testing_schema.mx_test_table'::regclass, 1310006, 't', '1073741824', '1610612735'),('mx_testing_schema.mx_test_table'::regclass, 1310007, 't', '1610612736', '2147483647')
|
|
||||||
SELECT pg_catalog.worker_record_sequence_dependency('mx_testing_schema.mx_test_table_col_3_seq'::regclass,'mx_testing_schema.mx_test_table'::regclass,'col_3')
|
SELECT pg_catalog.worker_record_sequence_dependency('mx_testing_schema.mx_test_table_col_3_seq'::regclass,'mx_testing_schema.mx_test_table'::regclass,'col_3')
|
||||||
SELECT worker_apply_sequence_command ('CREATE SEQUENCE IF NOT EXISTS mx_testing_schema.mx_test_table_col_3_seq INCREMENT BY 1 MINVALUE 1 MAXVALUE 9223372036854775807 START WITH 1 NO CYCLE','bigint')
|
SELECT worker_apply_sequence_command ('CREATE SEQUENCE IF NOT EXISTS mx_testing_schema.mx_test_table_col_3_seq INCREMENT BY 1 MINVALUE 1 MAXVALUE 9223372036854775807 START WITH 1 NO CYCLE','bigint')
|
||||||
SELECT worker_apply_sequence_command ('CREATE SEQUENCE IF NOT EXISTS public.user_defined_seq INCREMENT BY 1 MINVALUE 1 MAXVALUE 9223372036854775807 START WITH 1 NO CYCLE','bigint')
|
SELECT worker_apply_sequence_command ('CREATE SEQUENCE IF NOT EXISTS public.user_defined_seq INCREMENT BY 1 MINVALUE 1 MAXVALUE 9223372036854775807 START WITH 1 NO CYCLE','bigint')
|
||||||
SELECT worker_create_truncate_trigger('mx_testing_schema.mx_test_table')
|
SELECT worker_create_truncate_trigger('mx_testing_schema.mx_test_table')
|
||||||
SELECT worker_drop_distributed_table(logicalrelid::regclass::text) FROM pg_dist_partition
|
SELECT worker_drop_distributed_table(logicalrelid::regclass::text) FROM pg_dist_partition
|
||||||
TRUNCATE pg_dist_node CASCADE
|
TRUNCATE pg_dist_node CASCADE
|
||||||
|
WITH placement_data(shardid, shardstate, shardlength, groupid, placementid) AS (VALUES (1310000, 1, 0, 1, 100000), (1310001, 1, 0, 2, 100001), (1310002, 1, 0, 1, 100002), (1310003, 1, 0, 2, 100003), (1310004, 1, 0, 1, 100004), (1310005, 1, 0, 2, 100005), (1310006, 1, 0, 1, 100006), (1310007, 1, 0, 2, 100007)) SELECT citus_internal_add_placement_metadata(shardid, shardstate, shardlength, groupid, placementid) FROM placement_data;
|
||||||
|
WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) AS (VALUES ('mx_testing_schema.mx_test_table'::regclass, 1310000, 't'::"char", '-2147483648', '-1610612737'), ('mx_testing_schema.mx_test_table'::regclass, 1310001, 't'::"char", '-1610612736', '-1073741825'), ('mx_testing_schema.mx_test_table'::regclass, 1310002, 't'::"char", '-1073741824', '-536870913'), ('mx_testing_schema.mx_test_table'::regclass, 1310003, 't'::"char", '-536870912', '-1'), ('mx_testing_schema.mx_test_table'::regclass, 1310004, 't'::"char", '0', '536870911'), ('mx_testing_schema.mx_test_table'::regclass, 1310005, 't'::"char", '536870912', '1073741823'), ('mx_testing_schema.mx_test_table'::regclass, 1310006, 't'::"char", '1073741824', '1610612735'), ('mx_testing_schema.mx_test_table'::regclass, 1310007, 't'::"char", '1610612736', '2147483647')) SELECT citus_internal_add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data;
|
||||||
(17 rows)
|
(17 rows)
|
||||||
|
|
||||||
-- Show that append distributed tables are not included in the metadata snapshot
|
-- Show that append distributed tables are not included in the metadata snapshot
|
||||||
|
@ -159,15 +159,15 @@ SELECT unnest(master_metadata_snapshot()) order by 1;
|
||||||
CREATE INDEX mx_index ON mx_testing_schema.mx_test_table USING btree (col_2)
|
CREATE INDEX mx_index ON mx_testing_schema.mx_test_table USING btree (col_2)
|
||||||
CREATE TABLE mx_testing_schema.mx_test_table (col_1 integer, col_2 text NOT NULL, col_3 bigint DEFAULT nextval('mx_testing_schema.mx_test_table_col_3_seq'::regclass) NOT NULL, col_4 bigint DEFAULT nextval('public.user_defined_seq'::regclass))
|
CREATE TABLE mx_testing_schema.mx_test_table (col_1 integer, col_2 text NOT NULL, col_3 bigint DEFAULT nextval('mx_testing_schema.mx_test_table_col_3_seq'::regclass) NOT NULL, col_4 bigint DEFAULT nextval('public.user_defined_seq'::regclass))
|
||||||
INSERT INTO pg_dist_node (nodeid, groupid, nodename, nodeport, noderack, hasmetadata, metadatasynced, isactive, noderole, nodecluster) VALUES (1, 1, 'localhost', 57637, 'default', FALSE, FALSE, TRUE, 'primary'::noderole, 'default'),(2, 2, 'localhost', 57638, 'default', FALSE, FALSE, TRUE, 'primary'::noderole, 'default')
|
INSERT INTO pg_dist_node (nodeid, groupid, nodename, nodeport, noderack, hasmetadata, metadatasynced, isactive, noderole, nodecluster) VALUES (1, 1, 'localhost', 57637, 'default', FALSE, FALSE, TRUE, 'primary'::noderole, 'default'),(2, 2, 'localhost', 57638, 'default', FALSE, FALSE, TRUE, 'primary'::noderole, 'default')
|
||||||
INSERT INTO pg_dist_partition (logicalrelid, partmethod, partkey, colocationid, repmodel) VALUES ('mx_testing_schema.mx_test_table'::regclass, 'h', column_name_to_column('mx_testing_schema.mx_test_table','col_1'), 0, 's')
|
SELECT citus_internal_add_partition_metadata ('mx_testing_schema.mx_test_table'::regclass, 'h', 'col_1', 0, 's')
|
||||||
INSERT INTO pg_dist_placement (shardid, shardstate, shardlength, groupid, placementid) VALUES (1310000, 1, 0, 1, 100000),(1310001, 1, 0, 2, 100001),(1310002, 1, 0, 1, 100002),(1310003, 1, 0, 2, 100003),(1310004, 1, 0, 1, 100004),(1310005, 1, 0, 2, 100005),(1310006, 1, 0, 1, 100006),(1310007, 1, 0, 2, 100007)
|
|
||||||
INSERT INTO pg_dist_shard (logicalrelid, shardid, shardstorage, shardminvalue, shardmaxvalue) VALUES ('mx_testing_schema.mx_test_table'::regclass, 1310000, 't', '-2147483648', '-1610612737'),('mx_testing_schema.mx_test_table'::regclass, 1310001, 't', '-1610612736', '-1073741825'),('mx_testing_schema.mx_test_table'::regclass, 1310002, 't', '-1073741824', '-536870913'),('mx_testing_schema.mx_test_table'::regclass, 1310003, 't', '-536870912', '-1'),('mx_testing_schema.mx_test_table'::regclass, 1310004, 't', '0', '536870911'),('mx_testing_schema.mx_test_table'::regclass, 1310005, 't', '536870912', '1073741823'),('mx_testing_schema.mx_test_table'::regclass, 1310006, 't', '1073741824', '1610612735'),('mx_testing_schema.mx_test_table'::regclass, 1310007, 't', '1610612736', '2147483647')
|
|
||||||
SELECT pg_catalog.worker_record_sequence_dependency('mx_testing_schema.mx_test_table_col_3_seq'::regclass,'mx_testing_schema.mx_test_table'::regclass,'col_3')
|
SELECT pg_catalog.worker_record_sequence_dependency('mx_testing_schema.mx_test_table_col_3_seq'::regclass,'mx_testing_schema.mx_test_table'::regclass,'col_3')
|
||||||
SELECT worker_apply_sequence_command ('CREATE SEQUENCE IF NOT EXISTS mx_testing_schema.mx_test_table_col_3_seq INCREMENT BY 1 MINVALUE 1 MAXVALUE 9223372036854775807 START WITH 1 NO CYCLE','bigint')
|
SELECT worker_apply_sequence_command ('CREATE SEQUENCE IF NOT EXISTS mx_testing_schema.mx_test_table_col_3_seq INCREMENT BY 1 MINVALUE 1 MAXVALUE 9223372036854775807 START WITH 1 NO CYCLE','bigint')
|
||||||
SELECT worker_apply_sequence_command ('CREATE SEQUENCE IF NOT EXISTS public.user_defined_seq INCREMENT BY 1 MINVALUE 1 MAXVALUE 9223372036854775807 START WITH 1 NO CYCLE','bigint')
|
SELECT worker_apply_sequence_command ('CREATE SEQUENCE IF NOT EXISTS public.user_defined_seq INCREMENT BY 1 MINVALUE 1 MAXVALUE 9223372036854775807 START WITH 1 NO CYCLE','bigint')
|
||||||
SELECT worker_create_truncate_trigger('mx_testing_schema.mx_test_table')
|
SELECT worker_create_truncate_trigger('mx_testing_schema.mx_test_table')
|
||||||
SELECT worker_drop_distributed_table(logicalrelid::regclass::text) FROM pg_dist_partition
|
SELECT worker_drop_distributed_table(logicalrelid::regclass::text) FROM pg_dist_partition
|
||||||
TRUNCATE pg_dist_node CASCADE
|
TRUNCATE pg_dist_node CASCADE
|
||||||
|
WITH placement_data(shardid, shardstate, shardlength, groupid, placementid) AS (VALUES (1310000, 1, 0, 1, 100000), (1310001, 1, 0, 2, 100001), (1310002, 1, 0, 1, 100002), (1310003, 1, 0, 2, 100003), (1310004, 1, 0, 1, 100004), (1310005, 1, 0, 2, 100005), (1310006, 1, 0, 1, 100006), (1310007, 1, 0, 2, 100007)) SELECT citus_internal_add_placement_metadata(shardid, shardstate, shardlength, groupid, placementid) FROM placement_data;
|
||||||
|
WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) AS (VALUES ('mx_testing_schema.mx_test_table'::regclass, 1310000, 't'::"char", '-2147483648', '-1610612737'), ('mx_testing_schema.mx_test_table'::regclass, 1310001, 't'::"char", '-1610612736', '-1073741825'), ('mx_testing_schema.mx_test_table'::regclass, 1310002, 't'::"char", '-1073741824', '-536870913'), ('mx_testing_schema.mx_test_table'::regclass, 1310003, 't'::"char", '-536870912', '-1'), ('mx_testing_schema.mx_test_table'::regclass, 1310004, 't'::"char", '0', '536870911'), ('mx_testing_schema.mx_test_table'::regclass, 1310005, 't'::"char", '536870912', '1073741823'), ('mx_testing_schema.mx_test_table'::regclass, 1310006, 't'::"char", '1073741824', '1610612735'), ('mx_testing_schema.mx_test_table'::regclass, 1310007, 't'::"char", '1610612736', '2147483647')) SELECT citus_internal_add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data;
|
||||||
(17 rows)
|
(17 rows)
|
||||||
|
|
||||||
-- Show that range distributed tables are not included in the metadata snapshot
|
-- Show that range distributed tables are not included in the metadata snapshot
|
||||||
|
@ -183,15 +183,15 @@ SELECT unnest(master_metadata_snapshot()) order by 1;
|
||||||
CREATE INDEX mx_index ON mx_testing_schema.mx_test_table USING btree (col_2)
|
CREATE INDEX mx_index ON mx_testing_schema.mx_test_table USING btree (col_2)
|
||||||
CREATE TABLE mx_testing_schema.mx_test_table (col_1 integer, col_2 text NOT NULL, col_3 bigint DEFAULT nextval('mx_testing_schema.mx_test_table_col_3_seq'::regclass) NOT NULL, col_4 bigint DEFAULT nextval('public.user_defined_seq'::regclass))
|
CREATE TABLE mx_testing_schema.mx_test_table (col_1 integer, col_2 text NOT NULL, col_3 bigint DEFAULT nextval('mx_testing_schema.mx_test_table_col_3_seq'::regclass) NOT NULL, col_4 bigint DEFAULT nextval('public.user_defined_seq'::regclass))
|
||||||
INSERT INTO pg_dist_node (nodeid, groupid, nodename, nodeport, noderack, hasmetadata, metadatasynced, isactive, noderole, nodecluster) VALUES (1, 1, 'localhost', 57637, 'default', FALSE, FALSE, TRUE, 'primary'::noderole, 'default'),(2, 2, 'localhost', 57638, 'default', FALSE, FALSE, TRUE, 'primary'::noderole, 'default')
|
INSERT INTO pg_dist_node (nodeid, groupid, nodename, nodeport, noderack, hasmetadata, metadatasynced, isactive, noderole, nodecluster) VALUES (1, 1, 'localhost', 57637, 'default', FALSE, FALSE, TRUE, 'primary'::noderole, 'default'),(2, 2, 'localhost', 57638, 'default', FALSE, FALSE, TRUE, 'primary'::noderole, 'default')
|
||||||
INSERT INTO pg_dist_partition (logicalrelid, partmethod, partkey, colocationid, repmodel) VALUES ('mx_testing_schema.mx_test_table'::regclass, 'h', column_name_to_column('mx_testing_schema.mx_test_table','col_1'), 0, 's')
|
SELECT citus_internal_add_partition_metadata ('mx_testing_schema.mx_test_table'::regclass, 'h', 'col_1', 0, 's')
|
||||||
INSERT INTO pg_dist_placement (shardid, shardstate, shardlength, groupid, placementid) VALUES (1310000, 1, 0, 1, 100000),(1310001, 1, 0, 2, 100001),(1310002, 1, 0, 1, 100002),(1310003, 1, 0, 2, 100003),(1310004, 1, 0, 1, 100004),(1310005, 1, 0, 2, 100005),(1310006, 1, 0, 1, 100006),(1310007, 1, 0, 2, 100007)
|
|
||||||
INSERT INTO pg_dist_shard (logicalrelid, shardid, shardstorage, shardminvalue, shardmaxvalue) VALUES ('mx_testing_schema.mx_test_table'::regclass, 1310000, 't', '-2147483648', '-1610612737'),('mx_testing_schema.mx_test_table'::regclass, 1310001, 't', '-1610612736', '-1073741825'),('mx_testing_schema.mx_test_table'::regclass, 1310002, 't', '-1073741824', '-536870913'),('mx_testing_schema.mx_test_table'::regclass, 1310003, 't', '-536870912', '-1'),('mx_testing_schema.mx_test_table'::regclass, 1310004, 't', '0', '536870911'),('mx_testing_schema.mx_test_table'::regclass, 1310005, 't', '536870912', '1073741823'),('mx_testing_schema.mx_test_table'::regclass, 1310006, 't', '1073741824', '1610612735'),('mx_testing_schema.mx_test_table'::regclass, 1310007, 't', '1610612736', '2147483647')
|
|
||||||
SELECT pg_catalog.worker_record_sequence_dependency('mx_testing_schema.mx_test_table_col_3_seq'::regclass,'mx_testing_schema.mx_test_table'::regclass,'col_3')
|
SELECT pg_catalog.worker_record_sequence_dependency('mx_testing_schema.mx_test_table_col_3_seq'::regclass,'mx_testing_schema.mx_test_table'::regclass,'col_3')
|
||||||
SELECT worker_apply_sequence_command ('CREATE SEQUENCE IF NOT EXISTS mx_testing_schema.mx_test_table_col_3_seq INCREMENT BY 1 MINVALUE 1 MAXVALUE 9223372036854775807 START WITH 1 NO CYCLE','bigint')
|
SELECT worker_apply_sequence_command ('CREATE SEQUENCE IF NOT EXISTS mx_testing_schema.mx_test_table_col_3_seq INCREMENT BY 1 MINVALUE 1 MAXVALUE 9223372036854775807 START WITH 1 NO CYCLE','bigint')
|
||||||
SELECT worker_apply_sequence_command ('CREATE SEQUENCE IF NOT EXISTS public.user_defined_seq INCREMENT BY 1 MINVALUE 1 MAXVALUE 9223372036854775807 START WITH 1 NO CYCLE','bigint')
|
SELECT worker_apply_sequence_command ('CREATE SEQUENCE IF NOT EXISTS public.user_defined_seq INCREMENT BY 1 MINVALUE 1 MAXVALUE 9223372036854775807 START WITH 1 NO CYCLE','bigint')
|
||||||
SELECT worker_create_truncate_trigger('mx_testing_schema.mx_test_table')
|
SELECT worker_create_truncate_trigger('mx_testing_schema.mx_test_table')
|
||||||
SELECT worker_drop_distributed_table(logicalrelid::regclass::text) FROM pg_dist_partition
|
SELECT worker_drop_distributed_table(logicalrelid::regclass::text) FROM pg_dist_partition
|
||||||
TRUNCATE pg_dist_node CASCADE
|
TRUNCATE pg_dist_node CASCADE
|
||||||
|
WITH placement_data(shardid, shardstate, shardlength, groupid, placementid) AS (VALUES (1310000, 1, 0, 1, 100000), (1310001, 1, 0, 2, 100001), (1310002, 1, 0, 1, 100002), (1310003, 1, 0, 2, 100003), (1310004, 1, 0, 1, 100004), (1310005, 1, 0, 2, 100005), (1310006, 1, 0, 1, 100006), (1310007, 1, 0, 2, 100007)) SELECT citus_internal_add_placement_metadata(shardid, shardstate, shardlength, groupid, placementid) FROM placement_data;
|
||||||
|
WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) AS (VALUES ('mx_testing_schema.mx_test_table'::regclass, 1310000, 't'::"char", '-2147483648', '-1610612737'), ('mx_testing_schema.mx_test_table'::regclass, 1310001, 't'::"char", '-1610612736', '-1073741825'), ('mx_testing_schema.mx_test_table'::regclass, 1310002, 't'::"char", '-1073741824', '-536870913'), ('mx_testing_schema.mx_test_table'::regclass, 1310003, 't'::"char", '-536870912', '-1'), ('mx_testing_schema.mx_test_table'::regclass, 1310004, 't'::"char", '0', '536870911'), ('mx_testing_schema.mx_test_table'::regclass, 1310005, 't'::"char", '536870912', '1073741823'), ('mx_testing_schema.mx_test_table'::regclass, 1310006, 't'::"char", '1073741824', '1610612735'), ('mx_testing_schema.mx_test_table'::regclass, 1310007, 't'::"char", '1610612736', '2147483647')) SELECT citus_internal_add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data;
|
||||||
(17 rows)
|
(17 rows)
|
||||||
|
|
||||||
-- Test start_metadata_sync_to_node UDF
|
-- Test start_metadata_sync_to_node UDF
|
||||||
|
@ -1677,24 +1677,12 @@ SELECT unnest(master_metadata_snapshot()) order by 1;
|
||||||
CREATE TABLE public.mx_ref (col_1 integer, col_2 text)
|
CREATE TABLE public.mx_ref (col_1 integer, col_2 text)
|
||||||
CREATE TABLE public.test_table (id integer DEFAULT nextval('public.mx_test_sequence_0'::regclass), id2 integer DEFAULT nextval('public.mx_test_sequence_1'::regclass))
|
CREATE TABLE public.test_table (id integer DEFAULT nextval('public.mx_test_sequence_0'::regclass), id2 integer DEFAULT nextval('public.mx_test_sequence_1'::regclass))
|
||||||
INSERT INTO pg_dist_node (nodeid, groupid, nodename, nodeport, noderack, hasmetadata, metadatasynced, isactive, noderole, nodecluster) VALUES (4, 1, 'localhost', 8888, 'default', FALSE, FALSE, TRUE, 'secondary'::noderole, 'default'),(5, 1, 'localhost', 8889, 'default', FALSE, FALSE, TRUE, 'secondary'::noderole, 'second-cluster'),(1, 1, 'localhost', 57637, 'default', TRUE, TRUE, TRUE, 'primary'::noderole, 'default'),(7, 5, 'localhost', 57638, 'default', TRUE, TRUE, TRUE, 'primary'::noderole, 'default')
|
INSERT INTO pg_dist_node (nodeid, groupid, nodename, nodeport, noderack, hasmetadata, metadatasynced, isactive, noderole, nodecluster) VALUES (4, 1, 'localhost', 8888, 'default', FALSE, FALSE, TRUE, 'secondary'::noderole, 'default'),(5, 1, 'localhost', 8889, 'default', FALSE, FALSE, TRUE, 'secondary'::noderole, 'second-cluster'),(1, 1, 'localhost', 57637, 'default', TRUE, TRUE, TRUE, 'primary'::noderole, 'default'),(7, 5, 'localhost', 57638, 'default', TRUE, TRUE, TRUE, 'primary'::noderole, 'default')
|
||||||
INSERT INTO pg_dist_partition (logicalrelid, partmethod, partkey, colocationid, repmodel) VALUES ('mx_test_schema_1.mx_table_1'::regclass, 'h', column_name_to_column('mx_test_schema_1.mx_table_1','col1'), 3, 's')
|
SELECT citus_internal_add_partition_metadata ('mx_test_schema_1.mx_table_1'::regclass, 'h', 'col1', 3, 's')
|
||||||
INSERT INTO pg_dist_partition (logicalrelid, partmethod, partkey, colocationid, repmodel) VALUES ('mx_test_schema_2.mx_table_2'::regclass, 'h', column_name_to_column('mx_test_schema_2.mx_table_2','col1'), 3, 's')
|
SELECT citus_internal_add_partition_metadata ('mx_test_schema_2.mx_table_2'::regclass, 'h', 'col1', 3, 's')
|
||||||
INSERT INTO pg_dist_partition (logicalrelid, partmethod, partkey, colocationid, repmodel) VALUES ('mx_testing_schema.mx_test_table'::regclass, 'h', column_name_to_column('mx_testing_schema.mx_test_table','col_1'), 0, 's')
|
SELECT citus_internal_add_partition_metadata ('mx_testing_schema.mx_test_table'::regclass, 'h', 'col_1', 0, 's')
|
||||||
INSERT INTO pg_dist_partition (logicalrelid, partmethod, partkey, colocationid, repmodel) VALUES ('public.dist_table_1'::regclass, 'h', column_name_to_column('public.dist_table_1','a'), 10004, 's')
|
SELECT citus_internal_add_partition_metadata ('public.dist_table_1'::regclass, 'h', 'a', 10004, 's')
|
||||||
INSERT INTO pg_dist_partition (logicalrelid, partmethod, partkey, colocationid, repmodel) VALUES ('public.mx_ref'::regclass, 'n', NULL, 10002, 't')
|
SELECT citus_internal_add_partition_metadata ('public.mx_ref'::regclass, 'n', NULL, 10002, 't')
|
||||||
INSERT INTO pg_dist_partition (logicalrelid, partmethod, partkey, colocationid, repmodel) VALUES ('public.test_table'::regclass, 'h', column_name_to_column('public.test_table','id'), 10004, 's')
|
SELECT citus_internal_add_partition_metadata ('public.test_table'::regclass, 'h', 'id', 10004, 's')
|
||||||
INSERT INTO pg_dist_placement (shardid, shardstate, shardlength, groupid, placementid) VALUES (1310000, 1, 0, 1, 100000),(1310001, 1, 0, 5, 100001),(1310002, 1, 0, 1, 100002),(1310003, 1, 0, 5, 100003),(1310004, 1, 0, 1, 100004),(1310005, 1, 0, 5, 100005),(1310006, 1, 0, 1, 100006),(1310007, 1, 0, 5, 100007)
|
|
||||||
INSERT INTO pg_dist_placement (shardid, shardstate, shardlength, groupid, placementid) VALUES (1310020, 1, 0, 1, 100020),(1310021, 1, 0, 5, 100021),(1310022, 1, 0, 1, 100022),(1310023, 1, 0, 5, 100023),(1310024, 1, 0, 1, 100024)
|
|
||||||
INSERT INTO pg_dist_placement (shardid, shardstate, shardlength, groupid, placementid) VALUES (1310025, 1, 0, 1, 100025),(1310026, 1, 0, 5, 100026),(1310027, 1, 0, 1, 100027),(1310028, 1, 0, 5, 100028),(1310029, 1, 0, 1, 100029)
|
|
||||||
INSERT INTO pg_dist_placement (shardid, shardstate, shardlength, groupid, placementid) VALUES (1310073, 1, 0, 1, 100074),(1310073, 1, 0, 5, 100075)
|
|
||||||
INSERT INTO pg_dist_placement (shardid, shardstate, shardlength, groupid, placementid) VALUES (1310074, 1, 0, 1, 100076),(1310075, 1, 0, 5, 100077),(1310076, 1, 0, 1, 100078),(1310077, 1, 0, 5, 100079)
|
|
||||||
INSERT INTO pg_dist_placement (shardid, shardstate, shardlength, groupid, placementid) VALUES (1310083, 1, 0, 1, 100086),(1310084, 1, 0, 5, 100087),(1310085, 1, 0, 1, 100088),(1310086, 1, 0, 5, 100089)
|
|
||||||
INSERT INTO pg_dist_shard (logicalrelid, shardid, shardstorage, shardminvalue, shardmaxvalue) VALUES ('mx_test_schema_1.mx_table_1'::regclass, 1310020, 't', '-2147483648', '-1288490190'),('mx_test_schema_1.mx_table_1'::regclass, 1310021, 't', '-1288490189', '-429496731'),('mx_test_schema_1.mx_table_1'::regclass, 1310022, 't', '-429496730', '429496728'),('mx_test_schema_1.mx_table_1'::regclass, 1310023, 't', '429496729', '1288490187'),('mx_test_schema_1.mx_table_1'::regclass, 1310024, 't', '1288490188', '2147483647')
|
|
||||||
INSERT INTO pg_dist_shard (logicalrelid, shardid, shardstorage, shardminvalue, shardmaxvalue) VALUES ('mx_test_schema_2.mx_table_2'::regclass, 1310025, 't', '-2147483648', '-1288490190'),('mx_test_schema_2.mx_table_2'::regclass, 1310026, 't', '-1288490189', '-429496731'),('mx_test_schema_2.mx_table_2'::regclass, 1310027, 't', '-429496730', '429496728'),('mx_test_schema_2.mx_table_2'::regclass, 1310028, 't', '429496729', '1288490187'),('mx_test_schema_2.mx_table_2'::regclass, 1310029, 't', '1288490188', '2147483647')
|
|
||||||
INSERT INTO pg_dist_shard (logicalrelid, shardid, shardstorage, shardminvalue, shardmaxvalue) VALUES ('mx_testing_schema.mx_test_table'::regclass, 1310000, 't', '-2147483648', '-1610612737'),('mx_testing_schema.mx_test_table'::regclass, 1310001, 't', '-1610612736', '-1073741825'),('mx_testing_schema.mx_test_table'::regclass, 1310002, 't', '-1073741824', '-536870913'),('mx_testing_schema.mx_test_table'::regclass, 1310003, 't', '-536870912', '-1'),('mx_testing_schema.mx_test_table'::regclass, 1310004, 't', '0', '536870911'),('mx_testing_schema.mx_test_table'::regclass, 1310005, 't', '536870912', '1073741823'),('mx_testing_schema.mx_test_table'::regclass, 1310006, 't', '1073741824', '1610612735'),('mx_testing_schema.mx_test_table'::regclass, 1310007, 't', '1610612736', '2147483647')
|
|
||||||
INSERT INTO pg_dist_shard (logicalrelid, shardid, shardstorage, shardminvalue, shardmaxvalue) VALUES ('public.dist_table_1'::regclass, 1310074, 't', '-2147483648', '-1073741825'),('public.dist_table_1'::regclass, 1310075, 't', '-1073741824', '-1'),('public.dist_table_1'::regclass, 1310076, 't', '0', '1073741823'),('public.dist_table_1'::regclass, 1310077, 't', '1073741824', '2147483647')
|
|
||||||
INSERT INTO pg_dist_shard (logicalrelid, shardid, shardstorage, shardminvalue, shardmaxvalue) VALUES ('public.mx_ref'::regclass, 1310073, 't', NULL, NULL)
|
|
||||||
INSERT INTO pg_dist_shard (logicalrelid, shardid, shardstorage, shardminvalue, shardmaxvalue) VALUES ('public.test_table'::regclass, 1310083, 't', '-2147483648', '-1073741825'),('public.test_table'::regclass, 1310084, 't', '-1073741824', '-1'),('public.test_table'::regclass, 1310085, 't', '0', '1073741823'),('public.test_table'::regclass, 1310086, 't', '1073741824', '2147483647')
|
|
||||||
SELECT pg_catalog.worker_record_sequence_dependency('mx_testing_schema.mx_test_table_col_3_seq'::regclass,'mx_testing_schema.mx_test_table'::regclass,'col_3')
|
SELECT pg_catalog.worker_record_sequence_dependency('mx_testing_schema.mx_test_table_col_3_seq'::regclass,'mx_testing_schema.mx_test_table'::regclass,'col_3')
|
||||||
SELECT worker_apply_sequence_command ('CREATE SEQUENCE IF NOT EXISTS mx_testing_schema.mx_test_table_col_3_seq INCREMENT BY 1 MINVALUE 1 MAXVALUE 9223372036854775807 START WITH 1 NO CYCLE','bigint')
|
SELECT worker_apply_sequence_command ('CREATE SEQUENCE IF NOT EXISTS mx_testing_schema.mx_test_table_col_3_seq INCREMENT BY 1 MINVALUE 1 MAXVALUE 9223372036854775807 START WITH 1 NO CYCLE','bigint')
|
||||||
SELECT worker_apply_sequence_command ('CREATE SEQUENCE IF NOT EXISTS public.mx_test_sequence_0 INCREMENT BY 1 MINVALUE 1 MAXVALUE 2147483647 START WITH 1 NO CYCLE','integer')
|
SELECT worker_apply_sequence_command ('CREATE SEQUENCE IF NOT EXISTS public.mx_test_sequence_0 INCREMENT BY 1 MINVALUE 1 MAXVALUE 2147483647 START WITH 1 NO CYCLE','integer')
|
||||||
|
@ -1708,6 +1696,18 @@ SELECT unnest(master_metadata_snapshot()) order by 1;
|
||||||
SELECT worker_create_truncate_trigger('public.test_table')
|
SELECT worker_create_truncate_trigger('public.test_table')
|
||||||
SELECT worker_drop_distributed_table(logicalrelid::regclass::text) FROM pg_dist_partition
|
SELECT worker_drop_distributed_table(logicalrelid::regclass::text) FROM pg_dist_partition
|
||||||
TRUNCATE pg_dist_node CASCADE
|
TRUNCATE pg_dist_node CASCADE
|
||||||
|
WITH placement_data(shardid, shardstate, shardlength, groupid, placementid) AS (VALUES (1310000, 1, 0, 1, 100000), (1310001, 1, 0, 5, 100001), (1310002, 1, 0, 1, 100002), (1310003, 1, 0, 5, 100003), (1310004, 1, 0, 1, 100004), (1310005, 1, 0, 5, 100005), (1310006, 1, 0, 1, 100006), (1310007, 1, 0, 5, 100007)) SELECT citus_internal_add_placement_metadata(shardid, shardstate, shardlength, groupid, placementid) FROM placement_data;
|
||||||
|
WITH placement_data(shardid, shardstate, shardlength, groupid, placementid) AS (VALUES (1310020, 1, 0, 1, 100020), (1310021, 1, 0, 5, 100021), (1310022, 1, 0, 1, 100022), (1310023, 1, 0, 5, 100023), (1310024, 1, 0, 1, 100024)) SELECT citus_internal_add_placement_metadata(shardid, shardstate, shardlength, groupid, placementid) FROM placement_data;
|
||||||
|
WITH placement_data(shardid, shardstate, shardlength, groupid, placementid) AS (VALUES (1310025, 1, 0, 1, 100025), (1310026, 1, 0, 5, 100026), (1310027, 1, 0, 1, 100027), (1310028, 1, 0, 5, 100028), (1310029, 1, 0, 1, 100029)) SELECT citus_internal_add_placement_metadata(shardid, shardstate, shardlength, groupid, placementid) FROM placement_data;
|
||||||
|
WITH placement_data(shardid, shardstate, shardlength, groupid, placementid) AS (VALUES (1310073, 1, 0, 1, 100074), (1310073, 1, 0, 5, 100075)) SELECT citus_internal_add_placement_metadata(shardid, shardstate, shardlength, groupid, placementid) FROM placement_data;
|
||||||
|
WITH placement_data(shardid, shardstate, shardlength, groupid, placementid) AS (VALUES (1310074, 1, 0, 1, 100076), (1310075, 1, 0, 5, 100077), (1310076, 1, 0, 1, 100078), (1310077, 1, 0, 5, 100079)) SELECT citus_internal_add_placement_metadata(shardid, shardstate, shardlength, groupid, placementid) FROM placement_data;
|
||||||
|
WITH placement_data(shardid, shardstate, shardlength, groupid, placementid) AS (VALUES (1310083, 1, 0, 1, 100086), (1310084, 1, 0, 5, 100087), (1310085, 1, 0, 1, 100088), (1310086, 1, 0, 5, 100089)) SELECT citus_internal_add_placement_metadata(shardid, shardstate, shardlength, groupid, placementid) FROM placement_data;
|
||||||
|
WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) AS (VALUES ('mx_test_schema_1.mx_table_1'::regclass, 1310020, 't'::"char", '-2147483648', '-1288490190'), ('mx_test_schema_1.mx_table_1'::regclass, 1310021, 't'::"char", '-1288490189', '-429496731'), ('mx_test_schema_1.mx_table_1'::regclass, 1310022, 't'::"char", '-429496730', '429496728'), ('mx_test_schema_1.mx_table_1'::regclass, 1310023, 't'::"char", '429496729', '1288490187'), ('mx_test_schema_1.mx_table_1'::regclass, 1310024, 't'::"char", '1288490188', '2147483647')) SELECT citus_internal_add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data;
|
||||||
|
WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) AS (VALUES ('mx_test_schema_2.mx_table_2'::regclass, 1310025, 't'::"char", '-2147483648', '-1288490190'), ('mx_test_schema_2.mx_table_2'::regclass, 1310026, 't'::"char", '-1288490189', '-429496731'), ('mx_test_schema_2.mx_table_2'::regclass, 1310027, 't'::"char", '-429496730', '429496728'), ('mx_test_schema_2.mx_table_2'::regclass, 1310028, 't'::"char", '429496729', '1288490187'), ('mx_test_schema_2.mx_table_2'::regclass, 1310029, 't'::"char", '1288490188', '2147483647')) SELECT citus_internal_add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data;
|
||||||
|
WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) AS (VALUES ('mx_testing_schema.mx_test_table'::regclass, 1310000, 't'::"char", '-2147483648', '-1610612737'), ('mx_testing_schema.mx_test_table'::regclass, 1310001, 't'::"char", '-1610612736', '-1073741825'), ('mx_testing_schema.mx_test_table'::regclass, 1310002, 't'::"char", '-1073741824', '-536870913'), ('mx_testing_schema.mx_test_table'::regclass, 1310003, 't'::"char", '-536870912', '-1'), ('mx_testing_schema.mx_test_table'::regclass, 1310004, 't'::"char", '0', '536870911'), ('mx_testing_schema.mx_test_table'::regclass, 1310005, 't'::"char", '536870912', '1073741823'), ('mx_testing_schema.mx_test_table'::regclass, 1310006, 't'::"char", '1073741824', '1610612735'), ('mx_testing_schema.mx_test_table'::regclass, 1310007, 't'::"char", '1610612736', '2147483647')) SELECT citus_internal_add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data;
|
||||||
|
WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) AS (VALUES ('public.dist_table_1'::regclass, 1310074, 't'::"char", '-2147483648', '-1073741825'), ('public.dist_table_1'::regclass, 1310075, 't'::"char", '-1073741824', '-1'), ('public.dist_table_1'::regclass, 1310076, 't'::"char", '0', '1073741823'), ('public.dist_table_1'::regclass, 1310077, 't'::"char", '1073741824', '2147483647')) SELECT citus_internal_add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data;
|
||||||
|
WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) AS (VALUES ('public.mx_ref'::regclass, 1310073, 't'::"char", NULL, NULL)) SELECT citus_internal_add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data;
|
||||||
|
WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) AS (VALUES ('public.test_table'::regclass, 1310083, 't'::"char", '-2147483648', '-1073741825'), ('public.test_table'::regclass, 1310084, 't'::"char", '-1073741824', '-1'), ('public.test_table'::regclass, 1310085, 't'::"char", '0', '1073741823'), ('public.test_table'::regclass, 1310086, 't'::"char", '1073741824', '2147483647')) SELECT citus_internal_add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data;
|
||||||
(62 rows)
|
(62 rows)
|
||||||
|
|
||||||
-- shouldn't work since test_table is MX
|
-- shouldn't work since test_table is MX
|
||||||
|
|
|
@ -1304,6 +1304,7 @@ CREATE TABLE multi_modifications.local (a int default 1, b int);
|
||||||
INSERT INTO multi_modifications.local VALUES (default, (SELECT min(id) FROM summary_table));
|
INSERT INTO multi_modifications.local VALUES (default, (SELECT min(id) FROM summary_table));
|
||||||
ERROR: subqueries are not supported within INSERT queries
|
ERROR: subqueries are not supported within INSERT queries
|
||||||
HINT: Try rewriting your queries with 'INSERT INTO ... SELECT' syntax.
|
HINT: Try rewriting your queries with 'INSERT INTO ... SELECT' syntax.
|
||||||
|
DROP TABLE insufficient_shards;
|
||||||
DROP TABLE raw_table;
|
DROP TABLE raw_table;
|
||||||
DROP TABLE summary_table;
|
DROP TABLE summary_table;
|
||||||
DROP TABLE reference_raw_table;
|
DROP TABLE reference_raw_table;
|
||||||
|
|
|
@ -0,0 +1,535 @@
|
||||||
|
CREATE SCHEMA "Mx Regular User";
|
||||||
|
SET search_path TO "Mx Regular User";
|
||||||
|
-- sync the metadata to both nodes
|
||||||
|
SELECT start_metadata_sync_to_node('localhost', :worker_1_port);
|
||||||
|
start_metadata_sync_to_node
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
SELECT start_metadata_sync_to_node('localhost', :worker_2_port);
|
||||||
|
start_metadata_sync_to_node
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
-- create a role and give access one each node separately
|
||||||
|
-- and increase the error level to prevent enterprise to diverge
|
||||||
|
SET client_min_messages TO ERROR;
|
||||||
|
SET citus.enable_ddl_propagation TO OFF;
|
||||||
|
CREATE USER regular_mx_user WITH LOGIN;
|
||||||
|
GRANT ALL ON SCHEMA "Mx Regular User" TO regular_mx_user;
|
||||||
|
\c - postgres - :worker_1_port;
|
||||||
|
SET client_min_messages TO ERROR;
|
||||||
|
SET citus.enable_ddl_propagation TO OFF;
|
||||||
|
CREATE SCHEMA "Mx Regular User";
|
||||||
|
CREATE USER regular_mx_user WITH LOGIN;
|
||||||
|
GRANT ALL ON SCHEMA "Mx Regular User" TO regular_mx_user;
|
||||||
|
\c - postgres - :worker_2_port;
|
||||||
|
SET client_min_messages TO ERROR;
|
||||||
|
SET citus.enable_ddl_propagation TO OFF;
|
||||||
|
CREATE SCHEMA "Mx Regular User";
|
||||||
|
CREATE USER regular_mx_user WITH LOGIN;
|
||||||
|
GRANT ALL ON SCHEMA "Mx Regular User" TO regular_mx_user;
|
||||||
|
-- now connect with that user
|
||||||
|
\c - regular_mx_user - :master_port
|
||||||
|
SET search_path TO "Mx Regular User";
|
||||||
|
SET citus.next_shard_id TO 1560000;
|
||||||
|
-- make sure that we sync the metadata
|
||||||
|
SET citus.shard_replication_factor TO 1;
|
||||||
|
CREATE TABLE partitioned_table (long_column_names_1 int, long_column_names_2 int, long_column_names_3 int, long_column_names_4 int, long_column_names_5 int, long_column_names_6 int, long_column_names_7 int, long_column_names_8 int, long_column_names_9 int, long_column_names_10 int, long_column_names_11 timestamp) PARTITION BY RANGE (long_column_names_11);
|
||||||
|
CREATE TABLE very_long_child_partition_name_is_required_to_repro_the_bug PARTITION OF partitioned_table FOR VALUES FROM ('2011-01-01') TO ('2012-01-01');
|
||||||
|
SELECT create_distributed_table('partitioned_table', 'long_column_names_1');
|
||||||
|
create_distributed_table
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
SELECT bool_and(hasmetadata) FROM pg_dist_node WHERE nodename = 'localhost' and nodeport IN (:worker_1_port, :worker_2_port);
|
||||||
|
bool_and
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
t
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
-- show that we can rollback
|
||||||
|
BEGIN;
|
||||||
|
CREATE INDEX long_index_on_parent_table ON partitioned_table (long_column_names_1, long_column_names_2, long_column_names_3, long_column_names_4, long_column_names_5, long_column_names_6, long_column_names_11) INCLUDE (long_column_names_7, long_column_names_7, long_column_names_9, long_column_names_10);
|
||||||
|
ROLLBACK;
|
||||||
|
-- show that we can switch to sequential mode and still
|
||||||
|
-- sync the metadata to the nodes
|
||||||
|
BEGIN;
|
||||||
|
CREATE INDEX long_index_on_parent_table ON partitioned_table (long_column_names_1, long_column_names_2, long_column_names_3, long_column_names_4, long_column_names_5, long_column_names_6, long_column_names_11) INCLUDE (long_column_names_7, long_column_names_7, long_column_names_9, long_column_names_10);
|
||||||
|
show citus.multi_shard_modify_mode;
|
||||||
|
citus.multi_shard_modify_mode
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
sequential
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
COMMIT;
|
||||||
|
-- make sure that partitioned tables, columnar and conversion to columnar workes fine
|
||||||
|
-- on Citus MX with a non-super user
|
||||||
|
CREATE SEQUENCE my_mx_seq;
|
||||||
|
CREATE TABLE users_table_part(col_to_drop int, user_id int, value_1 int, value_2 int DEFAULT nextval('my_mx_seq'::regclass), value_3 bigserial) PARTITION BY RANGE (value_1);
|
||||||
|
CREATE TABLE users_table_part_0 PARTITION OF users_table_part FOR VALUES FROM (0) TO (1);
|
||||||
|
CREATE TABLE users_table_part_1 PARTITION OF users_table_part FOR VALUES FROM (1) TO (2);
|
||||||
|
SELECT create_distributed_table('users_table_part', 'user_id', colocate_with:='partitioned_table');
|
||||||
|
create_distributed_table
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
-- make sure that we can handle dropped columns nicely
|
||||||
|
ALTER TABLE users_table_part DROP COLUMN col_to_drop;
|
||||||
|
INSERT INTO users_table_part SELECT i, i %2, i %50 FROM generate_series(0, 100) i;
|
||||||
|
BEGIN;
|
||||||
|
-- make sure to use multiple connections
|
||||||
|
SET LOCAL citus.force_max_query_parallelization TO ON;
|
||||||
|
CREATE TABLE users_table_part_2 PARTITION OF users_table_part FOR VALUES FROM (2) TO (3);
|
||||||
|
INSERT INTO users_table_part SELECT i, i %3, i %50 FROM generate_series(0, 100) i;
|
||||||
|
CREATE TABLE users_table_part_3 (user_id int, value_1 int, value_2 int, value_3 bigserial);
|
||||||
|
ALTER TABLE users_table_part ATTACH PARTITION users_table_part_3 FOR VALUES FROM (3) TO (4);
|
||||||
|
CREATE TABLE users_table_part_4 PARTITION OF users_table_part FOR VALUES FROM (4) TO (5) USING COLUMNAR;;
|
||||||
|
COMMIT;
|
||||||
|
SELECT alter_table_set_access_method('users_table_part_0', 'columnar');
|
||||||
|
NOTICE: creating a new table for "Mx Regular User".users_table_part_0
|
||||||
|
NOTICE: moving the data of "Mx Regular User".users_table_part_0
|
||||||
|
NOTICE: dropping the old "Mx Regular User".users_table_part_0
|
||||||
|
NOTICE: renaming the new table to "Mx Regular User".users_table_part_0
|
||||||
|
alter_table_set_access_method
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
SELECT alter_table_set_access_method('users_table_part_0', 'heap');
|
||||||
|
NOTICE: creating a new table for "Mx Regular User".users_table_part_0
|
||||||
|
NOTICE: moving the data of "Mx Regular User".users_table_part_0
|
||||||
|
NOTICE: dropping the old "Mx Regular User".users_table_part_0
|
||||||
|
NOTICE: renaming the new table to "Mx Regular User".users_table_part_0
|
||||||
|
alter_table_set_access_method
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
BEGIN;
|
||||||
|
SET LOCAL citus.force_max_query_parallelization TO ON;
|
||||||
|
SELECT alter_table_set_access_method('users_table_part_0', 'columnar');
|
||||||
|
NOTICE: creating a new table for "Mx Regular User".users_table_part_0
|
||||||
|
NOTICE: moving the data of "Mx Regular User".users_table_part_0
|
||||||
|
NOTICE: dropping the old "Mx Regular User".users_table_part_0
|
||||||
|
NOTICE: renaming the new table to "Mx Regular User".users_table_part_0
|
||||||
|
alter_table_set_access_method
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
SELECT alter_table_set_access_method('users_table_part_0', 'heap');
|
||||||
|
NOTICE: creating a new table for "Mx Regular User".users_table_part_0
|
||||||
|
NOTICE: moving the data of "Mx Regular User".users_table_part_0
|
||||||
|
NOTICE: dropping the old "Mx Regular User".users_table_part_0
|
||||||
|
NOTICE: renaming the new table to "Mx Regular User".users_table_part_0
|
||||||
|
alter_table_set_access_method
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
ROLLBACK;
|
||||||
|
BEGIN;
|
||||||
|
SET LOCAL citus.force_max_query_parallelization TO ON;
|
||||||
|
SELECT undistribute_table('users_table_part');
|
||||||
|
NOTICE: converting the partitions of "Mx Regular User".users_table_part
|
||||||
|
NOTICE: creating a new table for "Mx Regular User".users_table_part_0
|
||||||
|
NOTICE: moving the data of "Mx Regular User".users_table_part_0
|
||||||
|
NOTICE: dropping the old "Mx Regular User".users_table_part_0
|
||||||
|
NOTICE: renaming the new table to "Mx Regular User".users_table_part_0
|
||||||
|
NOTICE: creating a new table for "Mx Regular User".users_table_part_1
|
||||||
|
NOTICE: moving the data of "Mx Regular User".users_table_part_1
|
||||||
|
NOTICE: dropping the old "Mx Regular User".users_table_part_1
|
||||||
|
NOTICE: renaming the new table to "Mx Regular User".users_table_part_1
|
||||||
|
NOTICE: creating a new table for "Mx Regular User".users_table_part_2
|
||||||
|
NOTICE: moving the data of "Mx Regular User".users_table_part_2
|
||||||
|
NOTICE: dropping the old "Mx Regular User".users_table_part_2
|
||||||
|
NOTICE: renaming the new table to "Mx Regular User".users_table_part_2
|
||||||
|
NOTICE: creating a new table for "Mx Regular User".users_table_part_3
|
||||||
|
NOTICE: moving the data of "Mx Regular User".users_table_part_3
|
||||||
|
NOTICE: dropping the old "Mx Regular User".users_table_part_3
|
||||||
|
NOTICE: renaming the new table to "Mx Regular User".users_table_part_3
|
||||||
|
NOTICE: creating a new table for "Mx Regular User".users_table_part_4
|
||||||
|
NOTICE: moving the data of "Mx Regular User".users_table_part_4
|
||||||
|
NOTICE: dropping the old "Mx Regular User".users_table_part_4
|
||||||
|
NOTICE: renaming the new table to "Mx Regular User".users_table_part_4
|
||||||
|
NOTICE: creating a new table for "Mx Regular User".users_table_part
|
||||||
|
NOTICE: dropping the old "Mx Regular User".users_table_part
|
||||||
|
NOTICE: renaming the new table to "Mx Regular User".users_table_part
|
||||||
|
undistribute_table
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
SELECT create_distributed_table('users_table_part', 'user_id');
|
||||||
|
NOTICE: Copying data from local table...
|
||||||
|
NOTICE: copying the data has completed
|
||||||
|
DETAIL: The local data in the table is no longer visible, but is still on disk.
|
||||||
|
HINT: To remove the local data, run: SELECT truncate_local_data_after_distributing_table($$"Mx Regular User".users_table_part_0$$)
|
||||||
|
NOTICE: Copying data from local table...
|
||||||
|
NOTICE: copying the data has completed
|
||||||
|
DETAIL: The local data in the table is no longer visible, but is still on disk.
|
||||||
|
HINT: To remove the local data, run: SELECT truncate_local_data_after_distributing_table($$"Mx Regular User".users_table_part_1$$)
|
||||||
|
NOTICE: Copying data from local table...
|
||||||
|
NOTICE: copying the data has completed
|
||||||
|
DETAIL: The local data in the table is no longer visible, but is still on disk.
|
||||||
|
HINT: To remove the local data, run: SELECT truncate_local_data_after_distributing_table($$"Mx Regular User".users_table_part_2$$)
|
||||||
|
create_distributed_table
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
COMMIT;
|
||||||
|
BEGIN;
|
||||||
|
-- make sure to use multiple connections
|
||||||
|
SET LOCAL citus.force_max_query_parallelization TO ON;
|
||||||
|
SELECT alter_distributed_table('users_table_part', shard_count:=9, cascade_to_colocated:=false);
|
||||||
|
NOTICE: converting the partitions of "Mx Regular User".users_table_part
|
||||||
|
NOTICE: creating a new table for "Mx Regular User".users_table_part_0
|
||||||
|
NOTICE: moving the data of "Mx Regular User".users_table_part_0
|
||||||
|
NOTICE: dropping the old "Mx Regular User".users_table_part_0
|
||||||
|
NOTICE: renaming the new table to "Mx Regular User".users_table_part_0
|
||||||
|
NOTICE: creating a new table for "Mx Regular User".users_table_part_1
|
||||||
|
NOTICE: moving the data of "Mx Regular User".users_table_part_1
|
||||||
|
NOTICE: dropping the old "Mx Regular User".users_table_part_1
|
||||||
|
NOTICE: renaming the new table to "Mx Regular User".users_table_part_1
|
||||||
|
NOTICE: creating a new table for "Mx Regular User".users_table_part_2
|
||||||
|
NOTICE: moving the data of "Mx Regular User".users_table_part_2
|
||||||
|
NOTICE: dropping the old "Mx Regular User".users_table_part_2
|
||||||
|
NOTICE: renaming the new table to "Mx Regular User".users_table_part_2
|
||||||
|
NOTICE: creating a new table for "Mx Regular User".users_table_part_3
|
||||||
|
NOTICE: moving the data of "Mx Regular User".users_table_part_3
|
||||||
|
NOTICE: dropping the old "Mx Regular User".users_table_part_3
|
||||||
|
NOTICE: renaming the new table to "Mx Regular User".users_table_part_3
|
||||||
|
NOTICE: creating a new table for "Mx Regular User".users_table_part_4
|
||||||
|
NOTICE: moving the data of "Mx Regular User".users_table_part_4
|
||||||
|
NOTICE: dropping the old "Mx Regular User".users_table_part_4
|
||||||
|
NOTICE: renaming the new table to "Mx Regular User".users_table_part_4
|
||||||
|
NOTICE: creating a new table for "Mx Regular User".users_table_part
|
||||||
|
NOTICE: dropping the old "Mx Regular User".users_table_part
|
||||||
|
NOTICE: renaming the new table to "Mx Regular User".users_table_part
|
||||||
|
alter_distributed_table
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
ROLLBACK;
|
||||||
|
BEGIN;
|
||||||
|
-- make sure to use multiple connections
|
||||||
|
SET LOCAL citus.force_max_query_parallelization TO ON;
|
||||||
|
ALTER TABLE users_table_part ADD COLUMN my_column INT DEFAULT 15;
|
||||||
|
CREATE INDEX test_index ON users_table_part(value_3, value_2);
|
||||||
|
CREATE INDEX test_index_on_child ON users_table_part_3(value_3, value_2);
|
||||||
|
ROLLBACK;
|
||||||
|
CREATE TABLE local_table_in_the_metadata (id int PRIMARY KEY, value_1 int);
|
||||||
|
CREATE TABLE reference_table(id int PRIMARY KEY, value_1 int);
|
||||||
|
SELECT create_reference_table('reference_table');
|
||||||
|
create_reference_table
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
CREATE TABLE on_delete_fkey_table(id int PRIMARY KEY, value_1 int);
|
||||||
|
SELECT create_distributed_table('on_delete_fkey_table', 'id', colocate_with:='partitioned_table');
|
||||||
|
create_distributed_table
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
ALTER TABLE reference_table ADD CONSTRAINT fkey_to_local FOREIGN KEY(id) REFERENCES local_table_in_the_metadata(id);
|
||||||
|
ALTER TABLE on_delete_fkey_table ADD CONSTRAINT veerrrrrrryyy_veerrrrrrryyy_veerrrrrrryyy_long_constraint_name FOREIGN KEY(value_1) REFERENCES reference_table(id) ON DELETE CASCADE;
|
||||||
|
INSERT INTO local_table_in_the_metadata SELECT i, i FROM generate_series(0, 100) i;
|
||||||
|
INSERT INTO reference_table SELECT i, i FROM generate_series(0, 100) i;
|
||||||
|
INSERT INTO on_delete_fkey_table SELECT i, i % 100 FROM generate_series(0, 1000) i;
|
||||||
|
-- make sure that we can handle switching to sequential execution nicely
|
||||||
|
-- on MX with a regular user
|
||||||
|
BEGIN;
|
||||||
|
DELETE FROM reference_table WHERE id > 50;
|
||||||
|
SHOW citus.multi_shard_modify_mode;
|
||||||
|
citus.multi_shard_modify_mode
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
sequential
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
ALTER TABLE on_delete_fkey_table ADD COLUMN t int DEFAULT 10;
|
||||||
|
SELECT avg(t) FROM on_delete_fkey_table;
|
||||||
|
avg
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
10.0000000000000000
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
ROLLBACK;
|
||||||
|
-- make sure to use multiple connections per node
|
||||||
|
SET citus.force_max_query_parallelization TO ON;
|
||||||
|
CREATE INDEX CONCURRENTLY concurrent_index_test ON on_delete_fkey_table(id);
|
||||||
|
CREATE UNIQUE INDEX unique_key_example ON on_delete_fkey_table(id, value_1);
|
||||||
|
BEGIN;
|
||||||
|
TRUNCATE local_table_in_the_metadata, reference_table, on_delete_fkey_table;
|
||||||
|
NOTICE: truncate cascades to table "reference_table_xxxxx"
|
||||||
|
SELECT count(*) FROM local_table_in_the_metadata;
|
||||||
|
count
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
0
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
SELECT count(*) FROM reference_table;
|
||||||
|
count
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
0
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
SELECT count(*) FROM on_delete_fkey_table;
|
||||||
|
count
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
0
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
ROLLBACK;
|
||||||
|
BEGIN;
|
||||||
|
SET citus.multi_shard_modify_mode TO 'sequential';
|
||||||
|
TRUNCATE on_delete_fkey_table CASCADE;
|
||||||
|
TRUNCATE reference_table CASCADE;
|
||||||
|
NOTICE: truncate cascades to table "on_delete_fkey_table"
|
||||||
|
SELECT count(*) FROM local_table_in_the_metadata JOIN reference_table USING(id) JOIN on_delete_fkey_table USING(id);
|
||||||
|
count
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
0
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
ROLLBACK;
|
||||||
|
-- join involving local, reference and distributed tables
|
||||||
|
SELECT count(*) FROM local_table_in_the_metadata JOIN reference_table USING(id) JOIN on_delete_fkey_table USING(id);
|
||||||
|
count
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
101
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
-- query with intermediate results
|
||||||
|
WITH cte_1 AS (SELECT * FROM on_delete_fkey_table ORDER BY 1,2 DESC LIMIT 10)
|
||||||
|
SELECT count(*) FROM cte_1;
|
||||||
|
count
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
10
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
-- query with intermediate results on remote nodes
|
||||||
|
WITH cte_1 AS (SELECT * FROM on_delete_fkey_table ORDER BY 1,2 DESC LIMIT 10)
|
||||||
|
SELECT count(*) FROM cte_1 JOIN on_delete_fkey_table USING(value_1);
|
||||||
|
count
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
101
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
-- repartition joins
|
||||||
|
SET citus.enable_repartition_joins to ON;
|
||||||
|
SELECT count(*) FROM on_delete_fkey_table o1 JOIN on_delete_fkey_table o2 USING(value_1);
|
||||||
|
count
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
10021
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
-- repartition INSERT .. SELECT
|
||||||
|
INSERT INTO on_delete_fkey_table (id, value_1) SELECT value_1, id FROM on_delete_fkey_table ON CONFLICT DO NOTHING;
|
||||||
|
-- make sure that we can create a type and use it in the same tx
|
||||||
|
BEGIN;
|
||||||
|
CREATE TYPE test_type AS (a int, b int);
|
||||||
|
CREATE TABLE composite_key (id int PRIMARY KEY, c int, data test_type);
|
||||||
|
SELECT create_distributed_table('composite_key', 'id', colocate_with:='partitioned_table');
|
||||||
|
create_distributed_table
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
COMMIT;
|
||||||
|
-- index statistics should work fine
|
||||||
|
CREATE INDEX test_index_on_parent ON users_table_part((value_3+value_2));
|
||||||
|
ALTER INDEX test_index_on_parent ALTER COLUMN 1 SET STATISTICS 4646;
|
||||||
|
DROP INDEX test_index_on_parent;
|
||||||
|
ALTER TABLE composite_key ALTER COLUMN c TYPE float USING (b::float + 0.5);
|
||||||
|
ERROR: column "b" does not exist
|
||||||
|
-- make sure that rebalancer works fine with a regular user on MX
|
||||||
|
-- first make sure that we can rollback
|
||||||
|
BEGIN;
|
||||||
|
SELECT citus_move_shard_placement(1560000, 'localhost', :worker_1_port, 'localhost', :worker_2_port, 'block_writes');
|
||||||
|
citus_move_shard_placement
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
ROLLBACK;
|
||||||
|
SELECT citus_move_shard_placement(1560000, 'localhost', :worker_1_port, 'localhost', :worker_2_port, 'block_writes');
|
||||||
|
citus_move_shard_placement
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
-- connect to the worker to see if the table has the correct owned and placement metadata
|
||||||
|
\c - postgres - :worker_2_port
|
||||||
|
SELECT
|
||||||
|
1560000, groupid = (SELECT groupid FROM pg_dist_node WHERE nodeport = :worker_2_port AND nodename = 'localhost' AND isactive)
|
||||||
|
FROM
|
||||||
|
pg_dist_placement
|
||||||
|
WHERE
|
||||||
|
shardid = 1560000;
|
||||||
|
?column? | ?column?
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
1560000 | t
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
-- also make sure that pg_dist_shard_placement is updated correctly
|
||||||
|
SELECT
|
||||||
|
nodeport = :worker_2_port
|
||||||
|
FROM pg_dist_shard_placement WHERE shardid = 1560000;
|
||||||
|
?column?
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
t
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
\c - postgres - :worker_1_port
|
||||||
|
SELECT
|
||||||
|
1560000, groupid = (SELECT groupid FROM pg_dist_node WHERE nodeport = :worker_2_port AND nodename = 'localhost' AND isactive)
|
||||||
|
FROM
|
||||||
|
pg_dist_placement
|
||||||
|
WHERE
|
||||||
|
shardid = 1560000;
|
||||||
|
?column? | ?column?
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
1560000 | t
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
-- also make sure that pg_dist_shard_placement is updated correctly
|
||||||
|
SELECT
|
||||||
|
nodeport = :worker_2_port
|
||||||
|
FROM pg_dist_shard_placement WHERE shardid = 1560000;
|
||||||
|
?column?
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
t
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
-- now connect with the user to the coordinator again
|
||||||
|
\c - regular_mx_user - :master_port
|
||||||
|
SET search_path TO "Mx Regular User";
|
||||||
|
-- make sure that we can still execute queries
|
||||||
|
SELECT count(*) FROM local_table_in_the_metadata JOIN reference_table USING(id) JOIN on_delete_fkey_table USING(id);
|
||||||
|
count
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
101
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
-- now, call directly the rebalancer, which should also work fine
|
||||||
|
SELECT rebalance_table_shards(shard_transfer_mode:='block_writes');
|
||||||
|
NOTICE: Moving shard xxxxx from localhost:xxxxx to localhost:xxxxx ...
|
||||||
|
rebalance_table_shards
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
-- make sure that we can still execute queries
|
||||||
|
SELECT count(*) FROM local_table_in_the_metadata JOIN reference_table USING(id) JOIN on_delete_fkey_table USING(id);
|
||||||
|
count
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
101
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
-- lets run some queries from the workers
|
||||||
|
\c - regular_mx_user - :worker_2_port
|
||||||
|
SET search_path TO "Mx Regular User";
|
||||||
|
SELECT count(*) FROM local_table_in_the_metadata JOIN reference_table USING(id) JOIN on_delete_fkey_table USING(id);
|
||||||
|
count
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
101
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
BEGIN;
|
||||||
|
TRUNCATE reference_table CASCADE;
|
||||||
|
NOTICE: truncate cascades to table "on_delete_fkey_table"
|
||||||
|
NOTICE: truncate cascades to table "on_delete_fkey_table_xxxxx"
|
||||||
|
NOTICE: truncate cascades to table "on_delete_fkey_table_xxxxx"
|
||||||
|
NOTICE: truncate cascades to table "on_delete_fkey_table_xxxxx"
|
||||||
|
ROLLBACK;
|
||||||
|
-- join involving local, reference and distributed tables
|
||||||
|
SELECT count(*) FROM local_table_in_the_metadata JOIN reference_table USING(id) JOIN on_delete_fkey_table USING(id);
|
||||||
|
count
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
101
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
-- query with intermediate results
|
||||||
|
WITH cte_1 AS (SELECT * FROM on_delete_fkey_table ORDER BY 1,2 DESC LIMIT 10)
|
||||||
|
SELECT count(*) FROM cte_1;
|
||||||
|
count
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
10
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
-- query with intermediate results on remote nodes
|
||||||
|
WITH cte_1 AS (SELECT * FROM on_delete_fkey_table ORDER BY 1,2 DESC LIMIT 10)
|
||||||
|
SELECT count(*) FROM cte_1 JOIN on_delete_fkey_table USING(value_1);
|
||||||
|
count
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
101
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
-- repartition joins
|
||||||
|
SET citus.enable_repartition_joins to ON;
|
||||||
|
SELECT count(*) FROM on_delete_fkey_table o1 JOIN on_delete_fkey_table o2 USING(value_1);
|
||||||
|
count
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
10021
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
BEGIN;
|
||||||
|
SET LOCAL citus.force_max_query_parallelization TO ON;
|
||||||
|
DELETE FROM on_delete_fkey_table;
|
||||||
|
WITH cte_1 AS (SELECT * FROM on_delete_fkey_table ORDER BY 1,2 DESC LIMIT 10)
|
||||||
|
SELECT count(*) FROM cte_1;
|
||||||
|
count
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
0
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
COMMIT;
|
||||||
|
\c - postgres - :master_port
|
||||||
|
-- resync the metadata to both nodes for test purposes and then stop
|
||||||
|
SELECT start_metadata_sync_to_node('localhost', :worker_1_port);
|
||||||
|
start_metadata_sync_to_node
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
SELECT start_metadata_sync_to_node('localhost', :worker_2_port);
|
||||||
|
start_metadata_sync_to_node
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
SELECT stop_metadata_sync_to_node('localhost', :worker_1_port);
|
||||||
|
NOTICE: dropping metadata on the node (localhost,57637)
|
||||||
|
stop_metadata_sync_to_node
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
SELECT stop_metadata_sync_to_node('localhost', :worker_2_port);
|
||||||
|
NOTICE: dropping metadata on the node (localhost,57638)
|
||||||
|
stop_metadata_sync_to_node
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
DROP SCHEMA "Mx Regular User" CASCADE;
|
||||||
|
NOTICE: drop cascades to 10 other objects
|
||||||
|
DETAIL: drop cascades to table "Mx Regular User".partitioned_table
|
||||||
|
drop cascades to sequence "Mx Regular User".my_mx_seq
|
||||||
|
drop cascades to table "Mx Regular User".users_table_part
|
||||||
|
drop cascades to table "Mx Regular User".local_table_in_the_metadata_1560131
|
||||||
|
drop cascades to table "Mx Regular User".reference_table
|
||||||
|
drop cascades to table "Mx Regular User".reference_table_1560126
|
||||||
|
drop cascades to table "Mx Regular User".on_delete_fkey_table
|
||||||
|
drop cascades to table "Mx Regular User".local_table_in_the_metadata
|
||||||
|
drop cascades to type "Mx Regular User".test_type
|
||||||
|
drop cascades to table "Mx Regular User".composite_key
|
|
@ -66,6 +66,10 @@ ORDER BY 1;
|
||||||
function citus_internal.replace_isolation_tester_func()
|
function citus_internal.replace_isolation_tester_func()
|
||||||
function citus_internal.restore_isolation_tester_func()
|
function citus_internal.restore_isolation_tester_func()
|
||||||
function citus_internal.upgrade_columnar_storage(regclass)
|
function citus_internal.upgrade_columnar_storage(regclass)
|
||||||
|
function citus_internal_add_partition_metadata(regclass,"char",text,integer,"char")
|
||||||
|
function citus_internal_add_placement_metadata(bigint,integer,bigint,integer,bigint)
|
||||||
|
function citus_internal_add_shard_metadata(regclass,bigint,"char",text,text)
|
||||||
|
function citus_internal_update_placement_metadata(bigint,integer,integer)
|
||||||
function citus_isolation_test_session_is_blocked(integer,integer[])
|
function citus_isolation_test_session_is_blocked(integer,integer[])
|
||||||
function citus_json_concatenate(json,json)
|
function citus_json_concatenate(json,json)
|
||||||
function citus_json_concatenate_final(json)
|
function citus_json_concatenate_final(json)
|
||||||
|
@ -248,5 +252,5 @@ ORDER BY 1;
|
||||||
view citus_worker_stat_activity
|
view citus_worker_stat_activity
|
||||||
view pg_dist_shard_placement
|
view pg_dist_shard_placement
|
||||||
view time_partitions
|
view time_partitions
|
||||||
(232 rows)
|
(236 rows)
|
||||||
|
|
||||||
|
|
|
@ -564,10 +564,13 @@ SELECT shardid, shardstate, nodename, nodeport
|
||||||
WHERE logicalrelid = 'numbers_hash_other'::regclass order by shardid, nodeport;
|
WHERE logicalrelid = 'numbers_hash_other'::regclass order by shardid, nodeport;
|
||||||
|
|
||||||
-- manually corrupt pg_dist_shard such that both copies of one shard is placed in
|
-- manually corrupt pg_dist_shard such that both copies of one shard is placed in
|
||||||
-- worker_1. This is to test the behavior when no replica of a shard is accessible.
|
-- worker_1+10. This is to test the behavior when no replica of a shard is accessible.
|
||||||
-- Whole copy operation is supposed to fail and rollback.
|
-- Whole copy operation is supposed to fail and rollback. We use session_replication_role
|
||||||
|
-- trick to disable the triggers on the pg_dist_shard_placement view
|
||||||
\c - :default_user
|
\c - :default_user
|
||||||
UPDATE pg_dist_shard_placement SET nodeport = :worker_1_port WHERE shardid = 560176;
|
SET session_replication_role = replica;
|
||||||
|
UPDATE pg_dist_shard_placement SET nodeport = :worker_1_port+10 WHERE shardid = 560176 and nodeport = :worker_2_port;
|
||||||
|
SET session_replication_role = DEFAULT;
|
||||||
|
|
||||||
-- disable test_user on the first worker
|
-- disable test_user on the first worker
|
||||||
\c - :default_user - :worker_1_port
|
\c - :default_user - :worker_1_port
|
||||||
|
|
|
@ -288,7 +288,7 @@ test: undistribute_table_cascade
|
||||||
test: create_citus_local_table_cascade
|
test: create_citus_local_table_cascade
|
||||||
test: fkeys_between_local_ref
|
test: fkeys_between_local_ref
|
||||||
test: auto_undist_citus_local
|
test: auto_undist_citus_local
|
||||||
|
test: mx_regular_user
|
||||||
test: remove_coordinator
|
test: remove_coordinator
|
||||||
|
|
||||||
# ----------
|
# ----------
|
||||||
|
|
|
@ -57,6 +57,7 @@ test: multi_mx_alter_distributed_table
|
||||||
|
|
||||||
# should be executed sequentially because it modifies metadata
|
# should be executed sequentially because it modifies metadata
|
||||||
test: local_shard_execution_dropped_column
|
test: local_shard_execution_dropped_column
|
||||||
|
test: metadata_sync_helpers
|
||||||
|
|
||||||
# test that no tests leaked intermediate results. This should always be last
|
# test that no tests leaked intermediate results. This should always be last
|
||||||
test: ensure_no_intermediate_data_leak
|
test: ensure_no_intermediate_data_leak
|
||||||
|
|
|
@ -773,10 +773,13 @@ SELECT shardid, shardstate, nodename, nodeport
|
||||||
(8 rows)
|
(8 rows)
|
||||||
|
|
||||||
-- manually corrupt pg_dist_shard such that both copies of one shard is placed in
|
-- manually corrupt pg_dist_shard such that both copies of one shard is placed in
|
||||||
-- worker_1. This is to test the behavior when no replica of a shard is accessible.
|
-- worker_1+10. This is to test the behavior when no replica of a shard is accessible.
|
||||||
-- Whole copy operation is supposed to fail and rollback.
|
-- Whole copy operation is supposed to fail and rollback. We use session_replication_role
|
||||||
|
-- trick to disable the triggers on the pg_dist_shard_placement view
|
||||||
\c - :default_user
|
\c - :default_user
|
||||||
UPDATE pg_dist_shard_placement SET nodeport = :worker_1_port WHERE shardid = 560176;
|
SET session_replication_role = replica;
|
||||||
|
UPDATE pg_dist_shard_placement SET nodeport = :worker_1_port+10 WHERE shardid = 560176 and nodeport = :worker_2_port;
|
||||||
|
SET session_replication_role = DEFAULT;
|
||||||
-- disable test_user on the first worker
|
-- disable test_user on the first worker
|
||||||
\c - :default_user - :worker_1_port
|
\c - :default_user - :worker_1_port
|
||||||
ALTER USER test_user WITH nologin;
|
ALTER USER test_user WITH nologin;
|
||||||
|
@ -843,7 +846,7 @@ SELECT shardid, shardstate, nodename, nodeport
|
||||||
560175 | 3 | localhost | 57637
|
560175 | 3 | localhost | 57637
|
||||||
560175 | 1 | localhost | 57638
|
560175 | 1 | localhost | 57638
|
||||||
560176 | 1 | localhost | 57637
|
560176 | 1 | localhost | 57637
|
||||||
560176 | 1 | localhost | 57637
|
560176 | 1 | localhost | 57638
|
||||||
560177 | 3 | localhost | 57637
|
560177 | 3 | localhost | 57637
|
||||||
560177 | 1 | localhost | 57638
|
560177 | 1 | localhost | 57638
|
||||||
(8 rows)
|
(8 rows)
|
||||||
|
|
|
@ -52,10 +52,10 @@ SELECT hasmetadata FROM pg_dist_node WHERE nodeport=:worker_2_proxy_port;
|
||||||
-- Check failures on DDL command propagation
|
-- Check failures on DDL command propagation
|
||||||
CREATE TABLE t2 (id int PRIMARY KEY);
|
CREATE TABLE t2 (id int PRIMARY KEY);
|
||||||
|
|
||||||
SELECT citus.mitmproxy('conn.onParse(query="^INSERT INTO pg_dist_placement").kill()');
|
SELECT citus.mitmproxy('conn.onParse(query="citus_internal_add_placement_metadata").kill()');
|
||||||
SELECT create_distributed_table('t2', 'id');
|
SELECT create_distributed_table('t2', 'id');
|
||||||
|
|
||||||
SELECT citus.mitmproxy('conn.onParse(query="^INSERT INTO pg_dist_shard").cancel(' || :pid || ')');
|
SELECT citus.mitmproxy('conn.onParse(query="citus_internal_add_shard_metadata").cancel(' || :pid || ')');
|
||||||
SELECT create_distributed_table('t2', 'id');
|
SELECT create_distributed_table('t2', 'id');
|
||||||
|
|
||||||
-- Verify that the table was not distributed
|
-- Verify that the table was not distributed
|
||||||
|
|
|
@ -0,0 +1,588 @@
|
||||||
|
CREATE SCHEMA metadata_sync_helpers;
|
||||||
|
SET search_path TO metadata_sync_helpers;
|
||||||
|
SET citus.next_shard_id TO 1420000;
|
||||||
|
SET citus.next_placement_id TO 1500000;
|
||||||
|
|
||||||
|
-- supress notice messages to make sure that the tests
|
||||||
|
-- do not diverge with enterprise
|
||||||
|
SET client_min_messages TO WARNING;
|
||||||
|
CREATE ROLE metadata_sync_helper_role WITH LOGIN;
|
||||||
|
GRANT ALL ON SCHEMA metadata_sync_helpers TO metadata_sync_helper_role;
|
||||||
|
RESET client_min_messages;
|
||||||
|
|
||||||
|
\c - metadata_sync_helper_role -
|
||||||
|
SET search_path TO metadata_sync_helpers;
|
||||||
|
CREATE TABLE test(col_1 int);
|
||||||
|
|
||||||
|
-- not in a distributed transaction
|
||||||
|
SELECT citus_internal_add_partition_metadata ('test'::regclass, 'h', 'col_1', 0, 's');
|
||||||
|
|
||||||
|
-- in a distributed transaction, but the application name is not Citus
|
||||||
|
BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
|
||||||
|
SELECT assign_distributed_transaction_id(0, 8, '2021-07-09 15:41:55.542377+02');
|
||||||
|
SELECT citus_internal_add_partition_metadata ('test'::regclass, 'h', 'col_1', 0, 's');
|
||||||
|
ROLLBACK;
|
||||||
|
|
||||||
|
-- in a distributed transaction and the application name is Citus
|
||||||
|
-- but we are on the coordinator, so still not allowed
|
||||||
|
BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
|
||||||
|
SELECT assign_distributed_transaction_id(0, 8, '2021-07-09 15:41:55.542377+02');
|
||||||
|
SET application_name to 'citus';
|
||||||
|
SELECT citus_internal_add_partition_metadata ('test'::regclass, 'h', 'col_1', 0, 's');
|
||||||
|
ROLLBACK;
|
||||||
|
|
||||||
|
-- connect back as super user, and then connect to the worker
|
||||||
|
-- with the superuser to make sure we can ingest metadata with
|
||||||
|
-- a regular user under the certain conditions
|
||||||
|
\c - postgres -
|
||||||
|
|
||||||
|
-- we don't need the table/schema anymore
|
||||||
|
SET client_min_messages TO ERROR;
|
||||||
|
DROP SCHEMA metadata_sync_helpers CASCADE;
|
||||||
|
DROP ROLE metadata_sync_helper_role;
|
||||||
|
|
||||||
|
\c - - - :worker_1_port
|
||||||
|
|
||||||
|
CREATE SCHEMA metadata_sync_helpers;
|
||||||
|
SET search_path TO metadata_sync_helpers;
|
||||||
|
|
||||||
|
CREATE TABLE test(col_1 int, col_2 int);
|
||||||
|
|
||||||
|
-- supress notice messages to make sure that the tests
|
||||||
|
-- do not diverge with enterprise
|
||||||
|
SET client_min_messages TO WARNING;
|
||||||
|
SET citus.enable_ddl_propagation TO OFF;
|
||||||
|
CREATE ROLE metadata_sync_helper_role WITH LOGIN;
|
||||||
|
GRANT ALL ON SCHEMA metadata_sync_helpers TO metadata_sync_helper_role;
|
||||||
|
RESET client_min_messages;
|
||||||
|
RESET citus.enable_ddl_propagation;
|
||||||
|
|
||||||
|
-- connect back with the regular user
|
||||||
|
\c - metadata_sync_helper_role - :worker_1_port
|
||||||
|
SET search_path TO metadata_sync_helpers;
|
||||||
|
|
||||||
|
-- in a distributed transaction and the application name is Citus
|
||||||
|
-- and we are on the worker, still not allowed because the user is not
|
||||||
|
-- owner of the table test
|
||||||
|
BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
|
||||||
|
SELECT assign_distributed_transaction_id(0, 8, '2021-07-09 15:41:55.542377+02');
|
||||||
|
SET application_name to 'citus';
|
||||||
|
SELECT citus_internal_add_partition_metadata ('test'::regclass, 'h', 'col_1', 0, 's');
|
||||||
|
ROLLBACK;
|
||||||
|
|
||||||
|
-- finally, a user can only add its own tables to the metadata
|
||||||
|
CREATE TABLE test_2(col_1 int, col_2 int);
|
||||||
|
BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
|
||||||
|
SELECT assign_distributed_transaction_id(0, 8, '2021-07-09 15:41:55.542377+02');
|
||||||
|
SET application_name to 'citus';
|
||||||
|
SELECT citus_internal_add_partition_metadata ('test_2'::regclass, 'h', 'col_1', 0, 's');
|
||||||
|
SELECT count(*) FROM pg_dist_partition WHERE logicalrelid = 'metadata_sync_helpers.test_2'::regclass;
|
||||||
|
ROLLBACK;
|
||||||
|
|
||||||
|
-- fails because there is no X distribution method
|
||||||
|
BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
|
||||||
|
SELECT assign_distributed_transaction_id(0, 8, '2021-07-09 15:41:55.542377+02');
|
||||||
|
SET application_name to 'citus';
|
||||||
|
SELECT citus_internal_add_partition_metadata ('test_2'::regclass, 'X', 'col_1', 0, 's');
|
||||||
|
ROLLBACK;
|
||||||
|
|
||||||
|
-- fails because there is the column does not exist
|
||||||
|
BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
|
||||||
|
SELECT assign_distributed_transaction_id(0, 8, '2021-07-09 15:41:55.542377+02');
|
||||||
|
SET application_name to 'citus';
|
||||||
|
SELECT citus_internal_add_partition_metadata ('test_2'::regclass, 'h', 'non_existing_col', 0, 's');
|
||||||
|
ROLLBACK;
|
||||||
|
|
||||||
|
--- fails because we do not allow NULL parameters
|
||||||
|
BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
|
||||||
|
SELECT assign_distributed_transaction_id(0, 8, '2021-07-09 15:41:55.542377+02');
|
||||||
|
SET application_name to 'citus';
|
||||||
|
SELECT citus_internal_add_partition_metadata (NULL, 'h', 'non_existing_col', 0, 's');
|
||||||
|
ROLLBACK;
|
||||||
|
|
||||||
|
-- fails because colocationId cannot be negative
|
||||||
|
BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
|
||||||
|
SELECT assign_distributed_transaction_id(0, 8, '2021-07-09 15:41:55.542377+02');
|
||||||
|
SET application_name to 'citus';
|
||||||
|
SELECT citus_internal_add_partition_metadata ('test_2'::regclass, 'h', 'col_1', -1, 's');
|
||||||
|
ROLLBACK;
|
||||||
|
|
||||||
|
-- fails because there is no X replication model
|
||||||
|
BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
|
||||||
|
SELECT assign_distributed_transaction_id(0, 8, '2021-07-09 15:41:55.542377+02');
|
||||||
|
SET application_name to 'citus';
|
||||||
|
SELECT citus_internal_add_partition_metadata ('test_2'::regclass, 'X', 'col_1', 0, 'X');
|
||||||
|
ROLLBACK;
|
||||||
|
|
||||||
|
-- the same table cannot be added twice, that is enforced by a primary key
|
||||||
|
BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
|
||||||
|
SELECT assign_distributed_transaction_id(0, 8, '2021-07-09 15:41:55.542377+02');
|
||||||
|
SET application_name to 'citus';
|
||||||
|
\set VERBOSITY terse
|
||||||
|
SELECT citus_internal_add_partition_metadata ('test_2'::regclass, 'h', 'col_1', 0, 's');
|
||||||
|
SELECT citus_internal_add_partition_metadata ('test_2'::regclass, 'h', 'col_1', 0, 's');
|
||||||
|
ROLLBACK;
|
||||||
|
|
||||||
|
-- the same table cannot be added twice, that is enforced by a primary key even if distribution key changes
|
||||||
|
BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
|
||||||
|
SELECT assign_distributed_transaction_id(0, 8, '2021-07-09 15:41:55.542377+02');
|
||||||
|
SET application_name to 'citus';
|
||||||
|
\set VERBOSITY terse
|
||||||
|
SELECT citus_internal_add_partition_metadata ('test_2'::regclass, 'h', 'col_1', 0, 's');
|
||||||
|
SELECT citus_internal_add_partition_metadata ('test_2'::regclass, 'h', 'col_2', 0, 's');
|
||||||
|
ROLLBACK;
|
||||||
|
|
||||||
|
-- hash distributed table cannot have NULL distribution key
|
||||||
|
BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
|
||||||
|
SELECT assign_distributed_transaction_id(0, 8, '2021-07-09 15:41:55.542377+02');
|
||||||
|
SET application_name to 'citus';
|
||||||
|
\set VERBOSITY terse
|
||||||
|
SELECT citus_internal_add_partition_metadata ('test_2'::regclass, 'h', NULL, 0, 's');
|
||||||
|
ROLLBACK;
|
||||||
|
|
||||||
|
-- even if metadata_sync_helper_role is not owner of the table test
|
||||||
|
-- the operator allowed metadata_sync_helper_role user to skip
|
||||||
|
-- checks, such as wrong partition method can be set
|
||||||
|
-- connect back with the regular user
|
||||||
|
\c - postgres - :worker_1_port
|
||||||
|
SET search_path TO metadata_sync_helpers;
|
||||||
|
|
||||||
|
ALTER SYSTEM SET citus.enable_manual_metadata_changes_for_user TO 'metadata_sync_helper_role';
|
||||||
|
SELECT pg_reload_conf();
|
||||||
|
SELECT pg_sleep(0.1);
|
||||||
|
|
||||||
|
\c - metadata_sync_helper_role - :worker_1_port
|
||||||
|
SET search_path TO metadata_sync_helpers;
|
||||||
|
|
||||||
|
BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
|
||||||
|
SELECT assign_distributed_transaction_id(0, 8, '2021-07-09 15:41:55.542377+02');
|
||||||
|
SET application_name to 'citus';
|
||||||
|
SELECT citus_internal_add_partition_metadata ('test_2'::regclass, 'X', 'col_1', 0, 's');
|
||||||
|
ROLLBACK;
|
||||||
|
|
||||||
|
-- should throw error even if we skip the checks, there are no such nodes
|
||||||
|
BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
|
||||||
|
SELECT assign_distributed_transaction_id(0, 8, '2021-07-09 15:41:55.542377+02');
|
||||||
|
SET application_name to 'citus';
|
||||||
|
\set VERBOSITY terse
|
||||||
|
SELECT citus_internal_update_placement_metadata(1420007, 10000, 11111);
|
||||||
|
ROLLBACK;
|
||||||
|
|
||||||
|
-- non-existing users should fail to pass the checks
|
||||||
|
\c - postgres - :worker_1_port
|
||||||
|
SET search_path TO metadata_sync_helpers;
|
||||||
|
ALTER SYSTEM SET citus.enable_manual_metadata_changes_for_user TO 'non_existing_user';
|
||||||
|
SELECT pg_reload_conf();
|
||||||
|
SELECT pg_sleep(0.1);
|
||||||
|
|
||||||
|
\c - metadata_sync_helper_role - :worker_1_port
|
||||||
|
SET search_path TO metadata_sync_helpers;
|
||||||
|
|
||||||
|
BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
|
||||||
|
SELECT assign_distributed_transaction_id(0, 8, '2021-07-09 15:41:55.542377+02');
|
||||||
|
SET application_name to 'citus';
|
||||||
|
SELECT citus_internal_add_partition_metadata ('test_2'::regclass, 'X', 'col_1', 0, 's');
|
||||||
|
ROLLBACK;
|
||||||
|
|
||||||
|
\c - postgres - :worker_1_port
|
||||||
|
SET search_path TO metadata_sync_helpers;
|
||||||
|
ALTER SYSTEM RESET citus.enable_manual_metadata_changes_for_user;
|
||||||
|
SELECT pg_reload_conf();
|
||||||
|
SELECT pg_sleep(0.1);
|
||||||
|
|
||||||
|
\c - metadata_sync_helper_role - :worker_1_port
|
||||||
|
SET search_path TO metadata_sync_helpers;
|
||||||
|
|
||||||
|
-- we also enforce some checks on the reference tables
|
||||||
|
-- cannot add because reference tables cannot have distribution column
|
||||||
|
CREATE TABLE test_ref(col_1 int, col_2 int);
|
||||||
|
BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
|
||||||
|
SELECT assign_distributed_transaction_id(0, 8, '2021-07-09 15:41:55.542377+02');
|
||||||
|
SET application_name to 'citus';
|
||||||
|
SELECT citus_internal_add_partition_metadata ('test_ref'::regclass, 'n', 'col_1', 0, 's');
|
||||||
|
ROLLBACK;
|
||||||
|
|
||||||
|
-- non-valid replication model
|
||||||
|
BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
|
||||||
|
SELECT assign_distributed_transaction_id(0, 8, '2021-07-09 15:41:55.542377+02');
|
||||||
|
SET application_name to 'citus';
|
||||||
|
SELECT citus_internal_add_partition_metadata ('test_ref'::regclass, 'n', NULL, 0, 'A');
|
||||||
|
ROLLBACK;
|
||||||
|
|
||||||
|
-- not-matching replication model for reference table
|
||||||
|
BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
|
||||||
|
SELECT assign_distributed_transaction_id(0, 8, '2021-07-09 15:41:55.542377+02');
|
||||||
|
SET application_name to 'citus';
|
||||||
|
SELECT citus_internal_add_partition_metadata ('test_ref'::regclass, 'n', NULL, 0, 'c');
|
||||||
|
ROLLBACK;
|
||||||
|
|
||||||
|
-- not-matching replication model for hash table
|
||||||
|
BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
|
||||||
|
SELECT assign_distributed_transaction_id(0, 8, '2021-07-09 15:41:55.542377+02');
|
||||||
|
SET application_name to 'citus';
|
||||||
|
SELECT citus_internal_add_partition_metadata ('test_2'::regclass, 'h', 'col_1', 0, 't');
|
||||||
|
ROLLBACK;
|
||||||
|
|
||||||
|
|
||||||
|
-- add entry for super user table
|
||||||
|
\c - postgres - :worker_1_port
|
||||||
|
SET search_path TO metadata_sync_helpers;
|
||||||
|
CREATE TABLE super_user_table(col_1 int);
|
||||||
|
BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
|
||||||
|
SELECT assign_distributed_transaction_id(0, 8, '2021-07-09 15:41:55.542377+02');
|
||||||
|
SET application_name to 'citus';
|
||||||
|
SELECT citus_internal_add_partition_metadata ('super_user_table'::regclass, 'h', 'col_1', 0, 's');
|
||||||
|
COMMIT;
|
||||||
|
|
||||||
|
-- now, lets check shard metadata
|
||||||
|
|
||||||
|
-- the user is only allowed to add a shard for add a table which they do not own
|
||||||
|
|
||||||
|
\c - metadata_sync_helper_role - :worker_1_port
|
||||||
|
SET search_path TO metadata_sync_helpers;
|
||||||
|
|
||||||
|
BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
|
||||||
|
SELECT assign_distributed_transaction_id(0, 8, '2021-07-09 15:41:55.542377+02');
|
||||||
|
SET application_name to 'citus';
|
||||||
|
\set VERBOSITY terse
|
||||||
|
WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue)
|
||||||
|
AS (VALUES ('super_user_table'::regclass, 1420000::bigint, 't'::"char", '-2147483648'::text, '-1610612737'::text))
|
||||||
|
SELECT citus_internal_add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data;
|
||||||
|
ROLLBACK;
|
||||||
|
|
||||||
|
-- the user is only allowed to add a shard for add a table which is in pg_dist_partition
|
||||||
|
BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
|
||||||
|
SELECT assign_distributed_transaction_id(0, 8, '2021-07-09 15:41:55.542377+02');
|
||||||
|
SET application_name to 'citus';
|
||||||
|
\set VERBOSITY terse
|
||||||
|
WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue)
|
||||||
|
AS (VALUES ('test_2'::regclass, 1420000::bigint, 't'::"char", '-2147483648'::text, '-1610612737'::text))
|
||||||
|
SELECT citus_internal_add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data;
|
||||||
|
ROLLBACK;
|
||||||
|
|
||||||
|
-- ok, now add the table to the pg_dist_partition
|
||||||
|
BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
|
||||||
|
SELECT assign_distributed_transaction_id(0, 8, '2021-07-09 15:41:55.542377+02');
|
||||||
|
SET application_name to 'citus';
|
||||||
|
SELECT citus_internal_add_partition_metadata ('test_2'::regclass, 'h', 'col_1', 0, 's');
|
||||||
|
SELECT citus_internal_add_partition_metadata ('test_ref'::regclass, 'n', NULL, 0, 't');
|
||||||
|
COMMIT;
|
||||||
|
|
||||||
|
-- invalid shard ids are not allowed
|
||||||
|
BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
|
||||||
|
SELECT assign_distributed_transaction_id(0, 8, '2021-07-09 15:41:55.542377+02');
|
||||||
|
SET application_name to 'citus';
|
||||||
|
\set VERBOSITY terse
|
||||||
|
WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue)
|
||||||
|
AS (VALUES ('test_2'::regclass, -1, 't'::"char", '-2147483648'::text, '-1610612737'::text))
|
||||||
|
SELECT citus_internal_add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data;
|
||||||
|
ROLLBACK;
|
||||||
|
|
||||||
|
-- invalid storage types are not allowed
|
||||||
|
BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
|
||||||
|
SELECT assign_distributed_transaction_id(0, 8, '2021-07-09 15:41:55.542377+02');
|
||||||
|
SET application_name to 'citus';
|
||||||
|
\set VERBOSITY terse
|
||||||
|
WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue)
|
||||||
|
AS (VALUES ('test_2'::regclass, 1420000, 'X'::"char", '-2147483648'::text, '-1610612737'::text))
|
||||||
|
SELECT citus_internal_add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data;
|
||||||
|
ROLLBACK;
|
||||||
|
|
||||||
|
-- NULL shard ranges are not allowed for hash distributed tables
|
||||||
|
BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
|
||||||
|
SELECT assign_distributed_transaction_id(0, 8, '2021-07-09 15:41:55.542377+02');
|
||||||
|
SET application_name to 'citus';
|
||||||
|
\set VERBOSITY terse
|
||||||
|
WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue)
|
||||||
|
AS (VALUES ('test_2'::regclass, 1420000, 't'::"char", NULL, '-1610612737'::text))
|
||||||
|
SELECT citus_internal_add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data;
|
||||||
|
ROLLBACK;
|
||||||
|
|
||||||
|
-- non-integer shard ranges are not allowed
|
||||||
|
BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
|
||||||
|
SELECT assign_distributed_transaction_id(0, 8, '2021-07-09 15:41:55.542377+02');
|
||||||
|
SET application_name to 'citus';
|
||||||
|
\set VERBOSITY terse
|
||||||
|
WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue)
|
||||||
|
AS (VALUES ('test_2'::regclass, 1420000::bigint, 't'::"char", 'non-int'::text, '-1610612737'::text))
|
||||||
|
SELECT citus_internal_add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data;
|
||||||
|
ROLLBACK;
|
||||||
|
|
||||||
|
-- shardMinValue should be smaller than shardMaxValue
|
||||||
|
BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
|
||||||
|
SELECT assign_distributed_transaction_id(0, 8, '2021-07-09 15:41:55.542377+02');
|
||||||
|
SET application_name to 'citus';
|
||||||
|
\set VERBOSITY terse
|
||||||
|
WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue)
|
||||||
|
AS (VALUES ('test_2'::regclass, 1420000::bigint, 't'::"char", '-1610612737'::text, '-2147483648'::text))
|
||||||
|
SELECT citus_internal_add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data;
|
||||||
|
ROLLBACK;
|
||||||
|
|
||||||
|
-- we do not allow overlapping shards for the same table
|
||||||
|
BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
|
||||||
|
SELECT assign_distributed_transaction_id(0, 8, '2021-07-09 15:41:55.542377+02');
|
||||||
|
SET application_name to 'citus';
|
||||||
|
\set VERBOSITY terse
|
||||||
|
WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue)
|
||||||
|
AS (VALUES ('test_2'::regclass, 1420000::bigint, 't'::"char", '10'::text, '20'::text),
|
||||||
|
('test_2'::regclass, 1420001::bigint, 't'::"char", '20'::text, '30'::text),
|
||||||
|
('test_2'::regclass, 1420002::bigint, 't'::"char", '10'::text, '50'::text))
|
||||||
|
SELECT citus_internal_add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data;
|
||||||
|
ROLLBACK;
|
||||||
|
|
||||||
|
\c - postgres - :worker_1_port
|
||||||
|
|
||||||
|
-- we do not allow wrong partmethod
|
||||||
|
-- so manually insert wrong partmethod for the sake of the test
|
||||||
|
SET search_path TO metadata_sync_helpers;
|
||||||
|
BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
|
||||||
|
SELECT assign_distributed_transaction_id(0, 8, '2021-07-09 15:41:55.542377+02');
|
||||||
|
SET application_name to 'citus';
|
||||||
|
\set VERBOSITY terse
|
||||||
|
UPDATE pg_dist_partition SET partmethod = 'X';
|
||||||
|
WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue)
|
||||||
|
AS (VALUES ('test_2'::regclass, 1420000::bigint, 't'::"char", '10'::text, '20'::text),
|
||||||
|
('test_2'::regclass, 1420001::bigint, 't'::"char", '20'::text, '30'::text))
|
||||||
|
SELECT citus_internal_add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data;
|
||||||
|
ROLLBACK;
|
||||||
|
|
||||||
|
-- we do not allow NULL shardMinMax values
|
||||||
|
-- so manually insert wrong partmethod for the sake of the test
|
||||||
|
SET search_path TO metadata_sync_helpers;
|
||||||
|
BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
|
||||||
|
SELECT assign_distributed_transaction_id(0, 8, '2021-07-09 15:41:55.542377+02');
|
||||||
|
SET application_name to 'citus';
|
||||||
|
\set VERBOSITY terse
|
||||||
|
WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue)
|
||||||
|
AS (VALUES ('test_2'::regclass, 1420000::bigint, 't'::"char", '10'::text, '20'::text))
|
||||||
|
SELECT citus_internal_add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data;
|
||||||
|
-- manually ingest NULL values, otherwise not likely unless metadata is corrupted
|
||||||
|
UPDATE pg_dist_shard SET shardminvalue = NULL WHERE shardid = 1420000;
|
||||||
|
WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue)
|
||||||
|
AS (VALUES ('test_2'::regclass, 1420001::bigint, 't'::"char", '20'::text, '30'::text))
|
||||||
|
SELECT citus_internal_add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data;
|
||||||
|
ROLLBACK;
|
||||||
|
|
||||||
|
\c - metadata_sync_helper_role - :worker_1_port
|
||||||
|
SET search_path TO metadata_sync_helpers;
|
||||||
|
|
||||||
|
-- now, add few shards
|
||||||
|
BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
|
||||||
|
SELECT assign_distributed_transaction_id(0, 8, '2021-07-09 15:41:55.542377+02');
|
||||||
|
SET application_name to 'citus';
|
||||||
|
\set VERBOSITY terse
|
||||||
|
WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue)
|
||||||
|
AS (VALUES ('test_2'::regclass, 1420000::bigint, 't'::"char", '11'::text, '20'::text),
|
||||||
|
('test_2'::regclass, 1420001::bigint, 't'::"char", '21'::text, '30'::text),
|
||||||
|
('test_2'::regclass, 1420002::bigint, 't'::"char", '31'::text, '40'::text),
|
||||||
|
('test_2'::regclass, 1420003::bigint, 't'::"char", '41'::text, '50'::text),
|
||||||
|
('test_2'::regclass, 1420004::bigint, 't'::"char", '51'::text, '60'::text),
|
||||||
|
('test_2'::regclass, 1420005::bigint, 't'::"char", '61'::text, '70'::text))
|
||||||
|
SELECT citus_internal_add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data;
|
||||||
|
COMMIT;
|
||||||
|
|
||||||
|
-- shardMin/MaxValues should be NULL for reference tables
|
||||||
|
BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
|
||||||
|
SELECT assign_distributed_transaction_id(0, 8, '2021-07-09 15:41:55.542377+02');
|
||||||
|
SET application_name to 'citus';
|
||||||
|
\set VERBOSITY terse
|
||||||
|
WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue)
|
||||||
|
AS (VALUES ('test_ref'::regclass, 1420003::bigint, 't'::"char", '-1610612737'::text, NULL))
|
||||||
|
SELECT citus_internal_add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data;
|
||||||
|
ROLLBACK;
|
||||||
|
|
||||||
|
-- reference tables cannot have multiple shards
|
||||||
|
BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
|
||||||
|
SELECT assign_distributed_transaction_id(0, 8, '2021-07-09 15:41:55.542377+02');
|
||||||
|
SET application_name to 'citus';
|
||||||
|
\set VERBOSITY terse
|
||||||
|
WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue)
|
||||||
|
AS (VALUES ('test_ref'::regclass, 1420006::bigint, 't'::"char", NULL, NULL),
|
||||||
|
('test_ref'::regclass, 1420007::bigint, 't'::"char", NULL, NULL))
|
||||||
|
SELECT citus_internal_add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data;
|
||||||
|
ROLLBACK;
|
||||||
|
|
||||||
|
-- finally, add a shard for reference tables
|
||||||
|
BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
|
||||||
|
SELECT assign_distributed_transaction_id(0, 8, '2021-07-09 15:41:55.542377+02');
|
||||||
|
SET application_name to 'citus';
|
||||||
|
\set VERBOSITY terse
|
||||||
|
WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue)
|
||||||
|
AS (VALUES ('test_ref'::regclass, 1420006::bigint, 't'::"char", NULL, NULL))
|
||||||
|
SELECT citus_internal_add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data;
|
||||||
|
COMMIT;
|
||||||
|
|
||||||
|
\c - postgres - :worker_1_port
|
||||||
|
SET search_path TO metadata_sync_helpers;
|
||||||
|
|
||||||
|
-- and a shard for the superuser table
|
||||||
|
BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
|
||||||
|
SELECT assign_distributed_transaction_id(0, 8, '2021-07-09 15:41:55.542377+02');
|
||||||
|
SET application_name to 'citus';
|
||||||
|
\set VERBOSITY terse
|
||||||
|
WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue)
|
||||||
|
AS (VALUES ('super_user_table'::regclass, 1420007::bigint, 't'::"char", '11'::text, '20'::text))
|
||||||
|
SELECT citus_internal_add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data;
|
||||||
|
COMMIT;
|
||||||
|
|
||||||
|
\c - metadata_sync_helper_role - :worker_1_port
|
||||||
|
SET search_path TO metadata_sync_helpers;
|
||||||
|
|
||||||
|
-- now, check placement metadata
|
||||||
|
|
||||||
|
-- shard does not exist
|
||||||
|
BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
|
||||||
|
SELECT assign_distributed_transaction_id(0, 8, '2021-07-09 15:41:55.542377+02');
|
||||||
|
SET application_name to 'citus';
|
||||||
|
\set VERBOSITY terse
|
||||||
|
WITH placement_data(shardid, shardstate, shardlength, groupid, placementid) AS
|
||||||
|
(VALUES (-10, 1, 0::bigint, 1::int, 1500000::bigint))
|
||||||
|
SELECT citus_internal_add_placement_metadata(shardid, shardstate, shardlength, groupid, placementid) FROM placement_data;
|
||||||
|
ROLLBACK;
|
||||||
|
|
||||||
|
-- invalid placementid
|
||||||
|
BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
|
||||||
|
SELECT assign_distributed_transaction_id(0, 8, '2021-07-09 15:41:55.542377+02');
|
||||||
|
SET application_name to 'citus';
|
||||||
|
\set VERBOSITY terse
|
||||||
|
WITH placement_data(shardid, shardstate, shardlength, groupid, placementid) AS
|
||||||
|
(VALUES (1420000, 1, 0::bigint, 1::int, -10))
|
||||||
|
SELECT citus_internal_add_placement_metadata(shardid, shardstate, shardlength, groupid, placementid) FROM placement_data;
|
||||||
|
ROLLBACK;
|
||||||
|
|
||||||
|
-- non-existing shard
|
||||||
|
BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
|
||||||
|
SELECT assign_distributed_transaction_id(0, 8, '2021-07-09 15:41:55.542377+02');
|
||||||
|
SET application_name to 'citus';
|
||||||
|
\set VERBOSITY terse
|
||||||
|
WITH placement_data(shardid, shardstate, shardlength, groupid, placementid) AS
|
||||||
|
(VALUES (1430100, 1, 0::bigint, 1::int, 10))
|
||||||
|
SELECT citus_internal_add_placement_metadata(shardid, shardstate, shardlength, groupid, placementid) FROM placement_data;
|
||||||
|
ROLLBACK;
|
||||||
|
|
||||||
|
-- invalid shard state
|
||||||
|
BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
|
||||||
|
SELECT assign_distributed_transaction_id(0, 8, '2021-07-09 15:41:55.542377+02');
|
||||||
|
SET application_name to 'citus';
|
||||||
|
\set VERBOSITY terse
|
||||||
|
WITH placement_data(shardid, shardstate, shardlength, groupid, placementid) AS
|
||||||
|
(VALUES (1420000, 10, 0::bigint, 1::int, 1500000))
|
||||||
|
SELECT citus_internal_add_placement_metadata(shardid, shardstate, shardlength, groupid, placementid) FROM placement_data;
|
||||||
|
ROLLBACK;
|
||||||
|
|
||||||
|
-- non-existing node with non-existing node-id 123123123
|
||||||
|
BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
|
||||||
|
SELECT assign_distributed_transaction_id(0, 8, '2021-07-09 15:41:55.542377+02');
|
||||||
|
SET application_name to 'citus';
|
||||||
|
\set VERBOSITY terse
|
||||||
|
WITH placement_data(shardid, shardstate, shardlength, groupid, placementid) AS
|
||||||
|
(VALUES ( 1420000, 1, 0::bigint, 123123123::int, 1500000))
|
||||||
|
SELECT citus_internal_add_placement_metadata(shardid, shardstate, shardlength, groupid, placementid) FROM placement_data;
|
||||||
|
ROLLBACK;
|
||||||
|
|
||||||
|
-- create a volatile function that returns the local node id
|
||||||
|
CREATE OR REPLACE FUNCTION get_node_id()
|
||||||
|
RETURNS INT AS $$
|
||||||
|
DECLARE localGroupId int;
|
||||||
|
BEGIN
|
||||||
|
SELECT
|
||||||
|
groupid into localGroupId
|
||||||
|
FROM
|
||||||
|
pg_dist_node
|
||||||
|
WHERE
|
||||||
|
nodeport = 57637 AND nodename = 'localhost' AND isactive AND nodecluster = 'default';
|
||||||
|
RETURN localGroupId;
|
||||||
|
END; $$ language plpgsql;
|
||||||
|
|
||||||
|
-- fails because we ingest more placements for the same shards to the same worker node
|
||||||
|
BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
|
||||||
|
SELECT assign_distributed_transaction_id(0, 8, '2021-07-09 15:41:55.542377+02');
|
||||||
|
SET application_name to 'citus';
|
||||||
|
\set VERBOSITY terse
|
||||||
|
WITH placement_data(shardid, shardstate, shardlength, groupid, placementid) AS
|
||||||
|
(VALUES (1420000, 1, 0::bigint, get_node_id(), 1500000),
|
||||||
|
(1420000, 1, 0::bigint, get_node_id(), 1500001))
|
||||||
|
SELECT citus_internal_add_placement_metadata(shardid, shardstate, shardlength, groupid, placementid) FROM placement_data;
|
||||||
|
ROLLBACK;
|
||||||
|
|
||||||
|
-- shard is not owned by us
|
||||||
|
BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
|
||||||
|
SELECT assign_distributed_transaction_id(0, 8, '2021-07-09 15:41:55.542377+02');
|
||||||
|
SET application_name to 'citus';
|
||||||
|
\set VERBOSITY terse
|
||||||
|
WITH placement_data(shardid, shardstate, shardlength, groupid, placementid) AS
|
||||||
|
(VALUES (1420007, 1, 0::bigint, get_node_id(), 1500000))
|
||||||
|
SELECT citus_internal_add_placement_metadata(shardid, shardstate, shardlength, groupid, placementid) FROM placement_data;
|
||||||
|
ROLLBACK;
|
||||||
|
|
||||||
|
-- sucessfully add placements
|
||||||
|
BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
|
||||||
|
SELECT assign_distributed_transaction_id(0, 8, '2021-07-09 15:41:55.542377+02');
|
||||||
|
SET application_name to 'citus';
|
||||||
|
\set VERBOSITY terse
|
||||||
|
WITH placement_data(shardid, shardstate, shardlength, groupid, placementid) AS
|
||||||
|
(VALUES (1420000, 1, 0::bigint, get_node_id(), 1500000),
|
||||||
|
(1420001, 1, 0::bigint, get_node_id(), 1500001),
|
||||||
|
(1420002, 1, 0::bigint, get_node_id(), 1500002),
|
||||||
|
(1420003, 1, 0::bigint, get_node_id(), 1500003),
|
||||||
|
(1420004, 1, 0::bigint, get_node_id(), 1500004),
|
||||||
|
(1420005, 1, 0::bigint, get_node_id(), 1500005))
|
||||||
|
SELECT citus_internal_add_placement_metadata(shardid, shardstate, shardlength, groupid, placementid) FROM placement_data;
|
||||||
|
COMMIT;
|
||||||
|
|
||||||
|
-- try to update placements
|
||||||
|
|
||||||
|
-- fails because we are trying to update it to non-existing node
|
||||||
|
BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
|
||||||
|
SELECT assign_distributed_transaction_id(0, 8, '2021-07-09 15:41:55.542377+02');
|
||||||
|
SET application_name to 'citus';
|
||||||
|
\set VERBOSITY terse
|
||||||
|
SELECT citus_internal_update_placement_metadata(1420000, get_node_id(), get_node_id()+1000);
|
||||||
|
COMMIT;
|
||||||
|
|
||||||
|
-- fails because the source node doesn't contain the shard
|
||||||
|
BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
|
||||||
|
SELECT assign_distributed_transaction_id(0, 8, '2021-07-09 15:41:55.542377+02');
|
||||||
|
SET application_name to 'citus';
|
||||||
|
\set VERBOSITY terse
|
||||||
|
SELECT citus_internal_update_placement_metadata(1420000, get_node_id()+10000, get_node_id());
|
||||||
|
COMMIT;
|
||||||
|
|
||||||
|
-- fails because shard does not exist
|
||||||
|
BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
|
||||||
|
SELECT assign_distributed_transaction_id(0, 8, '2021-07-09 15:41:55.542377+02');
|
||||||
|
SET application_name to 'citus';
|
||||||
|
\set VERBOSITY terse
|
||||||
|
SELECT citus_internal_update_placement_metadata(0, get_node_id(), get_node_id()+1);
|
||||||
|
COMMIT;
|
||||||
|
|
||||||
|
-- fails because none-existing shard
|
||||||
|
BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
|
||||||
|
SELECT assign_distributed_transaction_id(0, 8, '2021-07-09 15:41:55.542377+02');
|
||||||
|
SET application_name to 'citus';
|
||||||
|
\set VERBOSITY terse
|
||||||
|
SELECT citus_internal_update_placement_metadata(213123123123, get_node_id(), get_node_id()+1);
|
||||||
|
COMMIT;
|
||||||
|
|
||||||
|
-- fails because we do not own the shard
|
||||||
|
BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
|
||||||
|
SELECT assign_distributed_transaction_id(0, 8, '2021-07-09 15:41:55.542377+02');
|
||||||
|
SET application_name to 'citus';
|
||||||
|
\set VERBOSITY terse
|
||||||
|
SELECT citus_internal_update_placement_metadata(1420007, get_node_id(), get_node_id()+1);
|
||||||
|
COMMIT;
|
||||||
|
|
||||||
|
-- we don't need the table/schema anymore
|
||||||
|
-- connect back as super user to drop everything
|
||||||
|
\c - postgres - :worker_1_port
|
||||||
|
SET search_path TO metadata_sync_helpers;
|
||||||
|
-- remove the manually generated metadata
|
||||||
|
DELETE FROM pg_dist_placement WHERE shardid IN (SELECT shardid FROM pg_dist_shard WHERE logicalrelid IN ('test_ref'::regclass, 'test_2'::regclass));
|
||||||
|
DELETE FROM pg_dist_shard WHERE logicalrelid IN ('test_ref'::regclass, 'test_2'::regclass);
|
||||||
|
DELETE FROM pg_dist_partition WHERE logicalrelid IN ('test_ref'::regclass, 'test_2'::regclass);
|
||||||
|
|
||||||
|
SET client_min_messages TO ERROR;
|
||||||
|
SET citus.enable_ddl_propagation TO OFF;
|
||||||
|
DROP OWNED BY metadata_sync_helper_role;
|
||||||
|
DROP ROLE metadata_sync_helper_role;
|
||||||
|
DROP SCHEMA metadata_sync_helpers CASCADE;
|
|
@ -885,6 +885,7 @@ DELETE FROM summary_table WHERE id < (
|
||||||
CREATE TABLE multi_modifications.local (a int default 1, b int);
|
CREATE TABLE multi_modifications.local (a int default 1, b int);
|
||||||
INSERT INTO multi_modifications.local VALUES (default, (SELECT min(id) FROM summary_table));
|
INSERT INTO multi_modifications.local VALUES (default, (SELECT min(id) FROM summary_table));
|
||||||
|
|
||||||
|
DROP TABLE insufficient_shards;
|
||||||
DROP TABLE raw_table;
|
DROP TABLE raw_table;
|
||||||
DROP TABLE summary_table;
|
DROP TABLE summary_table;
|
||||||
DROP TABLE reference_raw_table;
|
DROP TABLE reference_raw_table;
|
||||||
|
|
|
@ -0,0 +1,268 @@
|
||||||
|
CREATE SCHEMA "Mx Regular User";
|
||||||
|
SET search_path TO "Mx Regular User";
|
||||||
|
|
||||||
|
-- sync the metadata to both nodes
|
||||||
|
SELECT start_metadata_sync_to_node('localhost', :worker_1_port);
|
||||||
|
SELECT start_metadata_sync_to_node('localhost', :worker_2_port);
|
||||||
|
|
||||||
|
-- create a role and give access one each node separately
|
||||||
|
-- and increase the error level to prevent enterprise to diverge
|
||||||
|
SET client_min_messages TO ERROR;
|
||||||
|
SET citus.enable_ddl_propagation TO OFF;
|
||||||
|
CREATE USER regular_mx_user WITH LOGIN;
|
||||||
|
GRANT ALL ON SCHEMA "Mx Regular User" TO regular_mx_user;
|
||||||
|
|
||||||
|
\c - postgres - :worker_1_port;
|
||||||
|
SET client_min_messages TO ERROR;
|
||||||
|
SET citus.enable_ddl_propagation TO OFF;
|
||||||
|
CREATE SCHEMA "Mx Regular User";
|
||||||
|
CREATE USER regular_mx_user WITH LOGIN;
|
||||||
|
GRANT ALL ON SCHEMA "Mx Regular User" TO regular_mx_user;
|
||||||
|
|
||||||
|
\c - postgres - :worker_2_port;
|
||||||
|
SET client_min_messages TO ERROR;
|
||||||
|
SET citus.enable_ddl_propagation TO OFF;
|
||||||
|
CREATE SCHEMA "Mx Regular User";
|
||||||
|
CREATE USER regular_mx_user WITH LOGIN;
|
||||||
|
GRANT ALL ON SCHEMA "Mx Regular User" TO regular_mx_user;
|
||||||
|
|
||||||
|
-- now connect with that user
|
||||||
|
\c - regular_mx_user - :master_port
|
||||||
|
SET search_path TO "Mx Regular User";
|
||||||
|
SET citus.next_shard_id TO 1560000;
|
||||||
|
|
||||||
|
-- make sure that we sync the metadata
|
||||||
|
SET citus.shard_replication_factor TO 1;
|
||||||
|
|
||||||
|
CREATE TABLE partitioned_table (long_column_names_1 int, long_column_names_2 int, long_column_names_3 int, long_column_names_4 int, long_column_names_5 int, long_column_names_6 int, long_column_names_7 int, long_column_names_8 int, long_column_names_9 int, long_column_names_10 int, long_column_names_11 timestamp) PARTITION BY RANGE (long_column_names_11);
|
||||||
|
CREATE TABLE very_long_child_partition_name_is_required_to_repro_the_bug PARTITION OF partitioned_table FOR VALUES FROM ('2011-01-01') TO ('2012-01-01');
|
||||||
|
|
||||||
|
SELECT create_distributed_table('partitioned_table', 'long_column_names_1');
|
||||||
|
SELECT bool_and(hasmetadata) FROM pg_dist_node WHERE nodename = 'localhost' and nodeport IN (:worker_1_port, :worker_2_port);
|
||||||
|
|
||||||
|
-- show that we can rollback
|
||||||
|
BEGIN;
|
||||||
|
CREATE INDEX long_index_on_parent_table ON partitioned_table (long_column_names_1, long_column_names_2, long_column_names_3, long_column_names_4, long_column_names_5, long_column_names_6, long_column_names_11) INCLUDE (long_column_names_7, long_column_names_7, long_column_names_9, long_column_names_10);
|
||||||
|
ROLLBACK;
|
||||||
|
|
||||||
|
-- show that we can switch to sequential mode and still
|
||||||
|
-- sync the metadata to the nodes
|
||||||
|
BEGIN;
|
||||||
|
CREATE INDEX long_index_on_parent_table ON partitioned_table (long_column_names_1, long_column_names_2, long_column_names_3, long_column_names_4, long_column_names_5, long_column_names_6, long_column_names_11) INCLUDE (long_column_names_7, long_column_names_7, long_column_names_9, long_column_names_10);
|
||||||
|
show citus.multi_shard_modify_mode;
|
||||||
|
COMMIT;
|
||||||
|
|
||||||
|
-- make sure that partitioned tables, columnar and conversion to columnar workes fine
|
||||||
|
-- on Citus MX with a non-super user
|
||||||
|
CREATE SEQUENCE my_mx_seq;
|
||||||
|
CREATE TABLE users_table_part(col_to_drop int, user_id int, value_1 int, value_2 int DEFAULT nextval('my_mx_seq'::regclass), value_3 bigserial) PARTITION BY RANGE (value_1);
|
||||||
|
CREATE TABLE users_table_part_0 PARTITION OF users_table_part FOR VALUES FROM (0) TO (1);
|
||||||
|
CREATE TABLE users_table_part_1 PARTITION OF users_table_part FOR VALUES FROM (1) TO (2);
|
||||||
|
SELECT create_distributed_table('users_table_part', 'user_id', colocate_with:='partitioned_table');
|
||||||
|
|
||||||
|
-- make sure that we can handle dropped columns nicely
|
||||||
|
ALTER TABLE users_table_part DROP COLUMN col_to_drop;
|
||||||
|
|
||||||
|
INSERT INTO users_table_part SELECT i, i %2, i %50 FROM generate_series(0, 100) i;
|
||||||
|
|
||||||
|
BEGIN;
|
||||||
|
-- make sure to use multiple connections
|
||||||
|
SET LOCAL citus.force_max_query_parallelization TO ON;
|
||||||
|
|
||||||
|
CREATE TABLE users_table_part_2 PARTITION OF users_table_part FOR VALUES FROM (2) TO (3);
|
||||||
|
INSERT INTO users_table_part SELECT i, i %3, i %50 FROM generate_series(0, 100) i;
|
||||||
|
|
||||||
|
CREATE TABLE users_table_part_3 (user_id int, value_1 int, value_2 int, value_3 bigserial);
|
||||||
|
ALTER TABLE users_table_part ATTACH PARTITION users_table_part_3 FOR VALUES FROM (3) TO (4);
|
||||||
|
CREATE TABLE users_table_part_4 PARTITION OF users_table_part FOR VALUES FROM (4) TO (5) USING COLUMNAR;;
|
||||||
|
COMMIT;
|
||||||
|
|
||||||
|
SELECT alter_table_set_access_method('users_table_part_0', 'columnar');
|
||||||
|
SELECT alter_table_set_access_method('users_table_part_0', 'heap');
|
||||||
|
|
||||||
|
BEGIN;
|
||||||
|
SET LOCAL citus.force_max_query_parallelization TO ON;
|
||||||
|
SELECT alter_table_set_access_method('users_table_part_0', 'columnar');
|
||||||
|
SELECT alter_table_set_access_method('users_table_part_0', 'heap');
|
||||||
|
ROLLBACK;
|
||||||
|
|
||||||
|
BEGIN;
|
||||||
|
SET LOCAL citus.force_max_query_parallelization TO ON;
|
||||||
|
SELECT undistribute_table('users_table_part');
|
||||||
|
SELECT create_distributed_table('users_table_part', 'user_id');
|
||||||
|
COMMIT;
|
||||||
|
|
||||||
|
BEGIN;
|
||||||
|
-- make sure to use multiple connections
|
||||||
|
SET LOCAL citus.force_max_query_parallelization TO ON;
|
||||||
|
SELECT alter_distributed_table('users_table_part', shard_count:=9, cascade_to_colocated:=false);
|
||||||
|
ROLLBACK;
|
||||||
|
|
||||||
|
BEGIN;
|
||||||
|
-- make sure to use multiple connections
|
||||||
|
SET LOCAL citus.force_max_query_parallelization TO ON;
|
||||||
|
ALTER TABLE users_table_part ADD COLUMN my_column INT DEFAULT 15;
|
||||||
|
CREATE INDEX test_index ON users_table_part(value_3, value_2);
|
||||||
|
CREATE INDEX test_index_on_child ON users_table_part_3(value_3, value_2);
|
||||||
|
ROLLBACK;
|
||||||
|
|
||||||
|
CREATE TABLE local_table_in_the_metadata (id int PRIMARY KEY, value_1 int);
|
||||||
|
|
||||||
|
CREATE TABLE reference_table(id int PRIMARY KEY, value_1 int);
|
||||||
|
SELECT create_reference_table('reference_table');
|
||||||
|
|
||||||
|
CREATE TABLE on_delete_fkey_table(id int PRIMARY KEY, value_1 int);
|
||||||
|
SELECT create_distributed_table('on_delete_fkey_table', 'id', colocate_with:='partitioned_table');
|
||||||
|
ALTER TABLE reference_table ADD CONSTRAINT fkey_to_local FOREIGN KEY(id) REFERENCES local_table_in_the_metadata(id);
|
||||||
|
ALTER TABLE on_delete_fkey_table ADD CONSTRAINT veerrrrrrryyy_veerrrrrrryyy_veerrrrrrryyy_long_constraint_name FOREIGN KEY(value_1) REFERENCES reference_table(id) ON DELETE CASCADE;
|
||||||
|
INSERT INTO local_table_in_the_metadata SELECT i, i FROM generate_series(0, 100) i;
|
||||||
|
INSERT INTO reference_table SELECT i, i FROM generate_series(0, 100) i;
|
||||||
|
INSERT INTO on_delete_fkey_table SELECT i, i % 100 FROM generate_series(0, 1000) i;
|
||||||
|
|
||||||
|
-- make sure that we can handle switching to sequential execution nicely
|
||||||
|
-- on MX with a regular user
|
||||||
|
BEGIN;
|
||||||
|
DELETE FROM reference_table WHERE id > 50;
|
||||||
|
SHOW citus.multi_shard_modify_mode;
|
||||||
|
ALTER TABLE on_delete_fkey_table ADD COLUMN t int DEFAULT 10;
|
||||||
|
SELECT avg(t) FROM on_delete_fkey_table;
|
||||||
|
ROLLBACK;
|
||||||
|
|
||||||
|
-- make sure to use multiple connections per node
|
||||||
|
SET citus.force_max_query_parallelization TO ON;
|
||||||
|
CREATE INDEX CONCURRENTLY concurrent_index_test ON on_delete_fkey_table(id);
|
||||||
|
CREATE UNIQUE INDEX unique_key_example ON on_delete_fkey_table(id, value_1);
|
||||||
|
|
||||||
|
BEGIN;
|
||||||
|
TRUNCATE local_table_in_the_metadata, reference_table, on_delete_fkey_table;
|
||||||
|
SELECT count(*) FROM local_table_in_the_metadata;
|
||||||
|
SELECT count(*) FROM reference_table;
|
||||||
|
SELECT count(*) FROM on_delete_fkey_table;
|
||||||
|
ROLLBACK;
|
||||||
|
|
||||||
|
BEGIN;
|
||||||
|
SET citus.multi_shard_modify_mode TO 'sequential';
|
||||||
|
TRUNCATE on_delete_fkey_table CASCADE;
|
||||||
|
TRUNCATE reference_table CASCADE;
|
||||||
|
SELECT count(*) FROM local_table_in_the_metadata JOIN reference_table USING(id) JOIN on_delete_fkey_table USING(id);
|
||||||
|
ROLLBACK;
|
||||||
|
|
||||||
|
-- join involving local, reference and distributed tables
|
||||||
|
SELECT count(*) FROM local_table_in_the_metadata JOIN reference_table USING(id) JOIN on_delete_fkey_table USING(id);
|
||||||
|
|
||||||
|
-- query with intermediate results
|
||||||
|
WITH cte_1 AS (SELECT * FROM on_delete_fkey_table ORDER BY 1,2 DESC LIMIT 10)
|
||||||
|
SELECT count(*) FROM cte_1;
|
||||||
|
|
||||||
|
-- query with intermediate results on remote nodes
|
||||||
|
WITH cte_1 AS (SELECT * FROM on_delete_fkey_table ORDER BY 1,2 DESC LIMIT 10)
|
||||||
|
SELECT count(*) FROM cte_1 JOIN on_delete_fkey_table USING(value_1);
|
||||||
|
|
||||||
|
-- repartition joins
|
||||||
|
SET citus.enable_repartition_joins to ON;
|
||||||
|
SELECT count(*) FROM on_delete_fkey_table o1 JOIN on_delete_fkey_table o2 USING(value_1);
|
||||||
|
|
||||||
|
-- repartition INSERT .. SELECT
|
||||||
|
INSERT INTO on_delete_fkey_table (id, value_1) SELECT value_1, id FROM on_delete_fkey_table ON CONFLICT DO NOTHING;
|
||||||
|
|
||||||
|
-- make sure that we can create a type and use it in the same tx
|
||||||
|
BEGIN;
|
||||||
|
CREATE TYPE test_type AS (a int, b int);
|
||||||
|
CREATE TABLE composite_key (id int PRIMARY KEY, c int, data test_type);
|
||||||
|
SELECT create_distributed_table('composite_key', 'id', colocate_with:='partitioned_table');
|
||||||
|
COMMIT;
|
||||||
|
|
||||||
|
-- index statistics should work fine
|
||||||
|
CREATE INDEX test_index_on_parent ON users_table_part((value_3+value_2));
|
||||||
|
ALTER INDEX test_index_on_parent ALTER COLUMN 1 SET STATISTICS 4646;
|
||||||
|
DROP INDEX test_index_on_parent;
|
||||||
|
|
||||||
|
ALTER TABLE composite_key ALTER COLUMN c TYPE float USING (b::float + 0.5);
|
||||||
|
|
||||||
|
-- make sure that rebalancer works fine with a regular user on MX
|
||||||
|
-- first make sure that we can rollback
|
||||||
|
BEGIN;
|
||||||
|
SELECT citus_move_shard_placement(1560000, 'localhost', :worker_1_port, 'localhost', :worker_2_port, 'block_writes');
|
||||||
|
ROLLBACK;
|
||||||
|
|
||||||
|
SELECT citus_move_shard_placement(1560000, 'localhost', :worker_1_port, 'localhost', :worker_2_port, 'block_writes');
|
||||||
|
|
||||||
|
-- connect to the worker to see if the table has the correct owned and placement metadata
|
||||||
|
\c - postgres - :worker_2_port
|
||||||
|
SELECT
|
||||||
|
1560000, groupid = (SELECT groupid FROM pg_dist_node WHERE nodeport = :worker_2_port AND nodename = 'localhost' AND isactive)
|
||||||
|
FROM
|
||||||
|
pg_dist_placement
|
||||||
|
WHERE
|
||||||
|
shardid = 1560000;
|
||||||
|
|
||||||
|
-- also make sure that pg_dist_shard_placement is updated correctly
|
||||||
|
SELECT
|
||||||
|
nodeport = :worker_2_port
|
||||||
|
FROM pg_dist_shard_placement WHERE shardid = 1560000;
|
||||||
|
|
||||||
|
\c - postgres - :worker_1_port
|
||||||
|
SELECT
|
||||||
|
1560000, groupid = (SELECT groupid FROM pg_dist_node WHERE nodeport = :worker_2_port AND nodename = 'localhost' AND isactive)
|
||||||
|
FROM
|
||||||
|
pg_dist_placement
|
||||||
|
WHERE
|
||||||
|
shardid = 1560000;
|
||||||
|
|
||||||
|
-- also make sure that pg_dist_shard_placement is updated correctly
|
||||||
|
SELECT
|
||||||
|
nodeport = :worker_2_port
|
||||||
|
FROM pg_dist_shard_placement WHERE shardid = 1560000;
|
||||||
|
|
||||||
|
-- now connect with the user to the coordinator again
|
||||||
|
\c - regular_mx_user - :master_port
|
||||||
|
SET search_path TO "Mx Regular User";
|
||||||
|
|
||||||
|
-- make sure that we can still execute queries
|
||||||
|
SELECT count(*) FROM local_table_in_the_metadata JOIN reference_table USING(id) JOIN on_delete_fkey_table USING(id);
|
||||||
|
|
||||||
|
-- now, call directly the rebalancer, which should also work fine
|
||||||
|
SELECT rebalance_table_shards(shard_transfer_mode:='block_writes');
|
||||||
|
-- make sure that we can still execute queries
|
||||||
|
SELECT count(*) FROM local_table_in_the_metadata JOIN reference_table USING(id) JOIN on_delete_fkey_table USING(id);
|
||||||
|
|
||||||
|
-- lets run some queries from the workers
|
||||||
|
\c - regular_mx_user - :worker_2_port
|
||||||
|
SET search_path TO "Mx Regular User";
|
||||||
|
SELECT count(*) FROM local_table_in_the_metadata JOIN reference_table USING(id) JOIN on_delete_fkey_table USING(id);
|
||||||
|
BEGIN;
|
||||||
|
TRUNCATE reference_table CASCADE;
|
||||||
|
ROLLBACK;
|
||||||
|
|
||||||
|
-- join involving local, reference and distributed tables
|
||||||
|
SELECT count(*) FROM local_table_in_the_metadata JOIN reference_table USING(id) JOIN on_delete_fkey_table USING(id);
|
||||||
|
|
||||||
|
-- query with intermediate results
|
||||||
|
WITH cte_1 AS (SELECT * FROM on_delete_fkey_table ORDER BY 1,2 DESC LIMIT 10)
|
||||||
|
SELECT count(*) FROM cte_1;
|
||||||
|
|
||||||
|
-- query with intermediate results on remote nodes
|
||||||
|
WITH cte_1 AS (SELECT * FROM on_delete_fkey_table ORDER BY 1,2 DESC LIMIT 10)
|
||||||
|
SELECT count(*) FROM cte_1 JOIN on_delete_fkey_table USING(value_1);
|
||||||
|
|
||||||
|
-- repartition joins
|
||||||
|
SET citus.enable_repartition_joins to ON;
|
||||||
|
SELECT count(*) FROM on_delete_fkey_table o1 JOIN on_delete_fkey_table o2 USING(value_1);
|
||||||
|
|
||||||
|
BEGIN;
|
||||||
|
SET LOCAL citus.force_max_query_parallelization TO ON;
|
||||||
|
DELETE FROM on_delete_fkey_table;
|
||||||
|
WITH cte_1 AS (SELECT * FROM on_delete_fkey_table ORDER BY 1,2 DESC LIMIT 10)
|
||||||
|
SELECT count(*) FROM cte_1;
|
||||||
|
COMMIT;
|
||||||
|
|
||||||
|
\c - postgres - :master_port
|
||||||
|
|
||||||
|
-- resync the metadata to both nodes for test purposes and then stop
|
||||||
|
SELECT start_metadata_sync_to_node('localhost', :worker_1_port);
|
||||||
|
SELECT start_metadata_sync_to_node('localhost', :worker_2_port);
|
||||||
|
|
||||||
|
SELECT stop_metadata_sync_to_node('localhost', :worker_1_port);
|
||||||
|
SELECT stop_metadata_sync_to_node('localhost', :worker_2_port);
|
||||||
|
|
||||||
|
DROP SCHEMA "Mx Regular User" CASCADE;
|
Loading…
Reference in New Issue