mirror of
https://github.com/minio/minio.git
synced 2024-12-24 06:05:55 -05:00
Deprecate and remove in-memory object caching (#5481)
in-memory caching cannot be cleanly implemented without the access to GC which Go doesn't naturally provide. At times we have seen that object caching is more of an hindrance rather than a boon for our use cases. Removing it completely from our implementation related to #5160 and #5182
This commit is contained in:
parent
1ebbc2ce88
commit
0c880bb852
@ -76,9 +76,6 @@ func handleCommonEnvVars() {
|
||||
globalProfiler = startProfiler(profiler)
|
||||
}
|
||||
|
||||
// Check if object cache is disabled.
|
||||
globalXLObjCacheDisabled = strings.EqualFold(os.Getenv("_MINIO_CACHE"), "off")
|
||||
|
||||
accessKey := os.Getenv("MINIO_ACCESS_KEY")
|
||||
secretKey := os.Getenv("MINIO_SECRET_KEY")
|
||||
if accessKey != "" && secretKey != "" {
|
||||
|
@ -143,9 +143,8 @@ var (
|
||||
// Time when object layer was initialized on start up.
|
||||
globalBootTime time.Time
|
||||
|
||||
globalActiveCred auth.Credentials
|
||||
globalPublicCerts []*x509.Certificate
|
||||
globalXLObjCacheDisabled bool
|
||||
globalActiveCred auth.Credentials
|
||||
globalPublicCerts []*x509.Certificate
|
||||
|
||||
globalIsEnvDomainName bool
|
||||
globalDomainName string // Root domain for virtual host style requests
|
||||
|
@ -106,14 +106,6 @@ func testGetObject(obj ObjectLayer, instanceType string, t TestErrHandler) {
|
||||
// Test case - 5.
|
||||
// Case with invalid object names.
|
||||
{bucketName, "", 0, 0, nil, nil, false, []byte(""), fmt.Errorf("%s", "Object name invalid: "+bucketName+"#")},
|
||||
// Test case - 6.
|
||||
// Valid object and bucket names but non-existent bucket.
|
||||
// {"abc", "def", 0, 0, nil, nil, false, []byte(""), fmt.Errorf("%s", "Bucket not found: abc")},
|
||||
// A custom writer is sent as an argument.
|
||||
// Its designed to return a EOF error after reading `n` bytes, where `n` is the argument when initializing the EOF writer.
|
||||
// This is to simulate the case of cache not filling up completly, since the EOFWriter doesn't allow the write to complete,
|
||||
// the cache gets filled up with partial data. The following up test case will read the object completly, tests the
|
||||
// purging of the cache during the incomplete write.
|
||||
// Test case - 7.
|
||||
{bucketName, objectName, 0, int64(len(bytesData[0].byteData)), buffers[0], NewEOFWriter(buffers[0], 100), false, []byte{}, io.EOF},
|
||||
// Test case with start offset set to 0 and length set to size of the object.
|
||||
@ -374,14 +366,6 @@ func testGetObjectDiskNotFound(obj ObjectLayer, instanceType string, disks []str
|
||||
// Test case - 5.
|
||||
// Case with invalid object names.
|
||||
{bucketName, "", 0, 0, nil, nil, false, []byte(""), fmt.Errorf("%s", "Object name invalid: "+bucketName+"#")},
|
||||
// Test case - 6.
|
||||
// Valid object and bucket names but non-existent bucket.
|
||||
// {"abc", "def", 0, 0, nil, nil, false, []byte(""), fmt.Errorf("%s", "Bucket not found: abc")},
|
||||
// A custom writer is sent as an argument.
|
||||
// Its designed to return a EOF error after reading `n` bytes, where `n` is the argument when initializing the EOF writer.
|
||||
// This is to simulate the case of cache not filling up completly, since the EOFWriter doesn't allow the write to complete,
|
||||
// the cache gets filled up with partial data. The following up test case will read the object completly, tests the
|
||||
// purging of the cache during the incomplete write.
|
||||
// Test case - 7.
|
||||
{bucketName, objectName, 0, int64(len(bytesData[0].byteData)), buffers[0], NewEOFWriter(buffers[0], 100), false, []byte{}, io.EOF},
|
||||
// Test case with start offset set to 0 and length set to size of the object.
|
||||
|
@ -53,42 +53,3 @@ func setMaxResources() (err error) {
|
||||
err = sys.SetMaxMemoryLimit(maxLimit, maxLimit)
|
||||
return err
|
||||
}
|
||||
|
||||
func getMaxCacheSize(curLimit, totalRAM uint64) (cacheSize uint64) {
|
||||
// Return zero if current limit or totalTAM is less than minRAMSize.
|
||||
if curLimit < minRAMSize || totalRAM < minRAMSize {
|
||||
return cacheSize
|
||||
}
|
||||
|
||||
// Return 50% of current rlimit or total RAM as cache size.
|
||||
if curLimit < totalRAM {
|
||||
cacheSize = curLimit / 2
|
||||
} else {
|
||||
cacheSize = totalRAM / 2
|
||||
}
|
||||
|
||||
return cacheSize
|
||||
}
|
||||
|
||||
// GetMaxCacheSize returns maximum cache size based on current RAM size and memory limit.
|
||||
func GetMaxCacheSize() (cacheSize uint64, err error) {
|
||||
// Get max memory limit
|
||||
var curLimit uint64
|
||||
if curLimit, _, err = sys.GetMaxMemoryLimit(); err != nil {
|
||||
return cacheSize, err
|
||||
}
|
||||
|
||||
// Get total RAM.
|
||||
var stats sys.Stats
|
||||
if stats, err = sys.GetStats(); err != nil {
|
||||
return cacheSize, err
|
||||
}
|
||||
|
||||
// In some OS like windows, maxLimit is zero. Set total RAM as maxLimit.
|
||||
if curLimit == 0 {
|
||||
curLimit = stats.TotalRAM
|
||||
}
|
||||
|
||||
cacheSize = getMaxCacheSize(curLimit, stats.TotalRAM)
|
||||
return cacheSize, err
|
||||
}
|
||||
|
@ -1,43 +0,0 @@
|
||||
/*
|
||||
* Minio Cloud Storage, (C) 2017 Minio, Inc.
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package cmd
|
||||
|
||||
import "testing"
|
||||
|
||||
func TestGetMaxCacheSize(t *testing.T) {
|
||||
testCases := []struct {
|
||||
curLimit uint64
|
||||
totalRAM uint64
|
||||
expectedResult uint64
|
||||
}{
|
||||
{uint64(0), uint64(0), uint64(0)},
|
||||
{minRAMSize, uint64(0), uint64(0)},
|
||||
{uint64(0), minRAMSize, uint64(0)},
|
||||
{uint64(18446744073709551615), uint64(8115998720), uint64(0)},
|
||||
{uint64(8115998720), uint64(16115998720), uint64(0)},
|
||||
{minRAMSize, minRAMSize, uint64(12884901888)},
|
||||
{minRAMSize, uint64(16115998720), uint64(0)},
|
||||
{uint64(18446744073709551615), uint64(10115998720), uint64(0)},
|
||||
}
|
||||
|
||||
for i, testCase := range testCases {
|
||||
cacheSize := getMaxCacheSize(testCase.curLimit, testCase.totalRAM)
|
||||
if testCase.expectedResult != cacheSize {
|
||||
t.Fatalf("Test %d, Expected: %v, Got: %v", i+1, testCase.expectedResult, cacheSize)
|
||||
}
|
||||
}
|
||||
}
|
@ -1657,12 +1657,6 @@ func initObjectLayer(endpoints EndpointList) (ObjectLayer, []StorageAPI, error)
|
||||
return nil, nil, err
|
||||
}
|
||||
|
||||
// Disabling the cache for integration tests.
|
||||
// Should use the object layer tests for validating cache.
|
||||
if xl, ok := objLayer.(*xlObjects); ok {
|
||||
xl.objCacheEnabled = false
|
||||
}
|
||||
|
||||
// Success.
|
||||
return objLayer, formattedDisks, nil
|
||||
}
|
||||
|
@ -20,7 +20,6 @@ import (
|
||||
"encoding/hex"
|
||||
"fmt"
|
||||
"io"
|
||||
"io/ioutil"
|
||||
"path"
|
||||
"strings"
|
||||
"sync"
|
||||
@ -1068,19 +1067,6 @@ func (xl xlObjects) CompleteMultipartUpload(bucket string, object string, upload
|
||||
return oi, toObjectErr(rErr, minioMetaMultipartBucket, uploadIDPath)
|
||||
}
|
||||
|
||||
defer func() {
|
||||
if xl.objCacheEnabled {
|
||||
// A new complete multipart upload invalidates any
|
||||
// previously cached object in memory.
|
||||
xl.objCache.Delete(path.Join(bucket, object))
|
||||
|
||||
// Prefetch the object from disk by triggering a fake GetObject call
|
||||
// Unlike a regular single PutObject, multipart PutObject is comes in
|
||||
// stages and it is harder to cache.
|
||||
go xl.GetObject(bucket, object, 0, objectSize, ioutil.Discard, s3MD5)
|
||||
}
|
||||
}()
|
||||
|
||||
if xl.isObject(bucket, object) {
|
||||
// Rename if an object already exists to temporary location.
|
||||
newUniqueID := mustGetUUID()
|
||||
|
@ -27,7 +27,6 @@ import (
|
||||
"github.com/minio/minio/pkg/errors"
|
||||
"github.com/minio/minio/pkg/hash"
|
||||
"github.com/minio/minio/pkg/mimedb"
|
||||
"github.com/minio/minio/pkg/objcache"
|
||||
)
|
||||
|
||||
// list all errors which can be ignored in object operations.
|
||||
@ -289,52 +288,6 @@ func (xl xlObjects) getObject(bucket, object string, startOffset int64, length i
|
||||
return errors.Trace(InvalidRange{startOffset, length, xlMeta.Stat.Size})
|
||||
}
|
||||
|
||||
// Save the writer.
|
||||
mw := writer
|
||||
|
||||
// Object cache enabled block.
|
||||
if xlMeta.Stat.Size > 0 && xl.objCacheEnabled {
|
||||
// Validate if we have previous cache.
|
||||
var cachedBuffer io.ReaderAt
|
||||
cachedBuffer, err = xl.objCache.Open(path.Join(bucket, object), modTime)
|
||||
if err == nil { // Cache hit
|
||||
// Create a new section reader, starting at an offset with length.
|
||||
reader := io.NewSectionReader(cachedBuffer, startOffset, length)
|
||||
|
||||
// Copy the data out.
|
||||
if _, err = io.Copy(writer, reader); err != nil {
|
||||
return errors.Trace(err)
|
||||
}
|
||||
|
||||
// Success.
|
||||
return nil
|
||||
|
||||
} // Cache miss.
|
||||
|
||||
// For unknown error, return and error out.
|
||||
if err != objcache.ErrKeyNotFoundInCache {
|
||||
return errors.Trace(err)
|
||||
} // Cache has not been found, fill the cache.
|
||||
|
||||
// Cache is only set if whole object is being read.
|
||||
if startOffset == 0 && length == xlMeta.Stat.Size {
|
||||
// Proceed to set the cache.
|
||||
var newBuffer io.WriteCloser
|
||||
// Create a new entry in memory of length.
|
||||
newBuffer, err = xl.objCache.Create(path.Join(bucket, object), length)
|
||||
if err == nil {
|
||||
// Create a multi writer to write to both memory and client response.
|
||||
mw = io.MultiWriter(newBuffer, writer)
|
||||
defer newBuffer.Close()
|
||||
}
|
||||
// Ignore error if cache is full, proceed to write the object.
|
||||
if err != nil && err != objcache.ErrCacheFull {
|
||||
// For any other error return here.
|
||||
return toObjectErr(errors.Trace(err), bucket, object)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
var totalBytesRead int64
|
||||
storage, err := NewErasureStorage(onlineDisks, xlMeta.Erasure.DataBlocks, xlMeta.Erasure.ParityBlocks, xlMeta.Erasure.BlockSize)
|
||||
if err != nil {
|
||||
@ -366,7 +319,7 @@ func (xl xlObjects) getObject(bucket, object string, startOffset int64, length i
|
||||
checksums[index] = checksumInfo.Hash
|
||||
}
|
||||
|
||||
file, err := storage.ReadFile(mw, bucket, pathJoin(object, partName), partOffset, readSize, partSize, checksums, algorithm, xlMeta.Erasure.BlockSize)
|
||||
file, err := storage.ReadFile(writer, bucket, pathJoin(object, partName), partOffset, readSize, partSize, checksums, algorithm, xlMeta.Erasure.BlockSize)
|
||||
if err != nil {
|
||||
return toObjectErr(err, bucket, object)
|
||||
}
|
||||
@ -663,27 +616,6 @@ func (xl xlObjects) putObject(bucket string, object string, data *hash.Reader, m
|
||||
// Limit the reader to its provided size if specified.
|
||||
var reader io.Reader = data
|
||||
|
||||
// Proceed to set the cache.
|
||||
var newBuffer io.WriteCloser
|
||||
|
||||
// If caching is enabled, proceed to set the cache.
|
||||
if data.Size() > 0 && xl.objCacheEnabled {
|
||||
// PutObject invalidates any previously cached object in memory.
|
||||
xl.objCache.Delete(path.Join(bucket, object))
|
||||
|
||||
// Create a new entry in memory of size.
|
||||
newBuffer, err = xl.objCache.Create(path.Join(bucket, object), data.Size())
|
||||
if err == nil {
|
||||
// Cache incoming data into a buffer
|
||||
reader = io.TeeReader(data, newBuffer)
|
||||
} else {
|
||||
// Return errors other than ErrCacheFull
|
||||
if err != objcache.ErrCacheFull {
|
||||
return ObjectInfo{}, toObjectErr(errors.Trace(err), bucket, object)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// Initialize parts metadata
|
||||
partsMetadata := make([]xlMetaV1, len(xl.storageDisks))
|
||||
|
||||
@ -811,12 +743,6 @@ func (xl xlObjects) putObject(bucket string, object string, data *hash.Reader, m
|
||||
return ObjectInfo{}, toObjectErr(err, bucket, object)
|
||||
}
|
||||
|
||||
// Once we have successfully renamed the object, Close the buffer which would
|
||||
// save the object on cache.
|
||||
if sizeWritten > 0 && xl.objCacheEnabled && newBuffer != nil {
|
||||
newBuffer.Close()
|
||||
}
|
||||
|
||||
// Object info is the same in all disks, so we can pick the first meta
|
||||
// of the first disk
|
||||
xlMeta = partsMetadata[0]
|
||||
@ -916,11 +842,6 @@ func (xl xlObjects) DeleteObject(bucket, object string) (err error) {
|
||||
return toObjectErr(err, bucket, object)
|
||||
}
|
||||
|
||||
if xl.objCacheEnabled {
|
||||
// Delete from the cache.
|
||||
xl.objCache.Delete(pathJoin(bucket, object))
|
||||
}
|
||||
|
||||
// Success.
|
||||
return nil
|
||||
}
|
||||
|
@ -184,8 +184,7 @@ func TestGetObjectNoQuorum(t *testing.T) {
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
// Disable caching to avoid returning early and not covering other code-paths
|
||||
xl.objCacheEnabled = false
|
||||
|
||||
// Make 9 disks offline, which leaves less than quorum number of disks
|
||||
// in a 16 disk XL setup. The original disks are 'replaced' with
|
||||
// naughtyDisks that fail after 'f' successful StorageAPI method
|
||||
|
34
cmd/xl-v1.go
34
cmd/xl-v1.go
@ -18,15 +18,12 @@ package cmd
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"runtime/debug"
|
||||
"sort"
|
||||
"sync"
|
||||
"time"
|
||||
|
||||
humanize "github.com/dustin/go-humanize"
|
||||
"github.com/minio/minio/pkg/disk"
|
||||
"github.com/minio/minio/pkg/errors"
|
||||
"github.com/minio/minio/pkg/objcache"
|
||||
)
|
||||
|
||||
// XL constants.
|
||||
@ -37,9 +34,6 @@ const (
|
||||
// Uploads metadata file carries per multipart object metadata.
|
||||
uploadsJSONFile = "uploads.json"
|
||||
|
||||
// Represents the minimum required RAM size to enable caching.
|
||||
minRAMSize = 24 * humanize.GiByte
|
||||
|
||||
// Maximum erasure blocks.
|
||||
maxErasureBlocks = 16
|
||||
|
||||
@ -55,12 +49,6 @@ type xlObjects struct {
|
||||
// ListObjects pool management.
|
||||
listPool *treeWalkPool
|
||||
|
||||
// Object cache for caching objects.
|
||||
objCache *objcache.Cache
|
||||
|
||||
// Object cache enabled.
|
||||
objCacheEnabled bool
|
||||
|
||||
// name space mutex for object layer
|
||||
nsMutex *nsLockMap
|
||||
}
|
||||
@ -112,28 +100,6 @@ func newXLObjects(storageDisks []StorageAPI) (ObjectLayer, error) {
|
||||
listPool: listPool,
|
||||
nsMutex: newNSLock(globalIsDistXL),
|
||||
}
|
||||
// Get cache size if _MINIO_CACHE environment variable is set.
|
||||
var maxCacheSize uint64
|
||||
if !globalXLObjCacheDisabled {
|
||||
maxCacheSize, err = GetMaxCacheSize()
|
||||
errorIf(err, "Unable to get maximum cache size")
|
||||
|
||||
// Enable object cache if cache size is more than zero
|
||||
xl.objCacheEnabled = maxCacheSize > 0
|
||||
}
|
||||
|
||||
// Check if object cache is enabled.
|
||||
if xl.objCacheEnabled {
|
||||
// Initialize object cache.
|
||||
objCache, oerr := objcache.New(maxCacheSize, objcache.DefaultExpiry)
|
||||
if oerr != nil {
|
||||
return nil, oerr
|
||||
}
|
||||
objCache.OnEviction = func(key string) {
|
||||
debug.FreeOSMemory()
|
||||
}
|
||||
xl.objCache = objCache
|
||||
}
|
||||
|
||||
// Initialize meta volume, if volume already exists ignores it.
|
||||
if err = initMetaVolume(xl.storageDisks); err != nil {
|
||||
|
@ -1,27 +0,0 @@
|
||||
## Object Caching
|
||||
|
||||
Object caching is on by default with following settings
|
||||
|
||||
- Cache size is 50% of your RAM size. Caching is disabled
|
||||
if your RAM size is smaller than 8GB.
|
||||
|
||||
- Expiration of each entries happen on every 72 hours.
|
||||
|
||||
- Garbage collection sweep of the expired entries happen every
|
||||
1/4th the set expiration hours value (every 18 hours).
|
||||
|
||||
NOTE: None of the settings can be configured manually.
|
||||
|
||||
### Behavior
|
||||
|
||||
Caching happens on both GET and PUT operations.
|
||||
|
||||
- GET caches new objects for entries not found in cache.
|
||||
Otherwise serves from the cache.
|
||||
|
||||
- PUT/POST caches all successfully uploaded objects. Replaces
|
||||
existing cached entry for the same object if needed.
|
||||
|
||||
NOTE: Expiration happens automatically based on the configured
|
||||
interval as explained above, frequently accessed objects stay
|
||||
alive in cache for a significantly longer time on every cache hit.
|
@ -1,64 +0,0 @@
|
||||
```
|
||||
PACKAGE DOCUMENTATION
|
||||
|
||||
package objcache
|
||||
import "github.com/minio/minio/pkg/objcache"
|
||||
|
||||
Package objcache implements in memory caching methods.
|
||||
|
||||
CONSTANTS
|
||||
|
||||
const (
|
||||
// NoExpiry represents caches to be permanent and can only be deleted.
|
||||
NoExpiry = time.Duration(0)
|
||||
|
||||
// DefaultExpiry represents three days time duration when individual entries will be expired.
|
||||
DefaultExpiry = time.Duration(3 * 24 * time.Hour)
|
||||
)
|
||||
|
||||
VARIABLES
|
||||
|
||||
var ErrCacheFull = errors.New("Not enough space in cache")
|
||||
ErrCacheFull - cache is full.
|
||||
|
||||
var ErrKeyNotFoundInCache = errors.New("Key not found in cache")
|
||||
ErrKeyNotFoundInCache - key not found in cache.
|
||||
|
||||
TYPES
|
||||
|
||||
type Cache struct {
|
||||
|
||||
// OnEviction - callback function for eviction
|
||||
OnEviction func(key string)
|
||||
// contains filtered or unexported fields
|
||||
}
|
||||
Cache holds the required variables to compose an in memory cache system
|
||||
which also provides expiring key mechanism and also maxSize.
|
||||
|
||||
func New(maxSize uint64, expiry time.Duration) *Cache
|
||||
New - Return a new cache with a given default expiry duration. If the
|
||||
expiry duration is less than one (or NoExpiry), the items in the cache
|
||||
never expire (by default), and must be deleted manually.
|
||||
|
||||
func (c *Cache) Create(key string, size int64) (w io.WriteCloser, err error)
|
||||
Create - validates if object size fits with in cache size limit and
|
||||
returns a io.WriteCloser to which object contents can be written and
|
||||
finally Close()'d. During Close() we checks if the amount of data
|
||||
written is equal to the size of the object, in which case it saves the
|
||||
contents to object cache.
|
||||
|
||||
func (c *Cache) Delete(key string)
|
||||
Delete - delete deletes an entry from the cache.
|
||||
|
||||
func (c *Cache) DeleteExpired()
|
||||
DeleteExpired - deletes all the expired entries from the cache.
|
||||
|
||||
func (c *Cache) Open(key string) (io.ReadSeeker, error)
|
||||
Open - open the in-memory file, returns an in memory read seeker.
|
||||
returns an error ErrNotFoundInCache, if the key does not exist.
|
||||
|
||||
func (c *Cache) StopExpiry()
|
||||
StopExpiry sends a message to the expiry routine to stop expiring cached
|
||||
entries. NOTE: once this is called, cached entries will not be expired
|
||||
if the consume has called this.
|
||||
```
|
@ -1,50 +0,0 @@
|
||||
/*
|
||||
* Minio Cloud Storage, (C) 2016 Minio, Inc.
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*
|
||||
*/
|
||||
|
||||
// Package objcache implements in memory caching methods.
|
||||
package objcache
|
||||
|
||||
// Used for adding entry to the object cache.
|
||||
// Implements io.WriteCloser
|
||||
type cappedWriter struct {
|
||||
offset int64
|
||||
cap int64
|
||||
buffer []byte
|
||||
onClose func() error
|
||||
}
|
||||
|
||||
// Write implements a limited writer, returns error.
|
||||
// if the writes go beyond allocated size.
|
||||
func (c *cappedWriter) Write(b []byte) (n int, err error) {
|
||||
if c.offset+int64(len(b)) > c.cap {
|
||||
return 0, ErrExcessData
|
||||
}
|
||||
n = copy(c.buffer[int(c.offset):int(c.offset)+len(b)], b)
|
||||
c.offset = c.offset + int64(n)
|
||||
return n, nil
|
||||
}
|
||||
|
||||
// Reset relinquishes the allocated underlying buffer.
|
||||
func (c *cappedWriter) Reset() {
|
||||
c.buffer = nil
|
||||
}
|
||||
|
||||
// On close, onClose() is called which checks if all object contents
|
||||
// have been written so that it can save the buffer to the cache.
|
||||
func (c cappedWriter) Close() (err error) {
|
||||
return c.onClose()
|
||||
}
|
@ -1,315 +0,0 @@
|
||||
/*
|
||||
* Minio Cloud Storage, (C) 2016, 2017 Minio, Inc.
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*
|
||||
*/
|
||||
|
||||
// Package objcache implements in memory caching methods.
|
||||
package objcache
|
||||
|
||||
import (
|
||||
"bytes"
|
||||
"errors"
|
||||
"io"
|
||||
"runtime/debug"
|
||||
"sync"
|
||||
"time"
|
||||
)
|
||||
|
||||
const (
|
||||
// NoExpiry represents caches to be permanent and can only be deleted.
|
||||
NoExpiry = time.Duration(0)
|
||||
|
||||
// DefaultExpiry represents three days time duration when individual entries will be expired.
|
||||
DefaultExpiry = time.Duration(3 * 24 * time.Hour)
|
||||
|
||||
// defaultBufferRatio represents default ratio used to calculate the
|
||||
// individual cache entry buffer size.
|
||||
defaultBufferRatio = uint64(10)
|
||||
|
||||
// defaultGCPercent represents default garbage collection target percentage.
|
||||
defaultGCPercent = 50
|
||||
)
|
||||
|
||||
var (
|
||||
// ErrKeyNotFoundInCache - key not found in cache.
|
||||
ErrKeyNotFoundInCache = errors.New("Key not found in cache")
|
||||
|
||||
// ErrCacheFull - cache is full.
|
||||
ErrCacheFull = errors.New("Not enough space in cache")
|
||||
|
||||
// ErrExcessData - excess data was attempted to be written on cache.
|
||||
ErrExcessData = errors.New("Attempted excess write on cache")
|
||||
)
|
||||
|
||||
// buffer represents the in memory cache of a single entry.
|
||||
// buffer carries value of the data and last accessed time.
|
||||
type buffer struct {
|
||||
value []byte // Value of the entry.
|
||||
lastAccessed time.Time // Represents time when value was last accessed.
|
||||
}
|
||||
|
||||
// Cache holds the required variables to compose an in memory cache system
|
||||
// which also provides expiring key mechanism and also maxSize.
|
||||
type Cache struct {
|
||||
// Mutex is used for handling the concurrent
|
||||
// read/write requests for cache
|
||||
mutex sync.Mutex
|
||||
|
||||
// Once is used for resetting GC once after
|
||||
// peak cache usage.
|
||||
onceGC sync.Once
|
||||
|
||||
// maxSize is a total size for overall cache
|
||||
maxSize uint64
|
||||
|
||||
// maxCacheEntrySize is a total size per key buffer.
|
||||
maxCacheEntrySize uint64
|
||||
|
||||
// currentSize is a current size in memory
|
||||
currentSize uint64
|
||||
|
||||
// OnEviction - callback function for eviction
|
||||
OnEviction func(key string)
|
||||
|
||||
// totalEvicted counter to keep track of total expirys
|
||||
totalEvicted int
|
||||
|
||||
// map of objectName and its contents
|
||||
entries map[string]*buffer
|
||||
|
||||
// Expiry in time duration.
|
||||
expiry time.Duration
|
||||
|
||||
// Stop garbage collection routine, stops any running GC routine.
|
||||
stopGC chan struct{}
|
||||
}
|
||||
|
||||
// New - Return a new cache with a given default expiry
|
||||
// duration. If the expiry duration is less than one
|
||||
// (or NoExpiry), the items in the cache never expire
|
||||
// (by default), and must be deleted manually.
|
||||
func New(maxSize uint64, expiry time.Duration) (c *Cache, err error) {
|
||||
if maxSize == 0 {
|
||||
err = errors.New("invalid maximum cache size")
|
||||
return c, err
|
||||
}
|
||||
|
||||
// A garbage collection is triggered when the ratio
|
||||
// of freshly allocated data to live data remaining
|
||||
// after the previous collection reaches this percentage.
|
||||
//
|
||||
// - https://golang.org/pkg/runtime/debug/#SetGCPercent
|
||||
//
|
||||
// This means that by default GC is triggered after
|
||||
// we've allocated an extra amount of memory proportional
|
||||
// to the amount already in use.
|
||||
//
|
||||
// If gcpercent=100 and we're using 4M, we'll gc again
|
||||
// when we get to 8M.
|
||||
//
|
||||
// Set this value to 40% if caching is enabled.
|
||||
debug.SetGCPercent(defaultGCPercent)
|
||||
|
||||
// Max cache entry size - indicates the
|
||||
// maximum buffer per key that can be held in
|
||||
// memory. Currently this value is 1/10th
|
||||
// the size of requested cache size.
|
||||
maxCacheEntrySize := func() uint64 {
|
||||
i := maxSize / defaultBufferRatio
|
||||
if i == 0 {
|
||||
i = maxSize
|
||||
}
|
||||
return i
|
||||
}()
|
||||
|
||||
c = &Cache{
|
||||
onceGC: sync.Once{},
|
||||
maxSize: maxSize,
|
||||
maxCacheEntrySize: maxCacheEntrySize,
|
||||
entries: make(map[string]*buffer),
|
||||
expiry: expiry,
|
||||
}
|
||||
|
||||
// We have expiry start the janitor routine.
|
||||
if expiry > 0 {
|
||||
// Initialize a new stop GC channel.
|
||||
c.stopGC = make(chan struct{})
|
||||
|
||||
// Start garbage collection routine to expire objects.
|
||||
c.StartGC()
|
||||
}
|
||||
|
||||
return c, nil
|
||||
}
|
||||
|
||||
// Create - validates if object size fits with in cache size limit and returns a io.WriteCloser
|
||||
// to which object contents can be written and finally Close()'d. During Close() we
|
||||
// checks if the amount of data written is equal to the size of the object, in which
|
||||
// case it saves the contents to object cache.
|
||||
func (c *Cache) Create(key string, size int64) (w io.WriteCloser, err error) {
|
||||
// Recovers any panic generated and return errors appropriately.
|
||||
defer func() {
|
||||
if r := recover(); r != nil {
|
||||
// Recover any panic and return ErrCacheFull.
|
||||
err = ErrCacheFull
|
||||
}
|
||||
}() // Do not crash the server.
|
||||
|
||||
valueLen := uint64(size)
|
||||
// Check if the size of the object is > 1/10th the size
|
||||
// of the cache, if yes then we ignore it.
|
||||
if valueLen > c.maxCacheEntrySize {
|
||||
return nil, ErrCacheFull
|
||||
}
|
||||
|
||||
c.mutex.Lock()
|
||||
// Check if the incoming size is going to exceed the
|
||||
// effective cache size, if yes return error instead.
|
||||
if c.currentSize+valueLen > c.maxSize {
|
||||
c.mutex.Unlock()
|
||||
return nil, ErrCacheFull
|
||||
}
|
||||
|
||||
// Change GC percent if the current cache usage might
|
||||
// become 75% of the maximum allowed usage, change
|
||||
// the GC percent.
|
||||
if c.currentSize+valueLen > (75 * c.maxSize / 100) {
|
||||
c.onceGC.Do(func() { debug.SetGCPercent(defaultGCPercent - 25) })
|
||||
}
|
||||
c.mutex.Unlock()
|
||||
|
||||
cbuf := &cappedWriter{
|
||||
offset: 0,
|
||||
cap: size,
|
||||
buffer: make([]byte, size),
|
||||
}
|
||||
|
||||
// Function called on close which saves the object contents
|
||||
// to the object cache.
|
||||
onClose := func() error {
|
||||
c.mutex.Lock()
|
||||
defer c.mutex.Unlock()
|
||||
if size != cbuf.offset {
|
||||
cbuf.Reset() // Reset resets the buffer to be empty.
|
||||
// Full object not available hence do not save buf to object cache.
|
||||
return io.ErrShortBuffer
|
||||
}
|
||||
|
||||
// Full object available in buf, save it to cache.
|
||||
c.entries[key] = &buffer{
|
||||
value: cbuf.buffer,
|
||||
lastAccessed: time.Now().UTC(), // Save last accessed time.
|
||||
}
|
||||
|
||||
// Account for the memory allocated above.
|
||||
c.currentSize += uint64(size)
|
||||
return nil
|
||||
}
|
||||
|
||||
// Object contents that is written - cappedWriter.Write(data)
|
||||
// will be accumulated in buf which implements io.Writer.
|
||||
cbuf.onClose = onClose
|
||||
|
||||
// Capped writer.
|
||||
return cbuf, nil
|
||||
}
|
||||
|
||||
// Open - open the in-memory file, returns an in memory read seeker.
|
||||
// returns an error ErrNotFoundInCache, if the key does not exist.
|
||||
// Returns ErrKeyNotFoundInCache if entry's lastAccessedTime is older
|
||||
// than objModTime.
|
||||
func (c *Cache) Open(key string, objModTime time.Time) (io.ReaderAt, error) {
|
||||
// Entry exists, return the readable buffer.
|
||||
c.mutex.Lock()
|
||||
defer c.mutex.Unlock()
|
||||
buf, ok := c.entries[key]
|
||||
if !ok {
|
||||
return nil, ErrKeyNotFoundInCache
|
||||
}
|
||||
|
||||
// Check if buf is recent copy of the object on disk.
|
||||
if buf.lastAccessed.Before(objModTime) {
|
||||
c.delete(key)
|
||||
return nil, ErrKeyNotFoundInCache
|
||||
}
|
||||
|
||||
buf.lastAccessed = time.Now().UTC()
|
||||
return bytes.NewReader(buf.value), nil
|
||||
}
|
||||
|
||||
// Delete - delete deletes an entry from the cache.
|
||||
func (c *Cache) Delete(key string) {
|
||||
c.mutex.Lock()
|
||||
c.delete(key)
|
||||
c.mutex.Unlock()
|
||||
if c.OnEviction != nil {
|
||||
c.OnEviction(key)
|
||||
}
|
||||
}
|
||||
|
||||
// gc - garbage collect all the expired entries from the cache.
|
||||
func (c *Cache) gc() {
|
||||
var evictedEntries []string
|
||||
c.mutex.Lock()
|
||||
for k, v := range c.entries {
|
||||
if c.expiry > 0 && time.Now().UTC().Sub(v.lastAccessed) > c.expiry {
|
||||
c.delete(k)
|
||||
evictedEntries = append(evictedEntries, k)
|
||||
}
|
||||
}
|
||||
c.mutex.Unlock()
|
||||
for _, k := range evictedEntries {
|
||||
if c.OnEviction != nil {
|
||||
c.OnEviction(k)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// StopGC sends a message to the expiry routine to stop
|
||||
// expiring cached entries. NOTE: once this is called, cached
|
||||
// entries will not be expired, be careful if you are using this.
|
||||
func (c *Cache) StopGC() {
|
||||
if c.stopGC != nil {
|
||||
c.stopGC <- struct{}{}
|
||||
}
|
||||
}
|
||||
|
||||
// StartGC starts running a routine ticking at expiry interval,
|
||||
// on each interval this routine does a sweep across the cache
|
||||
// entries and garbage collects all the expired entries.
|
||||
func (c *Cache) StartGC() {
|
||||
go func() {
|
||||
for {
|
||||
select {
|
||||
// Wait till cleanup interval and initiate delete expired entries.
|
||||
case <-time.After(c.expiry / 4):
|
||||
c.gc()
|
||||
// Stop the routine, usually called by the user of object cache during cleanup.
|
||||
case <-c.stopGC:
|
||||
return
|
||||
}
|
||||
}
|
||||
}()
|
||||
}
|
||||
|
||||
// Deletes a requested entry from the cache.
|
||||
func (c *Cache) delete(key string) {
|
||||
if _, ok := c.entries[key]; ok {
|
||||
deletedSize := uint64(len(c.entries[key].value))
|
||||
delete(c.entries, key)
|
||||
c.currentSize -= deletedSize
|
||||
c.totalEvicted++
|
||||
}
|
||||
}
|
@ -1,326 +0,0 @@
|
||||
/*
|
||||
* Minio Cloud Storage, (C) 2016 Minio, Inc.
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*
|
||||
*/
|
||||
|
||||
package objcache
|
||||
|
||||
import (
|
||||
"bytes"
|
||||
"io"
|
||||
"testing"
|
||||
"time"
|
||||
)
|
||||
|
||||
// TestObjectCache tests cases of object cache with expiry.
|
||||
func TestObjExpiry(t *testing.T) {
|
||||
// Non exhaustive list of all object cache behavior cases.
|
||||
testCases := []struct {
|
||||
expiry time.Duration
|
||||
cacheSize uint64
|
||||
err error
|
||||
closeErr error
|
||||
}{
|
||||
{
|
||||
expiry: 100 * time.Millisecond,
|
||||
cacheSize: 1024,
|
||||
err: ErrKeyNotFoundInCache,
|
||||
closeErr: nil,
|
||||
},
|
||||
}
|
||||
|
||||
// Test case 1 validates running of GC.
|
||||
testCase := testCases[0]
|
||||
cache, err := New(testCase.cacheSize, testCase.expiry)
|
||||
if err != nil {
|
||||
t.Fatalf("Unable to create new objcache")
|
||||
}
|
||||
|
||||
cache.OnEviction = func(key string) {}
|
||||
w, err := cache.Create("test", 1)
|
||||
if err != nil {
|
||||
t.Errorf("Test case 1 expected to pass, failed instead %s", err)
|
||||
}
|
||||
// Write a byte.
|
||||
w.Write([]byte("1"))
|
||||
if err = w.Close(); err != nil {
|
||||
t.Errorf("Test case 1 expected to pass, failed instead %s", err)
|
||||
}
|
||||
// Wait for 500 millisecond.
|
||||
time.Sleep(500 * time.Millisecond)
|
||||
// Setting objModTime to the beginning of golang's time.Time to avoid deletion of stale entry.
|
||||
fakeObjModTime := time.Time{}
|
||||
_, err = cache.Open("test", fakeObjModTime)
|
||||
if err != testCase.err {
|
||||
t.Errorf("Test case 1 expected %s, got instead %s", testCase.err, err)
|
||||
}
|
||||
}
|
||||
|
||||
// TestObjCache - tests various cases for object cache behavior.
|
||||
func TestObjCache(t *testing.T) {
|
||||
// Setting objModTime to the beginning of golang's time.Time to avoid deletion of stale entry.
|
||||
fakeObjModTime := time.Time{}
|
||||
|
||||
// Non exhaustive list of all object cache behavior cases.
|
||||
testCases := []struct {
|
||||
expiry time.Duration
|
||||
cacheSize uint64
|
||||
err error
|
||||
closeErr error
|
||||
}{
|
||||
// Validate if a key is not found in cache and Open fails.
|
||||
{
|
||||
expiry: NoExpiry,
|
||||
cacheSize: 1024,
|
||||
err: ErrKeyNotFoundInCache,
|
||||
},
|
||||
// Validate if cache indicates that it is full and Create fails.
|
||||
{
|
||||
expiry: NoExpiry,
|
||||
cacheSize: 1,
|
||||
err: ErrCacheFull,
|
||||
},
|
||||
// Validate if Create succeeds but Close fails to write to buffer.
|
||||
{
|
||||
expiry: NoExpiry,
|
||||
cacheSize: 2,
|
||||
closeErr: io.ErrShortBuffer,
|
||||
},
|
||||
// Validate that Create and Close succeed, making sure to update the cache.
|
||||
{
|
||||
expiry: NoExpiry,
|
||||
cacheSize: 1024,
|
||||
},
|
||||
// Validate that Delete succeeds and Open fails with key not found in cache.
|
||||
{
|
||||
expiry: NoExpiry,
|
||||
cacheSize: 1024,
|
||||
err: ErrKeyNotFoundInCache,
|
||||
},
|
||||
// Validate OnEviction function is called upon entry delete.
|
||||
{
|
||||
expiry: NoExpiry,
|
||||
cacheSize: 1024,
|
||||
},
|
||||
// Validate error excess data.
|
||||
{
|
||||
expiry: NoExpiry,
|
||||
cacheSize: 5,
|
||||
closeErr: ErrExcessData,
|
||||
},
|
||||
// Validate error excess data during write.
|
||||
{
|
||||
expiry: NoExpiry,
|
||||
cacheSize: 2048,
|
||||
err: ErrExcessData,
|
||||
},
|
||||
}
|
||||
|
||||
// Test 1 validating Open failure.
|
||||
testCase := testCases[0]
|
||||
cache, err := New(testCase.cacheSize, testCase.expiry)
|
||||
if err != nil {
|
||||
t.Fatalf("Unable to create new objcache")
|
||||
}
|
||||
|
||||
_, err = cache.Open("test", fakeObjModTime)
|
||||
if testCase.err != err {
|
||||
t.Errorf("Test case 2 expected to pass, failed instead %s", err)
|
||||
}
|
||||
|
||||
// Test 2 validating Create failure.
|
||||
testCase = testCases[1]
|
||||
cache, err = New(testCase.cacheSize, testCase.expiry)
|
||||
if err != nil {
|
||||
t.Fatalf("Unable to create new objcache")
|
||||
}
|
||||
|
||||
_, err = cache.Create("test", 2)
|
||||
if testCase.err != err {
|
||||
t.Errorf("Test case 2 expected to pass, failed instead %s", err)
|
||||
}
|
||||
|
||||
// Test 3 validating Create succeeds and returns a writer.
|
||||
// Subsequently we Close() without writing any data, to receive
|
||||
// `io.ErrShortBuffer`
|
||||
testCase = testCases[2]
|
||||
cache, err = New(testCase.cacheSize, testCase.expiry)
|
||||
if err != nil {
|
||||
t.Fatalf("Unable to create new objcache")
|
||||
}
|
||||
|
||||
w, err := cache.Create("test", 1)
|
||||
if testCase.err != err {
|
||||
t.Errorf("Test case 3 expected to pass, failed instead %s", err)
|
||||
}
|
||||
if err = w.Close(); err != testCase.closeErr {
|
||||
t.Errorf("Test case 3 expected to pass, failed instead %s", err)
|
||||
}
|
||||
|
||||
// Test 4 validates Create and Close succeeds successfully caching
|
||||
// the writes.
|
||||
testCase = testCases[3]
|
||||
cache, err = New(testCase.cacheSize, testCase.expiry)
|
||||
if err != nil {
|
||||
t.Fatalf("Unable to create new objcache")
|
||||
}
|
||||
|
||||
w, err = cache.Create("test", 5)
|
||||
if testCase.err != err {
|
||||
t.Errorf("Test case 4 expected to pass, failed instead %s", err)
|
||||
}
|
||||
// Write '5' bytes.
|
||||
w.Write([]byte("Hello"))
|
||||
// Close to successfully save into cache.
|
||||
if err = w.Close(); err != nil {
|
||||
t.Errorf("Test case 4 expected to pass, failed instead %s", err)
|
||||
}
|
||||
r, err := cache.Open("test", fakeObjModTime)
|
||||
if err != nil {
|
||||
t.Errorf("Test case 4 expected to pass, failed instead %s", err)
|
||||
}
|
||||
// Reads everything stored for key "test".
|
||||
cbytes := make([]byte, 5)
|
||||
_, err = r.ReadAt(cbytes, 0)
|
||||
if err != nil {
|
||||
t.Errorf("Test case 4 expected to pass, failed instead %s", err)
|
||||
}
|
||||
// Validate if read bytes match.
|
||||
if !bytes.Equal(cbytes, []byte("Hello")) {
|
||||
t.Errorf("Test case 4 expected to pass. wanted \"Hello\", got %s", string(cbytes))
|
||||
}
|
||||
|
||||
// Test 5 validates Delete succeeds and Open fails with err
|
||||
testCase = testCases[4]
|
||||
cache, err = New(testCase.cacheSize, testCase.expiry)
|
||||
if err != nil {
|
||||
t.Fatalf("Unable to create new objcache")
|
||||
}
|
||||
|
||||
w, err = cache.Create("test", 5)
|
||||
if err != nil {
|
||||
t.Errorf("Test case 5 expected to pass, failed instead %s", err)
|
||||
}
|
||||
// Write '5' bytes.
|
||||
w.Write([]byte("Hello"))
|
||||
// Close to successfully save into cache.
|
||||
if err = w.Close(); err != nil {
|
||||
t.Errorf("Test case 5 expected to pass, failed instead %s", err)
|
||||
}
|
||||
// Delete the cache entry.
|
||||
cache.Delete("test")
|
||||
_, err = cache.Open("test", fakeObjModTime)
|
||||
if testCase.err != err {
|
||||
t.Errorf("Test case 5 expected to pass, failed instead %s", err)
|
||||
}
|
||||
|
||||
// Test 6 validates OnEviction being called upon Delete is being invoked.
|
||||
testCase = testCases[5]
|
||||
cache, err = New(testCase.cacheSize, testCase.expiry)
|
||||
if err != nil {
|
||||
t.Fatalf("Unable to create new objcache")
|
||||
}
|
||||
|
||||
w, err = cache.Create("test", 5)
|
||||
if err != nil {
|
||||
t.Errorf("Test case 6 expected to pass, failed instead %s", err)
|
||||
}
|
||||
// Write '5' bytes.
|
||||
w.Write([]byte("Hello"))
|
||||
// Close to successfully save into cache.
|
||||
if err = w.Close(); err != nil {
|
||||
t.Errorf("Test case 6 expected to pass, failed instead %s", err)
|
||||
}
|
||||
var deleteKey string
|
||||
cache.OnEviction = func(key string) {
|
||||
deleteKey = key
|
||||
}
|
||||
// Delete the cache entry.
|
||||
cache.Delete("test")
|
||||
if deleteKey != "test" {
|
||||
t.Errorf("Test case 6 expected to pass, wanted \"test\", got %s", deleteKey)
|
||||
}
|
||||
|
||||
// Test 7 validates rejecting requests when excess data is being saved.
|
||||
testCase = testCases[6]
|
||||
cache, err = New(testCase.cacheSize, testCase.expiry)
|
||||
if err != nil {
|
||||
t.Fatalf("Unable to create new objcache")
|
||||
}
|
||||
|
||||
w, err = cache.Create("test1", 5)
|
||||
if err != nil {
|
||||
t.Errorf("Test case 7 expected to pass, failed instead %s", err)
|
||||
}
|
||||
// Write '5' bytes.
|
||||
w.Write([]byte("Hello"))
|
||||
// Close to successfully save into cache.
|
||||
if err = w.Close(); err != nil {
|
||||
t.Errorf("Test case 7 expected to pass, failed instead %s", err)
|
||||
}
|
||||
_, err = cache.Create("test2", 1)
|
||||
if err != ErrCacheFull {
|
||||
t.Errorf("Test case 7 expected to pass, failed instead %s", err)
|
||||
}
|
||||
|
||||
// Test 8 validates rejecting Writes which write excess data.
|
||||
testCase = testCases[7]
|
||||
cache, err = New(testCase.cacheSize, testCase.expiry)
|
||||
if err != nil {
|
||||
t.Fatalf("Unable to create new objcache")
|
||||
}
|
||||
|
||||
w, err = cache.Create("test1", 5)
|
||||
if err != nil {
|
||||
t.Errorf("Test case 8 expected to pass, failed instead %s", err)
|
||||
}
|
||||
// Write '5' bytes.
|
||||
n, err := w.Write([]byte("Hello"))
|
||||
if err != nil {
|
||||
t.Errorf("Test case 8 expected to pass, failed instead %s", err)
|
||||
}
|
||||
if n != 5 {
|
||||
t.Errorf("Test case 8 expected 5 bytes written, instead found %d", n)
|
||||
}
|
||||
// Write '1' more byte, should return error.
|
||||
n, err = w.Write([]byte("W"))
|
||||
if n == 0 && err != testCase.err {
|
||||
t.Errorf("Test case 8 expected to fail with ErrExcessData, but failed with %s instead", err)
|
||||
}
|
||||
}
|
||||
|
||||
// TestStateEntryPurge - tests if objCache purges stale entry and returns ErrKeyNotFoundInCache.
|
||||
func TestStaleEntryPurge(t *testing.T) {
|
||||
cache, err := New(1024, NoExpiry)
|
||||
if err != nil {
|
||||
t.Fatalf("Unable to create new objcache")
|
||||
}
|
||||
|
||||
w, err := cache.Create("test", 5)
|
||||
if err != nil {
|
||||
t.Errorf("Test case expected to pass, failed instead %s", err)
|
||||
}
|
||||
// Write '5' bytes.
|
||||
w.Write([]byte("Hello"))
|
||||
// Close to successfully save into cache.
|
||||
if err = w.Close(); err != nil {
|
||||
t.Errorf("Test case expected to pass, failed instead %s", err)
|
||||
}
|
||||
|
||||
_, err = cache.Open("test", time.Now().AddDate(0, 0, 1).UTC())
|
||||
if err != ErrKeyNotFoundInCache {
|
||||
t.Errorf("Test case expected to return ErrKeyNotFoundInCache, instead returned %s", err)
|
||||
}
|
||||
}
|
Loading…
Reference in New Issue
Block a user