summaryrefslogtreecommitdiffstats
path: root/modules/git
diff options
context:
space:
mode:
authorzeripath <art27@cantab.net>2021-06-20 23:00:46 +0100
committerGitHub <noreply@github.com>2021-06-21 01:00:46 +0300
commit23358bc55de67be132e3858a5d40f25dbdd0a769 (patch)
tree914386734d5dafc0bcf84253c55c67d6590092dc /modules/git
parent8fa3bbc42450fe34cc0cee3de566b17fa131d1c6 (diff)
downloadgitea-23358bc55de67be132e3858a5d40f25dbdd0a769.tar.gz
gitea-23358bc55de67be132e3858a5d40f25dbdd0a769.zip
Use git log name-status in get last commit (#16059)
* Improve get last commit using git log --name-status git log --name-status -c provides information about the diff between a commit and its parents. Using this and adjusting the algorithm to use the first change to a path allows for a much faster generation of commit info. There is a subtle change in the results generated but this will cause the results to more closely match those from elsewhere. Signed-off-by: Andrew Thornton <art27@cantab.net> Co-authored-by: 6543 <6543@obermui.de> Co-authored-by: techknowlogick <techknowlogick@gitea.io> Co-authored-by: Lauris BH <lauris@nix.lv>
Diffstat (limited to 'modules/git')
-rw-r--r--modules/git/batch_reader.go111
-rw-r--r--modules/git/commit_info_nogogit.go237
-rw-r--r--modules/git/last_commit_cache_nogogit.go5
-rw-r--r--modules/git/log_name_status.go398
-rw-r--r--modules/git/notes_nogogit.go2
-rw-r--r--modules/git/pipeline/lfs_nogogit.go6
-rw-r--r--modules/git/repo_language_stats_nogogit.go3
7 files changed, 467 insertions, 295 deletions
diff --git a/modules/git/batch_reader.go b/modules/git/batch_reader.go
index d6ee0ce8e0..678b184708 100644
--- a/modules/git/batch_reader.go
+++ b/modules/git/batch_reader.go
@@ -11,6 +11,9 @@ import (
"math"
"strconv"
"strings"
+
+ "github.com/djherbis/buffer"
+ "github.com/djherbis/nio/v3"
)
// WriteCloserError wraps an io.WriteCloser with an additional CloseWithError function
@@ -42,7 +45,7 @@ func CatFileBatchCheck(repoPath string) (WriteCloserError, *bufio.Reader, func()
}
}()
- // For simplicities sake we'll us a buffered reader to read from the cat-file --batch
+ // For simplicities sake we'll use a buffered reader to read from the cat-file --batch-check
batchReader := bufio.NewReader(batchStdoutReader)
return batchStdinWriter, batchReader, cancel
@@ -53,7 +56,7 @@ func CatFileBatch(repoPath string) (WriteCloserError, *bufio.Reader, func()) {
// We often want to feed the commits in order into cat-file --batch, followed by their trees and sub trees as necessary.
// so let's create a batch stdin and stdout
batchStdinReader, batchStdinWriter := io.Pipe()
- batchStdoutReader, batchStdoutWriter := io.Pipe()
+ batchStdoutReader, batchStdoutWriter := nio.Pipe(buffer.New(32 * 1024))
cancel := func() {
_ = batchStdinReader.Close()
_ = batchStdinWriter.Close()
@@ -74,7 +77,7 @@ func CatFileBatch(repoPath string) (WriteCloserError, *bufio.Reader, func()) {
}()
// For simplicities sake we'll us a buffered reader to read from the cat-file --batch
- batchReader := bufio.NewReader(batchStdoutReader)
+ batchReader := bufio.NewReaderSize(batchStdoutReader, 32*1024)
return batchStdinWriter, batchReader, cancel
}
@@ -84,22 +87,31 @@ func CatFileBatch(repoPath string) (WriteCloserError, *bufio.Reader, func()) {
// <sha> SP <type> SP <size> LF
// sha is a 40byte not 20byte here
func ReadBatchLine(rd *bufio.Reader) (sha []byte, typ string, size int64, err error) {
- sha, err = rd.ReadBytes(' ')
+ typ, err = rd.ReadString('\n')
if err != nil {
return
}
- sha = sha[:len(sha)-1]
-
- typ, err = rd.ReadString('\n')
- if err != nil {
+ if len(typ) == 1 {
+ typ, err = rd.ReadString('\n')
+ if err != nil {
+ return
+ }
+ }
+ idx := strings.IndexByte(typ, ' ')
+ if idx < 0 {
+ log("missing space typ: %s", typ)
+ err = ErrNotExist{ID: string(sha)}
return
}
+ sha = []byte(typ[:idx])
+ typ = typ[idx+1:]
- idx := strings.Index(typ, " ")
+ idx = strings.IndexByte(typ, ' ')
if idx < 0 {
err = ErrNotExist{ID: string(sha)}
return
}
+
sizeStr := typ[idx+1 : len(typ)-1]
typ = typ[:idx]
@@ -130,7 +142,7 @@ headerLoop:
}
// Discard the rest of the tag
- discard := size - n
+ discard := size - n + 1
for discard > math.MaxInt32 {
_, err := rd.Discard(math.MaxInt32)
if err != nil {
@@ -200,85 +212,42 @@ func To40ByteSHA(sha, out []byte) []byte {
return out
}
-// ParseTreeLineSkipMode reads an entry from a tree in a cat-file --batch stream
-// This simply skips the mode - saving a substantial amount of time and carefully avoids allocations - except where fnameBuf is too small.
+// ParseTreeLine reads an entry from a tree in a cat-file --batch stream
+// This carefully avoids allocations - except where fnameBuf is too small.
// It is recommended therefore to pass in an fnameBuf large enough to avoid almost all allocations
//
// Each line is composed of:
// <mode-in-ascii-dropping-initial-zeros> SP <fname> NUL <20-byte SHA>
//
// We don't attempt to convert the 20-byte SHA to 40-byte SHA to save a lot of time
-func ParseTreeLineSkipMode(rd *bufio.Reader, fnameBuf, shaBuf []byte) (fname, sha []byte, n int, err error) {
+func ParseTreeLine(rd *bufio.Reader, modeBuf, fnameBuf, shaBuf []byte) (mode, fname, sha []byte, n int, err error) {
var readBytes []byte
- // Skip the Mode
- readBytes, err = rd.ReadSlice(' ') // NB: DOES NOT ALLOCATE SIMPLY RETURNS SLICE WITHIN READER BUFFER
- if err != nil {
- return
- }
- n += len(readBytes)
- // Deal with the fname
+ // Read the Mode & fname
readBytes, err = rd.ReadSlice('\x00')
- copy(fnameBuf, readBytes)
- if len(fnameBuf) > len(readBytes) {
- fnameBuf = fnameBuf[:len(readBytes)] // cut the buf the correct size
- } else {
- fnameBuf = append(fnameBuf, readBytes[len(fnameBuf):]...) // extend the buf and copy in the missing bits
- }
- for err == bufio.ErrBufferFull { // Then we need to read more
- readBytes, err = rd.ReadSlice('\x00')
- fnameBuf = append(fnameBuf, readBytes...) // there is little point attempting to avoid allocations here so just extend
- }
- n += len(fnameBuf)
if err != nil {
return
}
- fnameBuf = fnameBuf[:len(fnameBuf)-1] // Drop the terminal NUL
- fname = fnameBuf // set the returnable fname to the slice
-
- // Now deal with the 20-byte SHA
- idx := 0
- for idx < 20 {
- read := 0
- read, err = rd.Read(shaBuf[idx:20])
- n += read
- if err != nil {
- return
- }
- idx += read
- }
- sha = shaBuf
- return
-}
-
-// ParseTreeLine reads an entry from a tree in a cat-file --batch stream
-// This carefully avoids allocations - except where fnameBuf is too small.
-// It is recommended therefore to pass in an fnameBuf large enough to avoid almost all allocations
-//
-// Each line is composed of:
-// <mode-in-ascii-dropping-initial-zeros> SP <fname> NUL <20-byte SHA>
-//
-// We don't attempt to convert the 20-byte SHA to 40-byte SHA to save a lot of time
-func ParseTreeLine(rd *bufio.Reader, modeBuf, fnameBuf, shaBuf []byte) (mode, fname, sha []byte, n int, err error) {
- var readBytes []byte
+ idx := bytes.IndexByte(readBytes, ' ')
+ if idx < 0 {
+ log("missing space in readBytes ParseTreeLine: %s", readBytes)
- // Read the Mode
- readBytes, err = rd.ReadSlice(' ')
- if err != nil {
+ err = &ErrNotExist{}
return
}
- n += len(readBytes)
- copy(modeBuf, readBytes)
- if len(modeBuf) > len(readBytes) {
- modeBuf = modeBuf[:len(readBytes)]
- } else {
- modeBuf = append(modeBuf, readBytes[len(modeBuf):]...)
+ n += idx + 1
+ copy(modeBuf, readBytes[:idx])
+ if len(modeBuf) >= idx {
+ modeBuf = modeBuf[:idx]
+ } else {
+ modeBuf = append(modeBuf, readBytes[len(modeBuf):idx]...)
}
- mode = modeBuf[:len(modeBuf)-1] // Drop the SP
+ mode = modeBuf
+
+ readBytes = readBytes[idx+1:]
// Deal with the fname
- readBytes, err = rd.ReadSlice('\x00')
copy(fnameBuf, readBytes)
if len(fnameBuf) > len(readBytes) {
fnameBuf = fnameBuf[:len(readBytes)]
@@ -297,7 +266,7 @@ func ParseTreeLine(rd *bufio.Reader, modeBuf, fnameBuf, shaBuf []byte) (mode, fn
fname = fnameBuf
// Deal with the 20-byte SHA
- idx := 0
+ idx = 0
for idx < 20 {
read := 0
read, err = rd.Read(shaBuf[idx:20])
diff --git a/modules/git/commit_info_nogogit.go b/modules/git/commit_info_nogogit.go
index f34bef9f01..2283510d96 100644
--- a/modules/git/commit_info_nogogit.go
+++ b/modules/git/commit_info_nogogit.go
@@ -7,15 +7,11 @@
package git
import (
- "bufio"
- "bytes"
"context"
"fmt"
"io"
- "math"
"path"
"sort"
- "strings"
)
// GetCommitsInfo gets information of all commits that are corresponding to these entries
@@ -43,21 +39,16 @@ func (tes Entries) GetCommitsInfo(ctx context.Context, commit *Commit, treePath
return nil, nil, err
}
- for i, found := range commits {
- if err := cache.Put(commit.ID.String(), path.Join(treePath, unHitPaths[i]), found.ID.String()); err != nil {
+ for pth, found := range commits {
+ if err := cache.Put(commit.ID.String(), path.Join(treePath, pth), found.ID.String()); err != nil {
return nil, nil, err
}
- revs[unHitPaths[i]] = found
+ revs[pth] = found
}
}
} else {
sort.Strings(entryPaths)
- revs = map[string]*Commit{}
- var foundCommits []*Commit
- foundCommits, err = GetLastCommitForPaths(ctx, commit, treePath, entryPaths)
- for i, found := range foundCommits {
- revs[entryPaths[i]] = found
- }
+ revs, err = GetLastCommitForPaths(ctx, commit, treePath, entryPaths)
}
if err != nil {
return nil, nil, err
@@ -86,6 +77,8 @@ func (tes Entries) GetCommitsInfo(ctx context.Context, commit *Commit, treePath
subModuleFile := NewSubModuleFile(entryCommit, subModuleURL, entry.ID.String())
commitsInfo[i].SubModuleFile = subModuleFile
}
+ } else {
+ log("missing commit for %s", entry.Name())
}
}
@@ -125,220 +118,24 @@ func getLastCommitForPathsByCache(ctx context.Context, commitID, treePath string
}
// GetLastCommitForPaths returns last commit information
-func GetLastCommitForPaths(ctx context.Context, commit *Commit, treePath string, paths []string) ([]*Commit, error) {
+func GetLastCommitForPaths(ctx context.Context, commit *Commit, treePath string, paths []string) (map[string]*Commit, error) {
// We read backwards from the commit to obtain all of the commits
-
- // We'll do this by using rev-list to provide us with parent commits in order
- revListReader, revListWriter := io.Pipe()
- defer func() {
- _ = revListWriter.Close()
- _ = revListReader.Close()
- }()
-
- go func() {
- stderr := strings.Builder{}
- err := NewCommand("rev-list", "--format=%T", commit.ID.String()).SetParentContext(ctx).RunInDirPipeline(commit.repo.Path, revListWriter, &stderr)
- if err != nil {
- _ = revListWriter.CloseWithError(ConcatenateError(err, (&stderr).String()))
- } else {
- _ = revListWriter.Close()
- }
- }()
+ revs, err := WalkGitLog(ctx, commit.repo, commit, treePath, paths...)
+ if err != nil {
+ return nil, err
+ }
batchStdinWriter, batchReader, cancel := commit.repo.CatFileBatch()
defer cancel()
- mapsize := 4096
- if len(paths) > mapsize {
- mapsize = len(paths)
- }
-
- path2idx := make(map[string]int, mapsize)
- for i, path := range paths {
- path2idx[path] = i
- }
-
- fnameBuf := make([]byte, 4096)
- modeBuf := make([]byte, 40)
-
- allShaBuf := make([]byte, (len(paths)+1)*20)
- shaBuf := make([]byte, 20)
- tmpTreeID := make([]byte, 40)
-
- // commits is the returnable commits matching the paths provided
- commits := make([]string, len(paths))
- // ids are the blob/tree ids for the paths
- ids := make([][]byte, len(paths))
-
- // We'll use a scanner for the revList because it's simpler than a bufio.Reader
- scan := bufio.NewScanner(revListReader)
-revListLoop:
- for scan.Scan() {
- // Get the next parent commit ID
- commitID := scan.Text()
- if !scan.Scan() {
- break revListLoop
- }
- commitID = commitID[7:]
- rootTreeID := scan.Text()
-
- // push the tree to the cat-file --batch process
- _, err := batchStdinWriter.Write([]byte(rootTreeID + "\n"))
- if err != nil {
- return nil, err
- }
-
- currentPath := ""
-
- // OK if the target tree path is "" and the "" is in the paths just set this now
- if treePath == "" && paths[0] == "" {
- // If this is the first time we see this set the id appropriate for this paths to this tree and set the last commit to curCommit
- if len(ids[0]) == 0 {
- ids[0] = []byte(rootTreeID)
- commits[0] = string(commitID)
- } else if bytes.Equal(ids[0], []byte(rootTreeID)) {
- commits[0] = string(commitID)
- }
- }
-
- treeReadingLoop:
- for {
- select {
- case <-ctx.Done():
- return nil, ctx.Err()
- default:
- }
- _, _, size, err := ReadBatchLine(batchReader)
- if err != nil {
- return nil, err
- }
-
- // Handle trees
-
- // n is counter for file position in the tree file
- var n int64
-
- // Two options: currentPath is the targetTreepath
- if treePath == currentPath {
- // We are in the right directory
- // Parse each tree line in turn. (don't care about mode here.)
- for n < size {
- fname, sha, count, err := ParseTreeLineSkipMode(batchReader, fnameBuf, shaBuf)
- shaBuf = sha
- if err != nil {
- return nil, err
- }
- n += int64(count)
- idx, ok := path2idx[string(fname)]
- if ok {
- // Now if this is the first time round set the initial Blob(ish) SHA ID and the commit
- if len(ids[idx]) == 0 {
- copy(allShaBuf[20*(idx+1):20*(idx+2)], shaBuf)
- ids[idx] = allShaBuf[20*(idx+1) : 20*(idx+2)]
- commits[idx] = string(commitID)
- } else if bytes.Equal(ids[idx], shaBuf) {
- commits[idx] = string(commitID)
- }
- }
- // FIXME: is there any order to the way strings are emitted from cat-file?
- // if there is - then we could skip once we've passed all of our data
- }
- if _, err := batchReader.Discard(1); err != nil {
- return nil, err
- }
-
- break treeReadingLoop
- }
-
- var treeID []byte
-
- // We're in the wrong directory
- // Find target directory in this directory
- idx := len(currentPath)
- if idx > 0 {
- idx++
- }
- target := strings.SplitN(treePath[idx:], "/", 2)[0]
-
- for n < size {
- // Read each tree entry in turn
- mode, fname, sha, count, err := ParseTreeLine(batchReader, modeBuf, fnameBuf, shaBuf)
- if err != nil {
- return nil, err
- }
- n += int64(count)
-
- // if we have found the target directory
- if bytes.Equal(fname, []byte(target)) && bytes.Equal(mode, []byte("40000")) {
- copy(tmpTreeID, sha)
- treeID = tmpTreeID
- break
- }
- }
-
- if n < size {
- // Discard any remaining entries in the current tree
- discard := size - n
- for discard > math.MaxInt32 {
- _, err := batchReader.Discard(math.MaxInt32)
- if err != nil {
- return nil, err
- }
- discard -= math.MaxInt32
- }
- _, err := batchReader.Discard(int(discard))
- if err != nil {
- return nil, err
- }
- }
- if _, err := batchReader.Discard(1); err != nil {
- return nil, err
- }
-
- // if we haven't found a treeID for the target directory our search is over
- if len(treeID) == 0 {
- break treeReadingLoop
- }
-
- // add the target to the current path
- if idx > 0 {
- currentPath += "/"
- }
- currentPath += target
-
- // if we've now found the current path check its sha id and commit status
- if treePath == currentPath && paths[0] == "" {
- if len(ids[0]) == 0 {
- copy(allShaBuf[0:20], treeID)
- ids[0] = allShaBuf[0:20]
- commits[0] = string(commitID)
- } else if bytes.Equal(ids[0], treeID) {
- commits[0] = string(commitID)
- }
- }
- treeID = To40ByteSHA(treeID, treeID)
- _, err = batchStdinWriter.Write(treeID)
- if err != nil {
- return nil, err
- }
- _, err = batchStdinWriter.Write([]byte("\n"))
- if err != nil {
- return nil, err
- }
- }
- }
- if scan.Err() != nil {
- return nil, scan.Err()
- }
-
- commitsMap := make(map[string]*Commit, len(commits))
+ commitsMap := map[string]*Commit{}
commitsMap[commit.ID.String()] = commit
- commitCommits := make([]*Commit, len(commits))
- for i, commitID := range commits {
+ commitCommits := map[string]*Commit{}
+ for path, commitID := range revs {
c, ok := commitsMap[commitID]
if ok {
- commitCommits[i] = c
+ commitCommits[path] = c
continue
}
@@ -364,8 +161,8 @@ revListLoop:
if _, err := batchReader.Discard(1); err != nil {
return nil, err
}
- commitCommits[i] = c
+ commitCommits[path] = c
}
- return commitCommits, scan.Err()
+ return commitCommits, nil
}
diff --git a/modules/git/last_commit_cache_nogogit.go b/modules/git/last_commit_cache_nogogit.go
index 3cbb0cca32..84c8ee132c 100644
--- a/modules/git/last_commit_cache_nogogit.go
+++ b/modules/git/last_commit_cache_nogogit.go
@@ -88,9 +88,8 @@ func (c *LastCommitCache) recursiveCache(ctx context.Context, commit *Commit, tr
return err
}
- for i, entryCommit := range commits {
- entry := entryPaths[i]
- if err := c.Put(commit.ID.String(), path.Join(treePath, entryPaths[i]), entryCommit.ID.String()); err != nil {
+ for entry, entryCommit := range commits {
+ if err := c.Put(commit.ID.String(), path.Join(treePath, entry), entryCommit.ID.String()); err != nil {
return err
}
if entryMap[entry].IsDir() {
diff --git a/modules/git/log_name_status.go b/modules/git/log_name_status.go
new file mode 100644
index 0000000000..803d614d61
--- /dev/null
+++ b/modules/git/log_name_status.go
@@ -0,0 +1,398 @@
+// Copyright 2021 The Gitea Authors. All rights reserved.
+// Use of this source code is governed by a MIT-style
+// license that can be found in the LICENSE file.
+
+package git
+
+import (
+ "bufio"
+ "bytes"
+ "context"
+ "io"
+ "path"
+ "sort"
+ "strings"
+
+ "github.com/djherbis/buffer"
+ "github.com/djherbis/nio/v3"
+)
+
+// LogNameStatusRepo opens git log --raw in the provided repo and returns a stdin pipe, a stdout reader and cancel function
+func LogNameStatusRepo(repository, head, treepath string, paths ...string) (*bufio.Reader, func()) {
+ // We often want to feed the commits in order into cat-file --batch, followed by their trees and sub trees as necessary.
+ // so let's create a batch stdin and stdout
+ stdoutReader, stdoutWriter := nio.Pipe(buffer.New(32 * 1024))
+ cancel := func() {
+ _ = stdoutReader.Close()
+ _ = stdoutWriter.Close()
+ }
+
+ args := make([]string, 0, 8+len(paths))
+ args = append(args, "log", "--name-status", "-c", "--format=commit%x00%H %P%x00", "--parents", "--no-renames", "-t", "-z", head, "--")
+ if len(paths) < 70 {
+ if treepath != "" {
+ args = append(args, treepath)
+ for _, pth := range paths {
+ if pth != "" {
+ args = append(args, path.Join(treepath, pth))
+ }
+ }
+ } else {
+ for _, pth := range paths {
+ if pth != "" {
+ args = append(args, pth)
+ }
+ }
+ }
+ } else if treepath != "" {
+ args = append(args, treepath)
+ }
+
+ go func() {
+ stderr := strings.Builder{}
+ err := NewCommand(args...).RunInDirFullPipeline(repository, stdoutWriter, &stderr, nil)
+ if err != nil {
+ _ = stdoutWriter.CloseWithError(ConcatenateError(err, (&stderr).String()))
+ } else {
+ _ = stdoutWriter.Close()
+ }
+ }()
+
+ // For simplicities sake we'll us a buffered reader to read from the cat-file --batch
+ bufReader := bufio.NewReaderSize(stdoutReader, 32*1024)
+
+ return bufReader, cancel
+}
+
+// LogNameStatusRepoParser parses a git log raw output from LogRawRepo
+type LogNameStatusRepoParser struct {
+ treepath string
+ paths []string
+ next []byte
+ buffull bool
+ rd *bufio.Reader
+ cancel func()
+}
+
+// NewLogNameStatusRepoParser returns a new parser for a git log raw output
+func NewLogNameStatusRepoParser(repository, head, treepath string, paths ...string) *LogNameStatusRepoParser {
+ rd, cancel := LogNameStatusRepo(repository, head, treepath, paths...)
+ return &LogNameStatusRepoParser{
+ treepath: treepath,
+ paths: paths,
+ rd: rd,
+ cancel: cancel,
+ }
+}
+
+// LogNameStatusCommitData represents a commit artefact from git log raw
+type LogNameStatusCommitData struct {
+ CommitID string
+ ParentIDs []string
+ Paths []bool
+}
+
+// Next returns the next LogStatusCommitData
+func (g *LogNameStatusRepoParser) Next(treepath string, paths2ids map[string]int, changed []bool, maxpathlen int) (*LogNameStatusCommitData, error) {
+ var err error
+ if g.next == nil || len(g.next) == 0 {
+ g.buffull = false
+ g.next, err = g.rd.ReadSlice('\x00')
+ if err != nil {
+ if err == bufio.ErrBufferFull {
+ g.buffull = true
+ } else if err == io.EOF {
+ return nil, nil
+ } else {
+ return nil, err
+ }
+ }
+ }
+
+ ret := LogNameStatusCommitData{}
+ if bytes.Equal(g.next, []byte("commit\000")) {
+ g.next, err = g.rd.ReadSlice('\x00')
+ if err != nil {
+ if err == bufio.ErrBufferFull {
+ g.buffull = true
+ } else if err == io.EOF {
+ return nil, nil
+ } else {
+ return nil, err
+ }
+ }
+ }
+
+ // Our "line" must look like: <commitid> SP (<parent> SP) * NUL
+ ret.CommitID = string(g.next[0:40])
+ parents := string(g.next[41:])
+ if g.buffull {
+ more, err := g.rd.ReadString('\x00')
+ if err != nil {
+ return nil, err
+ }
+ parents += more
+ }
+ parents = parents[:len(parents)-1]
+ ret.ParentIDs = strings.Split(parents, " ")
+
+ // now read the next "line"
+ g.buffull = false
+ g.next, err = g.rd.ReadSlice('\x00')
+ if err != nil {
+ if err == bufio.ErrBufferFull {
+ g.buffull = true
+ } else if err != io.EOF {
+ return nil, err
+ }
+ }
+
+ if err == io.EOF || !(g.next[0] == '\n' || g.next[0] == '\000') {
+ return &ret, nil
+ }
+
+ // Ok we have some changes.
+ // This line will look like: NL <fname> NUL
+ //
+ // Subsequent lines will not have the NL - so drop it here - g.bufffull must also be false at this point too.
+ if g.next[0] == '\n' {
+ g.next = g.next[1:]
+ } else {
+ g.buffull = false
+ g.next, err = g.rd.ReadSlice('\x00')
+ if err != nil {
+ if err == bufio.ErrBufferFull {
+ g.buffull = true
+ } else if err != io.EOF {
+ return nil, err
+ }
+ }
+ if g.next[0] == '\x00' {
+ g.buffull = false
+ g.next, err = g.rd.ReadSlice('\x00')
+ if err != nil {
+ if err == bufio.ErrBufferFull {
+ g.buffull = true
+ } else if err != io.EOF {
+ return nil, err
+ }
+ }
+ }
+ }
+
+ fnameBuf := make([]byte, 4096)
+
+diffloop:
+ for {
+ if err == io.EOF || bytes.Equal(g.next, []byte("commit\000")) {
+ return &ret, nil
+ }
+ g.next, err = g.rd.ReadSlice('\x00')
+ if err != nil {
+ if err == bufio.ErrBufferFull {
+ g.buffull = true
+ } else if err == io.EOF {
+ return &ret, nil
+ } else {
+ return nil, err
+ }
+ }
+ copy(fnameBuf, g.next)
+ if len(fnameBuf) < len(g.next) {
+ fnameBuf = append(fnameBuf, g.next[len(fnameBuf):]...)
+ } else {
+ fnameBuf = fnameBuf[:len(g.next)]
+ }
+ if err != nil {
+ if err != bufio.ErrBufferFull {
+ return nil, err
+ }
+ more, err := g.rd.ReadBytes('\x00')
+ if err != nil {
+ return nil, err
+ }
+ fnameBuf = append(fnameBuf, more...)
+ }
+
+ // read the next line
+ g.buffull = false
+ g.next, err = g.rd.ReadSlice('\x00')
+ if err != nil {
+ if err == bufio.ErrBufferFull {
+ g.buffull = true
+ } else if err != io.EOF {
+ return nil, err
+ }
+ }
+
+ if treepath != "" {
+ if !bytes.HasPrefix(fnameBuf, []byte(treepath)) {
+ fnameBuf = fnameBuf[:cap(fnameBuf)]
+ continue diffloop
+ }
+ }
+ fnameBuf = fnameBuf[len(treepath) : len(fnameBuf)-1]
+ if len(fnameBuf) > maxpathlen {
+ fnameBuf = fnameBuf[:cap(fnameBuf)]
+ continue diffloop
+ }
+ if len(fnameBuf) > 0 {
+ if len(treepath) > 0 {
+ if fnameBuf[0] != '/' || bytes.IndexByte(fnameBuf[1:], '/') >= 0 {
+ fnameBuf = fnameBuf[:cap(fnameBuf)]
+ continue diffloop
+ }
+ fnameBuf = fnameBuf[1:]
+ } else if bytes.IndexByte(fnameBuf, '/') >= 0 {
+ fnameBuf = fnameBuf[:cap(fnameBuf)]
+ continue diffloop
+ }
+ }
+
+ idx, ok := paths2ids[string(fnameBuf)]
+ if !ok {
+ fnameBuf = fnameBuf[:cap(fnameBuf)]
+ continue diffloop
+ }
+ if ret.Paths == nil {
+ ret.Paths = changed
+ }
+ changed[idx] = true
+ }
+}
+
+// Close closes the parser
+func (g *LogNameStatusRepoParser) Close() {
+ g.cancel()
+}
+
+// WalkGitLog walks the git log --name-status for the head commit in the provided treepath and files
+func WalkGitLog(ctx context.Context, repo *Repository, head *Commit, treepath string, paths ...string) (map[string]string, error) {
+ tree, err := head.SubTree(treepath)
+ if err != nil {
+ return nil, err
+ }
+
+ entries, err := tree.ListEntries()
+ if err != nil {
+ return nil, err
+ }
+
+ if len(paths) == 0 {
+ paths = make([]string, 0, len(entries)+1)
+ paths = append(paths, "")
+ for _, entry := range entries {
+ paths = append(paths, entry.Name())
+ }
+ } else {
+ sort.Strings(paths)
+ if paths[0] != "" {
+ paths = append([]string{""}, paths...)
+ }
+ // remove duplicates
+ for i := len(paths) - 1; i > 0; i-- {
+ if paths[i] == paths[i-1] {
+ paths = append(paths[:i-1], paths[i:]...)
+ }
+ }
+ }
+
+ path2idx := map[string]int{}
+ maxpathlen := len(treepath)
+
+ for i := range paths {
+ path2idx[paths[i]] = i
+ pthlen := len(paths[i]) + len(treepath) + 1
+ if pthlen > maxpathlen {
+ maxpathlen = pthlen
+ }
+ }
+
+ g := NewLogNameStatusRepoParser(repo.Path, head.ID.String(), treepath, paths...)
+ defer g.Close()
+
+ results := make([]string, len(paths))
+ remaining := len(paths)
+ nextRestart := (len(paths) * 3) / 4
+ if nextRestart > 70 {
+ nextRestart = 70
+ }
+ lastEmptyParent := head.ID.String()
+ commitSinceLastEmptyParent := uint64(0)
+ commitSinceNextRestart := uint64(0)
+ parentRemaining := map[string]bool{}
+
+ changed := make([]bool, len(paths))
+
+heaploop:
+ for {
+ select {
+ case <-ctx.Done():
+ return nil, ctx.Err()
+ default:
+ }
+ current, err := g.Next(treepath, path2idx, changed, maxpathlen)
+ if err != nil {
+ g.Close()
+ return nil, err
+ }
+ if current == nil {
+ break heaploop
+ }
+ delete(parentRemaining, current.CommitID)
+ if current.Paths != nil {
+ for i, found := range current.Paths {
+ if !found {
+ continue
+ }
+ changed[i] = false
+ if results[i] == "" {
+ results[i] = current.CommitID
+ delete(path2idx, paths[i])
+ remaining--
+ if results[0] == "" {
+ results[0] = current.CommitID
+ delete(path2idx, "")
+ remaining--
+ }
+ }
+ }
+ }
+
+ if remaining <= 0 {
+ break heaploop
+ }
+ commitSinceLastEmptyParent++
+ if len(parentRemaining) == 0 {
+ lastEmptyParent = current.CommitID
+ commitSinceLastEmptyParent = 0
+ }
+ if remaining <= nextRestart {
+ commitSinceNextRestart++
+ if 4*commitSinceNextRestart > 3*commitSinceLastEmptyParent {
+ g.Close()
+ remainingPaths := make([]string, 0, len(paths))
+ for i, pth := range paths {
+ if results[i] == "" {
+ remainingPaths = append(remainingPaths, pth)
+ }
+ }
+ g = NewLogNameStatusRepoParser(repo.Path, lastEmptyParent, treepath, remainingPaths...)
+ parentRemaining = map[string]bool{}
+ nextRestart = (remaining * 3) / 4
+ continue heaploop
+ }
+ }
+ for _, parent := range current.ParentIDs {
+ parentRemaining[parent] = true
+ }
+ }
+ g.Close()
+
+ resultsMap := map[string]string{}
+ for i, pth := range paths {
+ resultsMap[pth] = results[i]
+ }
+
+ return resultsMap, nil
+}
diff --git a/modules/git/notes_nogogit.go b/modules/git/notes_nogogit.go
index 2b92724995..267087a86f 100644
--- a/modules/git/notes_nogogit.go
+++ b/modules/git/notes_nogogit.go
@@ -68,7 +68,7 @@ func GetNote(ctx context.Context, repo *Repository, commitID string, note *Note)
if err != nil {
return err
}
- note.Commit = lastCommits[0]
+ note.Commit = lastCommits[path]
return nil
}
diff --git a/modules/git/pipeline/lfs_nogogit.go b/modules/git/pipeline/lfs_nogogit.go
index e618dd04b7..d3696fcda2 100644
--- a/modules/git/pipeline/lfs_nogogit.go
+++ b/modules/git/pipeline/lfs_nogogit.go
@@ -116,6 +116,9 @@ func FindLFSFile(repo *git.Repository, hash git.SHA1) ([]*LFSResult, error) {
if err != nil {
return nil, err
}
+ if _, err := batchReader.Discard(1); err != nil {
+ return nil, err
+ }
_, err := batchStdinWriter.Write([]byte(curCommit.Tree.ID.String() + "\n"))
if err != nil {
@@ -146,6 +149,9 @@ func FindLFSFile(repo *git.Repository, hash git.SHA1) ([]*LFSResult, error) {
paths = append(paths, curPath+string(fname)+"/")
}
}
+ if _, err := batchReader.Discard(1); err != nil {
+ return nil, err
+ }
if len(trees) > 0 {
_, err := batchStdinWriter.Write(trees[len(trees)-1])
if err != nil {
diff --git a/modules/git/repo_language_stats_nogogit.go b/modules/git/repo_language_stats_nogogit.go
index abbf5e943b..46b084cf01 100644
--- a/modules/git/repo_language_stats_nogogit.go
+++ b/modules/git/repo_language_stats_nogogit.go
@@ -49,6 +49,9 @@ func (repo *Repository) GetLanguageStats(commitID string) (map[string]int64, err
log("Unable to get commit for: %s. Err: %v", commitID, err)
return nil, err
}
+ if _, err = batchReader.Discard(1); err != nil {
+ return nil, err
+ }
tree := commit.Tree