gopls/internal/lsp/cache: remove package dependence on packages.Config

The cache.pkg type was a mix of metadata-related information and type
checking information, resulting in unnecessary relationships between
type-checking results (which are shared) and loading results (which are
not shared). As a result, the experimentalPackageCacheKey was more or
less a hope that these relationships were valid.

Avoid this relationship altogether by separating the shared
type-checking result from other derived calculations. This makes the
experimentalPackageCacheKey obsolete and lays the groundwork for
type-checking from export data.

Additionally:
- revisit the package cache key to ensure it covers all inputs into
  type-checking, and make it more similar to the analysis key
- remove methods from the source.Package API that return source.Package:
  we can't have edges between packages if they are going to be
  standalone
- remove the experimentalPackageCacheKey setting
- add a test for go list errors
- use the proper types.Sizes when type-checking
- address a comment from an earlier CL in completion_test.go

Fixes golang/go#57853

Change-Id: I238913c7c8305cb534db77ebec5f062e96ed2503
Reviewed-on: https://go-review.googlesource.com/c/tools/+/461944
Run-TryBot: Robert Findley <rfindley@google.com>
Reviewed-by: Alan Donovan <adonovan@google.com>
gopls-CI: kokoro <noreply+kokoro@google.com>
TryBot-Result: Gopher Robot <gobot@golang.org>
diff --git a/gopls/internal/lsp/cache/analysis.go b/gopls/internal/lsp/cache/analysis.go
index e9f1c7f..0005b0e 100644
--- a/gopls/internal/lsp/cache/analysis.go
+++ b/gopls/internal/lsp/cache/analysis.go
@@ -520,10 +520,8 @@
 	sz := m.TypesSizes.(*types.StdSizes)
 	fmt.Fprintf(hasher, "sizes: %d %d\n", sz.WordSize, sz.MaxAlign)
 
-	// metadata errors
-	for _, err := range m.Errors {
-		fmt.Fprintf(hasher, "error: %q", err)
-	}
+	// metadata errors: used for 'compiles' field
+	fmt.Fprintf(hasher, "errors: %d", len(m.Errors))
 
 	// module Go version
 	if m.Module != nil && m.Module.GoVersion != "" {
@@ -542,8 +540,8 @@
 		depIDs = append(depIDs, string(depID))
 	}
 	sort.Strings(depIDs)
