0
0
mirror of https://github.com/go-gitea/gitea.git synced 2026-01-22 06:45:08 +01:00

Refactor git command stdio pipe (#36393)

And remove the incorrect `ensureValidGitRepository`
This commit is contained in:
wxiaoguang 2026-01-18 02:11:46 +08:00 committed by GitHub
parent 7a2aac406d
commit 149f7a6f1f
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
9 changed files with 210 additions and 122 deletions

View File

@ -5,8 +5,11 @@ package git
import (
"context"
"os"
"path/filepath"
"code.gitea.io/gitea/modules/git/gitcmd"
"code.gitea.io/gitea/modules/util"
)
// catFileBatchCommand implements the CatFileBatch interface using the "cat-file --batch-command" command
@ -21,8 +24,8 @@ type catFileBatchCommand struct {
var _ CatFileBatch = (*catFileBatchCommand)(nil)
func newCatFileBatchCommand(ctx context.Context, repoPath string) (*catFileBatchCommand, error) {
if err := ensureValidGitRepository(ctx, repoPath); err != nil {
return nil, err
if _, err := os.Stat(repoPath); err != nil {
return nil, util.NewNotExistErrorf("repo %q doesn't exist", filepath.Base(repoPath))
}
return &catFileBatchCommand{ctx: ctx, repoPath: repoPath}, nil
}

View File

@ -6,8 +6,11 @@ package git
import (
"context"
"io"
"os"
"path/filepath"
"code.gitea.io/gitea/modules/git/gitcmd"
"code.gitea.io/gitea/modules/util"
)
// catFileBatchLegacy implements the CatFileBatch interface using the "cat-file --batch" command and "cat-file --batch-check" command
@ -24,8 +27,8 @@ type catFileBatchLegacy struct {
var _ CatFileBatchCloser = (*catFileBatchLegacy)(nil)
func newCatFileBatchLegacy(ctx context.Context, repoPath string) (*catFileBatchLegacy, error) {
if err := ensureValidGitRepository(ctx, repoPath); err != nil {
return nil, err
if _, err := os.Stat(repoPath); err != nil {
return nil, util.NewNotExistErrorf("repo %q doesn't exist", filepath.Base(repoPath))
}
return &catFileBatchLegacy{ctx: ctx, repoPath: repoPath}, nil
}

View File

@ -16,16 +16,12 @@ import (
"code.gitea.io/gitea/modules/log"
)
// writeCloserError wraps an io.WriteCloser with an additional CloseWithError function (for nio.Pipe)
type writeCloserError interface {
io.WriteCloser
CloseWithError(err error) error
}
type catFileBatchCommunicator struct {
cancel context.CancelFunc
reader *bufio.Reader
writer writeCloserError
writer io.Writer
debugGitCmd *gitcmd.Command
}
func (b *catFileBatchCommunicator) Close() {
@ -37,63 +33,41 @@ func (b *catFileBatchCommunicator) Close() {
}
}
// ensureValidGitRepository runs git rev-parse in the repository path - thus ensuring that the repository is a valid repository.
// Run before opening git cat-file.
// This is needed otherwise the git cat-file will hang for invalid repositories.
// FIXME: the comment is from https://github.com/go-gitea/gitea/pull/17991 but it doesn't seem to be true.
// The real problem is that Golang's Cmd.Wait hangs because it waits for the pipes to be closed, but we can't close the pipes before Wait returns
// Need to refactor to use StdinPipe and StdoutPipe
func ensureValidGitRepository(ctx context.Context, repoPath string) error {
stderr := strings.Builder{}
err := gitcmd.NewCommand("rev-parse").
WithDir(repoPath).
WithStderr(&stderr).
Run(ctx)
if err != nil {
return gitcmd.ConcatenateError(err, (&stderr).String())
}
return nil
}
// newCatFileBatch opens git cat-file --batch in the provided repo and returns a stdin pipe, a stdout reader and cancel function
func newCatFileBatch(ctx context.Context, repoPath string, cmdCatFile *gitcmd.Command) *catFileBatchCommunicator {
// We often want to feed the commits in order into cat-file --batch, followed by their trees and subtrees as necessary.
ctx, ctxCancel := context.WithCancelCause(ctx)
// so let's create a batch stdin and stdout
batchStdinReader, batchStdinWriter := io.Pipe()
batchStdoutReader, batchStdoutWriter := io.Pipe()
ctx, ctxCancel := context.WithCancel(ctx)
closed := make(chan struct{})
cancel := func() {
ctxCancel()
_ = batchStdinWriter.Close()
_ = batchStdoutReader.Close()
<-closed
var batchStdinWriter io.WriteCloser
var batchStdoutReader io.ReadCloser
stderr := strings.Builder{}
cmdCatFile = cmdCatFile.
WithDir(repoPath).
WithStdinWriter(&batchStdinWriter).
WithStdoutReader(&batchStdoutReader).
WithStderr(&stderr).
WithUseContextTimeout(true)
err := cmdCatFile.Start(ctx)
if err != nil {
log.Error("Unable to start git command %v: %v", cmdCatFile.LogString(), err)
// ideally here it should return the error, but it would require refactoring all callers
// so just return a dummy communicator that does nothing, almost the same behavior as before, not bad
return &catFileBatchCommunicator{
writer: io.Discard,
reader: bufio.NewReader(bytes.NewReader(nil)),
cancel: func() {
ctxCancel(err)
},
}
}
// Ensure cancel is called as soon as the provided context is cancelled
go func() {
<-ctx.Done()
cancel()
}()
go func() {
stderr := strings.Builder{}
err := cmdCatFile.
WithDir(repoPath).
WithStdin(batchStdinReader).
WithStdout(batchStdoutWriter).
WithStderr(&stderr).
WithUseContextTimeout(true).
Run(ctx)
err := cmdCatFile.Wait()
if err != nil {
_ = batchStdoutWriter.CloseWithError(gitcmd.ConcatenateError(err, (&stderr).String()))
_ = batchStdinReader.CloseWithError(gitcmd.ConcatenateError(err, (&stderr).String()))
} else {
_ = batchStdoutWriter.Close()
_ = batchStdinReader.Close()
log.Error("cat-file --batch command failed in repo %s: %v - stderr: %s", repoPath, err, stderr.String())
}
close(closed)
ctxCancel(err)
}()
// use a buffered reader to read from the cat-file --batch (StringReader.ReadString)
@ -102,7 +76,10 @@ func newCatFileBatch(ctx context.Context, repoPath string, cmdCatFile *gitcmd.Co
return &catFileBatchCommunicator{
writer: batchStdinWriter,
reader: batchReader,
cancel: cancel,
cancel: func() {
ctxCancel(nil)
},
debugGitCmd: cmdCatFile,
}
}

View File

@ -6,6 +6,7 @@ package git
import (
"io"
"path/filepath"
"sync"
"testing"
"code.gitea.io/gitea/modules/test"
@ -25,7 +26,14 @@ func TestCatFileBatch(t *testing.T) {
func testCatFileBatch(t *testing.T) {
t.Run("CorruptedGitRepo", func(t *testing.T) {
tmpDir := t.TempDir()
_, err := NewBatch(t.Context(), tmpDir)
batch, err := NewBatch(t.Context(), tmpDir)
// as long as the directory exists, no error, because we can't really know whether the git repo is valid until we run commands
require.NoError(t, err)
defer batch.Close()
_, err = batch.QueryInfo("e2129701f1a4d54dc44f03c93bca0a2aec7c5449")
require.Error(t, err)
_, err = batch.QueryInfo("e2129701f1a4d54dc44f03c93bca0a2aec7c5449")
require.Error(t, err)
})
@ -52,4 +60,30 @@ func testCatFileBatch(t *testing.T) {
require.NoError(t, err)
require.Equal(t, "file1\n", string(content))
})
t.Run("QueryTerminated", func(t *testing.T) {
var c *catFileBatchCommunicator
switch b := batch.(type) {
case *catFileBatchLegacy:
c = b.batchCheck
_, _ = c.writer.Write([]byte("in-complete-line-"))
case *catFileBatchCommand:
c = b.batch
_, _ = c.writer.Write([]byte("info"))
default:
t.FailNow()
return
}
wg := sync.WaitGroup{}
wg.Go(func() {
buf := make([]byte, 100)
_, _ = c.reader.Read(buf)
n, errRead := c.reader.Read(buf)
assert.Zero(t, n)
assert.ErrorIs(t, errRead, io.EOF) // the pipe is closed due to command being killed
})
c.debugGitCmd.DebugKill()
wg.Wait()
})
}

View File

@ -13,7 +13,6 @@ import (
"os"
"os/exec"
"path/filepath"
"runtime"
"strings"
"time"
@ -43,10 +42,19 @@ const DefaultLocale = "C"
type Command struct {
prog string
args []string
brokenArgs []string
preErrors []error
cmd *exec.Cmd // for debug purpose only
configArgs []string
opts runOpts
cmdCtx context.Context
cmdCancel context.CancelFunc
cmdFinished context.CancelFunc
cmdStartTime time.Time
cmdStdinWriter *io.WriteCloser
cmdStdoutReader *io.ReadCloser
cmdStderrReader *io.ReadCloser
}
func logArgSanitize(arg string) string {
@ -97,6 +105,10 @@ func NewCommand(args ...internal.CmdArg) *Command {
}
}
func (c *Command) handlePreErrorBrokenCommand(arg string) {
c.preErrors = append(c.preErrors, util.ErrorWrap(ErrBrokenCommand, `broken git command argument %q`, arg))
}
// isSafeArgumentValue checks if the argument is safe to be used as a value (not an option)
func isSafeArgumentValue(s string) bool {
return s == "" || s[0] != '-'
@ -124,7 +136,7 @@ func (c *Command) AddArguments(args ...internal.CmdArg) *Command {
// The values are treated as dynamic argument values. It equals to: AddArguments("--opt") then AddDynamicArguments(val).
func (c *Command) AddOptionValues(opt internal.CmdArg, args ...string) *Command {
if !isValidArgumentOption(string(opt)) {
c.brokenArgs = append(c.brokenArgs, string(opt))
c.handlePreErrorBrokenCommand(string(opt))
return c
}
c.args = append(c.args, string(opt))
@ -136,12 +148,12 @@ func (c *Command) AddOptionValues(opt internal.CmdArg, args ...string) *Command
// For example: AddOptionFormat("--opt=%s %s", val1, val2) means 1 argument: {"--opt=val1 val2"}.
func (c *Command) AddOptionFormat(opt string, args ...any) *Command {
if !isValidArgumentOption(opt) {
c.brokenArgs = append(c.brokenArgs, opt)
c.handlePreErrorBrokenCommand(opt)
return c
}
// a quick check to make sure the format string matches the number of arguments, to find low-level mistakes ASAP
if strings.Count(strings.ReplaceAll(opt, "%%", ""), "%") != len(args) {
c.brokenArgs = append(c.brokenArgs, opt)
c.handlePreErrorBrokenCommand(opt)
return c
}
s := fmt.Sprintf(opt, args...)
@ -155,10 +167,10 @@ func (c *Command) AddOptionFormat(opt string, args ...any) *Command {
func (c *Command) AddDynamicArguments(args ...string) *Command {
for _, arg := range args {
if !isSafeArgumentValue(arg) {
c.brokenArgs = append(c.brokenArgs, arg)
c.handlePreErrorBrokenCommand(arg)
}
}
if len(c.brokenArgs) != 0 {
if len(c.preErrors) != 0 {
return c
}
c.args = append(c.args, args...)
@ -178,7 +190,7 @@ func (c *Command) AddDashesAndList(list ...string) *Command {
func (c *Command) AddConfig(key, value string) *Command {
kv := key + "=" + value
if !isSafeArgumentValue(kv) {
c.brokenArgs = append(c.brokenArgs, key)
c.handlePreErrorBrokenCommand(kv)
} else {
c.configArgs = append(c.configArgs, "-c", kv)
}
@ -219,6 +231,7 @@ type runOpts struct {
// * `go { case <- parentContext.Done(): stdinWriter.Close() }` with `cmd.Run(DefaultTimeout)`: the command would have been killed by timeout but the Run doesn't return until stdinWriter.Close()
// * `go { if stdoutReader.Read() err != nil: stdinWriter.Close() }` with `cmd.Run()`: the stdoutReader may never return error if the command is killed by timeout
// In the future, ideally the git module itself should have full control of the stdin, to avoid such problems and make it easier to refactor to a better architecture.
// Use new functions like WithStdinWriter to avoid such problems.
Stdin io.Reader
PipelineFunc func(context.Context, context.CancelFunc) error
@ -281,16 +294,34 @@ func (c *Command) WithTimeout(timeout time.Duration) *Command {
return c
}
func (c *Command) WithStdoutReader(r *io.ReadCloser) *Command {
c.cmdStdoutReader = r
return c
}
// WithStdout is deprecated, use WithStdoutReader instead
func (c *Command) WithStdout(stdout io.Writer) *Command {
c.opts.Stdout = stdout
return c
}
func (c *Command) WithStderrReader(r *io.ReadCloser) *Command {
c.cmdStderrReader = r
return c
}
// WithStderr is deprecated, use WithStderrReader instead
func (c *Command) WithStderr(stderr io.Writer) *Command {
c.opts.Stderr = stderr
return c
}
func (c *Command) WithStdinWriter(w *io.WriteCloser) *Command {
c.cmdStdinWriter = w
return c
}
// WithStdin is deprecated, use WithStdinWriter instead
func (c *Command) WithStdin(stdin io.Reader) *Command {
c.opts.Stdin = stdin
return c
@ -329,11 +360,30 @@ func (c *Command) WithParentCallerInfo(optInfo ...string) *Command {
return c
}
// Run runs the command
func (c *Command) Run(ctx context.Context) error {
if len(c.brokenArgs) != 0 {
log.Error("git command is broken: %s, broken args: %s", c.LogString(), strings.Join(c.brokenArgs, " "))
return ErrBrokenCommand
func (c *Command) Start(ctx context.Context) (retErr error) {
if c.cmd != nil {
// this is a programming error, it will cause serious deadlock problems, so it must be fixed.
panic("git command has already been started")
}
defer func() {
if retErr != nil {
// release the pipes to avoid resource leak
safeClosePtrCloser(c.cmdStdoutReader)
safeClosePtrCloser(c.cmdStderrReader)
safeClosePtrCloser(c.cmdStdinWriter)
// if no error, cmdFinished will be called in "Wait" function
if c.cmdFinished != nil {
c.cmdFinished()
}
}
}()
if len(c.preErrors) != 0 {
// In most cases, such error shouldn't happen. If it happens, it must be a programming error, so we log it as error level with more details
err := errors.Join(c.preErrors...)
log.Error("git command: %s, error: %s", c.LogString(), err)
return err
}
// We must not change the provided options
@ -355,17 +405,13 @@ func (c *Command) Run(ctx context.Context) error {
span.SetAttributeString(gtprof.TraceAttrFuncCaller, c.opts.callerInfo)
span.SetAttributeString(gtprof.TraceAttrGitCommand, cmdLogString)
var cancel context.CancelFunc
var finished context.CancelFunc
if c.opts.UseContextTimeout {
ctx, cancel, finished = process.GetManager().AddContext(ctx, desc)
c.cmdCtx, c.cmdCancel, c.cmdFinished = process.GetManager().AddContext(ctx, desc)
} else {
ctx, cancel, finished = process.GetManager().AddContextTimeout(ctx, timeout, desc)
c.cmdCtx, c.cmdCancel, c.cmdFinished = process.GetManager().AddContextTimeout(ctx, timeout, desc)
}
defer finished()
startTime := time.Now()
c.cmdStartTime = time.Now()
cmd := exec.CommandContext(ctx, c.prog, append(c.configArgs, c.args...)...)
c.cmd = cmd // for debug purpose only
@ -381,9 +427,22 @@ func (c *Command) Run(ctx context.Context) error {
cmd.Stdout = c.opts.Stdout
cmd.Stderr = c.opts.Stderr
cmd.Stdin = c.opts.Stdin
if err := cmd.Start(); err != nil {
if _, err := safeAssignPipe(c.cmdStdinWriter, cmd.StdinPipe); err != nil {
return err
}
if _, err := safeAssignPipe(c.cmdStdoutReader, cmd.StdoutPipe); err != nil {
return err
}
if _, err := safeAssignPipe(c.cmdStderrReader, cmd.StderrPipe); err != nil {
return err
}
return cmd.Start()
}
func (c *Command) Wait() error {
defer c.cmdFinished()
cmd, ctx, cancel := c.cmd, c.cmdCtx, c.cmdCancel
if c.opts.PipelineFunc != nil {
err := c.opts.PipelineFunc(ctx, cancel)
@ -394,29 +453,30 @@ func (c *Command) Run(ctx context.Context) error {
}
}
err := cmd.Wait()
elapsed := time.Since(startTime)
errWait := cmd.Wait()
elapsed := time.Since(c.cmdStartTime)
if elapsed > time.Second {
log.Debug("slow git.Command.Run: %s (%s)", c, elapsed)
}
// We need to check if the context is canceled by the program on Windows.
// This is because Windows does not have signal checking when terminating the process.
// It always returns exit code 1, unlike Linux, which has many exit codes for signals.
// `err.Error()` returns "exit status 1" when using the `git check-attr` command after the context is canceled.
if runtime.GOOS == "windows" &&
err != nil &&
(err.Error() == "" || err.Error() == "exit status 1") &&
cmd.ProcessState.ExitCode() == 1 &&
ctx.Err() == context.Canceled {
return ctx.Err()
errCause := context.Cause(c.cmdCtx)
if errors.Is(errCause, context.Canceled) {
// if the ctx is canceled without other error, it must be caused by normal cancellation
return errCause
}
if errWait != nil {
// no matter whether there is other cause error, if "Wait" also has error,
// it's likely the error is caused by Wait error (from git command)
return errWait
}
return errCause
}
if err != nil && ctx.Err() != context.DeadlineExceeded {
func (c *Command) Run(ctx context.Context) (err error) {
if err = c.Start(ctx); err != nil {
return err
}
return ctx.Err()
return c.Wait()
}
type RunStdError interface {
@ -502,3 +562,7 @@ func (c *Command) runStdBytes(ctx context.Context) ( /*stdout*/ []byte /*stderr*
// even if there is no err, there could still be some stderr output
return stdoutBuf.Bytes(), stderrBuf.Bytes(), nil
}
func (c *Command) DebugKill() {
_ = c.cmd.Process.Kill()
}

View File

@ -15,8 +15,8 @@ func TestRunWithContextNoTimeout(t *testing.T) {
maxLoops := 10
// 'git --version' does not block so it must be finished before the timeout triggered.
cmd := NewCommand("--version")
for i := 0; i < maxLoops; i++ {
cmd := NewCommand("--version")
if err := cmd.Run(t.Context()); err != nil {
t.Fatal(err)
}
@ -27,8 +27,8 @@ func TestRunWithContextTimeout(t *testing.T) {
maxLoops := 10
// 'git hash-object --stdin' blocks on stdin so we can have the timeout triggered.
cmd := NewCommand("hash-object", "--stdin")
for i := 0; i < maxLoops; i++ {
cmd := NewCommand("hash-object", "--stdin")
if err := cmd.WithTimeout(1 * time.Millisecond).Run(t.Context()); err != nil {
if err != context.DeadlineExceeded {
t.Fatalf("Testing %d/%d: %v", i, maxLoops, err)

View File

@ -5,6 +5,7 @@ package gitcmd
import (
"fmt"
"io"
"code.gitea.io/gitea/modules/util"
)
@ -18,3 +19,30 @@ func ConcatenateError(err error, stderr string) error {
errMsg := fmt.Sprintf("%s - %s", err.Error(), stderr)
return util.ErrorWrap(&runStdError{err: err, stderr: stderr, errMsg: errMsg}, "%s", errMsg)
}
func safeClosePtrCloser[T *io.ReadCloser | *io.WriteCloser](c T) {
switch v := any(c).(type) {
case *io.ReadCloser:
if v != nil && *v != nil {
_ = (*v).Close()
}
case *io.WriteCloser:
if v != nil && *v != nil {
_ = (*v).Close()
}
default:
panic("unsupported type")
}
}
func safeAssignPipe[T any](p *T, fn func() (T, error)) (bool, error) {
if p == nil {
return false, nil
}
v, err := fn()
if err != nil {
return false, err
}
*p = v
return true, nil
}

View File

@ -67,16 +67,6 @@ func (repo *Repository) ConvertToGitID(commitID string) (ObjectID, error) {
return NewIDFromString(actualCommitID)
}
// IsCommitExist returns true if given commit exists in current repository.
func (repo *Repository) IsCommitExist(name string) bool {
hash, err := repo.ConvertToGitID(name)
if err != nil {
return false
}
_, err = repo.gogitRepo.CommitObject(plumbing.Hash(hash.RawValue()))
return err == nil
}
func (repo *Repository) getCommit(id ObjectID) (*Commit, error) {
var tagObject *object.Tag

View File

@ -44,23 +44,12 @@ func (repo *Repository) GetRefCommitID(name string) (string, error) {
info, err := batch.QueryInfo(name)
if IsErrNotExist(err) {
return "", ErrNotExist{name, ""}
} else if err != nil {
return "", err
}
return info.ID, nil
}
// IsCommitExist returns true if given commit exists in current repository.
func (repo *Repository) IsCommitExist(name string) bool {
if err := ensureValidGitRepository(repo.Ctx, repo.Path); err != nil {
log.Error("IsCommitExist: %v", err)
return false
}
_, _, err := gitcmd.NewCommand("cat-file", "-e").
AddDynamicArguments(name).
WithDir(repo.Path).
RunStdString(repo.Ctx)
return err == nil
}
func (repo *Repository) getCommit(id ObjectID) (*Commit, error) {
batch, cancel, err := repo.CatFileBatch(repo.Ctx)
if err != nil {