citus/src/backend/distributed/transaction/relation_access_tracking.c

1028 lines
29 KiB
C

/*-------------------------------------------------------------------------
*
* relation_access_tracking.c
*
* Transaction access tracking for Citus. The functions in this file
* are intended to track the relation accesses within a transaction. The
* logic here is mostly useful when a reference table is referred by
* a distributed table via a foreign key. Whenever such a pair of tables
* are acccesed inside a transaction, Citus should detect and act
* accordingly.
*
* Copyright (c) Citus Data, Inc.
*
*-------------------------------------------------------------------------
*/
#include "postgres.h"
#include "miscadmin.h"
#include "access/xact.h"
#include "distributed/colocation_utils.h"
#include "distributed/hash_helpers.h"
#include "distributed/multi_executor.h"
#include "distributed/multi_join_order.h"
#include "distributed/multi_partitioning_utils.h"
#include "distributed/metadata_cache.h"
#include "distributed/relation_access_tracking.h"
#include "utils/hsearch.h"
#include "utils/lsyscache.h"
/* Config variables managed via guc.c */
bool EnforceForeignKeyRestrictions = true;
#define PARALLEL_MODE_FLAG_OFFSET 3
/* simply set parallel bits as defined below for select, dml and ddl */
#define PARALLEL_ACCESS_MASK (int) (0 | \
(1 << (PLACEMENT_ACCESS_SELECT + \
PARALLEL_MODE_FLAG_OFFSET)) | \
(1 << (PLACEMENT_ACCESS_DML + \
PARALLEL_MODE_FLAG_OFFSET)) | \
(1 << (PLACEMENT_ACCESS_DDL + \
PARALLEL_MODE_FLAG_OFFSET)))
/*
* Hash table mapping relations to the
* (relationId) = (relationAccessType and relationAccessMode)
*
* RelationAccessHash is used to keep track of relation accesses types (e.g., select,
* dml or ddl) along with access modes (e.g., no access, sequential access or
* parallel access).
*
* We keep an integer per relation and use some of the bits to identify the access types
* and access modes.
*
* We store the access types in the first 3 bits:
* - 0th bit is set for SELECT accesses to a relation
* - 1st bit is set for DML accesses to a relation
* - 2nd bit is set for DDL accesses to a relation
*
* and, access modes in the next 3 bits:
* - 3rd bit is set for PARALLEL SELECT accesses to a relation
* - 4th bit is set for PARALLEL DML accesses to a relation
* - 5th bit is set for PARALLEL DDL accesses to a relation
*
*/
typedef struct RelationAccessHashKey
{
Oid relationId;
} RelationAccessHashKey;
typedef struct RelationAccessHashEntry
{
RelationAccessHashKey key;
int relationAccessMode;
} RelationAccessHashEntry;
static HTAB *RelationAccessHash;
/* functions related to access recording */
static void RecordRelationAccessBase(Oid relationId, ShardPlacementAccessType accessType);
static void RecordPlacementAccessToCache(Oid relationId,
ShardPlacementAccessType accessType);
static void RecordRelationParallelSelectAccessForTask(Task *task);
static void RecordRelationParallelModifyAccessForTask(Task *task);
static void RecordRelationParallelDDLAccessForTask(Task *task);
static RelationAccessMode GetRelationAccessMode(Oid relationId,
ShardPlacementAccessType accessType);
static void RecordParallelRelationAccess(Oid relationId, ShardPlacementAccessType
placementAccess);
static void RecordParallelRelationAccessToCache(Oid relationId,
ShardPlacementAccessType placementAccess);
/* functions related to access conflict checks */
static char * PlacementAccessTypeToText(ShardPlacementAccessType accessType);
static void CheckConflictingRelationAccesses(Oid relationId,
ShardPlacementAccessType accessType);
static bool HoldsConflictingLockWithReferencingRelations(Oid relationId,
ShardPlacementAccessType
placementAccess,
Oid *conflictingRelationId,
ShardPlacementAccessType *
conflictingAccessMode);
static void CheckConflictingParallelRelationAccesses(Oid relationId,
ShardPlacementAccessType
accessType);
static bool HoldsConflictingLockWithReferencedRelations(Oid relationId,
ShardPlacementAccessType
placementAccess,
Oid *conflictingRelationId,
ShardPlacementAccessType *
conflictingAccessMode);
/*
* Empty RelationAccessHash, without destroying the hash table itself.
*/
void
ResetRelationAccessHash()
{
hash_delete_all(RelationAccessHash);
}
/*
* Allocate RelationAccessHash.
*/
void
AllocateRelationAccessHash(void)
{
HASHCTL info;
memset(&info, 0, sizeof(info));
info.keysize = sizeof(RelationAccessHashKey);
info.entrysize = sizeof(RelationAccessHashEntry);
info.hash = tag_hash;
info.hcxt = ConnectionContext;
uint32 hashFlags = (HASH_ELEM | HASH_BLOBS | HASH_CONTEXT);
RelationAccessHash = hash_create("citus connection cache (relationid)",
8, &info, hashFlags);
}
/*
* RecordRelationAccessIfReferenceTable marks the relation accessed if it is a
* reference relation.
*
* The function is a wrapper around RecordRelationAccessBase().
*/
void
RecordRelationAccessIfReferenceTable(Oid relationId, ShardPlacementAccessType accessType)
{
if (!ShouldRecordRelationAccess())
{
return;
}
/*
* We keep track of relation accesses for the purposes of foreign keys to
* reference tables. So, other distributed tables are not relevant for now.
* Additionally, partitioned tables with lots of partitions might require
* recursively calling RecordRelationAccessBase(), so becareful about
* removing this check.
*/
if (PartitionMethod(relationId) != DISTRIBUTE_BY_NONE)
{
return;
}
RecordRelationAccessBase(relationId, accessType);
}
/*
* PlacementAccessTypeToText converts ShardPlacementAccessType to
* text representation.
*/
static char *
PlacementAccessTypeToText(ShardPlacementAccessType accessType)
{
switch (accessType)
{
case PLACEMENT_ACCESS_SELECT:
{
return "SELECT";
break;
}
case PLACEMENT_ACCESS_DML:
{
return "DML";
}
case PLACEMENT_ACCESS_DDL:
{
return "DDL";
}
default:
{
return "None";
break;
}
}
}
/*
* RecordRelationAccessBase associates the access to the distributed relation. The
* function takes partitioned relations into account as well.
*
* We implemented this function to prevent accessing placement metadata during
* recursive calls of the function itself (e.g., avoid
* RecordRelationAccessBase()).
*/
static void
RecordRelationAccessBase(Oid relationId, ShardPlacementAccessType accessType)
{
/*
* We call this only for reference tables, and we don't support partitioned
* reference tables.
*/
Assert(!PartitionedTable(relationId) && !PartitionTable(relationId));
/* make sure that this is not a conflicting access */
CheckConflictingRelationAccesses(relationId, accessType);
/* always record the relation that is being considered */
RecordPlacementAccessToCache(relationId, accessType);
}
/*
* RecordPlacementAccessToCache is a utility function which saves the given
* relation id's access to the RelationAccessHash.
*/
static void
RecordPlacementAccessToCache(Oid relationId, ShardPlacementAccessType accessType)
{
RelationAccessHashKey hashKey;
bool found = false;
hashKey.relationId = relationId;
RelationAccessHashEntry *hashEntry = hash_search(RelationAccessHash, &hashKey,
HASH_ENTER, &found);
if (!found)
{
hashEntry->relationAccessMode = 0;
}
/* set the bit representing the access type */
hashEntry->relationAccessMode |= (1 << (accessType));
}
/*
* RecordParallelRelationAccessForTaskList gets a task list and records
* the necessary parallel relation accesses for the task list.
*
* This function is used to enforce foreign keys from distributed
* tables to reference tables.
*/
void
RecordParallelRelationAccessForTaskList(List *taskList)
{
if (MultiShardConnectionType == SEQUENTIAL_CONNECTION)
{
/* sequential mode prevents parallel access */
return;
}
if (list_length(taskList) < 2)
{
/* single shard task doesn't mean parallel access in our definition */
return;
}
/*
* Since all the tasks in a task list is expected to operate on the same
* distributed table(s), we only need to process the first task.
*/
Task *firstTask = linitial(taskList);
if (firstTask->taskType == SQL_TASK)
{
RecordRelationParallelSelectAccessForTask(firstTask);
}
else if (firstTask->taskType == MODIFY_TASK)
{
if (firstTask->rowValuesLists != NIL)
{
/*
* We always run multi-row INSERTs in a sequential
* mode (hard-coded). Thus, we do not mark as parallel
* access even if the prerequisites hold.
*/
}
else
{
/*
* We prefer to mark with all remaining multi-shard modifications
* with both modify and select accesses.
*/
RecordRelationParallelModifyAccessForTask(firstTask);
RecordRelationParallelSelectAccessForTask(firstTask);
}
}
else
{
RecordRelationParallelDDLAccessForTask(firstTask);
}
}
/*
* RecordRelationParallelSelectAccessForTask goes over all the relations
* in the relationShardList and records the select access per each table.
*/
static void
RecordRelationParallelSelectAccessForTask(Task *task)
{
ListCell *relationShardCell = NULL;
Oid lastRelationId = InvalidOid;
/* no point in recoding accesses in non-transaction blocks, skip the loop */
if (!ShouldRecordRelationAccess())
{
return;
}
List *relationShardList = task->relationShardList;
foreach(relationShardCell, relationShardList)
{
RelationShard *relationShard = (RelationShard *) lfirst(relationShardCell);
Oid currentRelationId = relationShard->relationId;
/*
* An optimization, skip going to hash table if we've already
* recorded the relation.
*/
if (currentRelationId == lastRelationId)
{
continue;
}
RecordParallelSelectAccess(currentRelationId);
lastRelationId = currentRelationId;
}
}
/*
* RecordRelationParallelModifyAccessForTask gets a task and records
* the accesses. Note that the target relation is recorded with modify access
* where as the subqueries inside the modify query is recorded with select
* access.
*/
static void
RecordRelationParallelModifyAccessForTask(Task *task)
{
List *relationShardList = NULL;
ListCell *relationShardCell = NULL;
Oid lastRelationId = InvalidOid;
/* no point in recoding accesses in non-transaction blocks, skip the loop */
if (!ShouldRecordRelationAccess())
{
return;
}
/* anchor shard is always associated with modify access */
RecordParallelModifyAccess(RelationIdForShard(task->anchorShardId));
if (task->modifyWithSubquery)
{
relationShardList = task->relationShardList;
foreach(relationShardCell, relationShardList)
{
RelationShard *relationShard = (RelationShard *) lfirst(relationShardCell);
Oid currentRelationId = relationShard->relationId;
/*
* An optimization, skip going to hash table if we've already
* recorded the relation.
*/
if (currentRelationId == lastRelationId)
{
continue;
}
RecordParallelSelectAccess(currentRelationId);
lastRelationId = currentRelationId;
}
}
}
/*
* RecordRelationParallelDDLAccessForTask marks all the relationShards
* with parallel DDL access if exists. That case is valid for inter-shard
* DDL commands such as foreign key creation. The function also records
* the relation that anchorShardId belongs to.
*/
static void
RecordRelationParallelDDLAccessForTask(Task *task)
{
List *relationShardList = task->relationShardList;
ListCell *relationShardCell = NULL;
Oid lastRelationId = InvalidOid;
foreach(relationShardCell, relationShardList)
{
RelationShard *relationShard = (RelationShard *) lfirst(relationShardCell);
Oid currentRelationId = relationShard->relationId;
/*
* An optimization, skip going to hash table if we've already
* recorded the relation.
*/
if (currentRelationId == lastRelationId)
{
continue;
}
RecordParallelDDLAccess(currentRelationId);
lastRelationId = currentRelationId;
}
if (task->anchorShardId != INVALID_SHARD_ID)
{
RecordParallelDDLAccess(RelationIdForShard(task->anchorShardId));
}
}
/*
* RecordParallelSelectAccess is a wrapper around RecordParallelRelationAccess()
*/
void
RecordParallelSelectAccess(Oid relationId)
{
RecordParallelRelationAccess(relationId, PLACEMENT_ACCESS_SELECT);
}
/*
* RecordParallelModifyAccess is a wrapper around RecordParallelRelationAccess()
*/
void
RecordParallelModifyAccess(Oid relationId)
{
RecordParallelRelationAccess(relationId, PLACEMENT_ACCESS_DML);
}
/*
* RecordParallelDDLAccess is a wrapper around RecordParallelRelationAccess()
*/
void
RecordParallelDDLAccess(Oid relationId)
{
RecordParallelRelationAccess(relationId, PLACEMENT_ACCESS_DDL);
}
/*
* RecordParallelRelationAccess records the relation access mode as parallel
* for the given access type (e.g., select, dml or ddl) in the RelationAccessHash.
*
* The function also takes partitions and partitioned tables into account.
*/
static void
RecordParallelRelationAccess(Oid relationId, ShardPlacementAccessType placementAccess)
{
if (!ShouldRecordRelationAccess())
{
return;
}
/* act accordingly if it's a conflicting access */
CheckConflictingParallelRelationAccesses(relationId, placementAccess);
/* If a relation is partitioned, record accesses to all of its partitions as well. */
if (PartitionedTable(relationId))
{
List *partitionList = PartitionList(relationId);
ListCell *partitionCell = NULL;
foreach(partitionCell, partitionList)
{
Oid partitionOid = lfirst_oid(partitionCell);
/* recursively record all relation accesses of its partitions */
RecordParallelRelationAccess(partitionOid, placementAccess);
}
}
else if (PartitionTable(relationId))
{
Oid parentOid = PartitionParentOid(relationId);
/* only record the parent */
RecordParallelRelationAccessToCache(parentOid, placementAccess);
}
RecordParallelRelationAccessToCache(relationId, placementAccess);
}
/*
* RecordParallelRelationAccessToCache is a utility function which saves the given
* relation id's access to the RelationAccessHash.
*/
static void
RecordParallelRelationAccessToCache(Oid relationId,
ShardPlacementAccessType placementAccess)
{
RelationAccessHashKey hashKey;
bool found = false;
hashKey.relationId = relationId;
RelationAccessHashEntry *hashEntry = hash_search(RelationAccessHash, &hashKey,
HASH_ENTER, &found);
if (!found)
{
hashEntry->relationAccessMode = 0;
}
/* set the bit representing the access type */
hashEntry->relationAccessMode |= (1 << (placementAccess));
/* set the bit representing access mode */
int parallelRelationAccessBit = placementAccess + PARALLEL_MODE_FLAG_OFFSET;
hashEntry->relationAccessMode |= (1 << parallelRelationAccessBit);
}
/*
* ParallelQueryExecutedInTransaction returns true if any parallel query
* is executed in the current transaction.
*/
bool
ParallelQueryExecutedInTransaction(void)
{
HASH_SEQ_STATUS status;
if (!ShouldRecordRelationAccess() || RelationAccessHash == NULL)
{
return false;
}
hash_seq_init(&status, RelationAccessHash);
RelationAccessHashEntry *hashEntry = (RelationAccessHashEntry *) hash_seq_search(
&status);
while (hashEntry != NULL)
{
int relationAccessMode = hashEntry->relationAccessMode;
if ((relationAccessMode & PARALLEL_ACCESS_MASK))
{
hash_seq_term(&status);
return true;
}
hashEntry = (RelationAccessHashEntry *) hash_seq_search(&status);
}
return false;
}
/*
* GetRelationSelectAccessMode is a wrapper around GetRelationAccessMode.
*/
RelationAccessMode
GetRelationSelectAccessMode(Oid relationId)
{
return GetRelationAccessMode(relationId, PLACEMENT_ACCESS_SELECT);
}
/*
* GetRelationDMLAccessMode is a wrapper around GetRelationAccessMode.
*/
RelationAccessMode
GetRelationDMLAccessMode(Oid relationId)
{
return GetRelationAccessMode(relationId, PLACEMENT_ACCESS_DML);
}
/*
* GetRelationDDLAccessMode is a wrapper around GetRelationAccessMode.
*/
RelationAccessMode
GetRelationDDLAccessMode(Oid relationId)
{
return GetRelationAccessMode(relationId, PLACEMENT_ACCESS_DDL);
}
/*
* GetRelationAccessMode returns the relation access mode (e.g., none, sequential
* or parallel) for the given access type (e.g., select, dml or ddl).
*/
static RelationAccessMode
GetRelationAccessMode(Oid relationId, ShardPlacementAccessType accessType)
{
RelationAccessHashKey hashKey;
bool found = false;
int parallelRelationAccessBit = accessType + PARALLEL_MODE_FLAG_OFFSET;
/* no point in getting the mode when not inside a transaction block */
if (!ShouldRecordRelationAccess())
{
return RELATION_NOT_ACCESSED;
}
hashKey.relationId = relationId;
RelationAccessHashEntry *hashEntry = hash_search(RelationAccessHash, &hashKey,
HASH_FIND, &found);
if (!found)
{
/* relation not accessed at all */
return RELATION_NOT_ACCESSED;
}
int relationAcessMode = hashEntry->relationAccessMode;
if (!(relationAcessMode & (1 << accessType)))
{
/* relation not accessed with the given access type */
return RELATION_NOT_ACCESSED;
}
if (relationAcessMode & (1 << parallelRelationAccessBit))
{
return RELATION_PARALLEL_ACCESSED;
}
else
{
return RELATION_REFERENCE_ACCESSED;
}
}
/*
* ShouldRecordRelationAccess returns true when we should keep track
* of the relation accesses.
*
* In many cases, we'd only need IsMultiStatementTransaction(), however, for some
* cases such as CTEs, where Citus uses the same connections accross multiple queries,
* we should still record the relation accesses even not inside an explicit transaction
* block. Thus, keeping track of the relation accesses inside coordinated transactions
* is also required.
*/
bool
ShouldRecordRelationAccess()
{
if (EnforceForeignKeyRestrictions &&
(IsMultiStatementTransaction() || InCoordinatedTransaction()))
{
return true;
}
return false;
}
/*
* CheckConflictingRelationAccesses is mostly a wrapper around
* HoldsConflictingLockWithReferencingRelations(). We're only interested in accesses
* to reference tables that are referenced via a foreign constraint by a
* hash distributed tables.
*/
static void
CheckConflictingRelationAccesses(Oid relationId, ShardPlacementAccessType accessType)
{
Oid conflictingReferencingRelationId = InvalidOid;
ShardPlacementAccessType conflictingAccessType = PLACEMENT_ACCESS_SELECT;
if (!EnforceForeignKeyRestrictions || !IsDistributedTable(relationId))
{
return;
}
DistTableCacheEntry *cacheEntry = DistributedTableCacheEntry(relationId);
if (!(cacheEntry->partitionMethod == DISTRIBUTE_BY_NONE &&
cacheEntry->referencingRelationsViaForeignKey != NIL))
{
return;
}
if (HoldsConflictingLockWithReferencingRelations(relationId, accessType,
&conflictingReferencingRelationId,
&conflictingAccessType))
{
char *relationName = get_rel_name(relationId);
char *conflictingRelationName = get_rel_name(conflictingReferencingRelationId);
char *accessTypeText = PlacementAccessTypeToText(accessType);
char *conflictingAccessTypeText =
PlacementAccessTypeToText(conflictingAccessType);
ereport(ERROR, (errmsg("cannot execute %s on reference relation \"%s\" because "
"there was a parallel %s access to distributed relation "
"\"%s\" in the same transaction",
accessTypeText, relationName, conflictingAccessTypeText,
conflictingRelationName),
errhint("Try re-running the transaction with "
"\"SET LOCAL citus.multi_shard_modify_mode TO "
"\'sequential\';\"")));
}
else if (cacheEntry->referencingRelationsViaForeignKey != NIL &&
accessType > PLACEMENT_ACCESS_SELECT)
{
char *relationName = get_rel_name(relationId);
if (ParallelQueryExecutedInTransaction())
{
/*
* If there has already been a parallel query executed, the sequential mode
* would still use the already opened parallel connections to the workers,
* thus contradicting our purpose of using sequential mode.
*/
ereport(ERROR, (errmsg("cannot modify reference table \"%s\" because there "
"was a parallel operation on a distributed table",
relationName),
errdetail("When there is a foreign key to a reference "
"table, Citus needs to perform all operations "
"over a single connection per node to ensure "
"consistency."),
errhint("Try re-running the transaction with "
"\"SET LOCAL citus.multi_shard_modify_mode TO "
"\'sequential\';\"")));
}
else if (MultiShardConnectionType == PARALLEL_CONNECTION)
{
/*
* We can still continue with multi-shard queries in sequential mode, so
* set it.
*/
ereport(DEBUG1, (errmsg("switching to sequential query execution mode"),
errdetail(
"Reference relation \"%s\" is modified, which might lead "
"to data inconsistencies or distributed deadlocks via "
"parallel accesses to hash distributed relations due to "
"foreign keys. Any parallel modification to "
"those hash distributed relations in the same "
"transaction can only be executed in sequential query "
"execution mode", relationName)));
/*
* Switching to sequential mode is admittedly confusing and, could be useless
* and less performant in some cases. However, if we do not switch to
* sequential mode at this point, we'd loose the opportunity to do so
* later when a parallel query is executed on the hash distributed relations
* that are referencing this reference table.
*/
SetLocalMultiShardModifyModeToSequential();
}
}
}
/*
* CheckConflictingParallelRelationAccesses is mostly a wrapper around
* HoldsConflictingLockWithReferencedRelations(). We're only interested in parallel
* accesses to distributed tables that refers reference tables via foreign constraint.
*
*/
static void
CheckConflictingParallelRelationAccesses(Oid relationId, ShardPlacementAccessType
accessType)
{
Oid conflictingReferencingRelationId = InvalidOid;
ShardPlacementAccessType conflictingAccessType = PLACEMENT_ACCESS_SELECT;
if (!EnforceForeignKeyRestrictions || !IsDistributedTable(relationId))
{
return;
}
DistTableCacheEntry *cacheEntry = DistributedTableCacheEntry(relationId);
if (!(cacheEntry->partitionMethod == DISTRIBUTE_BY_HASH &&
cacheEntry->referencedRelationsViaForeignKey != NIL))
{
return;
}
if (MultiShardConnectionType == PARALLEL_CONNECTION &&
HoldsConflictingLockWithReferencedRelations(relationId, accessType,
&conflictingReferencingRelationId,
&conflictingAccessType))
{
char *relationName = get_rel_name(relationId);
char *conflictingRelationName = get_rel_name(conflictingReferencingRelationId);
char *accessTypeText = PlacementAccessTypeToText(accessType);
char *conflictingAccessTypeText =
PlacementAccessTypeToText(conflictingAccessType);
if (ParallelQueryExecutedInTransaction())
{
/*
* If there has already been a parallel query executed, the sequential mode
* would still use the already opened parallel connections to the workers,
* thus contradicting our purpose of using sequential mode.
*/
ereport(ERROR, (errmsg("cannot execute parallel %s on relation \"%s\" "
"after %s command on reference relation "
"\"%s\" because there is a foreign key between "
"them and \"%s\" has been accessed in this transaction",
accessTypeText, relationName,
conflictingAccessTypeText, conflictingRelationName,
conflictingRelationName),
errdetail("When there is a foreign key to a reference "
"table, Citus needs to perform all operations "
"over a single connection per node to ensure "
"consistency."),
errhint("Try re-running the transaction with "
"\"SET LOCAL citus.multi_shard_modify_mode TO "
"\'sequential\';\"")));
}
else
{
ereport(DEBUG1, (errmsg("switching to sequential query execution mode"),
errdetail("cannot execute parallel %s on relation \"%s\" "
"after %s command on reference relation "
"\"%s\" because there is a foreign key between "
"them and \"%s\" has been accessed in this transaction",
accessTypeText, relationName,
conflictingAccessTypeText, conflictingRelationName,
conflictingRelationName)));
SetLocalMultiShardModifyModeToSequential();
}
}
}
/*
* HoldsConflictingLockWithReferencedRelations returns true if the input relationId is a
* hash distributed table and it holds any conflicting locks with the reference tables that
* the distributed table has a foreign key to the reference table.
*/
static bool
HoldsConflictingLockWithReferencedRelations(Oid relationId, ShardPlacementAccessType
placementAccess,
Oid *conflictingRelationId,
ShardPlacementAccessType *
conflictingAccessMode)
{
DistTableCacheEntry *cacheEntry = DistributedTableCacheEntry(relationId);
ListCell *referencedRelationCell = NULL;
foreach(referencedRelationCell, cacheEntry->referencedRelationsViaForeignKey)
{
Oid referencedRelation = lfirst_oid(referencedRelationCell);
/* we're only interested in foreign keys to reference tables */
if (PartitionMethod(referencedRelation) != DISTRIBUTE_BY_NONE)
{
continue;
}
/*
* A select on a reference table could conflict with a DDL
* on a distributed table.
*/
RelationAccessMode selectMode = GetRelationSelectAccessMode(referencedRelation);
if (placementAccess == PLACEMENT_ACCESS_DDL &&
selectMode != RELATION_NOT_ACCESSED)
{
*conflictingRelationId = referencedRelation;
*conflictingAccessMode = PLACEMENT_ACCESS_SELECT;
return true;
}
/*
* Both DML and DDL operations on a reference table conflicts with
* any parallel operation on distributed tables.
*/
RelationAccessMode dmlMode = GetRelationDMLAccessMode(referencedRelation);
if (dmlMode != RELATION_NOT_ACCESSED)
{
*conflictingRelationId = referencedRelation;
*conflictingAccessMode = PLACEMENT_ACCESS_DML;
return true;
}
RelationAccessMode ddlMode = GetRelationDDLAccessMode(referencedRelation);
if (ddlMode != RELATION_NOT_ACCESSED)
{
*conflictingRelationId = referencedRelation;
*conflictingAccessMode = PLACEMENT_ACCESS_DDL;
return true;
}
}
return false;
}
/*
* HoldsConflictingLockWithReferencingRelations returns true when the input relationId is a
* reference table and it holds any conflicting locks with the distributed tables where
* the distributed table has a foreign key to the reference table.
*
* If returns true, the referencing relation and conflictingAccessMode are also set.
*/
static bool
HoldsConflictingLockWithReferencingRelations(Oid relationId, ShardPlacementAccessType
placementAccess, Oid *conflictingRelationId,
ShardPlacementAccessType *
conflictingAccessMode)
{
DistTableCacheEntry *cacheEntry = DistributedTableCacheEntry(relationId);
ListCell *referencingRelationCell = NULL;
bool holdsConflictingLocks = false;
Assert(PartitionMethod(relationId) == DISTRIBUTE_BY_NONE);
foreach(referencingRelationCell, cacheEntry->referencingRelationsViaForeignKey)
{
Oid referencingRelation = lfirst_oid(referencingRelationCell);
/*
* We're only interested in foreign keys to reference tables from
* hash distributed tables.
*/
if (!IsDistributedTable(referencingRelation) ||
PartitionMethod(referencingRelation) != DISTRIBUTE_BY_HASH)
{
continue;
}
/*
* Rules that we apply:
* - SELECT on a reference might table conflict with
* a previous parallel DDL on a distributed table
* - DML on a reference table might conflict with
* a previous parallel DML or DDL on a distributed
* table
* - DDL on a reference table might conflict with
* a parellel SELECT, DML or DDL on a distributed
* table
*/
if (placementAccess == PLACEMENT_ACCESS_SELECT)
{
RelationAccessMode ddlMode = GetRelationDDLAccessMode(referencingRelation);
if (ddlMode == RELATION_PARALLEL_ACCESSED)
{
/* SELECT on a distributed table conflicts with DDL / TRUNCATE */
holdsConflictingLocks = true;
*conflictingAccessMode = PLACEMENT_ACCESS_DDL;
}
}
else if (placementAccess == PLACEMENT_ACCESS_DML)
{
RelationAccessMode dmlMode = GetRelationDMLAccessMode(referencingRelation);
if (dmlMode == RELATION_PARALLEL_ACCESSED)
{
holdsConflictingLocks = true;
*conflictingAccessMode = PLACEMENT_ACCESS_DML;
}
RelationAccessMode ddlMode = GetRelationDDLAccessMode(referencingRelation);
if (ddlMode == RELATION_PARALLEL_ACCESSED)
{
/* SELECT on a distributed table conflicts with DDL / TRUNCATE */
holdsConflictingLocks = true;
*conflictingAccessMode = PLACEMENT_ACCESS_DDL;
}
}
else if (placementAccess == PLACEMENT_ACCESS_DDL)
{
RelationAccessMode selectMode = GetRelationSelectAccessMode(
referencingRelation);
if (selectMode == RELATION_PARALLEL_ACCESSED)
{
holdsConflictingLocks = true;
*conflictingAccessMode = PLACEMENT_ACCESS_SELECT;
}
RelationAccessMode dmlMode = GetRelationDMLAccessMode(referencingRelation);
if (dmlMode == RELATION_PARALLEL_ACCESSED)
{
holdsConflictingLocks = true;
*conflictingAccessMode = PLACEMENT_ACCESS_DML;
}
RelationAccessMode ddlMode = GetRelationDDLAccessMode(referencingRelation);
if (ddlMode == RELATION_PARALLEL_ACCESSED)
{
holdsConflictingLocks = true;
*conflictingAccessMode = PLACEMENT_ACCESS_DDL;
}
}
if (holdsConflictingLocks)
{
*conflictingRelationId = referencingRelation;
return true;
}
}
return false;
}