fix(memorycache): add a cloneable interface (#8414)

This commit is contained in:
Vibhu Pandey 2025-07-05 19:08:23 +05:30 committed by GitHub
parent 7d5e14abb6
commit 8274ebfe37
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
7 changed files with 290 additions and 317 deletions

View File

@ -11,11 +11,11 @@ import (
"github.com/SigNoz/signoz/pkg/factory"
"github.com/SigNoz/signoz/pkg/types/cachetypes"
"github.com/SigNoz/signoz/pkg/valuer"
go_cache "github.com/patrickmn/go-cache"
gocache "github.com/patrickmn/go-cache"
)
type provider struct {
cc *go_cache.Cache
cc *gocache.Cache
config cache.Config
settings factory.ScopedProviderSettings
}
@ -26,50 +26,75 @@ func NewFactory() factory.ProviderFactory[cache.Cache, cache.Config] {
func New(ctx context.Context, settings factory.ProviderSettings, config cache.Config) (cache.Cache, error) {
scopedProviderSettings := factory.NewScopedProviderSettings(settings, "github.com/SigNoz/signoz/pkg/cache/memorycache")
return &provider{cc: go_cache.New(config.Memory.TTL, config.Memory.CleanupInterval), settings: scopedProviderSettings, config: config}, nil
return &provider{
cc: gocache.New(config.Memory.TTL, config.Memory.CleanupInterval),
settings: scopedProviderSettings,
config: config,
}, nil
}
func (provider *provider) Set(ctx context.Context, orgID valuer.UUID, cacheKey string, data cachetypes.Cacheable, ttl time.Duration) error {
// check if the data being passed is a pointer and is not nil
err := cachetypes.ValidatePointer(data, "inmemory")
err := cachetypes.CheckCacheablePointer(data)
if err != nil {
return err
}
if ttl == 0 {
provider.settings.Logger().WarnContext(ctx, "zero value for TTL found. defaulting to the base TTL", "cache_key", cacheKey, "default_ttl", provider.config.Memory.TTL)
if cloneable, ok := data.(cachetypes.Cloneable); ok {
toCache := cloneable.Clone()
provider.cc.Set(strings.Join([]string{orgID.StringValue(), cacheKey}, "::"), toCache, ttl)
return nil
}
provider.cc.Set(strings.Join([]string{orgID.StringValue(), cacheKey}, "::"), data, ttl)
toCache, err := data.MarshalBinary()
if err != nil {
return err
}
provider.cc.Set(strings.Join([]string{orgID.StringValue(), cacheKey}, "::"), toCache, ttl)
return nil
}
func (provider *provider) Get(_ context.Context, orgID valuer.UUID, cacheKey string, dest cachetypes.Cacheable, allowExpired bool) error {
// check if the destination being passed is a pointer and is not nil
err := cachetypes.ValidatePointer(dest, "inmemory")
err := cachetypes.CheckCacheablePointer(dest)
if err != nil {
return err
}
// check if the destination value is settable
dstv := reflect.ValueOf(dest)
if !dstv.Elem().CanSet() {
return errors.Newf(errors.TypeInvalidInput, errors.CodeInvalidInput, "destination value is not settable, %s", dstv.Elem())
}
data, found := provider.cc.Get(strings.Join([]string{orgID.StringValue(), cacheKey}, "::"))
cachedData, found := provider.cc.Get(strings.Join([]string{orgID.StringValue(), cacheKey}, "::"))
if !found {
return errors.Newf(errors.TypeNotFound, errors.CodeNotFound, "key miss")
}
// check the type compatbility between the src and dest
srcv := reflect.ValueOf(data)
if !srcv.Type().AssignableTo(dstv.Type()) {
return errors.Newf(errors.TypeInvalidInput, errors.CodeInvalidInput, "src type is not assignable to dst type")
if cloneable, ok := cachedData.(cachetypes.Cloneable); ok {
// check if the destination value is settable
dstv := reflect.ValueOf(dest)
if !dstv.Elem().CanSet() {
return errors.Newf(errors.TypeInvalidInput, errors.CodeInvalidInput, "unsettable: (value: \"%s\")", dstv.Elem())
}
fromCache := cloneable.Clone()
// check the type compatbility between the src and dest
srcv := reflect.ValueOf(fromCache)
if !srcv.Type().AssignableTo(dstv.Type()) {
return errors.Newf(errors.TypeInvalidInput, errors.CodeInvalidInput, "unassignable: (src: \"%s\", dst: \"%s\")", srcv.Type().String(), dstv.Type().String())
}
// set the value to from src to dest
dstv.Elem().Set(srcv.Elem())
return nil
}
// set the value to from src to dest
dstv.Elem().Set(srcv.Elem())
return nil
if fromCache, ok := cachedData.([]byte); ok {
if err = dest.UnmarshalBinary(fromCache); err != nil {
return err
}
return nil
}
return errors.NewInternalf(errors.CodeInternal, "unrecognized: (value: \"%s\")", reflect.TypeOf(cachedData).String())
}
func (provider *provider) Delete(_ context.Context, orgID valuer.UUID, cacheKey string) {

View File

@ -3,247 +3,217 @@ package memorycache
import (
"context"
"encoding/json"
"fmt"
"strings"
"sync"
"testing"
"time"
"github.com/SigNoz/signoz/pkg/cache"
"github.com/SigNoz/signoz/pkg/factory/factorytest"
"github.com/SigNoz/signoz/pkg/types/cachetypes"
"github.com/SigNoz/signoz/pkg/valuer"
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/require"
)
// TestNew tests the New function
func TestNew(t *testing.T) {
opts := cache.Memory{
TTL: 10 * time.Second,
CleanupInterval: 10 * time.Second,
}
c, err := New(context.Background(), factorytest.NewSettings(), cache.Config{Provider: "memory", Memory: opts})
require.NoError(t, err)
assert.NotNil(t, c)
assert.NotNil(t, c.(*provider).cc)
}
type CacheableEntity struct {
type CloneableA struct {
Key string
Value int
Expiry time.Duration
}
func (ce CacheableEntity) MarshalBinary() ([]byte, error) {
return json.Marshal(ce)
func (cloneable *CloneableA) Clone() cachetypes.Cacheable {
return &CloneableA{
Key: cloneable.Key,
Value: cloneable.Value,
Expiry: cloneable.Expiry,
}
}
func (ce CacheableEntity) UnmarshalBinary(data []byte) error {
return nil
func (cloneable *CloneableA) MarshalBinary() ([]byte, error) {
return json.Marshal(cloneable)
}
type DCacheableEntity struct {
func (cloneable *CloneableA) UnmarshalBinary(data []byte) error {
return json.Unmarshal(data, cloneable)
}
type CacheableB struct {
Key string
Value int
Expiry time.Duration
}
func (dce DCacheableEntity) MarshalBinary() ([]byte, error) {
return json.Marshal(dce)
func (cacheable *CacheableB) MarshalBinary() ([]byte, error) {
return json.Marshal(cacheable)
}
func (dce DCacheableEntity) UnmarshalBinary(data []byte) error {
return nil
func (cacheable *CacheableB) UnmarshalBinary(data []byte) error {
return json.Unmarshal(data, cacheable)
}
// TestStore tests the Store function
// this should fail because of nil pointer error
func TestStoreWithNilPointer(t *testing.T) {
opts := cache.Memory{
func TestCloneableSetWithNilPointer(t *testing.T) {
cache, err := New(context.Background(), factorytest.NewSettings(), cache.Config{Provider: "memory", Memory: cache.Memory{
TTL: 10 * time.Second,
CleanupInterval: 10 * time.Second,
}
c, err := New(context.Background(), factorytest.NewSettings(), cache.Config{Provider: "memory", Memory: opts})
}})
require.NoError(t, err)
var storeCacheableEntity *CacheableEntity
assert.Error(t, c.Set(context.Background(), valuer.GenerateUUID(), "key", storeCacheableEntity, 10*time.Second))
var cloneable *CloneableA
assert.Error(t, cache.Set(context.Background(), valuer.GenerateUUID(), "key", cloneable, 10*time.Second))
}
// this should fail because of no pointer error
func TestStoreWithStruct(t *testing.T) {
opts := cache.Memory{
func TestCacheableSetWithNilPointer(t *testing.T) {
cache, err := New(context.Background(), factorytest.NewSettings(), cache.Config{Provider: "memory", Memory: cache.Memory{
TTL: 10 * time.Second,
CleanupInterval: 10 * time.Second,
}
c, err := New(context.Background(), factorytest.NewSettings(), cache.Config{Provider: "memory", Memory: opts})
}})
require.NoError(t, err)
var storeCacheableEntity CacheableEntity
assert.Error(t, c.Set(context.Background(), valuer.GenerateUUID(), "key", storeCacheableEntity, 10*time.Second))
var cacheable *CacheableB
assert.Error(t, cache.Set(context.Background(), valuer.GenerateUUID(), "key", cacheable, 10*time.Second))
}
func TestStoreWithNonNilPointer(t *testing.T) {
opts := cache.Memory{
func TestCloneableSetGet(t *testing.T) {
cache, err := New(context.Background(), factorytest.NewSettings(), cache.Config{Provider: "memory", Memory: cache.Memory{
TTL: 10 * time.Second,
CleanupInterval: 10 * time.Second,
}
c, err := New(context.Background(), factorytest.NewSettings(), cache.Config{Provider: "memory", Memory: opts})
}})
require.NoError(t, err)
storeCacheableEntity := &CacheableEntity{
Key: "some-random-key",
Value: 1,
Expiry: time.Microsecond,
}
assert.NoError(t, c.Set(context.Background(), valuer.GenerateUUID(), "key", storeCacheableEntity, 10*time.Second))
}
// TestRetrieve tests the Retrieve function
func TestRetrieveWithNilPointer(t *testing.T) {
opts := cache.Memory{
TTL: 10 * time.Second,
CleanupInterval: 10 * time.Second,
}
c, err := New(context.Background(), factorytest.NewSettings(), cache.Config{Provider: "memory", Memory: opts})
require.NoError(t, err)
storeCacheableEntity := &CacheableEntity{
Key: "some-random-key",
Value: 1,
Expiry: time.Microsecond,
}
orgID := valuer.GenerateUUID()
assert.NoError(t, c.Set(context.Background(), orgID, "key", storeCacheableEntity, 10*time.Second))
cloneable := &CloneableA{
Key: "some-random-key",
Value: 1,
Expiry: time.Microsecond,
}
var retrieveCacheableEntity *CacheableEntity
assert.NoError(t, cache.Set(context.Background(), orgID, "key", cloneable, 10*time.Second))
err = c.Get(context.Background(), orgID, "key", retrieveCacheableEntity, false)
provider := cache.(*provider)
insideCache, found := provider.cc.Get(strings.Join([]string{orgID.StringValue(), "key"}, "::"))
assert.True(t, found)
assert.IsType(t, &CloneableA{}, insideCache)
cached := new(CloneableA)
assert.NoError(t, cache.Get(context.Background(), orgID, "key", cached, false))
assert.Equal(t, cloneable, cached)
// confirm that the cached cloneable is a different pointer
assert.NotSame(t, cloneable, cached)
}
func TestCacheableSetGet(t *testing.T) {
cache, err := New(context.Background(), factorytest.NewSettings(), cache.Config{Provider: "memory", Memory: cache.Memory{
TTL: 10 * time.Second,
CleanupInterval: 10 * time.Second,
}})
require.NoError(t, err)
orgID := valuer.GenerateUUID()
cacheable := &CacheableB{
Key: "some-random-key",
Value: 1,
Expiry: time.Microsecond,
}
assert.NoError(t, cache.Set(context.Background(), orgID, "key", cacheable, 10*time.Second))
provider := cache.(*provider)
insideCache, found := provider.cc.Get(strings.Join([]string{orgID.StringValue(), "key"}, "::"))
assert.True(t, found)
assert.IsType(t, []byte{}, insideCache)
assert.Equal(t, "{\"Key\":\"some-random-key\",\"Value\":1,\"Expiry\":1000}", string(insideCache.([]byte)))
cached := new(CacheableB)
assert.NoError(t, cache.Get(context.Background(), orgID, "key", cached, false))
assert.Equal(t, cacheable, cached)
assert.NotSame(t, cacheable, cached)
}
func TestGetWithNilPointer(t *testing.T) {
cache, err := New(context.Background(), factorytest.NewSettings(), cache.Config{Provider: "memory", Memory: cache.Memory{
TTL: 10 * time.Second,
CleanupInterval: 10 * time.Second,
}})
require.NoError(t, err)
var cloneable *CloneableA
assert.Error(t, cache.Get(context.Background(), valuer.GenerateUUID(), "key", cloneable, false))
}
func TestSetGetWithDifferentTypes(t *testing.T) {
cache, err := New(context.Background(), factorytest.NewSettings(), cache.Config{Provider: "memory", Memory: cache.Memory{
TTL: 10 * time.Second,
CleanupInterval: 10 * time.Second,
}})
require.NoError(t, err)
orgID := valuer.GenerateUUID()
cloneable := &CloneableA{
Key: "some-random-key",
Value: 1,
Expiry: time.Microsecond,
}
assert.NoError(t, cache.Set(context.Background(), orgID, "key", cloneable, 10*time.Second))
cachedCacheable := new(CacheableB)
err = cache.Get(context.Background(), orgID, "key", cachedCacheable, false)
assert.Error(t, err)
}
func TestRetrieveWitNonPointer(t *testing.T) {
opts := cache.Memory{
func TestCloneableConcurrentSetGet(t *testing.T) {
cache, err := New(context.Background(), factorytest.NewSettings(), cache.Config{Provider: "memory", Memory: cache.Memory{
TTL: 10 * time.Second,
CleanupInterval: 10 * time.Second,
}
c, err := New(context.Background(), factorytest.NewSettings(), cache.Config{Provider: "memory", Memory: opts})
}})
require.NoError(t, err)
storeCacheableEntity := &CacheableEntity{
Key: "some-random-key",
Value: 1,
Expiry: time.Microsecond,
}
orgID := valuer.GenerateUUID()
assert.NoError(t, c.Set(context.Background(), orgID, "key", storeCacheableEntity, 10*time.Second))
numGoroutines := 100
done := make(chan bool, numGoroutines*2)
cloneables := make([]*CloneableA, numGoroutines)
mu := sync.Mutex{}
var retrieveCacheableEntity CacheableEntity
for i := 0; i < numGoroutines; i++ {
go func(id int) {
cloneable := &CloneableA{
Key: fmt.Sprintf("key-%d", id),
Value: id,
Expiry: 50 * time.Second,
}
err := cache.Set(context.Background(), orgID, fmt.Sprintf("key-%d", id), cloneable, 10*time.Second)
assert.NoError(t, err)
mu.Lock()
cloneables[id] = cloneable
mu.Unlock()
done <- true
}(i)
}
err = c.Get(context.Background(), orgID, "key", retrieveCacheableEntity, false)
assert.Error(t, err)
}
func TestRetrieveWithDifferentTypes(t *testing.T) {
opts := cache.Memory{
TTL: 10 * time.Second,
CleanupInterval: 10 * time.Second,
}
c, err := New(context.Background(), factorytest.NewSettings(), cache.Config{Provider: "memory", Memory: opts})
require.NoError(t, err)
orgID := valuer.GenerateUUID()
storeCacheableEntity := &CacheableEntity{
Key: "some-random-key",
Value: 1,
Expiry: time.Microsecond,
}
assert.NoError(t, c.Set(context.Background(), orgID, "key", storeCacheableEntity, 10*time.Second))
retrieveCacheableEntity := new(DCacheableEntity)
err = c.Get(context.Background(), orgID, "key", retrieveCacheableEntity, false)
assert.Error(t, err)
}
func TestRetrieveWithSameTypes(t *testing.T) {
opts := cache.Memory{
TTL: 10 * time.Second,
CleanupInterval: 10 * time.Second,
}
c, err := New(context.Background(), factorytest.NewSettings(), cache.Config{Provider: "memory", Memory: opts})
require.NoError(t, err)
orgID := valuer.GenerateUUID()
storeCacheableEntity := &CacheableEntity{
Key: "some-random-key",
Value: 1,
Expiry: time.Microsecond,
}
assert.NoError(t, c.Set(context.Background(), orgID, "key", storeCacheableEntity, 10*time.Second))
retrieveCacheableEntity := new(CacheableEntity)
err = c.Get(context.Background(), orgID, "key", retrieveCacheableEntity, false)
assert.NoError(t, err)
assert.Equal(t, storeCacheableEntity, retrieveCacheableEntity)
}
// TestRemove tests the Remove function
func TestRemove(t *testing.T) {
opts := cache.Memory{
TTL: 10 * time.Second,
CleanupInterval: 10 * time.Second,
}
c, err := New(context.Background(), factorytest.NewSettings(), cache.Config{Provider: "memory", Memory: opts})
require.NoError(t, err)
storeCacheableEntity := &CacheableEntity{
Key: "some-random-key",
Value: 1,
Expiry: time.Microsecond,
}
retrieveCacheableEntity := new(CacheableEntity)
orgID := valuer.GenerateUUID()
assert.NoError(t, c.Set(context.Background(), orgID, "key", storeCacheableEntity, 10*time.Second))
c.Delete(context.Background(), orgID, "key")
err = c.Get(context.Background(), orgID, "key", retrieveCacheableEntity, false)
assert.Error(t, err)
}
// TestBulkRemove tests the BulkRemove function
func TestBulkRemove(t *testing.T) {
opts := cache.Memory{
TTL: 10 * time.Second,
CleanupInterval: 10 * time.Second,
}
c, err := New(context.Background(), factorytest.NewSettings(), cache.Config{Provider: "memory", Memory: opts})
require.NoError(t, err)
orgID := valuer.GenerateUUID()
storeCacheableEntity := &CacheableEntity{
Key: "some-random-key",
Value: 1,
Expiry: time.Microsecond,
}
retrieveCacheableEntity := new(CacheableEntity)
assert.NoError(t, c.Set(context.Background(), orgID, "key1", storeCacheableEntity, 10*time.Second))
assert.NoError(t, c.Set(context.Background(), orgID, "key2", storeCacheableEntity, 10*time.Second))
c.DeleteMany(context.Background(), orgID, []string{"key1", "key2"})
err = c.Get(context.Background(), orgID, "key1", retrieveCacheableEntity, false)
assert.Error(t, err)
err = c.Get(context.Background(), orgID, "key2", retrieveCacheableEntity, false)
assert.Error(t, err)
}
// TestCache tests the cache
func TestCache(t *testing.T) {
opts := cache.Memory{
TTL: 10 * time.Second,
CleanupInterval: 10 * time.Second,
}
c, err := New(context.Background(), factorytest.NewSettings(), cache.Config{Provider: "memory", Memory: opts})
require.NoError(t, err)
orgID := valuer.GenerateUUID()
storeCacheableEntity := &CacheableEntity{
Key: "some-random-key",
Value: 1,
Expiry: time.Microsecond,
}
retrieveCacheableEntity := new(CacheableEntity)
assert.NoError(t, c.Set(context.Background(), orgID, "key", storeCacheableEntity, 10*time.Second))
err = c.Get(context.Background(), orgID, "key", retrieveCacheableEntity, false)
assert.NoError(t, err)
assert.Equal(t, storeCacheableEntity, retrieveCacheableEntity)
c.Delete(context.Background(), orgID, "key")
for i := 0; i < numGoroutines; i++ {
go func(id int) {
cachedCloneable := new(CloneableA)
err := cache.Get(context.Background(), orgID, fmt.Sprintf("key-%d", id), cachedCloneable, false)
// Some keys might not exist due to concurrent access, which is expected
_ = err
done <- true
}(i)
}
for i := 0; i < numGoroutines*2; i++ {
<-done
}
for i := 0; i < numGoroutines; i++ {
cachedCloneable := new(CloneableA)
assert.NoError(t, cache.Get(context.Background(), orgID, fmt.Sprintf("key-%d", i), cachedCloneable, false))
assert.Equal(t, fmt.Sprintf("key-%d", i), cachedCloneable.Key)
assert.Equal(t, i, cachedCloneable.Value)
// confirm that the cached cacheable is a different pointer
assert.NotSame(t, cachedCloneable, cloneables[i])
}
}

View File

@ -2,14 +2,13 @@ package rediscache
import (
"context"
"errors"
"strings"
"time"
"fmt"
"github.com/SigNoz/signoz/pkg/cache"
errorsV2 "github.com/SigNoz/signoz/pkg/errors"
"github.com/SigNoz/signoz/pkg/errors"
"github.com/SigNoz/signoz/pkg/factory"
"github.com/SigNoz/signoz/pkg/types/cachetypes"
"github.com/SigNoz/signoz/pkg/valuer"
@ -48,10 +47,12 @@ func (c *provider) Get(ctx context.Context, orgID valuer.UUID, cacheKey string,
err := c.client.Get(ctx, strings.Join([]string{orgID.StringValue(), cacheKey}, "::")).Scan(dest)
if err != nil {
if errors.Is(err, redis.Nil) {
return errorsV2.Newf(errorsV2.TypeNotFound, errorsV2.CodeNotFound, "key miss")
return errors.Newf(errors.TypeNotFound, errors.CodeNotFound, "key miss")
}
return err
}
return nil
}

View File

@ -8,114 +8,49 @@ import (
"time"
"github.com/SigNoz/signoz/pkg/factory"
"github.com/SigNoz/signoz/pkg/factory/factorytest"
"github.com/SigNoz/signoz/pkg/instrumentation/instrumentationtest"
"github.com/SigNoz/signoz/pkg/types/cachetypes"
"github.com/SigNoz/signoz/pkg/valuer"
"github.com/go-redis/redismock/v8"
"github.com/stretchr/testify/assert"
)
type CacheableEntity struct {
type CacheableA struct {
Key string
Value int
Expiry time.Duration
}
func (ce *CacheableEntity) MarshalBinary() ([]byte, error) {
return json.Marshal(ce)
func (cacheable *CacheableA) Clone() cachetypes.Cacheable {
return &CacheableA{
Key: cacheable.Key,
Value: cacheable.Value,
Expiry: cacheable.Expiry,
}
}
func (ce *CacheableEntity) UnmarshalBinary(data []byte) error {
return json.Unmarshal(data, ce)
func (cacheable *CacheableA) MarshalBinary() ([]byte, error) {
return json.Marshal(cacheable)
}
func (cacheable *CacheableA) UnmarshalBinary(data []byte) error {
return json.Unmarshal(data, cacheable)
}
func TestSet(t *testing.T) {
db, mock := redismock.NewClientMock()
cache := &provider{client: db, settings: factory.NewScopedProviderSettings(factorytest.NewSettings(), "github.com/SigNoz/signoz/pkg/cache/rediscache")}
storeCacheableEntity := &CacheableEntity{
providerSettings := instrumentationtest.New().ToProviderSettings()
cache := &provider{client: db, settings: factory.NewScopedProviderSettings(providerSettings, "github.com/SigNoz/signoz/pkg/cache/rediscache")}
cacheable := &CacheableA{
Key: "some-random-key",
Value: 1,
Expiry: time.Microsecond,
}
orgID := valuer.GenerateUUID()
mock.ExpectSet(strings.Join([]string{orgID.StringValue(), "key"}, "::"), storeCacheableEntity, 10*time.Second).RedisNil()
_ = cache.Set(context.Background(), orgID, "key", storeCacheableEntity, 10*time.Second)
mock.ExpectSet(strings.Join([]string{orgID.StringValue(), "key"}, "::"), cacheable, 10*time.Second).SetVal("ok")
if err := mock.ExpectationsWereMet(); err != nil {
t.Errorf("there were unfulfilled expectations: %s", err)
}
}
func TestGet(t *testing.T) {
db, mock := redismock.NewClientMock()
cache := &provider{client: db, settings: factory.NewScopedProviderSettings(factorytest.NewSettings(), "github.com/SigNoz/signoz/pkg/cache/rediscache")}
storeCacheableEntity := &CacheableEntity{
Key: "some-random-key",
Value: 1,
Expiry: time.Microsecond,
}
retrieveCacheableEntity := new(CacheableEntity)
orgID := valuer.GenerateUUID()
mock.ExpectSet(strings.Join([]string{orgID.StringValue(), "key"}, "::"), storeCacheableEntity, 10*time.Second).RedisNil()
_ = cache.Set(context.Background(), orgID, "key", storeCacheableEntity, 10*time.Second)
data, err := storeCacheableEntity.MarshalBinary()
assert.NoError(t, err)
mock.ExpectGet(strings.Join([]string{orgID.StringValue(), "key"}, "::")).SetVal(string(data))
err = cache.Get(context.Background(), orgID, "key", retrieveCacheableEntity, false)
if err != nil {
t.Errorf("unexpected error: %s", err)
}
assert.Equal(t, storeCacheableEntity, retrieveCacheableEntity)
if err := mock.ExpectationsWereMet(); err != nil {
t.Errorf("there were unfulfilled expectations: %s", err)
}
}
func TestDelete(t *testing.T) {
db, mock := redismock.NewClientMock()
cache := &provider{client: db, settings: factory.NewScopedProviderSettings(factorytest.NewSettings(), "github.com/SigNoz/signoz/pkg/cache/rediscache")}
storeCacheableEntity := &CacheableEntity{
Key: "some-random-key",
Value: 1,
Expiry: time.Microsecond,
}
orgID := valuer.GenerateUUID()
mock.ExpectSet(strings.Join([]string{orgID.StringValue(), "key"}, "::"), storeCacheableEntity, 10*time.Second).RedisNil()
_ = cache.Set(context.Background(), orgID, "key", storeCacheableEntity, 10*time.Second)
mock.ExpectDel(strings.Join([]string{orgID.StringValue(), "key"}, "::")).RedisNil()
cache.Delete(context.Background(), orgID, "key")
if err := mock.ExpectationsWereMet(); err != nil {
t.Errorf("there were unfulfilled expectations: %s", err)
}
}
func TestDeleteMany(t *testing.T) {
db, mock := redismock.NewClientMock()
cache := &provider{client: db, settings: factory.NewScopedProviderSettings(factorytest.NewSettings(), "github.com/SigNoz/signoz/pkg/cache/rediscache")}
storeCacheableEntity := &CacheableEntity{
Key: "some-random-key",
Value: 1,
Expiry: time.Microsecond,
}
orgID := valuer.GenerateUUID()
mock.ExpectSet(strings.Join([]string{orgID.StringValue(), "key"}, "::"), storeCacheableEntity, 10*time.Second).RedisNil()
_ = cache.Set(context.Background(), orgID, "key", storeCacheableEntity, 10*time.Second)
mock.ExpectSet(strings.Join([]string{orgID.StringValue(), "key2"}, "::"), storeCacheableEntity, 10*time.Second).RedisNil()
_ = cache.Set(context.Background(), orgID, "key2", storeCacheableEntity, 10*time.Second)
mock.ExpectDel(strings.Join([]string{orgID.StringValue(), "key"}, "::"), strings.Join([]string{orgID.StringValue(), "key2"}, "::")).RedisNil()
cache.DeleteMany(context.Background(), orgID, []string{"key", "key2"})
if err := mock.ExpectationsWereMet(); err != nil {
t.Errorf("there were unfulfilled expectations: %s", err)
}
assert.NoError(t, cache.Set(context.Background(), orgID, "key", cacheable, 10*time.Second))
assert.NoError(t, mock.ExpectationsWereMet())
}

View File

@ -1,6 +1,11 @@
package model
import "encoding/json"
import (
"encoding/json"
"maps"
"github.com/SigNoz/signoz/pkg/types/cachetypes"
)
type GetWaterfallSpansForTraceWithMetadataCache struct {
StartTime uint64 `json:"startTime"`
@ -14,6 +19,28 @@ type GetWaterfallSpansForTraceWithMetadataCache struct {
HasMissingSpans bool `json:"hasMissingSpans"`
}
func (c *GetWaterfallSpansForTraceWithMetadataCache) Clone() cachetypes.Cacheable {
copyOfServiceNameToTotalDurationMap := make(map[string]uint64)
maps.Copy(copyOfServiceNameToTotalDurationMap, c.ServiceNameToTotalDurationMap)
copyOfSpanIdToSpanNodeMap := make(map[string]*Span)
maps.Copy(copyOfSpanIdToSpanNodeMap, c.SpanIdToSpanNodeMap)
copyOfTraceRoots := make([]*Span, len(c.TraceRoots))
copy(copyOfTraceRoots, c.TraceRoots)
return &GetWaterfallSpansForTraceWithMetadataCache{
StartTime: c.StartTime,
EndTime: c.EndTime,
DurationNano: c.DurationNano,
TotalSpans: c.TotalSpans,
TotalErrorSpans: c.TotalErrorSpans,
ServiceNameToTotalDurationMap: copyOfServiceNameToTotalDurationMap,
SpanIdToSpanNodeMap: copyOfSpanIdToSpanNodeMap,
TraceRoots: copyOfTraceRoots,
HasMissingSpans: c.HasMissingSpans,
}
}
func (c *GetWaterfallSpansForTraceWithMetadataCache) MarshalBinary() (data []byte, err error) {
return json.Marshal(c)
}
@ -29,6 +56,16 @@ type GetFlamegraphSpansForTraceCache struct {
TraceRoots []*FlamegraphSpan `json:"traceRoots"`
}
func (c *GetFlamegraphSpansForTraceCache) Clone() cachetypes.Cacheable {
return &GetFlamegraphSpansForTraceCache{
StartTime: c.StartTime,
EndTime: c.EndTime,
DurationNano: c.DurationNano,
SelectedSpans: c.SelectedSpans,
TraceRoots: c.TraceRoots,
}
}
func (c *GetFlamegraphSpansForTraceCache) MarshalBinary() (data []byte, err error) {
return json.Marshal(c)
}

View File

@ -2,8 +2,9 @@ package model
import (
"encoding/json"
v3 "github.com/SigNoz/signoz/pkg/query-service/model/v3"
"time"
v3 "github.com/SigNoz/signoz/pkg/query-service/model/v3"
)
type UpdateMetricsMetadata struct {

View File

@ -12,22 +12,26 @@ type Cacheable interface {
encoding.BinaryUnmarshaler
}
func WrapCacheableErrors(rt reflect.Type, caller string) error {
if rt == nil {
return errors.Newf(errors.TypeInvalidInput, errors.CodeInvalidInput, "%s: (nil)", caller)
}
if rt.Kind() != reflect.Pointer {
return errors.Newf(errors.TypeInvalidInput, errors.CodeInvalidInput, "%s: (non-pointer \"%s\")", caller, rt.String())
}
return errors.Newf(errors.TypeInvalidInput, errors.CodeInvalidInput, "%s: (nil \"%s\")", caller, rt.String())
type Cloneable interface {
// Creates a deep copy of the Cacheable. This method is useful for memory caches to avoid the need for serialization/deserialization. It also prevents
// race conditions in the memory cache.
Clone() Cacheable
}
func ValidatePointer(dest any, caller string) error {
func CheckCacheablePointer(dest any) error {
rv := reflect.ValueOf(dest)
if rv.Kind() != reflect.Pointer || rv.IsNil() {
return WrapCacheableErrors(reflect.TypeOf(dest), caller)
rt := reflect.TypeOf(dest)
if rt == nil {
return errors.Newf(errors.TypeInvalidInput, errors.CodeInvalidInput, "cacheable: (nil)")
}
if rt.Kind() != reflect.Pointer {
return errors.Newf(errors.TypeInvalidInput, errors.CodeInvalidInput, "cacheable: (non-pointer \"%s\")", rt.String())
}
return errors.Newf(errors.TypeInvalidInput, errors.CodeInvalidInput, "cacheable: (nil \"%s\")", rt.String())
}
return nil
}