diff --git a/src/backend/distributed/commands/foreign_constraint.c b/src/backend/distributed/commands/foreign_constraint.c index dd46d377ae5..0d51563532d 100644 --- a/src/backend/distributed/commands/foreign_constraint.c +++ b/src/backend/distributed/commands/foreign_constraint.c @@ -132,7 +132,7 @@ EnsureNoFKeyFromTableType(Oid relationId, int tableTypeFlag) /* - * EnsureNoFKeyToTableType ensures that given relation is not referencing by any table specified + * EnsureNoFKeyToTableType ensures that given relation is not referencing any table specified * by table type flag. */ void diff --git a/src/backend/distributed/commands/multi_copy.c b/src/backend/distributed/commands/multi_copy.c index ec86f3f61ef..a684d06cc3f 100644 --- a/src/backend/distributed/commands/multi_copy.c +++ b/src/backend/distributed/commands/multi_copy.c @@ -1548,7 +1548,7 @@ CoerceColumnValue(Datum inputValue, CopyCoercionData *coercionPath) { switch (coercionPath->coercionType) { - case 0: + case COERCION_PATH_NONE: { return inputValue; /* this was a dropped column */ } diff --git a/src/backend/distributed/connection/connection_management.c b/src/backend/distributed/connection/connection_management.c index 46e757bfe9e..9439b38c5fb 100644 --- a/src/backend/distributed/connection/connection_management.c +++ b/src/backend/distributed/connection/connection_management.c @@ -371,7 +371,7 @@ StartNodeUserDatabaseConnection(uint32 flags, const char *hostname, int32 port, */ MultiConnection *connection = MemoryContextAllocZero(ConnectionContext, sizeof(MultiConnection)); - connection->initilizationState = POOL_STATE_NOT_INITIALIZED; + connection->initializationState = POOL_STATE_NOT_INITIALIZED; dlist_push_tail(entry->connections, &connection->connectionNode); /* these two flags are by nature cannot happen at the same time */ @@ -417,7 +417,7 @@ StartNodeUserDatabaseConnection(uint32 flags, const char *hostname, int32 port, * We've already incremented the counter above, so we should decrement * when we're done with the connection. */ - connection->initilizationState = POOL_STATE_COUNTER_INCREMENTED; + connection->initializationState = POOL_STATE_COUNTER_INCREMENTED; StartConnectionEstablishment(connection, &key); @@ -430,7 +430,7 @@ StartNodeUserDatabaseConnection(uint32 flags, const char *hostname, int32 port, } /* fully initialized the connection, record it */ - connection->initilizationState = POOL_STATE_INITIALIZED; + connection->initializationState = POOL_STATE_INITIALIZED; return connection; } @@ -486,7 +486,7 @@ FindAvailableConnection(dlist_head *connections, uint32 flags) continue; } - if (connection->initilizationState != POOL_STATE_INITIALIZED) + if (connection->initializationState != POOL_STATE_INITIALIZED) { /* * If the connection has not been initialized, it should not be @@ -780,7 +780,7 @@ ShutdownConnection(MultiConnection *connection) /* - * MultiConnectionStatePoll executes a PQconnectPoll on the connection to progres the + * MultiConnectionStatePoll executes a PQconnectPoll on the connection to progress the * connection establishment. The return value of this function indicates if the * MultiConnectionPollState has been changed, which could require a change to the WaitEventSet */ @@ -1182,10 +1182,10 @@ CitusPQFinish(MultiConnection *connection) } /* behave idempotently, there is no gurantee that CitusPQFinish() is called once */ - if (connection->initilizationState >= POOL_STATE_COUNTER_INCREMENTED) + if (connection->initializationState >= POOL_STATE_COUNTER_INCREMENTED) { DecrementSharedConnectionCounter(connection->hostname, connection->port); - connection->initilizationState = POOL_STATE_NOT_INITIALIZED; + connection->initializationState = POOL_STATE_NOT_INITIALIZED; } } @@ -1482,7 +1482,7 @@ ShouldShutdownConnection(MultiConnection *connection, const int cachedConnection * from their application name. */ return (IsCitusInternalBackend() || IsRebalancerInternalBackend()) || - connection->initilizationState != POOL_STATE_INITIALIZED || + connection->initializationState != POOL_STATE_INITIALIZED || cachedConnectionCount >= MaxCachedConnectionsPerWorker || connection->forceCloseAtTransactionEnd || PQstatus(connection->pgConn) != CONNECTION_OK || @@ -1541,7 +1541,7 @@ RestartConnection(MultiConnection *connection) * Not that we have to do this because ShutdownConnection() sets the * state to not initialized. */ - connection->initilizationState = POOL_STATE_INITIALIZED; + connection->initializationState = POOL_STATE_INITIALIZED; connection->connectionState = MULTI_CONNECTION_CONNECTING; } diff --git a/src/backend/distributed/connection/locally_reserved_shared_connections.c b/src/backend/distributed/connection/locally_reserved_shared_connections.c index 9f703dc6549..e3f7cb628eb 100644 --- a/src/backend/distributed/connection/locally_reserved_shared_connections.c +++ b/src/backend/distributed/connection/locally_reserved_shared_connections.c @@ -14,7 +14,7 @@ * (b) Reserving connections, the logic that this * file implements. * - * Finally, as the name already implies, once a node has reserved a shared + * Finally, as the name already implies, once a node has reserved a shared * connection, it is guaranteed to have the right to establish a connection * to the given remote node when needed. * @@ -505,7 +505,7 @@ IsReservationPossible(void) /* - * AllocateReservedConectionEntry allocates the required entry in the hash + * AllocateOrGetReservedConnectionEntry allocates the required entry in the hash * map by HASH_ENTER. The function throws an error if it cannot allocate * the entry. */ diff --git a/src/backend/distributed/connection/shared_connection_stats.c b/src/backend/distributed/connection/shared_connection_stats.c index 82ad2675673..fcd396fe4d4 100644 --- a/src/backend/distributed/connection/shared_connection_stats.c +++ b/src/backend/distributed/connection/shared_connection_stats.c @@ -339,7 +339,7 @@ TryToIncrementSharedConnectionCounter(const char *hostname, int port) LockConnectionSharedMemory(LW_EXCLUSIVE); /* - * As the hash map is allocated in shared memory, it doesn't rely on palloc for + * As the hash map is allocated in shared memory, it doesn't rely on palloc for * memory allocation, so we could get NULL via HASH_ENTER_NULL when there is no * space in the shared memory. That's why we prefer continuing the execution * instead of throwing an error. @@ -440,7 +440,7 @@ IncrementSharedConnectionCounter(const char *hostname, int port) LockConnectionSharedMemory(LW_EXCLUSIVE); /* - * As the hash map is allocated in shared memory, it doesn't rely on palloc for + * As the hash map is allocated in shared memory, it doesn't rely on palloc for * memory allocation, so we could get NULL via HASH_ENTER_NULL. That's why we prefer * continuing the execution instead of throwing an error. */ @@ -694,7 +694,7 @@ SharedConnectionStatsShmemInit(void) ConditionVariableInit(&ConnectionStatsSharedState->waitersConditionVariable); } - /* allocate hash table */ + /* allocate hash table */ SharedConnStatsHash = ShmemInitHash("Shared Conn. Stats Hash", MaxWorkerNodesTracked, MaxWorkerNodesTracked, &info, hashFlags); diff --git a/src/backend/distributed/executor/adaptive_executor.c b/src/backend/distributed/executor/adaptive_executor.c index 18dbed49d23..61a52e7c482 100644 --- a/src/backend/distributed/executor/adaptive_executor.c +++ b/src/backend/distributed/executor/adaptive_executor.c @@ -2045,7 +2045,7 @@ ProcessSessionsWithFailedWaitEventSetOperations(DistributedExecution *execution) /* * HasIncompleteConnectionEstablishment returns true if any of the connections - * that has been initiated by the executor is in initilization stage. + * that has been initiated by the executor is in initialization stage. */ static bool HasIncompleteConnectionEstablishment(DistributedExecution *execution) @@ -2658,7 +2658,7 @@ OpenNewConnections(WorkerPool *workerPool, int newConnectionCount, { /* * The worker pool has just started to establish connections. We need to - * defer this initilization after StartNodeUserDatabaseConnection() + * defer this initialization after StartNodeUserDatabaseConnection() * because for non-optional connections, we have some logic to wait * until a connection is allowed to be established. */ diff --git a/src/backend/distributed/executor/intermediate_results.c b/src/backend/distributed/executor/intermediate_results.c index 8179f00823c..d17e6521724 100644 --- a/src/backend/distributed/executor/intermediate_results.c +++ b/src/backend/distributed/executor/intermediate_results.c @@ -279,7 +279,7 @@ RemoteFileDestReceiverStartup(DestReceiver *dest, int operation, /* * PrepareIntermediateResultBroadcast gets a RemoteFileDestReceiver and does - * the necessary initilizations including initiating the remote connections + * the necessary initializations including initiating the remote connections * and creating the local file, which is necessary (it might be both). */ static void diff --git a/src/backend/distributed/operations/node_protocol.c b/src/backend/distributed/operations/node_protocol.c index dca9906a669..14287992e97 100644 --- a/src/backend/distributed/operations/node_protocol.c +++ b/src/backend/distributed/operations/node_protocol.c @@ -277,7 +277,7 @@ master_get_new_placementid(PG_FUNCTION_ARGS) /* * GetNextPlacementId allocates and returns a unique placementId for * the placement to be created. This allocation occurs both in shared memory - * and in write ahead logs; writing to logs avoids the risk of having shardId + * and in write ahead logs; writing to logs avoids the risk of having placementId * collisions. * * NB: This can be called by any user; for now we have decided that that's diff --git a/src/backend/distributed/operations/shard_cleaner.c b/src/backend/distributed/operations/shard_cleaner.c index e7ddaa97c9f..42877bf1023 100644 --- a/src/backend/distributed/operations/shard_cleaner.c +++ b/src/backend/distributed/operations/shard_cleaner.c @@ -450,7 +450,7 @@ CompareCleanupRecordsByObjectType(const void *leftElement, const void *rightElem /* - * InsertCleanupRecordInCurrentTransaction inserts a new pg_dist_cleanup_record entry + * InsertCleanupRecordInCurrentTransaction inserts a new pg_dist_cleanup entry * as part of the current transaction. This is primarily useful for deferred drop scenarios, * since these records would roll back in case of operation failure. */ @@ -497,8 +497,8 @@ InsertCleanupRecordInCurrentTransaction(CleanupObject objectType, /* - * InsertCleanupRecordInSubtransaction inserts a new pg_dist_cleanup_record entry - * in a separate transaction to ensure the record persists after rollback. We should + * InsertCleanupRecordInSubtransaction inserts a new pg_dist_cleanup entry in a + * separate transaction to ensure the record persists after rollback. We should * delete these records if the operation completes successfully. * * For failure scenarios, use a subtransaction (direct insert via localhost). @@ -541,7 +541,7 @@ InsertCleanupRecordInSubtransaction(CleanupObject objectType, /* - * DeleteCleanupRecordByRecordId deletes a cleanup record by record id. + * DeleteCleanupRecordByRecordIdOutsideTransaction deletes a cleanup record by record id. */ static void DeleteCleanupRecordByRecordIdOutsideTransaction(uint64 recordId) @@ -1106,7 +1106,7 @@ TupleToCleanupRecord(HeapTuple heapTuple, TupleDesc tupleDescriptor) /* * CleanupRecordExists returns whether a cleanup record with the given - * record ID exists in pg_dist_cleanup_record. + * record ID exists in pg_dist_cleanup. */ static bool CleanupRecordExists(uint64 recordId) @@ -1139,7 +1139,7 @@ CleanupRecordExists(uint64 recordId) /* - * DeleteCleanupRecordByRecordId deletes a single pg_dist_cleanup_record entry. + * DeleteCleanupRecordByRecordId deletes a single pg_dist_cleanup entry. */ static void DeleteCleanupRecordByRecordId(uint64 recordId) diff --git a/src/backend/distributed/operations/shard_rebalancer.c b/src/backend/distributed/operations/shard_rebalancer.c index d26c7baf237..e3ee4aa4d0b 100644 --- a/src/backend/distributed/operations/shard_rebalancer.c +++ b/src/backend/distributed/operations/shard_rebalancer.c @@ -882,7 +882,7 @@ ExecutePlacementUpdates(List *placementUpdateList, Oid shardReplicationModeOid, * ones) and the relation id of the target table. The dynamic shared memory * portion consists of a RebalanceMonitorHeader and multiple * PlacementUpdateEventProgress, one for each planned shard placement move. The - * dsm_handle of the created segment is savedin the progress of the current backend so + * dsm_handle of the created segment is saved in the progress of the current backend so * that it can be read by external agents such as get_rebalance_progress function by * calling pg_stat_get_progress_info UDF. Since currently only VACUUM commands are * officially allowed as the command type, we describe ourselves as a VACUUM command and diff --git a/src/backend/distributed/operations/shard_split.c b/src/backend/distributed/operations/shard_split.c index e2578c04c36..0772b03b488 100644 --- a/src/backend/distributed/operations/shard_split.c +++ b/src/backend/distributed/operations/shard_split.c @@ -216,7 +216,7 @@ ErrorIfCannotSplitShard(SplitOperation splitOperation, ShardInterval *sourceShar /* - * Exteded checks before we decide to split the shard. + * Extended checks before we decide to split the shard. * When all consumers (Example : ISOLATE_TENANT_TO_NEW_SHARD) directly call 'SplitShard' API, * this method will be merged with 'ErrorIfCannotSplitShard' above. */ @@ -425,7 +425,7 @@ GetWorkerNodesFromWorkerIds(List *nodeIdsForPlacementList) * 'shardInterval' : Source shard interval to be split. * 'shardSplitPointsList' : Split Points list for the source 'shardInterval'. * 'nodeIdsForPlacementList' : Placement list corresponding to split children. - * 'distributionColumnList' : Maps relation IDs to distribution columns. + * 'distributionColumnOverrides': Maps relation IDs to distribution columns. * If not specified, the distribution column is read * from the metadata. * 'colocatedShardIntervalList' : Shard interval list for colocation group. (only used for diff --git a/src/backend/distributed/operations/stage_protocol.c b/src/backend/distributed/operations/stage_protocol.c index 9411835a545..402f949b207 100644 --- a/src/backend/distributed/operations/stage_protocol.c +++ b/src/backend/distributed/operations/stage_protocol.c @@ -411,7 +411,7 @@ InsertShardPlacementRows(Oid relationId, int64 shardId, List *workerNodeList, /* * CreateShardsOnWorkers creates shards on worker nodes given the shard placements - * as a parameter The function creates the shards via the executor. This means + * as a parameter. The function creates the shards via the executor. This means * that it can adopt the number of connections required to create the shards. */ void diff --git a/src/backend/distributed/operations/worker_split_copy_udf.c b/src/backend/distributed/operations/worker_split_copy_udf.c index c154ac040d4..18fdbfc4a12 100644 --- a/src/backend/distributed/operations/worker_split_copy_udf.c +++ b/src/backend/distributed/operations/worker_split_copy_udf.c @@ -52,7 +52,7 @@ static char * TraceWorkerSplitCopyUdf(char *sourceShardToCopySchemaName, * worker_split_copy(source_shard_id bigint, splitCopyInfo pg_catalog.split_copy_info[]) * UDF to split copy shard to list of destination shards. * 'source_shard_id' : Source ShardId to split copy. - * 'splitCopyInfos' : Array of Split Copy Info (destination_shard's id, min/max ranges and node_id) + * 'splitCopyInfos' : Array of Split Copy Info (destination_shard's id, min/max ranges and node_id) */ Datum worker_split_copy(PG_FUNCTION_ARGS) @@ -139,7 +139,7 @@ TraceWorkerSplitCopyUdf(char *sourceShardToCopySchemaName, appendStringInfo(splitCopyTrace, "performing copy from shard %s to [", sourceShardToCopyQualifiedName); - /* split copy always has atleast two destinations */ + /* split copy always has at least two destinations */ int index = 1; int splitWayCount = list_length(splitCopyInfoList); SplitCopyInfo *splitCopyInfo = NULL; diff --git a/src/backend/distributed/operations/worker_split_shard_replication_setup_udf.c b/src/backend/distributed/operations/worker_split_shard_replication_setup_udf.c index 85c2328c776..4d116dfa13a 100644 --- a/src/backend/distributed/operations/worker_split_shard_replication_setup_udf.c +++ b/src/backend/distributed/operations/worker_split_shard_replication_setup_udf.c @@ -243,7 +243,7 @@ CreateShardSplitInfo(uint64 sourceShardIdToSplit, /* - * AddShardSplitInfoEntryForNodeInMap function add's ShardSplitInfo entry + * AddShardSplitInfoEntryForNodeInMap function adds ShardSplitInfo entry * to the hash map. The key is nodeId on which the new shard is to be placed. */ static void diff --git a/src/backend/distributed/transaction/worker_transaction.c b/src/backend/distributed/transaction/worker_transaction.c index b4a4976473c..a9a855fb1aa 100644 --- a/src/backend/distributed/transaction/worker_transaction.c +++ b/src/backend/distributed/transaction/worker_transaction.c @@ -124,7 +124,7 @@ SendCommandToWorkersWithMetadata(const char *command) * owner to ensure write access to the Citus metadata tables. * * Since we prevent to open superuser connections for metadata tables, it is - * discourated to use it. Consider using it only for propagating pg_dist_object + * discouraged to use it. Consider using it only for propagating pg_dist_object * tuples for dependent objects. */ void diff --git a/src/include/distributed/connection_management.h b/src/include/distributed/connection_management.h index f0812412323..158f0b1ce3d 100644 --- a/src/include/distributed/connection_management.h +++ b/src/include/distributed/connection_management.h @@ -134,7 +134,7 @@ enum MultiConnectionMode /* - * This state is used for keeping track of the initilization + * This state is used for keeping track of the initialization * of the underlying pg_conn struct. */ typedef enum MultiConnectionState @@ -149,7 +149,7 @@ typedef enum MultiConnectionState /* - * This state is used for keeping track of the initilization + * This state is used for keeping track of the initialization * of MultiConnection struct, not specifically the underlying * pg_conn. The state is useful to determine the action during * clean-up of connections. @@ -207,7 +207,7 @@ typedef struct MultiConnection instr_time connectionEstablishmentStart; instr_time connectionEstablishmentEnd; - /* membership in list of list of connections in ConnectionHashEntry */ + /* membership in list of connections in ConnectionHashEntry */ dlist_node connectionNode; /* information about the associated remote transaction */ @@ -229,7 +229,7 @@ typedef struct MultiConnection /* replication option */ bool requiresReplication; - MultiConnectionStructInitializationState initilizationState; + MultiConnectionStructInitializationState initializationState; } MultiConnection;