diff --git a/modules/git/catfile_batch_command.go b/modules/git/catfile_batch_command.go index ae3fb9e129..f5f0110195 100644 --- a/modules/git/catfile_batch_command.go +++ b/modules/git/catfile_batch_command.go @@ -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 } diff --git a/modules/git/catfile_batch_legacy.go b/modules/git/catfile_batch_legacy.go index 714ef022c3..07b875778a 100644 --- a/modules/git/catfile_batch_legacy.go +++ b/modules/git/catfile_batch_legacy.go @@ -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 } diff --git a/modules/git/catfile_batch_reader.go b/modules/git/catfile_batch_reader.go index 9ad49c98d9..2f65f522c5 100644 --- a/modules/git/catfile_batch_reader.go +++ b/modules/git/catfile_batch_reader.go @@ -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, } } diff --git a/modules/git/catfile_batch_test.go b/modules/git/catfile_batch_test.go index 2f5ade2b4e..355e8865e5 100644 --- a/modules/git/catfile_batch_test.go +++ b/modules/git/catfile_batch_test.go @@ -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() + }) } diff --git a/modules/git/gitcmd/command.go b/modules/git/gitcmd/command.go index 11dd46f472..f394ab7103 100644 --- a/modules/git/gitcmd/command.go +++ b/modules/git/gitcmd/command.go @@ -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() +} diff --git a/modules/git/gitcmd/command_race_test.go b/modules/git/gitcmd/command_race_test.go index c2f0b124a2..962470b6db 100644 --- a/modules/git/gitcmd/command_race_test.go +++ b/modules/git/gitcmd/command_race_test.go @@ -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) diff --git a/modules/git/gitcmd/utils.go b/modules/git/gitcmd/utils.go index 74d9d89e41..5c7546d7be 100644 --- a/modules/git/gitcmd/utils.go +++ b/modules/git/gitcmd/utils.go @@ -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 +} diff --git a/modules/git/repo_commit_gogit.go b/modules/git/repo_commit_gogit.go index c84aabde1a..550d153722 100644 --- a/modules/git/repo_commit_gogit.go +++ b/modules/git/repo_commit_gogit.go @@ -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 diff --git a/modules/git/repo_commit_nogogit.go b/modules/git/repo_commit_nogogit.go index 0147c82a01..2ddb527502 100644 --- a/modules/git/repo_commit_nogogit.go +++ b/modules/git/repo_commit_nogogit.go @@ -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 {