make boltdb-shipper use the generic indexshipper for managing index in object storage (#6226)

pull/6250/head
Sandeep Sukhani 3 years ago committed by GitHub
parent 67f104a783
commit 6ac955cc7d
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
  1. 4
      pkg/logcli/query/query.go
  2. 15
      pkg/loki/modules.go
  3. 12
      pkg/loki/modules_test.go
  4. 16
      pkg/storage/chunk/client/local/boltdb_index_client.go
  5. 8
      pkg/storage/factory.go
  6. 7
      pkg/storage/store.go
  7. 2
      pkg/storage/store_test.go
  8. 18
      pkg/storage/stores/indexshipper/downloads/table_manager.go
  9. 2
      pkg/storage/stores/indexshipper/downloads/table_manager_test.go
  10. 2
      pkg/storage/stores/indexshipper/gatewayclient/gateway_client.go
  11. 24
      pkg/storage/stores/indexshipper/gatewayclient/gateway_client_test.go
  12. 2
      pkg/storage/stores/indexshipper/gatewayclient/index_gateway_grpc_pool.go
  13. 43
      pkg/storage/stores/indexshipper/shipper.go
  14. 4
      pkg/storage/stores/shipper/compactor/deletion/delete_requests_table.go
  15. 6
      pkg/storage/stores/shipper/compactor/deletion/delete_requests_table_test.go
  16. 15
      pkg/storage/stores/shipper/compactor/retention/util_test.go
  17. 411
      pkg/storage/stores/shipper/downloads/index_set.go
  18. 117
      pkg/storage/stores/shipper/downloads/index_set_test.go
  19. 39
      pkg/storage/stores/shipper/downloads/metrics.go
  20. 373
      pkg/storage/stores/shipper/downloads/table.go
  21. 395
      pkg/storage/stores/shipper/downloads/table_manager.go
  22. 333
      pkg/storage/stores/shipper/downloads/table_manager_test.go
  23. 631
      pkg/storage/stores/shipper/downloads/table_test.go
  24. 59
      pkg/storage/stores/shipper/downloads/util.go
  25. 133
      pkg/storage/stores/shipper/index/indexfile/indexfile.go
  26. 13
      pkg/storage/stores/shipper/index/metrics.go
  27. 72
      pkg/storage/stores/shipper/index/querier.go
  28. 243
      pkg/storage/stores/shipper/index/table.go
  29. 98
      pkg/storage/stores/shipper/index/table_manager.go
  30. 78
      pkg/storage/stores/shipper/index/table_manager_test.go
  31. 454
      pkg/storage/stores/shipper/index/table_test.go
  32. 213
      pkg/storage/stores/shipper/shipper_index_client.go
  33. 99
      pkg/storage/stores/shipper/testutil/testutil.go
  34. 546
      pkg/storage/stores/shipper/uploads/table_test.go
  35. 10
      pkg/storage/stores/shipper/util/queries.go
  36. 6
      pkg/storage/stores/shipper/util/queries_test.go
  37. 1
      pkg/storage/stores/tsdb/store.go

@ -26,7 +26,7 @@ import (
"github.com/grafana/loki/pkg/logqlmodel/stats"
"github.com/grafana/loki/pkg/loki"
"github.com/grafana/loki/pkg/storage"
"github.com/grafana/loki/pkg/storage/stores/shipper"
"github.com/grafana/loki/pkg/storage/stores/indexshipper"
"github.com/grafana/loki/pkg/util/cfg"
util_log "github.com/grafana/loki/pkg/util/log"
"github.com/grafana/loki/pkg/util/marshal"
@ -189,7 +189,7 @@ func (q *Query) DoLocalQuery(out output.LogOutput, statistics bool, orgID string
return err
}
cm := storage.NewClientMetrics()
conf.StorageConfig.BoltDBShipperConfig.Mode = shipper.ModeReadOnly
conf.StorageConfig.BoltDBShipperConfig.Mode = indexshipper.ModeReadOnly
querier, err := storage.NewStore(conf.StorageConfig, conf.ChunkStoreConfig, conf.SchemaConfig, limits, cm, prometheus.DefaultRegisterer, util_log.Logger)
if err != nil {

@ -49,13 +49,12 @@ import (
"github.com/grafana/loki/pkg/storage/config"
"github.com/grafana/loki/pkg/storage/stores/indexshipper"
"github.com/grafana/loki/pkg/storage/stores/series/index"
"github.com/grafana/loki/pkg/storage/stores/shipper"
"github.com/grafana/loki/pkg/storage/stores/shipper/compactor"
"github.com/grafana/loki/pkg/storage/stores/shipper/compactor/deletion"
"github.com/grafana/loki/pkg/storage/stores/shipper/compactor/generationnumber"
shipper_index "github.com/grafana/loki/pkg/storage/stores/shipper/index"
"github.com/grafana/loki/pkg/storage/stores/shipper/indexgateway"
"github.com/grafana/loki/pkg/storage/stores/shipper/indexgateway/indexgatewaypb"
"github.com/grafana/loki/pkg/storage/stores/shipper/uploads"
"github.com/grafana/loki/pkg/usagestats"
"github.com/grafana/loki/pkg/util/httpreq"
util_log "github.com/grafana/loki/pkg/util/log"
@ -429,7 +428,7 @@ func (t *Loki) initStore() (_ services.Service, err error) {
t.Cfg.Ingester.RetainPeriod = t.Cfg.StorageConfig.IndexCacheValidity + 1*time.Minute
// We do not want ingester to unnecessarily keep downloading files
t.Cfg.StorageConfig.BoltDBShipperConfig.Mode = shipper.ModeWriteOnly
t.Cfg.StorageConfig.BoltDBShipperConfig.Mode = indexshipper.ModeWriteOnly
t.Cfg.StorageConfig.BoltDBShipperConfig.IngesterDBRetainPeriod = shipperQuerierIndexUpdateDelay(t.Cfg.StorageConfig.IndexCacheValidity, t.Cfg.StorageConfig.BoltDBShipperConfig.ResyncInterval)
t.Cfg.StorageConfig.TSDBShipperConfig.Mode = indexshipper.ModeWriteOnly
@ -437,10 +436,10 @@ func (t *Loki) initStore() (_ services.Service, err error) {
case t.Cfg.isModuleEnabled(Querier), t.Cfg.isModuleEnabled(Ruler), t.Cfg.isModuleEnabled(Read), t.isModuleActive(IndexGateway):
// We do not want query to do any updates to index
t.Cfg.StorageConfig.BoltDBShipperConfig.Mode = shipper.ModeReadOnly
t.Cfg.StorageConfig.BoltDBShipperConfig.Mode = indexshipper.ModeReadOnly
t.Cfg.StorageConfig.TSDBShipperConfig.Mode = indexshipper.ModeReadOnly
default:
t.Cfg.StorageConfig.BoltDBShipperConfig.Mode = shipper.ModeReadWrite
t.Cfg.StorageConfig.BoltDBShipperConfig.Mode = indexshipper.ModeReadWrite
t.Cfg.StorageConfig.BoltDBShipperConfig.IngesterDBRetainPeriod = shipperQuerierIndexUpdateDelay(t.Cfg.StorageConfig.IndexCacheValidity, t.Cfg.StorageConfig.BoltDBShipperConfig.ResyncInterval)
t.Cfg.StorageConfig.TSDBShipperConfig.Mode = indexshipper.ModeReadWrite
t.Cfg.StorageConfig.TSDBShipperConfig.IngesterDBRetainPeriod = shipperQuerierIndexUpdateDelay(t.Cfg.StorageConfig.IndexCacheValidity, t.Cfg.StorageConfig.TSDBShipperConfig.ResyncInterval)
@ -886,7 +885,7 @@ func (t *Loki) initIndexGatewayRing() (_ services.Service, err error) {
return
}
t.Cfg.StorageConfig.BoltDBShipperConfig.Mode = shipper.ModeReadOnly
t.Cfg.StorageConfig.BoltDBShipperConfig.Mode = indexshipper.ModeReadOnly
t.Cfg.IndexGateway.Ring.ListenPort = t.Cfg.Server.GRPCListenPort
managerMode := indexgateway.ClientMode
@ -974,7 +973,7 @@ func (t *Loki) deleteRequestsStore() (deletion.DeleteRequestsStore, error) {
}
func calculateMaxLookBack(pc config.PeriodConfig, maxLookBackConfig, minDuration time.Duration) (time.Duration, error) {
if pc.ObjectType != shipper.FilesystemObjectStoreType && maxLookBackConfig.Nanoseconds() != 0 {
if pc.ObjectType != indexshipper.FilesystemObjectStoreType && maxLookBackConfig.Nanoseconds() != 0 {
return 0, errors.New("it is an error to specify a non zero `query_store_max_look_back_period` value when using any object store other than `filesystem`")
}
@ -1013,7 +1012,7 @@ func shipperQuerierIndexUpdateDelay(cacheValidity, resyncInterval time.Duration)
// shipperIngesterIndexUploadDelay returns duration it could take for an index file containing id of a chunk to be uploaded to the shared store since it got flushed.
func shipperIngesterIndexUploadDelay() time.Duration {
return uploads.ShardDBsByDuration + shipper.UploadInterval
return shipper_index.ShardDBsByDuration + indexshipper.UploadInterval
}
// shipperMinIngesterQueryStoreDuration returns minimum duration(with some buffer) ingesters should query their stores to

@ -14,6 +14,7 @@ import (
"github.com/grafana/loki/pkg/storage"
"github.com/grafana/loki/pkg/storage/chunk/client/local"
"github.com/grafana/loki/pkg/storage/config"
"github.com/grafana/loki/pkg/storage/stores/indexshipper"
"github.com/grafana/loki/pkg/storage/stores/shipper"
)
@ -170,10 +171,13 @@ func TestMultiKVSetup(t *testing.T) {
cfg.StorageConfig = storage.Config{
FSConfig: local.FSConfig{Directory: dir},
BoltDBShipperConfig: shipper.Config{
SharedStoreType: config.StorageTypeFileSystem,
ActiveIndexDirectory: dir,
CacheLocation: dir,
Mode: shipper.ModeWriteOnly},
Config: indexshipper.Config{
SharedStoreType: config.StorageTypeFileSystem,
ActiveIndexDirectory: dir,
CacheLocation: dir,
Mode: indexshipper.ModeWriteOnly,
},
},
}
cfg.Ruler.Config.StoreConfig.Type = config.StorageTypeLocal
cfg.Ruler.Config.StoreConfig.Local.Directory = dir

@ -128,6 +128,10 @@ func (b *BoltIndexClient) Stop() {
}
func (b *BoltIndexClient) NewWriteBatch() index.WriteBatch {
return NewWriteBatch()
}
func NewWriteBatch() index.WriteBatch {
return &BoltWriteBatch{
Writes: map[string]TableWrites{},
}
@ -171,7 +175,7 @@ func (b *BoltIndexClient) GetDB(name string, operation int) (*bbolt.DB, error) {
return db, nil
}
func (b *BoltIndexClient) WriteToDB(_ context.Context, db *bbolt.DB, bucketName []byte, writes TableWrites) error {
func WriteToDB(_ context.Context, db *bbolt.DB, bucketName []byte, writes TableWrites) error {
return db.Update(func(tx *bbolt.Tx) error {
var b *bbolt.Bucket
if len(bucketName) == 0 {
@ -215,7 +219,7 @@ func (b *BoltIndexClient) BatchWrite(ctx context.Context, batch index.WriteBatch
return err
}
err = b.WriteToDB(ctx, db, IndexBucketName, writes)
err = WriteToDB(ctx, db, IndexBucketName, writes)
if err != nil {
return err
}
@ -238,10 +242,10 @@ func (b *BoltIndexClient) query(ctx context.Context, query index.Query, callback
return err
}
return b.QueryDB(ctx, db, IndexBucketName, query, callback)
return QueryDB(ctx, db, IndexBucketName, query, callback)
}
func (b *BoltIndexClient) QueryDB(ctx context.Context, db *bbolt.DB, bucketName []byte, query index.Query,
func QueryDB(ctx context.Context, db *bbolt.DB, bucketName []byte, query index.Query,
callback index.QueryPagesCallback,
) error {
return db.View(func(tx *bbolt.Tx) error {
@ -253,11 +257,11 @@ func (b *BoltIndexClient) QueryDB(ctx context.Context, db *bbolt.DB, bucketName
return nil
}
return b.QueryWithCursor(ctx, bucket.Cursor(), query, callback)
return QueryWithCursor(ctx, bucket.Cursor(), query, callback)
})
}
func (b *BoltIndexClient) QueryWithCursor(_ context.Context, c *bbolt.Cursor, query index.Query, callback index.QueryPagesCallback) error {
func QueryWithCursor(_ context.Context, c *bbolt.Cursor, query index.Query, callback index.QueryPagesCallback) error {
batch := batchPool.Get().(*cursorBatch)
defer batchPool.Put(batch)

@ -25,9 +25,10 @@ import (
"github.com/grafana/loki/pkg/storage/chunk/client/testutils"
"github.com/grafana/loki/pkg/storage/config"
"github.com/grafana/loki/pkg/storage/stores/indexshipper"
"github.com/grafana/loki/pkg/storage/stores/indexshipper/downloads"
"github.com/grafana/loki/pkg/storage/stores/indexshipper/gatewayclient"
"github.com/grafana/loki/pkg/storage/stores/series/index"
"github.com/grafana/loki/pkg/storage/stores/shipper"
"github.com/grafana/loki/pkg/storage/stores/shipper/downloads"
util_log "github.com/grafana/loki/pkg/util/log"
)
@ -120,6 +121,9 @@ func (cfg *Config) Validate() error {
if err := cfg.BoltDBShipperConfig.Validate(); err != nil {
return errors.Wrap(err, "invalid boltdb-shipper config")
}
if err := cfg.TSDBShipperConfig.Validate(); err != nil {
return errors.Wrap(err, "invalid tsdb config")
}
return nil
}
@ -157,7 +161,7 @@ func NewIndexClient(name string, cfg Config, schemaCfg config.SchemaConfig, limi
}
if shouldUseBoltDBIndexGatewayClient(cfg) {
gateway, err := shipper.NewGatewayClient(cfg.BoltDBShipperConfig.IndexGatewayClientConfig, registerer, util_log.Logger)
gateway, err := gatewayclient.NewGatewayClient(cfg.BoltDBShipperConfig.IndexGatewayClientConfig, registerer, util_log.Logger)
if err != nil {
return nil, err
}

@ -23,9 +23,10 @@ import (
"github.com/grafana/loki/pkg/storage/chunk/fetcher"
"github.com/grafana/loki/pkg/storage/config"
"github.com/grafana/loki/pkg/storage/stores"
"github.com/grafana/loki/pkg/storage/stores/indexshipper"
"github.com/grafana/loki/pkg/storage/stores/indexshipper/gatewayclient"
"github.com/grafana/loki/pkg/storage/stores/series"
"github.com/grafana/loki/pkg/storage/stores/series/index"
"github.com/grafana/loki/pkg/storage/stores/shipper"
"github.com/grafana/loki/pkg/storage/stores/shipper/indexgateway"
"github.com/grafana/loki/pkg/storage/stores/tsdb"
"github.com/grafana/loki/pkg/usagestats"
@ -186,7 +187,7 @@ func (s *store) chunkClientForPeriod(p config.PeriodConfig) (client.Client, erro
}
func shouldUseBoltDBIndexGatewayClient(cfg Config) bool {
if cfg.BoltDBShipperConfig.Mode != shipper.ModeReadOnly || cfg.BoltDBShipperConfig.IndexGatewayClientConfig.Disabled {
if cfg.BoltDBShipperConfig.Mode != indexshipper.ModeReadOnly || cfg.BoltDBShipperConfig.IndexGatewayClientConfig.Disabled {
return false
}
@ -217,7 +218,7 @@ func (s *store) storeForPeriod(p config.PeriodConfig, chunkClient client.Client,
// ToDo(Sandeep): Refactor code to not use boltdb-shipper index gateway client config
if shouldUseBoltDBIndexGatewayClient(s.cfg) {
// inject the index-gateway client into the index store
gw, err := shipper.NewGatewayClient(s.cfg.BoltDBShipperConfig.IndexGatewayClientConfig, indexClientReg, s.logger)
gw, err := gatewayclient.NewGatewayClient(s.cfg.BoltDBShipperConfig.IndexGatewayClientConfig, indexClientReg, s.logger)
if err != nil {
return nil, nil, nil, err
}

@ -25,6 +25,7 @@ import (
"github.com/grafana/loki/pkg/storage/chunk"
"github.com/grafana/loki/pkg/storage/chunk/client/local"
"github.com/grafana/loki/pkg/storage/config"
"github.com/grafana/loki/pkg/storage/stores/indexshipper"
"github.com/grafana/loki/pkg/storage/stores/shipper"
util_log "github.com/grafana/loki/pkg/util/log"
"github.com/grafana/loki/pkg/util/marshal"
@ -1000,6 +1001,7 @@ func TestStore_MultipleBoltDBShippersInConfig(t *testing.T) {
boltdbShipperConfig.ActiveIndexDirectory = path.Join(tempDir, "index")
boltdbShipperConfig.SharedStoreType = "filesystem"
boltdbShipperConfig.CacheLocation = path.Join(tempDir, "boltdb-shipper-cache")
boltdbShipperConfig.Mode = indexshipper.ModeReadWrite
// dates for activation of boltdb shippers
firstStoreDate := parseDate("2019-01-01")

@ -7,7 +7,6 @@ import (
"path/filepath"
"regexp"
"strconv"
"strings"
"sync"
"time"
@ -30,6 +29,11 @@ type Limits interface {
DefaultLimits() *validation.Limits
}
// IndexGatewayOwnsTenant is invoked by an IndexGateway instance and answers whether if the given tenant is assigned to this instance or not.
//
// It is only relevant by an IndexGateway in the ring mode and if it returns false for a given tenant, that tenant will be ignored by this IndexGateway during query readiness.
type IndexGatewayOwnsTenant func(tenant string) bool
type TableManager interface {
Stop()
ForEach(ctx context.Context, tableName, userID string, callback index.ForEachIndexCallback) error
@ -54,9 +58,11 @@ type tableManager struct {
ctx context.Context
cancel context.CancelFunc
wg sync.WaitGroup
ownsTenant IndexGatewayOwnsTenant
}
func NewTableManager(cfg Config, openIndexFileFunc index.OpenIndexFileFunc, indexStorageClient storage.Client) (TableManager, error) {
func NewTableManager(cfg Config, openIndexFileFunc index.OpenIndexFileFunc, indexStorageClient storage.Client, ownsTenantFn IndexGatewayOwnsTenant) (TableManager, error) {
if err := util.EnsureDirectory(cfg.CacheDir); err != nil {
return nil, err
}
@ -66,6 +72,7 @@ func NewTableManager(cfg Config, openIndexFileFunc index.OpenIndexFileFunc, inde
cfg: cfg,
openIndexFileFunc: openIndexFileFunc,
indexStorageClient: indexStorageClient,
ownsTenant: ownsTenantFn,
tables: make(map[string]Table),
ctx: ctx,
cancel: cancel,
@ -292,8 +299,7 @@ func (tm *tableManager) ensureQueryReadiness(ctx context.Context) error {
if err := table.EnsureQueryReadiness(ctx, usersToBeQueryReadyFor); err != nil {
return err
}
joinedUsers := strings.Join(usersToBeQueryReadyFor, ",")
level.Info(util_log.Logger).Log("msg", "index pre-download for query readiness completed", "users", joinedUsers, "duration", time.Since(perTableStart), "table", tableName)
level.Info(util_log.Logger).Log("msg", "index pre-download for query readiness completed", "users_len", len(usersToBeQueryReadyFor), "duration", time.Since(perTableStart), "table", tableName)
}
return nil
@ -317,6 +323,10 @@ func (tm *tableManager) findUsersInTableForQueryReadiness(tableNumber int64, use
continue
}
if tm.ownsTenant != nil && !tm.ownsTenant(userID) {
continue
}
if activeTableNumber-tableNumber <= int64(queryReadyNumDays) {
usersToBeQueryReadyFor = append(usersToBeQueryReadyFor, userID)
}

@ -42,7 +42,7 @@ func buildTestTableManager(t *testing.T, path string) (*tableManager, stopFunc)
}
tblManager, err := NewTableManager(cfg, func(s string) (index.Index, error) {
return openMockIndexFile(t, s), nil
}, indexStorageClient)
}, indexStorageClient, nil)
require.NoError(t, err)
return tblManager.(*tableManager), func() {

@ -1,37 +1,23 @@
package shipper
package gatewayclient
import (
"context"
"errors"
"fmt"
"net"
"os"
"path/filepath"
"strconv"
"sync"
"testing"
"time"
"github.com/grafana/dskit/flagext"
"github.com/prometheus/client_golang/prometheus"
"github.com/stretchr/testify/require"
"github.com/weaveworks/common/middleware"
"github.com/weaveworks/common/user"
"google.golang.org/grpc"
"google.golang.org/grpc/credentials/insecure"
"google.golang.org/grpc/test/bufconn"
"github.com/grafana/loki/pkg/storage/chunk/client/local"
"github.com/grafana/loki/pkg/storage/stores/series/index"
"github.com/grafana/loki/pkg/storage/stores/shipper/downloads"
"github.com/grafana/loki/pkg/storage/stores/shipper/indexgateway"
"github.com/grafana/loki/pkg/storage/stores/shipper/indexgateway/indexgatewaypb"
"github.com/grafana/loki/pkg/storage/stores/shipper/storage"
"github.com/grafana/loki/pkg/storage/stores/shipper/testutil"
"github.com/grafana/loki/pkg/storage/stores/shipper/util"
util_log "github.com/grafana/loki/pkg/util/log"
util_math "github.com/grafana/loki/pkg/util/math"
"github.com/grafana/loki/pkg/validation"
)
const (
@ -47,8 +33,8 @@ const (
valuePrefix = "value"
// the number of index entries for benchmarking will be divided amongst numTables
benchMarkNumEntries = 1000000
numTables = 50
//benchMarkNumEntries = 1000000
//numTables = 50
)
type mockIndexGatewayServer struct {
@ -165,6 +151,8 @@ func TestGatewayClient(t *testing.T) {
require.Equal(t, len(queries), numCallbacks)
}
/*
ToDo(Sandeep): Comment out benchmark code for now to fix circular dependency
func buildTableName(i int) string {
return fmt.Sprintf("%s%d", tableNamePrefix, i)
}
@ -304,7 +292,7 @@ func Benchmark_QueriesMatchingLargeNumOfRows(b *testing.B) {
})
}
benchmarkIndexQueries(b, queries)
}
}*/
func TestDoubleRegistration(t *testing.T) {
r := prometheus.NewRegistry()

@ -11,9 +11,11 @@ import (
"github.com/grafana/loki/pkg/storage/chunk/client"
"github.com/grafana/loki/pkg/storage/stores/indexshipper/downloads"
"github.com/grafana/loki/pkg/storage/stores/indexshipper/gatewayclient"
"github.com/grafana/loki/pkg/storage/stores/indexshipper/index"
"github.com/grafana/loki/pkg/storage/stores/indexshipper/uploads"
"github.com/grafana/loki/pkg/storage/stores/shipper/storage"
shipper_util "github.com/grafana/loki/pkg/storage/stores/shipper/util"
util_log "github.com/grafana/loki/pkg/util/log"
)
@ -27,6 +29,9 @@ const (
// ModeWriteOnly is to allow only write operations
ModeWriteOnly = Mode("WO")
// FilesystemObjectStoreType holds the periodic config type for the filesystem store
FilesystemObjectStoreType = "filesystem"
// UploadInterval defines interval for when we check if there are new index files to upload.
// It's also used to snapshot the currently written index tables so the snapshots can be used for reads.
UploadInterval = 1 * time.Minute
@ -47,13 +52,14 @@ type IndexShipper interface {
}
type Config struct {
ActiveIndexDirectory string `yaml:"active_index_directory"`
SharedStoreType string `yaml:"shared_store"`
SharedStoreKeyPrefix string `yaml:"shared_store_key_prefix"`
CacheLocation string `yaml:"cache_location"`
CacheTTL time.Duration `yaml:"cache_ttl"`
ResyncInterval time.Duration `yaml:"resync_interval"`
QueryReadyNumDays int `yaml:"query_ready_num_days"`
ActiveIndexDirectory string `yaml:"active_index_directory"`
SharedStoreType string `yaml:"shared_store"`
SharedStoreKeyPrefix string `yaml:"shared_store_key_prefix"`
CacheLocation string `yaml:"cache_location"`
CacheTTL time.Duration `yaml:"cache_ttl"`
ResyncInterval time.Duration `yaml:"resync_interval"`
QueryReadyNumDays int `yaml:"query_ready_num_days"`
IndexGatewayClientConfig gatewayclient.IndexGatewayClientConfig `yaml:"index_gateway_client"`
IngesterName string
Mode Mode
@ -62,6 +68,8 @@ type Config struct {
// RegisterFlagsWithPrefix registers flags.
func (cfg *Config) RegisterFlagsWithPrefix(prefix string, f *flag.FlagSet) {
cfg.IndexGatewayClientConfig.RegisterFlagsWithPrefix(prefix+".shipper.index-gateway-client", f)
f.StringVar(&cfg.ActiveIndexDirectory, prefix+".shipper.active-index-directory", "", "Directory where ingesters would write index files which would then be uploaded by shipper to configured storage")
f.StringVar(&cfg.SharedStoreType, prefix+".shipper.shared-store", "", "Shared store for keeping index files. Supported types: gcs, s3, azure, filesystem")
f.StringVar(&cfg.SharedStoreKeyPrefix, prefix+".shipper.shared-store.key-prefix", "index/", "Prefix to add to Object Keys in Shared store. Path separator(if any) should always be a '/'. Prefix should never start with a separator but should always end with it")
@ -71,6 +79,14 @@ func (cfg *Config) RegisterFlagsWithPrefix(prefix string, f *flag.FlagSet) {
f.IntVar(&cfg.QueryReadyNumDays, prefix+".shipper.query-ready-num-days", 0, "Number of days of common index to be kept downloaded for queries. For per tenant index query readiness, use limits overrides config.")
}
func (cfg *Config) Validate() error {
// set the default value for mode
if cfg.Mode == "" {
cfg.Mode = ModeReadWrite
}
return shipper_util.ValidateSharedStoreKeyPrefix(cfg.SharedStoreKeyPrefix)
}
type indexShipper struct {
cfg Config
openIndexFileFunc index.OpenIndexFileFunc
@ -82,13 +98,18 @@ type indexShipper struct {
// NewIndexShipper creates a shipper for providing index store functionality using index files and object storage.
// It manages the whole life cycle of uploading the index and downloading the index at query time.
func NewIndexShipper(cfg Config, storageClient client.ObjectClient, limits downloads.Limits, open index.OpenIndexFileFunc) (IndexShipper, error) {
func NewIndexShipper(cfg Config, storageClient client.ObjectClient, limits downloads.Limits, ownsTenantFn downloads.IndexGatewayOwnsTenant, open index.OpenIndexFileFunc) (IndexShipper, error) {
switch cfg.Mode {
case ModeReadOnly, ModeWriteOnly, ModeReadWrite:
default:
return nil, fmt.Errorf("invalid mode: %v", cfg.Mode)
}
shipper := indexShipper{
cfg: cfg,
openIndexFileFunc: open,
}
err := shipper.init(storageClient, limits)
err := shipper.init(storageClient, limits, ownsTenantFn)
if err != nil {
return nil, err
}
@ -98,7 +119,7 @@ func NewIndexShipper(cfg Config, storageClient client.ObjectClient, limits downl
return &shipper, nil
}
func (s *indexShipper) init(storageClient client.ObjectClient, limits downloads.Limits) error {
func (s *indexShipper) init(storageClient client.ObjectClient, limits downloads.Limits, ownsTenantFn downloads.IndexGatewayOwnsTenant) error {
indexStorageClient := storage.NewIndexStorageClient(storageClient, s.cfg.SharedStoreKeyPrefix)
if s.cfg.Mode != ModeReadOnly {
@ -122,7 +143,7 @@ func (s *indexShipper) init(storageClient client.ObjectClient, limits downloads.
QueryReadyNumDays: s.cfg.QueryReadyNumDays,
Limits: limits,
}
downloadsManager, err := downloads.NewTableManager(cfg, s.openIndexFileFunc, indexStorageClient)
downloadsManager, err := downloads.NewTableManager(cfg, s.openIndexFileFunc, indexStorageClient, ownsTenantFn)
if err != nil {
return err
}

@ -172,7 +172,7 @@ func (t *deleteRequestsTable) BatchWrite(ctx context.Context, batch index.WriteB
}
for _, tableWrites := range boltWriteBatch.Writes {
if err := t.boltdbIndexClient.WriteToDB(ctx, t.db, local.IndexBucketName, tableWrites); err != nil {
if err := local.WriteToDB(ctx, t.db, local.IndexBucketName, tableWrites); err != nil {
return err
}
}
@ -182,7 +182,7 @@ func (t *deleteRequestsTable) BatchWrite(ctx context.Context, batch index.WriteB
func (t *deleteRequestsTable) QueryPages(ctx context.Context, queries []index.Query, callback index.QueryPagesCallback) error {
for _, query := range queries {
if err := t.boltdbIndexClient.QueryDB(ctx, t.db, local.IndexBucketName, query, callback); err != nil {
if err := local.QueryDB(ctx, t.db, local.IndexBucketName, query, callback); err != nil {
return err
}
}

@ -40,7 +40,7 @@ func TestDeleteRequestsTable(t *testing.T) {
require.NoError(t, testDeleteRequestsTable.BatchWrite(context.Background(), batch))
// see if right records were written
testutil.TestSingleDBQuery(t, index.Query{}, testDeleteRequestsTable.db, local.IndexBucketName, testDeleteRequestsTable.boltdbIndexClient, 0, 10)
testutil.VerifySingleIndexFile(t, index.Query{}, testDeleteRequestsTable.db, local.IndexBucketName, 0, 10)
// upload the file to the storage
require.NoError(t, testDeleteRequestsTable.uploadFile())
@ -73,7 +73,7 @@ func TestDeleteRequestsTable(t *testing.T) {
require.NotEmpty(t, testDeleteRequestsTable.dbPath)
// validate records in local db
testutil.TestSingleDBQuery(t, index.Query{}, testDeleteRequestsTable.db, local.IndexBucketName, testDeleteRequestsTable.boltdbIndexClient, 0, 20)
testutil.VerifySingleIndexFile(t, index.Query{}, testDeleteRequestsTable.db, local.IndexBucketName, 0, 20)
}
func checkRecordsInStorage(t *testing.T, storageFilePath string, start, numRecords int) {
@ -93,5 +93,5 @@ func checkRecordsInStorage(t *testing.T, storageFilePath string, start, numRecor
defer boltdbIndexClient.Stop()
testutil.TestSingleDBQuery(t, index.Query{}, tempDB, local.IndexBucketName, boltdbIndexClient, start, numRecords)
testutil.VerifySingleIndexFile(t, index.Query{}, tempDB, local.IndexBucketName, start, numRecords)
}

@ -24,6 +24,7 @@ import (
"github.com/grafana/loki/pkg/storage/chunk/client/local"
chunk_util "github.com/grafana/loki/pkg/storage/chunk/client/util"
"github.com/grafana/loki/pkg/storage/config"
"github.com/grafana/loki/pkg/storage/stores/indexshipper"
"github.com/grafana/loki/pkg/storage/stores/shipper"
shipper_util "github.com/grafana/loki/pkg/storage/stores/shipper/util"
util_log "github.com/grafana/loki/pkg/util/log"
@ -259,12 +260,14 @@ func newTestStore(t testing.TB, clientMetrics storage.ClientMetrics) *testStore
MaxParallelGetChunk: 150,
BoltDBShipperConfig: shipper.Config{
ActiveIndexDirectory: indexDir,
SharedStoreType: "filesystem",
SharedStoreKeyPrefix: "index",
ResyncInterval: 1 * time.Millisecond,
IngesterName: "foo",
Mode: shipper.ModeReadWrite,
Config: indexshipper.Config{
ActiveIndexDirectory: indexDir,
SharedStoreType: "filesystem",
SharedStoreKeyPrefix: "index",
ResyncInterval: 1 * time.Millisecond,
IngesterName: "foo",
Mode: indexshipper.ModeReadWrite,
},
},
}

@ -1,411 +0,0 @@
package downloads
import (
"context"
"fmt"
"io"
"io/ioutil"
"os"
"path/filepath"
"sync"
"time"
"github.com/go-kit/log"
"github.com/go-kit/log/level"
"github.com/grafana/dskit/concurrency"
"github.com/grafana/dskit/tenant"
"go.etcd.io/bbolt"
"github.com/grafana/loki/pkg/storage/chunk/client/local"
chunk_util "github.com/grafana/loki/pkg/storage/chunk/client/util"
"github.com/grafana/loki/pkg/storage/stores/series/index"
"github.com/grafana/loki/pkg/storage/stores/shipper/storage"
shipper_util "github.com/grafana/loki/pkg/storage/stores/shipper/util"
util_log "github.com/grafana/loki/pkg/util/log"
"github.com/grafana/loki/pkg/util/spanlogger"
)
var errIndexListCacheTooStale = fmt.Errorf("index list cache too stale")
type IndexSet interface {
Init(forQuerying bool) error
Close()
MultiQueries(ctx context.Context, queries []index.Query, callback index.QueryPagesCallback) error
DropAllDBs() error
Err() error
LastUsedAt() time.Time
UpdateLastUsedAt()
Sync(ctx context.Context) (err error)
AwaitReady(ctx context.Context) error
}
// indexSet is a collection of multiple files created for a same table by various ingesters.
// All the public methods are concurrency safe and take care of mutexes to avoid any data race.
type indexSet struct {
baseIndexSet storage.IndexSet
tableName, userID string
cacheLocation string
boltDBIndexClient BoltDBIndexClient
logger log.Logger
lastUsedAt time.Time
dbs map[string]*bbolt.DB
dbsMtx *mtxWithReadiness
err error
cancelFunc context.CancelFunc // helps with cancellation of initialization if we are asked to stop.
}
func NewIndexSet(tableName, userID, cacheLocation string, baseIndexSet storage.IndexSet,
boltDBIndexClient BoltDBIndexClient, logger log.Logger) (IndexSet, error) {
if baseIndexSet.IsUserBasedIndexSet() && userID == "" {
return nil, fmt.Errorf("userID must not be empty")
} else if !baseIndexSet.IsUserBasedIndexSet() && userID != "" {
return nil, fmt.Errorf("userID must be empty")
}
err := chunk_util.EnsureDirectory(cacheLocation)
if err != nil {
return nil, err
}
is := indexSet{
baseIndexSet: baseIndexSet,
tableName: tableName,
userID: userID,
cacheLocation: cacheLocation,
boltDBIndexClient: boltDBIndexClient,
logger: logger,
lastUsedAt: time.Now(),
dbs: map[string]*bbolt.DB{},
dbsMtx: newMtxWithReadiness(),
cancelFunc: func() {},
}
return &is, nil
}
// Init downloads all the db files for the table from object storage.
func (t *indexSet) Init(forQuerying bool) (err error) {
// Using background context to avoid cancellation of download when request times out.
// We would anyways need the files for serving next requests.
ctx, cancelFunc := context.WithTimeout(context.Background(), downloadTimeout)
t.cancelFunc = cancelFunc
logger := spanlogger.FromContextWithFallback(ctx, t.logger)
defer func() {
if err != nil {
level.Error(t.logger).Log("msg", "failed to initialize table, cleaning it up", "err", err)
t.err = err
// cleaning up files due to error to avoid returning invalid results.
for fileName := range t.dbs {
if err := t.cleanupDB(fileName); err != nil {
level.Error(t.logger).Log("msg", "failed to cleanup partially downloaded file", "filename", fileName, "err", err)
}
}
}
t.cancelFunc()
t.dbsMtx.markReady()
}()
filesInfo, err := ioutil.ReadDir(t.cacheLocation)
if err != nil {
return err
}
// open all the locally present files first to avoid downloading them again during sync operation below.
for _, fileInfo := range filesInfo {
if fileInfo.IsDir() {
continue
}
fullPath := filepath.Join(t.cacheLocation, fileInfo.Name())
// if we fail to open a boltdb file, lets skip it and let sync operation re-download the file from storage.
boltdb, err := shipper_util.SafeOpenBoltdbFile(fullPath)
if err != nil {
level.Error(util_log.Logger).Log("msg", fmt.Sprintf("failed to open existing boltdb file %s, removing the file and continuing without it to let the sync operation catch up", fullPath), "err", err)
// Sometimes files get corrupted when the process gets killed in the middle of a download operation which causes boltdb client to panic.
// We already recover the panic but the lock on the file is not released by boltdb client which causes the reopening of the file to fail when the sync operation tries it.
// We want to remove the file failing to open to get rid of the lock.
if err := os.Remove(fullPath); err != nil {
level.Error(util_log.Logger).Log("msg", fmt.Sprintf("failed to remove boltdb file %s which failed to open", fullPath))
}
continue
}
t.dbs[fileInfo.Name()] = boltdb
}
level.Debug(logger).Log("msg", fmt.Sprintf("opened %d local files, now starting sync operation", len(t.dbs)))
// sync the table to get new files and remove the deleted ones from storage.
err = t.sync(ctx, false, forQuerying)
if err != nil {
return
}
level.Debug(logger).Log("msg", "finished syncing files")
return
}
// Close Closes references to all the dbs.
func (t *indexSet) Close() {
// stop the initialization if it is still ongoing.
t.cancelFunc()
err := t.dbsMtx.lock(context.Background())
if err != nil {
level.Error(t.logger).Log("msg", "failed to acquire lock for closing dbs", "err", err)
return
}
defer t.dbsMtx.unlock()
for name, db := range t.dbs {
if err := db.Close(); err != nil {
level.Error(t.logger).Log("msg", fmt.Sprintf("failed to close file %s", name), "err", err)
}
}
t.dbs = map[string]*bbolt.DB{}
}
// MultiQueries runs multiple queries without having to take lock multiple times for each query.
func (t *indexSet) MultiQueries(ctx context.Context, queries []index.Query, callback index.QueryPagesCallback) error {
userID, err := tenant.TenantID(ctx)
if err != nil {
return err
}
userIDBytes := shipper_util.GetUnsafeBytes(userID)
err = t.dbsMtx.rLock(ctx)
if err != nil {
return err
}
defer t.dbsMtx.rUnlock()
if t.err != nil {
return t.err
}
t.lastUsedAt = time.Now()
logger := util_log.WithContext(ctx, t.logger)
level.Debug(logger).Log("query-count", len(queries), "dbs-count", len(t.dbs))
for name, db := range t.dbs {
err := db.View(func(tx *bbolt.Tx) error {
bucket := tx.Bucket(userIDBytes)
if bucket == nil {
bucket = tx.Bucket(local.IndexBucketName)
if bucket == nil {
return nil
}
}
for _, query := range queries {
if err := t.boltDBIndexClient.QueryWithCursor(ctx, bucket.Cursor(), query, callback); err != nil {
return err
}
}
return nil
})
if err != nil {
return err
}
level.Debug(logger).Log("queried-db", name)
}
return nil
}
// DropAllDBs closes reference to all the open dbs and removes the local files.
func (t *indexSet) DropAllDBs() error {
err := t.dbsMtx.lock(context.Background())
if err != nil {
return err
}
defer t.dbsMtx.unlock()
for fileName := range t.dbs {
if err := t.cleanupDB(fileName); err != nil {
return err
}
}
return os.RemoveAll(t.cacheLocation)
}
// Err returns the err which is usually set when there was any issue in Init.
func (t *indexSet) Err() error {
return t.err
}
// LastUsedAt returns the time at which table was last used for querying.
func (t *indexSet) LastUsedAt() time.Time {
return t.lastUsedAt
}
func (t *indexSet) UpdateLastUsedAt() {
t.lastUsedAt = time.Now()
}
// cleanupDB closes and removes the local file.
func (t *indexSet) cleanupDB(fileName string) error {
df, ok := t.dbs[fileName]
if !ok {
return fmt.Errorf("file %s not found in files collection for cleaning up", fileName)
}
filePath := df.Path()
if err := df.Close(); err != nil {
return err
}
delete(t.dbs, fileName)
return os.Remove(filePath)
}
func (t *indexSet) Sync(ctx context.Context) (err error) {
return t.sync(ctx, true, false)
}
// sync downloads updated and new files from the storage relevant for the table and removes the deleted ones
func (t *indexSet) sync(ctx context.Context, lock, bypassListCache bool) (err error) {
level.Debug(t.logger).Log("msg", "syncing index files")
toDownload, toDelete, err := t.checkStorageForUpdates(ctx, lock, bypassListCache)
if err != nil {
return err
}
level.Debug(t.logger).Log("msg", "index sync updates", "toDownload", fmt.Sprint(toDownload), "toDelete", fmt.Sprint(toDelete))
downloadedFiles, err := t.doConcurrentDownload(ctx, toDownload)
if err != nil {
return err
}
// if we did not bypass list cache and skipped downloading all the new files due to them being removed by compaction,
// it means the cache is not valid anymore since compaction would have happened after last index list cache refresh.
// Let us return error to ask the caller to re-run the sync after the list cache refresh.
if !bypassListCache && len(downloadedFiles) == 0 && len(toDownload) > 0 {
level.Error(t.logger).Log("msg", "we skipped downloading all the new files, possibly removed by compaction", "files", toDownload)
return errIndexListCacheTooStale
}
if lock {
err = t.dbsMtx.lock(ctx)
if err != nil {
return err
}
defer t.dbsMtx.unlock()
}
for _, fileName := range downloadedFiles {
filePath := filepath.Join(t.cacheLocation, fileName)
boltdb, err := shipper_util.SafeOpenBoltdbFile(filePath)
if err != nil {
return err
}
t.dbs[fileName] = boltdb
}
for _, db := range toDelete {
err := t.cleanupDB(db)
if err != nil {
return err
}
}
return nil
}
// checkStorageForUpdates compares files from cache with storage and builds the list of files to be downloaded from storage and to be deleted from cache
func (t *indexSet) checkStorageForUpdates(ctx context.Context, lock, bypassListCache bool) (toDownload []storage.IndexFile, toDelete []string, err error) {
// listing tables from store
var files []storage.IndexFile
files, err = t.baseIndexSet.ListFiles(ctx, t.tableName, t.userID, bypassListCache)
if err != nil {
return
}
listedDBs := make(map[string]struct{}, len(files))
if lock {
err = t.dbsMtx.rLock(ctx)
if err != nil {
return nil, nil, err
}
defer t.dbsMtx.rUnlock()
}
for _, file := range files {
listedDBs[file.Name] = struct{}{}
// Checking whether file was already downloaded, if not, download it.
// We do not ever upload files in the object store with the same name but different contents so we do not consider downloading modified files again.
_, ok := t.dbs[file.Name]
if !ok {
toDownload = append(toDownload, file)
}
}
for db := range t.dbs {
if _, isOK := listedDBs[db]; !isOK {
toDelete = append(toDelete, db)
}
}
return
}
func (t *indexSet) AwaitReady(ctx context.Context) error {
return t.dbsMtx.awaitReady(ctx)
}
func (t *indexSet) downloadFileFromStorage(ctx context.Context, fileName, folderPathForTable string) error {
return shipper_util.DownloadFileFromStorage(filepath.Join(folderPathForTable, fileName), shipper_util.IsCompressedFile(fileName),
true, shipper_util.LoggerWithFilename(t.logger, fileName), func() (io.ReadCloser, error) {
return t.baseIndexSet.GetFile(ctx, t.tableName, t.userID, fileName)
})
}
// doConcurrentDownload downloads objects(files) concurrently. It ignores only missing file errors caused by removal of file by compaction.
// It returns the names of the files downloaded successfully and leaves it upto the caller to open those files.
func (t *indexSet) doConcurrentDownload(ctx context.Context, files []storage.IndexFile) ([]string, error) {
downloadedFiles := make([]string, 0, len(files))
downloadedFilesMtx := sync.Mutex{}
err := concurrency.ForEachJob(ctx, len(files), maxDownloadConcurrency, func(ctx context.Context, idx int) error {
fileName := files[idx].Name
err := t.downloadFileFromStorage(ctx, fileName, t.cacheLocation)
if err != nil {
if t.baseIndexSet.IsFileNotFoundErr(err) {
level.Info(util_log.Logger).Log("msg", fmt.Sprintf("ignoring missing file %s, possibly removed during compaction", fileName))
return nil
}
return err
}
downloadedFilesMtx.Lock()
downloadedFiles = append(downloadedFiles, fileName)
downloadedFilesMtx.Unlock()
return nil
})
if err != nil {
return nil, err
}
return downloadedFiles, nil
}

@ -1,117 +0,0 @@
package downloads
import (
"fmt"
"io/ioutil"
"os"
"path/filepath"
"testing"
"github.com/stretchr/testify/require"
"github.com/grafana/loki/pkg/storage/stores/series/index"
"github.com/grafana/loki/pkg/storage/stores/shipper/storage"
"github.com/grafana/loki/pkg/storage/stores/shipper/testutil"
util_log "github.com/grafana/loki/pkg/util/log"
)
const tableName = "test"
func buildTestIndexSet(t *testing.T, userID, path string) (*indexSet, stopFunc) {
boltDBIndexClient, storageClient := buildTestClients(t, path)
cachePath := filepath.Join(path, cacheDirName)
baseIndexSet := storage.NewIndexSet(storageClient, userID != "")
idxSet, err := NewIndexSet(tableName, userID, filepath.Join(cachePath, tableName, userID), baseIndexSet, boltDBIndexClient, util_log.Logger)
require.NoError(t, err)
require.NoError(t, idxSet.Init(false))
return idxSet.(*indexSet), func() {
idxSet.Close()
boltDBIndexClient.Stop()
}
}
func TestIndexSet_Init(t *testing.T) {
tempDir := t.TempDir()
objectStoragePath := filepath.Join(tempDir, objectsStorageDirName)
testDBs := map[string]testutil.DBConfig{}
checkIndexSet := func() {
indexSet, stopFunc := buildTestIndexSet(t, userID, tempDir)
require.Len(t, indexSet.dbs, len(testDBs))
testutil.TestSingleTableQuery(t, userID, []index.Query{{}}, indexSet, 0, len(testDBs)*10)
stopFunc()
}
// check index set without any local files and in storage
checkIndexSet()
// setup some dbs in object storage
for i := 0; i < 10; i++ {
testDBs[fmt.Sprint(i)] = testutil.DBConfig{
CompressFile: i%2 == 0,
DBRecords: testutil.DBRecords{
Start: i * 10,
NumRecords: 10,
},
}
}
testutil.SetupDBsAtPath(t, filepath.Join(objectStoragePath, tableName, userID), testDBs, nil)
// check index set twice; first run to have new files to download, second run to test with no changes in storage.
for i := 0; i < 2; i++ {
checkIndexSet()
}
// change a boltdb file to text file which would fail to open.
indexSetPathPathInCache := filepath.Join(tempDir, cacheDirName, tableName, userID)
require.NoError(t, ioutil.WriteFile(filepath.Join(indexSetPathPathInCache, "0"), []byte("invalid boltdb file"), 0o666))
// check index set with a corrupt file which should get downloaded again from storage
checkIndexSet()
// delete a file from storage which should get removed from local as well
indexSetPathPathInStorage := filepath.Join(objectStoragePath, tableName, userID)
require.NoError(t, os.Remove(filepath.Join(indexSetPathPathInStorage, "9")))
delete(testDBs, "9")
checkIndexSet()
}
func TestIndexSet_doConcurrentDownload(t *testing.T) {
tempDir := t.TempDir()
objectStoragePath := filepath.Join(tempDir, objectsStorageDirName)
for _, tc := range []int{0, 10, maxDownloadConcurrency, maxDownloadConcurrency * 2} {
t.Run(fmt.Sprintf("%d dbs", tc), func(t *testing.T) {
userID := fmt.Sprint(tc)
testDBs := map[string]testutil.DBConfig{}
for i := 0; i < tc; i++ {
testDBs[fmt.Sprint(i)] = testutil.DBConfig{
CompressFile: i%2 == 0,
DBRecords: testutil.DBRecords{
Start: i * 10,
NumRecords: 10,
},
}
}
testutil.SetupDBsAtPath(t, filepath.Join(objectStoragePath, tableName, userID), testDBs, nil)
indexSet, stopFunc := buildTestIndexSet(t, userID, tempDir)
defer func() {
stopFunc()
}()
// ensure that we have `tc` number of files downloaded and opened.
if tc > 0 {
require.Len(t, indexSet.dbs, tc)
}
testutil.TestSingleTableQuery(t, userID, []index.Query{{}}, indexSet, 0, tc*10)
})
}
}

@ -1,39 +0,0 @@
package downloads
import (
"github.com/prometheus/client_golang/prometheus"
"github.com/prometheus/client_golang/prometheus/promauto"
)
const (
statusFailure = "failure"
statusSuccess = "success"
)
type metrics struct {
queryTimeTableDownloadDurationSeconds *prometheus.CounterVec
tablesSyncOperationTotal *prometheus.CounterVec
tablesDownloadOperationDurationSeconds prometheus.Gauge
}
func newMetrics(r prometheus.Registerer) *metrics {
m := &metrics{
queryTimeTableDownloadDurationSeconds: promauto.With(r).NewCounterVec(prometheus.CounterOpts{
Namespace: "loki_boltdb_shipper",
Name: "query_time_table_download_duration_seconds",
Help: "Time (in seconds) spent in downloading of files per table at query time",
}, []string{"table"}),
tablesSyncOperationTotal: promauto.With(r).NewCounterVec(prometheus.CounterOpts{
Namespace: "loki_boltdb_shipper",
Name: "tables_sync_operation_total",
Help: "Total number of tables sync operations done by status",
}, []string{"status"}),
tablesDownloadOperationDurationSeconds: promauto.With(r).NewGauge(prometheus.GaugeOpts{
Namespace: "loki_boltdb_shipper",
Name: "tables_download_operation_duration_seconds",
Help: "Time (in seconds) spent in downloading updated files for all the tables",
}),
}
return m
}

@ -1,373 +0,0 @@
package downloads
import (
"context"
"fmt"
"io/ioutil"
"path/filepath"
"sync"
"time"
"github.com/go-kit/log"
"github.com/go-kit/log/level"
"github.com/grafana/dskit/concurrency"
"github.com/grafana/dskit/tenant"
"github.com/pkg/errors"
"go.etcd.io/bbolt"
chunk_util "github.com/grafana/loki/pkg/storage/chunk/client/util"
"github.com/grafana/loki/pkg/storage/stores/series/index"
"github.com/grafana/loki/pkg/storage/stores/shipper/storage"
util_log "github.com/grafana/loki/pkg/util/log"
"github.com/grafana/loki/pkg/util/spanlogger"
)
// timeout for downloading initial files for a table to avoid leaking resources by allowing it to take all the time.
const (
downloadTimeout = 5 * time.Minute
maxDownloadConcurrency = 50
)
type BoltDBIndexClient interface {
QueryWithCursor(_ context.Context, c *bbolt.Cursor, query index.Query, callback index.QueryPagesCallback) error
}
type Table interface {
Close()
MultiQueries(ctx context.Context, queries []index.Query, callback index.QueryPagesCallback) error
DropUnusedIndex(ttl time.Duration, now time.Time) (bool, error)
Sync(ctx context.Context) error
EnsureQueryReadiness(ctx context.Context, userIDs []string) error
}
// table is a collection of multiple files created for a same table by various ingesters.
// All the public methods are concurrency safe and take care of mutexes to avoid any data race.
type table struct {
name string
cacheLocation string
metrics *metrics
storageClient storage.Client
boltDBIndexClient BoltDBIndexClient
baseUserIndexSet, baseCommonIndexSet storage.IndexSet
logger log.Logger
indexSets map[string]IndexSet
indexSetsMtx sync.RWMutex
}
// NewTable just creates an instance of table without trying to load files from local storage or object store.
// It is used for initializing table at query time.
func NewTable(name, cacheLocation string, storageClient storage.Client, boltDBIndexClient BoltDBIndexClient, metrics *metrics) Table {
table := table{
name: name,
cacheLocation: cacheLocation,
metrics: metrics,
storageClient: storageClient,
baseUserIndexSet: storage.NewIndexSet(storageClient, true),
baseCommonIndexSet: storage.NewIndexSet(storageClient, false),
logger: log.With(util_log.Logger, "table-name", name),
boltDBIndexClient: boltDBIndexClient,
indexSets: map[string]IndexSet{},
}
return &table
}
// LoadTable loads a table from local storage(syncs the table too if we have it locally) or downloads it from the shared store.
// It is used for loading and initializing table at startup. It would initialize index sets which already had files locally.
func LoadTable(name, cacheLocation string, storageClient storage.Client, boltDBIndexClient BoltDBIndexClient, metrics *metrics) (Table, error) {
err := chunk_util.EnsureDirectory(cacheLocation)
if err != nil {
return nil, err
}
filesInfo, err := ioutil.ReadDir(cacheLocation)
if err != nil {
return nil, err
}
table := table{
name: name,
cacheLocation: cacheLocation,
metrics: metrics,
storageClient: storageClient,
baseUserIndexSet: storage.NewIndexSet(storageClient, true),
baseCommonIndexSet: storage.NewIndexSet(storageClient, false),
logger: log.With(util_log.Logger, "table-name", name),
boltDBIndexClient: boltDBIndexClient,
indexSets: map[string]IndexSet{},
}
level.Debug(table.logger).Log("msg", fmt.Sprintf("opening locally present files for table %s", name), "files", fmt.Sprint(filesInfo))
// common index files are outside the directories and user index files are in the directories
for _, fileInfo := range filesInfo {
if !fileInfo.IsDir() {
continue
}
userID := fileInfo.Name()
userIndexSet, err := NewIndexSet(name, userID, filepath.Join(cacheLocation, userID), table.baseUserIndexSet, boltDBIndexClient, loggerWithUserID(table.logger, userID))
if err != nil {
return nil, err
}
err = userIndexSet.Init(false)
if err != nil {
return nil, err
}
table.indexSets[userID] = userIndexSet
}
commonIndexSet, err := NewIndexSet(name, "", cacheLocation, table.baseCommonIndexSet, boltDBIndexClient, table.logger)
if err != nil {
return nil, err
}
err = commonIndexSet.Init(false)
if err != nil {
return nil, err
}
table.indexSets[""] = commonIndexSet
return &table, nil
}
// Close Closes references to all the dbs.
func (t *table) Close() {
t.indexSetsMtx.Lock()
defer t.indexSetsMtx.Unlock()
for _, userIndexSet := range t.indexSets {
userIndexSet.Close()
}
t.indexSets = map[string]IndexSet{}
}
// MultiQueries runs multiple queries without having to take lock multiple times for each query.
func (t *table) MultiQueries(ctx context.Context, queries []index.Query, callback index.QueryPagesCallback) error {
userID, err := tenant.TenantID(ctx)
if err != nil {
return err
}
// query both user and common index
for _, uid := range []string{userID, ""} {
indexSet, err := t.getOrCreateIndexSet(ctx, uid, true)
if err != nil {
return err
}
if indexSet.Err() != nil {
level.Error(util_log.WithContext(ctx, t.logger)).Log("msg", fmt.Sprintf("index set %s has some problem, cleaning it up", uid), "err", indexSet.Err())
if err := indexSet.DropAllDBs(); err != nil {
level.Error(t.logger).Log("msg", fmt.Sprintf("failed to cleanup broken index set %s", uid), "err", err)
}
t.indexSetsMtx.Lock()
delete(t.indexSets, userID)
t.indexSetsMtx.Unlock()
return indexSet.Err()
}
err = indexSet.MultiQueries(ctx, queries, callback)
if err != nil {
return err
}
}
return nil
}
func (t *table) findExpiredIndexSets(ttl time.Duration, now time.Time) []string {
t.indexSetsMtx.RLock()
defer t.indexSetsMtx.RUnlock()
var expiredIndexSets []string
commonIndexSetExpired := false
for userID, userIndexSet := range t.indexSets {
lastUsedAt := userIndexSet.LastUsedAt()
if lastUsedAt.Add(ttl).Before(now) {
if userID == "" {
// add the userID for common index set at the end of the list to make sure it is the last one cleaned up
// because we remove directories containing the index sets which in case of common index is
// the parent directory of all the user index sets.
commonIndexSetExpired = true
} else {
expiredIndexSets = append(expiredIndexSets, userID)
}
}
}
// common index set should expire only after all the user index sets have expired.
if commonIndexSetExpired && len(expiredIndexSets) == len(t.indexSets)-1 {
expiredIndexSets = append(expiredIndexSets, "")
}
return expiredIndexSets
}
// DropUnusedIndex drops the index set if it has not been queried for at least ttl duration.
// It returns true if the whole table gets dropped.
func (t *table) DropUnusedIndex(ttl time.Duration, now time.Time) (bool, error) {
indexSetsToCleanup := t.findExpiredIndexSets(ttl, now)
if len(indexSetsToCleanup) > 0 {
t.indexSetsMtx.Lock()
defer t.indexSetsMtx.Unlock()
for _, userID := range indexSetsToCleanup {
// additional check for cleaning up the common index set when it is the only one left.
// This is just for safety because the index sets could change between findExpiredIndexSets and the actual cleanup.
if userID == "" && len(t.indexSets) != 1 {
level.Info(t.logger).Log("msg", "skipping cleanup of common index set because we possibly have unexpired user index sets left")
continue
}
level.Info(t.logger).Log("msg", fmt.Sprintf("cleaning up expired index set %s", userID))
err := t.indexSets[userID].DropAllDBs()
if err != nil {
return false, err
}
delete(t.indexSets, userID)
}
return len(t.indexSets) == 0, nil
}
return false, nil
}
// Sync downloads updated and new files from the storage relevant for the table and removes the deleted ones
func (t *table) Sync(ctx context.Context) error {
level.Debug(t.logger).Log("msg", fmt.Sprintf("syncing files for table %s", t.name))
t.indexSetsMtx.RLock()
defer t.indexSetsMtx.RUnlock()
for userID, indexSet := range t.indexSets {
if err := indexSet.Sync(ctx); err != nil {
if errors.Is(err, errIndexListCacheTooStale) {
level.Info(t.logger).Log("msg", "we have hit stale list cache, refreshing it and running sync again")
t.storageClient.RefreshIndexListCache(ctx)
err = indexSet.Sync(ctx)
if err == nil {
continue
}
}
return errors.Wrap(err, fmt.Sprintf("failed to sync index set %s for table %s", userID, t.name))
}
}
return nil
}
// getOrCreateIndexSet gets or creates the index set for the userID.
// If it does not exist, it creates a new one and initializes it in a goroutine.
// Caller can use IndexSet.AwaitReady() to wait until the IndexSet gets ready, if required.
// forQuerying must be set to true only getting the index for querying since
// it captures the amount of time it takes to download the index at query time.
func (t *table) getOrCreateIndexSet(ctx context.Context, id string, forQuerying bool) (IndexSet, error) {
t.indexSetsMtx.RLock()
indexSet, ok := t.indexSets[id]
t.indexSetsMtx.RUnlock()
if ok {
return indexSet, nil
}
t.indexSetsMtx.Lock()
defer t.indexSetsMtx.Unlock()
indexSet, ok = t.indexSets[id]
if ok {
return indexSet, nil
}
var err error
baseIndexSet := t.baseUserIndexSet
if id == "" {
baseIndexSet = t.baseCommonIndexSet
}
// instantiate the index set, add it to the map
indexSet, err = NewIndexSet(t.name, id, filepath.Join(t.cacheLocation, id), baseIndexSet, t.boltDBIndexClient, loggerWithUserID(t.logger, id))
if err != nil {
return nil, err
}
t.indexSets[id] = indexSet
// initialize the index set in async mode, it would be upto the caller to wait for its readiness using IndexSet.AwaitReady()
go func() {
if forQuerying {
start := time.Now()
defer func() {
duration := time.Since(start)
t.metrics.queryTimeTableDownloadDurationSeconds.WithLabelValues(t.name).Add(duration.Seconds())
logger := spanlogger.FromContextWithFallback(ctx, loggerWithUserID(t.logger, id))
level.Info(logger).Log("msg", "downloaded index set at query time", "duration", duration)
}()
}
err := indexSet.Init(forQuerying)
if err != nil {
level.Error(t.logger).Log("msg", fmt.Sprintf("failed to init user index set %s", id), "err", err)
}
}()
return indexSet, nil
}
// EnsureQueryReadiness ensures that we have downloaded the common index as well as user index for the provided userIDs.
// When ensuring query readiness for a table, we will always download common index set because it can include index for one of the provided user ids.
func (t *table) EnsureQueryReadiness(ctx context.Context, userIDs []string) error {
commonIndexSet, err := t.getOrCreateIndexSet(ctx, "", false)
if err != nil {
return err
}
err = commonIndexSet.AwaitReady(ctx)
if err != nil {
return err
}
commonIndexSet.UpdateLastUsedAt()
missingUserIDs := make([]string, 0, len(userIDs))
t.indexSetsMtx.RLock()
for _, userID := range userIDs {
if userIndexSet, ok := t.indexSets[userID]; !ok {
missingUserIDs = append(missingUserIDs, userID)
} else {
userIndexSet.UpdateLastUsedAt()
}
}
t.indexSetsMtx.RUnlock()
return t.downloadUserIndexes(ctx, missingUserIDs)
}
// downloadUserIndexes downloads user specific index files concurrently.
func (t *table) downloadUserIndexes(ctx context.Context, userIDs []string) error {
return concurrency.ForEachJob(ctx, len(userIDs), maxDownloadConcurrency, func(ctx context.Context, idx int) error {
indexSet, err := t.getOrCreateIndexSet(ctx, userIDs[idx], false)
if err != nil {
return err
}
return indexSet.AwaitReady(ctx)
})
}
func loggerWithUserID(logger log.Logger, userID string) log.Logger {
if userID == "" {
return logger
}
return log.With(logger, "user-id", userID)
}

@ -1,395 +0,0 @@
package downloads
import (
"context"
"fmt"
"io/ioutil"
"path/filepath"
"regexp"
"strconv"
"sync"
"time"
"github.com/go-kit/log/level"
"github.com/prometheus/client_golang/prometheus"
chunk_util "github.com/grafana/loki/pkg/storage/chunk/client/util"
"github.com/grafana/loki/pkg/storage/stores/series/index"
"github.com/grafana/loki/pkg/storage/stores/shipper/storage"
"github.com/grafana/loki/pkg/storage/stores/shipper/util"
util_log "github.com/grafana/loki/pkg/util/log"
"github.com/grafana/loki/pkg/validation"
)
const (
cacheCleanupInterval = time.Hour
durationDay = 24 * time.Hour
)
type Limits interface {
AllByUserID() map[string]*validation.Limits
DefaultLimits() *validation.Limits
}
// IndexGatewayOwnsTenant is invoked by an IndexGateway instance and answers whether if the given tenant is assigned to this instance or not.
//
// It is only relevant by an IndexGateway in the ring mode and if it returns false for a given tenant, that tenant will be ignored by this IndexGateway during query readiness.
type IndexGatewayOwnsTenant func(tenant string) bool
type Config struct {
CacheDir string
SyncInterval time.Duration
CacheTTL time.Duration
QueryReadyNumDays int
Limits Limits
}
type TableManager struct {
cfg Config
boltIndexClient BoltDBIndexClient
indexStorageClient storage.Client
tables map[string]Table
tablesMtx sync.RWMutex
metrics *metrics
ctx context.Context
cancel context.CancelFunc
wg sync.WaitGroup
ownsTenant IndexGatewayOwnsTenant
}
func NewTableManager(cfg Config, boltIndexClient BoltDBIndexClient, indexStorageClient storage.Client, ownsTenantFn IndexGatewayOwnsTenant, registerer prometheus.Registerer) (*TableManager, error) {
if err := chunk_util.EnsureDirectory(cfg.CacheDir); err != nil {
return nil, err
}
ctx, cancel := context.WithCancel(context.Background())
tm := &TableManager{
cfg: cfg,
boltIndexClient: boltIndexClient,
indexStorageClient: indexStorageClient,
ownsTenant: ownsTenantFn,
tables: make(map[string]Table),
metrics: newMetrics(registerer),
ctx: ctx,
cancel: cancel,
}
// load the existing tables first.
err := tm.loadLocalTables()
if err != nil {
// call Stop to close open file references.
tm.Stop()
return nil, err
}
// download the missing tables.
err = tm.ensureQueryReadiness(ctx)
if err != nil {
// call Stop to close open file references.
tm.Stop()
return nil, err
}
go tm.loop()
return tm, nil
}
func (tm *TableManager) loop() {
tm.wg.Add(1)
defer tm.wg.Done()
syncTicker := time.NewTicker(tm.cfg.SyncInterval)
defer syncTicker.Stop()
cacheCleanupTicker := time.NewTicker(cacheCleanupInterval)
defer cacheCleanupTicker.Stop()
for {
select {
case <-syncTicker.C:
err := tm.syncTables(tm.ctx)
if err != nil {
level.Error(util_log.Logger).Log("msg", "error syncing local boltdb files with storage", "err", err)
}
// we need to keep ensuring query readiness to download every days new table which would otherwise be downloaded only during queries.
err = tm.ensureQueryReadiness(tm.ctx)
if err != nil {
level.Error(util_log.Logger).Log("msg", "error ensuring query readiness of tables", "err", err)
}
case <-cacheCleanupTicker.C:
err := tm.cleanupCache()
if err != nil {
level.Error(util_log.Logger).Log("msg", "error cleaning up expired tables", "err", err)
}
case <-tm.ctx.Done():
return
}
}
}
func (tm *TableManager) Stop() {
tm.cancel()
tm.wg.Wait()
tm.tablesMtx.Lock()
defer tm.tablesMtx.Unlock()
for _, table := range tm.tables {
table.Close()
}
}
func (tm *TableManager) QueryPages(ctx context.Context, queries []index.Query, callback index.QueryPagesCallback) error {
queriesByTable := util.QueriesByTable(queries)
for tableName, queries := range queriesByTable {
err := tm.query(ctx, tableName, queries, callback)
if err != nil {
return err
}
}
return nil
}
func (tm *TableManager) query(ctx context.Context, tableName string, queries []index.Query, callback index.QueryPagesCallback) error {
logger := util_log.WithContext(ctx, util_log.Logger)
level.Debug(logger).Log("table-name", tableName)
table, err := tm.getOrCreateTable(tableName)
if err != nil {
return err
}
return util.DoParallelQueries(ctx, table, queries, callback)
}
func (tm *TableManager) getOrCreateTable(tableName string) (Table, error) {
// if table is already there, use it.
tm.tablesMtx.RLock()
table, ok := tm.tables[tableName]
tm.tablesMtx.RUnlock()
if !ok {
tm.tablesMtx.Lock()
defer tm.tablesMtx.Unlock()
// check if some other competing goroutine got the lock before us and created the table, use it if so.
table, ok = tm.tables[tableName]
if !ok {
// table not found, creating one.
level.Info(util_log.Logger).Log("msg", fmt.Sprintf("downloading all files for table %s", tableName))
tablePath := filepath.Join(tm.cfg.CacheDir, tableName)
err := chunk_util.EnsureDirectory(tablePath)
if err != nil {
return nil, err
}
table = NewTable(tableName, filepath.Join(tm.cfg.CacheDir, tableName), tm.indexStorageClient, tm.boltIndexClient, tm.metrics)
tm.tables[tableName] = table
}
}
return table, nil
}
func (tm *TableManager) syncTables(ctx context.Context) error {
tm.tablesMtx.RLock()
defer tm.tablesMtx.RUnlock()
start := time.Now()
var err error
defer func() {
status := statusSuccess
if err != nil {
status = statusFailure
}
tm.metrics.tablesSyncOperationTotal.WithLabelValues(status).Inc()
tm.metrics.tablesDownloadOperationDurationSeconds.Set(time.Since(start).Seconds())
}()
level.Info(util_log.Logger).Log("msg", "syncing tables")
for _, table := range tm.tables {
err = table.Sync(ctx)
if err != nil {
return err
}
}
return nil
}
func (tm *TableManager) cleanupCache() error {
tm.tablesMtx.Lock()
defer tm.tablesMtx.Unlock()
level.Info(util_log.Logger).Log("msg", "cleaning tables cache")
for name, table := range tm.tables {
level.Info(util_log.Logger).Log("msg", fmt.Sprintf("cleaning up expired table %s", name))
isEmpty, err := table.DropUnusedIndex(tm.cfg.CacheTTL, time.Now())
if err != nil {
return err
}
if isEmpty {
delete(tm.tables, name)
}
}
return nil
}
// ensureQueryReadiness compares tables required for being query ready with the tables we already have and downloads the missing ones.
func (tm *TableManager) ensureQueryReadiness(ctx context.Context) error {
start := time.Now()
defer func() {
level.Info(util_log.Logger).Log("msg", "query readiness setup completed", "duration", time.Since(start))
}()
activeTableNumber := getActiveTableNumber()
// find the largest query readiness number
largestQueryReadinessNum := tm.cfg.QueryReadyNumDays
if defaultLimits := tm.cfg.Limits.DefaultLimits(); defaultLimits.QueryReadyIndexNumDays > largestQueryReadinessNum {
largestQueryReadinessNum = defaultLimits.QueryReadyIndexNumDays
}
queryReadinessNumByUserID := make(map[string]int)
for userID, limits := range tm.cfg.Limits.AllByUserID() {
if limits.QueryReadyIndexNumDays != 0 {
queryReadinessNumByUserID[userID] = limits.QueryReadyIndexNumDays
if limits.QueryReadyIndexNumDays > largestQueryReadinessNum {
largestQueryReadinessNum = limits.QueryReadyIndexNumDays
}
}
}
// return early if no table has to be downloaded for query readiness
if largestQueryReadinessNum == 0 {
return nil
}
tables, err := tm.indexStorageClient.ListTables(ctx)
if err != nil {
return err
}
// regex for finding daily tables which have a 5 digit number at the end.
re, err := regexp.Compile(`.+[0-9]{5}$`)
if err != nil {
return err
}
for _, tableName := range tables {
if !re.MatchString(tableName) {
continue
}
tableNumber, err := strconv.ParseInt(tableName[len(tableName)-5:], 10, 64)
if err != nil {
return err
}
// continue if the table is not within query readiness
if activeTableNumber-tableNumber > int64(largestQueryReadinessNum) {
continue
}
// list the users that have dedicated index files for this table
_, usersWithIndex, err := tm.indexStorageClient.ListFiles(ctx, tableName, false)
if err != nil {
return err
}
// find the users whos index we need to keep ready for querying from this table
usersToBeQueryReadyFor := tm.findUsersInTableForQueryReadiness(tableNumber, usersWithIndex, queryReadinessNumByUserID)
// continue if both user index and common index is not required to be downloaded for query readiness
if len(usersToBeQueryReadyFor) == 0 && activeTableNumber-tableNumber > int64(tm.cfg.QueryReadyNumDays) {
continue
}
table, err := tm.getOrCreateTable(tableName)
if err != nil {
return err
}
perTableStart := time.Now()
if err := table.EnsureQueryReadiness(ctx, usersToBeQueryReadyFor); err != nil {
return err
}
level.Info(util_log.Logger).Log("msg", "index pre-download for query readiness completed", "users_len", len(usersToBeQueryReadyFor), "duration", time.Since(perTableStart), "table", tableName)
}
return nil
}
// findUsersInTableForQueryReadiness returns the users that needs their index to be query ready based on the tableNumber and
// query readiness number provided per user
func (tm *TableManager) findUsersInTableForQueryReadiness(tableNumber int64, usersWithIndexInTable []string,
queryReadinessNumByUserID map[string]int,
) []string {
activeTableNumber := getActiveTableNumber()
usersToBeQueryReadyFor := []string{}
for _, userID := range usersWithIndexInTable {
// use the query readiness config for the user if it exists or use the default config
queryReadyNumDays, ok := queryReadinessNumByUserID[userID]
if !ok {
queryReadyNumDays = tm.cfg.Limits.DefaultLimits().QueryReadyIndexNumDays
}
if queryReadyNumDays == 0 {
continue
}
if tm.ownsTenant != nil && !tm.ownsTenant(userID) {
continue
}
if activeTableNumber-tableNumber <= int64(queryReadyNumDays) {
usersToBeQueryReadyFor = append(usersToBeQueryReadyFor, userID)
}
}
return usersToBeQueryReadyFor
}
// loadLocalTables loads tables present locally.
func (tm *TableManager) loadLocalTables() error {
filesInfo, err := ioutil.ReadDir(tm.cfg.CacheDir)
if err != nil {
return err
}
for _, fileInfo := range filesInfo {
if !fileInfo.IsDir() {
continue
}
level.Info(util_log.Logger).Log("msg", fmt.Sprintf("loading local table %s", fileInfo.Name()))
table, err := LoadTable(fileInfo.Name(), filepath.Join(tm.cfg.CacheDir, fileInfo.Name()), tm.indexStorageClient, tm.boltIndexClient, tm.metrics)
if err != nil {
return err
}
tm.tables[fileInfo.Name()] = table
}
return nil
}
func getActiveTableNumber() int64 {
periodSecs := int64(durationDay / time.Second)
return time.Now().Unix() / periodSecs
}

@ -1,333 +0,0 @@
package downloads
import (
"context"
"fmt"
"os"
"path/filepath"
"testing"
"time"
"github.com/stretchr/testify/require"
"github.com/grafana/loki/pkg/storage/stores/series/index"
"github.com/grafana/loki/pkg/storage/stores/shipper/storage"
"github.com/grafana/loki/pkg/storage/stores/shipper/testutil"
"github.com/grafana/loki/pkg/validation"
)
func buildTestTableManager(t *testing.T, path string) (*TableManager, stopFunc) {
boltDBIndexClient, indexStorageClient := buildTestClients(t, path)
cachePath := filepath.Join(path, cacheDirName)
cfg := Config{
CacheDir: cachePath,
SyncInterval: time.Hour,
CacheTTL: time.Hour,
Limits: &mockLimits{},
}
tableManager, err := NewTableManager(cfg, boltDBIndexClient, indexStorageClient, nil, nil)
require.NoError(t, err)
return tableManager, func() {
tableManager.Stop()
boltDBIndexClient.Stop()
}
}
func TestTableManager_QueryPages(t *testing.T) {
t.Run("QueryPages", func(t *testing.T) {
tempDir := t.TempDir()
objectStoragePath := filepath.Join(tempDir, objectsStorageDirName)
var queries []index.Query
for i, name := range []string{"table1", "table2"} {
testutil.SetupTable(t, filepath.Join(objectStoragePath, name), testutil.DBsConfig{
NumUnCompactedDBs: 5,
DBRecordsStart: i * 1000,
}, testutil.PerUserDBsConfig{
DBsConfig: testutil.DBsConfig{
NumUnCompactedDBs: 5,
DBRecordsStart: i*1000 + 500,
},
NumUsers: 1,
})
queries = append(queries, index.Query{TableName: name})
}
tableManager, stopFunc := buildTestTableManager(t, tempDir)
defer stopFunc()
testutil.TestMultiTableQuery(t, testutil.BuildUserID(0), queries, tableManager, 0, 2000)
})
t.Run("it doesn't deadlock when table create fails", func(t *testing.T) {
tempDir := t.TempDir()
require.NoError(t, os.Mkdir(filepath.Join(tempDir, "cache"), 0o777))
// This file forces chunk_util.EnsureDirectory to fail. Any write error would cause this
// deadlock
f, err := os.CreateTemp(filepath.Join(tempDir, "cache"), "not-a-directory")
require.NoError(t, err)
badTable := filepath.Base(f.Name())
tableManager, stopFunc := buildTestTableManager(t, tempDir)
defer stopFunc()
err = tableManager.query(context.Background(), badTable, nil, nil)
require.Error(t, err)
// This one deadlocks without the fix
err = tableManager.query(context.Background(), badTable, nil, nil)
require.Error(t, err)
})
}
func TestTableManager_cleanupCache(t *testing.T) {
tempDir := t.TempDir()
tableManager, stopFunc := buildTestTableManager(t, tempDir)
defer stopFunc()
// one table that would expire and other one won't
expiredTableName := "expired-table"
nonExpiredTableName := "non-expired-table"
tableManager.tables[expiredTableName] = &mockTable{}
tableManager.tables[nonExpiredTableName] = &mockTable{}
// call cleanupCache and verify that no tables are cleaned up because they are not yet expired.
require.NoError(t, tableManager.cleanupCache())
require.Len(t, tableManager.tables, 2)
// set the flag for expiredTable to expire.
tableManager.tables[expiredTableName].(*mockTable).tableExpired = true
// call the cleanupCache and verify that we still have nonExpiredTable and expiredTable is gone.
require.NoError(t, tableManager.cleanupCache())
require.Len(t, tableManager.tables, 1)
_, ok := tableManager.tables[expiredTableName]
require.False(t, ok)
_, ok = tableManager.tables[nonExpiredTableName]
require.True(t, ok)
}
func TestTableManager_ensureQueryReadiness(t *testing.T) {
activeTableNumber := getActiveTableNumber()
mockIndexStorageClient := &mockIndexStorageClient{
userIndexesInTables: map[string][]string{},
}
cfg := Config{
SyncInterval: time.Hour,
CacheTTL: time.Hour,
}
tableManager := &TableManager{
cfg: cfg,
indexStorageClient: mockIndexStorageClient,
tables: make(map[string]Table),
metrics: newMetrics(nil),
ctx: context.Background(),
cancel: func() {},
}
buildTableName := func(idx int) string {
return fmt.Sprintf("table_%d", activeTableNumber-int64(idx))
}
// setup 10 tables with 5 latest tables having user index for user1 and user2
for i := 0; i < 10; i++ {
tableName := buildTableName(i)
tableManager.tables[tableName] = &mockTable{}
mockIndexStorageClient.tablesInStorage = append(mockIndexStorageClient.tablesInStorage, tableName)
if i < 5 {
mockIndexStorageClient.userIndexesInTables[tableName] = []string{"user1", "user2"}
}
}
// function for resetting state of mockTables
resetTables := func() {
for _, table := range tableManager.tables {
table.(*mockTable).queryReadinessDoneForUsers = nil
}
}
for _, tc := range []struct {
name string
queryReadyNumDaysCfg int
queryReadinessLimits mockLimits
expectedQueryReadinessDoneForUsers map[string][]string
}{
{
name: "no query readiness configured",
queryReadinessLimits: mockLimits{},
},
{
name: "common index: 5 days",
queryReadyNumDaysCfg: 5,
expectedQueryReadinessDoneForUsers: map[string][]string{
buildTableName(0): {},
buildTableName(1): {},
buildTableName(2): {},
buildTableName(3): {},
buildTableName(4): {},
buildTableName(5): {}, // NOTE: we include an extra table since we are counting days back from current point in time
},
},
{
name: "common index: 20 days",
queryReadyNumDaysCfg: 20,
expectedQueryReadinessDoneForUsers: map[string][]string{
buildTableName(0): {},
buildTableName(1): {},
buildTableName(2): {},
buildTableName(3): {},
buildTableName(4): {},
buildTableName(5): {},
buildTableName(6): {},
buildTableName(7): {},
buildTableName(8): {},
buildTableName(9): {},
},
},
{
name: "user index default: 2 days",
queryReadinessLimits: mockLimits{
queryReadyIndexNumDaysDefault: 2,
},
expectedQueryReadinessDoneForUsers: map[string][]string{
buildTableName(0): {"user1", "user2"},
buildTableName(1): {"user1", "user2"},
buildTableName(2): {"user1", "user2"},
},
},
{
name: "common index: 5 days, user index default: 2 days",
queryReadinessLimits: mockLimits{
queryReadyIndexNumDaysDefault: 2,
},
queryReadyNumDaysCfg: 5,
expectedQueryReadinessDoneForUsers: map[string][]string{
buildTableName(0): {"user1", "user2"},
buildTableName(1): {"user1", "user2"},
buildTableName(2): {"user1", "user2"},
buildTableName(3): {},
buildTableName(4): {},
buildTableName(5): {},
},
},
{
name: "user1: 2 days",
queryReadinessLimits: mockLimits{
queryReadyIndexNumDaysByUser: map[string]int{"user1": 2},
},
expectedQueryReadinessDoneForUsers: map[string][]string{
buildTableName(0): {"user1"},
buildTableName(1): {"user1"},
buildTableName(2): {"user1"},
},
},
{
name: "user1: 2 days, user2: 20 days",
queryReadinessLimits: mockLimits{
queryReadyIndexNumDaysByUser: map[string]int{"user1": 2, "user2": 20},
},
expectedQueryReadinessDoneForUsers: map[string][]string{
buildTableName(0): {"user1", "user2"},
buildTableName(1): {"user1", "user2"},
buildTableName(2): {"user1", "user2"},
buildTableName(3): {"user2"},
buildTableName(4): {"user2"},
},
},
{
name: "user index default: 3 days, user1: 2 days",
queryReadinessLimits: mockLimits{
queryReadyIndexNumDaysDefault: 3,
queryReadyIndexNumDaysByUser: map[string]int{"user1": 2},
},
expectedQueryReadinessDoneForUsers: map[string][]string{
buildTableName(0): {"user1", "user2"},
buildTableName(1): {"user1", "user2"},
buildTableName(2): {"user1", "user2"},
buildTableName(3): {"user2"},
},
},
} {
tcCopy := tc
t.Run(tc.name, func(t *testing.T) {
resetTables()
tableManager.cfg.QueryReadyNumDays = tc.queryReadyNumDaysCfg
tableManager.cfg.Limits = &tcCopy.queryReadinessLimits
require.NoError(t, tableManager.ensureQueryReadiness(context.Background()))
for name, table := range tableManager.tables {
require.Equal(t, tc.expectedQueryReadinessDoneForUsers[name], table.(*mockTable).queryReadinessDoneForUsers, "table: %s", name)
}
})
}
}
type mockLimits struct {
queryReadyIndexNumDaysDefault int
queryReadyIndexNumDaysByUser map[string]int
}
func (m *mockLimits) AllByUserID() map[string]*validation.Limits {
allByUserID := map[string]*validation.Limits{}
for userID := range m.queryReadyIndexNumDaysByUser {
allByUserID[userID] = &validation.Limits{
QueryReadyIndexNumDays: m.queryReadyIndexNumDaysByUser[userID],
}
}
return allByUserID
}
func (m *mockLimits) DefaultLimits() *validation.Limits {
return &validation.Limits{
QueryReadyIndexNumDays: m.queryReadyIndexNumDaysDefault,
}
}
type mockTable struct {
tableExpired bool
queryReadinessDoneForUsers []string
}
func (m *mockTable) Close() {}
func (m *mockTable) MultiQueries(ctx context.Context, queries []index.Query, callback index.QueryPagesCallback) error {
return nil
}
func (m *mockTable) DropUnusedIndex(ttl time.Duration, now time.Time) (bool, error) {
return m.tableExpired, nil
}
func (m *mockTable) Sync(ctx context.Context) error {
return nil
}
func (m *mockTable) EnsureQueryReadiness(ctx context.Context, userIDs []string) error {
m.queryReadinessDoneForUsers = userIDs
return nil
}
type mockIndexStorageClient struct {
storage.Client
tablesInStorage []string
userIndexesInTables map[string][]string
}
func (m *mockIndexStorageClient) ListTables(ctx context.Context) ([]string, error) {
return m.tablesInStorage, nil
}
func (m *mockIndexStorageClient) ListFiles(ctx context.Context, tableName string, bypassCache bool) ([]storage.IndexFile, []string, error) {
return []storage.IndexFile{}, m.userIndexesInTables[tableName], nil
}

@ -1,631 +0,0 @@
package downloads
import (
"context"
"fmt"
"io/ioutil"
"os"
"path/filepath"
"strconv"
"sync"
"testing"
"time"
"github.com/pkg/errors"
"github.com/stretchr/testify/require"
"github.com/weaveworks/common/user"
"github.com/grafana/loki/pkg/storage/chunk/client/local"
"github.com/grafana/loki/pkg/storage/stores/series/index"
"github.com/grafana/loki/pkg/storage/stores/shipper/storage"
"github.com/grafana/loki/pkg/storage/stores/shipper/testutil"
util_log "github.com/grafana/loki/pkg/util/log"
)
const (
cacheDirName = "cache"
objectsStorageDirName = "objects"
userID = "user-id"
)
// storageClientWithFakeObjectsInList adds a fake object in the list call response which
// helps with testing the case where objects gets deleted in the middle of a Sync/Download operation due to compaction.
type storageClientWithFakeObjectsInList struct {
storage.Client
}
func newStorageClientWithFakeObjectsInList(storageClient storage.Client) storage.Client {
return storageClientWithFakeObjectsInList{storageClient}
}
func (o storageClientWithFakeObjectsInList) ListFiles(ctx context.Context, tableName string, _ bool) ([]storage.IndexFile, []string, error) {
files, userIDs, err := o.Client.ListFiles(ctx, tableName, true)
if err != nil {
return nil, nil, err
}
files = append(files, storage.IndexFile{
Name: "fake-object",
ModifiedAt: time.Now(),
})
return files, userIDs, nil
}
func (o storageClientWithFakeObjectsInList) ListUserFiles(ctx context.Context, tableName, userID string, _ bool) ([]storage.IndexFile, error) {
files, err := o.Client.ListUserFiles(ctx, tableName, userID, true)
if err != nil {
return nil, err
}
files = append(files, storage.IndexFile{
Name: "fake-object",
ModifiedAt: time.Now(),
})
return files, nil
}
type stopFunc func()
func buildTestClients(t *testing.T, path string) (*local.BoltIndexClient, storage.Client) {
cachePath := filepath.Join(path, cacheDirName)
boltDBIndexClient, err := local.NewBoltDBIndexClient(local.BoltDBConfig{Directory: cachePath})
require.NoError(t, err)
objectStoragePath := filepath.Join(path, objectsStorageDirName)
fsObjectClient, err := local.NewFSObjectClient(local.FSConfig{Directory: objectStoragePath})
require.NoError(t, err)
return boltDBIndexClient, storage.NewIndexStorageClient(fsObjectClient, "")
}
func buildTestTable(t *testing.T, path string) (*table, *local.BoltIndexClient, stopFunc) {
boltDBIndexClient, storageClient := buildTestClients(t, path)
cachePath := filepath.Join(path, cacheDirName)
table := NewTable(tableName, cachePath, storageClient, boltDBIndexClient, newMetrics(nil)).(*table)
_, usersWithIndex, err := table.storageClient.ListFiles(context.Background(), tableName, false)
require.NoError(t, err)
require.NoError(t, table.EnsureQueryReadiness(context.Background(), usersWithIndex))
return table, boltDBIndexClient, func() {
table.Close()
boltDBIndexClient.Stop()
}
}
type mockIndexSet struct {
IndexSet
queriesDone []index.Query
failQueries bool
lastUsedAt time.Time
}
func (m *mockIndexSet) MultiQueries(_ context.Context, queries []index.Query, _ index.QueryPagesCallback) error {
m.queriesDone = append(m.queriesDone, queries...)
return nil
}
func (m *mockIndexSet) Err() error {
var err error
if m.failQueries {
err = errors.New("fail queries")
}
return err
}
func (m *mockIndexSet) DropAllDBs() error {
return nil
}
func (m *mockIndexSet) LastUsedAt() time.Time {
return m.lastUsedAt
}
func (m *mockIndexSet) UpdateLastUsedAt() {
m.lastUsedAt = time.Now()
}
func TestTable_MultiQueries(t *testing.T) {
usersToSetup := []string{"user1", "user2"}
for name, tc := range map[string]struct {
withError bool
queryWithUserID string
}{
"without error": {
queryWithUserID: usersToSetup[0],
},
"with error": {
withError: true,
queryWithUserID: usersToSetup[0],
},
"query with user2": {
queryWithUserID: usersToSetup[1],
},
} {
t.Run(name, func(t *testing.T) {
table := table{
indexSets: map[string]IndexSet{},
logger: util_log.Logger,
}
table.indexSets[""] = &mockIndexSet{}
for _, userID := range usersToSetup {
table.indexSets[userID] = &mockIndexSet{failQueries: tc.withError}
}
var testQueries []index.Query
for i := 0; i < 5; i++ {
testQueries = append(testQueries, index.Query{
TableName: "test-table",
HashValue: fmt.Sprint(i),
RangeValuePrefix: []byte(fmt.Sprintf("range-value-prefix-%d", i)),
RangeValueStart: []byte(fmt.Sprintf("range-value-start-%d", i)),
ValueEqual: []byte(fmt.Sprintf("value-equal-%d", i)),
})
}
err := table.MultiQueries(user.InjectOrgID(context.Background(), tc.queryWithUserID), testQueries, func(query index.Query, batch index.ReadBatchResult) bool {
return true
})
if tc.withError {
require.Error(t, err)
require.Len(t, table.indexSets, len(usersToSetup))
ensureIndexSetExistsInTable(t, &table, "")
for _, userID := range usersToSetup {
if userID != tc.queryWithUserID {
ensureIndexSetExistsInTable(t, &table, userID)
}
}
} else {
require.NoError(t, err)
require.Len(t, table.indexSets, len(usersToSetup)+1)
// ensure that only common and user specific index sets are queried
for userID, indexSet := range table.indexSets {
if userID == "" || userID == tc.queryWithUserID {
require.EqualValues(t, testQueries, indexSet.(*mockIndexSet).queriesDone)
} else {
require.Len(t, indexSet.(*mockIndexSet).queriesDone, 0)
}
}
}
})
}
}
func TestTable_MultiQueries_Response(t *testing.T) {
tempDir := t.TempDir()
objectStoragePath := filepath.Join(tempDir, objectsStorageDirName)
testutil.SetupTable(t, filepath.Join(objectStoragePath, tableName), testutil.DBsConfig{
DBRecordsStart: 0,
NumUnCompactedDBs: 5,
}, testutil.PerUserDBsConfig{
DBsConfig: testutil.DBsConfig{
DBRecordsStart: 500,
NumUnCompactedDBs: 5,
NumCompactedDBs: 1,
},
NumUsers: 1,
})
table, _, stopFunc := buildTestTable(t, tempDir)
defer func() {
stopFunc()
}()
// build queries each looking for specific value from all the dbs
var queries []index.Query
for i := 0; i < 1000; i++ {
queries = append(queries, index.Query{ValueEqual: []byte(strconv.Itoa(i))})
}
// run the queries concurrently
var wg sync.WaitGroup
for i := 0; i < 10; i++ {
wg.Add(1)
go func() {
defer wg.Done()
// query for user 0 which has per user index setup which should return both user and common index.
testutil.TestSingleTableQuery(t, testutil.BuildUserID(0), queries, table, 0, 1000)
// query for user 1 which does not have per user index setup which should return only common index.
testutil.TestSingleTableQuery(t, testutil.BuildUserID(1), queries, table, 0, 500)
}()
}
wg.Wait()
}
func TestTable_DropUnusedIndex(t *testing.T) {
ttl := 24 * time.Hour
now := time.Now()
notExpiredIndexUserID := "not-expired-user-based-index"
expiredIndexUserID := "expired-user-based-index"
// initialize some indexSets with indexSet for expiredIndexUserID being expired
indexSets := map[string]IndexSet{
"": &mockIndexSet{lastUsedAt: time.Now()},
notExpiredIndexUserID: &mockIndexSet{lastUsedAt: time.Now().Add(-time.Hour)},
expiredIndexUserID: &mockIndexSet{lastUsedAt: now.Add(-25 * time.Hour)},
}
table := table{
indexSets: indexSets,
logger: util_log.Logger,
}
// ensure that we only find expiredIndexUserID to be dropped
require.Equal(t, []string{expiredIndexUserID}, table.findExpiredIndexSets(ttl, now))
// dropping unused indexSets should drop only index set for expiredIndexUserID
allIndexSetsDropped, err := table.DropUnusedIndex(ttl, now)
require.NoError(t, err)
require.False(t, allIndexSetsDropped)
// verify that we only dropped index set for expiredIndexUserID
require.Len(t, table.indexSets, 2)
ensureIndexSetExistsInTable(t, &table, "")
ensureIndexSetExistsInTable(t, &table, notExpiredIndexUserID)
// change the lastUsedAt for common index set to expire it
indexSets[""].(*mockIndexSet).lastUsedAt = now.Add(-25 * time.Hour)
// common index set should not get dropped since we still have notExpiredIndexUserID which is not expired
require.Equal(t, []string(nil), table.findExpiredIndexSets(ttl, now))
allIndexSetsDropped, err = table.DropUnusedIndex(ttl, now)
require.NoError(t, err)
require.False(t, allIndexSetsDropped)
// none of the index set should be dropped
require.Len(t, table.indexSets, 2)
ensureIndexSetExistsInTable(t, &table, "")
ensureIndexSetExistsInTable(t, &table, notExpiredIndexUserID)
// change the lastUsedAt for all indexSets so that all of them get dropped
for _, indexSets := range table.indexSets {
indexSets.(*mockIndexSet).lastUsedAt = now.Add(-25 * time.Hour)
}
// ensure that we get userID of common index set at the end
require.Equal(t, []string{notExpiredIndexUserID, ""}, table.findExpiredIndexSets(ttl, now))
allIndexSetsDropped, err = table.DropUnusedIndex(ttl, now)
require.NoError(t, err)
require.True(t, allIndexSetsDropped)
}
func TestTable_EnsureQueryReadiness(t *testing.T) {
tempDir := t.TempDir()
objectStoragePath := filepath.Join(tempDir, objectsStorageDirName)
// setup table in storage with 1 common db and 2 users with a db each
testutil.SetupTable(t, filepath.Join(objectStoragePath, tableName), testutil.DBsConfig{
DBRecordsStart: 0,
NumUnCompactedDBs: 1,
}, testutil.PerUserDBsConfig{
DBsConfig: testutil.DBsConfig{
DBRecordsStart: 100,
NumCompactedDBs: 1,
},
NumUsers: 2,
})
boltDBIndexClient, storageClient := buildTestClients(t, tempDir)
defer boltDBIndexClient.Stop()
for _, tc := range []struct {
name string
usersToDoQueryReadinessFor []string
}{
{
name: "only common index to be query ready",
},
{
name: "one of the users to be query ready",
usersToDoQueryReadinessFor: []string{testutil.BuildUserID(0)},
},
} {
t.Run(tc.name, func(t *testing.T) {
cachePath := t.TempDir()
table := NewTable(tableName, cachePath, storageClient, boltDBIndexClient, newMetrics(nil)).(*table)
defer func() {
table.Close()
}()
// EnsureQueryReadiness should update the last used at time of common index set
require.NoError(t, table.EnsureQueryReadiness(context.Background(), tc.usersToDoQueryReadinessFor))
require.Len(t, table.indexSets, len(tc.usersToDoQueryReadinessFor)+1)
for _, userID := range append(tc.usersToDoQueryReadinessFor, "") {
ensureIndexSetExistsInTable(t, table, userID)
require.InDelta(t, time.Now().Unix(), table.indexSets[userID].(*indexSet).lastUsedAt.Unix(), 5)
}
// change the last used at to verify that it gets updated when we do the query readiness again
for _, idxSet := range table.indexSets {
idxSet.(*indexSet).lastUsedAt = time.Now().Add(-time.Hour)
}
// Running it multiple times should not have an impact other than updating last used at time
for i := 0; i < 2; i++ {
require.NoError(t, table.EnsureQueryReadiness(context.Background(), tc.usersToDoQueryReadinessFor))
require.Len(t, table.indexSets, len(tc.usersToDoQueryReadinessFor)+1)
for _, userID := range append(tc.usersToDoQueryReadinessFor, "") {
ensureIndexSetExistsInTable(t, table, userID)
require.InDelta(t, time.Now().Unix(), table.indexSets[userID].(*indexSet).lastUsedAt.Unix(), 5)
}
}
})
}
}
func TestTable_Sync(t *testing.T) {
tempDir := t.TempDir()
objectStoragePath := filepath.Join(tempDir, objectsStorageDirName)
tablePathInStorage := filepath.Join(objectStoragePath, tableName)
// list of dbs to create except newDB that would be added later as part of updates
deleteDB := "delete"
noUpdatesDB := "no-updates"
newDB := "new"
testDBs := map[string]testutil.DBConfig{
deleteDB: {
DBRecords: testutil.DBRecords{
Start: 0,
NumRecords: 10,
},
},
noUpdatesDB: {
DBRecords: testutil.DBRecords{
Start: 10,
NumRecords: 10,
},
},
}
// setup the table in storage with some records
testutil.SetupDBsAtPath(t, filepath.Join(objectStoragePath, tableName), testDBs, nil)
// create table instance
table, boltdbClient, stopFunc := buildTestTable(t, tempDir)
defer func() {
stopFunc()
}()
// replace the storage client with the one that adds fake objects in the list call
table.storageClient = newStorageClientWithFakeObjectsInList(table.storageClient)
// query table to see it has expected records setup
testutil.TestSingleTableQuery(t, userID, []index.Query{{}}, table, 0, 20)
// add a sleep since we are updating a file and CI is sometimes too fast to create a difference in mtime of files
time.Sleep(time.Second)
// remove deleteDB and add the newDB
require.NoError(t, os.Remove(filepath.Join(tablePathInStorage, deleteDB)))
testutil.AddRecordsToDB(t, filepath.Join(tablePathInStorage, newDB), boltdbClient, 20, 10, nil)
// sync the table
table.storageClient.RefreshIndexListCache(context.Background())
require.NoError(t, table.Sync(context.Background()))
// query and verify table has expected records from new db and the records from deleted db are gone
testutil.TestSingleTableQuery(t, userID, []index.Query{{}}, table, 10, 20)
// verify files in cache where dbs for the table are synced to double check.
expectedFilesInDir := map[string]struct{}{
noUpdatesDB: {},
newDB: {},
}
filesInfo, err := ioutil.ReadDir(tablePathInStorage)
require.NoError(t, err)
require.Len(t, table.indexSets[""].(*indexSet).dbs, len(expectedFilesInDir))
for _, fileInfo := range filesInfo {
require.False(t, fileInfo.IsDir())
_, ok := expectedFilesInDir[fileInfo.Name()]
require.True(t, ok)
}
// let us simulate a compaction to test stale index list cache handling
// first, let us add a new file and refresh the index list cache
oneMoreDB := "one-more-db"
testutil.AddRecordsToDB(t, filepath.Join(tablePathInStorage, oneMoreDB), boltdbClient, 30, 10, nil)
table.storageClient.RefreshIndexListCache(context.Background())
// now, without syncing the table, let us compact the index in storage
compactedDBName := "compacted-db"
testutil.AddRecordsToDB(t, filepath.Join(tablePathInStorage, compactedDBName), boltdbClient, 10, 30, nil)
require.NoError(t, os.Remove(filepath.Join(tablePathInStorage, noUpdatesDB)))
require.NoError(t, os.Remove(filepath.Join(tablePathInStorage, newDB)))
require.NoError(t, os.Remove(filepath.Join(tablePathInStorage, oneMoreDB)))
// let us run a sync which should detect the stale index list cache and sync the table after refreshing the cache
require.NoError(t, table.Sync(context.Background()))
// query and verify table has expected records
testutil.TestSingleTableQuery(t, userID, []index.Query{{}}, table, 10, 30)
require.Len(t, table.indexSets[""].(*indexSet).dbs, 1)
}
func TestTable_QueryResponse(t *testing.T) {
tempDir := t.TempDir()
objectStoragePath := filepath.Join(tempDir, objectsStorageDirName)
tablePathInStorage := filepath.Join(objectStoragePath, tableName)
commonDBs := map[string]testutil.DBConfig{
"db1": {
CompressFile: true,
DBRecords: testutil.DBRecords{
Start: 0,
NumRecords: 10,
},
},
"duplicate_db1": {
CompressFile: true,
DBRecords: testutil.DBRecords{
Start: 0,
NumRecords: 10,
},
},
"db2": {
DBRecords: testutil.DBRecords{
Start: 10,
NumRecords: 10,
},
},
"partially_duplicate_db2": {
CompressFile: true,
DBRecords: testutil.DBRecords{
Start: 10,
NumRecords: 5,
},
},
"db3": {
DBRecords: testutil.DBRecords{
Start: 20,
NumRecords: 10,
},
},
}
userDBs := map[string]testutil.DBConfig{
"overlaps_with_common_dbs": {
CompressFile: true,
DBRecords: testutil.DBRecords{
Start: 10,
NumRecords: 30,
},
},
"same_db_again": {
DBRecords: testutil.DBRecords{
Start: 10,
NumRecords: 20,
},
},
"additional_records": {
CompressFile: true,
DBRecords: testutil.DBRecords{
Start: 30,
NumRecords: 10,
},
},
}
testutil.SetupDBsAtPath(t, tablePathInStorage, commonDBs, nil)
testutil.SetupDBsAtPath(t, filepath.Join(tablePathInStorage, userID), userDBs, nil)
table, _, stopFunc := buildTestTable(t, tempDir)
defer func() {
stopFunc()
}()
// build queries each looking for specific value from all the dbs
var queries []index.Query
for i := 5; i < 35; i++ {
queries = append(queries, index.Query{ValueEqual: []byte(strconv.Itoa(i))})
}
// Query the table with user id which has user specific index as well.
// Response should include records from both user and common index.
testutil.TestSingleTableQuery(t, userID, queries, table, 5, 30)
// Query the table with different user id which does not have user specific index.
// Response should include records only from common index.
testutil.TestSingleTableQuery(t, "fake", queries, table, 5, 25)
}
func TestLoadTable(t *testing.T) {
tempDir := t.TempDir()
objectStoragePath := filepath.Join(tempDir, objectsStorageDirName)
tablePathInStorage := filepath.Join(objectStoragePath, tableName)
commonDBs := make(map[string]testutil.DBConfig)
userDBs := make(map[string]testutil.DBConfig)
for i := 0; i < 10; i++ {
commonDBs[fmt.Sprint(i)] = testutil.DBConfig{
DBRecords: testutil.DBRecords{
Start: i,
NumRecords: 1,
},
}
userDBs[fmt.Sprint(i+10)] = testutil.DBConfig{
DBRecords: testutil.DBRecords{
Start: i + 10,
NumRecords: 1,
},
}
}
// setup the table in storage with some records
testutil.SetupDBsAtPath(t, tablePathInStorage, commonDBs, nil)
testutil.SetupDBsAtPath(t, filepath.Join(tablePathInStorage, userID), userDBs, nil)
boltDBIndexClient, storageClient := buildTestClients(t, tempDir)
tablePathInCache := filepath.Join(tempDir, cacheDirName, tableName)
storageClient = newStorageClientWithFakeObjectsInList(storageClient)
// try loading the table.
table, err := LoadTable(tableName, tablePathInCache, storageClient, boltDBIndexClient, newMetrics(nil))
require.NoError(t, err)
require.NotNil(t, table)
// query the loaded table to see if it has right data.
testutil.TestSingleTableQuery(t, userID, []index.Query{{}}, table, 0, 20)
// close the table to test reloading of table with already having files in the cache dir.
table.Close()
// change a boltdb file to text file which would fail to open.
require.NoError(t, ioutil.WriteFile(filepath.Join(tablePathInCache, "0"), []byte("invalid boltdb file"), 0o666))
require.NoError(t, ioutil.WriteFile(filepath.Join(tablePathInCache, userID, "10"), []byte("invalid boltdb file"), 0o666))
// verify that changed boltdb file can't be opened.
_, err = local.OpenBoltdbFile(filepath.Join(tablePathInCache, "0"))
require.Error(t, err)
// add some more files to the storage.
commonDBs = make(map[string]testutil.DBConfig)
userDBs = make(map[string]testutil.DBConfig)
for i := 20; i < 30; i++ {
commonDBs[fmt.Sprint(i)] = testutil.DBConfig{
DBRecords: testutil.DBRecords{
Start: i,
NumRecords: 1,
},
}
userDBs[fmt.Sprint(i+10)] = testutil.DBConfig{
DBRecords: testutil.DBRecords{
Start: i + 10,
NumRecords: 1,
},
}
}
testutil.SetupDBsAtPath(t, tablePathInStorage, commonDBs, nil)
testutil.SetupDBsAtPath(t, filepath.Join(tablePathInStorage, userID), userDBs, nil)
// try loading the table, it should skip loading corrupt file and reload it from storage.
table, err = LoadTable(tableName, tablePathInCache, storageClient, boltDBIndexClient, newMetrics(nil))
require.NoError(t, err)
require.NotNil(t, table)
defer table.Close()
// query the loaded table to see if it has right data.
testutil.TestSingleTableQuery(t, userID, []index.Query{{}}, table, 0, 40)
}
func ensureIndexSetExistsInTable(t *testing.T, table *table, indexSetName string) {
_, ok := table.indexSets[indexSetName]
require.True(t, ok)
}

@ -1,59 +0,0 @@
package downloads
import (
"context"
"sync"
)
// mtxWithReadiness combines a mutex with readiness channel. It would acquire lock only when the channel is closed to mark it ready.
type mtxWithReadiness struct {
mtx sync.RWMutex
ready chan struct{}
}
func newMtxWithReadiness() *mtxWithReadiness {
return &mtxWithReadiness{
ready: make(chan struct{}),
}
}
func (m *mtxWithReadiness) markReady() {
close(m.ready)
}
func (m *mtxWithReadiness) awaitReady(ctx context.Context) error {
select {
case <-ctx.Done():
return ctx.Err()
case <-m.ready:
return nil
}
}
func (m *mtxWithReadiness) lock(ctx context.Context) error {
err := m.awaitReady(ctx)
if err != nil {
return err
}
m.mtx.Lock()
return nil
}
func (m *mtxWithReadiness) unlock() {
m.mtx.Unlock()
}
func (m *mtxWithReadiness) rLock(ctx context.Context) error {
err := m.awaitReady(ctx)
if err != nil {
return err
}
m.mtx.RLock()
return nil
}
func (m *mtxWithReadiness) rUnlock() {
m.mtx.RUnlock()
}

@ -0,0 +1,133 @@
package indexfile
import (
"context"
"fmt"
"io"
"os"
"path"
"path/filepath"
"sync"
"time"
"github.com/go-kit/log/level"
"go.etcd.io/bbolt"
"github.com/grafana/loki/pkg/storage/chunk/client/local"
"github.com/grafana/loki/pkg/storage/stores/indexshipper/index"
series_index "github.com/grafana/loki/pkg/storage/stores/series/index"
"github.com/grafana/loki/pkg/storage/stores/shipper/util"
util_log "github.com/grafana/loki/pkg/util/log"
)
const TempFileSuffix = ".temp"
type IndexFile struct {
boltDB *bbolt.DB
name string
path string
issuedReaders []*os.File
issuedReadersMtx sync.Mutex
}
func OpenIndexFile(path string) (index.Index, error) {
boltdbFile, err := util.SafeOpenBoltdbFile(path)
if err != nil {
return nil, err
}
return &IndexFile{
boltDB: boltdbFile,
name: filepath.Base(path),
path: path,
}, nil
}
func BoltDBToIndexFile(boltdbFile *bbolt.DB, name string) index.Index {
return &IndexFile{
boltDB: boltdbFile,
name: name,
path: boltdbFile.Path(),
}
}
func (i *IndexFile) GetBoltDB() *bbolt.DB {
return i.boltDB
}
func (i *IndexFile) Path() string {
return i.path
}
func (i *IndexFile) Name() string {
return i.name
}
func (i *IndexFile) Reader() (io.ReadSeeker, error) {
filePath := path.Join(filepath.Dir(i.Path()), fmt.Sprintf("%d%s", time.Now().UnixNano(), TempFileSuffix))
f, err := os.Create(filePath)
if err != nil {
return nil, err
}
err = i.boltDB.View(func(tx *bbolt.Tx) (err error) {
_, err = tx.WriteTo(f)
return
})
if err != nil {
return nil, err
}
// flush the file to disk and seek the file to the beginning.
if err := f.Sync(); err != nil {
return nil, err
}
if _, err := f.Seek(0, 0); err != nil {
return nil, err
}
i.issuedReadersMtx.Lock()
defer i.issuedReadersMtx.Unlock()
i.issuedReaders = append(i.issuedReaders, f)
return f, nil
}
func (i *IndexFile) Close() error {
i.issuedReadersMtx.Lock()
defer i.issuedReadersMtx.Unlock()
// cleanup all the issued readers
for _, f := range i.issuedReaders {
if err := f.Close(); err != nil {
level.Error(util_log.Logger).Log("msg", "failed to close temp file", "path", f.Name(), "err", err)
}
if err := os.Remove(f.Name()); err != nil {
level.Error(util_log.Logger).Log("msg", "failed to remove temp file", "path", f.Name(), "err", err)
}
}
return i.boltDB.Close()
}
func QueryBoltDB(ctx context.Context, db *bbolt.DB, userID []byte, queries []series_index.Query, callback series_index.QueryPagesCallback) error {
return db.View(func(tx *bbolt.Tx) error {
bucket := tx.Bucket(userID)
if bucket == nil {
bucket = tx.Bucket(local.IndexBucketName)
if bucket == nil {
return nil
}
}
for _, query := range queries {
if err := local.QueryWithCursor(ctx, bucket.Cursor(), query, callback); err != nil {
return err
}
}
return nil
})
}

@ -1,27 +1,16 @@
package uploads
package index
import (
"github.com/prometheus/client_golang/prometheus"
"github.com/prometheus/client_golang/prometheus/promauto"
)
const (
statusFailure = "failure"
statusSuccess = "success"
)
type metrics struct {
tablesUploadOperationTotal *prometheus.CounterVec
openExistingFileFailuresTotal prometheus.Counter
}
func newMetrics(r prometheus.Registerer) *metrics {
return &metrics{
tablesUploadOperationTotal: promauto.With(r).NewCounterVec(prometheus.CounterOpts{
Namespace: "loki_boltdb_shipper",
Name: "tables_upload_operation_total",
Help: "Total number of upload operations done by status",
}, []string{"status"}),
openExistingFileFailuresTotal: promauto.With(r).NewCounter(prometheus.CounterOpts{
Namespace: "loki_boltdb_shipper",
Name: "open_existing_file_failures_total",

@ -0,0 +1,72 @@
package index
import (
"context"
"fmt"
"github.com/grafana/dskit/tenant"
"go.etcd.io/bbolt"
shipper_index "github.com/grafana/loki/pkg/storage/stores/indexshipper/index"
"github.com/grafana/loki/pkg/storage/stores/series/index"
"github.com/grafana/loki/pkg/storage/stores/shipper/index/indexfile"
"github.com/grafana/loki/pkg/storage/stores/shipper/util"
)
type Writer interface {
ForEach(ctx context.Context, tableName string, callback func(boltdb *bbolt.DB) error) error
}
type Querier interface {
QueryPages(ctx context.Context, queries []index.Query, callback index.QueryPagesCallback) error
}
type querier struct {
writer Writer
indexShipper Shipper
}
func NewQuerier(writer Writer, indexShipper Shipper) Querier {
return &querier{
writer: writer,
indexShipper: indexShipper,
}
}
// QueryPages queries both the writer and indexShipper for the given queries.
func (q *querier) QueryPages(ctx context.Context, queries []index.Query, callback index.QueryPagesCallback) error {
userID, err := tenant.TenantID(ctx)
if err != nil {
return err
}
userIDBytes := util.GetUnsafeBytes(userID)
queriesByTable := util.QueriesByTable(queries)
for table, queries := range queriesByTable {
err := util.DoParallelQueries(ctx, func(ctx context.Context, queries []index.Query, callback index.QueryPagesCallback) error {
// writer could be nil when running in ReadOnly mode
if q.writer != nil {
err := q.writer.ForEach(ctx, table, func(boltdb *bbolt.DB) error {
return indexfile.QueryBoltDB(ctx, boltdb, userIDBytes, queries, callback)
})
if err != nil {
return err
}
}
return q.indexShipper.ForEach(ctx, table, userID, func(idx shipper_index.Index) error {
boltdbIndexFile, ok := idx.(*indexfile.IndexFile)
if !ok {
return fmt.Errorf("unexpected index type %T", idx)
}
return indexfile.QueryBoltDB(ctx, boltdbIndexFile.GetBoltDB(), userIDBytes, queries, callback)
})
}, queries, callback)
if err != nil {
return err
}
}
return nil
}

@ -1,9 +1,8 @@
package uploads
package index
import (
"context"
"fmt"
"io"
"io/ioutil"
"os"
"path"
@ -14,14 +13,13 @@ import (
"time"
"github.com/go-kit/log/level"
"go.etcd.io/bbolt"
"github.com/grafana/dskit/tenant"
"go.etcd.io/bbolt"
"github.com/grafana/loki/pkg/chunkenc"
"github.com/grafana/loki/pkg/storage/chunk/client/local"
chunk_util "github.com/grafana/loki/pkg/storage/chunk/client/util"
"github.com/grafana/loki/pkg/storage/stores/series/index"
"github.com/grafana/loki/pkg/storage/stores/shipper/index/indexfile"
shipper_util "github.com/grafana/loki/pkg/storage/stores/shipper/util"
util_log "github.com/grafana/loki/pkg/util/log"
)
@ -30,9 +28,6 @@ const (
// create a new db sharded by time based on when write request is received
ShardDBsByDuration = 15 * time.Minute
// a temp file is created during uploads with name of the db + tempFileSuffix
tempFileSuffix = ".temp"
// a snapshot file is created with name of the db + snapshotFileSuffix periodically for read operation.
snapshotFileSuffix = ".snapshot"
)
@ -42,23 +37,19 @@ type BoltDBIndexClient interface {
WriteToDB(ctx context.Context, db *bbolt.DB, bucketName []byte, writes local.TableWrites) error
}
type StorageClient interface {
PutFile(ctx context.Context, tableName, fileName string, file io.ReadSeeker) error
}
type dbSnapshot struct {
boltdb *bbolt.DB
writesCount int
}
// Table is a collection of multiple dbs created for a same table by the ingester.
// Table is a collection of multiple index files created for a same table by the ingester.
// It is used on the write path for writing the index.
// All the public methods are concurrency safe and take care of mutexes to avoid any data race.
type Table struct {
name string
path string
uploader string
storageClient StorageClient
boltdbIndexClient BoltDBIndexClient
indexShipper Shipper
makePerTenantBuckets bool
dbs map[string]*bbolt.DB
@ -68,24 +59,20 @@ type Table struct {
dbSnapshotsMtx sync.RWMutex
modifyShardsSince int64
dbUploadTime map[string]time.Time
dbUploadTimeMtx sync.RWMutex
}
// NewTable create a new Table without looking for any existing local dbs belonging to the table.
func NewTable(path, uploader string, storageClient StorageClient, boltdbIndexClient BoltDBIndexClient, makePerTenantBuckets bool) (*Table, error) {
func NewTable(path, uploader string, indexShipper Shipper, makePerTenantBuckets bool) (*Table, error) {
err := chunk_util.EnsureDirectory(path)
if err != nil {
return nil, err
}
return newTableWithDBs(map[string]*bbolt.DB{}, path, uploader, storageClient, boltdbIndexClient, makePerTenantBuckets)
return newTableWithDBs(map[string]*bbolt.DB{}, path, uploader, indexShipper, makePerTenantBuckets)
}
// LoadTable loads local dbs belonging to the table and creates a new Table with references to dbs if there are any otherwise it doesn't create a table
func LoadTable(path, uploader string, storageClient StorageClient, boltdbIndexClient BoltDBIndexClient,
makePerTenantBuckets bool, metrics *metrics,
) (*Table, error) {
func LoadTable(path, uploader string, indexShipper Shipper, makePerTenantBuckets bool, metrics *metrics) (*Table, error) {
dbs, err := loadBoltDBsFromDir(path, metrics)
if err != nil {
return nil, err
@ -95,21 +82,17 @@ func LoadTable(path, uploader string, storageClient StorageClient, boltdbIndexCl
return nil, nil
}
return newTableWithDBs(dbs, path, uploader, storageClient, boltdbIndexClient, makePerTenantBuckets)
return newTableWithDBs(dbs, path, uploader, indexShipper, makePerTenantBuckets)
}
func newTableWithDBs(dbs map[string]*bbolt.DB, path, uploader string, storageClient StorageClient, boltdbIndexClient BoltDBIndexClient,
makePerTenantBuckets bool,
) (*Table, error) {
func newTableWithDBs(dbs map[string]*bbolt.DB, path, uploader string, indexShipper Shipper, makePerTenantBuckets bool) (*Table, error) {
return &Table{
name: filepath.Base(path),
path: path,
uploader: uploader,
storageClient: storageClient,
boltdbIndexClient: boltdbIndexClient,
indexShipper: indexShipper,
dbs: dbs,
dbSnapshots: map[string]*dbSnapshot{},
dbUploadTime: map[string]time.Time{},
modifyShardsSince: time.Now().Unix(),
makePerTenantBuckets: makePerTenantBuckets,
}, nil
@ -190,36 +173,12 @@ func (lt *Table) Snapshot() error {
return nil
}
// MultiQueries runs multiple queries without having to take lock multiple times for each query.
func (lt *Table) MultiQueries(ctx context.Context, queries []index.Query, callback index.QueryPagesCallback) error {
func (lt *Table) ForEach(_ context.Context, callback func(boltdb *bbolt.DB) error) error {
lt.dbSnapshotsMtx.RLock()
defer lt.dbSnapshotsMtx.RUnlock()
userID, err := tenant.TenantID(ctx)
if err != nil {
return err
}
userIDBytes := shipper_util.GetUnsafeBytes(userID)
for _, db := range lt.dbSnapshots {
err := db.boltdb.View(func(tx *bbolt.Tx) error {
bucket := tx.Bucket(userIDBytes)
if bucket == nil {
bucket = tx.Bucket(local.IndexBucketName)
if bucket == nil {
return nil
}
}
for _, query := range queries {
if err := lt.boltdbIndexClient.QueryWithCursor(ctx, bucket.Cursor(), query, callback); err != nil {
return err
}
}
return nil
})
if err != nil {
if err := callback(db.boltdb); err != nil {
return err
}
}
@ -285,7 +244,7 @@ func (lt *Table) write(ctx context.Context, tm time.Time, writes local.TableWrit
return err
}
return lt.boltdbIndexClient.WriteToDB(ctx, db, writeToBucket, writes)
return local.WriteToDB(ctx, db, writeToBucket, writes)
}
// Stop closes all the open dbs.
@ -302,184 +261,80 @@ func (lt *Table) Stop() {
lt.dbs = map[string]*bbolt.DB{}
}
// RemoveDB closes the db and removes the file locally.
func (lt *Table) RemoveDB(name string) error {
lt.dbsMtx.Lock()
defer lt.dbsMtx.Unlock()
func (lt *Table) removeSnapshotDB(name string) error {
lt.dbSnapshotsMtx.Lock()
defer lt.dbSnapshotsMtx.Unlock()
db, ok := lt.dbs[name]
db, ok := lt.dbSnapshots[name]
if !ok {
return nil
}
err := db.Close()
err := db.boltdb.Close()
if err != nil {
return err
}
delete(lt.dbs, name)
lt.dbUploadTimeMtx.Lock()
delete(lt.dbUploadTime, name)
lt.dbUploadTimeMtx.Unlock()
delete(lt.dbSnapshots, name)
return os.Remove(filepath.Join(lt.path, name))
return os.Remove(filepath.Join(lt.path, fmt.Sprintf("%s%s", name, snapshotFileSuffix)))
}
func (lt *Table) RemoveSnapshotDB(name string) error {
lt.dbSnapshotsMtx.Lock()
defer lt.dbSnapshotsMtx.Unlock()
db, ok := lt.dbSnapshots[name]
if !ok {
return nil
}
err := db.boltdb.Close()
// HandoverIndexesToShipper hands over the inactive dbs to shipper for uploading
func (lt *Table) HandoverIndexesToShipper(force bool) error {
indexesHandedOverToShipper, err := lt.handoverIndexesToShipper(force)
if err != nil {
return err
}
delete(lt.dbSnapshots, name)
lt.dbsMtx.Lock()
defer lt.dbsMtx.Unlock()
return os.Remove(filepath.Join(lt.path, fmt.Sprintf("%s%s", name, snapshotFileSuffix)))
for _, name := range indexesHandedOverToShipper {
delete(lt.dbs, name)
if err := lt.removeSnapshotDB(name); err != nil {
level.Error(util_log.Logger).Log("msg", fmt.Sprintf("failed to remove snapshot db %s", name))
}
}
return nil
}
// Upload uploads all the dbs which are never uploaded or have been modified since the last batch was uploaded.
func (lt *Table) Upload(ctx context.Context, force bool) error {
func (lt *Table) handoverIndexesToShipper(force bool) ([]string, error) {
lt.dbsMtx.RLock()
defer lt.dbsMtx.RUnlock()
uploadShardsBefore := fmt.Sprint(getOldestActiveShardTime().Unix())
handoverShardsBefore := fmt.Sprint(getOldestActiveShardTime().Unix())
// Adding check for considering only files which are sharded and have just an epoch in their name.
// Before introducing sharding we had a single file per table which were were moved inside the folder per table as part of migration.
// Before introducing sharding we had a single file per table which were moved inside the folder per table as part of migration.
// The files were named with <table_prefix><period>.
// Since sharding was introduced we have a new file every 15 mins and their names just include an epoch timestamp, for e.g `1597927538`.
// We can remove this check after we no longer support upgrading from 1.5.0.
filenameWithEpochRe, err := regexp.Compile(`^[0-9]{10}$`)
if err != nil {
return err
return nil, err
}
level.Info(util_log.Logger).Log("msg", fmt.Sprintf("uploading table %s", lt.name))
level.Info(util_log.Logger).Log("msg", fmt.Sprintf("handing over indexes to shipper %s", lt.name))
var indexesHandedOverToShipper []string
for name, db := range lt.dbs {
// doing string comparison between unix timestamps in string form since they are anyways of same length
if !force && filenameWithEpochRe.MatchString(name) && name >= uploadShardsBefore {
continue
}
// if the file is uploaded already do not upload it again.
lt.dbUploadTimeMtx.RLock()
_, ok := lt.dbUploadTime[name]
lt.dbUploadTimeMtx.RUnlock()
if ok {
if !force && filenameWithEpochRe.MatchString(name) && name >= handoverShardsBefore {
continue
}
err = lt.uploadDB(ctx, name, db)
err = lt.indexShipper.AddIndex(lt.name, "", indexfile.BoltDBToIndexFile(db, lt.buildFileName(name)))
if err != nil {
return err
return nil, err
}
lt.dbUploadTimeMtx.Lock()
lt.dbUploadTime[name] = time.Now()
lt.dbUploadTimeMtx.Unlock()
indexesHandedOverToShipper = append(indexesHandedOverToShipper, name)
}
level.Info(util_log.Logger).Log("msg", fmt.Sprintf("finished uploading table %s", lt.name))
return nil
}
func (lt *Table) uploadDB(ctx context.Context, name string, db *bbolt.DB) error {
level.Debug(util_log.Logger).Log("msg", fmt.Sprintf("uploading db %s from table %s", name, lt.name))
level.Info(util_log.Logger).Log("msg", fmt.Sprintf("finished handing over table %s", lt.name))
filePath := path.Join(lt.path, fmt.Sprintf("%s%s", name, tempFileSuffix))
f, err := os.Create(filePath)
if err != nil {
return err
}
defer func() {
if err := f.Close(); err != nil {
level.Error(util_log.Logger).Log("msg", "failed to close temp file", "path", filePath, "err", err)
}
if err := os.Remove(filePath); err != nil {
level.Error(util_log.Logger).Log("msg", "failed to remove temp file", "path", filePath, "err", err)
}
}()
err = db.View(func(tx *bbolt.Tx) (err error) {
compressedWriter := chunkenc.Gzip.GetWriter(f)
defer chunkenc.Gzip.PutWriter(compressedWriter)
defer func() {
cerr := compressedWriter.Close()
if err == nil {
err = cerr
}
}()
_, err = tx.WriteTo(compressedWriter)
return
})
if err != nil {
return err
}
// flush the file to disk and seek the file to the beginning.
if err := f.Sync(); err != nil {
return err
}
if _, err := f.Seek(0, 0); err != nil {
return err
}
fileName := lt.buildFileName(name)
return lt.storageClient.PutFile(ctx, lt.name, fileName, f)
}
// Cleanup removes dbs which are already uploaded and have not been modified for period longer than dbRetainPeriod.
// This is to avoid keeping all the files forever in the ingesters.
func (lt *Table) Cleanup(dbRetainPeriod time.Duration) error {
level.Info(util_log.Logger).Log("msg", fmt.Sprintf("cleaning up unwanted dbs from table %s", lt.name))
var filesToCleanup []string
cutoffTime := time.Now().Add(-dbRetainPeriod)
lt.dbsMtx.RLock()
for name := range lt.dbs {
lt.dbUploadTimeMtx.RLock()
dbUploadTime, ok := lt.dbUploadTime[name]
lt.dbUploadTimeMtx.RUnlock()
// consider files which are already uploaded and have mod time before cutoff time to retain files.
if ok && dbUploadTime.Before(cutoffTime) {
filesToCleanup = append(filesToCleanup, name)
}
}
lt.dbsMtx.RUnlock()
for i := range filesToCleanup {
level.Debug(util_log.Logger).Log("msg", fmt.Sprintf("removing db %s from table %s", filesToCleanup[i], lt.name))
if err := lt.RemoveDB(filesToCleanup[i]); err != nil {
return err
}
if err := lt.RemoveSnapshotDB(filesToCleanup[i]); err != nil {
level.Error(util_log.Logger).Log("msg", fmt.Sprintf("failed to remove snapshot db %s", filesToCleanup[i]))
}
}
return nil
return indexesHandedOverToShipper, nil
}
func (lt *Table) buildFileName(dbName string) string {
@ -491,7 +346,7 @@ func (lt *Table) buildFileName(dbName string) string {
fileName = lt.uploader
}
return fmt.Sprintf("%s.gz", fileName)
return fileName
}
func loadBoltDBsFromDir(dir string, metrics *metrics) (map[string]*bbolt.DB, error) {
@ -507,7 +362,7 @@ func loadBoltDBsFromDir(dir string, metrics *metrics) (map[string]*bbolt.DB, err
}
fullPath := filepath.Join(dir, fileInfo.Name())
if strings.HasSuffix(fileInfo.Name(), tempFileSuffix) || strings.HasSuffix(fileInfo.Name(), snapshotFileSuffix) {
if strings.HasSuffix(fileInfo.Name(), indexfile.TempFileSuffix) || strings.HasSuffix(fileInfo.Name(), snapshotFileSuffix) {
// If an ingester is killed abruptly in the middle of an upload operation it could leave out a temp file which holds the snapshot of db for uploading.
// Cleaning up those temp files to avoid problems.
if err := os.Remove(fullPath); err != nil {

@ -1,4 +1,4 @@
package uploads
package index
import (
"context"
@ -13,26 +13,26 @@ import (
"github.com/go-kit/log/level"
"github.com/prometheus/client_golang/prometheus"
"go.etcd.io/bbolt"
"github.com/grafana/loki/pkg/storage/chunk/client/local"
chunk_util "github.com/grafana/loki/pkg/storage/chunk/client/util"
"github.com/grafana/loki/pkg/storage/stores/indexshipper"
shipper_index "github.com/grafana/loki/pkg/storage/stores/indexshipper/index"
"github.com/grafana/loki/pkg/storage/stores/series/index"
"github.com/grafana/loki/pkg/storage/stores/shipper/util"
util_log "github.com/grafana/loki/pkg/util/log"
)
type Config struct {
Uploader string
IndexDir string
UploadInterval time.Duration
DBRetainPeriod time.Duration
MakePerTenantBuckets bool
}
type TableManager struct {
cfg Config
boltIndexClient BoltDBIndexClient
storageClient StorageClient
cfg Config
indexShipper Shipper
metrics *metrics
tables map[string]*Table
@ -43,15 +43,24 @@ type TableManager struct {
wg sync.WaitGroup
}
func NewTableManager(cfg Config, boltIndexClient BoltDBIndexClient, storageClient StorageClient, registerer prometheus.Registerer) (*TableManager, error) {
type Shipper interface {
AddIndex(tableName, userID string, index shipper_index.Index) error
ForEach(ctx context.Context, tableName, userID string, callback func(index shipper_index.Index) error) error
}
func NewTableManager(cfg Config, indexShipper Shipper, registerer prometheus.Registerer) (*TableManager, error) {
err := chunk_util.EnsureDirectory(cfg.IndexDir)
if err != nil {
return nil, err
}
ctx, cancel := context.WithCancel(context.Background())
tm := TableManager{
cfg: cfg,
boltIndexClient: boltIndexClient,
storageClient: storageClient,
metrics: newMetrics(registerer),
ctx: ctx,
cancel: cancel,
cfg: cfg,
indexShipper: indexShipper,
metrics: newMetrics(registerer),
ctx: ctx,
cancel: cancel,
}
tables, err := tm.loadTables()
@ -68,15 +77,15 @@ func (tm *TableManager) loop() {
tm.wg.Add(1)
defer tm.wg.Done()
tm.uploadTables(context.Background(), false)
tm.handoverIndexesToShipper(false)
syncTicker := time.NewTicker(tm.cfg.UploadInterval)
syncTicker := time.NewTicker(indexshipper.UploadInterval)
defer syncTicker.Stop()
for {
select {
case <-syncTicker.C:
tm.uploadTables(context.Background(), false)
tm.handoverIndexesToShipper(false)
case <-tm.ctx.Done():
return
}
@ -89,28 +98,16 @@ func (tm *TableManager) Stop() {
tm.cancel()
tm.wg.Wait()
tm.uploadTables(context.Background(), true)
tm.handoverIndexesToShipper(true)
}
func (tm *TableManager) QueryPages(ctx context.Context, queries []index.Query, callback index.QueryPagesCallback) error {
queriesByTable := util.QueriesByTable(queries)
for tableName, queries := range queriesByTable {
err := tm.query(ctx, tableName, queries, callback)
if err != nil {
return err
}
}
return nil
}
func (tm *TableManager) query(ctx context.Context, tableName string, queries []index.Query, callback index.QueryPagesCallback) error {
func (tm *TableManager) ForEach(ctx context.Context, tableName string, callback func(boltdb *bbolt.DB) error) error {
table, ok := tm.getTable(tableName)
if !ok {
return nil
}
return util.DoParallelQueries(ctx, table, queries, callback)
return table.ForEach(ctx, callback)
}
func (tm *TableManager) getTable(tableName string) (*Table, bool) {
@ -151,8 +148,7 @@ func (tm *TableManager) getOrCreateTable(tableName string) (*Table, error) {
table, ok = tm.tables[tableName]
if !ok {
var err error
table, err = NewTable(filepath.Join(tm.cfg.IndexDir, tableName), tm.cfg.Uploader, tm.storageClient,
tm.boltIndexClient, tm.cfg.MakePerTenantBuckets)
table, err = NewTable(filepath.Join(tm.cfg.IndexDir, tableName), tm.cfg.Uploader, tm.indexShipper, tm.cfg.MakePerTenantBuckets)
if err != nil {
return nil, err
}
@ -164,37 +160,26 @@ func (tm *TableManager) getOrCreateTable(tableName string) (*Table, error) {
return table, nil
}
func (tm *TableManager) uploadTables(ctx context.Context, force bool) {
func (tm *TableManager) handoverIndexesToShipper(force bool) {
tm.tablesMtx.RLock()
defer tm.tablesMtx.RUnlock()
level.Info(util_log.Logger).Log("msg", "uploading tables")
level.Info(util_log.Logger).Log("msg", "handing over indexes to shipper")
status := statusSuccess
for _, table := range tm.tables {
err := table.Snapshot()
if err != nil {
// we do not want to stop uploading of dbs due to failures in snapshotting them so logging just the error here.
level.Error(util_log.Logger).Log("msg", "failed to snapshot table for reads", "table", table.name, "err", err)
}
err = table.Upload(ctx, force)
err := table.HandoverIndexesToShipper(force)
if err != nil {
// continue uploading other tables while skipping cleanup for a failed one.
status = statusFailure
level.Error(util_log.Logger).Log("msg", "failed to upload dbs", "table", table.name, "err", err)
// continue handing over other tables while skipping cleanup for a failed one.
level.Error(util_log.Logger).Log("msg", "failed to handover index", "table", table.name, "err", err)
continue
}
// cleanup unwanted dbs from the table
err = table.Cleanup(tm.cfg.DBRetainPeriod)
err = table.Snapshot()
if err != nil {
// we do not want to stop uploading of dbs due to failures in cleaning them up so logging just the error here.
level.Error(util_log.Logger).Log("msg", "failed to cleanup uploaded dbs past their retention period", "table", table.name, "err", err)
// we do not want to stop handing over of index due to failures in snapshotting them so logging just the error here.
level.Error(util_log.Logger).Log("msg", "failed to snapshot table for reads", "table", table.name, "err", err)
}
}
tm.metrics.tablesUploadOperationTotal.WithLabelValues(status).Inc()
}
func (tm *TableManager) loadTables() (map[string]*Table, error) {
@ -239,8 +224,7 @@ func (tm *TableManager) loadTables() (map[string]*Table, error) {
}
level.Info(util_log.Logger).Log("msg", fmt.Sprintf("loading table %s", fileInfo.Name()))
table, err := LoadTable(filepath.Join(tm.cfg.IndexDir, fileInfo.Name()), tm.cfg.Uploader, tm.storageClient,
tm.boltIndexClient, tm.cfg.MakePerTenantBuckets, tm.metrics)
table, err := LoadTable(filepath.Join(tm.cfg.IndexDir, fileInfo.Name()), tm.cfg.Uploader, tm.indexShipper, tm.cfg.MakePerTenantBuckets, tm.metrics)
if err != nil {
return nil, err
}
@ -254,6 +238,12 @@ func (tm *TableManager) loadTables() (map[string]*Table, error) {
continue
}
// handover indexes to shipper since we won't modify them anymore.
err = table.HandoverIndexesToShipper(true)
if err != nil {
return nil, err
}
// Queries are only done against table snapshots so it's important we snapshot as soon as the table is loaded.
err = table.Snapshot()
if err != nil {

@ -1,53 +1,49 @@
package uploads
package index
import (
"context"
"os"
"path/filepath"
"testing"
"time"
"github.com/stretchr/testify/require"
"go.etcd.io/bbolt"
"github.com/grafana/loki/pkg/storage/chunk/client/local"
"github.com/grafana/loki/pkg/storage/chunk/client/util"
index_shipper "github.com/grafana/loki/pkg/storage/stores/indexshipper/index"
"github.com/grafana/loki/pkg/storage/stores/series/index"
"github.com/grafana/loki/pkg/storage/stores/shipper/index/indexfile"
"github.com/grafana/loki/pkg/storage/stores/shipper/testutil"
)
func buildTestTableManager(t *testing.T, testDir string) (*TableManager, *local.BoltIndexClient, stopFunc) {
func buildTestTableManager(t *testing.T, testDir string) (*TableManager, stopFunc) {
defer func() {
require.NoError(t, os.RemoveAll(testDir))
}()
boltDBIndexClient, storageClient := buildTestClients(t, testDir)
mockIndexShipper := newMockIndexShipper()
indexPath := filepath.Join(testDir, indexDirName)
cfg := Config{
Uploader: "test-table-manager",
IndexDir: indexPath,
UploadInterval: time.Hour,
Uploader: "test-table-manager",
IndexDir: indexPath,
}
tm, err := NewTableManager(cfg, boltDBIndexClient, storageClient, nil)
tm, err := NewTableManager(cfg, mockIndexShipper, nil)
require.NoError(t, err)
return tm, boltDBIndexClient, func() {
tm.Stop()
boltDBIndexClient.Stop()
}
return tm, tm.Stop
}
func TestLoadTables(t *testing.T) {
testDir := t.TempDir()
boltDBIndexClient, storageClient := buildTestClients(t, testDir)
mockIndexShipper := newMockIndexShipper()
indexPath := filepath.Join(testDir, indexDirName)
defer func() {
boltDBIndexClient.Stop()
}()
require.NoError(t, util.EnsureDirectory(indexPath))
// add a legacy db which is outside of table specific folder
testutil.AddRecordsToDB(t, filepath.Join(indexPath, "table0"), boltDBIndexClient, 0, 10, nil)
testutil.AddRecordsToDB(t, filepath.Join(indexPath, "table0"), 0, 10, nil)
// table1 with 2 dbs
testutil.SetupDBsAtPath(t, filepath.Join(indexPath, "table1"), map[string]testutil.DBConfig{
@ -90,12 +86,11 @@ func TestLoadTables(t *testing.T) {
}
cfg := Config{
Uploader: "test-table-manager",
IndexDir: indexPath,
UploadInterval: time.Hour,
Uploader: "test-table-manager",
IndexDir: indexPath,
}
tm, err := NewTableManager(cfg, boltDBIndexClient, storageClient, nil)
tm, err := NewTableManager(cfg, mockIndexShipper, nil)
require.NoError(t, err)
defer tm.Stop()
@ -106,14 +101,28 @@ func TestLoadTables(t *testing.T) {
require.True(t, !stat.IsDir())
for tableName, expectedIndex := range expectedTables {
testutil.TestSingleTableQuery(t, userID, []index.Query{{TableName: tableName}}, tm.tables[tableName], expectedIndex.start, expectedIndex.numRecords)
// loaded tables should not have any index files, it should have handed them over to index shipper
testutil.VerifyIndexes(t, userID, []index.Query{{TableName: tableName}},
func(ctx context.Context, table string, callback func(boltdb *bbolt.DB) error) error {
return tm.tables[tableName].ForEach(ctx, callback)
},
0, 0)
// see if index shipper has the index files
testutil.VerifyIndexes(t, userID, []index.Query{{TableName: tableName}},
func(ctx context.Context, table string, callback func(boltdb *bbolt.DB) error) error {
return tm.indexShipper.ForEach(ctx, table, userID, func(index index_shipper.Index) error {
return callback(index.(*indexfile.IndexFile).GetBoltDB())
})
},
expectedIndex.start, expectedIndex.numRecords)
}
}
func TestTableManager_BatchWrite(t *testing.T) {
testDir := t.TempDir()
tm, boltIndexClient, stopFunc := buildTestTableManager(t, testDir)
tm, stopFunc := buildTestTableManager(t, testDir)
defer func() {
stopFunc()
}()
@ -126,7 +135,7 @@ func TestTableManager_BatchWrite(t *testing.T) {
"table2": {start: 20, numRecords: 10},
}
writeBatch := boltIndexClient.NewWriteBatch()
writeBatch := local.NewWriteBatch()
for tableName, records := range tc {
testutil.AddRecordsToBatch(writeBatch, tableName, records.start, records.numRecords)
}
@ -137,14 +146,18 @@ func TestTableManager_BatchWrite(t *testing.T) {
for tableName, expectedIndex := range tc {
require.NoError(t, tm.tables[tableName].Snapshot())
testutil.TestSingleTableQuery(t, userID, []index.Query{{TableName: tableName}}, tm.tables[tableName], expectedIndex.start, expectedIndex.numRecords)
testutil.VerifyIndexes(t, userID, []index.Query{{TableName: tableName}},
func(ctx context.Context, table string, callback func(boltdb *bbolt.DB) error) error {
return tm.tables[tableName].ForEach(context.Background(), callback)
},
expectedIndex.start, expectedIndex.numRecords)
}
}
func TestTableManager_QueryPages(t *testing.T) {
func TestTableManager_ForEach(t *testing.T) {
testDir := t.TempDir()
tm, boltIndexClient, stopFunc := buildTestTableManager(t, testDir)
tm, stopFunc := buildTestTableManager(t, testDir)
defer func() {
stopFunc()
}()
@ -158,7 +171,7 @@ func TestTableManager_QueryPages(t *testing.T) {
}
var queries []index.Query
writeBatch := boltIndexClient.NewWriteBatch()
writeBatch := local.NewWriteBatch()
for tableName, records := range tc {
testutil.AddRecordsToBatch(writeBatch, tableName, records.start, records.numRecords)
queries = append(queries, index.Query{TableName: tableName})
@ -172,5 +185,10 @@ func TestTableManager_QueryPages(t *testing.T) {
require.NoError(t, table.Snapshot())
}
testutil.TestMultiTableQuery(t, userID, queries, tm, 0, 30)
testutil.VerifyIndexes(t, userID, queries,
func(ctx context.Context, table string, callback func(boltdb *bbolt.DB) error) error {
return tm.ForEach(ctx, table, callback)
},
0, 30)
}

@ -0,0 +1,454 @@
package index
import (
"context"
"fmt"
"io/ioutil"
"path/filepath"
"strconv"
"testing"
"time"
"github.com/stretchr/testify/require"
"github.com/weaveworks/common/user"
"go.etcd.io/bbolt"
"github.com/grafana/loki/pkg/storage/chunk/client/local"
"github.com/grafana/loki/pkg/storage/chunk/client/util"
shipper_index "github.com/grafana/loki/pkg/storage/stores/indexshipper/index"
"github.com/grafana/loki/pkg/storage/stores/series/index"
"github.com/grafana/loki/pkg/storage/stores/shipper/index/indexfile"
"github.com/grafana/loki/pkg/storage/stores/shipper/testutil"
)
const (
indexDirName = "index"
userID = "user-id"
)
type mockIndexShipper struct {
addedIndexes map[string][]shipper_index.Index
}
func newMockIndexShipper() Shipper {
return &mockIndexShipper{
addedIndexes: make(map[string][]shipper_index.Index),
}
}
func (m *mockIndexShipper) AddIndex(tableName, _ string, index shipper_index.Index) error {
m.addedIndexes[tableName] = append(m.addedIndexes[tableName], index)
return nil
}
func (m *mockIndexShipper) ForEach(ctx context.Context, tableName, _ string, callback func(index shipper_index.Index) error) error {
for _, idx := range m.addedIndexes[tableName] {
if err := callback(idx); err != nil {
return err
}
}
return nil
}
func (m *mockIndexShipper) hasIndex(tableName, indexName string) bool {
for _, index := range m.addedIndexes[tableName] {
if indexName == index.Name() {
return true
}
}
return false
}
type stopFunc func()
func buildTestTable(t *testing.T, path string, makePerTenantBuckets bool) (*Table, stopFunc) {
mockIndexShipper := newMockIndexShipper()
indexPath := filepath.Join(path, indexDirName)
require.NoError(t, util.EnsureDirectory(indexPath))
table, err := NewTable(indexPath, "test", mockIndexShipper, makePerTenantBuckets)
require.NoError(t, err)
return table, table.Stop
}
func TestLoadTable(t *testing.T) {
indexPath := t.TempDir()
boltDBIndexClient, err := local.NewBoltDBIndexClient(local.BoltDBConfig{Directory: indexPath})
require.NoError(t, err)
defer func() {
boltDBIndexClient.Stop()
}()
// setup some dbs with default bucket and per tenant bucket for a table at a path.
tablePath := filepath.Join(indexPath, "test-table")
testutil.SetupDBsAtPath(t, tablePath, map[string]testutil.DBConfig{
"db1": {
DBRecords: testutil.DBRecords{
Start: 0,
NumRecords: 10,
},
},
"db2": {
DBRecords: testutil.DBRecords{
Start: 10,
NumRecords: 10,
},
},
}, nil)
// change a boltdb file to text file which would fail to open.
invalidFilePath := filepath.Join(tablePath, "invalid")
require.NoError(t, ioutil.WriteFile(invalidFilePath, []byte("invalid boltdb file"), 0o666))
// verify that changed boltdb file can't be opened.
_, err = local.OpenBoltdbFile(invalidFilePath)
require.Error(t, err)
// try loading the table.
table, err := LoadTable(tablePath, "test", newMockIndexShipper(), false, newMetrics(nil))
require.NoError(t, err)
require.NotNil(t, table)
defer func() {
table.Stop()
}()
// verify that we still have 3 files(2 valid, 1 invalid)
filesInfo, err := ioutil.ReadDir(tablePath)
require.NoError(t, err)
require.Len(t, filesInfo, 3)
// query the loaded table to see if it has right data.
require.NoError(t, table.Snapshot())
testutil.VerifyIndexes(t, userID, []index.Query{{TableName: table.name}}, func(ctx context.Context, _ string, callback func(boltdb *bbolt.DB) error) error {
return table.ForEach(ctx, callback)
}, 0, 20)
}
func TestTable_Write(t *testing.T) {
for _, withPerTenantBucket := range []bool{false, true} {
t.Run(fmt.Sprintf("withPerTenantBucket=%v", withPerTenantBucket), func(t *testing.T) {
tempDir := t.TempDir()
table, stopFunc := buildTestTable(t, tempDir, withPerTenantBucket)
defer stopFunc()
now := time.Now()
// allow modifying last 5 shards
table.modifyShardsSince = now.Add(-5 * ShardDBsByDuration).Unix()
// a couple of times for which we want to do writes to make the table create different shards
testCases := []struct {
writeTime time.Time
dbName string // set only when it is supposed to be written to a different name than usual
}{
{
writeTime: now,
},
{
writeTime: now.Add(-(ShardDBsByDuration + 5*time.Minute)),
},
{
writeTime: now.Add(-(ShardDBsByDuration*3 + 3*time.Minute)),
},
{
writeTime: now.Add(-6 * ShardDBsByDuration), // write with time older than table.modifyShardsSince
dbName: fmt.Sprint(table.modifyShardsSince),
},
}
numFiles := 0
// performing writes and checking whether the index gets written to right shard
for i, tc := range testCases {
t.Run(fmt.Sprint(i), func(t *testing.T) {
batch := local.NewWriteBatch()
testutil.AddRecordsToBatch(batch, "test", i*10, 10)
require.NoError(t, table.write(user.InjectOrgID(context.Background(), userID), tc.writeTime, batch.(*local.BoltWriteBatch).Writes["test"]))
numFiles++
require.Equal(t, numFiles, len(table.dbs))
expectedDBName := tc.dbName
if expectedDBName == "" {
expectedDBName = fmt.Sprint(tc.writeTime.Truncate(ShardDBsByDuration).Unix())
}
db, ok := table.dbs[expectedDBName]
require.True(t, ok)
require.NoError(t, table.Snapshot())
// test that the table has current + previous records
testutil.VerifyIndexes(t, userID, []index.Query{{}},
func(ctx context.Context, _ string, callback func(boltdb *bbolt.DB) error) error {
return table.ForEach(ctx, callback)
},
0, (i+1)*10)
bucketToQuery := local.IndexBucketName
if withPerTenantBucket {
bucketToQuery = []byte(userID)
}
testutil.VerifySingleIndexFile(t, index.Query{}, db, bucketToQuery, i*10, 10)
})
}
})
}
}
func TestTable_HandoverIndexesToShipper(t *testing.T) {
for _, withPerTenantBucket := range []bool{false, true} {
t.Run(fmt.Sprintf("withPerTenantBucket=%v", withPerTenantBucket), func(t *testing.T) {
tempDir := t.TempDir()
table, stopFunc := buildTestTable(t, tempDir, withPerTenantBucket)
defer stopFunc()
now := time.Now()
// write a batch for now
batch := local.NewWriteBatch()
testutil.AddRecordsToBatch(batch, table.name, 0, 10)
require.NoError(t, table.write(user.InjectOrgID(context.Background(), userID), now, batch.(*local.BoltWriteBatch).Writes[table.name]))
// handover indexes from the table
require.NoError(t, table.HandoverIndexesToShipper(true))
require.Len(t, table.dbs, 0)
require.Len(t, table.dbSnapshots, 0)
// check that shipper has the data we handed over
indexShipper := table.indexShipper.(*mockIndexShipper)
require.Len(t, indexShipper.addedIndexes[table.name], 1)
testutil.VerifyIndexes(t, userID, []index.Query{{TableName: table.name}},
func(ctx context.Context, _ string, callback func(boltdb *bbolt.DB) error) error {
return indexShipper.ForEach(ctx, table.name, "", func(index shipper_index.Index) error {
return callback(index.(*indexfile.IndexFile).GetBoltDB())
})
},
0, 10)
// write a batch to another shard
batch = local.NewWriteBatch()
testutil.AddRecordsToBatch(batch, table.name, 10, 10)
require.NoError(t, table.write(user.InjectOrgID(context.Background(), userID), now.Add(ShardDBsByDuration), batch.(*local.BoltWriteBatch).Writes[table.name]))
// handover indexes from the table
require.NoError(t, table.HandoverIndexesToShipper(true))
require.Len(t, table.dbs, 0)
require.Len(t, table.dbSnapshots, 0)
// check that shipper got the new data we handed over
require.Len(t, indexShipper.addedIndexes[table.name], 2)
testutil.VerifyIndexes(t, userID, []index.Query{{TableName: table.name}},
func(ctx context.Context, _ string, callback func(boltdb *bbolt.DB) error) error {
return indexShipper.ForEach(ctx, table.name, "", func(index shipper_index.Index) error {
return callback(index.(*indexfile.IndexFile).GetBoltDB())
})
},
0, 20)
})
}
}
func Test_LoadBoltDBsFromDir(t *testing.T) {
indexPath := t.TempDir()
// setup some dbs with a snapshot file.
tablePath := testutil.SetupDBsAtPath(t, filepath.Join(indexPath, "test-table"), map[string]testutil.DBConfig{
"db1": {
DBRecords: testutil.DBRecords{
Start: 0,
NumRecords: 10,
},
},
"db1" + indexfile.TempFileSuffix: { // a snapshot file which should be ignored.
DBRecords: testutil.DBRecords{
Start: 0,
NumRecords: 10,
},
},
"db2": {
DBRecords: testutil.DBRecords{
Start: 10,
NumRecords: 10,
},
},
}, nil)
// create a boltdb file without bucket which should get removed
db, err := local.OpenBoltdbFile(filepath.Join(tablePath, "no-bucket"))
require.NoError(t, err)
require.NoError(t, db.Close())
// try loading the dbs
dbs, err := loadBoltDBsFromDir(tablePath, newMetrics(nil))
require.NoError(t, err)
// check that we have just 2 dbs
require.Len(t, dbs, 2)
require.NotNil(t, dbs["db1"])
require.NotNil(t, dbs["db2"])
// close all the open dbs
for _, boltdb := range dbs {
require.NoError(t, boltdb.Close())
}
filesInfo, err := ioutil.ReadDir(tablePath)
require.NoError(t, err)
require.Len(t, filesInfo, 2)
}
func TestTable_ImmutableUploads(t *testing.T) {
tempDir := t.TempDir()
indexShipper := newMockIndexShipper()
indexPath := filepath.Join(tempDir, indexDirName)
// shardCutoff is calculated based on when shards are considered to not be active anymore and are safe to be
// handed over to shipper for uploading.
shardCutoff := getOldestActiveShardTime()
// some dbs to setup
dbNames := []int64{
shardCutoff.Add(-ShardDBsByDuration).Unix(), // inactive shard, should handover
shardCutoff.Add(-1 * time.Minute).Unix(), // 1 minute before shard cutoff, should handover
time.Now().Truncate(ShardDBsByDuration).Unix(), // active shard, should not handover
}
dbs := map[string]testutil.DBConfig{}
for _, dbName := range dbNames {
dbs[fmt.Sprint(dbName)] = testutil.DBConfig{
DBRecords: testutil.DBRecords{
NumRecords: 10,
},
}
}
// setup some dbs for a table at a path.
tableName := "test-table"
tablePath := testutil.SetupDBsAtPath(t, filepath.Join(indexPath, tableName), dbs, nil)
table, err := LoadTable(tablePath, "test", indexShipper, false, newMetrics(nil))
require.NoError(t, err)
require.NotNil(t, table)
defer func() {
table.Stop()
}()
// db expected to be handed over without forcing it
expectedDBsToHandedOver := []int64{dbNames[0], dbNames[1]}
// handover dbs without forcing it which should not handover active shard or shard which has been active upto a minute back.
require.NoError(t, table.HandoverIndexesToShipper(false))
mockIndexShipper := table.indexShipper.(*mockIndexShipper)
// verify that only expected dbs are handed over
require.Len(t, mockIndexShipper.addedIndexes, 1)
require.Len(t, mockIndexShipper.addedIndexes[table.name], len(expectedDBsToHandedOver))
for _, expectedDB := range expectedDBsToHandedOver {
require.True(t, mockIndexShipper.hasIndex(tableName, table.buildFileName(fmt.Sprint(expectedDB))))
}
// force handover of dbs
require.NoError(t, table.HandoverIndexesToShipper(true))
expectedDBsToHandedOver = dbNames
// verify that all the dbs are handed over
require.Len(t, mockIndexShipper.addedIndexes, 1)
require.Len(t, mockIndexShipper.addedIndexes[table.name], len(expectedDBsToHandedOver))
for _, expectedDB := range expectedDBsToHandedOver {
require.True(t, mockIndexShipper.hasIndex(tableName, table.buildFileName(fmt.Sprint(expectedDB))))
}
// clear dbs handed over to shipper
mockIndexShipper.addedIndexes = map[string][]shipper_index.Index{}
// force handover of dbs
require.NoError(t, table.HandoverIndexesToShipper(true))
// make sure nothing was added to shipper again
require.Len(t, mockIndexShipper.addedIndexes, 0)
}
func TestTable_MultiQueries(t *testing.T) {
indexPath := t.TempDir()
boltDBIndexClient, err := local.NewBoltDBIndexClient(local.BoltDBConfig{Directory: indexPath})
require.NoError(t, err)
defer func() {
boltDBIndexClient.Stop()
}()
user1, user2 := "user1", "user2"
// setup some dbs with default bucket and per tenant bucket for a table at a path.
tablePath := filepath.Join(indexPath, "test-table")
testutil.SetupDBsAtPath(t, tablePath, map[string]testutil.DBConfig{
"db1": {
DBRecords: testutil.DBRecords{
NumRecords: 10,
},
},
"db2": {
DBRecords: testutil.DBRecords{
Start: 10,
NumRecords: 10,
},
},
}, nil)
testutil.SetupDBsAtPath(t, tablePath, map[string]testutil.DBConfig{
"db3": {
DBRecords: testutil.DBRecords{
Start: 20,
NumRecords: 10,
},
},
"db4": {
DBRecords: testutil.DBRecords{
Start: 30,
NumRecords: 10,
},
},
}, []byte(user1))
// try loading the table.
table, err := LoadTable(tablePath, "test", newMockIndexShipper(), false, newMetrics(nil))
require.NoError(t, err)
require.NotNil(t, table)
defer func() {
table.Stop()
}()
require.NoError(t, table.Snapshot())
// build queries each looking for specific value from all the dbs
var queries []index.Query
for i := 5; i < 35; i++ {
queries = append(queries, index.Query{TableName: table.name, ValueEqual: []byte(strconv.Itoa(i))})
}
// querying data for user1 should return both data from common index and user1's index
testutil.VerifyIndexes(t, user1, queries,
func(ctx context.Context, _ string, callback func(boltdb *bbolt.DB) error) error {
return table.ForEach(ctx, callback)
},
5, 30)
// querying data for user2 should return only common index
testutil.VerifyIndexes(t, user2, queries,
func(ctx context.Context, _ string, callback func(boltdb *bbolt.DB) error) error {
return table.ForEach(ctx, callback)
},
5, 15)
}

@ -17,181 +17,103 @@ import (
"github.com/grafana/loki/pkg/storage/chunk/client"
"github.com/grafana/loki/pkg/storage/chunk/client/local"
chunk_util "github.com/grafana/loki/pkg/storage/chunk/client/util"
"github.com/grafana/loki/pkg/storage/stores/series/index"
"github.com/grafana/loki/pkg/storage/stores/shipper/downloads"
"github.com/grafana/loki/pkg/storage/stores/shipper/storage"
"github.com/grafana/loki/pkg/storage/stores/shipper/uploads"
shipper_util "github.com/grafana/loki/pkg/storage/stores/shipper/util"
"github.com/grafana/loki/pkg/storage/chunk/client/util"
"github.com/grafana/loki/pkg/storage/stores/indexshipper"
"github.com/grafana/loki/pkg/storage/stores/indexshipper/downloads"
series_index "github.com/grafana/loki/pkg/storage/stores/series/index"
"github.com/grafana/loki/pkg/storage/stores/shipper/index"
"github.com/grafana/loki/pkg/storage/stores/shipper/index/indexfile"
util_log "github.com/grafana/loki/pkg/util/log"
"github.com/grafana/loki/pkg/util/spanlogger"
)
type Mode int
const (
// ModeReadWrite is to allow both read and write
ModeReadWrite Mode = iota
// ModeReadOnly is to allow only read operations
ModeReadOnly
// ModeWriteOnly is to allow only write operations
ModeWriteOnly
// FilesystemObjectStoreType holds the periodic config type for the filesystem store
FilesystemObjectStoreType = "filesystem"
// UploadInterval defines interval for when we check if there are new index files to upload.
UploadInterval = 1 * time.Minute
)
func (m Mode) String() string {
switch m {
case ModeReadWrite:
return "read-write"
case ModeReadOnly:
return "read-only"
case ModeWriteOnly:
return "write-only"
}
return "unknown"
}
type boltDBIndexClient interface {
QueryWithCursor(_ context.Context, c *bbolt.Cursor, query index.Query, callback index.QueryPagesCallback) error
NewWriteBatch() index.WriteBatch
WriteToDB(ctx context.Context, db *bbolt.DB, bucketName []byte, writes local.TableWrites) error
Stop()
}
type Config struct {
ActiveIndexDirectory string `yaml:"active_index_directory"`
SharedStoreType string `yaml:"shared_store"`
SharedStoreKeyPrefix string `yaml:"shared_store_key_prefix"`
CacheLocation string `yaml:"cache_location"`
CacheTTL time.Duration `yaml:"cache_ttl"`
ResyncInterval time.Duration `yaml:"resync_interval"`
QueryReadyNumDays int `yaml:"query_ready_num_days"`
IndexGatewayClientConfig IndexGatewayClientConfig `yaml:"index_gateway_client"`
BuildPerTenantIndex bool `yaml:"build_per_tenant_index"`
IngesterName string `yaml:"-"`
Mode Mode `yaml:"-"`
IngesterDBRetainPeriod time.Duration `yaml:"-"`
indexshipper.Config `yaml:",inline"`
BuildPerTenantIndex bool `yaml:"build_per_tenant_index"`
}
// RegisterFlags registers flags.
func (cfg *Config) RegisterFlags(f *flag.FlagSet) {
cfg.RegisterFlagsWithPrefix("boltdb.", f)
cfg.RegisterFlagsWithPrefix("boltdb", f)
}
func (cfg *Config) RegisterFlagsWithPrefix(prefix string, f *flag.FlagSet) {
cfg.IndexGatewayClientConfig.RegisterFlagsWithPrefix(prefix+"shipper.index-gateway-client", f)
f.StringVar(&cfg.ActiveIndexDirectory, prefix+"shipper.active-index-directory", "", "Directory where ingesters would write boltdb files which would then be uploaded by shipper to configured storage")
f.StringVar(&cfg.SharedStoreType, prefix+"shipper.shared-store", "", "Shared store for keeping boltdb files. Supported types: gcs, s3, azure, filesystem")
f.StringVar(&cfg.SharedStoreKeyPrefix, prefix+"shipper.shared-store.key-prefix", "index/", "Prefix to add to Object Keys in Shared store. Path separator(if any) should always be a '/'. Prefix should never start with a separator but should always end with it")
f.StringVar(&cfg.CacheLocation, prefix+"shipper.cache-location", "", "Cache location for restoring boltDB files for queries")
f.DurationVar(&cfg.CacheTTL, prefix+"shipper.cache-ttl", 24*time.Hour, "TTL for boltDB files restored in cache for queries")
f.DurationVar(&cfg.ResyncInterval, prefix+"shipper.resync-interval", 5*time.Minute, "Resync downloaded files with the storage")
f.IntVar(&cfg.QueryReadyNumDays, prefix+"shipper.query-ready-num-days", 0, "Number of days of common index to be kept downloaded for queries. For per tenant index query readiness, use limits overrides config.")
cfg.Config.RegisterFlagsWithPrefix(prefix, f)
f.BoolVar(&cfg.BuildPerTenantIndex, prefix+"shipper.build-per-tenant-index", false, "Build per tenant index files")
}
func (cfg *Config) Validate() error {
return shipper_util.ValidateSharedStoreKeyPrefix(cfg.SharedStoreKeyPrefix)
return cfg.Config.Validate()
}
type Shipper struct {
cfg Config
boltDBIndexClient boltDBIndexClient
uploadsManager *uploads.TableManager
downloadsManager *downloads.TableManager
type writer interface {
ForEach(ctx context.Context, tableName string, callback func(boltdb *bbolt.DB) error) error
BatchWrite(ctx context.Context, batch series_index.WriteBatch) error
Stop()
}
type indexClient struct {
cfg Config
indexShipper indexshipper.IndexShipper
writer writer
querier index.Querier
metrics *metrics
stopOnce sync.Once
}
// NewShipper creates a shipper for syncing local objects with a store
func NewShipper(cfg Config, storageClient client.ObjectClient, limits downloads.Limits, ownsTenantFn downloads.IndexGatewayOwnsTenant, registerer prometheus.Registerer) (index.Client, error) {
shipper := Shipper{
func NewShipper(cfg Config, storageClient client.ObjectClient, limits downloads.Limits, ownsTenantFn downloads.IndexGatewayOwnsTenant, registerer prometheus.Registerer) (series_index.Client, error) {
i := indexClient{
cfg: cfg,
metrics: newMetrics(registerer),
}
err := shipper.init(storageClient, limits, ownsTenantFn, registerer)
err := i.init(storageClient, limits, ownsTenantFn, registerer)
if err != nil {
return nil, err
}
level.Info(util_log.Logger).Log("msg", fmt.Sprintf("starting boltdb shipper in %s mode", cfg.Mode))
return &shipper, nil
return &i, nil
}
func (s *Shipper) init(storageClient client.ObjectClient, limits downloads.Limits, ownsTenantFn downloads.IndexGatewayOwnsTenant, registerer prometheus.Registerer) error {
// When we run with target querier we don't have ActiveIndexDirectory set so using CacheLocation instead.
// Also it doesn't matter which directory we use since BoltDBIndexClient doesn't do anything with it but it is good to have a valid path.
boltdbIndexClientDir := s.cfg.ActiveIndexDirectory
if boltdbIndexClientDir == "" {
boltdbIndexClientDir = s.cfg.CacheLocation
}
func (i *indexClient) init(storageClient client.ObjectClient, limits downloads.Limits, ownsTenantFn downloads.IndexGatewayOwnsTenant, registerer prometheus.Registerer) error {
var err error
s.boltDBIndexClient, err = local.NewBoltDBIndexClient(local.BoltDBConfig{Directory: boltdbIndexClientDir})
i.indexShipper, err = indexshipper.NewIndexShipper(i.cfg.Config, storageClient, limits, ownsTenantFn, indexfile.OpenIndexFile)
if err != nil {
return err
}
indexStorageClient := storage.NewIndexStorageClient(storageClient, s.cfg.SharedStoreKeyPrefix)
if s.cfg.Mode != ModeReadOnly {
uploader, err := s.getUploaderName()
if i.cfg.Mode != indexshipper.ModeReadOnly {
uploader, err := i.getUploaderName()
if err != nil {
return err
}
cfg := uploads.Config{
cfg := index.Config{
Uploader: uploader,
IndexDir: s.cfg.ActiveIndexDirectory,
UploadInterval: UploadInterval,
DBRetainPeriod: s.cfg.IngesterDBRetainPeriod,
MakePerTenantBuckets: s.cfg.BuildPerTenantIndex,
IndexDir: i.cfg.ActiveIndexDirectory,
DBRetainPeriod: i.cfg.IngesterDBRetainPeriod,
MakePerTenantBuckets: i.cfg.BuildPerTenantIndex,
}
uploadsManager, err := uploads.NewTableManager(cfg, s.boltDBIndexClient, indexStorageClient, registerer)
i.writer, err = index.NewTableManager(cfg, i.indexShipper, registerer)
if err != nil {
return err
}
s.uploadsManager = uploadsManager
}
if s.cfg.Mode != ModeWriteOnly {
cfg := downloads.Config{
CacheDir: s.cfg.CacheLocation,
SyncInterval: s.cfg.ResyncInterval,
CacheTTL: s.cfg.CacheTTL,
QueryReadyNumDays: s.cfg.QueryReadyNumDays,
Limits: limits,
}
downloadsManager, err := downloads.NewTableManager(cfg, s.boltDBIndexClient, indexStorageClient, ownsTenantFn, registerer)
if err != nil {
return err
}
s.downloadsManager = downloadsManager
}
i.querier = index.NewQuerier(i.writer, i.indexShipper)
return nil
}
// we would persist uploader name in <active-index-directory>/uploader/name file so that we use same name on subsequent restarts to
// avoid uploading same files again with different name. If the filed does not exist we would create one with uploader name set to
// ingester name and startup timestamp so that we randomise the name and do not override files from other ingesters.
func (s *Shipper) getUploaderName() (string, error) {
uploader := fmt.Sprintf("%s-%d", s.cfg.IngesterName, time.Now().UnixNano())
func (i *indexClient) getUploaderName() (string, error) {
uploader := fmt.Sprintf("%s-%d", i.cfg.IngesterName, time.Now().UnixNano())
uploaderFilePath := path.Join(s.cfg.ActiveIndexDirectory, "uploader", "name")
if err := chunk_util.EnsureDirectory(path.Dir(uploaderFilePath)); err != nil {
uploaderFilePath := path.Join(i.cfg.ActiveIndexDirectory, "uploader", "name")
if err := util.EnsureDirectory(path.Dir(uploaderFilePath)); err != nil {
return "", err
}
@ -214,54 +136,29 @@ func (s *Shipper) getUploaderName() (string, error) {
return uploader, nil
}
func (s *Shipper) Stop() {
s.stopOnce.Do(s.stop)
func (i *indexClient) Stop() {
i.stopOnce.Do(i.stop)
}
func (s *Shipper) stop() {
if s.uploadsManager != nil {
s.uploadsManager.Stop()
}
if s.downloadsManager != nil {
s.downloadsManager.Stop()
func (i *indexClient) stop() {
if i.writer != nil {
i.writer.Stop()
}
s.boltDBIndexClient.Stop()
i.indexShipper.Stop()
}
func (s *Shipper) NewWriteBatch() index.WriteBatch {
return s.boltDBIndexClient.NewWriteBatch()
func (i *indexClient) NewWriteBatch() series_index.WriteBatch {
return local.NewWriteBatch()
}
func (s *Shipper) BatchWrite(ctx context.Context, batch index.WriteBatch) error {
return instrument.CollectedRequest(ctx, "WRITE", instrument.NewHistogramCollector(s.metrics.requestDurationSeconds), instrument.ErrorCode, func(ctx context.Context) error {
return s.uploadsManager.BatchWrite(ctx, batch)
func (i *indexClient) BatchWrite(ctx context.Context, batch series_index.WriteBatch) error {
return instrument.CollectedRequest(ctx, "WRITE", instrument.NewHistogramCollector(i.metrics.requestDurationSeconds), instrument.ErrorCode, func(ctx context.Context) error {
return i.writer.BatchWrite(ctx, batch)
})
}
func (s *Shipper) QueryPages(ctx context.Context, queries []index.Query, callback index.QueryPagesCallback) error {
return instrument.CollectedRequest(ctx, "Shipper.Query", instrument.NewHistogramCollector(s.metrics.requestDurationSeconds), instrument.ErrorCode, func(ctx context.Context) error {
spanLogger := spanlogger.FromContext(ctx)
if s.uploadsManager != nil {
err := s.uploadsManager.QueryPages(ctx, queries, callback)
if err != nil {
return err
}
level.Debug(spanLogger).Log("queried", "uploads-manager")
}
if s.downloadsManager != nil {
err := s.downloadsManager.QueryPages(ctx, queries, callback)
if err != nil {
return err
}
level.Debug(spanLogger).Log("queried", "downloads-manager")
}
return nil
func (i *indexClient) QueryPages(ctx context.Context, queries []series_index.Query, callback series_index.QueryPagesCallback) error {
return instrument.CollectedRequest(ctx, "Shipper.Query", instrument.NewHistogramCollector(i.metrics.requestDurationSeconds), instrument.ErrorCode, func(ctx context.Context) error {
return i.querier.QueryPages(ctx, queries, callback)
})
}

@ -10,6 +10,7 @@ import (
"sync"
"testing"
"github.com/grafana/dskit/tenant"
"github.com/klauspost/compress/gzip"
"github.com/stretchr/testify/require"
"github.com/weaveworks/common/user"
@ -20,19 +21,19 @@ import (
"github.com/grafana/loki/pkg/storage/stores/series/index"
)
func AddRecordsToDB(t testing.TB, path string, dbClient *local.BoltIndexClient, start, numRecords int, bucketName []byte) {
func AddRecordsToDB(t testing.TB, path string, start, numRecords int, bucketName []byte) {
t.Helper()
db, err := local.OpenBoltdbFile(path)
require.NoError(t, err)
batch := dbClient.NewWriteBatch()
batch := local.NewWriteBatch()
AddRecordsToBatch(batch, "test", start, numRecords)
if len(bucketName) == 0 {
bucketName = local.IndexBucketName
}
require.NoError(t, dbClient.WriteToDB(context.Background(), db, bucketName, batch.(*local.BoltWriteBatch).Writes["test"]))
require.NoError(t, local.WriteToDB(context.Background(), db, bucketName, batch.(*local.BoltWriteBatch).Writes["test"]))
require.NoError(t, db.Sync())
require.NoError(t, db.Close())
@ -45,49 +46,46 @@ func AddRecordsToBatch(batch index.WriteBatch, tableName string, start, numRecor
}
}
type SingleTableQuerier interface {
MultiQueries(ctx context.Context, queries []index.Query, callback index.QueryPagesCallback) error
}
func TestSingleTableQuery(t *testing.T, userID string, queries []index.Query, querier SingleTableQuerier, start, numRecords int) {
t.Helper()
minValue := start
maxValue := start + numRecords
fetchedRecords := make(map[string]string)
err := querier.MultiQueries(user.InjectOrgID(context.Background(), userID), queries, makeTestCallback(t, minValue, maxValue, fetchedRecords))
// nolint
func queryIndexes(t *testing.T, ctx context.Context, queries []index.Query, indexIteratorFunc IndexIteratorFunc, callback index.QueryPagesCallback) {
userID, err := tenant.TenantID(ctx)
require.NoError(t, err)
require.Len(t, fetchedRecords, numRecords)
}
type SingleDBQuerier interface {
QueryDB(ctx context.Context, db *bbolt.DB, bucketName []byte, query index.Query, callback index.QueryPagesCallback) error
for _, query := range queries {
err := indexIteratorFunc(ctx, query.TableName, func(boltdb *bbolt.DB) error {
return queryBoltDB(ctx, boltdb, []byte(userID), []index.Query{query}, callback)
})
require.NoError(t, err)
}
}
func TestSingleDBQuery(t *testing.T, query index.Query, db *bbolt.DB, bucketName []byte, querier SingleDBQuerier, start, numRecords int) {
type IndexIteratorFunc func(ctx context.Context, table string, callback func(boltdb *bbolt.DB) error) error
func VerifyIndexes(t *testing.T, userID string, queries []index.Query, indexIteratorFunc IndexIteratorFunc, start, numRecords int) {
t.Helper()
minValue := start
maxValue := start + numRecords
fetchedRecords := make(map[string]string)
err := querier.QueryDB(context.Background(), db, bucketName, query, makeTestCallback(t, minValue, maxValue, fetchedRecords))
require.NoError(t, err)
queryIndexes(t, user.InjectOrgID(context.Background(), userID), queries, indexIteratorFunc, makeTestCallback(t, minValue, maxValue, fetchedRecords))
require.Len(t, fetchedRecords, numRecords)
}
type MultiTableQuerier interface {
QueryPages(ctx context.Context, queries []index.Query, callback index.QueryPagesCallback) error
type SingleDBQuerier interface {
QueryDB(ctx context.Context, db *bbolt.DB, bucketName []byte, query index.Query, callback index.QueryPagesCallback) error
}
func TestMultiTableQuery(t *testing.T, userID string, queries []index.Query, querier MultiTableQuerier, start, numRecords int) {
func VerifySingleIndexFile(t *testing.T, query index.Query, db *bbolt.DB, bucketName []byte, start, numRecords int) {
t.Helper()
minValue := start
maxValue := start + numRecords
fetchedRecords := make(map[string]string)
err := querier.QueryPages(user.InjectOrgID(context.Background(), userID), queries, makeTestCallback(t, minValue, maxValue, fetchedRecords))
err := db.View(func(tx *bbolt.Tx) error {
b := tx.Bucket(bucketName)
require.NotNil(t, b)
return local.QueryWithCursor(context.Background(), b.Cursor(), query, makeTestCallback(t, minValue, maxValue, fetchedRecords))
})
require.NoError(t, err)
require.Len(t, fetchedRecords, numRecords)
@ -114,32 +112,6 @@ func makeTestCallback(t *testing.T, minValue, maxValue int, records map[string]s
}
}
func CompareDBs(t *testing.T, db1, db2 *bbolt.DB) {
t.Helper()
db1Records := readDB(t, db1)
db2Records := readDB(t, db2)
require.Equal(t, db1Records, db2Records)
}
func readDB(t *testing.T, db *bbolt.DB) map[string]map[string]string {
t.Helper()
dbRecords := map[string]map[string]string{}
err := db.View(func(tx *bbolt.Tx) error {
return tx.ForEach(func(name []byte, b *bbolt.Bucket) error {
dbRecords[string(name)] = map[string]string{}
return b.ForEach(func(k, v []byte) error {
dbRecords[string(name)][string(k)] = string(v)
return nil
})
})
})
require.NoError(t, err)
return dbRecords
}
type DBConfig struct {
CompressFile bool
DBRecords
@ -159,7 +131,7 @@ func SetupDBsAtPath(t *testing.T, path string, dbs map[string]DBConfig, bucketNa
require.NoError(t, chunk_util.EnsureDirectory(path))
for name, dbConfig := range dbs {
AddRecordsToDB(t, filepath.Join(path, name), boltIndexClient, dbConfig.Start, dbConfig.NumRecords, bucketName)
AddRecordsToDB(t, filepath.Join(path, name), dbConfig.Start, dbConfig.NumRecords, bucketName)
if dbConfig.CompressFile {
compressFile(t, filepath.Join(path, name))
}
@ -301,3 +273,22 @@ func SetupTable(t *testing.T, path string, commonDBsConfig DBsConfig, perUserDBs
func BuildUserID(id int) string {
return fmt.Sprintf("user-%d", id)
}
func queryBoltDB(ctx context.Context, db *bbolt.DB, userID []byte, queries []index.Query, callback index.QueryPagesCallback) error {
return db.View(func(tx *bbolt.Tx) error {
bucket := tx.Bucket(userID)
if bucket == nil {
bucket = tx.Bucket(local.IndexBucketName)
if bucket == nil {
return nil
}
}
for _, query := range queries {
if err := local.QueryWithCursor(ctx, bucket.Cursor(), query, callback); err != nil {
return err
}
}
return nil
})
}

@ -1,546 +0,0 @@
package uploads
import (
"context"
"fmt"
"io"
"io/ioutil"
"os"
"path/filepath"
"strconv"
"strings"
"testing"
"time"
"github.com/klauspost/compress/gzip"
"github.com/stretchr/testify/require"
"github.com/weaveworks/common/user"
"github.com/grafana/loki/pkg/storage/chunk/client/local"
"github.com/grafana/loki/pkg/storage/stores/series/index"
"github.com/grafana/loki/pkg/storage/stores/shipper/storage"
"github.com/grafana/loki/pkg/storage/stores/shipper/testutil"
)
const (
indexDirName = "index"
objectsStorageDirName = "objects"
userID = "user-id"
)
func buildTestClients(t *testing.T, path string) (*local.BoltIndexClient, StorageClient) {
indexPath := filepath.Join(path, indexDirName)
boltDBIndexClient, err := local.NewBoltDBIndexClient(local.BoltDBConfig{Directory: indexPath})
require.NoError(t, err)
objectStoragePath := filepath.Join(path, objectsStorageDirName)
fsObjectClient, err := local.NewFSObjectClient(local.FSConfig{Directory: objectStoragePath})
require.NoError(t, err)
return boltDBIndexClient, storage.NewIndexStorageClient(fsObjectClient, "")
}
type stopFunc func()
func buildTestTable(t *testing.T, path string, makePerTenantBuckets bool) (*Table, *local.BoltIndexClient, stopFunc) {
boltDBIndexClient, fsObjectClient := buildTestClients(t, path)
indexPath := filepath.Join(path, indexDirName)
table, err := NewTable(indexPath, "test", fsObjectClient, boltDBIndexClient, makePerTenantBuckets)
require.NoError(t, err)
return table, boltDBIndexClient, func() {
table.Stop()
boltDBIndexClient.Stop()
}
}
func TestLoadTable(t *testing.T) {
indexPath := t.TempDir()
boltDBIndexClient, err := local.NewBoltDBIndexClient(local.BoltDBConfig{Directory: indexPath})
require.NoError(t, err)
defer func() {
boltDBIndexClient.Stop()
}()
// setup some dbs with default bucket and per tenant bucket for a table at a path.
tablePath := filepath.Join(indexPath, "test-table")
testutil.SetupDBsAtPath(t, tablePath, map[string]testutil.DBConfig{
"db1": {
DBRecords: testutil.DBRecords{
Start: 0,
NumRecords: 10,
},
},
"db2": {
DBRecords: testutil.DBRecords{
Start: 10,
NumRecords: 10,
},
},
}, nil)
testutil.SetupDBsAtPath(t, tablePath, map[string]testutil.DBConfig{
"db3": {
DBRecords: testutil.DBRecords{
Start: 20,
NumRecords: 10,
},
},
"db4": {
DBRecords: testutil.DBRecords{
Start: 30,
NumRecords: 10,
},
},
}, []byte(userID))
// change a boltdb file to text file which would fail to open.
invalidFilePath := filepath.Join(tablePath, "invalid")
require.NoError(t, ioutil.WriteFile(invalidFilePath, []byte("invalid boltdb file"), 0o666))
// verify that changed boltdb file can't be opened.
_, err = local.OpenBoltdbFile(invalidFilePath)
require.Error(t, err)
// try loading the table.
table, err := LoadTable(tablePath, "test", nil, boltDBIndexClient, false, newMetrics(nil))
require.NoError(t, err)
require.NotNil(t, table)
defer func() {
table.Stop()
}()
// verify that we still have 5 files(4 valid, 1 invalid)
filesInfo, err := ioutil.ReadDir(tablePath)
require.NoError(t, err)
require.Len(t, filesInfo, 5)
require.NoError(t, table.Snapshot())
// query the loaded table to see if it has right data.
testutil.TestSingleTableQuery(t, userID, []index.Query{{}}, table, 0, 40)
}
func TestTable_Write(t *testing.T) {
for _, withPerTenantBucket := range []bool{false, true} {
t.Run(fmt.Sprintf("withPerTenantBucket=%v", withPerTenantBucket), func(t *testing.T) {
tempDir := t.TempDir()
table, boltIndexClient, stopFunc := buildTestTable(t, tempDir, withPerTenantBucket)
defer stopFunc()
now := time.Now()
// allow modifying last 5 shards
table.modifyShardsSince = now.Add(-5 * ShardDBsByDuration).Unix()
// a couple of times for which we want to do writes to make the table create different shards
testCases := []struct {
writeTime time.Time
dbName string // set only when it is supposed to be written to a different name than usual
}{
{
writeTime: now,
},
{
writeTime: now.Add(-(ShardDBsByDuration + 5*time.Minute)),
},
{
writeTime: now.Add(-(ShardDBsByDuration*3 + 3*time.Minute)),
},
{
writeTime: now.Add(-6 * ShardDBsByDuration), // write with time older than table.modifyShardsSince
dbName: fmt.Sprint(table.modifyShardsSince),
},
}
numFiles := 0
// performing writes and checking whether the index gets written to right shard
for i, tc := range testCases {
t.Run(fmt.Sprint(i), func(t *testing.T) {
batch := boltIndexClient.NewWriteBatch()
testutil.AddRecordsToBatch(batch, "test", i*10, 10)
require.NoError(t, table.write(user.InjectOrgID(context.Background(), userID), tc.writeTime, batch.(*local.BoltWriteBatch).Writes["test"]))
numFiles++
require.Equal(t, numFiles, len(table.dbs))
expectedDBName := tc.dbName
if expectedDBName == "" {
expectedDBName = fmt.Sprint(tc.writeTime.Truncate(ShardDBsByDuration).Unix())
}
db, ok := table.dbs[expectedDBName]
require.True(t, ok)
require.NoError(t, table.Snapshot())
// test that the table has current + previous records
testutil.TestSingleTableQuery(t, userID, []index.Query{{}}, table, 0, (i+1)*10)
bucketToQuery := local.IndexBucketName
if withPerTenantBucket {
bucketToQuery = []byte(userID)
}
testutil.TestSingleDBQuery(t, index.Query{}, db, bucketToQuery, boltIndexClient, i*10, 10)
})
}
})
}
}
func TestTable_Upload(t *testing.T) {
for _, withPerTenantBucket := range []bool{false, true} {
t.Run(fmt.Sprintf("withPerTenantBucket=%v", withPerTenantBucket), func(t *testing.T) {
tempDir := t.TempDir()
table, boltIndexClient, stopFunc := buildTestTable(t, tempDir, withPerTenantBucket)
defer stopFunc()
now := time.Now()
// write a batch for now
batch := boltIndexClient.NewWriteBatch()
testutil.AddRecordsToBatch(batch, "test", 0, 10)
require.NoError(t, table.write(user.InjectOrgID(context.Background(), userID), now, batch.(*local.BoltWriteBatch).Writes["test"]))
// upload the table
require.NoError(t, table.Upload(context.Background(), true))
require.Len(t, table.dbs, 1)
// compare the local dbs for the table with the dbs in remote storage after upload to ensure they have same data
objectStorageDir := filepath.Join(tempDir, objectsStorageDirName)
compareTableWithStorage(t, table, objectStorageDir)
// write a batch to another shard
batch = boltIndexClient.NewWriteBatch()
testutil.AddRecordsToBatch(batch, "test", 20, 10)
require.NoError(t, table.write(user.InjectOrgID(context.Background(), userID), now.Add(ShardDBsByDuration), batch.(*local.BoltWriteBatch).Writes["test"]))
// upload the dbs to storage
require.NoError(t, table.Upload(context.Background(), true))
require.Len(t, table.dbs, 2)
// check local dbs with remote dbs to ensure they have same data
compareTableWithStorage(t, table, objectStorageDir)
})
}
}
func compareTableWithStorage(t *testing.T, table *Table, storageDir string) {
// use a temp dir for decompressing the files before comparison.
tempDir := t.TempDir()
for name, db := range table.dbs {
fileName := table.buildFileName(name)
// open compressed file from storage
compressedFile, err := os.Open(filepath.Join(storageDir, table.name, fileName))
require.NoError(t, err)
// get a compressed reader
compressedReader, err := gzip.NewReader(compressedFile)
require.NoError(t, err)
// create a temp file for writing decompressed file
decompressedFilePath := filepath.Join(tempDir, filepath.Base(fileName))
decompressedFile, err := os.Create(decompressedFilePath)
require.NoError(t, err)
// do the decompression
_, err = io.Copy(decompressedFile, compressedReader)
require.NoError(t, err)
// close the references
require.NoError(t, compressedFile.Close())
require.NoError(t, decompressedFile.Close())
storageDB, err := local.OpenBoltdbFile(decompressedFilePath)
require.NoError(t, err)
testutil.CompareDBs(t, db, storageDB)
require.NoError(t, storageDB.Close())
require.NoError(t, os.Remove(decompressedFilePath))
}
}
func TestTable_Cleanup(t *testing.T) {
testDir := t.TempDir()
boltDBIndexClient, storageClient := buildTestClients(t, testDir)
indexPath := filepath.Join(testDir, indexDirName)
defer func() {
boltDBIndexClient.Stop()
}()
dbRetainPeriod := time.Hour
// dbs for various scenarios to test
outsideRetention := filepath.Join(indexPath, "outside-retention")
inRetention := filepath.Join(indexPath, "in-retention")
notUploaded := filepath.Join(indexPath, "not-uploaded")
// build all the test dbs except for notUploaded
testutil.AddRecordsToDB(t, outsideRetention, boltDBIndexClient, 0, 10, nil)
testutil.AddRecordsToDB(t, inRetention, boltDBIndexClient, 10, 10, nil)
testutil.AddRecordsToDB(t, notUploaded, boltDBIndexClient, 20, 10, nil)
// load existing dbs
table, err := LoadTable(indexPath, "test", storageClient, boltDBIndexClient, false, newMetrics(nil))
require.NoError(t, err)
require.Len(t, table.dbs, 3)
defer func() {
table.Stop()
}()
require.NoError(t, table.Snapshot())
// no cleanup without upload
require.Len(t, table.dbs, 3)
require.Len(t, table.dbSnapshots, 3)
// upload outsideRetention and inRetention dbs
require.NoError(t, table.uploadDB(context.Background(), filepath.Base(outsideRetention), table.dbs[filepath.Base(outsideRetention)]))
require.NoError(t, table.uploadDB(context.Background(), filepath.Base(inRetention), table.dbs[filepath.Base(inRetention)]))
// change the upload time of outsideRetention to before dbRetainPeriod
table.dbUploadTime[filepath.Base(outsideRetention)] = time.Now().Add(-dbRetainPeriod).Add(-time.Minute)
// cleanup the dbs
require.NoError(t, table.Cleanup(dbRetainPeriod))
// there must be 2 dbs now, it should have cleaned up only outsideRetention
require.Len(t, table.dbs, 2)
require.Len(t, table.dbSnapshots, 2)
expectedDBs := []string{
inRetention,
fmt.Sprint(inRetention, snapshotFileSuffix),
notUploaded,
fmt.Sprint(notUploaded, snapshotFileSuffix),
}
// verify open dbs with the table and actual db files in the index directory
filesInfo, err := ioutil.ReadDir(indexPath)
require.NoError(t, err)
require.Len(t, filesInfo, len(expectedDBs))
for _, expectedDB := range expectedDBs {
if strings.HasSuffix(expectedDB, snapshotFileSuffix) {
_, ok := table.dbSnapshots[strings.TrimSuffix(filepath.Base(expectedDB), snapshotFileSuffix)]
require.True(t, ok)
} else {
_, ok := table.dbs[filepath.Base(expectedDB)]
require.True(t, ok)
}
_, err := os.Stat(expectedDB)
require.NoError(t, err)
}
}
func Test_LoadBoltDBsFromDir(t *testing.T) {
indexPath := t.TempDir()
// setup some dbs with a snapshot file.
tablePath := testutil.SetupDBsAtPath(t, filepath.Join(indexPath, "test-table"), map[string]testutil.DBConfig{
"db1": {
DBRecords: testutil.DBRecords{
Start: 0,
NumRecords: 10,
},
},
"db1" + tempFileSuffix: { // a snapshot file which should be ignored.
DBRecords: testutil.DBRecords{
Start: 0,
NumRecords: 10,
},
},
"db2": {
DBRecords: testutil.DBRecords{
Start: 10,
NumRecords: 10,
},
},
}, nil)
// create a boltdb file without bucket which should get removed
db, err := local.OpenBoltdbFile(filepath.Join(tablePath, "no-bucket"))
require.NoError(t, err)
require.NoError(t, db.Close())
// try loading the dbs
dbs, err := loadBoltDBsFromDir(tablePath, newMetrics(nil))
require.NoError(t, err)
// check that we have just 2 dbs
require.Len(t, dbs, 2)
require.NotNil(t, dbs["db1"])
require.NotNil(t, dbs["db2"])
// close all the open dbs
for _, boltdb := range dbs {
require.NoError(t, boltdb.Close())
}
filesInfo, err := ioutil.ReadDir(tablePath)
require.NoError(t, err)
require.Len(t, filesInfo, 2)
}
func TestTable_ImmutableUploads(t *testing.T) {
tempDir := t.TempDir()
boltDBIndexClient, storageClient := buildTestClients(t, tempDir)
indexPath := filepath.Join(tempDir, indexDirName)
defer func() {
boltDBIndexClient.Stop()
}()
// shardCutoff is calculated based on when shards are considered to not be active anymore and are safe to be uploaded.
shardCutoff := getOldestActiveShardTime()
// some dbs to setup
dbNames := []int64{
shardCutoff.Add(-ShardDBsByDuration).Unix(), // inactive shard, should upload
shardCutoff.Add(-1 * time.Minute).Unix(), // 1 minute before shard cutoff, should upload
time.Now().Truncate(ShardDBsByDuration).Unix(), // active shard, should not upload
}
dbs := map[string]testutil.DBConfig{}
for _, dbName := range dbNames {
dbs[fmt.Sprint(dbName)] = testutil.DBConfig{
DBRecords: testutil.DBRecords{
NumRecords: 10,
},
}
}
// setup some dbs for a table at a path.
tableName := "test-table"
tablePath := testutil.SetupDBsAtPath(t, filepath.Join(indexPath, tableName), dbs, nil)
table, err := LoadTable(tablePath, "test", storageClient, boltDBIndexClient, false, newMetrics(nil))
require.NoError(t, err)
require.NotNil(t, table)
defer func() {
table.Stop()
}()
// db expected to be uploaded without forcing the upload
expectedDBsToUpload := []int64{dbNames[0], dbNames[1]}
// upload dbs without forcing the upload which should not upload active shard or shard which has been active upto a minute back.
require.NoError(t, table.Upload(context.Background(), false))
// verify that only expected dbs are uploaded
objectStorageDir := filepath.Join(tempDir, objectsStorageDirName)
uploadedDBs, err := ioutil.ReadDir(filepath.Join(objectStorageDir, table.name))
require.NoError(t, err)
require.Len(t, uploadedDBs, len(expectedDBsToUpload))
for _, expectedDB := range expectedDBsToUpload {
require.FileExists(t, filepath.Join(objectStorageDir, tableName, table.buildFileName(fmt.Sprint(expectedDB))))
}
// force upload of dbs
require.NoError(t, table.Upload(context.Background(), true))
expectedDBsToUpload = dbNames
// verify that all the dbs are uploaded
uploadedDBs, err = ioutil.ReadDir(filepath.Join(objectStorageDir, table.name))
require.NoError(t, err)
require.Len(t, uploadedDBs, len(expectedDBsToUpload))
for _, expectedDB := range expectedDBsToUpload {
require.FileExists(t, filepath.Join(objectStorageDir, tableName, table.buildFileName(fmt.Sprint(expectedDB))))
}
// delete everything uploaded
dir, err := ioutil.ReadDir(filepath.Join(objectStorageDir, table.name))
require.NoError(t, err)
for _, d := range dir {
require.NoError(t, os.RemoveAll(filepath.Join(objectStorageDir, table.name, d.Name())))
}
// force upload of dbs
require.NoError(t, table.Upload(context.Background(), true))
// make sure nothing was re-uploaded
for _, expectedDB := range expectedDBsToUpload {
require.NoFileExists(t, filepath.Join(objectStorageDir, tableName, table.buildFileName(fmt.Sprint(expectedDB))))
}
}
func TestTable_MultiQueries(t *testing.T) {
indexPath := t.TempDir()
boltDBIndexClient, err := local.NewBoltDBIndexClient(local.BoltDBConfig{Directory: indexPath})
require.NoError(t, err)
defer func() {
boltDBIndexClient.Stop()
}()
user1, user2 := "user1", "user2"
// setup some dbs with default bucket and per tenant bucket for a table at a path.
tablePath := filepath.Join(indexPath, "test-table")
testutil.SetupDBsAtPath(t, tablePath, map[string]testutil.DBConfig{
"db1": {
DBRecords: testutil.DBRecords{
NumRecords: 10,
},
},
"db2": {
DBRecords: testutil.DBRecords{
Start: 10,
NumRecords: 10,
},
},
}, nil)
testutil.SetupDBsAtPath(t, tablePath, map[string]testutil.DBConfig{
"db3": {
DBRecords: testutil.DBRecords{
Start: 20,
NumRecords: 10,
},
},
"db4": {
DBRecords: testutil.DBRecords{
Start: 30,
NumRecords: 10,
},
},
}, []byte(user1))
// try loading the table.
table, err := LoadTable(tablePath, "test", nil, boltDBIndexClient, false, newMetrics(nil))
require.NoError(t, err)
require.NotNil(t, table)
defer func() {
table.Stop()
}()
require.NoError(t, table.Snapshot())
// build queries each looking for specific value from all the dbs
var queries []index.Query
for i := 5; i < 35; i++ {
queries = append(queries, index.Query{ValueEqual: []byte(strconv.Itoa(i))})
}
// querying data for user1 should return both data from common index and user1's index
testutil.TestSingleTableQuery(t, user1, queries, table, 5, 30)
// querying data for user2 should return only common index
testutil.TestSingleTableQuery(t, user2, queries, table, 5, 15)
}

@ -15,9 +15,7 @@ const (
maxConcurrency = 10
)
type TableQuerier interface {
MultiQueries(ctx context.Context, queries []index.Query, callback index.QueryPagesCallback) error
}
type QueryIndexFunc func(ctx context.Context, queries []index.Query, callback index.QueryPagesCallback) error
// QueriesByTable groups and returns queries by tables.
func QueriesByTable(queries []index.Query) map[string][]index.Query {
@ -33,12 +31,12 @@ func QueriesByTable(queries []index.Query) map[string][]index.Query {
return queriesByTable
}
func DoParallelQueries(ctx context.Context, tableQuerier TableQuerier, queries []index.Query, callback index.QueryPagesCallback) error {
func DoParallelQueries(ctx context.Context, queryIndex QueryIndexFunc, queries []index.Query, callback index.QueryPagesCallback) error {
if len(queries) == 0 {
return nil
}
if len(queries) <= maxQueriesBatch {
return tableQuerier.MultiQueries(ctx, queries, NewCallbackDeduper(callback, len(queries)))
return queryIndex(ctx, queries, NewCallbackDeduper(callback, len(queries)))
}
jobsCount := len(queries) / maxQueriesBatch
@ -47,7 +45,7 @@ func DoParallelQueries(ctx context.Context, tableQuerier TableQuerier, queries [
}
callback = NewSyncCallbackDeduper(callback, len(queries))
return concurrency.ForEachJob(ctx, jobsCount, maxConcurrency, func(ctx context.Context, idx int) error {
return tableQuerier.MultiQueries(ctx, queries[idx*maxQueriesBatch:util_math.Min((idx+1)*maxQueriesBatch, len(queries))], callback)
return queryIndex(ctx, queries[idx*maxQueriesBatch:util_math.Min((idx+1)*maxQueriesBatch, len(queries))], callback)
})
}

@ -64,7 +64,7 @@ func TestDoParallelQueries(t *testing.T) {
queries: map[string]index.Query{},
}
err := DoParallelQueries(context.Background(), &tableQuerier, queries, func(query index.Query, batch index.ReadBatchResult) bool {
err := DoParallelQueries(context.Background(), tableQuerier.MultiQueries, queries, func(query index.Query, batch index.ReadBatchResult) bool {
return false
})
require.NoError(t, err)
@ -280,7 +280,7 @@ func benchmarkMultiQueries(b *testing.B, n int) {
b.ReportAllocs()
b.ResetTimer()
for i := 0; i < b.N; i++ {
_ = DoParallelQueries(ctx, TableQuerierFunc(func(_ context.Context, queries []index.Query, callback index.QueryPagesCallback) error {
_ = DoParallelQueries(ctx, func(_ context.Context, queries []index.Query, callback index.QueryPagesCallback) error {
for _, query := range queries {
callback(query, batch{
hashValue: query.HashValue,
@ -300,7 +300,7 @@ func benchmarkMultiQueries(b *testing.B, n int) {
})
}
return nil
}), queries, callback)
}, queries, callback)
}
})
}

@ -25,6 +25,7 @@ func NewStore(indexShipperCfg indexshipper.Config, p config.PeriodConfig, f *fet
indexShipperCfg,
objectClient,
limits,
nil,
OpenShippableTSDB,
)
if err != nil {

Loading…
Cancel
Save