/* * ShardStorageType returns the shard storage type according to relation type. */ char ShardStorageType(Oid relationId) { char shardStorageType = 0; char relationType = get_rel_relkind(relationId); if (RegularTable(relationId)) { shardStorageType = SHARD_STORAGE_TABLE; } else if (relationType == RELKIND_FOREIGN_TABLE) { bool cstoreTable = CStoreTable(relationId); if (cstoreTable) { shardStorageType = SHARD_STORAGE_COLUMNAR; } else { shardStorageType = SHARD_STORAGE_FOREIGN; } } else { ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED), errmsg("unexpected relation type: %c", relationType))); } return shardStorageType; }
/* * citus_total_relation_size accepts a table name and returns a distributed table * and its indexes' total relation size. */ Datum citus_total_relation_size(PG_FUNCTION_ARGS) { Oid relationId = PG_GETARG_OID(0); uint64 totalRelationSize = 0; char *tableSizeFunction = PG_TOTAL_RELATION_SIZE_FUNCTION; CheckCitusVersion(ERROR); if (CStoreTable(relationId)) { tableSizeFunction = CSTORE_TABLE_SIZE_FUNCTION; } totalRelationSize = DistributedTableSize(relationId, tableSizeFunction); PG_RETURN_INT64(totalRelationSize); }
/* * CStoreProcessUtility is the hook for handling utility commands. This function * intercepts "COPY cstore_table FROM" statements, and redirectes execution to * CopyIntoCStoreTable function. For all other utility statements, the function * calls the previous utility hook or the standard utility command. */ static void CStoreProcessUtility(Node *parseTree, const char *queryString, ProcessUtilityContext context, ParamListInfo paramListInfo, DestReceiver *destReceiver, char *completionTag) { bool copyIntoCStoreTable = false; /* check if the statement is a "COPY cstore_table FROM ..." statement */ if (nodeTag(parseTree) == T_CopyStmt) { CopyStmt *copyStatement = (CopyStmt *) parseTree; if (copyStatement->is_from && CStoreTable(copyStatement->relation)) { copyIntoCStoreTable = true; } } if (copyIntoCStoreTable) { uint64 processed = CopyIntoCStoreTable((CopyStmt *) parseTree, queryString); if (completionTag != NULL) { snprintf(completionTag, COMPLETION_TAG_BUFSIZE, "COPY " UINT64_FORMAT, processed); } } else if (PreviousProcessUtilityHook != NULL) { PreviousProcessUtilityHook(parseTree, queryString, context, paramListInfo, destReceiver, completionTag); } else { standard_ProcessUtility(parseTree, queryString, context, paramListInfo, destReceiver, completionTag); } }
/* * 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_append_table_to_shard appends the given table's contents to the given * shard, and updates shard metadata on the master node. If the function fails * to append table data to all shard placements, it doesn't update any metadata * and errors out. Else if the function fails to append table data to some of * the shard placements, it marks those placements as invalid. These invalid * placements will get cleaned up during shard rebalancing. */ Datum master_append_table_to_shard(PG_FUNCTION_ARGS) { uint64 shardId = PG_GETARG_INT64(0); text *sourceTableNameText = PG_GETARG_TEXT_P(1); text *sourceNodeNameText = PG_GETARG_TEXT_P(2); uint32 sourceNodePort = PG_GETARG_UINT32(3); char *sourceTableName = text_to_cstring(sourceTableNameText); char *sourceNodeName = text_to_cstring(sourceNodeNameText); char *shardName = NULL; List *shardPlacementList = NIL; List *succeededPlacementList = NIL; List *failedPlacementList = NIL; ListCell *shardPlacementCell = NULL; ListCell *failedPlacementCell = NULL; uint64 newShardSize = 0; uint64 shardMaxSizeInBytes = 0; float4 shardFillLevel = 0.0; char partitionMethod = 0; ShardInterval *shardInterval = LoadShardInterval(shardId); Oid relationId = shardInterval->relationId; bool cstoreTable = CStoreTable(relationId); char storageType = shardInterval->storageType; EnsureTablePermissions(relationId, ACL_INSERT); if (storageType != SHARD_STORAGE_TABLE && !cstoreTable) { ereport(ERROR, (errmsg("cannot append to shardId " UINT64_FORMAT, shardId), errdetail("The underlying shard is not a regular table"))); } partitionMethod = PartitionMethod(relationId); if (partitionMethod == DISTRIBUTE_BY_HASH) { ereport(ERROR, (errmsg("cannot append to shardId " UINT64_FORMAT, shardId), errdetail("We currently don't support appending to shards " "in hash-partitioned tables"))); } /* * We lock on the shardId, but do not unlock. When the function returns, and * the transaction for this function commits, this lock will automatically * be released. This ensures appends to a shard happen in a serial manner. */ LockShardResource(shardId, AccessExclusiveLock); /* if shard doesn't have an alias, extend regular table name */ shardName = LoadShardAlias(relationId, shardId); if (shardName == NULL) { shardName = get_rel_name(relationId); AppendShardIdToName(&shardName, shardId); } shardPlacementList = FinalizedShardPlacementList(shardId); if (shardPlacementList == NIL) { ereport(ERROR, (errmsg("could not find any shard placements for shardId " UINT64_FORMAT, shardId), errhint("Try running master_create_empty_shard() first"))); } /* issue command to append table to each shard placement */ foreach(shardPlacementCell, shardPlacementList) { ShardPlacement *shardPlacement = (ShardPlacement *) lfirst(shardPlacementCell); char *workerName = shardPlacement->nodeName; uint32 workerPort = shardPlacement->nodePort; List *queryResultList = NIL; StringInfo workerAppendQuery = makeStringInfo(); appendStringInfo(workerAppendQuery, WORKER_APPEND_TABLE_TO_SHARD, quote_literal_cstr(shardName), quote_literal_cstr(sourceTableName), quote_literal_cstr(sourceNodeName), sourceNodePort); /* inserting data should be performed by the current user */ queryResultList = ExecuteRemoteQuery(workerName, workerPort, NULL, workerAppendQuery); if (queryResultList != NIL) { succeededPlacementList = lappend(succeededPlacementList, shardPlacement); } else { failedPlacementList = lappend(failedPlacementList, shardPlacement); } }
/* * 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); }
/* LocalTableSize returns the size on disk of the given table. */ static uint64 LocalTableSize(Oid relationId) { uint64 tableSize = 0; char relationType = 0; Datum relationIdDatum = ObjectIdGetDatum(relationId); relationType = get_rel_relkind(relationId); if (relationType == RELKIND_RELATION) { Datum tableSizeDatum = DirectFunctionCall1(pg_table_size, relationIdDatum); tableSize = DatumGetInt64(tableSizeDatum); } else if (relationType == RELKIND_FOREIGN_TABLE) { bool cstoreTable = CStoreTable(relationId); if (cstoreTable) { /* extract schema name of cstore */ Oid cstoreId = get_extension_oid(CSTORE_FDW_NAME, false); Oid cstoreSchemaOid = get_extension_schema(cstoreId); const char *cstoreSchemaName = get_namespace_name(cstoreSchemaOid); const int tableSizeArgumentCount = 1; Oid tableSizeFunctionOid = FunctionOid(cstoreSchemaName, CSTORE_TABLE_SIZE_FUNCTION_NAME, tableSizeArgumentCount); Datum tableSizeDatum = OidFunctionCall1(tableSizeFunctionOid, relationIdDatum); tableSize = DatumGetInt64(tableSizeDatum); } else { char *relationName = get_rel_name(relationId); struct stat fileStat; int statOK = 0; StringInfo localFilePath = makeStringInfo(); appendStringInfo(localFilePath, FOREIGN_CACHED_FILE_PATH, relationName); /* extract the file size using stat, analogous to pg_stat_file */ statOK = stat(localFilePath->data, &fileStat); if (statOK < 0) { ereport(ERROR, (errcode_for_file_access(), errmsg("could not stat file \"%s\": %m", localFilePath->data))); } tableSize = (uint64) fileStat.st_size; } } else { char *relationName = get_rel_name(relationId); ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED), errmsg("cannot get size for table \"%s\"", relationName), errdetail("Only regular and foreign tables are supported."))); } return tableSize; }
/* * 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(); }