submatview: reduce the getFromView implementation

Remove View.Result error return value, it was always nil, and seems like it will likely always remain nill
since it is simply reading a stored value.

Also replace some cache types with local types.
This commit is contained in:
Daniel Nephin 2021-02-23 14:27:24 -05:00
parent 1d9d7d0aa5
commit 832f1a2847
4 changed files with 41 additions and 60 deletions

View File

@ -139,7 +139,7 @@ func (s *streamingHealthState) Close() error {
} }
func (s *streamingHealthState) Fetch(opts cache.FetchOptions) (cache.FetchResult, error) { func (s *streamingHealthState) Fetch(opts cache.FetchOptions) (cache.FetchResult, error) {
result, err := s.materializer.Fetch(s.done, opts) result, err := s.materializer.getFromView(s.done, opts)
result.State = s result.State = s
return result, err return result, err
} }
@ -274,7 +274,7 @@ func sortCheckServiceNodes(serviceNodes *structs.IndexedCheckServiceNodes) {
} }
// Result returns the structs.IndexedCheckServiceNodes stored by this view. // Result returns the structs.IndexedCheckServiceNodes stored by this view.
func (s *healthView) Result(index uint64) (interface{}, error) { func (s *healthView) Result(index uint64) interface{} {
result := structs.IndexedCheckServiceNodes{ result := structs.IndexedCheckServiceNodes{
Nodes: make(structs.CheckServiceNodes, 0, len(s.state)), Nodes: make(structs.CheckServiceNodes, 0, len(s.state)),
QueryMeta: structs.QueryMeta{ QueryMeta: structs.QueryMeta{
@ -286,7 +286,7 @@ func (s *healthView) Result(index uint64) (interface{}, error) {
} }
sortCheckServiceNodes(&result) sortCheckServiceNodes(&result)
return &result, nil return &result
} }
func (s *healthView) Reset() { func (s *healthView) Reset() {

View File

@ -10,7 +10,6 @@ import (
"google.golang.org/grpc/codes" "google.golang.org/grpc/codes"
"google.golang.org/grpc/status" "google.golang.org/grpc/status"
"github.com/hashicorp/consul/agent/cache"
"github.com/hashicorp/consul/lib/retry" "github.com/hashicorp/consul/lib/retry"
"github.com/hashicorp/consul/proto/pbsubscribe" "github.com/hashicorp/consul/proto/pbsubscribe"
) )
@ -32,8 +31,7 @@ type View interface {
// separately and passed in in case the return type needs an Index field // separately and passed in in case the return type needs an Index field
// populating. This allows implementations to not worry about maintaining // populating. This allows implementations to not worry about maintaining
// indexes seen during Update. // indexes seen during Update.
// TODO: remove error return value. Result(index uint64) interface{}
Result(index uint64) (interface{}, error)
// Reset the view to the zero state, done in preparation for receiving a new // Reset the view to the zero state, done in preparation for receiving a new
// snapshot. // snapshot.
@ -217,80 +215,59 @@ func (m *Materializer) notifyUpdateLocked(err error) {
m.updateCh = make(chan struct{}) m.updateCh = make(chan struct{})
} }
// Fetch the value stored in the View. Fetch blocks until the index of the View type viewResult struct {
// is greater than opts.MinIndex, or the context is cancelled. Index uint64
func (m *Materializer) Fetch(done <-chan struct{}, opts cache.FetchOptions) (cache.FetchResult, error) { Value interface{}
var result cache.FetchResult }
// Get current view Result and index // getFromView blocks until the index of the View is greater than opts.MinIndex,
//or the context is cancelled.
func (m *Materializer) getFromView(ctx context.Context, minIndex uint64) (viewResult, error) {
m.lock.Lock() m.lock.Lock()
index := m.index
val, err := m.view.Result(m.index)
updateCh := m.updateCh
m.lock.Unlock()
if err != nil { result := viewResult{
return result, err Index: m.index,
Value: m.view.Result(m.index),
} }
result.Index = index updateCh := m.updateCh
result.Value = val m.lock.Unlock()
// If our index is > req.Index return right away. If index is zero then we // If our index is > req.Index return right away. If index is zero then we
// haven't loaded a snapshot at all yet which means we should wait for one on // haven't loaded a snapshot at all yet which means we should wait for one on
// the update chan. Note it's opts.MinIndex that the cache is using here the // the update chan. Note it's opts.MinIndex that the cache is using here the
// request min index might be different and from initial user request. // request min index might be different and from initial user request.
if index > 0 && index > opts.MinIndex { if result.Index > 0 && result.Index > minIndex {
return result, nil return result, nil
} }
// Watch for timeout of the Fetch. Note it's opts.Timeout not req.Timeout
// since that is the timeout the client requested from the cache Get while the
// options one is the internal "background refresh" timeout which is what the
// Fetch call should be using.
timeoutCh := time.After(opts.Timeout)
for { for {
select { select {
case <-updateCh: case <-updateCh:
// View updated, return the new result // View updated, return the new result
m.lock.Lock() m.lock.Lock()
result.Index = m.index result.Index = m.index
// Grab the new updateCh in case we need to keep waiting for the next
// update. if m.err != nil {
updateCh = m.updateCh m.lock.Unlock()
fetchErr := m.err return result, m.err
if fetchErr == nil {
// Only generate a new result if there was no error to avoid pointless
// work potentially shuffling the same data around.
result.Value, err = m.view.Result(m.index)
} }
result.Value = m.view.Result(m.index)
// Grab the new updateCh in case we need to keep waiting for the next update.
updateCh = m.updateCh
m.lock.Unlock() m.lock.Unlock()
// If there was a non-transient error return it if result.Index <= minIndex {
if fetchErr != nil {
return result, fetchErr
}
if err != nil {
return result, err
}
// Sanity check the update is actually later than the one the user
// requested.
if result.Index <= opts.MinIndex {
// The result is still older/same as the requested index, continue to // The result is still older/same as the requested index, continue to
// wait for further updates. // wait for further updates.
continue continue
} }
// Return the updated result
return result, nil return result, nil
case <-timeoutCh: case <-ctx.Done():
// Just return whatever we got originally, might still be empty return result, ctx.Err()
return result, nil
case <-done:
return result, context.Canceled
} }
} }
} }

View File

@ -86,18 +86,22 @@ func (s *Store) Get(
key, e := s.getEntry(req) key, e := s.getEntry(req)
defer s.releaseEntry(key) defer s.releaseEntry(key)
// TODO: no longer any need to return cache.FetchResult from Materializer.Fetch ctx, cancel := context.WithTimeout(ctx, info.Timeout)
// TODO: pass context instead of Done chan, also replaces Timeout param defer cancel()
result, err := e.materializer.Fetch(ctx.Done(), cache.FetchOptions{
MinIndex: info.MinIndex, result, err := e.materializer.getFromView(ctx, info.MinIndex)
Timeout: info.Timeout,
}) // TODO: does context.DeadlineExceeded need to be translated into a nil error
// to match the old interface?
return result.Value, cache.ResultMeta{Index: result.Index}, err return result.Value, cache.ResultMeta{Index: result.Index}, err
} }
// Notify the updateCh when there are updates to the entry identified by req. // Notify the updateCh when there are updates to the entry identified by req.
// See agent/cache.Cache.Notify for complete documentation. // See agent/cache.Cache.Notify for complete documentation.
//
// Request.CacheInfo().Timeout is ignored because it is not really relevant in
// this case. Instead set a deadline on the context.
func (s *Store) Notify( func (s *Store) Notify(
ctx context.Context, ctx context.Context,
req Request, req Request,
@ -112,7 +116,7 @@ func (s *Store) Notify(
index := info.MinIndex index := info.MinIndex
for { for {
result, err := e.materializer.Fetch(ctx.Done(), cache.FetchOptions{MinIndex: index}) result, err := e.materializer.getFromView(ctx, index)
switch { switch {
case ctx.Err() != nil: case ctx.Err() != nil:
return return

View File

@ -108,12 +108,12 @@ func (f *fakeView) Update(events []*pbsubscribe.Event) error {
return nil return nil
} }
func (f *fakeView) Result(index uint64) (interface{}, error) { func (f *fakeView) Result(index uint64) interface{} {
srvs := make([]*pbservice.CheckServiceNode, 0, len(f.srvs)) srvs := make([]*pbservice.CheckServiceNode, 0, len(f.srvs))
for _, srv := range f.srvs { for _, srv := range f.srvs {
srvs = append(srvs, srv) srvs = append(srvs, srv)
} }
return fakeResult{srvs: srvs, index: index}, nil return fakeResult{srvs: srvs, index: index}
} }
type fakeResult struct { type fakeResult struct {