-	for _, id := range depIDs {
-		vdep := vdeps[PackageID(id)]
+	for _, depID := range depIDs {
+		vdep := vdeps[PackageID(depID)]
 		fmt.Fprintf(hasher, "dep: %s\n", vdep.PkgPath)
 		fmt.Fprintf(hasher, "export: %s\n", vdep.DeepExportHash)
 
@@ -766,6 +764,7 @@
 	}
 
 	cfg := &types.Config{
+		Sizes: m.TypesSizes,
 		Error: func(e error) {
 			pkg.compiles = false // type error
 			pkg.typeErrors = append(pkg.typeErrors, e.(types.Error))
diff --git a/gopls/internal/lsp/cache/cache.go b/gopls/internal/lsp/cache/cache.go
index ef56b12..9da185c 100644
--- a/gopls/internal/lsp/cache/cache.go
+++ b/gopls/internal/lsp/cache/cache.go
@@ -209,7 +209,7 @@
 			typsCost := typesCost(v.pkg.types.Scope())
 			typInfoCost := typesInfoCost(v.pkg.typesInfo)
 			stat := packageStat{
-				id:        v.pkg.m.ID,
+				id:        v.pkg.id,
 				mode:      v.pkg.mode,
 				types:     typsCost,
 				typesInfo: typInfoCost,
diff --git a/gopls/internal/lsp/cache/check.go b/gopls/internal/lsp/cache/check.go
index 722691d..03691e5 100644
--- a/gopls/internal/lsp/cache/check.go
+++ b/gopls/internal/lsp/cache/check.go
@@ -5,21 +5,21 @@
 package cache
 
 import (
-	"bytes"
 	"context"
+	"crypto/sha256"
 	"errors"
 	"fmt"
 	"go/ast"
 	"go/types"
 	"path/filepath"
 	"regexp"
+	"sort"
 	"strings"
 	"sync"
 
 	"golang.org/x/mod/module"
 	"golang.org/x/sync/errgroup"
 	"golang.org/x/tools/go/ast/astutil"
-	"golang.org/x/tools/go/packages"
 	"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"
@@ -42,12 +42,14 @@
 
 type packageHandleKey source.Hash
 
-// A packageHandle is a handle to the future result of type-checking a package.
-// The resulting package is obtained from the await() method.
+// A packageHandle holds package information, some of which may not be fully
+// evaluated.
+//
+// The only methods on packageHandle that are safe to call before calling await
+// are Metadata and await itself.
 type packageHandle struct {
-	promise *memoize.Promise // [typeCheckResult]
-
-	// m is the metadata associated with the package.
+	// TODO(rfindley): remove metadata from packageHandle. It is only used for
+	// bug detection.
 	m *source.Metadata
 
 	// key is the hashed key for the package.
@@ -58,12 +60,29 @@
 	// enough. (The key for analysis actions could similarly
 	// 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 *pkg
+	pkg *syntaxPackage
 	err error
 }
 
@@ -101,7 +120,6 @@
 	// 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)
-	var depKey source.Hash // XOR of all unique deps
 	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
@@ -121,9 +139,6 @@
 			// checking the entire package. Leave depKeys[i] unset.
 			continue
 		}
-
-		depKey.XORWith(source.Hash(depHandle.key))
-
 		deps[depID] = depHandle
 	}
 
@@ -139,12 +154,29 @@
 		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.
-	experimentalKey := s.View().Options().ExperimentalPackageCacheKey
-	phKey := computePackageKey(m.ID, compiledGoFiles, m, depKey, mode, experimentalKey)
+	phKey := computePackageKey(s, inputs)
 	promise, release := s.store.Promise(phKey, func(ctx context.Context, arg interface{}) interface{} {
-		pkg, err := typeCheckImpl(ctx, arg.(*snapshot), goFiles, compiledGoFiles, m, mode, deps)
+		pkg, err := typeCheckImpl(ctx, arg.(*snapshot), inputs)
 		return typeCheckResult{pkg, err}
 	})
 
@@ -165,10 +197,11 @@
 	// handles for type-checking its immediate deps, at which
 	// point there will be no need to even access s.meta.)
 	if s.meta.metadata[ph.m.ID] != ph.m {
+		// TODO(rfindley): this should be bug.Errorf.
 		return nil, fmt.Errorf("stale metadata for %s", ph.m.ID)
 	}
 
-	// Check cache again in case another thread got there first.
+	// Check cache again in case another goroutine got there first.
 	if prev, ok := s.packages.Get(packageKey); ok {
 		prevPH := prev.(*packageHandle)
 		release()
@@ -223,61 +256,64 @@
 // computePackageKey returns a key representing the act of type checking
 // a package named id containing the specified files, metadata, and
 // combined dependency hash.
-func computePackageKey(id PackageID, files []source.FileHandle, m *source.Metadata, depsKey source.Hash, mode source.ParseMode, experimentalKey bool) packageHandleKey {
-	// TODO(adonovan): opt: no need to materalize the bytes; hash them directly.
-	// Also, use field separators to avoid spurious collisions.
-	b := bytes.NewBuffer(nil)
-	b.WriteString(string(id))
-	if m.Module != nil {
-		b.WriteString(m.Module.GoVersion) // go version affects type check errors.
-	}
-	if !experimentalKey {
-		// cfg was used to produce the other hashed inputs (package ID, parsed Go
-		// files, and deps). It should not otherwise affect the inputs to the type
-		// checker, so this experiment omits it. This should increase cache hits on
-		// the daemon as cfg contains the environment and working directory.
-		hc := hashConfig(m.Config)
-		b.Write(hc[:])
-	}
-	b.WriteByte(byte(mode))
-	b.Write(depsKey[:])
-	for _, file := range files {
-		b.WriteString(file.FileIdentity().String())
-	}
-	// Metadata errors are interpreted and memoized on the computed package, so
-	// we must hash them into the key here.
-	//
-	// TODO(rfindley): handle metadata diagnostics independently from
-	// type-checking diagnostics.
-	for _, err := range m.Errors {
-		b.WriteString(err.Msg)
-		b.WriteString(err.Pos)
-		b.WriteRune(rune(err.Kind))
-	}
-	return packageHandleKey(source.HashOf(b.Bytes()))
-}
+func computePackageKey(s *snapshot, inputs typeCheckInputs) packageHandleKey {
+	hasher := sha256.New()
 
-// hashConfig returns the hash for the *packages.Config.
-func hashConfig(config *packages.Config) source.Hash {
-	// TODO(adonovan): opt: don't materialize the bytes; hash them directly.
-	// Also, use sound field separators to avoid collisions.
-	var b bytes.Buffer
+	// In principle, a key must be the hash of an
+	// unambiguous encoding of all the relevant data.
+	// If it's ambiguous, we risk collisons.
 
-	// Dir, Mode, Env, BuildFlags are the parts of the config that can change.
-	b.WriteString(config.Dir)
-	b.WriteRune(rune(config.Mode))
+	// package identifiers
+	fmt.Fprintf(hasher, "package: %s %s %s\n", inputs.id, inputs.name, inputs.pkgPath)
 
-	for _, e := range config.Env {
-		b.WriteString(e)
+	// 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 {
+		importPaths = append(importPaths, string(impPath))
 	}
-	for _, f := range config.BuildFlags {
-		b.WriteString(f)
+	sort.Strings(importPaths)
+	for _, impPath := range importPaths {
+		fmt.Fprintf(hasher, "import %s %s", impPath, string(inputs.depsByImpPath[ImportPath(impPath)]))
 	}
-	return source.HashOf(b.Bytes())
+
+	// deps, in PackageID order
+	depIDs := make([]string, 0, len(inputs.deps))
+	for depID := range inputs.deps {
+		depIDs = append(depIDs, string(depID))
+	}
+	sort.Strings(depIDs)
+	for _, depID := range depIDs {
+		dep := inputs.deps[PackageID(depID)]
+		fmt.Fprintf(hasher, "dep: %s key:%s\n", dep.m.PkgPath, dep.key)
+	}
+
+	// file names and contents
+	fmt.Fprintf(hasher, "compiledGoFiles: %d\n", len(inputs.compiledGoFiles))
+	for _, fh := range inputs.compiledGoFiles {
+		fmt.Fprintln(hasher, fh.FileIdentity())
+	}
+	fmt.Fprintf(hasher, "goFiles: %d\n", len(inputs.goFiles))
+	for _, fh := range inputs.goFiles {
+		fmt.Fprintln(hasher, fh.FileIdentity())
+	}
+
+	// types sizes
+	sz := inputs.sizes.(*types.StdSizes)
+	fmt.Fprintf(hasher, "sizes: %d %d\n", sz.WordSize, sz.MaxAlign)
+
+	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) (*pkg, error) {
+func (ph *packageHandle) await(ctx context.Context, s *snapshot) (*syntaxPackage, error) {
 	v, err := s.awaitPromise(ctx, ph.promise)
 	if err != nil {
 		return nil, err
@@ -286,15 +322,7 @@
 	return data.pkg, data.err
 }
 
-func (ph *packageHandle) CompiledGoFiles() []span.URI {
-	return ph.m.CompiledGoFiles
-}
-
-func (ph *packageHandle) ID() string {
-	return string(ph.m.ID)
-}
-
-func (ph *packageHandle) cached() (*pkg, error) {
+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)
@@ -306,13 +334,13 @@
 // 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, goFiles, compiledGoFiles []source.FileHandle, m *source.Metadata, mode source.ParseMode, deps map[PackageID]*packageHandle) (*pkg, error) {
+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 deps {
+	for _, dep := range inputs.deps {
 		wg.Add(1)
 		go func(dep *packageHandle) {
 			dep.await(ctx, snapshot) // ignore result
@@ -328,56 +356,39 @@
 	defer wg.Wait()
 
 	var filter *unexportedFilter
-	if mode == source.ParseExported {
+	if inputs.mode == source.ParseExported {
 		filter = &unexportedFilter{uses: map[string]bool{}}
 	}
-	pkg, err := doTypeCheck(ctx, snapshot, goFiles, compiledGoFiles, m, mode, deps, filter)
+	pkg, err := doTypeCheck(ctx, snapshot, inputs, filter)
 	if err != nil {
 		return nil, err
 	}
 
-	if mode == source.ParseExported {
+	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, goFiles, compiledGoFiles, m, mode, deps, filter)
+			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, goFiles, compiledGoFiles, m, mode, deps, nil)
+			pkg, err = doTypeCheck(ctx, snapshot, inputs, nil)
 			if err != nil {
 				return nil, err
 			}
 		}
 	}
-	// If this is a replaced module in the workspace, the version is
-	// meaningless, and we don't want clients to access it.
-	if m.Module != nil {
-		pkg.version = &module.Version{
-			Path:    m.Module.Path,
-			Version: m.Module.Version,
-		}
-	}
 
 	// We don't care about a package's errors unless we have parsed it in full.
-	if mode != source.ParseFull {
+	if inputs.mode != source.ParseFull {
 		return pkg, nil
 	}
 
-	for _, e := range m.Errors {
-		diags, err := goPackagesErrorDiagnostics(snapshot, pkg, e)
-		if err != nil {
-			event.Error(ctx, "unable to compute positions for list errors", err, tag.Package.Of(string(pkg.ID())))
-			continue
-		}
-		pkg.diagnostics = append(pkg.diagnostics, diags...)
-	}
-
 	// Our heuristic for whether to show type checking errors is:
 	//  + If any file was 'fixed', don't show type checking errors as we
 	//    can't guarantee that they reference accurate locations in the source.
@@ -394,7 +405,7 @@
 	for _, e := range pkg.parseErrors {
 		diags, err := parseErrorDiagnostics(snapshot, pkg, e)
 		if err != nil {
-			event.Error(ctx, "unable to compute positions for parse errors", err, tag.Package.Of(string(pkg.ID())))
+			event.Error(ctx, "unable to compute positions for parse errors", err, tag.Package.Of(string(inputs.id)))
 			continue
 		}
 		for _, diag := range diags {
@@ -412,7 +423,7 @@
 	for _, e := range expandErrors(unexpanded, snapshot.View().Options().RelatedInformationSupported) {
 		diags, err := typeErrorDiagnostics(snapshot, pkg, e)
 		if err != nil {
-			event.Error(ctx, "unable to compute positions for type errors", err, tag.Package.Of(string(pkg.ID())))
+			event.Error(ctx, "unable to compute positions for type errors", err, tag.Package.Of(string(inputs.id)))
 			continue
 		}
 		pkg.typeErrors = append(pkg.typeErrors, e.primary)
@@ -426,27 +437,20 @@
 		}
 	}
 
-	depsErrors, err := snapshot.depsErrors(ctx, pkg)
-	if err != nil {
-		return nil, err
-	}
-	pkg.diagnostics = append(pkg.diagnostics, depsErrors...)
-
 	return pkg, nil
 }
 
 var goVersionRx = regexp.MustCompile(`^go([1-9][0-9]*)\.(0|[1-9][0-9]*)$`)
 
-func doTypeCheck(ctx context.Context, snapshot *snapshot, goFiles, compiledGoFiles []source.FileHandle, m *source.Metadata, mode source.ParseMode, deps map[PackageID]*packageHandle, astFilter *unexportedFilter) (*pkg, error) {
-	ctx, done := event.Start(ctx, "cache.typeCheck", tag.Package.Of(string(m.ID)))
+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()
 
-	pkg := &pkg{
-		m:     m,
-		mode:  mode,
+	pkg := &syntaxPackage{
+		id:    inputs.id,
+		mode:  inputs.mode,
 		fset:  snapshot.FileSet(), // must match parse call below (snapshot.ParseGo for now)
-		deps:  make(map[PackageID]*pkg),
-		types: types.NewPackage(string(m.PkgPath), string(m.Name)),
+		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),
@@ -461,9 +465,9 @@
 	// 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.
-	for _, fh := range goFiles {
-		goMode := mode
-		if mode == source.ParseExported {
+	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.
@@ -477,40 +481,32 @@
 	}
 
 	// Parse the CompiledGoFiles: those seen by the compiler/typechecker.
-	if err := parseCompiledGoFiles(ctx, compiledGoFiles, snapshot, mode, pkg, astFilter); err != nil {
+	if err := parseCompiledGoFiles(ctx, inputs.compiledGoFiles, snapshot, inputs.mode, pkg, astFilter); err != nil {
 		return nil, err
 	}
 
 	// Use the default type information for the unsafe package.
-	if m.PkgPath == "unsafe" {
+	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)
+		pkg.xrefs = xrefs.Index(pkg.compiledGoFiles, pkg.types, pkg.typesInfo)
 		return pkg, nil
 	}
 
-	if len(m.CompiledGoFiles) == 0 {
-		// No files most likely means go/packages failed. Try to attach error
-		// messages to the file as much as possible.
-		var found bool
-		for _, e := range m.Errors {
-			srcDiags, err := goPackagesErrorDiagnostics(snapshot, pkg, e)
-			if err != nil {
-				continue
-			}
-			found = true
-			pkg.diagnostics = append(pkg.diagnostics, srcDiags...)
-		}
-		if found {
-			return pkg, nil
-		}
-		return nil, fmt.Errorf("no parsed files for package %s, expected: %v, errors: %v", pkg.m.PkgPath, pkg.compiledGoFiles, m.Errors)
+	if len(pkg.compiledGoFiles) == 0 {
+		// No files most likely means go/packages failed.
+		//
+		// TODO(rfindley): in the past, we would capture go list errors in this
+		// case, to present go list errors to the user. However we had no tests for
+		// this behavior. It is unclear if anything better can be done here.
+		return nil, fmt.Errorf("no parsed files for package %s", inputs.pkgPath)
 	}
 
 	cfg := &types.Config{
+		Sizes: inputs.sizes,
 		Error: func(e error) {
 			pkg.typeErrors = append(pkg.typeErrors, e.(types.Error))
 		},
@@ -519,30 +515,30 @@
 			// based on the metadata before we start type checking,
 			// reporting them via types.Importer places the errors
 			// at the correct source location.
-			id, ok := pkg.m.DepsByImpPath[ImportPath(path)]
+			id, ok := inputs.depsByImpPath[ImportPath(path)]
 			if !ok {
 				// If the import declaration is broken,
 				// go list may fail to report metadata about it.
 				// See TestFixImportDecl for an example.
 				return nil, fmt.Errorf("missing metadata for import of %q", path)
 			}
-			dep, ok := deps[id] // id may be ""
+			dep, ok := inputs.deps[id] // id may be ""
 			if !ok {
 				return nil, snapshot.missingPkgError(path)
 			}
-			if !source.IsValidImport(m.PkgPath, dep.m.PkgPath) {
+			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 {
 				return nil, err
 			}
-			pkg.deps[depPkg.m.ID] = depPkg
 			return depPkg.types, nil
 		}),
 	}
-	if pkg.m.Module != nil && pkg.m.Module.GoVersion != "" {
-		goVersion := "go" + pkg.m.Module.GoVersion
+
+	if inputs.goVersion != "" {
+		goVersion := "go" + inputs.goVersion
 		// types.NewChecker panics if GoVersion is invalid. An unparsable mod
 		// file should probably stop us before we get here, but double check
 		// just in case.
@@ -551,7 +547,7 @@
 		}
 	}
 
-	if mode != source.ParseFull {
+	if inputs.mode != source.ParseFull {
 		cfg.DisableUnusedImportCheck = true
 		cfg.IgnoreFuncBodies = true
 	}
@@ -574,7 +570,7 @@
 	pkg.methodsets = methodsets.NewIndex(pkg.fset, pkg.types)
 
 	// Build global index of outbound cross-references.
-	pkg.xrefs = xrefs.Index(pkg)
+	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.
@@ -584,7 +580,7 @@
 	return pkg, nil
 }
 
-func parseCompiledGoFiles(ctx context.Context, compiledGoFiles []source.FileHandle, snapshot *snapshot, mode source.ParseMode, pkg *pkg, astFilter *unexportedFilter) error {
+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
@@ -633,11 +629,16 @@
 	return nil
 }
 
-func (s *snapshot) depsErrors(ctx context.Context, pkg *pkg) ([]*source.Diagnostic, error) {
+// depsErrors creates diagnostics for each metadata error (e.g. import cycle).
+// These may be attached to import declarations in the transitive source files
+// of pkg, or to 'requires' declarations in the package's go.mod file.
+//
+// TODO(rfindley): move this to errors.go
+func (s *snapshot) depsErrors(ctx context.Context, pkg *syntaxPackage, depsErrors []*packagesinternal.PackageError) ([]*source.Diagnostic, error) {
 	// Select packages that can't be found, and were imported in non-workspace packages.
 	// Workspace packages already show their own errors.
 	var relevantErrors []*packagesinternal.PackageError
-	for _, depsError := range pkg.m.DepsErrors {
+	for _, depsError := range depsErrors {
 		// Up to Go 1.15, the missing package was included in the stack, which
 		// was presumably a bug. We want the next one up.
 		directImporterIdx := len(depsError.ImportStack) - 1
@@ -665,7 +666,7 @@
 	allImports := map[string][]fileImport{}
 	for _, cgf := range pkg.compiledGoFiles {
 		// TODO(adonovan): modify Imports() to accept a single token.File (cgf.Tok).
-		for _, group := range astutil.Imports(s.FileSet(), cgf.File) {
+		for _, group := range astutil.Imports(pkg.fset, cgf.File) {
 			for _, imp := range group {
 				if imp.Path == nil {
 					continue
diff --git a/gopls/internal/lsp/cache/errors.go b/gopls/internal/lsp/cache/errors.go
index 8af1ac3..b0babc7 100644
--- a/gopls/internal/lsp/cache/errors.go
+++ b/gopls/internal/lsp/cache/errors.go
@@ -28,30 +28,20 @@
 	"golang.org/x/tools/internal/typesinternal"
 )
 
-func goPackagesErrorDiagnostics(snapshot *snapshot, pkg *pkg, e packages.Error) ([]*source.Diagnostic, error) {
-	if msg, spn, ok := parseGoListImportCycleError(snapshot, e, pkg); ok {
-		rng, err := spanToRange(pkg, spn)
-		if err != nil {
-			return nil, err
-		}
-		return []*source.Diagnostic{{
-			URI:      spn.URI(),
-			Range:    rng,
-			Severity: protocol.SeverityError,
-			Source:   source.ListError,
-			Message:  msg,
-		}}, nil
+func goPackagesErrorDiagnostics(e packages.Error, pkg *syntaxPackage, fromDir string) (diags []*source.Diagnostic, rerr error) {
+	if diag, ok := parseGoListImportCycleError(e, pkg); ok {
+		return []*source.Diagnostic{diag}, nil
 	}
 
 	var spn span.Span
 	if e.Pos == "" {
-		spn = parseGoListError(e.Msg, pkg.m.Config.Dir)
+		spn = parseGoListError(e.Msg, fromDir)
 		// We may not have been able to parse a valid span. Apply the errors to all files.
 		if _, err := spanToRange(pkg, spn); err != nil {
 			var diags []*source.Diagnostic
-			for _, cgf := range pkg.compiledGoFiles {
+			for _, pgf := range pkg.compiledGoFiles {
 				diags = append(diags, &source.Diagnostic{
-					URI:      cgf.URI,
+					URI:      pgf.URI,
 					Severity: protocol.SeverityError,
 					Source:   source.ListError,
 					Message:  e.Msg,
@@ -60,9 +50,20 @@
 			return diags, nil
 		}
 	} else {
-		spn = span.ParseInDir(e.Pos, pkg.m.Config.Dir)
+		spn = span.ParseInDir(e.Pos, fromDir)
 	}
 
+	// TODO(rfindley): in some cases the go command outputs invalid spans, for
+	// example (from TestGoListErrors):
+	//
+	//   package a
+	//   import
+	//
+	// In this case, the go command will complain about a.go:2:8, which is after
+	// the trailing newline but still considered to be on the second line, most
+	// likely because *token.File lacks information about newline termination.
+	//
+	// We could do better here by handling that case.
 	rng, err := spanToRange(pkg, spn)
 	if err != nil {
 		return nil, err
@@ -76,7 +77,7 @@
 	}}, nil
 }
 
-func parseErrorDiagnostics(snapshot *snapshot, pkg *pkg, errList scanner.ErrorList) ([]*source.Diagnostic, error) {
+func parseErrorDiagnostics(snapshot *snapshot, 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)
@@ -102,7 +103,7 @@
 var importErrorRe = regexp.MustCompile(`could not import ([^\s]+)`)
 var unsupportedFeatureRe = regexp.MustCompile(`.*require.* go(\d+\.\d+) or later`)
 
-func typeErrorDiagnostics(snapshot *snapshot, pkg *pkg, e extendedError) ([]*source.Diagnostic, error) {
+func typeErrorDiagnostics(snapshot *snapshot, pkg *syntaxPackage, e extendedError) ([]*source.Diagnostic, error) {
 	code, loc, err := typeErrorData(pkg, e.primary)
 	if err != nil {
 		return nil, err
@@ -286,7 +287,7 @@
 	return out
 }
 
-func typeErrorData(pkg *pkg, terr types.Error) (typesinternal.ErrorCode, protocol.Location, error) {
+func typeErrorData(pkg *syntaxPackage, terr types.Error) (typesinternal.ErrorCode, protocol.Location, error) {
 	ecode, start, end, ok := typesinternal.ReadGo116ErrorData(terr)
 	if !ok {
 		start, end = terr.Pos, terr.Pos
@@ -299,7 +300,7 @@
 	}
 	// go/types errors retain their FileSet.
 	// Sanity-check that we're using the right one.
-	fset := pkg.FileSet()
+	fset := pkg.fset
 	if fset != terr.Fset {
 		return 0, protocol.Location{}, bug.Errorf("wrong FileSet for type error")
 	}
@@ -320,7 +321,7 @@
 
 // spanToRange converts a span.Span to a protocol.Range,
 // assuming that the span belongs to the package whose diagnostics are being computed.
-func spanToRange(pkg *pkg, spn span.Span) (protocol.Range, error) {
+func spanToRange(pkg *syntaxPackage, spn span.Span) (protocol.Range, error) {
 	pgf, err := pkg.File(spn.URI())
 	if err != nil {
 		return protocol.Range{}, err
@@ -344,36 +345,39 @@
 	return span.ParseInDir(input[:msgIndex], wd)
 }
 
-func parseGoListImportCycleError(snapshot *snapshot, e packages.Error, pkg *pkg) (string, span.Span, bool) {
+func parseGoListImportCycleError(e packages.Error, pkg *syntaxPackage) (*source.Diagnostic, bool) {
 	re := regexp.MustCompile(`(.*): import stack: \[(.+)\]`)
 	matches := re.FindStringSubmatch(strings.TrimSpace(e.Msg))
 	if len(matches) < 3 {
-		return e.Msg, span.Span{}, false
+		return nil, false
 	}
 	msg := matches[1]
 	importList := strings.Split(matches[2], " ")
 	// Since the error is relative to the current package. The import that is causing
 	// the import cycle error is the second one in the list.
 	if len(importList) < 2 {
-		return msg, span.Span{}, false
+		return nil, false
 	}
 	// Imports have quotation marks around them.
 	circImp := strconv.Quote(importList[1])
-	for _, cgf := range pkg.compiledGoFiles {
+	for _, pgf := range pkg.compiledGoFiles {
 		// Search file imports for the import that is causing the import cycle.
-		for _, imp := range cgf.File.Imports {
+		for _, imp := range pgf.File.Imports {
 			if imp.Path.Value == circImp {
-				start, end, err := safetoken.Offsets(cgf.Tok, imp.Pos(), imp.End())
+				rng, err := pgf.PosMappedRange(imp.Pos(), imp.End())
 				if err != nil {
-					return msg, span.Span{}, false
+					return nil, false
 				}
-				spn, err := cgf.Mapper.OffsetSpan(start, end)
-				if err != nil {
-					return msg, span.Span{}, false
-				}
-				return msg, spn, true
+
+				return &source.Diagnostic{
+					URI:      pgf.URI,
+					Range:    rng.Range(),
+					Severity: protocol.SeverityError,
+					Source:   source.ListError,
+					Message:  msg,
+				}, true
 			}
 		}
 	}
-	return msg, span.Span{}, false
+	return nil, false
 }
diff --git a/gopls/internal/lsp/cache/error_test.go b/gopls/internal/lsp/cache/errors_test.go
similarity index 100%
rename from gopls/internal/lsp/cache/error_test.go
rename to gopls/internal/lsp/cache/errors_test.go
diff --git a/gopls/internal/lsp/cache/load.go b/gopls/internal/lsp/cache/load.go
index 1d69504..9b321ce 100644
--- a/gopls/internal/lsp/cache/load.go
+++ b/gopls/internal/lsp/cache/load.go
@@ -447,21 +447,13 @@
 		Name:       PackageName(pkg.Name),
 		ForTest:    PackagePath(packagesinternal.GetForTest(pkg)),
 		TypesSizes: pkg.TypesSizes,
-		Config:     cfg,
+		LoadDir:    cfg.Dir,
 		Module:     pkg.Module,
+		Errors:     pkg.Errors,
 		DepsErrors: packagesinternal.GetDepsErrors(pkg),
 	}
-	updates[id] = m
 
-	for _, err := range pkg.Errors {
-		// Filter out parse errors from go list. We'll get them when we
-		// actually parse, and buggy overlay support may generate spurious
-		// errors. (See TestNewModule_Issue38207.)
-		if strings.Contains(err.Msg, "expected '") {
-			continue
-		}
-		m.Errors = append(m.Errors, err)
-	}
+	updates[id] = m
 
 	for _, filename := range pkg.CompiledGoFiles {
 		uri := span.URIFromPath(filename)
diff --git a/gopls/internal/lsp/cache/pkg.go b/gopls/internal/lsp/cache/pkg.go
index 37aae50..bb4823c 100644
--- a/gopls/internal/lsp/cache/pkg.go
+++ b/gopls/internal/lsp/cache/pkg.go
@@ -5,19 +5,22 @@
 package cache
 
 import (
+	"context"
 	"fmt"
 	"go/ast"
 	"go/scanner"
 	"go/token"
 	"go/types"
+	"strings"
 
-	"golang.org/x/mod/module"
 	"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/event"
+	"golang.org/x/tools/internal/event/tag"
 	"golang.org/x/tools/internal/memoize"
 )
 
@@ -29,30 +32,93 @@
 	ImportPath  = source.ImportPath
 )
 
-// pkg contains parse trees and type information for a package.
-type pkg struct {
+// A Package is the union of snapshot-local information (Metadata) and shared
+// type-checking information (a syntaxPackage).
+//
+// TODO(rfindley): for now, we do not persist the post-processing of
+// loadDiagnostics, because the value of the snapshot.packages map  is just the
+// package handle. Fix this.
+type Package struct {
 	m               *source.Metadata
-	mode            source.ParseMode
+	pkg             *syntaxPackage
+	loadDiagnostics *memoize.Promise // post-processed errors from loading
+}
+
+func newPackage(m *source.Metadata, pkg *syntaxPackage) *Package {
+	p := &Package{
+		m:   m,
+		pkg: pkg,
+	}
+	if len(m.Errors) > 0 || len(m.DepsErrors) > 0 {
+		p.loadDiagnostics = memoize.NewPromise(fmt.Sprintf("loadDiagnostics(%s)", m.ID), func(ctx context.Context, arg interface{}) interface{} {
+			s := arg.(*snapshot)
+			var diags []*source.Diagnostic
+			for _, packagesErr := range p.m.Errors {
+				// Filter out parse errors from go list. We'll get them when we
+				// actually parse, and buggy overlay support may generate spurious
+				// errors. (See TestNewModule_Issue38207.)
+				if strings.Contains(packagesErr.Msg, "expected '") {
+					continue
+				}
+				pkgDiags, err := goPackagesErrorDiagnostics(packagesErr, p.pkg, p.m.LoadDir)
+				if err != nil {
+					// There are certain cases where the go command returns invalid
+					// positions, so we cannot panic or even bug.Reportf here.
+					event.Error(ctx, "unable to compute positions for list errors", err, tag.Package.Of(string(p.m.ID)))
+					continue
+				}
+				diags = append(diags, pkgDiags...)
+			}
+
+			// TODO(rfindley): this is buggy: an insignificant change to a modfile
+			// (or an unsaved modfile) could affect the position of deps errors,
+			// without invalidating the package.
+			depsDiags, err := s.depsErrors(ctx, p.pkg, p.m.DepsErrors)
+			if err != nil {
+				if ctx.Err() == nil {
+					// TODO(rfindley): consider making this a bug.Reportf. depsErrors should
+					// not normally fail.
+					event.Error(ctx, "unable to compute deps errors", err, tag.Package.Of(string(p.m.ID)))
+				}
+				return nil
+			}
+			diags = append(diags, depsDiags...)
+			return diags
+		})
+	}
+	return p
+}
+
+// syntaxPackage contains parse trees and type information for a package.
+type syntaxPackage struct {
+	// -- identifiers --
+	id   PackageID
+	mode source.ParseMode
+
+	// -- outputs --
 	fset            *token.FileSet // for now, same as the snapshot's FileSet
 	goFiles         []*source.ParsedGoFile
 	compiledGoFiles []*source.ParsedGoFile
 	diagnostics     []*source.Diagnostic
-	deps            map[PackageID]*pkg // use m.DepsBy{Pkg,Imp}Path to look up ID
-	version         *module.Version    // may be nil; may differ from m.Module.Version
 	parseErrors     []scanner.ErrorList
 	typeErrors      []types.Error
 	types           *types.Package
 	typesInfo       *types.Info
-	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
+	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
 }
 
-func (p *pkg) String() string { return string(p.ID()) }
+func (p *Package) String() string { return string(p.m.ID) }
+
+func (p *Package) Metadata() *source.Metadata {
+	return p.m
+}
 
 // A loadScope defines a package loading scope for use with go/packages.
+//
+// TODO(rfindley): move this to load.go.
 type loadScope interface {
 	aScope()
 }
@@ -70,127 +136,96 @@
 func (moduleLoadScope) aScope()  {}
 func (viewLoadScope) aScope()    {}
 
-func (p *pkg) ID() PackageID        { return p.m.ID }
-func (p *pkg) Name() PackageName    { return p.m.Name }
-func (p *pkg) PkgPath() PackagePath { return p.m.PkgPath }
-
-func (p *pkg) ParseMode() source.ParseMode {
-	return p.mode
+func (p *Package) ParseMode() source.ParseMode {
+	return p.pkg.mode
 }
 
-func (p *pkg) CompiledGoFiles() []*source.ParsedGoFile {
-	return p.compiledGoFiles
+func (p *Package) CompiledGoFiles() []*source.ParsedGoFile {
+	return p.pkg.compiledGoFiles
 }
 
-func (p *pkg) File(uri span.URI) (*source.ParsedGoFile, error) {
-	for _, cgf := range p.compiledGoFiles {
+func (p *Package) File(uri span.URI) (*source.ParsedGoFile, error) {
+	return p.pkg.File(uri)
+}
+
+func (pkg *syntaxPackage) File(uri span.URI) (*source.ParsedGoFile, error) {
+	for _, cgf := range pkg.compiledGoFiles {
 		if cgf.URI == uri {
 			return cgf, nil
 		}
 	}
-	for _, gf := range p.goFiles {
+	for _, gf := range pkg.goFiles {
 		if gf.URI == uri {
 			return gf, nil
 		}
 	}
-	return nil, fmt.Errorf("no parsed file for %s in %v", uri, p.m.ID)
+	return nil, fmt.Errorf("no parsed file for %s in %v", uri, pkg.id)
 }
 
-func (p *pkg) GetSyntax() []*ast.File {
+func (p *Package) GetSyntax() []*ast.File {
 	var syntax []*ast.File
-	for _, pgf := range p.compiledGoFiles {
+	for _, pgf := range p.pkg.compiledGoFiles {
 		syntax = append(syntax, pgf.File)
 	}
 	return syntax
 }
 
-func (p *pkg) FileSet() *token.FileSet {
-	return p.fset
+func (p *Package) FileSet() *token.FileSet {
+	return p.pkg.fset
 }
 
-func (p *pkg) GetTypes() *types.Package {
-	return p.types
+func (p *Package) GetTypes() *types.Package {
+	return p.pkg.types
 }
 
-func (p *pkg) GetTypesInfo() *types.Info {
-	return p.typesInfo
+func (p *Package) GetTypesInfo() *types.Info {
+	return p.pkg.typesInfo
 }
 
-func (p *pkg) GetTypesSizes() types.Sizes {
-	return p.m.TypesSizes
+func (p *Package) HasParseErrors() bool {
+	return len(p.pkg.parseErrors) != 0
 }
 
-func (p *pkg) ForTest() string {
-	return string(p.m.ForTest)
+func (p *Package) HasTypeErrors() bool {
+	return len(p.pkg.typeErrors) != 0
 }
 
-// DirectDep returns the directly imported dependency of this package,
-// given its PackagePath.  (If you have an ImportPath, e.g. a string
-// from an import declaration, use ResolveImportPath instead.
-// They may differ in case of vendoring.)
-func (p *pkg) DirectDep(pkgPath PackagePath) (source.Package, error) {
-	if id, ok := p.m.DepsByPkgPath[pkgPath]; ok {
-		if imp := p.deps[id]; imp != nil {
-			return imp, nil
-		}
-	}
-	return nil, fmt.Errorf("package does not import package with path %s", pkgPath)
-}
-
-// ResolveImportPath returns the directly imported dependency of this package,
-// given its ImportPath. See also DirectDep.
-func (p *pkg) ResolveImportPath(importPath ImportPath) (source.Package, error) {
-	if id, ok := p.m.DepsByImpPath[importPath]; ok && id != "" {
-		if imp := p.deps[id]; imp != nil {
-			return imp, nil
-		}
-	}
-	return nil, fmt.Errorf("package does not import %s", importPath)
-}
-
-func (p *pkg) Imports() []source.Package {
-	var result []source.Package // unordered
-	for _, dep := range p.deps {
-		result = append(result, dep)
-	}
-	return result
-}
-
-func (p *pkg) Version() *module.Version {
-	return p.version
-}
-
-func (p *pkg) HasListOrParseErrors() bool {
-	return len(p.m.Errors) != 0 || len(p.parseErrors) != 0
-}
-
-func (p *pkg) HasTypeErrors() bool {
-	return len(p.typeErrors) != 0
-}
-
-func (p *pkg) DiagnosticsForFile(uri span.URI) []*source.Diagnostic {
-	var res []*source.Diagnostic
-	for _, diag := range p.diagnostics {
+func (p *Package) DiagnosticsForFile(ctx context.Context, s source.Snapshot, uri span.URI) ([]*source.Diagnostic, error) {
+	var diags []*source.Diagnostic
+	for _, diag := range p.pkg.diagnostics {
 		if diag.URI == uri {
-			res = append(res, diag)
+			diags = append(diags, diag)
 		}
 	}
-	return res
+
+	if p.loadDiagnostics != nil {
+		res, err := p.loadDiagnostics.Get(ctx, s)
+		if err != nil {
+			return nil, err
+		}
+		for _, diag := range res.([]*source.Diagnostic) {
+			if diag.URI == uri {
+				diags = append(diags, diag)
+			}
+		}
+	}
+
+	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 *pkg) ReferencesTo(targets map[PackagePath]map[objectpath.Path]struct{}) []protocol.Location {
+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.xrefs, targets)
+	return xrefs.Lookup(p.m, p.pkg.xrefs, targets)
 }
 
-func (p *pkg) MethodSetsIndex() *methodsets.Index {
+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.methodsets
+	return p.pkg.methodsets
 }
diff --git a/gopls/internal/lsp/cache/snapshot.go b/gopls/internal/lsp/cache/snapshot.go
index 51d7a79..995fba3 100644
--- a/gopls/internal/lsp/cache/snapshot.go
+++ b/gopls/internal/lsp/cache/snapshot.go
@@ -118,7 +118,7 @@
 	// analyses maps an analysisKey (which identifies a package
 	// and a set of analyzers) to the handle for the future result
 	// of loading the package and analyzing it.
-	analyses *persistent.Map // from analysisKey to analysisHandle
+	analyses *persistent.Map // from analysisKey to analysisPromise
 
 	// workspacePackages contains the workspace's packages, which are loaded
 	// when the view is created.
@@ -626,10 +626,20 @@
 }
 
 // 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...
-	var phs []*packageHandle
-	for _, id := range ids {
+	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)
@@ -637,21 +647,36 @@
 
 		ph, err := s.buildPackageHandle(ctx, id, parseMode)
 		if err != nil {
-			return nil, err
+			if firstErr == nil {
+				firstErr = err
+			}
+			if ctx.Err() != nil {
+				return pkgs, firstErr
+			}
+			continue
 		}
-		phs = append(phs, ph)
+		phs[i] = ph
 	}
 
 	// ...then await them all.
-	var pkgs []source.Package
-	for _, ph := range phs {
-		pkg, err := ph.await(ctx, s)
-		if err != nil {
-			return nil, err
+	for i, ph := range phs {
+		if ph == nil {
+			continue
 		}
-		pkgs = append(pkgs, pkg)
+		p, err := ph.await(ctx, s)
+		if err != nil {
+			if firstErr == nil {
+				firstErr = err
+			}
+			if ctx.Err() != nil {
+				return pkgs, firstErr
+			}
+			continue
+		}
+		pkgs[i] = newPackage(ph.m, p)
 	}
-	return pkgs, nil
+
+	return pkgs, firstErr
 }
 
 func (s *snapshot) MetadataForFile(ctx context.Context, uri span.URI) ([]*source.Metadata, error) {
@@ -1058,7 +1083,7 @@
 	return meta, nil
 }
 
-func (s *snapshot) CachedImportPaths(ctx context.Context) (map[PackagePath]source.Package, error) {
+func (s *snapshot) CachedImportPaths(ctx context.Context) (map[PackagePath]*types.Package, error) {
 	// Don't reload workspace package metadata.
 	// This function is meant to only return currently cached information.
 	s.AwaitInitialized(ctx)
@@ -1066,24 +1091,41 @@
 	s.mu.Lock()
 	defer s.mu.Unlock()
 
-	results := map[PackagePath]source.Package{}
+	pkgs := make(map[PackagePath]*syntaxPackage)
+
+	// Find all cached packages that are imported a nonzero amount of time.
+	//
+	// TODO(rfindley): this is pre-existing behavior, and a test fails if we
+	// don't do the importCount filter, but why do we care if a package is
+	// imported a nonzero amount of times?
+	imported := make(map[PackagePath]bool)
 	s.packages.Range(func(_, v interface{}) {
-		cachedPkg, err := v.(*packageHandle).cached()
+		ph := v.(*packageHandle)
+		for dep := range ph.m.DepsByPkgPath {
+			imported[dep] = true
+		}
+		if ph.m.Name == "main" {
+			return
+		}
+		pkg, err := ph.cached()
 		if err != nil {
 			return
 		}
-		for _, newPkg := range cachedPkg.deps {
-			pkgPath := newPkg.PkgPath()
-			if oldPkg, ok := results[pkgPath]; ok {
-				// Using the same trick as NarrowestPackage, prefer non-variants.
-				if len(newPkg.compiledGoFiles) < len(oldPkg.(*pkg).compiledGoFiles) {
-					results[pkgPath] = newPkg
-				}
-			} else {
-				results[pkgPath] = newPkg
+		if old, ok := pkgs[ph.m.PkgPath]; ok {
+			if len(pkg.compiledGoFiles) < len(old.compiledGoFiles) {
+				pkgs[ph.m.PkgPath] = pkg
 			}
+		} else {
+			pkgs[ph.m.PkgPath] = pkg
 		}
 	})
+	results := make(map[PackagePath]*types.Package)
+	for pkgPath, pkg := range pkgs {
+		if imported[pkgPath] {
+			results[pkgPath] = pkg.types
+		}
+	}
+
 	return results, nil
 }