/* * master_get_new_shardid is a user facing wrapper function around GetNextShardId() * which allocates and returns a unique shardId for the shard to be created. * * NB: This can be called by any user; for now we have decided that that's * ok. We might want to restrict this to users part of a specific role or such * at some later point. */ Datum master_get_new_shardid(PG_FUNCTION_ARGS) { uint64 shardId = 0; Datum shardIdDatum = 0; EnsureCoordinator(); CheckCitusVersion(ERROR); shardId = GetNextShardId(); shardIdDatum = Int64GetDatum(shardId); PG_RETURN_DATUM(shardIdDatum); }
/* * CreateShardsWithRoundRobinPolicy creates empty shards for the given table * based on the specified number of initial shards. The function first updates * metadata on the coordinator 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. Finally, function creates empty shard placements on * worker nodes. */ void CreateShardsWithRoundRobinPolicy(Oid distributedTableId, int32 shardCount, int32 replicationFactor, bool useExclusiveConnections) { char shardStorageType = 0; List *workerNodeList = NIL; int32 workerNodeCount = 0; uint32 placementAttemptCount = 0; uint64 hashTokenIncrement = 0; List *existingShardList = NIL; int64 shardIndex = 0; DistTableCacheEntry *cacheEntry = DistributedTableCacheEntry(distributedTableId); bool colocatedShard = false; List *insertedShardPlacements = NIL; /* make sure table is hash partitioned */ CheckHashPartitionedTable(distributedTableId); /* * 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(distributedTableId); /* we plan to add shards: get an exclusive lock on relation oid */ LockRelationOid(distributedTableId, ExclusiveLock); /* validate that shards haven't already been created for this table */ existingShardList = LoadShardList(distributedTableId); if (existingShardList != NIL) { char *tableName = get_rel_name(distributedTableId); 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("shard_count must be positive"))); } /* make sure that at least one replica is specified */ if (replicationFactor <= 0) { ereport(ERROR, (errcode(ERRCODE_INVALID_PARAMETER_VALUE), errmsg("replication_factor must be positive"))); } /* make sure that RF=1 if the table is streaming replicated */ if (cacheEntry->replicationModel == REPLICATION_MODEL_STREAMING && replicationFactor > 1) { char *relationName = get_rel_name(cacheEntry->relationId); ereport(ERROR, (errcode(ERRCODE_INVALID_PARAMETER_VALUE), errmsg("using replication factor %d with the streaming " "replication model is not supported", replicationFactor), errdetail("The table %s is marked as streaming replicated and " "the shard replication factor of streaming replicated " "tables must be 1.", relationName), errhint("Use replication factor 1."))); } /* calculate the split of the hash space */ hashTokenIncrement = HASH_TOKEN_COUNT / shardCount; /* don't allow concurrent node list changes that require an exclusive lock */ LockRelationOid(DistNodeRelationId(), RowShareLock); /* load and sort the worker node list for deterministic placement */ workerNodeList = ActivePrimaryNodeList(); workerNodeList = SortList(workerNodeList, CompareWorkerNodes); /* make sure we don't process cancel signals until all shards are created */ HOLD_INTERRUPTS(); workerNodeCount = list_length(workerNodeList); if (replicationFactor > workerNodeCount) { ereport(ERROR, (errcode(ERRCODE_INVALID_PARAMETER_VALUE), errmsg("replication_factor (%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 */ shardStorageType = ShardStorageType(distributedTableId); for (shardIndex = 0; shardIndex < shardCount; shardIndex++) { uint32 roundRobinNodeIndex = shardIndex % workerNodeCount; /* initialize the hash token space for this shard */ text *minHashTokenText = NULL; text *maxHashTokenText = NULL; int32 shardMinHashToken = INT32_MIN + (shardIndex * hashTokenIncrement); int32 shardMaxHashToken = shardMinHashToken + (hashTokenIncrement - 1); uint64 shardId = GetNextShardId(); List *currentInsertedShardPlacements = NIL; /* if we are at the last shard, make sure the max token value is INT_MAX */ if (shardIndex == (shardCount - 1)) { shardMaxHashToken = INT32_MAX; } /* insert the shard metadata row along with its min/max values */ minHashTokenText = IntegerToText(shardMinHashToken); maxHashTokenText = IntegerToText(shardMaxHashToken); /* * Grabbing the shard metadata lock isn't technically necessary since * we already hold an exclusive lock on the partition table, but we'll * acquire it for the sake of completeness. As we're adding new active * placements, the mode must be exclusive. */ LockShardDistributionMetadata(shardId, ExclusiveLock); InsertShardRow(distributedTableId, shardId, shardStorageType, minHashTokenText, maxHashTokenText); currentInsertedShardPlacements = InsertShardPlacementRows(distributedTableId, shardId, workerNodeList, roundRobinNodeIndex, replicationFactor); insertedShardPlacements = list_concat(insertedShardPlacements, currentInsertedShardPlacements); } CreateShardsOnWorkers(distributedTableId, insertedShardPlacements, useExclusiveConnections, colocatedShard); if (QueryCancelPending) { ereport(WARNING, (errmsg("cancel requests are ignored during shard creation"))); QueryCancelPending = false; } RESUME_INTERRUPTS(); }
/* * 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); }