internal/lsp/cache: add an LRU parse cache
As work proceeds on incremental type-checking, two observations have
emerged from benchmarking:
- Using a global FileSet is impossible, as IImportShallow allocates a
large number of new token.Files (in early experiments 75%+ of in-use memory
was consumed by the FileSet!)
- Several benchmarks regressed with incremental type-checking due to
re-parsing package files following a change. Ideally after a single file
changes we would be able to re-typecheck packages containing that file
after only re-parsing the single file that changed.
These observations are in tension: because type-checking requires that
parsed ast.Files live in the same token.FileSet as the type-checked
package, we cannot naively save the results of parsing and still use a
package-scoped FileSet.
This CL seeks to address both observations, by introducing a new
mechanism for caching parsed files (a parseCache) that parses files in a
standalone FileSet offset to avoid collision with other parsed files.
This cache exposes a batch API to parse multiple files and return a
FileSet describing all of them.
Benchmarking indicates that this partially mitigates performance
regressions without sacrificing the memory improvement we by avoiding a
global cache of parsed files.
In this CL the parse cache is not yet integrated with type-checking, but
replaces certain call-sites where we previously tried to avoid parsing
through the cache.
For golang/go#57987
Change-Id: I840cf003db835a40721f086abcc7bf00486b8108
Reviewed-on: https://go-review.googlesource.com/c/tools/+/469858
Reviewed-by: Alan Donovan <adonovan@google.com>
TryBot-Result: Gopher Robot <gobot@golang.org>
gopls-CI: kokoro <noreply+kokoro@google.com>
Run-TryBot: Robert Findley <rfindley@google.com>
diff --git a/gopls/internal/lsp/cache/check.go b/gopls/internal/lsp/cache/check.go
index edefa43..adf50b6 100644
--- a/gopls/internal/lsp/cache/check.go
+++ b/gopls/internal/lsp/cache/check.go
@@ -677,7 +677,7 @@
if err != nil {
return nil, err
}
- fset := source.SingletonFileSet(pgf.Tok)
+ fset := source.FileSetFor(pgf.Tok)
// TODO(adonovan): modify Imports() to accept a single token.File (cgf.Tok).
for _, group := range astutil.Imports(fset, pgf.File) {
for _, imp := range group {
diff --git a/gopls/internal/lsp/cache/mod_tidy.go b/gopls/internal/lsp/cache/mod_tidy.go
index def10d5..0572e9d 100644
--- a/gopls/internal/lsp/cache/mod_tidy.go
+++ b/gopls/internal/lsp/cache/mod_tidy.go
@@ -203,7 +203,11 @@
// If -mod=readonly is not set we may have successfully imported
// packages from missing modules. Otherwise they'll be in
// MissingDependencies. Combine both.
- for imp := range parseImports(ctx, snapshot, goFiles) {
+ imps, err := parseImports(ctx, snapshot, goFiles)
+ if err != nil {
+ return nil, err
+ }
+ for imp := range imps {
if req, ok := missing[imp]; ok {
missingImports[imp] = req
break
@@ -446,19 +450,20 @@
//
// (We can't simply use Metadata.Imports because it is based on
// CompiledGoFiles, after cgo processing.)
-func parseImports(ctx context.Context, s *snapshot, files []source.FileHandle) map[string]bool {
- s.mu.Lock() // peekOrParse requires a locked snapshot (!)
- defer s.mu.Unlock()
+//
+// TODO(rfindley): this should key off source.ImportPath.
+func parseImports(ctx context.Context, s *snapshot, files []source.FileHandle) (map[string]bool, error) {
+ pgfs, _, err := s.parseCache.parseFiles(ctx, source.ParseHeader, files...)
+ if err != nil { // e.g. context cancellation
+ return nil, err
+ }
+
seen := make(map[string]bool)
- for _, file := range files {
- f, err := peekOrParse(ctx, s, file, source.ParseHeader)
- if err != nil {
- continue
- }
- for _, spec := range f.File.Imports {
+ for _, pgf := range pgfs {
+ for _, spec := range pgf.File.Imports {
path, _ := strconv.Unquote(spec.Path.Value)
seen[path] = true
}
}
- return seen
+ return seen, nil
}
diff --git a/gopls/internal/lsp/cache/parse.go b/gopls/internal/lsp/cache/parse.go
index 1f90be2..f9bf7d7 100644
--- a/gopls/internal/lsp/cache/parse.go
+++ b/gopls/internal/lsp/cache/parse.go
@@ -21,18 +21,13 @@
"golang.org/x/tools/gopls/internal/lsp/protocol"
"golang.org/x/tools/gopls/internal/lsp/safetoken"
"golang.org/x/tools/gopls/internal/lsp/source"
+ "golang.org/x/tools/gopls/internal/span"
"golang.org/x/tools/internal/diff"
"golang.org/x/tools/internal/event"
"golang.org/x/tools/internal/event/tag"
"golang.org/x/tools/internal/memoize"
)
-// parseKey uniquely identifies a parsed Go file.
-type parseKey struct {
- file source.FileIdentity
- mode source.ParseMode
-}
-
// ParseGo parses the file whose contents are provided by fh, using a cache.
// The resulting tree may have be fixed up.
//
@@ -111,22 +106,6 @@
return res.parsed, res.err
}
-// peekParseGoLocked peeks at the cache used by ParseGo but does not
-// populate it or wait for other threads to do so. On cache hit, it returns
-// the cache result of parseGoImpl; otherwise it returns (nil, nil).
-func (s *snapshot) peekParseGoLocked(fh source.FileHandle, mode source.ParseMode) (*source.ParsedGoFile, error) {
- entry, hit := s.parsedGoFiles.Get(parseKey{fh.FileIdentity(), mode})
- if !hit {
- return nil, nil // no-one has requested this file
- }
- v := entry.(*memoize.Promise).Cached()
- if v == nil {
- return nil, nil // parsing is still in progress
- }
- res := v.(parseGoResult)
- return res.parsed, res.err
-}
-
// parseGoResult holds the result of a call to parseGoImpl.
type parseGoResult struct {
parsed *source.ParsedGoFile
@@ -146,13 +125,17 @@
if err != nil {
return nil, err
}
+ return parseGoSrc(ctx, fset, fh.URI(), src, mode), nil
+}
+// parseGoSrc parses a buffer of Go source, repairing the tree if necessary.
+func parseGoSrc(ctx context.Context, fset *token.FileSet, uri span.URI, src []byte, mode source.ParseMode) (res *source.ParsedGoFile) {
parserMode := parser.AllErrors | parser.ParseComments
if mode == source.ParseHeader {
parserMode = parser.ImportsOnly | parser.ParseComments
}
- file, err := parser.ParseFile(fset, fh.URI().Filename(), src, parserMode)
+ file, err := parser.ParseFile(fset, uri.Filename(), src, parserMode)
var parseErr scanner.ErrorList
if err != nil {
// We passed a byte slice, so the only possible error is a parse error.
@@ -164,7 +147,7 @@
// file.Pos is the location of the package declaration (issue #53202). If there was
// none, we can't find the token.File that ParseFile created, and we
// have no choice but to recreate it.
- tok = fset.AddFile(fh.URI().Filename(), -1, len(src))
+ tok = fset.AddFile(uri.Filename(), -1, len(src))
tok.SetLinesForContent(src)
}
@@ -189,7 +172,7 @@
event.Log(ctx, fmt.Sprintf("fixSrc loop - last diff:\n%v", unified), tag.File.Of(tok.Name()))
}
- newFile, _ := parser.ParseFile(fset, fh.URI().Filename(), newSrc, parserMode)
+ newFile, _ := parser.ParseFile(fset, uri.Filename(), newSrc, parserMode)
if newFile != nil {
// Maintain the original parseError so we don't try formatting the doctored file.
file = newFile
@@ -202,15 +185,15 @@
}
return &source.ParsedGoFile{
- URI: fh.URI(),
+ URI: uri,
Mode: mode,
Src: src,
Fixed: fixed,
File: file,
Tok: tok,
- Mapper: protocol.NewMapper(fh.URI(), src),
+ Mapper: protocol.NewMapper(uri, src),
ParseErr: parseErr,
- }, nil
+ }
}
// An unexportedFilter removes as much unexported AST from a set of Files as possible.
diff --git a/gopls/internal/lsp/cache/parse_cache.go b/gopls/internal/lsp/cache/parse_cache.go
new file mode 100644
index 0000000..91ca091
--- /dev/null
+++ b/gopls/internal/lsp/cache/parse_cache.go
@@ -0,0 +1,298 @@
+// Copyright 2023 The Go Authors. All rights reserved.
+// Use of this source code is governed by a BSD-style
+// license that can be found in the LICENSE file.
+
+package cache
+
+import (
+ "container/heap"
+ "context"
+ "go/token"
+ "runtime"
+ "sort"
+ "sync"
+
+ "golang.org/x/sync/errgroup"
+ "golang.org/x/tools/gopls/internal/lsp/source"
+ "golang.org/x/tools/internal/memoize"
+)
+
+// This file contains an implementation of a bounded-size parse cache, that
+// offsets the base token.Pos value of each cached file so that they may be
+// later described by a single dedicated FileSet.
+//
+// This is achieved by tracking a monotonic offset in the token.Pos space, that
+// is incremented before parsing allow room for the resulting parsed file.
+
+// Keep 200 recently parsed files, based on the following rationale:
+// - One of the most important benefits of caching is avoiding re-parsing
+// everything in a package when working on a single file. No packages in
+// Kubernetes have > 200 files (only one has > 100).
+// - Experience has shown that ~1000 parsed files can use noticeable space.
+// 200 feels like a sweet spot between limiting cache size and optimizing
+// cache hits for low-latency operations.
+const parseCacheMaxFiles = 200
+
+// parsePadding is additional padding allocated between entries in the parse
+// cache to allow for increases in length (such as appending missing braces)
+// caused by fixAST.
+//
+// This is used to mitigate a chicken and egg problem: we must know the base
+// offset of the file we're about to parse, before we start parsing, and yet
+// src fixups may affect the actual size of the parsed content (and therefore
+// the offsets of subsequent files).
+//
+// When we encounter a file that no longer fits in its allocated space in the
+// fileset, we have no choice but to re-parse it. Leaving a generous padding
+// reduces the likelihood of this "slow path".
+//
+// This value is mutable for testing, so that we can exercise the slow path.
+var parsePadding = 1000 // mutable for testing
+
+// A parseCache holds a bounded number of recently accessed parsed Go files. As
+// new files are stored, older files may be evicted from the cache.
+//
+// The parseCache.parseFiles method exposes a batch API for parsing (and
+// caching) multiple files. This is necessary for type-checking, where files
+// must be parsed in a common fileset.
+type parseCache struct {
+ mu sync.Mutex
+ m map[parseKey]*parseCacheEntry
+ lru queue // min-atime priority queue of *parseCacheEntry
+ clock uint64 // clock time, incremented when the cache is updated
+ nextOffset token.Pos // token.Pos offset for the next parsed file
+}
+
+// parseKey uniquely identifies a parsed Go file.
+type parseKey struct {
+ file source.FileIdentity
+ mode source.ParseMode
+}
+
+type parseCacheEntry struct {
+ key parseKey
+ promise *memoize.Promise // memoize.Promise[*source.ParsedGoFile]
+ atime uint64 // clock time of last access
+ lruIndex int
+}
+
+// startParse prepares a parsing pass, using the following steps:
+// - search for cache hits
+// - create new promises for cache misses
+// - store as many new promises in the cache as space will allow
+//
+// The resulting slice has an entry for every given file handle, though some
+// entries may be nil if there was an error reading the file (in which case the
+// resulting error will be non-nil).
+func (c *parseCache) startParse(mode source.ParseMode, fhs ...source.FileHandle) ([]*memoize.Promise, error) {
+ c.mu.Lock()
+ defer c.mu.Unlock()
+
+ // Any parsing pass increments the clock, as we'll update access times.
+ // (technically, if fhs is empty this isn't necessary, but that's a degenerate case).
+ //
+ // All entries parsed from a single call get the same access time.
+ c.clock++
+
+ // Read file data and collect cacheable files.
+ var (
+ data = make([][]byte, len(fhs)) // file content for each readable file
+ promises = make([]*memoize.Promise, len(fhs))
+ firstReadError error // first error from fh.Read, or nil
+ )
+ for i, fh := range fhs {
+ src, err := fh.Read()
+ if err != nil {
+ if firstReadError == nil {
+ firstReadError = err
+ }
+ continue
+ }
+ data[i] = src
+
+ key := parseKey{
+ file: fh.FileIdentity(),
+ mode: mode,
+ }
+
+ // Check for a cache hit.
+ if e, ok := c.m[key]; ok {
+ e.atime = c.clock
+ heap.Fix(&c.lru, e.lruIndex)
+ promises[i] = e.promise
+ continue
+ }
+
+ // ...otherwise, create a new promise to parse with a non-overlapping offset
+ fset := token.NewFileSet()
+ if c.nextOffset > 0 {
+ // Add a dummy file so that this parsed file does not overlap with others.
+ fset.AddFile("", 1, int(c.nextOffset))
+ }
+ c.nextOffset += token.Pos(len(src) + parsePadding + 1) // leave room for src fixes
+ fh := fh
+ promise := memoize.NewPromise(string(fh.URI()), func(ctx context.Context, _ interface{}) interface{} {
+ return parseGoSrc(ctx, fset, fh.URI(), src, mode)
+ })
+ promises[i] = promise
+
+ var e *parseCacheEntry
+ if len(c.lru) < parseCacheMaxFiles {
+ // add new entry
+ e = new(parseCacheEntry)
+ if c.m == nil {
+ c.m = make(map[parseKey]*parseCacheEntry)
+ }
+ } else {
+ // evict oldest entry
+ e = heap.Pop(&c.lru).(*parseCacheEntry)
+ delete(c.m, e.key)
+ }
+ e.key = key
+ e.promise = promise
+ e.atime = c.clock
+ c.m[e.key] = e
+ heap.Push(&c.lru, e)
+ }
+
+ if len(c.m) != len(c.lru) {
+ panic("map and LRU are inconsistent")
+ }
+
+ return promises, firstReadError
+}
+
+// parseFiles returns a ParsedGoFile for the given file handles in the
+// requested parse mode.
+//
+// If parseFiles returns an error, it still returns a slice,
+// but with a nil entry for each file that could not be parsed.
+//
+// The second result is a FileSet describing all resulting parsed files.
+//
+// For parsed files that already exists in the cache, access time will be
+// updated. For others, parseFiles will parse and store as many results in the
+// cache as space allows.
+func (c *parseCache) parseFiles(ctx context.Context, mode source.ParseMode, fhs ...source.FileHandle) ([]*source.ParsedGoFile, *token.FileSet, error) {
+ promises, firstReadError := c.startParse(mode, fhs...)
+
+ // Await all parsing.
+ var g errgroup.Group
+ g.SetLimit(runtime.GOMAXPROCS(-1)) // parsing is CPU-bound.
+ pgfs := make([]*source.ParsedGoFile, len(fhs))
+ for i, promise := range promises {
+ if promise == nil {
+ continue
+ }
+ i := i
+ promise := promise
+ g.Go(func() error {
+ result, err := promise.Get(ctx, nil)
+ if err != nil {
+ return err
+ }
+ pgfs[i] = result.(*source.ParsedGoFile)
+ return nil
+ })
+ }
+ if err := g.Wait(); err != nil {
+ return nil, nil, err
+ }
+
+ // Construct a token.FileSet mapping all parsed files, and update their
+ // Tok to the corresponding file in the new fileset.
+ //
+ // In the unlikely event that a parsed file no longer fits in its allocated
+ // space in the FileSet range, it will need to be re-parsed.
+
+ var tokenFiles []*token.File
+ fileIndex := make(map[*token.File]int) // to look up original indexes after sorting
+ for i, pgf := range pgfs {
+ if pgf == nil {
+ continue
+ }
+ fileIndex[pgf.Tok] = i
+ tokenFiles = append(tokenFiles, pgf.Tok)
+ }
+
+ sort.Slice(tokenFiles, func(i, j int) bool {
+ return tokenFiles[i].Base() < tokenFiles[j].Base()
+ })
+
+ var needReparse []int // files requiring reparsing
+ out := tokenFiles[:0]
+ for i, f := range tokenFiles {
+ if i < len(tokenFiles)-1 && f.Base()+f.Size() >= tokenFiles[i+1].Base() {
+ if f != tokenFiles[i+1] { // no need to re-parse duplicates
+ needReparse = append(needReparse, fileIndex[f])
+ }
+ } else {
+ out = append(out, f)
+ }
+ }
+ fset := source.FileSetFor(out...)
+
+ // Re-parse any remaining files using the stitched fileSet.
+ for _, i := range needReparse {
+ // Start from scratch, rather than using ParsedGoFile.Src, so that source
+ // fixing operates exactly the same (note that fixing stops after a limited
+ // number of tries).
+ fh := fhs[i]
+ src, err := fh.Read()
+ if err != nil {
+ if firstReadError == nil {
+ firstReadError = err
+ }
+ continue
+ }
+ pgfs[i] = parseGoSrc(ctx, fset, fh.URI(), src, mode)
+ }
+
+ // Ensure each PGF refers to a token.File from the new FileSet.
+ for i, pgf := range pgfs {
+ if pgf == nil {
+ continue
+ }
+ newTok := fset.File(token.Pos(pgf.Tok.Base()))
+ if newTok == nil {
+ panic("internal error: missing tok for " + pgf.URI)
+ }
+ if newTok.Base() != pgf.Tok.Base() || newTok.Size() != pgf.Tok.Size() {
+ panic("internal error: mismatching token.File in synthetic FileSet")
+ }
+ pgf2 := *pgf
+ pgf2.Tok = newTok
+ pgfs[i] = &pgf2
+ }
+
+ return pgfs, fset, firstReadError
+}
+
+// -- priority queue boilerplate --
+
+// queue is a min-atime prority queue of cache entries.
+type queue []*parseCacheEntry
+
+func (q queue) Len() int { return len(q) }
+
+func (q queue) Less(i, j int) bool { return q[i].atime < q[j].atime }
+
+func (q queue) Swap(i, j int) {
+ q[i], q[j] = q[j], q[i]
+ q[i].lruIndex = i
+ q[j].lruIndex = j
+}
+
+func (q *queue) Push(x interface{}) {
+ e := x.(*parseCacheEntry)
+ e.lruIndex = len(*q)
+ *q = append(*q, e)
+}
+
+func (q *queue) Pop() interface{} {
+ last := len(*q) - 1
+ e := (*q)[last]
+ (*q)[last] = nil // aid GC
+ *q = (*q)[:last]
+ return e
+}
diff --git a/gopls/internal/lsp/cache/parse_cache_test.go b/gopls/internal/lsp/cache/parse_cache_test.go
new file mode 100644
index 0000000..97bff87
--- /dev/null
+++ b/gopls/internal/lsp/cache/parse_cache_test.go
@@ -0,0 +1,142 @@
+// Copyright 2023 The Go Authors. All rights reserved.
+// Use of this source code is governed by a BSD-style
+// license that can be found in the LICENSE file.
+
+package cache
+
+import (
+ "context"
+ "fmt"
+ "go/token"
+ "testing"
+
+ "golang.org/x/tools/gopls/internal/lsp/source"
+ "golang.org/x/tools/gopls/internal/span"
+)
+
+func TestParseCache(t *testing.T) {
+ ctx := context.Background()
+ uri := span.URI("file:///myfile")
+ fh := makeFakeFileHandle(uri, []byte("package p\n\nconst _ = \"foo\""))
+
+ var cache parseCache
+ pgfs1, _, err := cache.parseFiles(ctx, source.ParseFull, fh)
+ if err != nil {
+ t.Fatal(err)
+ }
+ pgf1 := pgfs1[0]
+ pgfs2, _, err := cache.parseFiles(ctx, source.ParseFull, fh)
+ pgf2 := pgfs2[0]
+ if err != nil {
+ t.Fatal(err)
+ }
+ if pgf1.File != pgf2.File {
+ t.Errorf("parseFiles(%q): unexpected cache miss on repeated call", uri)
+ }
+
+ // Fill up the cache with other files, but don't evict the file above.
+ files := []source.FileHandle{fh}
+ files = append(files, dummyFileHandles(parseCacheMaxFiles-1)...)
+ pgfs3, fset, err := cache.parseFiles(ctx, source.ParseFull, files...)
+ pgf3 := pgfs3[0]
+ if pgf3.File != pgf1.File {
+ t.Errorf("parseFiles(%q, ...): unexpected cache miss", uri)
+ }
+ if pgf3.Tok == pgf1.Tok {
+ t.Errorf("parseFiles(%q, ...): unexpectedly matching token file", uri)
+ }
+ if pgf3.Tok.Base() != pgf1.Tok.Base() || pgf3.Tok.Size() != pgf1.Tok.Size() {
+ t.Errorf("parseFiles(%q, ...): result.Tok has base: %d, size: %d, want (%d, %d)", uri, pgf3.Tok.Base(), pgf3.Tok.Size(), pgf1.Tok.Base(), pgf1.Tok.Size())
+ }
+ if tok := fset.File(token.Pos(pgf3.Tok.Base())); tok != pgf3.Tok {
+ t.Errorf("parseFiles(%q, ...): result.Tok not contained in FileSet", uri)
+ }
+
+ // Now overwrite the cache, after which we should get new results.
+ files = dummyFileHandles(parseCacheMaxFiles)
+ _, _, err = cache.parseFiles(ctx, source.ParseFull, files...)
+ if err != nil {
+ t.Fatal(err)
+ }
+ pgfs4, _, err := cache.parseFiles(ctx, source.ParseFull, fh)
+ if err != nil {
+ t.Fatal(err)
+ }
+ if pgfs4[0].File == pgf1.File {
+ t.Errorf("parseFiles(%q): unexpected cache hit after overwriting cache", uri)
+ }
+}
+
+func TestParseCache_Reparsing(t *testing.T) {
+ defer func(padding int) {
+ parsePadding = padding
+ }(parsePadding)
+ parsePadding = 0
+
+ files := dummyFileHandles(parseCacheMaxFiles)
+ danglingSelector := []byte("package p\nfunc _() {\n\tx.\n}")
+ files = append(files, makeFakeFileHandle("file:///bad1", danglingSelector))
+ files = append(files, makeFakeFileHandle("file:///bad2", danglingSelector))
+
+ // Parsing should succeed even though we overflow the padding.
+ var cache parseCache
+ _, _, err := cache.parseFiles(context.Background(), source.ParseFull, files...)
+ if err != nil {
+ t.Fatal(err)
+ }
+}
+
+func TestParseCache_Duplicates(t *testing.T) {
+ ctx := context.Background()
+ uri := span.URI("file:///myfile")
+ fh := makeFakeFileHandle(uri, []byte("package p\n\nconst _ = \"foo\""))
+
+ var cache parseCache
+ pgfs, _, err := cache.parseFiles(ctx, source.ParseFull, fh, fh)
+ if err != nil {
+ t.Fatal(err)
+ }
+ if pgfs[0].File != pgfs[1].File {
+ t.Errorf("parseFiles(fh, fh): = [%p, %p], want duplicate files", pgfs[0].File, pgfs[1].File)
+ }
+}
+
+func dummyFileHandles(n int) []source.FileHandle {
+ var fhs []source.FileHandle
+ for i := 0; i < n; i++ {
+ uri := span.URI(fmt.Sprintf("file:///_%d", i))
+ src := []byte(fmt.Sprintf("package p\nvar _ = %d", i))
+ fhs = append(fhs, makeFakeFileHandle(uri, src))
+ }
+ return fhs
+}
+
+func makeFakeFileHandle(uri span.URI, src []byte) fakeFileHandle {
+ return fakeFileHandle{
+ uri: uri,
+ data: src,
+ hash: source.HashOf(src),
+ }
+}
+
+type fakeFileHandle struct {
+ source.FileHandle
+ uri span.URI
+ data []byte
+ hash source.Hash
+}
+
+func (h fakeFileHandle) URI() span.URI {
+ return h.uri
+}
+
+func (h fakeFileHandle) Read() ([]byte, error) {
+ return h.data, nil
+}
+
+func (h fakeFileHandle) FileIdentity() source.FileIdentity {
+ return source.FileIdentity{
+ URI: h.uri,
+ Hash: h.hash,
+ }
+}
diff --git a/gopls/internal/lsp/cache/session.go b/gopls/internal/lsp/cache/session.go
index 5df540e..38e28ac 100644
--- a/gopls/internal/lsp/cache/session.go
+++ b/gopls/internal/lsp/cache/session.go
@@ -159,8 +159,9 @@
cancel: cancel,
store: s.cache.store,
packages: persistent.NewMap(packageKeyLessInterface),
- meta: &metadataGraph{},
+ meta: new(metadataGraph),
files: newFilesMap(),
+ parseCache: new(parseCache),
isActivePackageCache: newIsActivePackageCacheMap(),
parsedGoFiles: persistent.NewMap(parseKeyLessInterface),
parseKeysByURI: newParseKeysByURIMap(),
diff --git a/gopls/internal/lsp/cache/snapshot.go b/gopls/internal/lsp/cache/snapshot.go
index f93247a..7ad3255 100644
--- a/gopls/internal/lsp/cache/snapshot.go
+++ b/gopls/internal/lsp/cache/snapshot.go
@@ -89,6 +89,8 @@
// It may invalidated when a file's content changes.
files filesMap
+ // parseCache holds an LRU cache of recently parsed files.
+ parseCache *parseCache
// parsedGoFiles maps a parseKey to the handle of the future result of parsing it.
parsedGoFiles *persistent.Map // from parseKey to *memoize.Promise[parseGoResult]
@@ -1667,6 +1669,7 @@
isActivePackageCache: s.isActivePackageCache.Clone(),
analyses: s.analyses.Clone(),
files: s.files.Clone(),
+ parseCache: s.parseCache,
parsedGoFiles: s.parsedGoFiles.Clone(),
parseKeysByURI: s.parseKeysByURI.Clone(),
symbolizeHandles: s.symbolizeHandles.Clone(),
@@ -2084,19 +2087,22 @@
}
// Parse headers to compare package names and imports.
- oldHead, oldErr := peekOrParse(ctx, lockedSnapshot, oldFH, source.ParseHeader)
- newHead, newErr := peekOrParse(ctx, lockedSnapshot, newFH, source.ParseHeader)
+ oldHeads, _, oldErr := lockedSnapshot.parseCache.parseFiles(ctx, source.ParseHeader, oldFH)
+ newHeads, _, newErr := lockedSnapshot.parseCache.parseFiles(ctx, source.ParseHeader, newFH)
if oldErr != nil || newErr != nil {
- // TODO(rfindley): we can get here if newFH does not exists. There is
- // asymmetry here, in that newFH may be non-nil even if the underlying file
- // does not exist.
+ // TODO(rfindley): we can get here if newFH does not exist. There is
+ // asymmetry, in that newFH may be non-nil even if the underlying file does
+ // not exist.
//
// We should not produce a non-nil filehandle for a file that does not exist.
errChanged := (oldErr == nil) != (newErr == nil)
return errChanged, errChanged, (newErr != nil) // we don't know if an import was deleted
}
+ oldHead := oldHeads[0]
+ newHead := newHeads[0]
+
// `go list` fails completely if the file header cannot be parsed. If we go
// from a non-parsing state to a parsing state, we should reload.
if oldHead.ParseErr != nil && newHead.ParseErr == nil {
@@ -2133,10 +2139,10 @@
// Note: if this affects performance we can probably avoid parsing in the
// common case by first scanning the source for potential comments.
if !invalidate {
- origFull, oldErr := peekOrParse(ctx, lockedSnapshot, oldFH, source.ParseFull)
- currFull, newErr := peekOrParse(ctx, lockedSnapshot, newFH, source.ParseFull)
+ origFulls, _, oldErr := lockedSnapshot.parseCache.parseFiles(ctx, source.ParseFull, oldFH)
+ newFulls, _, newErr := lockedSnapshot.parseCache.parseFiles(ctx, source.ParseFull, newFH)
if oldErr == nil && newErr == nil {
- invalidate = magicCommentsChanged(origFull.File, currFull.File)
+ invalidate = magicCommentsChanged(origFulls[0].File, newFulls[0].File)
} else {
// At this point, we shouldn't ever fail to produce a ParsedGoFile, as
// we're already past header parsing.
@@ -2147,22 +2153,6 @@
return invalidate, pkgFileChanged, importDeleted
}
-// peekOrParse returns the cached ParsedGoFile if it exists,
-// otherwise parses without populating the cache.
-//
-// It returns an error if the file could not be read (note that parsing errors
-// are stored in ParsedGoFile.ParseErr).
-//
-// lockedSnapshot must be locked.
-func peekOrParse(ctx context.Context, lockedSnapshot *snapshot, fh source.FileHandle, mode source.ParseMode) (*source.ParsedGoFile, error) {
- // Peek in the cache without populating it.
- // We do this to reduce retained heap, not work.
- if parsed, _ := lockedSnapshot.peekParseGoLocked(fh, mode); parsed != nil {
- return parsed, nil // cache hit
- }
- return parseGoImpl(ctx, token.NewFileSet(), fh, mode)
-}
-
func magicCommentsChanged(original *ast.File, current *ast.File) bool {
oldComments := extractMagicComments(original)
newComments := extractMagicComments(current)
diff --git a/gopls/internal/lsp/cache/symbols.go b/gopls/internal/lsp/cache/symbols.go
index 7fa3584..b6ffffb 100644
--- a/gopls/internal/lsp/cache/symbols.go
+++ b/gopls/internal/lsp/cache/symbols.go
@@ -7,7 +7,6 @@
import (
"context"
"go/ast"
- "go/parser"
"go/token"
"go/types"
"strings"
@@ -38,8 +37,8 @@
}
type symbolHandleKey source.Hash
key := symbolHandleKey(fh.FileIdentity().Hash)
- promise, release := s.store.Promise(key, func(_ context.Context, arg interface{}) interface{} {
- symbols, err := symbolizeImpl(arg.(*snapshot), fh)
+ promise, release := s.store.Promise(key, func(ctx context.Context, arg interface{}) interface{} {
+ symbols, err := symbolizeImpl(ctx, arg.(*snapshot), fh)
return symbolizeResult{symbols, err}
})
@@ -62,50 +61,17 @@
// symbolizeImpl reads and parses a file and extracts symbols from it.
// It may use a parsed file already present in the cache but
// otherwise does not populate the cache.
-func symbolizeImpl(snapshot *snapshot, fh source.FileHandle) ([]source.Symbol, error) {
- src, err := fh.Read()
+func symbolizeImpl(ctx context.Context, snapshot *snapshot, fh source.FileHandle) ([]source.Symbol, error) {
+ pgfs, _, err := snapshot.parseCache.parseFiles(ctx, source.ParseFull, fh)
if err != nil {
return nil, err
}
- var (
- file *ast.File
- tokFile *token.File
- mapper *protocol.Mapper
- )
-
- // If the file has already been fully parsed through the
- // cache, we can just use the result. But we don't want to
- // populate the cache after a miss.
- snapshot.mu.Lock()
- pgf, _ := snapshot.peekParseGoLocked(fh, source.ParseFull)
- snapshot.mu.Unlock()
- if pgf != nil {
- file = pgf.File
- tokFile = pgf.Tok
- mapper = pgf.Mapper
- }
-
- // Otherwise, we parse the file ourselves. Notably we don't use parseGo here,
- // so that we can avoid parsing comments and can skip object resolution,
- // which has a meaningful impact on performance. Neither comments nor objects
- // are necessary for symbol construction.
- if file == nil {
- fset := token.NewFileSet()
- file, err = parser.ParseFile(fset, fh.URI().Filename(), src, skipObjectResolution)
- if file == nil {
- return nil, err
- }
- tokFile = fset.File(file.Package)
- mapper = protocol.NewMapper(fh.URI(), src)
- }
-
w := &symbolWalker{
- tokFile: tokFile,
- mapper: mapper,
+ tokFile: pgfs[0].Tok,
+ mapper: pgfs[0].Mapper,
}
-
- w.fileDecls(file.Decls)
+ w.fileDecls(pgfs[0].File.Decls)
return w.symbols, w.firstError
}
diff --git a/gopls/internal/lsp/command.go b/gopls/internal/lsp/command.go
index 3902b18..e6ec67f 100644
--- a/gopls/internal/lsp/command.go
+++ b/gopls/internal/lsp/command.go
@@ -744,7 +744,7 @@
if err != nil {
return err
}
- fset := source.SingletonFileSet(pgf.Tok)
+ fset := source.FileSetFor(pgf.Tok)
for _, group := range astutil.Imports(fset, pgf.File) {
for _, imp := range group {
if imp.Path == nil {
diff --git a/gopls/internal/lsp/source/format.go b/gopls/internal/lsp/source/format.go
index 10120c2..b8206ed 100644
--- a/gopls/internal/lsp/source/format.go
+++ b/gopls/internal/lsp/source/format.go
@@ -53,7 +53,7 @@
// This should be acceptable for all users, who likely be prompted to rebuild
// the LSP server on each Go release.
buf := &bytes.Buffer{}
- fset := SingletonFileSet(pgf.Tok)
+ fset := FileSetFor(pgf.Tok)
if err := format.Node(buf, fset, pgf.File); err != nil {
return nil, err
}
diff --git a/gopls/internal/lsp/source/hover.go b/gopls/internal/lsp/source/hover.go
index cc4d37b..25d85b4 100644
--- a/gopls/internal/lsp/source/hover.go
+++ b/gopls/internal/lsp/source/hover.go
@@ -195,7 +195,7 @@
spec2.Comment = nil
var b strings.Builder
b.WriteString("type ")
- fset := SingletonFileSet(declPGF.Tok)
+ fset := FileSetFor(declPGF.Tok)
if err := format.Node(&b, fset, &spec2); err != nil {
return protocol.Range{}, nil, err
}
diff --git a/gopls/internal/lsp/source/stub.go b/gopls/internal/lsp/source/stub.go
index 184fa9d..31d8135 100644
--- a/gopls/internal/lsp/source/stub.go
+++ b/gopls/internal/lsp/source/stub.go
@@ -232,7 +232,7 @@
NewText: []byte(edit.New),
})
}
- return SingletonFileSet(declPGF.Tok), // edits use declPGF.Tok
+ return FileSetFor(declPGF.Tok), // edits use declPGF.Tok
&analysis.SuggestedFix{TextEdits: edits},
nil
}
diff --git a/gopls/internal/lsp/source/types_format.go b/gopls/internal/lsp/source/types_format.go
index 318fab5..4d487d4 100644
--- a/gopls/internal/lsp/source/types_format.go
+++ b/gopls/internal/lsp/source/types_format.go
@@ -110,7 +110,7 @@
variadic = true
}
}
- fset := SingletonFileSet(builtin.Tok)
+ fset := FileSetFor(builtin.Tok)
params, _ := formatFieldList(ctx, fset, decl.Type.Params, variadic)
results, needResultParens := formatFieldList(ctx, fset, decl.Type.Results, false)
d := decl.Doc.Text()
diff --git a/gopls/internal/lsp/source/util.go b/gopls/internal/lsp/source/util.go
index 1dc7860..4149cea 100644
--- a/gopls/internal/lsp/source/util.go
+++ b/gopls/internal/lsp/source/util.go
@@ -127,18 +127,23 @@
// FormatNodeFile is like FormatNode, but requires only the token.File for the
// syntax containing the given ast node.
func FormatNodeFile(file *token.File, n ast.Node) string {
- fset := SingletonFileSet(file)
+ fset := FileSetFor(file)
return FormatNode(fset, n)
}
-// SingletonFileSet creates a new token.FileSet containing a file that is
-// identical to f (same base, size, and line), for use in APIs that require a
-// FileSet.
-func SingletonFileSet(f *token.File) *token.FileSet {
+// FileSetFor returns a new FileSet containing a sequence of new Files with
+// the same base, size, and line as the input files, for use in APIs that
+// require a FileSet.
+//
+// Precondition: the input files must be non-overlapping, and sorted in order
+// of their Base.
+func FileSetFor(files ...*token.File) *token.FileSet {
fset := token.NewFileSet()
- f2 := fset.AddFile(f.Name(), f.Base(), f.Size())
- lines := tokeninternal.GetLines(f)
- f2.SetLines(lines)
+ for _, f := range files {
+ f2 := fset.AddFile(f.Name(), f.Base(), f.Size())
+ lines := tokeninternal.GetLines(f)
+ f2.SetLines(lines)
+ }
return fset
}