polish the codebase by fixing dozens of typos (#7166)

pull/7163/head
zhjwpku 2023-09-01 18:21:53 +08:00 committed by GitHub
parent 05443a77ad
commit 5034f8eba5
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
16 changed files with 38 additions and 38 deletions

View File

@ -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. * by table type flag.
*/ */
void void

View File

@ -1548,7 +1548,7 @@ CoerceColumnValue(Datum inputValue, CopyCoercionData *coercionPath)
{ {
switch (coercionPath->coercionType) switch (coercionPath->coercionType)
{ {
case 0: case COERCION_PATH_NONE:
{ {
return inputValue; /* this was a dropped column */ return inputValue; /* this was a dropped column */
} }

View File

@ -371,7 +371,7 @@ StartNodeUserDatabaseConnection(uint32 flags, const char *hostname, int32 port,
*/ */
MultiConnection *connection = MemoryContextAllocZero(ConnectionContext, MultiConnection *connection = MemoryContextAllocZero(ConnectionContext,
sizeof(MultiConnection)); sizeof(MultiConnection));
connection->initilizationState = POOL_STATE_NOT_INITIALIZED; connection->initializationState = POOL_STATE_NOT_INITIALIZED;
dlist_push_tail(entry->connections, &connection->connectionNode); dlist_push_tail(entry->connections, &connection->connectionNode);
/* these two flags are by nature cannot happen at the same time */ /* 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 * We've already incremented the counter above, so we should decrement
* when we're done with the connection. * when we're done with the connection.
*/ */
connection->initilizationState = POOL_STATE_COUNTER_INCREMENTED; connection->initializationState = POOL_STATE_COUNTER_INCREMENTED;
StartConnectionEstablishment(connection, &key); StartConnectionEstablishment(connection, &key);
@ -430,7 +430,7 @@ StartNodeUserDatabaseConnection(uint32 flags, const char *hostname, int32 port,
} }
/* fully initialized the connection, record it */ /* fully initialized the connection, record it */
connection->initilizationState = POOL_STATE_INITIALIZED; connection->initializationState = POOL_STATE_INITIALIZED;
return connection; return connection;
} }
@ -486,7 +486,7 @@ FindAvailableConnection(dlist_head *connections, uint32 flags)
continue; continue;
} }
if (connection->initilizationState != POOL_STATE_INITIALIZED) if (connection->initializationState != POOL_STATE_INITIALIZED)
{ {
/* /*
* If the connection has not been initialized, it should not be * 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 * connection establishment. The return value of this function indicates if the
* MultiConnectionPollState has been changed, which could require a change to the WaitEventSet * 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 */ /* 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); 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. * from their application name.
*/ */
return (IsCitusInternalBackend() || IsRebalancerInternalBackend()) || return (IsCitusInternalBackend() || IsRebalancerInternalBackend()) ||
connection->initilizationState != POOL_STATE_INITIALIZED || connection->initializationState != POOL_STATE_INITIALIZED ||
cachedConnectionCount >= MaxCachedConnectionsPerWorker || cachedConnectionCount >= MaxCachedConnectionsPerWorker ||
connection->forceCloseAtTransactionEnd || connection->forceCloseAtTransactionEnd ||
PQstatus(connection->pgConn) != CONNECTION_OK || PQstatus(connection->pgConn) != CONNECTION_OK ||
@ -1541,7 +1541,7 @@ RestartConnection(MultiConnection *connection)
* Not that we have to do this because ShutdownConnection() sets the * Not that we have to do this because ShutdownConnection() sets the
* state to not initialized. * state to not initialized.
*/ */
connection->initilizationState = POOL_STATE_INITIALIZED; connection->initializationState = POOL_STATE_INITIALIZED;
connection->connectionState = MULTI_CONNECTION_CONNECTING; connection->connectionState = MULTI_CONNECTION_CONNECTING;
} }

View File

@ -14,7 +14,7 @@
* (b) Reserving connections, the logic that this * (b) Reserving connections, the logic that this
* file implements. * 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 * connection, it is guaranteed to have the right to establish a connection
* to the given remote node when needed. * 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 * map by HASH_ENTER. The function throws an error if it cannot allocate
* the entry. * the entry.
*/ */

View File

@ -339,7 +339,7 @@ TryToIncrementSharedConnectionCounter(const char *hostname, int port)
LockConnectionSharedMemory(LW_EXCLUSIVE); 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 * 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 * space in the shared memory. That's why we prefer continuing the execution
* instead of throwing an error. * instead of throwing an error.
@ -440,7 +440,7 @@ IncrementSharedConnectionCounter(const char *hostname, int port)
LockConnectionSharedMemory(LW_EXCLUSIVE); 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 * memory allocation, so we could get NULL via HASH_ENTER_NULL. That's why we prefer
* continuing the execution instead of throwing an error. * continuing the execution instead of throwing an error.
*/ */
@ -694,7 +694,7 @@ SharedConnectionStatsShmemInit(void)
ConditionVariableInit(&ConnectionStatsSharedState->waitersConditionVariable); ConditionVariableInit(&ConnectionStatsSharedState->waitersConditionVariable);
} }
/* allocate hash table */ /* allocate hash table */
SharedConnStatsHash = SharedConnStatsHash =
ShmemInitHash("Shared Conn. Stats Hash", MaxWorkerNodesTracked, ShmemInitHash("Shared Conn. Stats Hash", MaxWorkerNodesTracked,
MaxWorkerNodesTracked, &info, hashFlags); MaxWorkerNodesTracked, &info, hashFlags);

