Skip to content

Commit

Permalink
move cache TTLs to a struct
Browse files Browse the repository at this point in the history
Signed-off-by: akanshat <[email protected]>
  • Loading branch information
akanshat committed Dec 20, 2021
1 parent 088b9d3 commit a5e287a
Show file tree
Hide file tree
Showing 4 changed files with 92 additions and 84 deletions.
31 changes: 15 additions & 16 deletions pkg/cache/groupcache.go
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@ import (
"github.com/thanos-io/thanos/pkg/model"
"github.com/thanos-io/thanos/pkg/objstore"
"github.com/thanos-io/thanos/pkg/runutil"
"github.com/thanos-io/thanos/pkg/store/cache/cacheconfig"
"github.com/thanos-io/thanos/pkg/store/cache/cachekey"
"github.com/vimeo/galaxycache"
galaxyhttp "github.com/vimeo/galaxycache/http"
Expand Down Expand Up @@ -79,20 +80,18 @@ func parseGroupcacheConfig(conf []byte) (GroupcacheConfig, error) {

// NewGroupcache creates a new Groupcache instance.
func NewGroupcache(logger log.Logger, reg prometheus.Registerer, conf []byte, basepath string, r *route.Router, bucket objstore.Bucket,
isTSDBChunkFile, isMetaFile, isBlocksRootDir func(path string) bool,
MetaFileExistsTTL, MetafileDoesntExistTTL, MetafileContentTTL, ChunkObjectAttrsTTL, ChunkSubrangeTTL, BlocksIterTTL time.Duration) (*Groupcache, error) {
cacheConfig *cacheconfig.CachingWithBackendConfig) (*Groupcache, error) {
config, err := parseGroupcacheConfig(conf)
if err != nil {
return nil, err
}

return NewGroupcacheWithConfig(logger, reg, config, basepath, r, bucket, isTSDBChunkFile, isMetaFile, isBlocksRootDir, MetaFileExistsTTL, MetafileDoesntExistTTL, MetafileContentTTL, ChunkObjectAttrsTTL, ChunkSubrangeTTL, BlocksIterTTL)
return NewGroupcacheWithConfig(logger, reg, config, basepath, r, bucket, cacheConfig)
}

// NewGroupcacheWithConfig creates a new Groupcache instance with the given config.
func NewGroupcacheWithConfig(logger log.Logger, reg prometheus.Registerer, conf GroupcacheConfig, basepath string, r *route.Router, bucket objstore.Bucket,
isTSDBChunkFile, isMetaFile, isBlocksRootDir func(path string) bool,
MetaFileExistsTTL, MetafileDoesntExistTTL, MetafileContentTTL, ChunkObjectAttrsTTL, ChunkSubrangeTTL, BlocksIterTTL time.Duration) (*Groupcache, error) {
cacheConfig *cacheconfig.CachingWithBackendConfig) (*Groupcache, error) {
httpProto := galaxyhttp.NewHTTPFetchProtocol(&galaxyhttp.HTTPOptions{
BasePath: basepath,
})
Expand Down Expand Up @@ -145,8 +144,8 @@ func NewGroupcacheWithConfig(logger log.Logger, reg prometheus.Registerer, conf
return err
}

if isTSDBChunkFile(parsedData.Name) {
return dest.UnmarshalBinary(finalAttrs, time.Now().Add(ChunkObjectAttrsTTL))
if cacheconfig.IsTSDBChunkFile(parsedData.Name) {
return dest.UnmarshalBinary(finalAttrs, time.Now().Add(cacheConfig.ChunkObjectAttrsTTL))
}
panic("caching bucket layer must not call on unconfigured paths")
case cachekey.IterVerb:
Expand All @@ -163,8 +162,8 @@ func NewGroupcacheWithConfig(logger log.Logger, reg prometheus.Registerer, conf
return err
}

if isBlocksRootDir(parsedData.Name) {
return dest.UnmarshalBinary(encodedList, time.Now().Add(BlocksIterTTL))
if cacheconfig.IsBlocksRootDir(parsedData.Name) {
return dest.UnmarshalBinary(encodedList, time.Now().Add(cacheConfig.BlocksIterTTL))
}
panic("caching bucket layer must not call on unconfigured paths")
case cachekey.ContentVerb:
Expand All @@ -179,8 +178,8 @@ func NewGroupcacheWithConfig(logger log.Logger, reg prometheus.Registerer, conf
return err
}

if isMetaFile(parsedData.Name) {
return dest.UnmarshalBinary(b, time.Now().Add(MetafileContentTTL))
if cacheconfig.IsMetaFile(parsedData.Name) {
return dest.UnmarshalBinary(b, time.Now().Add(cacheConfig.MetafileContentTTL))
}
panic("caching bucket layer must not call on unconfigured paths")

Expand All @@ -190,11 +189,11 @@ func NewGroupcacheWithConfig(logger log.Logger, reg prometheus.Registerer, conf
return err
}

if isMetaFile(parsedData.Name) {
if cacheconfig.IsMetaFile(parsedData.Name) {
if exists {
return dest.UnmarshalBinary([]byte(strconv.FormatBool(exists)), time.Now().Add(MetaFileExistsTTL))
return dest.UnmarshalBinary([]byte(strconv.FormatBool(exists)), time.Now().Add(cacheConfig.MetafileExistsTTL))
} else {
return dest.UnmarshalBinary([]byte(strconv.FormatBool(exists)), time.Now().Add(MetafileDoesntExistTTL))
return dest.UnmarshalBinary([]byte(strconv.FormatBool(exists)), time.Now().Add(cacheConfig.MetafileDoesntExistTTL))
}
}
panic("caching bucket layer must not call on unconfigured paths")
Expand All @@ -211,8 +210,8 @@ func NewGroupcacheWithConfig(logger log.Logger, reg prometheus.Registerer, conf
return err
}

if isTSDBChunkFile(parsedData.Name) {
return dest.UnmarshalBinary(b, time.Now().Add(ChunkSubrangeTTL))
if cacheconfig.IsTSDBChunkFile(parsedData.Name) {
return dest.UnmarshalBinary(b, time.Now().Add(cacheConfig.ChunkSubrangeTTL))
}
panic("caching bucket layer must not call on unconfigured paths")

Expand Down
64 changes: 64 additions & 0 deletions pkg/store/cache/cacheconfig/cacheconfig.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,64 @@
// Copyright (c) The Thanos Authors.
// Licensed under the Apache License 2.0.

package cacheconfig

import (
"regexp"
"strings"
"time"

"github.com/thanos-io/thanos/pkg/block/metadata"
"github.com/thanos-io/thanos/pkg/model"
)

// BucketCacheProvider is a type used to evaluate all bucket cache providers.
type BucketCacheProvider string

// CachingWithBackendConfig is a configuration of caching bucket used by Store component.
type CachingWithBackendConfig struct {
Type BucketCacheProvider `yaml:"type"`
BackendConfig interface{} `yaml:"config"`

// Basic unit used to cache chunks.
ChunkSubrangeSize int64 `yaml:"chunk_subrange_size"`

// Maximum number of GetRange requests issued by this bucket for single GetRange call. Zero or negative value = unlimited.
MaxChunksGetRangeRequests int `yaml:"max_chunks_get_range_requests"`

MetafileMaxSize model.Bytes `yaml:"metafile_max_size"`

// TTLs for various cache items.
ChunkObjectAttrsTTL time.Duration `yaml:"chunk_object_attrs_ttl"`
ChunkSubrangeTTL time.Duration `yaml:"chunk_subrange_ttl"`

// How long to cache result of Iter call in root directory.
BlocksIterTTL time.Duration `yaml:"blocks_iter_ttl"`

// Config for Exists and Get operations for metadata files.
MetafileExistsTTL time.Duration `yaml:"metafile_exists_ttl"`
MetafileDoesntExistTTL time.Duration `yaml:"metafile_doesnt_exist_ttl"`
MetafileContentTTL time.Duration `yaml:"metafile_content_ttl"`
}

func (cfg *CachingWithBackendConfig) Defaults() {
cfg.ChunkSubrangeSize = 16000 // Equal to max chunk size.
cfg.ChunkObjectAttrsTTL = 24 * time.Hour
cfg.ChunkSubrangeTTL = 24 * time.Hour
cfg.MaxChunksGetRangeRequests = 3
cfg.BlocksIterTTL = 5 * time.Minute
cfg.MetafileExistsTTL = 2 * time.Hour
cfg.MetafileDoesntExistTTL = 15 * time.Minute
cfg.MetafileContentTTL = 24 * time.Hour
cfg.MetafileMaxSize = 1024 * 1024 // Equal to default MaxItemSize in memcached client.
}

var chunksMatcher = regexp.MustCompile(`^.*/chunks/\d+$`)

func IsTSDBChunkFile(name string) bool { return chunksMatcher.MatchString(name) }

func IsMetaFile(name string) bool {
return strings.HasSuffix(name, "/"+metadata.MetaFilename) || strings.HasSuffix(name, "/"+metadata.DeletionMarkFilename)
}

func IsBlocksRootDir(name string) bool { return name == "" }
78 changes: 11 additions & 67 deletions pkg/store/cache/caching_bucket_factory.go
Original file line number Diff line number Diff line change
Expand Up @@ -4,9 +4,7 @@
package storecache

import (
"regexp"
"strings"
"time"

"github.com/go-kit/log"
"github.com/go-kit/log/level"
Expand All @@ -15,66 +13,27 @@ import (
"github.com/prometheus/common/route"
"gopkg.in/yaml.v2"

"github.com/thanos-io/thanos/pkg/block/metadata"
cache "github.com/thanos-io/thanos/pkg/cache"
"github.com/thanos-io/thanos/pkg/cacheutil"
"github.com/thanos-io/thanos/pkg/model"
"github.com/thanos-io/thanos/pkg/objstore"
"github.com/thanos-io/thanos/pkg/store/cache/cacheconfig"
)

// BucketCacheProvider is a type used to evaluate all bucket cache providers.
type BucketCacheProvider string

const (
InMemoryBucketCacheProvider BucketCacheProvider = "IN-MEMORY" // In-memory cache-provider for caching bucket.
MemcachedBucketCacheProvider BucketCacheProvider = "MEMCACHED" // Memcached cache-provider for caching bucket.
RedisBucketCacheProvider BucketCacheProvider = "REDIS" // Redis cache-provider for caching bucket.
GroupcacheBucketCacheProvider BucketCacheProvider = "GROUPCACHE" // Groupcache cache-provider for caching bucket.
InMemoryBucketCacheProvider cacheconfig.BucketCacheProvider = "IN-MEMORY" // In-memory cache-provider for caching bucket.
MemcachedBucketCacheProvider cacheconfig.BucketCacheProvider = "MEMCACHED" // Memcached cache-provider for caching bucket.
RedisBucketCacheProvider cacheconfig.BucketCacheProvider = "REDIS" // Redis cache-provider for caching bucket.
GroupcacheBucketCacheProvider cacheconfig.BucketCacheProvider = "GROUPCACHE" // Groupcache cache-provider for caching bucket.
)

// CachingWithBackendConfig is a configuration of caching bucket used by Store component.
type CachingWithBackendConfig struct {
Type BucketCacheProvider `yaml:"type"`
BackendConfig interface{} `yaml:"config"`

// Basic unit used to cache chunks.
ChunkSubrangeSize int64 `yaml:"chunk_subrange_size"`

// Maximum number of GetRange requests issued by this bucket for single GetRange call. Zero or negative value = unlimited.
MaxChunksGetRangeRequests int `yaml:"max_chunks_get_range_requests"`

MetafileMaxSize model.Bytes `yaml:"metafile_max_size"`

// TTLs for various cache items.
ChunkObjectAttrsTTL time.Duration `yaml:"chunk_object_attrs_ttl"`
ChunkSubrangeTTL time.Duration `yaml:"chunk_subrange_ttl"`

// How long to cache result of Iter call in root directory.
BlocksIterTTL time.Duration `yaml:"blocks_iter_ttl"`

// Config for Exists and Get operations for metadata files.
MetafileExistsTTL time.Duration `yaml:"metafile_exists_ttl"`
MetafileDoesntExistTTL time.Duration `yaml:"metafile_doesnt_exist_ttl"`
MetafileContentTTL time.Duration `yaml:"metafile_content_ttl"`
}

func (cfg *CachingWithBackendConfig) Defaults() {
cfg.ChunkSubrangeSize = 16000 // Equal to max chunk size.
cfg.ChunkObjectAttrsTTL = 24 * time.Hour
cfg.ChunkSubrangeTTL = 24 * time.Hour
cfg.MaxChunksGetRangeRequests = 3
cfg.BlocksIterTTL = 5 * time.Minute
cfg.MetafileExistsTTL = 2 * time.Hour
cfg.MetafileDoesntExistTTL = 15 * time.Minute
cfg.MetafileContentTTL = 24 * time.Hour
cfg.MetafileMaxSize = 1024 * 1024 // Equal to default MaxItemSize in memcached client.
}

// NewCachingBucketFromYaml uses YAML configuration to create new caching bucket.
func NewCachingBucketFromYaml(yamlContent []byte, bucket objstore.Bucket, logger log.Logger, reg prometheus.Registerer, r *route.Router) (objstore.InstrumentedBucket, error) {
level.Info(logger).Log("msg", "loading caching bucket configuration")

config := &CachingWithBackendConfig{}
config := &cacheconfig.CachingWithBackendConfig{}
config.Defaults()

if err := yaml.UnmarshalStrict(yamlContent, config); err != nil {
Expand Down Expand Up @@ -104,10 +63,7 @@ func NewCachingBucketFromYaml(yamlContent []byte, bucket objstore.Bucket, logger
case string(GroupcacheBucketCacheProvider):
const basePath = "/_galaxycache/"

c, err = cache.NewGroupcache(logger, reg, backendConfig, basePath, r, bucket,
isTSDBChunkFile, isMetaFile, isBlocksRootDir,
config.MetafileExistsTTL, config.MetafileDoesntExistTTL, config.MetafileContentTTL,
config.ChunkObjectAttrsTTL, config.ChunkSubrangeTTL, config.BlocksIterTTL)
c, err = cache.NewGroupcache(logger, reg, backendConfig, basePath, r, bucket, config)
if err != nil {
return nil, errors.Wrap(err, "failed to create groupcache")
}
Expand All @@ -127,12 +83,12 @@ func NewCachingBucketFromYaml(yamlContent []byte, bucket objstore.Bucket, logger
cfg := NewCachingBucketConfig()

// Configure cache.
cfg.CacheGetRange("chunks", c, isTSDBChunkFile, config.ChunkSubrangeSize, config.ChunkObjectAttrsTTL, config.ChunkSubrangeTTL, config.MaxChunksGetRangeRequests)
cfg.CacheExists("meta.jsons", c, isMetaFile, config.MetafileExistsTTL, config.MetafileDoesntExistTTL)
cfg.CacheGet("meta.jsons", c, isMetaFile, int(config.MetafileMaxSize), config.MetafileContentTTL, config.MetafileExistsTTL, config.MetafileDoesntExistTTL)
cfg.CacheGetRange("chunks", c, cacheconfig.IsBlocksRootDir, config.ChunkSubrangeSize, config.ChunkObjectAttrsTTL, config.ChunkSubrangeTTL, config.MaxChunksGetRangeRequests)
cfg.CacheExists("meta.jsons", c, cacheconfig.IsMetaFile, config.MetafileExistsTTL, config.MetafileDoesntExistTTL)
cfg.CacheGet("meta.jsons", c, cacheconfig.IsMetaFile, int(config.MetafileMaxSize), config.MetafileContentTTL, config.MetafileExistsTTL, config.MetafileDoesntExistTTL)

// Cache Iter requests for root.
cfg.CacheIter("blocks-iter", c, isBlocksRootDir, config.BlocksIterTTL, JSONIterCodec{})
cfg.CacheIter("blocks-iter", c, cacheconfig.IsBlocksRootDir, config.BlocksIterTTL, JSONIterCodec{})

cb, err := NewCachingBucket(bucket, cfg, logger, reg)
if err != nil {
Expand All @@ -141,15 +97,3 @@ func NewCachingBucketFromYaml(yamlContent []byte, bucket objstore.Bucket, logger

return cb, nil
}

var chunksMatcher = regexp.MustCompile(`^.*/chunks/\d+$`)

func isTSDBChunkFile(name string) bool { return chunksMatcher.MatchString(name) }

func isMetaFile(name string) bool {
return strings.HasSuffix(name, "/"+metadata.MetaFilename) || strings.HasSuffix(name, "/"+metadata.DeletionMarkFilename)
}

func isBlocksRootDir(name string) bool {
return name == ""
}
3 changes: 2 additions & 1 deletion pkg/store/cache/caching_bucket_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@ import (

"github.com/thanos-io/thanos/pkg/objstore"
"github.com/thanos-io/thanos/pkg/runutil"
"github.com/thanos-io/thanos/pkg/store/cache/cacheconfig"
"github.com/thanos-io/thanos/pkg/store/cache/cachekey"
"github.com/thanos-io/thanos/pkg/testutil"
)
Expand Down Expand Up @@ -237,7 +238,7 @@ func TestChunksCaching(t *testing.T) {
}

cfg := NewCachingBucketConfig()
cfg.CacheGetRange("chunks", cache, isTSDBChunkFile, subrangeSize, time.Hour, time.Hour, tc.maxGetRangeRequests)
cfg.CacheGetRange("chunks", cache, cacheconfig.IsTSDBChunkFile, subrangeSize, time.Hour, time.Hour, tc.maxGetRangeRequests)

cachingBucket, err := NewCachingBucket(inmem, cfg, nil, nil)
testutil.Ok(t, err)
Expand Down

0 comments on commit a5e287a

Please sign in to comment.