From f8fb20cc958ca01dc3a3b0afb6a732a4f748c857 Mon Sep 17 00:00:00 2001 From: aykutbozkurt Date: Fri, 10 Mar 2023 12:11:16 +0300 Subject: [PATCH] =?UTF-8?q?PR=20#6728=20=C2=A0/=20commit=20-=207?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Remove unused old metadata sync methods. --- .../distributed/commands/dependencies.c | 62 ---- .../distributed/metadata/metadata_sync.c | 295 ----------------- .../distributed/metadata/node_metadata.c | 302 ------------------ src/include/distributed/metadata_sync.h | 3 - src/include/distributed/metadata_utility.h | 1 - src/include/distributed/worker_manager.h | 2 - 6 files changed, 665 deletions(-) diff --git a/src/backend/distributed/commands/dependencies.c b/src/backend/distributed/commands/dependencies.c index af84d2eb5..cda2237bc 100644 --- a/src/backend/distributed/commands/dependencies.c +++ b/src/backend/distributed/commands/dependencies.c @@ -529,68 +529,6 @@ GetAllDependencyCreateDDLCommands(const List *dependencies) } -/* - * ReplicateAllObjectsToNodeCommandList returns commands to replicate all - * previously marked objects to a worker node. The function also sets - * clusterHasDistributedFunction if there are any distributed functions. - */ -List * -ReplicateAllObjectsToNodeCommandList(const char *nodeName, int nodePort) -{ - /* since we are executing ddl commands disable propagation first, primarily for mx */ - List *ddlCommands = list_make1(DISABLE_DDL_PROPAGATION); - - /* - * collect all dependencies in creation order and get their ddl commands - */ - List *dependencies = GetDistributedObjectAddressList(); - - /* - * Depending on changes in the environment, such as the enable_metadata_sync guc - * there might be objects in the distributed object address list that should currently - * not be propagated by citus as they are 'not supported'. - */ - dependencies = FilterObjectAddressListByPredicate(dependencies, - &SupportedDependencyByCitus); - - /* - * When dependency lists are getting longer we see a delay in the creation time on the - * workers. We would like to inform the user. Currently we warn for lists greater than - * 100 items, where 100 is an arbitrarily chosen number. If we find it too high or too - * low we can adjust this based on experience. - */ - if (list_length(dependencies) > 100) - { - ereport(NOTICE, (errmsg("Replicating postgres objects to node %s:%d", nodeName, - nodePort), - errdetail("There are %d objects to replicate, depending on your " - "environment this might take a while", - list_length(dependencies)))); - } - - dependencies = OrderObjectAddressListInDependencyOrder(dependencies); - ObjectAddress *dependency = NULL; - foreach_ptr(dependency, dependencies) - { - if (IsAnyObjectAddressOwnedByExtension(list_make1(dependency), NULL)) - { - /* - * we expect extension-owned objects to be created as a result - * of the extension being created. - */ - continue; - } - - ddlCommands = list_concat(ddlCommands, - GetDependencyCreateDDLCommands(dependency)); - } - - ddlCommands = lappend(ddlCommands, ENABLE_DDL_PROPAGATION); - - return ddlCommands; -} - - /* * ShouldPropagate determines if we should be propagating anything */ diff --git a/src/backend/distributed/metadata/metadata_sync.c b/src/backend/distributed/metadata/metadata_sync.c index aeac16b89..ed89cf2c3 100644 --- a/src/backend/distributed/metadata/metadata_sync.c +++ b/src/backend/distributed/metadata/metadata_sync.c @@ -237,81 +237,6 @@ start_metadata_sync_to_all_nodes(PG_FUNCTION_ARGS) } -/* - * SyncNodeMetadataToNode is the internal API for - * start_metadata_sync_to_node(). - */ -void -SyncNodeMetadataToNode(const char *nodeNameString, int32 nodePort) -{ - char *escapedNodeName = quote_literal_cstr(nodeNameString); - - CheckCitusVersion(ERROR); - EnsureCoordinator(); - EnsureModificationsCanRun(); - - EnsureSequentialModeMetadataOperations(); - - LockRelationOid(DistNodeRelationId(), ExclusiveLock); - - WorkerNode *workerNode = FindWorkerNode(nodeNameString, nodePort); - if (workerNode == NULL) - { - ereport(ERROR, (errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE), - errmsg("you cannot sync metadata to a non-existent node"), - errhint("First, add the node with SELECT citus_add_node" - "(%s,%d)", escapedNodeName, nodePort))); - } - - if (!workerNode->isActive) - { - ereport(ERROR, (errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE), - errmsg("you cannot sync metadata to an inactive node"), - errhint("First, activate the node with " - "SELECT citus_activate_node(%s,%d)", - escapedNodeName, nodePort))); - } - - if (NodeIsCoordinator(workerNode)) - { - ereport(NOTICE, (errmsg("%s:%d is the coordinator and already contains " - "metadata, skipping syncing the metadata", - nodeNameString, nodePort))); - return; - } - - UseCoordinatedTransaction(); - - /* - * One would normally expect to set hasmetadata first, and then metadata sync. - * However, at this point we do the order reverse. - * We first set metadatasynced, and then hasmetadata; since setting columns for - * nodes with metadatasynced==false could cause errors. - * (See ErrorIfAnyMetadataNodeOutOfSync) - * We can safely do that because we are in a coordinated transaction and the changes - * are only visible to our own transaction. - * If anything goes wrong, we are going to rollback all the changes. - */ - workerNode = SetWorkerColumn(workerNode, Anum_pg_dist_node_metadatasynced, - BoolGetDatum(true)); - workerNode = SetWorkerColumn(workerNode, Anum_pg_dist_node_hasmetadata, BoolGetDatum( - true)); - - if (!NodeIsPrimary(workerNode)) - { - /* - * If this is a secondary node we can't actually sync metadata to it; we assume - * the primary node is receiving metadata. - */ - return; - } - - /* fail if metadata synchronization doesn't succeed */ - bool raiseInterrupts = true; - SyncNodeMetadataSnapshotToNode(workerNode, raiseInterrupts); -} - - /* * SyncCitusTableMetadata syncs citus table metadata to worker nodes with metadata. * Our definition of metadata includes the shell table and its inter relations with @@ -803,114 +728,6 @@ NodeMetadataCreateCommands(void) } -/* - * DistributedObjectMetadataSyncCommandList returns the necessary commands to create - * pg_dist_object entries on the new node. - */ -List * -DistributedObjectMetadataSyncCommandList(void) -{ - HeapTuple pgDistObjectTup = NULL; - Relation pgDistObjectRel = table_open(DistObjectRelationId(), AccessShareLock); - Relation pgDistObjectIndexRel = index_open(DistObjectPrimaryKeyIndexId(), - AccessShareLock); - TupleDesc pgDistObjectDesc = RelationGetDescr(pgDistObjectRel); - - List *objectAddressList = NIL; - List *distArgumentIndexList = NIL; - List *colocationIdList = NIL; - List *forceDelegationList = NIL; - - /* It is not strictly necessary to read the tuples in order. - * However, it is useful to get consistent behavior, both for regression - * tests and also in production systems. - */ - SysScanDesc pgDistObjectScan = systable_beginscan_ordered(pgDistObjectRel, - pgDistObjectIndexRel, NULL, - 0, NULL); - while (HeapTupleIsValid(pgDistObjectTup = systable_getnext_ordered(pgDistObjectScan, - ForwardScanDirection))) - { - Form_pg_dist_object pg_dist_object = (Form_pg_dist_object) GETSTRUCT( - pgDistObjectTup); - - ObjectAddress *address = palloc(sizeof(ObjectAddress)); - - ObjectAddressSubSet(*address, pg_dist_object->classid, pg_dist_object->objid, - pg_dist_object->objsubid); - - bool distributionArgumentIndexIsNull = false; - Datum distributionArgumentIndexDatum = - heap_getattr(pgDistObjectTup, - Anum_pg_dist_object_distribution_argument_index, - pgDistObjectDesc, - &distributionArgumentIndexIsNull); - int32 distributionArgumentIndex = DatumGetInt32(distributionArgumentIndexDatum); - - bool colocationIdIsNull = false; - Datum colocationIdDatum = - heap_getattr(pgDistObjectTup, - Anum_pg_dist_object_colocationid, - pgDistObjectDesc, - &colocationIdIsNull); - int32 colocationId = DatumGetInt32(colocationIdDatum); - - bool forceDelegationIsNull = false; - Datum forceDelegationDatum = - heap_getattr(pgDistObjectTup, - Anum_pg_dist_object_force_delegation, - pgDistObjectDesc, - &forceDelegationIsNull); - bool forceDelegation = DatumGetBool(forceDelegationDatum); - - objectAddressList = lappend(objectAddressList, address); - - if (distributionArgumentIndexIsNull) - { - distArgumentIndexList = lappend_int(distArgumentIndexList, - INVALID_DISTRIBUTION_ARGUMENT_INDEX); - } - else - { - distArgumentIndexList = lappend_int(distArgumentIndexList, - distributionArgumentIndex); - } - - if (colocationIdIsNull) - { - colocationIdList = lappend_int(colocationIdList, - INVALID_COLOCATION_ID); - } - else - { - colocationIdList = lappend_int(colocationIdList, colocationId); - } - - if (forceDelegationIsNull) - { - forceDelegationList = lappend_int(forceDelegationList, NO_FORCE_PUSHDOWN); - } - else - { - forceDelegationList = lappend_int(forceDelegationList, forceDelegation); - } - } - - systable_endscan_ordered(pgDistObjectScan); - index_close(pgDistObjectIndexRel, AccessShareLock); - relation_close(pgDistObjectRel, NoLock); - - char *workerMetadataUpdateCommand = - MarkObjectsDistributedCreateCommand(objectAddressList, - distArgumentIndexList, - colocationIdList, - forceDelegationList); - List *commandList = list_make1(workerMetadataUpdateCommand); - - return commandList; -} - - /* * CitusTableMetadataCreateCommandList returns the set of commands necessary to * create the given distributed table metadata on a worker. @@ -4081,118 +3898,6 @@ ColocationGroupDeleteCommand(uint32 colocationId) } -/* - * ColocationGroupCreateCommandList returns the full list of commands for syncing - * pg_dist_colocation. - */ -List * -ColocationGroupCreateCommandList(void) -{ - bool hasColocations = false; - - StringInfo colocationGroupCreateCommand = makeStringInfo(); - appendStringInfo(colocationGroupCreateCommand, - "WITH colocation_group_data (colocationid, shardcount, " - "replicationfactor, distributioncolumntype, " - "distributioncolumncollationname, " - "distributioncolumncollationschema) AS (VALUES "); - - Relation pgDistColocation = table_open(DistColocationRelationId(), AccessShareLock); - Relation colocationIdIndexRel = index_open(DistColocationIndexId(), AccessShareLock); - - /* - * It is not strictly necessary to read the tuples in order. - * However, it is useful to get consistent behavior, both for regression - * tests and also in production systems. - */ - SysScanDesc scanDescriptor = - systable_beginscan_ordered(pgDistColocation, colocationIdIndexRel, - NULL, 0, NULL); - - HeapTuple colocationTuple = systable_getnext_ordered(scanDescriptor, - ForwardScanDirection); - - while (HeapTupleIsValid(colocationTuple)) - { - if (hasColocations) - { - appendStringInfo(colocationGroupCreateCommand, ", "); - } - - hasColocations = true; - - Form_pg_dist_colocation colocationForm = - (Form_pg_dist_colocation) GETSTRUCT(colocationTuple); - - appendStringInfo(colocationGroupCreateCommand, - "(%d, %d, %d, %s, ", - colocationForm->colocationid, - colocationForm->shardcount, - colocationForm->replicationfactor, - RemoteTypeIdExpression(colocationForm->distributioncolumntype)); - - /* - * For collations, include the names in the VALUES section and then - * join with pg_collation. - */ - Oid distributionColumCollation = colocationForm->distributioncolumncollation; - if (distributionColumCollation != InvalidOid) - { - Datum collationIdDatum = ObjectIdGetDatum(distributionColumCollation); - HeapTuple collationTuple = SearchSysCache1(COLLOID, collationIdDatum); - - if (HeapTupleIsValid(collationTuple)) - { - Form_pg_collation collationform = - (Form_pg_collation) GETSTRUCT(collationTuple); - char *collationName = NameStr(collationform->collname); - char *collationSchemaName = get_namespace_name( - collationform->collnamespace); - - appendStringInfo(colocationGroupCreateCommand, - "%s, %s)", - quote_literal_cstr(collationName), - quote_literal_cstr(collationSchemaName)); - - ReleaseSysCache(collationTuple); - } - else - { - appendStringInfo(colocationGroupCreateCommand, - "NULL, NULL)"); - } - } - else - { - appendStringInfo(colocationGroupCreateCommand, - "NULL, NULL)"); - } - - colocationTuple = systable_getnext_ordered(scanDescriptor, ForwardScanDirection); - } - - systable_endscan_ordered(scanDescriptor); - index_close(colocationIdIndexRel, AccessShareLock); - table_close(pgDistColocation, AccessShareLock); - - if (!hasColocations) - { - return NIL; - } - - appendStringInfo(colocationGroupCreateCommand, - ") SELECT pg_catalog.citus_internal_add_colocation_metadata(" - "colocationid, shardcount, replicationfactor, " - "distributioncolumntype, coalesce(c.oid, 0)) " - "FROM colocation_group_data d LEFT JOIN pg_collation c " - "ON (d.distributioncolumncollationname = c.collname " - "AND d.distributioncolumncollationschema::regnamespace" - " = c.collnamespace)"); - - return list_make1(colocationGroupCreateCommand->data); -} - - /* * SetMetadataSyncNodesFromNodeList sets list of nodes that needs to be metadata * synced among given node list into metadataSyncContext. diff --git a/src/backend/distributed/metadata/node_metadata.c b/src/backend/distributed/metadata/node_metadata.c index 586006e16..581c04c8a 100644 --- a/src/backend/distributed/metadata/node_metadata.c +++ b/src/backend/distributed/metadata/node_metadata.c @@ -100,13 +100,8 @@ static void InsertPlaceholderCoordinatorRecord(void); static void InsertNodeRow(int nodeid, char *nodename, int32 nodeport, NodeMetadata *nodeMetadata); static void DeleteNodeRow(char *nodename, int32 nodeport); -static void SyncDistributedObjectsToNodeList(List *workerNodeList); -static void UpdateLocalGroupIdOnNode(WorkerNode *workerNode); -static void SyncPgDistTableMetadataToNodeList(List *nodeList); -static List * InterTableRelationshipCommandList(); static void BlockDistributedQueriesOnMetadataNodes(void); static WorkerNode * TupleToWorkerNode(TupleDesc tupleDescriptor, HeapTuple heapTuple); -static List * PropagateNodeWideObjectsCommandList(); static WorkerNode * ModifiableWorkerNode(const char *nodeName, int32 nodePort); static bool NodeIsLocal(WorkerNode *worker); static void SetLockTimeoutLocally(int32 lock_cooldown); @@ -650,303 +645,6 @@ master_set_node_property(PG_FUNCTION_ARGS) } -/* - * InterTableRelationshipCommandList returns the command list to - * set up the multiple integrations including - * - * (i) Foreign keys - * (ii) Partionining hierarchy - * - * for each citus table. - */ -static List * -InterTableRelationshipCommandList() -{ - List *distributedTableList = CitusTableList(); - List *propagatedTableList = NIL; - List *multipleTableIntegrationCommandList = NIL; - - CitusTableCacheEntry *cacheEntry = NULL; - foreach_ptr(cacheEntry, distributedTableList) - { - /* - * Skip foreign key and partition creation when we shouldn't need to sync - * tablem metadata or the Citus table is owned by an extension. - */ - if (ShouldSyncTableMetadata(cacheEntry->relationId) && - !IsTableOwnedByExtension(cacheEntry->relationId)) - { - propagatedTableList = lappend(propagatedTableList, cacheEntry); - } - } - - foreach_ptr(cacheEntry, propagatedTableList) - { - Oid relationId = cacheEntry->relationId; - - List *commandListForRelation = - InterTableRelationshipOfRelationCommandList(relationId); - - multipleTableIntegrationCommandList = list_concat( - multipleTableIntegrationCommandList, - commandListForRelation); - } - - multipleTableIntegrationCommandList = lcons(DISABLE_DDL_PROPAGATION, - multipleTableIntegrationCommandList); - multipleTableIntegrationCommandList = lappend(multipleTableIntegrationCommandList, - ENABLE_DDL_PROPAGATION); - - return multipleTableIntegrationCommandList; -} - - -/* - * PgDistTableMetadataSyncCommandList returns the command list to sync the pg_dist_* - * (except pg_dist_node) metadata. We call them as table metadata. - */ -List * -PgDistTableMetadataSyncCommandList(void) -{ - List *distributedTableList = CitusTableList(); - List *propagatedTableList = NIL; - List *metadataSnapshotCommandList = NIL; - - /* create the list of tables whose metadata will be created */ - CitusTableCacheEntry *cacheEntry = NULL; - foreach_ptr(cacheEntry, distributedTableList) - { - if (ShouldSyncTableMetadata(cacheEntry->relationId)) - { - propagatedTableList = lappend(propagatedTableList, cacheEntry); - } - } - - /* remove all dist table and object related metadata first */ - metadataSnapshotCommandList = lappend(metadataSnapshotCommandList, - DELETE_ALL_PARTITIONS); - metadataSnapshotCommandList = lappend(metadataSnapshotCommandList, DELETE_ALL_SHARDS); - metadataSnapshotCommandList = lappend(metadataSnapshotCommandList, - DELETE_ALL_PLACEMENTS); - metadataSnapshotCommandList = lappend(metadataSnapshotCommandList, - DELETE_ALL_DISTRIBUTED_OBJECTS); - metadataSnapshotCommandList = lappend(metadataSnapshotCommandList, - DELETE_ALL_COLOCATION); - - /* create pg_dist_partition, pg_dist_shard and pg_dist_placement entries */ - foreach_ptr(cacheEntry, propagatedTableList) - { - List *tableMetadataCreateCommandList = - CitusTableMetadataCreateCommandList(cacheEntry->relationId); - - metadataSnapshotCommandList = list_concat(metadataSnapshotCommandList, - tableMetadataCreateCommandList); - } - - /* commands to insert pg_dist_colocation entries */ - List *colocationGroupSyncCommandList = ColocationGroupCreateCommandList(); - metadataSnapshotCommandList = list_concat(metadataSnapshotCommandList, - colocationGroupSyncCommandList); - - List *distributedObjectSyncCommandList = DistributedObjectMetadataSyncCommandList(); - metadataSnapshotCommandList = list_concat(metadataSnapshotCommandList, - distributedObjectSyncCommandList); - - metadataSnapshotCommandList = lcons(DISABLE_DDL_PROPAGATION, - metadataSnapshotCommandList); - metadataSnapshotCommandList = lappend(metadataSnapshotCommandList, - ENABLE_DDL_PROPAGATION); - - return metadataSnapshotCommandList; -} - - -/* - * PropagateNodeWideObjectsCommandList is called during node activation to - * propagate any object that should be propagated for every node. These are - * generally not linked to any distributed object but change system wide behaviour. - */ -static List * -PropagateNodeWideObjectsCommandList() -{ - /* collect all commands */ - List *ddlCommands = NIL; - - if (EnableAlterRoleSetPropagation) - { - /* - * Get commands for database and postgres wide settings. Since these settings are not - * linked to any role that can be distributed we need to distribute them seperately - */ - List *alterRoleSetCommands = GenerateAlterRoleSetCommandForRole(InvalidOid); - ddlCommands = list_concat(ddlCommands, alterRoleSetCommands); - } - - if (list_length(ddlCommands) > 0) - { - /* if there are command wrap them in enable_ddl_propagation off */ - ddlCommands = lcons(DISABLE_DDL_PROPAGATION, ddlCommands); - ddlCommands = lappend(ddlCommands, ENABLE_DDL_PROPAGATION); - } - - return ddlCommands; -} - - -/* - * SyncDistributedObjectsCommandList returns commands to sync object dependencies - * to the given worker node. To be idempotent, it first drops the ones required to be - * dropped. - * - * Object dependencies include: - * - * - All dependencies (e.g., types, schemas, sequences) - * - All shell distributed tables - * - Inter relation between those shell tables - * - Node wide objects - * - * We also update the local group id here, as handling sequence dependencies - * requires it. - */ -List * -SyncDistributedObjectsCommandList(WorkerNode *workerNode) -{ - List *commandList = NIL; - - /* - * Propagate node wide objects. It includes only roles for now. - */ - commandList = list_concat(commandList, PropagateNodeWideObjectsCommandList()); - - /* - * Detach partitions, break dependencies between sequences and table then - * remove shell tables first. - */ - commandList = list_concat(commandList, DetachPartitionCommandList()); - commandList = lappend(commandList, BREAK_CITUS_TABLE_SEQUENCE_DEPENDENCY_COMMAND); - commandList = lappend(commandList, REMOVE_ALL_SHELL_TABLES_COMMAND); - - /* - * Replicate all objects of the pg_dist_object to the remote node. - */ - commandList = list_concat(commandList, ReplicateAllObjectsToNodeCommandList( - workerNode->workerName, workerNode->workerPort)); - - /* - * After creating each table, handle the inter table relationship between - * those tables. - */ - commandList = list_concat(commandList, InterTableRelationshipCommandList()); - - return commandList; -} - - -/* - * SyncDistributedObjectsToNodeList sync the distributed objects to the node. It includes - * - All dependencies (e.g., types, schemas, sequences) - * - All shell distributed table - * - Inter relation between those shell tables - * - * Note that we do not create the distributed dependencies on the coordinator - * since all the dependencies should be present in the coordinator already. - */ -static void -SyncDistributedObjectsToNodeList(List *workerNodeList) -{ - List *workerNodesToSync = NIL; - WorkerNode *workerNode = NULL; - foreach_ptr(workerNode, workerNodeList) - { - if (NodeIsCoordinator(workerNode)) - { - /* coordinator has all the objects */ - continue; - } - - if (!NodeIsPrimary(workerNode)) - { - /* secondary nodes gets the objects from their primaries via replication */ - continue; - } - - workerNodesToSync = lappend(workerNodesToSync, workerNode); - } - - if (workerNodesToSync == NIL) - { - return; - } - - EnsureSequentialModeMetadataOperations(); - - Assert(ShouldPropagate()); - - List *commandList = SyncDistributedObjectsCommandList(workerNode); - - /* send commands to new workers, the current user should be a superuser */ - Assert(superuser()); - SendMetadataCommandListToWorkerListInCoordinatedTransaction( - workerNodesToSync, - CurrentUserName(), - commandList); -} - - -/* - * UpdateLocalGroupIdOnNode updates local group id on node. - */ -static void -UpdateLocalGroupIdOnNode(WorkerNode *workerNode) -{ - if (NodeIsPrimary(workerNode) && !NodeIsCoordinator(workerNode)) - { - List *commandList = list_make1(LocalGroupIdUpdateCommand(workerNode->groupId)); - - /* send commands to new workers, the current user should be a superuser */ - Assert(superuser()); - SendMetadataCommandListToWorkerListInCoordinatedTransaction( - list_make1(workerNode), - CurrentUserName(), - commandList); - } -} - - -/* - * SyncPgDistTableMetadataToNodeList syncs the pg_dist_partition, pg_dist_shard - * pg_dist_placement and pg_dist_object metadata entries. - * - */ -static void -SyncPgDistTableMetadataToNodeList(List *nodeList) -{ - /* send commands to new workers, the current user should be a superuser */ - Assert(superuser()); - - List *nodesWithMetadata = NIL; - WorkerNode *workerNode = NULL; - foreach_ptr(workerNode, nodeList) - { - if (NodeIsPrimary(workerNode) && !NodeIsCoordinator(workerNode)) - { - nodesWithMetadata = lappend(nodesWithMetadata, workerNode); - } - } - - if (nodesWithMetadata == NIL) - { - return; - } - - List *syncPgDistMetadataCommandList = PgDistTableMetadataSyncCommandList(); - SendMetadataCommandListToWorkerListInCoordinatedTransaction( - nodesWithMetadata, - CurrentUserName(), - syncPgDistMetadataCommandList); -} - - /* * ModifiableWorkerNode gets the requested WorkerNode and also gets locks * required for modifying it. This fails if the node does not exist. diff --git a/src/include/distributed/metadata_sync.h b/src/include/distributed/metadata_sync.h index 0b32d34d8..6f8254071 100644 --- a/src/include/distributed/metadata_sync.h +++ b/src/include/distributed/metadata_sync.h @@ -73,7 +73,6 @@ extern void citus_internal_add_placement_metadata_internal(int64 shardId, int64 shardLength, int32 groupId, int64 placementId); -extern void SyncNodeMetadataToNode(const char *nodeNameString, int32 nodePort); extern void SyncCitusTableMetadata(Oid relationId); extern void EnsureSequentialModeMetadataOperations(void); extern bool ClusterHasKnownMetadataWorkers(void); @@ -85,8 +84,6 @@ extern Oid FetchRelationIdFromPgPartitionHeapTuple(HeapTuple heapTuple, TupleDesc tupleDesc); extern bool ShouldSyncSequenceMetadata(Oid relationId); extern List * NodeMetadataCreateCommands(void); -extern List * DistributedObjectMetadataSyncCommandList(void); -extern List * ColocationGroupCreateCommandList(void); extern List * CitusTableMetadataCreateCommandList(Oid relationId); extern List * NodeMetadataDropCommands(void); extern char * MarkObjectsDistributedCreateCommand(List *addresses, diff --git a/src/include/distributed/metadata_utility.h b/src/include/distributed/metadata_utility.h index acb4ae5da..64d2e3557 100644 --- a/src/include/distributed/metadata_utility.h +++ b/src/include/distributed/metadata_utility.h @@ -338,7 +338,6 @@ extern List * GetAllDependencyCreateDDLCommands(const List *dependencies); extern bool ShouldPropagate(void); extern bool ShouldPropagateCreateInCoordinatedTransction(void); extern bool ShouldPropagateAnyObject(List *addresses); -extern List * ReplicateAllObjectsToNodeCommandList(const char *nodeName, int nodePort); /* Remaining metadata utility functions */ extern Oid TableOwnerOid(Oid relationId); diff --git a/src/include/distributed/worker_manager.h b/src/include/distributed/worker_manager.h index 39fa3e612..bfc18c919 100644 --- a/src/include/distributed/worker_manager.h +++ b/src/include/distributed/worker_manager.h @@ -102,8 +102,6 @@ extern WorkerNode * SetWorkerColumnLocalOnly(WorkerNode *workerNode, int columnI Datum value); extern uint32 CountPrimariesWithMetadata(void); extern WorkerNode * GetFirstPrimaryWorkerNode(void); -extern List * SyncDistributedObjectsCommandList(WorkerNode *workerNode); -extern List * PgDistTableMetadataSyncCommandList(void); /* Function declarations for worker node utilities */ extern int CompareWorkerNodes(const void *leftElement, const void *rightElement);