/* * master_get_table_ddl_events takes in a relation name, and returns the set of * DDL commands needed to reconstruct the relation. The returned DDL commands * are similar in flavor to schema definitions that pgdump returns. The function * errors if given relation does not exist. */ Datum master_get_table_ddl_events(PG_FUNCTION_ARGS) { FuncCallContext *functionContext = NULL; ListCell *tableDDLEventCell = NULL; /* * On the very first call to this function, we first use the given relation * name to get to the relation. We then recreate the list of DDL statements * issued for this relation, and save the first statement's position in the * function context. */ if (SRF_IS_FIRSTCALL()) { text *relationName = PG_GETARG_TEXT_P(0); Oid relationId = ResolveRelationId(relationName); MemoryContext oldContext = NULL; List *tableDDLEventList = NIL; /* create a function context for cross-call persistence */ functionContext = SRF_FIRSTCALL_INIT(); /* switch to memory context appropriate for multiple function calls */ oldContext = MemoryContextSwitchTo(functionContext->multi_call_memory_ctx); /* allocate DDL statements, and then save position in DDL statements */ tableDDLEventList = GetTableDDLEvents(relationId); tableDDLEventCell = list_head(tableDDLEventList); functionContext->user_fctx = tableDDLEventCell; MemoryContextSwitchTo(oldContext); } /* * On every call to this function, we get the current position in the * statement list. We then iterate to the next position in the list and * return the current statement, if we have not yet reached the end of * list. */ functionContext = SRF_PERCALL_SETUP(); tableDDLEventCell = (ListCell *) functionContext->user_fctx; if (tableDDLEventCell != NULL) { char *ddlStatement = (char *) lfirst(tableDDLEventCell); text *ddlStatementText = cstring_to_text(ddlStatement); functionContext->user_fctx = lnext(tableDDLEventCell); SRF_RETURN_NEXT(functionContext, PointerGetDatum(ddlStatementText)); } else { SRF_RETURN_DONE(functionContext); } }
/* * worker_copy_shard_placement implements a internal UDF to copy a table's data from * a healthy placement into a receiving table on an unhealthy placement. This * function returns a boolean reflecting success or failure. */ Datum worker_copy_shard_placement(PG_FUNCTION_ARGS) { text *shardRelationNameText = PG_GETARG_TEXT_P(0); text *nodeNameText = PG_GETARG_TEXT_P(1); int32 nodePort = PG_GETARG_INT32(2); char *shardRelationName = text_to_cstring(shardRelationNameText); char *nodeName = text_to_cstring(nodeNameText); bool fetchSuccessful = false; Oid shardRelationId = ResolveRelationId(shardRelationNameText); Relation shardTable = heap_open(shardRelationId, RowExclusiveLock); TupleDesc tupleDescriptor = RelationGetDescr(shardTable); Tuplestorestate *tupleStore = tuplestore_begin_heap(false, false, work_mem); StringInfo selectAllQuery = NULL; ShardPlacement *placement = NULL; Task *task = NULL; selectAllQuery = makeStringInfo(); appendStringInfo(selectAllQuery, SELECT_ALL_QUERY, quote_identifier(shardRelationName)); placement = (ShardPlacement *) palloc0(sizeof(ShardPlacement)); placement->nodeName = nodeName; placement->nodePort = nodePort; task = (Task *) palloc0(sizeof(Task)); task->queryString = selectAllQuery; task->taskPlacementList = list_make1(placement); fetchSuccessful = ExecuteTaskAndStoreResults(task, tupleDescriptor, tupleStore); if (!fetchSuccessful) { ereport(ERROR, (errmsg("could not store shard rows from healthy placement"), errhint("Consult recent messages in the server logs for " "details."))); } CopyDataFromTupleStoreToRelation(tupleStore, shardTable); tuplestore_end(tupleStore); heap_close(shardTable, RowExclusiveLock); PG_RETURN_VOID(); }
/* * GetNextPlacementId allocates and returns a unique placementId for * the placement to be created. This allocation occurs both in shared memory * and in write ahead logs; writing to logs avoids the risk of having shardId * collisions. * * 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. */ uint64 GetNextPlacementId(void) { text *sequenceName = NULL; Oid sequenceId = InvalidOid; Datum sequenceIdDatum = 0; Oid savedUserId = InvalidOid; int savedSecurityContext = 0; Datum placementIdDatum = 0; uint64 placementId = 0; /* * In regression tests, we would like to generate placement IDs consistently * even if the tests run in parallel. Instead of the sequence, we can use * the next_placement_id GUC to specify which shard ID the current session * should generate next. The GUC is automatically increased by 1 every time * a new placement ID is generated. */ if (NextPlacementId > 0) { placementId = NextPlacementId; NextPlacementId += 1; return placementId; } sequenceName = cstring_to_text(PLACEMENTID_SEQUENCE_NAME); sequenceId = ResolveRelationId(sequenceName); sequenceIdDatum = ObjectIdGetDatum(sequenceId); GetUserIdAndSecContext(&savedUserId, &savedSecurityContext); SetUserIdAndSecContext(CitusExtensionOwner(), SECURITY_LOCAL_USERID_CHANGE); /* generate new and unique placement id from sequence */ placementIdDatum = DirectFunctionCall1(nextval_oid, sequenceIdDatum); SetUserIdAndSecContext(savedUserId, savedSecurityContext); placementId = DatumGetInt64(placementIdDatum); return placementId; }
/* * master_create_worker_shards is a user facing function to create worker shards * for the given relation in round robin order. */ 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); /* do not add any data */ bool useExclusiveConnections = false; EnsureCoordinator(); CheckCitusVersion(ERROR); CreateShardsWithRoundRobinPolicy(distributedTableId, shardCount, replicationFactor, useExclusiveConnections); PG_RETURN_VOID(); }
/* * master_get_new_shardid allocates and returns a unique shardId for the shard * to be created. This allocation occurs both in shared memory and in write * ahead logs; writing to logs avoids the risk of having shardId collisions. * * Please note that the caller is still responsible for finalizing shard data * and the shardId with the master node. Further note that this function relies * on an internal sequence created in initdb to generate unique identifiers. * * 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) { text *sequenceName = cstring_to_text(SHARDID_SEQUENCE_NAME); Oid sequenceId = ResolveRelationId(sequenceName); Datum sequenceIdDatum = ObjectIdGetDatum(sequenceId); Oid savedUserId = InvalidOid; int savedSecurityContext = 0; Datum shardIdDatum = 0; GetUserIdAndSecContext(&savedUserId, &savedSecurityContext); SetUserIdAndSecContext(CitusExtensionOwner(), SECURITY_LOCAL_USERID_CHANGE); /* generate new and unique shardId from sequence */ shardIdDatum = DirectFunctionCall1(nextval_oid, sequenceIdDatum); SetUserIdAndSecContext(savedUserId, savedSecurityContext); PG_RETURN_DATUM(shardIdDatum); }
/* * master_create_empty_shard creates an empty shard for the given distributed * table. For this, the function first gets a list of candidate nodes, connects * to these nodes, and issues DDL commands on the nodes to create empty shard * placements. The function then updates metadata on the master node to make * this shard (and its placements) visible. */ Datum master_create_empty_shard(PG_FUNCTION_ARGS) { text *relationNameText = PG_GETARG_TEXT_P(0); char *relationName = text_to_cstring(relationNameText); Datum shardIdDatum = 0; int64 shardId = INVALID_SHARD_ID; List *ddlEventList = NULL; uint32 attemptableNodeCount = 0; uint32 liveNodeCount = 0; uint32 candidateNodeCount = 0; List *candidateNodeList = NIL; text *nullMinValue = NULL; text *nullMaxValue = NULL; char partitionMethod = 0; char storageType = SHARD_STORAGE_TABLE; Oid relationId = ResolveRelationId(relationNameText); char *relationOwner = TableOwner(relationId); EnsureTablePermissions(relationId, ACL_INSERT); CheckDistributedTable(relationId); if (CStoreTable(relationId)) { storageType = SHARD_STORAGE_COLUMNAR; } partitionMethod = PartitionMethod(relationId); if (partitionMethod == DISTRIBUTE_BY_HASH) { ereport(ERROR, (errmsg("relation \"%s\" is a hash partitioned table", relationName), errdetail("We currently don't support creating shards " "on hash-partitioned tables"))); } /* generate new and unique shardId from sequence */ shardIdDatum = master_get_new_shardid(NULL); shardId = DatumGetInt64(shardIdDatum); /* get table DDL commands to replay on the worker node */ ddlEventList = GetTableDDLEvents(relationId); /* if enough live nodes, add an extra candidate node as backup */ attemptableNodeCount = ShardReplicationFactor; liveNodeCount = WorkerGetLiveNodeCount(); if (liveNodeCount > ShardReplicationFactor) { attemptableNodeCount = ShardReplicationFactor + 1; } /* first retrieve a list of random nodes for shard placements */ while (candidateNodeCount < attemptableNodeCount) { WorkerNode *candidateNode = WorkerGetCandidateNode(candidateNodeList); if (candidateNode == NULL) { ereport(ERROR, (errmsg("could only find %u of %u possible nodes", candidateNodeCount, attemptableNodeCount))); } candidateNodeList = lappend(candidateNodeList, candidateNode); candidateNodeCount++; } CreateShardPlacements(shardId, ddlEventList, relationOwner, candidateNodeList, 0, ShardReplicationFactor); InsertShardRow(relationId, shardId, storageType, nullMinValue, nullMaxValue); PG_RETURN_INT64(shardId); }
/* * master_get_table_metadata takes in a relation name, and returns partition * related metadata for the relation. These metadata are grouped and returned in * a tuple, and are used by the caller when creating new shards. The function * errors if given relation does not exist, or is not partitioned. */ Datum master_get_table_metadata(PG_FUNCTION_ARGS) { text *relationName = PG_GETARG_TEXT_P(0); Oid relationId = ResolveRelationId(relationName); DistTableCacheEntry *partitionEntry = NULL; TypeFuncClass resultTypeClass = 0; Datum partitionKeyExpr = 0; Datum partitionKey = 0; Datum metadataDatum = 0; HeapTuple metadataTuple = NULL; TupleDesc metadataDescriptor = NULL; uint64 shardMaxSizeInBytes = 0; char relationType = 0; char storageType = 0; Datum values[TABLE_METADATA_FIELDS]; bool isNulls[TABLE_METADATA_FIELDS]; /* find partition tuple for partitioned relation */ partitionEntry = DistributedTableCacheEntry(relationId); /* create tuple descriptor for return value */ resultTypeClass = get_call_result_type(fcinfo, NULL, &metadataDescriptor); if (resultTypeClass != TYPEFUNC_COMPOSITE) { ereport(ERROR, (errmsg("return type must be a row type"))); } /* get decompiled expression tree for partition key */ partitionKeyExpr = PointerGetDatum(cstring_to_text(partitionEntry->partitionKeyString)); partitionKey = DirectFunctionCall2(pg_get_expr, partitionKeyExpr, ObjectIdGetDatum(relationId)); /* form heap tuple for table metadata */ memset(values, 0, sizeof(values)); memset(isNulls, false, sizeof(isNulls)); shardMaxSizeInBytes = (int64) ShardMaxSize * 1024L; /* get storage type */ relationType = get_rel_relkind(relationId); if (relationType == RELKIND_RELATION) { storageType = SHARD_STORAGE_TABLE; } else if (relationType == RELKIND_FOREIGN_TABLE) { bool cstoreTable = CStoreTable(relationId); if (cstoreTable) { storageType = SHARD_STORAGE_COLUMNAR; } else { storageType = SHARD_STORAGE_FOREIGN; } } values[0] = ObjectIdGetDatum(relationId); values[1] = storageType; values[2] = partitionEntry->partitionMethod; values[3] = partitionKey; values[4] = Int32GetDatum(ShardReplicationFactor); values[5] = Int64GetDatum(shardMaxSizeInBytes); values[6] = Int32GetDatum(ShardPlacementPolicy); metadataTuple = heap_form_tuple(metadataDescriptor, values, isNulls); metadataDatum = HeapTupleGetDatum(metadataTuple); PG_RETURN_DATUM(metadataDatum); }
/* * master_create_distributed_table inserts the table and partition column * information into the partition metadata table. Note that this function * currently assumes the table is hash partitioned. */ Datum master_create_distributed_table(PG_FUNCTION_ARGS) { text *tableNameText = PG_GETARG_TEXT_P(0); text *partitionColumnText = PG_GETARG_TEXT_P(1); char partitionMethod = PG_GETARG_CHAR(2); Oid distributedTableId = ResolveRelationId(tableNameText); char relationKind = '\0'; char *partitionColumnName = text_to_cstring(partitionColumnText); char *tableName = text_to_cstring(tableNameText); Var *partitionColumn = NULL; /* verify target relation is either regular or foreign table */ relationKind = get_rel_relkind(distributedTableId); if (relationKind != RELKIND_RELATION && relationKind != RELKIND_FOREIGN_TABLE) { ereport(ERROR, (errcode(ERRCODE_WRONG_OBJECT_TYPE), errmsg("cannot distribute relation: %s", tableName), errdetail("Distributed relations must be regular or " "foreign tables."))); } /* this will error out if no column exists with the specified name */ partitionColumn = ColumnNameToColumn(distributedTableId, partitionColumnName); /* check for support function needed by specified partition method */ if (partitionMethod == HASH_PARTITION_TYPE) { Oid hashSupportFunction = SupportFunctionForColumn(partitionColumn, HASH_AM_OID, HASHPROC); if (hashSupportFunction == InvalidOid) { ereport(ERROR, (errcode(ERRCODE_UNDEFINED_FUNCTION), errmsg("could not identify a hash function for type %s", format_type_be(partitionColumn->vartype)), errdatatype(partitionColumn->vartype), errdetail("Partition column types must have a hash function " "defined to use hash partitioning."))); } } else if (partitionMethod == RANGE_PARTITION_TYPE) { Oid btreeSupportFunction = InvalidOid; /* * Error out immediately since we don't yet support range partitioning, * but the checks below are ready for when we do. * * TODO: Remove when range partitioning is supported. */ ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED), errmsg("pg_shard only supports hash partitioning"))); btreeSupportFunction = SupportFunctionForColumn(partitionColumn, BTREE_AM_OID, BTORDER_PROC); if (btreeSupportFunction == InvalidOid) { ereport(ERROR, (errcode(ERRCODE_UNDEFINED_FUNCTION), errmsg("could not identify a comparison function for type %s", format_type_be(partitionColumn->vartype)), errdatatype(partitionColumn->vartype), errdetail("Partition column types must have a comparison function " "defined to use range partitioning."))); } } /* insert row into the partition metadata table */ InsertPartitionRow(distributedTableId, partitionMethod, partitionColumnText); PG_RETURN_VOID(); }
/* * 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(); }
/* * master_create_empty_shard creates an empty shard for the given distributed * table. For this, the function first gets a list of candidate nodes, connects * to these nodes, and issues DDL commands on the nodes to create empty shard * placements. The function then updates metadata on the master node to make * this shard (and its placements) visible. */ Datum master_create_empty_shard(PG_FUNCTION_ARGS) { text *relationNameText = PG_GETARG_TEXT_P(0); char *relationName = text_to_cstring(relationNameText); List *workerNodeList = WorkerNodeList(); Datum shardIdDatum = 0; int64 shardId = INVALID_SHARD_ID; List *ddlEventList = NULL; uint32 attemptableNodeCount = 0; uint32 liveNodeCount = 0; uint32 candidateNodeIndex = 0; List *candidateNodeList = NIL; text *nullMinValue = NULL; text *nullMaxValue = NULL; char partitionMethod = 0; char storageType = SHARD_STORAGE_TABLE; Oid relationId = ResolveRelationId(relationNameText); char relationKind = get_rel_relkind(relationId); char *relationOwner = TableOwner(relationId); EnsureTablePermissions(relationId, ACL_INSERT); CheckDistributedTable(relationId); /* * We check whether the table is a foreign table or not. If it is, we set * storage type as foreign also. Only exception is if foreign table is a * foreign cstore table, in this case we set storage type as columnar. * * i.e. While setting storage type, columnar has priority over foreign. */ if (relationKind == RELKIND_FOREIGN_TABLE) { bool cstoreTable = cstoreTable = CStoreTable(relationId); if (cstoreTable) { storageType = SHARD_STORAGE_COLUMNAR; } else { storageType = SHARD_STORAGE_FOREIGN; } } partitionMethod = PartitionMethod(relationId); if (partitionMethod == DISTRIBUTE_BY_HASH) { ereport(ERROR, (errmsg("relation \"%s\" is a hash partitioned table", relationName), errdetail("We currently don't support creating shards " "on hash-partitioned tables"))); } /* generate new and unique shardId from sequence */ shardIdDatum = master_get_new_shardid(NULL); shardId = DatumGetInt64(shardIdDatum); /* get table DDL commands to replay on the worker node */ ddlEventList = GetTableDDLEvents(relationId); /* if enough live nodes, add an extra candidate node as backup */ attemptableNodeCount = ShardReplicationFactor; liveNodeCount = WorkerGetLiveNodeCount(); if (liveNodeCount > ShardReplicationFactor) { attemptableNodeCount = ShardReplicationFactor + 1; } /* first retrieve a list of random nodes for shard placements */ while (candidateNodeIndex < attemptableNodeCount) { WorkerNode *candidateNode = NULL; if (ShardPlacementPolicy == SHARD_PLACEMENT_LOCAL_NODE_FIRST) { candidateNode = WorkerGetLocalFirstCandidateNode(candidateNodeList); } else if (ShardPlacementPolicy == SHARD_PLACEMENT_ROUND_ROBIN) { candidateNode = WorkerGetRoundRobinCandidateNode(workerNodeList, shardId, candidateNodeIndex); } else if (ShardPlacementPolicy == SHARD_PLACEMENT_RANDOM) { candidateNode = WorkerGetRandomCandidateNode(candidateNodeList); } else { ereport(ERROR, (errmsg("unrecognized shard placement policy"))); } if (candidateNode == NULL) { ereport(ERROR, (errmsg("could only find %u of %u possible nodes", candidateNodeIndex, attemptableNodeCount))); } candidateNodeList = lappend(candidateNodeList, candidateNode); candidateNodeIndex++; } CreateShardPlacements(relationId, shardId, ddlEventList, relationOwner, candidateNodeList, 0, ShardReplicationFactor); InsertShardRow(relationId, shardId, storageType, nullMinValue, nullMaxValue); PG_RETURN_INT64(shardId); }
/* * 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 *relationOwner = NULL; char shardStorageType = '\0'; List *workerNodeList = NIL; List *ddlCommandList = NIL; int32 workerNodeCount = 0; uint32 placementAttemptCount = 0; uint64 hashTokenIncrement = 0; List *existingShardList = NIL; int64 shardIndex = 0; /* 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 metadata lock */ LockRelationDistributionMetadata(distributedTableId, ExclusiveLock); relationOwner = TableOwner(distributedTableId); /* validate that shards haven't already been created for this table */ existingShardList = LoadShardList(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("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"))); } /* calculate the split of the hash space */ hashTokenIncrement = HASH_TOKEN_COUNT / shardCount; /* load and sort the worker node list for deterministic placement */ workerNodeList = WorkerNodeList(); 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 = GetTableDDLEvents(distributedTableId); 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 */ if (relationKind == RELKIND_FOREIGN_TABLE) { bool cstoreTable = CStoreTable(distributedTableId); if (cstoreTable) { shardStorageType = SHARD_STORAGE_COLUMNAR; } else { shardStorageType = SHARD_STORAGE_FOREIGN; } } else { shardStorageType = SHARD_STORAGE_TABLE; } 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); Datum shardIdDatum = master_get_new_shardid(NULL); int64 shardId = DatumGetInt64(shardIdDatum); /* 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); CreateShardPlacements(shardId, ddlCommandList, relationOwner, workerNodeList, roundRobinNodeIndex, replicationFactor); 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(); }