gopls/internal/cache: express packageHandle as a state machine

In preparation for storing active packages on packageHandle, express the
various package handle states using a new packageState type, and hold on
to package handle data even if their local files may have changed, as
long as their metadata did not change.

Also: rename buildPackageHandle to evaluatePackageHandle, which better
matches its meaning, and move the package ID index to the View, since it
is shared across all snapshots.

Change-Id: I2c14613d320b1121f20bb3960d42370bef5ad98b
Reviewed-on: https://go-review.googlesource.com/c/tools/+/614164
LUCI-TryBot-Result: Go LUCI <golang-scoped@luci-project-accounts.iam.gserviceaccount.com>
Reviewed-by: Alan Donovan <adonovan@google.com>
This commit is contained in:
Rob Findley 2024-09-19 13:05:48 +00:00 коммит произвёл Robert Findley
Родитель dd745ec14b
Коммит a19eef6bcb
4 изменённых файлов: 194 добавлений и 180 удалений

340
gopls/internal/cache/check.go поставляемый
Просмотреть файл

@ -77,6 +77,7 @@ func (b *typeCheckBatch) addHandles(handles map[PackageID]*packageHandle) {
b.handleMu.Lock() b.handleMu.Lock()
defer b.handleMu.Unlock() defer b.handleMu.Unlock()
for id, ph := range handles { for id, ph := range handles {
assert(ph.state == validKey, "invalid handle")
if alt, ok := b._handles[id]; ok { if alt, ok := b._handles[id]; ok {
// Once handles have been reevaluated, they should not change. Therefore, // Once handles have been reevaluated, they should not change. Therefore,
// we should only ever encounter exactly one handle instance for a given // we should only ever encounter exactly one handle instance for a given
@ -832,40 +833,62 @@ func (b *typeCheckBatch) importLookup(mp *metadata.Package) func(PackagePath) Pa
} }
} }
// A packageHandle holds inputs required to compute a Package, including // A packageState is the state of a [packageHandle]; see below for details.
// metadata, derived diagnostics, files, and settings. Additionally, type packageState uint8
// packageHandles manage a key for these inputs, to use in looking up
// precomputed results. const (
validMetadata packageState = iota // the package has valid metadata (initial state)
validLocalData // local package files have been analyzed
validKey // dependencies have been analyzed, and key produced
)
// A packageHandle holds information derived from a metadata.Package, and
// records its degree of validity as state changes occur: successful analysis
// causes the state to progress; invalidation due to changes causes it to
// regress.
// //
// packageHandles may be invalid following an invalidation via snapshot.clone, // In the initial state (validMetadata), all we know is the metadata for the
// but the handles returned by getPackageHandles will always be valid. // package itself. This is the lowest state, and it cannot become invalid
// because the metadata for a given snapshot never changes. (Each handle is
// implicitly associated with a Snapshot.)
// //
// packageHandles are critical for implementing "precise pruning" in gopls: // After the files of the package have been read (validLocalData), we can
// packageHandle.key is a hash of a precise set of inputs, such as package // perform computations that are local to that package, such as parsing, or
// files and "reachable" syntax, that may affect type checking. // building the symbol reference graph (SRG). This information is invalidated
// by a change to any file in the package. The local information is thus
// sufficient to form a cache key for saved parsed trees or the SRG.
// //
// packageHandles also keep track of state that allows gopls to compute, and // Once all dependencies have been analyzed (validKey), we can type-check the
// then quickly recompute, these keys. This state is split into two categories: // package. This information is invalidated by any change to the package
// - local state, which depends only on the package's local files and metadata // itself, or to any dependency that is transitively reachable through the SRG.
// - other state, which includes data derived from dependencies. // The cache key for saved type information must thus incorporate information
// from all reachable dependencies. This reachability analysis implements what
// we sometimes refer to as "precise pruning", or fine-grained invalidation:
// https://go.dev/blog/gopls-scalability#invalidation
// //
// Dividing the data in this way allows gopls to minimize invalidation when a // Following a change, the packageHandle is cloned in the new snapshot with a
// package is modified. For example, any change to a package file fully // new state set to its least known valid state, as described above: if package
// invalidates the package handle. On the other hand, if that change was not // files changed, it is reset to validMetadata; if dependencies changed, it is
// metadata-affecting it may be the case that packages indirectly depending on // reset to validLocalData. However, the derived data from its previous state
// the modified package are unaffected by the change. For that reason, we have // is not yet removed, as keys may not have changed after they are reevaluated,
// two types of invalidation, corresponding to the two types of data above: // in which case we can avoid recomputing the derived data.
// - deletion of the handle, which occurs when the package itself changes
// - clearing of the validated field, which marks the package as possibly
// invalid.
// //
// With the second type of invalidation, packageHandles are re-evaluated from the // See [packageHandleBuilder.evaluatePackageHandle] for more details of the
// bottom up. If this process encounters a packageHandle whose deps have not // reevaluation algorithm.
// changed (as detected by the depkeys field), then the packageHandle in //
// question must also not have changed, and we need not re-evaluate its key. // packageHandles are immutable once they are stored in the Snapshot.packages
// map: any changes to packageHandle fields evaluatePackageHandle must be made
// to a cloned packageHandle, and inserted back into Snapshot.packages. Data
// referred to by the packageHandle may be shared by multiple clones, and so
// referents must not be mutated.
type packageHandle struct { type packageHandle struct {
mp *metadata.Package mp *metadata.Package
// state indicates which data below are still valid.
state packageState
// Local data:
// loadDiagnostics memoizes the result of processing error messages from // loadDiagnostics memoizes the result of processing error messages from
// go/packages (i.e. `go list`). // go/packages (i.e. `go list`).
// //
@ -883,27 +906,19 @@ type packageHandle struct {
// (Nevertheless, since the lifetime of load diagnostics matches that of the // (Nevertheless, since the lifetime of load diagnostics matches that of the
// Metadata, it is convenient to memoize them here.) // Metadata, it is convenient to memoize them here.)
loadDiagnostics []*Diagnostic loadDiagnostics []*Diagnostic
// Local data:
// localInputs holds all local type-checking localInputs, excluding // localInputs holds all local type-checking localInputs, excluding
// dependencies. // dependencies.
localInputs typeCheckInputs localInputs *typeCheckInputs
// localKey is a hash of localInputs. // localKey is a hash of localInputs.
localKey file.Hash localKey file.Hash
// refs is the result of syntactic dependency analysis produced by the // refs is the result of syntactic dependency analysis produced by the
// typerefs package. // typerefs package. Derived from localInputs.
refs map[string][]typerefs.Symbol refs map[string][]typerefs.Symbol
// Data derived from dependencies: // Keys, computed through reachability analysis of dependencies.
// validated indicates whether the current packageHandle is known to have a
// valid key. Invalidated package handles are stored for packages whose
// type information may have changed.
validated bool
// depKeys records the key of each dependency that was used to calculate the // depKeys records the key of each dependency that was used to calculate the
// key above. If the handle becomes invalid, we must re-check that each still // key below. If state < validKey, we must re-check that each still matches.
// matches.
depKeys map[PackageID]file.Hash depKeys map[PackageID]file.Hash
// key is the hashed key for the package. // key is the hashed key for the package.
// //
@ -912,36 +927,35 @@ type packageHandle struct {
key file.Hash key file.Hash
} }
// clone returns a copy of the receiver with the validated bit set to the // clone returns a shallow copy of the receiver.
// provided value. func (ph *packageHandle) clone() *packageHandle {
func (ph *packageHandle) clone(validated bool) *packageHandle { clone := *ph
copy := *ph return &clone
copy.validated = validated
return &copy
} }
// getPackageHandles gets package handles for all given ids and their // getPackageHandles gets package handles for all given ids and their
// dependencies, recursively. // dependencies, recursively. The resulting [packageHandle] values are fully
// evaluated (their state will be at least validKey).
func (s *Snapshot) getPackageHandles(ctx context.Context, ids []PackageID) (map[PackageID]*packageHandle, error) { func (s *Snapshot) getPackageHandles(ctx context.Context, ids []PackageID) (map[PackageID]*packageHandle, error) {
// perform a two-pass traversal. // perform a two-pass traversal.
// //
// On the first pass, build up a bidirectional graph of handle nodes, and collect leaves. // On the first pass, build up a bidirectional graph of handle nodes, and collect leaves.
// Then build package handles from bottom up. // Then build package handles from bottom up.
s.mu.Lock() // guard s.meta and s.packages below
b := &packageHandleBuilder{ b := &packageHandleBuilder{
s: s, s: s,
transitiveRefs: make(map[typerefs.IndexID]*partialRefs), transitiveRefs: make(map[typerefs.IndexID]*partialRefs),
nodes: make(map[typerefs.IndexID]*handleNode), nodes: make(map[typerefs.IndexID]*handleNode),
} }
meta := s.MetadataGraph()
var leaves []*handleNode var leaves []*handleNode
var makeNode func(*handleNode, PackageID) *handleNode var makeNode func(*handleNode, PackageID) *handleNode
makeNode = func(from *handleNode, id PackageID) *handleNode { makeNode = func(from *handleNode, id PackageID) *handleNode {
idxID := b.s.pkgIndex.IndexID(id) idxID := s.view.pkgIndex.IndexID(id)
n, ok := b.nodes[idxID] n, ok := b.nodes[idxID]
if !ok { if !ok {
mp := s.meta.Packages[id] mp := meta.Packages[id]
if mp == nil { if mp == nil {
panic(fmt.Sprintf("nil metadata for %q", id)) panic(fmt.Sprintf("nil metadata for %q", id))
} }
@ -950,9 +964,6 @@ func (s *Snapshot) getPackageHandles(ctx context.Context, ids []PackageID) (map[
idxID: idxID, idxID: idxID,
unfinishedSuccs: int32(len(mp.DepsByPkgPath)), unfinishedSuccs: int32(len(mp.DepsByPkgPath)),
} }
if entry, hit := b.s.packages.Get(mp.ID); hit {
n.ph = entry
}
if n.unfinishedSuccs == 0 { if n.unfinishedSuccs == 0 {
leaves = append(leaves, n) leaves = append(leaves, n)
} else { } else {
@ -971,12 +982,10 @@ func (s *Snapshot) getPackageHandles(ctx context.Context, ids []PackageID) (map[
} }
for _, id := range ids { for _, id := range ids {
if ctx.Err() != nil { if ctx.Err() != nil {
s.mu.Unlock()
return nil, ctx.Err() return nil, ctx.Err()
} }
makeNode(nil, id) makeNode(nil, id)
} }
s.mu.Unlock()
g, ctx := errgroup.WithContext(ctx) g, ctx := errgroup.WithContext(ctx)
@ -997,15 +1006,16 @@ func (s *Snapshot) getPackageHandles(ctx context.Context, ids []PackageID) (map[
return ctx.Err() return ctx.Err()
} }
b.buildPackageHandle(ctx, n) if err := b.evaluatePackageHandle(ctx, n); err != nil {
return err
}
for _, pred := range n.preds { for _, pred := range n.preds {
if atomic.AddInt32(&pred.unfinishedSuccs, -1) == 0 { if atomic.AddInt32(&pred.unfinishedSuccs, -1) == 0 {
enqueue(pred) enqueue(pred)
} }
} }
return nil
return n.err
}) })
} }
for _, leaf := range leaves { for _, leaf := range leaves {
@ -1047,7 +1057,6 @@ type handleNode struct {
mp *metadata.Package mp *metadata.Package
idxID typerefs.IndexID idxID typerefs.IndexID
ph *packageHandle ph *packageHandle
err error
preds []*handleNode preds []*handleNode
succs map[PackageID]*handleNode succs map[PackageID]*handleNode
unfinishedSuccs int32 unfinishedSuccs int32
@ -1073,7 +1082,7 @@ func (b *packageHandleBuilder) getTransitiveRefs(pkgID PackageID) map[string]*ty
b.transitiveRefsMu.Lock() b.transitiveRefsMu.Lock()
defer b.transitiveRefsMu.Unlock() defer b.transitiveRefsMu.Unlock()
idxID := b.s.pkgIndex.IndexID(pkgID) idxID := b.s.view.pkgIndex.IndexID(pkgID)
trefs, ok := b.transitiveRefs[idxID] trefs, ok := b.transitiveRefs[idxID]
if !ok { if !ok {
trefs = &partialRefs{ trefs = &partialRefs{
@ -1084,12 +1093,12 @@ func (b *packageHandleBuilder) getTransitiveRefs(pkgID PackageID) map[string]*ty
if !trefs.complete { if !trefs.complete {
trefs.complete = true trefs.complete = true
ph := b.nodes[idxID].ph node := b.nodes[idxID]
for name := range ph.refs { for name := range node.ph.refs {
if ('A' <= name[0] && name[0] <= 'Z') || token.IsExported(name) { if ('A' <= name[0] && name[0] <= 'Z') || token.IsExported(name) {
if _, ok := trefs.refs[name]; !ok { if _, ok := trefs.refs[name]; !ok {
pkgs := b.s.pkgIndex.NewSet() pkgs := b.s.view.pkgIndex.NewSet()
for _, sym := range ph.refs[name] { for _, sym := range node.ph.refs[name] {
pkgs.Add(sym.Package) pkgs.Add(sym.Package)
otherSet := b.getOneTransitiveRefLocked(sym) otherSet := b.getOneTransitiveRefLocked(sym)
pkgs.Union(otherSet) pkgs.Union(otherSet)
@ -1140,7 +1149,7 @@ func (b *packageHandleBuilder) getOneTransitiveRefLocked(sym typerefs.Symbol) *t
// point release. // point release.
// //
// TODO(rfindley): in the future, we should turn this into an assertion. // TODO(rfindley): in the future, we should turn this into an assertion.
bug.Reportf("missing reference to package %s", b.s.pkgIndex.PackageID(sym.Package)) bug.Reportf("missing reference to package %s", b.s.view.pkgIndex.PackageID(sym.Package))
return nil return nil
} }
@ -1152,7 +1161,7 @@ func (b *packageHandleBuilder) getOneTransitiveRefLocked(sym typerefs.Symbol) *t
// See the "cycle detected" bug report above. // See the "cycle detected" bug report above.
trefs.refs[sym.Name] = nil trefs.refs[sym.Name] = nil
pkgs := b.s.pkgIndex.NewSet() pkgs := b.s.view.pkgIndex.NewSet()
for _, sym2 := range n.ph.refs[sym.Name] { for _, sym2 := range n.ph.refs[sym.Name] {
pkgs.Add(sym2.Package) pkgs.Add(sym2.Package)
otherSet := b.getOneTransitiveRefLocked(sym2) otherSet := b.getOneTransitiveRefLocked(sym2)
@ -1164,153 +1173,152 @@ func (b *packageHandleBuilder) getOneTransitiveRefLocked(sym typerefs.Symbol) *t
return pkgs return pkgs
} }
// buildPackageHandle gets or builds a package handle for the given id, storing // evaluatePackageHandle recomputes the derived information in the package handle.
// its result in the snapshot.packages map. // On success, the handle's state is validKey.
// //
// buildPackageHandle must only be called from getPackageHandles. // evaluatePackageHandle must only be called from getPackageHandles.
func (b *packageHandleBuilder) buildPackageHandle(ctx context.Context, n *handleNode) { func (b *packageHandleBuilder) evaluatePackageHandle(ctx context.Context, n *handleNode) (err error) {
var prevPH *packageHandle // Initialize n.ph.
if n.ph != nil { var hit bool
// Existing package handle: if it is valid, return it. Otherwise, create a b.s.mu.Lock()
// copy to update. n.ph, hit = b.s.packages.Get(n.mp.ID)
if n.ph.validated { b.s.mu.Unlock()
return
} if hit && n.ph.state >= validKey {
prevPH = n.ph return nil // already valid
// Either prevPH is still valid, or we will update the key and depKeys of
// this copy. In either case, the result will be valid.
n.ph = prevPH.clone(true)
} else { } else {
// We'll need to update the package handle. Since this could happen
// concurrently, make a copy.
if hit {
n.ph = n.ph.clone()
} else {
n.ph = &packageHandle{
mp: n.mp,
state: validMetadata,
}
}
}
defer func() {
if err == nil {
assert(n.ph.state == validKey, "invalid handle")
// Record the now valid key in the snapshot.
// There may be a race, so avoid the write if the recorded handle is
// already valid.
b.s.mu.Lock()
if alt, ok := b.s.packages.Get(n.mp.ID); !ok || alt.state < n.ph.state {
b.s.packages.Set(n.mp.ID, n.ph, nil)
} else {
n.ph = alt
}
b.s.mu.Unlock()
}
}()
// Invariant: n.ph is either
// - a new handle in state validMetadata, or
// - a clone of an existing handle in state validMetadata or validLocalData.
// State transition: validMetadata -> validLocalInputs.
localKeyChanged := false
if n.ph.state < validLocalData {
prevLocalKey := n.ph.localKey // may be zero
// No package handle: read and analyze the package syntax. // No package handle: read and analyze the package syntax.
inputs, err := b.s.typeCheckInputs(ctx, n.mp) inputs, err := b.s.typeCheckInputs(ctx, n.mp)
if err != nil { if err != nil {
n.err = err return err
return
} }
refs, err := b.s.typerefs(ctx, n.mp, inputs.compiledGoFiles) refs, err := b.s.typerefs(ctx, n.mp, inputs.compiledGoFiles)
if err != nil { if err != nil {
n.err = err return err
return
}
n.ph = &packageHandle{
mp: n.mp,
loadDiagnostics: computeLoadDiagnostics(ctx, b.s, n.mp),
localInputs: inputs,
localKey: localPackageKey(inputs),
refs: refs,
validated: true,
} }
n.ph.loadDiagnostics = computeLoadDiagnostics(ctx, b.s, n.mp)
n.ph.localInputs = inputs
n.ph.localKey = localPackageKey(inputs)
n.ph.refs = refs
n.ph.state = validLocalData
localKeyChanged = n.ph.localKey != prevLocalKey
} }
// ph either did not exist, or was invalid. We must re-evaluate deps and key. assert(n.ph.state == validLocalData, "unexpected handle state")
if err := b.evaluatePackageHandle(prevPH, n); err != nil {
n.err = err
return
}
assert(n.ph.validated, "unvalidated handle") // Optimization: if the local package information did not change, nor did any
// of the dependencies, we don't need to re-run the reachability algorithm.
// Ensure the result (or an equivalent) is recorded in the snapshot.
b.s.mu.Lock()
defer b.s.mu.Unlock()
// Check that the metadata has not changed
// (which should invalidate this handle).
// //
// TODO(rfindley): eventually promote this to an assert. // Concretely: suppose A -> B -> C -> D, where '->' means "imports". If I
// TODO(rfindley): move this to after building the package handle graph? // type in a function body of D, I will probably invalidate types in D that C
if b.s.meta.Packages[n.mp.ID] != n.mp { // uses, because positions change, and therefore the package key of C will
bug.Reportf("stale metadata for %s", n.mp.ID) // change. But B probably doesn't reach any types in D, and therefore the
} // package key of B will not change. We still need to re-run the reachability
// algorithm on B to confirm. But if the key of B did not change, we don't
// even need to run the reachability algorithm on A.
if !localKeyChanged &&
n.ph.depKeys != nil && // n.ph was previously evaluated
len(n.ph.depKeys) == len(n.succs) {
// Check the packages map again in case another goroutine got there first. unchanged := true
if alt, ok := b.s.packages.Get(n.mp.ID); ok && alt.validated {
if alt.mp != n.mp {
bug.Reportf("existing package handle does not match for %s", n.mp.ID)
}
n.ph = alt
} else {
b.s.packages.Set(n.mp.ID, n.ph, nil)
}
}
// evaluatePackageHandle validates and/or computes the key of ph, setting key,
// depKeys, and the validated flag on ph.
//
// It uses prevPH to avoid recomputing keys that can't have changed, since
// their depKeys did not change.
//
// See the documentation for packageHandle for more details about packageHandle
// state, and see the documentation for the typerefs package for more details
// about precise reachability analysis.
func (b *packageHandleBuilder) evaluatePackageHandle(prevPH *packageHandle, n *handleNode) error {
// Opt: if no dep keys have changed, we need not re-evaluate the key.
if prevPH != nil {
depsChanged := false
assert(len(prevPH.depKeys) == len(n.succs), "mismatching dep count")
for id, succ := range n.succs { for id, succ := range n.succs {
oldKey, ok := prevPH.depKeys[id] oldKey, ok := n.ph.depKeys[id]
assert(ok, "missing dep") assert(ok, "missing dep")
if oldKey != succ.ph.key { if oldKey != succ.ph.key {
depsChanged = true unchanged = false
break break
} }
} }
if !depsChanged { if unchanged {
return nil // key cannot have changed n.ph.state = validKey
return nil
} }
} }
// Deps have changed, so we must re-evaluate the key. // State transition: validLocalInputs -> validKey
//
// If we get here, it must be the case that deps have changed, so we must
// run the reachability algorithm.
n.ph.depKeys = make(map[PackageID]file.Hash) n.ph.depKeys = make(map[PackageID]file.Hash)
// See the typerefs package: the reachable set of packages is defined to be // See the typerefs package: the reachable set of packages is defined to be
// the set of packages containing syntax that is reachable through the // the set of packages containing syntax that is reachable through the
// exported symbols in the dependencies of n.ph. // exported symbols in the dependencies of n.ph.
reachable := b.s.pkgIndex.NewSet() reachable := b.s.view.pkgIndex.NewSet()
for depID, succ := range n.succs { for depID, succ := range n.succs {
n.ph.depKeys[depID] = succ.ph.key n.ph.depKeys[depID] = succ.ph.key
reachable.Add(succ.idxID) reachable.Add(succ.idxID)
trefs := b.getTransitiveRefs(succ.mp.ID) trefs := b.getTransitiveRefs(succ.mp.ID)
if trefs == nil { assert(trefs != nil, "nil trefs")
// A predecessor failed to build due to e.g. context cancellation.
return fmt.Errorf("missing transitive refs for %s", succ.mp.ID)
}
for _, set := range trefs { for _, set := range trefs {
reachable.Union(set) reachable.Union(set)
} }
} }
// Collect reachable handles. // Collect reachable nodes.
var reachableHandles []*packageHandle var reachableNodes []*handleNode
// In the presence of context cancellation, any package may be missing. // In the presence of context cancellation, any package may be missing.
// We need all dependencies to produce a valid key. // We need all dependencies to produce a valid key.
missingReachablePackage := false
reachable.Elems(func(id typerefs.IndexID) { reachable.Elems(func(id typerefs.IndexID) {
dh := b.nodes[id] dh := b.nodes[id]
if dh == nil { if dh == nil {
missingReachablePackage = true // Previous code reported an error (not a bug) here.
bug.Reportf("missing reachable node for %q", id)
} else { } else {
assert(dh.ph.validated, "unvalidated dependency") reachableNodes = append(reachableNodes, dh)
reachableHandles = append(reachableHandles, dh.ph)
} }
}) })
if missingReachablePackage {
return fmt.Errorf("missing reachable package")
}
// Sort for stability. // Sort for stability.
sort.Slice(reachableHandles, func(i, j int) bool { sort.Slice(reachableNodes, func(i, j int) bool {
return reachableHandles[i].mp.ID < reachableHandles[j].mp.ID return reachableNodes[i].mp.ID < reachableNodes[j].mp.ID
}) })
// Key is the hash of the local key, and the local key of all reachable // Key is the hash of the local key, and the local key of all reachable
// packages. // packages.
depHasher := sha256.New() depHasher := sha256.New()
depHasher.Write(n.ph.localKey[:]) depHasher.Write(n.ph.localKey[:])
for _, rph := range reachableHandles { for _, dh := range reachableNodes {
depHasher.Write(rph.localKey[:]) depHasher.Write(dh.ph.localKey[:])
} }
depHasher.Sum(n.ph.key[:0]) depHasher.Sum(n.ph.key[:0])
n.ph.state = validKey
return nil return nil
} }
@ -1329,7 +1337,7 @@ func (s *Snapshot) typerefs(ctx context.Context, mp *metadata.Package, cgfs []fi
if err != nil { if err != nil {
return nil, err return nil, err
} }
classes := typerefs.Decode(s.pkgIndex, data) classes := typerefs.Decode(s.view.pkgIndex, data)
refs := make(map[string][]typerefs.Symbol) refs := make(map[string][]typerefs.Symbol)
for _, class := range classes { for _, class := range classes {
for _, decl := range class.Decls { for _, decl := range class.Decls {
@ -1419,7 +1427,7 @@ type typeCheckInputs struct {
viewType ViewType viewType ViewType
} }
func (s *Snapshot) typeCheckInputs(ctx context.Context, mp *metadata.Package) (typeCheckInputs, error) { func (s *Snapshot) typeCheckInputs(ctx context.Context, mp *metadata.Package) (*typeCheckInputs, error) {
// Read both lists of files of this package. // Read both lists of files of this package.
// //
// Parallelism is not necessary here as the files will have already been // Parallelism is not necessary here as the files will have already been
@ -1432,11 +1440,11 @@ func (s *Snapshot) typeCheckInputs(ctx context.Context, mp *metadata.Package) (t
// The need should be rare. // The need should be rare.
goFiles, err := readFiles(ctx, s, mp.GoFiles) goFiles, err := readFiles(ctx, s, mp.GoFiles)
if err != nil { if err != nil {
return typeCheckInputs{}, err return nil, err
} }
compiledGoFiles, err := readFiles(ctx, s, mp.CompiledGoFiles) compiledGoFiles, err := readFiles(ctx, s, mp.CompiledGoFiles)
if err != nil { if err != nil {
return typeCheckInputs{}, err return nil, err
} }
goVersion := "" goVersion := ""
@ -1444,7 +1452,7 @@ func (s *Snapshot) typeCheckInputs(ctx context.Context, mp *metadata.Package) (t
goVersion = mp.Module.GoVersion goVersion = mp.Module.GoVersion
} }
return typeCheckInputs{ return &typeCheckInputs{
id: mp.ID, id: mp.ID,
pkgPath: mp.PkgPath, pkgPath: mp.PkgPath,
name: mp.Name, name: mp.Name,
@ -1475,7 +1483,7 @@ func readFiles(ctx context.Context, fs file.Source, uris []protocol.DocumentURI)
// localPackageKey returns a key for local inputs into type-checking, excluding // localPackageKey returns a key for local inputs into type-checking, excluding
// dependency information: files, metadata, and configuration. // dependency information: files, metadata, and configuration.
func localPackageKey(inputs typeCheckInputs) file.Hash { func localPackageKey(inputs *typeCheckInputs) file.Hash {
hasher := sha256.New() hasher := sha256.New()
// In principle, a key must be the hash of an // In principle, a key must be the hash of an
@ -1669,7 +1677,7 @@ func (b *typeCheckBatch) checkPackage(ctx context.Context, ph *packageHandle) (*
// e.g. "go1" or "go1.2" or "go1.2.3" // e.g. "go1" or "go1.2" or "go1.2.3"
var goVersionRx = regexp.MustCompile(`^go[1-9][0-9]*(?:\.(0|[1-9][0-9]*)){0,2}$`) var goVersionRx = regexp.MustCompile(`^go[1-9][0-9]*(?:\.(0|[1-9][0-9]*)){0,2}$`)
func (b *typeCheckBatch) typesConfig(ctx context.Context, inputs typeCheckInputs, onError func(e error)) *types.Config { func (b *typeCheckBatch) typesConfig(ctx context.Context, inputs *typeCheckInputs, onError func(e error)) *types.Config {
cfg := &types.Config{ cfg := &types.Config{
Sizes: inputs.sizes, Sizes: inputs.sizes,
Error: onError, Error: onError,
@ -1914,7 +1922,7 @@ func missingPkgError(from PackageID, pkgPath string, viewType ViewType) error {
// sequence to a terminal). // sequence to a terminal).
// //
// Fields in typeCheckInputs may affect the resulting diagnostics. // Fields in typeCheckInputs may affect the resulting diagnostics.
func typeErrorsToDiagnostics(pkg *syntaxPackage, inputs typeCheckInputs, errs []types.Error) []*Diagnostic { func typeErrorsToDiagnostics(pkg *syntaxPackage, inputs *typeCheckInputs, errs []types.Error) []*Diagnostic {
var result []*Diagnostic var result []*Diagnostic
// batch records diagnostics for a set of related types.Errors. // batch records diagnostics for a set of related types.Errors.

2
gopls/internal/cache/session.go поставляемый
Просмотреть файл

@ -230,6 +230,7 @@ func (s *Session) createView(ctx context.Context, def *viewDefinition) (*View, *
initialWorkspaceLoad: make(chan struct{}), initialWorkspaceLoad: make(chan struct{}),
initializationSema: make(chan struct{}, 1), initializationSema: make(chan struct{}, 1),
baseCtx: baseCtx, baseCtx: baseCtx,
pkgIndex: typerefs.NewPackageIndex(),
parseCache: s.parseCache, parseCache: s.parseCache,
ignoreFilter: ignoreFilter, ignoreFilter: ignoreFilter,
fs: s.overlayFS, fs: s.overlayFS,
@ -257,7 +258,6 @@ func (s *Session) createView(ctx context.Context, def *viewDefinition) (*View, *
modTidyHandles: new(persistent.Map[protocol.DocumentURI, *memoize.Promise]), modTidyHandles: new(persistent.Map[protocol.DocumentURI, *memoize.Promise]),
modVulnHandles: new(persistent.Map[protocol.DocumentURI, *memoize.Promise]), modVulnHandles: new(persistent.Map[protocol.DocumentURI, *memoize.Promise]),
modWhyHandles: new(persistent.Map[protocol.DocumentURI, *memoize.Promise]), modWhyHandles: new(persistent.Map[protocol.DocumentURI, *memoize.Promise]),
pkgIndex: typerefs.NewPackageIndex(),
moduleUpgrades: new(persistent.Map[protocol.DocumentURI, map[string]string]), moduleUpgrades: new(persistent.Map[protocol.DocumentURI, map[string]string]),
vulns: new(persistent.Map[protocol.DocumentURI, *vulncheck.Result]), vulns: new(persistent.Map[protocol.DocumentURI, *vulncheck.Result]),
} }

28
gopls/internal/cache/snapshot.go поставляемый
Просмотреть файл

@ -32,7 +32,6 @@ import (
"golang.org/x/tools/gopls/internal/cache/methodsets" "golang.org/x/tools/gopls/internal/cache/methodsets"
"golang.org/x/tools/gopls/internal/cache/parsego" "golang.org/x/tools/gopls/internal/cache/parsego"
"golang.org/x/tools/gopls/internal/cache/testfuncs" "golang.org/x/tools/gopls/internal/cache/testfuncs"
"golang.org/x/tools/gopls/internal/cache/typerefs"
"golang.org/x/tools/gopls/internal/cache/xrefs" "golang.org/x/tools/gopls/internal/cache/xrefs"
"golang.org/x/tools/gopls/internal/file" "golang.org/x/tools/gopls/internal/file"
"golang.org/x/tools/gopls/internal/filecache" "golang.org/x/tools/gopls/internal/filecache"
@ -191,9 +190,6 @@ type Snapshot struct {
importGraphDone chan struct{} // closed when importGraph is set; may be nil importGraphDone chan struct{} // closed when importGraph is set; may be nil
importGraph *importGraph // copied from preceding snapshot and re-evaluated importGraph *importGraph // copied from preceding snapshot and re-evaluated
// pkgIndex is an index of package IDs, for efficient storage of typerefs.
pkgIndex *typerefs.PackageIndex
// moduleUpgrades tracks known upgrades for module paths in each modfile. // moduleUpgrades tracks known upgrades for module paths in each modfile.
// Each modfile has a map of module name to upgrade version. // Each modfile has a map of module name to upgrade version.
moduleUpgrades *persistent.Map[protocol.DocumentURI, map[string]string] moduleUpgrades *persistent.Map[protocol.DocumentURI, map[string]string]
@ -1686,7 +1682,6 @@ func (s *Snapshot) clone(ctx, bgCtx context.Context, changed StateChange, done f
modWhyHandles: cloneWithout(s.modWhyHandles, changedFiles, &needsDiagnosis), modWhyHandles: cloneWithout(s.modWhyHandles, changedFiles, &needsDiagnosis),
modVulnHandles: cloneWithout(s.modVulnHandles, changedFiles, &needsDiagnosis), modVulnHandles: cloneWithout(s.modVulnHandles, changedFiles, &needsDiagnosis),
importGraph: s.importGraph, importGraph: s.importGraph,
pkgIndex: s.pkgIndex,
moduleUpgrades: cloneWith(s.moduleUpgrades, changed.ModuleUpgrades), moduleUpgrades: cloneWith(s.moduleUpgrades, changed.ModuleUpgrades),
vulns: cloneWith(s.vulns, changed.Vulns), vulns: cloneWith(s.vulns, changed.Vulns),
} }
@ -1950,14 +1945,21 @@ func (s *Snapshot) clone(ctx, bgCtx context.Context, changed StateChange, done f
// Invalidated package information. // Invalidated package information.
for id, invalidateMetadata := range idsToInvalidate { for id, invalidateMetadata := range idsToInvalidate {
if _, ok := directIDs[id]; ok || invalidateMetadata { // See the [packageHandle] documentation for more details about this
if result.packages.Delete(id) { // invalidation.
needsDiagnosis = true if ph, ok := result.packages.Get(id); ok {
} needsDiagnosis = true
} else { if invalidateMetadata {
if entry, hit := result.packages.Get(id); hit { result.packages.Delete(id)
needsDiagnosis = true } else {
ph := entry.clone(false) // If the package was just invalidated by a dependency, its local
// inputs are still valid.
ph = ph.clone()
if _, ok := directIDs[id]; ok {
ph.state = validMetadata // local inputs changed
} else {
ph.state = min(ph.state, validLocalData) // a dependency changed
}
result.packages.Set(id, ph, nil) result.packages.Set(id, ph, nil)
} }
} }

4
gopls/internal/cache/view.go поставляемый
Просмотреть файл

@ -27,6 +27,7 @@ import (
"time" "time"
"golang.org/x/tools/gopls/internal/cache/metadata" "golang.org/x/tools/gopls/internal/cache/metadata"
"golang.org/x/tools/gopls/internal/cache/typerefs"
"golang.org/x/tools/gopls/internal/file" "golang.org/x/tools/gopls/internal/file"
"golang.org/x/tools/gopls/internal/protocol" "golang.org/x/tools/gopls/internal/protocol"
"golang.org/x/tools/gopls/internal/settings" "golang.org/x/tools/gopls/internal/settings"
@ -106,6 +107,9 @@ type View struct {
importsState *importsState importsState *importsState
// pkgIndex is an index of package IDs, for efficient storage of typerefs.
pkgIndex *typerefs.PackageIndex
// parseCache holds an LRU cache of recently parsed files. // parseCache holds an LRU cache of recently parsed files.
parseCache *parseCache parseCache *parseCache