From 20abc4d2b5e06efc006a15220a05d894b992e968 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Philip=20Dub=C3=A9?= Date: Thu, 27 Feb 2020 15:54:49 +0000 Subject: [PATCH] Replace foreach with foreach_ptr/foreach_oid (#3544) --- src/backend/distributed/commands/collation.c | 5 +- .../commands/create_distributed_table.c | 7 +- .../distributed/commands/dependencies.c | 40 ++---- src/backend/distributed/commands/extension.c | 53 ++++---- src/backend/distributed/commands/function.c | 33 ++--- src/backend/distributed/commands/index.c | 36 +++--- src/backend/distributed/commands/role.c | 7 +- src/backend/distributed/commands/schema.c | 17 ++- src/backend/distributed/commands/sequence.c | 7 +- src/backend/distributed/commands/table.c | 78 +++++------- src/backend/distributed/commands/transmit.c | 25 ++-- src/backend/distributed/commands/truncate.c | 28 ++--- src/backend/distributed/commands/type.c | 20 ++- .../distributed/commands/utility_hook.c | 23 ++-- src/backend/distributed/commands/vacuum.c | 40 +++--- .../connection/connection_management.c | 31 ++--- .../connection/placement_connection.c | 15 +-- .../distributed/connection/remote_commands.c | 16 +-- .../distributed/executor/citus_custom_scan.c | 7 +- .../directed_acyclic_graph_execution.c | 35 +++--- .../executor/distributed_execution_locks.c | 31 ++--- .../distributed_intermediate_results.c | 52 ++++---- .../executor/insert_select_executor.c | 29 ++--- .../executor/intermediate_results.c | 28 ++--- .../distributed/executor/local_executor.c | 33 ++--- .../distributed/executor/multi_executor.c | 17 +-- .../executor/multi_server_executor.c | 9 +- .../executor/multi_task_tracker_executor.c | 104 ++++++--------- .../distributed/executor/placement_access.c | 7 +- .../executor/repartition_join_execution.c | 38 +++--- .../distributed/executor/subplan_execution.c | 6 +- .../master/citus_create_restore_point.c | 17 +-- .../distributed/master/master_create_shards.c | 5 +- .../master/master_delete_protocol.c | 18 +-- .../master/master_metadata_utility.c | 33 ++--- .../distributed/master/master_repair_shards.c | 58 ++++----- .../master/master_stage_protocol.c | 42 +++---- .../distributed/master/master_truncate.c | 7 +- .../distributed/master/worker_node_manager.c | 18 ++- src/backend/distributed/metadata/dependency.c | 14 +-- .../distributed/metadata/metadata_cache.c | 24 ++-- .../distributed/metadata/metadata_sync.c | 107 ++++++---------- .../distributed/metadata/node_metadata.c | 4 +- .../distributed/progress/multi_progress.c | 8 +- .../distributed/relay/relay_event_utility.c | 23 ++-- .../distributed/test/colocation_utils.c | 5 +- src/backend/distributed/test/create_shards.c | 5 +- .../distributed/test/deparse_shard_query.c | 12 +- .../test/distributed_intermediate_results.c | 13 +- .../distributed/test/distribution_metadata.c | 23 ++-- src/backend/distributed/test/metadata_sync.c | 11 +- .../distributed/test/partitioning_utils.c | 7 +- src/backend/distributed/test/progress_utils.c | 6 +- .../distributed/test/prune_shard_list.c | 7 +- .../distributed/transaction/backend_data.c | 16 +-- .../transaction/citus_dist_stat_activity.c | 32 ++--- .../distributed_deadlock_detection.c | 21 ++-- .../distributed/transaction/lock_graph.c | 16 ++- .../transaction/relation_access_tracking.c | 34 ++--- .../transaction/remote_transaction.c | 26 ++-- .../transaction/transaction_management.c | 11 +- .../transaction/transaction_recovery.c | 7 +- .../transaction/worker_transaction.c | 119 +++++++----------- .../distributed/utils/colocation_utils.c | 11 +- .../utils/foreign_key_relationship.c | 20 +-- src/backend/distributed/utils/listutils.c | 28 ++--- .../distributed/utils/reference_table_utils.c | 39 +++--- src/backend/distributed/utils/resource_lock.c | 53 +++----- .../distributed/utils/shardinterval_utils.c | 21 ++-- .../distributed/utils/statistics_collection.c | 7 +- src/backend/distributed/worker/task_tracker.c | 12 +- .../worker/worker_data_fetch_protocol.c | 14 +-- .../distributed/worker/worker_drop_protocol.c | 14 +-- .../worker/worker_file_access_protocol.c | 7 +- .../distributed/intermediate_results.h | 9 +- src/include/distributed/master_protocol.h | 10 +- src/include/distributed/metadata_sync.h | 15 +-- src/include/distributed/worker_manager.h | 4 +- src/include/distributed/worker_transaction.h | 22 ++-- 79 files changed, 749 insertions(+), 1163 deletions(-) diff --git a/src/backend/distributed/commands/collation.c b/src/backend/distributed/commands/collation.c index 8b7063cf1..91b824c2d 100644 --- a/src/backend/distributed/commands/collation.c +++ b/src/backend/distributed/commands/collation.c @@ -190,14 +190,13 @@ static List * FilterNameListForDistributedCollations(List *objects, bool missing_ok, List **objectAddresses) { - ListCell *objectCell = NULL; List *result = NIL; *objectAddresses = NIL; - foreach(objectCell, objects) + List *collName = NULL; + foreach_ptr(collName, objects) { - List *collName = lfirst(objectCell); Oid collOid = get_collation_oid(collName, true); ObjectAddress collAddress = { 0 }; diff --git a/src/backend/distributed/commands/create_distributed_table.c b/src/backend/distributed/commands/create_distributed_table.c index 60fe3b5b6..b47fb8a8f 100644 --- a/src/backend/distributed/commands/create_distributed_table.c +++ b/src/backend/distributed/commands/create_distributed_table.c @@ -37,6 +37,7 @@ #include "distributed/colocation_utils.h" #include "distributed/commands.h" #include "distributed/distribution_column.h" +#include "distributed/listutils.h" #include "distributed/master_metadata_utility.h" #include "distributed/master_protocol.h" #include "distributed/metadata_cache.h" @@ -396,11 +397,9 @@ CreateDistributedTable(Oid relationId, Var *distributionColumn, char distributio if (PartitionedTable(relationId)) { List *partitionList = PartitionList(relationId); - ListCell *partitionCell = NULL; - - foreach(partitionCell, partitionList) + Oid partitionRelationId = InvalidOid; + foreach_oid(partitionRelationId, partitionList) { - Oid partitionRelationId = lfirst_oid(partitionCell); CreateDistributedTable(partitionRelationId, distributionColumn, distributionMethod, colocateWithTableName, viaDeprecatedAPI); diff --git a/src/backend/distributed/commands/dependencies.c b/src/backend/distributed/commands/dependencies.c index d41cf2168..dc21cca08 100644 --- a/src/backend/distributed/commands/dependencies.c +++ b/src/backend/distributed/commands/dependencies.c @@ -15,6 +15,7 @@ #include "commands/extension.h" #include "distributed/commands.h" #include "distributed/connection_management.h" +#include "distributed/listutils.h" #include "distributed/metadata/dependency.h" #include "distributed/metadata/distobject.h" #include "distributed/metadata_sync.h" @@ -50,23 +51,14 @@ bool EnableDependencyCreation = true; void EnsureDependenciesExistOnAllNodes(const ObjectAddress *target) { - /* local variables to work with dependencies */ List *dependenciesWithCommands = NIL; - ListCell *dependencyCell = NULL; - - /* local variables to collect ddl commands */ List *ddlCommands = NULL; - /* local variables to work with worker nodes */ - ListCell *workerNodeCell = NULL; - - /* - * collect all dependencies in creation order and get their ddl commands - */ + /* collect all dependencies in creation order and get their ddl commands */ List *dependencies = GetDependenciesForObject(target); - foreach(dependencyCell, dependencies) + ObjectAddress *dependency = NULL; + foreach_ptr(dependency, dependencies) { - ObjectAddress *dependency = (ObjectAddress *) lfirst(dependencyCell); List *dependencyCommands = GetDependencyCreateDDLCommands(dependency); ddlCommands = list_concat(ddlCommands, dependencyCommands); @@ -105,9 +97,8 @@ EnsureDependenciesExistOnAllNodes(const ObjectAddress *target) * to the nodes before marking the objects as distributed these objects would never be * created on the workers when they get added, causing shards to fail to create. */ - foreach(dependencyCell, dependenciesWithCommands) + foreach_ptr(dependency, dependenciesWithCommands) { - ObjectAddress *dependency = (ObjectAddress *) lfirst(dependencyCell); MarkObjectDistributed(dependency); } @@ -121,10 +112,9 @@ EnsureDependenciesExistOnAllNodes(const ObjectAddress *target) } - foreach(workerNodeCell, workerNodeList) + WorkerNode *workerNode = NULL; + foreach_ptr(workerNode, workerNodeList) { - WorkerNode *workerNode = (WorkerNode *) lfirst(workerNodeCell); - const char *nodeName = workerNode->workerName; uint32 nodePort = workerNode->workerPort; @@ -145,16 +135,14 @@ GetDistributableDependenciesForObject(const ObjectAddress *target) { /* local variables to work with dependencies */ List *distributableDependencies = NIL; - ListCell *dependencyCell = NULL; /* collect all dependencies in creation order */ List *dependencies = GetDependenciesForObject(target); /* filter the ones that can be distributed */ - foreach(dependencyCell, dependencies) + ObjectAddress *dependency = NULL; + foreach_ptr(dependency, dependencies) { - ObjectAddress *dependency = (ObjectAddress *) lfirst(dependencyCell); - /* * TODO: maybe we can optimize the logic applied in below line. Actually we * do not need to create ddl commands as we are not ensuring their existence @@ -258,7 +246,6 @@ GetDependencyCreateDDLCommands(const ObjectAddress *dependency) void ReplicateAllDependenciesToNode(const char *nodeName, int nodePort) { - ListCell *dependencyCell = NULL; List *ddlCommands = NIL; /* @@ -290,9 +277,9 @@ ReplicateAllDependenciesToNode(const char *nodeName, int nodePort) } dependencies = OrderObjectAddressListInDependencyOrder(dependencies); - foreach(dependencyCell, dependencies) + ObjectAddress *dependency = NULL; + foreach_ptr(dependency, dependencies) { - ObjectAddress *dependency = (ObjectAddress *) lfirst(dependencyCell); ddlCommands = list_concat(ddlCommands, GetDependencyCreateDDLCommands(dependency)); } @@ -367,11 +354,10 @@ static List * FilterObjectAddressListByPredicate(List *objectAddressList, AddressPredicate predicate) { List *result = NIL; - ListCell *objectAddressListCell = NULL; - foreach(objectAddressListCell, objectAddressList) + ObjectAddress *address = NULL; + foreach_ptr(address, objectAddressList) { - ObjectAddress *address = (ObjectAddress *) lfirst(objectAddressListCell); if (predicate(address)) { result = lappend(result, address); diff --git a/src/backend/distributed/commands/extension.c b/src/backend/distributed/commands/extension.c index 521d5458e..6b47be681 100644 --- a/src/backend/distributed/commands/extension.c +++ b/src/backend/distributed/commands/extension.c @@ -17,6 +17,7 @@ #include "distributed/commands.h" #include "distributed/commands/utility_hook.h" #include "distributed/deparser.h" +#include "distributed/listutils.h" #include "distributed/master_protocol.h" #include "distributed/metadata_sync.h" #include "distributed/metadata/dependency.h" @@ -208,7 +209,7 @@ AddSchemaFieldIfMissing(CreateExtensionStmt *createExtensionStmt) * As we already created the extension by standard_ProcessUtility, * we actually know the schema it belongs to */ - bool missingOk = false; + const bool missingOk = false; Oid extensionOid = get_extension_oid(createExtensionStmt->extname, missingOk); Oid extensionSchemaOid = get_extension_schema(extensionOid); char *extensionSchemaName = get_namespace_name(extensionSchemaOid); @@ -238,7 +239,6 @@ List * PreprocessDropExtensionStmt(Node *node, const char *queryString) { DropStmt *stmt = castNode(DropStmt, node); - ListCell *addressCell = NULL; if (!ShouldPropagateExtensionCommand(node)) { @@ -278,9 +278,9 @@ PreprocessDropExtensionStmt(Node *node, const char *queryString) distributedExtensions); /* unmark each distributed extension */ - foreach(addressCell, distributedExtensionAddresses) + ObjectAddress *address = NULL; + foreach_ptr(address, distributedExtensionAddresses) { - ObjectAddress *address = (ObjectAddress *) lfirst(addressCell); UnmarkObjectDistributed(address); } @@ -293,7 +293,6 @@ PreprocessDropExtensionStmt(Node *node, const char *queryString) */ stmt->objects = distributedExtensions; const char *deparsedStmt = DeparseTreeNode((Node *) stmt); - stmt->objects = allDroppedExtensions; /* @@ -318,12 +317,11 @@ FilterDistributedExtensions(List *extensionObjectList) { List *extensionNameList = NIL; - bool missingOk = true; - ListCell *objectCell = NULL; - - foreach(objectCell, extensionObjectList) + Value *objectName = NULL; + foreach_ptr(objectName, extensionObjectList) { - char *extensionName = strVal(lfirst(objectCell)); + const char *extensionName = strVal(objectName); + const bool missingOk = true; Oid extensionOid = get_extension_oid(extensionName, missingOk); @@ -340,7 +338,7 @@ FilterDistributedExtensions(List *extensionObjectList) continue; } - extensionNameList = lappend(extensionNameList, makeString(extensionName)); + extensionNameList = lappend(extensionNameList, objectName); } return extensionNameList; @@ -358,17 +356,15 @@ ExtensionNameListToObjectAddressList(List *extensionObjectList) { List *extensionObjectAddressList = NIL; - ListCell *objectCell = NULL; - - foreach(objectCell, extensionObjectList) + Value *objectName; + foreach_ptr(objectName, extensionObjectList) { /* * We set missingOk to false as we assume all the objects in * extensionObjectList list are valid and distributed. */ - bool missingOk = false; - - const char *extensionName = strVal(lfirst(objectCell)); + const char *extensionName = strVal(objectName); + const bool missingOk = false; ObjectAddress *address = palloc0(sizeof(ObjectAddress)); @@ -553,18 +549,15 @@ PostprocessAlterExtensionCitusUpdateStmt(Node *node) static void MarkExistingObjectDependenciesDistributedIfSupported() { - ListCell *listCell = NULL; - /* resulting object addresses to be marked as distributed */ List *resultingObjectAddresses = NIL; /* resolve dependencies of distributed tables */ List *distributedTableOidList = DistTableOidList(); - foreach(listCell, distributedTableOidList) + Oid distributedTableOid = InvalidOid; + foreach_oid(distributedTableOid, distributedTableOidList) { - Oid distributedTableOid = lfirst_oid(listCell); - ObjectAddress tableAddress = { 0 }; ObjectAddressSet(tableAddress, RelationRelationId, distributedTableOid); @@ -578,10 +571,9 @@ MarkExistingObjectDependenciesDistributedIfSupported() /* resolve dependencies of the objects in pg_dist_object*/ List *distributedObjectAddressList = GetDistributedObjectAddressList(); - foreach(listCell, distributedObjectAddressList) + ObjectAddress *distributedObjectAddress = NULL; + foreach_ptr(distributedObjectAddress, distributedObjectAddressList) { - ObjectAddress *distributedObjectAddress = (ObjectAddress *) lfirst(listCell); - List *distributableDependencyObjectAddresses = GetDistributableDependenciesForObject(distributedObjectAddress); @@ -592,9 +584,9 @@ MarkExistingObjectDependenciesDistributedIfSupported() /* remove duplicates from object addresses list for efficiency */ List *uniqueObjectAddresses = GetUniqueDependenciesList(resultingObjectAddresses); - foreach(listCell, uniqueObjectAddresses) + ObjectAddress *objectAddress = NULL; + foreach_ptr(objectAddress, uniqueObjectAddresses) { - ObjectAddress *objectAddress = (ObjectAddress *) lfirst(listCell); MarkObjectDistributed(objectAddress); } } @@ -729,8 +721,6 @@ IsCreateAlterExtensionUpdateCitusStmt(Node *parseTree) bool IsDropCitusStmt(Node *parseTree) { - ListCell *objectCell = NULL; - /* if it is not a DropStmt, it is needless to search for citus */ if (!IsA(parseTree, DropStmt)) { @@ -738,9 +728,10 @@ IsDropCitusStmt(Node *parseTree) } /* now that we have a DropStmt, check if citus is among the objects to dropped */ - foreach(objectCell, ((DropStmt *) parseTree)->objects) + Value *objectName; + foreach_ptr(objectName, ((DropStmt *) parseTree)->objects) { - const char *extensionName = strVal(lfirst(objectCell)); + const char *extensionName = strVal(objectName); if (strncasecmp(extensionName, "citus", NAMEDATALEN) == 0) { diff --git a/src/backend/distributed/commands/function.c b/src/backend/distributed/commands/function.c index 2aef95c6d..d8b6bf2bc 100644 --- a/src/backend/distributed/commands/function.c +++ b/src/backend/distributed/commands/function.c @@ -35,6 +35,7 @@ #include "distributed/commands.h" #include "distributed/commands/utility_hook.h" #include "distributed/deparser.h" +#include "distributed/listutils.h" #include "distributed/maintenanced.h" #include "distributed/master_metadata_utility.h" #include "distributed/master_protocol.h" @@ -1027,13 +1028,11 @@ static void TriggerSyncMetadataToPrimaryNodes(void) { List *workerList = ActivePrimaryWorkerNodeList(ShareLock); - ListCell *workerCell = NULL; bool triggerMetadataSync = false; - foreach(workerCell, workerList) + WorkerNode *workerNode = NULL; + foreach_ptr(workerNode, workerList) { - WorkerNode *workerNode = (WorkerNode *) lfirst(workerCell); - /* if already has metadata, no need to do it again */ if (!workerNode->hasMetadata) { @@ -1212,7 +1211,6 @@ CreateFunctionStmtObjectAddress(Node *node, bool missing_ok) { CreateFunctionStmt *stmt = castNode(CreateFunctionStmt, node); ObjectType objectType = OBJECT_FUNCTION; - ListCell *parameterCell = NULL; if (stmt->is_procedure) { @@ -1222,9 +1220,9 @@ CreateFunctionStmtObjectAddress(Node *node, bool missing_ok) ObjectWithArgs *objectWithArgs = makeNode(ObjectWithArgs); objectWithArgs->objname = stmt->funcname; - foreach(parameterCell, stmt->parameters) + FunctionParameter *funcParam = NULL; + foreach_ptr(funcParam, stmt->parameters) { - FunctionParameter *funcParam = castNode(FunctionParameter, lfirst(parameterCell)); objectWithArgs->objargs = lappend(objectWithArgs->objargs, funcParam->argType); } @@ -1243,16 +1241,15 @@ ObjectAddress DefineAggregateStmtObjectAddress(Node *node, bool missing_ok) { DefineStmt *stmt = castNode(DefineStmt, node); - ListCell *parameterCell = NULL; Assert(stmt->kind == OBJECT_AGGREGATE); ObjectWithArgs *objectWithArgs = makeNode(ObjectWithArgs); objectWithArgs->objname = stmt->defnames; - foreach(parameterCell, linitial(stmt->args)) + FunctionParameter *funcParam = NULL; + foreach_ptr(funcParam, linitial(stmt->args)) { - FunctionParameter *funcParam = castNode(FunctionParameter, lfirst(parameterCell)); objectWithArgs->objargs = lappend(objectWithArgs->objargs, funcParam->argType); } @@ -1403,8 +1400,6 @@ PreprocessDropFunctionStmt(Node *node, const char *queryString) List *deletingObjectWithArgsList = stmt->objects; List *distributedObjectWithArgsList = NIL; List *distributedFunctionAddresses = NIL; - ListCell *addressCell = NULL; - ListCell *objectWithArgsListCell = NULL; AssertObjectTypeIsFunctional(stmt->removeType); @@ -1436,9 +1431,9 @@ PreprocessDropFunctionStmt(Node *node, const char *queryString) * iterate over all functions to be dropped and filter to keep only distributed * functions. */ - foreach(objectWithArgsListCell, deletingObjectWithArgsList) + ObjectWithArgs *func = NULL; + foreach_ptr(func, deletingObjectWithArgsList) { - ObjectWithArgs *func = castNode(ObjectWithArgs, lfirst(objectWithArgsListCell)); ObjectAddress address = FunctionToObjectAddress(stmt->removeType, func, stmt->missing_ok); @@ -1469,9 +1464,9 @@ PreprocessDropFunctionStmt(Node *node, const char *queryString) EnsureSequentialModeForFunctionDDL(); /* remove the entries for the distributed objects on dropping */ - foreach(addressCell, distributedFunctionAddresses) + ObjectAddress *address = NULL; + foreach_ptr(address, distributedFunctionAddresses) { - ObjectAddress *address = (ObjectAddress *) lfirst(addressCell); UnmarkObjectDistributed(address); } @@ -1824,11 +1819,9 @@ FunctionToObjectAddress(ObjectType objectType, ObjectWithArgs *objectWithArgs, static void ErrorIfUnsupportedAlterFunctionStmt(AlterFunctionStmt *stmt) { - ListCell *actionCell = NULL; - - foreach(actionCell, stmt->actions) + DefElem *action = NULL; + foreach_ptr(action, stmt->actions) { - DefElem *action = castNode(DefElem, lfirst(actionCell)); if (strcmp(action->defname, "set") == 0) { VariableSetStmt *setStmt = castNode(VariableSetStmt, action->arg); diff --git a/src/backend/distributed/commands/index.c b/src/backend/distributed/commands/index.c index 0fc43afa8..8dbd5df05 100644 --- a/src/backend/distributed/commands/index.c +++ b/src/backend/distributed/commands/index.c @@ -25,6 +25,7 @@ #include "distributed/commands/utility_hook.h" #include "distributed/deparse_shard_query.h" #include "distributed/distributed_planner.h" +#include "distributed/listutils.h" #include "distributed/master_protocol.h" #include "distributed/metadata_cache.h" #include "distributed/multi_physical_planner.h" @@ -309,20 +310,19 @@ PreprocessDropIndexStmt(Node *node, const char *dropIndexCommand) { DropStmt *dropIndexStatement = castNode(DropStmt, node); List *ddlJobs = NIL; - ListCell *dropObjectCell = NULL; Oid distributedIndexId = InvalidOid; Oid distributedRelationId = InvalidOid; Assert(dropIndexStatement->removeType == OBJECT_INDEX); /* check if any of the indexes being dropped belong to a distributed table */ - foreach(dropObjectCell, dropIndexStatement->objects) + List *objectNameList = NULL; + foreach_ptr(objectNameList, dropIndexStatement->objects) { struct DropRelationCallbackState state; uint32 rvrFlags = RVR_MISSING_OK; LOCKMODE lockmode = AccessExclusiveLock; - List *objectNameList = (List *) lfirst(dropObjectCell); RangeVar *rangeVar = makeRangeVarFromNameList(objectNameList); /* @@ -463,13 +463,11 @@ PostprocessIndexStmt(Node *node, const char *queryString) void ErrorIfUnsupportedAlterIndexStmt(AlterTableStmt *alterTableStatement) { - List *commandList = alterTableStatement->cmds; - ListCell *commandCell = NULL; - /* error out if any of the subcommands are unsupported */ - foreach(commandCell, commandList) + List *commandList = alterTableStatement->cmds; + AlterTableCmd *command = NULL; + foreach_ptr(command, commandList) { - AlterTableCmd *command = (AlterTableCmd *) lfirst(commandCell); AlterTableType alterTableType = command->subtype; switch (alterTableType) @@ -508,7 +506,6 @@ CreateIndexTaskList(Oid relationId, IndexStmt *indexStmt) { List *taskList = NIL; List *shardIntervalList = LoadShardIntervalList(relationId); - ListCell *shardIntervalCell = NULL; StringInfoData ddlString; uint64 jobId = INVALID_JOB_ID; int taskId = 1; @@ -518,9 +515,9 @@ CreateIndexTaskList(Oid relationId, IndexStmt *indexStmt) /* lock metadata before getting placement lists */ LockShardListMetadata(shardIntervalList, ShareLock); - foreach(shardIntervalCell, shardIntervalList) + ShardInterval *shardInterval = NULL; + foreach_ptr(shardInterval, shardIntervalList) { - ShardInterval *shardInterval = (ShardInterval *) lfirst(shardIntervalCell); uint64 shardId = shardInterval->shardId; deparse_shard_index_statement(indexStmt, relationId, shardId, &ddlString); @@ -553,7 +550,6 @@ CreateReindexTaskList(Oid relationId, ReindexStmt *reindexStmt) { List *taskList = NIL; List *shardIntervalList = LoadShardIntervalList(relationId); - ListCell *shardIntervalCell = NULL; StringInfoData ddlString; uint64 jobId = INVALID_JOB_ID; int taskId = 1; @@ -563,9 +559,9 @@ CreateReindexTaskList(Oid relationId, ReindexStmt *reindexStmt) /* lock metadata before getting placement lists */ LockShardListMetadata(shardIntervalList, ShareLock); - foreach(shardIntervalCell, shardIntervalList) + ShardInterval *shardInterval = NULL; + foreach_ptr(shardInterval, shardIntervalList) { - ShardInterval *shardInterval = (ShardInterval *) lfirst(shardIntervalCell); uint64 shardId = shardInterval->shardId; deparse_shard_reindex_statement(reindexStmt, relationId, shardId, &ddlString); @@ -794,7 +790,6 @@ ErrorIfUnsupportedIndexStmt(IndexStmt *createIndexStatement) LOCKMODE lockMode = ShareLock; Oid relationId = RangeVarGetRelid(relation, lockMode, missingOk); char partitionMethod = PartitionMethod(relationId); - ListCell *indexParameterCell = NULL; bool indexContainsPartitionColumn = false; /* @@ -815,10 +810,10 @@ ErrorIfUnsupportedIndexStmt(IndexStmt *createIndexStatement) Var *partitionKey = ForceDistPartitionKey(relationId); List *indexParameterList = createIndexStatement->indexParams; - foreach(indexParameterCell, indexParameterList) + IndexElem *indexElement = NULL; + foreach_ptr(indexElement, indexParameterList) { - IndexElem *indexElement = (IndexElem *) lfirst(indexParameterCell); - char *columnName = indexElement->name; + const char *columnName = indexElement->name; /* column name is null for index expressions, skip it */ if (columnName == NULL) @@ -872,7 +867,6 @@ DropIndexTaskList(Oid relationId, Oid indexId, DropStmt *dropStmt) { List *taskList = NIL; List *shardIntervalList = LoadShardIntervalList(relationId); - ListCell *shardIntervalCell = NULL; char *indexName = get_rel_name(indexId); Oid schemaId = get_rel_namespace(indexId); char *schemaName = get_namespace_name(schemaId); @@ -885,9 +879,9 @@ DropIndexTaskList(Oid relationId, Oid indexId, DropStmt *dropStmt) /* lock metadata before getting placement lists */ LockShardListMetadata(shardIntervalList, ShareLock); - foreach(shardIntervalCell, shardIntervalList) + ShardInterval *shardInterval = NULL; + foreach_ptr(shardInterval, shardIntervalList) { - ShardInterval *shardInterval = (ShardInterval *) lfirst(shardIntervalCell); uint64 shardId = shardInterval->shardId; char *shardIndexName = pstrdup(indexName); diff --git a/src/backend/distributed/commands/role.c b/src/backend/distributed/commands/role.c index 127e92528..a0061137a 100644 --- a/src/backend/distributed/commands/role.c +++ b/src/backend/distributed/commands/role.c @@ -21,6 +21,7 @@ #include "distributed/commands.h" #include "distributed/commands/utility_hook.h" #include "distributed/deparser.h" +#include "distributed/listutils.h" #include "distributed/master_protocol.h" #include "distributed/worker_transaction.h" #include "nodes/makefuncs.h" @@ -46,7 +47,6 @@ List * PostprocessAlterRoleStmt(Node *node, const char *queryString) { AlterRoleStmt *stmt = castNode(AlterRoleStmt, node); - ListCell *optionCell = NULL; if (!EnableAlterRolePropagation || !IsCoordinator()) { @@ -60,10 +60,9 @@ PostprocessAlterRoleStmt(Node *node, const char *queryString) */ LockRelationOid(DistNodeRelationId(), RowShareLock); - foreach(optionCell, stmt->options) + DefElem *option = NULL; + foreach_ptr(option, stmt->options) { - DefElem *option = (DefElem *) lfirst(optionCell); - if (strcasecmp(option->defname, "password") == 0) { Oid roleOid = get_rolespec_oid(stmt->role, true); diff --git a/src/backend/distributed/commands/schema.c b/src/backend/distributed/commands/schema.c index 92cc79698..73c670a36 100644 --- a/src/backend/distributed/commands/schema.c +++ b/src/backend/distributed/commands/schema.c @@ -22,6 +22,7 @@ #include #include "distributed/commands/utility_hook.h" #include "distributed/deparser.h" +#include "distributed/listutils.h" #include "distributed/metadata/distobject.h" #include "distributed/metadata_cache.h" #include @@ -51,18 +52,16 @@ PreprocessDropSchemaStmt(Node *node, const char *queryString) int scanKeyCount = 1; Oid scanIndexId = InvalidOid; bool useIndex = false; - ListCell *dropSchemaCell; if (dropStatement->behavior != DROP_CASCADE) { return NIL; } - foreach(dropSchemaCell, dropStatement->objects) + Value *schemaValue = NULL; + foreach_ptr(schemaValue, dropStatement->objects) { - Value *schemaValue = (Value *) lfirst(dropSchemaCell); - char *schemaString = strVal(schemaValue); - + const char *schemaString = strVal(schemaValue); Oid namespaceOid = get_namespace_oid(schemaString, true); if (namespaceOid == InvalidOid) @@ -152,11 +151,11 @@ static List * FilterDistributedSchemas(List *schemas) { List *distributedSchemas = NIL; - ListCell *cell = NULL; - foreach(cell, schemas) + Value *schemaValue = NULL; + foreach_ptr(schemaValue, schemas) { - char *schemaName = strVal(lfirst(cell)); + const char *schemaName = strVal(schemaValue); Oid schemaOid = get_namespace_oid(schemaName, true); if (!OidIsValid(schemaOid)) @@ -172,7 +171,7 @@ FilterDistributedSchemas(List *schemas) continue; } - distributedSchemas = lappend(distributedSchemas, makeString(schemaName)); + distributedSchemas = lappend(distributedSchemas, schemaValue); } return distributedSchemas; diff --git a/src/backend/distributed/commands/sequence.c b/src/backend/distributed/commands/sequence.c index ef59bda4d..1bcd0b4f3 100644 --- a/src/backend/distributed/commands/sequence.c +++ b/src/backend/distributed/commands/sequence.c @@ -15,6 +15,7 @@ #include "catalog/namespace.h" #include "commands/defrem.h" #include "distributed/commands.h" +#include "distributed/listutils.h" #include "distributed/metadata_cache.h" #include "nodes/parsenodes.h" @@ -113,11 +114,9 @@ ErrorIfDistributedAlterSeqOwnedBy(AlterSeqStmt *alterSeqStmt) static bool OptionsSpecifyOwnedBy(List *optionList, Oid *ownedByTableId) { - ListCell *optionCell = NULL; - - foreach(optionCell, optionList) + DefElem *defElem = NULL; + foreach_ptr(defElem, optionList) { - DefElem *defElem = (DefElem *) lfirst(optionCell); if (strcmp(defElem->defname, "owned_by") == 0) { List *ownedByNames = defGetQualifiedName(defElem); diff --git a/src/backend/distributed/commands/table.c b/src/backend/distributed/commands/table.c index ea2f4865f..f4aad4f36 100644 --- a/src/backend/distributed/commands/table.c +++ b/src/backend/distributed/commands/table.c @@ -24,6 +24,7 @@ #include "distributed/commands/utility_hook.h" #include "distributed/deparser.h" #include "distributed/deparse_shard_query.h" +#include "distributed/listutils.h" #include "distributed/master_protocol.h" #include "distributed/metadata_sync.h" #include "distributed/multi_executor.h" @@ -67,16 +68,14 @@ List * PreprocessDropTableStmt(Node *node, const char *queryString) { DropStmt *dropTableStatement = castNode(DropStmt, node); - ListCell *dropTableCell = NULL; Assert(dropTableStatement->removeType == OBJECT_TABLE); - foreach(dropTableCell, dropTableStatement->objects) + List *tableNameList = NULL; + foreach_ptr(tableNameList, dropTableStatement->objects) { - List *tableNameList = (List *) lfirst(dropTableCell); RangeVar *tableRangeVar = makeRangeVarFromNameList(tableNameList); bool missingOK = true; - ListCell *partitionCell = NULL; Oid relationId = RangeVarGetRelid(tableRangeVar, AccessShareLock, missingOK); @@ -108,9 +107,9 @@ PreprocessDropTableStmt(Node *node, const char *queryString) SendCommandToWorkersWithMetadata(DISABLE_DDL_PROPAGATION); - foreach(partitionCell, partitionList) + Oid partitionRelationId = InvalidOid; + foreach_oid(partitionRelationId, partitionList) { - Oid partitionRelationId = lfirst_oid(partitionCell); char *detachPartitionCommand = GenerateDetachPartitionCommand(partitionRelationId); @@ -204,12 +203,9 @@ PostprocessAlterTableStmtAttachPartition(AlterTableStmt *alterTableStatement, const char *queryString) { List *commandList = alterTableStatement->cmds; - ListCell *commandCell = NULL; - - foreach(commandCell, commandList) + AlterTableCmd *alterTableCommand = NULL; + foreach_ptr(alterTableCommand, commandList) { - AlterTableCmd *alterTableCommand = (AlterTableCmd *) lfirst(commandCell); - if (alterTableCommand->subtype == AT_AttachPartition) { Oid relationId = AlterTableLookupRelation(alterTableStatement, NoLock); @@ -290,7 +286,6 @@ PreprocessAlterTableStmt(Node *node, const char *alterTableCommand) { AlterTableStmt *alterTableStatement = castNode(AlterTableStmt, node); Oid rightRelationId = InvalidOid; - ListCell *commandCell = NULL; bool executeSequentially = false; /* first check whether a distributed relation is affected */ @@ -346,10 +341,9 @@ PreprocessAlterTableStmt(Node *node, const char *alterTableCommand) * foreign constraint in master. Validity will be checked in workers anyway. */ List *commandList = alterTableStatement->cmds; - - foreach(commandCell, commandList) + AlterTableCmd *command = NULL; + foreach_ptr(command, commandList) { - AlterTableCmd *command = (AlterTableCmd *) lfirst(commandCell); AlterTableType alterTableType = command->subtype; if (alterTableType == AT_AddConstraint) @@ -389,10 +383,9 @@ PreprocessAlterTableStmt(Node *node, const char *alterTableCommand) ColumnDef *columnDefinition = (ColumnDef *) command->def; List *columnConstraints = columnDefinition->constraints; - ListCell *columnConstraint = NULL; - foreach(columnConstraint, columnConstraints) + Constraint *constraint = NULL; + foreach_ptr(constraint, columnConstraints) { - Constraint *constraint = (Constraint *) lfirst(columnConstraint); if (constraint->contype == CONSTR_FOREIGN) { rightRelationId = RangeVarGetRelid(constraint->pktable, lockmode, @@ -546,8 +539,6 @@ Node * WorkerProcessAlterTableStmt(AlterTableStmt *alterTableStatement, const char *alterTableCommand) { - ListCell *commandCell = NULL; - /* first check whether a distributed relation is affected */ if (alterTableStatement->relation == NULL) { @@ -574,10 +565,9 @@ WorkerProcessAlterTableStmt(AlterTableStmt *alterTableStatement, * foreign constraint in master. Validity will be checked in workers anyway. */ List *commandList = alterTableStatement->cmds; - - foreach(commandCell, commandList) + AlterTableCmd *command = NULL; + foreach_ptr(command, commandList) { - AlterTableCmd *command = (AlterTableCmd *) lfirst(commandCell); AlterTableType alterTableType = command->subtype; if (alterTableType == AT_AddConstraint) @@ -636,9 +626,6 @@ IsAlterTableRenameStmt(RenameStmt *renameStmt) void ErrorIfAlterDropsPartitionColumn(AlterTableStmt *alterTableStatement) { - List *commandList = alterTableStatement->cmds; - ListCell *commandCell = NULL; - /* first check whether a distributed relation is affected */ if (alterTableStatement->relation == NULL) { @@ -659,9 +646,10 @@ ErrorIfAlterDropsPartitionColumn(AlterTableStmt *alterTableStatement) } /* then check if any of subcommands drop partition column.*/ - foreach(commandCell, commandList) + List *commandList = alterTableStatement->cmds; + AlterTableCmd *command = NULL; + foreach_ptr(command, commandList) { - AlterTableCmd *command = (AlterTableCmd *) lfirst(commandCell); AlterTableType alterTableType = command->subtype; if (alterTableType == AT_DropColumn) { @@ -685,9 +673,6 @@ ErrorIfAlterDropsPartitionColumn(AlterTableStmt *alterTableStatement) void PostprocessAlterTableStmt(AlterTableStmt *alterTableStatement) { - List *commandList = alterTableStatement->cmds; - ListCell *commandCell = NULL; - LOCKMODE lockmode = AlterTableGetLockLevel(alterTableStatement->cmds); Oid relationId = AlterTableLookupRelation(alterTableStatement, lockmode); @@ -699,9 +684,10 @@ PostprocessAlterTableStmt(AlterTableStmt *alterTableStatement) EnsureDependenciesExistOnAllNodes(&tableAddress); } - foreach(commandCell, commandList) + List *commandList = alterTableStatement->cmds; + AlterTableCmd *command = NULL; + foreach_ptr(command, commandList) { - AlterTableCmd *command = (AlterTableCmd *) lfirst(commandCell); AlterTableType alterTableType = command->subtype; if (alterTableType == AT_AddConstraint) @@ -723,8 +709,6 @@ PostprocessAlterTableStmt(AlterTableStmt *alterTableStatement) } else if (alterTableType == AT_AddColumn) { - ListCell *columnConstraint = NULL; - ColumnDef *columnDefinition = (ColumnDef *) command->def; List *columnConstraints = columnDefinition->constraints; if (columnConstraints) @@ -737,10 +721,9 @@ PostprocessAlterTableStmt(AlterTableStmt *alterTableStatement) continue; } - foreach(columnConstraint, columnConstraints) + Constraint *constraint = NULL; + foreach_ptr(constraint, columnConstraints) { - Constraint *constraint = (Constraint *) lfirst(columnConstraint); - if (constraint->conname == NULL && (constraint->contype == CONSTR_PRIMARY || constraint->contype == CONSTR_UNIQUE || @@ -861,8 +844,6 @@ void ErrorIfUnsupportedConstraint(Relation relation, char distributionMethod, Var *distributionColumn, uint32 colocationId) { - ListCell *indexOidCell = NULL; - /* * We first perform check for foreign constraints. It is important to do this check * before next check, because other types of constraints are allowed on reference @@ -889,9 +870,9 @@ ErrorIfUnsupportedConstraint(Relation relation, char distributionMethod, char *relationName = RelationGetRelationName(relation); List *indexOidList = RelationGetIndexList(relation); - foreach(indexOidCell, indexOidList) + Oid indexOid = InvalidOid; + foreach_oid(indexOid, indexOidList) { - Oid indexOid = lfirst_oid(indexOidCell); Relation indexDesc = index_open(indexOid, RowExclusiveLock); bool hasDistributionColumn = false; @@ -979,13 +960,11 @@ ErrorIfUnsupportedConstraint(Relation relation, char distributionMethod, static void ErrorIfUnsupportedAlterTableStmt(AlterTableStmt *alterTableStatement) { - List *commandList = alterTableStatement->cmds; - ListCell *commandCell = NULL; - /* error out if any of the subcommands are unsupported */ - foreach(commandCell, commandList) + List *commandList = alterTableStatement->cmds; + AlterTableCmd *command = NULL; + foreach_ptr(command, commandList) { - AlterTableCmd *command = (AlterTableCmd *) lfirst(commandCell); AlterTableType alterTableType = command->subtype; switch (alterTableType) @@ -1213,10 +1192,9 @@ SetupExecutionModeForAlterTable(Oid relationId, AlterTableCmd *command) ColumnDef *columnDefinition = (ColumnDef *) command->def; List *columnConstraints = columnDefinition->constraints; - ListCell *columnConstraint = NULL; - foreach(columnConstraint, columnConstraints) + Constraint *constraint = NULL; + foreach_ptr(constraint, columnConstraints) { - Constraint *constraint = (Constraint *) lfirst(columnConstraint); if (constraint->contype == CONSTR_FOREIGN) { Oid rightRelationId = RangeVarGetRelid(constraint->pktable, NoLock, diff --git a/src/backend/distributed/commands/transmit.c b/src/backend/distributed/commands/transmit.c index d925d847c..8ce727017 100644 --- a/src/backend/distributed/commands/transmit.c +++ b/src/backend/distributed/commands/transmit.c @@ -16,6 +16,7 @@ #include #include "commands/defrem.h" +#include "distributed/listutils.h" #include "distributed/relay_utility.h" #include "distributed/transmit.h" #include "distributed/worker_protocol.h" @@ -318,13 +319,11 @@ IsTransmitStmt(Node *parsetree) if (IsA(parsetree, CopyStmt)) { CopyStmt *copyStatement = (CopyStmt *) parsetree; - ListCell *optionCell = NULL; /* Extract options from the statement node tree */ - foreach(optionCell, copyStatement->options) + DefElem *defel = NULL; + foreach_ptr(defel, copyStatement->options) { - DefElem *defel = (DefElem *) lfirst(optionCell); - if (strncmp(defel->defname, "format", NAMEDATALEN) == 0 && strncmp(defGetString(defel), "transmit", NAMEDATALEN) == 0) { @@ -344,22 +343,24 @@ IsTransmitStmt(Node *parsetree) char * TransmitStatementUser(CopyStmt *copyStatement) { - ListCell *optionCell = NULL; - char *userName = NULL; - AssertArg(IsTransmitStmt((Node *) copyStatement)); - foreach(optionCell, copyStatement->options) + DefElem *lastUserDefElem = NULL; + DefElem *defel = NULL; + foreach_ptr(defel, copyStatement->options) { - DefElem *defel = (DefElem *) lfirst(optionCell); - if (strncmp(defel->defname, "user", NAMEDATALEN) == 0) { - userName = defGetString(defel); + lastUserDefElem = defel; } } - return userName; + if (lastUserDefElem == NULL) + { + return NULL; + } + + return defGetString(lastUserDefElem); } diff --git a/src/backend/distributed/commands/truncate.c b/src/backend/distributed/commands/truncate.c index 3910bb2ea..f147c7a21 100644 --- a/src/backend/distributed/commands/truncate.c +++ b/src/backend/distributed/commands/truncate.c @@ -64,10 +64,9 @@ static void ErrorIfUnsupportedTruncateStmt(TruncateStmt *truncateStatement) { List *relationList = truncateStatement->relations; - ListCell *relationCell = NULL; - foreach(relationCell, relationList) + RangeVar *rangeVar = NULL; + foreach_ptr(rangeVar, relationList) { - RangeVar *rangeVar = (RangeVar *) lfirst(relationCell); Oid relationId = RangeVarGetRelid(rangeVar, NoLock, false); char relationKind = get_rel_relkind(relationId); if (IsDistributedTable(relationId) && @@ -90,11 +89,9 @@ ErrorIfUnsupportedTruncateStmt(TruncateStmt *truncateStatement) static void EnsurePartitionTableNotReplicatedForTruncate(TruncateStmt *truncateStatement) { - ListCell *relationCell = NULL; - - foreach(relationCell, truncateStatement->relations) + RangeVar *rangeVar = NULL; + foreach_ptr(rangeVar, truncateStatement->relations) { - RangeVar *rangeVar = (RangeVar *) lfirst(relationCell); Oid relationId = RangeVarGetRelid(rangeVar, NoLock, false); if (!IsDistributedTable(relationId)) @@ -118,12 +115,11 @@ static void ExecuteTruncateStmtSequentialIfNecessary(TruncateStmt *command) { List *relationList = command->relations; - ListCell *relationCell = NULL; bool failOK = false; - foreach(relationCell, relationList) + RangeVar *rangeVar = NULL; + foreach_ptr(rangeVar, relationList) { - RangeVar *rangeVar = (RangeVar *) lfirst(relationCell); Oid relationId = RangeVarGetRelid(rangeVar, NoLock, failOK); if (IsDistributedTable(relationId) && @@ -168,7 +164,6 @@ static void LockTruncatedRelationMetadataInWorkers(TruncateStmt *truncateStatement) { List *distributedRelationList = NIL; - ListCell *relationCell = NULL; /* nothing to do if there is no metadata at worker nodes */ if (!ClusterHasKnownMetadataWorkers()) @@ -176,9 +171,9 @@ LockTruncatedRelationMetadataInWorkers(TruncateStmt *truncateStatement) return; } - foreach(relationCell, truncateStatement->relations) + RangeVar *rangeVar = NULL; + foreach_ptr(rangeVar, truncateStatement->relations) { - RangeVar *rangeVar = (RangeVar *) lfirst(relationCell); Oid relationId = RangeVarGetRelid(rangeVar, NoLock, false); Oid referencingRelationId = InvalidOid; @@ -249,16 +244,15 @@ AcquireDistributedLockOnRelations(List *relationIdList, LOCKMODE lockMode) { char *qualifiedRelationName = generate_qualified_relation_name(relationId); StringInfo lockRelationCommand = makeStringInfo(); - ListCell *workerNodeCell = NULL; appendStringInfo(lockRelationCommand, LOCK_RELATION_IF_EXISTS, quote_literal_cstr(qualifiedRelationName), lockModeText); - foreach(workerNodeCell, workerNodeList) + WorkerNode *workerNode = NULL; + foreach_ptr(workerNode, workerNodeList) { - WorkerNode *workerNode = (WorkerNode *) lfirst(workerNodeCell); - char *nodeName = workerNode->workerName; + const char *nodeName = workerNode->workerName; int nodePort = workerNode->workerPort; /* if local node is one of the targets, acquire the lock locally */ diff --git a/src/backend/distributed/commands/type.c b/src/backend/distributed/commands/type.c index 2e46a0f41..93d75e272 100644 --- a/src/backend/distributed/commands/type.c +++ b/src/backend/distributed/commands/type.c @@ -54,6 +54,7 @@ #include "distributed/commands.h" #include "distributed/commands/utility_hook.h" #include "distributed/deparser.h" +#include "distributed/listutils.h" #include "distributed/metadata/distobject.h" #include "distributed/metadata/namespace.h" #include "distributed/metadata_sync.h" @@ -452,7 +453,6 @@ PreprocessDropTypeStmt(Node *node, const char *queryString) * the old list to put back */ List *oldTypes = stmt->objects; - ListCell *addressCell = NULL; if (!ShouldPropagate()) { @@ -478,9 +478,9 @@ PreprocessDropTypeStmt(Node *node, const char *queryString) * remove the entries for the distributed objects on dropping */ List *distributedTypeAddresses = TypeNameListToObjectAddresses(distributedTypes); - foreach(addressCell, distributedTypeAddresses) + ObjectAddress *address = NULL; + foreach_ptr(address, distributedTypeAddresses) { - ObjectAddress *address = (ObjectAddress *) lfirst(addressCell); UnmarkObjectDistributed(address); } @@ -489,14 +489,14 @@ PreprocessDropTypeStmt(Node *node, const char *queryString) * deparse to an executable sql statement for the workers */ stmt->objects = distributedTypes; - const char *dropStmtSql = DeparseTreeNode((Node *) stmt); + char *dropStmtSql = DeparseTreeNode((Node *) stmt); stmt->objects = oldTypes; EnsureSequentialModeForTypeDDL(); /* to prevent recursion with mx we disable ddl propagation */ List *commands = list_make3(DISABLE_DDL_PROPAGATION, - (void *) dropStmtSql, + dropStmtSql, ENABLE_DDL_PROPAGATION); return NodeDDLTaskList(ALL_WORKERS, commands); @@ -1113,11 +1113,10 @@ GenerateBackupNameForTypeCollision(const ObjectAddress *address) static List * FilterNameListForDistributedTypes(List *objects, bool missing_ok) { - ListCell *objectCell = NULL; List *result = NIL; - foreach(objectCell, objects) + TypeName *typeName = NULL; + foreach_ptr(typeName, objects) { - TypeName *typeName = castNode(TypeName, lfirst(objectCell)); Oid typeOid = LookupTypeNameOid(NULL, typeName, missing_ok); ObjectAddress typeAddress = { 0 }; @@ -1144,11 +1143,10 @@ FilterNameListForDistributedTypes(List *objects, bool missing_ok) static List * TypeNameListToObjectAddresses(List *objects) { - ListCell *objectCell = NULL; List *result = NIL; - foreach(objectCell, objects) + TypeName *typeName = NULL; + foreach_ptr(typeName, objects) { - TypeName *typeName = castNode(TypeName, lfirst(objectCell)); Oid typeOid = LookupTypeNameOid(NULL, typeName, false); ObjectAddress *typeAddress = palloc0(sizeof(ObjectAddress)); ObjectAddressSet(*typeAddress, TypeRelationId, typeOid); diff --git a/src/backend/distributed/commands/utility_hook.c b/src/backend/distributed/commands/utility_hook.c index 92c2552d3..7ed7a1928 100644 --- a/src/backend/distributed/commands/utility_hook.c +++ b/src/backend/distributed/commands/utility_hook.c @@ -166,13 +166,11 @@ multi_ProcessUtility(PlannedStmt *pstmt, if (IsTransactionBlock()) { - ListCell *optionCell = NULL; bool analyze = false; - foreach(optionCell, explainStmt->options) + DefElem *option = NULL; + foreach_ptr(option, explainStmt->options) { - DefElem *option = (DefElem *) lfirst(optionCell); - if (strcmp(option->defname, "analyze") == 0) { analyze = defGetBoolean(option); @@ -807,12 +805,12 @@ CurrentSearchPath(void) { StringInfo currentSearchPath = makeStringInfo(); List *searchPathList = fetch_search_path(false); - ListCell *searchPathCell; bool schemaAdded = false; - foreach(searchPathCell, searchPathList) + Oid searchPathOid = InvalidOid; + foreach_oid(searchPathOid, searchPathList) { - char *schemaName = get_namespace_name(lfirst_oid(searchPathCell)); + char *schemaName = get_namespace_name(searchPathOid); /* watch out for deleted namespace */ if (schemaName) @@ -871,7 +869,6 @@ DDLTaskList(Oid relationId, const char *commandString) { List *taskList = NIL; List *shardIntervalList = LoadShardIntervalList(relationId); - ListCell *shardIntervalCell = NULL; Oid schemaId = get_rel_namespace(relationId); char *schemaName = get_namespace_name(schemaId); char *escapedSchemaName = quote_literal_cstr(schemaName); @@ -882,9 +879,9 @@ DDLTaskList(Oid relationId, const char *commandString) /* lock metadata before getting placement lists */ LockShardListMetadata(shardIntervalList, ShareLock); - foreach(shardIntervalCell, shardIntervalList) + ShardInterval *shardInterval = NULL; + foreach_ptr(shardInterval, shardIntervalList) { - ShardInterval *shardInterval = (ShardInterval *) lfirst(shardIntervalCell); uint64 shardId = shardInterval->shardId; StringInfo applyCommand = makeStringInfo(); @@ -921,7 +918,6 @@ NodeDDLTaskList(TargetWorkerSet targets, List *commands) { List *workerNodes = TargetWorkerSetNodeList(targets, NoLock); char *concatenatedCommands = StringJoin(commands, ';'); - ListCell *workerNodeCell = NULL; if (list_length(workerNodes) <= 0) { @@ -936,10 +932,9 @@ NodeDDLTaskList(TargetWorkerSet targets, List *commands) task->taskType = DDL_TASK; SetTaskQueryString(task, concatenatedCommands); - foreach(workerNodeCell, workerNodes) + WorkerNode *workerNode = NULL; + foreach_ptr(workerNode, workerNodes) { - WorkerNode *workerNode = (WorkerNode *) lfirst(workerNodeCell); - ShardPlacement *targetPlacement = CitusMakeNode(ShardPlacement); targetPlacement->nodeName = workerNode->workerName; targetPlacement->nodePort = workerNode->workerPort; diff --git a/src/backend/distributed/commands/vacuum.c b/src/backend/distributed/commands/vacuum.c index e426b2c8c..6574d5da6 100644 --- a/src/backend/distributed/commands/vacuum.c +++ b/src/backend/distributed/commands/vacuum.c @@ -17,6 +17,7 @@ #include "distributed/commands.h" #include "distributed/commands/utility_hook.h" #include "distributed/deparse_shard_query.h" +#include "distributed/listutils.h" #include "distributed/metadata_cache.h" #include "distributed/multi_executor.h" #include "distributed/resource_lock.h" @@ -64,17 +65,15 @@ PostprocessVacuumStmt(VacuumStmt *vacuumStmt, const char *vacuumCommand) { int relationIndex = 0; List *vacuumRelationList = ExtractVacuumTargetRels(vacuumStmt); - ListCell *vacuumRelationCell = NULL; List *relationIdList = NIL; - ListCell *relationIdCell = NULL; CitusVacuumParams vacuumParams = VacuumStmtParams(vacuumStmt); LOCKMODE lockMode = (vacuumParams.options & VACOPT_FULL) ? AccessExclusiveLock : ShareUpdateExclusiveLock; int executedVacuumCount = 0; - foreach(vacuumRelationCell, vacuumRelationList) + RangeVar *vacuumRelation = NULL; + foreach_ptr(vacuumRelation, vacuumRelationList) { - RangeVar *vacuumRelation = (RangeVar *) lfirst(vacuumRelationCell); Oid relationId = RangeVarGetRelid(vacuumRelation, lockMode, false); relationIdList = lappend_oid(relationIdList, relationId); } @@ -87,9 +86,9 @@ PostprocessVacuumStmt(VacuumStmt *vacuumStmt, const char *vacuumCommand) } /* execute vacuum on distributed tables */ - foreach(relationIdCell, relationIdList) + Oid relationId = InvalidOid; + foreach_oid(relationId, relationIdList) { - Oid relationId = lfirst_oid(relationIdCell); if (IsDistributedTable(relationId)) { /* @@ -131,7 +130,6 @@ IsDistributedVacuumStmt(int vacuumOptions, List *vacuumRelationIdList) { const char *stmtName = (vacuumOptions & VACOPT_VACUUM) ? "VACUUM" : "ANALYZE"; bool distributeStmt = false; - ListCell *relationIdCell = NULL; int distributedRelationCount = 0; /* @@ -147,9 +145,9 @@ IsDistributedVacuumStmt(int vacuumOptions, List *vacuumRelationIdList) "distributed tables.", stmtName))); } - foreach(relationIdCell, vacuumRelationIdList) + Oid relationId = InvalidOid; + foreach_oid(relationId, vacuumRelationIdList) { - Oid relationId = lfirst_oid(relationIdCell); if (OidIsValid(relationId) && IsDistributedTable(relationId)) { distributedRelationCount++; @@ -185,7 +183,6 @@ static List * VacuumTaskList(Oid relationId, CitusVacuumParams vacuumParams, List *vacuumColumnList) { List *taskList = NIL; - ListCell *shardIntervalCell = NULL; uint64 jobId = INVALID_JOB_ID; int taskId = 1; StringInfo vacuumString = DeparseVacuumStmtPrefix(vacuumParams); @@ -209,9 +206,9 @@ VacuumTaskList(Oid relationId, CitusVacuumParams vacuumParams, List *vacuumColum /* grab shard lock before getting placement list */ LockShardListMetadata(shardIntervalList, ShareLock); - foreach(shardIntervalCell, shardIntervalList) + ShardInterval *shardInterval = NULL; + foreach_ptr(shardInterval, shardIntervalList) { - ShardInterval *shardInterval = (ShardInterval *) lfirst(shardIntervalCell); uint64 shardId = shardInterval->shardId; char *shardName = pstrdup(tableName); @@ -353,7 +350,6 @@ static char * DeparseVacuumColumnNames(List *columnNameList) { StringInfo columnNames = makeStringInfo(); - ListCell *columnNameCell = NULL; if (columnNameList == NIL) { @@ -362,11 +358,10 @@ DeparseVacuumColumnNames(List *columnNameList) appendStringInfoString(columnNames, " ("); - foreach(columnNameCell, columnNameList) + Value *columnName = NULL; + foreach_ptr(columnName, columnNameList) { - char *columnName = strVal(lfirst(columnNameCell)); - - appendStringInfo(columnNames, "%s,", columnName); + appendStringInfo(columnNames, "%s,", strVal(columnName)); } columnNames->data[columnNames->len - 1] = ')'; @@ -398,10 +393,9 @@ ExtractVacuumTargetRels(VacuumStmt *vacuumStmt) { List *vacuumList = NIL; - ListCell *vacuumRelationCell = NULL; - foreach(vacuumRelationCell, vacuumStmt->rels) + VacuumRelation *vacuumRelation = NULL; + foreach_ptr(vacuumRelation, vacuumStmt->rels) { - VacuumRelation *vacuumRelation = (VacuumRelation *) lfirst(vacuumRelationCell); vacuumList = lappend(vacuumList, vacuumRelation->relation); } @@ -427,17 +421,15 @@ VacuumStmtParams(VacuumStmt *vacstmt) bool freeze = false; bool full = false; bool disable_page_skipping = false; - ListCell *lc; /* Set default value */ params.index_cleanup = VACOPT_TERNARY_DEFAULT; params.truncate = VACOPT_TERNARY_DEFAULT; /* Parse options list */ - foreach(lc, vacstmt->options) + DefElem *opt = NULL; + foreach_ptr(opt, vacstmt->options) { - DefElem *opt = (DefElem *) lfirst(lc); - /* Parse common options for VACUUM and ANALYZE */ if (strcmp(opt->defname, "verbose") == 0) { diff --git a/src/backend/distributed/connection/connection_management.c b/src/backend/distributed/connection/connection_management.c index 3e8d992f2..9f34e5ac0 100644 --- a/src/backend/distributed/connection/connection_management.c +++ b/src/backend/distributed/connection/connection_management.c @@ -21,6 +21,7 @@ #include "commands/dbcommands.h" #include "distributed/connection_management.h" #include "distributed/errormessage.h" +#include "distributed/listutils.h" #include "distributed/log_utils.h" #include "distributed/memutils.h" #include "distributed/metadata_cache.h" @@ -222,12 +223,11 @@ StartWorkerListConnections(List *workerNodeList, uint32 flags, const char *user, const char *database) { List *connectionList = NIL; - ListCell *workerNodeCell = NULL; - foreach(workerNodeCell, workerNodeList) + WorkerNode *workerNode = NULL; + foreach_ptr(workerNode, workerNodeList) { - WorkerNode *workerNode = (WorkerNode *) lfirst(workerNodeCell); - char *nodeName = workerNode->workerName; + const char *nodeName = workerNode->workerName; int nodePort = workerNode->workerPort; int connectionFlags = 0; @@ -599,8 +599,6 @@ EventSetSizeForConnectionList(List *connections) static WaitEventSet * WaitEventSetFromMultiConnectionStates(List *connections, int *waitCount) { - ListCell *connectionCell = NULL; - const int eventSetSize = EventSetSizeForConnectionList(connections); int numEventsAdded = 0; @@ -621,11 +619,9 @@ WaitEventSetFromMultiConnectionStates(List *connections, int *waitCount) AddWaitEventToSet(waitEventSet, WL_LATCH_SET, PGINVALID_SOCKET, MyLatch, NULL); numEventsAdded += 2; - foreach(connectionCell, connections) + MultiConnectionPollState *connectionState = NULL; + foreach_ptr(connectionState, connections) { - MultiConnectionPollState *connectionState = (MultiConnectionPollState *) lfirst( - connectionCell); - if (numEventsAdded >= eventSetSize) { /* room for events to schedule is exhausted */ @@ -688,15 +684,14 @@ FinishConnectionListEstablishment(List *multiConnectionList) INSTR_TIME_SET_CURRENT(connectionStart); List *connectionStates = NULL; - ListCell *multiConnectionCell = NULL; WaitEventSet *waitEventSet = NULL; bool waitEventSetRebuild = true; int waitCount = 0; - foreach(multiConnectionCell, multiConnectionList) + MultiConnection *connection = NULL; + foreach_ptr(connection, multiConnectionList) { - MultiConnection *connection = (MultiConnection *) lfirst(multiConnectionCell); MultiConnectionPollState *connectionState = palloc0(sizeof(MultiConnectionPollState)); @@ -802,9 +797,8 @@ FinishConnectionListEstablishment(List *multiConnectionList) */ if (connectionState->phase == MULTI_CONNECTION_PHASE_CONNECTED) { - MultiConnection *connection = connectionState->connection; - - connection->connectionState = MULTI_CONNECTION_CONNECTED; + connectionState->connection->connectionState = + MULTI_CONNECTION_CONNECTED; } } } @@ -876,10 +870,9 @@ MillisecondsToTimeout(instr_time start, long msAfterStart) static void CloseNotReadyMultiConnectionStates(List *connectionStates) { - ListCell *connectionStateCell = NULL; - foreach(connectionStateCell, connectionStates) + MultiConnectionPollState *connectionState = NULL; + foreach_ptr(connectionState, connectionStates) { - MultiConnectionPollState *connectionState = lfirst(connectionStateCell); MultiConnection *connection = connectionState->connection; if (connectionState->phase != MULTI_CONNECTION_PHASE_CONNECTING) diff --git a/src/backend/distributed/connection/placement_connection.c b/src/backend/distributed/connection/placement_connection.c index f92a93831..27cab75e0 100644 --- a/src/backend/distributed/connection/placement_connection.c +++ b/src/backend/distributed/connection/placement_connection.c @@ -15,6 +15,7 @@ #include "distributed/colocation_utils.h" #include "distributed/connection_management.h" #include "distributed/hash_helpers.h" +#include "distributed/listutils.h" #include "distributed/master_protocol.h" #include "distributed/metadata_cache.h" #include "distributed/distributed_planner.h" @@ -338,13 +339,11 @@ StartPlacementListConnection(uint32 flags, List *placementAccessList, void AssignPlacementListToConnection(List *placementAccessList, MultiConnection *connection) { - ListCell *placementAccessCell = NULL; - char *userName = connection->user; + const char *userName = connection->user; - foreach(placementAccessCell, placementAccessList) + ShardPlacementAccess *placementAccess = NULL; + foreach_ptr(placementAccess, placementAccessList) { - ShardPlacementAccess *placementAccess = - (ShardPlacementAccess *) lfirst(placementAccessCell); ShardPlacement *placement = placementAccess->placement; ShardPlacementAccessType accessType = placementAccess->accessType; @@ -490,7 +489,6 @@ static MultiConnection * FindPlacementListConnection(int flags, List *placementAccessList, const char *userName) { bool foundModifyingConnection = false; - ListCell *placementAccessCell = NULL; MultiConnection *chosenConnection = NULL; /* @@ -506,10 +504,9 @@ FindPlacementListConnection(int flags, List *placementAccessList, const char *us * If placements have only been read in this transaction, then use the last * suitable connection found for a placement in the placementAccessList. */ - foreach(placementAccessCell, placementAccessList) + ShardPlacementAccess *placementAccess = NULL; + foreach_ptr(placementAccess, placementAccessList) { - ShardPlacementAccess *placementAccess = - (ShardPlacementAccess *) lfirst(placementAccessCell); ShardPlacement *placement = placementAccess->placement; ShardPlacementAccessType accessType = placementAccess->accessType; diff --git a/src/backend/distributed/connection/remote_commands.c b/src/backend/distributed/connection/remote_commands.c index 7c3ef2d0b..a2160a891 100644 --- a/src/backend/distributed/connection/remote_commands.c +++ b/src/backend/distributed/connection/remote_commands.c @@ -15,6 +15,7 @@ #include "distributed/connection_management.h" #include "distributed/errormessage.h" +#include "distributed/listutils.h" #include "distributed/log_utils.h" #include "distributed/remote_commands.h" #include "distributed/cancel_utils.h" @@ -369,12 +370,9 @@ LogRemoteCommand(MultiConnection *connection, const char *command) void ExecuteCriticalRemoteCommandList(MultiConnection *connection, List *commandList) { - ListCell *commandCell = NULL; - - foreach(commandCell, commandList) + const char *command = NULL; + foreach_ptr(command, commandList) { - char *command = (char *) lfirst(commandCell); - ExecuteCriticalRemoteCommand(connection, command); } } @@ -781,7 +779,6 @@ WaitForAllConnections(List *connectionList, bool raiseInterrupts) int totalConnectionCount = list_length(connectionList); int pendingConnectionsStartIndex = 0; int connectionIndex = 0; - ListCell *connectionCell = NULL; MultiConnection **allConnections = palloc(totalConnectionCount * sizeof(MultiConnection *)); @@ -790,11 +787,10 @@ WaitForAllConnections(List *connectionList, bool raiseInterrupts) WaitEventSet *waitEventSet = NULL; /* convert connection list to an array such that we can move items around */ - foreach(connectionCell, connectionList) + MultiConnection *connectionItem = NULL; + foreach_ptr(connectionItem, connectionList) { - MultiConnection *connection = (MultiConnection *) lfirst(connectionCell); - - allConnections[connectionIndex] = connection; + allConnections[connectionIndex] = connectionItem; connectionReady[connectionIndex] = false; connectionIndex++; } diff --git a/src/backend/distributed/executor/citus_custom_scan.c b/src/backend/distributed/executor/citus_custom_scan.c index c11cafb4b..8b7ea274a 100644 --- a/src/backend/distributed/executor/citus_custom_scan.c +++ b/src/backend/distributed/executor/citus_custom_scan.c @@ -20,6 +20,7 @@ #include "distributed/distributed_execution_locks.h" #include "distributed/insert_select_executor.h" #include "distributed/insert_select_planner.h" +#include "distributed/listutils.h" #include "distributed/local_executor.h" #include "distributed/multi_executor.h" #include "distributed/multi_server_executor.h" @@ -536,12 +537,10 @@ CacheLocalPlanForTask(Task *task, DistributedPlan *originalDistributedPlan) PlannedStmt * GetCachedLocalPlan(Task *task, DistributedPlan *distributedPlan) { - ListCell *cachedLocalPlanCell = NULL; List *cachedPlanList = distributedPlan->workerJob->localPlannedStatements; - foreach(cachedLocalPlanCell, cachedPlanList) + LocalPlannedStatement *localPlannedStatement = NULL; + foreach_ptr(localPlannedStatement, cachedPlanList) { - LocalPlannedStatement *localPlannedStatement = lfirst(cachedLocalPlanCell); - if (localPlannedStatement->shardId == task->anchorShardId && localPlannedStatement->localGroupId == GetLocalGroupId()) { diff --git a/src/backend/distributed/executor/directed_acyclic_graph_execution.c b/src/backend/distributed/executor/directed_acyclic_graph_execution.c index 9c177fa95..bd9fa4011 100644 --- a/src/backend/distributed/executor/directed_acyclic_graph_execution.c +++ b/src/backend/distributed/executor/directed_acyclic_graph_execution.c @@ -11,17 +11,17 @@ #include "access/hash.h" #include "distributed/hash_helpers.h" -#include "distributed/directed_acyclic_graph_execution.h" -#include "distributed/multi_physical_planner.h" #include "distributed/adaptive_executor.h" -#include "distributed/worker_manager.h" -#include "distributed/multi_server_executor.h" -#include "distributed/worker_transaction.h" -#include "distributed/worker_manager.h" -#include "distributed/transaction_management.h" -#include "distributed/multi_task_tracker_executor.h" +#include "distributed/directed_acyclic_graph_execution.h" +#include "distributed/listutils.h" #include "distributed/metadata_cache.h" +#include "distributed/multi_physical_planner.h" +#include "distributed/multi_server_executor.h" +#include "distributed/multi_task_tracker_executor.h" +#include "distributed/transaction_management.h" #include "distributed/transmit.h" +#include "distributed/worker_manager.h" +#include "distributed/worker_transaction.h" typedef struct TaskHashKey { @@ -83,13 +83,10 @@ static List * FindExecutableTasks(List *allTasks, HTAB *completedTasks) { List *curTasks = NIL; - ListCell *taskCell = NULL; - - foreach(taskCell, allTasks) + Task *task = NULL; + foreach_ptr(task, allTasks) { - Task *task = (Task *) lfirst(taskCell); - if (IsAllDependencyCompleted(task, completedTasks) && !IsTaskAlreadyCompleted(task, completedTasks)) { @@ -107,13 +104,11 @@ FindExecutableTasks(List *allTasks, HTAB *completedTasks) static void AddCompletedTasks(List *curCompletedTasks, HTAB *completedTasks) { - ListCell *taskCell = NULL; - bool found; - foreach(taskCell, curCompletedTasks) + Task *task = NULL; + foreach_ptr(task, curCompletedTasks) { - Task *task = (Task *) lfirst(taskCell); TaskHashKey taskKey = { task->jobId, task->taskId }; hash_search(completedTasks, &taskKey, HASH_ENTER, &found); } @@ -155,13 +150,11 @@ IsTaskAlreadyCompleted(Task *task, HTAB *completedTasks) static bool IsAllDependencyCompleted(Task *targetTask, HTAB *completedTasks) { - ListCell *taskCell = NULL; bool found = false; - - foreach(taskCell, targetTask->dependentTaskList) + Task *task = NULL; + foreach_ptr(task, targetTask->dependentTaskList) { - Task *task = (Task *) lfirst(taskCell); TaskHashKey taskKey = { task->jobId, task->taskId }; hash_search(completedTasks, &taskKey, HASH_FIND, &found); diff --git a/src/backend/distributed/executor/distributed_execution_locks.c b/src/backend/distributed/executor/distributed_execution_locks.c index 9b41548c1..43ec15298 100644 --- a/src/backend/distributed/executor/distributed_execution_locks.c +++ b/src/backend/distributed/executor/distributed_execution_locks.c @@ -9,6 +9,7 @@ *------------------------------------------------------------------------- */ #include "distributed/distributed_execution_locks.h" +#include "distributed/listutils.h" #include "distributed/metadata_cache.h" #include "distributed/multi_executor.h" #include "distributed/multi_partitioning_utils.h" @@ -87,11 +88,9 @@ AcquireExecutorShardLocks(Task *task, RowModifyLevel modLevel) void AcquireExecutorMultiShardLocks(List *taskList) { - ListCell *taskCell = NULL; - - foreach(taskCell, taskList) + Task *task = NULL; + foreach_ptr(task, taskList) { - Task *task = (Task *) lfirst(taskCell); LOCKMODE lockMode = NoLock; if (task->anchorShardId == INVALID_SHARD_ID) @@ -227,8 +226,6 @@ RequiresConsistentSnapshot(Task *task) void AcquireMetadataLocks(List *taskList) { - ListCell *taskCell = NULL; - /* * Note: to avoid the overhead of additional sorting, we assume tasks * to be already sorted by shard ID such that deadlocks are avoided. @@ -236,10 +233,9 @@ AcquireMetadataLocks(List *taskList) * command right now. */ - foreach(taskCell, taskList) + Task *task = NULL; + foreach_ptr(task, taskList) { - Task *task = (Task *) lfirst(taskCell); - LockShardDistributionMetadata(task->anchorShardId, ShareLock); } } @@ -354,7 +350,6 @@ AcquireExecutorShardLockForRowModify(Task *task, RowModifyLevel modLevel) static void AcquireExecutorShardLocksForRelationRowLockList(List *relationRowLockList) { - ListCell *relationRowLockCell = NULL; LOCKMODE rowLockMode = NoLock; if (relationRowLockList == NIL) @@ -379,9 +374,9 @@ AcquireExecutorShardLocksForRelationRowLockList(List *relationRowLockList) * with each other but conflicts with modify commands, we get ShareLock for * them. */ - foreach(relationRowLockCell, relationRowLockList) + RelationRowLock *relationRowLock = NULL; + foreach_ptr(relationRowLock, relationRowLockList) { - RelationRowLock *relationRowLock = lfirst(relationRowLockCell); LockClauseStrength rowLockStrength = relationRowLock->rowLockStrength; Oid relationId = relationRowLock->relationId; @@ -412,11 +407,9 @@ AcquireExecutorShardLocksForRelationRowLockList(List *relationRowLockList) void LockPartitionsInRelationList(List *relationIdList, LOCKMODE lockmode) { - ListCell *relationIdCell = NULL; - - foreach(relationIdCell, relationIdList) + Oid relationId = InvalidOid; + foreach_oid(relationId, relationIdList) { - Oid relationId = lfirst_oid(relationIdCell); if (PartitionedTable(relationId)) { LockPartitionRelations(relationId, lockmode); @@ -439,11 +432,9 @@ LockPartitionRelations(Oid relationId, LOCKMODE lockMode) * locks. */ List *partitionList = PartitionList(relationId); - ListCell *partitionCell = NULL; - - foreach(partitionCell, partitionList) + Oid partitionRelationId = InvalidOid; + foreach_oid(partitionRelationId, partitionList) { - Oid partitionRelationId = lfirst_oid(partitionCell); LockRelationOid(partitionRelationId, lockMode); } } diff --git a/src/backend/distributed/executor/distributed_intermediate_results.c b/src/backend/distributed/executor/distributed_intermediate_results.c index 40688f04a..4a02b9002 100644 --- a/src/backend/distributed/executor/distributed_intermediate_results.c +++ b/src/backend/distributed/executor/distributed_intermediate_results.c @@ -19,6 +19,7 @@ #include "catalog/pg_type.h" #include "distributed/deparse_shard_query.h" #include "distributed/intermediate_results.h" +#include "distributed/listutils.h" #include "distributed/master_metadata_utility.h" #include "distributed/metadata_cache.h" #include "distributed/multi_executor.h" @@ -54,7 +55,7 @@ typedef struct NodeToNodeFragmentsTransfer /* forward declarations of local functions */ -static void WrapTasksForPartitioning(char *resultIdPrefix, List *selectTaskList, +static void WrapTasksForPartitioning(const char *resultIdPrefix, List *selectTaskList, int partitionColumnIndex, DistTableCacheEntry *targetRelation, bool binaryFormat); @@ -65,7 +66,7 @@ static ArrayType * CreateArrayFromDatums(Datum *datumArray, bool *nullsArray, in static void ShardMinMaxValueArrays(ShardInterval **shardIntervalArray, int shardCount, Oid intervalTypeId, ArrayType **minValueArray, ArrayType **maxValueArray); -static char * SourceShardPrefix(char *resultPrefix, uint64 shardId); +static char * SourceShardPrefix(const char *resultPrefix, uint64 shardId); static DistributedResultFragment * TupleToDistributedResultFragment( TupleTableSlot *tupleSlot, DistTableCacheEntry *targetRelation); static Tuplestorestate * ExecuteSelectTasksIntoTupleStore(List *taskList, @@ -96,7 +97,7 @@ static void ExecuteFetchTaskList(List *fetchTaskList); * partitioning. */ List ** -RedistributeTaskListResults(char *resultIdPrefix, List *selectTaskList, +RedistributeTaskListResults(const char *resultIdPrefix, List *selectTaskList, int partitionColumnIndex, DistTableCacheEntry *targetRelation, bool binaryFormat) @@ -131,7 +132,7 @@ RedistributeTaskListResults(char *resultIdPrefix, List *selectTaskList, * partitioning. */ List * -PartitionTasklistResults(char *resultIdPrefix, List *selectTaskList, +PartitionTasklistResults(const char *resultIdPrefix, List *selectTaskList, int partitionColumnIndex, DistTableCacheEntry *targetRelation, bool binaryFormat) @@ -165,12 +166,11 @@ PartitionTasklistResults(char *resultIdPrefix, List *selectTaskList, * match the tuple descriptor in ExecutePartitionTaskList(). */ static void -WrapTasksForPartitioning(char *resultIdPrefix, List *selectTaskList, +WrapTasksForPartitioning(const char *resultIdPrefix, List *selectTaskList, int partitionColumnIndex, DistTableCacheEntry *targetRelation, bool binaryFormat) { - ListCell *taskCell = NULL; ShardInterval **shardIntervalArray = targetRelation->sortedShardIntervalArray; int shardCount = targetRelation->shardIntervalArrayLength; @@ -193,9 +193,9 @@ WrapTasksForPartitioning(char *resultIdPrefix, List *selectTaskList, StringInfo maxValuesString = ArrayObjectToString(maxValueArray, TEXTOID, intervalTypeMod); - foreach(taskCell, selectTaskList) + Task *selectTask = NULL; + foreach_ptr(selectTask, selectTaskList) { - Task *selectTask = (Task *) lfirst(taskCell); List *shardPlacementList = selectTask->taskPlacementList; char *taskPrefix = SourceShardPrefix(resultIdPrefix, selectTask->anchorShardId); char *partitionMethodString = targetRelation->partitionMethod == 'h' ? @@ -208,10 +208,9 @@ WrapTasksForPartitioning(char *resultIdPrefix, List *selectTaskList, * so we form a different query per placement, each of which returning * the node id of the placement. */ - ListCell *placementCell = NULL; - foreach(placementCell, shardPlacementList) + ShardPlacement *shardPlacement = NULL; + foreach_ptr(shardPlacement, shardPlacementList) { - ShardPlacement *shardPlacement = lfirst(placementCell); StringInfo wrappedQuery = makeStringInfo(); appendStringInfo(wrappedQuery, "SELECT %u, partition_index" @@ -241,7 +240,7 @@ WrapTasksForPartitioning(char *resultIdPrefix, List *selectTaskList, * given anchor shard id. */ static char * -SourceShardPrefix(char *resultPrefix, uint64 shardId) +SourceShardPrefix(const char *resultPrefix, uint64 shardId) { StringInfo taskPrefix = makeStringInfo(); @@ -440,10 +439,9 @@ ColocateFragmentsWithRelation(List *fragmentList, DistTableCacheEntry *targetRel int shardCount = targetRelation->shardIntervalArrayLength; List **shardResultIdList = palloc0(shardCount * sizeof(List *)); - ListCell *fragmentCell = NULL; - foreach(fragmentCell, fragmentList) + DistributedResultFragment *sourceFragment = NULL; + foreach_ptr(sourceFragment, fragmentList) { - DistributedResultFragment *sourceFragment = lfirst(fragmentCell); int shardIndex = sourceFragment->targetShardIndex; shardResultIdList[shardIndex] = lappend(shardResultIdList[shardIndex], @@ -470,16 +468,13 @@ ColocationTransfers(List *fragmentList, DistTableCacheEntry *targetRelation) HTAB *transferHash = hash_create("Fragment Transfer Hash", 32, &transferHashInfo, HASH_ELEM | HASH_CONTEXT | HASH_BLOBS); - ListCell *fragmentCell = NULL; - foreach(fragmentCell, fragmentList) + DistributedResultFragment *fragment = NULL; + foreach_ptr(fragment, fragmentList) { - DistributedResultFragment *fragment = lfirst(fragmentCell); List *placementList = ActiveShardPlacementList(fragment->targetShardId); - - ListCell *placementCell = NULL; - foreach(placementCell, placementList) + ShardPlacement *placement = NULL; + foreach_ptr(placement, placementList) { - ShardPlacement *placement = lfirst(placementCell); NodePair transferKey = { .sourceNodeId = fragment->nodeId, .targetNodeId = placement->nodeId @@ -528,12 +523,10 @@ static List * FragmentTransferTaskList(List *fragmentListTransfers) { List *fetchTaskList = NIL; - ListCell *transferCell = NULL; - foreach(transferCell, fragmentListTransfers) + NodeToNodeFragmentsTransfer *fragmentsTransfer = NULL; + foreach_ptr(fragmentsTransfer, fragmentListTransfers) { - NodeToNodeFragmentsTransfer *fragmentsTransfer = lfirst(transferCell); - uint32 targetNodeId = fragmentsTransfer->nodes.targetNodeId; /* these should have already been pruned away in ColocationTransfers */ @@ -566,7 +559,6 @@ FragmentTransferTaskList(List *fragmentListTransfers) static char * QueryStringForFragmentsTransfer(NodeToNodeFragmentsTransfer *fragmentsTransfer) { - ListCell *fragmentCell = NULL; StringInfo queryString = makeStringInfo(); StringInfo fragmentNamesArrayString = makeStringInfo(); int fragmentCount = 0; @@ -575,10 +567,10 @@ QueryStringForFragmentsTransfer(NodeToNodeFragmentsTransfer *fragmentsTransfer) appendStringInfoString(fragmentNamesArrayString, "ARRAY["); - foreach(fragmentCell, fragmentsTransfer->fragmentList) + DistributedResultFragment *fragment = NULL; + foreach_ptr(fragment, fragmentsTransfer->fragmentList) { - DistributedResultFragment *fragment = lfirst(fragmentCell); - char *fragmentName = fragment->resultId; + const char *fragmentName = fragment->resultId; if (fragmentCount > 0) { diff --git a/src/backend/distributed/executor/insert_select_executor.c b/src/backend/distributed/executor/insert_select_executor.c index a1b3928f0..ba0cc8db8 100644 --- a/src/backend/distributed/executor/insert_select_executor.c +++ b/src/backend/distributed/executor/insert_select_executor.c @@ -300,7 +300,6 @@ CoordinatorInsertSelectExecScanInternal(CustomScanState *node) * distributed INSERT...SELECT from a set of intermediate results * to the target relation. */ - ListCell *taskCell = NULL; List *prunedTaskList = NIL; shardStateHash = ExecutePlanIntoColocatedIntermediateResults( @@ -321,9 +320,9 @@ CoordinatorInsertSelectExecScanInternal(CustomScanState *node) * written to them. Prune those tasks out by only including tasks * on shards with connections. */ - foreach(taskCell, taskList) + Task *task = NULL; + foreach_ptr(task, taskList) { - Task *task = (Task *) lfirst(taskCell); uint64 shardId = task->anchorShardId; bool shardModified = false; @@ -417,7 +416,6 @@ static Query * WrapSubquery(Query *subquery) { ParseState *pstate = make_parsestate(NULL); - ListCell *selectTargetCell = NULL; List *newTargetList = NIL; Query *outerQuery = makeNode(Query); @@ -436,10 +434,9 @@ WrapSubquery(Query *subquery) outerQuery->jointree = makeFromExpr(list_make1(newRangeTableRef), NULL); /* create a target list that matches the SELECT */ - foreach(selectTargetCell, subquery->targetList) + TargetEntry *selectTargetEntry = NULL; + foreach_ptr(selectTargetEntry, subquery->targetList) { - TargetEntry *selectTargetEntry = (TargetEntry *) lfirst(selectTargetCell); - /* exactly 1 entry in FROM */ int indexInRangeTable = 1; @@ -648,14 +645,12 @@ ExecutePlanIntoRelation(Oid targetRelationId, List *insertTargetList, static List * BuildColumnNameListFromTargetList(Oid targetRelationId, List *insertTargetList) { - ListCell *insertTargetCell = NULL; List *columnNameList = NIL; /* build the list of column names for the COPY statement */ - foreach(insertTargetCell, insertTargetList) + TargetEntry *insertTargetEntry = NULL; + foreach_ptr(insertTargetEntry, insertTargetList) { - TargetEntry *insertTargetEntry = (TargetEntry *) lfirst(insertTargetCell); - columnNameList = lappend(columnNameList, insertTargetEntry->resname); } @@ -671,14 +666,12 @@ BuildColumnNameListFromTargetList(Oid targetRelationId, List *insertTargetList) static int PartitionColumnIndexFromColumnList(Oid relationId, List *columnNameList) { - ListCell *columnNameCell = NULL; Var *partitionColumn = PartitionColumn(relationId, 0); int partitionColumnIndex = 0; - foreach(columnNameCell, columnNameList) + const char *columnName = NULL; + foreach_ptr(columnName, columnNameList) { - char *columnName = (char *) lfirst(columnNameCell); - AttrNumber attrNumber = get_attnum(relationId, columnName); /* check whether this is the partition column */ @@ -804,10 +797,10 @@ AddInsertSelectCasts(List *insertTargetList, List *selectTargetList, /* selectEntry->resno must be the ordinal number of the entry */ selectTargetList = list_concat(projectedEntries, nonProjectedEntries); int entryResNo = 1; - foreach(selectEntryCell, selectTargetList) + TargetEntry *selectTargetEntry = NULL; + foreach_ptr(selectTargetEntry, selectTargetList) { - TargetEntry *selectEntry = lfirst(selectEntryCell); - selectEntry->resno = entryResNo++; + selectTargetEntry->resno = entryResNo++; } heap_close(distributedRelation, NoLock); diff --git a/src/backend/distributed/executor/intermediate_results.c b/src/backend/distributed/executor/intermediate_results.c index eadccdeec..2cc875d5d 100644 --- a/src/backend/distributed/executor/intermediate_results.c +++ b/src/backend/distributed/executor/intermediate_results.c @@ -21,6 +21,7 @@ #include "distributed/commands/multi_copy.h" #include "distributed/connection_management.h" #include "distributed/intermediate_results.h" +#include "distributed/listutils.h" #include "distributed/master_metadata_utility.h" #include "distributed/metadata_cache.h" #include "distributed/multi_client_executor.h" @@ -51,7 +52,7 @@ typedef struct RemoteFileDestReceiver /* public DestReceiver interface */ DestReceiver pub; - char *resultId; + const char *resultId; /* descriptor of the tuples that are sent to the worker */ TupleDesc tupleDescriptor; @@ -197,7 +198,7 @@ create_intermediate_result(PG_FUNCTION_ARGS) * coordinated transaction is started prior to using the DestReceiver. */ DestReceiver * -CreateRemoteFileDestReceiver(char *resultId, EState *executorState, +CreateRemoteFileDestReceiver(const char *resultId, EState *executorState, List *initialNodeList, bool writeLocalFile) { RemoteFileDestReceiver *resultDest = (RemoteFileDestReceiver *) palloc0( @@ -238,9 +239,7 @@ RemoteFileDestReceiverStartup(DestReceiver *dest, int operation, const char *nullPrintCharacter = "\\N"; List *initialNodeList = resultDest->initialNodeList; - ListCell *initialNodeCell = NULL; List *connectionList = NIL; - ListCell *connectionCell = NULL; resultDest->tupleDescriptor = inputTupleDescriptor; @@ -272,10 +271,10 @@ RemoteFileDestReceiverStartup(DestReceiver *dest, int operation, fileMode)); } - foreach(initialNodeCell, initialNodeList) + WorkerNode *workerNode = NULL; + foreach_ptr(workerNode, initialNodeList) { - WorkerNode *workerNode = (WorkerNode *) lfirst(initialNodeCell); - char *nodeName = workerNode->workerName; + const char *nodeName = workerNode->workerName; int nodePort = workerNode->workerPort; /* @@ -298,10 +297,9 @@ RemoteFileDestReceiverStartup(DestReceiver *dest, int operation, /* must open transaction blocks to use intermediate results */ RemoteTransactionsBeginIfNecessary(connectionList); - foreach(connectionCell, connectionList) + MultiConnection *connection = NULL; + foreach_ptr(connection, connectionList) { - MultiConnection *connection = (MultiConnection *) lfirst(connectionCell); - StringInfo copyCommand = ConstructCopyResultStatement(resultId); bool querySent = SendRemoteCommand(connection, copyCommand->data); @@ -311,9 +309,8 @@ RemoteFileDestReceiverStartup(DestReceiver *dest, int operation, } } - foreach(connectionCell, connectionList) + foreach_ptr(connection, connectionList) { - MultiConnection *connection = (MultiConnection *) lfirst(connectionCell); bool raiseInterrupts = true; PGresult *result = GetRemoteCommandResult(connection, raiseInterrupts); @@ -469,10 +466,9 @@ RemoteFileDestReceiverShutdown(DestReceiver *destReceiver) static void BroadcastCopyData(StringInfo dataBuffer, List *connectionList) { - ListCell *connectionCell = NULL; - foreach(connectionCell, connectionList) + MultiConnection *connection = NULL; + foreach_ptr(connection, connectionList) { - MultiConnection *connection = (MultiConnection *) lfirst(connectionCell); SendCopyDataOverConnection(dataBuffer, connection); } } @@ -678,7 +674,7 @@ RemoveIntermediateResultsDirectory(void) * or -1 if the file does not exist. */ int64 -IntermediateResultSize(char *resultId) +IntermediateResultSize(const char *resultId) { struct stat fileStat; diff --git a/src/backend/distributed/executor/local_executor.c b/src/backend/distributed/executor/local_executor.c index 2006a6261..c520cd6d4 100644 --- a/src/backend/distributed/executor/local_executor.c +++ b/src/backend/distributed/executor/local_executor.c @@ -76,6 +76,7 @@ #include "distributed/citus_custom_scan.h" #include "distributed/citus_ruleutils.h" #include "distributed/deparse_shard_query.h" +#include "distributed/listutils.h" #include "distributed/local_executor.h" #include "distributed/multi_executor.h" #include "distributed/master_protocol.h" @@ -129,7 +130,6 @@ ExecuteLocalTaskList(CitusScanState *scanState, List *taskList) ParamListInfo paramListInfo = copyParamList(executorState->es_param_list_info); int numParams = 0; Oid *parameterTypes = NULL; - ListCell *taskCell = NULL; uint64 totalRowsProcessed = 0; if (paramListInfo != NULL) @@ -143,10 +143,9 @@ ExecuteLocalTaskList(CitusScanState *scanState, List *taskList) numParams = paramListInfo->numParams; } - foreach(taskCell, taskList) + Task *task = NULL; + foreach_ptr(task, taskList) { - Task *task = (Task *) lfirst(taskCell); - /* * If we have a valid shard id, a distributed table will be accessed * during execution. Record it to apply the restrictions related to @@ -171,10 +170,10 @@ ExecuteLocalTaskList(CitusScanState *scanState, List *taskList) RowShareLock : AccessShareLock); - ListCell *oidCell = NULL; - foreach(oidCell, localPlan->relationOids) + Oid relationId = InvalidOid; + foreach_oid(relationId, localPlan->relationOids) { - LockRelationOid(lfirst_oid(oidCell), lockMode); + LockRelationOid(relationId, lockMode); } } else @@ -264,15 +263,12 @@ void ExtractLocalAndRemoteTasks(bool readOnly, List *taskList, List **localTaskList, List **remoteTaskList) { - ListCell *taskCell = NULL; - *remoteTaskList = NIL; *localTaskList = NIL; - foreach(taskCell, taskList) + Task *task = NULL; + foreach_ptr(task, taskList) { - Task *task = (Task *) lfirst(taskCell); - List *localTaskPlacementList = NULL; List *remoteTaskPlacementList = NULL; @@ -338,17 +334,14 @@ static void SplitLocalAndRemotePlacements(List *taskPlacementList, List **localTaskPlacementList, List **remoteTaskPlacementList) { - ListCell *placementCell = NULL; int32 localGroupId = GetLocalGroupId(); *localTaskPlacementList = NIL; *remoteTaskPlacementList = NIL; - foreach(placementCell, taskPlacementList) + ShardPlacement *taskPlacement = NULL; + foreach_ptr(taskPlacement, taskPlacementList) { - ShardPlacement *taskPlacement = - (ShardPlacement *) lfirst(placementCell); - if (taskPlacement->groupId == localGroupId) { *localTaskPlacementList = lappend(*localTaskPlacementList, taskPlacement); @@ -494,13 +487,11 @@ ShouldExecuteTasksLocally(List *taskList) bool TaskAccessesLocalNode(Task *task) { - ListCell *placementCell = NULL; int localGroupId = GetLocalGroupId(); - foreach(placementCell, task->taskPlacementList) + ShardPlacement *taskPlacement = NULL; + foreach_ptr(taskPlacement, task->taskPlacementList) { - ShardPlacement *taskPlacement = (ShardPlacement *) lfirst(placementCell); - if (taskPlacement->groupId == localGroupId) { return true; diff --git a/src/backend/distributed/executor/multi_executor.c b/src/backend/distributed/executor/multi_executor.c index 567b2bb6f..59826fe64 100644 --- a/src/backend/distributed/executor/multi_executor.c +++ b/src/backend/distributed/executor/multi_executor.c @@ -386,7 +386,6 @@ void LoadTuplesIntoTupleStore(CitusScanState *citusScanState, Job *workerJob) { List *workerTaskList = workerJob->taskList; - ListCell *workerTaskCell = NULL; bool randomAccess = true; bool interTransactions = false; char *copyFormat = "text"; @@ -402,10 +401,9 @@ LoadTuplesIntoTupleStore(CitusScanState *citusScanState, Job *workerJob) copyFormat = "binary"; } - foreach(workerTaskCell, workerTaskList) + Task *workerTask = NULL; + foreach_ptr(workerTask, workerTaskList) { - Task *workerTask = (Task *) lfirst(workerTaskCell); - StringInfo jobDirectoryName = MasterJobDirectoryName(workerTask->jobId); StringInfo taskFilename = TaskFilename(jobDirectoryName, workerTask->taskId); @@ -496,17 +494,15 @@ SortTupleStore(CitusScanState *scanState) Oid *collations = (Oid *) palloc(numberOfSortKeys * sizeof(Oid)); bool *nullsFirst = (bool *) palloc(numberOfSortKeys * sizeof(bool)); - ListCell *targetCell = NULL; int sortKeyIndex = 0; - /* * Iterate on the returning target list and generate the necessary information * for sorting the tuples. */ - foreach(targetCell, targetList) + TargetEntry *returningEntry = NULL; + foreach_ptr(returningEntry, targetList) { - TargetEntry *returningEntry = (TargetEntry *) lfirst(targetCell); Oid sortop = InvalidOid; /* determine the sortop, we don't need anything else */ @@ -740,7 +736,6 @@ IsLocalReferenceTableJoinPlan(PlannedStmt *plan) { bool hasReferenceTable = false; bool hasLocalTable = false; - ListCell *rangeTableCell = NULL; bool hasReferenceTableReplica = false; /* @@ -784,9 +779,9 @@ IsLocalReferenceTableJoinPlan(PlannedStmt *plan) * It doesn't contain optimized away table accesses (due to join optimization), * which is fine for our purpose. */ - foreach(rangeTableCell, plan->rtable) + RangeTblEntry *rangeTableEntry = NULL; + foreach_ptr(rangeTableEntry, plan->rtable) { - RangeTblEntry *rangeTableEntry = (RangeTblEntry *) lfirst(rangeTableCell); bool onlySearchPath = false; /* diff --git a/src/backend/distributed/executor/multi_server_executor.c b/src/backend/distributed/executor/multi_server_executor.c index 3758ff476..77527e4d0 100644 --- a/src/backend/distributed/executor/multi_server_executor.c +++ b/src/backend/distributed/executor/multi_server_executor.c @@ -19,6 +19,8 @@ #include +#include "distributed/listutils.h" +#include "distributed/log_utils.h" #include "distributed/multi_client_executor.h" #include "distributed/multi_executor.h" #include "distributed/multi_physical_planner.h" @@ -27,7 +29,6 @@ #include "distributed/master_protocol.h" #include "distributed/subplan_execution.h" #include "distributed/worker_protocol.h" -#include "distributed/log_utils.h" #include "utils/lsyscache.h" int RemoteTaskCheckInterval = 100; /* per cycle sleep interval in millisecs */ @@ -143,11 +144,9 @@ JobExecutorType(DistributedPlan *distributedPlan) static bool HasReplicatedDistributedTable(List *relationOids) { - ListCell *oidCell = NULL; - - foreach(oidCell, relationOids) + Oid oid; + foreach_oid(oid, relationOids) { - Oid oid = lfirst_oid(oidCell); char partitionMethod = PartitionMethod(oid); if (partitionMethod == DISTRIBUTE_BY_NONE) { diff --git a/src/backend/distributed/executor/multi_task_tracker_executor.c b/src/backend/distributed/executor/multi_task_tracker_executor.c index 77da0a6e0..5dd0db6ea 100644 --- a/src/backend/distributed/executor/multi_task_tracker_executor.c +++ b/src/backend/distributed/executor/multi_task_tracker_executor.c @@ -30,6 +30,7 @@ #include "distributed/connection_management.h" #include "distributed/deparse_shard_query.h" #include "distributed/distributed_execution_locks.h" +#include "distributed/listutils.h" #include "distributed/local_executor.h" #include "distributed/metadata_cache.h" #include "distributed/multi_client_executor.h" @@ -160,7 +161,6 @@ void MultiTaskTrackerExecute(Job *job) { List *jobTaskList = job->taskList; - ListCell *taskAndExecutionCell = NULL; uint32 topLevelTaskCount = 0; uint32 failedTaskId = 0; bool allTasksCompleted = false; @@ -191,10 +191,9 @@ MultiTaskTrackerExecute(Job *job) * We now count the number of "top level" tasks in the query tree. Once they * complete, we'll need to fetch these tasks' results to the master node. */ - foreach(taskAndExecutionCell, taskAndExecutionList) + Task *task = NULL; + foreach_ptr(task, taskAndExecutionList) { - Task *task = (Task *) lfirst(taskAndExecutionCell); - bool topLevelTask = TopLevelTask(task); if (topLevelTask) { @@ -235,9 +234,8 @@ MultiTaskTrackerExecute(Job *job) double acceptableHealthyTrackerCount = 0.0; /* first, loop around all tasks and manage them */ - foreach(taskAndExecutionCell, taskAndExecutionList) + foreach_ptr(task, taskAndExecutionList) { - Task *task = (Task *) lfirst(taskAndExecutionCell); TaskExecution *taskExecution = task->taskExecution; TaskTracker *execTaskTracker = ResolveTaskTracker(taskTrackerHash, @@ -300,13 +298,10 @@ MultiTaskTrackerExecute(Job *job) } /* second, loop around "top level" tasks to fetch their results */ - taskAndExecutionCell = NULL; - foreach(taskAndExecutionCell, taskAndExecutionList) + foreach_ptr(task, taskAndExecutionList) { - Task *task = (Task *) lfirst(taskAndExecutionCell); TaskExecution *taskExecution = task->taskExecution; - /* * We find the tasks that appear in the top level of the query tree, * and start fetching their results to the master node. @@ -471,8 +466,6 @@ TaskAndExecutionList(List *jobTaskList) List *taskQueue = list_copy(jobTaskList); while (taskQueue != NIL) { - ListCell *dependentTaskCell = NULL; - /* pop first element from the task queue */ Task *task = (Task *) linitial(taskQueue); taskQueue = list_delete_first(taskQueue); @@ -501,6 +494,7 @@ TaskAndExecutionList(List *jobTaskList) * taskHash is used to reduce the complexity of keeping track of * the tasks that are already encountered. */ + ListCell *dependentTaskCell = NULL; foreach(dependentTaskCell, dependendTaskList) { Task *dependendTask = lfirst(dependentTaskCell); @@ -685,10 +679,9 @@ TrackerHash(const char *taskTrackerHashName, List *workerNodeList, char *userNam uint32 taskTrackerHashSize = list_length(workerNodeList); HTAB *taskTrackerHash = TrackerHashCreate(taskTrackerHashName, taskTrackerHashSize); - ListCell *workerNodeCell = NULL; - foreach(workerNodeCell, workerNodeList) + WorkerNode *workerNode = NULL; + foreach_ptr(workerNode, workerNodeList) { - WorkerNode *workerNode = (WorkerNode *) lfirst(workerNodeCell); char *nodeName = workerNode->workerName; uint32 nodePort = workerNode->workerPort; @@ -1505,11 +1498,10 @@ static bool TaskExecutionsCompleted(List *taskList) { bool taskExecutionsComplete = true; - ListCell *taskCell = NULL; - foreach(taskCell, taskList) + Task *task = NULL; + foreach_ptr(task, taskList) { - Task *task = (Task *) lfirst(taskCell); TaskExecution *taskExecution = task->taskExecution; uint32 nodeIndex = taskExecution->currentNodeIndex; @@ -1791,8 +1783,6 @@ static List * ConstrainedTaskList(List *taskAndExecutionList, Task *task) { List *constrainedTaskList = NIL; - ListCell *mergeTaskCell = NULL; - ListCell *upstreamTaskCell = NULL; /* * We first check if this task depends on any merge tasks. If it does *not*, @@ -1807,9 +1797,9 @@ ConstrainedTaskList(List *taskAndExecutionList, Task *task) } /* we first add merge tasks and their dependencies to our constraint group */ - foreach(mergeTaskCell, mergeTaskList) + Task *mergeTask = NULL; + foreach_ptr(mergeTask, mergeTaskList) { - Task *mergeTask = (Task *) lfirst(mergeTaskCell); List *dependentTaskList = mergeTask->dependentTaskList; constrainedTaskList = lappend(constrainedTaskList, mergeTask); @@ -1830,9 +1820,9 @@ ConstrainedTaskList(List *taskAndExecutionList, Task *task) constrainingTask); Assert(upstreamTaskList != NIL); - foreach(upstreamTaskCell, upstreamTaskList) + Task *upstreamTask = NULL; + foreach_ptr(upstreamTask, upstreamTaskList) { - Task *upstreamTask = (Task *) lfirst(upstreamTaskCell); List *dependentTaskList = upstreamTask->dependentTaskList; /* @@ -1884,13 +1874,11 @@ static List * UpstreamDependencyList(List *taskAndExecutionList, Task *searchedTask) { List *upstreamTaskList = NIL; - ListCell *taskAndExecutionCell = NULL; - foreach(taskAndExecutionCell, taskAndExecutionList) + Task *upstreamTask = NULL; + foreach_ptr(upstreamTask, taskAndExecutionList) { - Task *upstreamTask = (Task *) lfirst(taskAndExecutionCell); List *dependentTaskList = upstreamTask->dependentTaskList; - ListCell *dependentTaskCell = NULL; /* * The given task and its upstream dependency cannot be of the same type. @@ -1906,9 +1894,9 @@ UpstreamDependencyList(List *taskAndExecutionList, Task *searchedTask) * We walk over the upstream task's dependency list, and check if any of * them is the task we are looking for. */ - foreach(dependentTaskCell, dependentTaskList) + Task *dependentTask = NULL; + foreach_ptr(dependentTask, dependentTaskList) { - Task *dependentTask = (Task *) lfirst(dependentTaskCell); if (TasksEqual(dependentTask, searchedTask)) { upstreamTaskList = lappend(upstreamTaskList, upstreamTask); @@ -1983,11 +1971,10 @@ static List * MergeTaskList(List *taskList) { List *mergeTaskList = NIL; - ListCell *taskCell = NULL; - foreach(taskCell, taskList) + Task *task = NULL; + foreach_ptr(task, taskList) { - Task *task = (Task *) lfirst(taskCell); if (task->taskType == MERGE_TASK) { mergeTaskList = lappend(mergeTaskList, task); @@ -2008,15 +1995,14 @@ static void ReassignTaskList(List *taskList) { List *completedTaskList = NIL; - ListCell *taskCell = NULL; /* * As an optimization, we first find the SQL tasks whose results we already * fetched to the master node. We don't need to re-execute these SQL tasks. */ - foreach(taskCell, taskList) + Task *task = NULL; + foreach_ptr(task, taskList) { - Task *task = (Task *) lfirst(taskCell); TaskExecution *taskExecution = task->taskExecution; bool transmitCompleted = TransmitExecutionCompleted(taskExecution); @@ -2028,10 +2014,8 @@ ReassignTaskList(List *taskList) taskList = TaskListDifference(taskList, completedTaskList); - taskCell = NULL; - foreach(taskCell, taskList) + foreach_ptr(task, taskList) { - Task *task = (Task *) lfirst(taskCell); TaskExecution *taskExecution = task->taskExecution; uint32 currentNodeIndex = taskExecution->currentNodeIndex; @@ -2059,10 +2043,9 @@ ReassignTaskList(List *taskList) static void ReassignMapFetchTaskList(List *mapFetchTaskList) { - ListCell *mapFetchTaskCell = NULL; - foreach(mapFetchTaskCell, mapFetchTaskList) + Task *mapFetchTask = NULL; + foreach_ptr(mapFetchTask, mapFetchTaskList) { - Task *mapFetchTask = (Task *) lfirst(mapFetchTaskCell); TaskExecution *mapFetchTaskExecution = mapFetchTask->taskExecution; TaskExecStatus *taskStatusArray = mapFetchTaskExecution->taskStatusArray; @@ -2288,15 +2271,12 @@ AssignQueuedTasks(TaskTracker *taskTracker) void *queryResult = NULL; int rowCount = 0; int columnCount = 0; - ListCell *taskCell = NULL; bool batchSuccess = MultiClientSendQuery(connectionId, assignTaskBatchQuery->data); - foreach(taskCell, tasksToAssignList) + foreach_ptr(taskState, tasksToAssignList) { - taskState = (TrackerTaskState *) lfirst(taskCell); - if (!batchSuccess) { taskState->status = TASK_CLIENT_SIDE_ASSIGN_FAILED; @@ -2349,7 +2329,6 @@ TaskStatusBatchList(TaskTracker *taskTracker) int32 assignedTaskIndex = 0; List *assignedTaskList = taskTracker->assignedTaskList; List *taskStatusBatchList = NIL; - ListCell *taskCell = NULL; int32 assignedTaskCount = list_length(assignedTaskList); if (assignedTaskCount == 0) @@ -2364,9 +2343,9 @@ TaskStatusBatchList(TaskTracker *taskTracker) currentTaskIndex = -1; } - foreach(taskCell, assignedTaskList) + TrackerTaskState *assignedTask = NULL; + foreach_ptr(assignedTask, assignedTaskList) { - TrackerTaskState *assignedTask = (TrackerTaskState *) lfirst(taskCell); TaskStatus taskStatus = assignedTask->status; bool taskRunning = false; @@ -2403,12 +2382,10 @@ static StringInfo TaskStatusBatchQuery(List *taskList) { StringInfo taskStatusBatchQuery = makeStringInfo(); - ListCell *taskCell = NULL; - foreach(taskCell, taskList) + TrackerTaskState *taskState = NULL; + foreach_ptr(taskState, taskList) { - TrackerTaskState *taskState = (TrackerTaskState *) lfirst(taskCell); - appendStringInfo(taskStatusBatchQuery, TASK_STATUS_QUERY, taskState->jobId, taskState->taskId); } @@ -2427,16 +2404,15 @@ TaskStatusBatchQuery(List *taskList) static void ReceiveTaskStatusBatchQueryResponse(TaskTracker *taskTracker) { - ListCell *taskCell = NULL; List *checkedTaskList = taskTracker->connectionBusyOnTaskList; int32 connectionId = taskTracker->connectionId; int rowCount = 0; int columnCount = 0; void *queryResult = NULL; - foreach(taskCell, checkedTaskList) + TrackerTaskState *checkedTask = NULL; + foreach_ptr(checkedTask, checkedTaskList) { - TrackerTaskState *checkedTask = (TrackerTaskState *) lfirst(taskCell); TaskStatus taskStatus = TASK_STATUS_INVALID_FIRST; BatchQueryStatus queryStatus = MultiClientBatchResult(connectionId, &queryResult, @@ -2621,9 +2597,6 @@ static void TrackerCleanupResources(HTAB *taskTrackerHash, HTAB *transmitTrackerHash, List *jobIdList, List *taskList) { - ListCell *taskCell = NULL; - ListCell *jobIdCell = NULL; - /* * We are done with query execution. We now wait for open requests to the task * trackers to complete and cancel any open requests to the transmit trackers. @@ -2632,9 +2605,9 @@ TrackerCleanupResources(HTAB *taskTrackerHash, HTAB *transmitTrackerHash, TrackerHashCancelActiveRequest(transmitTrackerHash); /* only close open files; open connections are owned by trackers */ - foreach(taskCell, taskList) + Task *task = NULL; + foreach_ptr(task, taskList) { - Task *task = (Task *) lfirst(taskCell); TaskExecution *taskExecution = task->taskExecution; CleanupTaskExecution(taskExecution); @@ -2645,10 +2618,9 @@ TrackerCleanupResources(HTAB *taskTrackerHash, HTAB *transmitTrackerHash, * For each executed job, we create a special task to clean up its resources * on worker nodes, and send this clean-up task to all task trackers. */ - foreach(jobIdCell, jobIdList) + uint64 *jobIdPointer = NULL; + foreach_ptr(jobIdPointer, jobIdList) { - uint64 *jobIdPointer = (uint64 *) lfirst(jobIdCell); - Task *jobCleanupTask = JobCleanupTask(*jobIdPointer); TrackerHashCleanupJob(taskTrackerHash, jobCleanupTask); } @@ -2814,7 +2786,6 @@ TrackerHashCleanupJob(HTAB *taskTrackerHash, Task *jobCleanupTask) while (list_length(remainingTaskTrackerList) > 0 && !timedOut) { List *activeTackTrackerList = remainingTaskTrackerList; - ListCell *activeTaskTrackerCell = NULL; remainingTaskTrackerList = NIL; @@ -2823,9 +2794,8 @@ TrackerHashCleanupJob(HTAB *taskTrackerHash, Task *jobCleanupTask) timedOut = TimestampDifferenceExceeds(startTime, currentTime, NodeConnectionTimeout); - foreach(activeTaskTrackerCell, activeTackTrackerList) + foreach_ptr(taskTracker, activeTackTrackerList) { - taskTracker = (TaskTracker *) lfirst(activeTaskTrackerCell); int32 connectionId = taskTracker->connectionId; char *nodeName = taskTracker->workerName; uint32 nodePort = taskTracker->workerPort; diff --git a/src/backend/distributed/executor/placement_access.c b/src/backend/distributed/executor/placement_access.c index c3cab3f01..557dbcad6 100644 --- a/src/backend/distributed/executor/placement_access.c +++ b/src/backend/distributed/executor/placement_access.c @@ -9,6 +9,7 @@ *------------------------------------------------------------------------- */ #include "distributed/placement_access.h" +#include "distributed/listutils.h" #include "distributed/metadata_cache.h" static List * BuildPlacementSelectList(int32 groupId, List *relationShardList); @@ -122,13 +123,11 @@ static List * BuildPlacementAccessList(int32 groupId, List *relationShardList, ShardPlacementAccessType accessType) { - ListCell *relationShardCell = NULL; List *placementAccessList = NIL; - foreach(relationShardCell, relationShardList) + RelationShard *relationShard = NULL; + foreach_ptr(relationShard, relationShardList) { - RelationShard *relationShard = (RelationShard *) lfirst(relationShardCell); - ShardPlacement *placement = FindShardPlacementOnGroup(groupId, relationShard->shardId); if (placement == NULL) diff --git a/src/backend/distributed/executor/repartition_join_execution.c b/src/backend/distributed/executor/repartition_join_execution.c index 6202dd1e2..e170a8d50 100644 --- a/src/backend/distributed/executor/repartition_join_execution.c +++ b/src/backend/distributed/executor/repartition_join_execution.c @@ -29,21 +29,18 @@ #include "utils/builtins.h" #include "distributed/hash_helpers.h" -#include "distributed/directed_acyclic_graph_execution.h" -#include "distributed/multi_physical_planner.h" #include "distributed/adaptive_executor.h" -#include "distributed/worker_manager.h" -#include "distributed/metadata_cache.h" -#include "distributed/multi_server_executor.h" -#include "distributed/repartition_join_execution.h" -#include "distributed/worker_transaction.h" -#include "distributed/worker_manager.h" -#include "distributed/transaction_management.h" -#include "distributed/multi_task_tracker_executor.h" -#include "distributed/worker_transaction.h" -#include "distributed/metadata_cache.h" +#include "distributed/directed_acyclic_graph_execution.h" #include "distributed/listutils.h" +#include "distributed/metadata_cache.h" +#include "distributed/multi_physical_planner.h" +#include "distributed/multi_server_executor.h" +#include "distributed/multi_task_tracker_executor.h" +#include "distributed/repartition_join_execution.h" +#include "distributed/transaction_management.h" #include "distributed/transmit.h" +#include "distributed/worker_manager.h" +#include "distributed/worker_transaction.h" static List * CreateTemporarySchemasForMergeTasks(Job *topLevelJob); @@ -109,12 +106,11 @@ ExtractJobsInJobTree(Job *job) static void TraverseJobTree(Job *curJob, List **jobIds) { - ListCell *jobCell = NULL; *jobIds = lappend(*jobIds, (void *) curJob->jobId); - foreach(jobCell, curJob->dependentJobList) + Job *childJob = NULL; + foreach_ptr(childJob, curJob->dependentJobList) { - Job *childJob = (Job *) lfirst(jobCell); TraverseJobTree(childJob, jobIds); } } @@ -127,11 +123,11 @@ static char * GenerateCreateSchemasCommand(List *jobIds, char *ownerName) { StringInfo createSchemaCommand = makeStringInfo(); - ListCell *jobIdCell = NULL; - foreach(jobIdCell, jobIds) + void *jobIdPointer = NULL; + foreach_ptr(jobIdPointer, jobIds) { - uint64 jobId = (uint64) lfirst(jobIdCell); + uint64 jobId = (uint64) jobIdPointer; appendStringInfo(createSchemaCommand, WORKER_CREATE_SCHEMA_QUERY, jobId, quote_literal_cstr(ownerName)); } @@ -150,11 +146,11 @@ static char * GenerateJobCommands(List *jobIds, char *templateCommand) { StringInfo createSchemaCommand = makeStringInfo(); - ListCell *jobIdCell = NULL; - foreach(jobIdCell, jobIds) + void *jobIdPointer = NULL; + foreach_ptr(jobIdPointer, jobIds) { - uint64 jobId = (uint64) lfirst(jobIdCell); + uint64 jobId = (uint64) jobIdPointer; appendStringInfo(createSchemaCommand, templateCommand, jobId); } return createSchemaCommand->data; diff --git a/src/backend/distributed/executor/subplan_execution.c b/src/backend/distributed/executor/subplan_execution.c index 93a1087a5..b82667368 100644 --- a/src/backend/distributed/executor/subplan_execution.c +++ b/src/backend/distributed/executor/subplan_execution.c @@ -12,6 +12,7 @@ #include "distributed/intermediate_result_pruning.h" #include "distributed/intermediate_results.h" +#include "distributed/listutils.h" #include "distributed/multi_executor.h" #include "distributed/multi_physical_planner.h" #include "distributed/recursive_planning.h" @@ -35,7 +36,6 @@ ExecuteSubPlans(DistributedPlan *distributedPlan) { uint64 planId = distributedPlan->planId; List *subPlanList = distributedPlan->subPlanList; - ListCell *subPlanCell = NULL; if (subPlanList == NIL) { @@ -54,9 +54,9 @@ ExecuteSubPlans(DistributedPlan *distributedPlan) */ UseCoordinatedTransaction(); - foreach(subPlanCell, subPlanList) + DistributedSubPlan *subPlan = NULL; + foreach_ptr(subPlan, subPlanList) { - DistributedSubPlan *subPlan = (DistributedSubPlan *) lfirst(subPlanCell); PlannedStmt *plannedStmt = subPlan->plan; uint32 subPlanId = subPlan->subPlanId; ParamListInfo params = NULL; diff --git a/src/backend/distributed/master/citus_create_restore_point.c b/src/backend/distributed/master/citus_create_restore_point.c index babf84624..78d6f9e32 100644 --- a/src/backend/distributed/master/citus_create_restore_point.c +++ b/src/backend/distributed/master/citus_create_restore_point.c @@ -16,6 +16,7 @@ #include "access/xlog_internal.h" #include "catalog/pg_type.h" #include "distributed/connection_management.h" +#include "distributed/listutils.h" #include "distributed/master_metadata_utility.h" #include "distributed/metadata_cache.h" #include "distributed/remote_commands.h" @@ -114,15 +115,13 @@ static List * OpenConnectionsToAllWorkerNodes(LOCKMODE lockMode) { List *connectionList = NIL; - ListCell *workerNodeCell = NULL; int connectionFlags = FORCE_NEW_CONNECTION; List *workerNodeList = ActivePrimaryWorkerNodeList(lockMode); - foreach(workerNodeCell, workerNodeList) + WorkerNode *workerNode = NULL; + foreach_ptr(workerNode, workerNodeList) { - WorkerNode *workerNode = (WorkerNode *) lfirst(workerNodeCell); - MultiConnection *connection = StartNodeConnection(connectionFlags, workerNode->workerName, workerNode->workerPort); @@ -158,15 +157,13 @@ BlockDistributedTransactions(void) static void CreateRemoteRestorePoints(char *restoreName, List *connectionList) { - ListCell *connectionCell = NULL; int parameterCount = 1; Oid parameterTypes[1] = { TEXTOID }; const char *parameterValues[1] = { restoreName }; - foreach(connectionCell, connectionList) + MultiConnection *connection = NULL; + foreach_ptr(connection, connectionList) { - MultiConnection *connection = (MultiConnection *) lfirst(connectionCell); - int querySent = SendRemoteCommandParams(connection, CREATE_RESTORE_POINT_COMMAND, parameterCount, parameterTypes, parameterValues); @@ -176,10 +173,8 @@ CreateRemoteRestorePoints(char *restoreName, List *connectionList) } } - foreach(connectionCell, connectionList) + foreach_ptr(connection, connectionList) { - MultiConnection *connection = (MultiConnection *) lfirst(connectionCell); - PGresult *result = GetRemoteCommandResult(connection, true); if (!IsResponseOK(result)) { diff --git a/src/backend/distributed/master/master_create_shards.c b/src/backend/distributed/master/master_create_shards.c index 98e0975d8..37a158591 100644 --- a/src/backend/distributed/master/master_create_shards.c +++ b/src/backend/distributed/master/master_create_shards.c @@ -238,7 +238,6 @@ void CreateColocatedShards(Oid targetRelationId, Oid sourceRelationId, bool useExclusiveConnections) { - ListCell *sourceShardCell = NULL; bool colocatedShard = true; List *insertedShardPlacements = NIL; @@ -276,9 +275,9 @@ CreateColocatedShards(Oid targetRelationId, Oid sourceRelationId, bool char targetShardStorageType = ShardStorageType(targetRelationId); - foreach(sourceShardCell, sourceShardIntervalList) + ShardInterval *sourceShardInterval = NULL; + foreach_ptr(sourceShardInterval, sourceShardIntervalList) { - ShardInterval *sourceShardInterval = (ShardInterval *) lfirst(sourceShardCell); uint64 sourceShardId = sourceShardInterval->shardId; uint64 newShardId = GetNextShardId(); diff --git a/src/backend/distributed/master/master_delete_protocol.c b/src/backend/distributed/master/master_delete_protocol.c index 78f418a10..0d9e79b64 100644 --- a/src/backend/distributed/master/master_delete_protocol.c +++ b/src/backend/distributed/master/master_delete_protocol.c @@ -382,7 +382,6 @@ DropShards(Oid relationId, char *schemaName, char *relationName, } ShardInterval *shardInterval = NULL; - foreach_ptr(shardInterval, deletableShardIntervalList) { uint64 shardId = shardInterval->shardId; @@ -396,7 +395,6 @@ DropShards(Oid relationId, char *schemaName, char *relationName, List *shardPlacementList = ShardPlacementList(shardId); ShardPlacement *shardPlacement = NULL; - foreach_ptr(shardPlacement, shardPlacementList) { uint64 shardPlacementId = shardPlacement->placementId; @@ -559,13 +557,11 @@ CheckDeleteCriteria(Node *deleteCriteria) } else if (IsA(deleteCriteria, BoolExpr)) { - ListCell *opExpressionCell = NULL; BoolExpr *deleteCriteriaExpression = (BoolExpr *) deleteCriteria; List *opExpressionList = deleteCriteriaExpression->args; - - foreach(opExpressionCell, opExpressionList) + Expr *opExpression = NULL; + foreach_ptr(opExpression, opExpressionList) { - Expr *opExpression = (Expr *) lfirst(opExpressionCell); if (!SimpleOpExpression(opExpression)) { simpleOpExpression = false; @@ -595,12 +591,11 @@ static void CheckPartitionColumn(Oid relationId, Node *whereClause) { Var *partitionColumn = ForceDistPartitionKey(relationId); - ListCell *columnCell = NULL; List *columnList = pull_var_clause_default(whereClause); - foreach(columnCell, columnList) + Var *var = NULL; + foreach_ptr(var, columnList) { - Var *var = (Var *) lfirst(columnCell); if (var->varattno != partitionColumn->varattno) { ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED), @@ -624,7 +619,6 @@ ShardsMatchingDeleteCriteria(Oid relationId, List *shardIntervalList, Node *deleteCriteria) { List *dropShardIntervalList = NIL; - ListCell *shardIntervalCell = NULL; /* build the base expression for constraint */ Index rangeTableIndex = 1; @@ -635,9 +629,9 @@ ShardsMatchingDeleteCriteria(Oid relationId, List *shardIntervalList, List *deleteCriteriaList = list_make1(deleteCriteria); /* walk over shard list and check if shards can be dropped */ - foreach(shardIntervalCell, shardIntervalList) + ShardInterval *shardInterval = NULL; + foreach_ptr(shardInterval, shardIntervalList) { - ShardInterval *shardInterval = (ShardInterval *) lfirst(shardIntervalCell); if (shardInterval->minValueExists && shardInterval->maxValueExists) { List *restrictInfoList = NIL; diff --git a/src/backend/distributed/master/master_metadata_utility.c b/src/backend/distributed/master/master_metadata_utility.c index ca3c162ed..e21f85487 100644 --- a/src/backend/distributed/master/master_metadata_utility.c +++ b/src/backend/distributed/master/master_metadata_utility.c @@ -161,7 +161,6 @@ citus_relation_size(PG_FUNCTION_ARGS) static uint64 DistributedTableSize(Oid relationId, char *sizeQuery) { - ListCell *workerNodeCell = NULL; uint64 totalRelationSize = 0; if (XactModificationLevel == XACT_MODIFICATION_DATA) @@ -182,10 +181,9 @@ DistributedTableSize(Oid relationId, char *sizeQuery) ErrorIfNotSuitableToGetSize(relationId); List *workerNodeList = ActiveReadableNodeList(); - - foreach(workerNodeCell, workerNodeList) + WorkerNode *workerNode = NULL; + foreach_ptr(workerNode, workerNodeList) { - WorkerNode *workerNode = (WorkerNode *) lfirst(workerNodeCell); uint64 relationSizeOnNode = DistributedTableSizeOnWorker(workerNode, relationId, sizeQuery); totalRelationSize += relationSizeOnNode; @@ -338,13 +336,12 @@ StringInfo GenerateSizeQueryOnMultiplePlacements(List *shardIntervalList, char *sizeQuery) { StringInfo selectQuery = makeStringInfo(); - ListCell *shardIntervalCell = NULL; appendStringInfo(selectQuery, "SELECT "); - foreach(shardIntervalCell, shardIntervalList) + ShardInterval *shardInterval = NULL; + foreach_ptr(shardInterval, shardIntervalList) { - ShardInterval *shardInterval = (ShardInterval *) lfirst(shardIntervalCell); uint64 shardId = shardInterval->shardId; Oid schemaId = get_rel_namespace(shardInterval->relationId); char *schemaName = get_namespace_name(schemaId); @@ -433,12 +430,11 @@ uint32 TableShardReplicationFactor(Oid relationId) { uint32 replicationCount = 0; - ListCell *shardCell = NULL; List *shardIntervalList = LoadShardIntervalList(relationId); - foreach(shardCell, shardIntervalList) + ShardInterval *shardInterval = NULL; + foreach_ptr(shardInterval, shardIntervalList) { - ShardInterval *shardInterval = (ShardInterval *) lfirst(shardCell); uint64 shardId = shardInterval->shardId; List *shardPlacementList = ShardPlacementList(shardId); @@ -695,10 +691,9 @@ ActiveShardPlacementList(uint64 shardId) List *activePlacementList = NIL; List *shardPlacementList = ShardPlacementList(shardId); - ListCell *shardPlacementCell = NULL; - foreach(shardPlacementCell, shardPlacementList) + ShardPlacement *shardPlacement = NULL; + foreach_ptr(shardPlacement, shardPlacementList) { - ShardPlacement *shardPlacement = (ShardPlacement *) lfirst(shardPlacementCell); if (shardPlacement->shardState == SHARD_STATE_ACTIVE) { activePlacementList = lappend(activePlacementList, shardPlacement); @@ -1198,11 +1193,10 @@ UpdatePartitionShardPlacementStates(ShardPlacement *parentShardPlacement, char s /* this function should only be called for partitioned tables */ Assert(PartitionedTable(partitionedTableOid)); - ListCell *partitionOidCell = NULL; List *partitionList = PartitionList(partitionedTableOid); - foreach(partitionOidCell, partitionList) + Oid partitionOid = InvalidOid; + foreach_oid(partitionOid, partitionList) { - Oid partitionOid = lfirst_oid(partitionOidCell); uint64 partitionShardId = ColocatedShardIdInRelation(partitionOid, parentShardInterval->shardIndex); @@ -1226,12 +1220,9 @@ static ShardPlacement * ShardPlacementOnGroup(uint64 shardId, int groupId) { List *placementList = ShardPlacementList(shardId); - ListCell *placementCell = NULL; - - foreach(placementCell, placementList) + ShardPlacement *placement = NULL; + foreach_ptr(placement, placementList) { - ShardPlacement *placement = (ShardPlacement *) lfirst(placementCell); - if (placement->groupId == groupId) { return placement; diff --git a/src/backend/distributed/master/master_repair_shards.c b/src/backend/distributed/master/master_repair_shards.c index 34d11a6e9..62704f6d2 100644 --- a/src/backend/distributed/master/master_repair_shards.c +++ b/src/backend/distributed/master/master_repair_shards.c @@ -50,14 +50,14 @@ /* local function forward declarations */ static char LookupShardTransferMode(Oid shardReplicationModeOid); -static void RepairShardPlacement(int64 shardId, char *sourceNodeName, - int32 sourceNodePort, char *targetNodeName, +static void RepairShardPlacement(int64 shardId, const char *sourceNodeName, + int32 sourceNodePort, const char *targetNodeName, int32 targetNodePort); static List * CopyPartitionShardsCommandList(ShardInterval *shardInterval, - char *sourceNodeName, + const char *sourceNodeName, int32 sourceNodePort); -static void EnsureShardCanBeRepaired(int64 shardId, char *sourceNodeName, - int32 sourceNodePort, char *targetNodeName, +static void EnsureShardCanBeRepaired(int64 shardId, const char *sourceNodeName, + int32 sourceNodePort, const char *targetNodeName, int32 targetNodePort); static List * RecreateTableDDLCommandList(Oid relationId); static List * WorkerApplyShardDDLCommandList(List *ddlCommandList, int64 shardId); @@ -137,13 +137,9 @@ master_move_shard_placement(PG_FUNCTION_ARGS) void BlockWritesToShardList(List *shardList) { - ListCell *shardCell = NULL; - - - foreach(shardCell, shardList) + ShardInterval *shard = NULL; + foreach_ptr(shard, shardList) { - ShardInterval *shard = (ShardInterval *) lfirst(shardCell); - /* * We need to lock the referenced reference table metadata to avoid * asynchronous shard copy in case of cascading DML operations. @@ -213,8 +209,8 @@ LookupShardTransferMode(Oid shardReplicationModeOid) * This function is not co-location aware. It only repairs given shard. */ static void -RepairShardPlacement(int64 shardId, char *sourceNodeName, int32 sourceNodePort, - char *targetNodeName, int32 targetNodePort) +RepairShardPlacement(int64 shardId, const char *sourceNodeName, int32 sourceNodePort, + const char *targetNodeName, int32 targetNodePort) { ShardInterval *shardInterval = LoadShardInterval(shardId); Oid distributedTableId = shardInterval->relationId; @@ -333,19 +329,18 @@ RepairShardPlacement(int64 shardId, char *sourceNodeName, int32 sourceNodePort, * of the input shardInterval. */ static List * -CopyPartitionShardsCommandList(ShardInterval *shardInterval, char *sourceNodeName, +CopyPartitionShardsCommandList(ShardInterval *shardInterval, const char *sourceNodeName, int32 sourceNodePort) { Oid distributedTableId = shardInterval->relationId; - ListCell *partitionOidCell = NULL; List *ddlCommandList = NIL; Assert(PartitionedTableNoLock(distributedTableId)); List *partitionList = PartitionList(distributedTableId); - foreach(partitionOidCell, partitionList) + Oid partitionOid = InvalidOid; + foreach_oid(partitionOid, partitionList) { - Oid partitionOid = lfirst_oid(partitionOidCell); uint64 partitionShardId = ColocatedShardIdInRelation(partitionOid, shardInterval->shardIndex); ShardInterval *partitionShardInterval = LoadShardInterval(partitionShardId); @@ -370,8 +365,8 @@ CopyPartitionShardsCommandList(ShardInterval *shardInterval, char *sourceNodeNam * node and inactive node on the target node. */ static void -EnsureShardCanBeRepaired(int64 shardId, char *sourceNodeName, int32 sourceNodePort, - char *targetNodeName, int32 targetNodePort) +EnsureShardCanBeRepaired(int64 shardId, const char *sourceNodeName, int32 sourceNodePort, + const char *targetNodeName, int32 targetNodePort) { List *shardPlacementList = ShardPlacementList(shardId); @@ -401,14 +396,12 @@ EnsureShardCanBeRepaired(int64 shardId, char *sourceNodeName, int32 sourceNodePo * placement exists in the provided list. */ ShardPlacement * -SearchShardPlacementInList(List *shardPlacementList, char *nodeName, uint32 nodePort) +SearchShardPlacementInList(List *shardPlacementList, const char *nodeName, + uint32 nodePort) { - ListCell *shardPlacementCell = NULL; - - foreach(shardPlacementCell, shardPlacementList) + ShardPlacement *shardPlacement = NULL; + foreach_ptr(shardPlacement, shardPlacementList) { - ShardPlacement *shardPlacement = lfirst(shardPlacementCell); - if (strncmp(nodeName, shardPlacement->nodeName, MAX_NODE_LENGTH) == 0 && nodePort == shardPlacement->nodePort) { @@ -425,7 +418,8 @@ SearchShardPlacementInList(List *shardPlacementList, char *nodeName, uint32 node * error if no such placement exists in the provided list. */ ShardPlacement * -ForceSearchShardPlacementInList(List *shardPlacementList, char *nodeName, uint32 nodePort) +ForceSearchShardPlacementInList(List *shardPlacementList, const char *nodeName, + uint32 nodePort) { ShardPlacement *placement = SearchShardPlacementInList(shardPlacementList, nodeName, nodePort); @@ -446,7 +440,7 @@ ForceSearchShardPlacementInList(List *shardPlacementList, char *nodeName, uint32 * the data by the flag includeDataCopy. */ List * -CopyShardCommandList(ShardInterval *shardInterval, char *sourceNodeName, +CopyShardCommandList(ShardInterval *shardInterval, const char *sourceNodeName, int32 sourceNodePort, bool includeDataCopy) { int64 shardId = shardInterval->shardId; @@ -526,7 +520,6 @@ CopyShardForeignConstraintCommandListGrouped(ShardInterval *shardInterval, int shardIndex = 0; List *commandList = GetTableForeignConstraintCommands(shardInterval->relationId); - ListCell *commandCell = NULL; /* we will only use shardIndex if there is a foreign constraint */ if (commandList != NIL) @@ -537,9 +530,9 @@ CopyShardForeignConstraintCommandListGrouped(ShardInterval *shardInterval, *colocatedShardForeignConstraintCommandList = NIL; *referenceTableForeignConstraintList = NIL; - foreach(commandCell, commandList) + const char *command = NULL; + foreach_ptr(command, commandList) { - char *command = (char *) lfirst(commandCell); char *escapedCommand = quote_literal_cstr(command); uint64 referencedShardId = INVALID_SHARD_ID; @@ -681,11 +674,10 @@ static List * WorkerApplyShardDDLCommandList(List *ddlCommandList, int64 shardId) { List *applyDdlCommandList = NIL; - ListCell *ddlCommandCell = NULL; - foreach(ddlCommandCell, ddlCommandList) + const char *ddlCommand = NULL; + foreach_ptr(ddlCommand, ddlCommandList) { - char *ddlCommand = lfirst(ddlCommandCell); char *escapedDdlCommand = quote_literal_cstr(ddlCommand); StringInfo applyDdlCommand = makeStringInfo(); diff --git a/src/backend/distributed/master/master_stage_protocol.c b/src/backend/distributed/master/master_stage_protocol.c index 28cac5035..01dc2bc92 100644 --- a/src/backend/distributed/master/master_stage_protocol.c +++ b/src/backend/distributed/master/master_stage_protocol.c @@ -62,7 +62,7 @@ /* Local functions forward declarations */ static List * RelationShardListForShardCreate(ShardInterval *shardInterval); static bool WorkerShardStats(ShardPlacement *placement, Oid relationId, - char *shardName, uint64 *shardSize, + const char *shardName, uint64 *shardSize, text **shardMinValue, text **shardMaxValue); /* exports for SQL callable functions */ @@ -230,10 +230,8 @@ master_append_table_to_shard(PG_FUNCTION_ARGS) char *sourceTableName = text_to_cstring(sourceTableNameText); char *sourceNodeName = text_to_cstring(sourceNodeNameText); - ListCell *shardPlacementCell = NULL; float4 shardFillLevel = 0.0; - CheckCitusVersion(ERROR); ShardInterval *shardInterval = LoadShardInterval(shardId); @@ -289,9 +287,9 @@ master_append_table_to_shard(PG_FUNCTION_ARGS) UseCoordinatedTransaction(); /* issue command to append table to each shard placement */ - foreach(shardPlacementCell, shardPlacementList) + ShardPlacement *shardPlacement = NULL; + foreach_ptr(shardPlacement, shardPlacementList) { - ShardPlacement *shardPlacement = (ShardPlacement *) lfirst(shardPlacementCell); MultiConnection *connection = GetPlacementConnection(FOR_DML, shardPlacement, NULL); PGresult *queryResult = NULL; @@ -488,15 +486,14 @@ CreateShardsOnWorkers(Oid distributedRelationId, List *shardPlacements, includeSequenceDefaults); List *foreignConstraintCommandList = GetTableForeignConstraintCommands(distributedRelationId); - ListCell *shardPlacementCell = NULL; int taskId = 1; List *taskList = NIL; int poolSize = 1; - foreach(shardPlacementCell, shardPlacements) + ShardPlacement *shardPlacement = NULL; + foreach_ptr(shardPlacement, shardPlacements) { - ShardPlacement *shardPlacement = (ShardPlacement *) lfirst(shardPlacementCell); uint64 shardId = shardPlacement->shardId; ShardInterval *shardInterval = LoadShardInterval(shardId); int shardIndex = -1; @@ -565,7 +562,6 @@ RelationShardListForShardCreate(ShardInterval *shardInterval) List *referencedRelationList = cacheEntry->referencedRelationsViaForeignKey; List *referencingRelationList = cacheEntry->referencingRelationsViaForeignKey; int shardIndex = -1; - ListCell *fkeyRelationIdCell = NULL; /* list_concat_*() modifies the first arg, so make a copy first */ List *allForeignKeyRelations = list_copy(referencedRelationList); @@ -586,9 +582,9 @@ RelationShardListForShardCreate(ShardInterval *shardInterval) /* all foregin key constraint relations */ - foreach(fkeyRelationIdCell, allForeignKeyRelations) + Oid fkeyRelationid = InvalidOid; + foreach_oid(fkeyRelationid, allForeignKeyRelations) { - Oid fkeyRelationid = lfirst_oid(fkeyRelationIdCell); uint64 fkeyShardId = INVALID_SHARD_ID; if (!IsDistributedTable(fkeyRelationid)) @@ -661,12 +657,10 @@ WorkerCreateShardCommandList(Oid relationId, int shardIndex, uint64 shardId, Oid schemaId = get_rel_namespace(relationId); char *schemaName = get_namespace_name(schemaId); char *escapedSchemaName = quote_literal_cstr(schemaName); - ListCell *ddlCommandCell = NULL; - ListCell *foreignConstraintCommandCell = NULL; - foreach(ddlCommandCell, ddlCommandList) + const char *ddlCommand = NULL; + foreach_ptr(ddlCommand, ddlCommandList) { - char *ddlCommand = (char *) lfirst(ddlCommandCell); char *escapedDDLCommand = quote_literal_cstr(ddlCommand); StringInfo applyDDLCommand = makeStringInfo(); @@ -685,9 +679,9 @@ WorkerCreateShardCommandList(Oid relationId, int shardIndex, uint64 shardId, commandList = lappend(commandList, applyDDLCommand->data); } - foreach(foreignConstraintCommandCell, foreignConstraintCommandList) + const char *command = NULL; + foreach_ptr(command, foreignConstraintCommandList) { - char *command = (char *) lfirst(foreignConstraintCommandCell); char *escapedCommand = quote_literal_cstr(command); uint64 referencedShardId = INVALID_SHARD_ID; @@ -764,7 +758,6 @@ UpdateShardStatistics(int64 shardId) Oid relationId = shardInterval->relationId; char storageType = shardInterval->storageType; char partitionType = PartitionMethod(relationId); - ListCell *shardPlacementCell = NULL; bool statsOK = false; uint64 shardSize = 0; text *minValue = NULL; @@ -782,10 +775,9 @@ UpdateShardStatistics(int64 shardId) List *shardPlacementList = ActiveShardPlacementList(shardId); /* get shard's statistics from a shard placement */ - foreach(shardPlacementCell, shardPlacementList) + ShardPlacement *placement = NULL; + foreach_ptr(placement, shardPlacementList) { - ShardPlacement *placement = (ShardPlacement *) lfirst(shardPlacementCell); - statsOK = WorkerShardStats(placement, relationId, shardQualifiedName, &shardSize, &minValue, &maxValue); if (statsOK) @@ -812,10 +804,8 @@ UpdateShardStatistics(int64 shardId) HOLD_INTERRUPTS(); /* update metadata for each shard placement we appended to */ - shardPlacementCell = NULL; - foreach(shardPlacementCell, shardPlacementList) + foreach_ptr(placement, shardPlacementList) { - ShardPlacement *placement = (ShardPlacement *) lfirst(shardPlacementCell); uint64 placementId = placement->placementId; int32 groupId = placement->groupId; @@ -848,7 +838,7 @@ UpdateShardStatistics(int64 shardId) * we assume have changed after new table data have been appended to the shard. */ static bool -WorkerShardStats(ShardPlacement *placement, Oid relationId, char *shardName, +WorkerShardStats(ShardPlacement *placement, Oid relationId, const char *shardName, uint64 *shardSize, text **shardMinValue, text **shardMaxValue) { StringInfo tableSizeQuery = makeStringInfo(); @@ -956,7 +946,7 @@ WorkerShardStats(ShardPlacement *placement, Oid relationId, char *shardName, * extracts referenced table id from it. */ Oid -ForeignConstraintGetReferencedTableId(char *queryString) +ForeignConstraintGetReferencedTableId(const char *queryString) { Node *queryNode = ParseTreeNode(queryString); AlterTableStmt *foreignConstraintStmt = (AlterTableStmt *) queryNode; diff --git a/src/backend/distributed/master/master_truncate.c b/src/backend/distributed/master/master_truncate.c index 8647eaf5b..d6d4d56e0 100644 --- a/src/backend/distributed/master/master_truncate.c +++ b/src/backend/distributed/master/master_truncate.c @@ -17,6 +17,7 @@ #include "commands/trigger.h" #include "distributed/commands/utility_hook.h" #include "distributed/deparse_shard_query.h" +#include "distributed/listutils.h" #include "distributed/master_metadata_utility.h" #include "distributed/master_protocol.h" #include "distributed/multi_executor.h" @@ -103,11 +104,9 @@ TruncateTaskList(Oid relationId) /* lock metadata before getting placement lists */ LockShardListMetadata(shardIntervalList, ShareLock); - ListCell *shardIntervalCell = NULL; - - foreach(shardIntervalCell, shardIntervalList) + ShardInterval *shardInterval = NULL; + foreach_ptr(shardInterval, shardIntervalList) { - ShardInterval *shardInterval = (ShardInterval *) lfirst(shardIntervalCell); uint64 shardId = shardInterval->shardId; char *shardRelationName = pstrdup(relationName); diff --git a/src/backend/distributed/master/worker_node_manager.c b/src/backend/distributed/master/worker_node_manager.c index f06dafa12..e0e38e08c 100644 --- a/src/backend/distributed/master/worker_node_manager.c +++ b/src/backend/distributed/master/worker_node_manager.c @@ -16,6 +16,7 @@ #include "commands/dbcommands.h" #include "distributed/hash_helpers.h" +#include "distributed/listutils.h" #include "distributed/metadata_cache.h" #include "distributed/multi_client_executor.h" #include "distributed/worker_manager.h" @@ -536,20 +537,18 @@ OddNumber(uint32 number) static bool ListMember(List *currentList, WorkerNode *workerNode) { - bool listMember = false; Size keySize = WORKER_LENGTH + sizeof(uint32); - ListCell *currentCell = NULL; - foreach(currentCell, currentList) + WorkerNode *currentNode = NULL; + foreach_ptr(currentNode, currentList) { - WorkerNode *currentNode = (WorkerNode *) lfirst(currentCell); if (WorkerNodeCompare(workerNode, currentNode, keySize) == 0) { - listMember = true; + return true; } } - return listMember; + return false; } @@ -604,13 +603,10 @@ WorkerNode * GetFirstPrimaryWorkerNode(void) { List *workerNodeList = ActivePrimaryWorkerNodeList(NoLock); - ListCell *workerNodeCell = NULL; WorkerNode *firstWorkerNode = NULL; - - foreach(workerNodeCell, workerNodeList) + WorkerNode *workerNode = NULL; + foreach_ptr(workerNode, workerNodeList) { - WorkerNode *workerNode = (WorkerNode *) lfirst(workerNodeCell); - if (firstWorkerNode == NULL || CompareWorkerNodes(&workerNode, &firstWorkerNode) < 0) { diff --git a/src/backend/distributed/metadata/dependency.c b/src/backend/distributed/metadata/dependency.c index af8792132..043c16410 100644 --- a/src/backend/distributed/metadata/dependency.c +++ b/src/backend/distributed/metadata/dependency.c @@ -86,7 +86,6 @@ GetUniqueDependenciesList(List *objectAddressesList) InitObjectAddressCollector(&objectAddressCollector); ObjectAddress *objectAddress = NULL; - foreach_ptr(objectAddress, objectAddressesList) { if (IsObjectAddressCollected(objectAddress, &objectAddressCollector)) @@ -111,7 +110,6 @@ List * GetDependenciesForObject(const ObjectAddress *target) { ObjectAddressCollector collector = { 0 }; - InitObjectAddressCollector(&collector); recurse_pg_depend(target, @@ -140,14 +138,11 @@ List * OrderObjectAddressListInDependencyOrder(List *objectAddressList) { ObjectAddressCollector collector = { 0 }; - ListCell *objectAddressCell = NULL; - InitObjectAddressCollector(&collector); - foreach(objectAddressCell, objectAddressList) + ObjectAddress *objectAddress = NULL; + foreach_ptr(objectAddress, objectAddressList) { - ObjectAddress *objectAddress = (ObjectAddress *) lfirst(objectAddressCell); - if (IsObjectAddressCollected(objectAddress, &collector)) { /* skip objects that are already ordered */ @@ -201,7 +196,6 @@ recurse_pg_depend(const ObjectAddress *target, ScanKeyData key[2]; HeapTuple depTup = NULL; List *pgDependEntries = NIL; - ListCell *pgDependCell = NULL; if (TargetObjectVisited(collector, target)) { @@ -249,9 +243,9 @@ recurse_pg_depend(const ObjectAddress *target, /* * Iterate all entries and recurse depth first */ - foreach(pgDependCell, pgDependEntries) + Form_pg_depend pg_depend = NULL; + foreach_ptr(pg_depend, pgDependEntries) { - Form_pg_depend pg_depend = (Form_pg_depend) lfirst(pgDependCell); ObjectAddress address = { 0 }; ObjectAddressSet(address, pg_depend->refclassid, pg_depend->refobjid); diff --git a/src/backend/distributed/metadata/metadata_cache.c b/src/backend/distributed/metadata/metadata_cache.c index c7aa0e7b1..c4f1ba037 100644 --- a/src/backend/distributed/metadata/metadata_cache.c +++ b/src/backend/distributed/metadata/metadata_cache.c @@ -34,6 +34,7 @@ #include "distributed/citus_ruleutils.h" #include "distributed/function_utils.h" #include "distributed/foreign_key_relationship.h" +#include "distributed/listutils.h" #include "distributed/master_metadata_utility.h" #include "distributed/metadata/pg_dist_object.h" #include "distributed/metadata_cache.h" @@ -332,17 +333,15 @@ List * DistributedTableList(void) { List *distributedTableList = NIL; - ListCell *distTableOidCell = NULL; Assert(CitusHasBeenLoaded() && CheckCitusVersion(WARNING)); /* first, we need to iterate over pg_dist_partition */ List *distTableOidList = DistTableOidList(); - foreach(distTableOidCell, distTableOidList) + Oid relationId = InvalidOid; + foreach_oid(relationId, distTableOidList) { - Oid relationId = lfirst_oid(distTableOidCell); - DistTableCacheEntry *cacheEntry = DistributedTableCacheEntry(relationId); distributedTableList = lappend(distributedTableList, cacheEntry); @@ -1137,7 +1136,6 @@ BuildCachedShardList(DistTableCacheEntry *cacheEntry) { Relation distShardRelation = heap_open(DistShardRelationId(), AccessShareLock); TupleDesc distShardTupleDesc = RelationGetDescr(distShardRelation); - ListCell *distShardTupleCell = NULL; int arrayIndex = 0; shardIntervalArray = MemoryContextAllocZero(MetadataCacheMemoryContext, @@ -1153,9 +1151,9 @@ BuildCachedShardList(DistTableCacheEntry *cacheEntry) shardIntervalArrayLength * sizeof(int)); - foreach(distShardTupleCell, distShardTupleList) + HeapTuple shardTuple = NULL; + foreach_ptr(shardTuple, distShardTupleList) { - HeapTuple shardTuple = lfirst(distShardTupleCell); ShardInterval *shardInterval = TupleToShardInterval(shardTuple, distShardTupleDesc, intervalTypeId, @@ -1291,7 +1289,6 @@ BuildCachedShardList(DistTableCacheEntry *cacheEntry) { ShardInterval *shardInterval = sortedShardIntervalArray[shardIndex]; bool foundInCache = false; - ListCell *placementCell = NULL; int placementOffset = 0; ShardCacheEntry *shardEntry = hash_search(DistShardCacheHash, @@ -1323,11 +1320,9 @@ BuildCachedShardList(DistTableCacheEntry *cacheEntry) MemoryContext oldContext = MemoryContextSwitchTo(MetadataCacheMemoryContext); GroupShardPlacement *placementArray = palloc0(numberOfPlacements * sizeof(GroupShardPlacement)); - foreach(placementCell, placementList) + GroupShardPlacement *srcPlacement = NULL; + foreach_ptr(srcPlacement, placementList) { - GroupShardPlacement *srcPlacement = - (GroupShardPlacement *) lfirst(placementCell); - placementArray[placementOffset] = *srcPlacement; placementOffset++; } @@ -3024,7 +3019,6 @@ PrepareWorkerNodeCache(void) static void InitializeWorkerNodeCache(void) { - ListCell *workerNodeCell = NULL; HASHCTL info; long maxTableSize = (long) MaxWorkerNodesTracked; bool includeNodesFromOtherClusters = false; @@ -3057,9 +3051,9 @@ InitializeWorkerNodeCache(void) newWorkerNodeCount); /* iterate over the worker node list */ - foreach(workerNodeCell, workerNodeList) + WorkerNode *currentNode = NULL; + foreach_ptr(currentNode, workerNodeList) { - WorkerNode *currentNode = lfirst(workerNodeCell); bool handleFound = false; /* search for the worker node in the hash, and then insert the values */ diff --git a/src/backend/distributed/metadata/metadata_sync.c b/src/backend/distributed/metadata/metadata_sync.c index b0f5fb139..36915580d 100644 --- a/src/backend/distributed/metadata/metadata_sync.c +++ b/src/backend/distributed/metadata/metadata_sync.c @@ -55,8 +55,8 @@ static char * LocalGroupIdUpdateCommand(int32 groupId); -static void UpdateDistNodeBoolAttr(char *nodeName, int32 nodePort, int attrNum, - bool value); +static void UpdateDistNodeBoolAttr(const char *nodeName, int32 nodePort, + int attrNum, bool value); static List * SequenceDDLCommandsForTable(Oid relationId); static char * TruncateTriggerCreateCommand(Oid relationId); static char * SchemaOwnerName(Oid objectId); @@ -87,18 +87,18 @@ start_metadata_sync_to_node(PG_FUNCTION_ARGS) char *nodeNameString = text_to_cstring(nodeName); - StartMetadatSyncToNode(nodeNameString, nodePort); + StartMetadataSyncToNode(nodeNameString, nodePort); PG_RETURN_VOID(); } /* - * StartMetadatSyncToNode is the internal API for + * StartMetadataSyncToNode is the internal API for * start_metadata_sync_to_node(). */ void -StartMetadatSyncToNode(char *nodeNameString, int32 nodePort) +StartMetadataSyncToNode(const char *nodeNameString, int32 nodePort) { char *escapedNodeName = quote_literal_cstr(nodeNameString); @@ -301,10 +301,9 @@ SyncMetadataSnapshotToNode(WorkerNode *workerNode, bool raiseOnError) * rollbacks the transaction, and otherwise commits. */ bool -SendOptionalCommandListToWorkerInTransaction(char *nodeName, int32 nodePort, - char *nodeUser, List *commandList) +SendOptionalCommandListToWorkerInTransaction(const char *nodeName, int32 nodePort, + const char *nodeUser, List *commandList) { - ListCell *commandCell = NULL; int connectionFlags = FORCE_NEW_CONNECTION; bool failed = false; @@ -315,10 +314,9 @@ SendOptionalCommandListToWorkerInTransaction(char *nodeName, int32 nodePort, RemoteTransactionBegin(workerConnection); /* iterate over the commands and execute them in the same connection */ - foreach(commandCell, commandList) + const char *commandString = NULL; + foreach_ptr(commandString, commandList) { - char *commandString = lfirst(commandCell); - if (ExecuteOptionalRemoteCommand(workerConnection, commandString, NULL) != 0) { failed = true; @@ -362,7 +360,6 @@ MetadataCreateCommands(void) List *propagatedTableList = NIL; bool includeNodesFromOtherClusters = true; List *workerNodeList = ReadDistNode(includeNodesFromOtherClusters); - ListCell *distributedTableCell = NULL; bool includeSequenceDefaults = true; /* make sure we have deterministic output for our tests */ @@ -374,10 +371,9 @@ MetadataCreateCommands(void) nodeListInsertCommand); /* create the list of tables whose metadata will be created */ - foreach(distributedTableCell, distributedTableList) + DistTableCacheEntry *cacheEntry = NULL; + foreach_ptr(cacheEntry, distributedTableList) { - DistTableCacheEntry *cacheEntry = - (DistTableCacheEntry *) lfirst(distributedTableCell); if (ShouldSyncTableMetadata(cacheEntry->relationId)) { propagatedTableList = lappend(propagatedTableList, cacheEntry); @@ -385,10 +381,8 @@ MetadataCreateCommands(void) } /* create the tables, but not the metadata */ - foreach(distributedTableCell, propagatedTableList) + foreach_ptr(cacheEntry, propagatedTableList) { - DistTableCacheEntry *cacheEntry = - (DistTableCacheEntry *) lfirst(distributedTableCell); Oid relationId = cacheEntry->relationId; ObjectAddress tableAddress = { 0 }; @@ -414,11 +408,8 @@ MetadataCreateCommands(void) } /* construct the foreign key constraints after all tables are created */ - foreach(distributedTableCell, propagatedTableList) + foreach_ptr(cacheEntry, propagatedTableList) { - DistTableCacheEntry *cacheEntry = - (DistTableCacheEntry *) lfirst(distributedTableCell); - List *foreignConstraintCommands = GetTableForeignConstraintCommands(cacheEntry->relationId); @@ -427,11 +418,8 @@ MetadataCreateCommands(void) } /* construct partitioning hierarchy after all tables are created */ - foreach(distributedTableCell, propagatedTableList) + foreach_ptr(cacheEntry, propagatedTableList) { - DistTableCacheEntry *cacheEntry = - (DistTableCacheEntry *) lfirst(distributedTableCell); - if (PartitionTable(cacheEntry->relationId)) { char *alterTableAttachPartitionCommands = @@ -443,10 +431,8 @@ MetadataCreateCommands(void) } /* after all tables are created, create the metadata */ - foreach(distributedTableCell, propagatedTableList) + foreach_ptr(cacheEntry, propagatedTableList) { - DistTableCacheEntry *cacheEntry = - (DistTableCacheEntry *) lfirst(distributedTableCell); Oid clusteredTableId = cacheEntry->relationId; /* add the table metadata command first*/ @@ -568,7 +554,6 @@ MetadataDropCommands(void) char * NodeListInsertCommand(List *workerNodeList) { - ListCell *workerNodeCell = NULL; StringInfo nodeListInsertCommand = makeStringInfo(); int workerCount = list_length(workerNodeList); int processedWorkerNodeCount = 0; @@ -594,9 +579,9 @@ NodeListInsertCommand(List *workerNodeList) "noderack, hasmetadata, metadatasynced, isactive, noderole, nodecluster) VALUES "); /* iterate over the worker nodes, add the values */ - foreach(workerNodeCell, workerNodeList) + WorkerNode *workerNode = NULL; + foreach_ptr(workerNode, workerNodeList) { - WorkerNode *workerNode = (WorkerNode *) lfirst(workerNodeCell); char *hasMetadataString = workerNode->hasMetadata ? "TRUE" : "FALSE"; char *metadataSyncedString = workerNode->metadataSynced ? "TRUE" : "FALSE"; char *isActiveString = workerNode->isActive ? "TRUE" : "FALSE"; @@ -678,7 +663,7 @@ DistributionCreateCommand(DistTableCacheEntry *cacheEntry) * to drop a distributed table and its metadata on a remote node. */ char * -DistributionDeleteCommand(char *schemaName, char *tableName) +DistributionDeleteCommand(const char *schemaName, const char *tableName) { StringInfo deleteDistributionCommand = makeStringInfo(); @@ -722,7 +707,6 @@ List * ShardListInsertCommand(List *shardIntervalList) { List *commandList = NIL; - ListCell *shardIntervalCell = NULL; StringInfo insertPlacementCommand = makeStringInfo(); StringInfo insertShardCommand = makeStringInfo(); int shardCount = list_length(shardIntervalList); @@ -735,18 +719,15 @@ ShardListInsertCommand(List *shardIntervalList) } /* add placements to insertPlacementCommand */ - foreach(shardIntervalCell, shardIntervalList) + ShardInterval *shardInterval = NULL; + foreach_ptr(shardInterval, shardIntervalList) { - ShardInterval *shardInterval = (ShardInterval *) lfirst(shardIntervalCell); uint64 shardId = shardInterval->shardId; - List *shardPlacementList = ActiveShardPlacementList(shardId); - ListCell *shardPlacementCell = NULL; - foreach(shardPlacementCell, shardPlacementList) + ShardPlacement *placement = NULL; + foreach_ptr(placement, shardPlacementList) { - ShardPlacement *placement = (ShardPlacement *) lfirst(shardPlacementCell); - if (insertPlacementCommand->len == 0) { /* generate the shard placement query without any values yet */ @@ -782,9 +763,8 @@ ShardListInsertCommand(List *shardIntervalList) "VALUES "); /* now add shards to insertShardCommand */ - foreach(shardIntervalCell, shardIntervalList) + foreach_ptr(shardInterval, shardIntervalList) { - ShardInterval *shardInterval = (ShardInterval *) lfirst(shardIntervalCell); uint64 shardId = shardInterval->shardId; Oid distributedRelationId = shardInterval->relationId; char *qualifiedRelationName = generate_qualified_relation_name( @@ -974,7 +954,7 @@ LocalGroupIdUpdateCommand(int32 groupId) * pg_dist_node to hasMetadata. */ void -MarkNodeHasMetadata(char *nodeName, int32 nodePort, bool hasMetadata) +MarkNodeHasMetadata(const char *nodeName, int32 nodePort, bool hasMetadata) { UpdateDistNodeBoolAttr(nodeName, nodePort, Anum_pg_dist_node_hasmetadata, @@ -987,7 +967,7 @@ MarkNodeHasMetadata(char *nodeName, int32 nodePort, bool hasMetadata) * specified worker in pg_dist_node to the given value. */ void -MarkNodeMetadataSynced(char *nodeName, int32 nodePort, bool synced) +MarkNodeMetadataSynced(const char *nodeName, int32 nodePort, bool synced) { UpdateDistNodeBoolAttr(nodeName, nodePort, Anum_pg_dist_node_metadatasynced, @@ -1000,7 +980,7 @@ MarkNodeMetadataSynced(char *nodeName, int32 nodePort, bool synced) * to the given value. */ static void -UpdateDistNodeBoolAttr(char *nodeName, int32 nodePort, int attrNum, bool value) +UpdateDistNodeBoolAttr(const char *nodeName, int32 nodePort, int attrNum, bool value) { const bool indexOK = false; @@ -1059,12 +1039,11 @@ SequenceDDLCommandsForTable(Oid relationId) { List *sequenceDDLList = NIL; List *ownedSequences = getOwnedSequences(relationId, InvalidAttrNumber); - ListCell *listCell; char *ownerName = TableOwner(relationId); - foreach(listCell, ownedSequences) + Oid sequenceOid = InvalidOid; + foreach_oid(sequenceOid, ownedSequences) { - Oid sequenceOid = (Oid) lfirst_oid(listCell); char *sequenceDef = pg_get_sequencedef_string(sequenceOid); char *escapedSequenceDef = quote_literal_cstr(sequenceDef); StringInfo wrappedSequenceDef = makeStringInfo(); @@ -1276,11 +1255,10 @@ static bool HasMetadataWorkers(void) { List *workerNodeList = ActivePrimaryWorkerNodeList(NoLock); - ListCell *workerNodeCell = NULL; - foreach(workerNodeCell, workerNodeList) + WorkerNode *workerNode = NULL; + foreach_ptr(workerNode, workerNodeList) { - WorkerNode *workerNode = (WorkerNode *) lfirst(workerNodeCell); if (workerNode->hasMetadata) { return true; @@ -1302,16 +1280,14 @@ void CreateTableMetadataOnWorkers(Oid relationId) { List *commandList = GetDistributedTableDDLEvents(relationId); - ListCell *commandCell = NULL; /* prevent recursive propagation */ SendCommandToWorkersWithMetadata(DISABLE_DDL_PROPAGATION); /* send the commands one by one */ - foreach(commandCell, commandList) + const char *command = NULL; + foreach_ptr(command, commandList) { - char *command = (char *) lfirst(commandCell); - SendCommandToWorkersWithMetadata(command); } } @@ -1331,24 +1307,20 @@ DetachPartitionCommandList(void) { List *detachPartitionCommandList = NIL; List *distributedTableList = DistributedTableList(); - ListCell *distributedTableCell = NULL; /* we iterate over all distributed partitioned tables and DETACH their partitions */ - foreach(distributedTableCell, distributedTableList) + DistTableCacheEntry *cacheEntry = NULL; + foreach_ptr(cacheEntry, distributedTableList) { - DistTableCacheEntry *cacheEntry = - (DistTableCacheEntry *) lfirst(distributedTableCell); - ListCell *partitionCell = NULL; - if (!PartitionedTable(cacheEntry->relationId)) { continue; } List *partitionList = PartitionList(cacheEntry->relationId); - foreach(partitionCell, partitionList) + Oid partitionRelationId = InvalidOid; + foreach_oid(partitionRelationId, partitionList) { - Oid partitionRelationId = lfirst_oid(partitionCell); char *detachPartitionCommand = GenerateDetachPartitionCommand(partitionRelationId); @@ -1384,7 +1356,6 @@ DetachPartitionCommandList(void) MetadataSyncResult SyncMetadataToNodes(void) { - ListCell *workerCell = NULL; MetadataSyncResult result = METADATA_SYNC_SUCCESS; if (!IsCoordinator()) @@ -1403,11 +1374,9 @@ SyncMetadataToNodes(void) } List *workerList = ActivePrimaryWorkerNodeList(NoLock); - - foreach(workerCell, workerList) + WorkerNode *workerNode = NULL; + foreach_ptr(workerNode, workerList) { - WorkerNode *workerNode = lfirst(workerCell); - if (workerNode->hasMetadata && !workerNode->metadataSynced) { bool raiseInterrupts = false; diff --git a/src/backend/distributed/metadata/node_metadata.c b/src/backend/distributed/metadata/node_metadata.c index a9a87e51c..779c4362d 100644 --- a/src/backend/distributed/metadata/node_metadata.c +++ b/src/backend/distributed/metadata/node_metadata.c @@ -857,7 +857,7 @@ get_shard_id_for_distribution_column(PG_FUNCTION_ARGS) * if it already exists. Else, the function returns NULL. */ WorkerNode * -FindWorkerNode(char *nodeName, int32 nodePort) +FindWorkerNode(const char *nodeName, int32 nodePort) { HTAB *workerNodeHash = GetWorkerNodeHash(); bool handleFound = false; @@ -886,7 +886,7 @@ FindWorkerNode(char *nodeName, int32 nodePort) * if it exists otherwise it errors out. */ WorkerNode * -ForceFindWorkerNode(char *nodeName, int32 nodePort) +ForceFindWorkerNode(const char *nodeName, int32 nodePort) { WorkerNode *node = FindWorkerNode(nodeName, nodePort); if (node == NULL) diff --git a/src/backend/distributed/progress/multi_progress.c b/src/backend/distributed/progress/multi_progress.c index 66c764c47..23a40f723 100644 --- a/src/backend/distributed/progress/multi_progress.c +++ b/src/backend/distributed/progress/multi_progress.c @@ -12,6 +12,7 @@ #include "pgstat.h" #include "distributed/function_utils.h" +#include "distributed/listutils.h" #include "distributed/multi_progress.h" #include "distributed/version_compat.h" #include "storage/dsm.h" @@ -228,12 +229,9 @@ MonitorDataFromDSMHandle(dsm_handle dsmHandle, dsm_segment **attachedSegment) void DetachFromDSMSegments(List *dsmSegmentList) { - ListCell *dsmSegmentCell = NULL; - - foreach(dsmSegmentCell, dsmSegmentList) + dsm_segment *dsmSegment = NULL; + foreach_ptr(dsmSegment, dsmSegmentList) { - dsm_segment *dsmSegment = (dsm_segment *) lfirst(dsmSegmentCell); - dsm_detach(dsmSegment); } } diff --git a/src/backend/distributed/relay/relay_event_utility.c b/src/backend/distributed/relay/relay_event_utility.c index deb2d3e0c..1fb721a4f 100644 --- a/src/backend/distributed/relay/relay_event_utility.c +++ b/src/backend/distributed/relay/relay_event_utility.c @@ -31,6 +31,7 @@ #include "catalog/pg_constraint.h" #include "distributed/citus_safe_lib.h" #include "distributed/commands.h" +#include "distributed/listutils.h" #include "distributed/metadata_cache.h" #include "distributed/relay_utility.h" #include "distributed/version_compat.h" @@ -105,7 +106,6 @@ RelayEventExtendNames(Node *parseTree, char *schemaName, uint64 shardId) char **relationSchemaName = &(alterTableStmt->relation->schemaname); List *commandList = alterTableStmt->cmds; - ListCell *commandCell = NULL; /* prefix with schema name if it is not added already */ SetSchemaNameIfNotExist(relationSchemaName, schemaName); @@ -113,10 +113,9 @@ RelayEventExtendNames(Node *parseTree, char *schemaName, uint64 shardId) /* append shardId to base relation name */ AppendShardIdToName(relationName, shardId); - foreach(commandCell, commandList) + AlterTableCmd *command = NULL; + foreach_ptr(command, commandList) { - AlterTableCmd *command = (AlterTableCmd *) lfirst(commandCell); - if (command->subtype == AT_AddConstraint) { Constraint *constraint = (Constraint *) command->def; @@ -327,11 +326,9 @@ RelayEventExtendNames(Node *parseTree, char *schemaName, uint64 shardId) if (grantStmt->targtype == ACL_TARGET_OBJECT && grantStmt->objtype == OBJECT_TABLE) { - ListCell *lc; - - foreach(lc, grantStmt->objects) + RangeVar *relation = NULL; + foreach_ptr(relation, grantStmt->objects) { - RangeVar *relation = (RangeVar *) lfirst(lc); char **relationName = &(relation->relname); char **relationSchemaName = &(relation->schemaname); @@ -524,11 +521,10 @@ RelayEventExtendNamesForInterShardCommands(Node *parseTree, uint64 leftShardId, { AlterTableStmt *alterTableStmt = (AlterTableStmt *) parseTree; List *commandList = alterTableStmt->cmds; - ListCell *commandCell = NULL; - foreach(commandCell, commandList) + AlterTableCmd *command = NULL; + foreach_ptr(command, commandList) { - AlterTableCmd *command = (AlterTableCmd *) lfirst(commandCell); char **referencedTableName = NULL; char **relationSchemaName = NULL; @@ -553,10 +549,9 @@ RelayEventExtendNamesForInterShardCommands(Node *parseTree, uint64 leftShardId, ColumnDef *columnDefinition = (ColumnDef *) command->def; List *columnConstraints = columnDefinition->constraints; - ListCell *columnConstraint = NULL; - foreach(columnConstraint, columnConstraints) + Constraint *constraint = NULL; + foreach_ptr(constraint, columnConstraints) { - Constraint *constraint = (Constraint *) lfirst(columnConstraint); if (constraint->contype == CONSTR_FOREIGN) { referencedTableName = &(constraint->pktable->relname); diff --git a/src/backend/distributed/test/colocation_utils.c b/src/backend/distributed/test/colocation_utils.c index 3060beb3c..ad6ebd728 100644 --- a/src/backend/distributed/test/colocation_utils.c +++ b/src/backend/distributed/test/colocation_utils.c @@ -84,15 +84,14 @@ get_colocated_table_array(PG_FUNCTION_ARGS) Oid distributedTableId = PG_GETARG_OID(0); List *colocatedTableList = ColocatedTableList(distributedTableId); - ListCell *colocatedTableCell = NULL; int colocatedTableCount = list_length(colocatedTableList); Datum *colocatedTablesDatumArray = palloc0(colocatedTableCount * sizeof(Datum)); Oid arrayTypeId = OIDOID; int colocatedTableIndex = 0; - foreach(colocatedTableCell, colocatedTableList) + Oid colocatedTableId = InvalidOid; + foreach_oid(colocatedTableId, colocatedTableList) { - Oid colocatedTableId = lfirst_oid(colocatedTableCell); Datum colocatedTableDatum = ObjectIdGetDatum(colocatedTableId); colocatedTablesDatumArray[colocatedTableIndex] = colocatedTableDatum; diff --git a/src/backend/distributed/test/create_shards.c b/src/backend/distributed/test/create_shards.c index 21f25701d..4ed1db7c7 100644 --- a/src/backend/distributed/test/create_shards.c +++ b/src/backend/distributed/test/create_shards.c @@ -43,10 +43,9 @@ sort_names(PG_FUNCTION_ARGS) (int (*)(const void *, const void *))(&CompareStrings)); StringInfo sortedNames = makeStringInfo(); - ListCell *nameCell = NULL; - foreach(nameCell, nameList) + const char *name = NULL; + foreach_ptr(name, nameList) { - char *name = lfirst(nameCell); appendStringInfo(sortedNames, "%s\n", name); } diff --git a/src/backend/distributed/test/deparse_shard_query.c b/src/backend/distributed/test/deparse_shard_query.c index fa31845db..3f948ce6b 100644 --- a/src/backend/distributed/test/deparse_shard_query.c +++ b/src/backend/distributed/test/deparse_shard_query.c @@ -17,6 +17,7 @@ #include #include "catalog/pg_type.h" +#include "distributed/listutils.h" #include "distributed/master_protocol.h" #include "distributed/citus_ruleutils.h" #include "distributed/insert_select_planner.h" @@ -44,20 +45,17 @@ deparse_shard_query_test(PG_FUNCTION_ARGS) char *queryStringChar = text_to_cstring(queryString); List *parseTreeList = pg_parse_query(queryStringChar); - ListCell *parseTreeCell = NULL; - foreach(parseTreeCell, parseTreeList) + Node *parsetree = NULL; + foreach_ptr(parsetree, parseTreeList) { - Node *parsetree = (Node *) lfirst(parseTreeCell); - ListCell *queryTreeCell = NULL; - List *queryTreeList = pg_analyze_and_rewrite((RawStmt *) parsetree, queryStringChar, NULL, 0, NULL); - foreach(queryTreeCell, queryTreeList) + Query *query = NULL; + foreach_ptr(query, queryTreeList) { - Query *query = lfirst(queryTreeCell); StringInfo buffer = makeStringInfo(); /* reoreder the target list only for INSERT .. SELECT queries */ diff --git a/src/backend/distributed/test/distributed_intermediate_results.c b/src/backend/distributed/test/distributed_intermediate_results.c index 5465a0393..19b0287fa 100644 --- a/src/backend/distributed/test/distributed_intermediate_results.c +++ b/src/backend/distributed/test/distributed_intermediate_results.c @@ -22,6 +22,7 @@ #include "distributed/commands/multi_copy.h" #include "distributed/connection_management.h" #include "distributed/intermediate_results.h" +#include "distributed/listutils.h" #include "distributed/multi_executor.h" #include "distributed/remote_commands.h" #include "distributed/tuplestore.h" @@ -79,12 +80,9 @@ partition_task_list_results(PG_FUNCTION_ARGS) TupleDesc tupleDescriptor = NULL; Tuplestorestate *tupleStore = SetupTuplestore(fcinfo, &tupleDescriptor); - ListCell *fragmentCell = NULL; - - foreach(fragmentCell, fragmentList) + DistributedResultFragment *fragment = NULL; + foreach_ptr(fragment, fragmentList) { - DistributedResultFragment *fragment = lfirst(fragmentCell); - bool columnNulls[5] = { 0 }; Datum columnValues[5] = { CStringGetTextDatum(fragment->resultId), @@ -161,11 +159,10 @@ redistribute_task_list_results(PG_FUNCTION_ARGS) Datum *resultIdValues = palloc0(fragmentCount * sizeof(Datum)); List *sortedResultIds = SortList(shardResultIds[shardIndex], pg_qsort_strcmp); - ListCell *resultIdCell = NULL; + const char *resultId = NULL; int resultIdIndex = 0; - foreach(resultIdCell, sortedResultIds) + foreach_ptr(resultId, sortedResultIds) { - char *resultId = lfirst(resultIdCell); resultIdValues[resultIdIndex++] = CStringGetTextDatum(resultId); } diff --git a/src/backend/distributed/test/distribution_metadata.c b/src/backend/distributed/test/distribution_metadata.c index 6e8d0ef8e..aa1a85235 100644 --- a/src/backend/distributed/test/distribution_metadata.c +++ b/src/backend/distributed/test/distribution_metadata.c @@ -62,7 +62,6 @@ Datum load_shard_id_array(PG_FUNCTION_ARGS) { Oid distributedTableId = PG_GETARG_OID(0); - ListCell *shardCell = NULL; int shardIdIndex = 0; Oid shardIdTypeId = INT8OID; @@ -71,10 +70,10 @@ load_shard_id_array(PG_FUNCTION_ARGS) int shardIdCount = list_length(shardList); Datum *shardIdDatumArray = palloc0(shardIdCount * sizeof(Datum)); - foreach(shardCell, shardList) + ShardInterval *shardInterval = NULL; + foreach_ptr(shardInterval, shardList) { - ShardInterval *shardId = (ShardInterval *) lfirst(shardCell); - Datum shardIdDatum = Int64GetDatum(shardId->shardId); + Datum shardIdDatum = Int64GetDatum(shardInterval->shardId); shardIdDatumArray[shardIdIndex] = shardIdDatum; shardIdIndex++; @@ -123,7 +122,6 @@ load_shard_placement_array(PG_FUNCTION_ARGS) int64 shardId = PG_GETARG_INT64(0); bool onlyActive = PG_GETARG_BOOL(1); List *placementList = NIL; - ListCell *placementCell = NULL; int placementIndex = 0; Oid placementTypeId = TEXTOID; StringInfo placementInfo = makeStringInfo(); @@ -142,9 +140,9 @@ load_shard_placement_array(PG_FUNCTION_ARGS) int placementCount = list_length(placementList); Datum *placementDatumArray = palloc0(placementCount * sizeof(Datum)); - foreach(placementCell, placementList) + ShardPlacement *placement = NULL; + foreach_ptr(placement, placementList) { - ShardPlacement *placement = (ShardPlacement *) lfirst(placementCell); appendStringInfo(placementInfo, "%s:%d", placement->nodeName, placement->nodePort); @@ -256,20 +254,17 @@ relation_count_in_query(PG_FUNCTION_ARGS) char *queryStringChar = text_to_cstring(queryString); List *parseTreeList = pg_parse_query(queryStringChar); - ListCell *parseTreeCell = NULL; - foreach(parseTreeCell, parseTreeList) + Node *parsetree = NULL; + foreach_ptr(parsetree, parseTreeList) { - Node *parsetree = (Node *) lfirst(parseTreeCell); - ListCell *queryTreeCell = NULL; - List *queryTreeList = pg_analyze_and_rewrite((RawStmt *) parsetree, queryStringChar, NULL, 0, NULL); - foreach(queryTreeCell, queryTreeList) + Query *query = NULL; + foreach_ptr(query, queryTreeList) { - Query *query = lfirst(queryTreeCell); List *rangeTableList = NIL; ExtractRangeTableRelationWalker((Node *) query, &rangeTableList); diff --git a/src/backend/distributed/test/metadata_sync.c b/src/backend/distributed/test/metadata_sync.c index 9d4a31cf4..e7450454a 100644 --- a/src/backend/distributed/test/metadata_sync.c +++ b/src/backend/distributed/test/metadata_sync.c @@ -40,7 +40,6 @@ master_metadata_snapshot(PG_FUNCTION_ARGS) List *dropSnapshotCommands = MetadataDropCommands(); List *createSnapshotCommands = MetadataCreateCommands(); List *snapshotCommandList = NIL; - ListCell *snapshotCommandCell = NULL; int snapshotCommandIndex = 0; Oid ddlCommandTypeId = TEXTOID; @@ -50,9 +49,9 @@ master_metadata_snapshot(PG_FUNCTION_ARGS) int snapshotCommandCount = list_length(snapshotCommandList); Datum *snapshotCommandDatumArray = palloc0(snapshotCommandCount * sizeof(Datum)); - foreach(snapshotCommandCell, snapshotCommandList) + const char *metadataSnapshotCommand = NULL; + foreach_ptr(metadataSnapshotCommand, snapshotCommandList) { - char *metadataSnapshotCommand = (char *) lfirst(snapshotCommandCell); Datum metadataSnapshotCommandDatum = CStringGetTextDatum(metadataSnapshotCommand); snapshotCommandDatumArray[snapshotCommandIndex] = metadataSnapshotCommandDatum; @@ -77,13 +76,11 @@ wait_until_metadata_sync(PG_FUNCTION_ARGS) uint32 timeout = PG_GETARG_UINT32(0); List *workerList = ActivePrimaryWorkerNodeList(NoLock); - ListCell *workerCell = NULL; bool waitNotifications = false; - foreach(workerCell, workerList) + WorkerNode *workerNode = NULL; + foreach_ptr(workerNode, workerList) { - WorkerNode *workerNode = (WorkerNode *) lfirst(workerCell); - /* if already has metadata, no need to do it again */ if (workerNode->hasMetadata && !workerNode->metadataSynced) { diff --git a/src/backend/distributed/test/partitioning_utils.c b/src/backend/distributed/test/partitioning_utils.c index 8dcd81549..95adaddf6 100644 --- a/src/backend/distributed/test/partitioning_utils.c +++ b/src/backend/distributed/test/partitioning_utils.c @@ -80,14 +80,11 @@ print_partitions(PG_FUNCTION_ARGS) StringInfo resultRelationNames = makeStringInfo(); List *partitionList = PartitionList(PG_GETARG_OID(0)); - ListCell *partitionOidCell = NULL; - partitionList = SortList(partitionList, CompareOids); - foreach(partitionOidCell, partitionList) + Oid partitionOid = InvalidOid; + foreach_oid(partitionOid, partitionList) { - Oid partitionOid = lfirst_oid(partitionOidCell); - /* at least one table is already added, add comma */ if (resultRelationNames->len > 0) { diff --git a/src/backend/distributed/test/progress_utils.c b/src/backend/distributed/test/progress_utils.c index 2d9475a6b..0d776c963 100644 --- a/src/backend/distributed/test/progress_utils.c +++ b/src/backend/distributed/test/progress_utils.c @@ -18,6 +18,7 @@ #include +#include "distributed/listutils.h" #include "distributed/multi_progress.h" #include "distributed/tuplestore.h" #include "nodes/execnodes.h" @@ -86,13 +87,12 @@ show_progress(PG_FUNCTION_ARGS) uint64 magicNumber = PG_GETARG_INT64(0); List *attachedDSMSegments = NIL; List *monitorList = ProgressMonitorList(magicNumber, &attachedDSMSegments); - ListCell *monitorCell = NULL; TupleDesc tupdesc; Tuplestorestate *tupstore = SetupTuplestore(fcinfo, &tupdesc); - foreach(monitorCell, monitorList) + ProgressMonitorData *monitor = NULL; + foreach_ptr(monitor, monitorList) { - ProgressMonitorData *monitor = lfirst(monitorCell); uint64 *steps = monitor->steps; for (int stepIndex = 0; stepIndex < monitor->stepCount; stepIndex++) diff --git a/src/backend/distributed/test/prune_shard_list.c b/src/backend/distributed/test/prune_shard_list.c index 60100eafd..f735964a8 100644 --- a/src/backend/distributed/test/prune_shard_list.c +++ b/src/backend/distributed/test/prune_shard_list.c @@ -206,7 +206,6 @@ MakeTextPartitionExpression(Oid distributedTableId, text *value) static ArrayType * PrunedShardIdsForTable(Oid distributedTableId, List *whereClauseList) { - ListCell *shardCell = NULL; int shardIdIndex = 0; Oid shardIdTypeId = INT8OID; Index tableId = 1; @@ -217,10 +216,10 @@ PrunedShardIdsForTable(Oid distributedTableId, List *whereClauseList) int shardIdCount = list_length(shardList); Datum *shardIdDatumArray = palloc0(shardIdCount * sizeof(Datum)); - foreach(shardCell, shardList) + ShardInterval *shardInterval = NULL; + foreach_ptr(shardInterval, shardList) { - ShardInterval *shardId = (ShardInterval *) lfirst(shardCell); - Datum shardIdDatum = Int64GetDatum(shardId->shardId); + Datum shardIdDatum = Int64GetDatum(shardInterval->shardId); shardIdDatumArray[shardIdIndex] = shardIdDatum; shardIdIndex++; diff --git a/src/backend/distributed/transaction/backend_data.c b/src/backend/distributed/transaction/backend_data.c index 268d9b521..c29f8a6cb 100644 --- a/src/backend/distributed/transaction/backend_data.c +++ b/src/backend/distributed/transaction/backend_data.c @@ -215,9 +215,7 @@ get_global_active_transactions(PG_FUNCTION_ARGS) { TupleDesc tupleDescriptor = NULL; List *workerNodeList = ActivePrimaryWorkerNodeList(NoLock); - ListCell *workerNodeCell = NULL; List *connectionList = NIL; - ListCell *connectionCell = NULL; StringInfo queryToSend = makeStringInfo(); CheckCitusVersion(ERROR); @@ -229,10 +227,10 @@ get_global_active_transactions(PG_FUNCTION_ARGS) StoreAllActiveTransactions(tupleStore, tupleDescriptor); /* open connections in parallel */ - foreach(workerNodeCell, workerNodeList) + WorkerNode *workerNode = NULL; + foreach_ptr(workerNode, workerNodeList) { - WorkerNode *workerNode = (WorkerNode *) lfirst(workerNodeCell); - char *nodeName = workerNode->workerName; + const char *nodeName = workerNode->workerName; int nodePort = workerNode->workerPort; int connectionFlags = 0; @@ -251,10 +249,9 @@ get_global_active_transactions(PG_FUNCTION_ARGS) FinishConnectionListEstablishment(connectionList); /* send commands in parallel */ - foreach(connectionCell, connectionList) + MultiConnection *connection = NULL; + foreach_ptr(connection, connectionList) { - MultiConnection *connection = (MultiConnection *) lfirst(connectionCell); - int querySent = SendRemoteCommand(connection, queryToSend->data); if (querySent == 0) { @@ -263,9 +260,8 @@ get_global_active_transactions(PG_FUNCTION_ARGS) } /* receive query results */ - foreach(connectionCell, connectionList) + foreach_ptr(connection, connectionList) { - MultiConnection *connection = (MultiConnection *) lfirst(connectionCell); bool raiseInterrupts = true; Datum values[ACTIVE_TRANSACTION_COLUMN_COUNT]; bool isNulls[ACTIVE_TRANSACTION_COLUMN_COUNT]; diff --git a/src/backend/distributed/transaction/citus_dist_stat_activity.c b/src/backend/distributed/transaction/citus_dist_stat_activity.c index d383fee82..1af7e2347 100644 --- a/src/backend/distributed/transaction/citus_dist_stat_activity.c +++ b/src/backend/distributed/transaction/citus_dist_stat_activity.c @@ -312,9 +312,7 @@ static List * CitusStatActivity(const char *statQuery) { List *workerNodeList = ActivePrimaryWorkerNodeList(NoLock); - ListCell *workerNodeCell = NULL; List *connectionList = NIL; - ListCell *connectionCell = NULL; /* * For the local node, we can avoid opening connections. This might be @@ -332,10 +330,10 @@ CitusStatActivity(const char *statQuery) char *nodeUser = CurrentUserName(); /* open connections in parallel */ - foreach(workerNodeCell, workerNodeList) + WorkerNode *workerNode = NULL; + foreach_ptr(workerNode, workerNodeList) { - WorkerNode *workerNode = (WorkerNode *) lfirst(workerNodeCell); - char *nodeName = workerNode->workerName; + const char *nodeName = workerNode->workerName; int nodePort = workerNode->workerPort; int connectionFlags = 0; @@ -355,10 +353,9 @@ CitusStatActivity(const char *statQuery) FinishConnectionListEstablishment(connectionList); /* send commands in parallel */ - foreach(connectionCell, connectionList) + MultiConnection *connection = NULL; + foreach_ptr(connection, connectionList) { - MultiConnection *connection = (MultiConnection *) lfirst(connectionCell); - int querySent = SendRemoteCommand(connection, statQuery); if (querySent == 0) { @@ -367,9 +364,8 @@ CitusStatActivity(const char *statQuery) } /* receive query results */ - foreach(connectionCell, connectionList) + foreach_ptr(connection, connectionList) { - MultiConnection *connection = (MultiConnection *) lfirst(connectionCell); bool raiseInterrupts = true; PGresult *result = GetRemoteCommandResult(connection, raiseInterrupts); @@ -424,8 +420,6 @@ GetLocalNodeCitusDistStat(const char *statQuery) { List *citusStatsList = NIL; - ListCell *workerNodeCell = NULL; - if (IsCoordinator()) { /* @@ -442,13 +436,12 @@ GetLocalNodeCitusDistStat(const char *statQuery) /* get the current worker's node stats */ List *workerNodeList = ActivePrimaryWorkerNodeList(NoLock); - foreach(workerNodeCell, workerNodeList) + WorkerNode *workerNode = NULL; + foreach_ptr(workerNode, workerNodeList) { - WorkerNode *workerNode = (WorkerNode *) lfirst(workerNodeCell); - if (workerNode->groupId == localGroupId) { - char *nodeName = workerNode->workerName; + const char *nodeName = workerNode->workerName; int nodePort = workerNode->workerPort; citusStatsList = LocalNodeCitusDistStat(statQuery, nodeName, nodePort); @@ -908,15 +901,12 @@ ParseXIDField(PGresult *result, int rowIndex, int colIndex) static void ReturnCitusDistStats(List *citusStatsList, FunctionCallInfo fcinfo) { - ListCell *citusStatsCell = NULL; - TupleDesc tupleDesc; Tuplestorestate *tupleStore = SetupTuplestore(fcinfo, &tupleDesc); - foreach(citusStatsCell, citusStatsList) + CitusDistStat *citusDistStat = NULL; + foreach_ptr(citusDistStat, citusStatsList) { - CitusDistStat *citusDistStat = (CitusDistStat *) lfirst(citusStatsCell); - Datum values[CITUS_DIST_STAT_ACTIVITY_COLS]; bool nulls[CITUS_DIST_STAT_ACTIVITY_COLS]; diff --git a/src/backend/distributed/transaction/distributed_deadlock_detection.c b/src/backend/distributed/transaction/distributed_deadlock_detection.c index 1d7e81909..112aafc92 100644 --- a/src/backend/distributed/transaction/distributed_deadlock_detection.c +++ b/src/backend/distributed/transaction/distributed_deadlock_detection.c @@ -18,10 +18,10 @@ #include "distributed/backend_data.h" #include "distributed/distributed_deadlock_detection.h" #include "distributed/errormessage.h" -#include "distributed/log_utils.h" #include "distributed/hash_helpers.h" #include "distributed/listutils.h" #include "distributed/lock_graph.h" +#include "distributed/log_utils.h" #include "distributed/metadata_cache.h" #include "distributed/transaction_identifier.h" #include "nodes/pg_list.h" @@ -153,7 +153,6 @@ CheckForDistributedDeadlocks(void) if (deadlockFound) { TransactionNode *youngestAliveTransaction = NULL; - ListCell *participantTransactionCell = NULL; /* * There should generally be at least two transactions to get into a @@ -174,10 +173,9 @@ CheckForDistributedDeadlocks(void) * We're also searching for the youngest transactions initiated by * this node. */ - foreach(participantTransactionCell, deadlockPath) + TransactionNode *currentNode = NULL; + foreach_ptr(currentNode, deadlockPath) { - TransactionNode *currentNode = - (TransactionNode *) lfirst(participantTransactionCell); bool transactionAssociatedWithProc = AssociateDistributedTransactionWithBackendProc(currentNode); @@ -299,16 +297,13 @@ static void PrependOutgoingNodesToQueue(TransactionNode *transactionNode, int currentStackDepth, List **toBeVisitedNodes) { - ListCell *currentWaitForCell = NULL; - /* as we traverse outgoing edges, increment the depth */ currentStackDepth++; /* prepend to the list to continue depth-first search */ - foreach(currentWaitForCell, transactionNode->waitsFor) + TransactionNode *waitForTransaction = NULL; + foreach_ptr(waitForTransaction, transactionNode->waitsFor) { - TransactionNode *waitForTransaction = - (TransactionNode *) lfirst(currentWaitForCell); QueuedTransactionNode *queuedNode = palloc0(sizeof(QueuedTransactionNode)); queuedNode->transactionNode = waitForTransaction; @@ -670,12 +665,10 @@ char * WaitsForToString(List *waitsFor) { StringInfo transactionIdStr = makeStringInfo(); - ListCell *waitsForCell = NULL; - foreach(waitsForCell, waitsFor) + TransactionNode *waitingNode = NULL; + foreach_ptr(waitingNode, waitsFor) { - TransactionNode *waitingNode = (TransactionNode *) lfirst(waitsForCell); - if (transactionIdStr->len != 0) { appendStringInfoString(transactionIdStr, ","); diff --git a/src/backend/distributed/transaction/lock_graph.c b/src/backend/distributed/transaction/lock_graph.c index 70d7d9fe0..417becd10 100644 --- a/src/backend/distributed/transaction/lock_graph.c +++ b/src/backend/distributed/transaction/lock_graph.c @@ -21,6 +21,7 @@ #include "distributed/backend_data.h" #include "distributed/connection_management.h" #include "distributed/hash_helpers.h" +#include "distributed/listutils.h" #include "distributed/lock_graph.h" #include "distributed/metadata_cache.h" #include "distributed/remote_commands.h" @@ -90,19 +91,17 @@ WaitGraph * BuildGlobalWaitGraph(void) { List *workerNodeList = ActiveReadableNodeList(); - ListCell *workerNodeCell = NULL; char *nodeUser = CitusExtensionOwnerName(); List *connectionList = NIL; - ListCell *connectionCell = NULL; int localNodeId = GetLocalGroupId(); WaitGraph *waitGraph = BuildLocalWaitGraph(); /* open connections in parallel */ - foreach(workerNodeCell, workerNodeList) + WorkerNode *workerNode = NULL; + foreach_ptr(workerNode, workerNodeList) { - WorkerNode *workerNode = (WorkerNode *) lfirst(workerNodeCell); - char *nodeName = workerNode->workerName; + const char *nodeName = workerNode->workerName; int nodePort = workerNode->workerPort; int connectionFlags = 0; @@ -122,9 +121,9 @@ BuildGlobalWaitGraph(void) FinishConnectionListEstablishment(connectionList); /* send commands in parallel */ - foreach(connectionCell, connectionList) + MultiConnection *connection = NULL; + foreach_ptr(connection, connectionList) { - MultiConnection *connection = (MultiConnection *) lfirst(connectionCell); const char *command = "SELECT * FROM dump_local_wait_edges()"; int querySent = SendRemoteCommand(connection, command); @@ -135,9 +134,8 @@ BuildGlobalWaitGraph(void) } /* receive dump_local_wait_edges results */ - foreach(connectionCell, connectionList) + foreach_ptr(connection, connectionList) { - MultiConnection *connection = (MultiConnection *) lfirst(connectionCell); bool raiseInterrupts = true; PGresult *result = GetRemoteCommandResult(connection, raiseInterrupts); diff --git a/src/backend/distributed/transaction/relation_access_tracking.c b/src/backend/distributed/transaction/relation_access_tracking.c index 4139e656d..cef71e5c8 100644 --- a/src/backend/distributed/transaction/relation_access_tracking.c +++ b/src/backend/distributed/transaction/relation_access_tracking.c @@ -20,6 +20,7 @@ #include "access/xact.h" #include "distributed/colocation_utils.h" #include "distributed/hash_helpers.h" +#include "distributed/listutils.h" #include "distributed/multi_executor.h" #include "distributed/multi_join_order.h" #include "distributed/multi_partitioning_utils.h" @@ -325,7 +326,6 @@ RecordParallelRelationAccessForTaskList(List *taskList) static void RecordRelationParallelSelectAccessForTask(Task *task) { - ListCell *relationShardCell = NULL; Oid lastRelationId = InvalidOid; /* no point in recoding accesses in non-transaction blocks, skip the loop */ @@ -336,9 +336,9 @@ RecordRelationParallelSelectAccessForTask(Task *task) List *relationShardList = task->relationShardList; - foreach(relationShardCell, relationShardList) + RelationShard *relationShard = NULL; + foreach_ptr(relationShard, relationShardList) { - RelationShard *relationShard = (RelationShard *) lfirst(relationShardCell); Oid currentRelationId = relationShard->relationId; /* @@ -367,7 +367,6 @@ 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 */ @@ -382,9 +381,9 @@ RecordRelationParallelModifyAccessForTask(Task *task) if (task->modifyWithSubquery) { relationShardList = task->relationShardList; - foreach(relationShardCell, relationShardList) + RelationShard *relationShard = NULL; + foreach_ptr(relationShard, relationShardList) { - RelationShard *relationShard = (RelationShard *) lfirst(relationShardCell); Oid currentRelationId = relationShard->relationId; /* @@ -414,12 +413,11 @@ static void RecordRelationParallelDDLAccessForTask(Task *task) { List *relationShardList = task->relationShardList; - ListCell *relationShardCell = NULL; Oid lastRelationId = InvalidOid; - foreach(relationShardCell, relationShardList) + RelationShard *relationShard = NULL; + foreach_ptr(relationShard, relationShardList) { - RelationShard *relationShard = (RelationShard *) lfirst(relationShardCell); Oid currentRelationId = relationShard->relationId; /* @@ -493,12 +491,10 @@ RecordParallelRelationAccess(Oid relationId, ShardPlacementAccessType placementA if (PartitionedTable(relationId)) { List *partitionList = PartitionList(relationId); - ListCell *partitionCell = NULL; - foreach(partitionCell, partitionList) + Oid partitionOid = InvalidOid; + foreach_oid(partitionOid, partitionList) { - Oid partitionOid = lfirst_oid(partitionCell); - /* recursively record all relation accesses of its partitions */ RecordParallelRelationAccess(partitionOid, placementAccess); } @@ -887,12 +883,10 @@ HoldsConflictingLockWithReferencedRelations(Oid relationId, ShardPlacementAccess conflictingAccessMode) { DistTableCacheEntry *cacheEntry = DistributedTableCacheEntry(relationId); - ListCell *referencedRelationCell = NULL; - foreach(referencedRelationCell, cacheEntry->referencedRelationsViaForeignKey) + Oid referencedRelation = InvalidOid; + foreach_oid(referencedRelation, cacheEntry->referencedRelationsViaForeignKey) { - Oid referencedRelation = lfirst_oid(referencedRelationCell); - /* we're only interested in foreign keys to reference tables */ if (PartitionMethod(referencedRelation) != DISTRIBUTE_BY_NONE) { @@ -954,15 +948,13 @@ HoldsConflictingLockWithReferencingRelations(Oid relationId, ShardPlacementAcces conflictingAccessMode) { DistTableCacheEntry *cacheEntry = DistributedTableCacheEntry(relationId); - ListCell *referencingRelationCell = NULL; bool holdsConflictingLocks = false; Assert(PartitionMethod(relationId) == DISTRIBUTE_BY_NONE); - foreach(referencingRelationCell, cacheEntry->referencingRelationsViaForeignKey) + Oid referencingRelation = InvalidOid; + foreach_oid(referencingRelation, cacheEntry->referencingRelationsViaForeignKey) { - Oid referencingRelation = lfirst_oid(referencingRelationCell); - /* * We're only interested in foreign keys to reference tables from * hash distributed tables. diff --git a/src/backend/distributed/transaction/remote_transaction.c b/src/backend/distributed/transaction/remote_transaction.c index 6c4a011a7..0a2fd9d6b 100644 --- a/src/backend/distributed/transaction/remote_transaction.c +++ b/src/backend/distributed/transaction/remote_transaction.c @@ -18,6 +18,7 @@ #include "distributed/backend_data.h" #include "distributed/citus_safe_lib.h" #include "distributed/connection_management.h" +#include "distributed/listutils.h" #include "distributed/metadata_cache.h" #include "distributed/remote_commands.h" #include "distributed/remote_transaction.h" @@ -59,7 +60,6 @@ void StartRemoteTransactionBegin(struct MultiConnection *connection) { RemoteTransaction *transaction = &connection->remoteTransaction; - ListCell *subIdCell = NULL; Assert(transaction->transactionState == REMOTE_TRANS_NOT_STARTED); @@ -75,10 +75,10 @@ StartRemoteTransactionBegin(struct MultiConnection *connection) List *activeSubXacts = ActiveSubXactContexts(); transaction->lastSuccessfulSubXact = TopSubTransactionId; transaction->lastQueuedSubXact = TopSubTransactionId; - foreach(subIdCell, activeSubXacts) - { - SubXactContext *subXactState = lfirst(subIdCell); + SubXactContext *subXactState = NULL; + foreach_ptr(subXactState, activeSubXacts) + { /* append SET LOCAL state from when SAVEPOINT was encountered... */ if (subXactState->setLocalCmds != NULL) { @@ -189,21 +189,17 @@ RemoteTransactionBegin(struct MultiConnection *connection) void RemoteTransactionListBegin(List *connectionList) { - ListCell *connectionCell = NULL; + MultiConnection *connection = NULL; /* send BEGIN to all nodes */ - foreach(connectionCell, connectionList) + foreach_ptr(connection, connectionList) { - MultiConnection *connection = (MultiConnection *) lfirst(connectionCell); - StartRemoteTransactionBegin(connection); } /* wait for BEGIN to finish on all nodes */ - foreach(connectionCell, connectionList) + foreach_ptr(connection, connectionList) { - MultiConnection *connection = (MultiConnection *) lfirst(connectionCell); - FinishRemoteTransactionBegin(connection); } } @@ -605,7 +601,7 @@ RemoteTransactionBeginIfNecessary(MultiConnection *connection) void RemoteTransactionsBeginIfNecessary(List *connectionList) { - ListCell *connectionCell = NULL; + MultiConnection *connection = NULL; /* * Don't do anything if not in a coordinated transaction. That allows the @@ -618,9 +614,8 @@ RemoteTransactionsBeginIfNecessary(List *connectionList) } /* issue BEGIN to all connections needing it */ - foreach(connectionCell, connectionList) + foreach_ptr(connection, connectionList) { - MultiConnection *connection = (MultiConnection *) lfirst(connectionCell); RemoteTransaction *transaction = &connection->remoteTransaction; /* can't send BEGIN if a command already is in progress */ @@ -643,9 +638,8 @@ RemoteTransactionsBeginIfNecessary(List *connectionList) WaitForAllConnections(connectionList, raiseInterrupts); /* get result of all the BEGINs */ - foreach(connectionCell, connectionList) + foreach_ptr(connection, connectionList) { - MultiConnection *connection = (MultiConnection *) lfirst(connectionCell); RemoteTransaction *transaction = &connection->remoteTransaction; /* diff --git a/src/backend/distributed/transaction/transaction_management.c b/src/backend/distributed/transaction/transaction_management.c index a547ddec7..2dfedf353 100644 --- a/src/backend/distributed/transaction/transaction_management.c +++ b/src/backend/distributed/transaction/transaction_management.c @@ -25,6 +25,7 @@ #include "distributed/distributed_planner.h" #include "distributed/hash_helpers.h" #include "distributed/intermediate_results.h" +#include "distributed/listutils.h" #include "distributed/local_executor.h" #include "distributed/multi_executor.h" #include "distributed/transaction_management.h" @@ -595,16 +596,15 @@ PopSubXact(SubTransactionId subId) List * ActiveSubXacts(void) { - ListCell *subXactCell = NULL; List *activeSubXactsReversed = NIL; /* * activeSubXactContexts is in reversed temporal order, so we reverse it to get it * in temporal order. */ - foreach(subXactCell, activeSubXactContexts) + SubXactContext *state = NULL; + foreach_ptr(state, activeSubXactContexts) { - SubXactContext *state = lfirst(subXactCell); activeSubXactsReversed = lcons_int(state->subId, activeSubXactsReversed); } @@ -616,16 +616,15 @@ ActiveSubXacts(void) List * ActiveSubXactContexts(void) { - ListCell *subXactCell = NULL; List *reversedSubXactStates = NIL; /* * activeSubXactContexts is in reversed temporal order, so we reverse it to get it * in temporal order. */ - foreach(subXactCell, activeSubXactContexts) + SubXactContext *state = NULL; + foreach_ptr(state, activeSubXactContexts) { - SubXactContext *state = lfirst(subXactCell); reversedSubXactStates = lcons(state, reversedSubXactStates); } diff --git a/src/backend/distributed/transaction/transaction_recovery.c b/src/backend/distributed/transaction/transaction_recovery.c index 64736c9da..132db008f 100644 --- a/src/backend/distributed/transaction/transaction_recovery.c +++ b/src/backend/distributed/transaction/transaction_recovery.c @@ -113,15 +113,12 @@ LogTransactionRecord(int32 groupId, char *transactionName) int RecoverTwoPhaseCommits(void) { - ListCell *workerNodeCell = NULL; int recoveredTransactionCount = 0; List *workerList = ActivePrimaryNodeList(NoLock); - - foreach(workerNodeCell, workerList) + WorkerNode *workerNode = NULL; + foreach_ptr(workerNode, workerList) { - WorkerNode *workerNode = (WorkerNode *) lfirst(workerNodeCell); - recoveredTransactionCount += RecoverWorkerTransactions(workerNode); } diff --git a/src/backend/distributed/transaction/worker_transaction.c b/src/backend/distributed/transaction/worker_transaction.c index 1de3e9e79..f8be9cd4e 100644 --- a/src/backend/distributed/transaction/worker_transaction.c +++ b/src/backend/distributed/transaction/worker_transaction.c @@ -44,12 +44,13 @@ static void SendCommandToWorkersParamsInternal(TargetWorkerSet targetWorkerSet, const char *const *parameterValues); static void ErrorIfAnyMetadataNodeOutOfSync(List *metadataNodeList); static void SendCommandListToAllWorkersInternal(List *commandList, bool failOnError, - char *superuser); -static List * OpenConnectionsToWorkersInParallel(TargetWorkerSet targetWorkerSet, const - char *user); + const char *superuser); +static List * OpenConnectionsToWorkersInParallel(TargetWorkerSet targetWorkerSet, + const char *user); static void GetConnectionsResults(List *connectionList, bool failOnError); -static void SendCommandToWorkersOutsideTransaction(TargetWorkerSet targetWorkerSet, const - char *command, const char *user, bool +static void SendCommandToWorkersOutsideTransaction(TargetWorkerSet targetWorkerSet, + const char *command, const char *user, + bool failOnError); /* @@ -57,7 +58,7 @@ static void SendCommandToWorkersOutsideTransaction(TargetWorkerSet targetWorkerS * 2PC. */ void -SendCommandToWorker(char *nodeName, int32 nodePort, const char *command) +SendCommandToWorker(const char *nodeName, int32 nodePort, const char *command) { const char *nodeUser = CitusExtensionOwnerName(); SendCommandToWorkerAsUser(nodeName, nodePort, nodeUser, command); @@ -73,13 +74,12 @@ SendCommandToWorkersAsUser(TargetWorkerSet targetWorkerSet, const char *nodeUser const char *command) { List *workerNodeList = TargetWorkerSetNodeList(targetWorkerSet, ShareLock); - ListCell *workerNodeCell = NULL; /* run commands serially */ - foreach(workerNodeCell, workerNodeList) + WorkerNode *workerNode = NULL; + foreach_ptr(workerNode, workerNodeList) { - WorkerNode *workerNode = (WorkerNode *) lfirst(workerNodeCell); - char *nodeName = workerNode->workerName; + const char *nodeName = workerNode->workerName; int nodePort = workerNode->workerPort; SendCommandToWorkerAsUser(nodeName, nodePort, nodeUser, command); @@ -92,7 +92,7 @@ SendCommandToWorkersAsUser(TargetWorkerSet targetWorkerSet, const char *nodeUser * as part of the 2PC. */ void -SendCommandToWorkerAsUser(char *nodeName, int32 nodePort, const char *nodeUser, +SendCommandToWorkerAsUser(const char *nodeName, int32 nodePort, const char *nodeUser, const char *command) { uint32 connectionFlags = 0; @@ -130,9 +130,9 @@ SendCommandToWorkersWithMetadata(const char *command) * all workers as a superuser. */ void -SendCommandToAllWorkers(char *command, char *superuser) +SendCommandToAllWorkers(const char *command, const char *superuser) { - SendCommandListToAllWorkers(list_make1(command), superuser); + SendCommandListToAllWorkers(list_make1((char *) command), superuser); } @@ -141,7 +141,7 @@ SendCommandToAllWorkers(char *command, char *superuser) * a single transaction. */ void -SendCommandListToAllWorkers(List *commandList, char *superuser) +SendCommandListToAllWorkers(List *commandList, const char *superuser) { SendCommandListToAllWorkersInternal(commandList, true, superuser); } @@ -153,14 +153,14 @@ SendCommandListToAllWorkers(List *commandList, char *superuser) * workers even if it fails in one of them. */ static void -SendCommandListToAllWorkersInternal(List *commandList, bool failOnError, char *superuser) +SendCommandListToAllWorkersInternal(List *commandList, bool failOnError, const + char *superuser) { - ListCell *workerNodeCell = NULL; List *workerNodeList = ActivePrimaryWorkerNodeList(NoLock); - foreach(workerNodeCell, workerNodeList) + WorkerNode *workerNode = NULL; + foreach_ptr(workerNode, workerNodeList) { - WorkerNode *workerNode = (WorkerNode *) lfirst(workerNodeCell); if (failOnError) { SendCommandListToWorkerInSingleTransaction(workerNode->workerName, @@ -185,7 +185,7 @@ SendCommandListToAllWorkersInternal(List *commandList, bool failOnError, char *s * so this method doesnt return any error. */ void -SendOptionalCommandListToAllWorkers(List *commandList, char *superuser) +SendOptionalCommandListToAllWorkers(List *commandList, const char *superuser) { SendCommandListToAllWorkersInternal(commandList, false, superuser); } @@ -199,13 +199,11 @@ List * TargetWorkerSetNodeList(TargetWorkerSet targetWorkerSet, LOCKMODE lockMode) { List *workerNodeList = ActivePrimaryWorkerNodeList(lockMode); - ListCell *workerNodeCell = NULL; List *result = NIL; - foreach(workerNodeCell, workerNodeList) + WorkerNode *workerNode = NULL; + foreach_ptr(workerNode, workerNodeList) { - WorkerNode *workerNode = (WorkerNode *) lfirst(workerNodeCell); - if (targetWorkerSet == WORKERS_WITH_METADATA && !workerNode->hasMetadata) { continue; @@ -235,17 +233,15 @@ SendBareCommandListToMetadataWorkers(List *commandList) { TargetWorkerSet targetWorkerSet = WORKERS_WITH_METADATA; List *workerNodeList = TargetWorkerSetNodeList(targetWorkerSet, ShareLock); - ListCell *workerNodeCell = NULL; char *nodeUser = CitusExtensionOwnerName(); - ListCell *commandCell = NULL; ErrorIfAnyMetadataNodeOutOfSync(workerNodeList); /* run commands serially */ - foreach(workerNodeCell, workerNodeList) + WorkerNode *workerNode = NULL; + foreach_ptr(workerNode, workerNodeList) { - WorkerNode *workerNode = (WorkerNode *) lfirst(workerNodeCell); - char *nodeName = workerNode->workerName; + const char *nodeName = workerNode->workerName; int nodePort = workerNode->workerPort; int connectionFlags = FORCE_NEW_CONNECTION; @@ -255,10 +251,9 @@ SendBareCommandListToMetadataWorkers(List *commandList) nodeUser, NULL); /* iterate over the commands and execute them in the same connection */ - foreach(commandCell, commandList) + const char *commandString = NULL; + foreach_ptr(commandString, commandList) { - char *commandString = lfirst(commandCell); - ExecuteCriticalRemoteCommand(workerConnection, commandString); } @@ -277,15 +272,13 @@ SendBareOptionalCommandListToAllWorkersAsUser(List *commandList, const char *use { TargetWorkerSet targetWorkerSet = ALL_WORKERS; List *workerNodeList = TargetWorkerSetNodeList(targetWorkerSet, ShareLock); - ListCell *workerNodeCell = NULL; - ListCell *commandCell = NULL; int maxError = RESPONSE_OKAY; /* run commands serially */ - foreach(workerNodeCell, workerNodeList) + WorkerNode *workerNode = NULL; + foreach_ptr(workerNode, workerNodeList) { - WorkerNode *workerNode = (WorkerNode *) lfirst(workerNodeCell); - char *nodeName = workerNode->workerName; + const char *nodeName = workerNode->workerName; int nodePort = workerNode->workerPort; int connectionFlags = FORCE_NEW_CONNECTION; @@ -295,9 +288,9 @@ SendBareOptionalCommandListToAllWorkersAsUser(List *commandList, const char *use NULL); /* iterate over the commands and execute them in the same connection */ - foreach(commandCell, commandList) + const char *commandString = NULL; + foreach_ptr(commandString, commandList) { - char *commandString = lfirst(commandCell); int result = ExecuteOptionalRemoteCommand(workerConnection, commandString, NULL); if (result != RESPONSE_OKAY) @@ -374,18 +367,15 @@ SendCommandToWorkersOutsideTransaction(TargetWorkerSet targetWorkerSet, const char *command, const char *user, bool failOnError) { - ListCell *connectionCell = NULL; - List *connectionList = OpenConnectionsToWorkersInParallel(targetWorkerSet, user); /* finish opening connections */ FinishConnectionListEstablishment(connectionList); /* send commands in parallel */ - foreach(connectionCell, connectionList) + MultiConnection *connection = NULL; + foreach_ptr(connection, connectionList) { - MultiConnection *connection = (MultiConnection *) lfirst(connectionCell); - int querySent = SendRemoteCommand(connection, command); if (failOnError && querySent == 0) { @@ -404,15 +394,14 @@ SendCommandToWorkersOutsideTransaction(TargetWorkerSet targetWorkerSet, const static List * OpenConnectionsToWorkersInParallel(TargetWorkerSet targetWorkerSet, const char *user) { - ListCell *workerNodeCell = NULL; List *connectionList = NIL; List *workerNodeList = TargetWorkerSetNodeList(targetWorkerSet, ShareLock); - foreach(workerNodeCell, workerNodeList) + WorkerNode *workerNode = NULL; + foreach_ptr(workerNode, workerNodeList) { - WorkerNode *workerNode = (WorkerNode *) lfirst(workerNodeCell); - char *nodeName = workerNode->workerName; + const char *nodeName = workerNode->workerName; int nodePort = workerNode->workerPort; int32 connectionFlags = OUTSIDE_TRANSACTION; @@ -432,11 +421,9 @@ OpenConnectionsToWorkersInParallel(TargetWorkerSet targetWorkerSet, const char * static void GetConnectionsResults(List *connectionList, bool failOnError) { - ListCell *connectionCell = NULL; - - foreach(connectionCell, connectionList) + MultiConnection *connection = NULL; + foreach_ptr(connection, connectionList) { - MultiConnection *connection = (MultiConnection *) lfirst(connectionCell); bool raiseInterrupt = false; PGresult *result = GetRemoteCommandResult(connection, raiseInterrupt); @@ -471,18 +458,16 @@ SendCommandToWorkersParamsInternal(TargetWorkerSet targetWorkerSet, const char * const char *const *parameterValues) { List *connectionList = NIL; - ListCell *connectionCell = NULL; List *workerNodeList = TargetWorkerSetNodeList(targetWorkerSet, ShareLock); - ListCell *workerNodeCell = NULL; UseCoordinatedTransaction(); CoordinatedTransactionUse2PC(); /* open connections in parallel */ - foreach(workerNodeCell, workerNodeList) + WorkerNode *workerNode = NULL; + foreach_ptr(workerNode, workerNodeList) { - WorkerNode *workerNode = (WorkerNode *) lfirst(workerNodeCell); - char *nodeName = workerNode->workerName; + const char *nodeName = workerNode->workerName; int nodePort = workerNode->workerPort; int32 connectionFlags = 0; @@ -501,10 +486,9 @@ SendCommandToWorkersParamsInternal(TargetWorkerSet targetWorkerSet, const char * RemoteTransactionsBeginIfNecessary(connectionList); /* send commands in parallel */ - foreach(connectionCell, connectionList) + MultiConnection *connection = NULL; + foreach_ptr(connection, connectionList) { - MultiConnection *connection = (MultiConnection *) lfirst(connectionCell); - int querySent = SendRemoteCommandParams(connection, command, parameterCount, parameterTypes, parameterValues); if (querySent == 0) @@ -514,10 +498,8 @@ SendCommandToWorkersParamsInternal(TargetWorkerSet targetWorkerSet, const char * } /* get results */ - foreach(connectionCell, connectionList) + foreach_ptr(connection, connectionList) { - MultiConnection *connection = (MultiConnection *) lfirst(connectionCell); - PGresult *result = GetRemoteCommandResult(connection, true); if (!IsResponseOK(result)) { @@ -557,7 +539,6 @@ void SendCommandListToWorkerInSingleTransaction(const char *nodeName, int32 nodePort, const char *nodeUser, List *commandList) { - ListCell *commandCell = NULL; int connectionFlags = FORCE_NEW_CONNECTION; MultiConnection *workerConnection = GetNodeUserDatabaseConnection(connectionFlags, @@ -568,10 +549,9 @@ SendCommandListToWorkerInSingleTransaction(const char *nodeName, int32 nodePort, RemoteTransactionBegin(workerConnection); /* iterate over the commands and execute them in the same connection */ - foreach(commandCell, commandList) + const char *commandString = NULL; + foreach_ptr(commandString, commandList) { - char *commandString = lfirst(commandCell); - ExecuteCriticalRemoteCommand(workerConnection, commandString); } @@ -597,12 +577,9 @@ SendCommandListToWorkerInSingleTransaction(const char *nodeName, int32 nodePort, static void ErrorIfAnyMetadataNodeOutOfSync(List *metadataNodeList) { - ListCell *workerNodeCell = NULL; - - foreach(workerNodeCell, metadataNodeList) + WorkerNode *metadataNode = NULL; + foreach_ptr(metadataNode, metadataNodeList) { - WorkerNode *metadataNode = lfirst(workerNodeCell); - Assert(metadataNode->hasMetadata); if (!metadataNode->metadataSynced) diff --git a/src/backend/distributed/utils/colocation_utils.c b/src/backend/distributed/utils/colocation_utils.c index 63e39eaa5..917a0c0fe 100644 --- a/src/backend/distributed/utils/colocation_utils.c +++ b/src/backend/distributed/utils/colocation_utils.c @@ -107,16 +107,14 @@ get_colocated_shard_array(PG_FUNCTION_ARGS) ShardInterval *shardInterval = LoadShardInterval(shardId); List *colocatedShardList = ColocatedShardIntervalList(shardInterval); - ListCell *colocatedShardCell = NULL; int colocatedShardCount = list_length(colocatedShardList); Datum *colocatedShardsDatumArray = palloc0(colocatedShardCount * sizeof(Datum)); Oid arrayTypeId = OIDOID; int colocatedShardIndex = 0; - foreach(colocatedShardCell, colocatedShardList) + ShardInterval *colocatedShardInterval = NULL; + foreach_ptr(colocatedShardInterval, colocatedShardList) { - ShardInterval *colocatedShardInterval = (ShardInterval *) lfirst( - colocatedShardCell); uint64 colocatedShardId = colocatedShardInterval->shardId; Datum colocatedShardDatum = Int64GetDatum(colocatedShardId); @@ -837,7 +835,6 @@ ColocatedShardIntervalList(ShardInterval *shardInterval) { Oid distributedTableId = shardInterval->relationId; List *colocatedShardList = NIL; - ListCell *colocatedTableCell = NULL; DistTableCacheEntry *cacheEntry = DistributedTableCacheEntry(distributedTableId); char partitionMethod = cacheEntry->partitionMethod; @@ -863,9 +860,9 @@ ColocatedShardIntervalList(ShardInterval *shardInterval) /* ShardIndex have to find index of given shard */ Assert(shardIntervalIndex >= 0); - foreach(colocatedTableCell, colocatedTableList) + Oid colocatedTableId = InvalidOid; + foreach_oid(colocatedTableId, colocatedTableList) { - Oid colocatedTableId = lfirst_oid(colocatedTableCell); DistTableCacheEntry *colocatedTableCacheEntry = DistributedTableCacheEntry(colocatedTableId); diff --git a/src/backend/distributed/utils/foreign_key_relationship.c b/src/backend/distributed/utils/foreign_key_relationship.c index 9ecb5d643..6ae9f1ee4 100644 --- a/src/backend/distributed/utils/foreign_key_relationship.c +++ b/src/backend/distributed/utils/foreign_key_relationship.c @@ -126,7 +126,6 @@ GetForeignConstraintRelationshipHelper(Oid relationId, bool isReferencing) { List *foreignConstraintList = NIL; List *foreignNodeList = NIL; - ListCell *nodeCell = NULL; bool isFound = false; CreateForeignConstraintRelationshipGraph(); @@ -151,11 +150,9 @@ GetForeignConstraintRelationshipHelper(Oid relationId, bool isReferencing) * We need only their OIDs, we get back node list to make their visited * variable to false for using them iteratively. */ - foreach(nodeCell, foreignNodeList) + ForeignConstraintRelationshipNode *currentNode = NULL; + foreach_ptr(currentNode, foreignNodeList) { - ForeignConstraintRelationshipNode *currentNode = - (ForeignConstraintRelationshipNode *) lfirst(nodeCell); - foreignConstraintList = lappend_oid(foreignConstraintList, currentNode->relationId); currentNode->visited = false; @@ -257,7 +254,6 @@ static void GetConnectedListHelper(ForeignConstraintRelationshipNode *node, List **adjacentNodeList, bool isReferencing) { - ListCell *nodeCell = NULL; List *neighbourList = NIL; node->visited = true; @@ -271,10 +267,9 @@ GetConnectedListHelper(ForeignConstraintRelationshipNode *node, List **adjacentN neighbourList = node->adjacencyList; } - foreach(nodeCell, neighbourList) + ForeignConstraintRelationshipNode *neighborNode = NULL; + foreach_ptr(neighborNode, neighbourList) { - ForeignConstraintRelationshipNode *neighborNode = - (ForeignConstraintRelationshipNode *) lfirst(nodeCell); if (neighborNode->visited == false) { *adjacentNodeList = lappend(*adjacentNodeList, neighborNode); @@ -298,7 +293,6 @@ PopulateAdjacencyLists(void) Oid prevReferencingOid = InvalidOid; Oid prevReferencedOid = InvalidOid; List *frelEdgeList = NIL; - ListCell *frelEdgeCell = NULL; Relation pgConstraint = heap_open(ConstraintRelationId, AccessShareLock); @@ -327,11 +321,9 @@ PopulateAdjacencyLists(void) */ frelEdgeList = SortList(frelEdgeList, CompareForeignConstraintRelationshipEdges); - foreach(frelEdgeCell, frelEdgeList) + ForeignConstraintRelationshipEdge *currentFConstraintRelationshipEdge = NULL; + foreach_ptr(currentFConstraintRelationshipEdge, frelEdgeList) { - ForeignConstraintRelationshipEdge *currentFConstraintRelationshipEdge = - (ForeignConstraintRelationshipEdge *) lfirst(frelEdgeCell); - /* we just saw this edge, no need to add it twice */ if (currentFConstraintRelationshipEdge->referencingRelationOID == prevReferencingOid && diff --git a/src/backend/distributed/utils/listutils.c b/src/backend/distributed/utils/listutils.c index 831418f0f..0aa0b0cd2 100644 --- a/src/backend/distributed/utils/listutils.c +++ b/src/backend/distributed/utils/listutils.c @@ -40,10 +40,9 @@ SortList(List *pointerList, int (*comparisonFunction)(const void *, const void * uint32 arraySize = (uint32) list_length(pointerList); void **array = (void **) palloc0(arraySize * sizeof(void *)); - ListCell *pointerCell = NULL; - foreach(pointerCell, pointerList) + void *pointer = NULL; + foreach_ptr(pointer, pointerList) { - void *pointer = lfirst(pointerCell); array[arrayIndex] = pointer; arrayIndex++; @@ -78,12 +77,12 @@ PointerArrayFromList(List *pointerList) { int pointerCount = list_length(pointerList); void **pointerArray = (void **) palloc0(pointerCount * sizeof(void *)); - ListCell *pointerCell = NULL; int pointerIndex = 0; - foreach(pointerCell, pointerList) + void *pointer = NULL; + foreach_ptr(pointer, pointerList) { - pointerArray[pointerIndex] = (void *) lfirst(pointerCell); + pointerArray[pointerIndex] = pointer; pointerIndex += 1; } @@ -125,7 +124,6 @@ HTAB * ListToHashSet(List *itemList, Size keySize, bool isStringList) { HASHCTL info; - ListCell *itemCell = NULL; int flags = HASH_ELEM | HASH_CONTEXT; /* allocate sufficient capacity for O(1) expected look-up time */ @@ -144,9 +142,9 @@ ListToHashSet(List *itemList, Size keySize, bool isStringList) HTAB *itemSet = hash_create("ListToHashSet", capacity, &info, flags); - foreach(itemCell, itemList) + void *item = NULL; + foreach_ptr(item, itemList) { - void *item = lfirst(itemCell); bool foundInSet = false; hash_search(itemSet, item, HASH_ENTER, &foundInSet); @@ -164,13 +162,11 @@ ListToHashSet(List *itemList, Size keySize, bool isStringList) char * StringJoin(List *stringList, char delimiter) { - ListCell *stringCell = NULL; StringInfo joinedString = makeStringInfo(); - foreach(stringCell, stringList) + const char *command = NULL; + foreach_ptr(command, stringList) { - const char *command = lfirst(stringCell); - appendStringInfoString(joinedString, command); appendStringInfoChar(joinedString, delimiter); } @@ -189,11 +185,11 @@ ListTake(List *pointerList, int size) { List *result = NIL; int listIndex = 0; - ListCell *pointerCell = NULL; - foreach(pointerCell, pointerList) + void *pointer = NULL; + foreach_ptr(pointer, pointerList) { - result = lappend(result, lfirst(pointerCell)); + result = lappend(result, pointer); listIndex++; if (listIndex >= size) { diff --git a/src/backend/distributed/utils/reference_table_utils.c b/src/backend/distributed/utils/reference_table_utils.c index 8c720a44c..8c19dc6da 100644 --- a/src/backend/distributed/utils/reference_table_utils.c +++ b/src/backend/distributed/utils/reference_table_utils.c @@ -121,22 +121,20 @@ void ReplicateAllReferenceTablesToNode(char *nodeName, int nodePort) { List *referenceTableList = ReferenceTableOidList(); - ListCell *referenceTableCell = NULL; /* if there is no reference table, we do not need to replicate anything */ if (list_length(referenceTableList) > 0) { List *referenceShardIntervalList = NIL; - ListCell *referenceShardIntervalCell = NULL; /* * We sort the reference table list to prevent deadlocks in concurrent * ReplicateAllReferenceTablesToAllNodes calls. */ referenceTableList = SortList(referenceTableList, CompareOids); - foreach(referenceTableCell, referenceTableList) + Oid referenceTableId = InvalidOid; + foreach_oid(referenceTableId, referenceTableList) { - Oid referenceTableId = lfirst_oid(referenceTableCell); List *shardIntervalList = LoadShardIntervalList(referenceTableId); ShardInterval *shardInterval = (ShardInterval *) linitial(shardIntervalList); @@ -149,10 +147,9 @@ ReplicateAllReferenceTablesToNode(char *nodeName, int nodePort) BlockWritesToShardList(referenceShardIntervalList); } - foreach(referenceShardIntervalCell, referenceShardIntervalList) + ShardInterval *shardInterval = NULL; + foreach_ptr(shardInterval, referenceShardIntervalList) { - ShardInterval *shardInterval = (ShardInterval *) lfirst( - referenceShardIntervalCell); uint64 shardId = shardInterval->shardId; LockShardDistributionMetadata(shardId, ExclusiveLock); @@ -161,10 +158,8 @@ ReplicateAllReferenceTablesToNode(char *nodeName, int nodePort) } /* create foreign constraints between reference tables */ - foreach(referenceShardIntervalCell, referenceShardIntervalList) + foreach_ptr(shardInterval, referenceShardIntervalList) { - ShardInterval *shardInterval = - (ShardInterval *) lfirst(referenceShardIntervalCell); char *tableOwner = TableOwner(shardInterval->relationId); List *commandList = CopyShardForeignConstraintCommandList(shardInterval); @@ -235,8 +230,6 @@ ReplicateSingleShardTableToAllNodes(Oid relationId) static void ReplicateShardToAllNodes(ShardInterval *shardInterval) { - ListCell *workerNodeCell = NULL; - /* prevent concurrent pg_dist_node changes */ List *workerNodeList = ReferenceTablePlacementNodeList(ShareLock); @@ -246,9 +239,9 @@ ReplicateShardToAllNodes(ShardInterval *shardInterval) * the metadata to reflect newly copied shard. */ workerNodeList = SortList(workerNodeList, CompareWorkerNodes); - foreach(workerNodeCell, workerNodeList) + WorkerNode *workerNode = NULL; + foreach_ptr(workerNode, workerNodeList) { - WorkerNode *workerNode = (WorkerNode *) lfirst(workerNodeCell); char *nodeName = workerNode->workerName; uint32 nodePort = workerNode->workerPort; @@ -407,7 +400,6 @@ DeleteAllReferenceTablePlacementsFromNodeGroup(int32 groupId) { List *referenceTableList = ReferenceTableOidList(); List *referenceShardIntervalList = NIL; - ListCell *referenceTableCell = NULL; /* if there are no reference tables, we do not need to do anything */ if (list_length(referenceTableList) == 0) @@ -427,11 +419,10 @@ DeleteAllReferenceTablePlacementsFromNodeGroup(int32 groupId) BlockWritesToShardList(referenceShardIntervalList); } - foreach(referenceTableCell, referenceTableList) + StringInfo deletePlacementCommand = makeStringInfo(); + Oid referenceTableId = InvalidOid; + foreach_oid(referenceTableId, referenceTableList) { - StringInfo deletePlacementCommand = makeStringInfo(); - - Oid referenceTableId = lfirst_oid(referenceTableCell); List *placements = GroupShardPlacementsForTableOnGroup(referenceTableId, groupId); if (list_length(placements) == 0) @@ -446,6 +437,7 @@ DeleteAllReferenceTablePlacementsFromNodeGroup(int32 groupId) DeleteShardPlacementRow(placement->placementId); + resetStringInfo(deletePlacementCommand); appendStringInfo(deletePlacementCommand, "DELETE FROM pg_dist_placement WHERE placementid = " UINT64_FORMAT, @@ -465,15 +457,12 @@ DeleteAllReferenceTablePlacementsFromNodeGroup(int32 groupId) List * ReferenceTableOidList() { - List *distTableOidList = DistTableOidList(); - ListCell *distTableOidCell = NULL; - List *referenceTableList = NIL; - foreach(distTableOidCell, distTableOidList) + List *distTableOidList = DistTableOidList(); + Oid relationId = InvalidOid; + foreach_oid(relationId, distTableOidList) { - Oid relationId = lfirst_oid(distTableOidCell); - DistTableCacheEntry *cacheEntry = DistributedTableCacheEntry(relationId); if (cacheEntry->partitionMethod == DISTRIBUTE_BY_NONE) diff --git a/src/backend/distributed/utils/resource_lock.c b/src/backend/distributed/utils/resource_lock.c index 216e2caf3..a2e8bee82 100644 --- a/src/backend/distributed/utils/resource_lock.c +++ b/src/backend/distributed/utils/resource_lock.c @@ -172,7 +172,6 @@ static void LockShardListResourcesOnFirstWorker(LOCKMODE lockmode, List *shardIntervalList) { StringInfo lockCommand = makeStringInfo(); - ListCell *shardIntervalCell = NULL; int processedShardIntervalCount = 0; int totalShardIntervalCount = list_length(shardIntervalList); WorkerNode *firstWorkerNode = GetFirstPrimaryWorkerNode(); @@ -181,9 +180,9 @@ LockShardListResourcesOnFirstWorker(LOCKMODE lockmode, List *shardIntervalList) appendStringInfo(lockCommand, "SELECT lock_shard_resources(%d, ARRAY[", lockmode); - foreach(shardIntervalCell, shardIntervalList) + ShardInterval *shardInterval = NULL; + foreach_ptr(shardInterval, shardIntervalList) { - ShardInterval *shardInterval = (ShardInterval *) lfirst(shardIntervalCell); int64 shardId = shardInterval->shardId; appendStringInfo(lockCommand, "%lu", shardId); @@ -260,7 +259,6 @@ void LockShardListMetadataOnWorkers(LOCKMODE lockmode, List *shardIntervalList) { StringInfo lockCommand = makeStringInfo(); - ListCell *shardIntervalCell = NULL; int processedShardIntervalCount = 0; int totalShardIntervalCount = list_length(shardIntervalList); @@ -271,9 +269,9 @@ LockShardListMetadataOnWorkers(LOCKMODE lockmode, List *shardIntervalList) appendStringInfo(lockCommand, "SELECT lock_shard_metadata(%d, ARRAY[", lockmode); - foreach(shardIntervalCell, shardIntervalList) + ShardInterval *shardInterval = NULL; + foreach_ptr(shardInterval, shardIntervalList) { - ShardInterval *shardInterval = (ShardInterval *) lfirst(shardIntervalCell); int64 shardId = shardInterval->shardId; appendStringInfo(lockCommand, "%lu", shardId); @@ -391,7 +389,6 @@ SetLocktagForShardDistributionMetadata(int64 shardId, LOCKTAG *tag) void LockReferencedReferenceShardDistributionMetadata(uint64 shardId, LOCKMODE lockMode) { - ListCell *shardIntervalCell = NULL; Oid relationId = RelationIdForShard(shardId); DistTableCacheEntry *cacheEntry = DistributedTableCacheEntry(relationId); @@ -403,10 +400,9 @@ LockReferencedReferenceShardDistributionMetadata(uint64 shardId, LOCKMODE lockMo LockShardListMetadataOnWorkers(lockMode, shardIntervalList); } - foreach(shardIntervalCell, shardIntervalList) + ShardInterval *shardInterval = NULL; + foreach_ptr(shardInterval, shardIntervalList) { - ShardInterval *shardInterval = (ShardInterval *) lfirst(shardIntervalCell); - LockShardDistributionMetadata(shardInterval->shardId, lockMode); } } @@ -423,7 +419,6 @@ LockReferencedReferenceShardDistributionMetadata(uint64 shardId, LOCKMODE lockMo static void LockReferencedReferenceShardResources(uint64 shardId, LOCKMODE lockMode) { - ListCell *shardIntervalCell = NULL; Oid relationId = RelationIdForShard(shardId); DistTableCacheEntry *cacheEntry = DistributedTableCacheEntry(relationId); @@ -449,10 +444,9 @@ LockReferencedReferenceShardResources(uint64 shardId, LOCKMODE lockMode) LockShardListResourcesOnFirstWorker(lockMode, referencedShardIntervalList); } - foreach(shardIntervalCell, referencedShardIntervalList) + ShardInterval *referencedShardInterval = NULL; + foreach_ptr(referencedShardInterval, referencedShardIntervalList) { - ShardInterval *referencedShardInterval = (ShardInterval *) lfirst( - shardIntervalCell); LockShardResource(referencedShardInterval->shardId, lockMode); } } @@ -466,12 +460,10 @@ List * GetSortedReferenceShardIntervals(List *relationList) { List *shardIntervalList = NIL; - ListCell *relationCell = NULL; - foreach(relationCell, relationList) + Oid relationId = InvalidOid; + foreach_oid(relationId, relationList) { - Oid relationId = lfirst_oid(relationCell); - if (PartitionMethod(relationId) != DISTRIBUTE_BY_NONE) { continue; @@ -560,14 +552,12 @@ UnlockJobResource(uint64 jobId, LOCKMODE lockmode) void LockShardListMetadata(List *shardIntervalList, LOCKMODE lockMode) { - ListCell *shardIntervalCell = NULL; - /* lock shards in order of shard id to prevent deadlock */ shardIntervalList = SortList(shardIntervalList, CompareShardIntervalsById); - foreach(shardIntervalCell, shardIntervalList) + ShardInterval *shardInterval = NULL; + foreach_ptr(shardInterval, shardIntervalList) { - ShardInterval *shardInterval = (ShardInterval *) lfirst(shardIntervalCell); int64 shardId = shardInterval->shardId; LockShardDistributionMetadata(shardId, lockMode); @@ -582,16 +572,13 @@ LockShardListMetadata(List *shardIntervalList, LOCKMODE lockMode) void LockShardsInPlacementListMetadata(List *shardPlacementList, LOCKMODE lockMode) { - ListCell *shardPlacementCell = NULL; - /* lock shards in order of shard id to prevent deadlock */ shardPlacementList = SortList(shardPlacementList, CompareShardPlacementsByShardId); - foreach(shardPlacementCell, shardPlacementList) + GroupShardPlacement *placement = NULL; + foreach_ptr(placement, shardPlacementList) { - GroupShardPlacement *placement = - (GroupShardPlacement *) lfirst(shardPlacementCell); int64 shardId = placement->shardId; LockShardDistributionMetadata(shardId, lockMode); @@ -647,14 +634,12 @@ SerializeNonCommutativeWrites(List *shardIntervalList, LOCKMODE lockMode) static void LockShardListResources(List *shardIntervalList, LOCKMODE lockMode) { - ListCell *shardIntervalCell = NULL; - /* lock shards in order of shard id to prevent deadlock */ shardIntervalList = SortList(shardIntervalList, CompareShardIntervalsById); - foreach(shardIntervalCell, shardIntervalList) + ShardInterval *shardInterval = NULL; + foreach_ptr(shardInterval, shardIntervalList) { - ShardInterval *shardInterval = (ShardInterval *) lfirst(shardIntervalCell); int64 shardId = shardInterval->shardId; LockShardResource(shardId, lockMode); @@ -669,14 +654,12 @@ LockShardListResources(List *shardIntervalList, LOCKMODE lockMode) void LockRelationShardResources(List *relationShardList, LOCKMODE lockMode) { - ListCell *relationShardCell = NULL; - /* lock shards in a consistent order to prevent deadlock */ relationShardList = SortList(relationShardList, CompareRelationShards); - foreach(relationShardCell, relationShardList) + RelationShard *relationShard = NULL; + foreach_ptr(relationShard, relationShardList) { - RelationShard *relationShard = (RelationShard *) lfirst(relationShardCell); uint64 shardId = relationShard->shardId; if (shardId != INVALID_SHARD_ID) diff --git a/src/backend/distributed/utils/shardinterval_utils.c b/src/backend/distributed/utils/shardinterval_utils.c index 24c85edec..63c0e7d17 100644 --- a/src/backend/distributed/utils/shardinterval_utils.c +++ b/src/backend/distributed/utils/shardinterval_utils.c @@ -15,6 +15,7 @@ #include "catalog/pg_am.h" #include "catalog/pg_collation.h" #include "catalog/pg_type.h" +#include "distributed/listutils.h" #include "distributed/metadata_cache.h" #include "distributed/multi_join_order.h" #include "distributed/distributed_planner.h" @@ -34,12 +35,10 @@ ShardInterval * LowestShardIntervalById(List *shardIntervalList) { ShardInterval *lowestShardInterval = NULL; - ListCell *shardIntervalCell = NULL; - foreach(shardIntervalCell, shardIntervalList) + ShardInterval *shardInterval = NULL; + foreach_ptr(shardInterval, shardIntervalList) { - ShardInterval *shardInterval = (ShardInterval *) lfirst(shardIntervalCell); - if (lowestShardInterval == NULL || lowestShardInterval->shardId > shardInterval->shardId) { @@ -460,12 +459,12 @@ SingleReplicatedTable(Oid relationId) return false; } - /* checking only for the first shard id should suffice */ - Oid shardId = (*(uint64 *) linitial(shardList)); - /* for hash distributed tables, it is sufficient to only check one shard */ if (PartitionMethod(relationId) == DISTRIBUTE_BY_HASH) { + /* checking only for the first shard id should suffice */ + uint64 shardId = *(uint64 *) linitial(shardList); + shardPlacementList = ShardPlacementList(shardId); if (list_length(shardPlacementList) != 1) { @@ -475,12 +474,10 @@ SingleReplicatedTable(Oid relationId) else { List *shardIntervalList = LoadShardList(relationId); - ListCell *shardIntervalCell = NULL; - - foreach(shardIntervalCell, shardIntervalList) + uint64 *shardIdPointer = NULL; + foreach_ptr(shardIdPointer, shardIntervalList) { - uint64 *shardIdPointer = (uint64 *) lfirst(shardIntervalCell); - shardId = (*shardIdPointer); + uint64 shardId = *shardIdPointer; shardPlacementList = ShardPlacementList(shardId); if (list_length(shardPlacementList) != 1) diff --git a/src/backend/distributed/utils/statistics_collection.c b/src/backend/distributed/utils/statistics_collection.c index 81d51c598..47a6e082e 100644 --- a/src/backend/distributed/utils/statistics_collection.c +++ b/src/backend/distributed/utils/statistics_collection.c @@ -24,6 +24,7 @@ PG_FUNCTION_INFO_V1(citus_server_id); #include #include "access/xact.h" +#include "distributed/listutils.h" #include "distributed/metadata_cache.h" #include "distributed/multi_join_order.h" #include "distributed/shardinterval_utils.h" @@ -174,12 +175,10 @@ static uint64 DistributedTablesSize(List *distTableOids) { uint64 totalSize = 0; - ListCell *distTableOidCell = NULL; - foreach(distTableOidCell, distTableOids) + Oid relationId = InvalidOid; + foreach_oid(relationId, distTableOids) { - Oid relationId = lfirst_oid(distTableOidCell); - /* * Relations can get dropped after getting the Oid list and before we * reach here. Acquire a lock to make sure the relation is available diff --git a/src/backend/distributed/worker/task_tracker.c b/src/backend/distributed/worker/task_tracker.c index 486c4e83e..0a72579f1 100644 --- a/src/backend/distributed/worker/task_tracker.c +++ b/src/backend/distributed/worker/task_tracker.c @@ -31,6 +31,7 @@ #include "commands/dbcommands.h" #include "distributed/citus_safe_lib.h" +#include "distributed/listutils.h" #include "distributed/multi_client_executor.h" #include "distributed/multi_server_executor.h" #include "distributed/task_tracker.h" @@ -370,16 +371,14 @@ TrackerCleanupJobSchemas(void) * this function to read from pg_database directly. */ List *databaseNameList = NIL; - ListCell *databaseNameCell = NULL; const uint64 jobId = RESERVED_JOB_ID; uint32 taskIndex = 1; LWLockAcquire(&WorkerTasksSharedState->taskHashLock, LW_EXCLUSIVE); - foreach(databaseNameCell, databaseNameList) + const char *databaseName = NULL; + foreach_ptr(databaseName, databaseNameList) { - char *databaseName = (char *) lfirst(databaseNameCell); - /* template0 database does not accept connections */ int skipDatabaseName = strncmp(databaseName, TEMPLATE0_NAME, NAMEDATALEN); if (skipDatabaseName == 0) @@ -790,10 +789,9 @@ CompareTasksByTime(const void *first, const void *second) static void ScheduleWorkerTasks(HTAB *WorkerTasksHash, List *schedulableTaskList) { - ListCell *schedulableTaskCell = NULL; - foreach(schedulableTaskCell, schedulableTaskList) + WorkerTask *schedulableTask = NULL; + foreach_ptr(schedulableTask, schedulableTaskList) { - WorkerTask *schedulableTask = (WorkerTask *) lfirst(schedulableTaskCell); void *hashKey = (void *) schedulableTask; WorkerTask *taskToSchedule = (WorkerTask *) hash_search(WorkerTasksHash, hashKey, diff --git a/src/backend/distributed/worker/worker_data_fetch_protocol.c b/src/backend/distributed/worker/worker_data_fetch_protocol.c index 9e3a364f1..d0486f70a 100644 --- a/src/backend/distributed/worker/worker_data_fetch_protocol.c +++ b/src/backend/distributed/worker/worker_data_fetch_protocol.c @@ -30,6 +30,7 @@ #include "distributed/citus_ruleutils.h" #include "distributed/commands/utility_hook.h" #include "distributed/connection_management.h" +#include "distributed/listutils.h" #include "distributed/master_protocol.h" #include "distributed/metadata_cache.h" #include "distributed/multi_client_executor.h" @@ -712,8 +713,6 @@ worker_append_table_to_shard(PG_FUNCTION_ARGS) static bool check_log_statement(List *statementList) { - ListCell *statementCell; - if (log_statement == LOGSTMT_NONE) { return false; @@ -725,10 +724,9 @@ check_log_statement(List *statementList) } /* else we have to inspect the statement(s) to see whether to log */ - foreach(statementCell, statementList) + Node *statement = NULL; + foreach_ptr(statement, statementList) { - Node *statement = (Node *) lfirst(statementCell); - if (GetCommandLogLevel(statement) <= log_statement) { return true; @@ -826,12 +824,8 @@ static void SetDefElemArg(AlterSeqStmt *statement, const char *name, Node *arg) { DefElem *defElem = NULL; - ListCell *optionCell = NULL; - - foreach(optionCell, statement->options) + foreach_ptr(defElem, statement->options) { - defElem = (DefElem *) lfirst(optionCell); - if (strcmp(defElem->defname, name) == 0) { pfree(defElem->arg); diff --git a/src/backend/distributed/worker/worker_drop_protocol.c b/src/backend/distributed/worker/worker_drop_protocol.c index bf5aa27c5..8ca056399 100644 --- a/src/backend/distributed/worker/worker_drop_protocol.c +++ b/src/backend/distributed/worker/worker_drop_protocol.c @@ -20,6 +20,7 @@ #include "catalog/pg_foreign_server.h" #include "distributed/citus_ruleutils.h" #include "distributed/distribution_column.h" +#include "distributed/listutils.h" #include "distributed/master_metadata_utility.h" #include "distributed/master_protocol.h" #include "distributed/metadata_cache.h" @@ -51,7 +52,6 @@ worker_drop_distributed_table(PG_FUNCTION_ARGS) Oid relationId = ResolveRelationId(relationName, true); ObjectAddress distributedTableObject = { InvalidOid, InvalidOid, 0 }; - ListCell *shardCell = NULL; char relationKind = '\0'; CheckCitusVersion(ERROR); @@ -108,17 +108,15 @@ worker_drop_distributed_table(PG_FUNCTION_ARGS) } /* iterate over shardList to delete the corresponding rows */ - foreach(shardCell, shardList) + uint64 *shardIdPointer = NULL; + foreach_ptr(shardIdPointer, shardList) { - ListCell *shardPlacementCell = NULL; - uint64 *shardIdPointer = (uint64 *) lfirst(shardCell); - uint64 shardId = (*shardIdPointer); + uint64 shardId = *shardIdPointer; List *shardPlacementList = ShardPlacementList(shardId); - foreach(shardPlacementCell, shardPlacementList) + ShardPlacement *placement = NULL; + foreach_ptr(placement, shardPlacementList) { - ShardPlacement *placement = (ShardPlacement *) lfirst(shardPlacementCell); - /* delete the row from pg_dist_placement */ DeleteShardPlacementRow(placement->placementId); } diff --git a/src/backend/distributed/worker/worker_file_access_protocol.c b/src/backend/distributed/worker/worker_file_access_protocol.c index a0cad944b..9bbd5b222 100644 --- a/src/backend/distributed/worker/worker_file_access_protocol.c +++ b/src/backend/distributed/worker/worker_file_access_protocol.c @@ -15,6 +15,7 @@ #include "funcapi.h" #include "commands/defrem.h" +#include "distributed/listutils.h" #include "distributed/master_protocol.h" #include "distributed/worker_protocol.h" #include "foreign/foreign.h" @@ -39,13 +40,11 @@ worker_foreign_file_path(PG_FUNCTION_ARGS) Oid relationId = ResolveRelationId(foreignTableName, false); ForeignTable *foreignTable = GetForeignTable(relationId); - ListCell *optionCell = NULL; - CheckCitusVersion(ERROR); - foreach(optionCell, foreignTable->options) + DefElem *option = NULL; + foreach_ptr(option, foreignTable->options) { - DefElem *option = (DefElem *) lfirst(optionCell); char *optionName = option->defname; int compareResult = strncmp(optionName, FOREIGN_FILENAME_OPTION, MAXPGPATH); diff --git a/src/include/distributed/intermediate_results.h b/src/include/distributed/intermediate_results.h index d7671ab6b..852a828be 100644 --- a/src/include/distributed/intermediate_results.h +++ b/src/include/distributed/intermediate_results.h @@ -49,23 +49,24 @@ typedef struct DistributedResultFragment /* intermediate_results.c */ -extern DestReceiver * CreateRemoteFileDestReceiver(char *resultId, EState *executorState, +extern DestReceiver * CreateRemoteFileDestReceiver(const char *resultId, + EState *executorState, List *initialNodeList, bool writeLocalFile); extern void SendQueryResultViaCopy(const char *resultId); extern void ReceiveQueryResultViaCopy(const char *resultId); extern void RemoveIntermediateResultsDirectory(void); -extern int64 IntermediateResultSize(char *resultId); +extern int64 IntermediateResultSize(const char *resultId); extern char * QueryResultFileName(const char *resultId); extern char * CreateIntermediateResultsDirectory(void); /* distributed_intermediate_results.c */ -extern List ** RedistributeTaskListResults(char *resultIdPrefix, +extern List ** RedistributeTaskListResults(const char *resultIdPrefix, List *selectTaskList, int partitionColumnIndex, DistTableCacheEntry *targetRelation, bool binaryFormat); -extern List * PartitionTasklistResults(char *resultIdPrefix, List *selectTaskList, +extern List * PartitionTasklistResults(const char *resultIdPrefix, List *selectTaskList, int partitionColumnIndex, DistTableCacheEntry *distributionScheme, bool binaryFormat); diff --git a/src/include/distributed/master_protocol.h b/src/include/distributed/master_protocol.h index ebb18c778..b9f5b2671 100644 --- a/src/include/distributed/master_protocol.h +++ b/src/include/distributed/master_protocol.h @@ -121,7 +121,7 @@ extern void CreateReferenceTableShard(Oid distributedTableId); extern List * WorkerCreateShardCommandList(Oid relationId, int shardIndex, uint64 shardId, List *ddlCommandList, List *foreignConstraintCommandList); -extern Oid ForeignConstraintGetReferencedTableId(char *queryString); +extern Oid ForeignConstraintGetReferencedTableId(const char *queryString); extern void CheckHashPartitionedTable(Oid distributedTableId); extern void CheckTableSchemaNameForDrop(Oid relationId, char **schemaName, char **tableName); @@ -155,7 +155,8 @@ extern Datum isolate_tenant_to_new_shard(PG_FUNCTION_ARGS); extern Datum master_copy_shard_placement(PG_FUNCTION_ARGS); /* function declarations for shard copy functinality */ -extern List * CopyShardCommandList(ShardInterval *shardInterval, char *sourceNodeName, +extern List * CopyShardCommandList(ShardInterval *shardInterval, const + char *sourceNodeName, int32 sourceNodePort, bool includeData); extern List * CopyShardForeignConstraintCommandList(ShardInterval *shardInterval); extern void CopyShardForeignConstraintCommandListGrouped(ShardInterval *shardInterval, @@ -164,8 +165,9 @@ extern void CopyShardForeignConstraintCommandListGrouped(ShardInterval *shardInt List ** referenceTableForeignConstraintList); extern ShardPlacement * SearchShardPlacementInList(List *shardPlacementList, - char *nodeName, uint32 nodePort); + const char *nodeName, uint32 nodePort); extern ShardPlacement * ForceSearchShardPlacementInList(List *shardPlacementList, - char *nodeName, uint32 nodePort); + const char *nodeName, + uint32 nodePort); #endif /* MASTER_PROTOCOL_H */ diff --git a/src/include/distributed/metadata_sync.h b/src/include/distributed/metadata_sync.h index a69c1a62c..efe11c6cb 100644 --- a/src/include/distributed/metadata_sync.h +++ b/src/include/distributed/metadata_sync.h @@ -28,15 +28,15 @@ typedef enum } MetadataSyncResult; /* Functions declarations for metadata syncing */ -extern void StartMetadatSyncToNode(char *nodeNameString, int32 nodePort); +extern void StartMetadataSyncToNode(const char *nodeNameString, int32 nodePort); extern bool ClusterHasKnownMetadataWorkers(void); extern bool ShouldSyncTableMetadata(Oid relationId); extern List * MetadataCreateCommands(void); extern List * GetDistributedTableDDLEvents(Oid relationId); extern List * MetadataDropCommands(void); extern char * DistributionCreateCommand(DistTableCacheEntry *cacheEntry); -extern char * DistributionDeleteCommand(char *schemaName, - char *tableName); +extern char * DistributionDeleteCommand(const char *schemaName, + const char *tableName); extern char * TableOwnerResetCommand(Oid distributedRelationId); extern char * NodeListInsertCommand(List *workerNodeList); extern List * ShardListInsertCommand(List *shardIntervalList); @@ -50,11 +50,12 @@ extern List * GrantOnSchemaDDLCommands(Oid schemaId); extern char * PlacementUpsertCommand(uint64 shardId, uint64 placementId, int shardState, uint64 shardLength, int32 groupId); extern void CreateTableMetadataOnWorkers(Oid relationId); -extern void MarkNodeHasMetadata(char *nodeName, int32 nodePort, bool hasMetadata); -extern void MarkNodeMetadataSynced(char *nodeName, int32 nodePort, bool synced); +extern void MarkNodeHasMetadata(const char *nodeName, int32 nodePort, bool hasMetadata); +extern void MarkNodeMetadataSynced(const char *nodeName, int32 nodePort, bool synced); extern MetadataSyncResult SyncMetadataToNodes(void); -extern bool SendOptionalCommandListToWorkerInTransaction(char *nodeName, int32 nodePort, - char *nodeUser, +extern bool SendOptionalCommandListToWorkerInTransaction(const char *nodeName, int32 + nodePort, + const char *nodeUser, List *commandList); #define DELETE_ALL_NODES "TRUNCATE pg_dist_node CASCADE" diff --git a/src/include/distributed/worker_manager.h b/src/include/distributed/worker_manager.h index 72acfafb9..f5bb00ba9 100644 --- a/src/include/distributed/worker_manager.h +++ b/src/include/distributed/worker_manager.h @@ -78,8 +78,8 @@ extern bool NodeCanHaveDistTablePlacements(WorkerNode *node); extern uint32 ActiveReadableWorkerNodeCount(void); extern List * ActiveReadableWorkerNodeList(void); extern List * ActiveReadableNodeList(void); -extern WorkerNode * FindWorkerNode(char *nodeName, int32 nodePort); -extern WorkerNode * ForceFindWorkerNode(char *nodeName, int32 nodePort); +extern WorkerNode * FindWorkerNode(const char *nodeName, int32 nodePort); +extern WorkerNode * ForceFindWorkerNode(const char *nodeName, int32 nodePort); extern WorkerNode * FindWorkerNodeAnyCluster(const char *nodeName, int32 nodePort); extern List * ReadDistNode(bool includeNodesFromOtherClusters); extern void EnsureCoordinator(void); diff --git a/src/include/distributed/worker_transaction.h b/src/include/distributed/worker_transaction.h index dac399361..f1c243cc9 100644 --- a/src/include/distributed/worker_transaction.h +++ b/src/include/distributed/worker_transaction.h @@ -31,30 +31,30 @@ typedef enum TargetWorkerSet /* Functions declarations for worker transactions */ extern List * GetWorkerTransactions(void); extern List * TargetWorkerSetNodeList(TargetWorkerSet targetWorkerSet, LOCKMODE lockMode); -extern void SendCommandToWorker(char *nodeName, int32 nodePort, const char *command); +extern void SendCommandToWorker(const char *nodeName, int32 nodePort, const + char *command); extern void SendCommandToWorkersAsUser(TargetWorkerSet targetWorkerSet, const char *nodeUser, const char *command); -extern void SendCommandToWorkerAsUser(char *nodeName, int32 nodePort, +extern void SendCommandToWorkerAsUser(const char *nodeName, int32 nodePort, const char *nodeUser, const char *command); extern void SendCommandToWorkersWithMetadata(const char *command); extern void SendBareCommandListToMetadataWorkers(List *commandList); extern int SendBareOptionalCommandListToAllWorkersAsUser(List *commandList, const char *user); extern void EnsureNoModificationsHaveBeenDone(void); -extern void SendCommandListToAllWorkers(List *commandList, char *superuser); -extern void SendOptionalCommandListToAllWorkers(List *commandList, char *superuser); -extern void SendCommandToAllWorkers(char *command, char *superuser); +extern void SendCommandListToAllWorkers(List *commandList, const char *superuser); +extern void SendOptionalCommandListToAllWorkers(List *commandList, const char *superuser); +extern void SendCommandToAllWorkers(const char *command, const char *superuser); extern void SendCommandListToWorkerInSingleTransaction(const char *nodeName, int32 nodePort, const char *nodeUser, List *commandList); -extern void SendCommandToWorkersOptionalInParallel(TargetWorkerSet targetWorkerSet, const - char *command, +extern void SendCommandToWorkersOptionalInParallel(TargetWorkerSet targetWorkerSet, + const char *command, const char *user); -void SendCommandToWorkersInParallel(TargetWorkerSet targetWorkerSet, const - char *command, - const char *user); -extern void RemoveWorkerTransaction(char *nodeName, int32 nodePort); +void SendCommandToWorkersInParallel(TargetWorkerSet targetWorkerSet, + const char *command, const char *user); +extern void RemoveWorkerTransaction(const char *nodeName, int32 nodePort); /* helper functions for worker transactions */ extern bool IsWorkerTransactionActive(void);