From cd53fc9cbf9dba863bde143f4d0792cfc23fcb7e Mon Sep 17 00:00:00 2001 From: Sandeep Sukhani Date: Mon, 23 Mar 2020 12:21:17 +0530 Subject: [PATCH 01/13] flush boltdb to object store files are stored in folder per periodic table and are named after ingester flushed every 15 mins to make index available to other services files are also flushed before ingester stops to avoid any data loss new stores can be implemented easily ingester to also query store when using boltdb --- go.mod | 1 + pkg/ingester/flush_test.go | 4 +- pkg/ingester/ingester.go | 42 ++- pkg/ingester/ingester_test.go | 7 + pkg/ingester/instance.go | 17 +- pkg/logql/stats/context.go | 3 +- pkg/logql/stats/grpc.go | 74 +++-- pkg/logql/stats/grpc_test.go | 20 +- pkg/loki/modules.go | 40 +++ pkg/loki/modules_test.go | 30 ++ pkg/storage/store.go | 30 +- pkg/storage/stores/factory.go | 30 ++ .../stores/local/boltdb_index_client.go | 73 +++++ pkg/storage/stores/local/downloads.go | 142 +++++++++ pkg/storage/stores/local/shipper.go | 285 ++++++++++++++++++ pkg/storage/stores/local/uploads.go | 101 +++++++ 16 files changed, 844 insertions(+), 55 deletions(-) create mode 100644 pkg/storage/stores/factory.go create mode 100644 pkg/storage/stores/local/boltdb_index_client.go create mode 100644 pkg/storage/stores/local/downloads.go create mode 100644 pkg/storage/stores/local/shipper.go create mode 100644 pkg/storage/stores/local/uploads.go diff --git a/go.mod b/go.mod index 5c925cf393e8..f03c7d95fc44 100644 --- a/go.mod +++ b/go.mod @@ -51,6 +51,7 @@ require ( github.com/uber/jaeger-client-go v2.20.1+incompatible github.com/ugorji/go v1.1.7 // indirect github.com/weaveworks/common v0.0.0-20200310113808-2708ba4e60a4 + go.etcd.io/bbolt v1.3.3 go.etcd.io/etcd v0.0.0-20200401174654-e694b7bb0875 // indirect golang.org/x/net v0.0.0-20200226121028-0de0cce0169b google.golang.org/grpc v1.25.1 diff --git a/pkg/ingester/flush_test.go b/pkg/ingester/flush_test.go index d1a99bdff2fe..82e236e61c76 100644 --- a/pkg/ingester/flush_test.go +++ b/pkg/ingester/flush_test.go @@ -7,6 +7,8 @@ import ( "testing" "time" + "github.com/grafana/loki/pkg/logql" + "github.com/cortexproject/cortex/pkg/chunk" "github.com/cortexproject/cortex/pkg/ring" "github.com/cortexproject/cortex/pkg/ring/kv" @@ -223,7 +225,7 @@ func (s *testStore) IsLocal() bool { return false } -func (s *testStore) LazyQuery(ctx context.Context, req *logproto.QueryRequest) (iter.EntryIterator, error) { +func (s *testStore) LazyQuery(ctx context.Context, req logql.SelectParams) (iter.EntryIterator, error) { return nil, nil } diff --git a/pkg/ingester/ingester.go b/pkg/ingester/ingester.go index 7a5b8c1bf4ee..cf3a6b9f80b4 100644 --- a/pkg/ingester/ingester.go +++ b/pkg/ingester/ingester.go @@ -63,6 +63,9 @@ type Config struct { // For testing, you can override the address and ID of this ingester. ingesterClientFactory func(cfg client.Config, addr string) (client.HealthAndIngesterClient, error) + + QueryStore bool `yaml:"-"` + QueryStoreMaxLookBackPeriod time.Duration `yaml:"-"` } // RegisterFlags registers the flags. @@ -113,6 +116,7 @@ type Ingester struct { // ChunkStore is the interface we need to store chunks. type ChunkStore interface { Put(ctx context.Context, chunks []chunk.Chunk) error + LazyQuery(ctx context.Context, req logql.SelectParams) (iter.EntryIterator, error) } // New makes a new Ingester. @@ -241,13 +245,47 @@ func (i *Ingester) getOrCreateInstance(instanceID string) *instance { // Query the ingests for log streams matching a set of matchers. func (i *Ingester) Query(req *logproto.QueryRequest, queryServer logproto.Querier_QueryServer) error { - instanceID, err := user.ExtractOrgID(queryServer.Context()) + // initialize stats collection for ingester queries and set grpc trailer with stats. + ctx := stats.NewContext(queryServer.Context()) + defer stats.SendAsTrailer(ctx, queryServer) + + instanceID, err := user.ExtractOrgID(ctx) if err != nil { return err } instance := i.getOrCreateInstance(instanceID) - return instance.Query(req, queryServer) + itr, err := instance.Query(ctx, req) + if err != nil { + return err + } + + defer helpers.LogError("closing iterator", itr.Close) + + if i.cfg.QueryStore { + start := req.Start + end := req.End + if i.cfg.QueryStoreMaxLookBackPeriod != 0 { + oldestStartTime := time.Now().Add(-i.cfg.QueryStoreMaxLookBackPeriod) + if oldestStartTime.After(req.Start) { + start = oldestStartTime + } + } + + if start.Before(end) { + req.Start = start + req.End = end + + storeItr, err := i.store.LazyQuery(ctx, logql.SelectParams{QueryRequest: req}) + if err != nil { + return err + } + + itr.Push(storeItr) + } + } + + return sendBatches(queryServer.Context(), itr, queryServer, req.Limit) } // Label returns the set of labels for the stream this ingester knows about. diff --git a/pkg/ingester/ingester_test.go b/pkg/ingester/ingester_test.go index acd00e7dff20..40a0684f4989 100644 --- a/pkg/ingester/ingester_test.go +++ b/pkg/ingester/ingester_test.go @@ -7,6 +7,9 @@ import ( "testing" "time" + "github.com/grafana/loki/pkg/iter" + "github.com/grafana/loki/pkg/logql" + "github.com/stretchr/testify/require" "github.com/weaveworks/common/httpgrpc" "github.com/weaveworks/common/user" @@ -247,6 +250,10 @@ func (s *mockStore) Put(ctx context.Context, chunks []chunk.Chunk) error { return nil } +func (s *mockStore) LazyQuery(ctx context.Context, req logql.SelectParams) (iter.EntryIterator, error) { + return nil, nil +} + type mockQuerierServer struct { ctx context.Context resps []*logproto.QueryResponse diff --git a/pkg/ingester/instance.go b/pkg/ingester/instance.go index 5724effb5756..d10957e87fa3 100644 --- a/pkg/ingester/instance.go +++ b/pkg/ingester/instance.go @@ -186,18 +186,14 @@ func (i *instance) getLabelsFromFingerprint(fp model.Fingerprint) labels.Labels return s.labels } -func (i *instance) Query(req *logproto.QueryRequest, queryServer logproto.Querier_QueryServer) error { - // initialize stats collection for ingester queries and set grpc trailer with stats. - ctx := stats.NewContext(queryServer.Context()) - defer stats.SendAsTrailer(ctx, queryServer) - +func (i *instance) Query(ctx context.Context, req *logproto.QueryRequest) (iter.HeapIterator, error) { expr, err := (logql.SelectParams{QueryRequest: req}).LogSelector() if err != nil { - return err + return nil, err } filter, err := expr.Filter() if err != nil { - return err + return nil, err } ingStats := stats.GetIngesterData(ctx) @@ -215,13 +211,10 @@ func (i *instance) Query(req *logproto.QueryRequest, queryServer logproto.Querie }, ) if err != nil { - return err + return nil, err } - iter := iter.NewHeapIterator(ctx, iters, req.Direction) - defer helpers.LogError("closing iterator", iter.Close) - - return sendBatches(ctx, iter, queryServer, req.Limit) + return iter.NewHeapIterator(ctx, iters, req.Direction), nil } func (i *instance) Label(_ context.Context, req *logproto.LabelRequest) (*logproto.LabelResponse, error) { diff --git a/pkg/logql/stats/context.go b/pkg/logql/stats/context.go index 75dacb9e5912..6b5a6800b3e6 100644 --- a/pkg/logql/stats/context.go +++ b/pkg/logql/stats/context.go @@ -138,9 +138,8 @@ func GetStoreData(ctx context.Context) *StoreData { // Snapshot compute query statistics from a context using the total exec time. func Snapshot(ctx context.Context, execTime time.Duration) Result { - var res Result // ingester data is decoded from grpc trailers. - res.Ingester = decodeTrailers(ctx) + res := decodeTrailers(ctx) // collect data from store. s, ok := ctx.Value(storeKey).(*StoreData) if ok { diff --git a/pkg/logql/stats/grpc.go b/pkg/logql/stats/grpc.go index 8dad127c6b60..e1640fda5edc 100644 --- a/pkg/logql/stats/grpc.go +++ b/pkg/logql/stats/grpc.go @@ -14,6 +14,7 @@ import ( const ( ingesterDataKey = "ingester_data" chunkDataKey = "chunk_data" + storeDataKey = "store_data" ) type trailerCollector struct { @@ -70,32 +71,45 @@ func encodeTrailer(ctx context.Context) (metadata.MD, error) { } meta.Set(chunkDataKey, data) } + + storeData, ok := ctx.Value(storeKey).(*StoreData) + if ok { + data, err := jsoniter.MarshalToString(storeData) + if err != nil { + return meta, err + } + meta.Set(storeDataKey, data) + } + return meta, nil } -func decodeTrailers(ctx context.Context) Ingester { - var res Ingester +func decodeTrailers(ctx context.Context) Result { + var res Result collector, ok := ctx.Value(trailersKey).(*trailerCollector) if !ok { return res } - res.TotalReached = int32(len(collector.trailers)) + res.Ingester.TotalReached = int32(len(collector.trailers)) for _, meta := range collector.trailers { ing := decodeTrailer(meta) - res.TotalChunksMatched += ing.TotalChunksMatched - res.TotalBatches += ing.TotalBatches - res.TotalLinesSent += ing.TotalLinesSent - res.HeadChunkBytes += ing.HeadChunkBytes - res.HeadChunkLines += ing.HeadChunkLines - res.DecompressedBytes += ing.DecompressedBytes - res.DecompressedLines += ing.DecompressedLines - res.CompressedBytes += ing.CompressedBytes - res.TotalDuplicates += ing.TotalDuplicates + res.Ingester.TotalChunksMatched += ing.Ingester.TotalChunksMatched + res.Ingester.TotalBatches += ing.Ingester.TotalBatches + res.Ingester.TotalLinesSent += ing.Ingester.TotalLinesSent + res.Ingester.HeadChunkBytes += ing.Ingester.HeadChunkBytes + res.Ingester.HeadChunkLines += ing.Ingester.HeadChunkLines + res.Ingester.DecompressedBytes += ing.Ingester.DecompressedBytes + res.Ingester.DecompressedLines += ing.Ingester.DecompressedLines + res.Ingester.CompressedBytes += ing.Ingester.CompressedBytes + res.Ingester.TotalDuplicates += ing.Ingester.TotalDuplicates + res.Store.TotalChunksRef += ing.Store.TotalChunksRef + res.Store.TotalChunksDownloaded += ing.Store.TotalChunksDownloaded + res.Store.ChunksDownloadTime += ing.Store.ChunksDownloadTime } return res } -func decodeTrailer(meta *metadata.MD) Ingester { +func decodeTrailer(meta *metadata.MD) Result { var ingData IngesterData values := meta.Get(ingesterDataKey) if len(values) == 1 { @@ -110,15 +124,29 @@ func decodeTrailer(meta *metadata.MD) Ingester { level.Warn(util.Logger).Log("msg", "could not unmarshal chunk data", "err", err) } } - return Ingester{ - TotalChunksMatched: ingData.TotalChunksMatched, - TotalBatches: ingData.TotalBatches, - TotalLinesSent: ingData.TotalLinesSent, - HeadChunkBytes: chunkData.HeadChunkBytes, - HeadChunkLines: chunkData.HeadChunkLines, - DecompressedBytes: chunkData.DecompressedBytes, - DecompressedLines: chunkData.DecompressedLines, - CompressedBytes: chunkData.CompressedBytes, - TotalDuplicates: chunkData.TotalDuplicates, + var storeData StoreData + values = meta.Get(storeDataKey) + if len(values) == 1 { + if err := jsoniter.UnmarshalFromString(values[0], &storeData); err != nil { + level.Warn(util.Logger).Log("msg", "could not unmarshal chunk data", "err", err) + } + } + return Result{ + Ingester: Ingester{ + TotalChunksMatched: ingData.TotalChunksMatched, + TotalBatches: ingData.TotalBatches, + TotalLinesSent: ingData.TotalLinesSent, + HeadChunkBytes: chunkData.HeadChunkBytes, + HeadChunkLines: chunkData.HeadChunkLines, + DecompressedBytes: chunkData.DecompressedBytes, + DecompressedLines: chunkData.DecompressedLines, + CompressedBytes: chunkData.CompressedBytes, + TotalDuplicates: chunkData.TotalDuplicates, + }, + Store: Store{ + TotalChunksRef: storeData.TotalChunksRef, + TotalChunksDownloaded: storeData.TotalChunksDownloaded, + ChunksDownloadTime: storeData.ChunksDownloadTime.Seconds(), + }, } } diff --git a/pkg/logql/stats/grpc_test.go b/pkg/logql/stats/grpc_test.go index 92f68a31cb40..d44538f81cae 100644 --- a/pkg/logql/stats/grpc_test.go +++ b/pkg/logql/stats/grpc_test.go @@ -82,16 +82,16 @@ func TestCollectTrailer(t *testing.T) { t.Fatal(err) } res := decodeTrailers(ctx) - require.Equal(t, int32(2), res.TotalReached) - require.Equal(t, int64(2), res.TotalChunksMatched) - require.Equal(t, int64(4), res.TotalBatches) - require.Equal(t, int64(6), res.TotalLinesSent) - require.Equal(t, int64(2), res.HeadChunkBytes) - require.Equal(t, int64(2), res.HeadChunkLines) - require.Equal(t, int64(2), res.DecompressedBytes) - require.Equal(t, int64(2), res.DecompressedLines) - require.Equal(t, int64(2), res.CompressedBytes) - require.Equal(t, int64(2), res.TotalDuplicates) + require.Equal(t, int32(2), res.Ingester.TotalReached) + require.Equal(t, int64(2), res.Ingester.TotalChunksMatched) + require.Equal(t, int64(4), res.Ingester.TotalBatches) + require.Equal(t, int64(6), res.Ingester.TotalLinesSent) + require.Equal(t, int64(2), res.Ingester.HeadChunkBytes) + require.Equal(t, int64(2), res.Ingester.HeadChunkLines) + require.Equal(t, int64(2), res.Ingester.DecompressedBytes) + require.Equal(t, int64(2), res.Ingester.DecompressedLines) + require.Equal(t, int64(2), res.Ingester.CompressedBytes) + require.Equal(t, int64(2), res.Ingester.TotalDuplicates) } type ingesterFn func(*logproto.QueryRequest, logproto.Querier_QueryServer) error diff --git a/pkg/loki/modules.go b/pkg/loki/modules.go index de6717ae6642..8858d517ea79 100644 --- a/pkg/loki/modules.go +++ b/pkg/loki/modules.go @@ -5,9 +5,14 @@ import ( "fmt" "net/http" "os" + "sort" "strings" "time" + "github.com/prometheus/common/model" + + "github.com/grafana/loki/pkg/storage/stores/local" + "github.com/cortexproject/cortex/pkg/chunk" "github.com/cortexproject/cortex/pkg/chunk/storage" "github.com/cortexproject/cortex/pkg/querier/frontend" @@ -197,6 +202,14 @@ func (t *Loki) initIngester() (err error) { t.cfg.Ingester.LifecyclerConfig.RingConfig.KVStore.Multi.ConfigProvider = multiClientRuntimeConfigChannel(t.runtimeConfig) t.cfg.Ingester.LifecyclerConfig.RingConfig.KVStore.MemberlistKV = t.memberlistKV.GetMemberlistKV t.cfg.Ingester.LifecyclerConfig.ListenPort = &t.cfg.Server.GRPCListenPort + + // We want ingester to also query the store when using boltdb-shipper + if activeIndexType(t.cfg.SchemaConfig) == local.BoltDBShipperType { + t.cfg.Ingester.QueryStore = true + // When using shipper, limit max look back for query to MaxChunkAge + upload interval by shipper + 15 mins to query only data whose index is not pushed yet + t.cfg.Ingester.QueryStoreMaxLookBackPeriod = t.cfg.Ingester.MaxChunkAge + local.ShipperFileUploadInterval + (15 * time.Minute) + } + t.ingester, err = ingester.New(t.cfg.Ingester, t.cfg.IngesterClient, t.store, t.overrides) if err != nil { return @@ -277,6 +290,20 @@ func (t *Loki) stopTableManager() error { } func (t *Loki) initStore() (err error) { + if activeIndexType(t.cfg.SchemaConfig) == local.BoltDBShipperType { + t.cfg.StorageConfig.BoltDBShipperConfig.IngesterName = t.cfg.Ingester.LifecyclerConfig.ID + switch t.cfg.Target { + case Ingester: + // We do not want ingester to unnecessarily keep downloading files + t.cfg.StorageConfig.BoltDBShipperConfig.Mode = local.ShipperModeWriteOnly + case Querier: + // We do not want query to do any updates to index + t.cfg.StorageConfig.BoltDBShipperConfig.Mode = local.ShipperModeReadOnly + default: + t.cfg.StorageConfig.BoltDBShipperConfig.Mode = local.ShipperModeReadWrite + } + } + t.store, err = loki_storage.NewStore(t.cfg.StorageConfig, t.cfg.ChunkStoreConfig, t.cfg.SchemaConfig, t.overrides) return } @@ -473,3 +500,16 @@ var modules = map[moduleName]module{ deps: []moduleName{Querier, Ingester, Distributor, TableManager}, }, } + +// activeIndexType type returns index type which would be applicable to logs that would be pushed starting now +// Note: Another periodic config can be applicable in future which can change index type +func activeIndexType(cfg chunk.SchemaConfig) string { + now := model.Now() + i := sort.Search(len(cfg.Configs), func(i int) bool { + return cfg.Configs[i].From.Time > now + }) + if i > 0 { + i-- + } + return cfg.Configs[i].IndexType +} diff --git a/pkg/loki/modules_test.go b/pkg/loki/modules_test.go index 99832efd86c1..c30624738820 100644 --- a/pkg/loki/modules_test.go +++ b/pkg/loki/modules_test.go @@ -2,7 +2,10 @@ package loki import ( "testing" + "time" + "github.com/cortexproject/cortex/pkg/chunk" + "github.com/prometheus/common/model" "github.com/stretchr/testify/assert" ) @@ -35,3 +38,30 @@ func TestUniqueDeps(t *testing.T) { expected := []moduleName{Server, Overrides, Distributor, Ingester} assert.Equal(t, expected, uniqueDeps(input)) } + +func TestActiveIndexType(t *testing.T) { + var cfg chunk.SchemaConfig + + // just one PeriodConfig in the past + cfg.Configs = []chunk.PeriodConfig{{ + From: chunk.DayTime{Time: model.Now().Add(-24 * time.Hour)}, + IndexType: "first", + }} + + assert.Equal(t, "first", activeIndexType(cfg)) + + // add a newer PeriodConfig in the past which should be considered + cfg.Configs = append(cfg.Configs, chunk.PeriodConfig{ + From: chunk.DayTime{Time: model.Now().Add(-12 * time.Hour)}, + IndexType: "second", + }) + assert.Equal(t, "second", activeIndexType(cfg)) + + // add a newer PeriodConfig in the future which should not be considered + cfg.Configs = append(cfg.Configs, chunk.PeriodConfig{ + From: chunk.DayTime{Time: model.Now().Add(time.Hour)}, + IndexType: "third", + }) + assert.Equal(t, "second", activeIndexType(cfg)) + +} diff --git a/pkg/storage/store.go b/pkg/storage/store.go index b03992c45b1f..57454a47c57b 100644 --- a/pkg/storage/store.go +++ b/pkg/storage/store.go @@ -5,25 +5,28 @@ import ( "flag" "sort" + "github.com/cortexproject/cortex/pkg/chunk" + cortex_local "github.com/cortexproject/cortex/pkg/chunk/local" + "github.com/cortexproject/cortex/pkg/chunk/storage" "github.com/prometheus/common/model" "github.com/prometheus/prometheus/pkg/labels" "github.com/weaveworks/common/user" - "github.com/cortexproject/cortex/pkg/chunk" - "github.com/cortexproject/cortex/pkg/chunk/storage" - "github.com/grafana/loki/pkg/chunkenc" "github.com/grafana/loki/pkg/iter" "github.com/grafana/loki/pkg/logproto" "github.com/grafana/loki/pkg/logql" "github.com/grafana/loki/pkg/logql/stats" + "github.com/grafana/loki/pkg/storage/stores" + "github.com/grafana/loki/pkg/storage/stores/local" "github.com/grafana/loki/pkg/util" ) // Config is the loki storage configuration type Config struct { - storage.Config `yaml:",inline"` - MaxChunkBatchSize int `yaml:"max_chunk_batch_size"` + storage.Config `yaml:",inline"` + MaxChunkBatchSize int `yaml:"max_chunk_batch_size"` + BoltDBShipperConfig local.ShipperConfig `yaml:"boltdb_shipper_config"` } // RegisterFlags adds the flags required to configure this flag set. @@ -46,6 +49,10 @@ type store struct { // NewStore creates a new Loki Store using configuration supplied. func NewStore(cfg Config, storeCfg chunk.StoreConfig, schemaCfg chunk.SchemaConfig, limits storage.StoreLimits) (Store, error) { + err := registerCustomIndexClients(cfg) + if err != nil { + return nil, err + } s, err := storage.NewStore(cfg.Config, storeCfg, schemaCfg, limits) if err != nil { return nil, err @@ -201,3 +208,16 @@ func filterChunksByTime(from, through model.Time, chunks []chunk.Chunk) []chunk. } return filtered } + +func registerCustomIndexClients(cfg Config) error { + storage.RegisterIndexClient(local.BoltDBShipperType, func() (chunk.IndexClient, error) { + objectClient, err := stores.NewObjectClient(cfg.BoltDBShipperConfig.StoreConfig) + if err != nil { + return nil, err + } + + return local.NewBoltDBIndexClient(cortex_local.BoltDBConfig{Directory: cfg.BoltDBShipperConfig.ActiveIndexDirectory}, objectClient, cfg.BoltDBShipperConfig) + }) + + return nil +} diff --git a/pkg/storage/stores/factory.go b/pkg/storage/stores/factory.go new file mode 100644 index 000000000000..b877b9e27f74 --- /dev/null +++ b/pkg/storage/stores/factory.go @@ -0,0 +1,30 @@ +package stores + +import ( + "context" + "fmt" + + "github.com/cortexproject/cortex/pkg/chunk" + "github.com/cortexproject/cortex/pkg/chunk/aws" + "github.com/cortexproject/cortex/pkg/chunk/azure" + "github.com/cortexproject/cortex/pkg/chunk/gcp" + cortex_local "github.com/cortexproject/cortex/pkg/chunk/local" + + "github.com/grafana/loki/pkg/storage/stores/local" +) + +// NewObjectClient makes a new ObjectClient of the desired type. +func NewObjectClient(cfg local.StoreConfig) (chunk.ObjectClient, error) { + switch cfg.Store { + case "aws", "s3": + return aws.NewS3ObjectClient(cfg.AWSStorageConfig.S3Config) + case "gcs": + return gcp.NewGCSObjectClient(context.Background(), cfg.GCSConfig) + case "azure": + return azure.NewBlobStorage(&cfg.Azure) + case "filesystem": + return cortex_local.NewFSObjectClient(cfg.FSConfig) + default: + return nil, fmt.Errorf("unrecognized storage client %v, choose one of: aws, s3, gcp, azure, filesystem", cfg.Store) + } +} diff --git a/pkg/storage/stores/local/boltdb_index_client.go b/pkg/storage/stores/local/boltdb_index_client.go new file mode 100644 index 000000000000..8e43b9962344 --- /dev/null +++ b/pkg/storage/stores/local/boltdb_index_client.go @@ -0,0 +1,73 @@ +package local + +import ( + "context" + "sync" + + "github.com/cortexproject/cortex/pkg/chunk" + "github.com/cortexproject/cortex/pkg/chunk/local" + chunk_util "github.com/cortexproject/cortex/pkg/chunk/util" + "go.etcd.io/bbolt" +) + +type BoltdbIndexClientWithArchiver struct { + *local.BoltIndexClient + + shipper *Shipper + + done chan struct{} + wait sync.WaitGroup +} + +// NewBoltDBIndexClient creates a new IndexClient that used BoltDB. +func NewBoltDBIndexClient(cfg local.BoltDBConfig, archiveStoreClient chunk.ObjectClient, archiverCfg ShipperConfig) (chunk.IndexClient, error) { + boltDBIndexClient, err := local.NewBoltDBIndexClient(cfg) + if err != nil { + return nil, err + } + + archiver, err := NewShipper(archiverCfg, archiveStoreClient, func(name string, operation int) (db *bbolt.DB, e error) { + return boltDBIndexClient.GetDB(name, operation) + }) + if err != nil { + return nil, err + } + + indexClient := BoltdbIndexClientWithArchiver{ + BoltIndexClient: boltDBIndexClient, + shipper: archiver, + done: make(chan struct{}), + } + + return &indexClient, nil +} + +func (b *BoltdbIndexClientWithArchiver) Stop() { + close(b.done) + + b.BoltIndexClient.Stop() + b.shipper.Stop() + + b.wait.Wait() +} + +func (b *BoltdbIndexClientWithArchiver) QueryPages(ctx context.Context, queries []chunk.IndexQuery, callback func(chunk.IndexQuery, chunk.ReadBatch) (shouldContinue bool)) error { + return chunk_util.DoParallelQueries(ctx, b.query, queries, callback) +} + +func (b *BoltdbIndexClientWithArchiver) query(ctx context.Context, query chunk.IndexQuery, callback func(chunk.ReadBatch) (shouldContinue bool)) error { + db, err := b.GetDB(query.TableName, local.DBOperationRead) + if err != nil && err != local.ErrUnexistentBoltDB { + return err + } + + if db != nil { + if err := b.QueryDB(ctx, db, query, callback); err != nil { + return err + } + } + + return b.shipper.forEach(ctx, query.TableName, func(db *bbolt.DB) error { + return b.QueryDB(ctx, db, query, callback) + }) +} diff --git a/pkg/storage/stores/local/downloads.go b/pkg/storage/stores/local/downloads.go new file mode 100644 index 000000000000..e50fb69e5319 --- /dev/null +++ b/pkg/storage/stores/local/downloads.go @@ -0,0 +1,142 @@ +package local + +import ( + "context" + "fmt" + "io" + "os" + "path" + "strings" + "time" + + "github.com/cortexproject/cortex/pkg/chunk" + "github.com/cortexproject/cortex/pkg/chunk/local" + chunk_util "github.com/cortexproject/cortex/pkg/chunk/util" + "github.com/cortexproject/cortex/pkg/util" + "github.com/go-kit/kit/log/level" +) + +// downloadFilesForPeriod downloads all the files from for given period from all the uploaders +func (a *Shipper) downloadFilesForPeriod(ctx context.Context, period string, fc *filesCollection) error { + toDownload, toDelete, err := a.checkStorageForUpdates(ctx, period, fc) + if err != nil { + return err + } + + for _, afd := range toDownload { + err := a.downloadFile(ctx, period, afd, fc) + if err != nil { + return err + } + } + + for _, afd := range toDelete { + err := a.deleteFileFromCache(period, afd.uploader, fc) + 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 (a *Shipper) checkStorageForUpdates(ctx context.Context, period string, fc *filesCollection) (toDownload []shippedFileDetails, toDelete []shippedFileDetails, err error) { + if a.cfg.Mode == ShipperModeWriteOnly { + return + } + + fc.RLock() + defer fc.RUnlock() + + // listing tables from store + var objects []chunk.StorageObject + objects, err = a.storageClient.List(ctx, period) + if err != nil { + return + } + + listedUploaders := make(map[string]struct{}, len(objects)) + + for _, object := range objects { + uploader := strings.Split(object.Key, "/")[1] + listedUploaders[uploader] = struct{}{} + + // Checking whether file was updated in the store after we downloaded it, if not, no need to include it in updates + downloadedFileDetails, ok := fc.files[uploader] + if !ok || downloadedFileDetails.mtime != object.ModifiedAt { + toDownload = append(toDownload, shippedFileDetails{uploader: uploader, mtime: object.ModifiedAt}) + } + } + + for uploader, fileDetails := range fc.files { + if _, isOK := listedUploaders[uploader]; !isOK { + toDelete = append(toDelete, shippedFileDetails{uploader: uploader, mtime: fileDetails.mtime}) + } + } + + return +} + +// downloadFile downloads a file from storage to cache. +// It first downloads it to a temp file so that we close the existing file(if already exists), replace it with new one and then reopen it. +func (a *Shipper) downloadFile(ctx context.Context, period string, afd shippedFileDetails, fc *filesCollection) error { + objectKey := fmt.Sprintf("%s/%s", period, afd.uploader) + readCloser, err := a.storageClient.GetObject(ctx, objectKey) + if err != nil { + return err + } + + defer func() { + if err := readCloser.Close(); err != nil { + level.Error(util.Logger) + } + }() + + downloadPath := path.Join(a.cfg.CacheLocation, period) + err = chunk_util.EnsureDirectory(downloadPath) + if err != nil { + return err + } + + // download the file temporarily with some other name to allow boltdb client to close the existing file first if it exists + tempFilePath := path.Join(downloadPath, fmt.Sprintf("%s.%d", afd.uploader, time.Now().Unix())) + + f, err := os.Create(tempFilePath) + if err != nil { + return err + } + + _, err = io.Copy(f, readCloser) + if err != nil { + return err + } + + fc.Lock() + defer fc.Unlock() + + af, ok := fc.files[afd.uploader] + if ok { + if err := af.boltdb.Close(); err != nil { + return err + } + } else { + af = shippedFile{downloadLocation: path.Join(downloadPath, afd.uploader)} + } + + // move the file from temp location to actual location + err = os.Rename(tempFilePath, af.downloadLocation) + if err != nil { + return err + } + + af.mtime = afd.mtime + af.boltdb, err = local.OpenBoltdbFile(af.downloadLocation) + if err != nil { + return err + } + + fc.files[afd.uploader] = af + + return nil +} diff --git a/pkg/storage/stores/local/shipper.go b/pkg/storage/stores/local/shipper.go new file mode 100644 index 000000000000..ee5c084e1462 --- /dev/null +++ b/pkg/storage/stores/local/shipper.go @@ -0,0 +1,285 @@ +package local + +import ( + "context" + "flag" + "fmt" + "os" + "path" + "sync" + "time" + + "github.com/cortexproject/cortex/pkg/chunk" + "github.com/cortexproject/cortex/pkg/chunk/aws" + "github.com/cortexproject/cortex/pkg/chunk/azure" + "github.com/cortexproject/cortex/pkg/chunk/gcp" + "github.com/cortexproject/cortex/pkg/chunk/local" + chunk_util "github.com/cortexproject/cortex/pkg/chunk/util" + "github.com/cortexproject/cortex/pkg/util" + "github.com/go-kit/kit/log/level" + "go.etcd.io/bbolt" +) + +const ( + // ShipperModeReadWrite is to allow both read and write + ShipperModeReadWrite = iota + // ShipperModeReadOnly is to allow only read operations + ShipperModeReadOnly + // ShipperModeWriteOnly is to allow only write operations + ShipperModeWriteOnly + + // ShipperFileUploadInterval defines interval for uploading active boltdb files from local which are being written to by ingesters. + ShipperFileUploadInterval = 15 * time.Minute + + cacheCleanupInterval = 24 * time.Hour + + // BoltDBShipperType holds the index type for using boltdb with shipper which keeps flushing them to a shared storage + BoltDBShipperType = "boltdb-shipper" +) + +type StoreConfig struct { + Store string `yaml:"store"` + AWSStorageConfig aws.StorageConfig `yaml:"aws"` + GCSConfig gcp.GCSConfig `yaml:"gcs"` + FSConfig local.FSConfig `yaml:"filesystem"` + Azure azure.BlobStorageConfig `yaml:"azure"` +} + +type ShipperConfig struct { + ActiveIndexDirectory string `yaml:"active_index_directory"` + CacheLocation string `yaml:"cache_location"` + CacheTTL time.Duration `yaml:"cache_ttl"` + StoreConfig StoreConfig `yaml:"store_config"` + ResyncInterval time.Duration `yaml:"resync_interval"` + IngesterName string `yaml:"-"` + Mode int `yaml:"-"` + BoltdbDirectory string `yaml:"-"` +} + +// RegisterFlags registers flags. +func (cfg *ShipperConfig) RegisterFlags(f *flag.FlagSet) { + storeFlagsPrefix := "boltdb.shipper." + cfg.StoreConfig.AWSStorageConfig.RegisterFlagsWithPrefix(storeFlagsPrefix, f) + cfg.StoreConfig.GCSConfig.RegisterFlagsWithPrefix(storeFlagsPrefix, f) + cfg.StoreConfig.FSConfig.RegisterFlagsWithPrefix(storeFlagsPrefix, f) + + f.StringVar(&cfg.ActiveIndexDirectory, "boltdb.active-index-directory", "filesystem", "Directory where ingesters would write boltdb files which would then be uploaded by shipper to configured storage") + f.StringVar(&cfg.StoreConfig.Store, "boltdb.shipper.store", "filesystem", "Store for keeping boltdb files") + f.StringVar(&cfg.CacheLocation, "boltdb.shipper.cache-location", "", "Cache location for restoring boltDB files for queries") + f.DurationVar(&cfg.CacheTTL, "boltdb.shipper.cache-ttl", 24*time.Hour, "TTL for boltDB files restored in cache for queries") + f.DurationVar(&cfg.ResyncInterval, "boltdb.shipper.resync-interval", 5*time.Minute, "Resync downloaded files with the storage") +} + +type shippedFileDetails struct { + uploader string + mtime time.Time +} + +type shippedFile struct { + downloadLocation string + mtime time.Time + boltdb *bbolt.DB +} + +// filesCollection holds info about shipped boltdb index files by other uploaders(ingesters). +// It is generally used to hold boltdb files created by all the ingesters for same period i.e with same name. +// In the object store files are uploaded as / to manage files with same name from different ingesters +type filesCollection struct { + sync.RWMutex + lastUsedAt time.Time + files map[string]shippedFile +} + +type Shipper struct { + cfg ShipperConfig + localBoltdbGetter func(name string, operation int) (*bbolt.DB, error) + + // downloadedPeriods holds mapping for period -> filesCollection. + // Here period is name of the file created by ingesters for a specific period. + downloadedPeriods map[string]*filesCollection + downloadedPeriodsMtx sync.RWMutex + storageClient chunk.ObjectClient + done chan struct{} + + uploader string + uploadedFilesMtime map[string]time.Time + uploadedFilesMtimeMtx sync.RWMutex +} + +// NewShipper creates a shipper for syncing local objects with a store +func NewShipper(cfg ShipperConfig, storageClient chunk.ObjectClient, localBoltdbGetter func(name string, operation int) (*bbolt.DB, error)) (*Shipper, error) { + if err := chunk_util.EnsureDirectory(cfg.CacheLocation); err != nil { + return nil, err + } + + shipper := Shipper{ + cfg: cfg, + localBoltdbGetter: localBoltdbGetter, + downloadedPeriods: map[string]*filesCollection{}, + storageClient: storageClient, + done: make(chan struct{}), + // We would use ingester name and startup timestamp for naming files while uploading so that + // ingester does not override old files when using same id + uploader: fmt.Sprintf("%s-%d", cfg.IngesterName, time.Now().Unix()), + } + + go shipper.loop() + + return &shipper, nil +} + +func (a *Shipper) loop() { + resyncTicker := time.NewTicker(a.cfg.ResyncInterval) + defer resyncTicker.Stop() + + uploadFilesTicker := time.NewTicker(ShipperFileUploadInterval) + defer uploadFilesTicker.Stop() + + cacheCleanupTicker := time.NewTicker(cacheCleanupInterval) + defer cacheCleanupTicker.Stop() + + for { + select { + case <-resyncTicker.C: + err := a.syncLocalWithStorage(context.Background()) + if err != nil { + level.Error(util.Logger).Log("msg", "error syncing local boltdb files with storage", "err", err) + } + case <-uploadFilesTicker.C: + err := a.uploadFiles(context.Background()) + if err != nil { + level.Error(util.Logger).Log("msg", "error pushing archivable files to store", "err", err) + } + case <-cacheCleanupTicker.C: + err := a.cleanupCache() + if err != nil { + level.Error(util.Logger).Log("msg", "error cleaning up expired tables", "err", err) + } + case <-a.done: + return + } + } +} + +// Stop the shipper and push all the local files to the store +func (a *Shipper) Stop() { + close(a.done) + + // Push all boltdb files to storage before returning + err := a.syncLocalWithStorage(context.Background()) + if err != nil { + level.Error(util.Logger).Log("msg", "error pushing archivable files to store", "err", err) + } + + a.downloadedPeriodsMtx.Lock() + defer a.downloadedPeriodsMtx.Unlock() + + for _, fc := range a.downloadedPeriods { + fc.Lock() + for _, fl := range fc.files { + _ = fl.boltdb.Close() + } + fc.Unlock() + } +} + +// cleanupCache removes all the files for a period which has not be queried for using the configured TTL +func (a *Shipper) cleanupCache() error { + a.downloadedPeriodsMtx.Lock() + defer a.downloadedPeriodsMtx.Unlock() + + for period, fc := range a.downloadedPeriods { + if fc.lastUsedAt.Add(a.cfg.CacheTTL).Before(time.Now()) { + for uploader := range fc.files { + if err := a.deleteFileFromCache(period, uploader, fc); err != nil { + return err + } + } + + delete(a.downloadedPeriods, period) + } + } + + return nil +} + +// syncLocalWithStorage syncs all the periods that we have in the cache with the storage +// i.e download new and updated files and remove files which were delete from the storage. +func (a *Shipper) syncLocalWithStorage(ctx context.Context) error { + a.downloadedPeriodsMtx.RLock() + defer a.downloadedPeriodsMtx.RUnlock() + + for period := range a.downloadedPeriods { + if err := a.downloadFilesForPeriod(ctx, period, a.downloadedPeriods[period]); err != nil { + return err + } + } + + return nil +} + +// deleteFileFromCache removes a file from cache. +// It takes care of locking the filesCollection, closing the boltdb file and removing the file from cache +func (a *Shipper) deleteFileFromCache(period, uploader string, fc *filesCollection) error { + fc.Lock() + defer fc.Unlock() + + if err := fc.files[uploader].boltdb.Close(); err != nil { + return err + } + + delete(fc.files, uploader) + + return os.Remove(path.Join(a.cfg.CacheLocation, period, uploader)) +} + +func (a *Shipper) getFilesCollection(period string, createIfNotExists bool) *filesCollection { + a.downloadedPeriodsMtx.RLock() + fc, ok := a.downloadedPeriods[period] + a.downloadedPeriodsMtx.RUnlock() + + if !ok && createIfNotExists { + a.downloadedPeriodsMtx.Lock() + defer a.downloadedPeriodsMtx.Unlock() + + fc, ok = a.downloadedPeriods[period] + if ok { + return fc + } + + fc = &filesCollection{files: map[string]shippedFile{}} + a.downloadedPeriods[period] = fc + + } + + return fc +} + +func (a *Shipper) forEach(ctx context.Context, period string, callback func(db *bbolt.DB) error) error { + fc := a.getFilesCollection(period, false) + + if fc == nil { + fc = a.getFilesCollection(period, true) + + if err := a.downloadFilesForPeriod(ctx, period, fc); err != nil { + return err + } + + a.downloadedPeriodsMtx.RLock() + fc = a.downloadedPeriods[period] + a.downloadedPeriodsMtx.RUnlock() + } + + fc.RLock() + defer fc.RUnlock() + + fc.lastUsedAt = time.Now() + + for uploader := range fc.files { + if err := callback(fc.files[uploader].boltdb); err != nil { + return err + } + } + + return nil +} diff --git a/pkg/storage/stores/local/uploads.go b/pkg/storage/stores/local/uploads.go new file mode 100644 index 000000000000..f9273f449eb1 --- /dev/null +++ b/pkg/storage/stores/local/uploads.go @@ -0,0 +1,101 @@ +package local + +import ( + "context" + "fmt" + "io/ioutil" + "os" + "path" + "time" + + "github.com/cortexproject/cortex/pkg/chunk/local" + chunk_util "github.com/cortexproject/cortex/pkg/chunk/util" + "github.com/cortexproject/cortex/pkg/util" + "github.com/go-kit/kit/log/level" + "go.etcd.io/bbolt" +) + +// uploadFiles uploads all new and updated files to storage. +// It uploads the files from configured boltdb dir where ingester writes the index. +func (a *Shipper) uploadFiles(ctx context.Context) error { + if a.cfg.Mode == ShipperModeReadOnly { + return nil + } + + filesInfo, err := ioutil.ReadDir(a.cfg.BoltdbDirectory) + if err != nil { + return err + } + + for _, fileInfo := range filesInfo { + if fileInfo.IsDir() { + continue + } + + a.uploadedFilesMtimeMtx.RLock() + // Checking whether file is updated after last push, if not skipping it + uploadedFileMtime, ok := a.uploadedFilesMtime[fileInfo.Name()] + a.uploadedFilesMtimeMtx.RUnlock() + + if ok && uploadedFileMtime.Equal(fileInfo.ModTime()) { + continue + } + + err := a.uploadFile(ctx, fileInfo.Name()) + if err != nil { + return err + } + + a.uploadedFilesMtimeMtx.Lock() + a.uploadedFilesMtime[fileInfo.Name()] = fileInfo.ModTime() + a.uploadedFilesMtimeMtx.Unlock() + } + + return nil +} + +// uploadFile uploads one of the files locally written by ingesters to storage. +func (a *Shipper) uploadFile(ctx context.Context, period string) error { + if a.cfg.Mode == ShipperModeReadWrite { + return nil + } + + snapshotPath := path.Join(a.cfg.CacheLocation, period) + err := chunk_util.EnsureDirectory(snapshotPath) + if err != nil { + return err + } + + filePath := path.Join(snapshotPath, fmt.Sprintf("%s.%d", a.uploader, time.Now().Unix())) + f, err := os.Open(filePath) + if err != nil { + return err + } + + db, err := a.localBoltdbGetter(period, local.DBOperationRead) + if err != nil { + return err + } + + err = db.View(func(tx *bbolt.Tx) error { + _, err := tx.WriteTo(f) + return err + }) + if err != nil { + return err + } + + defer func() { + if err := f.Close(); err != nil { + level.Error(util.Logger) + } + + if err := os.Remove(filePath); err != nil { + level.Error(util.Logger) + } + }() + + // Files are stored with / + objectKey := fmt.Sprintf("%s/%s", period, a.uploader) + return a.storageClient.PutObject(ctx, objectKey, f) +} From aa6742125b73241daec9bcbbdda89f6f413f0aa8 Mon Sep 17 00:00:00 2001 From: Sandeep Sukhani Date: Fri, 27 Mar 2020 17:22:40 +0530 Subject: [PATCH 02/13] changes suggested from PR review --- pkg/ingester/ingester.go | 26 +++++++++++++------ pkg/ingester/instance.go | 4 +-- pkg/loki/modules.go | 2 +- pkg/storage/store.go | 20 +++++++++----- .../stores/local/boltdb_index_client.go | 6 ++--- pkg/storage/stores/local/downloads.go | 4 +++ pkg/storage/stores/local/shipper.go | 20 ++++++++------ pkg/storage/stores/local/uploads.go | 26 +++++++++++++------ 8 files changed, 70 insertions(+), 38 deletions(-) diff --git a/pkg/ingester/ingester.go b/pkg/ingester/ingester.go index cf3a6b9f80b4..5a2536882de7 100644 --- a/pkg/ingester/ingester.go +++ b/pkg/ingester/ingester.go @@ -255,13 +255,11 @@ func (i *Ingester) Query(req *logproto.QueryRequest, queryServer logproto.Querie } instance := i.getOrCreateInstance(instanceID) - itr, err := instance.Query(ctx, req) + itrs, err := instance.Query(ctx, req) if err != nil { return err } - defer helpers.LogError("closing iterator", itr.Close) - if i.cfg.QueryStore { start := req.Start end := req.End @@ -273,19 +271,22 @@ func (i *Ingester) Query(req *logproto.QueryRequest, queryServer logproto.Querie } if start.Before(end) { - req.Start = start - req.End = end + storeRequest := recreateRequestWithTime(req, start, end) - storeItr, err := i.store.LazyQuery(ctx, logql.SelectParams{QueryRequest: req}) + storeItr, err := i.store.LazyQuery(ctx, logql.SelectParams{QueryRequest: storeRequest}) if err != nil { return err } - itr.Push(storeItr) + itrs = append(itrs, storeItr) } } - return sendBatches(queryServer.Context(), itr, queryServer, req.Limit) + heapItr := iter.NewHeapIterator(ctx, itrs, req.Direction) + + defer helpers.LogError("closing iterator", heapItr.Close) + + return sendBatches(queryServer.Context(), heapItr, queryServer, req.Limit) } // Label returns the set of labels for the stream this ingester knows about. @@ -394,3 +395,12 @@ func (i *Ingester) TailersCount(ctx context.Context, in *logproto.TailersCountRe return &resp, nil } + +// creates a new QueryRequest with a query range +func recreateRequestWithTime(req *logproto.QueryRequest, start, end time.Time) *logproto.QueryRequest { + newRequest := *req + newRequest.Start = start + newRequest.End = end + + return &newRequest +} diff --git a/pkg/ingester/instance.go b/pkg/ingester/instance.go index d10957e87fa3..7d05eb97608c 100644 --- a/pkg/ingester/instance.go +++ b/pkg/ingester/instance.go @@ -186,7 +186,7 @@ func (i *instance) getLabelsFromFingerprint(fp model.Fingerprint) labels.Labels return s.labels } -func (i *instance) Query(ctx context.Context, req *logproto.QueryRequest) (iter.HeapIterator, error) { +func (i *instance) Query(ctx context.Context, req *logproto.QueryRequest) ([]iter.EntryIterator, error) { expr, err := (logql.SelectParams{QueryRequest: req}).LogSelector() if err != nil { return nil, err @@ -214,7 +214,7 @@ func (i *instance) Query(ctx context.Context, req *logproto.QueryRequest) (iter. return nil, err } - return iter.NewHeapIterator(ctx, iters, req.Direction), nil + return iters, nil } func (i *instance) Label(_ context.Context, req *logproto.LabelRequest) (*logproto.LabelResponse, error) { diff --git a/pkg/loki/modules.go b/pkg/loki/modules.go index 8858d517ea79..256850b2998a 100644 --- a/pkg/loki/modules.go +++ b/pkg/loki/modules.go @@ -256,7 +256,7 @@ func (t *Loki) initTableManager() error { os.Exit(1) } - tableClient, err := storage.NewTableClient(lastConfig.IndexType, t.cfg.StorageConfig.Config) + tableClient, err := loki_storage.NewTableClient(lastConfig.IndexType, t.cfg.StorageConfig) if err != nil { return err } diff --git a/pkg/storage/store.go b/pkg/storage/store.go index 57454a47c57b..9c9aec0e61f2 100644 --- a/pkg/storage/store.go +++ b/pkg/storage/store.go @@ -32,6 +32,7 @@ type Config struct { // RegisterFlags adds the flags required to configure this flag set. func (cfg *Config) RegisterFlags(f *flag.FlagSet) { cfg.Config.RegisterFlags(f) + cfg.BoltDBShipperConfig.RegisterFlags(f) f.IntVar(&cfg.MaxChunkBatchSize, "max-chunk-batch-size", 50, "The maximum number of chunks to fetch per batch.") } @@ -49,10 +50,8 @@ type store struct { // NewStore creates a new Loki Store using configuration supplied. func NewStore(cfg Config, storeCfg chunk.StoreConfig, schemaCfg chunk.SchemaConfig, limits storage.StoreLimits) (Store, error) { - err := registerCustomIndexClients(cfg) - if err != nil { - return nil, err - } + registerCustomIndexClients(cfg) + s, err := storage.NewStore(cfg.Config, storeCfg, schemaCfg, limits) if err != nil { return nil, err @@ -63,6 +62,15 @@ func NewStore(cfg Config, storeCfg chunk.StoreConfig, schemaCfg chunk.SchemaConf }, nil } +// NewTableClient creates a new TableClient using configuration supplied. +func NewTableClient(name string, cfg Config) (chunk.TableClient, error) { + if name == local.BoltDBShipperType { + name = "boltdb" + cfg.FSConfig = cortex_local.FSConfig{Directory: cfg.BoltDBShipperConfig.ActiveIndexDirectory} + } + return storage.NewTableClient(name, cfg.Config) +} + // decodeReq sanitizes an incoming request, rounds bounds, and appends the __name__ matcher func decodeReq(req logql.SelectParams) ([]*labels.Matcher, logql.LineFilter, model.Time, model.Time, error) { expr, err := req.LogSelector() @@ -209,7 +217,7 @@ func filterChunksByTime(from, through model.Time, chunks []chunk.Chunk) []chunk. return filtered } -func registerCustomIndexClients(cfg Config) error { +func registerCustomIndexClients(cfg Config) { storage.RegisterIndexClient(local.BoltDBShipperType, func() (chunk.IndexClient, error) { objectClient, err := stores.NewObjectClient(cfg.BoltDBShipperConfig.StoreConfig) if err != nil { @@ -218,6 +226,4 @@ func registerCustomIndexClients(cfg Config) error { return local.NewBoltDBIndexClient(cortex_local.BoltDBConfig{Directory: cfg.BoltDBShipperConfig.ActiveIndexDirectory}, objectClient, cfg.BoltDBShipperConfig) }) - - return nil } diff --git a/pkg/storage/stores/local/boltdb_index_client.go b/pkg/storage/stores/local/boltdb_index_client.go index 8e43b9962344..43e6afd7f6ee 100644 --- a/pkg/storage/stores/local/boltdb_index_client.go +++ b/pkg/storage/stores/local/boltdb_index_client.go @@ -26,16 +26,14 @@ func NewBoltDBIndexClient(cfg local.BoltDBConfig, archiveStoreClient chunk.Objec return nil, err } - archiver, err := NewShipper(archiverCfg, archiveStoreClient, func(name string, operation int) (db *bbolt.DB, e error) { - return boltDBIndexClient.GetDB(name, operation) - }) + shipper, err := NewShipper(archiverCfg, archiveStoreClient, boltDBIndexClient) if err != nil { return nil, err } indexClient := BoltdbIndexClientWithArchiver{ BoltIndexClient: boltDBIndexClient, - shipper: archiver, + shipper: shipper, done: make(chan struct{}), } diff --git a/pkg/storage/stores/local/downloads.go b/pkg/storage/stores/local/downloads.go index e50fb69e5319..3a2c86b0417d 100644 --- a/pkg/storage/stores/local/downloads.go +++ b/pkg/storage/stores/local/downloads.go @@ -60,6 +60,10 @@ func (a *Shipper) checkStorageForUpdates(ctx context.Context, period string, fc for _, object := range objects { uploader := strings.Split(object.Key, "/")[1] + // don't include the file which was uploaded by same ingester + if uploader == a.uploader { + continue + } listedUploaders[uploader] = struct{}{} // Checking whether file was updated in the store after we downloaded it, if not, no need to include it in updates diff --git a/pkg/storage/stores/local/shipper.go b/pkg/storage/stores/local/shipper.go index ee5c084e1462..8f8ab4e70066 100644 --- a/pkg/storage/stores/local/shipper.go +++ b/pkg/storage/stores/local/shipper.go @@ -29,7 +29,7 @@ const ( ShipperModeWriteOnly // ShipperFileUploadInterval defines interval for uploading active boltdb files from local which are being written to by ingesters. - ShipperFileUploadInterval = 15 * time.Minute + ShipperFileUploadInterval = 15 * time.Second cacheCleanupInterval = 24 * time.Hour @@ -37,6 +37,10 @@ const ( BoltDBShipperType = "boltdb-shipper" ) +type BoltDBGetter interface { + GetDB(name string, operation int) (*bbolt.DB, error) +} + type StoreConfig struct { Store string `yaml:"store"` AWSStorageConfig aws.StorageConfig `yaml:"aws"` @@ -53,7 +57,6 @@ type ShipperConfig struct { ResyncInterval time.Duration `yaml:"resync_interval"` IngesterName string `yaml:"-"` Mode int `yaml:"-"` - BoltdbDirectory string `yaml:"-"` } // RegisterFlags registers flags. @@ -63,7 +66,7 @@ func (cfg *ShipperConfig) RegisterFlags(f *flag.FlagSet) { cfg.StoreConfig.GCSConfig.RegisterFlagsWithPrefix(storeFlagsPrefix, f) cfg.StoreConfig.FSConfig.RegisterFlagsWithPrefix(storeFlagsPrefix, f) - f.StringVar(&cfg.ActiveIndexDirectory, "boltdb.active-index-directory", "filesystem", "Directory where ingesters would write boltdb files which would then be uploaded by shipper to configured storage") + f.StringVar(&cfg.ActiveIndexDirectory, "boltdb.active-index-directory", "", "Directory where ingesters would write boltdb files which would then be uploaded by shipper to configured storage") f.StringVar(&cfg.StoreConfig.Store, "boltdb.shipper.store", "filesystem", "Store for keeping boltdb files") f.StringVar(&cfg.CacheLocation, "boltdb.shipper.cache-location", "", "Cache location for restoring boltDB files for queries") f.DurationVar(&cfg.CacheTTL, "boltdb.shipper.cache-ttl", 24*time.Hour, "TTL for boltDB files restored in cache for queries") @@ -91,8 +94,8 @@ type filesCollection struct { } type Shipper struct { - cfg ShipperConfig - localBoltdbGetter func(name string, operation int) (*bbolt.DB, error) + cfg ShipperConfig + boltDBGetter BoltDBGetter // downloadedPeriods holds mapping for period -> filesCollection. // Here period is name of the file created by ingesters for a specific period. @@ -107,20 +110,21 @@ type Shipper struct { } // NewShipper creates a shipper for syncing local objects with a store -func NewShipper(cfg ShipperConfig, storageClient chunk.ObjectClient, localBoltdbGetter func(name string, operation int) (*bbolt.DB, error)) (*Shipper, error) { +func NewShipper(cfg ShipperConfig, storageClient chunk.ObjectClient, boltDBGetter BoltDBGetter) (*Shipper, error) { if err := chunk_util.EnsureDirectory(cfg.CacheLocation); err != nil { return nil, err } shipper := Shipper{ cfg: cfg, - localBoltdbGetter: localBoltdbGetter, + boltDBGetter: boltDBGetter, downloadedPeriods: map[string]*filesCollection{}, storageClient: storageClient, done: make(chan struct{}), // We would use ingester name and startup timestamp for naming files while uploading so that // ingester does not override old files when using same id - uploader: fmt.Sprintf("%s-%d", cfg.IngesterName, time.Now().Unix()), + uploader: fmt.Sprintf("%s-%d", cfg.IngesterName, time.Now().Unix()), + uploadedFilesMtime: map[string]time.Time{}, } go shipper.loop() diff --git a/pkg/storage/stores/local/uploads.go b/pkg/storage/stores/local/uploads.go index f9273f449eb1..18a52ec4f82c 100644 --- a/pkg/storage/stores/local/uploads.go +++ b/pkg/storage/stores/local/uploads.go @@ -22,7 +22,7 @@ func (a *Shipper) uploadFiles(ctx context.Context) error { return nil } - filesInfo, err := ioutil.ReadDir(a.cfg.BoltdbDirectory) + filesInfo, err := ioutil.ReadDir(a.cfg.ActiveIndexDirectory) if err != nil { return err } @@ -56,7 +56,7 @@ func (a *Shipper) uploadFiles(ctx context.Context) error { // uploadFile uploads one of the files locally written by ingesters to storage. func (a *Shipper) uploadFile(ctx context.Context, period string) error { - if a.cfg.Mode == ShipperModeReadWrite { + if a.cfg.Mode == ShipperModeReadOnly { return nil } @@ -67,12 +67,18 @@ func (a *Shipper) uploadFile(ctx context.Context, period string) error { } filePath := path.Join(snapshotPath, fmt.Sprintf("%s.%d", a.uploader, time.Now().Unix())) - f, err := os.Open(filePath) + f, err := os.Create(filePath) if err != nil { return err } - db, err := a.localBoltdbGetter(period, local.DBOperationRead) + defer func() { + if err := os.Remove(filePath); err != nil { + level.Error(util.Logger) + } + }() + + db, err := a.boltDBGetter.GetDB(period, local.DBOperationRead) if err != nil { return err } @@ -85,14 +91,18 @@ func (a *Shipper) uploadFile(ctx context.Context, period string) error { return err } + if err := f.Sync(); err != nil { + return err + } + + if _, err := f.Seek(0, 0); err != nil { + return err + } + defer func() { if err := f.Close(); err != nil { level.Error(util.Logger) } - - if err := os.Remove(filePath); err != nil { - level.Error(util.Logger) - } }() // Files are stored with / From 83a28a0848e386a873235e8cc7a0f5819372b824 Mon Sep 17 00:00:00 2001 From: Sandeep Sukhani Date: Mon, 30 Mar 2020 19:08:29 +0530 Subject: [PATCH 03/13] refactored some code to fix minor issues --- cmd/loki/loki-local-config.yaml | 10 +- pkg/storage/stores/local/downloads.go | 173 +++++++++++++++++--------- pkg/storage/stores/local/shipper.go | 52 ++++---- 3 files changed, 149 insertions(+), 86 deletions(-) diff --git a/cmd/loki/loki-local-config.yaml b/cmd/loki/loki-local-config.yaml index 390f542a8c16..12c7bdd88fdf 100644 --- a/cmd/loki/loki-local-config.yaml +++ b/cmd/loki/loki-local-config.yaml @@ -18,7 +18,7 @@ ingester: schema_config: configs: - from: 2018-04-15 - store: boltdb + store: boltdb-shipper object_store: filesystem schema: v11 index: @@ -32,6 +32,14 @@ storage_config: filesystem: directory: /loki/chunks + boltdb_shipper_config: + active_index_directory: /tmp/loki/index + cache_location: /tmp/loki/boltdb-cache + store_config: + store: filesystem + filesystem: + directory: /tmp/loki/boltdb-store + limits_config: enforce_metric_name: false reject_old_samples: true diff --git a/pkg/storage/stores/local/downloads.go b/pkg/storage/stores/local/downloads.go index 3a2c86b0417d..6f4a961e0b0e 100644 --- a/pkg/storage/stores/local/downloads.go +++ b/pkg/storage/stores/local/downloads.go @@ -16,39 +16,12 @@ import ( "github.com/go-kit/kit/log/level" ) -// downloadFilesForPeriod downloads all the files from for given period from all the uploaders -func (a *Shipper) downloadFilesForPeriod(ctx context.Context, period string, fc *filesCollection) error { - toDownload, toDelete, err := a.checkStorageForUpdates(ctx, period, fc) - if err != nil { - return err - } - - for _, afd := range toDownload { - err := a.downloadFile(ctx, period, afd, fc) - if err != nil { - return err - } - } - - for _, afd := range toDelete { - err := a.deleteFileFromCache(period, afd.uploader, fc) - 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 (a *Shipper) checkStorageForUpdates(ctx context.Context, period string, fc *filesCollection) (toDownload []shippedFileDetails, toDelete []shippedFileDetails, err error) { +func (a *Shipper) checkStorageForUpdates(ctx context.Context, period string, fc *filesCollection) (toDownload []chunk.StorageObject, toDelete []string, err error) { if a.cfg.Mode == ShipperModeWriteOnly { return } - fc.RLock() - defer fc.RUnlock() - // listing tables from store var objects []chunk.StorageObject objects, err = a.storageClient.List(ctx, period) @@ -69,49 +42,56 @@ func (a *Shipper) checkStorageForUpdates(ctx context.Context, period string, fc // Checking whether file was updated in the store after we downloaded it, if not, no need to include it in updates downloadedFileDetails, ok := fc.files[uploader] if !ok || downloadedFileDetails.mtime != object.ModifiedAt { - toDownload = append(toDownload, shippedFileDetails{uploader: uploader, mtime: object.ModifiedAt}) + toDownload = append(toDownload, object) } } - for uploader, fileDetails := range fc.files { + for uploader := range fc.files { if _, isOK := listedUploaders[uploader]; !isOK { - toDelete = append(toDelete, shippedFileDetails{uploader: uploader, mtime: fileDetails.mtime}) + toDelete = append(toDelete, uploader) } } return } -// downloadFile downloads a file from storage to cache. -// It first downloads it to a temp file so that we close the existing file(if already exists), replace it with new one and then reopen it. -func (a *Shipper) downloadFile(ctx context.Context, period string, afd shippedFileDetails, fc *filesCollection) error { - objectKey := fmt.Sprintf("%s/%s", period, afd.uploader) - readCloser, err := a.storageClient.GetObject(ctx, objectKey) +// syncFilesForPeriod downloads updated and new files from for given period from all the uploaders and removes deleted ones +func (a *Shipper) syncFilesForPeriod(ctx context.Context, period string, fc *filesCollection) error { + fc.RLock() + toDownload, toDelete, err := a.checkStorageForUpdates(ctx, period, fc) + fc.RUnlock() + if err != nil { return err } - defer func() { - if err := readCloser.Close(); err != nil { - level.Error(util.Logger) + for _, storageObject := range toDownload { + err = a.downloadFile(ctx, period, storageObject, fc) + if err != nil { + return err } - }() + } - downloadPath := path.Join(a.cfg.CacheLocation, period) - err = chunk_util.EnsureDirectory(downloadPath) - if err != nil { - return err + for _, uploader := range toDelete { + err := a.deleteFileFromCache(period, uploader, fc) + if err != nil { + return err + } } - // download the file temporarily with some other name to allow boltdb client to close the existing file first if it exists - tempFilePath := path.Join(downloadPath, fmt.Sprintf("%s.%d", afd.uploader, time.Now().Unix())) + return nil +} - f, err := os.Create(tempFilePath) - if err != nil { - return err - } +// It first downloads file to a temp location so that we close the existing file(if already exists), replace it with new one and then reopen it. +func (a *Shipper) downloadFile(ctx context.Context, period string, storageObject chunk.StorageObject, fc *filesCollection) error { + uploader := strings.Split(storageObject.Key, "/")[1] + folderPath, _ := a.getFolderPathForPeriod(period, false) + filePath := path.Join(folderPath, uploader) - _, err = io.Copy(f, readCloser) + // download the file temporarily with some other name to allow boltdb client to close the existing file first if it exists + tempFilePath := path.Join(folderPath, fmt.Sprintf("%s.%d", uploader, time.Now().Unix())) + + err := a.getFileFromStorage(ctx, storageObject.Key, tempFilePath) if err != nil { return err } @@ -119,28 +99,105 @@ func (a *Shipper) downloadFile(ctx context.Context, period string, afd shippedFi fc.Lock() defer fc.Unlock() - af, ok := fc.files[afd.uploader] + df, ok := fc.files[uploader] if ok { - if err := af.boltdb.Close(); err != nil { + if err := df.boltdb.Close(); err != nil { return err } } else { - af = shippedFile{downloadLocation: path.Join(downloadPath, afd.uploader)} + df = downloadedFiles{} } // move the file from temp location to actual location - err = os.Rename(tempFilePath, af.downloadLocation) + err = os.Rename(tempFilePath, filePath) + if err != nil { + return err + } + + df.mtime = storageObject.ModifiedAt + df.boltdb, err = local.OpenBoltdbFile(filePath) if err != nil { return err } - af.mtime = afd.mtime - af.boltdb, err = local.OpenBoltdbFile(af.downloadLocation) + fc.files[uploader] = df + + return nil +} + +// getFileFromStorage downloads a file from storage to given location. +func (a *Shipper) getFileFromStorage(ctx context.Context, objectKey, destination string) error { + readCloser, err := a.storageClient.GetObject(ctx, objectKey) + if err != nil { + return err + } + + defer func() { + if err := readCloser.Close(); err != nil { + level.Error(util.Logger) + } + }() + + f, err := os.Create(destination) + if err != nil { + return err + } + + _, err = io.Copy(f, readCloser) + return err +} + +// downloadFilesForPeriod should be called when files for a period does not exist i.e they were never downloaded or got cleaned up later on by TTL +// While files are being downloaded it will block all reads/writes on filesCollection by taking an exclusive lock +func (a *Shipper) downloadFilesForPeriod(ctx context.Context, period string, fc *filesCollection) error { + fc.Lock() + defer fc.Unlock() + + objects, err := a.storageClient.List(ctx, period) if err != nil { return err } - fc.files[afd.uploader] = af + folderPath, err := a.getFolderPathForPeriod(period, true) + if err != nil { + return err + } + + for _, object := range objects { + uploader := getUploaderFromObjectKey(object.Key) + filePath := path.Join(folderPath, uploader) + df := downloadedFiles{} + + err := a.getFileFromStorage(ctx, object.Key, filePath) + if err != nil { + return err + } + + df.mtime = object.ModifiedAt + df.boltdb, err = local.OpenBoltdbFile(filePath) + if err != nil { + return err + } + + fc.files[uploader] = df + } return nil } + +func (a *Shipper) getFolderPathForPeriod(period string, ensureExists bool) (string, error) { + folderPath := path.Join(a.cfg.CacheLocation, period) + + if ensureExists { + err := chunk_util.EnsureDirectory(folderPath) + if err != nil { + return "", err + } + } + + return folderPath, nil +} + +func getUploaderFromObjectKey(objectKey string) string { + return strings.Split(objectKey, "/")[1] +} diff --git a/pkg/storage/stores/local/shipper.go b/pkg/storage/stores/local/shipper.go index 8f8ab4e70066..3da31fb9f408 100644 --- a/pkg/storage/stores/local/shipper.go +++ b/pkg/storage/stores/local/shipper.go @@ -73,15 +73,9 @@ func (cfg *ShipperConfig) RegisterFlags(f *flag.FlagSet) { f.DurationVar(&cfg.ResyncInterval, "boltdb.shipper.resync-interval", 5*time.Minute, "Resync downloaded files with the storage") } -type shippedFileDetails struct { - uploader string - mtime time.Time -} - -type shippedFile struct { - downloadLocation string - mtime time.Time - boltdb *bbolt.DB +type downloadedFiles struct { + mtime time.Time + boltdb *bbolt.DB } // filesCollection holds info about shipped boltdb index files by other uploaders(ingesters). @@ -90,7 +84,7 @@ type shippedFile struct { type filesCollection struct { sync.RWMutex lastUsedAt time.Time - files map[string]shippedFile + files map[string]downloadedFiles } type Shipper struct { @@ -214,7 +208,7 @@ func (a *Shipper) syncLocalWithStorage(ctx context.Context) error { defer a.downloadedPeriodsMtx.RUnlock() for period := range a.downloadedPeriods { - if err := a.downloadFilesForPeriod(ctx, period, a.downloadedPeriods[period]); err != nil { + if err := a.syncFilesForPeriod(ctx, period, a.downloadedPeriods[period]); err != nil { return err } } @@ -237,41 +231,45 @@ func (a *Shipper) deleteFileFromCache(period, uploader string, fc *filesCollecti return os.Remove(path.Join(a.cfg.CacheLocation, period, uploader)) } -func (a *Shipper) getFilesCollection(period string, createIfNotExists bool) *filesCollection { +func (a *Shipper) getFilesCollection(ctx context.Context, period string, createIfNotExists bool) (*filesCollection, error) { a.downloadedPeriodsMtx.RLock() fc, ok := a.downloadedPeriods[period] a.downloadedPeriodsMtx.RUnlock() if !ok && createIfNotExists { a.downloadedPeriodsMtx.Lock() - defer a.downloadedPeriodsMtx.Unlock() - fc, ok = a.downloadedPeriods[period] if ok { - return fc + a.downloadedPeriodsMtx.Unlock() } - fc = &filesCollection{files: map[string]shippedFile{}} + fc = &filesCollection{files: map[string]downloadedFiles{}} a.downloadedPeriods[period] = fc - } - return fc + return fc, nil } func (a *Shipper) forEach(ctx context.Context, period string, callback func(db *bbolt.DB) error) error { - fc := a.getFilesCollection(period, false) - - if fc == nil { - fc = a.getFilesCollection(period, true) + a.downloadedPeriodsMtx.RLock() + fc, ok := a.downloadedPeriods[period] + a.downloadedPeriodsMtx.RUnlock() - if err := a.downloadFilesForPeriod(ctx, period, fc); err != nil { - return err + if !ok { + a.downloadedPeriodsMtx.Lock() + fc, ok = a.downloadedPeriods[period] + if ok { + a.downloadedPeriodsMtx.Unlock() + } else { + fc = &filesCollection{files: map[string]downloadedFiles{}} + a.downloadedPeriods[period] = fc + a.downloadedPeriodsMtx.Unlock() + + if err := a.downloadFilesForPeriod(ctx, period, fc); err != nil { + return err + } } - a.downloadedPeriodsMtx.RLock() - fc = a.downloadedPeriods[period] - a.downloadedPeriodsMtx.RUnlock() } fc.RLock() From a4685972ad303135e3d8081b384a464223d76640 Mon Sep 17 00:00:00 2001 From: Sandeep Sukhani Date: Tue, 31 Mar 2020 11:28:46 +0530 Subject: [PATCH 04/13] revert accidentally pushed change --- cmd/loki/loki-local-config.yaml | 10 +--------- 1 file changed, 1 insertion(+), 9 deletions(-) diff --git a/cmd/loki/loki-local-config.yaml b/cmd/loki/loki-local-config.yaml index 12c7bdd88fdf..390f542a8c16 100644 --- a/cmd/loki/loki-local-config.yaml +++ b/cmd/loki/loki-local-config.yaml @@ -18,7 +18,7 @@ ingester: schema_config: configs: - from: 2018-04-15 - store: boltdb-shipper + store: boltdb object_store: filesystem schema: v11 index: @@ -32,14 +32,6 @@ storage_config: filesystem: directory: /loki/chunks - boltdb_shipper_config: - active_index_directory: /tmp/loki/index - cache_location: /tmp/loki/boltdb-cache - store_config: - store: filesystem - filesystem: - directory: /tmp/loki/boltdb-store - limits_config: enforce_metric_name: false reject_old_samples: true From e5f8199da1c50b6952407157145877d89bb14192 Mon Sep 17 00:00:00 2001 From: Sandeep Sukhani Date: Wed, 1 Apr 2020 19:58:33 +0530 Subject: [PATCH 05/13] persisting uploader name across restarts, detecting objectstore type from periodic config, other refactorings --- go.mod | 2 + go.sum | 2 + pkg/loki/modules.go | 6 +- pkg/storage/store.go | 31 ++- pkg/storage/stores/factory.go | 11 +- .../stores/local/boltdb_index_client.go | 20 +- pkg/storage/stores/local/downloads.go | 38 ++-- pkg/storage/stores/local/shipper.go | 188 ++++++++++-------- pkg/storage/stores/local/uploads.go | 34 ++-- pkg/storage/stores/util/object_client.go | 47 +++++ 10 files changed, 233 insertions(+), 146 deletions(-) create mode 100644 pkg/storage/stores/util/object_client.go diff --git a/go.mod b/go.mod index f03c7d95fc44..c38e13068f4b 100644 --- a/go.mod +++ b/go.mod @@ -69,3 +69,5 @@ replace k8s.io/client-go => k8s.io/client-go v0.0.0-20190620085101-78d2af792bab replace github.com/Azure/azure-sdk-for-go => github.com/Azure/azure-sdk-for-go v36.2.0+incompatible replace github.com/Azure/go-autorest => github.com/Azure/go-autorest v13.3.0+incompatible + +replace github.com/cortexproject/cortex => github.com/sandeepsukhani/cortex v0.0.0-20200401102156-8377099831f7 diff --git a/go.sum b/go.sum index 419cf1e48c19..44710cccc8ff 100644 --- a/go.sum +++ b/go.sum @@ -716,6 +716,8 @@ github.com/samuel/go-zookeeper v0.0.0-20190810000440-0ceca61e4d75 h1:cA+Ubq9qEVI github.com/samuel/go-zookeeper v0.0.0-20190810000440-0ceca61e4d75/go.mod h1:gi+0XIa01GRL2eRQVjQkKGqKF3SF9vZR/HnPullcV2E= github.com/samuel/go-zookeeper v0.0.0-20190923202752-2cc03de413da h1:p3Vo3i64TCLY7gIfzeQaUJ+kppEO5WQG3cL8iE8tGHU= github.com/samuel/go-zookeeper v0.0.0-20190923202752-2cc03de413da/go.mod h1:gi+0XIa01GRL2eRQVjQkKGqKF3SF9vZR/HnPullcV2E= +github.com/sandeepsukhani/cortex v0.0.0-20200401102156-8377099831f7 h1:yoacVlbP2nrspG+JX4fnzPQVQJia0fYq283Kfb+59zE= +github.com/sandeepsukhani/cortex v0.0.0-20200401102156-8377099831f7/go.mod h1:dMuT8RuWexf371937IhTj7/Ha3P/+Aog3pddNtV6Jo0= github.com/santhosh-tekuri/jsonschema v1.2.4/go.mod h1:TEAUOeZSmIxTTuHatJzrvARHiuO9LYd+cIxzgEHCQI4= github.com/satori/go.uuid v0.0.0-20160603004225-b111a074d5ef/go.mod h1:dA0hQrYB0VpLJoorglMZABFdXlWrHn1NEOzdhQKdks0= github.com/satori/go.uuid v1.2.0/go.mod h1:dA0hQrYB0VpLJoorglMZABFdXlWrHn1NEOzdhQKdks0= diff --git a/pkg/loki/modules.go b/pkg/loki/modules.go index 256850b2998a..3c98f2d71133 100644 --- a/pkg/loki/modules.go +++ b/pkg/loki/modules.go @@ -9,10 +9,6 @@ import ( "strings" "time" - "github.com/prometheus/common/model" - - "github.com/grafana/loki/pkg/storage/stores/local" - "github.com/cortexproject/cortex/pkg/chunk" "github.com/cortexproject/cortex/pkg/chunk/storage" "github.com/cortexproject/cortex/pkg/querier/frontend" @@ -25,6 +21,7 @@ import ( "github.com/go-kit/kit/log/level" "github.com/prometheus/client_golang/prometheus" + "github.com/prometheus/common/model" httpgrpc_server "github.com/weaveworks/common/httpgrpc/server" "github.com/weaveworks/common/middleware" "github.com/weaveworks/common/server" @@ -36,6 +33,7 @@ import ( "github.com/grafana/loki/pkg/querier" "github.com/grafana/loki/pkg/querier/queryrange" loki_storage "github.com/grafana/loki/pkg/storage" + "github.com/grafana/loki/pkg/storage/stores/local" "github.com/grafana/loki/pkg/util/validation" ) diff --git a/pkg/storage/store.go b/pkg/storage/store.go index 9c9aec0e61f2..8a30054b2b15 100644 --- a/pkg/storage/store.go +++ b/pkg/storage/store.go @@ -50,7 +50,7 @@ type store struct { // NewStore creates a new Loki Store using configuration supplied. func NewStore(cfg Config, storeCfg chunk.StoreConfig, schemaCfg chunk.SchemaConfig, limits storage.StoreLimits) (Store, error) { - registerCustomIndexClients(cfg) + registerCustomIndexClients(cfg, schemaCfg) s, err := storage.NewStore(cfg.Config, storeCfg, schemaCfg, limits) if err != nil { @@ -62,7 +62,8 @@ func NewStore(cfg Config, storeCfg chunk.StoreConfig, schemaCfg chunk.SchemaConf }, nil } -// NewTableClient creates a new TableClient using configuration supplied. +// NewTableClient creates a TableClient for managing tables for index/chunk store. +// ToDo: Add support in Cortex for registering custom table client like index client. func NewTableClient(name string, cfg Config) (chunk.TableClient, error) { if name == local.BoltDBShipperType { name = "boltdb" @@ -217,9 +218,31 @@ func filterChunksByTime(from, through model.Time, chunks []chunk.Chunk) []chunk. return filtered } -func registerCustomIndexClients(cfg Config) { +func registerCustomIndexClients(cfg Config, schemaCfg chunk.SchemaConfig) { + boltdbShipperInstances := 0 storage.RegisterIndexClient(local.BoltDBShipperType, func() (chunk.IndexClient, error) { - objectClient, err := stores.NewObjectClient(cfg.BoltDBShipperConfig.StoreConfig) + // since we do not know which object client is being used for the period for which we are creating this index client, + // we need to iterate through all the periodic configs to find the right one. + // We maintain number of instances that we have already created in boltdbShipperInstances and then count the number of + // encounters of BoltDBShipperType until we find the right periodic config for getting the ObjectType. + // This is done assuming we are creating index client in the order of periodic configs. + // Note: We are assuming that user would never store chunks in table based store otherwise NewObjectClient would return an error. + + // ToDo: Try passing on ObjectType from Cortex to the callback for creating custom index client. + boltdbShipperEncounter := 0 + objectStoreType := "" + for _, config := range schemaCfg.Configs { + if config.IndexType == local.BoltDBShipperType { + boltdbShipperEncounter += 1 + if boltdbShipperEncounter > boltdbShipperInstances { + objectStoreType = config.ObjectType + break + } + } + } + + boltdbShipperInstances += 1 + objectClient, err := stores.NewObjectClient(objectStoreType, cfg.Config) if err != nil { return nil, err } diff --git a/pkg/storage/stores/factory.go b/pkg/storage/stores/factory.go index b877b9e27f74..59af4e6c598f 100644 --- a/pkg/storage/stores/factory.go +++ b/pkg/storage/stores/factory.go @@ -9,22 +9,21 @@ import ( "github.com/cortexproject/cortex/pkg/chunk/azure" "github.com/cortexproject/cortex/pkg/chunk/gcp" cortex_local "github.com/cortexproject/cortex/pkg/chunk/local" - - "github.com/grafana/loki/pkg/storage/stores/local" + "github.com/cortexproject/cortex/pkg/chunk/storage" ) // NewObjectClient makes a new ObjectClient of the desired type. -func NewObjectClient(cfg local.StoreConfig) (chunk.ObjectClient, error) { - switch cfg.Store { +func NewObjectClient(storeType string, cfg storage.Config) (chunk.ObjectClient, error) { + switch storeType { case "aws", "s3": return aws.NewS3ObjectClient(cfg.AWSStorageConfig.S3Config) case "gcs": return gcp.NewGCSObjectClient(context.Background(), cfg.GCSConfig) case "azure": - return azure.NewBlobStorage(&cfg.Azure) + return azure.NewBlobStorage(&cfg.AzureStorageConfig) case "filesystem": return cortex_local.NewFSObjectClient(cfg.FSConfig) default: - return nil, fmt.Errorf("unrecognized storage client %v, choose one of: aws, s3, gcp, azure, filesystem", cfg.Store) + return nil, fmt.Errorf("unrecognized storage client %v, choose one of: aws, s3, gcp, azure, filesystem", storeType) } } diff --git a/pkg/storage/stores/local/boltdb_index_client.go b/pkg/storage/stores/local/boltdb_index_client.go index 43e6afd7f6ee..a518b1594f95 100644 --- a/pkg/storage/stores/local/boltdb_index_client.go +++ b/pkg/storage/stores/local/boltdb_index_client.go @@ -2,7 +2,6 @@ package local import ( "context" - "sync" "github.com/cortexproject/cortex/pkg/chunk" "github.com/cortexproject/cortex/pkg/chunk/local" @@ -10,13 +9,9 @@ import ( "go.etcd.io/bbolt" ) -type BoltdbIndexClientWithArchiver struct { +type BoltdbIndexClientWithShipper struct { *local.BoltIndexClient - shipper *Shipper - - done chan struct{} - wait sync.WaitGroup } // NewBoltDBIndexClient creates a new IndexClient that used BoltDB. @@ -31,29 +26,24 @@ func NewBoltDBIndexClient(cfg local.BoltDBConfig, archiveStoreClient chunk.Objec return nil, err } - indexClient := BoltdbIndexClientWithArchiver{ + indexClient := BoltdbIndexClientWithShipper{ BoltIndexClient: boltDBIndexClient, shipper: shipper, - done: make(chan struct{}), } return &indexClient, nil } -func (b *BoltdbIndexClientWithArchiver) Stop() { - close(b.done) - +func (b *BoltdbIndexClientWithShipper) Stop() { b.BoltIndexClient.Stop() b.shipper.Stop() - - b.wait.Wait() } -func (b *BoltdbIndexClientWithArchiver) QueryPages(ctx context.Context, queries []chunk.IndexQuery, callback func(chunk.IndexQuery, chunk.ReadBatch) (shouldContinue bool)) error { +func (b *BoltdbIndexClientWithShipper) QueryPages(ctx context.Context, queries []chunk.IndexQuery, callback func(chunk.IndexQuery, chunk.ReadBatch) (shouldContinue bool)) error { return chunk_util.DoParallelQueries(ctx, b.query, queries, callback) } -func (b *BoltdbIndexClientWithArchiver) query(ctx context.Context, query chunk.IndexQuery, callback func(chunk.ReadBatch) (shouldContinue bool)) error { +func (b *BoltdbIndexClientWithShipper) query(ctx context.Context, query chunk.IndexQuery, callback func(chunk.ReadBatch) (shouldContinue bool)) error { db, err := b.GetDB(query.TableName, local.DBOperationRead) if err != nil && err != local.ErrUnexistentBoltDB { return err diff --git a/pkg/storage/stores/local/downloads.go b/pkg/storage/stores/local/downloads.go index 6f4a961e0b0e..29bce78e3592 100644 --- a/pkg/storage/stores/local/downloads.go +++ b/pkg/storage/stores/local/downloads.go @@ -17,14 +17,14 @@ import ( ) // 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 (a *Shipper) checkStorageForUpdates(ctx context.Context, period string, fc *filesCollection) (toDownload []chunk.StorageObject, toDelete []string, err error) { - if a.cfg.Mode == ShipperModeWriteOnly { +func (s *Shipper) checkStorageForUpdates(ctx context.Context, period string, fc *filesCollection) (toDownload []chunk.StorageObject, toDelete []string, err error) { + if s.cfg.Mode == ShipperModeWriteOnly { return } // listing tables from store var objects []chunk.StorageObject - objects, err = a.storageClient.List(ctx, period) + objects, err = s.storageClient.List(ctx, period) if err != nil { return } @@ -34,7 +34,7 @@ func (a *Shipper) checkStorageForUpdates(ctx context.Context, period string, fc for _, object := range objects { uploader := strings.Split(object.Key, "/")[1] // don't include the file which was uploaded by same ingester - if uploader == a.uploader { + if uploader == s.uploader { continue } listedUploaders[uploader] = struct{}{} @@ -56,9 +56,9 @@ func (a *Shipper) checkStorageForUpdates(ctx context.Context, period string, fc } // syncFilesForPeriod downloads updated and new files from for given period from all the uploaders and removes deleted ones -func (a *Shipper) syncFilesForPeriod(ctx context.Context, period string, fc *filesCollection) error { +func (s *Shipper) syncFilesForPeriod(ctx context.Context, period string, fc *filesCollection) error { fc.RLock() - toDownload, toDelete, err := a.checkStorageForUpdates(ctx, period, fc) + toDownload, toDelete, err := s.checkStorageForUpdates(ctx, period, fc) fc.RUnlock() if err != nil { @@ -66,14 +66,14 @@ func (a *Shipper) syncFilesForPeriod(ctx context.Context, period string, fc *fil } for _, storageObject := range toDownload { - err = a.downloadFile(ctx, period, storageObject, fc) + err = s.downloadFile(ctx, period, storageObject, fc) if err != nil { return err } } for _, uploader := range toDelete { - err := a.deleteFileFromCache(period, uploader, fc) + err := s.deleteFileFromCache(period, uploader, fc) if err != nil { return err } @@ -83,15 +83,15 @@ func (a *Shipper) syncFilesForPeriod(ctx context.Context, period string, fc *fil } // It first downloads file to a temp location so that we close the existing file(if already exists), replace it with new one and then reopen it. -func (a *Shipper) downloadFile(ctx context.Context, period string, storageObject chunk.StorageObject, fc *filesCollection) error { +func (s *Shipper) downloadFile(ctx context.Context, period string, storageObject chunk.StorageObject, fc *filesCollection) error { uploader := strings.Split(storageObject.Key, "/")[1] - folderPath, _ := a.getFolderPathForPeriod(period, false) + folderPath, _ := s.getFolderPathForPeriod(period, false) filePath := path.Join(folderPath, uploader) // download the file temporarily with some other name to allow boltdb client to close the existing file first if it exists tempFilePath := path.Join(folderPath, fmt.Sprintf("%s.%d", uploader, time.Now().Unix())) - err := a.getFileFromStorage(ctx, storageObject.Key, tempFilePath) + err := s.getFileFromStorage(ctx, storageObject.Key, tempFilePath) if err != nil { return err } @@ -126,8 +126,8 @@ func (a *Shipper) downloadFile(ctx context.Context, period string, storageObject } // getFileFromStorage downloads a file from storage to given location. -func (a *Shipper) getFileFromStorage(ctx context.Context, objectKey, destination string) error { - readCloser, err := a.storageClient.GetObject(ctx, objectKey) +func (s *Shipper) getFileFromStorage(ctx context.Context, objectKey, destination string) error { + readCloser, err := s.storageClient.GetObject(ctx, objectKey) if err != nil { return err } @@ -149,16 +149,16 @@ func (a *Shipper) getFileFromStorage(ctx context.Context, objectKey, destination // downloadFilesForPeriod should be called when files for a period does not exist i.e they were never downloaded or got cleaned up later on by TTL // While files are being downloaded it will block all reads/writes on filesCollection by taking an exclusive lock -func (a *Shipper) downloadFilesForPeriod(ctx context.Context, period string, fc *filesCollection) error { +func (s *Shipper) downloadFilesForPeriod(ctx context.Context, period string, fc *filesCollection) error { fc.Lock() defer fc.Unlock() - objects, err := a.storageClient.List(ctx, period) + objects, err := s.storageClient.List(ctx, period) if err != nil { return err } - folderPath, err := a.getFolderPathForPeriod(period, true) + folderPath, err := s.getFolderPathForPeriod(period, true) if err != nil { return err } @@ -168,7 +168,7 @@ func (a *Shipper) downloadFilesForPeriod(ctx context.Context, period string, fc filePath := path.Join(folderPath, uploader) df := downloadedFiles{} - err := a.getFileFromStorage(ctx, object.Key, filePath) + err := s.getFileFromStorage(ctx, object.Key, filePath) if err != nil { return err } @@ -185,8 +185,8 @@ func (a *Shipper) downloadFilesForPeriod(ctx context.Context, period string, fc return nil } -func (a *Shipper) getFolderPathForPeriod(period string, ensureExists bool) (string, error) { - folderPath := path.Join(a.cfg.CacheLocation, period) +func (s *Shipper) getFolderPathForPeriod(period string, ensureExists bool) (string, error) { + folderPath := path.Join(s.cfg.CacheLocation, period) if ensureExists { err := chunk_util.EnsureDirectory(folderPath) diff --git a/pkg/storage/stores/local/shipper.go b/pkg/storage/stores/local/shipper.go index 3da31fb9f408..e8da71ab4ada 100644 --- a/pkg/storage/stores/local/shipper.go +++ b/pkg/storage/stores/local/shipper.go @@ -4,20 +4,19 @@ import ( "context" "flag" "fmt" + "io/ioutil" "os" "path" "sync" "time" "github.com/cortexproject/cortex/pkg/chunk" - "github.com/cortexproject/cortex/pkg/chunk/aws" - "github.com/cortexproject/cortex/pkg/chunk/azure" - "github.com/cortexproject/cortex/pkg/chunk/gcp" - "github.com/cortexproject/cortex/pkg/chunk/local" chunk_util "github.com/cortexproject/cortex/pkg/chunk/util" - "github.com/cortexproject/cortex/pkg/util" + pkg_util "github.com/cortexproject/cortex/pkg/util" "github.com/go-kit/kit/log/level" "go.etcd.io/bbolt" + + "github.com/grafana/loki/pkg/storage/stores/util" ) const ( @@ -29,31 +28,24 @@ const ( ShipperModeWriteOnly // ShipperFileUploadInterval defines interval for uploading active boltdb files from local which are being written to by ingesters. - ShipperFileUploadInterval = 15 * time.Second - - cacheCleanupInterval = 24 * time.Hour + ShipperFileUploadInterval = 15 * time.Minute // BoltDBShipperType holds the index type for using boltdb with shipper which keeps flushing them to a shared storage BoltDBShipperType = "boltdb-shipper" + + cacheCleanupInterval = 24 * time.Hour + storageKeyPrefix = "index/" ) type BoltDBGetter interface { GetDB(name string, operation int) (*bbolt.DB, error) } -type StoreConfig struct { - Store string `yaml:"store"` - AWSStorageConfig aws.StorageConfig `yaml:"aws"` - GCSConfig gcp.GCSConfig `yaml:"gcs"` - FSConfig local.FSConfig `yaml:"filesystem"` - Azure azure.BlobStorageConfig `yaml:"azure"` -} - type ShipperConfig struct { ActiveIndexDirectory string `yaml:"active_index_directory"` + SharedStoreType string `yaml:"shared_store_type"` CacheLocation string `yaml:"cache_location"` CacheTTL time.Duration `yaml:"cache_ttl"` - StoreConfig StoreConfig `yaml:"store_config"` ResyncInterval time.Duration `yaml:"resync_interval"` IngesterName string `yaml:"-"` Mode int `yaml:"-"` @@ -61,13 +53,8 @@ type ShipperConfig struct { // RegisterFlags registers flags. func (cfg *ShipperConfig) RegisterFlags(f *flag.FlagSet) { - storeFlagsPrefix := "boltdb.shipper." - cfg.StoreConfig.AWSStorageConfig.RegisterFlagsWithPrefix(storeFlagsPrefix, f) - cfg.StoreConfig.GCSConfig.RegisterFlagsWithPrefix(storeFlagsPrefix, f) - cfg.StoreConfig.FSConfig.RegisterFlagsWithPrefix(storeFlagsPrefix, f) - f.StringVar(&cfg.ActiveIndexDirectory, "boltdb.active-index-directory", "", "Directory where ingesters would write boltdb files which would then be uploaded by shipper to configured storage") - f.StringVar(&cfg.StoreConfig.Store, "boltdb.shipper.store", "filesystem", "Store for keeping boltdb files") + f.StringVar(&cfg.SharedStoreType, "boltdb.shipper.shared-store", "", "Shared store for keeping boltdb files. Supported types: gcs, s3, azure, filesystem") f.StringVar(&cfg.CacheLocation, "boltdb.shipper.cache-location", "", "Cache location for restoring boltDB files for queries") f.DurationVar(&cfg.CacheTTL, "boltdb.shipper.cache-ttl", 24*time.Hour, "TTL for boltDB files restored in cache for queries") f.DurationVar(&cfg.ResyncInterval, "boltdb.shipper.resync-interval", 5*time.Minute, "Resync downloaded files with the storage") @@ -96,38 +83,76 @@ type Shipper struct { downloadedPeriods map[string]*filesCollection downloadedPeriodsMtx sync.RWMutex storageClient chunk.ObjectClient - done chan struct{} uploader string uploadedFilesMtime map[string]time.Time uploadedFilesMtimeMtx sync.RWMutex + + done chan struct{} + wait sync.WaitGroup } // NewShipper creates a shipper for syncing local objects with a store func NewShipper(cfg ShipperConfig, storageClient chunk.ObjectClient, boltDBGetter BoltDBGetter) (*Shipper, error) { - if err := chunk_util.EnsureDirectory(cfg.CacheLocation); err != nil { + err := chunk_util.EnsureDirectory(cfg.CacheLocation) + if err != nil { return nil, err } shipper := Shipper{ - cfg: cfg, - boltDBGetter: boltDBGetter, - downloadedPeriods: map[string]*filesCollection{}, - storageClient: storageClient, - done: make(chan struct{}), - // We would use ingester name and startup timestamp for naming files while uploading so that - // ingester does not override old files when using same id - uploader: fmt.Sprintf("%s-%d", cfg.IngesterName, time.Now().Unix()), + cfg: cfg, + boltDBGetter: boltDBGetter, + downloadedPeriods: map[string]*filesCollection{}, + storageClient: util.NewPrefixedObjectClient(storageClient, storageKeyPrefix), + done: make(chan struct{}), uploadedFilesMtime: map[string]time.Time{}, } + shipper.uploader, err = shipper.getUploaderName() + if err != nil { + return nil, err + } + + shipper.wait.Add(1) go shipper.loop() return &shipper, nil } -func (a *Shipper) loop() { - resyncTicker := time.NewTicker(a.cfg.ResyncInterval) +// we would persist uploader name in /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().Unix()) + + uploaderFilePath := path.Join(s.cfg.ActiveIndexDirectory, "uploader", "name") + if err := chunk_util.EnsureDirectory(path.Dir(uploaderFilePath)); err != nil { + return "", err + } + + _, err := os.Stat(uploaderFilePath) + if err != nil { + if !os.IsNotExist(err) { + return "", err + } + if err := ioutil.WriteFile(uploaderFilePath, []byte(uploader), 0666); err != nil { + return "", err + } + } else { + ub, err := ioutil.ReadFile(uploaderFilePath) + if err != nil { + return "", err + } + uploader = string(ub) + } + + return uploader, nil +} + +func (s *Shipper) loop() { + defer s.wait.Done() + + resyncTicker := time.NewTicker(s.cfg.ResyncInterval) defer resyncTicker.Stop() uploadFilesTicker := time.NewTicker(ShipperFileUploadInterval) @@ -139,40 +164,41 @@ func (a *Shipper) loop() { for { select { case <-resyncTicker.C: - err := a.syncLocalWithStorage(context.Background()) + err := s.syncLocalWithStorage(context.Background()) if err != nil { - level.Error(util.Logger).Log("msg", "error syncing local boltdb files with storage", "err", err) + level.Error(pkg_util.Logger).Log("msg", "error syncing local boltdb files with storage", "err", err) } case <-uploadFilesTicker.C: - err := a.uploadFiles(context.Background()) + err := s.uploadFiles(context.Background()) if err != nil { - level.Error(util.Logger).Log("msg", "error pushing archivable files to store", "err", err) + level.Error(pkg_util.Logger).Log("msg", "error pushing archivable files to store", "err", err) } case <-cacheCleanupTicker.C: - err := a.cleanupCache() + err := s.cleanupCache() if err != nil { - level.Error(util.Logger).Log("msg", "error cleaning up expired tables", "err", err) + level.Error(pkg_util.Logger).Log("msg", "error cleaning up expired tables", "err", err) } - case <-a.done: + case <-s.done: return } } } // Stop the shipper and push all the local files to the store -func (a *Shipper) Stop() { - close(a.done) +func (s *Shipper) Stop() { + close(s.done) + s.wait.Wait() // Push all boltdb files to storage before returning - err := a.syncLocalWithStorage(context.Background()) + err := s.uploadFiles(context.Background()) if err != nil { - level.Error(util.Logger).Log("msg", "error pushing archivable files to store", "err", err) + level.Error(pkg_util.Logger).Log("msg", "error pushing archivable files to store", "err", err) } - a.downloadedPeriodsMtx.Lock() - defer a.downloadedPeriodsMtx.Unlock() + s.downloadedPeriodsMtx.Lock() + defer s.downloadedPeriodsMtx.Unlock() - for _, fc := range a.downloadedPeriods { + for _, fc := range s.downloadedPeriods { fc.Lock() for _, fl := range fc.files { _ = fl.boltdb.Close() @@ -182,19 +208,19 @@ func (a *Shipper) Stop() { } // cleanupCache removes all the files for a period which has not be queried for using the configured TTL -func (a *Shipper) cleanupCache() error { - a.downloadedPeriodsMtx.Lock() - defer a.downloadedPeriodsMtx.Unlock() +func (s *Shipper) cleanupCache() error { + s.downloadedPeriodsMtx.Lock() + defer s.downloadedPeriodsMtx.Unlock() - for period, fc := range a.downloadedPeriods { - if fc.lastUsedAt.Add(a.cfg.CacheTTL).Before(time.Now()) { + for period, fc := range s.downloadedPeriods { + if fc.lastUsedAt.Add(s.cfg.CacheTTL).Before(time.Now()) { for uploader := range fc.files { - if err := a.deleteFileFromCache(period, uploader, fc); err != nil { + if err := s.deleteFileFromCache(period, uploader, fc); err != nil { return err } } - delete(a.downloadedPeriods, period) + delete(s.downloadedPeriods, period) } } @@ -203,12 +229,12 @@ func (a *Shipper) cleanupCache() error { // syncLocalWithStorage syncs all the periods that we have in the cache with the storage // i.e download new and updated files and remove files which were delete from the storage. -func (a *Shipper) syncLocalWithStorage(ctx context.Context) error { - a.downloadedPeriodsMtx.RLock() - defer a.downloadedPeriodsMtx.RUnlock() +func (s *Shipper) syncLocalWithStorage(ctx context.Context) error { + s.downloadedPeriodsMtx.RLock() + defer s.downloadedPeriodsMtx.RUnlock() - for period := range a.downloadedPeriods { - if err := a.syncFilesForPeriod(ctx, period, a.downloadedPeriods[period]); err != nil { + for period := range s.downloadedPeriods { + if err := s.syncFilesForPeriod(ctx, period, s.downloadedPeriods[period]); err != nil { return err } } @@ -218,7 +244,7 @@ func (a *Shipper) syncLocalWithStorage(ctx context.Context) error { // deleteFileFromCache removes a file from cache. // It takes care of locking the filesCollection, closing the boltdb file and removing the file from cache -func (a *Shipper) deleteFileFromCache(period, uploader string, fc *filesCollection) error { +func (s *Shipper) deleteFileFromCache(period, uploader string, fc *filesCollection) error { fc.Lock() defer fc.Unlock() @@ -228,44 +254,44 @@ func (a *Shipper) deleteFileFromCache(period, uploader string, fc *filesCollecti delete(fc.files, uploader) - return os.Remove(path.Join(a.cfg.CacheLocation, period, uploader)) + return os.Remove(path.Join(s.cfg.CacheLocation, period, uploader)) } -func (a *Shipper) getFilesCollection(ctx context.Context, period string, createIfNotExists bool) (*filesCollection, error) { - a.downloadedPeriodsMtx.RLock() - fc, ok := a.downloadedPeriods[period] - a.downloadedPeriodsMtx.RUnlock() +func (s *Shipper) getFilesCollection(ctx context.Context, period string, createIfNotExists bool) (*filesCollection, error) { + s.downloadedPeriodsMtx.RLock() + fc, ok := s.downloadedPeriods[period] + s.downloadedPeriodsMtx.RUnlock() if !ok && createIfNotExists { - a.downloadedPeriodsMtx.Lock() - fc, ok = a.downloadedPeriods[period] + s.downloadedPeriodsMtx.Lock() + fc, ok = s.downloadedPeriods[period] if ok { - a.downloadedPeriodsMtx.Unlock() + s.downloadedPeriodsMtx.Unlock() } fc = &filesCollection{files: map[string]downloadedFiles{}} - a.downloadedPeriods[period] = fc + s.downloadedPeriods[period] = fc } return fc, nil } -func (a *Shipper) forEach(ctx context.Context, period string, callback func(db *bbolt.DB) error) error { - a.downloadedPeriodsMtx.RLock() - fc, ok := a.downloadedPeriods[period] - a.downloadedPeriodsMtx.RUnlock() +func (s *Shipper) forEach(ctx context.Context, period string, callback func(db *bbolt.DB) error) error { + s.downloadedPeriodsMtx.RLock() + fc, ok := s.downloadedPeriods[period] + s.downloadedPeriodsMtx.RUnlock() if !ok { - a.downloadedPeriodsMtx.Lock() - fc, ok = a.downloadedPeriods[period] + s.downloadedPeriodsMtx.Lock() + fc, ok = s.downloadedPeriods[period] if ok { - a.downloadedPeriodsMtx.Unlock() + s.downloadedPeriodsMtx.Unlock() } else { fc = &filesCollection{files: map[string]downloadedFiles{}} - a.downloadedPeriods[period] = fc - a.downloadedPeriodsMtx.Unlock() + s.downloadedPeriods[period] = fc + s.downloadedPeriodsMtx.Unlock() - if err := a.downloadFilesForPeriod(ctx, period, fc); err != nil { + if err := s.downloadFilesForPeriod(ctx, period, fc); err != nil { return err } } diff --git a/pkg/storage/stores/local/uploads.go b/pkg/storage/stores/local/uploads.go index 18a52ec4f82c..ddf1df064c7b 100644 --- a/pkg/storage/stores/local/uploads.go +++ b/pkg/storage/stores/local/uploads.go @@ -17,12 +17,12 @@ import ( // uploadFiles uploads all new and updated files to storage. // It uploads the files from configured boltdb dir where ingester writes the index. -func (a *Shipper) uploadFiles(ctx context.Context) error { - if a.cfg.Mode == ShipperModeReadOnly { +func (s *Shipper) uploadFiles(ctx context.Context) error { + if s.cfg.Mode == ShipperModeReadOnly { return nil } - filesInfo, err := ioutil.ReadDir(a.cfg.ActiveIndexDirectory) + filesInfo, err := ioutil.ReadDir(s.cfg.ActiveIndexDirectory) if err != nil { return err } @@ -32,41 +32,41 @@ func (a *Shipper) uploadFiles(ctx context.Context) error { continue } - a.uploadedFilesMtimeMtx.RLock() + s.uploadedFilesMtimeMtx.RLock() // Checking whether file is updated after last push, if not skipping it - uploadedFileMtime, ok := a.uploadedFilesMtime[fileInfo.Name()] - a.uploadedFilesMtimeMtx.RUnlock() + uploadedFileMtime, ok := s.uploadedFilesMtime[fileInfo.Name()] + s.uploadedFilesMtimeMtx.RUnlock() if ok && uploadedFileMtime.Equal(fileInfo.ModTime()) { continue } - err := a.uploadFile(ctx, fileInfo.Name()) + err := s.uploadFile(ctx, fileInfo.Name()) if err != nil { return err } - a.uploadedFilesMtimeMtx.Lock() - a.uploadedFilesMtime[fileInfo.Name()] = fileInfo.ModTime() - a.uploadedFilesMtimeMtx.Unlock() + s.uploadedFilesMtimeMtx.Lock() + s.uploadedFilesMtime[fileInfo.Name()] = fileInfo.ModTime() + s.uploadedFilesMtimeMtx.Unlock() } return nil } // uploadFile uploads one of the files locally written by ingesters to storage. -func (a *Shipper) uploadFile(ctx context.Context, period string) error { - if a.cfg.Mode == ShipperModeReadOnly { +func (s *Shipper) uploadFile(ctx context.Context, period string) error { + if s.cfg.Mode == ShipperModeReadOnly { return nil } - snapshotPath := path.Join(a.cfg.CacheLocation, period) + snapshotPath := path.Join(s.cfg.CacheLocation, period) err := chunk_util.EnsureDirectory(snapshotPath) if err != nil { return err } - filePath := path.Join(snapshotPath, fmt.Sprintf("%s.%d", a.uploader, time.Now().Unix())) + filePath := path.Join(snapshotPath, fmt.Sprintf("%s.%d", s.uploader, time.Now().Unix())) f, err := os.Create(filePath) if err != nil { return err @@ -78,7 +78,7 @@ func (a *Shipper) uploadFile(ctx context.Context, period string) error { } }() - db, err := a.boltDBGetter.GetDB(period, local.DBOperationRead) + db, err := s.boltDBGetter.GetDB(period, local.DBOperationRead) if err != nil { return err } @@ -106,6 +106,6 @@ func (a *Shipper) uploadFile(ctx context.Context, period string) error { }() // Files are stored with / - objectKey := fmt.Sprintf("%s/%s", period, a.uploader) - return a.storageClient.PutObject(ctx, objectKey, f) + objectKey := fmt.Sprintf("%s/%s", period, s.uploader) + return s.storageClient.PutObject(ctx, objectKey, f) } diff --git a/pkg/storage/stores/util/object_client.go b/pkg/storage/stores/util/object_client.go new file mode 100644 index 000000000000..bb95ca3ee17a --- /dev/null +++ b/pkg/storage/stores/util/object_client.go @@ -0,0 +1,47 @@ +package util + +import ( + "context" + "io" + "strings" + + "github.com/cortexproject/cortex/pkg/chunk" +) + +type PrefixedObjectClient struct { + downstreamClient chunk.ObjectClient + prefix string +} + +func (p PrefixedObjectClient) PutObject(ctx context.Context, objectKey string, object io.ReadSeeker) error { + return p.downstreamClient.PutObject(ctx, p.prefix+objectKey, object) +} + +func (p PrefixedObjectClient) GetObject(ctx context.Context, objectKey string) (io.ReadCloser, error) { + return p.downstreamClient.GetObject(ctx, p.prefix+objectKey) +} + +func (p PrefixedObjectClient) List(ctx context.Context, prefix string) ([]chunk.StorageObject, error) { + objects, err := p.downstreamClient.List(ctx, p.prefix+prefix) + if err != nil { + return nil, err + } + + for i := range objects { + objects[i].Key = strings.TrimPrefix(objects[i].Key, p.prefix) + } + + return objects, nil +} + +func (p PrefixedObjectClient) DeleteObject(ctx context.Context, objectKey string) error { + return p.downstreamClient.DeleteObject(ctx, p.prefix+objectKey) +} + +func (p PrefixedObjectClient) Stop() { + p.downstreamClient.Stop() +} + +func NewPrefixedObjectClient(downstreamClient chunk.ObjectClient, prefix string) chunk.ObjectClient { + return PrefixedObjectClient{downstreamClient: downstreamClient, prefix: prefix} +} From 2b1c53047231778ca0b9e2cf4b307c62916dbf73 Mon Sep 17 00:00:00 2001 From: Sandeep Sukhani Date: Thu, 2 Apr 2020 14:43:33 +0530 Subject: [PATCH 06/13] updated cli flag for active directory --- pkg/storage/stores/local/shipper.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pkg/storage/stores/local/shipper.go b/pkg/storage/stores/local/shipper.go index e8da71ab4ada..836505c67fb7 100644 --- a/pkg/storage/stores/local/shipper.go +++ b/pkg/storage/stores/local/shipper.go @@ -53,7 +53,7 @@ type ShipperConfig struct { // RegisterFlags registers flags. func (cfg *ShipperConfig) RegisterFlags(f *flag.FlagSet) { - f.StringVar(&cfg.ActiveIndexDirectory, "boltdb.active-index-directory", "", "Directory where ingesters would write boltdb files which would then be uploaded by shipper to configured storage") + f.StringVar(&cfg.ActiveIndexDirectory, "boltdb.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, "boltdb.shipper.shared-store", "", "Shared store for keeping boltdb files. Supported types: gcs, s3, azure, filesystem") f.StringVar(&cfg.CacheLocation, "boltdb.shipper.cache-location", "", "Cache location for restoring boltDB files for queries") f.DurationVar(&cfg.CacheTTL, "boltdb.shipper.cache-ttl", 24*time.Hour, "TTL for boltDB files restored in cache for queries") From 8348471d9063d8d16cf1ff0c656ea5458a76c62f Mon Sep 17 00:00:00 2001 From: Sandeep Sukhani Date: Wed, 8 Apr 2020 20:27:43 +0530 Subject: [PATCH 07/13] add tests for boltdb shipper and update vendor --- cmd/loki/loki-local-config.yaml | 13 +- go.mod | 2 +- .../stores/local/boltdb_index_client.go | 2 +- pkg/storage/stores/local/downloads.go | 20 ++- pkg/storage/stores/local/downloads_test.go | 112 ++++++++++++ pkg/storage/stores/local/shipper.go | 2 + pkg/storage/stores/local/uploads.go | 2 + pkg/storage/stores/local/uploads_test.go | 161 ++++++++++++++++++ 8 files changed, 306 insertions(+), 8 deletions(-) create mode 100644 pkg/storage/stores/local/downloads_test.go create mode 100644 pkg/storage/stores/local/uploads_test.go diff --git a/cmd/loki/loki-local-config.yaml b/cmd/loki/loki-local-config.yaml index 390f542a8c16..4d5d1de792ca 100644 --- a/cmd/loki/loki-local-config.yaml +++ b/cmd/loki/loki-local-config.yaml @@ -18,11 +18,11 @@ ingester: schema_config: configs: - from: 2018-04-15 - store: boltdb - object_store: filesystem + store: boltdb-shipper + object_store: gcs schema: v11 index: - prefix: index_ + prefix: loki_boltdb_shipper_index_ period: 168h storage_config: @@ -32,6 +32,13 @@ storage_config: filesystem: directory: /loki/chunks + gcs: + bucket_name: 'dev-us-central1-loki-boltdb-shipper' + + boltdb_shipper_config: + active_index_directory: /tmp/loki/index + cache_location: /tmp/loki/boltdb-cache + limits_config: enforce_metric_name: false reject_old_samples: true diff --git a/go.mod b/go.mod index c38e13068f4b..8624e86064df 100644 --- a/go.mod +++ b/go.mod @@ -70,4 +70,4 @@ replace github.com/Azure/azure-sdk-for-go => github.com/Azure/azure-sdk-for-go v replace github.com/Azure/go-autorest => github.com/Azure/go-autorest v13.3.0+incompatible -replace github.com/cortexproject/cortex => github.com/sandeepsukhani/cortex v0.0.0-20200401102156-8377099831f7 +replace github.com/cortexproject/cortex => github.com/grafana/cortex v0.0.0-20200408151555-2e824a9e4de8 diff --git a/pkg/storage/stores/local/boltdb_index_client.go b/pkg/storage/stores/local/boltdb_index_client.go index a518b1594f95..286780d79966 100644 --- a/pkg/storage/stores/local/boltdb_index_client.go +++ b/pkg/storage/stores/local/boltdb_index_client.go @@ -35,8 +35,8 @@ func NewBoltDBIndexClient(cfg local.BoltDBConfig, archiveStoreClient chunk.Objec } func (b *BoltdbIndexClientWithShipper) Stop() { - b.BoltIndexClient.Stop() b.shipper.Stop() + b.BoltIndexClient.Stop() } func (b *BoltdbIndexClientWithShipper) QueryPages(ctx context.Context, queries []chunk.IndexQuery, callback func(chunk.IndexQuery, chunk.ReadBatch) (shouldContinue bool)) error { diff --git a/pkg/storage/stores/local/downloads.go b/pkg/storage/stores/local/downloads.go index 29bce78e3592..66197c9d6cdd 100644 --- a/pkg/storage/stores/local/downloads.go +++ b/pkg/storage/stores/local/downloads.go @@ -24,7 +24,7 @@ func (s *Shipper) checkStorageForUpdates(ctx context.Context, period string, fc // listing tables from store var objects []chunk.StorageObject - objects, err = s.storageClient.List(ctx, period) + objects, err = s.storageClient.List(ctx, period+"/") if err != nil { return } @@ -57,6 +57,8 @@ func (s *Shipper) checkStorageForUpdates(ctx context.Context, period string, fc // syncFilesForPeriod downloads updated and new files from for given period from all the uploaders and removes deleted ones func (s *Shipper) syncFilesForPeriod(ctx context.Context, period string, fc *filesCollection) error { + level.Debug(util.Logger).Log("msg", fmt.Sprintf("syncing files for period %s", period)) + fc.RLock() toDownload, toDelete, err := s.checkStorageForUpdates(ctx, period, fc) fc.RUnlock() @@ -144,7 +146,13 @@ func (s *Shipper) getFileFromStorage(ctx context.Context, objectKey, destination } _, err = io.Copy(f, readCloser) - return err + if err != nil { + return err + } + + level.Info(util.Logger).Log("msg", fmt.Sprintf("downloaded file %s", objectKey)) + + return f.Sync() } // downloadFilesForPeriod should be called when files for a period does not exist i.e they were never downloaded or got cleaned up later on by TTL @@ -153,11 +161,13 @@ func (s *Shipper) downloadFilesForPeriod(ctx context.Context, period string, fc fc.Lock() defer fc.Unlock() - objects, err := s.storageClient.List(ctx, period) + objects, err := s.storageClient.List(ctx, period+"/") if err != nil { return err } + level.Debug(util.Logger).Log("msg", fmt.Sprintf("list of files to download for period %s: %s", period, objects)) + folderPath, err := s.getFolderPathForPeriod(period, true) if err != nil { return err @@ -165,6 +175,10 @@ func (s *Shipper) downloadFilesForPeriod(ctx context.Context, period string, fc for _, object := range objects { uploader := getUploaderFromObjectKey(object.Key) + if uploader == s.uploader { + continue + } + filePath := path.Join(folderPath, uploader) df := downloadedFiles{} diff --git a/pkg/storage/stores/local/downloads_test.go b/pkg/storage/stores/local/downloads_test.go new file mode 100644 index 000000000000..bdf6cc6cd2b0 --- /dev/null +++ b/pkg/storage/stores/local/downloads_test.go @@ -0,0 +1,112 @@ +package local + +import ( + "context" + "io/ioutil" + "os" + "path/filepath" + "strconv" + "testing" + + "github.com/cortexproject/cortex/pkg/chunk" + "github.com/stretchr/testify/require" +) + +func queryTestBoltdb(t *testing.T, boltdbIndexClient *BoltdbIndexClientWithShipper, query chunk.IndexQuery) map[string]string { + resp := map[string]string{} + + require.NoError(t, boltdbIndexClient.query(context.Background(), query, func(batch chunk.ReadBatch) (shouldContinue bool) { + itr := batch.Iterator() + for itr.Next() { + resp[string(itr.RangeValue())] = string(itr.Value()) + } + return true + })) + + return resp +} + +func writeTestData(t *testing.T, indexClient chunk.IndexClient, tableName string, numRecords, startValue int) { + batch := indexClient.NewWriteBatch() + for i := 0; i < numRecords; i++ { + value := []byte(strconv.Itoa(startValue + i)) + batch.Add(tableName, "", value, value) + } + + require.NoError(t, indexClient.BatchWrite(context.Background(), batch)) + +} + +func TestShipper_Downloads(t *testing.T) { + tempDirForTests, err := ioutil.TempDir("", "test-dir") + require.NoError(t, err) + + defer func() { + require.NoError(t, os.RemoveAll(tempDirForTests)) + }() + + localStoreLocation, err := ioutil.TempDir(tempDirForTests, "local-store") + require.NoError(t, err) + + boltDBWithShipper1 := createTestBoltDBWithShipper(t, tempDirForTests, "ingester1", localStoreLocation) + boltDBWithShipper2 := createTestBoltDBWithShipper(t, tempDirForTests, "ingester2", localStoreLocation) + + // add a file to boltDBWithShipper1 + writeTestData(t, boltDBWithShipper1, "1", 10, 0) + + // upload files from boltDBWithShipper1 + require.NoError(t, boltDBWithShipper1.shipper.uploadFiles(context.Background())) + + // query data for same table from boltDBWithShipper2 + resp := queryTestBoltdb(t, boltDBWithShipper2, chunk.IndexQuery{ + TableName: "1", + }) + + // make sure we got same data that was added from boltDBWithShipper1 + checkExpectedKVsInBoltdbResp(t, resp, 10, 0) + + // add more data to the previous file added to boltDBWithShipper1 and the upload it + writeTestData(t, boltDBWithShipper1, "1", 10, 10) + require.NoError(t, boltDBWithShipper1.shipper.uploadFiles(context.Background())) + + // sync files in boltDBWithShipper2 + require.NoError(t, boltDBWithShipper2.shipper.syncLocalWithStorage(context.Background())) + + // query data for same table from boltDBWithShipper2 + resp = queryTestBoltdb(t, boltDBWithShipper2, chunk.IndexQuery{ + TableName: "1", + }) + + // make sure we also got new data that was added from boltDBWithShipper1 + checkExpectedKVsInBoltdbResp(t, resp, 20, 0) + + // add some data for same table in boltDBWithShipper2 + writeTestData(t, boltDBWithShipper2, "1", 10, 20) + + // query data for same table from boltDBWithShipper2 + resp = queryTestBoltdb(t, boltDBWithShipper2, chunk.IndexQuery{ + TableName: "1", + }) + + // make sure we data from boltDBWithShipper1 and boltDBWithShipper2 + checkExpectedKVsInBoltdbResp(t, resp, 30, 0) + + // stop boltDBWithShipper1 + boltDBWithShipper1.Stop() + + // delete the file from the store that was uploaded by boltDBWithShipper1 + require.NoError(t, os.Remove(filepath.Join(localStoreLocation, storageKeyPrefix, "1", boltDBWithShipper1.shipper.uploader))) + + // sync files in boltDBWithShipper2 + require.NoError(t, boltDBWithShipper2.shipper.syncLocalWithStorage(context.Background())) + + // query data for same table from boltDBWithShipper2 + resp = queryTestBoltdb(t, boltDBWithShipper2, chunk.IndexQuery{ + TableName: "1", + }) + + // make sure we got only data that was added to boltDBWithShipper2 + checkExpectedKVsInBoltdbResp(t, resp, 10, 20) + + boltDBWithShipper2.Stop() +} diff --git a/pkg/storage/stores/local/shipper.go b/pkg/storage/stores/local/shipper.go index 836505c67fb7..d6631ee3f52e 100644 --- a/pkg/storage/stores/local/shipper.go +++ b/pkg/storage/stores/local/shipper.go @@ -287,6 +287,8 @@ func (s *Shipper) forEach(ctx context.Context, period string, callback func(db * if ok { s.downloadedPeriodsMtx.Unlock() } else { + level.Info(pkg_util.Logger).Log("msg", fmt.Sprintf("downloading all files for period %s", period)) + fc = &filesCollection{files: map[string]downloadedFiles{}} s.downloadedPeriods[period] = fc s.downloadedPeriodsMtx.Unlock() diff --git a/pkg/storage/stores/local/uploads.go b/pkg/storage/stores/local/uploads.go index ddf1df064c7b..8f6bd271d3b9 100644 --- a/pkg/storage/stores/local/uploads.go +++ b/pkg/storage/stores/local/uploads.go @@ -60,6 +60,8 @@ func (s *Shipper) uploadFile(ctx context.Context, period string) error { return nil } + level.Debug(util.Logger).Log("msg", fmt.Sprintf("uploading file for period %s", period)) + snapshotPath := path.Join(s.cfg.CacheLocation, period) err := chunk_util.EnsureDirectory(snapshotPath) if err != nil { diff --git a/pkg/storage/stores/local/uploads_test.go b/pkg/storage/stores/local/uploads_test.go new file mode 100644 index 000000000000..1b00dbc47c41 --- /dev/null +++ b/pkg/storage/stores/local/uploads_test.go @@ -0,0 +1,161 @@ +package local + +import ( + "context" + "io/ioutil" + "os" + "path/filepath" + "strconv" + "testing" + "time" + + "github.com/cortexproject/cortex/pkg/chunk/local" + "github.com/cortexproject/cortex/pkg/chunk/util" + "github.com/stretchr/testify/require" + "go.etcd.io/bbolt" +) + +const testBucketName = "testBucket" + +func createTestBoltDBWithShipper(t *testing.T, parentTempDir, ingesterName, localStoreLocation string) *BoltdbIndexClientWithShipper { + cacheLocation := filepath.Join(parentTempDir, ingesterName, "cache") + boltdbFilesLocation := filepath.Join(parentTempDir, ingesterName, "boltdb") + + require.NoError(t, util.EnsureDirectory(cacheLocation)) + require.NoError(t, util.EnsureDirectory(boltdbFilesLocation)) + + shipperConfig := ShipperConfig{ + ActiveIndexDirectory: boltdbFilesLocation, + CacheLocation: cacheLocation, + CacheTTL: 1 * time.Hour, + ResyncInterval: 1 * time.Hour, + IngesterName: ingesterName, + Mode: ShipperModeReadWrite, + } + + archiveStoreClient, err := local.NewFSObjectClient(local.FSConfig{ + Directory: localStoreLocation, + }) + require.NoError(t, err) + + boltdbIndexClientWithShipper, err := NewBoltDBIndexClient(local.BoltDBConfig{Directory: shipperConfig.ActiveIndexDirectory}, archiveStoreClient, shipperConfig) + require.NoError(t, err) + + return boltdbIndexClientWithShipper.(*BoltdbIndexClientWithShipper) +} + +func addTestRecordsToBoltDBFile(t *testing.T, boltdb *bbolt.DB, numRecords int, start int) { + err := boltdb.Update(func(tx *bbolt.Tx) error { + b, err := tx.CreateBucketIfNotExists([]byte(testBucketName)) + if err != nil { + return err + } + + for i := 0; i < numRecords; i++ { + kv := []byte(strconv.Itoa(start + i)) + + err = b.Put(kv, kv) + if err != nil { + return err + } + } + return nil + }) + + require.NoError(t, err) +} + +func readAllKVsFromBoltdbFile(t *testing.T, boltdb *bbolt.DB) map[string]string { + resp := map[string]string{} + + err := boltdb.View(func(tx *bbolt.Tx) error { + b := tx.Bucket([]byte(testBucketName)) + require.NotNil(t, b) + + return b.ForEach(func(k, v []byte) error { + resp[string(k)] = string(v) + return nil + }) + }) + + require.NoError(t, err) + + return resp +} + +func readAllKVsFromBoltdbFileAtPath(t *testing.T, path string) map[string]string { + boltDBFile, err := local.OpenBoltdbFile(path) + require.NoError(t, err) + + defer func() { + require.NoError(t, boltDBFile.Close()) + }() + + return readAllKVsFromBoltdbFile(t, boltDBFile) +} + +func checkExpectedKVsInBoltdbResp(t *testing.T, resp map[string]string, expectedNumRecords, start int) { + require.Equal(t, expectedNumRecords, len(resp), "responses", resp) + + for i := 0; i < expectedNumRecords; i++ { + expectedKV := strconv.Itoa(start + i) + + val, ok := resp[expectedKV] + require.Equal(t, true, ok) + require.Equal(t, expectedKV, val) + } +} + +func TestShipper_Uploads(t *testing.T) { + tempDirForTests, err := ioutil.TempDir("", "test-dir") + require.NoError(t, err) + + defer func() { + require.NoError(t, os.RemoveAll(tempDirForTests)) + }() + + localStoreLocation, err := ioutil.TempDir(tempDirForTests, "local-store") + require.NoError(t, err) + + boltDBWithShipper := createTestBoltDBWithShipper(t, tempDirForTests, "ingester", localStoreLocation) + + // create a boltdb file for boltDBWithShipper to test upload. + boltdbFile1, err := boltDBWithShipper.GetDB("file1", local.DBOperationWrite) + require.NoError(t, err) + file1PathInStorage := filepath.Join(localStoreLocation, storageKeyPrefix, filepath.Base(boltdbFile1.Path()), boltDBWithShipper.shipper.uploader) + + // add some test records to boltdbFile1 + addTestRecordsToBoltDBFile(t, boltdbFile1, 10, 1) + + // Upload files from boltDBWithShipper + err = boltDBWithShipper.shipper.uploadFiles(context.Background()) + require.NoError(t, err) + + // open boltdbFile1 and verify it has expected records + checkExpectedKVsInBoltdbResp(t, readAllKVsFromBoltdbFileAtPath(t, file1PathInStorage), 10, 1) + + // create another boltdb file for boltDBWithShipper to test upload. + boltdbFile2, err := boltDBWithShipper.GetDB("file2", local.DBOperationWrite) + require.NoError(t, err) + file2PathInStorage := filepath.Join(localStoreLocation, storageKeyPrefix, filepath.Base(boltdbFile2.Path()), boltDBWithShipper.shipper.uploader) + + // add some test records to boltdbFile2 and some more records to boltdbFile1 + addTestRecordsToBoltDBFile(t, boltdbFile2, 10, 1) + addTestRecordsToBoltDBFile(t, boltdbFile1, 10, 11) + + // Upload files from boltDBWithShipper + err = boltDBWithShipper.shipper.uploadFiles(context.Background()) + require.NoError(t, err) + + // open boltdbFile1 and boltdbFile2 and verify it has expected records + checkExpectedKVsInBoltdbResp(t, readAllKVsFromBoltdbFileAtPath(t, file2PathInStorage), 10, 1) + checkExpectedKVsInBoltdbResp(t, readAllKVsFromBoltdbFileAtPath(t, file1PathInStorage), 20, 1) + + // modify boltdbFile2 again + addTestRecordsToBoltDBFile(t, boltdbFile2, 10, 11) + + // stop boltDBWithShipper to make it upload all the new and changed to store + boltDBWithShipper.Stop() + + checkExpectedKVsInBoltdbResp(t, readAllKVsFromBoltdbFileAtPath(t, file2PathInStorage), 20, 1) +} From b58cfb3023c8f7a0aa1ecdbdeea63a92b8e0a784 Mon Sep 17 00:00:00 2001 From: Sandeep Sukhani Date: Sat, 11 Apr 2020 23:45:48 +0530 Subject: [PATCH 08/13] syncing boltdb files to disk during update --- pkg/storage/stores/local/downloads_test.go | 8 +++++++- pkg/storage/stores/local/uploads_test.go | 2 ++ 2 files changed, 9 insertions(+), 1 deletion(-) diff --git a/pkg/storage/stores/local/downloads_test.go b/pkg/storage/stores/local/downloads_test.go index bdf6cc6cd2b0..696b9e71bf04 100644 --- a/pkg/storage/stores/local/downloads_test.go +++ b/pkg/storage/stores/local/downloads_test.go @@ -8,6 +8,8 @@ import ( "strconv" "testing" + "github.com/cortexproject/cortex/pkg/chunk/local" + "github.com/cortexproject/cortex/pkg/chunk" "github.com/stretchr/testify/require" ) @@ -26,7 +28,7 @@ func queryTestBoltdb(t *testing.T, boltdbIndexClient *BoltdbIndexClientWithShipp return resp } -func writeTestData(t *testing.T, indexClient chunk.IndexClient, tableName string, numRecords, startValue int) { +func writeTestData(t *testing.T, indexClient *BoltdbIndexClientWithShipper, tableName string, numRecords, startValue int) { batch := indexClient.NewWriteBatch() for i := 0; i < numRecords; i++ { value := []byte(strconv.Itoa(startValue + i)) @@ -35,6 +37,10 @@ func writeTestData(t *testing.T, indexClient chunk.IndexClient, tableName string require.NoError(t, indexClient.BatchWrite(context.Background(), batch)) + boltdb, err := indexClient.GetDB(tableName, local.DBOperationWrite) + require.NoError(t, err) + + require.NoError(t, boltdb.Sync()) } func TestShipper_Downloads(t *testing.T) { diff --git a/pkg/storage/stores/local/uploads_test.go b/pkg/storage/stores/local/uploads_test.go index 1b00dbc47c41..2d445e7c93d6 100644 --- a/pkg/storage/stores/local/uploads_test.go +++ b/pkg/storage/stores/local/uploads_test.go @@ -59,10 +59,12 @@ func addTestRecordsToBoltDBFile(t *testing.T, boltdb *bbolt.DB, numRecords int, return err } } + return nil }) require.NoError(t, err) + require.NoError(t, boltdb.Sync()) } func readAllKVsFromBoltdbFile(t *testing.T, boltdb *bbolt.DB) map[string]string { From 4c632047e18d3a6192e580724061ff27da5f73da Mon Sep 17 00:00:00 2001 From: Sandeep Sukhani Date: Sun, 12 Apr 2020 01:15:08 +0530 Subject: [PATCH 09/13] sleep in tests to let mtime of boltdb file be changed --- pkg/storage/stores/local/downloads_test.go | 3 +++ pkg/storage/stores/local/uploads_test.go | 2 ++ 2 files changed, 5 insertions(+) diff --git a/pkg/storage/stores/local/downloads_test.go b/pkg/storage/stores/local/downloads_test.go index 696b9e71bf04..47710e892243 100644 --- a/pkg/storage/stores/local/downloads_test.go +++ b/pkg/storage/stores/local/downloads_test.go @@ -7,6 +7,7 @@ import ( "path/filepath" "strconv" "testing" + "time" "github.com/cortexproject/cortex/pkg/chunk/local" @@ -29,6 +30,8 @@ func queryTestBoltdb(t *testing.T, boltdbIndexClient *BoltdbIndexClientWithShipp } func writeTestData(t *testing.T, indexClient *BoltdbIndexClientWithShipper, tableName string, numRecords, startValue int) { + time.Sleep(time.Second / 2) + batch := indexClient.NewWriteBatch() for i := 0; i < numRecords; i++ { value := []byte(strconv.Itoa(startValue + i)) diff --git a/pkg/storage/stores/local/uploads_test.go b/pkg/storage/stores/local/uploads_test.go index 2d445e7c93d6..d54463fa2b67 100644 --- a/pkg/storage/stores/local/uploads_test.go +++ b/pkg/storage/stores/local/uploads_test.go @@ -45,6 +45,8 @@ func createTestBoltDBWithShipper(t *testing.T, parentTempDir, ingesterName, loca } func addTestRecordsToBoltDBFile(t *testing.T, boltdb *bbolt.DB, numRecords int, start int) { + time.Sleep(time.Second / 2) + err := boltdb.Update(func(tx *bbolt.Tx) error { b, err := tx.CreateBucketIfNotExists([]byte(testBucketName)) if err != nil { From 567c70feff342ecaba61d3b52455b7b85b8866c8 Mon Sep 17 00:00:00 2001 From: Sandeep Sukhani Date: Fri, 17 Apr 2020 14:05:19 +0530 Subject: [PATCH 10/13] changes suggested from PR review --- cmd/loki/loki-local-config.yaml | 13 +---- pkg/ingester/ingester.go | 45 +++++++------- pkg/ingester/ingester_test.go | 74 ++++++++++++++++++++++++ pkg/storage/store.go | 4 +- pkg/storage/stores/local/downloads.go | 3 +- pkg/storage/stores/local/shipper.go | 19 ------ pkg/storage/stores/local/uploads.go | 3 +- pkg/storage/stores/local/uploads_test.go | 4 +- 8 files changed, 108 insertions(+), 57 deletions(-) diff --git a/cmd/loki/loki-local-config.yaml b/cmd/loki/loki-local-config.yaml index 4d5d1de792ca..390f542a8c16 100644 --- a/cmd/loki/loki-local-config.yaml +++ b/cmd/loki/loki-local-config.yaml @@ -18,11 +18,11 @@ ingester: schema_config: configs: - from: 2018-04-15 - store: boltdb-shipper - object_store: gcs + store: boltdb + object_store: filesystem schema: v11 index: - prefix: loki_boltdb_shipper_index_ + prefix: index_ period: 168h storage_config: @@ -32,13 +32,6 @@ storage_config: filesystem: directory: /loki/chunks - gcs: - bucket_name: 'dev-us-central1-loki-boltdb-shipper' - - boltdb_shipper_config: - active_index_directory: /tmp/loki/index - cache_location: /tmp/loki/boltdb-cache - limits_config: enforce_metric_name: false reject_old_samples: true diff --git a/pkg/ingester/ingester.go b/pkg/ingester/ingester.go index 5a2536882de7..e035fea66787 100644 --- a/pkg/ingester/ingester.go +++ b/pkg/ingester/ingester.go @@ -260,26 +260,13 @@ func (i *Ingester) Query(req *logproto.QueryRequest, queryServer logproto.Querie return err } - if i.cfg.QueryStore { - start := req.Start - end := req.End - if i.cfg.QueryStoreMaxLookBackPeriod != 0 { - oldestStartTime := time.Now().Add(-i.cfg.QueryStoreMaxLookBackPeriod) - if oldestStartTime.After(req.Start) { - start = oldestStartTime - } + if storeReq := buildStoreRequest(i.cfg, req); storeReq != nil { + storeItr, err := i.store.LazyQuery(ctx, logql.SelectParams{QueryRequest: storeReq}) + if err != nil { + return err } - if start.Before(end) { - storeRequest := recreateRequestWithTime(req, start, end) - - storeItr, err := i.store.LazyQuery(ctx, logql.SelectParams{QueryRequest: storeRequest}) - if err != nil { - return err - } - - itrs = append(itrs, storeItr) - } + itrs = append(itrs, storeItr) } heapItr := iter.NewHeapIterator(ctx, itrs, req.Direction) @@ -396,8 +383,26 @@ func (i *Ingester) TailersCount(ctx context.Context, in *logproto.TailersCountRe return &resp, nil } -// creates a new QueryRequest with a query range -func recreateRequestWithTime(req *logproto.QueryRequest, start, end time.Time) *logproto.QueryRequest { +// buildStoreRequest returns a store request from an ingester request, returns nit if QueryStore is set to false in configuration. +// The request may be truncated due to QueryStoreMaxLookBackPeriod which limits the range of request to make sure +// we only query enough to not miss any data and not add too to many duplicates by covering the who time range in query. +func buildStoreRequest(cfg Config, req *logproto.QueryRequest) *logproto.QueryRequest { + if !cfg.QueryStore { + return nil + } + start := req.Start + end := req.End + if cfg.QueryStoreMaxLookBackPeriod != 0 { + oldestStartTime := time.Now().Add(-cfg.QueryStoreMaxLookBackPeriod) + if oldestStartTime.After(req.Start) { + start = oldestStartTime + } + } + + if start.After(end) { + return nil + } + newRequest := *req newRequest.Start = start newRequest.End = end diff --git a/pkg/ingester/ingester_test.go b/pkg/ingester/ingester_test.go index 40a0684f4989..4bf9f5b0062b 100644 --- a/pkg/ingester/ingester_test.go +++ b/pkg/ingester/ingester_test.go @@ -276,3 +276,77 @@ func defaultLimitsTestConfig() validation.Limits { flagext.DefaultValues(&limits) return limits } + +func TestIngester_buildStoreRequest(t *testing.T) { + ingesterQueryRequest := logproto.QueryRequest{ + Selector: `{foo="bar"}`, + Limit: 100, + } + + now := time.Now() + + for _, tc := range []struct { + name string + queryStore bool + maxLookBackPeriod time.Duration + ingesterQueryRequest *logproto.QueryRequest + expectedStoreQueryRequest *logproto.QueryRequest + }{ + { + name: "do not query store", + queryStore: false, + ingesterQueryRequest: recreateRequestWithTime(ingesterQueryRequest, now.Add(-time.Minute), now), + expectedStoreQueryRequest: nil, + }, + { + name: "query store with max look back covering whole request duration", + queryStore: true, + maxLookBackPeriod: time.Hour, + ingesterQueryRequest: recreateRequestWithTime(ingesterQueryRequest, now.Add(-10*time.Minute), now), + expectedStoreQueryRequest: recreateRequestWithTime(ingesterQueryRequest, now.Add(-10*time.Minute), now), + }, + { + name: "query store with max look back covering partial request duration", + queryStore: true, + maxLookBackPeriod: time.Hour, + ingesterQueryRequest: recreateRequestWithTime(ingesterQueryRequest, now.Add(-2*time.Hour), now), + expectedStoreQueryRequest: recreateRequestWithTime(ingesterQueryRequest, now.Add(-time.Hour), now), + }, + { + name: "query store with max look back not covering request duration at all", + queryStore: true, + maxLookBackPeriod: time.Hour, + ingesterQueryRequest: recreateRequestWithTime(ingesterQueryRequest, now.Add(-4*time.Hour), now.Add(-2*time.Hour)), + expectedStoreQueryRequest: nil, + }, + } { + t.Run(tc.name, func(t *testing.T) { + ingesterConfig := defaultIngesterTestConfig(t) + ingesterConfig.QueryStore = tc.queryStore + ingesterConfig.QueryStoreMaxLookBackPeriod = tc.maxLookBackPeriod + storeRequest := buildStoreRequest(ingesterConfig, tc.ingesterQueryRequest) + if tc.expectedStoreQueryRequest == nil { + require.Nil(t, storeRequest) + return + } + + // because start time of store could be changed and built based on time when function is called we can't predict expected start time. + // So allowing upto 1s difference between expected and actual start time of store query request. + require.Equal(t, tc.expectedStoreQueryRequest.Selector, storeRequest.Selector) + require.Equal(t, tc.expectedStoreQueryRequest.Limit, storeRequest.Limit) + require.Equal(t, tc.expectedStoreQueryRequest.End, storeRequest.End) + + if storeRequest.Start.Sub(tc.expectedStoreQueryRequest.Start) > time.Second { + t.Fatalf("expected upto 1s difference in expected and acutal store request end time but got %d", storeRequest.End.Sub(tc.expectedStoreQueryRequest.End)) + } + }) + } +} + +func recreateRequestWithTime(req logproto.QueryRequest, start, end time.Time) *logproto.QueryRequest { + newReq := req + newReq.Start = start + newReq.End = end + + return &newReq +} diff --git a/pkg/storage/store.go b/pkg/storage/store.go index 8a30054b2b15..8a0cca4dadb3 100644 --- a/pkg/storage/store.go +++ b/pkg/storage/store.go @@ -233,7 +233,7 @@ func registerCustomIndexClients(cfg Config, schemaCfg chunk.SchemaConfig) { objectStoreType := "" for _, config := range schemaCfg.Configs { if config.IndexType == local.BoltDBShipperType { - boltdbShipperEncounter += 1 + boltdbShipperEncounter++ if boltdbShipperEncounter > boltdbShipperInstances { objectStoreType = config.ObjectType break @@ -241,7 +241,7 @@ func registerCustomIndexClients(cfg Config, schemaCfg chunk.SchemaConfig) { } } - boltdbShipperInstances += 1 + boltdbShipperInstances++ objectClient, err := stores.NewObjectClient(objectStoreType, cfg.Config) if err != nil { return nil, err diff --git a/pkg/storage/stores/local/downloads.go b/pkg/storage/stores/local/downloads.go index 66197c9d6cdd..b98e2df0215e 100644 --- a/pkg/storage/stores/local/downloads.go +++ b/pkg/storage/stores/local/downloads.go @@ -7,7 +7,6 @@ import ( "os" "path" "strings" - "time" "github.com/cortexproject/cortex/pkg/chunk" "github.com/cortexproject/cortex/pkg/chunk/local" @@ -91,7 +90,7 @@ func (s *Shipper) downloadFile(ctx context.Context, period string, storageObject filePath := path.Join(folderPath, uploader) // download the file temporarily with some other name to allow boltdb client to close the existing file first if it exists - tempFilePath := path.Join(folderPath, fmt.Sprintf("%s.%d", uploader, time.Now().Unix())) + tempFilePath := path.Join(folderPath, fmt.Sprintf("%s.%s", uploader, "temp")) err := s.getFileFromStorage(ctx, storageObject.Key, tempFilePath) if err != nil { diff --git a/pkg/storage/stores/local/shipper.go b/pkg/storage/stores/local/shipper.go index d6631ee3f52e..bddcc93ab5dd 100644 --- a/pkg/storage/stores/local/shipper.go +++ b/pkg/storage/stores/local/shipper.go @@ -257,25 +257,6 @@ func (s *Shipper) deleteFileFromCache(period, uploader string, fc *filesCollecti return os.Remove(path.Join(s.cfg.CacheLocation, period, uploader)) } -func (s *Shipper) getFilesCollection(ctx context.Context, period string, createIfNotExists bool) (*filesCollection, error) { - s.downloadedPeriodsMtx.RLock() - fc, ok := s.downloadedPeriods[period] - s.downloadedPeriodsMtx.RUnlock() - - if !ok && createIfNotExists { - s.downloadedPeriodsMtx.Lock() - fc, ok = s.downloadedPeriods[period] - if ok { - s.downloadedPeriodsMtx.Unlock() - } - - fc = &filesCollection{files: map[string]downloadedFiles{}} - s.downloadedPeriods[period] = fc - } - - return fc, nil -} - func (s *Shipper) forEach(ctx context.Context, period string, callback func(db *bbolt.DB) error) error { s.downloadedPeriodsMtx.RLock() fc, ok := s.downloadedPeriods[period] diff --git a/pkg/storage/stores/local/uploads.go b/pkg/storage/stores/local/uploads.go index 8f6bd271d3b9..b44e52928900 100644 --- a/pkg/storage/stores/local/uploads.go +++ b/pkg/storage/stores/local/uploads.go @@ -6,7 +6,6 @@ import ( "io/ioutil" "os" "path" - "time" "github.com/cortexproject/cortex/pkg/chunk/local" chunk_util "github.com/cortexproject/cortex/pkg/chunk/util" @@ -68,7 +67,7 @@ func (s *Shipper) uploadFile(ctx context.Context, period string) error { return err } - filePath := path.Join(snapshotPath, fmt.Sprintf("%s.%d", s.uploader, time.Now().Unix())) + filePath := path.Join(snapshotPath, fmt.Sprintf("%s.%s", s.uploader, "temp")) f, err := os.Create(filePath) if err != nil { return err diff --git a/pkg/storage/stores/local/uploads_test.go b/pkg/storage/stores/local/uploads_test.go index d54463fa2b67..4a71da5194b7 100644 --- a/pkg/storage/stores/local/uploads_test.go +++ b/pkg/storage/stores/local/uploads_test.go @@ -145,7 +145,7 @@ func TestShipper_Uploads(t *testing.T) { // add some test records to boltdbFile2 and some more records to boltdbFile1 addTestRecordsToBoltDBFile(t, boltdbFile2, 10, 1) - addTestRecordsToBoltDBFile(t, boltdbFile1, 10, 11) + addTestRecordsToBoltDBFile(t, boltdbFile1, 5, 11) // Upload files from boltDBWithShipper err = boltDBWithShipper.shipper.uploadFiles(context.Background()) @@ -153,7 +153,7 @@ func TestShipper_Uploads(t *testing.T) { // open boltdbFile1 and boltdbFile2 and verify it has expected records checkExpectedKVsInBoltdbResp(t, readAllKVsFromBoltdbFileAtPath(t, file2PathInStorage), 10, 1) - checkExpectedKVsInBoltdbResp(t, readAllKVsFromBoltdbFileAtPath(t, file1PathInStorage), 20, 1) + checkExpectedKVsInBoltdbResp(t, readAllKVsFromBoltdbFileAtPath(t, file1PathInStorage), 15, 1) // modify boltdbFile2 again addTestRecordsToBoltDBFile(t, boltdbFile2, 10, 11) From eb1381a64fc168ddd34a5ef78be9875df67fee20 Mon Sep 17 00:00:00 2001 From: Sandeep Sukhani Date: Sat, 18 Apr 2020 18:58:47 +0530 Subject: [PATCH 11/13] revert go mod changes for using custom branch for cortex --- go.mod | 2 - .../cortex/pkg/chunk/chunk_store.go | 1 - .../pkg/chunk/local/fs_object_client.go | 14 +- .../pkg/chunk/storage/caching_index_client.go | 1 - vendor/github.com/thanos-io/thanos/LICENSE | 201 ------------------ 5 files changed, 2 insertions(+), 217 deletions(-) delete mode 100644 vendor/github.com/thanos-io/thanos/LICENSE diff --git a/go.mod b/go.mod index 8624e86064df..f03c7d95fc44 100644 --- a/go.mod +++ b/go.mod @@ -69,5 +69,3 @@ replace k8s.io/client-go => k8s.io/client-go v0.0.0-20190620085101-78d2af792bab replace github.com/Azure/azure-sdk-for-go => github.com/Azure/azure-sdk-for-go v36.2.0+incompatible replace github.com/Azure/go-autorest => github.com/Azure/go-autorest v13.3.0+incompatible - -replace github.com/cortexproject/cortex => github.com/grafana/cortex v0.0.0-20200408151555-2e824a9e4de8 diff --git a/vendor/github.com/cortexproject/cortex/pkg/chunk/chunk_store.go b/vendor/github.com/cortexproject/cortex/pkg/chunk/chunk_store.go index 02cc7da879ba..4406b0db5d54 100644 --- a/vendor/github.com/cortexproject/cortex/pkg/chunk/chunk_store.go +++ b/vendor/github.com/cortexproject/cortex/pkg/chunk/chunk_store.go @@ -119,7 +119,6 @@ func newStore(cfg StoreConfig, schema StoreSchema, index IndexClient, chunks Cli func (c *store) Stop() { c.storage.Stop() c.Fetcher.Stop() - c.index.Stop() } // Put implements ChunkStore diff --git a/vendor/github.com/cortexproject/cortex/pkg/chunk/local/fs_object_client.go b/vendor/github.com/cortexproject/cortex/pkg/chunk/local/fs_object_client.go index 40b081c78052..a10bd297648a 100644 --- a/vendor/github.com/cortexproject/cortex/pkg/chunk/local/fs_object_client.go +++ b/vendor/github.com/cortexproject/cortex/pkg/chunk/local/fs_object_client.go @@ -11,7 +11,6 @@ import ( "time" "github.com/go-kit/kit/log/level" - "github.com/thanos-io/thanos/pkg/runutil" "github.com/cortexproject/cortex/pkg/chunk" "github.com/cortexproject/cortex/pkg/chunk/util" @@ -75,19 +74,10 @@ func (f *FSObjectClient) PutObject(ctx context.Context, objectKey string, object return err } - defer runutil.CloseWithLogOnErr(pkgUtil.Logger, fl, "fullPath: %s", fullPath) + defer fl.Close() _, err = io.Copy(fl, object) - if err != nil { - return err - } - - err = fl.Sync() - if err != nil { - return err - } - - return fl.Close() + return err } // List only objects from the store non-recursively diff --git a/vendor/github.com/cortexproject/cortex/pkg/chunk/storage/caching_index_client.go b/vendor/github.com/cortexproject/cortex/pkg/chunk/storage/caching_index_client.go index 7af46eea056d..f1863653b792 100644 --- a/vendor/github.com/cortexproject/cortex/pkg/chunk/storage/caching_index_client.go +++ b/vendor/github.com/cortexproject/cortex/pkg/chunk/storage/caching_index_client.go @@ -63,7 +63,6 @@ func newCachingIndexClient(client chunk.IndexClient, c cache.Cache, validity tim func (s *cachingIndexClient) Stop() { s.cache.Stop() - s.IndexClient.Stop() } func (s *cachingIndexClient) QueryPages(ctx context.Context, queries []chunk.IndexQuery, callback func(chunk.IndexQuery, chunk.ReadBatch) (shouldContinue bool)) error { diff --git a/vendor/github.com/thanos-io/thanos/LICENSE b/vendor/github.com/thanos-io/thanos/LICENSE deleted file mode 100644 index 8dada3edaf50..000000000000 --- a/vendor/github.com/thanos-io/thanos/LICENSE +++ /dev/null @@ -1,201 +0,0 @@ - Apache License - Version 2.0, January 2004 - http://www.apache.org/licenses/ - - TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION - - 1. Definitions. - - "License" shall mean the terms and conditions for use, reproduction, - and distribution as defined by Sections 1 through 9 of this document. - - "Licensor" shall mean the copyright owner or entity authorized by - the copyright owner that is granting the License. - - "Legal Entity" shall mean the union of the acting entity and all - other entities that control, are controlled by, or are under common - control with that entity. For the purposes of this definition, - "control" means (i) the power, direct or indirect, to cause the - direction or management of such entity, whether by contract or - otherwise, or (ii) ownership of fifty percent (50%) or more of the - outstanding shares, or (iii) beneficial ownership of such entity. - - "You" (or "Your") shall mean an individual or Legal Entity - exercising permissions granted by this License. - - "Source" form shall mean the preferred form for making modifications, - including but not limited to software source code, documentation - source, and configuration files. - - "Object" form shall mean any form resulting from mechanical - transformation or translation of a Source form, including but - not limited to compiled object code, generated documentation, - and conversions to other media types. - - "Work" shall mean the work of authorship, whether in Source or - Object form, made available under the License, as indicated by a - copyright notice that is included in or attached to the work - (an example is provided in the Appendix below). - - "Derivative Works" shall mean any work, whether in Source or Object - form, that is based on (or derived from) the Work and for which the - editorial revisions, annotations, elaborations, or other modifications - represent, as a whole, an original work of authorship. For the purposes - of this License, Derivative Works shall not include works that remain - separable from, or merely link (or bind by name) to the interfaces of, - the Work and Derivative Works thereof. - - "Contribution" shall mean any work of authorship, including - the original version of the Work and any modifications or additions - to that Work or Derivative Works thereof, that is intentionally - submitted to Licensor for inclusion in the Work by the copyright owner - or by an individual or Legal Entity authorized to submit on behalf of - the copyright owner. For the purposes of this definition, "submitted" - means any form of electronic, verbal, or written communication sent - to the Licensor or its representatives, including but not limited to - communication on electronic mailing lists, source code control systems, - and issue tracking systems that are managed by, or on behalf of, the - Licensor for the purpose of discussing and improving the Work, but - excluding communication that is conspicuously marked or otherwise - designated in writing by the copyright owner as "Not a Contribution." - - "Contributor" shall mean Licensor and any individual or Legal Entity - on behalf of whom a Contribution has been received by Licensor and - subsequently incorporated within the Work. - - 2. Grant of Copyright License. Subject to the terms and conditions of - this License, each Contributor hereby grants to You a perpetual, - worldwide, non-exclusive, no-charge, royalty-free, irrevocable - copyright license to reproduce, prepare Derivative Works of, - publicly display, publicly perform, sublicense, and distribute the - Work and such Derivative Works in Source or Object form. - - 3. Grant of Patent License. Subject to the terms and conditions of - this License, each Contributor hereby grants to You a perpetual, - worldwide, non-exclusive, no-charge, royalty-free, irrevocable - (except as stated in this section) patent license to make, have made, - use, offer to sell, sell, import, and otherwise transfer the Work, - where such license applies only to those patent claims licensable - by such Contributor that are necessarily infringed by their - Contribution(s) alone or by combination of their Contribution(s) - with the Work to which such Contribution(s) was submitted. If You - institute patent litigation against any entity (including a - cross-claim or counterclaim in a lawsuit) alleging that the Work - or a Contribution incorporated within the Work constitutes direct - or contributory patent infringement, then any patent licenses - granted to You under this License for that Work shall terminate - as of the date such litigation is filed. - - 4. Redistribution. You may reproduce and distribute copies of the - Work or Derivative Works thereof in any medium, with or without - modifications, and in Source or Object form, provided that You - meet the following conditions: - - (a) You must give any other recipients of the Work or - Derivative Works a copy of this License; and - - (b) You must cause any modified files to carry prominent notices - stating that You changed the files; and - - (c) You must retain, in the Source form of any Derivative Works - that You distribute, all copyright, patent, trademark, and - attribution notices from the Source form of the Work, - excluding those notices that do not pertain to any part of - the Derivative Works; and - - (d) If the Work includes a "NOTICE" text file as part of its - distribution, then any Derivative Works that You distribute must - include a readable copy of the attribution notices contained - within such NOTICE file, excluding those notices that do not - pertain to any part of the Derivative Works, in at least one - of the following places: within a NOTICE text file distributed - as part of the Derivative Works; within the Source form or - documentation, if provided along with the Derivative Works; or, - within a display generated by the Derivative Works, if and - wherever such third-party notices normally appear. The contents - of the NOTICE file are for informational purposes only and - do not modify the License. You may add Your own attribution - notices within Derivative Works that You distribute, alongside - or as an addendum to the NOTICE text from the Work, provided - that such additional attribution notices cannot be construed - as modifying the License. - - You may add Your own copyright statement to Your modifications and - may provide additional or different license terms and conditions - for use, reproduction, or distribution of Your modifications, or - for any such Derivative Works as a whole, provided Your use, - reproduction, and distribution of the Work otherwise complies with - the conditions stated in this License. - - 5. Submission of Contributions. Unless You explicitly state otherwise, - any Contribution intentionally submitted for inclusion in the Work - by You to the Licensor shall be under the terms and conditions of - this License, without any additional terms or conditions. - Notwithstanding the above, nothing herein shall supersede or modify - the terms of any separate license agreement you may have executed - with Licensor regarding such Contributions. - - 6. Trademarks. This License does not grant permission to use the trade - names, trademarks, service marks, or product names of the Licensor, - except as required for reasonable and customary use in describing the - origin of the Work and reproducing the content of the NOTICE file. - - 7. Disclaimer of Warranty. Unless required by applicable law or - agreed to in writing, Licensor provides the Work (and each - Contributor provides its Contributions) on an "AS IS" BASIS, - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or - implied, including, without limitation, any warranties or conditions - of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A - PARTICULAR PURPOSE. You are solely responsible for determining the - appropriateness of using or redistributing the Work and assume any - risks associated with Your exercise of permissions under this License. - - 8. Limitation of Liability. In no event and under no legal theory, - whether in tort (including negligence), contract, or otherwise, - unless required by applicable law (such as deliberate and grossly - negligent acts) or agreed to in writing, shall any Contributor be - liable to You for damages, including any direct, indirect, special, - incidental, or consequential damages of any character arising as a - result of this License or out of the use or inability to use the - Work (including but not limited to damages for loss of goodwill, - work stoppage, computer failure or malfunction, or any and all - other commercial damages or losses), even if such Contributor - has been advised of the possibility of such damages. - - 9. Accepting Warranty or Additional Liability. While redistributing - the Work or Derivative Works thereof, You may choose to offer, - and charge a fee for, acceptance of support, warranty, indemnity, - or other liability obligations and/or rights consistent with this - License. However, in accepting such obligations, You may act only - on Your own behalf and on Your sole responsibility, not on behalf - of any other Contributor, and only if You agree to indemnify, - defend, and hold each Contributor harmless for any liability - incurred by, or claims asserted against, such Contributor by reason - of your accepting any such warranty or additional liability. - - END OF TERMS AND CONDITIONS - - APPENDIX: How to apply the Apache License to your work. - - To apply the Apache License to your work, attach the following - boilerplate notice, with the fields enclosed by brackets "{}" - replaced with your own identifying information. (Don't include - the brackets!) The text should be enclosed in the appropriate - comment syntax for the file format. We also recommend that a - file or class name and description of purpose be included on the - same "printed page" as the copyright notice for easier - identification within third-party archives. - - Copyright {yyyy} {name of copyright owner} - - Licensed under the Apache License, Version 2.0 (the "License"); - you may not use this file except in compliance with the License. - You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - - Unless required by applicable law or agreed to in writing, software - distributed under the License is distributed on an "AS IS" BASIS, - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - See the License for the specific language governing permissions and - limitations under the License. From 753f57c6feff9d63a1125a4c16e774ece1a866da Mon Sep 17 00:00:00 2001 From: Sandeep Sukhani Date: Sat, 18 Apr 2020 19:34:10 +0530 Subject: [PATCH 12/13] fix merge conflict --- go.sum | 2 - pkg/ingester/ingester.go | 4 + pkg/storage/store.go | 7 +- pkg/storage/stores/factory.go | 29 --- .../cortex/pkg/chunk/chunk_store.go | 1 + .../pkg/chunk/local/fs_object_client.go | 14 +- .../pkg/chunk/storage/caching_index_client.go | 1 + vendor/github.com/thanos-io/thanos/LICENSE | 201 ++++++++++++++++++ 8 files changed, 222 insertions(+), 37 deletions(-) delete mode 100644 pkg/storage/stores/factory.go create mode 100644 vendor/github.com/thanos-io/thanos/LICENSE diff --git a/go.sum b/go.sum index 44710cccc8ff..419cf1e48c19 100644 --- a/go.sum +++ b/go.sum @@ -716,8 +716,6 @@ github.com/samuel/go-zookeeper v0.0.0-20190810000440-0ceca61e4d75 h1:cA+Ubq9qEVI github.com/samuel/go-zookeeper v0.0.0-20190810000440-0ceca61e4d75/go.mod h1:gi+0XIa01GRL2eRQVjQkKGqKF3SF9vZR/HnPullcV2E= github.com/samuel/go-zookeeper v0.0.0-20190923202752-2cc03de413da h1:p3Vo3i64TCLY7gIfzeQaUJ+kppEO5WQG3cL8iE8tGHU= github.com/samuel/go-zookeeper v0.0.0-20190923202752-2cc03de413da/go.mod h1:gi+0XIa01GRL2eRQVjQkKGqKF3SF9vZR/HnPullcV2E= -github.com/sandeepsukhani/cortex v0.0.0-20200401102156-8377099831f7 h1:yoacVlbP2nrspG+JX4fnzPQVQJia0fYq283Kfb+59zE= -github.com/sandeepsukhani/cortex v0.0.0-20200401102156-8377099831f7/go.mod h1:dMuT8RuWexf371937IhTj7/Ha3P/+Aog3pddNtV6Jo0= github.com/santhosh-tekuri/jsonschema v1.2.4/go.mod h1:TEAUOeZSmIxTTuHatJzrvARHiuO9LYd+cIxzgEHCQI4= github.com/satori/go.uuid v0.0.0-20160603004225-b111a074d5ef/go.mod h1:dA0hQrYB0VpLJoorglMZABFdXlWrHn1NEOzdhQKdks0= github.com/satori/go.uuid v1.2.0/go.mod h1:dA0hQrYB0VpLJoorglMZABFdXlWrHn1NEOzdhQKdks0= diff --git a/pkg/ingester/ingester.go b/pkg/ingester/ingester.go index e035fea66787..cc15eadf2a3c 100644 --- a/pkg/ingester/ingester.go +++ b/pkg/ingester/ingester.go @@ -22,8 +22,12 @@ import ( "github.com/cortexproject/cortex/pkg/util/services" "github.com/grafana/loki/pkg/chunkenc" + "github.com/grafana/loki/pkg/helpers" "github.com/grafana/loki/pkg/ingester/client" + "github.com/grafana/loki/pkg/iter" "github.com/grafana/loki/pkg/logproto" + "github.com/grafana/loki/pkg/logql" + "github.com/grafana/loki/pkg/logql/stats" "github.com/grafana/loki/pkg/util/validation" ) diff --git a/pkg/storage/store.go b/pkg/storage/store.go index 8a0cca4dadb3..5402f0144965 100644 --- a/pkg/storage/store.go +++ b/pkg/storage/store.go @@ -17,7 +17,6 @@ import ( "github.com/grafana/loki/pkg/logproto" "github.com/grafana/loki/pkg/logql" "github.com/grafana/loki/pkg/logql/stats" - "github.com/grafana/loki/pkg/storage/stores" "github.com/grafana/loki/pkg/storage/stores/local" "github.com/grafana/loki/pkg/util" ) @@ -220,7 +219,7 @@ func filterChunksByTime(from, through model.Time, chunks []chunk.Chunk) []chunk. func registerCustomIndexClients(cfg Config, schemaCfg chunk.SchemaConfig) { boltdbShipperInstances := 0 - storage.RegisterIndexClient(local.BoltDBShipperType, func() (chunk.IndexClient, error) { + storage.RegisterIndexStore(local.BoltDBShipperType, func() (chunk.IndexClient, error) { // since we do not know which object client is being used for the period for which we are creating this index client, // we need to iterate through all the periodic configs to find the right one. // We maintain number of instances that we have already created in boltdbShipperInstances and then count the number of @@ -242,11 +241,11 @@ func registerCustomIndexClients(cfg Config, schemaCfg chunk.SchemaConfig) { } boltdbShipperInstances++ - objectClient, err := stores.NewObjectClient(objectStoreType, cfg.Config) + objectClient, err := storage.NewObjectClient(objectStoreType, cfg.Config) if err != nil { return nil, err } return local.NewBoltDBIndexClient(cortex_local.BoltDBConfig{Directory: cfg.BoltDBShipperConfig.ActiveIndexDirectory}, objectClient, cfg.BoltDBShipperConfig) - }) + }, nil) } diff --git a/pkg/storage/stores/factory.go b/pkg/storage/stores/factory.go deleted file mode 100644 index 59af4e6c598f..000000000000 --- a/pkg/storage/stores/factory.go +++ /dev/null @@ -1,29 +0,0 @@ -package stores - -import ( - "context" - "fmt" - - "github.com/cortexproject/cortex/pkg/chunk" - "github.com/cortexproject/cortex/pkg/chunk/aws" - "github.com/cortexproject/cortex/pkg/chunk/azure" - "github.com/cortexproject/cortex/pkg/chunk/gcp" - cortex_local "github.com/cortexproject/cortex/pkg/chunk/local" - "github.com/cortexproject/cortex/pkg/chunk/storage" -) - -// NewObjectClient makes a new ObjectClient of the desired type. -func NewObjectClient(storeType string, cfg storage.Config) (chunk.ObjectClient, error) { - switch storeType { - case "aws", "s3": - return aws.NewS3ObjectClient(cfg.AWSStorageConfig.S3Config) - case "gcs": - return gcp.NewGCSObjectClient(context.Background(), cfg.GCSConfig) - case "azure": - return azure.NewBlobStorage(&cfg.AzureStorageConfig) - case "filesystem": - return cortex_local.NewFSObjectClient(cfg.FSConfig) - default: - return nil, fmt.Errorf("unrecognized storage client %v, choose one of: aws, s3, gcp, azure, filesystem", storeType) - } -} diff --git a/vendor/github.com/cortexproject/cortex/pkg/chunk/chunk_store.go b/vendor/github.com/cortexproject/cortex/pkg/chunk/chunk_store.go index 4406b0db5d54..02cc7da879ba 100644 --- a/vendor/github.com/cortexproject/cortex/pkg/chunk/chunk_store.go +++ b/vendor/github.com/cortexproject/cortex/pkg/chunk/chunk_store.go @@ -119,6 +119,7 @@ func newStore(cfg StoreConfig, schema StoreSchema, index IndexClient, chunks Cli func (c *store) Stop() { c.storage.Stop() c.Fetcher.Stop() + c.index.Stop() } // Put implements ChunkStore diff --git a/vendor/github.com/cortexproject/cortex/pkg/chunk/local/fs_object_client.go b/vendor/github.com/cortexproject/cortex/pkg/chunk/local/fs_object_client.go index a10bd297648a..40b081c78052 100644 --- a/vendor/github.com/cortexproject/cortex/pkg/chunk/local/fs_object_client.go +++ b/vendor/github.com/cortexproject/cortex/pkg/chunk/local/fs_object_client.go @@ -11,6 +11,7 @@ import ( "time" "github.com/go-kit/kit/log/level" + "github.com/thanos-io/thanos/pkg/runutil" "github.com/cortexproject/cortex/pkg/chunk" "github.com/cortexproject/cortex/pkg/chunk/util" @@ -74,10 +75,19 @@ func (f *FSObjectClient) PutObject(ctx context.Context, objectKey string, object return err } - defer fl.Close() + defer runutil.CloseWithLogOnErr(pkgUtil.Logger, fl, "fullPath: %s", fullPath) _, err = io.Copy(fl, object) - return err + if err != nil { + return err + } + + err = fl.Sync() + if err != nil { + return err + } + + return fl.Close() } // List only objects from the store non-recursively diff --git a/vendor/github.com/cortexproject/cortex/pkg/chunk/storage/caching_index_client.go b/vendor/github.com/cortexproject/cortex/pkg/chunk/storage/caching_index_client.go index f1863653b792..7af46eea056d 100644 --- a/vendor/github.com/cortexproject/cortex/pkg/chunk/storage/caching_index_client.go +++ b/vendor/github.com/cortexproject/cortex/pkg/chunk/storage/caching_index_client.go @@ -63,6 +63,7 @@ func newCachingIndexClient(client chunk.IndexClient, c cache.Cache, validity tim func (s *cachingIndexClient) Stop() { s.cache.Stop() + s.IndexClient.Stop() } func (s *cachingIndexClient) QueryPages(ctx context.Context, queries []chunk.IndexQuery, callback func(chunk.IndexQuery, chunk.ReadBatch) (shouldContinue bool)) error { diff --git a/vendor/github.com/thanos-io/thanos/LICENSE b/vendor/github.com/thanos-io/thanos/LICENSE new file mode 100644 index 000000000000..8dada3edaf50 --- /dev/null +++ b/vendor/github.com/thanos-io/thanos/LICENSE @@ -0,0 +1,201 @@ + Apache License + Version 2.0, January 2004 + http://www.apache.org/licenses/ + + TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION + + 1. Definitions. + + "License" shall mean the terms and conditions for use, reproduction, + and distribution as defined by Sections 1 through 9 of this document. + + "Licensor" shall mean the copyright owner or entity authorized by + the copyright owner that is granting the License. + + "Legal Entity" shall mean the union of the acting entity and all + other entities that control, are controlled by, or are under common + control with that entity. For the purposes of this definition, + "control" means (i) the power, direct or indirect, to cause the + direction or management of such entity, whether by contract or + otherwise, or (ii) ownership of fifty percent (50%) or more of the + outstanding shares, or (iii) beneficial ownership of such entity. + + "You" (or "Your") shall mean an individual or Legal Entity + exercising permissions granted by this License. + + "Source" form shall mean the preferred form for making modifications, + including but not limited to software source code, documentation + source, and configuration files. + + "Object" form shall mean any form resulting from mechanical + transformation or translation of a Source form, including but + not limited to compiled object code, generated documentation, + and conversions to other media types. + + "Work" shall mean the work of authorship, whether in Source or + Object form, made available under the License, as indicated by a + copyright notice that is included in or attached to the work + (an example is provided in the Appendix below). + + "Derivative Works" shall mean any work, whether in Source or Object + form, that is based on (or derived from) the Work and for which the + editorial revisions, annotations, elaborations, or other modifications + represent, as a whole, an original work of authorship. For the purposes + of this License, Derivative Works shall not include works that remain + separable from, or merely link (or bind by name) to the interfaces of, + the Work and Derivative Works thereof. + + "Contribution" shall mean any work of authorship, including + the original version of the Work and any modifications or additions + to that Work or Derivative Works thereof, that is intentionally + submitted to Licensor for inclusion in the Work by the copyright owner + or by an individual or Legal Entity authorized to submit on behalf of + the copyright owner. For the purposes of this definition, "submitted" + means any form of electronic, verbal, or written communication sent + to the Licensor or its representatives, including but not limited to + communication on electronic mailing lists, source code control systems, + and issue tracking systems that are managed by, or on behalf of, the + Licensor for the purpose of discussing and improving the Work, but + excluding communication that is conspicuously marked or otherwise + designated in writing by the copyright owner as "Not a Contribution." + + "Contributor" shall mean Licensor and any individual or Legal Entity + on behalf of whom a Contribution has been received by Licensor and + subsequently incorporated within the Work. + + 2. Grant of Copyright License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + copyright license to reproduce, prepare Derivative Works of, + publicly display, publicly perform, sublicense, and distribute the + Work and such Derivative Works in Source or Object form. + + 3. Grant of Patent License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + (except as stated in this section) patent license to make, have made, + use, offer to sell, sell, import, and otherwise transfer the Work, + where such license applies only to those patent claims licensable + by such Contributor that are necessarily infringed by their + Contribution(s) alone or by combination of their Contribution(s) + with the Work to which such Contribution(s) was submitted. If You + institute patent litigation against any entity (including a + cross-claim or counterclaim in a lawsuit) alleging that the Work + or a Contribution incorporated within the Work constitutes direct + or contributory patent infringement, then any patent licenses + granted to You under this License for that Work shall terminate + as of the date such litigation is filed. + + 4. Redistribution. You may reproduce and distribute copies of the + Work or Derivative Works thereof in any medium, with or without + modifications, and in Source or Object form, provided that You + meet the following conditions: + + (a) You must give any other recipients of the Work or + Derivative Works a copy of this License; and + + (b) You must cause any modified files to carry prominent notices + stating that You changed the files; and + + (c) You must retain, in the Source form of any Derivative Works + that You distribute, all copyright, patent, trademark, and + attribution notices from the Source form of the Work, + excluding those notices that do not pertain to any part of + the Derivative Works; and + + (d) If the Work includes a "NOTICE" text file as part of its + distribution, then any Derivative Works that You distribute must + include a readable copy of the attribution notices contained + within such NOTICE file, excluding those notices that do not + pertain to any part of the Derivative Works, in at least one + of the following places: within a NOTICE text file distributed + as part of the Derivative Works; within the Source form or + documentation, if provided along with the Derivative Works; or, + within a display generated by the Derivative Works, if and + wherever such third-party notices normally appear. The contents + of the NOTICE file are for informational purposes only and + do not modify the License. You may add Your own attribution + notices within Derivative Works that You distribute, alongside + or as an addendum to the NOTICE text from the Work, provided + that such additional attribution notices cannot be construed + as modifying the License. + + You may add Your own copyright statement to Your modifications and + may provide additional or different license terms and conditions + for use, reproduction, or distribution of Your modifications, or + for any such Derivative Works as a whole, provided Your use, + reproduction, and distribution of the Work otherwise complies with + the conditions stated in this License. + + 5. Submission of Contributions. Unless You explicitly state otherwise, + any Contribution intentionally submitted for inclusion in the Work + by You to the Licensor shall be under the terms and conditions of + this License, without any additional terms or conditions. + Notwithstanding the above, nothing herein shall supersede or modify + the terms of any separate license agreement you may have executed + with Licensor regarding such Contributions. + + 6. Trademarks. This License does not grant permission to use the trade + names, trademarks, service marks, or product names of the Licensor, + except as required for reasonable and customary use in describing the + origin of the Work and reproducing the content of the NOTICE file. + + 7. Disclaimer of Warranty. Unless required by applicable law or + agreed to in writing, Licensor provides the Work (and each + Contributor provides its Contributions) on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or + implied, including, without limitation, any warranties or conditions + of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A + PARTICULAR PURPOSE. You are solely responsible for determining the + appropriateness of using or redistributing the Work and assume any + risks associated with Your exercise of permissions under this License. + + 8. Limitation of Liability. In no event and under no legal theory, + whether in tort (including negligence), contract, or otherwise, + unless required by applicable law (such as deliberate and grossly + negligent acts) or agreed to in writing, shall any Contributor be + liable to You for damages, including any direct, indirect, special, + incidental, or consequential damages of any character arising as a + result of this License or out of the use or inability to use the + Work (including but not limited to damages for loss of goodwill, + work stoppage, computer failure or malfunction, or any and all + other commercial damages or losses), even if such Contributor + has been advised of the possibility of such damages. + + 9. Accepting Warranty or Additional Liability. While redistributing + the Work or Derivative Works thereof, You may choose to offer, + and charge a fee for, acceptance of support, warranty, indemnity, + or other liability obligations and/or rights consistent with this + License. However, in accepting such obligations, You may act only + on Your own behalf and on Your sole responsibility, not on behalf + of any other Contributor, and only if You agree to indemnify, + defend, and hold each Contributor harmless for any liability + incurred by, or claims asserted against, such Contributor by reason + of your accepting any such warranty or additional liability. + + END OF TERMS AND CONDITIONS + + APPENDIX: How to apply the Apache License to your work. + + To apply the Apache License to your work, attach the following + boilerplate notice, with the fields enclosed by brackets "{}" + replaced with your own identifying information. (Don't include + the brackets!) The text should be enclosed in the appropriate + comment syntax for the file format. We also recommend that a + file or class name and description of purpose be included on the + same "printed page" as the copyright notice for easier + identification within third-party archives. + + Copyright {yyyy} {name of copyright owner} + + Licensed under the Apache License, Version 2.0 (the "License"); + you may not use this file except in compliance with the License. + You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + + Unless required by applicable law or agreed to in writing, software + distributed under the License is distributed on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + See the License for the specific language governing permissions and + limitations under the License. From cdbbec329b385cba560a3624653fbf982e5d4821 Mon Sep 17 00:00:00 2001 From: Sandeep Sukhani Date: Mon, 20 Apr 2020 15:40:48 +0530 Subject: [PATCH 13/13] add documentation for boltdb shipper and remove unwanted config --- docs/README.md | 1 + docs/operations/storage/boltdb-shipper.md | 86 +++++++++++++++++++++++ pkg/storage/stores/local/shipper.go | 2 - 3 files changed, 87 insertions(+), 2 deletions(-) create mode 100644 docs/operations/storage/boltdb-shipper.md diff --git a/docs/README.md b/docs/README.md index 790e2b3ad5c8..6f87f3b9fadc 100644 --- a/docs/README.md +++ b/docs/README.md @@ -45,6 +45,7 @@ simplifies the operation and significantly lowers the cost of Loki. 4. [Storage](operations/storage/README.md) 1. [Table Manager](operations/storage/table-manager.md) 2. [Retention](operations/storage/retention.md) + 3. [BoltDB Shipper](operations/storage/boltdb-shipper.md) 5. [Multi-tenancy](operations/multi-tenancy.md) 6. [Loki Canary](operations/loki-canary.md) 8. [HTTP API](api.md) diff --git a/docs/operations/storage/boltdb-shipper.md b/docs/operations/storage/boltdb-shipper.md new file mode 100644 index 000000000000..f6e70d7d217a --- /dev/null +++ b/docs/operations/storage/boltdb-shipper.md @@ -0,0 +1,86 @@ +# Loki with BoltDB Shipper + +:warning: BoltDB Shipper is still an experimental feature. It is not recommended to be used in production environments. + +BoltDB Shipper lets you run Loki without any dependency on NoSQL stores for storing index. +It locally stores the index in BoltDB files instead and keeps shipping those files to a shared object store i.e the same object store which is being used for storing chunks. +It also keeps syncing BoltDB files from shared object store to a configured local directory for getting index entries created by other services of same Loki cluster. +This helps run Loki with one less dependency and also saves costs in storage since object stores are likely to be much cheaper compared to cost of a hosted NoSQL store or running a self hosted instance of Cassandra. + +## Example Configuration + +Example configuration with GCS: + +```yaml +schema_config: + configs: + - from: 2018-04-15 + store: boltdb-shipper + object_store: gcs + schema: v11 + index: + prefix: loki_index_ + period: 168h + +storage_config: + gcs: + bucket_name: GCS_BUCKET_NAME + + boltdb_shipper_config: + active_index_directory: /loki/index + cache_location: /loki/boltdb-cache +``` + +This would run Loki with BoltDB Shipper storing BoltDB files locally at `/loki/index` and chunks at configured `GCS_BUCKET_NAME`. +It would also keep shipping BoltDB files periodically to same configured bucket. +It would also keep downloading BoltDB files from shared bucket uploaded by other ingesters to `/loki/boltdb-cache` folder locally. + +## Operational Details + +Loki can be configured to run as just a single vertically scaled instance or as a cluster of horizontally scaled single binary(running all Loki services) instances or in micro-services mode running just one of the services in each instance. +When it comes to reads and writes, Ingesters are the ones which writes the index and chunks to stores and Queriers are the ones which reads index and chunks from the store for serving requests. + +Before we get into more details, it is important to understand how Loki manages index in stores. Loki shards index as per configured period which defaults to 7 days i.e when it comes to table based stores like Bigtable/Cassandra/DynamoDB there would be separate table per week containing index for that week. +In case of BoltDB files there is no concept of tables so it creates a BoltDB file per week. Files/Tables created per week are identified by a configured `prefix_` + ``. +Here `` in case of default config would be week number since epoch. +For example, if you have prefix set to `loki_index_` and a write requests comes in on 20th April 2020, it would be stored in table/file named `loki_index_2624` because it has been `2623` weeks since epoch and we are in `2624`th week. +Since sharding of index creates multiple files when using BoltDB, BoltDB Shipper would create a folder per week and add files for that week in that folder and names those files after ingesters which created them. + +To show how BoltDB files in shared object store would look like, let us consider 2 ingesters named `ingester-0` and `ingester-1` running in a Loki cluster and +they both having shipped files for week `2623` and `2624` with prefix `loki_index_`, here is how the files would look like: + +``` +└── index + ├── loki_index_2623 + │ ├── ingester-0 + │ └── ingester-1 + └── loki_index_2624 + ├── ingester-0 + └── ingester-1 +``` +*NOTE: We also add a timestamp to names of the files to randomize the names to avoid overwriting files when running Ingesters with same name and not have a persistent storage. Timestamps not shown here for simplification* + +Let us talk about more in depth about how both Ingesters and Queriers work when running them with BoltDB Shipper. + +### Ingesters + +Ingesters keep writing the index to BoltDB files in `active_index_directory` and BoltDB Shipper keeps looking for new and updated files in that directory every 15 Minutes to upload them to the shared object store. +When running Loki in clustered mode there could be multiple ingesters serving write requests hence each of them generating BoltDB files locally. + +*NOTE: To avoid any loss of index when Ingester crashes it is recommended to run Ingesters as statefulset(when using k8s) with a persistent storage for storing index files.* + +Another important detail to note is when chunks are flushed they are available for reads in object store instantly while index is not since we only upload them every 15 Minutes with BoltDB shipper. +To avoid missing logs from queries which happen to be indexed in BoltDB files which are not shipped yet, while serving queries for in-memory logs, Ingesters would also do a store query for `now()` - (`max_chunk_age` + `30 Min`) to ``. + +### Queriers + +Queriers lazily loads BoltDB files from shared object store to configured `cache_location`. +When a querier receives a read request, query range from request is resolved to period numbers and all the files for those period numbers are downloaded to `cache_location` if not already. +Once we have downloaded files for a period we keep looking for updates in shared object store and download them every 15 Minutes by default. +Frequency for checking updates can be configured with `resync_interval` config. + +To avoid keeping downloaded index files forever there is a ttl for them which defaults to 24 hours, which means if index files for a period are not used for 24 hours they would be removed from cache location. +ttl can be configured using `cache_ttl` config. + + + diff --git a/pkg/storage/stores/local/shipper.go b/pkg/storage/stores/local/shipper.go index bddcc93ab5dd..59f72ef1f64a 100644 --- a/pkg/storage/stores/local/shipper.go +++ b/pkg/storage/stores/local/shipper.go @@ -43,7 +43,6 @@ type BoltDBGetter interface { type ShipperConfig struct { ActiveIndexDirectory string `yaml:"active_index_directory"` - SharedStoreType string `yaml:"shared_store_type"` CacheLocation string `yaml:"cache_location"` CacheTTL time.Duration `yaml:"cache_ttl"` ResyncInterval time.Duration `yaml:"resync_interval"` @@ -54,7 +53,6 @@ type ShipperConfig struct { // RegisterFlags registers flags. func (cfg *ShipperConfig) RegisterFlags(f *flag.FlagSet) { f.StringVar(&cfg.ActiveIndexDirectory, "boltdb.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, "boltdb.shipper.shared-store", "", "Shared store for keeping boltdb files. Supported types: gcs, s3, azure, filesystem") f.StringVar(&cfg.CacheLocation, "boltdb.shipper.cache-location", "", "Cache location for restoring boltDB files for queries") f.DurationVar(&cfg.CacheTTL, "boltdb.shipper.cache-ttl", 24*time.Hour, "TTL for boltDB files restored in cache for queries") f.DurationVar(&cfg.ResyncInterval, "boltdb.shipper.resync-interval", 5*time.Minute, "Resync downloaded files with the storage")