View File

@ -2045,7 +2045,7 @@ ProcessSessionsWithFailedWaitEventSetOperations(DistributedExecution *execution)
/* /*
* HasIncompleteConnectionEstablishment returns true if any of the connections * 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 static bool
HasIncompleteConnectionEstablishment(DistributedExecution *execution) HasIncompleteConnectionEstablishment(DistributedExecution *execution)
@ -2658,7 +2658,7 @@ OpenNewConnections(WorkerPool *workerPool, int newConnectionCount,
{ {
/* /*
* The worker pool has just started to establish connections. We need to * 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 * because for non-optional connections, we have some logic to wait
* until a connection is allowed to be established. * until a connection is allowed to be established.
*/ */

View File

@ -279,7 +279,7 @@ RemoteFileDestReceiverStartup(DestReceiver *dest, int operation,
/* /*
* PrepareIntermediateResultBroadcast gets a RemoteFileDestReceiver and does * 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). * and creating the local file, which is necessary (it might be both).
*/ */
static void static void

View File

@ -277,7 +277,7 @@ master_get_new_placementid(PG_FUNCTION_ARGS)
/* /*
* GetNextPlacementId allocates and returns a unique placementId for * GetNextPlacementId allocates and returns a unique placementId for
* the placement to be created. This allocation occurs both in shared memory * 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. * collisions.
* *
* NB: This can be called by any user; for now we have decided that that's * NB: This can be called by any user; for now we have decided that that's

View File

@ -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, * 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. * 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 * InsertCleanupRecordInSubtransaction inserts a new pg_dist_cleanup entry in a
* in a separate transaction to ensure the record persists after rollback. We should * separate transaction to ensure the record persists after rollback. We should
* delete these records if the operation completes successfully. * delete these records if the operation completes successfully.
* *
* For failure scenarios, use a subtransaction (direct insert via localhost). * 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 static void
DeleteCleanupRecordByRecordIdOutsideTransaction(uint64 recordId) DeleteCleanupRecordByRecordIdOutsideTransaction(uint64 recordId)
@ -1106,7 +1106,7 @@ TupleToCleanupRecord(HeapTuple heapTuple, TupleDesc tupleDescriptor)
/* /*
* CleanupRecordExists returns whether a cleanup record with the given * 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 static bool
CleanupRecordExists(uint64 recordId) 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 static void
DeleteCleanupRecordByRecordId(uint64 recordId) DeleteCleanupRecordByRecordId(uint64 recordId)

View File

@ -882,7 +882,7 @@ ExecutePlacementUpdates(List *placementUpdateList, Oid shardReplicationModeOid,
* ones) and the relation id of the target table. The dynamic shared memory * ones) and the relation id of the target table. The dynamic shared memory
* portion consists of a RebalanceMonitorHeader and multiple * portion consists of a RebalanceMonitorHeader and multiple
* PlacementUpdateEventProgress, one for each planned shard placement move. The * 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 * 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 * 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 * officially allowed as the command type, we describe ourselves as a VACUUM command and

View File

@ -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, * When all consumers (Example : ISOLATE_TENANT_TO_NEW_SHARD) directly call 'SplitShard' API,
* this method will be merged with 'ErrorIfCannotSplitShard' above. * this method will be merged with 'ErrorIfCannotSplitShard' above.
*/ */
@ -425,7 +425,7 @@ GetWorkerNodesFromWorkerIds(List *nodeIdsForPlacementList)
* 'shardInterval' : Source shard interval to be split. * 'shardInterval' : Source shard interval to be split.
* 'shardSplitPointsList' : Split Points list for the source 'shardInterval'. * 'shardSplitPointsList' : Split Points list for the source 'shardInterval'.
* 'nodeIdsForPlacementList' : Placement list corresponding to split children. * '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 * If not specified, the distribution column is read
* from the metadata. * from the metadata.
* 'colocatedShardIntervalList' : Shard interval list for colocation group. (only used for * 'colocatedShardIntervalList' : Shard interval list for colocation group. (only used for

View File

@ -411,7 +411,7 @@ InsertShardPlacementRows(Oid relationId, int64 shardId, List *workerNodeList,
/* /*
* CreateShardsOnWorkers creates shards on worker nodes given the shard placements * 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. * that it can adopt the number of connections required to create the shards.
*/ */
void void

View File

@ -52,7 +52,7 @@ static char * TraceWorkerSplitCopyUdf(char *sourceShardToCopySchemaName,
* worker_split_copy(source_shard_id bigint, splitCopyInfo pg_catalog.split_copy_info[]) * worker_split_copy(source_shard_id bigint, splitCopyInfo pg_catalog.split_copy_info[])
* UDF to split copy shard to list of destination shards. * UDF to split copy shard to list of destination shards.
* 'source_shard_id' : Source ShardId to split copy. * '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 Datum
worker_split_copy(PG_FUNCTION_ARGS) worker_split_copy(PG_FUNCTION_ARGS)
@ -139,7 +139,7 @@ TraceWorkerSplitCopyUdf(char *sourceShardToCopySchemaName,
appendStringInfo(splitCopyTrace, "performing copy from shard %s to [", appendStringInfo(splitCopyTrace, "performing copy from shard %s to [",
sourceShardToCopyQualifiedName); sourceShardToCopyQualifiedName);
/* split copy always has atleast two destinations */ /* split copy always has at least two destinations */
int index = 1; int index = 1;
int splitWayCount = list_length(splitCopyInfoList); int splitWayCount = list_length(splitCopyInfoList);
SplitCopyInfo *splitCopyInfo = NULL; SplitCopyInfo *splitCopyInfo = NULL;

View File

@ -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. * to the hash map. The key is nodeId on which the new shard is to be placed.
*/ */
static void static void

View File

@ -124,7 +124,7 @@ SendCommandToWorkersWithMetadata(const char *command)
* owner to ensure write access to the Citus metadata tables. * owner to ensure write access to the Citus metadata tables.
* *
* Since we prevent to open superuser connections for metadata tables, it is * 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. * tuples for dependent objects.
*/ */
void void

View File

@ -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. * of the underlying pg_conn struct.
*/ */
typedef enum MultiConnectionState 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 * of MultiConnection struct, not specifically the underlying
* pg_conn. The state is useful to determine the action during * pg_conn. The state is useful to determine the action during
* clean-up of connections. * clean-up of connections.
@ -207,7 +207,7 @@ typedef struct MultiConnection
instr_time connectionEstablishmentStart; instr_time connectionEstablishmentStart;
instr_time connectionEstablishmentEnd; instr_time connectionEstablishmentEnd;
/* membership in list of list of connections in ConnectionHashEntry */ /* membership in list of connections in ConnectionHashEntry */
dlist_node connectionNode; dlist_node connectionNode;
/* information about the associated remote transaction */ /* information about the associated remote transaction */
@ -229,7 +229,7 @@ typedef struct MultiConnection
/* replication option */ /* replication option */
bool requiresReplication; bool requiresReplication;
MultiConnectionStructInitializationState initilizationState; MultiConnectionStructInitializationState initializationState;
} MultiConnection; } MultiConnection;