Welcome to mirror list, hosted at ThFree Co, Russian Federation.

gitlab.com/gitlab-org/gitaly.git - Unnamed repository; edit this file 'description' to name the repository.
summaryrefslogtreecommitdiff
diff options
context:
space:
mode:
authorWill Chandler <wchandler@gitlab.com>2023-11-20 20:37:32 +0300
committerWill Chandler <wchandler@gitlab.com>2023-11-20 20:37:32 +0300
commit276bfb610a543a5de88a1d5f69219811dad13a6b (patch)
tree9ed26512c3c2f81e43b286a1e7573368c2a77ccb
parent9dbf48d9b74ea2247409802155564db02a99ede9 (diff)
parentadc156fe95433002eb33ffb1c13cd2d267d14c70 (diff)
Merge branch 'smh-snapshot-multiple-repos' into 'master'
Allow snapshotting multiple repositories See merge request https://gitlab.com/gitlab-org/gitaly/-/merge_requests/6522 Merged-by: Will Chandler <wchandler@gitlab.com> Approved-by: Quang-Minh Nguyen <qmnguyen@gitlab.com> Approved-by: Will Chandler <wchandler@gitlab.com> Reviewed-by: Will Chandler <wchandler@gitlab.com> Co-authored-by: Sami Hiltunen <shiltunen@gitlab.com>
-rw-r--r--internal/gitaly/storage/storagemgr/partition_manager.go4
-rw-r--r--internal/gitaly/storage/storagemgr/snapshot.go158
-rw-r--r--internal/gitaly/storage/storagemgr/transaction_manager.go149
-rw-r--r--internal/gitaly/storage/storagemgr/transaction_manager_test.go478
4 files changed, 657 insertions, 132 deletions
diff --git a/internal/gitaly/storage/storagemgr/partition_manager.go b/internal/gitaly/storage/storagemgr/partition_manager.go
index e59190bf3..899ad6a97 100644
--- a/internal/gitaly/storage/storagemgr/partition_manager.go
+++ b/internal/gitaly/storage/storagemgr/partition_manager.go
@@ -30,7 +30,7 @@ var ErrPartitionManagerClosed = errors.New("partition manager closed")
// transactionManager is the interface of TransactionManager as used by PartitionManager. See the
// TransactionManager's documentation for more details.
type transactionManager interface {
- Begin(context.Context, string, bool) (*Transaction, error)
+ Begin(context.Context, string, []string, bool) (*Transaction, error)
Run() error
Close()
isClosing() bool
@@ -474,7 +474,7 @@ func (pm *PartitionManager) Begin(ctx context.Context, storageName, relativePath
ptn.pendingTransactionCount++
storageMgr.mu.Unlock()
- transaction, err := ptn.transactionManager.Begin(ctx, relativePath, readOnly)
+ transaction, err := ptn.transactionManager.Begin(ctx, relativePath, nil, readOnly)
if err != nil {
// The pending transaction count needs to be decremented since the transaction is no longer
// inflight. A transaction failing does not necessarily mean the transaction manager has
diff --git a/internal/gitaly/storage/storagemgr/snapshot.go b/internal/gitaly/storage/storagemgr/snapshot.go
new file mode 100644
index 000000000..1b90a978e
--- /dev/null
+++ b/internal/gitaly/storage/storagemgr/snapshot.go
@@ -0,0 +1,158 @@
+package storagemgr
+
+import (
+ "context"
+ "errors"
+ "fmt"
+ "io/fs"
+ "os"
+ "path/filepath"
+
+ "gitlab.com/gitlab-org/gitaly/v16/internal/git/housekeeping"
+ "gitlab.com/gitlab-org/gitaly/v16/internal/helper/perm"
+)
+
+// snapshot represents a snapshot of a partition's state at a given time.
+type snapshot struct {
+ prefix string
+}
+
+// relativePath returns the given relative path rewritten to point to the relative
+// path in the snapshot.
+func (s snapshot) relativePath(relativePath string) string {
+ return filepath.Join(s.prefix, relativePath)
+}
+
+// newSnapshot creates a snapshot of the given relative paths and their alternates under snapshotPath.
+func newSnapshot(ctx context.Context, storagePath, snapshotPath string, relativePaths []string) (snapshot, error) {
+ snapshotPrefix, err := filepath.Rel(storagePath, snapshotPath)
+ if err != nil {
+ return snapshot{}, fmt.Errorf("rel snapshot prefix: %w", err)
+ }
+
+ if err := createRepositorySnapshots(ctx, storagePath, snapshotPrefix, relativePaths); err != nil {
+ return snapshot{}, fmt.Errorf("create repository snapshots: %w", err)
+ }
+
+ return snapshot{prefix: snapshotPrefix}, nil
+}
+
+// createRepositorySnapshots creates a snapshot of the partition containing all repositories at the given relative paths
+// and their alternates.
+func createRepositorySnapshots(ctx context.Context, storagePath, snapshotPrefix string, relativePaths []string) error {
+ snapshottedRepositories := make(map[string]struct{}, len(relativePaths))
+ for _, relativePath := range relativePaths {
+ if _, ok := snapshottedRepositories[relativePath]; ok {
+ continue
+ }
+
+ sourcePath := filepath.Join(storagePath, relativePath)
+ targetPath := filepath.Join(storagePath, snapshotPrefix, relativePath)
+ if err := createRepositorySnapshot(ctx, sourcePath, targetPath); err != nil {
+ return fmt.Errorf("create snapshot: %w", err)
+ }
+
+ snapshottedRepositories[relativePath] = struct{}{}
+
+ // Read the repository's 'objects/info/alternates' file to figure out whether it is connected
+ // to an alternate. If so, we need to include the alternate repository in the snapshot along
+ // with the repository itself to ensure the objects from the alternate are also available.
+ if alternate, err := readAlternatesFile(targetPath); err != nil && !errors.Is(err, errNoAlternate) {
+ return fmt.Errorf("get alternate path: %w", err)
+ } else if alternate != "" {
+ // The repository had an alternate. The path is a relative from the repository's 'objects' directory
+ // to the alternate's 'objects' directory. Build the relative path of the alternate repository.
+ alternateRelativePath := filepath.Dir(filepath.Join(relativePath, "objects", alternate))
+ if _, ok := snapshottedRepositories[alternateRelativePath]; ok {
+ continue
+ }
+
+ // Include the alternate repository in the snapshot as well.
+ if err := createRepositorySnapshot(ctx,
+ filepath.Join(storagePath, alternateRelativePath),
+ filepath.Join(storagePath, snapshotPrefix, alternateRelativePath),
+ ); err != nil {
+ return fmt.Errorf("create alternate snapshot: %w", err)
+ }
+
+ snapshottedRepositories[alternateRelativePath] = struct{}{}
+ }
+ }
+
+ return nil
+}
+
+// createRepositorySnapshot snapshots a repository's current state at snapshotPath. This is done by
+// recreating the repository's directory structure and hard linking the repository's files in their
+// correct locations there. This effectively does a copy-free clone of the repository. Since the files
+// are shared between the snapshot and the repository, they must not be modified. Git doesn't modify
+// existing files but writes new ones so this property is upheld.
+func createRepositorySnapshot(ctx context.Context, repositoryPath, snapshotPath string) error {
+ // This creates the parent directory hierarchy regardless of whether the repository exists or not. It also
+ // doesn't consider the permissions in the storage. While not 100% correct, we have no logic that cares about
+ // the storage hierarchy above repositories.
+ //
+ // The repository's directory itself is not yet created as whether it should be created depends on whether the
+ // repository exists or not.
+ if err := os.MkdirAll(filepath.Dir(snapshotPath), perm.PrivateDir); err != nil {
+ return fmt.Errorf("create parent directory hierarchy: %w", err)
+ }
+
+ if err := createDirectorySnapshot(ctx, repositoryPath, snapshotPath, map[string]struct{}{
+ // Don't include worktrees in the snapshot. All of the worktrees in the repository should be leftover
+ // state from before transaction management was introduced as the transactions would create their
+ // worktrees in the snapshot.
+ housekeeping.WorktreesPrefix: {},
+ housekeeping.GitlabWorktreePrefix: {},
+ }); err != nil {
+ return fmt.Errorf("create directory snapshot: %w", err)
+ }
+
+ return nil
+}
+
+// createDirectorySnapshot recursively recreates the directory structure from originalDirectory into
+// snapshotDirectory and hard links files into the same locations in snapshotDirectory.
+//
+// skipRelativePaths can be provided to skip certain entries from being included in the snapshot.
+func createDirectorySnapshot(ctx context.Context, originalDirectory, snapshotDirectory string, skipRelativePaths map[string]struct{}) error {
+ if err := filepath.Walk(originalDirectory, func(oldPath string, info fs.FileInfo, err error) error {
+ if err != nil {
+ if errors.Is(err, fs.ErrNotExist) && oldPath == originalDirectory {
+ // The directory being snapshotted does not exist. This is fine as the transaction
+ // may be about to create it.
+ return nil
+ }
+
+ return err
+ }
+
+ relativePath, err := filepath.Rel(originalDirectory, oldPath)
+ if err != nil {
+ return fmt.Errorf("rel: %w", err)
+ }
+
+ if _, ok := skipRelativePaths[relativePath]; ok {
+ return fs.SkipDir
+ }
+
+ newPath := filepath.Join(snapshotDirectory, relativePath)
+ if info.IsDir() {
+ if err := os.Mkdir(newPath, info.Mode().Perm()); err != nil {
+ return fmt.Errorf("create dir: %w", err)
+ }
+ } else if info.Mode().IsRegular() {
+ if err := os.Link(oldPath, newPath); err != nil {
+ return fmt.Errorf("link file: %w", err)
+ }
+ } else {
+ return fmt.Errorf("unsupported file mode: %q", info.Mode())
+ }
+
+ return nil
+ }); err != nil {
+ return fmt.Errorf("walk: %w", err)
+ }
+
+ return nil
+}
diff --git a/internal/gitaly/storage/storagemgr/transaction_manager.go b/internal/gitaly/storage/storagemgr/transaction_manager.go
index d8be4bbc1..78ba2fd5b 100644
--- a/internal/gitaly/storage/storagemgr/transaction_manager.go
+++ b/internal/gitaly/storage/storagemgr/transaction_manager.go
@@ -184,9 +184,6 @@ type Transaction struct {
// quarantineDirectory is the directory within the stagingDirectory where the new objects of the
// transaction are quarantined.
quarantineDirectory string
- // snapshotBaseRelativePath is the relative path of the snapshot directory in the storage.
- // It's used to rewrite the repositories to point to their snapshots.
- snapshotBaseRelativePath string
// packPrefix contains the prefix (`pack-<digest>`) of the transaction's pack if the transaction
// had objects to log.
packPrefix string
@@ -197,6 +194,9 @@ type Transaction struct {
// snapshotLSN is the log sequence number which this transaction is reading the repository's
// state at.
snapshotLSN LSN
+ // snapshot is the transaction's snapshot of the partition. It's used to rewrite relative paths to
+ // point to the snapshot instead of the actual repositories.
+ snapshot snapshot
// stagingRepository is a repository that is used to stage the transaction. If there are quarantined
// objects, it has the quarantine applied so the objects are available for verification and packing.
// Generally the staging repository is the actual repository instance. If the repository doesn't exist
@@ -223,9 +223,12 @@ type Transaction struct {
//
// relativePath is the relative path of the target repository the transaction is operating on.
//
+// snapshottedRelativePaths are the relative paths to snapshot in addition to target repository.
+// These are read-only as the transaction can only perform changes against the target repository.
+//
// readOnly indicates whether this is a read-only transaction. Read-only transactions are not
// configured with a quarantine directory and do not commit a log entry.
-func (mgr *TransactionManager) Begin(ctx context.Context, relativePath string, readOnly bool) (_ *Transaction, returnedErr error) {
+func (mgr *TransactionManager) Begin(ctx context.Context, relativePath string, snapshottedRelativePaths []string, readOnly bool) (_ *Transaction, returnedErr error) {
// Wait until the manager has been initialized so the notification channels
// and the LSNs are loaded.
select {
@@ -290,45 +293,22 @@ func (mgr *TransactionManager) Begin(ctx context.Context, relativePath string, r
return nil, fmt.Errorf("mkdir temp: %w", err)
}
- if txn.snapshotBaseRelativePath, err = filepath.Rel(
+ mgr.stateLock.RLock()
+ defer mgr.stateLock.RUnlock()
+ if txn.snapshot, err = newSnapshot(ctx,
mgr.storagePath,
filepath.Join(txn.stagingDirectory, "snapshot"),
+ append(snapshottedRelativePaths, txn.relativePath),
); err != nil {
- return nil, fmt.Errorf("snapshot root relative path: %w", err)
- }
-
- snapshotRepositoryPath := mgr.getAbsolutePath(txn.snapshotRelativePath(txn.relativePath))
- if err := mgr.createRepositorySnapshot(ctx,
- mgr.getAbsolutePath(txn.relativePath),
- snapshotRepositoryPath,
- ); err != nil {
- return nil, fmt.Errorf("create snapshot: %w", err)
- }
-
- // Read the repository's 'objects/info/alternates' file to figure out whether it is connected
- // to an alternate. If so, we need to include the alternate repository in the snapshot along
- // with the repository itself to ensure the objects from the alternate are also available.
- if alternate, err := readAlternatesFile(snapshotRepositoryPath); err != nil && !errors.Is(err, errNoAlternate) {
- return nil, fmt.Errorf("get alternate path: %w", err)
- } else if alternate != "" {
- // The repository had an alternate. The path is a relative from the repository's 'objects' directory
- // to the alternates 'objects' directory. Build the relative path of the alternate repository.
- alternateRelativePath := filepath.Dir(filepath.Join(txn.relativePath, "objects", alternate))
- // Include the alternate repository in the snapshot as well.
- if err := mgr.createRepositorySnapshot(ctx,
- mgr.getAbsolutePath(alternateRelativePath),
- mgr.getAbsolutePath(txn.snapshotRelativePath(alternateRelativePath)),
- ); err != nil {
- return nil, fmt.Errorf("create alternate snapshot: %w", err)
- }
+ return nil, fmt.Errorf("new snapshot: %w", err)
}
- txn.repositoryExists, err = mgr.doesRepositoryExist(txn.snapshotRelativePath(txn.relativePath))
+ txn.repositoryExists, err = mgr.doesRepositoryExist(txn.snapshot.relativePath(txn.relativePath))
if err != nil {
return nil, fmt.Errorf("does repository exist: %w", err)
}
- txn.snapshotRepository = mgr.repositoryFactory.Build(txn.snapshotRelativePath(txn.relativePath))
+ txn.snapshotRepository = mgr.repositoryFactory.Build(txn.snapshot.relativePath(txn.relativePath))
if !txn.readOnly {
if txn.repositoryExists {
txn.quarantineDirectory = filepath.Join(txn.stagingDirectory, "quarantine")
@@ -341,7 +321,7 @@ func (mgr *TransactionManager) Begin(ctx context.Context, relativePath string, r
return nil, fmt.Errorf("quarantine: %w", err)
}
} else {
- txn.quarantineDirectory = filepath.Join(mgr.storagePath, txn.snapshotRelativePath(txn.relativePath), "objects")
+ txn.quarantineDirectory = filepath.Join(mgr.storagePath, txn.snapshot.relativePath(txn.relativePath), "objects")
}
}
@@ -349,108 +329,29 @@ func (mgr *TransactionManager) Begin(ctx context.Context, relativePath string, r
}
}
-// snapshotRelativePath returns a rewritten relative path that points to the passed in relative path's
-// location in the snapshot.
-func (txn *Transaction) snapshotRelativePath(relativePath string) string {
- return filepath.Join(txn.snapshotBaseRelativePath, relativePath)
-}
-
// RewriteRepository returns a copy of the repository that has been set up to correctly access
// the repository in the transaction's snapshot.
func (txn *Transaction) RewriteRepository(repo *gitalypb.Repository) *gitalypb.Repository {
rewritten := proto.Clone(repo).(*gitalypb.Repository)
- rewritten.RelativePath = txn.snapshotRelativePath(repo.RelativePath)
- rewritten.GitObjectDirectory = txn.snapshotRepository.GetGitObjectDirectory()
- rewritten.GitAlternateObjectDirectories = txn.snapshotRepository.GetGitAlternateObjectDirectories()
+ rewritten.RelativePath = txn.snapshot.relativePath(repo.RelativePath)
+
+ if repo.RelativePath == txn.relativePath {
+ rewritten.GitObjectDirectory = txn.snapshotRepository.GetGitObjectDirectory()
+ rewritten.GitAlternateObjectDirectories = txn.snapshotRepository.GetGitAlternateObjectDirectories()
+ }
+
return rewritten
}
// OriginalRepository returns the repository as it was before rewriting it to point to the snapshot.
func (txn *Transaction) OriginalRepository(repo *gitalypb.Repository) *gitalypb.Repository {
original := proto.Clone(repo).(*gitalypb.Repository)
- original.RelativePath = strings.TrimPrefix(repo.RelativePath, txn.snapshotBaseRelativePath+string(os.PathSeparator))
+ original.RelativePath = strings.TrimPrefix(repo.RelativePath, txn.snapshot.prefix+string(os.PathSeparator))
original.GitObjectDirectory = ""
original.GitAlternateObjectDirectories = nil
return original
}
-// createRepositorySnapshot snapshots the repository's current state at snapshotPath. This is done by
-// recreating the repository's directory structure and hard linking the repository's files in their
-// correct locations there. This effectively does a copy-free clone of the repository. Since the files
-// are shared between the snapshot and the repository, they must not be modified. Git doesn't modify
-// existing files but writes new ones so this property is upheld.
-func (mgr *TransactionManager) createRepositorySnapshot(ctx context.Context, repositoryPath, snapshotPath string) error {
- // This creates the parent directory hierarchy regardless of whether the repository exists or not. It also
- // doesn't consider the permissions in the storage. While not 100% correct, we have no logic that cares about
- // the storage hierarchy above repositories.
- //
- // The repository's directory itself is not yet created as whether it should be created depends on whether the
- // repository exists or not.
- if err := os.MkdirAll(filepath.Dir(snapshotPath), perm.PrivateDir); err != nil {
- return fmt.Errorf("create parent directory hierarchy: %w", err)
- }
-
- mgr.stateLock.RLock()
- defer mgr.stateLock.RUnlock()
- if err := createDirectorySnapshot(ctx, repositoryPath, snapshotPath, map[string]struct{}{
- // Don't include worktrees in the snapshot. All of the worktrees in the repository should be leftover
- // state from before transaction management was introduced as the transactions would create their
- // worktrees in the snapshot.
- housekeeping.WorktreesPrefix: {},
- housekeeping.GitlabWorktreePrefix: {},
- }); err != nil {
- return fmt.Errorf("create directory snapshot: %w", err)
- }
-
- return nil
-}
-
-// createDirectorySnapshot recursively recreates the directory structure from originalDirectory into
-// snapshotDirectory and hard links files into the same locations in snapshotDirectory.
-//
-// skipRelativePaths can be provided to skip certain entries from being included in the snapshot.
-func createDirectorySnapshot(ctx context.Context, originalDirectory, snapshotDirectory string, skipRelativePaths map[string]struct{}) error {
- if err := filepath.Walk(originalDirectory, func(oldPath string, info fs.FileInfo, err error) error {
- if err != nil {
- if errors.Is(err, fs.ErrNotExist) && oldPath == originalDirectory {
- // The directory being snapshotted does not exist. This is fine as the transaction
- // may be about to create it.
- return nil
- }
-
- return err
- }
-
- relativePath, err := filepath.Rel(originalDirectory, oldPath)
- if err != nil {
- return fmt.Errorf("rel: %w", err)
- }
-
- if _, ok := skipRelativePaths[relativePath]; ok {
- return fs.SkipDir
- }
-
- newPath := filepath.Join(snapshotDirectory, relativePath)
- if info.IsDir() {
- if err := os.Mkdir(newPath, info.Mode().Perm()); err != nil {
- return fmt.Errorf("create dir: %w", err)
- }
- } else if info.Mode().IsRegular() {
- if err := os.Link(oldPath, newPath); err != nil {
- return fmt.Errorf("link file: %w", err)
- }
- } else {
- return fmt.Errorf("unsupported file mode: %q", info.Mode())
- }
-
- return nil
- }); err != nil {
- return fmt.Errorf("walk: %w", err)
- }
-
- return nil
-}
-
func (txn *Transaction) updateState(newState transactionState) error {
switch txn.state {
case transactionStateOpen:
@@ -914,9 +815,9 @@ func (mgr *TransactionManager) stageRepositoryCreation(ctx context.Context, tran
// Repository had no alternate.
} else {
alternateObjectsDir, err := filepath.Rel(
- mgr.getAbsolutePath(transaction.snapshotBaseRelativePath),
+ mgr.getAbsolutePath(transaction.snapshot.prefix),
mgr.getAbsolutePath(
- filepath.Join(transaction.snapshotBaseRelativePath, transaction.relativePath, "objects", alternate),
+ filepath.Join(transaction.snapshot.prefix, transaction.relativePath, "objects", alternate),
),
)
if err != nil {
diff --git a/internal/gitaly/storage/storagemgr/transaction_manager_test.go b/internal/gitaly/storage/storagemgr/transaction_manager_test.go
index 214ff8ef4..5ddaf9f95 100644
--- a/internal/gitaly/storage/storagemgr/transaction_manager_test.go
+++ b/internal/gitaly/storage/storagemgr/transaction_manager_test.go
@@ -317,6 +317,9 @@ func TestTransactionManager(t *testing.T) {
TransactionID int
// RelativePath is the relative path of the repository this transaction is operating on.
RelativePath string
+ // SnapshottedRelativePaths are the relative paths of the repositories to include in the snapshot
+ // in addition to the target repository.
+ SnapshottedRelativePaths []string
// ReadOnly indicates whether this is a read-only transaction.
ReadOnly bool
// Context is the context to use for the Begin call.
@@ -5735,6 +5738,469 @@ func TestTransactionManager(t *testing.T) {
},
},
},
+ {
+ desc: "multiple repositories can be included in transaction's snapshot",
+ steps: steps{
+ RemoveRepository{},
+ StartManager{},
+ Begin{
+ TransactionID: 1,
+ RelativePath: "repository-1",
+ },
+ CreateRepository{
+ TransactionID: 1,
+ References: map[git.ReferenceName]git.ObjectID{
+ "refs/heads/main": setup.Commits.First.OID,
+ },
+ Packs: [][]byte{setup.Commits.First.Pack},
+ CustomHooks: validCustomHooks(t),
+ },
+ Commit{
+ TransactionID: 1,
+ },
+ Begin{
+ TransactionID: 2,
+ RelativePath: "repository-2",
+ ExpectedSnapshotLSN: 1,
+ },
+ CreateRepository{
+ TransactionID: 2,
+ References: map[git.ReferenceName]git.ObjectID{
+ "refs/heads/branch": setup.Commits.Third.OID,
+ },
+ DefaultBranch: "refs/heads/branch",
+ Packs: [][]byte{
+ setup.Commits.First.Pack,
+ setup.Commits.Second.Pack,
+ setup.Commits.Third.Pack,
+ },
+ },
+ Commit{
+ TransactionID: 2,
+ },
+ Begin{
+ TransactionID: 3,
+ RelativePath: "repository-3",
+ ExpectedSnapshotLSN: 2,
+ },
+ CreateRepository{
+ TransactionID: 3,
+ // Set repository-2 as repository-3's alternate to assert the
+ // snasphotted repositories' alternates are also included.
+ Alternate: "../../repository-2/objects",
+ },
+ Commit{
+ TransactionID: 3,
+ },
+ Begin{
+ TransactionID: 4,
+ // Create a repository that is not snapshotted to assert it's not included
+ // in the snapshot.
+ RelativePath: "repository-4",
+ ExpectedSnapshotLSN: 3,
+ },
+ CreateRepository{
+ TransactionID: 4,
+ },
+ Commit{
+ TransactionID: 4,
+ },
+ Begin{
+ TransactionID: 5,
+ RelativePath: "repository-1",
+ SnapshottedRelativePaths: []string{"repository-3"},
+ ExpectedSnapshotLSN: 4,
+ },
+ RepositoryAssertion{
+ TransactionID: 5,
+ Repositories: RepositoryStates{
+ "repository-1": {
+ DefaultBranch: "refs/heads/main",
+ References: []git.Reference{
+ {Name: "refs/heads/main", Target: setup.Commits.First.OID.String()},
+ },
+ Objects: []git.ObjectID{
+ setup.ObjectHash.EmptyTreeOID,
+ setup.Commits.First.OID,
+ },
+ CustomHooks: testhelper.DirectoryState{
+ "/": {Mode: umask.Mask(fs.ModeDir | perm.PrivateDir)},
+ "/pre-receive": {
+ Mode: umask.Mask(fs.ModePerm),
+ Content: []byte("hook content"),
+ },
+ "/private-dir": {Mode: umask.Mask(fs.ModeDir | perm.PrivateDir)},
+ "/private-dir/private-file": {Mode: umask.Mask(perm.PrivateFile), Content: []byte("private content")},
+ },
+ },
+ "repository-2": {
+ DefaultBranch: "refs/heads/branch",
+ References: []git.Reference{
+ {Name: "refs/heads/branch", Target: setup.Commits.Third.OID.String()},
+ },
+ Objects: []git.ObjectID{
+ setup.ObjectHash.EmptyTreeOID,
+ setup.Commits.First.OID,
+ setup.Commits.Second.OID,
+ setup.Commits.Third.OID,
+ },
+ },
+ "repository-3": {
+ DefaultBranch: "refs/heads/main",
+ Objects: []git.ObjectID{
+ setup.ObjectHash.EmptyTreeOID,
+ setup.Commits.First.OID,
+ setup.Commits.Second.OID,
+ setup.Commits.Third.OID,
+ },
+ Alternate: "../../repository-2/objects",
+ },
+ },
+ },
+ Rollback{
+ TransactionID: 5,
+ },
+ },
+ expectedState: StateAssertion{
+ Database: DatabaseState{
+ string(keyAppliedLSN(partitionID)): LSN(4).toProto(),
+ },
+ Repositories: RepositoryStates{
+ "repository-1": {
+ References: []git.Reference{
+ {Name: "refs/heads/main", Target: setup.Commits.First.OID.String()},
+ },
+ Objects: []git.ObjectID{
+ setup.ObjectHash.EmptyTreeOID,
+ setup.Commits.First.OID,
+ },
+ CustomHooks: testhelper.DirectoryState{
+ "/": {Mode: umask.Mask(fs.ModeDir | perm.PrivateDir)},
+ "/pre-receive": {
+ Mode: umask.Mask(fs.ModePerm),
+ Content: []byte("hook content"),
+ },
+ "/private-dir": {Mode: umask.Mask(fs.ModeDir | perm.PrivateDir)},
+ "/private-dir/private-file": {Mode: umask.Mask(perm.PrivateFile), Content: []byte("private content")},
+ },
+ },
+ "repository-2": {
+ DefaultBranch: "refs/heads/branch",
+ References: []git.Reference{
+ {Name: "refs/heads/branch", Target: setup.Commits.Third.OID.String()},
+ },
+ Objects: []git.ObjectID{
+ setup.ObjectHash.EmptyTreeOID,
+ setup.Commits.First.OID,
+ setup.Commits.Second.OID,
+ setup.Commits.Third.OID,
+ },
+ },
+ "repository-3": {
+ Objects: []git.ObjectID{
+ setup.ObjectHash.EmptyTreeOID,
+ setup.Commits.First.OID,
+ setup.Commits.Second.OID,
+ setup.Commits.Third.OID,
+ },
+ Alternate: "../../repository-2/objects",
+ },
+ "repository-4": {
+ Objects: []git.ObjectID{},
+ },
+ },
+ Directory: testhelper.DirectoryState{
+ "/": {Mode: fs.ModeDir | perm.PrivateDir},
+ "/wal": {Mode: fs.ModeDir | perm.PrivateDir},
+ "/wal/1": {Mode: fs.ModeDir | perm.PrivateDir},
+ "/wal/1/objects.idx": indexFileDirectoryEntry(setup.Config),
+ "/wal/1/objects.pack": packFileDirectoryEntry(
+ setup.Config,
+ []git.ObjectID{
+ setup.ObjectHash.EmptyTreeOID,
+ setup.Commits.First.OID,
+ },
+ ),
+ "/wal/1/objects.rev": reverseIndexFileDirectoryEntry(setup.Config),
+ "/wal/2": {Mode: fs.ModeDir | perm.PrivateDir},
+ "/wal/2/objects.idx": indexFileDirectoryEntry(setup.Config),
+ "/wal/2/objects.pack": packFileDirectoryEntry(
+ setup.Config,
+ []git.ObjectID{
+ setup.ObjectHash.EmptyTreeOID,
+ setup.Commits.First.OID,
+ setup.Commits.Second.OID,
+ setup.Commits.Third.OID,
+ },
+ ),
+ "/wal/2/objects.rev": reverseIndexFileDirectoryEntry(setup.Config),
+ },
+ },
+ },
+ {
+ desc: "additional repository is included in the snapshot explicitly and implicitly",
+ steps: steps{
+ RemoveRepository{},
+ StartManager{},
+ Begin{
+ TransactionID: 1,
+ RelativePath: "pool",
+ },
+ CreateRepository{
+ TransactionID: 1,
+ References: map[git.ReferenceName]git.ObjectID{
+ "refs/heads/main": setup.Commits.First.OID,
+ },
+ Packs: [][]byte{setup.Commits.First.Pack},
+ },
+ Commit{
+ TransactionID: 1,
+ },
+ Begin{
+ TransactionID: 2,
+ RelativePath: "member",
+ ExpectedSnapshotLSN: 1,
+ },
+ CreateRepository{
+ TransactionID: 2,
+ References: map[git.ReferenceName]git.ObjectID{
+ "refs/heads/branch": setup.Commits.Second.OID,
+ },
+ DefaultBranch: "refs/heads/branch",
+ Packs: [][]byte{
+ setup.Commits.First.Pack,
+ setup.Commits.Second.Pack,
+ },
+ Alternate: "../../pool/objects",
+ },
+ Commit{
+ TransactionID: 2,
+ },
+ Begin{
+ TransactionID: 3,
+ RelativePath: "member",
+ // The pool is included explicitly here, and also implicitly through
+ // the alternate link of member.
+ SnapshottedRelativePaths: []string{"pool"},
+ ExpectedSnapshotLSN: 2,
+ },
+ RepositoryAssertion{
+ TransactionID: 3,
+ Repositories: RepositoryStates{
+ "pool": {
+ DefaultBranch: "refs/heads/main",
+ References: []git.Reference{
+ {Name: "refs/heads/main", Target: setup.Commits.First.OID.String()},
+ },
+ Objects: []git.ObjectID{
+ setup.ObjectHash.EmptyTreeOID,
+ setup.Commits.First.OID,
+ },
+ },
+ "member": {
+ DefaultBranch: "refs/heads/branch",
+ References: []git.Reference{
+ {Name: "refs/heads/branch", Target: setup.Commits.Second.OID.String()},
+ },
+ Objects: []git.ObjectID{
+ setup.ObjectHash.EmptyTreeOID,
+ setup.Commits.First.OID,
+ setup.Commits.Second.OID,
+ },
+ Alternate: "../../pool/objects",
+ },
+ },
+ },
+ Rollback{
+ TransactionID: 3,
+ },
+ },
+ expectedState: StateAssertion{
+ Database: DatabaseState{
+ string(keyAppliedLSN(partitionID)): LSN(2).toProto(),
+ },
+ Repositories: RepositoryStates{
+ "pool": {
+ References: []git.Reference{
+ {Name: "refs/heads/main", Target: setup.Commits.First.OID.String()},
+ },
+ Objects: []git.ObjectID{
+ setup.ObjectHash.EmptyTreeOID,
+ setup.Commits.First.OID,
+ },
+ },
+ "member": {
+ DefaultBranch: "refs/heads/branch",
+ References: []git.Reference{
+ {Name: "refs/heads/branch", Target: setup.Commits.Second.OID.String()},
+ },
+ Objects: []git.ObjectID{
+ setup.ObjectHash.EmptyTreeOID,
+ setup.Commits.First.OID,
+ setup.Commits.Second.OID,
+ },
+ Alternate: "../../pool/objects",
+ },
+ },
+ Directory: testhelper.DirectoryState{
+ "/": {Mode: fs.ModeDir | perm.PrivateDir},
+ "/wal": {Mode: fs.ModeDir | perm.PrivateDir},
+ "/wal/1": {Mode: fs.ModeDir | perm.PrivateDir},
+ "/wal/1/objects.idx": indexFileDirectoryEntry(setup.Config),
+ "/wal/1/objects.pack": packFileDirectoryEntry(
+ setup.Config,
+ []git.ObjectID{
+ setup.ObjectHash.EmptyTreeOID,
+ setup.Commits.First.OID,
+ },
+ ),
+ "/wal/1/objects.rev": reverseIndexFileDirectoryEntry(setup.Config),
+ "/wal/2": {Mode: fs.ModeDir | perm.PrivateDir},
+ "/wal/2/objects.idx": indexFileDirectoryEntry(setup.Config),
+ "/wal/2/objects.pack": packFileDirectoryEntry(
+ setup.Config,
+ []git.ObjectID{
+ setup.ObjectHash.EmptyTreeOID,
+ setup.Commits.First.OID,
+ setup.Commits.Second.OID,
+ },
+ ),
+ "/wal/2/objects.rev": reverseIndexFileDirectoryEntry(setup.Config),
+ },
+ },
+ },
+ {
+ desc: "target repository is included in the snapshot explicitly and implicitly",
+ steps: steps{
+ RemoveRepository{},
+ StartManager{},
+ Begin{
+ TransactionID: 1,
+ RelativePath: "pool",
+ },
+ CreateRepository{
+ TransactionID: 1,
+ References: map[git.ReferenceName]git.ObjectID{
+ "refs/heads/main": setup.Commits.First.OID,
+ },
+ Packs: [][]byte{setup.Commits.First.Pack},
+ },
+ Commit{
+ TransactionID: 1,
+ },
+ Begin{
+ TransactionID: 2,
+ RelativePath: "member",
+ ExpectedSnapshotLSN: 1,
+ },
+ CreateRepository{
+ TransactionID: 2,
+ References: map[git.ReferenceName]git.ObjectID{
+ "refs/heads/branch": setup.Commits.Second.OID,
+ },
+ DefaultBranch: "refs/heads/branch",
+ Packs: [][]byte{
+ setup.Commits.First.Pack,
+ setup.Commits.Second.Pack,
+ },
+ Alternate: "../../pool/objects",
+ },
+ Commit{
+ TransactionID: 2,
+ },
+ Begin{
+ TransactionID: 3,
+ // The pool is targeted, and also implicitly included through
+ // the alternate link of member.
+ RelativePath: "pool",
+ SnapshottedRelativePaths: []string{"member"},
+ ExpectedSnapshotLSN: 2,
+ },
+ RepositoryAssertion{
+ TransactionID: 3,
+ Repositories: RepositoryStates{
+ "pool": {
+ DefaultBranch: "refs/heads/main",
+ References: []git.Reference{
+ {Name: "refs/heads/main", Target: setup.Commits.First.OID.String()},
+ },
+ Objects: []git.ObjectID{
+ setup.ObjectHash.EmptyTreeOID,
+ setup.Commits.First.OID,
+ },
+ },
+ "member": {
+ DefaultBranch: "refs/heads/branch",
+ References: []git.Reference{
+ {Name: "refs/heads/branch", Target: setup.Commits.Second.OID.String()},
+ },
+ Objects: []git.ObjectID{
+ setup.ObjectHash.EmptyTreeOID,
+ setup.Commits.First.OID,
+ setup.Commits.Second.OID,
+ },
+ Alternate: "../../pool/objects",
+ },
+ },
+ },
+ Rollback{
+ TransactionID: 3,
+ },
+ },
+ expectedState: StateAssertion{
+ Database: DatabaseState{
+ string(keyAppliedLSN(partitionID)): LSN(2).toProto(),
+ },
+ Repositories: RepositoryStates{
+ "pool": {
+ References: []git.Reference{
+ {Name: "refs/heads/main", Target: setup.Commits.First.OID.String()},
+ },
+ Objects: []git.ObjectID{
+ setup.ObjectHash.EmptyTreeOID,
+ setup.Commits.First.OID,
+ },
+ },
+ "member": {
+ DefaultBranch: "refs/heads/branch",
+ References: []git.Reference{
+ {Name: "refs/heads/branch", Target: setup.Commits.Second.OID.String()},
+ },
+ Objects: []git.ObjectID{
+ setup.ObjectHash.EmptyTreeOID,
+ setup.Commits.First.OID,
+ setup.Commits.Second.OID,
+ },
+ Alternate: "../../pool/objects",
+ },
+ },
+ Directory: testhelper.DirectoryState{
+ "/": {Mode: fs.ModeDir | perm.PrivateDir},
+ "/wal": {Mode: fs.ModeDir | perm.PrivateDir},
+ "/wal/1": {Mode: fs.ModeDir | perm.PrivateDir},
+ "/wal/1/objects.idx": indexFileDirectoryEntry(setup.Config),
+ "/wal/1/objects.pack": packFileDirectoryEntry(
+ setup.Config,
+ []git.ObjectID{
+ setup.ObjectHash.EmptyTreeOID,
+ setup.Commits.First.OID,
+ },
+ ),
+ "/wal/1/objects.rev": reverseIndexFileDirectoryEntry(setup.Config),
+ "/wal/2": {Mode: fs.ModeDir | perm.PrivateDir},
+ "/wal/2/objects.idx": indexFileDirectoryEntry(setup.Config),
+ "/wal/2/objects.pack": packFileDirectoryEntry(
+ setup.Config,
+ []git.ObjectID{
+ setup.ObjectHash.EmptyTreeOID,
+ setup.Commits.First.OID,
+ setup.Commits.Second.OID,
+ },
+ ),
+ "/wal/2/objects.rev": reverseIndexFileDirectoryEntry(setup.Config),
+ },
+ },
+ },
}
type invalidReferenceTestCase struct {
@@ -5939,7 +6405,7 @@ func TestTransactionManager(t *testing.T) {
beginCtx = step.Context
}
- transaction, err := transactionManager.Begin(beginCtx, step.RelativePath, step.ReadOnly)
+ transaction, err := transactionManager.Begin(beginCtx, step.RelativePath, step.SnapshottedRelativePaths, step.ReadOnly)
require.Equal(t, step.ExpectedError, err)
if err == nil {
require.Equal(t, step.ExpectedSnapshotLSN, transaction.SnapshotLSN())
@@ -6102,7 +6568,7 @@ func TestTransactionManager(t *testing.T) {
RequireRepositories(t, ctx, setup.Config,
// Assert the contents of the transaction's snapshot.
- filepath.Join(setup.Config.Storages[0].Path, transaction.snapshotBaseRelativePath),
+ filepath.Join(setup.Config.Storages[0].Path, transaction.snapshot.prefix),
// Rewrite all of the repositories to point to their snapshots.
func(relativePath string) *localrepo.Repo {
return setup.RepositoryFactory.Build(
@@ -6204,7 +6670,7 @@ func checkManagerError(t *testing.T, ctx context.Context, managerErrChannel chan
// Begin a transaction to wait until the manager has applied all log entries currently
// committed. This ensures the disk state assertions run with all log entries fully applied
// to the repository.
- tx, err := mgr.Begin(ctx, "non-existent", false)
+ tx, err := mgr.Begin(ctx, "non-existent", nil, false)
require.NoError(t, err)
require.NoError(t, tx.Rollback())
@@ -6352,7 +6818,7 @@ func BenchmarkTransactionManager(b *testing.B) {
require.NoError(b, err)
for j := 0; j < tc.concurrentUpdaters; j++ {
- transaction, err := manager.Begin(ctx, repo.RelativePath, false)
+ transaction, err := manager.Begin(ctx, repo.RelativePath, nil, false)
require.NoError(b, err)
transaction.UpdateReferences(getReferenceUpdates(j, objectHash.ZeroOID, commit1))
require.NoError(b, transaction.Commit(ctx))
@@ -6372,7 +6838,7 @@ func BenchmarkTransactionManager(b *testing.B) {
currentReferences := getReferenceUpdates(i, commit1, commit2)
nextReferences := getReferenceUpdates(i, commit2, commit1)
- transaction, err := manager.Begin(ctx, relativePath, false)
+ transaction, err := manager.Begin(ctx, relativePath, nil, false)
require.NoError(b, err)
transaction.UpdateReferences(currentReferences)
@@ -6384,7 +6850,7 @@ func BenchmarkTransactionManager(b *testing.B) {
defer transactionWG.Done()
for range transactionChan {
- transaction, err := manager.Begin(ctx, relativePath, false)
+ transaction, err := manager.Begin(ctx, relativePath, nil, false)
require.NoError(b, err)
transaction.UpdateReferences(nextReferences)
assert.NoError(b, transaction.Commit(ctx))