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:
authorChristian Couder <chriscool@tuxfamily.org>2021-03-09 19:11:50 +0300
committerChristian Couder <chriscool@tuxfamily.org>2021-03-09 19:11:50 +0300
commitd8881723dd308150dced165769fb70e224019854 (patch)
tree1c5a26c36cd3994f381b7b9d5382a21888c7c1a3
parent58bf16b78b3c99757a2f283a5befe57a2cb7f009 (diff)
parent80056dad431eea918fd836a74589441a5e69e89b (diff)
Merge branch 'pks-git-smallish-refactorings' into 'master'
git: Random set of small refactorings See merge request gitlab-org/gitaly!3229
-rw-r--r--internal/git/catfile/batch.go321
-rw-r--r--internal/git/catfile/batch_check_process.go (renamed from internal/git/catfile/batchcheck.go)10
-rw-r--r--internal/git/catfile/batch_process.go152
-rw-r--r--internal/git/catfile/batch_test.go (renamed from internal/git/catfile/catfile_test.go)0
-rw-r--r--internal/git/catfile/catfile.go272
-rw-r--r--internal/git/catfile/objectinfo.go11
-rw-r--r--internal/git/catfile/objectinfo_test.go (renamed from internal/git/catfile/parse_test.go)0
-rw-r--r--internal/git/localrepo/objects.go30
-rw-r--r--internal/git/localrepo/objects_test.go14
-rw-r--r--internal/git/log/commit.go2
-rw-r--r--internal/git/log/commit_test.go14
-rw-r--r--internal/git2go/apply_test.go12
-rw-r--r--internal/git2go/commit_test.go52
-rw-r--r--internal/gitaly/service/blob/get_blob.go2
-rw-r--r--internal/gitaly/service/blob/lfs_pointers.go2
-rw-r--r--internal/gitaly/service/commit/tree_entries_helper.go4
-rw-r--r--internal/gitaly/service/commit/tree_entry.go2
-rw-r--r--internal/gitaly/service/conflicts/resolve_conflicts_test.go2
-rw-r--r--internal/gitaly/service/operations/commit_files.go8
-rw-r--r--internal/gitaly/service/operations/tags.go6
-rw-r--r--internal/gitaly/service/ref/list_new_blobs.go6
-rw-r--r--internal/gitaly/service/repository/apply_gitattributes.go7
-rw-r--r--internal/gitaly/service/repository/raw_changes.go2
23 files changed, 477 insertions, 454 deletions
diff --git a/internal/git/catfile/batch.go b/internal/git/catfile/batch.go
index cdb7b9e02..a59f74973 100644
--- a/internal/git/catfile/batch.go
+++ b/internal/git/catfile/batch.go
@@ -1,152 +1,273 @@
package catfile
import (
- "bufio"
"context"
- "fmt"
- "io"
- "io/ioutil"
"sync"
"github.com/opentracing/opentracing-go"
+ "github.com/prometheus/client_golang/prometheus"
"gitlab.com/gitlab-org/gitaly/internal/git"
"gitlab.com/gitlab-org/gitaly/internal/git/repository"
- "gitlab.com/gitlab-org/labkit/correlation"
+ "gitlab.com/gitlab-org/gitaly/internal/metadata"
)
-// batch encapsulates a 'git cat-file --batch' process
-type batchProcess struct {
- r *bufio.Reader
- w io.WriteCloser
-
- // n is a state machine that tracks how much data we still have to read
- // from r. Legal states are: n==0, this means we can do a new request on
- // the cat-file process. n==1, this means that we have to discard a
- // trailing newline. n>0, this means we are in the middle of reading a
- // raw git object.
- n int64
-
- // Even though the batch type should not be used concurrently, I think
- // that if that does happen by mistake we should give proper errors
- // instead of doing unsafe memory writes (to n) and failing in some
- // unpredictable way.
+var catfileCacheCounter = prometheus.NewCounterVec(
+ prometheus.CounterOpts{
+ Name: "gitaly_catfile_cache_total",
+ Help: "Counter of catfile cache hit/miss",
+ },
+ []string{"type"},
+)
+
+var currentCatfileProcesses = prometheus.NewGauge(
+ prometheus.GaugeOpts{
+ Name: "gitaly_catfile_processes",
+ Help: "Gauge of active catfile processes",
+ },
+)
+
+var totalCatfileProcesses = prometheus.NewCounter(
+ prometheus.CounterOpts{
+ Name: "gitaly_catfile_processes_total",
+ Help: "Counter of catfile processes",
+ },
+)
+
+var catfileLookupCounter = prometheus.NewCounterVec(
+ prometheus.CounterOpts{
+ Name: "gitaly_catfile_lookups_total",
+ Help: "Git catfile lookups by object type",
+ },
+ []string{"type"},
+)
+
+const (
+ // SessionIDField is the gRPC metadata field we use to store the gitaly session ID.
+ SessionIDField = "gitaly-session-id"
+)
+
+func init() {
+ prometheus.MustRegister(catfileCacheCounter)
+ prometheus.MustRegister(currentCatfileProcesses)
+ prometheus.MustRegister(totalCatfileProcesses)
+ prometheus.MustRegister(catfileLookupCounter)
+}
+
+// Batch abstracts 'git cat-file --batch' and 'git cat-file --batch-check'.
+// It lets you retrieve object metadata and raw objects from a Git repo.
+//
+// A Batch instance can only serve single request at a time. If you want to
+// use it across multiple goroutines you need to add your own locking.
+type Batch interface {
+ Info(ctx context.Context, revision git.Revision) (*ObjectInfo, error)
+ Tree(ctx context.Context, revision git.Revision) (*Object, error)
+ Commit(ctx context.Context, revision git.Revision) (*Object, error)
+ Blob(ctx context.Context, revision git.Revision) (*Object, error)
+ Tag(ctx context.Context, revision git.Revision) (*Object, error)
+}
+
+type batch struct {
sync.Mutex
+ *batchCheckProcess
+ *batchProcess
+ cancel func()
+ closed bool
}
-func newBatchProcess(ctx context.Context, gitCmdFactory git.CommandFactory, repo repository.GitRepo) (*batchProcess, error) {
- totalCatfileProcesses.Inc()
- b := &batchProcess{}
+// Info returns an ObjectInfo if spec exists. If the revision does not exist
+// the error is of type NotFoundError.
+func (c *batch) Info(ctx context.Context, revision git.Revision) (*ObjectInfo, error) {
+ return c.batchCheckProcess.info(revision)
+}
- var stdinReader io.Reader
- stdinReader, b.w = io.Pipe()
+// Tree returns a raw tree object. It is an error if the revision does not
+// point to a tree. To prevent this, use Info to resolve the revision and check
+// the object type. Caller must consume the Reader before making another call
+// on C.
+func (c *batch) Tree(ctx context.Context, revision git.Revision) (*Object, error) {
+ return c.batchProcess.reader(revision, "tree")
+}
- // batch processes are long-lived and reused across RPCs,
- // so we de-correlate the process from the RPC
- ctx = correlation.ContextWithCorrelation(ctx, "")
- ctx = opentracing.ContextWithSpan(ctx, nil)
+// Commit returns a raw commit object. It is an error if the revision does not
+// point to a commit. To prevent this, use Info to resolve the revision and
+// check the object type. Caller must consume the Reader before making another
+// call on C.
+func (c *batch) Commit(ctx context.Context, revision git.Revision) (*Object, error) {
+ return c.batchProcess.reader(revision, "commit")
+}
- batchCmd, err := gitCmdFactory.New(ctx, repo, nil,
- git.SubCmd{
- Name: "cat-file",
- Flags: []git.Option{
- git.Flag{Name: "--batch"},
- },
- },
- git.WithStdin(stdinReader),
- )
- if err != nil {
- return nil, err
- }
+// Blob returns a reader for the requested blob. The entire blob must be
+// read before any new objects can be requested from this Batch instance.
+//
+// It is an error if the revision does not point to a blob. To prevent this,
+// use Info to resolve the revision and check the object type.
+func (c *batch) Blob(ctx context.Context, revision git.Revision) (*Object, error) {
+ return c.batchProcess.reader(revision, "blob")
+}
+
+// Tag returns a raw tag object. Caller must consume the Reader before
+// making another call on C.
+func (c *batch) Tag(ctx context.Context, revision git.Revision) (*Object, error) {
+ return c.batchProcess.reader(revision, "tag")
+}
- b.r = bufio.NewReader(batchCmd)
+// Close closes the writers for batchCheckProcess and batchProcess. This is only used for cached
+// Batches
+func (c *batch) Close() {
+ c.Lock()
+ defer c.Unlock()
- currentCatfileProcesses.Inc()
- go func() {
- <-ctx.Done()
- // This Close() is crucial to prevent leaking file descriptors.
- b.w.Close()
- currentCatfileProcesses.Dec()
- }()
+ if c.closed {
+ return
+ }
- if injectSpawnErrors {
- // Testing only: intentionally leak process
- return nil, &simulatedBatchSpawnError{}
+ c.closed = true
+ if c.cancel != nil {
+ // both c.batchProcess and c.batchCheckProcess have goroutines that listen on
+ // ctx.Done() when this is cancelled, it will cause those goroutines to close both
+ // writers
+ c.cancel()
}
+}
- return b, nil
+func (c *batch) isClosed() bool {
+ c.Lock()
+ defer c.Unlock()
+ return c.closed
}
-func (b *batchProcess) reader(revision git.Revision, expectedType string) (*Object, error) {
- b.Lock()
- defer b.Unlock()
+// New returns a new Batch instance. It is important that ctx gets canceled
+// somewhere, because if it doesn't the cat-file processes spawned by
+// New() never terminate.
+func New(ctx context.Context, gitCmdFactory git.CommandFactory, repo repository.GitRepo) (Batch, error) {
+ if ctx.Done() == nil {
+ panic("empty ctx.Done() in catfile.Batch.New()")
+ }
- if b.n == 1 {
- // Consume linefeed
- if _, err := b.r.ReadByte(); err != nil {
+ sessionID := metadata.GetValue(ctx, SessionIDField)
+ if sessionID == "" {
+ c, err := newBatch(ctx, gitCmdFactory, repo)
+ if err != nil {
return nil, err
}
- b.n--
+ return newInstrumentedBatch(c), err
}
- if b.n != 0 {
- return nil, fmt.Errorf("cannot create new Object: batch contains %d unread bytes", b.n)
- }
+ cacheKey := newCacheKey(sessionID, repo)
+ requestDone := ctx.Done()
- if _, err := fmt.Fprintln(b.w, revision.String()); err != nil {
- return nil, err
+ if c, ok := cache.Checkout(cacheKey); ok {
+ go returnToCacheWhenDone(requestDone, cache, cacheKey, c)
+ return newInstrumentedBatch(c), nil
}
- oi, err := ParseObjectInfo(b.r)
+ // if we are using caching, create a fresh context for the new batch
+ // and initialize the new batch with a cache key and cancel function
+ cacheCtx, cacheCancel := context.WithCancel(context.Background())
+ c, err := newBatch(cacheCtx, gitCmdFactory, repo)
if err != nil {
+ cacheCancel()
return nil, err
}
- b.n = oi.Size + 1
+ c.cancel = cacheCancel
+ go returnToCacheWhenDone(requestDone, cache, cacheKey, c)
- if oi.Type != expectedType {
- // This is a programmer error and it should never happen. But if it does,
- // we need to leave the cat-file process in a good state
- if _, err := io.CopyN(ioutil.Discard, b.r, b.n); err != nil {
- return nil, err
- }
- b.n = 0
+ return newInstrumentedBatch(c), nil
+}
- return nil, NotFoundError{error: fmt.Errorf("expected %s to be a %s, got %s", oi.Oid, expectedType, oi.Type)}
+func returnToCacheWhenDone(done <-chan struct{}, bc *batchCache, cacheKey key, c *batch) {
+ <-done
+
+ if c == nil || c.isClosed() {
+ return
+ }
+
+ if c.hasUnreadData() {
+ catfileCacheCounter.WithLabelValues("dirty").Inc()
+ c.Close()
+ return
}
- return &Object{
- ObjectInfo: *oi,
- Reader: &batchReader{
- batchProcess: b,
- r: io.LimitReader(b.r, oi.Size),
- },
- }, nil
+ bc.Add(cacheKey, c)
}
-func (b *batchProcess) consume(nBytes int) {
- b.n -= int64(nBytes)
- if b.n < 1 {
- panic("too many bytes read from batch")
+var injectSpawnErrors = false
+
+type simulatedBatchSpawnError struct{}
+
+func (simulatedBatchSpawnError) Error() string { return "simulated spawn error" }
+
+func newBatch(ctx context.Context, gitCmdFactory git.CommandFactory, repo repository.GitRepo) (_ *batch, err error) {
+ ctx, cancel := context.WithCancel(ctx)
+ defer func() {
+ if err != nil {
+ cancel()
+ }
+ }()
+
+ batchProcess, err := newBatchProcess(ctx, gitCmdFactory, repo)
+ if err != nil {
+ return nil, err
+ }
+
+ batchCheckProcess, err := newBatchCheckProcess(ctx, gitCmdFactory, repo)
+ if err != nil {
+ return nil, err
}
+
+ return &batch{batchProcess: batchProcess, batchCheckProcess: batchCheckProcess}, nil
}
-func (b *batchProcess) hasUnreadData() bool {
- b.Lock()
- defer b.Unlock()
+func newInstrumentedBatch(c Batch) Batch {
+ return &instrumentedBatch{c}
+}
- return b.n > 1
+type instrumentedBatch struct {
+ Batch
}
-type batchReader struct {
- *batchProcess
- r io.Reader
+func (ib *instrumentedBatch) Info(ctx context.Context, revision git.Revision) (*ObjectInfo, error) {
+ span, ctx := opentracing.StartSpanFromContext(ctx, "Batch.Info", opentracing.Tag{"revision", revision})
+ defer span.Finish()
+
+ catfileLookupCounter.WithLabelValues("info").Inc()
+
+ return ib.Batch.Info(ctx, revision)
+}
+
+func (ib *instrumentedBatch) Tree(ctx context.Context, revision git.Revision) (*Object, error) {
+ span, ctx := opentracing.StartSpanFromContext(ctx, "Batch.Tree", opentracing.Tag{"revision", revision})
+ defer span.Finish()
+
+ catfileLookupCounter.WithLabelValues("tree").Inc()
+
+ return ib.Batch.Tree(ctx, revision)
+}
+
+func (ib *instrumentedBatch) Commit(ctx context.Context, revision git.Revision) (*Object, error) {
+ span, ctx := opentracing.StartSpanFromContext(ctx, "Batch.Commit", opentracing.Tag{"revision", revision})
+ defer span.Finish()
+
+ catfileLookupCounter.WithLabelValues("commit").Inc()
+
+ return ib.Batch.Commit(ctx, revision)
+}
+
+func (ib *instrumentedBatch) Blob(ctx context.Context, revision git.Revision) (*Object, error) {
+ span, ctx := opentracing.StartSpanFromContext(ctx, "Batch.Blob", opentracing.Tag{"revision", revision})
+ defer span.Finish()
+
+ catfileLookupCounter.WithLabelValues("blob").Inc()
+
+ return ib.Batch.Blob(ctx, revision)
}
-func (br *batchReader) Read(p []byte) (int, error) {
- br.batchProcess.Lock()
- defer br.batchProcess.Unlock()
+func (ib *instrumentedBatch) Tag(ctx context.Context, revision git.Revision) (*Object, error) {
+ span, ctx := opentracing.StartSpanFromContext(ctx, "Batch.Tag", opentracing.Tag{"revision", revision})
+ defer span.Finish()
+
+ catfileLookupCounter.WithLabelValues("tag").Inc()
- n, err := br.r.Read(p)
- br.batchProcess.consume(n)
- return n, err
+ return ib.Batch.Tag(ctx, revision)
}
diff --git a/internal/git/catfile/batchcheck.go b/internal/git/catfile/batch_check_process.go
index b1fa8085b..ac7b5e109 100644
--- a/internal/git/catfile/batchcheck.go
+++ b/internal/git/catfile/batch_check_process.go
@@ -13,15 +13,15 @@ import (
"gitlab.com/gitlab-org/labkit/correlation"
)
-// batchCheck encapsulates a 'git cat-file --batch-check' process
-type batchCheck struct {
+// batchCheckProcess encapsulates a 'git cat-file --batch-check' process
+type batchCheckProcess struct {
r *bufio.Reader
w io.WriteCloser
sync.Mutex
}
-func newBatchCheck(ctx context.Context, gitCmdFactory git.CommandFactory, repo repository.GitRepo) (*batchCheck, error) {
- bc := &batchCheck{}
+func newBatchCheckProcess(ctx context.Context, gitCmdFactory git.CommandFactory, repo repository.GitRepo) (*batchCheckProcess, error) {
+ bc := &batchCheckProcess{}
var stdinReader io.Reader
stdinReader, bc.w = io.Pipe()
@@ -59,7 +59,7 @@ func newBatchCheck(ctx context.Context, gitCmdFactory git.CommandFactory, repo r
return bc, nil
}
-func (bc *batchCheck) info(revision git.Revision) (*ObjectInfo, error) {
+func (bc *batchCheckProcess) info(revision git.Revision) (*ObjectInfo, error) {
bc.Lock()
defer bc.Unlock()
diff --git a/internal/git/catfile/batch_process.go b/internal/git/catfile/batch_process.go
new file mode 100644
index 000000000..cdb7b9e02
--- /dev/null
+++ b/internal/git/catfile/batch_process.go
@@ -0,0 +1,152 @@
+package catfile
+
+import (
+ "bufio"
+ "context"
+ "fmt"
+ "io"
+ "io/ioutil"
+ "sync"
+
+ "github.com/opentracing/opentracing-go"
+ "gitlab.com/gitlab-org/gitaly/internal/git"
+ "gitlab.com/gitlab-org/gitaly/internal/git/repository"
+ "gitlab.com/gitlab-org/labkit/correlation"
+)
+
+// batch encapsulates a 'git cat-file --batch' process
+type batchProcess struct {
+ r *bufio.Reader
+ w io.WriteCloser
+
+ // n is a state machine that tracks how much data we still have to read
+ // from r. Legal states are: n==0, this means we can do a new request on
+ // the cat-file process. n==1, this means that we have to discard a
+ // trailing newline. n>0, this means we are in the middle of reading a
+ // raw git object.
+ n int64
+
+ // Even though the batch type should not be used concurrently, I think
+ // that if that does happen by mistake we should give proper errors
+ // instead of doing unsafe memory writes (to n) and failing in some
+ // unpredictable way.
+ sync.Mutex
+}
+
+func newBatchProcess(ctx context.Context, gitCmdFactory git.CommandFactory, repo repository.GitRepo) (*batchProcess, error) {
+ totalCatfileProcesses.Inc()
+ b := &batchProcess{}
+
+ var stdinReader io.Reader
+ stdinReader, b.w = io.Pipe()
+
+ // batch processes are long-lived and reused across RPCs,
+ // so we de-correlate the process from the RPC
+ ctx = correlation.ContextWithCorrelation(ctx, "")
+ ctx = opentracing.ContextWithSpan(ctx, nil)
+
+ batchCmd, err := gitCmdFactory.New(ctx, repo, nil,
+ git.SubCmd{
+ Name: "cat-file",
+ Flags: []git.Option{
+ git.Flag{Name: "--batch"},
+ },
+ },
+ git.WithStdin(stdinReader),
+ )
+ if err != nil {
+ return nil, err
+ }
+
+ b.r = bufio.NewReader(batchCmd)
+
+ currentCatfileProcesses.Inc()
+ go func() {
+ <-ctx.Done()
+ // This Close() is crucial to prevent leaking file descriptors.
+ b.w.Close()
+ currentCatfileProcesses.Dec()
+ }()
+
+ if injectSpawnErrors {
+ // Testing only: intentionally leak process
+ return nil, &simulatedBatchSpawnError{}
+ }
+
+ return b, nil
+}
+
+func (b *batchProcess) reader(revision git.Revision, expectedType string) (*Object, error) {
+ b.Lock()
+ defer b.Unlock()
+
+ if b.n == 1 {
+ // Consume linefeed
+ if _, err := b.r.ReadByte(); err != nil {
+ return nil, err
+ }
+ b.n--
+ }
+
+ if b.n != 0 {
+ return nil, fmt.Errorf("cannot create new Object: batch contains %d unread bytes", b.n)
+ }
+
+ if _, err := fmt.Fprintln(b.w, revision.String()); err != nil {
+ return nil, err
+ }
+
+ oi, err := ParseObjectInfo(b.r)
+ if err != nil {
+ return nil, err
+ }
+
+ b.n = oi.Size + 1
+
+ if oi.Type != expectedType {
+ // This is a programmer error and it should never happen. But if it does,
+ // we need to leave the cat-file process in a good state
+ if _, err := io.CopyN(ioutil.Discard, b.r, b.n); err != nil {
+ return nil, err
+ }
+ b.n = 0
+
+ return nil, NotFoundError{error: fmt.Errorf("expected %s to be a %s, got %s", oi.Oid, expectedType, oi.Type)}
+ }
+
+ return &Object{
+ ObjectInfo: *oi,
+ Reader: &batchReader{
+ batchProcess: b,
+ r: io.LimitReader(b.r, oi.Size),
+ },
+ }, nil
+}
+
+func (b *batchProcess) consume(nBytes int) {
+ b.n -= int64(nBytes)
+ if b.n < 1 {
+ panic("too many bytes read from batch")
+ }
+}
+
+func (b *batchProcess) hasUnreadData() bool {
+ b.Lock()
+ defer b.Unlock()
+
+ return b.n > 1
+}
+
+type batchReader struct {
+ *batchProcess
+ r io.Reader
+}
+
+func (br *batchReader) Read(p []byte) (int, error) {
+ br.batchProcess.Lock()
+ defer br.batchProcess.Unlock()
+
+ n, err := br.r.Read(p)
+ br.batchProcess.consume(n)
+ return n, err
+}
diff --git a/internal/git/catfile/catfile_test.go b/internal/git/catfile/batch_test.go
index 448382fbe..448382fbe 100644
--- a/internal/git/catfile/catfile_test.go
+++ b/internal/git/catfile/batch_test.go
diff --git a/internal/git/catfile/catfile.go b/internal/git/catfile/catfile.go
deleted file mode 100644
index 15078ead3..000000000
--- a/internal/git/catfile/catfile.go
+++ /dev/null
@@ -1,272 +0,0 @@
-package catfile
-
-import (
- "context"
- "sync"
-
- "github.com/opentracing/opentracing-go"
- "github.com/prometheus/client_golang/prometheus"
- "gitlab.com/gitlab-org/gitaly/internal/git"
- "gitlab.com/gitlab-org/gitaly/internal/git/repository"
- "gitlab.com/gitlab-org/gitaly/internal/metadata"
-)
-
-var catfileCacheCounter = prometheus.NewCounterVec(
- prometheus.CounterOpts{
- Name: "gitaly_catfile_cache_total",
- Help: "Counter of catfile cache hit/miss",
- },
- []string{"type"},
-)
-
-var currentCatfileProcesses = prometheus.NewGauge(
- prometheus.GaugeOpts{
- Name: "gitaly_catfile_processes",
- Help: "Gauge of active catfile processes",
- },
-)
-
-var totalCatfileProcesses = prometheus.NewCounter(
- prometheus.CounterOpts{
- Name: "gitaly_catfile_processes_total",
- Help: "Counter of catfile processes",
- },
-)
-
-var catfileLookupCounter = prometheus.NewCounterVec(
- prometheus.CounterOpts{
- Name: "gitaly_catfile_lookups_total",
- Help: "Git catfile lookups by object type",
- },
- []string{"type"},
-)
-
-const (
- // SessionIDField is the gRPC metadata field we use to store the gitaly session ID.
- SessionIDField = "gitaly-session-id"
-)
-
-func init() {
- prometheus.MustRegister(catfileCacheCounter)
- prometheus.MustRegister(currentCatfileProcesses)
- prometheus.MustRegister(totalCatfileProcesses)
- prometheus.MustRegister(catfileLookupCounter)
-}
-
-// Batch abstracts 'git cat-file --batch' and 'git cat-file --batch-check'.
-// It lets you retrieve object metadata and raw objects from a Git repo.
-//
-// A Batch instance can only serve single request at a time. If you want to
-// use it across multiple goroutines you need to add your own locking.
-type Batch interface {
- Info(ctx context.Context, revision git.Revision) (*ObjectInfo, error)
- Tree(ctx context.Context, revision git.Revision) (*Object, error)
- Commit(ctx context.Context, revision git.Revision) (*Object, error)
- Blob(ctx context.Context, revision git.Revision) (*Object, error)
- Tag(ctx context.Context, revision git.Revision) (*Object, error)
-}
-
-type batch struct {
- sync.Mutex
- *batchCheck
- *batchProcess
- cancel func()
- closed bool
-}
-
-// Info returns an ObjectInfo if spec exists. If the revision does not exist
-// the error is of type NotFoundError.
-func (c *batch) Info(ctx context.Context, revision git.Revision) (*ObjectInfo, error) {
- return c.batchCheck.info(revision)
-}
-
-// Tree returns a raw tree object. It is an error if the revision does not
-// point to a tree. To prevent this, use Info to resolve the revision and check
-// the object type. Caller must consume the Reader before making another call
-// on C.
-func (c *batch) Tree(ctx context.Context, revision git.Revision) (*Object, error) {
- return c.batchProcess.reader(revision, "tree")
-}
-
-// Commit returns a raw commit object. It is an error if the revision does not
-// point to a commit. To prevent this, use Info to resolve the revision and
-// check the object type. Caller must consume the Reader before making another
-// call on C.
-func (c *batch) Commit(ctx context.Context, revision git.Revision) (*Object, error) {
- return c.batchProcess.reader(revision, "commit")
-}
-
-// Blob returns a reader for the requested blob. The entire blob must be
-// read before any new objects can be requested from this Batch instance.
-//
-// It is an error if the revision does not point to a blob. To prevent this,
-// use Info to resolve the revision and check the object type.
-func (c *batch) Blob(ctx context.Context, revision git.Revision) (*Object, error) {
- return c.batchProcess.reader(revision, "blob")
-}
-
-// Tag returns a raw tag object. Caller must consume the Reader before
-// making another call on C.
-func (c *batch) Tag(ctx context.Context, revision git.Revision) (*Object, error) {
- return c.batchProcess.reader(revision, "tag")
-}
-
-// Close closes the writers for batchCheck and batch. This is only used for
-// cached Batches
-func (c *batch) Close() {
- c.Lock()
- defer c.Unlock()
-
- if c.closed {
- return
- }
-
- c.closed = true
- if c.cancel != nil {
- // both c.batch and c.batchCheck have goroutines that listen on <ctx.Done()
- // when this is cancelled, it will cause those goroutines to close both writers
- c.cancel()
- }
-}
-
-func (c *batch) isClosed() bool {
- c.Lock()
- defer c.Unlock()
- return c.closed
-}
-
-// New returns a new Batch instance. It is important that ctx gets canceled
-// somewhere, because if it doesn't the cat-file processes spawned by
-// New() never terminate.
-func New(ctx context.Context, gitCmdFactory git.CommandFactory, repo repository.GitRepo) (Batch, error) {
- if ctx.Done() == nil {
- panic("empty ctx.Done() in catfile.Batch.New()")
- }
-
- sessionID := metadata.GetValue(ctx, SessionIDField)
- if sessionID == "" {
- c, err := newBatch(ctx, gitCmdFactory, repo)
- if err != nil {
- return nil, err
- }
- return newInstrumentedBatch(c), err
- }
-
- cacheKey := newCacheKey(sessionID, repo)
- requestDone := ctx.Done()
-
- if c, ok := cache.Checkout(cacheKey); ok {
- go returnWhenDone(requestDone, cache, cacheKey, c)
- return newInstrumentedBatch(c), nil
- }
-
- // if we are using caching, create a fresh context for the new batch
- // and initialize the new batch with a cache key and cancel function
- cacheCtx, cacheCancel := context.WithCancel(context.Background())
- c, err := newBatch(cacheCtx, gitCmdFactory, repo)
- if err != nil {
- cacheCancel()
- return nil, err
- }
-
- c.cancel = cacheCancel
- go returnWhenDone(requestDone, cache, cacheKey, c)
-
- return newInstrumentedBatch(c), nil
-}
-
-func returnWhenDone(done <-chan struct{}, bc *batchCache, cacheKey key, c *batch) {
- <-done
-
- if c == nil || c.isClosed() {
- return
- }
-
- if c.hasUnreadData() {
- catfileCacheCounter.WithLabelValues("dirty").Inc()
- c.Close()
- return
- }
-
- bc.Add(cacheKey, c)
-}
-
-var injectSpawnErrors = false
-
-type simulatedBatchSpawnError struct{}
-
-func (simulatedBatchSpawnError) Error() string { return "simulated spawn error" }
-
-func newBatch(ctx context.Context, gitCmdFactory git.CommandFactory, repo repository.GitRepo) (_ *batch, err error) {
- ctx, cancel := context.WithCancel(ctx)
- defer func() {
- if err != nil {
- cancel()
- }
- }()
-
- b, err := newBatchProcess(ctx, gitCmdFactory, repo)
- if err != nil {
- return nil, err
- }
-
- batchCheck, err := newBatchCheck(ctx, gitCmdFactory, repo)
- if err != nil {
- return nil, err
- }
-
- return &batch{batchProcess: b, batchCheck: batchCheck}, nil
-}
-
-func newInstrumentedBatch(c Batch) Batch {
- return &instrumentedBatch{c}
-}
-
-type instrumentedBatch struct {
- Batch
-}
-
-func (ib *instrumentedBatch) Info(ctx context.Context, revision git.Revision) (*ObjectInfo, error) {
- span, ctx := opentracing.StartSpanFromContext(ctx, "Batch.Info", opentracing.Tag{"revision", revision})
- defer span.Finish()
-
- catfileLookupCounter.WithLabelValues("info").Inc()
-
- return ib.Batch.Info(ctx, revision)
-}
-
-func (ib *instrumentedBatch) Tree(ctx context.Context, revision git.Revision) (*Object, error) {
- span, ctx := opentracing.StartSpanFromContext(ctx, "Batch.Tree", opentracing.Tag{"revision", revision})
- defer span.Finish()
-
- catfileLookupCounter.WithLabelValues("tree").Inc()
-
- return ib.Batch.Tree(ctx, revision)
-}
-
-func (ib *instrumentedBatch) Commit(ctx context.Context, revision git.Revision) (*Object, error) {
- span, ctx := opentracing.StartSpanFromContext(ctx, "Batch.Commit", opentracing.Tag{"revision", revision})
- defer span.Finish()
-
- catfileLookupCounter.WithLabelValues("commit").Inc()
-
- return ib.Batch.Commit(ctx, revision)
-}
-
-func (ib *instrumentedBatch) Blob(ctx context.Context, revision git.Revision) (*Object, error) {
- span, ctx := opentracing.StartSpanFromContext(ctx, "Batch.Blob", opentracing.Tag{"revision", revision})
- defer span.Finish()
-
- catfileLookupCounter.WithLabelValues("blob").Inc()
-
- return ib.Batch.Blob(ctx, revision)
-}
-
-func (ib *instrumentedBatch) Tag(ctx context.Context, revision git.Revision) (*Object, error) {
- span, ctx := opentracing.StartSpanFromContext(ctx, "Batch.Tag", opentracing.Tag{"revision", revision})
- defer span.Finish()
-
- catfileLookupCounter.WithLabelValues("tag").Inc()
-
- return ib.Batch.Tag(ctx, revision)
-}
diff --git a/internal/git/catfile/objectinfo.go b/internal/git/catfile/objectinfo.go
index 7be477e24..601784aa8 100644
--- a/internal/git/catfile/objectinfo.go
+++ b/internal/git/catfile/objectinfo.go
@@ -5,11 +5,13 @@ import (
"fmt"
"strconv"
"strings"
+
+ "gitlab.com/gitlab-org/gitaly/internal/git"
)
// ObjectInfo represents a header returned by `git cat-file --batch`
type ObjectInfo struct {
- Oid string
+ Oid git.ObjectID
Type string
Size int64
}
@@ -45,13 +47,18 @@ func ParseObjectInfo(stdout *bufio.Reader) (*ObjectInfo, error) {
return nil, fmt.Errorf("invalid info line: %q", infoLine)
}
+ oid, err := git.NewObjectIDFromHex(info[0])
+ if err != nil {
+ return nil, fmt.Errorf("parse object ID: %w", err)
+ }
+
objectSize, err := strconv.ParseInt(info[2], 10, 64)
if err != nil {
return nil, fmt.Errorf("parse object size: %w", err)
}
return &ObjectInfo{
- Oid: info[0],
+ Oid: oid,
Type: info[1],
Size: objectSize,
}, nil
diff --git a/internal/git/catfile/parse_test.go b/internal/git/catfile/objectinfo_test.go
index 46e9b39e2..46e9b39e2 100644
--- a/internal/git/catfile/parse_test.go
+++ b/internal/git/catfile/objectinfo_test.go
diff --git a/internal/git/localrepo/objects.go b/internal/git/localrepo/objects.go
index d2105d694..6d3c9a3bb 100644
--- a/internal/git/localrepo/objects.go
+++ b/internal/git/localrepo/objects.go
@@ -24,7 +24,7 @@ var (
// WriteBlob writes a blob to the repository's object database and
// returns its object ID. Path is used by git to decide which filters to
// run on the content.
-func (repo *Repo) WriteBlob(ctx context.Context, path string, content io.Reader) (string, error) {
+func (repo *Repo) WriteBlob(ctx context.Context, path string, content io.Reader) (git.ObjectID, error) {
stdout := &bytes.Buffer{}
stderr := &bytes.Buffer{}
@@ -49,7 +49,12 @@ func (repo *Repo) WriteBlob(ctx context.Context, path string, content io.Reader)
return "", errorWithStderr(err, stderr.Bytes())
}
- return text.ChompBytes(stdout.Bytes()), nil
+ oid, err := git.NewObjectIDFromHex(text.ChompBytes(stdout.Bytes()))
+ if err != nil {
+ return "", err
+ }
+
+ return oid, nil
}
// FormatTagError is used by FormatTag() below
@@ -77,7 +82,12 @@ func (e FormatTagError) Error() string {
// timestamp here would at best be annoying, and at worst run up
// against some other assumption (e.g. that some hook check isn't as
// strict on locally generated data).
-func FormatTag(objectID, objectType string, tagName, userName, userEmail, tagBody []byte, committerDate time.Time) (string, error) {
+func FormatTag(
+ objectID git.ObjectID,
+ objectType string,
+ tagName, userName, userEmail, tagBody []byte,
+ committerDate time.Time,
+) (string, error) {
if committerDate.IsZero() {
committerDate = time.Now()
}
@@ -86,7 +96,7 @@ func FormatTag(objectID, objectType string, tagName, userName, userEmail, tagBod
"type %s\n" +
"tag %s\n" +
"tagger %s <%s> %d +0000\n"
- tagBuf := fmt.Sprintf(tagHeaderFormat, objectID, objectType, tagName, userName, userEmail, committerDate.Unix())
+ tagBuf := fmt.Sprintf(tagHeaderFormat, objectID.String(), objectType, tagName, userName, userEmail, committerDate.Unix())
maxHeaderLines := 4
actualHeaderLines := strings.Count(tagBuf, "\n")
@@ -116,7 +126,13 @@ func (e MktagError) Error() string {
//
// It's important that this be git-mktag and not git-hash-object due
// to its fsck sanity checking semantics.
-func (repo *Repo) WriteTag(ctx context.Context, objectID, objectType string, tagName, userName, userEmail, tagBody []byte, committerDate time.Time) (string, error) {
+func (repo *Repo) WriteTag(
+ ctx context.Context,
+ objectID git.ObjectID,
+ objectType string,
+ tagName, userName, userEmail, tagBody []byte,
+ committerDate time.Time,
+) (string, error) {
stdout := &bytes.Buffer{}
stderr := &bytes.Buffer{}
@@ -153,7 +169,7 @@ func (err InvalidObjectError) Error() string { return fmt.Sprintf("invalid objec
// ReadObject reads an object from the repository's object database. InvalidObjectError
// is returned if the oid does not refer to a valid object.
-func (repo *Repo) ReadObject(ctx context.Context, oid string) ([]byte, error) {
+func (repo *Repo) ReadObject(ctx context.Context, oid git.ObjectID) ([]byte, error) {
const msgInvalidObject = "fatal: Not a valid object name "
stdout := &bytes.Buffer{}
@@ -162,7 +178,7 @@ func (repo *Repo) ReadObject(ctx context.Context, oid string) ([]byte, error) {
git.SubCmd{
Name: "cat-file",
Flags: []git.Option{git.Flag{"-p"}},
- Args: []string{oid},
+ Args: []string{oid.String()},
},
git.WithStdout(stdout),
git.WithStderr(stderr),
diff --git a/internal/git/localrepo/objects_test.go b/internal/git/localrepo/objects_test.go
index 84cccb7b6..91bbcb7c0 100644
--- a/internal/git/localrepo/objects_test.go
+++ b/internal/git/localrepo/objects_test.go
@@ -117,7 +117,7 @@ func TestRepo_WriteBlob(t *testing.T) {
func TestFormatTag(t *testing.T) {
for _, tc := range []struct {
desc string
- objectID string
+ objectID git.ObjectID
objectType string
tagName []byte
userName []byte
@@ -130,7 +130,7 @@ func TestFormatTag(t *testing.T) {
// internal/gitaly/service/operations/tags_test.go
{
desc: "basic signature",
- objectID: git.ZeroOID.String(),
+ objectID: git.ZeroOID,
objectType: "commit",
tagName: []byte("my-tag"),
userName: []byte("root"),
@@ -139,7 +139,7 @@ func TestFormatTag(t *testing.T) {
},
{
desc: "basic signature",
- objectID: git.ZeroOID.String(),
+ objectID: git.ZeroOID,
objectType: "commit",
tagName: []byte("my-tag\ninjection"),
userName: []byte("root"),
@@ -149,7 +149,7 @@ func TestFormatTag(t *testing.T) {
},
{
desc: "signature with fixed time",
- objectID: git.ZeroOID.String(),
+ objectID: git.ZeroOID,
objectType: "commit",
tagName: []byte("my-tag"),
userName: []byte("root"),
@@ -182,7 +182,7 @@ func TestRepo_WriteTag(t *testing.T) {
for _, tc := range []struct {
desc string
- objectID string
+ objectID git.ObjectID
objectType string
tagName []byte
userName []byte
@@ -231,13 +231,13 @@ func TestRepo_ReadObject(t *testing.T) {
for _, tc := range []struct {
desc string
- oid string
+ oid git.ObjectID
content string
error error
}{
{
desc: "invalid object",
- oid: git.ZeroOID.String(),
+ oid: git.ZeroOID,
error: InvalidObjectError(git.ZeroOID.String()),
},
{
diff --git a/internal/git/log/commit.go b/internal/git/log/commit.go
index d1d60a31f..15f23b83b 100644
--- a/internal/git/log/commit.go
+++ b/internal/git/log/commit.go
@@ -109,7 +109,7 @@ func splitRawCommit(r io.Reader) ([]byte, []byte, error) {
func buildCommit(header, body []byte, info *catfile.ObjectInfo) (*gitalypb.GitCommit, error) {
commit := &gitalypb.GitCommit{
- Id: info.Oid,
+ Id: info.Oid.String(),
BodySize: int64(len(body)),
Body: body,
Subject: subjectFromBody(body),
diff --git a/internal/git/log/commit_test.go b/internal/git/log/commit_test.go
index fe8492e29..f9b9580dc 100644
--- a/internal/git/log/commit_test.go
+++ b/internal/git/log/commit_test.go
@@ -65,13 +65,13 @@ func TestParseRawCommit(t *testing.T) {
{
desc: "empty commit object",
in: []byte{},
- out: &gitalypb.GitCommit{Id: info.Oid},
+ out: &gitalypb.GitCommit{Id: info.Oid.String()},
},
{
desc: "no email",
in: []byte("author Jane Doe"),
out: &gitalypb.GitCommit{
- Id: info.Oid,
+ Id: info.Oid.String(),
Author: &gitalypb.CommitAuthor{Name: []byte("Jane Doe")},
},
},
@@ -79,7 +79,7 @@ func TestParseRawCommit(t *testing.T) {
desc: "unmatched <",
in: []byte("author Jane Doe <janedoe@example.com"),
out: &gitalypb.GitCommit{
- Id: info.Oid,
+ Id: info.Oid.String(),
Author: &gitalypb.CommitAuthor{Name: []byte("Jane Doe")},
},
},
@@ -87,7 +87,7 @@ func TestParseRawCommit(t *testing.T) {
desc: "unmatched >",
in: []byte("author Jane Doe janedoe@example.com>"),
out: &gitalypb.GitCommit{
- Id: info.Oid,
+ Id: info.Oid.String(),
Author: &gitalypb.CommitAuthor{Name: []byte("Jane Doe janedoe@example.com>")},
},
},
@@ -95,7 +95,7 @@ func TestParseRawCommit(t *testing.T) {
desc: "missing date",
in: []byte("author Jane Doe <janedoe@example.com> "),
out: &gitalypb.GitCommit{
- Id: info.Oid,
+ Id: info.Oid.String(),
Author: &gitalypb.CommitAuthor{Name: []byte("Jane Doe"), Email: []byte("janedoe@example.com")},
},
},
@@ -103,7 +103,7 @@ func TestParseRawCommit(t *testing.T) {
desc: "date too high",
in: []byte("author Jane Doe <janedoe@example.com> 9007199254740993 +0200"),
out: &gitalypb.GitCommit{
- Id: info.Oid,
+ Id: info.Oid.String(),
Author: &gitalypb.CommitAuthor{
Name: []byte("Jane Doe"),
Email: []byte("janedoe@example.com"),
@@ -116,7 +116,7 @@ func TestParseRawCommit(t *testing.T) {
desc: "date negative",
in: []byte("author Jane Doe <janedoe@example.com> -1 +0200"),
out: &gitalypb.GitCommit{
- Id: info.Oid,
+ Id: info.Oid.String(),
Author: &gitalypb.CommitAuthor{
Name: []byte("Jane Doe"),
Email: []byte("janedoe@example.com"),
diff --git a/internal/git2go/apply_test.go b/internal/git2go/apply_test.go
index e3ed37d8c..6f303257c 100644
--- a/internal/git2go/apply_test.go
+++ b/internal/git2go/apply_test.go
@@ -42,7 +42,7 @@ func TestExecutor_Apply(t *testing.T) {
Author: author,
Committer: committer,
Message: "base commit",
- Actions: []Action{CreateFile{Path: "file", OID: oidBase}},
+ Actions: []Action{CreateFile{Path: "file", OID: oidBase.String()}},
})
require.NoError(t, err)
@@ -51,7 +51,7 @@ func TestExecutor_Apply(t *testing.T) {
Author: author,
Committer: committer,
Message: "commit with ab",
- Actions: []Action{CreateFile{Path: "file", OID: oidA}},
+ Actions: []Action{CreateFile{Path: "file", OID: oidA.String()}},
})
require.NoError(t, err)
@@ -61,7 +61,7 @@ func TestExecutor_Apply(t *testing.T) {
Committer: committer,
Message: "commit with a",
Parent: parentCommitSHA,
- Actions: []Action{UpdateFile{Path: "file", OID: oidA}},
+ Actions: []Action{UpdateFile{Path: "file", OID: oidA.String()}},
})
require.NoError(t, err)
@@ -71,7 +71,7 @@ func TestExecutor_Apply(t *testing.T) {
Committer: committer,
Message: "commit to b",
Parent: parentCommitSHA,
- Actions: []Action{UpdateFile{Path: "file", OID: oidB}},
+ Actions: []Action{UpdateFile{Path: "file", OID: oidB.String()}},
})
require.NoError(t, err)
@@ -81,7 +81,7 @@ func TestExecutor_Apply(t *testing.T) {
Committer: committer,
Message: "commit a -> b",
Parent: updateToA,
- Actions: []Action{UpdateFile{Path: "file", OID: oidB}},
+ Actions: []Action{UpdateFile{Path: "file", OID: oidB.String()}},
})
require.NoError(t, err)
@@ -90,7 +90,7 @@ func TestExecutor_Apply(t *testing.T) {
Author: author,
Committer: committer,
Message: "commit with other-file",
- Actions: []Action{CreateFile{Path: "other-file", OID: oidA}},
+ Actions: []Action{CreateFile{Path: "other-file", OID: oidA.String()}},
})
require.NoError(t, err)
diff --git a/internal/git2go/commit_test.go b/internal/git2go/commit_test.go
index 6e689a314..c4c8427f7 100644
--- a/internal/git2go/commit_test.go
+++ b/internal/git2go/commit_test.go
@@ -120,7 +120,7 @@ func TestExecutor_Commit(t *testing.T) {
steps: []step{
{
actions: []Action{
- CreateFile{Path: "file", OID: originalFile},
+ CreateFile{Path: "file", OID: originalFile.String()},
},
treeEntries: []gittest.TreeEntry{
{Mode: DefaultMode, Path: "file", Content: "original"},
@@ -139,7 +139,7 @@ func TestExecutor_Commit(t *testing.T) {
steps: []step{
{
actions: []Action{
- CreateFile{Path: "file", OID: originalFile},
+ CreateFile{Path: "file", OID: originalFile.String()},
},
treeEntries: []gittest.TreeEntry{
{Mode: DefaultMode, Path: "file", Content: "original"},
@@ -152,8 +152,8 @@ func TestExecutor_Commit(t *testing.T) {
steps: []step{
{
actions: []Action{
- CreateFile{Path: "file", OID: originalFile},
- CreateFile{Path: "file", OID: updatedFile},
+ CreateFile{Path: "file", OID: originalFile.String()},
+ CreateFile{Path: "file", OID: updatedFile.String()},
},
error: FileExistsError("file"),
},
@@ -165,7 +165,7 @@ func TestExecutor_Commit(t *testing.T) {
{
actions: []Action{
CreateDirectory{Path: "directory"},
- CreateFile{Path: "directory", OID: originalFile},
+ CreateFile{Path: "directory", OID: originalFile.String()},
},
treeEntries: []gittest.TreeEntry{
{Mode: DefaultMode, Path: "directory", Content: "original"},
@@ -178,8 +178,8 @@ func TestExecutor_Commit(t *testing.T) {
steps: []step{
{
actions: []Action{
- CreateFile{Path: "file", OID: originalFile},
- UpdateFile{Path: "file", OID: updatedFile},
+ CreateFile{Path: "file", OID: originalFile.String()},
+ UpdateFile{Path: "file", OID: updatedFile.String()},
},
treeEntries: []gittest.TreeEntry{
{Mode: DefaultMode, Path: "file", Content: "updated"},
@@ -192,7 +192,7 @@ func TestExecutor_Commit(t *testing.T) {
steps: []step{
{
actions: []Action{
- CreateFile{Path: "file", OID: originalFile},
+ CreateFile{Path: "file", OID: originalFile.String()},
},
treeEntries: []gittest.TreeEntry{
{Mode: DefaultMode, Path: "file", Content: "original"},
@@ -200,7 +200,7 @@ func TestExecutor_Commit(t *testing.T) {
},
{
actions: []Action{
- UpdateFile{Path: "file", OID: updatedFile},
+ UpdateFile{Path: "file", OID: updatedFile.String()},
},
treeEntries: []gittest.TreeEntry{
{Mode: DefaultMode, Path: "file", Content: "updated"},
@@ -213,7 +213,7 @@ func TestExecutor_Commit(t *testing.T) {
steps: []step{
{
actions: []Action{
- UpdateFile{Path: "non-existing", OID: updatedFile},
+ UpdateFile{Path: "non-existing", OID: updatedFile.String()},
},
error: FileNotFoundError("non-existing"),
},
@@ -224,8 +224,8 @@ func TestExecutor_Commit(t *testing.T) {
steps: []step{
{
actions: []Action{
- CreateFile{Path: "original-file", OID: originalFile},
- MoveFile{Path: "original-file", NewPath: "moved-file", OID: originalFile},
+ CreateFile{Path: "original-file", OID: originalFile.String()},
+ MoveFile{Path: "original-file", NewPath: "moved-file", OID: originalFile.String()},
},
treeEntries: []gittest.TreeEntry{
{Mode: DefaultMode, Path: "moved-file", Content: "original"},
@@ -250,7 +250,7 @@ func TestExecutor_Commit(t *testing.T) {
steps: []step{
{
actions: []Action{
- CreateFile{Path: "original-file", OID: originalFile},
+ CreateFile{Path: "original-file", OID: originalFile.String()},
},
treeEntries: []gittest.TreeEntry{
{Mode: DefaultMode, Path: "original-file", Content: "original"},
@@ -282,8 +282,8 @@ func TestExecutor_Commit(t *testing.T) {
steps: []step{
{
actions: []Action{
- CreateFile{Path: "source-file", OID: originalFile},
- CreateFile{Path: "already-existing", OID: updatedFile},
+ CreateFile{Path: "source-file", OID: originalFile.String()},
+ CreateFile{Path: "already-existing", OID: updatedFile.String()},
MoveFile{Path: "source-file", NewPath: "already-existing"},
},
error: FileExistsError("already-existing"),
@@ -297,7 +297,7 @@ func TestExecutor_Commit(t *testing.T) {
steps: []step{
{
actions: []Action{
- CreateFile{Path: "file", OID: originalFile},
+ CreateFile{Path: "file", OID: originalFile.String()},
CreateDirectory{Path: "already-existing"},
MoveFile{Path: "file", NewPath: "already-existing"},
},
@@ -312,7 +312,7 @@ func TestExecutor_Commit(t *testing.T) {
steps: []step{
{
actions: []Action{
- CreateFile{Path: "original-file", OID: originalFile},
+ CreateFile{Path: "original-file", OID: originalFile.String()},
},
treeEntries: []gittest.TreeEntry{
{Mode: DefaultMode, Path: "original-file", Content: "original"},
@@ -320,7 +320,7 @@ func TestExecutor_Commit(t *testing.T) {
},
{
actions: []Action{
- MoveFile{Path: "original-file", NewPath: "moved-file", OID: updatedFile},
+ MoveFile{Path: "original-file", NewPath: "moved-file", OID: updatedFile.String()},
},
treeEntries: []gittest.TreeEntry{
{Mode: DefaultMode, Path: "moved-file", Content: "updated"},
@@ -344,7 +344,7 @@ func TestExecutor_Commit(t *testing.T) {
steps: []step{
{
actions: []Action{
- CreateFile{Path: "file-1", OID: originalFile},
+ CreateFile{Path: "file-1", OID: originalFile.String()},
ChangeFileMode{Path: "file-1", ExecutableMode: true},
},
treeEntries: []gittest.TreeEntry{
@@ -366,7 +366,7 @@ func TestExecutor_Commit(t *testing.T) {
steps: []step{
{
actions: []Action{
- CreateFile{Path: "file-1", OID: originalFile},
+ CreateFile{Path: "file-1", OID: originalFile.String()},
ChangeFileMode{Path: "file-1", ExecutableMode: true},
},
treeEntries: []gittest.TreeEntry{
@@ -380,7 +380,7 @@ func TestExecutor_Commit(t *testing.T) {
steps: []step{
{
actions: []Action{
- CreateFile{Path: "file-1", OID: originalFile},
+ CreateFile{Path: "file-1", OID: originalFile.String()},
},
treeEntries: []gittest.TreeEntry{
{Mode: DefaultMode, Path: "file-1", Content: "original"},
@@ -412,8 +412,8 @@ func TestExecutor_Commit(t *testing.T) {
steps: []step{
{
actions: []Action{
- CreateFile{Path: "file-1", OID: originalFile},
- CreateFile{Path: "file-2", OID: updatedFile},
+ CreateFile{Path: "file-1", OID: originalFile.String()},
+ CreateFile{Path: "file-2", OID: updatedFile.String()},
MoveFile{Path: "file-1", NewPath: "file-2"},
},
error: FileExistsError("file-2"),
@@ -436,7 +436,7 @@ func TestExecutor_Commit(t *testing.T) {
steps: []step{
{
actions: []Action{
- CreateFile{Path: "file-1", OID: originalFile},
+ CreateFile{Path: "file-1", OID: originalFile.String()},
DeleteFile{Path: "file-1"},
},
},
@@ -447,7 +447,7 @@ func TestExecutor_Commit(t *testing.T) {
steps: []step{
{
actions: []Action{
- CreateFile{Path: "file-1", OID: originalFile},
+ CreateFile{Path: "file-1", OID: originalFile.String()},
},
treeEntries: []gittest.TreeEntry{
{Mode: DefaultMode, Path: "file-1", Content: "original"},
@@ -500,7 +500,7 @@ func TestExecutor_Commit(t *testing.T) {
func getCommit(t testing.TB, ctx context.Context, repo *localrepo.Repo, oid string) commit {
t.Helper()
- data, err := repo.ReadObject(ctx, oid)
+ data, err := repo.ReadObject(ctx, git.ObjectID(oid))
require.NoError(t, err)
var commit commit
diff --git a/internal/gitaly/service/blob/get_blob.go b/internal/gitaly/service/blob/get_blob.go
index 1f1d7f38e..213552416 100644
--- a/internal/gitaly/service/blob/get_blob.go
+++ b/internal/gitaly/service/blob/get_blob.go
@@ -39,7 +39,7 @@ func (s *server) GetBlob(in *gitalypb.GetBlobRequest, stream gitalypb.BlobServic
}
firstMessage := &gitalypb.GetBlobResponse{
Size: objectInfo.Size,
- Oid: objectInfo.Oid,
+ Oid: objectInfo.Oid.String(),
}
if readLimit == 0 {
diff --git a/internal/gitaly/service/blob/lfs_pointers.go b/internal/gitaly/service/blob/lfs_pointers.go
index 03bf3f802..92c1948af 100644
--- a/internal/gitaly/service/blob/lfs_pointers.go
+++ b/internal/gitaly/service/blob/lfs_pointers.go
@@ -387,7 +387,7 @@ func readLFSPointers(
lfsPointers = append(lfsPointers, &gitalypb.LFSPointer{
Data: data,
Size: int64(len(data)),
- Oid: objectInfo.Oid,
+ Oid: objectInfo.Oid.String(),
})
// Exit early in case we've got all LFS pointers. We want to do this here instead of
diff --git a/internal/gitaly/service/commit/tree_entries_helper.go b/internal/gitaly/service/commit/tree_entries_helper.go
index 4f48814a0..2fcbd84d4 100644
--- a/internal/gitaly/service/commit/tree_entries_helper.go
+++ b/internal/gitaly/service/commit/tree_entries_helper.go
@@ -125,7 +125,7 @@ func treeEntries(ctx context.Context, c catfile.Batch, revision, path string, ro
return nil, err
}
- rootOid = rootTreeInfo.Oid
+ rootOid = rootTreeInfo.Oid.String()
}
treeObj, err := c.Tree(ctx, git.Revision(fmt.Sprintf("%s:%s", revision, path)))
@@ -136,7 +136,7 @@ func treeEntries(ctx context.Context, c catfile.Batch, revision, path string, ro
return nil, err
}
- entries, err := extractEntryInfoFromTreeData(treeObj, revision, rootOid, path, treeObj.Oid)
+ entries, err := extractEntryInfoFromTreeData(treeObj, revision, rootOid, path, treeObj.Oid.String())
if err != nil {
return nil, err
}
diff --git a/internal/gitaly/service/commit/tree_entry.go b/internal/gitaly/service/commit/tree_entry.go
index f5b0c6da5..9f67cc33b 100644
--- a/internal/gitaly/service/commit/tree_entry.go
+++ b/internal/gitaly/service/commit/tree_entry.go
@@ -83,7 +83,7 @@ func sendTreeEntry(stream gitalypb.CommitService_TreeEntryServer, c catfile.Batc
response := &gitalypb.TreeEntryResponse{
Type: gitalypb.TreeEntryResponse_BLOB,
- Oid: objectInfo.Oid,
+ Oid: objectInfo.Oid.String(),
Size: objectInfo.Size,
Mode: treeEntry.Mode,
}
diff --git a/internal/gitaly/service/conflicts/resolve_conflicts_test.go b/internal/gitaly/service/conflicts/resolve_conflicts_test.go
index 6b2bf0e74..bee92f864 100644
--- a/internal/gitaly/service/conflicts/resolve_conflicts_test.go
+++ b/internal/gitaly/service/conflicts/resolve_conflicts_test.go
@@ -121,7 +121,7 @@ func testSuccessfulResolveConflictsRequest(t *testing.T, ctx context.Context) {
testhelper.MustRunCommand(t, nil, "git", "-C", testRepoPath, "read-tree", branch)
testhelper.MustRunCommand(t, nil,
"git", "-C", testRepoPath,
- "update-index", "--add", "--cacheinfo", "100644", blobID, missingAncestorPath,
+ "update-index", "--add", "--cacheinfo", "100644", blobID.String(), missingAncestorPath,
)
treeID := bytes.TrimSpace(
testhelper.MustRunCommand(t, nil,
diff --git a/internal/gitaly/service/operations/commit_files.go b/internal/gitaly/service/operations/commit_files.go
index 8aa3791bf..e01bcf3cd 100644
--- a/internal/gitaly/service/operations/commit_files.go
+++ b/internal/gitaly/service/operations/commit_files.go
@@ -270,7 +270,7 @@ func (s *Server) userCommitFiles(ctx context.Context, header *gitalypb.UserCommi
}
actions = append(actions, git2go.CreateFile{
- OID: blobID,
+ OID: blobID.String(),
Path: path,
ExecutableMode: pbAction.header.ExecuteFilemode,
})
@@ -285,7 +285,7 @@ func (s *Server) userCommitFiles(ctx context.Context, header *gitalypb.UserCommi
return fmt.Errorf("validate previous path: %w", err)
}
- var oid string
+ var oid git.ObjectID
if !pbAction.header.InferContent {
var err error
oid, err = localRepo.WriteBlob(ctx, path, content)
@@ -297,7 +297,7 @@ func (s *Server) userCommitFiles(ctx context.Context, header *gitalypb.UserCommi
actions = append(actions, git2go.MoveFile{
Path: prevPath,
NewPath: path,
- OID: oid,
+ OID: oid.String(),
})
case gitalypb.UserCommitFilesActionHeader_UPDATE:
oid, err := localRepo.WriteBlob(ctx, path, content)
@@ -307,7 +307,7 @@ func (s *Server) userCommitFiles(ctx context.Context, header *gitalypb.UserCommi
actions = append(actions, git2go.UpdateFile{
Path: path,
- OID: oid,
+ OID: oid.String(),
})
case gitalypb.UserCommitFilesActionHeader_DELETE:
actions = append(actions, git2go.DeleteFile{
diff --git a/internal/gitaly/service/operations/tags.go b/internal/gitaly/service/operations/tags.go
index c3dd6cbc7..cd9520bfb 100644
--- a/internal/gitaly/service/operations/tags.go
+++ b/internal/gitaly/service/operations/tags.go
@@ -141,7 +141,7 @@ func (s *Server) UserCreateTag(ctx context.Context, req *gitalypb.UserCreateTagR
// At this point we'll either be pointing to an object we were
// provided with, or creating a new tag object and pointing to
// that.
- refObjectID := targetObjectID
+ refObjectID := targetObjectID.String()
var tagObject *gitalypb.Tag
if makingTag {
localRepo := localrepo.New(s.gitCmdFactory, repo, s.cfg)
@@ -185,7 +185,7 @@ func (s *Server) UserCreateTag(ctx context.Context, req *gitalypb.UserCreateTagR
} else {
tagObject = &gitalypb.Tag{
Name: req.TagName,
- Id: peeledTargetObjectID,
+ Id: peeledTargetObjectID.String(),
//TargetCommit: is filled in below if needed
}
}
@@ -249,7 +249,7 @@ func (s *Server) UserCreateTag(ctx context.Context, req *gitalypb.UserCreateTagR
// Save ourselves looking this up earlier in case update-ref
// died
if peeledTargetObjectType == "commit" {
- peeledTargetCommit, err := log.GetCommitCatfile(ctx, catFile, git.Revision(peeledTargetObjectID))
+ peeledTargetCommit, err := log.GetCommitCatfile(ctx, catFile, peeledTargetObjectID.Revision())
if err != nil {
return nil, status.Error(codes.Internal, err.Error())
}
diff --git a/internal/gitaly/service/ref/list_new_blobs.go b/internal/gitaly/service/ref/list_new_blobs.go
index 44c546295..98f7c6ac0 100644
--- a/internal/gitaly/service/ref/list_new_blobs.go
+++ b/internal/gitaly/service/ref/list_new_blobs.go
@@ -62,7 +62,11 @@ func (s *server) listNewBlobs(in *gitalypb.ListNewBlobsRequest, stream gitalypb.
continue
}
- newBlobs = append(newBlobs, &gitalypb.NewBlobObject{Oid: info.Oid, Size: info.Size, Path: []byte(parts[1])})
+ newBlobs = append(newBlobs, &gitalypb.NewBlobObject{
+ Oid: info.Oid.String(),
+ Size: info.Size,
+ Path: []byte(parts[1]),
+ })
if len(newBlobs) >= 1000 {
response := &gitalypb.ListNewBlobsResponse{NewBlobObjects: newBlobs}
if err := stream.Send(response); err != nil {
diff --git a/internal/gitaly/service/repository/apply_gitattributes.go b/internal/gitaly/service/repository/apply_gitattributes.go
index 6c4335f84..0f0728d29 100644
--- a/internal/gitaly/service/repository/apply_gitattributes.go
+++ b/internal/gitaly/service/repository/apply_gitattributes.go
@@ -83,13 +83,8 @@ func (s *server) applyGitattributes(ctx context.Context, c catfile.Batch, repoPa
return err
}
- blobOID, err := git.NewObjectIDFromHex(blobInfo.Oid)
- if err != nil {
- return err
- }
-
// Vote on the contents of the newly written gitattributes file.
- if err := s.vote(ctx, blobOID); err != nil {
+ if err := s.vote(ctx, blobInfo.Oid); err != nil {
return fmt.Errorf("could not commit gitattributes: %w", err)
}
diff --git a/internal/gitaly/service/repository/raw_changes.go b/internal/gitaly/service/repository/raw_changes.go
index 9db19100c..df4b89e2b 100644
--- a/internal/gitaly/service/repository/raw_changes.go
+++ b/internal/gitaly/service/repository/raw_changes.go
@@ -158,7 +158,7 @@ func changeFromDiff(ctx context.Context, batch catfile.Batch, d *rawdiff.Diff) (
return nil, fmt.Errorf("find %q: %v", shortBlobID, err)
}
- resp.BlobId = info.Oid
+ resp.BlobId = info.Oid.String()
resp.Size = info.Size
}