diff --git a/common/cache/cache.go b/common/cache/cache.go index 9944b1f6de0..77c51e82afe 100644 --- a/common/cache/cache.go +++ b/common/cache/cache.go @@ -51,8 +51,7 @@ type Cache interface { // Iterator returns the iterator of the cache Iterator() Iterator - // Size returns current size of the Cache, the size definition is implementation of SizeGetter interface - // for the entry size, if the entry does not implement SizeGetter interface, the size is 1 + // Size returns the number of entries currently stored in the Cache Size() int } diff --git a/common/cache/lru.go b/common/cache/lru.go index 35163179f36..c87cefe372f 100644 --- a/common/cache/lru.go +++ b/common/cache/lru.go @@ -33,9 +33,7 @@ import ( var ( // ErrCacheFull is returned if Put fails due to cache being filled with pinned elements - ErrCacheFull = errors.New("cache capacity is fully occupied with pinned elements") - // ErrCacheItemTooLarge is returned if Put fails due to item size being larger than max cache capacity - ErrCacheItemTooLarge = errors.New("cache item size is larger than max cache capacity") + ErrCacheFull = errors.New("Cache capacity is fully occupied with pinned elements") ) // lru is a concurrent fixed size cache that evicts elements in lru order @@ -45,7 +43,6 @@ type ( byAccess *list.List byKey map[interface{}]*list.Element maxSize int - currSize int ttl time.Duration pin bool } @@ -61,7 +58,6 @@ type ( createTime time.Time value interface{} refCount int - size int } ) @@ -87,7 +83,6 @@ func (it *iteratorImpl) Next() Entry { entry = &entryImpl{ key: entry.key, value: entry.value, - size: entry.size, createTime: entry.createTime, } it.prepareNext() @@ -129,10 +124,6 @@ func (entry *entryImpl) Value() interface{} { return entry.value } -func (entry *entryImpl) Size() int { - return entry.size -} - func (entry *entryImpl) CreateTime() time.Time { return entry.createTime } @@ -148,7 +139,6 @@ func New(maxSize int, opts *Options) Cache { byKey: make(map[interface{}]*list.Element, opts.InitialCapacity), ttl: opts.TTL, maxSize: maxSize, - currSize: 0, pin: opts.Pin, } } @@ -249,15 +239,12 @@ func (c *lru) Release(key interface{}) { entry.refCount-- } -// Size returns the current size of the lru, useful if cache is not full. This size is calculated by summing -// the size of all entries in the cache. And the entry size is calculated by the size of the value. -// The size of the value is calculated implementing the Sizeable interface. If the value does not implement -// the Sizeable interface, the size is 1. +// Size returns the number of entries currently in the lru, useful if cache is not full func (c *lru) Size() int { c.mut.Lock() defer c.mut.Unlock() - return c.currSize + return len(c.byKey) } // Put puts a new value associated with a given key, returning the existing value (if present) @@ -266,22 +253,9 @@ func (c *lru) putInternal(key interface{}, value interface{}, allowUpdate bool) if c.maxSize == 0 { return nil, nil } - entrySize := getSize(value) - if entrySize > c.maxSize { - return nil, ErrCacheItemTooLarge - } - c.mut.Lock() defer c.mut.Unlock() - c.currSize += entrySize - c.tryEvictUntilEnoughSpace() - // If there is still not enough space, remove the new entry size from the current size and return an error - if c.currSize > c.maxSize { - c.currSize -= entrySize - return nil, ErrCacheFull - } - elt := c.byKey[key] if elt != nil { entry := elt.Value.(*entryImpl) @@ -308,7 +282,6 @@ func (c *lru) putInternal(key interface{}, value interface{}, allowUpdate bool) entry := &entryImpl{ key: key, value: value, - size: entrySize, } if c.pin { @@ -319,6 +292,13 @@ func (c *lru) putInternal(key interface{}, value interface{}, allowUpdate bool) entry.createTime = time.Now().UTC() } + if len(c.byKey) >= c.maxSize { + c.evictOnceInternal() + } + if len(c.byKey) >= c.maxSize { + return nil, ErrCacheFull + } + element := c.byAccess.PushFront(entry) c.byKey[key] = element return nil, nil @@ -326,17 +306,16 @@ func (c *lru) putInternal(key interface{}, value interface{}, allowUpdate bool) func (c *lru) deleteInternal(element *list.Element) { entry := c.byAccess.Remove(element).(*entryImpl) - c.currSize -= entry.Size() delete(c.byKey, entry.key) } -// tryEvictUntilEnoughSpace try to evict entries until there is enough space for the new entry -func (c *lru) tryEvictUntilEnoughSpace() { +func (c *lru) evictOnceInternal() { element := c.byAccess.Back() - for c.currSize > c.maxSize && element != nil { + for element != nil { entry := element.Value.(*entryImpl) if entry.refCount == 0 { c.deleteInternal(element) + return } // entry.refCount > 0 diff --git a/common/cache/lru_test.go b/common/cache/lru_test.go index dee27c5eb44..053a9b4e4da 100644 --- a/common/cache/lru_test.go +++ b/common/cache/lru_test.go @@ -25,28 +25,16 @@ package cache import ( - "math/rand" "sync" "testing" "time" - "github.com/google/uuid" "github.com/stretchr/testify/assert" ) -type ( - keyType struct { - dummyString string - dummyInt int - } - - testEntryWithCacheSize struct { - cacheSize int - } -) - -func (c *testEntryWithCacheSize) CacheSize() int { - return c.cacheSize +type keyType struct { + dummyString string + dummyInt int } func TestLRU(t *testing.T) { @@ -314,50 +302,3 @@ func TestZeroSizeCache(t *testing.T) { assert.Nil(t, err) assert.Equal(t, 0, cache.Size()) } - -func TestCache_ItemSizeTooLarge(t *testing.T) { - t.Parallel() - - maxTotalBytes := 10 - cache := NewLRU(maxTotalBytes) - - res := cache.Put(uuid.New(), &testEntryWithCacheSize{maxTotalBytes}) - assert.Equal(t, res, nil) - - res, err := cache.PutIfNotExist(uuid.New(), &testEntryWithCacheSize{maxTotalBytes + 1}) - assert.Equal(t, err, ErrCacheItemTooLarge) - assert.Equal(t, res, nil) - -} - -func TestCache_ItemHasCacheSizeDefined(t *testing.T) { - t.Parallel() - - maxTotalBytes := 10 - cache := NewLRU(maxTotalBytes) - - numPuts := rand.Intn(1024) - - startWG := sync.WaitGroup{} - endWG := sync.WaitGroup{} - - startWG.Add(numPuts) - endWG.Add(numPuts) - - go func() { - startWG.Wait() - assert.True(t, cache.Size() < maxTotalBytes) - }() - for i := 0; i < numPuts; i++ { - go func() { - defer endWG.Done() - - startWG.Wait() - key := uuid.New() - cache.Put(key, &testEntryWithCacheSize{rand.Int()}) - }() - startWG.Done() - } - - endWG.Wait() -} diff --git a/common/cache/size_getter.go b/common/cache/size_getter.go deleted file mode 100644 index 6b1d603d4b6..00000000000 --- a/common/cache/size_getter.go +++ /dev/null @@ -1,42 +0,0 @@ -// The MIT License -// -// Copyright (c) 2023 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - -//go:generate mockgen -copyright_file ../../LICENSE -package $GOPACKAGE -source $GOFILE -destination size_getter_mock.go - -package cache - -// SizeGetter is an interface that can be implemented by cache entries to provide their size -type ( - SizeGetter interface { - CacheSize() int - } -) - -func getSize(value interface{}) int { - if v, ok := value.(SizeGetter); ok { - return v.CacheSize() - } - // if the object does not have a CacheSize() method, assume is count limit cache, which size should be 1 - return 1 -} diff --git a/common/cache/size_getter_mock.go b/common/cache/size_getter_mock.go deleted file mode 100644 index bcbc710e13f..00000000000 --- a/common/cache/size_getter_mock.go +++ /dev/null @@ -1,72 +0,0 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - -// Code generated by MockGen. DO NOT EDIT. -// Source: size_getter.go - -// Package cache is a generated GoMock package. -package cache - -import ( - reflect "reflect" - - gomock "github.com/golang/mock/gomock" -) - -// MockSizeGetter is a mock of SizeGetter interface. -type MockSizeGetter struct { - ctrl *gomock.Controller - recorder *MockSizeGetterMockRecorder -} - -// MockSizeGetterMockRecorder is the mock recorder for MockSizeGetter. -type MockSizeGetterMockRecorder struct { - mock *MockSizeGetter -} - -// NewMockSizeGetter creates a new mock instance. -func NewMockSizeGetter(ctrl *gomock.Controller) *MockSizeGetter { - mock := &MockSizeGetter{ctrl: ctrl} - mock.recorder = &MockSizeGetterMockRecorder{mock} - return mock -} - -// EXPECT returns an object that allows the caller to indicate expected use. -func (m *MockSizeGetter) EXPECT() *MockSizeGetterMockRecorder { - return m.recorder -} - -// CacheSize mocks base method. -func (m *MockSizeGetter) CacheSize() int { - m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "CacheSize") - ret0, _ := ret[0].(int) - return ret0 -} - -// CacheSize indicates an expected call of CacheSize. -func (mr *MockSizeGetterMockRecorder) CacheSize() *gomock.Call { - mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "CacheSize", reflect.TypeOf((*MockSizeGetter)(nil).CacheSize)) -} diff --git a/service/history/configs/config.go b/service/history/configs/config.go index 43bba3d129f..34a4b85c278 100644 --- a/service/history/configs/config.go +++ b/service/history/configs/config.go @@ -352,10 +352,9 @@ func NewConfig( HistoryCacheTTL: dc.GetDurationProperty(dynamicconfig.HistoryCacheTTL, time.Hour), HistoryCacheNonUserContextLockTimeout: dc.GetDurationProperty(dynamicconfig.HistoryCacheNonUserContextLockTimeout, 500*time.Millisecond), - EventsCacheInitialSize: dc.GetIntProperty(dynamicconfig.EventsCacheInitialSize, 128*1024), // 128KB - EventsCacheMaxSize: dc.GetIntProperty(dynamicconfig.EventsCacheMaxSize, 512*1024), // 512KB - EventsCacheTTL: dc.GetDurationProperty(dynamicconfig.EventsCacheTTL, time.Hour), - + EventsCacheInitialSize: dc.GetIntProperty(dynamicconfig.EventsCacheInitialSize, 128), + EventsCacheMaxSize: dc.GetIntProperty(dynamicconfig.EventsCacheMaxSize, 512), + EventsCacheTTL: dc.GetDurationProperty(dynamicconfig.EventsCacheTTL, time.Hour), RangeSizeBits: 20, // 20 bits for sequencer, 2^20 sequence number for any range AcquireShardInterval: dc.GetDurationProperty(dynamicconfig.AcquireShardInterval, time.Minute), AcquireShardConcurrency: dc.GetIntProperty(dynamicconfig.AcquireShardConcurrency, 10), diff --git a/service/history/events/cache.go b/service/history/events/cache.go index 685c4282c55..c613fc21c5d 100644 --- a/service/history/events/cache.go +++ b/service/history/events/cache.go @@ -65,10 +65,6 @@ type ( metricsHandler metrics.Handler shardID int32 } - - historyEventCacheItemImpl struct { - event *historypb.HistoryEvent - } ) var ( @@ -124,9 +120,9 @@ func (e *CacheImpl) GetEvent(ctx context.Context, key EventKey, firstEventID int // Test hook for disabling cache if !e.disabled { - eventItem, cacheHit := e.Cache.Get(key).(*historyEventCacheItemImpl) + event, cacheHit := e.Cache.Get(key).(*historypb.HistoryEvent) if cacheHit { - return eventItem.event, nil + return event, nil } } @@ -145,7 +141,7 @@ func (e *CacheImpl) GetEvent(ctx context.Context, key EventKey, firstEventID int // If invalid, return event anyway, but don't store in cache if validKey { - e.put(key, event) + e.Put(key, event) } return event, nil } @@ -159,7 +155,7 @@ func (e *CacheImpl) PutEvent(key EventKey, event *historypb.HistoryEvent) { if !e.validateKey(key) { return } - e.put(key, event) + e.Put(key, event) } func (e *CacheImpl) DeleteEvent(key EventKey) { @@ -214,21 +210,3 @@ func (e *CacheImpl) getHistoryEventFromStore( return nil, errEventNotFoundInBatch } - -func (e *CacheImpl) put(key EventKey, event *historypb.HistoryEvent) interface{} { - return e.Put(key, newHistoryEventCacheItem(event)) -} - -var _ cache.SizeGetter = (*historyEventCacheItemImpl)(nil) - -func newHistoryEventCacheItem( - event *historypb.HistoryEvent, -) *historyEventCacheItemImpl { - return &historyEventCacheItemImpl{ - event: event, - } -} - -func (h *historyEventCacheItemImpl) CacheSize() int { - return h.event.Size() -}