pfsdb

package
v2.9.4 Latest Latest
Warning

This package is not in the latest version of its module.

Go to latest
Published: Apr 8, 2024 License: Apache-2.0 Imports: 22 Imported by: 0

Documentation

Overview

Package pfsdb contains the database schema that PFS uses.

Index

Constants

View Source
const (
	BranchesChannelName     = "pfs_branches"
	BranchesRepoChannelName = "pfs_branches_repo_"
)
View Source
const (
	BranchColumnID        = branchColumn("branch.id")
	BranchColumnRepoID    = branchColumn("branch.repo_id")
	BranchColumnCreatedAt = branchColumn("branch.created_at")
	BranchColumnUpdatedAt = branchColumn("branch.updated_at")
)
View Source
const (
	// CommitsChannelName is used to watch events for the commits table.
	CommitsChannelName     = "pfs_commits"
	CommitsRepoChannelName = "pfs_commits_repo_"
	CommitChannelName      = "pfs_commits_"
)
View Source
const (
	SortOrderNone = sortOrder("")
	SortOrderAsc  = sortOrder("ASC")
	SortOrderDesc = sortOrder("DESC")
)
View Source
const (
	RepoColumnID        = repoColumn("repo.id")
	RepoColumnCreatedAt = repoColumn("repo.created_at")
	RepoColumnUpdatedAt = repoColumn("repo.updated_at")
)
View Source
const (
	// ReposChannelName is used to watch events for the repos table.
	ReposChannelName = "pfs_repos"
)

Variables

View Source
var (
	CommitColumnID        = commitColumn("commit.int_id")
	CommitColumnSetID     = commitColumn("commit.commit_set_id")
	CommitColumnRepoID    = commitColumn("commit.repo_id")
	CommitColumnOrigin    = commitColumn("commit.origin")
	CommitColumnCreatedAt = commitColumn("commit.created_at")
	CommitColumnUpdatedAt = commitColumn("commit.updated_at")
)
View Source
var (
	ProjectColumnID        = projectColumn("project.id")
	ProjectColumnCreatedAt = projectColumn("project.created_at")
	ProjectColumnUpdatedAt = projectColumn("project.updated_at")
)
View Source
var BranchesRepoIndex = &col.Index{
	Name: "repo",
	Extract: func(val proto.Message) string {
		return RepoKey(val.(*pfs.BranchInfo).Branch.Repo)
	},
}
View Source
var CommitsBranchlessIndex = &col.Index{
	Name: "branchless",
	Extract: func(val proto.Message) string {
		return CommitKey(val.(*pfs.CommitInfo).Commit)
	},
}
View Source
var CommitsCommitSetIndex = &col.Index{
	Name: "commitset",
	Extract: func(val proto.Message) string {
		return val.(*pfs.CommitInfo).Commit.Id
	},
}
View Source
var CommitsRepoIndex = &col.Index{
	Name: "repo",
	Extract: func(val proto.Message) string {
		return RepoKey(val.(*pfs.CommitInfo).Commit.Repo)
	},
}

Functions

func AddCommit

func AddCommit(tx *pachsql.Tx, commit *pfs.Commit) error

func AddCommitProvenance

func AddCommitProvenance(tx *pachsql.Tx, from, to *pfs.Commit) error

func BranchKey

func BranchKey(branch *pfs.Branch) string

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

func BranchesInRepoChannel(repoID RepoID) string

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 CommitDirectProvenance(ctx context.Context, extCtx sqlx.ExtContext, id CommitID) ([]*pfs.Commit, error)

func CommitKey

func CommitKey(commit *pfs.Commit) string

func CommitSetProvenance

func CommitSetProvenance(tx *pachsql.Tx, id string) (_ []*pfs.Commit, retErr error)

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

func CommitSetSubvenance(tx *pachsql.Tx, id string) (_ []*pfs.Commit, retErr error)

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

func CommitsInRepoChannel(repoID RepoID) string

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

func CreateDirectBranchProvenance(ctx context.Context, ext sqlx.ExtContext, from, to BranchID) error

CreateBranchProvenance creates a provenance relationship between two branches.

func CreateProject added in v2.8.0

func CreateProject(ctx context.Context, tx *pachsql.Tx, project *pfs.ProjectInfo) error

CreateProject creates an entry in the core.projects table.

func DeleteBranch added in v2.8.0

func DeleteBranch(ctx context.Context, tx *pachsql.Tx, b *BranchInfoWithID, force bool) error

DeleteBranch deletes a branch.

func DeleteBranchTrigger added in v2.8.0

func DeleteBranchTrigger(ctx context.Context, tx *pachsql.Tx, from BranchID) error

