mirror of
https://github.com/zitadel/zitadel.git
synced 2025-08-11 18:07:31 +00:00
feat(storage): generic cache interface (#8628)
# Which Problems Are Solved We identified the need of caching. Currently we have a number of places where we use different ways of caching, like go maps or LRU. We might also want shared chaches in the future, like Redis-based or in special SQL tables. # How the Problems Are Solved Define a generic Cache interface which allows different implementations. - A noop implementation is provided and enabled as. - An implementation using go maps is provided - disabled in defaults.yaml - enabled in integration tests - Authz middleware instance objects are cached using the interface. # Additional Changes - Enabled integration test command raceflag - Fix a race condition in the limits integration test client - Fix a number of flaky integration tests. (Because zitadel is super fast now!) 🎸 🚀 # Additional Context Related to https://github.com/zitadel/zitadel/issues/8648
This commit is contained in:
106
internal/cache/cache.go
vendored
Normal file
106
internal/cache/cache.go
vendored
Normal file
@@ -0,0 +1,106 @@
|
||||
// Package cache provides abstraction of cache implementations that can be used by zitadel.
|
||||
package cache
|
||||
|
||||
import (
|
||||
"context"
|
||||
"time"
|
||||
|
||||
"github.com/zitadel/logging"
|
||||
)
|
||||
|
||||
// Cache stores objects with a value of type `V`.
|
||||
// Objects may be referred to by one or more indices.
|
||||
// Implementations may encode the value for storage.
|
||||
// This means non-exported fields may be lost and objects
|
||||
// with function values may fail to encode.
|
||||
// See https://pkg.go.dev/encoding/json#Marshal for example.
|
||||
//
|
||||
// `I` is the type by which indices are identified,
|
||||
// typically an enum for type-safe access.
|
||||
// Indices are defined when calling the constructor of an implementation of this interface.
|
||||
// It is illegal to refer to an idex not defined during construction.
|
||||
//
|
||||
// `K` is the type used as key in each index.
|
||||
// Due to the limitations in type constraints, all indices use the same key type.
|
||||
//
|
||||
// Implementations are free to use stricter type constraints or fixed typing.
|
||||
type Cache[I, K comparable, V Entry[I, K]] interface {
|
||||
// Get an object through specified index.
|
||||
// An [IndexUnknownError] may be returned if the index is unknown.
|
||||
// [ErrCacheMiss] is returned if the key was not found in the index,
|
||||
// or the object is not valid.
|
||||
Get(ctx context.Context, index I, key K) (V, bool)
|
||||
|
||||
// Set an object.
|
||||
// Keys are created on each index based in the [Entry.Keys] method.
|
||||
// If any key maps to an existing object, the object is invalidated,
|
||||
// regardless if the object has other keys defined in the new entry.
|
||||
// This to prevent ghost objects when an entry reduces the amount of keys
|
||||
// for a given index.
|
||||
Set(ctx context.Context, value V)
|
||||
|
||||
// Invalidate an object through specified index.
|
||||
// Implementations may choose to instantly delete the object,
|
||||
// defer until prune or a separate cleanup routine.
|
||||
// Invalidated object are no longer returned from Get.
|
||||
// It is safe to call Invalidate multiple times or on non-existing entries.
|
||||
Invalidate(ctx context.Context, index I, key ...K) error
|
||||
|
||||
// Delete one or more keys from a specific index.
|
||||
// An [IndexUnknownError] may be returned if the index is unknown.
|
||||
// The referred object is not invalidated and may still be accessible though
|
||||
// other indices and keys.
|
||||
// It is safe to call Delete multiple times or on non-existing entries
|
||||
Delete(ctx context.Context, index I, key ...K) error
|
||||
|
||||
// Truncate deletes all cached objects.
|
||||
Truncate(ctx context.Context) error
|
||||
|
||||
// Close the cache. Subsequent calls to the cache are not allowed.
|
||||
Close(ctx context.Context) error
|
||||
}
|
||||
|
||||
// Entry contains a value of type `V` to be cached.
|
||||
//
|
||||
// `I` is the type by which indices are identified,
|
||||
// typically an enum for type-safe access.
|
||||
//
|
||||
// `K` is the type used as key in an index.
|
||||
// Due to the limitations in type constraints, all indices use the same key type.
|
||||
type Entry[I, K comparable] interface {
|
||||
// Keys returns which keys map to the object in a specified index.
|
||||
// May return nil if the index in unknown or when there are no keys.
|
||||
Keys(index I) (key []K)
|
||||
}
|
||||
|
||||
type CachesConfig struct {
|
||||
Connectors struct {
|
||||
Memory MemoryConnectorConfig
|
||||
// SQL database.Config
|
||||
// Redis redis.Config?
|
||||
}
|
||||
Instance *CacheConfig
|
||||
}
|
||||
|
||||
type CacheConfig struct {
|
||||
Connector string
|
||||
|
||||
// Age since an object was added to the cache,
|
||||
// after which the object is considered invalid.
|
||||
// 0 disables max age checks.
|
||||
MaxAge time.Duration
|
||||
|
||||
// Age since last use (Get) of an object,
|
||||
// after which the object is considered invalid.
|
||||
// 0 disables last use age checks.
|
||||
LastUseAge time.Duration
|
||||
|
||||
// Log allows logging of the specific cache.
|
||||
// By default only errors are logged to stdout.
|
||||
Log *logging.Config
|
||||
}
|
||||
|
||||
type MemoryConnectorConfig struct {
|
||||
Enabled bool
|
||||
AutoPrune AutoPruneConfig
|
||||
}
|
29
internal/cache/error.go
vendored
Normal file
29
internal/cache/error.go
vendored
Normal file
@@ -0,0 +1,29 @@
|
||||
package cache
|
||||
|
||||
import (
|
||||
"errors"
|
||||
"fmt"
|
||||
)
|
||||
|
||||
type IndexUnknownError[I comparable] struct {
|
||||
index I
|
||||
}
|
||||
|
||||
func NewIndexUnknownErr[I comparable](index I) error {
|
||||
return IndexUnknownError[I]{index}
|
||||
}
|
||||
|
||||
func (i IndexUnknownError[I]) Error() string {
|
||||
return fmt.Sprintf("index %v unknown", i.index)
|
||||
}
|
||||
|
||||
func (a IndexUnknownError[I]) Is(err error) bool {
|
||||
if b, ok := err.(IndexUnknownError[I]); ok {
|
||||
return a.index == b.index
|
||||
}
|
||||
return false
|
||||
}
|
||||
|
||||
var (
|
||||
ErrCacheMiss = errors.New("cache miss")
|
||||
)
|
204
internal/cache/gomap/gomap.go
vendored
Normal file
204
internal/cache/gomap/gomap.go
vendored
Normal file
@@ -0,0 +1,204 @@
|
||||
package gomap
|
||||
|
||||
import (
|
||||
"context"
|
||||
"errors"
|
||||
"log/slog"
|
||||
"maps"
|
||||
"os"
|
||||
"sync"
|
||||
"sync/atomic"
|
||||
"time"
|
||||
|
||||
"github.com/zitadel/zitadel/internal/cache"
|
||||
)
|
||||
|
||||
type mapCache[I, K comparable, V cache.Entry[I, K]] struct {
|
||||
config *cache.CacheConfig
|
||||
indexMap map[I]*index[K, V]
|
||||
logger *slog.Logger
|
||||
}
|
||||
|
||||
// NewCache returns an in-memory Cache implementation based on the builtin go map type.
|
||||
// Object values are stored as-is and there is no encoding or decoding involved.
|
||||
func NewCache[I, K comparable, V cache.Entry[I, K]](background context.Context, indices []I, config cache.CacheConfig) cache.PrunerCache[I, K, V] {
|
||||
m := &mapCache[I, K, V]{
|
||||
config: &config,
|
||||
indexMap: make(map[I]*index[K, V], len(indices)),
|
||||
logger: slog.New(slog.NewTextHandler(os.Stderr, &slog.HandlerOptions{
|
||||
AddSource: true,
|
||||
Level: slog.LevelError,
|
||||
})),
|
||||
}
|
||||
if config.Log != nil {
|
||||
m.logger = config.Log.Slog()
|
||||
}
|
||||
m.logger.InfoContext(background, "map cache logging enabled")
|
||||
|
||||
for _, name := range indices {
|
||||
m.indexMap[name] = &index[K, V]{
|
||||
config: m.config,
|
||||
entries: make(map[K]*entry[V]),
|
||||
}
|
||||
}
|
||||
return m
|
||||
}
|
||||
|
||||
func (c *mapCache[I, K, V]) Get(ctx context.Context, index I, key K) (value V, ok bool) {
|
||||
i, ok := c.indexMap[index]
|
||||
if !ok {
|
||||
c.logger.ErrorContext(ctx, "map cache get", "err", cache.NewIndexUnknownErr(index), "index", index, "key", key)
|
||||
return value, false
|
||||
}
|
||||
entry, err := i.Get(key)
|
||||
if err == nil {
|
||||
c.logger.DebugContext(ctx, "map cache get", "index", index, "key", key)
|
||||
return entry.value, true
|
||||
}
|
||||
if errors.Is(err, cache.ErrCacheMiss) {
|
||||
c.logger.InfoContext(ctx, "map cache get", "err", err, "index", index, "key", key)
|
||||
return value, false
|
||||
}
|
||||
c.logger.ErrorContext(ctx, "map cache get", "err", cache.NewIndexUnknownErr(index), "index", index, "key", key)
|
||||
return value, false
|
||||
}
|
||||
|
||||
func (c *mapCache[I, K, V]) Set(ctx context.Context, value V) {
|
||||
now := time.Now()
|
||||
entry := &entry[V]{
|
||||
value: value,
|
||||
created: now,
|
||||
}
|
||||
entry.lastUse.Store(now.UnixMicro())
|
||||
|
||||
for name, i := range c.indexMap {
|
||||
keys := value.Keys(name)
|
||||
i.Set(keys, entry)
|
||||
c.logger.DebugContext(ctx, "map cache set", "index", name, "keys", keys)
|
||||
}
|
||||
}
|
||||
|
||||
func (c *mapCache[I, K, V]) Invalidate(ctx context.Context, index I, keys ...K) error {
|
||||
i, ok := c.indexMap[index]
|
||||
if !ok {
|
||||
return cache.NewIndexUnknownErr(index)
|
||||
}
|
||||
i.Invalidate(keys)
|
||||
c.logger.DebugContext(ctx, "map cache invalidate", "index", index, "keys", keys)
|
||||
return nil
|
||||
}
|
||||
|
||||
func (c *mapCache[I, K, V]) Delete(ctx context.Context, index I, keys ...K) error {
|
||||
i, ok := c.indexMap[index]
|
||||
if !ok {
|
||||
return cache.NewIndexUnknownErr(index)
|
||||
}
|
||||
i.Delete(keys)
|
||||
c.logger.DebugContext(ctx, "map cache delete", "index", index, "keys", keys)
|
||||
return nil
|
||||
}
|
||||
|
||||
func (c *mapCache[I, K, V]) Prune(ctx context.Context) error {
|
||||
for name, index := range c.indexMap {
|
||||
index.Prune()
|
||||
c.logger.DebugContext(ctx, "map cache prune", "index", name)
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (c *mapCache[I, K, V]) Truncate(ctx context.Context) error {
|
||||
for name, index := range c.indexMap {
|
||||
index.Truncate()
|
||||
c.logger.DebugContext(ctx, "map cache clear", "index", name)
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (c *mapCache[I, K, V]) Close(ctx context.Context) error {
|
||||
return ctx.Err()
|
||||
}
|
||||
|
||||
type index[K comparable, V any] struct {
|
||||
mutex sync.RWMutex
|
||||
config *cache.CacheConfig
|
||||
entries map[K]*entry[V]
|
||||
}
|
||||
|
||||
func (i *index[K, V]) Get(key K) (*entry[V], error) {
|
||||
i.mutex.RLock()
|
||||
entry, ok := i.entries[key]
|
||||
i.mutex.RUnlock()
|
||||
if ok && entry.isValid(i.config) {
|
||||
return entry, nil
|
||||
}
|
||||
return nil, cache.ErrCacheMiss
|
||||
}
|
||||
|
||||
func (c *index[K, V]) Set(keys []K, entry *entry[V]) {
|
||||
c.mutex.Lock()
|
||||
for _, key := range keys {
|
||||
c.entries[key] = entry
|
||||
}
|
||||
c.mutex.Unlock()
|
||||
}
|
||||
|
||||
func (i *index[K, V]) Invalidate(keys []K) {
|
||||
i.mutex.RLock()
|
||||
for _, key := range keys {
|
||||
if entry, ok := i.entries[key]; ok {
|
||||
entry.invalid.Store(true)
|
||||
}
|
||||
}
|
||||
i.mutex.RUnlock()
|
||||
}
|
||||
|
||||
func (c *index[K, V]) Delete(keys []K) {
|
||||
c.mutex.Lock()
|
||||
for _, key := range keys {
|
||||
delete(c.entries, key)
|
||||
}
|
||||
c.mutex.Unlock()
|
||||
}
|
||||
|
||||
func (c *index[K, V]) Prune() {
|
||||
c.mutex.Lock()
|
||||
maps.DeleteFunc(c.entries, func(_ K, entry *entry[V]) bool {
|
||||
return !entry.isValid(c.config)
|
||||
})
|
||||
c.mutex.Unlock()
|
||||
}
|
||||
|
||||
func (c *index[K, V]) Truncate() {
|
||||
c.mutex.Lock()
|
||||
c.entries = make(map[K]*entry[V])
|
||||
c.mutex.Unlock()
|
||||
}
|
||||
|
||||
type entry[V any] struct {
|
||||
value V
|
||||
created time.Time
|
||||
invalid atomic.Bool
|
||||
lastUse atomic.Int64 // UnixMicro time
|
||||
}
|
||||
|
||||
func (e *entry[V]) isValid(c *cache.CacheConfig) bool {
|
||||
if e.invalid.Load() {
|
||||
return false
|
||||
}
|
||||
now := time.Now()
|
||||
if c.MaxAge > 0 {
|
||||
if e.created.Add(c.MaxAge).Before(now) {
|
||||
e.invalid.Store(true)
|
||||
return false
|
||||
}
|
||||
}
|
||||
if c.LastUseAge > 0 {
|
||||
lastUse := e.lastUse.Load()
|
||||
if time.UnixMicro(lastUse).Add(c.LastUseAge).Before(now) {
|
||||
e.invalid.Store(true)
|
||||
return false
|
||||
}
|
||||
e.lastUse.CompareAndSwap(lastUse, now.UnixMicro())
|
||||
}
|
||||
return true
|
||||
}
|
334
internal/cache/gomap/gomap_test.go
vendored
Normal file
334
internal/cache/gomap/gomap_test.go
vendored
Normal file
@@ -0,0 +1,334 @@
|
||||
package gomap
|
||||
|
||||
import (
|
||||
"context"
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
"github.com/stretchr/testify/assert"
|
||||
"github.com/stretchr/testify/require"
|
||||
"github.com/zitadel/logging"
|
||||
|
||||
"github.com/zitadel/zitadel/internal/cache"
|
||||
)
|
||||
|
||||
type testIndex int
|
||||
|
||||
const (
|
||||
testIndexID testIndex = iota
|
||||
testIndexName
|
||||
)
|
||||
|
||||
var testIndices = []testIndex{
|
||||
testIndexID,
|
||||
testIndexName,
|
||||
}
|
||||
|
||||
type testObject struct {
|
||||
id string
|
||||
names []string
|
||||
}
|
||||
|
||||
func (o *testObject) Keys(index testIndex) []string {
|
||||
switch index {
|
||||
case testIndexID:
|
||||
return []string{o.id}
|
||||
case testIndexName:
|
||||
return o.names
|
||||
default:
|
||||
return nil
|
||||
}
|
||||
}
|
||||
|
||||
func Test_mapCache_Get(t *testing.T) {
|
||||
c := NewCache[testIndex, string, *testObject](context.Background(), testIndices, cache.CacheConfig{
|
||||
MaxAge: time.Second,
|
||||
LastUseAge: time.Second / 4,
|
||||
Log: &logging.Config{
|
||||
Level: "debug",
|
||||
AddSource: true,
|
||||
},
|
||||
})
|
||||
defer c.Close(context.Background())
|
||||
obj := &testObject{
|
||||
id: "id",
|
||||
names: []string{"foo", "bar"},
|
||||
}
|
||||
c.Set(context.Background(), obj)
|
||||
|
||||
type args struct {
|
||||
index testIndex
|
||||
key string
|
||||
}
|
||||
tests := []struct {
|
||||
name string
|
||||
args args
|
||||
want *testObject
|
||||
wantOk bool
|
||||
}{
|
||||
{
|
||||
name: "ok",
|
||||
args: args{
|
||||
index: testIndexID,
|
||||
key: "id",
|
||||
},
|
||||
want: obj,
|
||||
wantOk: true,
|
||||
},
|
||||
{
|
||||
name: "miss",
|
||||
args: args{
|
||||
index: testIndexID,
|
||||
key: "spanac",
|
||||
},
|
||||
want: nil,
|
||||
wantOk: false,
|
||||
},
|
||||
{
|
||||
name: "unknown index",
|
||||
args: args{
|
||||
index: 99,
|
||||
key: "id",
|
||||
},
|
||||
want: nil,
|
||||
wantOk: false,
|
||||
},
|
||||
}
|
||||
for _, tt := range tests {
|
||||
t.Run(tt.name, func(t *testing.T) {
|
||||
got, ok := c.Get(context.Background(), tt.args.index, tt.args.key)
|
||||
assert.Equal(t, tt.want, got)
|
||||
assert.Equal(t, tt.wantOk, ok)
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
func Test_mapCache_Invalidate(t *testing.T) {
|
||||
c := NewCache[testIndex, string, *testObject](context.Background(), testIndices, cache.CacheConfig{
|
||||
MaxAge: time.Second,
|
||||
LastUseAge: time.Second / 4,
|
||||
Log: &logging.Config{
|
||||
Level: "debug",
|
||||
AddSource: true,
|
||||
},
|
||||
})
|
||||
defer c.Close(context.Background())
|
||||
obj := &testObject{
|
||||
id: "id",
|
||||
names: []string{"foo", "bar"},
|
||||
}
|
||||
c.Set(context.Background(), obj)
|
||||
err := c.Invalidate(context.Background(), testIndexName, "bar")
|
||||
require.NoError(t, err)
|
||||
got, ok := c.Get(context.Background(), testIndexID, "id")
|
||||
assert.Nil(t, got)
|
||||
assert.False(t, ok)
|
||||
}
|
||||
|
||||
func Test_mapCache_Delete(t *testing.T) {
|
||||
c := NewCache[testIndex, string, *testObject](context.Background(), testIndices, cache.CacheConfig{
|
||||
MaxAge: time.Second,
|
||||
LastUseAge: time.Second / 4,
|
||||
Log: &logging.Config{
|
||||
Level: "debug",
|
||||
AddSource: true,
|
||||
},
|
||||
})
|
||||
defer c.Close(context.Background())
|
||||
obj := &testObject{
|
||||
id: "id",
|
||||
names: []string{"foo", "bar"},
|
||||
}
|
||||
c.Set(context.Background(), obj)
|
||||
err := c.Delete(context.Background(), testIndexName, "bar")
|
||||
require.NoError(t, err)
|
||||
|
||||
// Shouldn't find object by deleted name
|
||||
got, ok := c.Get(context.Background(), testIndexName, "bar")
|
||||
assert.Nil(t, got)
|
||||
assert.False(t, ok)
|
||||
|
||||
// Should find object by other name
|
||||
got, ok = c.Get(context.Background(), testIndexName, "foo")
|
||||
assert.Equal(t, obj, got)
|
||||
assert.True(t, ok)
|
||||
|
||||
// Should find object by id
|
||||
got, ok = c.Get(context.Background(), testIndexID, "id")
|
||||
assert.Equal(t, obj, got)
|
||||
assert.True(t, ok)
|
||||
}
|
||||
|
||||
func Test_mapCache_Prune(t *testing.T) {
|
||||
c := NewCache[testIndex, string, *testObject](context.Background(), testIndices, cache.CacheConfig{
|
||||
MaxAge: time.Second,
|
||||
LastUseAge: time.Second / 4,
|
||||
Log: &logging.Config{
|
||||
Level: "debug",
|
||||
AddSource: true,
|
||||
},
|
||||
})
|
||||
defer c.Close(context.Background())
|
||||
|
||||
objects := []*testObject{
|
||||
{
|
||||
id: "id1",
|
||||
names: []string{"foo", "bar"},
|
||||
},
|
||||
{
|
||||
id: "id2",
|
||||
names: []string{"hello"},
|
||||
},
|
||||
}
|
||||
for _, obj := range objects {
|
||||
c.Set(context.Background(), obj)
|
||||
}
|
||||
// invalidate one entry
|
||||
err := c.Invalidate(context.Background(), testIndexName, "bar")
|
||||
require.NoError(t, err)
|
||||
|
||||
err = c.(cache.Pruner).Prune(context.Background())
|
||||
require.NoError(t, err)
|
||||
|
||||
// Other object should still be found
|
||||
got, ok := c.Get(context.Background(), testIndexID, "id2")
|
||||
assert.Equal(t, objects[1], got)
|
||||
assert.True(t, ok)
|
||||
}
|
||||
|
||||
func Test_mapCache_Truncate(t *testing.T) {
|
||||
c := NewCache[testIndex, string, *testObject](context.Background(), testIndices, cache.CacheConfig{
|
||||
MaxAge: time.Second,
|
||||
LastUseAge: time.Second / 4,
|
||||
Log: &logging.Config{
|
||||
Level: "debug",
|
||||
AddSource: true,
|
||||
},
|
||||
})
|
||||
defer c.Close(context.Background())
|
||||
objects := []*testObject{
|
||||
{
|
||||
id: "id1",
|
||||
names: []string{"foo", "bar"},
|
||||
},
|
||||
{
|
||||
id: "id2",
|
||||
names: []string{"hello"},
|
||||
},
|
||||
}
|
||||
for _, obj := range objects {
|
||||
c.Set(context.Background(), obj)
|
||||
}
|
||||
|
||||
err := c.Truncate(context.Background())
|
||||
require.NoError(t, err)
|
||||
|
||||
mc := c.(*mapCache[testIndex, string, *testObject])
|
||||
for _, index := range mc.indexMap {
|
||||
index.mutex.RLock()
|
||||
assert.Len(t, index.entries, 0)
|
||||
index.mutex.RUnlock()
|
||||
}
|
||||
}
|
||||
|
||||
func Test_entry_isValid(t *testing.T) {
|
||||
type fields struct {
|
||||
created time.Time
|
||||
invalid bool
|
||||
lastUse time.Time
|
||||
}
|
||||
tests := []struct {
|
||||
name string
|
||||
fields fields
|
||||
config *cache.CacheConfig
|
||||
want bool
|
||||
}{
|
||||
{
|
||||
name: "invalid",
|
||||
fields: fields{
|
||||
created: time.Now(),
|
||||
invalid: true,
|
||||
lastUse: time.Now(),
|
||||
},
|
||||
config: &cache.CacheConfig{
|
||||
MaxAge: time.Minute,
|
||||
LastUseAge: time.Second,
|
||||
},
|
||||
want: false,
|
||||
},
|
||||
{
|
||||
name: "max age exceeded",
|
||||
fields: fields{
|
||||
created: time.Now().Add(-(time.Minute + time.Second)),
|
||||
invalid: false,
|
||||
lastUse: time.Now(),
|
||||
},
|
||||
config: &cache.CacheConfig{
|
||||
MaxAge: time.Minute,
|
||||
LastUseAge: time.Second,
|
||||
},
|
||||
want: false,
|
||||
},
|
||||
{
|
||||
name: "max age disabled",
|
||||
fields: fields{
|
||||
created: time.Now().Add(-(time.Minute + time.Second)),
|
||||
invalid: false,
|
||||
lastUse: time.Now(),
|
||||
},
|
||||
config: &cache.CacheConfig{
|
||||
LastUseAge: time.Second,
|
||||
},
|
||||
want: true,
|
||||
},
|
||||
{
|
||||
name: "last use age exceeded",
|
||||
fields: fields{
|
||||
created: time.Now().Add(-(time.Minute / 2)),
|
||||
invalid: false,
|
||||
lastUse: time.Now().Add(-(time.Second * 2)),
|
||||
},
|
||||
config: &cache.CacheConfig{
|
||||
MaxAge: time.Minute,
|
||||
LastUseAge: time.Second,
|
||||
},
|
||||
want: false,
|
||||
},
|
||||
{
|
||||
name: "last use age disabled",
|
||||
fields: fields{
|
||||
created: time.Now().Add(-(time.Minute / 2)),
|
||||
invalid: false,
|
||||
lastUse: time.Now().Add(-(time.Second * 2)),
|
||||
},
|
||||
config: &cache.CacheConfig{
|
||||
MaxAge: time.Minute,
|
||||
},
|
||||
want: true,
|
||||
},
|
||||
{
|
||||
name: "valid",
|
||||
fields: fields{
|
||||
created: time.Now(),
|
||||
invalid: false,
|
||||
lastUse: time.Now(),
|
||||
},
|
||||
config: &cache.CacheConfig{
|
||||
MaxAge: time.Minute,
|
||||
LastUseAge: time.Second,
|
||||
},
|
||||
want: true,
|
||||
},
|
||||
}
|
||||
for _, tt := range tests {
|
||||
t.Run(tt.name, func(t *testing.T) {
|
||||
e := &entry[any]{
|
||||
created: tt.fields.created,
|
||||
}
|
||||
e.invalid.Store(tt.fields.invalid)
|
||||
e.lastUse.Store(tt.fields.lastUse.UnixMicro())
|
||||
got := e.isValid(tt.config)
|
||||
assert.Equal(t, tt.want, got)
|
||||
})
|
||||
}
|
||||
}
|
22
internal/cache/noop/noop.go
vendored
Normal file
22
internal/cache/noop/noop.go
vendored
Normal file
@@ -0,0 +1,22 @@
|
||||
package noop
|
||||
|
||||
import (
|
||||
"context"
|
||||
|
||||
"github.com/zitadel/zitadel/internal/cache"
|
||||
)
|
||||
|
||||
type noop[I, K comparable, V cache.Entry[I, K]] struct{}
|
||||
|
||||
// NewCache returns a cache that does nothing
|
||||
func NewCache[I, K comparable, V cache.Entry[I, K]]() cache.Cache[I, K, V] {
|
||||
return noop[I, K, V]{}
|
||||
}
|
||||
|
||||
func (noop[I, K, V]) Set(context.Context, V) {}
|
||||
func (noop[I, K, V]) Get(context.Context, I, K) (value V, ok bool) { return }
|
||||
func (noop[I, K, V]) Invalidate(context.Context, I, ...K) (err error) { return }
|
||||
func (noop[I, K, V]) Delete(context.Context, I, ...K) (err error) { return }
|
||||
func (noop[I, K, V]) Prune(context.Context) (err error) { return }
|
||||
func (noop[I, K, V]) Truncate(context.Context) (err error) { return }
|
||||
func (noop[I, K, V]) Close(context.Context) (err error) { return }
|
76
internal/cache/pruner.go
vendored
Normal file
76
internal/cache/pruner.go
vendored
Normal file
@@ -0,0 +1,76 @@
|
||||
package cache
|
||||
|
||||
import (
|
||||
"context"
|
||||
"math/rand"
|
||||
"time"
|
||||
|
||||
"github.com/jonboulle/clockwork"
|
||||
"github.com/zitadel/logging"
|
||||
)
|
||||
|
||||
// Pruner is an optional [Cache] interface.
|
||||
type Pruner interface {
|
||||
// Prune deletes all invalidated or expired objects.
|
||||
Prune(ctx context.Context) error
|
||||
}
|
||||
|
||||
type PrunerCache[I, K comparable, V Entry[I, K]] interface {
|
||||
Cache[I, K, V]
|
||||
Pruner
|
||||
}
|
||||
|
||||
type AutoPruneConfig struct {
|
||||
// Interval at which the cache is automatically pruned.
|
||||
// 0 or lower disables automatic pruning.
|
||||
Interval time.Duration
|
||||
|
||||
// Timeout for an automatic prune.
|
||||
// It is recommended to keep the value shorter than AutoPruneInterval
|
||||
// 0 or lower disables automatic pruning.
|
||||
Timeout time.Duration
|
||||
}
|
||||
|
||||
func (c AutoPruneConfig) StartAutoPrune(background context.Context, pruner Pruner, name string) (close func()) {
|
||||
return c.startAutoPrune(background, pruner, name, clockwork.NewRealClock())
|
||||
}
|
||||
|
||||
func (c *AutoPruneConfig) startAutoPrune(background context.Context, pruner Pruner, name string, clock clockwork.Clock) (close func()) {
|
||||
if c.Interval <= 0 {
|
||||
return func() {}
|
||||
}
|
||||
background, cancel := context.WithCancel(background)
|
||||
// randomize the first interval
|
||||
timer := clock.NewTimer(time.Duration(rand.Int63n(int64(c.Interval))))
|
||||
go c.pruneTimer(background, pruner, name, timer)
|
||||
return cancel
|
||||
}
|
||||
|
||||
func (c *AutoPruneConfig) pruneTimer(background context.Context, pruner Pruner, name string, timer clockwork.Timer) {
|
||||
defer func() {
|
||||
if !timer.Stop() {
|
||||
<-timer.Chan()
|
||||
}
|
||||
}()
|
||||
|
||||
for {
|
||||
select {
|
||||
case <-background.Done():
|
||||
return
|
||||
case <-timer.Chan():
|
||||
timer.Reset(c.Interval)
|
||||
err := c.doPrune(background, pruner)
|
||||
logging.OnError(err).WithField("name", name).Error("cache auto prune")
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func (c *AutoPruneConfig) doPrune(background context.Context, pruner Pruner) error {
|
||||
ctx, cancel := context.WithCancel(background)
|
||||
defer cancel()
|
||||
if c.Timeout > 0 {
|
||||
ctx, cancel = context.WithTimeout(background, c.Timeout)
|
||||
defer cancel()
|
||||
}
|
||||
return pruner.Prune(ctx)
|
||||
}
|
43
internal/cache/pruner_test.go
vendored
Normal file
43
internal/cache/pruner_test.go
vendored
Normal file
@@ -0,0 +1,43 @@
|
||||
package cache
|
||||
|
||||
import (
|
||||
"context"
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
"github.com/jonboulle/clockwork"
|
||||
"github.com/stretchr/testify/assert"
|
||||
)
|
||||
|
||||
type testPruner struct {
|
||||
called chan struct{}
|
||||
}
|
||||
|
||||
func (p *testPruner) Prune(context.Context) error {
|
||||
p.called <- struct{}{}
|
||||
return nil
|
||||
}
|
||||
|
||||
func TestAutoPruneConfig_startAutoPrune(t *testing.T) {
|
||||
c := AutoPruneConfig{
|
||||
Interval: time.Second,
|
||||
Timeout: time.Millisecond,
|
||||
}
|
||||
ctx, cancel := context.WithTimeout(context.Background(), time.Second)
|
||||
defer cancel()
|
||||
|
||||
pruner := testPruner{
|
||||
called: make(chan struct{}),
|
||||
}
|
||||
clock := clockwork.NewFakeClock()
|
||||
close := c.startAutoPrune(ctx, &pruner, "foo", clock)
|
||||
defer close()
|
||||
clock.Advance(time.Second)
|
||||
|
||||
select {
|
||||
case _, ok := <-pruner.called:
|
||||
assert.True(t, ok)
|
||||
case <-ctx.Done():
|
||||
t.Fatal(ctx.Err())
|
||||
}
|
||||
}
|
Reference in New Issue
Block a user