diff --git a/src/backend/distributed/commands/create_distributed_table.c b/src/backend/distributed/commands/create_distributed_table.c index 7b936e53b..d54fc02f3 100644 --- a/src/backend/distributed/commands/create_distributed_table.c +++ b/src/backend/distributed/commands/create_distributed_table.c @@ -265,7 +265,7 @@ create_distributed_table(PG_FUNCTION_ARGS) /* - * CreateReferenceTable creates a distributed table with the given relationId. The + * create_reference_table creates a distributed table with the given relationId. The * created table has one shard and replication factor is set to the active worker * count. In fact, the above is the definition of a reference table in Citus. */ diff --git a/src/backend/distributed/commands/extension.c b/src/backend/distributed/commands/extension.c index 9b57a3ea4..6688be031 100644 --- a/src/backend/distributed/commands/extension.c +++ b/src/backend/distributed/commands/extension.c @@ -540,10 +540,10 @@ PostprocessAlterExtensionCitusUpdateStmt(Node *node) /* - * MarkAllExistingObjectsDistributed marks all objects that could be distributed by - * resolving dependencies of "existing distributed tables" and "already distributed - * objects" to introduce the objects created in older versions of Citus to distributed - * object infrastructure as well. + * MarkExistingObjectDependenciesDistributedIfSupported marks all objects that could + * be distributed by resolving dependencies of "existing distributed tables" and + * "already distributed objects" to introduce the objects created in older versions + * of Citus to distributed object infrastructure as well. * * Note that this function is not responsible for ensuring if dependencies exist on * nodes and satisfying these dependendencies if not exists, which is already done by diff --git a/src/backend/distributed/commands/foreign_constraint.c b/src/backend/distributed/commands/foreign_constraint.c index c0a32da1a..a7e0ce6f8 100644 --- a/src/backend/distributed/commands/foreign_constraint.c +++ b/src/backend/distributed/commands/foreign_constraint.c @@ -722,7 +722,7 @@ get_relation_constraint_oid_compat(HeapTuple heapTuple) /* - * HasForeignKeyToLocalTable returns true if relation has foreign key + * HasForeignKeyWithLocalTable returns true if relation has foreign key * relationship with a local table. */ bool @@ -857,8 +857,8 @@ TableReferencing(Oid relationId) /* - * ConstraintWithNameIsOfType is a wrapper around ConstraintWithNameIsOfType that returns true - * if given constraint name identifies a uniqueness constraint, i.e: + * ConstraintIsAUniquenessConstraint is a wrapper around ConstraintWithNameIsOfType + * that returns true if given constraint name identifies a uniqueness constraint, i.e: * - primary key constraint, or * - unique constraint */ diff --git a/src/backend/distributed/commands/index.c b/src/backend/distributed/commands/index.c index 5a196db91..38ac46a2d 100644 --- a/src/backend/distributed/commands/index.c +++ b/src/backend/distributed/commands/index.c @@ -331,7 +331,7 @@ ExecuteFunctionOnEachTableIndex(Oid relationId, PGIndexProcessor pgIndexProcesso /* - * SwitchToSequentialOrLocalExecutionIfIndexNameTooLong generates the longest index name + * SwitchToSequentialAndLocalExecutionIfIndexNameTooLong generates the longest index name * on the shards of the partitions, and if exceeds the limit switches to sequential and * local execution to prevent self-deadlocks. */ diff --git a/src/backend/distributed/commands/rename.c b/src/backend/distributed/commands/rename.c index db1689ef2..7b6b71839 100644 --- a/src/backend/distributed/commands/rename.c +++ b/src/backend/distributed/commands/rename.c @@ -137,7 +137,7 @@ PreprocessRenameStmt(Node *node, const char *renameCommand, /* - * ErrorIfDistributedRenameStmt errors out if the corresponding rename statement + * ErrorIfUnsupportedRenameStmt errors out if the corresponding rename statement * operates on any part of a distributed table other than a column. * * Note: This function handles RenameStmt applied to relations handed by Citus. diff --git a/src/backend/distributed/commands/table.c b/src/backend/distributed/commands/table.c index e5a8101ea..3fa4e2343 100644 --- a/src/backend/distributed/commands/table.c +++ b/src/backend/distributed/commands/table.c @@ -1576,7 +1576,7 @@ IsAlterTableRenameStmt(RenameStmt *renameStmt) /* - * ErrorIfDropPartitionColumn checks if any subcommands of the given alter table + * ErrorIfAlterDropsPartitionColumn checks if any subcommands of the given alter table * command is a DROP COLUMN command which drops the partition column of a distributed * table. If there is such a subcommand, this function errors out. */ @@ -2967,7 +2967,7 @@ AlterInvolvesPartitionColumn(AlterTableStmt *alterTableStatement, /* - * ErrorIfUnsopprtedAlterAddConstraintStmt runs the constraint checks on distributed + * ErrorIfUnsupportedAlterAddConstraintStmt runs the constraint checks on distributed * table using the same logic with create_distributed_table. */ static void diff --git a/src/backend/distributed/commands/utility_hook.c b/src/backend/distributed/commands/utility_hook.c index 12fdc5f45..cae66049f 100644 --- a/src/backend/distributed/commands/utility_hook.c +++ b/src/backend/distributed/commands/utility_hook.c @@ -98,7 +98,7 @@ static bool ShouldUndistributeCitusLocalTables(void); /* - * ProcessUtilityForParseTree is a convenience method to create a PlannedStmt out of + * ProcessUtilityParseTree is a convenience method to create a PlannedStmt out of * pieces of a utility statement before invoking ProcessUtility. */ void diff --git a/src/backend/distributed/commands/vacuum.c b/src/backend/distributed/commands/vacuum.c index 37acfd6ba..3b57c9d36 100644 --- a/src/backend/distributed/commands/vacuum.c +++ b/src/backend/distributed/commands/vacuum.c @@ -198,7 +198,7 @@ ExecuteVacuumOnDistributedTables(VacuumStmt *vacuumStmt, List *relationIdList, /* - * IsSupportedDistributedVacuumStmt returns whether distributed execution of a + * IsDistributedVacuumStmt returns whether distributed execution of a * given VacuumStmt is supported. The provided relationId list represents * the list of tables targeted by the provided statement. * diff --git a/src/backend/distributed/connection/connection_configuration.c b/src/backend/distributed/connection/connection_configuration.c index 43ceaefd7..a0bd31e25 100644 --- a/src/backend/distributed/connection/connection_configuration.c +++ b/src/backend/distributed/connection/connection_configuration.c @@ -43,7 +43,7 @@ static Size CalculateMaxSize(void); static int uri_prefix_length(const char *connstr); /* - * InitConnParms initializes the ConnParams field to point to enough memory to + * InitConnParams initializes the ConnParams field to point to enough memory to * store settings for every valid libpq value, though these regions are set to * zeros from the outset and the size appropriately also set to zero. * diff --git a/src/backend/distributed/connection/remote_commands.c b/src/backend/distributed/connection/remote_commands.c index 89af02224..b566b53da 100644 --- a/src/backend/distributed/connection/remote_commands.c +++ b/src/backend/distributed/connection/remote_commands.c @@ -80,7 +80,7 @@ ForgetResults(MultiConnection *connection) /* - * ClearResultsInternal clears a connection from pending activity, + * ClearResults clears a connection from pending activity, * returns true if all pending commands return success. It raises * error if raiseErrors flag is set, any command fails and transaction * is marked critical. diff --git a/src/backend/distributed/deparser/deparse_role_stmts.c b/src/backend/distributed/deparser/deparse_role_stmts.c index 51eb13a79..047bdbb60 100644 --- a/src/backend/distributed/deparser/deparse_role_stmts.c +++ b/src/backend/distributed/deparser/deparse_role_stmts.c @@ -174,8 +174,8 @@ AppendRoleOption(StringInfo buf, ListCell *optionCell) /* - * AppendAlterRoleStmt generates the string representation of the - * AlterRoleStmt and appends it to the buffer. + * AppendAlterRoleSetStmt generates the string representation of the + * AlterRoleSetStmt and appends it to the buffer. */ static void AppendAlterRoleSetStmt(StringInfo buf, AlterRoleSetStmt *stmt) diff --git a/src/backend/distributed/deparser/qualify_statistics_stmt.c b/src/backend/distributed/deparser/qualify_statistics_stmt.c index 2a8cb1eaa..b8a7ce30e 100644 --- a/src/backend/distributed/deparser/qualify_statistics_stmt.c +++ b/src/backend/distributed/deparser/qualify_statistics_stmt.c @@ -146,8 +146,8 @@ QualifyAlterStatisticsStmt(Node *node) #endif /* - * QualifyAlterStatisticsStmt qualifies AlterOwnerStmt's with schema name for - * ALTER STATISTICS .. OWNER TO statements. + * QualifyAlterStatisticsOwnerStmt qualifies AlterOwnerStmt's with schema + * name for ALTER STATISTICS .. OWNER TO statements. */ void QualifyAlterStatisticsOwnerStmt(Node *node) diff --git a/src/backend/distributed/executor/adaptive_executor.c b/src/backend/distributed/executor/adaptive_executor.c index 21c1b4e1e..090472c27 100644 --- a/src/backend/distributed/executor/adaptive_executor.c +++ b/src/backend/distributed/executor/adaptive_executor.c @@ -974,7 +974,7 @@ ExecuteTaskListOutsideTransaction(RowModifyLevel modLevel, List *taskList, /* - * ExecuteTaskListIntoTupleStore is a proxy to ExecuteTaskListExtended() with defaults + * ExecuteTaskListIntoTupleDest is a proxy to ExecuteTaskListExtended() with defaults * for some of the arguments. */ uint64 @@ -2418,8 +2418,8 @@ RunDistributedExecution(DistributedExecution *execution) /* - * ProcessSessionsWithFailedEventSetOperations goes over the session list and - * processes sessions with failed wait event set operations. + * ProcessSessionsWithFailedWaitEventSetOperations goes over the session list + * and processes sessions with failed wait event set operations. * * Failed sessions are not going to generate any further events, so it is our * only chance to process the failure by calling into `ConnectionStateMachine`. diff --git a/src/backend/distributed/executor/distributed_intermediate_results.c b/src/backend/distributed/executor/distributed_intermediate_results.c index c30ea8cbb..8a29e633d 100644 --- a/src/backend/distributed/executor/distributed_intermediate_results.c +++ b/src/backend/distributed/executor/distributed_intermediate_results.c @@ -297,7 +297,7 @@ CreatePartitioningTupleDest(CitusTableCacheEntry *targetRelation) /* - * PartitioningTupleDestTupleDescForQuery implements TupleDestination->putTuple for + * PartitioningTupleDestPutTuple implements TupleDestination->putTuple for * PartitioningTupleDest. */ static void diff --git a/src/backend/distributed/executor/local_executor.c b/src/backend/distributed/executor/local_executor.c index 1844a3246..48e056cca 100644 --- a/src/backend/distributed/executor/local_executor.c +++ b/src/backend/distributed/executor/local_executor.c @@ -144,7 +144,7 @@ GetCurrentLocalExecutionStatus(void) /* - * ExecuteLocalTasks executes the given tasks locally. + * ExecuteLocalTaskList executes the given tasks locally. * * The function goes over the task list and executes them locally. * The returning tuples (if any) is stored in the tupleStoreState. diff --git a/src/backend/distributed/executor/tuple_destination.c b/src/backend/distributed/executor/tuple_destination.c index 13dedac08..42dbf001e 100644 --- a/src/backend/distributed/executor/tuple_destination.c +++ b/src/backend/distributed/executor/tuple_destination.c @@ -205,7 +205,7 @@ CreateTupleDestNone(void) /* - * TupleStoreTupleDestPutTuple implements TupleDestination->putTuple for + * TupleDestNonePutTuple implements TupleDestination->putTuple for * no-op tuple destination. */ static void @@ -218,7 +218,7 @@ TupleDestNonePutTuple(TupleDestination *self, Task *task, /* - * TupleStoreTupleDestTupleDescForQuery implements TupleDestination->TupleDescForQuery + * TupleDestNoneTupleDescForQuery implements TupleDestination->TupleDescForQuery * for no-op tuple destination. */ static TupleDesc @@ -262,7 +262,7 @@ TupleDestDestReceiverStartup(DestReceiver *destReceiver, int operation, /* - * TupleDestDestReceiverStartup implements DestReceiver->receiveSlot for + * TupleDestDestReceiverReceive implements DestReceiver->receiveSlot for * TupleDestDestReceiver. */ static bool @@ -292,7 +292,7 @@ TupleDestDestReceiverReceive(TupleTableSlot *slot, /* - * TupleDestDestReceiverStartup implements DestReceiver->rShutdown for + * TupleDestDestReceiverShutdown implements DestReceiver->rShutdown for * TupleDestDestReceiver. */ static void @@ -303,7 +303,7 @@ TupleDestDestReceiverShutdown(DestReceiver *destReceiver) /* - * TupleDestDestReceiverStartup implements DestReceiver->rDestroy for + * TupleDestDestReceiverDestroy implements DestReceiver->rDestroy for * TupleDestDestReceiver. */ static void diff --git a/src/backend/distributed/metadata/metadata_cache.c b/src/backend/distributed/metadata/metadata_cache.c index fa01aa42a..9db80a6cc 100644 --- a/src/backend/distributed/metadata/metadata_cache.c +++ b/src/backend/distributed/metadata/metadata_cache.c @@ -942,7 +942,7 @@ InitializeTableCacheEntry(int64 shardId) /* - * RefreshInvalidTableCacheEntry checks if the cache entry is still valid and + * RefreshTableCacheEntryIfInvalid checks if the cache entry is still valid and * refreshes it in cache when it's not. It returns true if it refreshed the * entry in the cache and false if it didn't. */ @@ -3618,7 +3618,7 @@ ResetCitusTableCacheEntry(CitusTableCacheEntry *cacheEntry) /* - * RemoveShardIdCacheEntries removes all shard ID cache entries belonging to the + * RemoveStaleShardIdCacheEntries removes all shard ID cache entries belonging to the * given table entry. If the shard ID belongs to a different (newer) table entry, * we leave it in place. */ diff --git a/src/backend/distributed/metadata/node_metadata.c b/src/backend/distributed/metadata/node_metadata.c index 5640f2c81..49c7be38f 100644 --- a/src/backend/distributed/metadata/node_metadata.c +++ b/src/backend/distributed/metadata/node_metadata.c @@ -701,7 +701,7 @@ GroupForNode(char *nodeName, int nodePort) /* - * NodeIsPrimaryAndLocal returns whether the argument represents the local + * NodeIsPrimaryAndRemote returns whether the argument represents the remote * primary node. */ bool diff --git a/src/backend/distributed/operations/citus_tools.c b/src/backend/distributed/operations/citus_tools.c index 9b2eedefb..de267a822 100644 --- a/src/backend/distributed/operations/citus_tools.c +++ b/src/backend/distributed/operations/citus_tools.c @@ -206,7 +206,7 @@ ParseCommandParameters(FunctionCallInfo fcinfo, StringInfo **nodeNameArray, /* - * ExecuteCommandsInParellelAndStoreResults connects to each node specified in + * ExecuteCommandsInParallelAndStoreResults connects to each node specified in * nodeNameArray and nodePortArray, and executes command in commandStringArray * in parallel fashion. Execution success status and result is reported for * each command in statusArray and resultStringArray. Each array contains diff --git a/src/backend/distributed/operations/repair_shards.c b/src/backend/distributed/operations/repair_shards.c index 8a7a276f0..2b3f44ddd 100644 --- a/src/backend/distributed/operations/repair_shards.c +++ b/src/backend/distributed/operations/repair_shards.c @@ -525,7 +525,7 @@ ErrorIfMoveUnsupportedTableType(Oid relationId) /* - * BlockWritesToColocatedShardList blocks writes to all shards in the given shard + * BlockWritesToShardList blocks writes to all shards in the given shard * list. The function assumes that all the shards in the list are colocated. */ void diff --git a/src/backend/distributed/operations/shard_cleaner.c b/src/backend/distributed/operations/shard_cleaner.c index 5423e2745..d07403355 100644 --- a/src/backend/distributed/operations/shard_cleaner.c +++ b/src/backend/distributed/operations/shard_cleaner.c @@ -217,7 +217,7 @@ DropOrphanedShards(bool waitForLocks) /* - * TryLockRelationAndCleanup tries to lock the given relation + * TryLockRelationAndPlacementCleanup tries to lock the given relation * and the placement cleanup. If it cannot, it returns false. * */ diff --git a/src/backend/distributed/operations/shard_rebalancer.c b/src/backend/distributed/operations/shard_rebalancer.c index 282dc13b5..4e24d8055 100644 --- a/src/backend/distributed/operations/shard_rebalancer.c +++ b/src/backend/distributed/operations/shard_rebalancer.c @@ -2048,7 +2048,7 @@ CompareShardCostAsc(const void *void1, const void *void2) /* - * CompareShardCostAsc can be used to sort shard costs from high cost to low + * CompareShardCostDesc can be used to sort shard costs from high cost to low * cost. */ static int @@ -2109,8 +2109,8 @@ CompareDisallowedPlacementAsc(const void *void1, const void *void2) /* - * CompareDisallowedPlacementAsc can be used to sort disallowed placements from - * low cost to high cost. + * CompareDisallowedPlacementDesc can be used to sort disallowed placements from + * high cost to low cost. */ static int CompareDisallowedPlacementDesc(const void *a, const void *b) @@ -2619,7 +2619,7 @@ ActivePlacementsHash(List *shardPlacementList) /* - * PlacementsHashFinds returns true if there exists a shard placement with the + * PlacementsHashFind returns true if there exists a shard placement with the * given workerNode and shard id in the given placements hash, otherwise it * returns false. */ @@ -2681,7 +2681,7 @@ PlacementsHashRemove(HTAB *placementsHash, uint64 shardId, WorkerNode *workerNod /* - * ShardPlacementCompare compares two shard placements using shard id, node name, + * PlacementsHashCompare compares two shard placements using shard id, node name, * and node port number. */ static int @@ -2722,7 +2722,7 @@ PlacementsHashCompare(const void *lhsKey, const void *rhsKey, Size keySize) /* - * ShardPlacementHashCode computes the hash code for a shard placement from the + * PlacementsHashHashCode computes the hash code for a shard placement from the * placement's shard id, node name, and node port number. */ static uint32 @@ -2924,7 +2924,7 @@ EnsureNodeCapacityUDF(Oid functionOid) /* - * EnsureNodeCapacityUDF checks that the UDF matching the oid has the correct + * EnsureShardAllowedOnNodeUDF checks that the UDF matching the oid has the correct * signature to be used as a NodeCapacity function. The expected signature is: * * shard_allowed_on_node(shardid bigint, nodeid int) returns boolean diff --git a/src/backend/distributed/planner/multi_explain.c b/src/backend/distributed/planner/multi_explain.c index 7fdb6d8e7..f4c38a3bd 100644 --- a/src/backend/distributed/planner/multi_explain.c +++ b/src/backend/distributed/planner/multi_explain.c @@ -1635,7 +1635,7 @@ ExplainOneQuery(Query *query, int cursorOptions, /* - * ExplainAnalyzeWorkerPlan produces explain output into es. If es->analyze, it also executes + * ExplainWorkerPlan produces explain output into es. If es->analyze, it also executes * the given plannedStmt and sends the results to dest. It puts total time to execute in * executionDurationMillisec. * diff --git a/src/backend/distributed/planner/multi_logical_optimizer.c b/src/backend/distributed/planner/multi_logical_optimizer.c index 93a35be37..cb415f49a 100644 --- a/src/backend/distributed/planner/multi_logical_optimizer.c +++ b/src/backend/distributed/planner/multi_logical_optimizer.c @@ -2626,7 +2626,7 @@ ProcessHavingClauseForWorkerQuery(Node *originalHavingQual, /* - * PrcoessDistinctClauseForWorkerQuery gets the inputs and modifies the outputs + * ProcessDistinctClauseForWorkerQuery gets the inputs and modifies the outputs * such that worker query's DISTINCT and DISTINCT ON clauses are set accordingly. * Note the function may or may not decide to pushdown the DISTINCT and DISTINCT * on clauses based on the inputs. @@ -2789,7 +2789,7 @@ ProcessLimitOrderByForWorkerQuery(OrderByLimitReference orderByLimitReference, /* - * BuildLimitOrderByReference is a helper function that simply builds + * BuildOrderByLimitReference is a helper function that simply builds * the necessary information for processing the limit and order by. * The return value should be used in a read-only manner. */ diff --git a/src/backend/distributed/planner/multi_router_planner.c b/src/backend/distributed/planner/multi_router_planner.c index 9bf8766fd..6274eb061 100644 --- a/src/backend/distributed/planner/multi_router_planner.c +++ b/src/backend/distributed/planner/multi_router_planner.c @@ -264,7 +264,7 @@ CreateModifyPlan(Query *originalQuery, Query *query, /* - * CreateSingleTaskRouterPlan creates a physical plan for given SELECT query. + * CreateSingleTaskRouterSelectPlan creates a physical plan for given SELECT query. * The returned plan is a router task that returns query results from a single worker. * If not router plannable, the returned plan's planningError describes the problem. */ @@ -1878,7 +1878,7 @@ RouterJob(Query *originalQuery, PlannerRestrictionContext *plannerRestrictionCon /* - * SingleShardRouterTaskList is a wrapper around other corresponding task + * GenerateSingleShardRouterTaskList is a wrapper around other corresponding task * list generation functions specific to single shard selects and modifications. * * The function updates the input job's taskList in-place. diff --git a/src/backend/distributed/planner/query_pushdown_planning.c b/src/backend/distributed/planner/query_pushdown_planning.c index e8452a357..f78cec339 100644 --- a/src/backend/distributed/planner/query_pushdown_planning.c +++ b/src/backend/distributed/planner/query_pushdown_planning.c @@ -77,7 +77,7 @@ static DeferredErrorMessage * DeferredErrorIfUnsupportedRecurringTuplesJoin( PlannerRestrictionContext *plannerRestrictionContext); static DeferredErrorMessage * DeferErrorIfUnsupportedTableCombination(Query *queryTree); static DeferredErrorMessage * DeferErrorIfSubqueryRequiresMerge(Query *subqueryTree); -static bool ExtractSetOperationStatmentWalker(Node *node, List **setOperationList); +static bool ExtractSetOperationStatementWalker(Node *node, List **setOperationList); static RecurringTuplesType FetchFirstRecurType(PlannerInfo *plannerInfo, Relids relids); static bool ContainsRecurringRTE(RangeTblEntry *rangeTableEntry, @@ -1254,8 +1254,8 @@ DeferErrorIfUnsupportedUnionQuery(Query *subqueryTree) ListCell *setOperationStatmentCell = NULL; RecurringTuplesType recurType = RECURRING_TUPLES_INVALID; - ExtractSetOperationStatmentWalker((Node *) subqueryTree->setOperations, - &setOperationStatementList); + ExtractSetOperationStatementWalker((Node *) subqueryTree->setOperations, + &setOperationStatementList); foreach(setOperationStatmentCell, setOperationStatementList) { SetOperationStmt *setOperation = @@ -1343,7 +1343,7 @@ DeferErrorIfUnsupportedUnionQuery(Query *subqueryTree) * and finds all set operations in the tree. */ static bool -ExtractSetOperationStatmentWalker(Node *node, List **setOperationList) +ExtractSetOperationStatementWalker(Node *node, List **setOperationList) { if (node == NULL) { @@ -1358,7 +1358,7 @@ ExtractSetOperationStatmentWalker(Node *node, List **setOperationList) } bool walkerResult = expression_tree_walker(node, - ExtractSetOperationStatmentWalker, + ExtractSetOperationStatementWalker, setOperationList); return walkerResult; diff --git a/src/backend/distributed/planner/recursive_planning.c b/src/backend/distributed/planner/recursive_planning.c index b2a8140da..9138b1b80 100644 --- a/src/backend/distributed/planner/recursive_planning.c +++ b/src/backend/distributed/planner/recursive_planning.c @@ -564,8 +564,8 @@ RecursivelyPlanNonColocatedJoinWalker(Node *joinNode, /* - * RecursivelyPlanNonColocatedJoinWalker gets a query and walks over its sublinks - * to find subqueries that live in WHERE clause. + * RecursivelyPlanNonColocatedSubqueriesInWhere gets a query and walks over its + * sublinks to find subqueries that live in WHERE clause. * * When a subquery found, it's checked whether the subquery is colocated with the * anchor subquery specified in the nonColocatedJoinContext. If not, @@ -1122,7 +1122,7 @@ IsRelationLocalTableOrMatView(Oid relationId) /* - * RecursivelyPlanQuery recursively plans a query, replaces it with a + * RecursivelyPlanSubquery recursively plans a query, replaces it with a * result query and returns the subplan. * * Before we recursively plan the given subquery, we should ensure diff --git a/src/backend/distributed/transaction/distributed_deadlock_detection.c b/src/backend/distributed/transaction/distributed_deadlock_detection.c index 475b56784..f9e4adca0 100644 --- a/src/backend/distributed/transaction/distributed_deadlock_detection.c +++ b/src/backend/distributed/transaction/distributed_deadlock_detection.c @@ -220,7 +220,7 @@ CheckForDistributedDeadlocks(void) /* - * CheckDeadlockForDistributedTransaction does a DFS starting with the given + * CheckDeadlockForTransactionNode does a DFS starting with the given * transaction node and checks for a cycle (i.e., the node can be reached again * while traversing the graph). * diff --git a/src/backend/distributed/transaction/remote_transaction.c b/src/backend/distributed/transaction/remote_transaction.c index 969f0d583..ba67843de 100644 --- a/src/backend/distributed/transaction/remote_transaction.c +++ b/src/backend/distributed/transaction/remote_transaction.c @@ -51,7 +51,7 @@ static void Assign2PCIdentifier(MultiConnection *connection); /* - * StartRemoteTransactionBeging initiates beginning the remote transaction in + * StartRemoteTransactionBegin initiates beginning the remote transaction in * a non-blocking manner. The function sends "BEGIN" followed by * assign_distributed_transaction_id() to assign the distributed transaction * id on the remote node. diff --git a/src/backend/distributed/utils/colocation_utils.c b/src/backend/distributed/utils/colocation_utils.c index fa698f88a..51d8f9fbd 100644 --- a/src/backend/distributed/utils/colocation_utils.c +++ b/src/backend/distributed/utils/colocation_utils.c @@ -693,7 +693,7 @@ CheckDistributionColumnType(Oid sourceRelationId, Oid targetRelationId) /* - * GetColumnTypeEquality checks if distribution column types and collations + * EnsureColumnTypeEquality checks if distribution column types and collations * of the given columns are same. The function sets the boolean pointers. */ void diff --git a/src/backend/distributed/utils/resource_lock.c b/src/backend/distributed/utils/resource_lock.c index f3c6b449f..885ce39ac 100644 --- a/src/backend/distributed/utils/resource_lock.c +++ b/src/backend/distributed/utils/resource_lock.c @@ -649,7 +649,7 @@ LockShardListMetadata(List *shardIntervalList, LOCKMODE lockMode) /* - * LockPlacementListMetadata takes locks on the metadata of all shards in + * LockShardsInPlacementListMetadata takes locks on the metadata of all shards in * shardPlacementList to prevent concurrent placement changes. */ void