func DeleteCommit added in v2.8.0

func DeleteCommit(ctx context.Context, tx *pachsql.Tx, commit *pfs.Commit) error

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

func DeleteProject(ctx context.Context, tx *pachsql.Tx, projectName string) error

DeleteProject deletes an entry in the core.projects table.

func DeleteRepo added in v2.8.0

func DeleteRepo(ctx context.Context, tx *pachsql.Tx, repoProject, repoName, repoType string) error

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 ForEachProject(ctx context.Context, tx *pachsql.Tx, cb func(projectWithID ProjectWithID) error) error

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

func GetBranchInfo(ctx context.Context, tx *pachsql.Tx, id BranchID) (*pfs.BranchInfo, error)

GetBranchInfo returns a *pfs.BranchInfo by id.

func GetBranchProvenance added in v2.8.0

func GetBranchProvenance(ctx context.Context, ext sqlx.ExtContext, id BranchID) ([]*pfs.Branch, error)

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

func GetBranchSubvenance(ctx context.Context, ext sqlx.ExtContext, id BranchID) ([]*pfs.Branch, error)

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 GetBranchTrigger(ctx context.Context, ext sqlx.ExtContext, from BranchID) (*pfs.Trigger, error)

func GetCommit added in v2.8.0

func GetCommit(ctx context.Context, tx *pachsql.Tx, id CommitID) (*pfs.CommitInfo, error)

GetCommit returns the commitInfo where int_id=id.

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 GetCommitSubvenance(ctx context.Context, tx *pachsql.Tx, commit *pfs.Commit) ([]*pfs.Commit, error)

func GetDirectBranchProvenance added in v2.8.0

func GetDirectBranchProvenance(ctx context.Context, ext sqlx.ExtContext, id BranchID) ([]*pfs.Branch, error)

GetDirectBranchProvenance returns the direct provenance of a branch, i.e. all branches that it directly depends on.

func GetDirectBranchSubvenance added in v2.9.0

func GetDirectBranchSubvenance(ctx context.Context, ext sqlx.ExtContext, id BranchID) ([]*pfs.Branch, error)

func GetProject added in v2.8.0

func GetProject(ctx context.Context, tx *pachsql.Tx, id ProjectID) (*pfs.ProjectInfo, error)

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

func GetRepo(ctx context.Context, tx *pachsql.Tx, id RepoID) (*pfs.RepoInfo, error)

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 GetTriggeredBranches added in v2.9.0

func GetTriggeredBranches(ctx context.Context, ext sqlx.ExtContext, bid BranchID) ([]*pfs.Branch, error)

GetTriggeredBranches lists all the branches that are directly triggered by a branch

func GetTriggeringBranches added in v2.9.0

func GetTriggeringBranches(ctx context.Context, ext sqlx.ExtContext, bid BranchID) ([]*pfs.Branch, error)

GetTriggeringBranches lists all the branches that would directly trigger a branch

func IsChildCommitNotFound added in v2.8.0

func IsChildCommitNotFound(err error) bool

func IsDuplicateKeyErr added in v2.8.0

func IsDuplicateKeyErr(err error) bool

func IsErrProjectAlreadyExists added in v2.8.0

func IsErrProjectAlreadyExists(err error) bool

func IsErrRepoNotFound added in v2.8.0

func IsErrRepoNotFound(err error) bool

func IsNotFoundError added in v2.8.0

func IsNotFoundError(err error) bool

func IsParentCommitNotFound added in v2.8.0

func IsParentCommitNotFound(err error) bool

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 ParseBranch(key string) *pfs.Branch

func ParseCommit

func ParseCommit(key string) *pfs.Commit

func ParseRepo

func ParseRepo(key string) *pfs.Repo

func ProjectKey

func ProjectKey(project *pfs.Project) string

func RepoKey

func RepoKey(repo *pfs.Repo) string

func ResolveCommitProvenance

func ResolveCommitProvenance(tx *pachsql.Tx, repo *pfs.Repo, commitSet string) (*pfs.Commit, error)

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 UpsertBranchTrigger(ctx context.Context, tx *pachsql.Tx, from BranchID, to BranchID, trigger *pfs.Trigger) error

func UpsertProject added in v2.8.0

func UpsertProject(ctx context.Context, tx *pachsql.Tx, project *pfs.ProjectInfo) error

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.

func WatchBranchesInRepo added in v2.8.2

func WatchBranchesInRepo(ctx context.Context, db *pachsql.DB, listener collection.PostgresListener, repoID RepoID, onUpsert branchUpsertHandler, onDelete branchDeleteHandler) error

