gopls/internal/lsp/cache: make type-checking incremental
In this CL type-checked packages are made entirely independent of each
other, and package export data and indexes are stored in a file cache.
As a result, gopls uses significantly less memory, and (with a warm
cache) starts significantly faster. Other benchmarks have regressed
slightly due to the additional I/O and export data loading, but not
significantly so, and we have some ideas for how to further narrow or
even close the performance gap.
In the benchmarks below, based on the x/tools repository, we can see
that in-use memory was reduced by 88%, and startup time with a warm
cache by 65% (this is the best case where nothing has changed). Other
benchmarks regressed by 10-50%, much of which can be addressed by
improvements to the objectpath package (golang/go#51017), and by making
package data serialization asynchronous to type-checking.
Notably, we observe larger regressions in implementations, references,
and rename because the index implementations (by Alan Donovan) preceded
this change to type-checking, and so these benchmark statistics compare
in-memory index performance to on-disk index performance. Again, we can
optimize these if necessary by keeping certain index information in
memory, or by decoding more selectively.
name old in_use_bytes new in_use_bytes delta
InitialWorkspaceLoad/tools-12 432M ± 2% 50M ± 2% -88.54% (p=0.000 n=10+10)
name old time/op new time/op delta
StructCompletion/tools-12 27.2ms ± 5% 31.8ms ± 9% +16.99% (p=0.000 n=9+9)
ImportCompletion/tools-12 2.07ms ± 8% 2.21ms ± 6% +6.64% (p=0.004 n=9+9)
SliceCompletion/tools-12 29.0ms ± 5% 32.7ms ± 5% +12.78% (p=0.000 n=10+9)
FuncDeepCompletion/tools-12 39.6ms ± 6% 39.3ms ± 3% ~ (p=0.853 n=10+10)
CompletionFollowingEdit/tools-12 72.7ms ± 7% 108.1ms ± 7% +48.59% (p=0.000 n=9+9)
Definition/tools-12 525µs ± 6% 601µs ± 2% +14.33% (p=0.000 n=9+10)
DidChange/tools-12 6.17ms ± 7% 6.77ms ± 2% +9.64% (p=0.000 n=10+10)
Hover/tools-12 2.11ms ± 5% 2.61ms ± 3% +23.87% (p=0.000 n=10+10)
Implementations/tools-12 4.04ms ± 3% 60.19ms ± 3% +1389.77% (p=0.000 n=9+10)
InitialWorkspaceLoad/tools-12 3.84s ± 4% 1.33s ± 2% -65.47% (p=0.000 n=10+9)
References/tools-12 9.72ms ± 6% 24.28ms ± 6% +149.83% (p=0.000 n=10+10)
Rename/tools-12 121ms ± 8% 168ms ±12% +38.92% (p=0.000 n=10+10)
WorkspaceSymbols/tools-12 14.4ms ± 6% 15.6ms ± 3% +8.76% (p=0.000 n=9+10)
This CL is one step closer to the end* of a long journey to reduce
memory usage and statefulness in gopls, so that it can be more
performant and reliable.
Specifically, this CL implements a new type-checking pass that loads and
stores export data, cross references, serialized diagnostics, and method
set indexes in the file system. Concurrent type-checking passes may
share in-progress work, but after type-checking only active packages are
kept in memory. Consequently, there can be no global relationship
between type-checked packages. The work to break any dependence on
global relationships was done over a long time leading up to this CL.
In order to approach the previous type-checking performance, the
following new optimizations are made:
- the global FileSet is completely removed: repeatedly importing from
export data resulted in a tremendous amount of unnecessary token.File
information, and so FileSets had to be scoped to packages
- files are parsed as a batch and stored in the LRU cache implemented
in the preceding CL
- type-checking is also turned into a batch process, so that
overlapping nodes in the package graph may be shared during large
type-checking operations such as the initial workspace load
This new execution model enables several simplifications:
- We no longer need to trim the AST before type-checking:
TypeCheckMode and ParseExported are gone.
- We no longer need to do careful bookkeeping around parsed files: all
parsing uses the LRU parse cache.
- It is no longer necessary to estimate cache heap usage in debug
information.
There is still much more to do. This new model for gopls's execution
requires significant testing and experimentation. There may be new bugs
in the complicated new algorithms that enable this change, or bugs
related to the new reliance on export data (this may be the first time
export data for packages with type errors is significantly exercised).
There may be new environments where the new execution model does not
have the same beneficial effect. (On the other hand, there may be
some where it has an even more beneficial effect, such as resource
limited environments like dev containers.) There are also a lot of new
opportunities for optimization now that we are no longer tied to a rigid
structure of in-memory data.
*Furthermore, the following planned work is simply not done yet:
- Implement precise pruning based on "deep" hash of imports.
- Rewrite unimported completion, now that we no longer have cached
import paths.
For golang/go#57987
Change-Id: Iedfc16656f79e314be448b892b710b9e63f72551
Reviewed-on: https://go-review.googlesource.com/c/tools/+/466975
Run-TryBot: Robert Findley <rfindley@google.com>
TryBot-Result: Gopher Robot <gobot@golang.org>
gopls-CI: kokoro <noreply+kokoro@google.com>
Reviewed-by: Alan Donovan <adonovan@google.com>
diff --git a/gopls/internal/lsp/cache/analysis.go b/gopls/internal/lsp/cache/analysis.go
index 1e19108..eac23cd 100644
--- a/gopls/internal/lsp/cache/analysis.go
+++ b/gopls/internal/lsp/cache/analysis.go
@@ -11,6 +11,7 @@
"context"
"crypto/sha256"
"encoding/gob"
+ "encoding/json"
"errors"
"fmt"
"go/ast"
@@ -1155,14 +1156,18 @@
}
}
-// -- data types for serialization of analysis.Diagnostic --
+// -- data types for serialization of analysis.Diagnostic and source.Diagnostic --
type gobDiagnostic struct {
Location protocol.Location
- Category string
+ Severity protocol.DiagnosticSeverity
+ Code string
+ CodeHref string
+ Source string
Message string
SuggestedFixes []gobSuggestedFix
Related []gobRelatedInformation
+ Tags []protocol.DiagnosticTag
}
type gobRelatedInformation struct {
@@ -1171,8 +1176,16 @@
}
type gobSuggestedFix struct {
- Message string
- TextEdits []gobTextEdit
+ Message string
+ TextEdits []gobTextEdit
+ Command *gobCommand
+ ActionKind protocol.CodeActionKind
+}
+
+type gobCommand struct {
+ Title string
+ Command string
+ Arguments []json.RawMessage
}
type gobTextEdit struct {
@@ -1218,11 +1231,17 @@
if err != nil {
return gobDiagnostic{}, err
}
+
return gobDiagnostic{
- Location: loc,
- Category: diag.Category,
+ Location: loc,
+ // Severity for analysis diagnostics is dynamic, based on user
+ // configuration per analyzer.
+ // Code and CodeHref are unset for Analysis diagnostics,
+ // TODO(rfindley): set Code fields if/when golang/go#57906 is accepted.
+ Source: diag.Category,
Message: diag.Message,
- Related: related,
SuggestedFixes: fixes,
+ Related: related,
+ // Analysis diagnostics do not contain tags.
}, nil
}
diff --git a/gopls/internal/lsp/cache/cache.go b/gopls/internal/lsp/cache/cache.go
index edf1d0e..24bd842 100644
--- a/gopls/internal/lsp/cache/cache.go
+++ b/gopls/internal/lsp/cache/cache.go
@@ -6,13 +6,7 @@
import (
"context"
- "fmt"
- "go/ast"
- "go/token"
- "go/types"
- "html/template"
"reflect"
- "sort"
"strconv"
"sync/atomic"
@@ -29,22 +23,15 @@
// Both the fset and store may be nil, but if store is non-nil so must be fset
// (and they must always be used together), otherwise it may be possible to get
// cached data referencing token.Pos values not mapped by the FileSet.
-func New(fset *token.FileSet, store *memoize.Store) *Cache {
+func New(store *memoize.Store) *Cache {
index := atomic.AddInt64(&cacheIndex, 1)
- if store != nil && fset == nil {
- panic("non-nil store with nil fset")
- }
- if fset == nil {
- fset = token.NewFileSet()
- }
if store == nil {
store = &memoize.Store{}
}
c := &Cache{
id: strconv.FormatInt(index, 10),
- fset: fset,
store: store,
memoizedFS: &memoizedFS{filesByID: map[robustio.FileID][]*DiskFile{}},
}
@@ -56,8 +43,7 @@
// TODO(rfindley): once fset and store need not be bundled together, the Cache
// type can be eliminated.
type Cache struct {
- id string
- fset *token.FileSet
+ id string
store *memoize.Store
@@ -90,98 +76,3 @@
func (c *Cache) ID() string { return c.id }
func (c *Cache) MemStats() map[reflect.Type]int { return c.store.Stats() }
-
-type packageStat struct {
- id PackageID
- mode source.ParseMode
- file int64
- ast int64
- types int64
- typesInfo int64
- total int64
-}
-
-func (c *Cache) PackageStats(withNames bool) template.HTML {
- var packageStats []packageStat
- c.store.DebugOnlyIterate(func(k, v interface{}) {
- switch k.(type) {
- case packageHandleKey:
- v := v.(typeCheckResult)
- if v.pkg == nil {
- break
- }
- typsCost := typesCost(v.pkg.types.Scope())
- typInfoCost := typesInfoCost(v.pkg.typesInfo)
- stat := packageStat{
- id: v.pkg.id,
- mode: v.pkg.mode,
- types: typsCost,
- typesInfo: typInfoCost,
- }
- for _, f := range v.pkg.compiledGoFiles {
- stat.file += int64(len(f.Src))
- stat.ast += astCost(f.File)
- }
- stat.total = stat.file + stat.ast + stat.types + stat.typesInfo
- packageStats = append(packageStats, stat)
- }
- })
- var totalCost int64
- for _, stat := range packageStats {
- totalCost += stat.total
- }
- sort.Slice(packageStats, func(i, j int) bool {
- return packageStats[i].total > packageStats[j].total
- })
- html := "<table><thead><td>Name</td><td>total = file + ast + types + types info</td></thead>\n"
- human := func(n int64) string {
- return fmt.Sprintf("%.2f", float64(n)/(1024*1024))
- }
- var printedCost int64
- for _, stat := range packageStats {
- name := stat.id
- if !withNames {
- name = "-"
- }
- html += fmt.Sprintf("<tr><td>%v (%v)</td><td>%v = %v + %v + %v + %v</td></tr>\n", name, stat.mode,
- human(stat.total), human(stat.file), human(stat.ast), human(stat.types), human(stat.typesInfo))
- printedCost += stat.total
- if float64(printedCost) > float64(totalCost)*.9 {
- break
- }
- }
- html += "</table>\n"
- return template.HTML(html)
-}
-
-func astCost(f *ast.File) int64 {
- if f == nil {
- return 0
- }
- var count int64
- ast.Inspect(f, func(_ ast.Node) bool {
- count += 32 // nodes are pretty small.
- return true
- })
- return count
-}
-
-func typesCost(scope *types.Scope) int64 {
- cost := 64 + int64(scope.Len())*128 // types.object looks pretty big
- for i := 0; i < scope.NumChildren(); i++ {
- cost += typesCost(scope.Child(i))
- }
- return cost
-}
-
-func typesInfoCost(info *types.Info) int64 {
- // Most of these refer to existing objects, with the exception of InitOrder, Selections, and Types.
- cost := 24*len(info.Defs) +
- 32*len(info.Implicits) +
- 256*len(info.InitOrder) + // these are big, but there aren't many of them.
- 32*len(info.Scopes) +
- 128*len(info.Selections) + // wild guess
- 128*len(info.Types) + // wild guess
- 32*len(info.Uses)
- return int64(cost)
-}
diff --git a/gopls/internal/lsp/cache/check.go b/gopls/internal/lsp/cache/check.go
index adf50b6..451604f 100644
--- a/gopls/internal/lsp/cache/check.go
+++ b/gopls/internal/lsp/cache/check.go
@@ -7,11 +7,11 @@
import (
"context"
"crypto/sha256"
- "errors"
"fmt"
"go/ast"
+ "go/token"
"go/types"
- "path/filepath"
+ "log"
"regexp"
"sort"
"strings"
@@ -20,6 +20,7 @@
"golang.org/x/mod/module"
"golang.org/x/sync/errgroup"
"golang.org/x/tools/go/ast/astutil"
+ "golang.org/x/tools/gopls/internal/lsp/filecache"
"golang.org/x/tools/gopls/internal/lsp/protocol"
"golang.org/x/tools/gopls/internal/lsp/source"
"golang.org/x/tools/gopls/internal/lsp/source/methodsets"
@@ -28,16 +29,532 @@
"golang.org/x/tools/internal/bug"
"golang.org/x/tools/internal/event"
"golang.org/x/tools/internal/event/tag"
+ "golang.org/x/tools/internal/gcimporter"
"golang.org/x/tools/internal/memoize"
"golang.org/x/tools/internal/packagesinternal"
"golang.org/x/tools/internal/typeparams"
"golang.org/x/tools/internal/typesinternal"
)
-// A packageKey identifies a packageHandle in the snapshot.packages map.
-type packageKey struct {
- mode source.ParseMode
- id PackageID
+// A typeCheckBatch holds data for a logical type-checking operation, which may
+// type-check many unrelated packages.
+//
+// It shares state such as parsed files and imports, to optimize type-checking
+// for packages with overlapping dependency graphs.
+type typeCheckBatch struct {
+ meta *metadataGraph
+
+ parsedFiles map[span.URI]*source.ParsedGoFile // parsed files necessary for type-checking
+ fset *token.FileSet // FileSet describing all parsed files
+
+ // Promises holds promises to either read export data for the package, or
+ // parse and type-check its syntax.
+ //
+ // The return value of these promises is not used: after promises are
+ // awaited, they must write an entry into the imports map.
+ promises map[PackageID]*memoize.Promise
+
+ mu sync.Mutex
+ needFiles map[span.URI]source.FileHandle // de-duplicated file handles required for type-checking
+ imports map[PackageID]pkgOrErr // types.Packages to use for importing
+ exportData map[PackageID][]byte
+ packages map[PackageID]*Package
+}
+
+type pkgOrErr struct {
+ pkg *types.Package
+ err error
+}
+
+// TypeCheck type-checks the specified packages.
+//
+// The resulting packages slice always contains len(ids) entries, though some
+// of them may be nil if (and only if) the resulting error is non-nil.
+//
+// An error is returned if any of the requested packages fail to type-check.
+// This is different from having type-checking errors: a failure to type-check
+// indicates context cancellation or otherwise significant failure to perform
+// the type-checking operation.
+func (s *snapshot) TypeCheck(ctx context.Context, ids ...PackageID) ([]source.Package, error) {
+ // Check for existing active packages.
+ //
+ // Since gopls can't depend on package identity, any instance of the
+ // requested package must be ok to return.
+ //
+ // This is an optimization to avoid redundant type-checking: following
+ // changes to an open package many LSP clients send several successive
+ // requests for package information for the modified package (semantic
+ // tokens, code lens, inlay hints, etc.)
+ pkgs := make([]source.Package, len(ids))
+ needSyntax := make(map[PackageID]bool)
+ for i, id := range ids {
+ if pkg := s.getActivePackage(id); pkg != nil {
+ pkgs[i] = pkg
+ } else {
+ needSyntax[id] = true
+ }
+ }
+
+ if len(needSyntax) == 0 {
+ return pkgs, nil
+ }
+
+ // Build up shared state for efficient type-checking.
+ b := &typeCheckBatch{
+ parsedFiles: make(map[span.URI]*source.ParsedGoFile),
+ // fset is built during the parsing pass.
+ needFiles: make(map[span.URI]source.FileHandle),
+
+ promises: make(map[PackageID]*memoize.Promise),
+ imports: make(map[PackageID]pkgOrErr),
+ exportData: make(map[PackageID][]byte),
+ packages: make(map[PackageID]*Package),
+ }
+
+ // Capture metadata once to ensure a consistent view.
+ s.mu.Lock()
+ b.meta = s.meta
+ s.mu.Unlock()
+
+ // -- Step 1: assemble the promises graph --
+
+ var (
+ needExportData = make(map[PackageID]packageHandleKey)
+ packageHandles = make(map[PackageID]*packageHandle)
+ )
+
+ // collectPromises collects promises to load packages from export data or
+ // type-check.
+ var collectPromises func(PackageID) error
+ collectPromises = func(id PackageID) error {
+ if _, ok := b.promises[id]; ok {
+ return nil
+ }
+ b.promises[id] = nil // break cycles
+
+ m := b.meta.metadata[id]
+ if m == nil {
+ return bug.Errorf("missing metadata for %v", id)
+ }
+ for _, id := range m.DepsByPkgPath {
+ if err := collectPromises(id); err != nil {
+ return err
+ }
+ }
+
+ // Note that we can't reuse active packages here, as they will have the
+ // wrong FileSet. Any active packages that exist as dependencies of other
+ // packages will need to be loaded from export data.
+ ph, err := s.buildPackageHandle(ctx, id)
+ if err != nil {
+ return err
+ }
+ packageHandles[id] = ph
+
+ if needSyntax[id] {
+ // We will need to parse and type-check this package.
+ //
+ // We may also need to parse and type-check if export data is missing,
+ // but that is handled after fetching export data below.
+ b.addNeededFiles(ph)
+ } else if id != "unsafe" { // we can't load export data for unsafe
+ needExportData[id] = ph.key
+ }
+
+ debugName := fmt.Sprintf("check(%s)", id)
+ b.promises[id] = memoize.NewPromise(debugName, func(ctx context.Context, _ interface{}) interface{} {
+ var res pkgOrErr
+ if err := b.awaitPredecessors(ctx, ph.m); err != nil {
+ res.err = err
+ } else {
+ b.mu.Lock()
+ data, ok := b.exportData[id]
+ b.mu.Unlock()
+
+ if ok {
+ // We need export data, and have it.
+ res.pkg, res.err = b.importPackage(ctx, m, data)
+ } else if !needSyntax[id] {
+ // We need only a types.Package, but don't have export data.
+ // Type-check as fast as possible (skipping function bodies).
+ res.pkg, res.err = b.checkPackageForImport(ctx, ph)
+ } else {
+ // We need a syntax package.
+ var pkg *Package
+ pkg, res.err = b.checkPackage(ctx, ph)
+ if res.err == nil {
+ res.pkg = pkg.pkg.types
+ b.mu.Lock()
+ b.packages[id] = pkg
+ b.mu.Unlock()
+ }
+ }
+ }
+
+ b.mu.Lock()
+ b.imports[m.ID] = res
+ b.mu.Unlock()
+ return nil
+ })
+ return nil
+ }
+ for id := range needSyntax {
+ collectPromises(id)
+ }
+
+ // -- Step 2: collect export data --
+ //
+ // This must be done before parsing in order to determine which files must be
+ // parsed.
+ {
+ var g errgroup.Group
+ for id, key := range needExportData {
+ id := id
+ key := key
+ g.Go(func() error {
+ data, err := filecache.Get(exportDataKind, key)
+ if err != nil {
+ if err == filecache.ErrNotFound {
+ ph := packageHandles[id]
+ b.addNeededFiles(ph) // we will need to parse and type check
+ return nil // ok: we will type check later
+ }
+ return err
+ }
+ b.mu.Lock()
+ b.exportData[id] = data
+ b.mu.Unlock()
+ return nil
+ })
+ }
+ if err := g.Wait(); err != nil {
+ return pkgs, err
+ }
+ }
+
+ // -- Step 3: parse files required for type checking. --
+ //
+ // Parse all necessary files in parallel. Unfortunately we can't start
+ // parsing each package's file as soon as we discover that it is a syntax
+ // package, because the parseCache cannot add files to an existing FileSet.
+ {
+ var fhs []source.FileHandle
+ for _, fh := range b.needFiles {
+ fhs = append(fhs, fh)
+ }
+ pgfs, fset, err := s.parseCache.parseFiles(ctx, source.ParseFull, fhs...)
+ if err != nil {
+ return pkgs, err
+ }
+ for _, pgf := range pgfs {
+ b.parsedFiles[pgf.URI] = pgf
+ }
+ b.fset = fset
+ }
+
+ // -- Step 4: await type-checking. --
+ //
+ // Start a single goroutine for each promise.
+ {
+ var g errgroup.Group
+ // TODO(rfindley): find a good way to limit concurrency of type-checking,
+ // which is CPU bound at this point.
+ //
+ // (calling g.SetLimit here is mostly ineffective, as promises are
+ // recursively concurrent.)
+ for _, promise := range b.promises {
+ promise := promise
+ g.Go(func() error {
+ _, err := promise.Get(ctx, nil)
+ return err
+ })
+ }
+ if err := g.Wait(); err != nil {
+ return pkgs, err
+ }
+ }
+
+ // Fill in the gaps of the results slice.
+ var firstErr error
+ for i, id := range ids {
+ if pkgs[i] != nil {
+ continue
+ }
+ if err := b.imports[id].err; err != nil {
+ if firstErr == nil {
+ firstErr = err
+ }
+ continue
+ }
+ pkg := b.packages[id]
+ if pkg == nil {
+ panic("nil package")
+ }
+ if alt := s.memoizeActivePackage(id, pkg); alt != nil && alt != pkg {
+ // pkg is an open package, but we've lost a race and an existing package
+ // has already been memoized.
+ pkg = alt
+ }
+ pkgs[i] = pkg
+ }
+
+ return pkgs, firstErr
+}
+
+// addNeededFiles records the files necessary for type-checking ph, for later
+// parsing.
+func (b *typeCheckBatch) addNeededFiles(ph *packageHandle) {
+ b.mu.Lock()
+ defer b.mu.Unlock()
+
+ // Technically for export-only packages we only need compiledGoFiles, but
+ // these slices are usually redundant.
+ for _, fh := range ph.inputs.goFiles {
+ b.needFiles[fh.URI()] = fh
+ }
+ for _, fh := range ph.inputs.compiledGoFiles {
+ b.needFiles[fh.URI()] = fh
+ }
+}
+
+// importPackage loads the given package from its export data in p.exportData
+// (which must already be populated).
+func (b *typeCheckBatch) importPackage(ctx context.Context, m *source.Metadata, data []byte) (*types.Package, error) {
+ impMap, errMap := b.importMap(m.ID)
+ // Any failure to populate an import will cause confusing errors from
+ // IImportShallow below.
+ for path, err := range errMap {
+ return nil, fmt.Errorf("error importing %q for %q: %v", path, m.ID, err)
+ }
+
+ // TODO(rfindley): collect "deep" hashes here using the provided
+ // callback, for precise pruning.
+ imported, err := gcimporter.IImportShallow(b.fset, impMap, data, string(m.PkgPath), func(*types.Package, string) {})
+ if err != nil {
+ return nil, bug.Errorf("invalid export data for %q: %v", m.ID, err)
+ }
+ return imported, nil
+}
+
+// checkPackageForImport type checks, but skips function bodies and does not
+// record syntax information.
+func (b *typeCheckBatch) checkPackageForImport(ctx context.Context, ph *packageHandle) (*types.Package, error) {
+ if ph.m.ID == "unsafe" {
+ return types.Unsafe, nil
+ }
+ impMap, errMap := b.importMap(ph.inputs.id)
+ onError := func(e error) {
+ // Ignore errors for exporting.
+ }
+ cfg := b.typesConfig(ph.inputs, onError, impMap, errMap)
+ var files []*ast.File
+ for _, fh := range ph.inputs.compiledGoFiles {
+ pgf := b.parsedFiles[fh.URI()]
+ if pgf == nil {
+ return nil, fmt.Errorf("compiled go file %q failed to parse", fh.URI().Filename())
+ }
+ files = append(files, pgf.File)
+ }
+ cfg.IgnoreFuncBodies = true
+ pkg := types.NewPackage(string(ph.inputs.pkgPath), string(ph.inputs.name))
+ check := types.NewChecker(cfg, b.fset, pkg, nil)
+
+ _ = check.Files(files) // ignore errors
+
+ // If the context was cancelled, we may have returned a ton of transient
+ // errors to the type checker. Swallow them.
+ if ctx.Err() != nil {
+ return nil, ctx.Err()
+ }
+
+ // Asynchronously record export data.
+ go func() {
+ exportData, err := gcimporter.IExportShallow(b.fset, pkg)
+ if err != nil {
+ bug.Reportf("exporting package %v: %v", ph.m.ID, err)
+ return
+ }
+ if err := filecache.Set(exportDataKind, ph.key, exportData); err != nil {
+ event.Error(ctx, fmt.Sprintf("storing export data for %s", ph.m.ID), err)
+ }
+ }()
+ return pkg, nil
+}
+
+// checkPackage "fully type checks" to produce a syntax package.
+func (b *typeCheckBatch) checkPackage(ctx context.Context, ph *packageHandle) (*Package, error) {
+ // TODO(rfindley): refactor to inline typeCheckImpl here. There is no need
+ // for so many layers to build up the package
+ // (checkPackage->typeCheckImpl->doTypeCheck).
+ pkg, err := typeCheckImpl(ctx, b, ph.inputs)
+
+ if err == nil {
+ // Write package data to disk asynchronously.
+ go func() {
+ toCache := map[string][]byte{
+ xrefsKind: pkg.xrefs,
+ methodSetsKind: pkg.methodsets.Encode(),
+ diagnosticsKind: encodeDiagnostics(pkg.diagnostics),
+ }
+
+ if ph.m.ID != "unsafe" { // unsafe cannot be exported
+ exportData, err := gcimporter.IExportShallow(pkg.fset, pkg.types)
+ if err != nil {
+ bug.Reportf("exporting package %v: %v", ph.m.ID, err)
+ } else {
+ toCache[exportDataKind] = exportData
+ }
+ }
+
+ for kind, data := range toCache {
+ if err := filecache.Set(kind, ph.key, data); err != nil {
+ event.Error(ctx, fmt.Sprintf("storing %s data for %s", kind, ph.m.ID), err)
+ }
+ }
+ }()
+ }
+
+ return &Package{ph.m, pkg}, err
+}
+
+// awaitPredecessors awaits all promises for m.DepsByPkgPath, returning an
+// error if awaiting failed due to context cancellation or if there was an
+// unrecoverable error loading export data.
+func (b *typeCheckBatch) awaitPredecessors(ctx context.Context, m *source.Metadata) error {
+ for _, depID := range m.DepsByPkgPath {
+ depID := depID
+ if p, ok := b.promises[depID]; ok {
+ if _, err := p.Get(ctx, nil); err != nil {
+ return err
+ }
+ }
+ }
+ return nil
+}
+
+// importMap returns an import map for the given package ID, populated with
+// type-checked packages for its dependencies. It is intended for compatibility
+// with gcimporter.IImportShallow, so the first result uses the map signature
+// of that API, where keys are package path strings.
+//
+// importMap must only be used once all promises for dependencies of id have
+// been awaited.
+//
+// For any missing packages, importMap returns an entry in the resulting errMap
+// reporting the error for that package.
+//
+// Invariant: for all recursive dependencies, either impMap[path] or
+// errMap[path] is set.
+func (b *typeCheckBatch) importMap(id PackageID) (impMap map[string]*types.Package, errMap map[PackagePath]error) {
+ impMap = make(map[string]*types.Package)
+ outerID := id
+ var populateDepsOf func(m *source.Metadata)
+ populateDepsOf = func(parent *source.Metadata) {
+ for _, id := range parent.DepsByPkgPath {
+ m := b.meta.metadata[id]
+ if _, ok := impMap[string(m.PkgPath)]; ok {
+ continue
+ }
+ if _, ok := errMap[m.PkgPath]; ok {
+ continue
+ }
+ b.mu.Lock()
+ result, ok := b.imports[m.ID]
+ b.mu.Unlock()
+ if !ok {
+ panic(fmt.Sprintf("import map for %q missing package data for %q", outerID, m.ID))
+ }
+ // We may fail to produce a package due to e.g. context cancellation
+ // (handled elsewhere), or some catastrophic failure such as a package with
+ // no files.
+ switch {
+ case result.err != nil:
+ if errMap == nil {
+ errMap = make(map[PackagePath]error)
+ }
+ errMap[m.PkgPath] = result.err
+ case result.pkg != nil:
+ impMap[string(m.PkgPath)] = result.pkg
+ default:
+ panic("invalid import for " + id)
+ }
+ populateDepsOf(m)
+ }
+ }
+ m := b.meta.metadata[id]
+ populateDepsOf(m)
+ return impMap, errMap
+}
+
+// packageData holds binary data (e.g. types, xrefs) extracted from a syntax
+// package.
+type packageData struct {
+ m *source.Metadata
+ data []byte
+}
+
+// getPackageData gets package data (e.g. types, xrefs) for the requested ids,
+// either loading from the file-based cache or type-checking and extracting
+// data using the provided get function.
+func (s *snapshot) getPackageData(ctx context.Context, kind string, ids []PackageID, get func(*syntaxPackage) []byte) ([]*packageData, error) {
+ var needIDs []PackageID
+ keys := make([]packageHandleKey, len(ids))
+ pkgData := make([]*packageData, len(ids))
+ var firstErr error
+ // Compute package keys and query file cache.
+ for i, id := range ids {
+ ph, err := s.buildPackageHandle(ctx, id)
+ if err != nil {
+ if firstErr == nil {
+ firstErr = err
+ }
+ if ctx.Err() != nil {
+ return pkgData, firstErr
+ }
+ continue
+ }
+ keys[i] = ph.key
+ data, err := filecache.Get(kind, ph.key)
+ switch err {
+ case nil:
+ pkgData[i] = &packageData{m: ph.m, data: data}
+ case filecache.ErrNotFound:
+ needIDs = append(needIDs, id)
+ default:
+ if firstErr == nil {
+ firstErr = err
+ }
+ if ctx.Err() != nil {
+ return pkgData, firstErr
+ }
+ }
+ }
+
+ // Type-check the packages for which we got file-cache misses.
+ pkgs, err := s.TypeCheck(ctx, needIDs...)
+ if err != nil {
+ return nil, err
+ }
+
+ pkgMap := make(map[PackageID]source.Package)
+ for i, id := range needIDs {
+ pkgMap[id] = pkgs[i]
+ }
+
+ // Fill in the gaps using data derived from type checking.
+ for i, id := range ids {
+ if pkgData[i] != nil {
+ continue
+ }
+ result := pkgMap[id]
+ if result == nil {
+ panic(fmt.Sprintf("missing type-check result for %s", id))
+ }
+ data := get(result.(*Package).pkg)
+ pkgData[i] = &packageData{m: result.Metadata(), data: data}
+ }
+
+ return pkgData, firstErr
}
type packageHandleKey source.Hash
@@ -48,10 +565,10 @@
// The only methods on packageHandle that are safe to call before calling await
// are Metadata and await itself.
type packageHandle struct {
- // TODO(rfindley): remove metadata from packageHandle. It is only used for
- // bug detection.
m *source.Metadata
+ inputs typeCheckInputs
+
// key is the hashed key for the package.
//
// It includes the all bits of the transitive closure of
@@ -61,29 +578,9 @@
// hash only Facts of direct dependencies.)
key packageHandleKey
- // The shared type-checking promise.
- promise *memoize.Promise // [typeCheckResult]
-}
-
-// typeCheckInputs contains the inputs of a call to typeCheckImpl, which
-// type-checks a package.
-type typeCheckInputs struct {
- id PackageID
- pkgPath PackagePath
- name PackageName
- mode source.ParseMode
- goFiles, compiledGoFiles []source.FileHandle
- sizes types.Sizes
- deps map[PackageID]*packageHandle
- depsByImpPath map[ImportPath]PackageID
- goVersion string // packages.Module.GoVersion, e.g. "1.18"
-}
-
-// typeCheckResult contains the result of a call to
-// typeCheckImpl, which type-checks a package.
-type typeCheckResult struct {
- pkg *syntaxPackage
- err error
+ // Note: as an optimization, we could join in-flight type-checking by
+ // recording a transient ref-counted promise here.
+ // (This was done previously, but proved to be a premature optimization).
}
// buildPackageHandle returns a handle for the future results of
@@ -91,11 +588,9 @@
// It assumes that the given ID already has metadata available, so it does not
// attempt to reload missing or invalid metadata. The caller must reload
// metadata if needed.
-func (s *snapshot) buildPackageHandle(ctx context.Context, id PackageID, mode source.ParseMode) (*packageHandle, error) {
- packageKey := packageKey{id: id, mode: mode}
-
+func (s *snapshot) buildPackageHandle(ctx context.Context, id PackageID) (*packageHandle, error) {
s.mu.Lock()
- entry, hit := s.packages.Get(packageKey)
+ entry, hit := s.packages.Get(id)
m := s.meta.metadata[id]
s.mu.Unlock()
@@ -107,83 +602,17 @@
return entry.(*packageHandle), nil
}
- // Begin computing the key by getting the depKeys for all dependencies.
- // This requires reading the transitive closure of dependencies' source files.
- //
- // It is tempting to parallelize the recursion here, but
- // without de-duplication of subtasks this would lead to an
- // exponential amount of work, and computing the key is
- // expensive as it reads all the source files transitively.
- // Notably, we don't update the s.packages cache until the
- // entire key has been computed.
- // TODO(adonovan): use a promise cache to ensure that the key
- // for each package is computed by at most one thread, then do
- // the recursive key building of dependencies in parallel.
- deps := make(map[PackageID]*packageHandle)
- for _, depID := range m.DepsByPkgPath {
- depHandle, err := s.buildPackageHandle(ctx, depID, s.workspaceParseMode(depID))
- // Don't use invalid metadata for dependencies if the top-level
- // metadata is valid. We only load top-level packages, so if the
- // top-level is valid, all of its dependencies should be as well.
- if err != nil {
- event.Error(ctx, fmt.Sprintf("%s: no dep handle for %s", id, depID), err, source.SnapshotLabels(s)...)
-
- // This check ensures we break out of the slow
- // buildPackageHandle recursion quickly when
- // context cancelation is detected within GetFile.
- if ctx.Err() != nil {
- return nil, ctx.Err() // cancelled
- }
-
- // One bad dependency should not prevent us from
- // checking the entire package. Leave depKeys[i] unset.
- continue
- }
- deps[depID] = depHandle
- }
-
- // Read both lists of files of this package, in parallel.
- //
- // goFiles aren't presented to the type checker--nor
- // are they included in the key, unsoundly--but their
- // syntax trees are available from (*pkg).File(URI).
- // TODO(adonovan): consider parsing them on demand?
- // The need should be rare.
- goFiles, compiledGoFiles, err := readGoFiles(ctx, s, m)
+ inputs, err := s.typeCheckInputs(ctx, m)
if err != nil {
return nil, err
}
-
- goVersion := ""
- if m.Module != nil && m.Module.GoVersion != "" {
- goVersion = m.Module.GoVersion
- }
-
- inputs := typeCheckInputs{
- id: m.ID,
- pkgPath: m.PkgPath,
- name: m.Name,
- mode: mode,
- goFiles: goFiles,
- compiledGoFiles: compiledGoFiles,
- sizes: m.TypesSizes,
- deps: deps,
- depsByImpPath: m.DepsByImpPath,
- goVersion: goVersion,
- }
-
// All the file reading has now been done.
// Create a handle for the result of type checking.
phKey := computePackageKey(s, inputs)
- promise, release := s.store.Promise(phKey, func(ctx context.Context, arg interface{}) interface{} {
- pkg, err := typeCheckImpl(ctx, arg.(*snapshot), inputs)
- return typeCheckResult{pkg, err}
- })
-
ph := &packageHandle{
- promise: promise,
- m: m,
- key: phKey,
+ m: m,
+ inputs: inputs,
+ key: phKey,
}
s.mu.Lock()
@@ -202,21 +631,100 @@
}
// Check cache again in case another goroutine got there first.
- if prev, ok := s.packages.Get(packageKey); ok {
+ if prev, ok := s.packages.Get(id); ok {
prevPH := prev.(*packageHandle)
- release()
if prevPH.m != ph.m {
return nil, bug.Errorf("existing package handle does not match for %s", ph.m.ID)
}
return prevPH, nil
}
- // Update the map.
- s.packages.Set(packageKey, ph, func(_, _ interface{}) { release() })
-
+ s.packages.Set(id, ph, nil)
return ph, nil
}
+// typeCheckInputs contains the inputs of a call to typeCheckImpl, which
+// type-checks a package.
+//
+// Part of the purpose of this type is to keep type checking in-sync with the
+// package handle key, by explicitly identifying the inputs to type checking.
+type typeCheckInputs struct {
+ id PackageID
+
+ // Used for type checking:
+ pkgPath PackagePath
+ name PackageName
+ goFiles, compiledGoFiles []source.FileHandle
+ sizes types.Sizes
+ deps map[PackageID]*packageHandle
+ depsByImpPath map[ImportPath]PackageID
+ goVersion string // packages.Module.GoVersion, e.g. "1.18"
+
+ // Used for type check diagnostics:
+ relatedInformation bool
+ linkTarget string
+ moduleMode bool
+}
+
+func (s *snapshot) typeCheckInputs(ctx context.Context, m *source.Metadata) (typeCheckInputs, error) {
+ deps := make(map[PackageID]*packageHandle)
+ for _, depID := range m.DepsByPkgPath {
+ depHandle, err := s.buildPackageHandle(ctx, depID)
+ if err != nil {
+ // If err is non-nil, we either have an invalid dependency, or a
+ // catastrophic failure to read a file (context cancellation or
+ // permission issues).
+ //
+ // We don't want one bad dependency to prevent us from type-checking the
+ // package -- we should instead get an import error. So we only abort
+ // this operation if the context is cancelled.
+ //
+ // We could do a better job of handling permission errors on files, but
+ // this is rare, and it is reasonable to treat the same an invalid
+ // dependency.
+ event.Error(ctx, fmt.Sprintf("%s: no dep handle for %s", m.ID, depID), err, source.SnapshotLabels(s)...)
+ if ctx.Err() != nil {
+ return typeCheckInputs{}, ctx.Err() // cancelled
+ }
+ continue
+ }
+ deps[depID] = depHandle
+ }
+
+ // Read both lists of files of this package, in parallel.
+ //
+ // goFiles aren't presented to the type checker--nor
+ // are they included in the key, unsoundly--but their
+ // syntax trees are available from (*pkg).File(URI).
+ // TODO(adonovan): consider parsing them on demand?
+ // The need should be rare.
+ goFiles, compiledGoFiles, err := readGoFiles(ctx, s, m)
+ if err != nil {
+ return typeCheckInputs{}, err
+ }
+
+ goVersion := ""
+ if m.Module != nil && m.Module.GoVersion != "" {
+ goVersion = m.Module.GoVersion
+ }
+
+ return typeCheckInputs{
+ id: m.ID,
+ pkgPath: m.PkgPath,
+ name: m.Name,
+ goFiles: goFiles,
+ compiledGoFiles: compiledGoFiles,
+ sizes: m.TypesSizes,
+ deps: deps,
+ depsByImpPath: m.DepsByImpPath,
+ goVersion: goVersion,
+
+ relatedInformation: s.view.Options().RelatedInformationSupported,
+ linkTarget: s.view.Options().LinkTarget,
+ moduleMode: s.moduleMode(),
+ }, nil
+}
+
// readGoFiles reads the content of Metadata.GoFiles and
// Metadata.CompiledGoFiles, in parallel.
func readGoFiles(ctx context.Context, s *snapshot, m *source.Metadata) (goFiles, compiledGoFiles []source.FileHandle, err error) {
@@ -237,22 +745,6 @@
group.Wait()
}
-func (s *snapshot) workspaceParseMode(id PackageID) source.ParseMode {
- s.mu.Lock()
- defer s.mu.Unlock()
- _, ws := s.workspacePackages[id]
- if !ws {
- return source.ParseExported
- }
- if s.view.Options().MemoryMode == source.ModeNormal {
- return source.ParseFull
- }
- if s.isActiveLocked(id) {
- return source.ParseFull
- }
- return source.ParseExported
-}
-
// computePackageKey returns a key representing the act of type checking
// a package named id containing the specified files, metadata, and
// combined dependency hash.
@@ -269,9 +761,6 @@
// module Go version
fmt.Fprintf(hasher, "go %s\n", inputs.goVersion)
- // parse mode
- fmt.Fprintf(hasher, "mode %d\n", inputs.mode)
-
// import map
importPaths := make([]string, 0, len(inputs.depsByImpPath))
for impPath := range inputs.depsByImpPath {
@@ -307,87 +796,28 @@
sz := inputs.sizes.(*types.StdSizes)
fmt.Fprintf(hasher, "sizes: %d %d\n", sz.WordSize, sz.MaxAlign)
+ fmt.Fprintf(hasher, "relatedInformation: %t\n", inputs.relatedInformation)
+ fmt.Fprintf(hasher, "linkTarget: %s\n", inputs.linkTarget)
+ fmt.Fprintf(hasher, "moduleMode: %t\n", inputs.moduleMode)
+
var hash [sha256.Size]byte
hasher.Sum(hash[:0])
return packageHandleKey(hash)
}
-// await waits for typeCheckImpl to complete and returns its result.
-func (ph *packageHandle) await(ctx context.Context, s *snapshot) (*syntaxPackage, error) {
- v, err := s.awaitPromise(ctx, ph.promise)
- if err != nil {
- return nil, err
- }
- data := v.(typeCheckResult)
- return data.pkg, data.err
-}
-
-func (ph *packageHandle) cached() (*syntaxPackage, error) {
- v := ph.promise.Cached()
- if v == nil {
- return nil, fmt.Errorf("no cached type information for %s", ph.m.PkgPath)
- }
- data := v.(typeCheckResult)
- return data.pkg, data.err
-}
-
// typeCheckImpl type checks the parsed source files in compiledGoFiles.
// (The resulting pkg also holds the parsed but not type-checked goFiles.)
// deps holds the future results of type-checking the direct dependencies.
-func typeCheckImpl(ctx context.Context, snapshot *snapshot, inputs typeCheckInputs) (*syntaxPackage, error) {
- // Start type checking of direct dependencies,
- // in parallel and asynchronously.
- // As the type checker imports each of these
- // packages, it will wait for its completion.
- var wg sync.WaitGroup
- for _, dep := range inputs.deps {
- wg.Add(1)
- go func(dep *packageHandle) {
- dep.await(ctx, snapshot) // ignore result
- wg.Done()
- }(dep)
- }
- // The 'defer' below is unusual but intentional:
- // it is not necessary that each call to dep.check
- // complete before type checking begins, as the type
- // checker will wait for those it needs. But they do
- // need to complete before this function returns and
- // the snapshot is possibly destroyed.
- defer wg.Wait()
+func typeCheckImpl(ctx context.Context, b *typeCheckBatch, inputs typeCheckInputs) (*syntaxPackage, error) {
+ ctx, done := event.Start(ctx, "cache.typeCheck", tag.Package.Of(string(inputs.id)))
+ defer done()
- var filter *unexportedFilter
- if inputs.mode == source.ParseExported {
- filter = &unexportedFilter{uses: map[string]bool{}}
- }
- pkg, err := doTypeCheck(ctx, snapshot, inputs, filter)
+ pkg, err := doTypeCheck(ctx, b, inputs)
if err != nil {
return nil, err
}
-
- if inputs.mode == source.ParseExported {
- // The AST filtering is a little buggy and may remove things it
- // shouldn't. If we only got undeclared name errors, try one more
- // time keeping those names.
- missing, unexpected := filter.ProcessErrors(pkg.typeErrors)
- if len(unexpected) == 0 && len(missing) != 0 {
- pkg, err = doTypeCheck(ctx, snapshot, inputs, filter)
- if err != nil {
- return nil, err
- }
- missing, unexpected = filter.ProcessErrors(pkg.typeErrors)
- }
- if len(unexpected) != 0 || len(missing) != 0 {
- pkg, err = doTypeCheck(ctx, snapshot, inputs, nil)
- if err != nil {
- return nil, err
- }
- }
- }
-
- // We don't care about a package's errors unless we have parsed it in full.
- if inputs.mode != source.ParseFull {
- return pkg, nil
- }
+ pkg.methodsets = methodsets.NewIndex(pkg.fset, pkg.types)
+ pkg.xrefs = xrefs.Index(pkg.compiledGoFiles, pkg.types, pkg.typesInfo)
// Our heuristic for whether to show type checking errors is:
// + If any file was 'fixed', don't show type checking errors as we
@@ -403,7 +833,7 @@
// files.
unparseable := map[span.URI]bool{}
for _, e := range pkg.parseErrors {
- diags, err := parseErrorDiagnostics(snapshot, pkg, e)
+ diags, err := parseErrorDiagnostics(pkg, e)
if err != nil {
event.Error(ctx, "unable to compute positions for parse errors", err, tag.Package.Of(string(inputs.id)))
continue
@@ -420,8 +850,8 @@
unexpanded := pkg.typeErrors
pkg.typeErrors = nil
- for _, e := range expandErrors(unexpanded, snapshot.View().Options().RelatedInformationSupported) {
- diags, err := typeErrorDiagnostics(snapshot, pkg, e)
+ for _, e := range expandErrors(unexpanded, inputs.relatedInformation) {
+ diags, err := typeErrorDiagnostics(inputs.moduleMode, inputs.linkTarget, pkg, e)
if err != nil {
event.Error(ctx, "unable to compute positions for type errors", err, tag.Package.Of(string(inputs.id)))
continue
@@ -442,16 +872,12 @@
var goVersionRx = regexp.MustCompile(`^go([1-9][0-9]*)\.(0|[1-9][0-9]*)$`)
-func doTypeCheck(ctx context.Context, snapshot *snapshot, inputs typeCheckInputs, astFilter *unexportedFilter) (*syntaxPackage, error) {
- ctx, done := event.Start(ctx, "cache.typeCheck", tag.Package.Of(string(inputs.id)))
- defer done()
-
+func doTypeCheck(ctx context.Context, b *typeCheckBatch, inputs typeCheckInputs) (*syntaxPackage, error) {
+ impMap, errMap := b.importMap(inputs.id)
pkg := &syntaxPackage{
- id: inputs.id,
- mode: inputs.mode,
- fset: snapshot.view.fset, // must match parse call below (snapshot.ParseGo for now)
- types: types.NewPackage(string(inputs.pkgPath), string(inputs.name)),
- importMap: new(importMap),
+ id: inputs.id,
+ fset: b.fset, // must match parse call below
+ types: types.NewPackage(string(inputs.pkgPath), string(inputs.name)),
typesInfo: &types.Info{
Types: make(map[ast.Expr]types.TypeAndValue),
Defs: make(map[*ast.Ident]types.Object),
@@ -460,41 +886,37 @@
Selections: make(map[*ast.SelectorExpr]*types.Selection),
Scopes: make(map[ast.Node]*types.Scope),
},
+ importMap: impMap,
}
typeparams.InitInstanceInfo(pkg.typesInfo)
- defer func() { pkg.importMap.types = pkg.types }() // simplifies early return in "unsafe"
- // Parse the non-compiled GoFiles. (These aren't presented to
- // the type checker but are part of the returned pkg.)
- // TODO(adonovan): opt: parallelize parsing.
+ // Collect parsed files from the type check pass, capturing parse errors from
+ // compiled files.
for _, fh := range inputs.goFiles {
- goMode := inputs.mode
- if inputs.mode == source.ParseExported {
- // This package is being loaded only for type information,
- // to which non-compiled Go files are irrelevant,
- // so parse only the header.
- goMode = source.ParseHeader
- }
- pgf, err := snapshot.ParseGo(ctx, fh, goMode)
- if err != nil {
- return nil, err
+ pgf := b.parsedFiles[fh.URI()]
+ if pgf == nil {
+ // If go/packages told us that a file is in a package, it should be
+ // parseable (after all, it was parsed by go list).
+ return nil, bug.Errorf("go file %q failed to parse", fh.URI().Filename())
}
pkg.goFiles = append(pkg.goFiles, pgf)
}
-
- // Parse the CompiledGoFiles: those seen by the compiler/typechecker.
- if err := parseCompiledGoFiles(ctx, inputs.compiledGoFiles, snapshot, inputs.mode, pkg, astFilter); err != nil {
- return nil, err
+ for _, fh := range inputs.compiledGoFiles {
+ pgf := b.parsedFiles[fh.URI()]
+ if pgf == nil {
+ return nil, fmt.Errorf("compiled go file %q failed to parse", fh.URI().Filename())
+ }
+ if pgf.ParseErr != nil {
+ pkg.parseErrors = append(pkg.parseErrors, pgf.ParseErr)
+ }
+ pkg.compiledGoFiles = append(pkg.compiledGoFiles, pgf)
}
// Use the default type information for the unsafe package.
if inputs.pkgPath == "unsafe" {
// Don't type check Unsafe: it's unnecessary, and doing so exposes a data
// race to Unsafe.completed.
- // TODO(adonovan): factor (tail-merge) with the normal control path.
pkg.types = types.Unsafe
- pkg.methodsets = methodsets.NewIndex(pkg.fset, pkg.types)
- pkg.xrefs = xrefs.Index(pkg.compiledGoFiles, pkg.types, pkg.typesInfo)
return pkg, nil
}
@@ -507,11 +929,33 @@
return nil, fmt.Errorf("no parsed files for package %s", inputs.pkgPath)
}
+ onError := func(e error) {
+ pkg.typeErrors = append(pkg.typeErrors, e.(types.Error))
+ }
+ cfg := b.typesConfig(inputs, onError, impMap, errMap)
+
+ check := types.NewChecker(cfg, pkg.fset, pkg.types, pkg.typesInfo)
+
+ var files []*ast.File
+ for _, cgf := range pkg.compiledGoFiles {
+ files = append(files, cgf.File)
+ }
+
+ // Type checking errors are handled via the config, so ignore them here.
+ _ = check.Files(files) // 50us-15ms, depending on size of package
+
+ // If the context was cancelled, we may have returned a ton of transient
+ // errors to the type checker. Swallow them.
+ if ctx.Err() != nil {
+ return nil, ctx.Err()
+ }
+ return pkg, nil
+}
+
+func (b *typeCheckBatch) typesConfig(inputs typeCheckInputs, onError func(e error), impMap map[string]*types.Package, errMap map[PackagePath]error) *types.Config {
cfg := &types.Config{
Sizes: inputs.sizes,
- Error: func(e error) {
- pkg.typeErrors = append(pkg.typeErrors, e.(types.Error))
- },
+ Error: onError,
Importer: importerFunc(func(path string) (*types.Package, error) {
// While all of the import errors could be reported
// based on the metadata before we start type checking,
@@ -524,19 +968,23 @@
// See TestFixImportDecl for an example.
return nil, fmt.Errorf("missing metadata for import of %q", path)
}
- dep, ok := inputs.deps[id] // id may be ""
+ depPH := inputs.deps[id]
+ if depPH == nil {
+ // e.g. missing metadata for dependencies in buildPackageHandle
+ return nil, missingPkgError(path, inputs.moduleMode)
+ }
+ if !source.IsValidImport(inputs.pkgPath, depPH.m.PkgPath) {
+ return nil, fmt.Errorf("invalid use of internal package %q", path)
+ }
+ pkg, ok := impMap[string(depPH.m.PkgPath)]
if !ok {
- return nil, snapshot.missingPkgError(path)
- }
- if !source.IsValidImport(inputs.pkgPath, dep.m.PkgPath) {
- return nil, fmt.Errorf("invalid use of internal package %s", path)
- }
- depPkg, err := dep.await(ctx, snapshot)
- if err != nil {
+ err := errMap[depPH.m.PkgPath]
+ if err == nil {
+ log.Fatalf("neither pkg nor error is set")
+ }
return nil, err
}
- pkg.importMap.union(depPkg.importMap)
- return depPkg.types, nil
+ return pkg, nil
}),
}
@@ -550,86 +998,10 @@
}
}
- if inputs.mode != source.ParseFull {
- cfg.DisableUnusedImportCheck = true
- cfg.IgnoreFuncBodies = true
- }
-
// We want to type check cgo code if go/types supports it.
// We passed typecheckCgo to go/packages when we Loaded.
typesinternal.SetUsesCgo(cfg)
-
- check := types.NewChecker(cfg, pkg.fset, pkg.types, pkg.typesInfo)
-
- var files []*ast.File
- for _, cgf := range pkg.compiledGoFiles {
- files = append(files, cgf.File)
- }
-
- // Type checking errors are handled via the config, so ignore them here.
- _ = check.Files(files) // 50us-15ms, depending on size of package
-
- // Build global index of method sets for 'implementations' queries.
- pkg.methodsets = methodsets.NewIndex(pkg.fset, pkg.types)
-
- // Build global index of outbound cross-references.
- pkg.xrefs = xrefs.Index(pkg.compiledGoFiles, pkg.types, pkg.typesInfo)
-
- // If the context was cancelled, we may have returned a ton of transient
- // errors to the type checker. Swallow them.
- if ctx.Err() != nil {
- return nil, ctx.Err()
- }
- return pkg, nil
-}
-
-func parseCompiledGoFiles(ctx context.Context, compiledGoFiles []source.FileHandle, snapshot *snapshot, mode source.ParseMode, pkg *syntaxPackage, astFilter *unexportedFilter) error {
- // TODO(adonovan): opt: parallelize this loop, which takes 1-25ms.
- for _, fh := range compiledGoFiles {
- var pgf *source.ParsedGoFile
- var err error
- // Only parse Full through the cache -- we need to own Exported ASTs
- // to prune them.
- if mode == source.ParseFull {
- pgf, err = snapshot.ParseGo(ctx, fh, mode)
- } else {
- pgf, err = parseGoImpl(ctx, pkg.fset, fh, mode) // ~20us/KB
- }
- if err != nil {
- return err
- }
- pkg.compiledGoFiles = append(pkg.compiledGoFiles, pgf)
- if pgf.ParseErr != nil {
- pkg.parseErrors = append(pkg.parseErrors, pgf.ParseErr)
- }
- // If we have fixed parse errors in any of the files, we should hide type
- // errors, as they may be completely nonsensical.
- pkg.hasFixedFiles = pkg.hasFixedFiles || pgf.Fixed
- }
-
- // Optionally remove parts that don't affect the exported API.
- if mode == source.ParseExported {
- // TODO(adonovan): opt: experiment with pre-parser
- // trimming, either a scanner-based implementation
- // such as https://go.dev/play/p/KUrObH1YkX8 (~31%
- // speedup), or a byte-oriented implementation (2x
- // speedup).
- if astFilter != nil {
- // aggressive pruning based on reachability
- var files []*ast.File
- for _, cgf := range pkg.compiledGoFiles {
- files = append(files, cgf.File)
- }
- astFilter.Filter(files)
- } else {
- // simple trimming of function bodies
- for _, cgf := range pkg.compiledGoFiles {
- trimAST(cgf.File)
- }
- }
- }
-
- return nil
+ return cfg
}
// depsErrors creates diagnostics for each metadata error (e.g. import cycle).
@@ -768,22 +1140,17 @@
// missingPkgError returns an error message for a missing package that varies
// based on the user's workspace mode.
-func (s *snapshot) missingPkgError(pkgPath string) error {
- var b strings.Builder
- if s.workspaceMode()&moduleMode == 0 {
- gorootSrcPkg := filepath.FromSlash(filepath.Join(s.view.goroot, "src", pkgPath))
- fmt.Fprintf(&b, "cannot find package %q in any of \n\t%s (from $GOROOT)", pkgPath, gorootSrcPkg)
- for _, gopath := range filepath.SplitList(s.view.gopath) {
- gopathSrcPkg := filepath.FromSlash(filepath.Join(gopath, "src", pkgPath))
- fmt.Fprintf(&b, "\n\t%s (from $GOPATH)", gopathSrcPkg)
- }
+func missingPkgError(pkgPath string, moduleMode bool) error {
+ // TODO(rfindley): improve this error. Previous versions of this error had
+ // access to the full snapshot, and could provide more information (such as
+ // the initialization error).
+ if moduleMode {
+ // Previously, we would present the initialization error here.
+ return fmt.Errorf("no required module provides package %q", pkgPath)
} else {
- fmt.Fprintf(&b, "no required module provides package %q", pkgPath)
- if err := s.getInitializationError(); err != nil {
- fmt.Fprintf(&b, "\n(workspace configuration error: %s)", err.MainError)
- }
+ // Previously, we would list the directories in GOROOT and GOPATH here.
+ return fmt.Errorf("cannot find package %q in GOROOT or GOPATH", pkgPath)
}
- return errors.New(b.String())
}
type extendedError struct {
diff --git a/gopls/internal/lsp/cache/errors.go b/gopls/internal/lsp/cache/errors.go
index 2d21705..07783f4 100644
--- a/gopls/internal/lsp/cache/errors.go
+++ b/gopls/internal/lsp/cache/errors.go
@@ -85,7 +85,7 @@
}}, nil
}
-func parseErrorDiagnostics(snapshot *snapshot, pkg *syntaxPackage, errList scanner.ErrorList) ([]*source.Diagnostic, error) {
+func parseErrorDiagnostics(pkg *syntaxPackage, errList scanner.ErrorList) ([]*source.Diagnostic, error) {
// The first parser error is likely the root cause of the problem.
if errList.Len() <= 0 {
return nil, fmt.Errorf("no errors in %v", errList)
@@ -111,7 +111,7 @@
var importErrorRe = regexp.MustCompile(`could not import ([^\s]+)`)
var unsupportedFeatureRe = regexp.MustCompile(`.*require.* go(\d+\.\d+) or later`)
-func typeErrorDiagnostics(snapshot *snapshot, pkg *syntaxPackage, e extendedError) ([]*source.Diagnostic, error) {
+func typeErrorDiagnostics(moduleMode bool, linkTarget string, pkg *syntaxPackage, e extendedError) ([]*source.Diagnostic, error) {
code, loc, err := typeErrorData(pkg, e.primary)
if err != nil {
return nil, err
@@ -125,7 +125,7 @@
}
if code != 0 {
diag.Code = code.String()
- diag.CodeHref = typesCodeHref(snapshot, code)
+ diag.CodeHref = typesCodeHref(linkTarget, code)
}
switch code {
case typesinternal.UnusedVar, typesinternal.UnusedImport:
@@ -144,13 +144,13 @@
}
if match := importErrorRe.FindStringSubmatch(e.primary.Msg); match != nil {
- diag.SuggestedFixes, err = goGetQuickFixes(snapshot.moduleMode(), loc.URI.SpanURI(), match[1])
+ diag.SuggestedFixes, err = goGetQuickFixes(moduleMode, loc.URI.SpanURI(), match[1])
if err != nil {
return nil, err
}
}
if match := unsupportedFeatureRe.FindStringSubmatch(e.primary.Msg); match != nil {
- diag.SuggestedFixes, err = editGoDirectiveQuickFix(snapshot, loc.URI.SpanURI(), match[1])
+ diag.SuggestedFixes, err = editGoDirectiveQuickFix(moduleMode, loc.URI.SpanURI(), match[1])
if err != nil {
return nil, err
}
@@ -159,6 +159,7 @@
}
func goGetQuickFixes(moduleMode bool, uri span.URI, pkg string) ([]source.SuggestedFix, error) {
+ // Go get only supports module mode for now.
if !moduleMode {
return nil, nil
}
@@ -174,9 +175,9 @@
return []source.SuggestedFix{source.SuggestedFixFromCommand(cmd, protocol.QuickFix)}, nil
}
-func editGoDirectiveQuickFix(snapshot *snapshot, uri span.URI, version string) ([]source.SuggestedFix, error) {
+func editGoDirectiveQuickFix(moduleMode bool, uri span.URI, version string) ([]source.SuggestedFix, error) {
// Go mod edit only supports module mode.
- if snapshot.workspaceMode()&moduleMode == 0 {
+ if !moduleMode {
return nil, nil
}
title := fmt.Sprintf("go mod edit -go=%s", version)
@@ -190,6 +191,112 @@
return []source.SuggestedFix{source.SuggestedFixFromCommand(cmd, protocol.QuickFix)}, nil
}
+// encodeDiagnostics gob-encodes the given diagnostics.
+func encodeDiagnostics(srcDiags []*source.Diagnostic) []byte {
+ var gobDiags []gobDiagnostic
+ for _, srcDiag := range srcDiags {
+ var gobFixes []gobSuggestedFix
+ for _, srcFix := range srcDiag.SuggestedFixes {
+ gobFix := gobSuggestedFix{
+ Message: srcFix.Title,
+ ActionKind: srcFix.ActionKind,
+ }
+ for uri, srcEdits := range srcFix.Edits {
+ for _, srcEdit := range srcEdits {
+ gobFix.TextEdits = append(gobFix.TextEdits, gobTextEdit{
+ Location: protocol.Location{
+ URI: protocol.URIFromSpanURI(uri),
+ Range: srcEdit.Range,
+ },
+ NewText: []byte(srcEdit.NewText),
+ })
+ }
+ }
+ if srcCmd := srcFix.Command; srcCmd != nil {
+ gobFix.Command = &gobCommand{
+ Title: srcCmd.Title,
+ Command: srcCmd.Command,
+ Arguments: srcCmd.Arguments,
+ }
+ }
+ gobFixes = append(gobFixes, gobFix)
+ }
+ var gobRelated []gobRelatedInformation
+ for _, srcRel := range srcDiag.Related {
+ gobRel := gobRelatedInformation(srcRel)
+ gobRelated = append(gobRelated, gobRel)
+ }
+ gobDiag := gobDiagnostic{
+ Location: protocol.Location{
+ URI: protocol.URIFromSpanURI(srcDiag.URI),
+ Range: srcDiag.Range,
+ },
+ Severity: srcDiag.Severity,
+ Code: srcDiag.Code,
+ CodeHref: srcDiag.CodeHref,
+ Source: string(srcDiag.Source),
+ Message: srcDiag.Message,
+ SuggestedFixes: gobFixes,
+ Related: gobRelated,
+ Tags: srcDiag.Tags,
+ }
+ gobDiags = append(gobDiags, gobDiag)
+ }
+ return mustEncode(gobDiags)
+}
+
+// decodeDiagnostics decodes the given gob-encoded diagnostics.
+func decodeDiagnostics(data []byte) []*source.Diagnostic {
+ var gobDiags []gobDiagnostic
+ mustDecode(data, &gobDiags)
+ var srcDiags []*source.Diagnostic
+ for _, gobDiag := range gobDiags {
+ var srcFixes []source.SuggestedFix
+ for _, gobFix := range gobDiag.SuggestedFixes {
+ srcFix := source.SuggestedFix{
+ Title: gobFix.Message,
+ ActionKind: gobFix.ActionKind,
+ }
+ for _, gobEdit := range gobFix.TextEdits {
+ if srcFix.Edits == nil {
+ srcFix.Edits = make(map[span.URI][]protocol.TextEdit)
+ }
+ srcEdit := protocol.TextEdit{
+ Range: gobEdit.Location.Range,
+ NewText: string(gobEdit.NewText),
+ }
+ uri := gobEdit.Location.URI.SpanURI()
+ srcFix.Edits[uri] = append(srcFix.Edits[uri], srcEdit)
+ }
+ if gobCmd := gobFix.Command; gobCmd != nil {
+ gobFix.Command = &gobCommand{
+ Title: gobCmd.Title,
+ Command: gobCmd.Command,
+ Arguments: gobCmd.Arguments,
+ }
+ }
+ srcFixes = append(srcFixes, srcFix)
+ }
+ var srcRelated []protocol.DiagnosticRelatedInformation
+ for _, gobRel := range gobDiag.Related {
+ srcRel := protocol.DiagnosticRelatedInformation(gobRel)
+ srcRelated = append(srcRelated, srcRel)
+ }
+ srcDiag := &source.Diagnostic{
+ URI: gobDiag.Location.URI.SpanURI(),
+ Range: gobDiag.Location.Range,
+ Severity: gobDiag.Severity,
+ Source: source.AnalyzerErrorKind(gobDiag.Source),
+ Message: gobDiag.Message,
+ Tags: gobDiag.Tags,
+ Related: srcRelated,
+ SuggestedFixes: srcFixes,
+ }
+ srcDiags = append(srcDiags, srcDiag)
+ }
+ return srcDiags
+}
+
// toSourceDiagnostic converts a gobDiagnostic to "source" form.
func toSourceDiagnostic(srcAnalyzer *source.Analyzer, gobDiag *gobDiagnostic) *source.Diagnostic {
var related []protocol.DiagnosticRelatedInformation
@@ -223,11 +330,10 @@
}
diag := &source.Diagnostic{
- // TODO(adonovan): is this sound? See dual conversion in posToLocation.
- URI: span.URI(gobDiag.Location.URI),
+ URI: gobDiag.Location.URI.SpanURI(),
Range: gobDiag.Location.Range,
Severity: severity,
- Source: source.AnalyzerErrorKind(gobDiag.Category),
+ Source: source.AnalyzerErrorKind(gobDiag.Source),
Message: gobDiag.Message,
Related: related,
SuggestedFixes: fixes,
@@ -259,9 +365,8 @@
return len(fixes) > 0
}
-func typesCodeHref(snapshot *snapshot, code typesinternal.ErrorCode) string {
- target := snapshot.View().Options().LinkTarget
- return source.BuildLink(target, "golang.org/x/tools/internal/typesinternal", code.String())
+func typesCodeHref(linkTarget string, code typesinternal.ErrorCode) string {
+ return source.BuildLink(linkTarget, "golang.org/x/tools/internal/typesinternal", code.String())
}
func suggestedAnalysisFixes(diag *gobDiagnostic, kinds []protocol.CodeActionKind) []source.SuggestedFix {
diff --git a/gopls/internal/lsp/cache/graph.go b/gopls/internal/lsp/cache/graph.go
index 8e9e5d9..f304112 100644
--- a/gopls/internal/lsp/cache/graph.go
+++ b/gopls/internal/lsp/cache/graph.go
@@ -25,6 +25,11 @@
ids map[span.URI][]PackageID
}
+// Metadata implements the source.MetadataSource interface.
+func (g *metadataGraph) Metadata(id PackageID) *source.Metadata {
+ return g.metadata[id]
+}
+
// Clone creates a new metadataGraph, applying the given updates to the
// receiver.
func (g *metadataGraph) Clone(updates map[PackageID]*source.Metadata) *metadataGraph {
@@ -49,6 +54,8 @@
}
// build constructs g.importedBy and g.uris from g.metadata.
+//
+// TODO(rfindley): we should enforce that the graph is acyclic here.
func (g *metadataGraph) build() {
// Build the import graph.
g.importedBy = make(map[PackageID][]PackageID)
diff --git a/gopls/internal/lsp/cache/load.go b/gopls/internal/lsp/cache/load.go
index 22af16c..c4bdf1c 100644
--- a/gopls/internal/lsp/cache/load.go
+++ b/gopls/internal/lsp/cache/load.go
@@ -220,8 +220,8 @@
}
// Assert the invariant.
s.packages.Range(func(k, v interface{}) {
- pk, ph := k.(packageKey), v.(*packageHandle)
- if s.meta.metadata[pk.id] != ph.m {
+ id, ph := k.(PackageID), v.(*packageHandle)
+ if s.meta.metadata[id] != ph.m {
// TODO(adonovan): upgrade to unconditional panic after Jan 2023.
bug.Reportf("inconsistent metadata")
}
@@ -241,18 +241,24 @@
}
s.meta = meta
s.workspacePackages = workspacePackages
+ s.resetActivePackagesLocked()
- s.resetIsActivePackageLocked()
s.dumpWorkspace("load")
s.mu.Unlock()
// Recompute the workspace package handle for any packages we invalidated.
//
- // This is (putatively) an optimization since handle
- // construction prefetches the content of all Go source files.
- // It is safe to ignore errors, or omit this step entirely.
+ // This is (putatively) an optimization since handle construction prefetches
+ // the content of all Go source files.
+ //
+ // However, one necessary side effect of this operation is that we are
+ // guaranteed to visit all package files during load. This is required for
+ // e.g. determining the set of directories to watch.
+ //
+ // TODO(rfindley, golang/go#57558): determine the set of directories based on
+ // loaded packages, and skip this precomputation.
for _, m := range updates {
- s.buildPackageHandle(ctx, m.ID, s.workspaceParseMode(m.ID)) // ignore error
+ s.buildPackageHandle(ctx, m.ID) // ignore error
}
if len(moduleErrs) > 0 {
diff --git a/gopls/internal/lsp/cache/maps.go b/gopls/internal/lsp/cache/maps.go
index baa0deb..0ad4ac9 100644
--- a/gopls/internal/lsp/cache/maps.go
+++ b/gopls/internal/lsp/cache/maps.go
@@ -61,105 +61,8 @@
m.impl.Delete(key)
}
-func parseKeyLessInterface(a, b interface{}) bool {
- return parseKeyLess(a.(parseKey), b.(parseKey))
-}
-
-func parseKeyLess(a, b parseKey) bool {
- if a.mode != b.mode {
- return a.mode < b.mode
- }
- if a.file.Hash != b.file.Hash {
- return a.file.Hash.Less(b.file.Hash)
- }
- return a.file.URI < b.file.URI
-}
-
-type isActivePackageCacheMap struct {
- impl *persistent.Map
-}
-
-func newIsActivePackageCacheMap() isActivePackageCacheMap {
- return isActivePackageCacheMap{
- impl: persistent.NewMap(func(a, b interface{}) bool {
- return a.(PackageID) < b.(PackageID)
- }),
- }
-}
-
-func (m isActivePackageCacheMap) Clone() isActivePackageCacheMap {
- return isActivePackageCacheMap{
- impl: m.impl.Clone(),
- }
-}
-
-func (m isActivePackageCacheMap) Destroy() {
- m.impl.Destroy()
-}
-
-func (m isActivePackageCacheMap) Get(key PackageID) (bool, bool) {
- value, ok := m.impl.Get(key)
- if !ok {
- return false, false
- }
- return value.(bool), true
-}
-
-func (m isActivePackageCacheMap) Set(key PackageID, value bool) {
- m.impl.Set(key, value, nil)
-}
-
-type parseKeysByURIMap struct {
- impl *persistent.Map
-}
-
-func newParseKeysByURIMap() parseKeysByURIMap {
- return parseKeysByURIMap{
- impl: persistent.NewMap(uriLessInterface),
- }
-}
-
-func (m parseKeysByURIMap) Clone() parseKeysByURIMap {
- return parseKeysByURIMap{
- impl: m.impl.Clone(),
- }
-}
-
-func (m parseKeysByURIMap) Destroy() {
- m.impl.Destroy()
-}
-
-func (m parseKeysByURIMap) Get(key span.URI) ([]parseKey, bool) {
- value, ok := m.impl.Get(key)
- if !ok {
- return nil, false
- }
- return value.([]parseKey), true
-}
-
-func (m parseKeysByURIMap) Range(do func(key span.URI, value []parseKey)) {
- m.impl.Range(func(key, value interface{}) {
- do(key.(span.URI), value.([]parseKey))
- })
-}
-
-func (m parseKeysByURIMap) Set(key span.URI, value []parseKey) {
- m.impl.Set(key, value, nil)
-}
-
-func (m parseKeysByURIMap) Delete(key span.URI) {
- m.impl.Delete(key)
-}
-
-func packageKeyLessInterface(x, y interface{}) bool {
- return packageKeyLess(x.(packageKey), y.(packageKey))
-}
-
-func packageKeyLess(x, y packageKey) bool {
- if x.mode != y.mode {
- return x.mode < y.mode
- }
- return x.id < y.id
+func packageIDLessInterface(x, y interface{}) bool {
+ return x.(PackageID) < y.(PackageID)
}
type knownDirsSet struct {
diff --git a/gopls/internal/lsp/cache/parse.go b/gopls/internal/lsp/cache/parse.go
index f9bf7d7..1654708 100644
--- a/gopls/internal/lsp/cache/parse.go
+++ b/gopls/internal/lsp/cache/parse.go
@@ -12,11 +12,8 @@
"go/parser"
"go/scanner"
"go/token"
- "go/types"
"path/filepath"
"reflect"
- "strconv"
- "strings"
"golang.org/x/tools/gopls/internal/lsp/protocol"
"golang.org/x/tools/gopls/internal/lsp/safetoken"
@@ -25,91 +22,16 @@
"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"
)
// ParseGo parses the file whose contents are provided by fh, using a cache.
-// The resulting tree may have be fixed up.
-//
-// Token position information will be added to the snapshot's FileSet.
-//
-// The parser mode must not be ParseExported: that mode is used during
-// type checking to destructively trim the tree to reduce work,
-// which is not safe for values from a shared cache.
-// TODO(adonovan): opt: shouldn't parseGoImpl do the trimming?
-// Then we can cache the result since it would never change.
-//
-// TODO(adonovan): in the absence of any way to add existing an
-// token.File to a new FileSet (see go.dev/issue/53200), caching ASTs
-// implies a global FileSet.
+// The resulting tree may have beeen fixed up.
func (s *snapshot) ParseGo(ctx context.Context, fh source.FileHandle, mode source.ParseMode) (*source.ParsedGoFile, error) {
- if mode == source.ParseExported {
- panic("only type checking should use Exported")
- }
-
- key := parseKey{
- file: fh.FileIdentity(),
- mode: mode,
- }
-
- s.mu.Lock()
- entry, hit := s.parsedGoFiles.Get(key)
- s.mu.Unlock()
-
- // cache miss?
- if !hit {
- promise, release := s.store.Promise(key, func(ctx context.Context, arg interface{}) interface{} {
- parsed, err := parseGoImpl(ctx, arg.(*snapshot).view.fset, fh, mode)
- return parseGoResult{parsed, err}
- })
-
- s.mu.Lock()
- // Check cache again in case another thread got there first.
- if prev, ok := s.parsedGoFiles.Get(key); ok {
- entry = prev
- release()
- } else {
- entry = promise
- s.parsedGoFiles.Set(key, entry, func(_, _ interface{}) { release() })
-
- // In order to correctly invalidate the key above, we must keep track of
- // the parse key just created.
- //
- // TODO(rfindley): use a two-level map URI->parseKey->promise.
- keys, _ := s.parseKeysByURI.Get(fh.URI())
-
- // Only record the new key if it doesn't exist. This is overly cautious:
- // we should only be setting the key if it doesn't exist. However, this
- // logic will be replaced soon, and erring on the side of caution seemed
- // wise.
- foundKey := false
- for _, existing := range keys {
- if existing == key {
- foundKey = true
- break
- }
- }
- if !foundKey {
- keys = append(keys, key)
- s.parseKeysByURI.Set(fh.URI(), keys)
- }
- }
- s.mu.Unlock()
- }
-
- // Await result.
- v, err := s.awaitPromise(ctx, entry.(*memoize.Promise))
+ pgfs, _, err := s.parseCache.parseFiles(ctx, mode, fh)
if err != nil {
return nil, err
}
- res := v.(parseGoResult)
- return res.parsed, res.err
-}
-
-// parseGoResult holds the result of a call to parseGoImpl.
-type parseGoResult struct {
- parsed *source.ParsedGoFile
- err error
+ return pgfs[0], nil
}
// parseGoImpl parses the Go source file whose content is provided by fh.
@@ -196,340 +118,6 @@
}
}
-// An unexportedFilter removes as much unexported AST from a set of Files as possible.
-type unexportedFilter struct {
- uses map[string]bool
-}
-
-// Filter records uses of unexported identifiers and filters out all other
-// unexported declarations.
-func (f *unexportedFilter) Filter(files []*ast.File) {
- // Iterate to fixed point -- unexported types can include other unexported types.
- oldLen := len(f.uses)
- for {
- for _, file := range files {
- f.recordUses(file)
- }
- if len(f.uses) == oldLen {
- break
- }
- oldLen = len(f.uses)
- }
-
- for _, file := range files {
- var newDecls []ast.Decl
- for _, decl := range file.Decls {
- if f.filterDecl(decl) {
- newDecls = append(newDecls, decl)
- }
- }
- file.Decls = newDecls
- file.Scope = nil
- file.Unresolved = nil
- file.Comments = nil
- trimAST(file)
- }
-}
-
-func (f *unexportedFilter) keep(ident *ast.Ident) bool {
- return ast.IsExported(ident.Name) || f.uses[ident.Name]
-}
-
-func (f *unexportedFilter) filterDecl(decl ast.Decl) bool {
- switch decl := decl.(type) {
- case *ast.FuncDecl:
- if ident := source.RecvIdent(decl.Recv); ident != nil && !f.keep(ident) {
- return false
- }
- return f.keep(decl.Name)
- case *ast.GenDecl:
- if decl.Tok == token.CONST {
- // Constants can involve iota, and iota is hard to deal with.
- return true
- }
- var newSpecs []ast.Spec
- for _, spec := range decl.Specs {
- if f.filterSpec(spec) {
- newSpecs = append(newSpecs, spec)
- }
- }
- decl.Specs = newSpecs
- return len(newSpecs) != 0
- case *ast.BadDecl:
- return false
- }
- panic(fmt.Sprintf("unknown ast.Decl %T", decl))
-}
-
-func (f *unexportedFilter) filterSpec(spec ast.Spec) bool {
- switch spec := spec.(type) {
- case *ast.ImportSpec:
- return true
- case *ast.ValueSpec:
- var newNames []*ast.Ident
- for _, name := range spec.Names {
- if f.keep(name) {
- newNames = append(newNames, name)
- }
- }
- spec.Names = newNames
- return len(spec.Names) != 0
- case *ast.TypeSpec:
- if !f.keep(spec.Name) {
- return false
- }
- switch typ := spec.Type.(type) {
- case *ast.StructType:
- // In practice this no longer filters anything;
- // see comment at StructType case in recordUses.
- f.filterFieldList(typ.Fields)
- case *ast.InterfaceType:
- f.filterFieldList(typ.Methods)
- }
- return true
- }
- panic(fmt.Sprintf("unknown ast.Spec %T", spec))
-}
-
-func (f *unexportedFilter) filterFieldList(fields *ast.FieldList) {
- var newFields []*ast.Field
- for _, field := range fields.List {
- if len(field.Names) == 0 {
- // Keep embedded fields: they can export methods and fields.
- newFields = append(newFields, field)
- }
- for _, name := range field.Names {
- if f.keep(name) {
- newFields = append(newFields, field)
- break
- }
- }
- }
- fields.List = newFields
-}
-
-func (f *unexportedFilter) recordUses(file *ast.File) {
- for _, decl := range file.Decls {
- switch decl := decl.(type) {
- case *ast.FuncDecl:
- // Ignore methods on dropped types.
- if ident := source.RecvIdent(decl.Recv); ident != nil && !f.keep(ident) {
- break
- }
- // Ignore functions with dropped names.
- if !f.keep(decl.Name) {
- break
- }
- f.recordFuncType(decl.Type)
- case *ast.GenDecl:
- for _, spec := range decl.Specs {
- switch spec := spec.(type) {
- case *ast.ValueSpec:
- for i, name := range spec.Names {
- // Don't mess with constants -- iota is hard.
- if f.keep(name) || decl.Tok == token.CONST {
- f.recordIdents(spec.Type)
- if len(spec.Values) > i {
- f.recordIdents(spec.Values[i])
- }
- }
- }
- case *ast.TypeSpec:
- switch typ := spec.Type.(type) {
- case *ast.StructType:
- // We used to trim unexported fields but this
- // had observable consequences. For example,
- // the 'fieldalignment' analyzer would compute
- // incorrect diagnostics from the size and
- // offsets, and the UI hover information for
- // types was inaccurate. So now we keep them.
- if typ.Fields != nil {
- for _, field := range typ.Fields.List {
- f.recordIdents(field.Type)
- }
- }
- case *ast.InterfaceType:
- f.recordInterfaceMethodUses(typ.Methods)
- }
- }
- }
- }
- }
-}
-
-// recordIdents records unexported identifiers in an Expr in uses.
-// These may be types, e.g. in map[key]value, function names, e.g. in foo(),
-// or simple variable references. References that will be discarded, such
-// as those in function literal bodies, are ignored.
-func (f *unexportedFilter) recordIdents(x ast.Expr) {
- ast.Inspect(x, func(n ast.Node) bool {
- if n == nil {
- return false
- }
- if complit, ok := n.(*ast.CompositeLit); ok {
- // We clear out composite literal contents; just record their type.
- f.recordIdents(complit.Type)
- return false
- }
- if flit, ok := n.(*ast.FuncLit); ok {
- f.recordFuncType(flit.Type)
- return false
- }
- if ident, ok := n.(*ast.Ident); ok && !ast.IsExported(ident.Name) {
- f.uses[ident.Name] = true
- }
- return true
- })
-}
-
-// recordFuncType records the types mentioned by a function type.
-func (f *unexportedFilter) recordFuncType(fn *ast.FuncType) {
- // Parameter and result types of retained functions need to be retained.
- if fn.Params != nil {
- for _, field := range fn.Params.List {
- f.recordIdents(field.Type)
- }
- }
- if fn.Results != nil {
- for _, field := range fn.Results.List {
- f.recordIdents(field.Type)
- }
- }
-}
-
-// recordInterfaceMethodUses records unexported identifiers used in interface methods.
-func (f *unexportedFilter) recordInterfaceMethodUses(methods *ast.FieldList) {
- if methods != nil {
- for _, method := range methods.List {
- if len(method.Names) == 0 {
- // I, pkg.I, I[T] -- embedded interface:
- // may contribute exported names.
- f.recordIdents(method.Type)
- } else if ft, ok := method.Type.(*ast.FuncType); ok {
- // f(T) -- ordinary interface method:
- // needs all its types retained.
- f.recordFuncType(ft)
- }
- }
- }
-}
-
-// ProcessErrors records additional uses from errors, returning the new uses
-// and any unexpected errors.
-func (f *unexportedFilter) ProcessErrors(errors []types.Error) (map[string]bool, []types.Error) {
- var unexpected []types.Error
- missing := map[string]bool{}
- for _, err := range errors {
- if strings.Contains(err.Msg, "missing return") {
- continue
- }
- const undeclared = "undeclared name: "
- if strings.HasPrefix(err.Msg, undeclared) {
- missing[strings.TrimPrefix(err.Msg, undeclared)] = true
- f.uses[strings.TrimPrefix(err.Msg, undeclared)] = true
- continue
- }
- unexpected = append(unexpected, err)
- }
- return missing, unexpected
-}
-
-// trimAST clears any part of the AST not relevant to type checking
-// the package-level declarations.
-func trimAST(file *ast.File) {
- // Eliminate bodies of top-level functions, methods, inits.
- for _, decl := range file.Decls {
- if fn, ok := decl.(*ast.FuncDecl); ok {
- fn.Body = nil
- }
- }
-
- // Simplify remaining declarations.
- ast.Inspect(file, func(n ast.Node) bool {
- switch n := n.(type) {
- case *ast.FuncLit:
- // Eliminate bodies of literal functions.
- // func() { ... } => func() {}
- n.Body.List = nil
- case *ast.CompositeLit:
- // types.Info.Types for long slice/array literals are particularly
- // expensive. Try to clear them out: T{e, ..., e} => T{}
- at, ok := n.Type.(*ast.ArrayType)
- if !ok {
- // Map or struct literal: no harm removing all its fields.
- n.Elts = nil
- break
- }
-
- // Removing the elements from an ellipsis array changes its type.
- // Try to set the length explicitly so we can continue.
- // [...]T{e, ..., e} => [3]T[]{}
- if _, ok := at.Len.(*ast.Ellipsis); ok {
- length, ok := arrayLength(n)
- if !ok {
- break
- }
- at.Len = &ast.BasicLit{
- Kind: token.INT,
- Value: fmt.Sprint(length),
- ValuePos: at.Len.Pos(),
- }
- }
- n.Elts = nil
- }
- return true
- })
-}
-
-// arrayLength returns the length of some simple forms of ellipsis array literal.
-// Notably, it handles the tables in golang.org/x/text.
-func arrayLength(array *ast.CompositeLit) (int, bool) {
- litVal := func(expr ast.Expr) (int, bool) {
- lit, ok := expr.(*ast.BasicLit)
- if !ok {
- return 0, false
- }
- val, err := strconv.ParseInt(lit.Value, 10, 64)
- if err != nil {
- return 0, false
- }
- return int(val), true
- }
- largestKey := -1
- for _, elt := range array.Elts {
- kve, ok := elt.(*ast.KeyValueExpr)
- if !ok {
- continue
- }
- switch key := kve.Key.(type) {
- case *ast.BasicLit:
- if val, ok := litVal(key); ok && largestKey < val {
- largestKey = val
- }
- case *ast.BinaryExpr:
- // golang.org/x/text uses subtraction (and only subtraction) in its indices.
- if key.Op != token.SUB {
- break
- }
- x, ok := litVal(key.X)
- if !ok {
- break
- }
- y, ok := litVal(key.Y)
- if !ok {
- break
- }
- if val := x - y; largestKey < val {
- largestKey = val
- }
- }
- }
- if largestKey != -1 {
- return largestKey + 1, true
- }
- return len(array.Elts), true
-}
-
// fixAST inspects the AST and potentially modifies any *ast.BadStmts so that it can be
// type-checked more effectively.
//
diff --git a/gopls/internal/lsp/cache/parse_test.go b/gopls/internal/lsp/cache/parse_test.go
deleted file mode 100644
index e8db645..0000000
--- a/gopls/internal/lsp/cache/parse_test.go
+++ /dev/null
@@ -1,217 +0,0 @@
-// Copyright 2019 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 (
- "bytes"
- "go/ast"
- "go/format"
- "go/parser"
- "go/token"
- "go/types"
- "reflect"
- "sort"
- "testing"
-
- "golang.org/x/tools/go/packages"
-)
-
-func TestArrayLength(t *testing.T) {
- tests := []struct {
- expr string
- length int
- }{
- {`[...]int{0,1,2,3,4,5,6,7,8,9}`, 10},
- {`[...]int{9:0}`, 10},
- {`[...]int{19-10:0}`, 10},
- {`[...]int{19-10:0, 17-10:0, 18-10:0}`, 10},
- }
-
- for _, tt := range tests {
- expr, err := parser.ParseExpr(tt.expr)
- if err != nil {
- t.Fatal(err)
- }
- l, ok := arrayLength(expr.(*ast.CompositeLit))
- if !ok {
- t.Errorf("arrayLength did not recognize expression %#v", expr)
- }
- if l != tt.length {
- t.Errorf("arrayLength(%#v) = %v, want %v", expr, l, tt.length)
- }
- }
-}
-
-func TestTrim(t *testing.T) {
- tests := []struct {
- name string
- file string
- kept []string
- }{
- {
- name: "delete_unused",
- file: `
-type x struct{}
-func y()
-var z int
-`,
- kept: []string{},
- },
- {
- // From the common type in testing.
- name: "unexported_embedded",
- file: `
-type x struct {}
-type Exported struct { x }
-`,
- kept: []string{"Exported", "x"},
- },
- {
- // From the d type in unicode.
- name: "exported_field_unexported_type",
- file: `
-type x struct {}
-type Exported struct {
- X x
-}
-`,
- kept: []string{"Exported", "x"},
- },
- {
- // From errNotExist in io/fs.
- name: "exported_var_function_call",
- file: `
-func x() int { return 0 }
-var Exported = x()
-`,
- kept: []string{"Exported", "x"},
- },
- {
- // From DefaultServeMux in net/http.
- name: "exported_pointer_to_unexported_var",
- file: `
-var Exported = &x
-var x int
-`,
- kept: []string{"Exported", "x"},
- },
- {
- // From DefaultWriter in goldmark/renderer/html.
- name: "exported_pointer_to_composite_lit",
- file: `
-var Exported = &x{}
-type x struct{}
-`,
- kept: []string{"Exported", "x"},
- },
- {
- // From SelectDir in reflect.
- name: "leave_constants",
- file: `
-type Enum int
-const (
- _ Enum = iota
- EnumOne
-)
-`,
- kept: []string{"Enum", "EnumOne"},
- },
- {
- name: "constant_conversion",
- file: `
-type x int
-const (
- foo x = 0
-)
-`,
- kept: []string{"x", "foo"},
- },
- {
- name: "unexported_return",
- file: `
-type x int
-func Exported() x {}
-type y int
-type Interface interface {
- Exported() y
-}
-`,
- kept: []string{"Exported", "Interface", "x", "y"},
- },
- {
- name: "drop_composite_literals",
- file: `
-type x int
-type Exported struct {
- foo x
-}
-var Var = Exported{foo:1}
-`,
- kept: []string{"Exported", "Var", "x"},
- },
- {
- name: "drop_function_literals",
- file: `
-type x int
-var Exported = func() { return x(0) }
-`,
- kept: []string{"Exported"},
- },
- {
- name: "missing_receiver_panic",
- file: `
- func() foo() {}
-`,
- kept: []string{},
- },
- }
-
- for _, tt := range tests {
- t.Run(tt.name, func(t *testing.T) {
- fset := token.NewFileSet()
- file, err := parser.ParseFile(fset, "main.go", "package main\n\n"+tt.file, parser.AllErrors)
- if err != nil {
- t.Fatal(err)
- }
- filter := &unexportedFilter{uses: map[string]bool{}}
- filter.Filter([]*ast.File{file})
- pkg := types.NewPackage("main", "main")
- checker := types.NewChecker(&types.Config{
- DisableUnusedImportCheck: true,
- }, fset, pkg, nil)
- if err := checker.Files([]*ast.File{file}); err != nil {
- t.Error(err)
- }
- names := pkg.Scope().Names()
- sort.Strings(names)
- sort.Strings(tt.kept)
- if !reflect.DeepEqual(names, tt.kept) {
- t.Errorf("package contains names %v, wanted %v", names, tt.kept)
- }
- })
- }
-}
-
-func TestPkg(t *testing.T) {
- t.Skip("for manual debugging")
- fset := token.NewFileSet()
- pkgs, err := packages.Load(&packages.Config{
- Mode: packages.NeedSyntax | packages.NeedFiles,
- Fset: fset,
- }, "io")
- if err != nil {
- t.Fatal(err)
- }
- if len(pkgs[0].Errors) != 0 {
- t.Fatal(pkgs[0].Errors)
- }
- filter := &unexportedFilter{uses: map[string]bool{}}
- filter.Filter(pkgs[0].Syntax)
- for _, file := range pkgs[0].Syntax {
- buf := &bytes.Buffer{}
- format.Node(buf, fset, file)
- t.Log(buf.String())
- }
-}
diff --git a/gopls/internal/lsp/cache/pkg.go b/gopls/internal/lsp/cache/pkg.go
index 76060ef..11346fc 100644
--- a/gopls/internal/lsp/cache/pkg.go
+++ b/gopls/internal/lsp/cache/pkg.go
@@ -12,11 +12,8 @@
"go/token"
"go/types"
- "golang.org/x/tools/go/types/objectpath"
- "golang.org/x/tools/gopls/internal/lsp/protocol"
"golang.org/x/tools/gopls/internal/lsp/source"
"golang.org/x/tools/gopls/internal/lsp/source/methodsets"
- "golang.org/x/tools/gopls/internal/lsp/source/xrefs"
"golang.org/x/tools/gopls/internal/span"
"golang.org/x/tools/internal/memoize"
)
@@ -55,53 +52,17 @@
typeErrors []types.Error
types *types.Package
typesInfo *types.Info
- importMap *importMap // required for DependencyTypes (until we have shallow export data)
- hasFixedFiles bool // if true, AST was sufficiently mangled that we should hide type errors
- xrefs []byte // serializable index of outbound cross-references
- analyses memoize.Store // maps analyzer.Name to Promise[actionResult]
- methodsets *methodsets.Index // index of method sets of package-level types
+ importMap map[string]*types.Package // keys are PackagePaths
+ hasFixedFiles bool // if true, AST was sufficiently mangled that we should hide type errors
+ analyses memoize.Store // maps analyzer.Name to Promise[actionResult]
+ xrefs []byte
+ methodsets *methodsets.Index
}
func (p *Package) String() string { return string(p.m.ID) }
func (p *Package) Metadata() *source.Metadata { return p.m }
-// An importMap is an mapping from source.PackagePath to types.Package
-// of the dependencies of a syntaxPackage. Once constructed (by calls
-// to union) it is never subsequently modified.
-type importMap struct {
- // Concretely, it is a node that contains one element of the
- // mapping and whose deps are edges in DAG that comprises the
- // rest of the mapping. This design optimizes union over get.
- //
- // TODO(adonovan): simplify when we use shallow export data.
- // At that point it becomes a simple lookup in the importers
- // map, which should be retained in syntaxPackage.
- // (Otherwise this implementation could expose types.Packages
- // that represent an old state that has since changed, but
- // not in a way that is consequential to a downstream package.)
-
- types *types.Package // map entry for types.Path => types
- deps []*importMap // all other entries
-}
-
-func (m *importMap) union(dep *importMap) { m.deps = append(m.deps, dep) }
-
-func (m *importMap) get(path source.PackagePath, seen map[*importMap]bool) *types.Package {
- if !seen[m] {
- seen[m] = true
- if source.PackagePath(m.types.Path()) == path {
- return m.types
- }
- for _, dep := range m.deps {
- if pkg := dep.get(path, seen); pkg != nil {
- return pkg
- }
- }
- }
- return nil
-}
-
// A loadScope defines a package loading scope for use with go/packages.
//
// TODO(rfindley): move this to load.go.
@@ -173,7 +134,10 @@
// dependencies of p, or if no symbols from that package were
// referenced during the type-checking of p.
func (p *Package) DependencyTypes(path source.PackagePath) *types.Package {
- return p.pkg.importMap.get(path, make(map[*importMap]bool))
+ if path == p.m.PkgPath {
+ return p.pkg.types
+ }
+ return p.pkg.importMap[string(path)]
}
func (p *Package) HasParseErrors() bool {
@@ -199,20 +163,3 @@
return diags, nil
}
-
-// ReferencesTo returns the location of each reference within package p
-// to one of the target objects denoted by the pair (package path, object path).
-func (p *Package) ReferencesTo(targets map[PackagePath]map[objectpath.Path]struct{}) []protocol.Location {
- // TODO(adonovan): In future, p.xrefs will be retrieved from a
- // section of the cache file produced by type checking.
- // (Other sections will include the package's export data,
- // "implements" relations, exported symbols, etc.)
- // For now we just hang it off the pkg.
- return xrefs.Lookup(p.m, p.pkg.xrefs, targets)
-}
-
-func (p *Package) MethodSetsIndex() *methodsets.Index {
- // TODO(adonovan): In future, p.methodsets will be retrieved from a
- // section of the cache file produced by type checking.
- return p.pkg.methodsets
-}
diff --git a/gopls/internal/lsp/cache/session.go b/gopls/internal/lsp/cache/session.go
index 38e28ac..f8d5835 100644
--- a/gopls/internal/lsp/cache/session.go
+++ b/gopls/internal/lsp/cache/session.go
@@ -121,7 +121,6 @@
v := &View{
id: strconv.FormatInt(index, 10),
- fset: s.cache.fset,
gocmdRunner: s.gocmdRunner,
initialWorkspaceLoad: make(chan struct{}),
initializationSema: make(chan struct{}, 1),
@@ -158,13 +157,11 @@
backgroundCtx: backgroundCtx,
cancel: cancel,
store: s.cache.store,
- packages: persistent.NewMap(packageKeyLessInterface),
+ packages: persistent.NewMap(packageIDLessInterface),
meta: new(metadataGraph),
files: newFilesMap(),
parseCache: new(parseCache),
- isActivePackageCache: newIsActivePackageCacheMap(),
- parsedGoFiles: persistent.NewMap(parseKeyLessInterface),
- parseKeysByURI: newParseKeysByURIMap(),
+ activePackages: persistent.NewMap(packageIDLessInterface),
symbolizeHandles: persistent.NewMap(uriLessInterface),
analyses: persistent.NewMap(analysisKeyLessInterface),
workspacePackages: make(map[PackageID]PackagePath),
diff --git a/gopls/internal/lsp/cache/snapshot.go b/gopls/internal/lsp/cache/snapshot.go
index 7ad3255..cd8e498 100644
--- a/gopls/internal/lsp/cache/snapshot.go
+++ b/gopls/internal/lsp/cache/snapshot.go
@@ -28,7 +28,11 @@
"golang.org/x/sync/errgroup"
"golang.org/x/tools/go/packages"
+ "golang.org/x/tools/go/types/objectpath"
+ "golang.org/x/tools/gopls/internal/lsp/protocol"
"golang.org/x/tools/gopls/internal/lsp/source"
+ "golang.org/x/tools/gopls/internal/lsp/source/methodsets"
+ "golang.org/x/tools/gopls/internal/lsp/source/xrefs"
"golang.org/x/tools/gopls/internal/span"
"golang.org/x/tools/internal/bug"
"golang.org/x/tools/internal/event"
@@ -91,14 +95,6 @@
// 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]
-
- // parseKeysByURI records the set of keys of parsedGoFiles that
- // need to be invalidated for each URI.
- // TODO(adonovan): opt: parseKey = ParseMode + URI, so this could
- // be just a set of ParseModes, or we could loop over AllParseModes.
- parseKeysByURI parseKeysByURIMap
// symbolizeHandles maps each file URI to a handle for the future
// result of computing the symbols declared in that file.
@@ -111,11 +107,13 @@
// - packages.Get(id).meta == meta.metadata[id] for all ids
// - if a package is in packages, then all of its dependencies should also
// be in packages, unless there is a missing import
- packages *persistent.Map // from packageKey to *packageHandle
+ packages *persistent.Map // from packageID to *packageHandle
- // isActivePackageCache maps package ID to the cached value if it is active or not.
- // It may be invalidated when metadata changes or a new file is opened or closed.
- isActivePackageCache isActivePackageCacheMap
+ // activePackages maps a package ID to a memoized active package, or nil if
+ // the package is known not to be open.
+ //
+ // IDs not contained in the map are not known to be open or not open.
+ activePackages *persistent.Map // from packageID to *Package
// analyses maps an analysisKey (which identifies a package
// and a set of analyzers) to the handle for the future result
@@ -136,6 +134,9 @@
// unloadableFiles keeps track of files that we've failed to load.
unloadableFiles map[span.URI]struct{}
+ // TODO(rfindley): rename the handles below to "promises". A promise is
+ // different from a handle (we mutate the package handle.)
+
// parseModHandles keeps track of any parseModHandles for the snapshot.
// The handles need not refer to only the view's go.mod file.
parseModHandles *persistent.Map // from span.URI to *memoize.Promise[parseModResult]
@@ -233,11 +234,9 @@
}
s.packages.Destroy()
- s.isActivePackageCache.Destroy()
+ s.activePackages.Destroy()
s.analyses.Destroy()
s.files.Destroy()
- s.parsedGoFiles.Destroy()
- s.parseKeysByURI.Destroy()
s.knownSubdirs.Destroy()
s.symbolizeHandles.Destroy()
s.parseModHandles.Destroy()
@@ -624,58 +623,72 @@
return overlays
}
-// TypeCheck type-checks the specified packages in the given mode.
-//
-// The resulting packages slice always contains len(ids) entries, though some
-// of them may be nil if (and only if) the resulting error is non-nil.
-//
-// An error is returned if any of the packages fail to type-check. This is
-// different from having type-checking errors: a failure to type-check
-// indicates context cancellation or otherwise significant failure to perform
-// the type-checking operation.
-func (s *snapshot) TypeCheck(ctx context.Context, mode source.TypecheckMode, ids ...PackageID) ([]source.Package, error) {
- // Build all the handles...
- phs := make([]*packageHandle, len(ids))
- pkgs := make([]source.Package, len(ids))
- var firstErr error
- for i, id := range ids {
- parseMode := source.ParseFull
- if mode == source.TypecheckWorkspace {
- parseMode = s.workspaceParseMode(id)
- }
+// Package data kinds, identifying various package data that may be stored in
+// the file cache.
+const (
+ xrefsKind = "xrefs"
+ methodSetsKind = "methodsets"
+ exportDataKind = "export"
+ diagnosticsKind = "diagnostics"
+)
- ph, err := s.buildPackageHandle(ctx, id, parseMode)
- if err != nil {
- if firstErr == nil {
- firstErr = err
+func (s *snapshot) PackageDiagnostics(ctx context.Context, ids ...PackageID) (map[span.URI][]*source.Diagnostic, error) {
+ // TODO(rfindley): opt: avoid unnecessary encode->decode after type-checking.
+ data, err := s.getPackageData(ctx, diagnosticsKind, ids, func(p *syntaxPackage) []byte {
+ return encodeDiagnostics(p.diagnostics)
+ })
+ perFile := make(map[span.URI][]*source.Diagnostic)
+ for _, data := range data {
+ if data != nil {
+ for _, diag := range data.m.Diagnostics {
+ perFile[diag.URI] = append(perFile[diag.URI], diag)
}
- if ctx.Err() != nil {
- return pkgs, firstErr
+ diags := decodeDiagnostics(data.data)
+ for _, diag := range diags {
+ perFile[diag.URI] = append(perFile[diag.URI], diag)
}
- continue
}
- phs[i] = ph
}
+ return perFile, err
+}
- // ...then await them all.
- for i, ph := range phs {
- if ph == nil {
- continue
+func (s *snapshot) References(ctx context.Context, ids ...PackageID) ([]source.XrefIndex, error) {
+ data, err := s.getPackageData(ctx, xrefsKind, ids, func(p *syntaxPackage) []byte { return p.xrefs })
+ indexes := make([]source.XrefIndex, len(ids))
+ for i, data := range data {
+ if data != nil {
+ indexes[i] = XrefIndex{m: data.m, data: data.data}
}
- p, err := ph.await(ctx, s)
- if err != nil {
- if firstErr == nil {
- firstErr = err
- }
- if ctx.Err() != nil {
- return pkgs, firstErr
- }
- continue
- }
- pkgs[i] = &Package{ph.m, p}
}
+ return indexes, err
+}
- return pkgs, firstErr
+// An XrefIndex is a helper for looking up a package in a given package.
+type XrefIndex struct {
+ m *source.Metadata
+ data []byte
+}
+
+func (index XrefIndex) Lookup(targets map[PackagePath]map[objectpath.Path]struct{}) []protocol.Location {
+ return xrefs.Lookup(index.m, index.data, targets)
+}
+
+func (s *snapshot) MethodSets(ctx context.Context, ids ...PackageID) ([]*methodsets.Index, error) {
+ // TODO(rfindley): opt: avoid unnecessary encode->decode after type-checking.
+ data, err := s.getPackageData(ctx, methodSetsKind, ids, func(p *syntaxPackage) []byte {
+ return p.methodsets.Encode()
+ })
+ indexes := make([]*methodsets.Index, len(ids))
+ for i, data := range data {
+ if data != nil {
+ indexes[i] = methodsets.Decode(data.data)
+ } else if ids[i] == "unsafe" {
+ indexes[i] = &methodsets.Index{}
+ } else {
+ panic(fmt.Sprintf("nil data for %s", ids[i]))
+ }
+ }
+ return indexes, err
}
func (s *snapshot) MetadataForFile(ctx context.Context, uri span.URI) ([]*source.Metadata, error) {
@@ -793,35 +806,55 @@
return meta
}
-func (s *snapshot) isActiveLocked(id PackageID) (active bool) {
- if seen, ok := s.isActivePackageCache.Get(id); ok {
- return seen
+// -- Active package tracking --
+//
+// We say a package is "active" if any of its files are open. After
+// type-checking we keep active packages in memory. The activePackages
+// peristent map does bookkeeping for the set of active packages.
+
+// getActivePackage returns a the memoized active package for id, if it exists.
+// If id is not active or has not yet been type-checked, it returns nil.
+func (s *snapshot) getActivePackage(id PackageID) *Package {
+ s.mu.Lock()
+ defer s.mu.Unlock()
+
+ if value, ok := s.activePackages.Get(id); ok {
+ return value.(*Package) // possibly nil, if we have already checked this id.
}
- defer func() {
- s.isActivePackageCache.Set(id, active)
- }()
- m, ok := s.meta.metadata[id]
- if !ok {
- return false
- }
- for _, cgf := range m.CompiledGoFiles {
- if s.isOpenLocked(cgf) {
- return true
- }
- }
- // TODO(rfindley): it looks incorrect that we don't also check GoFiles here.
- // If a CGo file is open, we want to consider the package active.
- for _, dep := range m.DepsByPkgPath {
- if s.isActiveLocked(dep) {
- return true
- }
- }
- return false
+ return nil
}
-func (s *snapshot) resetIsActivePackageLocked() {
- s.isActivePackageCache.Destroy()
- s.isActivePackageCache = newIsActivePackageCacheMap()
+// memoizeActivePackage checks if pkg is active, and if so either records it in
+// the active packages map or returns the existing memoized active package for id.
+//
+// The resulting package is non-nil if and only if the specified package is open.
+func (s *snapshot) memoizeActivePackage(id PackageID, pkg *Package) (active *Package) {
+ s.mu.Lock()
+ defer s.mu.Unlock()
+
+ if value, ok := s.activePackages.Get(id); ok {
+ return value.(*Package) // possibly nil, if we have already checked this id.
+ }
+
+ defer func() {
+ s.activePackages.Set(id, active, nil) // store the result either way: remember that pkg is not open
+ }()
+ for _, cgf := range pkg.Metadata().GoFiles {
+ if s.isOpenLocked(cgf) {
+ return pkg
+ }
+ }
+ for _, cgf := range pkg.Metadata().CompiledGoFiles {
+ if s.isOpenLocked(cgf) {
+ return pkg
+ }
+ }
+ return nil
+}
+
+func (s *snapshot) resetActivePackagesLocked() {
+ s.activePackages.Destroy()
+ s.activePackages = persistent.NewMap(packageIDLessInterface)
}
const fileExtensions = "go,mod,sum,work"
@@ -1008,21 +1041,7 @@
if err := s.awaitLoaded(ctx); err != nil {
return nil, err
}
-
- if s.view.Options().MemoryMode == source.ModeNormal {
- return s.workspaceMetadata(), nil
- }
-
- // ModeDegradeClosed
- s.mu.Lock()
- defer s.mu.Unlock()
- var active []*source.Metadata
- for id := range s.workspacePackages {
- if s.isActiveLocked(id) {
- active = append(active, s.Metadata(id))
- }
- }
- return active, nil
+ return s.workspaceMetadata(), nil
}
// Symbols extracts and returns symbol information for every file contained in
@@ -1095,22 +1114,11 @@
}
func (s *snapshot) CachedPackages(ctx context.Context) []source.Package {
- // Don't reload workspace package metadata.
- // This function is meant to only return currently cached information.
- s.AwaitInitialized(ctx)
-
- s.mu.Lock()
- defer s.mu.Unlock()
-
- var pkgs []source.Package
- s.packages.Range(func(_, v interface{}) {
- ph := v.(*packageHandle)
- pkg, err := ph.cached()
- if err == nil {
- pkgs = append(pkgs, &Package{ph.m, pkg})
- }
- })
- return pkgs
+ // Cached packages do not make sense with incremental gopls.
+ //
+ // TODO(golang/go#58663): re-implement unimported completions to not depend
+ // on cached import paths.
+ return nil
}
// TODO(rfindley): clarify that this is only active modules. Or update to just
@@ -1666,12 +1674,10 @@
initialized: s.initialized,
initializedErr: s.initializedErr,
packages: s.packages.Clone(),
- isActivePackageCache: s.isActivePackageCache.Clone(),
+ activePackages: s.activePackages.Clone(),
analyses: s.analyses.Clone(),
files: s.files.Clone(),
parseCache: s.parseCache,
- parsedGoFiles: s.parsedGoFiles.Clone(),
- parseKeysByURI: s.parseKeysByURI.Clone(),
symbolizeHandles: s.symbolizeHandles.Clone(),
workspacePackages: make(map[PackageID]PackagePath, len(s.workspacePackages)),
unloadableFiles: make(map[span.URI]struct{}, len(s.unloadableFiles)),
@@ -1707,37 +1713,6 @@
result.unloadableFiles[k] = v
}
- // TODO(adonovan): merge loops over "changes".
- for uri, change := range changes {
- // Optimization: if the content did not change, we don't need to evict the
- // parsed file. This is not the case for e.g. the files map, which may
- // switch from on-disk state to overlay. Parsed files depend only on
- // content and parse mode (which is captured in the parse key).
- //
- // NOTE: This also makes it less likely that we re-parse a file due to a
- // cache-miss but get a cache-hit for the corresponding package. In the
- // past, there was code that relied on ParseGo returning the type-checked
- // syntax tree. That code was wrong, but avoiding invalidation here limits
- // the blast radius of these types of bugs.
- if !change.isUnchanged {
- keys, ok := result.parseKeysByURI.Get(uri)
- if ok {
- for _, key := range keys {
- result.parsedGoFiles.Delete(key)
- }
- result.parseKeysByURI.Delete(uri)
- }
- }
-
- // Invalidate go.mod-related handles.
- result.modTidyHandles.Delete(uri)
- result.modWhyHandles.Delete(uri)
- result.modVulnHandles.Delete(uri)
-
- // Invalidate handles for cached symbols.
- result.symbolizeHandles.Delete(uri)
- }
-
// Add all of the known subdirectories, but don't update them for the
// changed files. We need to rebuild the workspace module to know the
// true set of known subdirectories, but we don't want to do that in clone.
@@ -1764,6 +1739,14 @@
anyFileAdded := false // adding a file can resolve missing dependencies
for uri, change := range changes {
+ // Invalidate go.mod-related handles.
+ result.modTidyHandles.Delete(uri)
+ result.modWhyHandles.Delete(uri)
+ result.modVulnHandles.Delete(uri)
+
+ // Invalidate handles for cached symbols.
+ result.symbolizeHandles.Delete(uri)
+
// The original FileHandle for this URI is cached on the snapshot.
originalFH, _ := s.files.Get(uri)
var originalOpen, newOpen bool
@@ -1877,10 +1860,8 @@
// Delete invalidated package type information.
for id := range idsToInvalidate {
- for _, mode := range source.AllParseModes {
- key := packageKey{mode, id}
- result.packages.Delete(key)
- }
+ result.packages.Delete(id)
+ result.activePackages.Delete(id)
}
// Delete invalidated analysis actions.
@@ -1962,7 +1943,7 @@
// Update workspace and active packages, if necessary.
if result.meta != s.meta || anyFileOpenedOrClosed {
result.workspacePackages = computeWorkspacePackagesLocked(result, result.meta)
- result.resetIsActivePackageLocked()
+ result.resetActivePackagesLocked()
} else {
result.workspacePackages = s.workspacePackages
}
diff --git a/gopls/internal/lsp/cache/view.go b/gopls/internal/lsp/cache/view.go
index 51660b3..bc106d1 100644
--- a/gopls/internal/lsp/cache/view.go
+++ b/gopls/internal/lsp/cache/view.go
@@ -11,7 +11,6 @@
"encoding/json"
"errors"
"fmt"
- "go/token"
"io/ioutil"
"os"
"path"
@@ -39,7 +38,6 @@
type View struct {
id string
- fset *token.FileSet // shared FileSet
gocmdRunner *gocommand.Runner // limits go command concurrency
// baseCtx is the context handed to NewView. This is the parent of all
diff --git a/gopls/internal/lsp/cache/view_test.go b/gopls/internal/lsp/cache/view_test.go
index 4b45681..9e6d23b 100644
--- a/gopls/internal/lsp/cache/view_test.go
+++ b/gopls/internal/lsp/cache/view_test.go
@@ -96,7 +96,7 @@
rel := fake.RelativeTo(dir)
folderURI := span.URIFromPath(rel.AbsPath(test.folder))
excludeNothing := func(string) bool { return false }
- got, err := findWorkspaceModFile(ctx, folderURI, New(nil, nil), excludeNothing)
+ got, err := findWorkspaceModFile(ctx, folderURI, New(nil), excludeNothing)
if err != nil {
t.Fatal(err)
}
diff --git a/gopls/internal/lsp/cmd/capabilities_test.go b/gopls/internal/lsp/cmd/capabilities_test.go
index 4b38db7..753d9bf 100644
--- a/gopls/internal/lsp/cmd/capabilities_test.go
+++ b/gopls/internal/lsp/cmd/capabilities_test.go
@@ -43,7 +43,7 @@
params.Capabilities.Workspace.Configuration = true
// Send an initialize request to the server.
- c.Server = lsp.NewServer(cache.NewSession(ctx, cache.New(nil, nil), app.options), c.Client)
+ c.Server = lsp.NewServer(cache.NewSession(ctx, cache.New(nil), app.options), c.Client)
result, err := c.Server.Initialize(ctx, params)
if err != nil {
t.Fatal(err)
diff --git a/gopls/internal/lsp/cmd/cmd.go b/gopls/internal/lsp/cmd/cmd.go
index 0943cce..a48eb16 100644
--- a/gopls/internal/lsp/cmd/cmd.go
+++ b/gopls/internal/lsp/cmd/cmd.go
@@ -284,7 +284,7 @@
switch {
case app.Remote == "":
connection := newConnection(app)
- connection.Server = lsp.NewServer(cache.NewSession(ctx, cache.New(nil, nil), app.options), connection.Client)
+ connection.Server = lsp.NewServer(cache.NewSession(ctx, cache.New(nil), app.options), connection.Client)
ctx = protocol.WithClient(ctx, connection.Client)
return connection, connection.initialize(ctx, app.options)
case strings.HasPrefix(app.Remote, "internal@"):
diff --git a/gopls/internal/lsp/cmd/serve.go b/gopls/internal/lsp/cmd/serve.go
index 44d4b1d..df42e79 100644
--- a/gopls/internal/lsp/cmd/serve.go
+++ b/gopls/internal/lsp/cmd/serve.go
@@ -101,7 +101,7 @@
return fmt.Errorf("creating forwarder: %w", err)
}
} else {
- ss = lsprpc.NewStreamServer(cache.New(nil, nil), isDaemon, s.app.options)
+ ss = lsprpc.NewStreamServer(cache.New(nil), isDaemon, s.app.options)
}
var network, addr string
diff --git a/gopls/internal/lsp/code_action.go b/gopls/internal/lsp/code_action.go
index eaf5e43..230cdd8 100644
--- a/gopls/internal/lsp/code_action.go
+++ b/gopls/internal/lsp/code_action.go
@@ -158,7 +158,7 @@
// Type-check the package and also run analysis,
// then combine their diagnostics.
- pkg, _, err := source.PackageForFile(ctx, snapshot, fh.URI(), source.TypecheckFull, source.NarrowestPackage)
+ pkg, _, err := source.PackageForFile(ctx, snapshot, fh.URI(), source.NarrowestPackage)
if err != nil {
return nil, err
}
diff --git a/gopls/internal/lsp/debug/serve.go b/gopls/internal/lsp/debug/serve.go
index e79e88c..4bbcbcb 100644
--- a/gopls/internal/lsp/debug/serve.go
+++ b/gopls/internal/lsp/debug/serve.go
@@ -461,6 +461,13 @@
mux.HandleFunc("/info", render(InfoTmpl, i.getInfo))
mux.HandleFunc("/memory", render(MemoryTmpl, getMemory))
+ // Internal debugging helpers.
+ mux.HandleFunc("/_dogc", func(w http.ResponseWriter, r *http.Request) {
+ runtime.GC()
+ runtime.GC()
+ runtime.GC()
+ http.Error(w, "OK", 200)
+ })
mux.HandleFunc("/_makeabug", func(w http.ResponseWriter, r *http.Request) {
bug.Report("bug here", nil)
http.Error(w, "made a bug", http.StatusOK)
@@ -540,16 +547,6 @@
return err
}
- for _, cache := range i.State.Caches() {
- cf, err := zipw.Create(fmt.Sprintf("cache-%v.html", cache.ID()))
- if err != nil {
- return err
- }
- if _, err := cf.Write([]byte(cache.PackageStats(withNames))); err != nil {
- return err
- }
- }
-
if err := zipw.Close(); err != nil {
return err
}
@@ -821,8 +818,6 @@
{{define "body"}}
<h2>memoize.Store entries</h2>
<ul>{{range $k,$v := .MemStats}}<li>{{$k}} - {{$v}}</li>{{end}}</ul>
-<h2>Per-package usage - not accurate, for guidance only</h2>
-{{.PackageStats true}}
{{end}}
`))
diff --git a/gopls/internal/lsp/diagnostics.go b/gopls/internal/lsp/diagnostics.go
index 87d788f..99319bc 100644
--- a/gopls/internal/lsp/diagnostics.go
+++ b/gopls/internal/lsp/diagnostics.go
@@ -375,12 +375,11 @@
return
}
- pkgs, err := snapshot.TypeCheck(ctx, source.TypecheckFull, m.ID)
+ diags, err := snapshot.PackageDiagnostics(ctx, m.ID)
if err != nil {
- event.Error(ctx, "warning: typecheck failed", err, append(source.SnapshotLabels(snapshot), tag.Package.Of(string(m.ID)))...)
+ event.Error(ctx, "warning: diagnostics failed", err, append(source.SnapshotLabels(snapshot), tag.Package.Of(string(m.ID)))...)
return
}
- pkg := pkgs[0]
// Get diagnostics from analysis framework.
// This includes type-error analyzers, which suggest fixes to compiler errors.
@@ -395,23 +394,18 @@
}
// For each file, update the server's diagnostics state.
- for _, cgf := range pkg.CompiledGoFiles() {
+ for _, uri := range m.CompiledGoFiles {
// builtin.go exists only for documentation purposes and
// is not valid Go code. Don't report distracting errors.
- if snapshot.IsBuiltin(ctx, cgf.URI) {
+ if snapshot.IsBuiltin(ctx, uri) {
continue
}
- pkgDiags, err := pkg.DiagnosticsForFile(ctx, snapshot, cgf.URI)
- if err != nil {
- event.Error(ctx, "warning: getting package diagnostics", err, append(source.SnapshotLabels(snapshot), tag.Package.Of(string(m.ID)))...)
- return
- }
-
+ pkgDiags := diags[uri]
var tdiags, adiags []*source.Diagnostic
- source.CombineDiagnostics(pkgDiags, analysisDiags[cgf.URI], &tdiags, &adiags)
- s.storeDiagnostics(snapshot, cgf.URI, typeCheckSource, tdiags, true)
- s.storeDiagnostics(snapshot, cgf.URI, analysisSource, adiags, true)
+ source.CombineDiagnostics(pkgDiags, analysisDiags[uri], &tdiags, &adiags)
+ s.storeDiagnostics(snapshot, uri, typeCheckSource, tdiags, true)
+ s.storeDiagnostics(snapshot, uri, analysisSource, adiags, true)
}
// If gc optimization details are requested, add them to the
diff --git a/gopls/internal/lsp/lsp_test.go b/gopls/internal/lsp/lsp_test.go
index bcc1f14..75c46fe 100644
--- a/gopls/internal/lsp/lsp_test.go
+++ b/gopls/internal/lsp/lsp_test.go
@@ -54,7 +54,7 @@
func testLSP(t *testing.T, datum *tests.Data) {
ctx := tests.Context(t)
- session := cache.NewSession(ctx, cache.New(nil, nil), nil)
+ session := cache.NewSession(ctx, cache.New(nil), nil)
options := source.DefaultOptions().Clone()
tests.DefaultOptions(options)
session.SetOptions(options)
@@ -78,7 +78,31 @@
datum.ModfileFlagAvailable = len(snapshot.ModFiles()) > 0 && testenv.Go1Point() >= 14
release()
+ // Open all files for performance reasons. This is done because gopls only
+ // keeps active packages in memory for open files.
+ //
+ // In practice clients will only send document-oriented requests for open
+ // files.
var modifications []source.FileModification
+ for _, module := range datum.Exported.Modules {
+ for name := range module.Files {
+ filename := datum.Exported.File(module.Name, name)
+ if filepath.Ext(filename) != ".go" {
+ continue
+ }
+ content, err := datum.Exported.FileContents(filename)
+ if err != nil {
+ t.Fatal(err)
+ }
+ modifications = append(modifications, source.FileModification{
+ URI: span.URIFromPath(filename),
+ Action: source.Open,
+ Version: -1,
+ Text: content,
+ LanguageID: "go",
+ })
+ }
+ }
for filename, content := range datum.Config.Overlay {
if filepath.Ext(filename) != ".go" {
continue
diff --git a/gopls/internal/lsp/lsprpc/lsprpc_test.go b/gopls/internal/lsp/lsprpc/lsprpc_test.go
index 0dc78e6..6fb8372 100644
--- a/gopls/internal/lsp/lsprpc/lsprpc_test.go
+++ b/gopls/internal/lsp/lsprpc/lsprpc_test.go
@@ -57,7 +57,7 @@
client := FakeClient{Logs: make(chan string, 10)}
ctx = debug.WithInstance(ctx, "", "")
- ss := NewStreamServer(cache.New(nil, nil), false, nil)
+ ss := NewStreamServer(cache.New(nil), false, nil)
ss.serverForTest = server
ts := servertest.NewPipeServer(ss, nil)
defer checkClose(t, ts.Close)
@@ -120,7 +120,7 @@
func setupForwarding(ctx context.Context, t *testing.T, s protocol.Server) (direct, forwarded servertest.Connector, cleanup func()) {
t.Helper()
serveCtx := debug.WithInstance(ctx, "", "")
- ss := NewStreamServer(cache.New(nil, nil), false, nil)
+ ss := NewStreamServer(cache.New(nil), false, nil)
ss.serverForTest = s
tsDirect := servertest.NewTCPServer(serveCtx, ss, nil)
@@ -215,7 +215,7 @@
clientCtx := debug.WithInstance(baseCtx, "", "")
serverCtx := debug.WithInstance(baseCtx, "", "")
- cache := cache.New(nil, nil)
+ cache := cache.New(nil)
ss := NewStreamServer(cache, false, nil)
tsBackend := servertest.NewTCPServer(serverCtx, ss, nil)
diff --git a/gopls/internal/lsp/mod/diagnostics.go b/gopls/internal/lsp/mod/diagnostics.go
index 4891cd0..746a14e 100644
--- a/gopls/internal/lsp/mod/diagnostics.go
+++ b/gopls/internal/lsp/mod/diagnostics.go
@@ -95,24 +95,18 @@
event.Error(ctx, fmt.Sprintf("workspace packages: diagnosing %s", pm.URI), err)
}
if err == nil {
- // Type-check packages in parallel and gather list/parse/type errors.
- // (This may be the first operation after the initial metadata load
- // to demand type-checking of all active packages.)
+ // Note: the call to PackageDiagnostics below may be the first operation
+ // after the initial metadata load, and therefore result in type-checking
+ // or loading many packages.
ids := make([]source.PackageID, len(active))
for i, meta := range active {
ids[i] = meta.ID
}
- pkgs, err := snapshot.TypeCheck(ctx, source.TypecheckFull, ids...)
+ diags, err := snapshot.PackageDiagnostics(ctx, ids...)
if err != nil {
return nil, err
}
- for _, pkg := range pkgs {
- pkgDiags, err := pkg.DiagnosticsForFile(ctx, snapshot, fh.URI())
- if err != nil {
- return nil, err
- }
- diagnostics = append(diagnostics, pkgDiags...)
- }
+ diagnostics = append(diagnostics, diags[fh.URI()]...)
}
tidied, err := snapshot.ModTidy(ctx, pm)
diff --git a/gopls/internal/lsp/mod/mod_test.go b/gopls/internal/lsp/mod/mod_test.go
index eead8ac..c2aa1af 100644
--- a/gopls/internal/lsp/mod/mod_test.go
+++ b/gopls/internal/lsp/mod/mod_test.go
@@ -24,7 +24,7 @@
func TestModfileRemainsUnchanged(t *testing.T) {
ctx := tests.Context(t)
- session := cache.NewSession(ctx, cache.New(nil, nil), nil)
+ session := cache.NewSession(ctx, cache.New(nil), nil)
options := source.DefaultOptions().Clone()
tests.DefaultOptions(options)
options.TempModfile = true
diff --git a/gopls/internal/lsp/regtest/marker.go b/gopls/internal/lsp/regtest/marker.go
index c10445b..6a6e616 100644
--- a/gopls/internal/lsp/regtest/marker.go
+++ b/gopls/internal/lsp/regtest/marker.go
@@ -258,8 +258,8 @@
}
// Opt: use a shared cache.
- // TODO: opt: use a memoize store with no eviction.
- cache := cache.New(nil, nil)
+ // TODO(rfindley): opt: use a memoize store with no eviction.
+ cache := cache.New(nil)
for _, test := range tests {
t.Run(test.name, func(t *testing.T) {
diff --git a/gopls/internal/lsp/regtest/regtest.go b/gopls/internal/lsp/regtest/regtest.go
index b2ef357..5a5232f 100644
--- a/gopls/internal/lsp/regtest/regtest.go
+++ b/gopls/internal/lsp/regtest/regtest.go
@@ -8,7 +8,6 @@
"context"
"flag"
"fmt"
- "go/token"
"io/ioutil"
"os"
"runtime"
@@ -118,7 +117,6 @@
PrintGoroutinesOnFailure: *printGoroutinesOnFailure,
SkipCleanup: *skipCleanup,
OptionsHook: hook,
- fset: token.NewFileSet(),
store: memoize.NewStore(memoize.NeverEvict),
}
diff --git a/gopls/internal/lsp/regtest/runner.go b/gopls/internal/lsp/regtest/runner.go
index 20dac84..5f556b2 100644
--- a/gopls/internal/lsp/regtest/runner.go
+++ b/gopls/internal/lsp/regtest/runner.go
@@ -8,7 +8,6 @@
"bytes"
"context"
"fmt"
- "go/token"
"io"
"io/ioutil"
"net"
@@ -118,7 +117,6 @@
// Immutable state shared across test invocations
goplsPath string // path to the gopls executable (for SeparateProcess mode)
tempDir string // shared parent temp directory
- fset *token.FileSet // shared FileSet
store *memoize.Store // shared store
// Lazily allocated resources
@@ -336,7 +334,7 @@
// defaultServer handles the Default execution mode.
func (r *Runner) defaultServer(optsHook func(*source.Options)) jsonrpc2.StreamServer {
- return lsprpc.NewStreamServer(cache.New(r.fset, r.store), false, optsHook)
+ return lsprpc.NewStreamServer(cache.New(r.store), false, optsHook)
}
// experimentalServer handles the Experimental execution mode.
@@ -345,7 +343,7 @@
optsHook(o)
o.EnableAllExperiments()
}
- return lsprpc.NewStreamServer(cache.New(nil, nil), false, options)
+ return lsprpc.NewStreamServer(cache.New(nil), false, options)
}
// forwardedServer handles the Forwarded execution mode.
@@ -353,7 +351,7 @@
r.tsOnce.Do(func() {
ctx := context.Background()
ctx = debug.WithInstance(ctx, "", "off")
- ss := lsprpc.NewStreamServer(cache.New(nil, nil), false, optsHook)
+ ss := lsprpc.NewStreamServer(cache.New(nil), false, optsHook)
r.ts = servertest.NewTCPServer(ctx, ss, nil)
})
return newForwarder("tcp", r.ts.Addr)
diff --git a/gopls/internal/lsp/semantic.go b/gopls/internal/lsp/semantic.go
index d26c10f..46f9483 100644
--- a/gopls/internal/lsp/semantic.go
+++ b/gopls/internal/lsp/semantic.go
@@ -93,7 +93,7 @@
if kind != source.Go {
return nil, nil
}
- pkg, pgf, err := source.PackageForFile(ctx, snapshot, fh.URI(), source.TypecheckFull, source.NarrowestPackage)
+ pkg, pgf, err := source.PackageForFile(ctx, snapshot, fh.URI(), source.NarrowestPackage)
if err != nil {
return nil, err
}
diff --git a/gopls/internal/lsp/source/call_hierarchy.go b/gopls/internal/lsp/source/call_hierarchy.go
index bc7984a..2bdf7df 100644
--- a/gopls/internal/lsp/source/call_hierarchy.go
+++ b/gopls/internal/lsp/source/call_hierarchy.go
@@ -27,7 +27,7 @@
ctx, done := event.Start(ctx, "source.PrepareCallHierarchy")
defer done()
- pkg, pgf, err := PackageForFile(ctx, snapshot, fh.URI(), TypecheckFull, NarrowestPackage)
+ pkg, pgf, err := PackageForFile(ctx, snapshot, fh.URI(), NarrowestPackage)
if err != nil {
return nil, err
}
@@ -182,7 +182,7 @@
ctx, done := event.Start(ctx, "source.OutgoingCalls")
defer done()
- pkg, pgf, err := PackageForFile(ctx, snapshot, fh.URI(), TypecheckFull, NarrowestPackage)
+ pkg, pgf, err := PackageForFile(ctx, snapshot, fh.URI(), NarrowestPackage)
if err != nil {
return nil, err
}
@@ -221,7 +221,7 @@
}
// Use TypecheckFull as we want to inspect the body of the function declaration.
- declPkg, declPGF, err := PackageForFile(ctx, snapshot, uri, TypecheckFull, NarrowestPackage)
+ declPkg, declPGF, err := PackageForFile(ctx, snapshot, uri, NarrowestPackage)
if err != nil {
return nil, err
}
diff --git a/gopls/internal/lsp/source/code_lens.go b/gopls/internal/lsp/source/code_lens.go
index 5a18e53..ef1c3aa 100644
--- a/gopls/internal/lsp/source/code_lens.go
+++ b/gopls/internal/lsp/source/code_lens.go
@@ -100,7 +100,7 @@
if !strings.HasSuffix(fh.URI().Filename(), "_test.go") {
return out, nil
}
- pkg, pgf, err := PackageForFile(ctx, snapshot, fh.URI(), TypecheckFull, NarrowestPackage)
+ pkg, pgf, err := PackageForFile(ctx, snapshot, fh.URI(), NarrowestPackage)
if err != nil {
return out, err
}
diff --git a/gopls/internal/lsp/source/completion/completion.go b/gopls/internal/lsp/source/completion/completion.go
index e43ebc9..7ecd7a7 100644
--- a/gopls/internal/lsp/source/completion/completion.go
+++ b/gopls/internal/lsp/source/completion/completion.go
@@ -435,7 +435,7 @@
startTime := time.Now()
- pkg, pgf, err := source.PackageForFile(ctx, snapshot, fh.URI(), source.TypecheckFull, source.NarrowestPackage)
+ pkg, pgf, err := source.PackageForFile(ctx, snapshot, fh.URI(), source.NarrowestPackage)
if err != nil || pgf.File.Package == token.NoPos {
// If we can't parse this file or find position for the package
// keyword, it may be missing a package declaration. Try offering
diff --git a/gopls/internal/lsp/source/completion/format.go b/gopls/internal/lsp/source/completion/format.go
index fdbe124..634cdc8 100644
--- a/gopls/internal/lsp/source/completion/format.go
+++ b/gopls/internal/lsp/source/completion/format.go
@@ -81,7 +81,7 @@
detail = "struct{...}" // for anonymous structs
} else if obj.IsField() {
var err error
- detail, err = source.FormatVarType(ctx, c.snapshot, c.pkg, c.file, obj, c.qf, c.mq)
+ detail, err = source.FormatVarType(ctx, c.snapshot, c.pkg, obj, c.qf, c.mq)
if err != nil {
return CompletionItem{}, err
}
@@ -133,7 +133,7 @@
switch mod {
case invoke:
if sig, ok := funcType.Underlying().(*types.Signature); ok {
- s, err := source.NewSignature(ctx, c.snapshot, c.pkg, c.file, sig, nil, c.qf, c.mq)
+ s, err := source.NewSignature(ctx, c.snapshot, c.pkg, sig, nil, c.qf, c.mq)
if err != nil {
return CompletionItem{}, err
}
diff --git a/gopls/internal/lsp/source/completion/literal.go b/gopls/internal/lsp/source/completion/literal.go
index 6777f73..06ed559 100644
--- a/gopls/internal/lsp/source/completion/literal.go
+++ b/gopls/internal/lsp/source/completion/literal.go
@@ -202,7 +202,7 @@
// If the param has no name in the signature, guess a name based
// on the type. Use an empty qualifier to ignore the package.
// For example, we want to name "http.Request" "r", not "hr".
- typeName, err := source.FormatVarType(ctx, c.snapshot, c.pkg, c.file, p,
+ typeName, err := source.FormatVarType(ctx, c.snapshot, c.pkg, p,
func(p *types.Package) string { return "" },
func(source.PackageName, source.ImportPath, source.PackagePath) string { return "" })
if err != nil {
@@ -272,7 +272,7 @@
// of "i int, j int".
if i == sig.Params().Len()-1 || !types.Identical(p.Type(), sig.Params().At(i+1).Type()) {
snip.WriteText(" ")
- typeStr, err := source.FormatVarType(ctx, c.snapshot, c.pkg, c.file, p, c.qf, c.mq)
+ typeStr, err := source.FormatVarType(ctx, c.snapshot, c.pkg, p, c.qf, c.mq)
if err != nil {
// In general, the only error we should encounter while formatting is
// context cancellation.
@@ -330,7 +330,7 @@
snip.WriteText(name + " ")
}
- text, err := source.FormatVarType(ctx, c.snapshot, c.pkg, c.file, r, c.qf, c.mq)
+ text, err := source.FormatVarType(ctx, c.snapshot, c.pkg, r, c.qf, c.mq)
if err != nil {
// In general, the only error we should encounter while formatting is
// context cancellation.
diff --git a/gopls/internal/lsp/source/definition.go b/gopls/internal/lsp/source/definition.go
index 013a77f..cb87eb0 100644
--- a/gopls/internal/lsp/source/definition.go
+++ b/gopls/internal/lsp/source/definition.go
@@ -22,7 +22,7 @@
ctx, done := event.Start(ctx, "source.Definition")
defer done()
- pkg, pgf, err := PackageForFile(ctx, snapshot, fh.URI(), TypecheckFull, NarrowestPackage)
+ pkg, pgf, err := PackageForFile(ctx, snapshot, fh.URI(), NarrowestPackage)
if err != nil {
return nil, err
}
diff --git a/gopls/internal/lsp/source/diagnostics.go b/gopls/internal/lsp/source/diagnostics.go
index 2159720..e3f3598 100644
--- a/gopls/internal/lsp/source/diagnostics.go
+++ b/gopls/internal/lsp/source/diagnostics.go
@@ -71,7 +71,7 @@
if err != nil {
return nil, nil, err
}
- pkg, _, err := PackageForFile(ctx, snapshot, uri, TypecheckFull, NarrowestPackage)
+ pkg, _, err := PackageForFile(ctx, snapshot, uri, NarrowestPackage)
if err != nil {
return nil, nil, err
}
diff --git a/gopls/internal/lsp/source/fix.go b/gopls/internal/lsp/source/fix.go
index cbfe490..2ed55c4 100644
--- a/gopls/internal/lsp/source/fix.go
+++ b/gopls/internal/lsp/source/fix.go
@@ -55,7 +55,7 @@
// singleFile calls analyzers that expect inputs for a single file
func singleFile(sf singleFileFixFunc) SuggestedFixFunc {
return func(ctx context.Context, snapshot Snapshot, fh FileHandle, pRng protocol.Range) (*token.FileSet, *analysis.SuggestedFix, error) {
- pkg, pgf, err := PackageForFile(ctx, snapshot, fh.URI(), TypecheckFull, NarrowestPackage)
+ pkg, pgf, err := PackageForFile(ctx, snapshot, fh.URI(), NarrowestPackage)
if err != nil {
return nil, nil, err
}
diff --git a/gopls/internal/lsp/source/highlight.go b/gopls/internal/lsp/source/highlight.go
index 6438f03..a190f48 100644
--- a/gopls/internal/lsp/source/highlight.go
+++ b/gopls/internal/lsp/source/highlight.go
@@ -23,7 +23,7 @@
// We always want fully parsed files for highlight, regardless
// of whether the file belongs to a workspace package.
- pkg, pgf, err := PackageForFile(ctx, snapshot, fh.URI(), TypecheckFull, NarrowestPackage)
+ pkg, pgf, err := PackageForFile(ctx, snapshot, fh.URI(), NarrowestPackage)
if err != nil {
return nil, fmt.Errorf("getting package for Highlight: %w", err)
}
diff --git a/gopls/internal/lsp/source/hover.go b/gopls/internal/lsp/source/hover.go
index 25d85b4..136a302 100644
--- a/gopls/internal/lsp/source/hover.go
+++ b/gopls/internal/lsp/source/hover.go
@@ -87,7 +87,7 @@
// hovering at the position, it returns _, nil, nil: an error is only returned
// if the position is valid but we fail to compute hover information.
func hover(ctx context.Context, snapshot Snapshot, fh FileHandle, pp protocol.Position) (protocol.Range, *HoverJSON, error) {
- pkg, pgf, err := PackageForFile(ctx, snapshot, fh.URI(), TypecheckFull, NarrowestPackage)
+ pkg, pgf, err := PackageForFile(ctx, snapshot, fh.URI(), NarrowestPackage)
if err != nil {
return protocol.Range{}, nil, err
}
diff --git a/gopls/internal/lsp/source/implementation.go b/gopls/internal/lsp/source/implementation.go
index db216e6..72ec90d 100644
--- a/gopls/internal/lsp/source/implementation.go
+++ b/gopls/internal/lsp/source/implementation.go
@@ -95,7 +95,7 @@
for i, m := range declMetas {
ids[i] = m.ID
}
- localPkgs, err := snapshot.TypeCheck(ctx, TypecheckFull, ids...)
+ localPkgs, err := snapshot.TypeCheck(ctx, ids...)
if err != nil {
return nil, err
}
@@ -170,7 +170,7 @@
}
globalIDs = append(globalIDs, m.ID)
}
- globalPkgs, err := snapshot.TypeCheck(ctx, TypecheckFull, globalIDs...)
+ indexes, err := snapshot.MethodSets(ctx, globalIDs...)
if err != nil {
return nil, err
}
@@ -196,10 +196,10 @@
})
}
// global search
- for _, globalPkg := range globalPkgs {
- globalPkg := globalPkg
+ for _, index := range indexes {
+ index := index
group.Go(func() error {
- for _, res := range globalPkg.MethodSetsIndex().Search(key, queryMethodID) {
+ for _, res := range index.Search(key, queryMethodID) {
loc := res.Location
// Map offsets to protocol.Locations in parallel (may involve I/O).
group.Go(func() error {
@@ -244,7 +244,7 @@
func typeDeclPosition(ctx context.Context, snapshot Snapshot, uri span.URI, ppos protocol.Position) (token.Position, error) {
var noPosn token.Position
- pkg, pgf, err := PackageForFile(ctx, snapshot, uri, TypecheckFull, WidestPackage)
+ pkg, pgf, err := PackageForFile(ctx, snapshot, uri, WidestPackage)
if err != nil {
return noPosn, err
}
diff --git a/gopls/internal/lsp/source/inlay_hint.go b/gopls/internal/lsp/source/inlay_hint.go
index 9e40ba1..671d405 100644
--- a/gopls/internal/lsp/source/inlay_hint.go
+++ b/gopls/internal/lsp/source/inlay_hint.go
@@ -82,7 +82,7 @@
ctx, done := event.Start(ctx, "source.InlayHint")
defer done()
- pkg, pgf, err := PackageForFile(ctx, snapshot, fh.URI(), TypecheckFull, NarrowestPackage)
+ pkg, pgf, err := PackageForFile(ctx, snapshot, fh.URI(), NarrowestPackage)
if err != nil {
return nil, fmt.Errorf("getting file for InlayHint: %w", err)
}
diff --git a/gopls/internal/lsp/source/linkname.go b/gopls/internal/lsp/source/linkname.go
index 9017d14..c8afcdf 100644
--- a/gopls/internal/lsp/source/linkname.go
+++ b/gopls/internal/lsp/source/linkname.go
@@ -112,7 +112,7 @@
}
// When found, type check the desired package (snapshot.TypeCheck in TypecheckFull mode),
- pkgs, err := snapshot.TypeCheck(ctx, TypecheckFull, pkgMeta.ID)
+ pkgs, err := snapshot.TypeCheck(ctx, pkgMeta.ID)
if err != nil {
return nil, err
}
diff --git a/gopls/internal/lsp/source/methodsets/methodsets.go b/gopls/internal/lsp/source/methodsets/methodsets.go
index f297840..af836a5 100644
--- a/gopls/internal/lsp/source/methodsets/methodsets.go
+++ b/gopls/internal/lsp/source/methodsets/methodsets.go
@@ -44,10 +44,13 @@
// single 64-bit mask is quite effective. See CL 452060 for details.
import (
+ "bytes"
+ "encoding/gob"
"fmt"
"go/token"
"go/types"
"hash/crc32"
+ "log"
"strconv"
"strings"
@@ -64,6 +67,32 @@
pkg gobPackage
}
+// Decode decodes the given gob-encoded data as an Index.
+func Decode(data []byte) *Index {
+ var pkg gobPackage
+ mustDecode(data, &pkg)
+ return &Index{pkg}
+}
+
+// Encode encodes the receiver as gob-encoded data.
+func (index *Index) Encode() []byte {
+ return mustEncode(index.pkg)
+}
+
+func mustEncode(x interface{}) []byte {
+ var buf bytes.Buffer
+ if err := gob.NewEncoder(&buf).Encode(x); err != nil {
+ log.Fatalf("internal error encoding %T: %v", x, err)
+ }
+ return buf.Bytes()
+}
+
+func mustDecode(data []byte, ptr interface{}) {
+ if err := gob.NewDecoder(bytes.NewReader(data)).Decode(ptr); err != nil {
+ log.Fatalf("internal error decoding %T: %v", ptr, err)
+ }
+}
+
// NewIndex returns a new index of method-set information for all
// package-level types in the specified package.
func NewIndex(fset *token.FileSet, pkg *types.Package) *Index {
@@ -474,6 +503,6 @@
// A gobPosition records the file, offset, and length of an identifier.
type gobPosition struct {
- File int // index into gopPackage.Strings
+ File int // index into gobPackage.Strings
Offset, Len int // in bytes
}
diff --git a/gopls/internal/lsp/source/references.go b/gopls/internal/lsp/source/references.go
index 219ad6c..8f0fbfc 100644
--- a/gopls/internal/lsp/source/references.go
+++ b/gopls/internal/lsp/source/references.go
@@ -204,7 +204,7 @@
// declaration (e.g. because the _test.go files can change the
// meaning of a field or method selection), but the narrower
// package reports the more broadly referenced object.
- pkg, pgf, err := PackageForFile(ctx, snapshot, uri, TypecheckFull, NarrowestPackage)
+ pkg, pgf, err := PackageForFile(ctx, snapshot, uri, NarrowestPackage)
if err != nil {
return nil, err
}
@@ -346,12 +346,22 @@
}
// Compute global references for selected reverse dependencies.
- for _, m := range globalScope {
- m := m
- group.Go(func() error {
- return globalReferences(ctx, snapshot, m, globalTargets, report)
- })
- }
+ group.Go(func() error {
+ var globalIDs []PackageID
+ for id := range globalScope {
+ globalIDs = append(globalIDs, id)
+ }
+ indexes, err := snapshot.References(ctx, globalIDs...)
+ if err != nil {
+ return err
+ }
+ for _, index := range indexes {
+ for _, loc := range index.Lookup(globalTargets) {
+ report(loc, false)
+ }
+ }
+ return nil
+ })
if err := group.Wait(); err != nil {
return nil, err
@@ -379,20 +389,21 @@
allIDs = append(allIDs, m.ID)
}
// Search the methodset index of each package in the workspace.
- allPkgs, err := snapshot.TypeCheck(ctx, TypecheckFull, allIDs...)
+ indexes, err := snapshot.MethodSets(ctx, allIDs...)
if err != nil {
return err
}
var group errgroup.Group
- for _, pkg := range allPkgs {
- pkg := pkg
+ for i, index := range indexes {
+ i := i
+ index := index
group.Go(func() error {
// Consult index for matching methods.
- results := pkg.MethodSetsIndex().Search(key, method.Name())
+ results := index.Search(key, method.Name())
// Expand global search scope to include rdeps of this pkg.
if len(results) > 0 {
- rdeps, err := snapshot.ReverseDependencies(ctx, pkg.Metadata().ID, true)
+ rdeps, err := snapshot.ReverseDependencies(ctx, allIDs[i], true)
if err != nil {
return err
}
@@ -420,7 +431,7 @@
// localReferences reports each reference to the object
// declared at the specified URI/offset within its enclosing package m.
func localReferences(ctx context.Context, snapshot Snapshot, declURI span.URI, declOffset int, m *Metadata, report func(loc protocol.Location, isDecl bool)) error {
- pkgs, err := snapshot.TypeCheck(ctx, TypecheckFull, m.ID)
+ pkgs, err := snapshot.TypeCheck(ctx, m.ID)
if err != nil {
return err
}
@@ -554,23 +565,6 @@
return targets, path[0], nil
}
-// globalReferences reports each cross-package reference to one of the
-// target objects denoted by (package path, object path).
-func globalReferences(ctx context.Context, snapshot Snapshot, m *Metadata, targets map[PackagePath]map[objectpath.Path]unit, report func(loc protocol.Location, isDecl bool)) error {
- // TODO(adonovan): opt: don't actually type-check here,
- // since we quite intentionally don't look at type information.
- // Instead, access the reference index computed during
- // type checking that will in due course be a file-based cache.
- pkgs, err := snapshot.TypeCheck(ctx, TypecheckFull, m.ID)
- if err != nil {
- return err
- }
- for _, loc := range pkgs[0].ReferencesTo(targets) {
- report(loc, false)
- }
- return nil
-}
-
// mustLocation reports the location interval a syntax node,
// which must belong to m.File.
//
diff --git a/gopls/internal/lsp/source/rename.go b/gopls/internal/lsp/source/rename.go
index 63c6b7f..c67f15c 100644
--- a/gopls/internal/lsp/source/rename.go
+++ b/gopls/internal/lsp/source/rename.go
@@ -116,7 +116,7 @@
// which means we return (nil, nil) at the protocol
// layer. This seems like a bug, or at best an exploitation of
// knowledge of VSCode-specific behavior. Can we avoid that?
- pkg, pgf, err := PackageForFile(ctx, snapshot, f.URI(), TypecheckFull, NarrowestPackage)
+ pkg, pgf, err := PackageForFile(ctx, snapshot, f.URI(), NarrowestPackage)
if err != nil {
return nil, nil, err
}
@@ -295,7 +295,7 @@
// Type-check the referring package and locate the object(s).
// We choose the widest variant as, for non-exported
// identifiers, it is the only package we need.
- pkg, pgf, err := PackageForFile(ctx, snapshot, f.URI(), TypecheckFull, WidestPackage)
+ pkg, pgf, err := PackageForFile(ctx, snapshot, f.URI(), WidestPackage)
if err != nil {
return nil, err
}
@@ -463,6 +463,10 @@
ids = append(ids, id)
}
+ // Sort the packages into some topological order of the
+ // (unfiltered) metadata graph.
+ SortPostOrder(snapshot, ids)
+
// Dependencies must be visited first since they can expand
// the search set. Ideally we would process the (filtered) set
// of packages in the parallel postorder of the snapshot's
@@ -474,34 +478,32 @@
//
// Type checking is by far the dominant cost, so
// overlapping it with renaming may not be worthwhile.
- pkgs, err := snapshot.TypeCheck(ctx, TypecheckFull, ids...)
- if err != nil {
- return nil, err
- }
+ return snapshot.TypeCheck(ctx, ids...)
+}
- // Sort the packages into some topological order of the
- // (unfiltered) metadata graph.
+// SortPostOrder sorts the IDs so that if x depends on y, then y appears before x.
+func SortPostOrder(meta MetadataSource, ids []PackageID) {
postorder := make(map[PackageID]int)
+ order := 0
var visit func(PackageID)
visit = func(id PackageID) {
if _, ok := postorder[id]; !ok {
postorder[id] = -1 // break recursion
- if m := snapshot.Metadata(id); m != nil {
+ if m := meta.Metadata(id); m != nil {
for _, depID := range m.DepsByPkgPath {
visit(depID)
}
}
- postorder[id] = len(postorder)
+ order++
+ postorder[id] = order
}
}
for _, id := range ids {
visit(id)
}
- sort.Slice(pkgs, func(i, j int) bool {
- return postorder[pkgs[i].Metadata().ID] < postorder[pkgs[j].Metadata().ID]
+ sort.Slice(ids, func(i, j int) bool {
+ return postorder[ids[i]] < postorder[ids[j]]
})
-
- return pkgs, err
}
// renameExported renames the object denoted by (pkgPath, objPath)
@@ -881,7 +883,7 @@
for id := range needsTypeCheck {
ids = append(ids, id)
}
- pkgs, err := snapshot.TypeCheck(ctx, TypecheckFull, ids...)
+ pkgs, err := snapshot.TypeCheck(ctx, ids...)
if err != nil {
return err
}
diff --git a/gopls/internal/lsp/source/signature_help.go b/gopls/internal/lsp/source/signature_help.go
index 307de70..716de2d 100644
--- a/gopls/internal/lsp/source/signature_help.go
+++ b/gopls/internal/lsp/source/signature_help.go
@@ -23,7 +23,7 @@
// We need full type-checking here, as we must type-check function bodies in
// order to provide signature help at the requested position.
- pkg, pgf, err := PackageForFile(ctx, snapshot, fh.URI(), TypecheckFull, NarrowestPackage)
+ pkg, pgf, err := PackageForFile(ctx, snapshot, fh.URI(), NarrowestPackage)
if err != nil {
return nil, 0, fmt.Errorf("getting file for SignatureHelp: %w", err)
}
@@ -107,7 +107,7 @@
name = "func"
}
mq := MetadataQualifierForFile(snapshot, pgf.File, pkg.Metadata())
- s, err := NewSignature(ctx, snapshot, pkg, pgf.File, sig, comment, qf, mq)
+ s, err := NewSignature(ctx, snapshot, pkg, sig, comment, qf, mq)
if err != nil {
return nil, 0, err
}
diff --git a/gopls/internal/lsp/source/stub.go b/gopls/internal/lsp/source/stub.go
index 31d8135..6bbc1db 100644
--- a/gopls/internal/lsp/source/stub.go
+++ b/gopls/internal/lsp/source/stub.go
@@ -29,7 +29,7 @@
// methods of the concrete type that is assigned to an interface type
// at the cursor position.
func stubSuggestedFixFunc(ctx context.Context, snapshot Snapshot, fh FileHandle, rng protocol.Range) (*token.FileSet, *analysis.SuggestedFix, error) {
- pkg, pgf, err := PackageForFile(ctx, snapshot, fh.URI(), TypecheckFull, NarrowestPackage)
+ pkg, pgf, err := PackageForFile(ctx, snapshot, fh.URI(), NarrowestPackage)
if err != nil {
return nil, nil, fmt.Errorf("GetTypedFile: %w", err)
}
diff --git a/gopls/internal/lsp/source/type_definition.go b/gopls/internal/lsp/source/type_definition.go
index 6808f8e..104b7ac 100644
--- a/gopls/internal/lsp/source/type_definition.go
+++ b/gopls/internal/lsp/source/type_definition.go
@@ -18,7 +18,7 @@
ctx, done := event.Start(ctx, "source.TypeDefinition")
defer done()
- pkg, pgf, err := PackageForFile(ctx, snapshot, fh.URI(), TypecheckFull, NarrowestPackage)
+ pkg, pgf, err := PackageForFile(ctx, snapshot, fh.URI(), NarrowestPackage)
if err != nil {
return nil, err
}
diff --git a/gopls/internal/lsp/source/types_format.go b/gopls/internal/lsp/source/types_format.go
index 4d487d4..46e2602 100644
--- a/gopls/internal/lsp/source/types_format.go
+++ b/gopls/internal/lsp/source/types_format.go
@@ -196,7 +196,7 @@
}
// NewSignature returns formatted signature for a types.Signature struct.
-func NewSignature(ctx context.Context, s Snapshot, pkg Package, srcFile *ast.File, sig *types.Signature, comment *ast.CommentGroup, qf types.Qualifier, mq MetadataQualifier) (*signature, error) {
+func NewSignature(ctx context.Context, s Snapshot, pkg Package, sig *types.Signature, comment *ast.CommentGroup, qf types.Qualifier, mq MetadataQualifier) (*signature, error) {
var tparams []string
tpList := typeparams.ForSignature(sig)
for i := 0; i < tpList.Len(); i++ {
@@ -209,7 +209,7 @@
params := make([]string, 0, sig.Params().Len())
for i := 0; i < sig.Params().Len(); i++ {
el := sig.Params().At(i)
- typ, err := FormatVarType(ctx, s, pkg, srcFile, el, qf, mq)
+ typ, err := FormatVarType(ctx, s, pkg, el, qf, mq)
if err != nil {
return nil, err
}
@@ -227,7 +227,7 @@
needResultParens = true
}
el := sig.Results().At(i)
- typ, err := FormatVarType(ctx, s, pkg, srcFile, el, qf, mq)
+ typ, err := FormatVarType(ctx, s, pkg, el, qf, mq)
if err != nil {
return nil, err
}
@@ -266,7 +266,7 @@
//
// TODO(rfindley): this function could return the actual name used in syntax,
// for better parameter names.
-func FormatVarType(ctx context.Context, snapshot Snapshot, srcpkg Package, srcFile *ast.File, obj *types.Var, qf types.Qualifier, mq MetadataQualifier) (string, error) {
+func FormatVarType(ctx context.Context, snapshot Snapshot, srcpkg Package, obj *types.Var, qf types.Qualifier, mq MetadataQualifier) (string, error) {
// TODO(rfindley): This looks wrong. The previous comment said:
// "If the given expr refers to a type parameter, then use the
// object's Type instead of the type parameter declaration. This helps
diff --git a/gopls/internal/lsp/source/util.go b/gopls/internal/lsp/source/util.go
index 4149cea..82cb8d0 100644
--- a/gopls/internal/lsp/source/util.go
+++ b/gopls/internal/lsp/source/util.go
@@ -228,35 +228,6 @@
return search(m)
}
-// recursiveDeps finds unique transitive dependencies of m, including m itself.
-//
-// Invariant: for the resulting slice res, res[0] == m.ID.
-//
-// TODO(rfindley): consider replacing this with a snapshot.ForwardDependencies
-// method, or exposing the metadata graph itself.
-func recursiveDeps(s MetadataSource, m *Metadata) []PackageID {
- seen := make(map[PackageID]bool)
- var ids []PackageID
- var add func(*Metadata)
- add = func(m *Metadata) {
- if seen[m.ID] {
- return
- }
- seen[m.ID] = true
- ids = append(ids, m.ID)
- for _, dep := range m.DepsByPkgPath {
- m := s.Metadata(dep)
- if m == nil {
- bug.Reportf("nil metadata for %q", dep)
- continue
- }
- add(m)
- }
- }
- add(m)
- return ids
-}
-
// UnquoteImportPath returns the unquoted import path of s,
// or "" if the path is not properly quoted.
func UnquoteImportPath(s *ast.ImportSpec) ImportPath {
@@ -557,28 +528,6 @@
return strings.Contains(string(id), "command-line-arguments")
}
-// RecvIdent returns the type identifier of a method receiver.
-// e.g. A for all of A, *A, A[T], *A[T], etc.
-func RecvIdent(recv *ast.FieldList) *ast.Ident {
- if recv == nil || len(recv.List) == 0 {
- return nil
- }
- x := recv.List[0].Type
- if star, ok := x.(*ast.StarExpr); ok {
- x = star.X
- }
- switch ix := x.(type) { // check for instantiated receivers
- case *ast.IndexExpr:
- x = ix.X
- case *typeparams.IndexListExpr:
- x = ix.X
- }
- if ident, ok := x.(*ast.Ident); ok {
- return ident
- }
- return nil
-}
-
// embeddedIdent returns the type name identifier for an embedding x, if x in a
// valid embedding. Otherwise, it returns nil.
//
diff --git a/gopls/internal/lsp/source/view.go b/gopls/internal/lsp/source/view.go
index 7623b60..ebb9128 100644
--- a/gopls/internal/lsp/source/view.go
+++ b/gopls/internal/lsp/source/view.go
@@ -193,7 +193,26 @@
// and returns them in the same order as the ids.
// The resulting packages' types may belong to different importers,
// so types from different packages are incommensurable.
- TypeCheck(ctx context.Context, mode TypecheckMode, ids ...PackageID) ([]Package, error)
+ TypeCheck(ctx context.Context, ids ...PackageID) ([]Package, error)
+
+ // PackageDiagnostics returns diagnostics for files contained in specified
+ // packages.
+ //
+ // If these diagnostics cannot be loaded from cache, the requested packages
+ // may be type-checked.
+ PackageDiagnostics(ctx context.Context, ids ...PackageID) (map[span.URI][]*Diagnostic, error)
+
+ // References returns cross-references indexes for the specified packages.
+ //
+ // If these indexes cannot be loaded from cache, the requested packages may
+ // be type-checked.
+ References(ctx context.Context, ids ...PackageID) ([]XrefIndex, error)
+
+ // MethodSets returns method-set indexes for the specified packages.
+ //
+ // If these indexes cannot be loaded from cache, the requested packages may
+ // be type-checked.
+ MethodSets(ctx context.Context, ids ...PackageID) ([]*methodsets.Index, error)
// GetCriticalError returns any critical errors in the workspace.
//
@@ -201,6 +220,10 @@
GetCriticalError(ctx context.Context) *CriticalError
}
+type XrefIndex interface {
+ Lookup(targets map[PackagePath]map[objectpath.Path]struct{}) (locs []protocol.Location)
+}
+
// SnapshotLabels returns a new slice of labels that should be used for events
// related to a snapshot.
func SnapshotLabels(snapshot Snapshot) []label.Label {
@@ -214,7 +237,7 @@
//
// Type-checking is expensive. Call snapshot.ParseGo if all you need
// is a parse tree, or snapshot.MetadataForFile if you only need metadata.
-func PackageForFile(ctx context.Context, snapshot Snapshot, uri span.URI, mode TypecheckMode, pkgSel PackageSelector) (Package, *ParsedGoFile, error) {
+func PackageForFile(ctx context.Context, snapshot Snapshot, uri span.URI, pkgSel PackageSelector) (Package, *ParsedGoFile, error) {
metas, err := snapshot.MetadataForFile(ctx, uri)
if err != nil {
return nil, nil, err
@@ -228,7 +251,7 @@
case WidestPackage:
metas = metas[len(metas)-1:]
}
- pkgs, err := snapshot.TypeCheck(ctx, mode, metas[0].ID)
+ pkgs, err := snapshot.TypeCheck(ctx, metas[0].ID)
if err != nil {
return nil, nil, err
}
@@ -588,33 +611,12 @@
// This is the mode used when attempting to examine the package graph structure.
ParseHeader ParseMode = iota
- // ParseExported specifies that the package is used only as a dependency,
- // and only its exported declarations are needed. More may be included if
- // necessary to avoid type errors.
- ParseExported
-
// ParseFull specifies the full AST is needed.
// This is used for files of direct interest where the entire contents must
// be considered.
ParseFull
)
-// AllParseModes contains all possible values of ParseMode.
-// It is used for cache invalidation on a file content change.
-var AllParseModes = []ParseMode{ParseHeader, ParseExported, ParseFull}
-
-// TypecheckMode controls what kind of parsing should be done (see ParseMode)
-// while type checking a package.
-type TypecheckMode int
-
-const (
- // TypecheckFull means to use ParseFull.
- TypecheckFull TypecheckMode = iota
- // TypecheckWorkspace means to use ParseFull for workspace packages, and
- // ParseExported for others.
- TypecheckWorkspace
-)
-
// A FileHandle is an interface to files tracked by the LSP session, which may
// be either files read from disk, or open in the editor session (overlays).
type FileHandle interface {
@@ -800,8 +802,6 @@
DependencyTypes(PackagePath) *types.Package // nil for indirect dependency of no consequence
HasTypeErrors() bool
DiagnosticsForFile(ctx context.Context, s Snapshot, uri span.URI) ([]*Diagnostic, error)
- ReferencesTo(map[PackagePath]map[objectpath.Path]unit) []protocol.Location
- MethodSetsIndex() *methodsets.Index
}
type unit = struct{}
diff --git a/gopls/internal/lsp/testdata/bad/bad0.go b/gopls/internal/lsp/testdata/bad/bad0.go
index 9eedf4a..0f23a39 100644
--- a/gopls/internal/lsp/testdata/bad/bad0.go
+++ b/gopls/internal/lsp/testdata/bad/bad0.go
@@ -3,7 +3,7 @@
package bad
-import _ "golang.org/lsptests/assign/internal/secret" //@diag("\"golang.org/lsptests/assign/internal/secret\"", "compiler", "could not import golang.org/lsptests/assign/internal/secret \\(invalid use of internal package golang.org/lsptests/assign/internal/secret\\)", "error")
+import _ "golang.org/lsptests/assign/internal/secret" //@diag("\"golang.org/lsptests/assign/internal/secret\"", "compiler", "could not import golang.org/lsptests/assign/internal/secret \\(invalid use of internal package \"golang.org/lsptests/assign/internal/secret\"\\)", "error")
func stuff() { //@item(stuff, "stuff", "func()", "func")
x := "heeeeyyyy"
diff --git a/gopls/internal/lsp/tests/tests.go b/gopls/internal/lsp/tests/tests.go
index 5bf8a92..00521f5 100644
--- a/gopls/internal/lsp/tests/tests.go
+++ b/gopls/internal/lsp/tests/tests.go
@@ -611,6 +611,7 @@
t.Run("UnimportedCompletion", func(t *testing.T) {
t.Helper()
+ t.Skip("golang/go#58663: currently broken with incremental gopls")
eachCompletion(t, data.UnimportedCompletions, tests.UnimportedCompletion)
})
diff --git a/gopls/internal/regtest/bench/stress_test.go b/gopls/internal/regtest/bench/stress_test.go
index 11c511f..471e2e1 100644
--- a/gopls/internal/regtest/bench/stress_test.go
+++ b/gopls/internal/regtest/bench/stress_test.go
@@ -45,7 +45,7 @@
t.Fatal(err)
}
- server := lsprpc.NewStreamServer(cache.New(nil, nil), false, hooks.Options)
+ server := lsprpc.NewStreamServer(cache.New(nil), false, hooks.Options)
ts := servertest.NewPipeServer(server, jsonrpc2.NewRawStream)
ctx := context.Background()
diff --git a/gopls/internal/regtest/completion/completion_test.go b/gopls/internal/regtest/completion/completion_test.go
index 9859158..a52fcc8 100644
--- a/gopls/internal/regtest/completion/completion_test.go
+++ b/gopls/internal/regtest/completion/completion_test.go
@@ -232,11 +232,9 @@
})
}
+// TODO(rfindley): audit/clean up call sites for this helper, to ensure
+// consistent test errors.
func compareCompletionLabels(want []string, gotItems []protocol.CompletionItem) string {
- if len(gotItems) != len(want) {
- return fmt.Sprintf("got %v completion(s), want %v", len(gotItems), len(want))
- }
-
var got []string
for _, item := range gotItems {
got = append(got, item.Label)
@@ -246,12 +244,13 @@
}
}
- for i, v := range got {
- if v != want[i] {
- return fmt.Sprintf("%d completion result not the same: got %q, want %q", i, v, want[i])
- }
+ if len(got) == 0 && len(want) == 0 {
+ return "" // treat nil and the empty slice as equivalent
}
+ if diff := cmp.Diff(want, got); diff != "" {
+ return fmt.Sprintf("completion item mismatch (-want +got):\n%s", diff)
+ }
return ""
}
@@ -498,6 +497,8 @@
}
func TestUnimportedCompletion_VSCodeIssue1489(t *testing.T) {
+ t.Skip("golang/go#58663: currently broken with incremental gopls")
+
const src = `
-- go.mod --
module mod.com
@@ -537,7 +538,7 @@
func TestDefinition(t *testing.T) {
testenv.NeedsGo1Point(t, 17) // in go1.16, The FieldList in func x is not empty
- stuff := `
+ files := `
-- go.mod --
module mod.com
@@ -569,20 +570,16 @@
{"func Te(t *testing.T)", "Te", []string{"TestMain", "Test"}},
}
fname := "a_test.go"
- Run(t, stuff, func(t *testing.T, env *Env) {
+ Run(t, files, func(t *testing.T, env *Env) {
env.OpenFile(fname)
env.Await(env.DoneWithOpen())
- for _, tst := range tests {
- env.SetBufferContent(fname, "package foo\n"+tst.line)
- loc := env.RegexpSearch(fname, tst.pat)
- loc.Range.Start.Character += uint32(protocol.UTF16Len([]byte(tst.pat)))
+ for _, test := range tests {
+ env.SetBufferContent(fname, "package foo\n"+test.line)
+ loc := env.RegexpSearch(fname, test.pat)
+ loc.Range.Start.Character += uint32(protocol.UTF16Len([]byte(test.pat)))
completions := env.Completion(loc)
- result := compareCompletionLabels(tst.want, completions.Items)
- if result != "" {
- t.Errorf("\npat:%q line:%q failed: %s:%q", tst.pat, tst.line, result, tst.want)
- for i, it := range completions.Items {
- t.Errorf("%d got %q %q", i, it.Label, it.Detail)
- }
+ if diff := compareCompletionLabels(test.want, completions.Items); diff != "" {
+ t.Error(diff)
}
}
})
diff --git a/gopls/internal/regtest/diagnostics/diagnostics_test.go b/gopls/internal/regtest/diagnostics/diagnostics_test.go
index f96a0aa..b4b962e 100644
--- a/gopls/internal/regtest/diagnostics/diagnostics_test.go
+++ b/gopls/internal/regtest/diagnostics/diagnostics_test.go
@@ -1637,7 +1637,7 @@
).Run(t, mod, func(t *testing.T, env *Env) {
env.OnceMet(
InitialWorkspaceLoad,
- Diagnostics(env.AtRegexp("main.go", `"nosuchpkg"`), WithMessage(`cannot find package "nosuchpkg" in any of`)),
+ Diagnostics(env.AtRegexp("main.go", `"nosuchpkg"`), WithMessage(`cannot find package "nosuchpkg"`)),
)
})
})
diff --git a/gopls/internal/regtest/misc/leak_test.go b/gopls/internal/regtest/misc/leak_test.go
index 28a5843..88d2f01 100644
--- a/gopls/internal/regtest/misc/leak_test.go
+++ b/gopls/internal/regtest/misc/leak_test.go
@@ -31,7 +31,7 @@
println("1")
}
`
- c := cache.New(nil, nil)
+ c := cache.New(nil)
env := setupEnv(t, files, c)
env.Await(InitialWorkspaceLoad)
env.OpenFile("a.go")