/* * master_create_worker_shards creates empty shards for the given table based * on the specified number of initial shards. The function first gets a list of * candidate nodes and issues DDL commands on the nodes to create empty shard * placements on those nodes. The function then updates metadata on the master * node to make this shard (and its placements) visible. Note that the function * assumes the table is hash partitioned and calculates the min/max hash token * ranges for each shard, giving them an equal split of the hash space. */ Datum master_create_worker_shards(PG_FUNCTION_ARGS) { text *tableNameText = PG_GETARG_TEXT_P(0); int32 shardCount = PG_GETARG_INT32(1); int32 replicationFactor = PG_GETARG_INT32(2); Oid distributedTableId = ResolveRelationId(tableNameText); char relationKind = get_rel_relkind(distributedTableId); char *tableName = text_to_cstring(tableNameText); char shardStorageType = '\0'; int32 shardIndex = 0; List *workerNodeList = NIL; List *ddlCommandList = NIL; int32 workerNodeCount = 0; uint32 placementAttemptCount = 0; uint32 hashTokenIncrement = 0; List *existingShardList = NIL; /* make sure table is hash partitioned */ CheckHashPartitionedTable(distributedTableId); /* validate that shards haven't already been created for this table */ existingShardList = LoadShardIntervalList(distributedTableId); if (existingShardList != NIL) { ereport(ERROR, (errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE), errmsg("table \"%s\" has already had shards created for it", tableName))); } /* make sure that at least one shard is specified */ if (shardCount <= 0) { ereport(ERROR, (errcode(ERRCODE_INVALID_PARAMETER_VALUE), errmsg("shardCount must be positive"))); } /* make sure that at least one replica is specified */ if (replicationFactor <= 0) { ereport(ERROR, (errcode(ERRCODE_INVALID_PARAMETER_VALUE), errmsg("replicationFactor must be positive"))); } /* calculate the split of the hash space */ hashTokenIncrement = UINT_MAX / shardCount; /* load and sort the worker node list for deterministic placement */ workerNodeList = ParseWorkerNodeFile(WORKER_LIST_FILENAME); workerNodeList = SortList(workerNodeList, CompareWorkerNodes); /* make sure we don't process cancel signals until all shards are created */ HOLD_INTERRUPTS(); /* retrieve the DDL commands for the table */ ddlCommandList = TableDDLCommandList(distributedTableId); workerNodeCount = list_length(workerNodeList); if (replicationFactor > workerNodeCount) { ereport(ERROR, (errcode(ERRCODE_INVALID_PARAMETER_VALUE), errmsg("replicationFactor (%d) exceeds number of worker nodes " "(%d)", replicationFactor, workerNodeCount), errhint("Add more worker nodes or try again with a lower " "replication factor."))); } /* if we have enough nodes, add an extra placement attempt for backup */ placementAttemptCount = (uint32) replicationFactor; if (workerNodeCount > replicationFactor) { placementAttemptCount++; } /* set shard storage type according to relation type */ if (relationKind == RELKIND_FOREIGN_TABLE) { shardStorageType = SHARD_STORAGE_FOREIGN; } else { shardStorageType = SHARD_STORAGE_TABLE; } for (shardIndex = 0; shardIndex < shardCount; shardIndex++) { uint64 shardId = NextSequenceId(SHARD_ID_SEQUENCE_NAME); int32 placementCount = 0; uint32 placementIndex = 0; uint32 roundRobinNodeIndex = shardIndex % workerNodeCount; List *extendedDDLCommands = ExtendedDDLCommandList(distributedTableId, shardId, ddlCommandList); /* initialize the hash token space for this shard */ text *minHashTokenText = NULL; text *maxHashTokenText = NULL; int32 shardMinHashToken = INT_MIN + (shardIndex * hashTokenIncrement); int32 shardMaxHashToken = shardMinHashToken + hashTokenIncrement - 1; /* if we are at the last shard, make sure the max token value is INT_MAX */ if (shardIndex == (shardCount - 1)) { shardMaxHashToken = INT_MAX; } for (placementIndex = 0; placementIndex < placementAttemptCount; placementIndex++) { int32 candidateNodeIndex = (roundRobinNodeIndex + placementIndex) % workerNodeCount; WorkerNode *candidateNode = (WorkerNode *) list_nth(workerNodeList, candidateNodeIndex); char *nodeName = candidateNode->nodeName; uint32 nodePort = candidateNode->nodePort; bool created = ExecuteRemoteCommandList(nodeName, nodePort, extendedDDLCommands); if (created) { uint64 shardPlacementId = 0; ShardState shardState = STATE_FINALIZED; shardPlacementId = NextSequenceId(SHARD_PLACEMENT_ID_SEQUENCE_NAME); InsertShardPlacementRow(shardPlacementId, shardId, shardState, nodeName, nodePort); placementCount++; } else { ereport(WARNING, (errmsg("could not create shard on \"%s:%u\"", nodeName, nodePort))); } if (placementCount >= replicationFactor) { break; } } /* check if we created enough shard replicas */ if (placementCount < replicationFactor) { ereport(ERROR, (errmsg("could not satisfy specified replication factor"), errdetail("Created %d shard replicas, less than the " "requested replication factor of %d.", placementCount, replicationFactor))); } /* insert the shard metadata row along with its min/max values */ minHashTokenText = IntegerToText(shardMinHashToken); maxHashTokenText = IntegerToText(shardMaxHashToken); InsertShardRow(distributedTableId, shardId, shardStorageType, minHashTokenText, maxHashTokenText); } if (QueryCancelPending) { ereport(WARNING, (errmsg("cancel requests are ignored during shard creation"))); QueryCancelPending = false; } RESUME_INTERRUPTS(); PG_RETURN_VOID(); }
/* * CreateColocatedShards creates shards for the target relation colocated with * the source relation. */ void CreateColocatedShards(Oid targetRelationId, Oid sourceRelationId, bool useExclusiveConnections) { char targetShardStorageType = 0; List *existingShardList = NIL; List *sourceShardIntervalList = NIL; ListCell *sourceShardCell = NULL; bool colocatedShard = true; List *insertedShardPlacements = NIL; /* make sure that tables are hash partitioned */ CheckHashPartitionedTable(targetRelationId); CheckHashPartitionedTable(sourceRelationId); /* * In contrast to append/range partitioned tables it makes more sense to * require ownership privileges - shards for hash-partitioned tables are * only created once, not continually during ingest as for the other * partitioning types. */ EnsureTableOwner(targetRelationId); /* we plan to add shards: get an exclusive lock on target relation oid */ LockRelationOid(targetRelationId, ExclusiveLock); /* we don't want source table to get dropped before we colocate with it */ LockRelationOid(sourceRelationId, AccessShareLock); /* prevent placement changes of the source relation until we colocate with them */ sourceShardIntervalList = LoadShardIntervalList(sourceRelationId); LockShardListMetadata(sourceShardIntervalList, ShareLock); /* validate that shards haven't already been created for this table */ existingShardList = LoadShardList(targetRelationId); if (existingShardList != NIL) { char *targetRelationName = get_rel_name(targetRelationId); ereport(ERROR, (errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE), errmsg("table \"%s\" has already had shards created for it", targetRelationName))); } targetShardStorageType = ShardStorageType(targetRelationId); foreach(sourceShardCell, sourceShardIntervalList) { ShardInterval *sourceShardInterval = (ShardInterval *) lfirst(sourceShardCell); uint64 sourceShardId = sourceShardInterval->shardId; uint64 newShardId = GetNextShardId(); ListCell *sourceShardPlacementCell = NULL; int32 shardMinValue = DatumGetInt32(sourceShardInterval->minValue); int32 shardMaxValue = DatumGetInt32(sourceShardInterval->maxValue); text *shardMinValueText = IntegerToText(shardMinValue); text *shardMaxValueText = IntegerToText(shardMaxValue); List *sourceShardPlacementList = ShardPlacementList(sourceShardId); InsertShardRow(targetRelationId, newShardId, targetShardStorageType, shardMinValueText, shardMaxValueText); foreach(sourceShardPlacementCell, sourceShardPlacementList) { ShardPlacement *sourcePlacement = (ShardPlacement *) lfirst(sourceShardPlacementCell); uint32 groupId = sourcePlacement->groupId; const RelayFileState shardState = FILE_FINALIZED; const uint64 shardSize = 0; uint64 shardPlacementId = 0; ShardPlacement *shardPlacement = NULL; /* * Optimistically add shard placement row the pg_dist_shard_placement, in case * of any error it will be roll-backed. */ shardPlacementId = InsertShardPlacementRow(newShardId, INVALID_PLACEMENT_ID, shardState, shardSize, groupId); shardPlacement = LoadShardPlacement(newShardId, shardPlacementId); insertedShardPlacements = lappend(insertedShardPlacements, shardPlacement); }
/* * master_copy_shard_placement implements a user-facing UDF to copy data from * a healthy (source) node to an inactive (target) node. To accomplish this it * entirely recreates the table structure before copying all data. During this * time all modifications are paused to the shard. After successful repair, the * inactive placement is marked healthy and modifications may continue. If the * repair fails at any point, this function throws an error, leaving the node * in an unhealthy state. */ Datum master_copy_shard_placement(PG_FUNCTION_ARGS) { int64 shardId = PG_GETARG_INT64(0); text *sourceNodeName = PG_GETARG_TEXT_P(1); int32 sourceNodePort = PG_GETARG_INT32(2); text *targetNodeName = PG_GETARG_TEXT_P(3); int32 targetNodePort = PG_GETARG_INT32(4); ShardInterval *shardInterval = LoadShardInterval(shardId); Oid distributedTableId = shardInterval->relationId; List *shardPlacementList = NIL; ShardPlacement *sourcePlacement = NULL; ShardPlacement *targetPlacement = NULL; List *ddlCommandList = NIL; bool recreated = false; bool dataCopied = false; /* * By taking an exclusive lock on the shard, we both stop all modifications * (INSERT, UPDATE, or DELETE) and prevent concurrent repair operations from * being able to operate on this shard. */ LockShard(shardId, ExclusiveLock); shardPlacementList = LoadShardPlacementList(shardId); sourcePlacement = SearchShardPlacementInList(shardPlacementList, sourceNodeName, sourceNodePort); if (sourcePlacement->shardState != STATE_FINALIZED) { ereport(ERROR, (errcode(ERRCODE_INVALID_PARAMETER_VALUE), errmsg("source placement must be in finalized state"))); } targetPlacement = SearchShardPlacementInList(shardPlacementList, targetNodeName, targetNodePort); if (targetPlacement->shardState != STATE_INACTIVE) { ereport(ERROR, (errcode(ERRCODE_INVALID_PARAMETER_VALUE), errmsg("target placement must be in inactive state"))); } /* retrieve the DDL commands for the table and run them */ ddlCommandList = RecreateTableDDLCommandList(distributedTableId, shardId); recreated = ExecuteRemoteCommandList(targetPlacement->nodeName, targetPlacement->nodePort, ddlCommandList); if (!recreated) { ereport(ERROR, (errmsg("could not recreate shard table"), errhint("Consult recent messages in the server logs for " "details."))); } HOLD_INTERRUPTS(); dataCopied = CopyDataFromFinalizedPlacement(distributedTableId, shardId, sourcePlacement, targetPlacement); if (!dataCopied) { ereport(ERROR, (errmsg("could not copy shard data"), errhint("Consult recent messages in the server logs for " "details."))); } /* the placement is repaired, so return to finalized state */ DeleteShardPlacementRow(targetPlacement->id); InsertShardPlacementRow(targetPlacement->id, targetPlacement->shardId, STATE_FINALIZED, targetPlacement->nodeName, targetPlacement->nodePort); RESUME_INTERRUPTS(); PG_RETURN_VOID(); }