diff --git a/src/backend/distributed/commands/function.c b/src/backend/distributed/commands/function.c index 0c363df34..9bffb1bac 100644 --- a/src/backend/distributed/commands/function.c +++ b/src/backend/distributed/commands/function.c @@ -123,8 +123,8 @@ create_distributed_function(PG_FUNCTION_ARGS) if (PG_ARGISNULL(1)) { /* - * Using the default value, so distribute the function but do not set - * the distribution argument. + * Using the default value, so distribute the function but do not set + * the distribution argument. */ distributionArgumentName = NULL; } diff --git a/src/backend/distributed/commands/transmit.c b/src/backend/distributed/commands/transmit.c index 9b47115db..d925d847c 100644 --- a/src/backend/distributed/commands/transmit.c +++ b/src/backend/distributed/commands/transmit.c @@ -140,8 +140,8 @@ FileOpenForTransmit(const char *filename, int fileFlags, int fileMode) { struct stat fileStat; - int fileStated = stat(filename, &fileStat); - if (fileStated >= 0) + int statOK = stat(filename, &fileStat); + if (statOK >= 0) { if (S_ISDIR(fileStat.st_mode)) { diff --git a/src/backend/distributed/commands/type.c b/src/backend/distributed/commands/type.c index e7a54dff0..fec7e68fb 100644 --- a/src/backend/distributed/commands/type.c +++ b/src/backend/distributed/commands/type.c @@ -833,7 +833,7 @@ CompositeTypeStmtObjectAddress(Node *node, bool missing_ok) /* * CreateEnumStmtObjectAddress finds the ObjectAddress for the enum type described by the - * CreateEnumStmt. If missing_ok is false this function throws an error if the type does + * CreateEnumStmt. If missing_ok is false this function throws an error if the type does * not exist. * * Never returns NULL, but the objid in the address could be invalid if missing_ok was set @@ -854,7 +854,7 @@ CreateEnumStmtObjectAddress(Node *node, bool missing_ok) /* * AlterTypeStmtObjectAddress finds the ObjectAddress for the type described by the ALTER - * TYPE statement. If missing_ok is false this function throws an error if the type does + * TYPE statement. If missing_ok is false this function throws an error if the type does * not exist. * * Never returns NULL, but the objid in the address could be invalid if missing_ok was set diff --git a/src/backend/distributed/executor/citus_custom_scan.c b/src/backend/distributed/executor/citus_custom_scan.c index 075bd8732..89f232eb6 100644 --- a/src/backend/distributed/executor/citus_custom_scan.c +++ b/src/backend/distributed/executor/citus_custom_scan.c @@ -265,7 +265,7 @@ CitusBeginScanWithCoordinatorProcessing(CustomScanState *node, EState *estate, i EState *executorState = planState->state; ResetExecutionParameters(executorState); - /* we're done, we don't want to evaluate functions for SELECT queries */ + /* we're done, we don't want to evaluate functions for SELECT queries */ return; } else if (jobQuery->commandType == CMD_SELECT && workerJob->deferredPruning) @@ -274,7 +274,7 @@ CitusBeginScanWithCoordinatorProcessing(CustomScanState *node, EState *estate, i * Evaluate parameters, because the parameters are only avaliable on the * coordinator and are required for pruning. * - * But, we don't want to evaluate functions for read-only queries on the + * But, we don't want to evaluate functions for read-only queries on the * coordinator as the volatile functions could yield different * results per shard (also per row) and could have side-effects. * diff --git a/src/backend/distributed/executor/local_executor.c b/src/backend/distributed/executor/local_executor.c index 075d87374..3156d9dac 100644 --- a/src/backend/distributed/executor/local_executor.c +++ b/src/backend/distributed/executor/local_executor.c @@ -265,7 +265,7 @@ ExtractLocalAndRemoteTasks(bool readOnly, List *taskList, List **localTaskList, if (list_length(task->taskPlacementList) == 1) { /* - * At this point, the task has a single placement (e.g,. anchor shard is + * At this point, the task has a single placement (e.g,. anchor shard is * distributed table's shard). So, it is either added to local or remote * taskList. */ diff --git a/src/backend/distributed/master/master_stage_protocol.c b/src/backend/distributed/master/master_stage_protocol.c index 819f16b15..0488608ae 100644 --- a/src/backend/distributed/master/master_stage_protocol.c +++ b/src/backend/distributed/master/master_stage_protocol.c @@ -476,7 +476,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 @@ -629,7 +629,7 @@ RelationShardListForShardCreate(ShardInterval *shardInterval) } - /* if partitioned table, make sure to record the parent table */ + /* if partitioned table, make sure to record the parent table */ if (PartitionTable(relationId)) { RelationShard *parentRelationShard = CitusMakeNode(RelationShard); diff --git a/src/backend/distributed/master/pg_worker_list.conf.sample b/src/backend/distributed/master/pg_worker_list.conf.sample deleted file mode 100644 index a9e5d740b..000000000 --- a/src/backend/distributed/master/pg_worker_list.conf.sample +++ /dev/null @@ -1,27 +0,0 @@ -# ------------------------------------------ -# Citus Database Worker Node Membership List -# ------------------------------------------ -# -# This file contains list of worker node names; these names are used both for -# initializing the worker nodes, and later for communicating with them. Records -# in this file are in the following format: -# -# HOSTNAME [PORT] [RACK] -# -# (The uppercase items must be replaced by actual values.) -# -# HOSTNAME specifies the DNS resolvable host name for the worker node. In test -# environments, localhost may be used to loopback to the current node. -# -# PORT specifies the port number to connect to at the specified host. This value -# is optional; in its absence, the port configuration value is used as the -# default. -# -# RACK specifies the host's network location for the purposes of performing rack -# aware data distribution. This value is optional; in its absence, a generic -# value is used as the default. - -# Put your actual configuration here -# ---------------------------------- -# -# HOSTNAME [PORT] [RACK] diff --git a/src/backend/distributed/planner/distributed_planner.c b/src/backend/distributed/planner/distributed_planner.c index 5af86bad3..63dc1e28c 100644 --- a/src/backend/distributed/planner/distributed_planner.c +++ b/src/backend/distributed/planner/distributed_planner.c @@ -1139,7 +1139,7 @@ RecordSubPlansUsedInPlan(DistributedPlan *plan, Query *originalQuery) * may still prefer to write locally. * * If any of the subplansInHaving is used in other parts of the query, - * we'll later merge it those subPlans and send it to remote. + * we'll later merge those subPlans and send to remote. */ UpdateUsedPlanListLocation(subplansInHaving, SUBPLAN_ACCESS_LOCAL); } diff --git a/src/backend/distributed/planner/insert_select_planner.c b/src/backend/distributed/planner/insert_select_planner.c index c0e446e3a..44b3af4bf 100644 --- a/src/backend/distributed/planner/insert_select_planner.c +++ b/src/backend/distributed/planner/insert_select_planner.c @@ -492,7 +492,7 @@ RouterModifyTaskForShardInterval(Query *originalQuery, ShardInterval *shardInter } /* - * We also need to add shard interval range to the subquery in case + * We also need to add shard interval range to the subquery in case * the partition qual not distributed all tables such as some * subqueries in WHERE clause. * diff --git a/src/backend/distributed/planner/multi_logical_optimizer.c b/src/backend/distributed/planner/multi_logical_optimizer.c index 0a3957da4..36c863b96 100644 --- a/src/backend/distributed/planner/multi_logical_optimizer.c +++ b/src/backend/distributed/planner/multi_logical_optimizer.c @@ -2432,8 +2432,7 @@ ProcessHavingClauseForWorkerQuery(Node *originalHavingQual, * We converted the having expression to a list in subquery pushdown * planner. However, this query cannot be parsed as it is in the worker. * We should convert this back to being explicit for worker query - * so that it can be parsed when it hits to the standard planner in - * worker. + * so that it can be parsed when it hits the standard planner in worker. */ if (IsA(originalHavingQual, List)) { @@ -4085,7 +4084,7 @@ FindReferencedTableColumn(Expr *columnExpression, List *parentQueryList, Query * /* * We currently don't support finding partition keys in the subqueries - * that references from outer subqueries. For example, in corrolated + * that reference outer subqueries. For example, in corrolated * subqueries in WHERE clause, we don't support use of partition keys * in the subquery that is referred from the outer query. */ diff --git a/src/backend/distributed/planner/multi_logical_planner.c b/src/backend/distributed/planner/multi_logical_planner.c index 4cf967171..c8ee68fe3 100644 --- a/src/backend/distributed/planner/multi_logical_planner.c +++ b/src/backend/distributed/planner/multi_logical_planner.c @@ -130,7 +130,7 @@ static MultiNode * ApplyCartesianProduct(MultiNode *leftNode, MultiNode *rightNo /* * MultiLogicalPlanCreate takes in both the original query and its corresponding modified * query tree yield by the standard planner. It uses helper functions to create logical - * plan and adds a root node to top of it. The original query is only used for subquery + * plan and adds a root node to top of it. The original query is only used for subquery * pushdown planning. * * We also pass queryTree and plannerRestrictionContext to the planner. They @@ -334,14 +334,14 @@ TargetListOnPartitionColumn(Query *query, List *targetEntryList) /* - * AllTargetExpressionsAreColumnReferences returns true if non of the + * AllTargetExpressionsAreColumnReferences returns true if none of the * elements in the target entry list belong to an outer query (for * example the query is a sublink and references to another query * in the from list). * - * The function also returns true if any of the target entries is not + * The function also returns true if any of the target entries is not * a column itself. This might be too restrictive, but, given that we're - * handling a very specific type of queries, that seems acceptable for now. + * handling very specific type of queries, that seems acceptable for now. */ static bool AllTargetExpressionsAreColumnReferences(List *targetEntryList) diff --git a/src/backend/distributed/planner/multi_physical_planner.c b/src/backend/distributed/planner/multi_physical_planner.c index 94bc98a17..b4ad3c893 100644 --- a/src/backend/distributed/planner/multi_physical_planner.c +++ b/src/backend/distributed/planner/multi_physical_planner.c @@ -951,7 +951,7 @@ TargetEntryList(List *expressionList) /* - * AddAnyValueAggregates wraps all vars that do not apear in the GROUP BY + * AddAnyValueAggregates wraps all vars that do not appear in the GROUP BY * clause or are inside an aggregate function in an any_value aggregate * function. This is needed for repartition joins because primary keys are not * present on intermediate tables. @@ -4871,7 +4871,7 @@ TaskListAppendUnique(List *list, Task *task) /* - * TaskListConcatUnique append to list1 each member of list2 that isn't + * TaskListConcatUnique append to list1 each member of list2 that isn't * already in list1. Whether an element is already a member of the list * is determined via TaskListMember(). */ diff --git a/src/backend/distributed/planner/query_pushdown_planning.c b/src/backend/distributed/planner/query_pushdown_planning.c index 76ee0b9f4..d6655b618 100644 --- a/src/backend/distributed/planner/query_pushdown_planning.c +++ b/src/backend/distributed/planner/query_pushdown_planning.c @@ -1544,7 +1544,7 @@ SubqueryPushdownMultiNodeTree(Query *queryTree) * SELECT * s1.a AS worker_column_0, * s2.c AS worker_column_1, - * s2.b AS as worker_column_2 + * s2.b AS worker_column_2 * FROM (some subquery) s1, (some subquery) s2 * WHERE s1.a = s2.a) worker_subquery * GROUP BY worker_column_0 @@ -1712,7 +1712,7 @@ FlattenJoinVarsMutator(Node *node, Query *queryTree) return (Node *) column; } - /* join RTE does not have and alias defined at this level, deeper look is needed */ + /* join RTE does not have an alias defined at this level, deeper look is needed */ Assert(column->varattno > 0); Node *newColumn = (Node *) list_nth(rte->joinaliasvars, column->varattno - 1); Assert(newColumn != NULL); @@ -1836,7 +1836,7 @@ UpdateColumnToMatchingTargetEntry(Var *column, Node *flattenedExpr, List *target else if (IsA(targetEntry->expr, CoalesceExpr)) { /* - * FlattenJoinVars() flattens full oter joins' columns that is + * FlattenJoinVars() flattens full outer joins' columns that is * in the USING part into COALESCE(left_col, right_col) */ diff --git a/src/backend/distributed/planner/recursive_planning.c b/src/backend/distributed/planner/recursive_planning.c index 18cd2f8f8..68c465308 100644 --- a/src/backend/distributed/planner/recursive_planning.c +++ b/src/backend/distributed/planner/recursive_planning.c @@ -896,7 +896,7 @@ ShouldRecursivelyPlanSubquery(Query *subquery, RecursivePlanningContext *context /* * Citus can pushdown this subquery, no need to recursively - * plan which is much expensive than pushdown. + * plan which is much more expensive than pushdown. */ return false; } @@ -1565,7 +1565,7 @@ BuildSubPlanResultQuery(List *targetEntryList, List *columnAliasList, char *resu /* - * BuildSubPlanResultQuery returns a query of the form: + * BuildReadIntermediateResultsArrayQuery returns a query of the form: * * SELECT * diff --git a/src/backend/distributed/planner/relation_restriction_equivalence.c b/src/backend/distributed/planner/relation_restriction_equivalence.c index 63a55692e..4c323cef8 100644 --- a/src/backend/distributed/planner/relation_restriction_equivalence.c +++ b/src/backend/distributed/planner/relation_restriction_equivalence.c @@ -231,7 +231,7 @@ ContextContainsLocalRelation(RelationRestrictionContext *restrictionContext) * select count(*) from ( * select user_id, user_id from users_table * union - * select 2, user_id from users_table) u; + * select 2, user_id from users_table) u; * * For the above query, although the second item in the target list make this query * safe to push down, the function would fail to return true. diff --git a/src/backend/distributed/transaction/distributed_deadlock_detection.c b/src/backend/distributed/transaction/distributed_deadlock_detection.c index 5a141164a..1d7e81909 100644 --- a/src/backend/distributed/transaction/distributed_deadlock_detection.c +++ b/src/backend/distributed/transaction/distributed_deadlock_detection.c @@ -226,7 +226,7 @@ CheckForDistributedDeadlocks(void) * transaction node and checks for a cycle (i.e., the node can be reached again * while traversing the graph). * - * Finding a cycle indicates a distributed deadlock and the function returns + * Finding a cycle indicates a distributed deadlock and the function returns * true on that case. Also, the deadlockPath is filled with the transaction * nodes that form the cycle. */ diff --git a/src/backend/distributed/transaction/lock_graph.c b/src/backend/distributed/transaction/lock_graph.c index 911a286c7..70d7d9fe0 100644 --- a/src/backend/distributed/transaction/lock_graph.c +++ b/src/backend/distributed/transaction/lock_graph.c @@ -450,7 +450,7 @@ BuildLocalWaitGraph(void) * In general for the purpose of distributed deadlock detection, we should * skip if the process blocked on the locks that may not be part of deadlocks. * Those locks are held for a short duration while the relation or the index - * is actually extended on the disk and released as soon as the extension is + * is actually extended on the disk and released as soon as the extension is * done, even before the execution of the command that triggered the extension * finishes. Thus, recording such waits on our lock graphs could yield detecting * wrong distributed deadlocks. diff --git a/src/backend/distributed/transaction/transaction_recovery.c b/src/backend/distributed/transaction/transaction_recovery.c index 330ac2a16..64736c9da 100644 --- a/src/backend/distributed/transaction/transaction_recovery.c +++ b/src/backend/distributed/transaction/transaction_recovery.c @@ -305,7 +305,7 @@ RecoverWorkerTransactions(WorkerNode *workerNode) * * If a transaction started and committed just after we observed the * set of prepared transactions, and just before we called - * ActiveDistributedTransactionNumbers, then we would see a recovery + * ActiveDistributedTransactionNumbers, then we would see a recovery * record without a prepared transaction in pendingTransactionSet, * but there may be prepared transactions that failed to commit. * We should not delete the records for those prepared transactions, diff --git a/src/backend/distributed/utils/citus_clauses.c b/src/backend/distributed/utils/citus_clauses.c index 861f6f90f..8f1efdda2 100644 --- a/src/backend/distributed/utils/citus_clauses.c +++ b/src/backend/distributed/utils/citus_clauses.c @@ -147,7 +147,7 @@ PartiallyEvaluateExpression(Node *expression, /* - * ShouldEvaluateFunctionWithMasterContext is a helper function which is used to + * ShouldEvaluateFunctionWithMasterContext is a helper function which is used to * decide whether the function/expression should be evaluated with the input * masterEvaluationContext. */ diff --git a/src/backend/distributed/utils/colocation_utils.c b/src/backend/distributed/utils/colocation_utils.c index 8162ddf9b..63e39eaa5 100644 --- a/src/backend/distributed/utils/colocation_utils.c +++ b/src/backend/distributed/utils/colocation_utils.c @@ -335,7 +335,7 @@ ShardsIntervalsEqual(ShardInterval *leftShardInterval, ShardInterval *rightShard char leftIntervalPartitionMethod = PartitionMethod(leftShardInterval->relationId); char rightIntervalPartitionMethod = PartitionMethod(rightShardInterval->relationId); - /* if both shards are the same, return true */ + /* if both shards are the same, return true */ if (leftShardInterval->shardId == rightShardInterval->shardId) { return true; diff --git a/src/backend/distributed/worker/worker_partition_protocol.c b/src/backend/distributed/worker/worker_partition_protocol.c index f930006c1..5ebbc245b 100644 --- a/src/backend/distributed/worker/worker_partition_protocol.c +++ b/src/backend/distributed/worker/worker_partition_protocol.c @@ -722,8 +722,8 @@ CitusRemoveDirectory(const char *filename) struct stat fileStat; int removed = 0; - int fileStated = stat(filename, &fileStat); - if (fileStated < 0) + int statOK = stat(filename, &fileStat); + if (statOK < 0) { if (errno == ENOENT) {