-
Notifications
You must be signed in to change notification settings - Fork 10.3k
cache: Added consistent reads for cache #21428
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
base: main
Are you sure you want to change the base?
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -1,3 +1,5 @@ | ||
| # etcd cache | ||
|
|
||
| Experimental etcd client cache library. | ||
|
|
||
| **Note:** gRPC proxy is not supported. The cache relies on `RequestProgress` RPCs, which the gRPC proxy does not forward. | ||
| Original file line number | Diff line number | Diff line change | ||||||||||||||||||||||||||
|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|
|
|
@@ -23,6 +23,7 @@ import ( | |||||||||||||||||||||||||||
| "time" | ||||||||||||||||||||||||||||
|
|
||||||||||||||||||||||||||||
| pb "go.etcd.io/etcd/api/v3/etcdserverpb" | ||||||||||||||||||||||||||||
| "go.etcd.io/etcd/api/v3/v3rpc/rpctypes" | ||||||||||||||||||||||||||||
| clientv3 "go.etcd.io/etcd/client/v3" | ||||||||||||||||||||||||||||
| ) | ||||||||||||||||||||||||||||
|
|
||||||||||||||||||||||||||||
|
|
@@ -31,24 +32,33 @@ var ( | |||||||||||||||||||||||||||
| ErrUnsupportedRequest = errors.New("cache: unsupported request parameters") | ||||||||||||||||||||||||||||
| // Returned when the requested key or key‑range is invalid (empty or reversed) or lies outside c.prefix. | ||||||||||||||||||||||||||||
| ErrKeyRangeInvalid = errors.New("cache: invalid or out‑of‑range key range") | ||||||||||||||||||||||||||||
| // Returned when the cache timed out waiting for the requested revision | ||||||||||||||||||||||||||||
| ErrCacheTimeout = errors.New("cache: timed out waiting for revision") | ||||||||||||||||||||||||||||
| ) | ||||||||||||||||||||||||||||
|
|
||||||||||||||||||||||||||||
| // Cache buffers a single etcd Watch for a given key‐prefix and fan‑outs local watchers. | ||||||||||||||||||||||||||||
| // | ||||||||||||||||||||||||||||
| // Note: gRPC proxy is not supported. Cache relies on RequestProgress RPCs, | ||||||||||||||||||||||||||||
| // which the gRPC proxy does not forward. | ||||||||||||||||||||||||||||
| type Cache struct { | ||||||||||||||||||||||||||||
| prefix string // prefix is the key-prefix this shard is responsible for ("" = root). | ||||||||||||||||||||||||||||
| cfg Config // immutable runtime configuration | ||||||||||||||||||||||||||||
| watcher clientv3.Watcher | ||||||||||||||||||||||||||||
| kv clientv3.KV | ||||||||||||||||||||||||||||
| demux *demux // demux fans incoming events out to active watchers and manages resync. | ||||||||||||||||||||||||||||
| store *store // last‑observed snapshot | ||||||||||||||||||||||||||||
| ready *ready | ||||||||||||||||||||||||||||
| stop context.CancelFunc | ||||||||||||||||||||||||||||
| waitGroup sync.WaitGroup | ||||||||||||||||||||||||||||
| internalCtx context.Context | ||||||||||||||||||||||||||||
| prefix string // prefix is the key-prefix this shard is responsible for ("" = root). | ||||||||||||||||||||||||||||
| cfg Config // immutable runtime configuration | ||||||||||||||||||||||||||||
| watcher clientv3.Watcher | ||||||||||||||||||||||||||||
| kv clientv3.KV | ||||||||||||||||||||||||||||
| demux *demux // demux fans incoming events out to active watchers and manages resync. | ||||||||||||||||||||||||||||
| store *store // last‑observed snapshot | ||||||||||||||||||||||||||||
| ready *ready | ||||||||||||||||||||||||||||
| stop context.CancelFunc | ||||||||||||||||||||||||||||
| waitGroup sync.WaitGroup | ||||||||||||||||||||||||||||
| internalCtx context.Context | ||||||||||||||||||||||||||||
| progressRequestor progressRequestor | ||||||||||||||||||||||||||||
| } | ||||||||||||||||||||||||||||
|
|
||||||||||||||||||||||||||||
| // New builds a cache shard that watches only the requested prefix. | ||||||||||||||||||||||||||||
| // For the root cache pass "". | ||||||||||||||||||||||||||||
| // | ||||||||||||||||||||||||||||
| // Note: gRPC proxy is not supported. Cache relies on RequestProgress RPCs, | ||||||||||||||||||||||||||||
| // which the gRPC proxy does not forward. | ||||||||||||||||||||||||||||
| func New(client *clientv3.Client, prefix string, opts ...Option) (*Cache, error) { | ||||||||||||||||||||||||||||
| cfg := defaultConfig() | ||||||||||||||||||||||||||||
| for _, opt := range opts { | ||||||||||||||||||||||||||||
|
|
@@ -65,23 +75,28 @@ func New(client *clientv3.Client, prefix string, opts ...Option) (*Cache, error) | |||||||||||||||||||||||||||
| internalCtx, cancel := context.WithCancel(context.Background()) | ||||||||||||||||||||||||||||
|
|
||||||||||||||||||||||||||||
| cache := &Cache{ | ||||||||||||||||||||||||||||
| prefix: prefix, | ||||||||||||||||||||||||||||
| cfg: cfg, | ||||||||||||||||||||||||||||
| watcher: client.Watcher, | ||||||||||||||||||||||||||||
| kv: client.KV, | ||||||||||||||||||||||||||||
| store: newStore(cfg.BTreeDegree, cfg.HistoryWindowSize), | ||||||||||||||||||||||||||||
| ready: newReady(), | ||||||||||||||||||||||||||||
| stop: cancel, | ||||||||||||||||||||||||||||
| internalCtx: internalCtx, | ||||||||||||||||||||||||||||
| prefix: prefix, | ||||||||||||||||||||||||||||
| cfg: cfg, | ||||||||||||||||||||||||||||
| watcher: client.Watcher, | ||||||||||||||||||||||||||||
| kv: client.KV, | ||||||||||||||||||||||||||||
| store: newStore(cfg.BTreeDegree, cfg.HistoryWindowSize), | ||||||||||||||||||||||||||||
| ready: newReady(), | ||||||||||||||||||||||||||||
| stop: cancel, | ||||||||||||||||||||||||||||
| internalCtx: internalCtx, | ||||||||||||||||||||||||||||
| progressRequestor: newConditionalProgressRequestor(client.Watcher, cfg.Progress), | ||||||||||||||||||||||||||||
| } | ||||||||||||||||||||||||||||
|
|
||||||||||||||||||||||||||||
| cache.demux = NewDemux(internalCtx, &cache.waitGroup, cfg.HistoryWindowSize, cfg.ResyncInterval) | ||||||||||||||||||||||||||||
|
|
||||||||||||||||||||||||||||
| cache.waitGroup.Add(1) | ||||||||||||||||||||||||||||
| cache.waitGroup.Add(2) | ||||||||||||||||||||||||||||
| go func() { | ||||||||||||||||||||||||||||
| defer cache.waitGroup.Done() | ||||||||||||||||||||||||||||
| cache.getWatchLoop() | ||||||||||||||||||||||||||||
| }() | ||||||||||||||||||||||||||||
| go func() { | ||||||||||||||||||||||||||||
| defer cache.waitGroup.Done() | ||||||||||||||||||||||||||||
| cache.progressRequestor.run(internalCtx) | ||||||||||||||||||||||||||||
| }() | ||||||||||||||||||||||||||||
|
|
||||||||||||||||||||||||||||
| return cache, nil | ||||||||||||||||||||||||||||
| } | ||||||||||||||||||||||||||||
|
|
@@ -161,6 +176,12 @@ func (c *Cache) Get(ctx context.Context, key string, opts ...clientv3.OpOption) | |||||||||||||||||||||||||||
| endKey := op.RangeBytes() | ||||||||||||||||||||||||||||
| requestedRev := op.Rev() | ||||||||||||||||||||||||||||
|
|
||||||||||||||||||||||||||||
| if !op.IsSerializable() { | ||||||||||||||||||||||||||||
| if err := c.waitTillRevision(ctx, requestedRev); err != nil { | ||||||||||||||||||||||||||||
| return nil, err | ||||||||||||||||||||||||||||
| } | ||||||||||||||||||||||||||||
|
Comment on lines
+180
to
+182
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
Suggested change
|
||||||||||||||||||||||||||||
| } | ||||||||||||||||||||||||||||
|
|
||||||||||||||||||||||||||||
| kvs, latestRev, err := c.store.Get(startKey, endKey, requestedRev) | ||||||||||||||||||||||||||||
| if err != nil { | ||||||||||||||||||||||||||||
| return nil, err | ||||||||||||||||||||||||||||
|
|
@@ -196,6 +217,73 @@ func (c *Cache) WaitForRevision(ctx context.Context, rev int64) error { | |||||||||||||||||||||||||||
| } | ||||||||||||||||||||||||||||
| } | ||||||||||||||||||||||||||||
|
|
||||||||||||||||||||||||||||
| func (c *Cache) serverRevision(ctx context.Context) (int64, error) { | ||||||||||||||||||||||||||||
| key := c.prefix | ||||||||||||||||||||||||||||
| if key == "" { | ||||||||||||||||||||||||||||
| key = "/" | ||||||||||||||||||||||||||||
| } | ||||||||||||||||||||||||||||
| resp, err := c.kv.Get(ctx, key, clientv3.WithLimit(1), clientv3.WithCountOnly()) | ||||||||||||||||||||||||||||
| if err != nil { | ||||||||||||||||||||||||||||
| return 0, err | ||||||||||||||||||||||||||||
| } | ||||||||||||||||||||||||||||
| return resp.Header.Revision, nil | ||||||||||||||||||||||||||||
| } | ||||||||||||||||||||||||||||
|
|
||||||||||||||||||||||||||||
| // linearizableRevision returns the target revision for a consistent read. | ||||||||||||||||||||||||||||
| // If rev is non-zero it validates that the server has reached it; otherwise | ||||||||||||||||||||||||||||
| // it returns the server's latest committed revision. | ||||||||||||||||||||||||||||
| func (c *Cache) linearizableRevision(ctx context.Context, rev int64) (int64, error) { | ||||||||||||||||||||||||||||
| serverRev, err := c.serverRevision(ctx) | ||||||||||||||||||||||||||||
| if err != nil { | ||||||||||||||||||||||||||||
| return 0, err | ||||||||||||||||||||||||||||
| } | ||||||||||||||||||||||||||||
| if rev == 0 { | ||||||||||||||||||||||||||||
| return serverRev, nil | ||||||||||||||||||||||||||||
| } | ||||||||||||||||||||||||||||
| if serverRev < rev { | ||||||||||||||||||||||||||||
| return 0, rpctypes.ErrFutureRev | ||||||||||||||||||||||||||||
| } | ||||||||||||||||||||||||||||
| return rev, nil | ||||||||||||||||||||||||||||
| } | ||||||||||||||||||||||||||||
|
|
||||||||||||||||||||||||||||
| // waitTillRevision blocks until the local cache revision reaches rev, | ||||||||||||||||||||||||||||
|
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Writing comments doest help by itself, usually it's a sign of code not being clean itself. General rule: Comment should explain WHY something is being do e, not WHATis being done. Don't write comments that just explain the function. Rather think how to improve the code to make it obvious |
||||||||||||||||||||||||||||
| // using the server's latest revision as the linearizable target. | ||||||||||||||||||||||||||||
| func (c *Cache) waitTillRevision(ctx context.Context, rev int64) error { | ||||||||||||||||||||||||||||
| if rev != 0 && c.store.LatestRev() >= rev { | ||||||||||||||||||||||||||||
|
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Sorry my suggestion was wrong, we always need to read server Revision.
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Why would that be? Did you got that reason from the failing robustness test? I didn't got a chance to check it, let me check it
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Etcd server always linearizes read and thus returns linearized revision. Cache needs to mirror that behavior to be compatible. |
||||||||||||||||||||||||||||
| return nil | ||||||||||||||||||||||||||||
| } | ||||||||||||||||||||||||||||
|
|
||||||||||||||||||||||||||||
| rev, err := c.linearizableRevision(ctx, rev) | ||||||||||||||||||||||||||||
|
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Please move the function out of |
||||||||||||||||||||||||||||
| if err != nil { | ||||||||||||||||||||||||||||
| return err | ||||||||||||||||||||||||||||
| } | ||||||||||||||||||||||||||||
|
Comment on lines
+251
to
+259
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
Suggested change
|
||||||||||||||||||||||||||||
|
|
||||||||||||||||||||||||||||
| if c.store.LatestRev() >= rev { | ||||||||||||||||||||||||||||
| return nil | ||||||||||||||||||||||||||||
| } | ||||||||||||||||||||||||||||
|
|
||||||||||||||||||||||||||||
| c.progressRequestor.add() | ||||||||||||||||||||||||||||
| defer c.progressRequestor.remove() | ||||||||||||||||||||||||||||
|
|
||||||||||||||||||||||||||||
| ticker := time.NewTicker(revisionPollInterval) | ||||||||||||||||||||||||||||
| defer ticker.Stop() | ||||||||||||||||||||||||||||
| timeout := time.After(c.cfg.WaitTimeout) | ||||||||||||||||||||||||||||
|
|
||||||||||||||||||||||||||||
| // TODO: rewrite from periodic polling to passive notification | ||||||||||||||||||||||||||||
| for { | ||||||||||||||||||||||||||||
serathius marked this conversation as resolved.
Show resolved
Hide resolved
|
||||||||||||||||||||||||||||
| if c.store.LatestRev() >= rev { | ||||||||||||||||||||||||||||
| return nil | ||||||||||||||||||||||||||||
| } | ||||||||||||||||||||||||||||
| select { | ||||||||||||||||||||||||||||
| case <-ticker.C: | ||||||||||||||||||||||||||||
| case <-timeout: | ||||||||||||||||||||||||||||
| return ErrCacheTimeout | ||||||||||||||||||||||||||||
| case <-ctx.Done(): | ||||||||||||||||||||||||||||
| return ctx.Err() | ||||||||||||||||||||||||||||
| } | ||||||||||||||||||||||||||||
| } | ||||||||||||||||||||||||||||
| } | ||||||||||||||||||||||||||||
|
|
||||||||||||||||||||||||||||
| // Close cancels the private context and blocks until all goroutines return. | ||||||||||||||||||||||||||||
| func (c *Cache) Close() { | ||||||||||||||||||||||||||||
| c.stop() | ||||||||||||||||||||||||||||
|
|
@@ -358,9 +446,6 @@ func (c *Cache) validateGet(key string, op clientv3.Op) (KeyPredicate, error) { | |||||||||||||||||||||||||||
| return nil, fmt.Errorf("%w: MinCreateRev(%d) not supported", ErrUnsupportedRequest, op.MinCreateRev()) | ||||||||||||||||||||||||||||
| case op.MaxCreateRev() != 0: | ||||||||||||||||||||||||||||
| return nil, fmt.Errorf("%w: MaxCreateRev(%d) not supported", ErrUnsupportedRequest, op.MaxCreateRev()) | ||||||||||||||||||||||||||||
| // cache now only serves serializable reads of the latest revision (rev == 0). | ||||||||||||||||||||||||||||
| case !op.IsSerializable(): | ||||||||||||||||||||||||||||
| return nil, fmt.Errorf("%w: non-serializable request", ErrUnsupportedRequest) | ||||||||||||||||||||||||||||
| } | ||||||||||||||||||||||||||||
|
|
||||||||||||||||||||||||||||
| startKey := []byte(key) | ||||||||||||||||||||||||||||
|
|
||||||||||||||||||||||||||||
Uh oh!
There was an error while loading. Please reload this page.