Documentation ¶
Overview ¶
Package pfsdb contains the database schema that PFS uses.
Index ¶
- Constants
- Variables
- func AddCommit(tx *pachsql.Tx, commit *pfs.Commit) error
- func AddCommitProvenance(tx *pachsql.Tx, from, to *pfs.Commit) error
- func BranchKey(branch *pfs.Branch) string
- func Branches(db *pachsql.DB, listener col.PostgresListener) col.PostgresCollection
- func BranchesInRepoChannel(repoID RepoID) string
- func CommitDirectProvenance(ctx context.Context, extCtx sqlx.ExtContext, id CommitID) ([]*pfs.Commit, error)
- func CommitKey(commit *pfs.Commit) string
- func CommitSetProvenance(tx *pachsql.Tx, id string) (_ []*pfs.Commit, retErr error)
- func CommitSetSubvenance(tx *pachsql.Tx, id string) (_ []*pfs.Commit, retErr error)
- func Commits(db *pachsql.DB, listener col.PostgresListener) col.PostgresCollection
- func CommitsInRepoChannel(repoID RepoID) string
- func CreateCommitAncestries(ctx context.Context, tx *pachsql.Tx, parentCommit CommitID, ...) error
- func CreateCommitChildren(ctx context.Context, tx *pachsql.Tx, parentCommit CommitID, ...) error
- func CreateCommitParent(ctx context.Context, tx *pachsql.Tx, parentCommit *pfs.Commit, ...) error
- func CreateDirectBranchProvenance(ctx context.Context, tx *pachsql.Tx, from, to BranchID) error
- func CreateProject(ctx context.Context, tx *pachsql.Tx, project *pfs.ProjectInfo) error
- func DeleteBranch(ctx context.Context, tx *pachsql.Tx, id BranchID, force bool) error
- func DeleteBranchTrigger(ctx context.Context, tx *pachsql.Tx, from BranchID) error
- func DeleteCommit(ctx context.Context, tx *pachsql.Tx, commit *pfs.Commit) error
- func DeleteProject(ctx context.Context, tx *pachsql.Tx, projectName string) error
- func DeleteRepo(ctx context.Context, tx *pachsql.Tx, repoProject, repoName, repoType string) error
- func ForEachBranch(ctx context.Context, tx *pachsql.Tx, filter *pfs.Branch, ...) error
- func ForEachCommit(ctx context.Context, db *pachsql.DB, filter *pfs.Commit, ...) error
- func ForEachCommitTxByFilter(ctx context.Context, tx *pachsql.Tx, filter *pfs.Commit, ...) error
- func ForEachProject(ctx context.Context, tx *pachsql.Tx, ...) error
- func ForEachRepo(ctx context.Context, tx *pachsql.Tx, filter *pfs.Repo, ...) error
- func GetBranchInfo(ctx context.Context, tx *pachsql.Tx, id BranchID) (*pfs.BranchInfo, error)
- func GetBranchProvenance(ctx context.Context, tx *pachsql.Tx, id BranchID) ([]*pfs.Branch, error)
- func GetBranchSubvenance(ctx context.Context, tx *pachsql.Tx, id BranchID) ([]*pfs.Branch, error)
- func GetBranchTrigger(ctx context.Context, tx *pachsql.Tx, from BranchID) (*pfs.Trigger, error)
- func GetCommit(ctx context.Context, tx *pachsql.Tx, id CommitID) (*pfs.CommitInfo, error)
- func GetCommitByCommitKey(ctx context.Context, tx *pachsql.Tx, commit *pfs.Commit) (*pfs.CommitInfo, error)
- func GetCommitChildren(ctx context.Context, tx *pachsql.Tx, parentCommit CommitID) ([]*pfs.Commit, error)
- func GetCommitParent(ctx context.Context, tx *pachsql.Tx, childCommit CommitID) (*pfs.Commit, error)
- func GetCommitSubvenance(ctx context.Context, tx *pachsql.Tx, commit *pfs.Commit) ([]*pfs.Commit, error)
- func GetDirectBranchProvenance(ctx context.Context, tx *pachsql.Tx, id BranchID) ([]*pfs.Branch, error)
- func GetProject(ctx context.Context, tx *pachsql.Tx, id ProjectID) (*pfs.ProjectInfo, error)
- func GetProjectByName(ctx context.Context, tx *pachsql.Tx, projectName string) (*pfs.ProjectInfo, error)
- func GetRepo(ctx context.Context, tx *pachsql.Tx, id RepoID) (*pfs.RepoInfo, error)
- func GetRepoByName(ctx context.Context, tx *pachsql.Tx, repoProject, repoName, repoType string) (*pfs.RepoInfo, error)
- func IsChildCommitNotFound(err error) bool
- func IsDuplicateKeyErr(err error) bool
- func IsErrProjectAlreadyExists(err error) bool
- func IsErrRepoNotFound(err error) bool
- func IsNotFoundError(err error) bool
- func IsParentCommitNotFound(err error) bool
- func ListCommitTxByFilter(ctx context.Context, tx *pachsql.Tx, filter *pfs.Commit, ...) ([]*pfs.CommitInfo, error)
- func OrderByQuery[T ColumnName](orderBys ...OrderByColumn[T]) string
- func ParseBranch(key string) *pfs.Branch
- func ParseCommit(key string) *pfs.Commit
- func ParseRepo(key string) *pfs.Repo
- func ProjectKey(project *pfs.Project) string
- func RepoKey(repo *pfs.Repo) string
- func ResolveCommitProvenance(tx *pachsql.Tx, repo *pfs.Repo, commitSet string) (*pfs.Commit, error)
- func UpdateCommit(ctx context.Context, tx *pachsql.Tx, id CommitID, commitInfo *pfs.CommitInfo, ...) error
- func UpdateProject(ctx context.Context, tx *pachsql.Tx, id ProjectID, project *pfs.ProjectInfo) error
- func UpsertBranchTrigger(ctx context.Context, tx *pachsql.Tx, from BranchID, to BranchID, ...) error
- func UpsertProject(ctx context.Context, tx *pachsql.Tx, project *pfs.ProjectInfo) error
- type AncestryOpt
- type Branch
- type BranchID
- type BranchInfoWithID
- type BranchIterator
- type BranchNotFoundError
- type BranchProvCycleError
- type BranchTrigger
- type ChildCommitNotFoundError
- type ColumnName
- type Commit
- type CommitAlreadyExistsError
- type CommitID
- func CreateCommit(ctx context.Context, tx *pachsql.Tx, commitInfo *pfs.CommitInfo, ...) (CommitID, error)
- func GetCommitID(ctx context.Context, tx *pachsql.Tx, commit *pfs.Commit) (CommitID, error)
- func UpsertCommit(ctx context.Context, tx *pachsql.Tx, commitInfo *pfs.CommitInfo, ...) (CommitID, error)
- type CommitIterator
- type CommitMissingInfoError
- type CommitNotFoundError
- type CommitWithID
- type CreatedAtUpdatedAt
- type ModelType
- type OrderByBranchColumn
- type OrderByColumn
- type OrderByCommitColumn
- type OrderByProjectColumn
- type OrderByRepoColumn
- type ParentCommitNotFoundError
- type Project
- type ProjectAlreadyExistsError
- type ProjectID
- type ProjectIterator
- type ProjectNotFoundError
- type ProjectWithID
- type Repo
- type RepoID
- type RepoInfoWithID
- type RepoIterator
- type RepoNotFoundError
Constants ¶
const ( BranchesChannelName = "pfs_branches" BranchesRepoChannelName = "pfs_branches_repo_" )
const ( BranchColumnID = branchColumn("branch.id") BranchColumnCreatedAt = branchColumn("branch.created_at") BranchColumnUpdatedAt = branchColumn("branch.updated_at") )
const ( // CommitsChannelName is used to watch events for the commits table. CommitsChannelName = "pfs_commits" CommitsRepoChannelName = "pfs_commits_repo_" CommitChannelName = "pfs_commits_" )
const ( SortOrderNone = sortOrder("") SortOrderAsc = sortOrder("ASC") SortOrderDesc = sortOrder("DESC") )
const ( RepoColumnID = repoColumn("repo.id") RepoColumnCreatedAt = repoColumn("repo.created_at") RepoColumnUpdatedAt = repoColumn("repo.updated_at") )
const (
// ReposChannelName is used to watch events for the repos table.
ReposChannelName = "pfs_repos"
)
Variables ¶
var ( CommitColumnID = commitColumn("commit.int_id") CommitColumnSetID = commitColumn("commit.commit_set_id") CommitColumnOrigin = commitColumn("commit.origin") CommitColumnCreatedAt = commitColumn("commit.created_at") CommitColumnUpdatedAt = commitColumn("commit.updated_at") )
var ( ProjectColumnID = projectColumn("project.id") ProjectColumnCreatedAt = projectColumn("project.created_at") ProjectColumnUpdatedAt = projectColumn("project.updated_at") )
var BranchesRepoIndex = &col.Index{ Name: "repo", Extract: func(val proto.Message) string { return RepoKey(val.(*pfs.BranchInfo).Branch.Repo) }, }
var CommitsBranchlessIndex = &col.Index{ Name: "branchless", Extract: func(val proto.Message) string { return CommitKey(val.(*pfs.CommitInfo).Commit) }, }
var CommitsCommitSetIndex = &col.Index{ Name: "commitset", Extract: func(val proto.Message) string { return val.(*pfs.CommitInfo).Commit.Id }, }
var CommitsRepoIndex = &col.Index{ Name: "repo", Extract: func(val proto.Message) string { return RepoKey(val.(*pfs.CommitInfo).Commit.Repo) }, }
Functions ¶
func Branches ¶
func Branches(db *pachsql.DB, listener col.PostgresListener) col.PostgresCollection
Branches returns a collection of branches
func BranchesInRepoChannel ¶ added in v2.8.0
BranchesInRepoChannel returns the name of the channel that is notified when branches in repo 'repoID' are created, updated, or deleted
func CommitDirectProvenance ¶ added in v2.8.0
func CommitSetProvenance ¶
CommitSetProvenance returns all the commit IDs that are in the provenance of all the commits in this commit set.
TODO(provenance): is 'SELECT DISTINCT commit_id' a performance concern?
func CommitSetSubvenance ¶
CommitSetSubvenance returns all the commit IDs that contain commits in this commit set in their full (transitive) provenance
func Commits ¶
func Commits(db *pachsql.DB, listener col.PostgresListener) col.PostgresCollection
Commits returns a collection of commits
func CommitsInRepoChannel ¶ added in v2.8.0
CommitsInRepoChannel returns the name of the channel that is notified when commits in repo 'repoID' are created, updated, or deleted
func CreateCommitAncestries ¶ added in v2.8.0
func CreateCommitAncestries(ctx context.Context, tx *pachsql.Tx, parentCommit CommitID, childrenCommits []CommitID) error
CreateCommitAncestries inserts ancestry relationships where the ids of both parent and children are known.
func CreateCommitChildren ¶ added in v2.8.0
func CreateCommitChildren(ctx context.Context, tx *pachsql.Tx, parentCommit CommitID, childCommits []*pfs.Commit) error
CreateCommitChildren inserts ancestry relationships using a single query for all of the children.
func CreateCommitParent ¶ added in v2.8.0
func CreateCommitParent(ctx context.Context, tx *pachsql.Tx, parentCommit *pfs.Commit, childCommit CommitID) error
CreateCommitParent inserts a single ancestry relationship where the child is known and parent must be derived.
func CreateDirectBranchProvenance ¶ added in v2.8.0
CreateBranchProvenance creates a provenance relationship between two branches.
func CreateProject ¶ added in v2.8.0
CreateProject creates an entry in the core.projects table.
func DeleteBranch ¶ added in v2.8.0
DeleteBranch deletes a branch.
func DeleteBranchTrigger ¶ added in v2.8.0
func DeleteCommit ¶ added in v2.8.0
DeleteCommit deletes an entry in the pfs.commits table. It also repoints the references in the commit_ancestry table. The caller is responsible for updating branchesg.
func DeleteProject ¶ added in v2.8.0
DeleteProject deletes an entry in the core.projects table.
func DeleteRepo ¶ added in v2.8.0
DeleteRepo deletes an entry in the pfs.repos table.
func ForEachBranch ¶ added in v2.8.0
func ForEachBranch(ctx context.Context, tx *pachsql.Tx, filter *pfs.Branch, cb func(branchInfoWithID BranchInfoWithID) error, orderBys ...OrderByBranchColumn) error
func ForEachCommit ¶ added in v2.8.0
func ForEachCommit(ctx context.Context, db *pachsql.DB, filter *pfs.Commit, cb func(commitWithID CommitWithID) error, orderBys ...OrderByCommitColumn) error
func ForEachCommitTxByFilter ¶ added in v2.8.0
func ForEachCommitTxByFilter(ctx context.Context, tx *pachsql.Tx, filter *pfs.Commit, cb func(commitWithID CommitWithID) error, orderBys ...OrderByCommitColumn) error
func ForEachProject ¶ added in v2.8.0
func ForEachRepo ¶ added in v2.8.0
func ForEachRepo(ctx context.Context, tx *pachsql.Tx, filter *pfs.Repo, cb func(repoWithID RepoInfoWithID) error, orderBys ...OrderByRepoColumn) error
func GetBranchInfo ¶ added in v2.8.0
GetBranchInfo returns a *pfs.BranchInfo by id.
func GetBranchProvenance ¶ added in v2.8.0
GetBranchProvenance returns the full provenance of a branch, i.e. all branches that it either directly or transitively depends on.
func GetBranchSubvenance ¶ added in v2.8.0
GetBranchSubvenance returns the full subvenance of a branch, i.e. all branches that either directly or transitively depend on it.
func GetBranchTrigger ¶ added in v2.8.0
func GetCommitByCommitKey ¶ added in v2.8.0
func GetCommitByCommitKey(ctx context.Context, tx *pachsql.Tx, commit *pfs.Commit) (*pfs.CommitInfo, error)
GetCommitByCommitKey is like GetCommit but derives the int_id on behalf of the caller.
func GetCommitChildren ¶ added in v2.8.0
func GetCommitChildren(ctx context.Context, tx *pachsql.Tx, parentCommit CommitID) ([]*pfs.Commit, error)
GetCommitChildren uses the pfs.commit_ancestry and pfs.commits tables to retrieve commits of all of the children given an int_id of the parent.
func GetCommitParent ¶ added in v2.8.0
func GetCommitParent(ctx context.Context, tx *pachsql.Tx, childCommit CommitID) (*pfs.Commit, error)
GetCommitParent uses the pfs.commit_ancestry and pfs.commits tables to retrieve a commit given an int_id of one of its children.
func GetCommitSubvenance ¶ added in v2.8.0
func GetDirectBranchProvenance ¶ added in v2.8.0
func GetDirectBranchProvenance(ctx context.Context, tx *pachsql.Tx, id BranchID) ([]*pfs.Branch, error)
GetDirectBranchProvenance returns the direct provenance of a branch, i.e. all branches that it directly depends on.
func GetProject ¶ added in v2.8.0
GetProject is like GetProjectByName, but retrieves an entry using the row id.
func GetProjectByName ¶ added in v2.8.0
func GetProjectByName(ctx context.Context, tx *pachsql.Tx, projectName string) (*pfs.ProjectInfo, error)
GetProjectByName retrieves an entry from the core.projects table by project name.
func GetRepo ¶ added in v2.8.0
todo(fahad): rewrite branch related code during the branches migration. GetRepo retrieves an entry from the pfs.repos table by using the row id.
func GetRepoByName ¶ added in v2.8.0
func GetRepoByName(ctx context.Context, tx *pachsql.Tx, repoProject, repoName, repoType string) (*pfs.RepoInfo, error)
GetRepoByName retrieves an entry from the pfs.repos table by project, repo name, and type.
func IsChildCommitNotFound ¶ added in v2.8.0
func IsDuplicateKeyErr ¶ added in v2.8.0
func IsErrProjectAlreadyExists ¶ added in v2.8.0
func IsErrRepoNotFound ¶ added in v2.8.0
func IsNotFoundError ¶ added in v2.8.0
func IsParentCommitNotFound ¶ added in v2.8.0
func ListCommitTxByFilter ¶ added in v2.8.0
func ListCommitTxByFilter(ctx context.Context, tx *pachsql.Tx, filter *pfs.Commit, orderBys ...OrderByCommitColumn) ([]*pfs.CommitInfo, error)
func OrderByQuery ¶ added in v2.8.0
func OrderByQuery[T ColumnName](orderBys ...OrderByColumn[T]) string
func ParseBranch ¶ added in v2.6.6
func ParseCommit ¶
func ProjectKey ¶
func ResolveCommitProvenance ¶
returns the commit of a certain repo in a commit set.
func UpdateCommit ¶ added in v2.8.0
func UpdateCommit(ctx context.Context, tx *pachsql.Tx, id CommitID, commitInfo *pfs.CommitInfo, opts ...AncestryOpt) error
UpdateCommit overwrites an existing commit entry by CommitID as well as the corresponding ancestry entries.
func UpdateProject ¶ added in v2.8.0
func UpdateProject(ctx context.Context, tx *pachsql.Tx, id ProjectID, project *pfs.ProjectInfo) error
UpdateProject overwrites an existing project entry by ID.
func UpsertBranchTrigger ¶ added in v2.8.0
func UpsertProject ¶ added in v2.8.0
UpsertProject updates all fields of an existing project entry in the core.projects table by name. If 'upsert' is set to true, UpsertProject() will attempt to call CreateProject() if the entry does not exist.
Types ¶
type AncestryOpt ¶ added in v2.8.0
AncestryOpt allows users to create commitInfos and skip creating the ancestry information. This allows a user to create the commits in an arbitrary order, then create their ancestry later.
type Branch ¶ added in v2.8.0
type Branch struct { ID BranchID `db:"id"` Head Commit `db:"head"` Repo Repo `db:"repo"` Name string `db:"name"` CreatedAtUpdatedAt }
Branch is a row in the pfs.branches table.
func (Branch) GetCreatedAtUpdatedAt ¶ added in v2.8.0
func (branch Branch) GetCreatedAtUpdatedAt() CreatedAtUpdatedAt
type BranchID ¶ added in v2.8.0
type BranchID uint64
BranchID is the row id for a branch entry in postgres.
func GetBranchID ¶ added in v2.8.0
GetBranchID returns the id of a branch given a set strings that uniquely identify a branch.
func UpsertBranch ¶ added in v2.8.0
func UpsertBranch(ctx context.Context, tx *pachsql.Tx, branchInfo *pfs.BranchInfo) (BranchID, error)
UpsertBranch creates a branch if it does not exist, or updates the head if the branch already exists. If direct provenance is specified, it will be used to update the branch's provenance relationships.
type BranchInfoWithID ¶ added in v2.8.0
type BranchInfoWithID struct { ID BranchID Revision int64 *pfs.BranchInfo }
func GetBranchInfoWithID ¶ added in v2.8.0
func GetBranchInfoWithID(ctx context.Context, tx *pachsql.Tx, b *pfs.Branch) (*BranchInfoWithID, error)
GetBranchInfoWithID returns a *pfs.BranchInfo by name
func ListBranches ¶ added in v2.8.0
func ListBranches(ctx context.Context, tx *pachsql.Tx, filter *pfs.Branch, orderBys ...OrderByBranchColumn) ([]BranchInfoWithID, error)
type BranchIterator ¶ added in v2.8.0
type BranchIterator struct {
// contains filtered or unexported fields
}
func NewBranchIterator ¶ added in v2.8.0
func NewBranchIterator(ctx context.Context, tx *pachsql.Tx, startPage, pageSize uint64, filter *pfs.Branch, orderBys ...OrderByBranchColumn) (*BranchIterator, error)
func (*BranchIterator) Next ¶ added in v2.8.0
func (i *BranchIterator) Next(ctx context.Context, dst *BranchInfoWithID) error
type BranchNotFoundError ¶ added in v2.8.0
BranchNotFoundError is returned when a branch is not found in postgres.
func (*BranchNotFoundError) Error ¶ added in v2.8.0
func (err *BranchNotFoundError) Error() string
func (*BranchNotFoundError) GRPCStatus ¶ added in v2.8.0
func (err *BranchNotFoundError) GRPCStatus() *status.Status
type BranchProvCycleError ¶ added in v2.8.0
type BranchProvCycleError struct {
From, To string
}
BranchProvCycleError is returned when a cycle is detected at branch creation time.
func (*BranchProvCycleError) Error ¶ added in v2.8.0
func (err *BranchProvCycleError) Error() string
func (*BranchProvCycleError) GRPCStatus ¶ added in v2.8.0
func (err *BranchProvCycleError) GRPCStatus() *status.Status
type BranchTrigger ¶ added in v2.8.0
type BranchTrigger struct { FromBranch Branch `db:"from_branch"` ToBranch Branch `db:"to_branch"` CronSpec string `db:"cron_spec"` RateLimitSpec string `db:"rate_limit_spec"` Size string `db:"size"` NumCommits int64 `db:"num_commits"` AllConditions bool `db:"all_conditions"` }
func (*BranchTrigger) Pb ¶ added in v2.8.0
func (trigger *BranchTrigger) Pb() *pfs.Trigger
type ChildCommitNotFoundError ¶ added in v2.8.0
ChildCommitNotFoundError is returned when a commit's child is not found in postgres.
func (*ChildCommitNotFoundError) Error ¶ added in v2.8.0
func (err *ChildCommitNotFoundError) Error() string
func (*ChildCommitNotFoundError) GRPCStatus ¶ added in v2.8.0
func (err *ChildCommitNotFoundError) GRPCStatus() *status.Status
type ColumnName ¶ added in v2.8.0
type ColumnName interface { string | projectColumn | branchColumn | commitColumn | repoColumn }
type Commit ¶ added in v2.8.0
type Commit struct { ID CommitID `db:"int_id"` CommitSetID string `db:"commit_set_id"` CommitID string `db:"commit_id"` Origin string `db:"origin"` Description string `db:"description"` StartTime sql.NullTime `db:"start_time"` FinishingTime sql.NullTime `db:"finishing_time"` FinishedTime sql.NullTime `db:"finished_time"` CompactingTime sql.NullInt64 `db:"compacting_time_s"` ValidatingTime sql.NullInt64 `db:"validating_time_s"` Error string `db:"error"` Size int64 `db:"size"` // BranchName is used to derive the BranchID in commit related queries. BranchName sql.NullString `db:"branch_name"` BranchID sql.NullInt64 `db:"branch_id"` Repo Repo `db:"repo"` CreatedAtUpdatedAt }
func (Commit) GetCreatedAtUpdatedAt ¶ added in v2.8.0
func (commit Commit) GetCreatedAtUpdatedAt() CreatedAtUpdatedAt
type CommitAlreadyExistsError ¶ added in v2.8.0
type CommitAlreadyExistsError struct {
CommitID string
}
CommitAlreadyExistsError is returned when a commit with the same name already exists in postgres.
func (*CommitAlreadyExistsError) Error ¶ added in v2.8.0
func (err *CommitAlreadyExistsError) Error() string
Error satisfies the error interface.
func (*CommitAlreadyExistsError) GRPCStatus ¶ added in v2.8.0
func (err *CommitAlreadyExistsError) GRPCStatus() *status.Status
type CommitID ¶ added in v2.8.0
type CommitID uint64
CommitID is the row id for a commit entry in postgres.
func CreateCommit ¶ added in v2.8.0
func CreateCommit(ctx context.Context, tx *pachsql.Tx, commitInfo *pfs.CommitInfo, opts ...AncestryOpt) (CommitID, error)
CreateCommit creates an entry in the pfs.commits table. If the commit has a parent or children, it will attempt to create entries in the pfs.commit_ancestry table unless options are provided to skip ancestry creation.
func GetCommitID ¶ added in v2.8.0
GetCommitID returns the int_id of a commit in postgres.
func UpsertCommit ¶ added in v2.8.0
func UpsertCommit(ctx context.Context, tx *pachsql.Tx, commitInfo *pfs.CommitInfo, opts ...AncestryOpt) (CommitID, error)
UpsertCommit will attempt to insert a commit and its ancestry relationships. If the commit already exists, it will update its description.
type CommitIterator ¶ added in v2.8.0
type CommitIterator struct {
// contains filtered or unexported fields
}
func NewCommitsIterator ¶ added in v2.8.0
func NewCommitsIterator(ctx context.Context, extCtx sqlx.ExtContext, startPage, pageSize uint64, filter *pfs.Commit, orderBys ...OrderByCommitColumn) (*CommitIterator, error)
func (*CommitIterator) Next ¶ added in v2.8.0
func (i *CommitIterator) Next(ctx context.Context, dst *CommitWithID) error
type CommitMissingInfoError ¶ added in v2.8.0
type CommitMissingInfoError struct {
Field string
}
CommitMissingInfoError is returned when a commitInfo is missing a field.
func (*CommitMissingInfoError) Error ¶ added in v2.8.0
func (err *CommitMissingInfoError) Error() string
func (*CommitMissingInfoError) GRPCStatus ¶ added in v2.8.0
func (err *CommitMissingInfoError) GRPCStatus() *status.Status
type CommitNotFoundError ¶ added in v2.8.0
CommitNotFoundError is returned by GetCommit() when a commit is not found in postgres.
func (*CommitNotFoundError) Error ¶ added in v2.8.0
func (err *CommitNotFoundError) Error() string
func (*CommitNotFoundError) GRPCStatus ¶ added in v2.8.0
func (err *CommitNotFoundError) GRPCStatus() *status.Status
type CommitWithID ¶ added in v2.8.0
type CommitWithID struct { ID CommitID CommitInfo *pfs.CommitInfo Revision int64 }
CommitWithID is returned by the commit iterator.
func GetCommitWithIDByKey ¶ added in v2.8.0
type CreatedAtUpdatedAt ¶ added in v2.8.0
type ModelType ¶ added in v2.8.0
type ModelType interface { Repo | Commit | Branch | Project GetCreatedAtUpdatedAt() CreatedAtUpdatedAt }
type OrderByBranchColumn ¶ added in v2.8.0
type OrderByBranchColumn OrderByColumn[branchColumn]
type OrderByColumn ¶ added in v2.8.0
type OrderByColumn[T ColumnName] struct { Column T Order sortOrder }
type OrderByCommitColumn ¶ added in v2.8.0
type OrderByCommitColumn OrderByColumn[commitColumn]
type OrderByProjectColumn ¶ added in v2.8.0
type OrderByProjectColumn OrderByColumn[projectColumn]
type OrderByRepoColumn ¶ added in v2.8.0
type OrderByRepoColumn OrderByColumn[repoColumn]
type ParentCommitNotFoundError ¶ added in v2.8.0
ParentCommitNotFoundError is returned when a commit's parent is not found in postgres.
func (*ParentCommitNotFoundError) Error ¶ added in v2.8.0
func (err *ParentCommitNotFoundError) Error() string
func (*ParentCommitNotFoundError) GRPCStatus ¶ added in v2.8.0
func (err *ParentCommitNotFoundError) GRPCStatus() *status.Status
type Project ¶ added in v2.8.0
type Project struct { ID ProjectID `db:"id"` Name string `db:"name"` Description string `db:"description"` CreatedAtUpdatedAt }
func (Project) GetCreatedAtUpdatedAt ¶ added in v2.8.0
func (project Project) GetCreatedAtUpdatedAt() CreatedAtUpdatedAt
func (*Project) PbInfo ¶ added in v2.8.0
func (project *Project) PbInfo() *pfs.ProjectInfo
type ProjectAlreadyExistsError ¶ added in v2.8.0
type ProjectAlreadyExistsError struct {
Name string
}
ProjectAlreadyExistsError is returned by CreateProject() when a project with the same name already exists in postgres.
func (*ProjectAlreadyExistsError) Error ¶ added in v2.8.0
func (err *ProjectAlreadyExistsError) Error() string
Error satisfies the error interface.
func (*ProjectAlreadyExistsError) GRPCStatus ¶ added in v2.8.0
func (err *ProjectAlreadyExistsError) GRPCStatus() *status.Status
func (*ProjectAlreadyExistsError) Is ¶ added in v2.8.0
func (err *ProjectAlreadyExistsError) Is(other error) bool
type ProjectID ¶ added in v2.8.0
type ProjectID uint64
ProjectID is the row id for a project entry in postgres.
type ProjectIterator ¶ added in v2.8.0
type ProjectIterator struct {
// contains filtered or unexported fields
}
func NewProjectIterator ¶ added in v2.8.0
func NewProjectIterator(ctx context.Context, extCtx sqlx.ExtContext, startPage, pageSize uint64, filter *pfs.Project, orderBys ...OrderByProjectColumn) (*ProjectIterator, error)
func (*ProjectIterator) Next ¶ added in v2.8.0
func (i *ProjectIterator) Next(ctx context.Context, dst *ProjectWithID) error
type ProjectNotFoundError ¶ added in v2.8.0
ProjectNotFoundError is returned by GetProject() when a project is not found in postgres.
func (*ProjectNotFoundError) Error ¶ added in v2.8.0
func (err *ProjectNotFoundError) Error() string
Error satisfies the error interface.
func (*ProjectNotFoundError) GRPCStatus ¶ added in v2.8.0
func (err *ProjectNotFoundError) GRPCStatus() *status.Status
func (*ProjectNotFoundError) Is ¶ added in v2.8.0
func (err *ProjectNotFoundError) Is(other error) bool
type ProjectWithID ¶ added in v2.8.0
type ProjectWithID struct { ProjectInfo *pfs.ProjectInfo ID ProjectID Revision int64 }
func ListProject ¶ added in v2.8.0
type Repo ¶ added in v2.8.0
type Repo struct { ID RepoID `db:"id"` Project Project `db:"project"` Name string `db:"name"` Type string `db:"type"` Description string `db:"description"` CreatedAtUpdatedAt BranchesNames string `db:"branches"` }
Repo is a row in the pfs.repos table.
func (Repo) GetCreatedAtUpdatedAt ¶ added in v2.8.0
func (repo Repo) GetCreatedAtUpdatedAt() CreatedAtUpdatedAt
type RepoID ¶ added in v2.8.0
type RepoID uint64
RepoID is the row id for a repo entry in postgres.
type RepoInfoWithID ¶ added in v2.8.0
RepoInfoWithID is an (id, repoInfo) tuple returned by the repo iterator.
type RepoIterator ¶ added in v2.8.0
type RepoIterator struct {
// contains filtered or unexported fields
}
func NewRepoIterator ¶ added in v2.8.0
func NewRepoIterator(ctx context.Context, tx *pachsql.Tx, startPage, pageSize uint64, filter *pfs.Repo, orderBys ...OrderByRepoColumn) (*RepoIterator, error)
func (*RepoIterator) Next ¶ added in v2.8.0
func (i *RepoIterator) Next(ctx context.Context, dst *RepoInfoWithID) error
type RepoNotFoundError ¶ added in v2.8.0
RepoNotFoundError is returned by GetRepo() when a repo is not found in postgres.
func (*RepoNotFoundError) Error ¶ added in v2.8.0
func (err *RepoNotFoundError) Error() string
Error satisfies the error interface.
func (*RepoNotFoundError) GRPCStatus ¶ added in v2.8.0
func (err *RepoNotFoundError) GRPCStatus() *status.Status