func WatchCommit added in v2.8.1

func WatchCommit(ctx context.Context, db *pachsql.DB, listener collection.PostgresListener, commitID CommitID, onUpsert commitUpsertHandler, onDelete commitDeleteHandler) error

WatchCommit creates a watcher and watches for changes to a single commit.

func WatchCommits added in v2.8.1

func WatchCommits(ctx context.Context, db *pachsql.DB, listener collection.PostgresListener, onUpsert commitUpsertHandler, onDelete commitDeleteHandler) error

WatchCommits creates a watcher and watches the pfs.commits table for changes.

func WatchCommitsInRepo added in v2.8.1

func WatchCommitsInRepo(ctx context.Context, db *pachsql.DB, listener collection.PostgresListener, repoID RepoID, onUpsert commitUpsertHandler, onDelete commitDeleteHandler) error

WatchCommitsInRepo creates a watcher and watches for commits in a repo.

func WatchRepos added in v2.8.2

func WatchRepos(ctx context.Context, db *pachsql.DB, listener collection.PostgresListener, onUpsert repoUpsertHandler, onDelete repoDeleteHandler) error

Types

type AncestryOpt added in v2.8.0

type AncestryOpt struct {
	SkipChildren bool
	SkipParent   bool
}

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

func (*Branch) Pb added in v2.8.0

func (branch *Branch) Pb() *pfs.Branch

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

func GetBranchID(ctx context.Context, tx *pachsql.Tx, branch *pfs.Branch) (BranchID, error)

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, ext sqlx.ExtContext, startPage, pageSize uint64, filter *pfs.Branch, orderBys ...OrderByBranchColumn) (*BranchIterator, error)

func (*BranchIterator) Next added in v2.8.0

type BranchNotFoundError added in v2.8.0

type BranchNotFoundError struct {
	ID        BranchID
	BranchKey string
}

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

type ChildCommitNotFoundError struct {
	Repo           string
	ParentRowID    CommitID
	ParentCommitID string
}

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

func (*Commit) Pb added in v2.8.0

func (commit *Commit) Pb() *pfs.Commit

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

func GetCommitID(ctx context.Context, tx *pachsql.Tx, commit *pfs.Commit) (CommitID, error)

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

type CommitNotFoundError struct {
	RowID    CommitID
	CommitID string
}

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

func GetCommitWithIDByKey(ctx context.Context, tx *pachsql.Tx, commit *pfs.Commit) (*CommitWithID, error)

type CreatedAtUpdatedAt added in v2.8.0

type CreatedAtUpdatedAt struct {
	CreatedAt time.Time `db:"created_at"`
	UpdatedAt time.Time `db:"updated_at"`
}

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

type ParentCommitNotFoundError struct {
	ChildRowID    CommitID
	ChildCommitID string
}

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) Pb added in v2.8.0

func (project *Project) Pb() *pfs.Project

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

type ProjectNotFoundError struct {
	Name string
	ID   ProjectID
}

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

func ListProject(ctx context.Context, tx *pachsql.Tx) ([]ProjectWithID, error)

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

func (*Repo) Pb added in v2.8.0

func (repo *Repo) Pb() *pfs.Repo

func (*Repo) PbInfo added in v2.8.0

func (repo *Repo) PbInfo() (*pfs.RepoInfo, error)

type RepoID added in v2.8.0

type RepoID uint64

RepoID is the row id for a repo entry in postgres.

func GetRepoID added in v2.8.0

func GetRepoID(ctx context.Context, tx *pachsql.Tx, repoProject, repoName, repoType string) (RepoID, error)

func UpsertRepo added in v2.8.0

func UpsertRepo(ctx context.Context, tx *pachsql.Tx, repo *pfs.RepoInfo) (RepoID, error)

UpsertRepo will attempt to insert a repo, and return its ID. If the repo already exists, it will update its description.

type RepoInfoWithID added in v2.8.0

type RepoInfoWithID struct {
	ID       RepoID
	RepoInfo *pfs.RepoInfo
	Revision int64
}

RepoInfoWithID is an (id, repoInfo) tuple returned by the repo iterator.

func ListRepo added in v2.8.0

func ListRepo(ctx context.Context, tx *pachsql.Tx, filter *pfs.Repo, orderBys ...OrderByRepoColumn) ([]RepoInfoWithID, error)

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, ext sqlx.ExtContext, 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

type RepoNotFoundError struct {
	Project string
	Name    string
	Type    string
	ID      RepoID
}

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

Jump to

Keyboard shortcuts

? : This menu
/ : Search site
f or F : Jump to
y or Y : Canonical URL