Like Prometheus, but for logs.
You can not select more than 25 topics Topics must start with a letter or number, can include dashes ('-') and can be up to 35 characters long.
loki/pkg/storage/factory.go

336 lines
15 KiB

package storage
import (
"context"
"flag"
"fmt"
"strings"
"time"
"github.com/go-kit/log/level"
"github.com/pkg/errors"
"github.com/prometheus/client_golang/prometheus"
"github.com/grafana/loki/pkg/storage/chunk/cache"
"github.com/grafana/loki/pkg/storage/chunk/client"
"github.com/grafana/loki/pkg/storage/chunk/client/aws"
"github.com/grafana/loki/pkg/storage/chunk/client/azure"
Feat: add Baidu Cloud BOS as storage backends for Loki #4788 (#5848) * feat: add baidu bce bos storage support Signed-off-by: arcosx <arcosx@outlook.com> * add baidu bce bos as chunk storage backend Signed-off-by: arcosx <arcosx@outlook.com> * fix: some doc error && rewrite bad code Signed-off-by: arcosx <arcosx@outlook.com> * fix: add the `BceServiceError` source link Signed-off-by: arcosx <arcosx@outlook.com> * Update CHANGELOG.md Co-authored-by: Karen Miller <84039272+KMiller-Grafana@users.noreply.github.com> * Update docs/sources/configuration/_index.md Co-authored-by: Karen Miller <84039272+KMiller-Grafana@users.noreply.github.com> Signed-off-by: arcosx <arcosx@outlook.com> * Update docs/sources/configuration/_index.md Co-authored-by: Karen Miller <84039272+KMiller-Grafana@users.noreply.github.com> * Update docs/sources/configuration/_index.md Co-authored-by: Karen Miller <84039272+KMiller-Grafana@users.noreply.github.com> * Update docs/sources/configuration/_index.md Co-authored-by: Karen Miller <84039272+KMiller-Grafana@users.noreply.github.com> * Update docs/sources/configuration/_index.md Co-authored-by: Karen Miller <84039272+KMiller-Grafana@users.noreply.github.com> * Update pkg/storage/chunk/client/baidubce/bos_storage_client.go Co-authored-by: Karen Miller <84039272+KMiller-Grafana@users.noreply.github.com> * Update pkg/storage/chunk/client/baidubce/bos_storage_client.go Co-authored-by: Karen Miller <84039272+KMiller-Grafana@users.noreply.github.com> * Update pkg/storage/chunk/client/baidubce/bos_storage_client.go Co-authored-by: Karen Miller <84039272+KMiller-Grafana@users.noreply.github.com> Co-authored-by: Karen Miller <84039272+KMiller-Grafana@users.noreply.github.com>
3 years ago
"github.com/grafana/loki/pkg/storage/chunk/client/baidubce"
"github.com/grafana/loki/pkg/storage/chunk/client/cassandra"
"github.com/grafana/loki/pkg/storage/chunk/client/gcp"
"github.com/grafana/loki/pkg/storage/chunk/client/grpc"
"github.com/grafana/loki/pkg/storage/chunk/client/hedging"
"github.com/grafana/loki/pkg/storage/chunk/client/local"
"github.com/grafana/loki/pkg/storage/chunk/client/openstack"
"github.com/grafana/loki/pkg/storage/chunk/client/testutils"
"github.com/grafana/loki/pkg/storage/config"
TSDB shipper + WAL (#6049) * begins speccing out TSDB Head * auto incrementing series ref + mempostings * mintime/maxtime methods * tsdb head IndexReader impl * head correctly populates ref lookup * tsdb head tests * adds prometheus license to tsdb head * linting * [WIP] speccing out tsdb head wal * fix length check and adds tsdb wal encoding tests * exposes wal structs & removes closed semantics * logs start time in the tsdb wal * wal interface + testing * exports walrecord + returns ref when appending * specs out head manager * tsdb head manager wal initialization * tsdb wal rotation * wals dont use node name, but tsdb files do * cleans up fn signature * multi tsdb idx now just wraps Index interfaces * no longer sorts indices when creating multi-idx * tenantHeads & HeadManger index impls * head mgr tests * bugfixes & head manager tests * tsdb dir selection now helper fns * period utility * pulls out more code to helpers, fixes some var races * head recovery is more generic * tsdb manager builds from wals * pulls more helpers out of headmanager * lockedIdx, Close() on idx, tsdbManager update * removes mmap from index reader implementation * tsdb file * adds tsdb shipper config and refactors initStore * removes unused tsdbManager code * implements stores.Index and stores.ChunkWriter for tsdb * chunk.Data now supports an Entries() method * moves walreader to new util/wal pkg to avoid circular dep + tsdb storage alignment * tsdb store * passes indexWriter to chunkWriter * build a tsdb per index bucket in according with shipper conventions * dont open tsdb files until necessary for indexshipper * tsdbManager Index impl * tsdb defaults + initStore fix for invalid looping * fixes UsingTSDB helper * disables deleteRequestStore when using TSDB * pass limits to tsdb store * always start headmanager for tsdb Signed-off-by: Owen Diehl <ow.diehl@gmail.com> * fixes copy bug Signed-off-by: Owen Diehl <ow.diehl@gmail.com> * more logging Signed-off-by: Owen Diehl <ow.diehl@gmail.com> * fixes duplicate tenant label bug Signed-off-by: Owen Diehl <ow.diehl@gmail.com> * debug logs, uses label builder, removes __name__=logs for tsdb Signed-off-by: Owen Diehl <ow.diehl@gmail.com> * tsdb fixes labels at earlier pt Signed-off-by: Owen Diehl <ow.diehl@gmail.com> * account for setting tenant label in head manager test * changing tsdb dir names * identifier interface, builder to tsdb pkg * tsdb version path prefix * fixes buildfromwals identifier * fixes tsdb shipper paths * split buckets once per user set * refactors combining single and multi tenant tsdb indices on shipper reads * indexshipper ignores old gzip logic * method name refactor * remove unused record type * removes v1 prefix in tsdb paths and refactores indices method * ignores double optimization in tsdb looking for multitenant idx, shipper handles this * removes 5-ln requirement on shipper tablename regexp * groups identifiers, begins removing multitenant prefix in shipped files * passses open fn to indexshipper * exposes RealByteSlice * TSDBFile no longer needs a file descriptor, parses gzip extensions * method signature fixing * stop masquerading as compressed indices post-download in indexshipper * variable bucket regexp * removes accidental configs committed * label matcher handling for multitenancy and metricname in tsdb * explicitly require fingerprint when creating tsdb index * only add tenant label when creating multitenant tsdb write fingerprints without synthetic tenant label strip out tenant labels from queries * linting + unused removal * more linting :( * goimports * removes uploadername from indexshipper * maxuint32 for arm32 builds * tsdb chunk filterer support * always set ingester name when using object storage index Co-authored-by: Sandeep Sukhani <sandeep.d.sukhani@gmail.com>
3 years ago
"github.com/grafana/loki/pkg/storage/stores/indexshipper"
"github.com/grafana/loki/pkg/storage/stores/indexshipper/downloads"
"github.com/grafana/loki/pkg/storage/stores/indexshipper/gatewayclient"
"github.com/grafana/loki/pkg/storage/stores/series/index"
"github.com/grafana/loki/pkg/storage/stores/shipper"
util_log "github.com/grafana/loki/pkg/util/log"
)
// BoltDB Shipper is supposed to be run as a singleton.
// This could also be done in NewBoltDBIndexClientWithShipper factory method but we are doing it here because that method is used
// in tests for creating multiple instances of it at a time.
var boltDBIndexClientWithShipper index.Client
// StoreLimits helps get Limits specific to Queries for Stores
type StoreLimits interface {
downloads.Limits
CardinalityLimit(userID string) int
MaxChunksPerQueryFromStore(userID string) int
MaxQueryLength(userID string) time.Duration
}
// Config chooses which storage client to use.
type Config struct {
Feat: add Baidu Cloud BOS as storage backends for Loki #4788 (#5848) * feat: add baidu bce bos storage support Signed-off-by: arcosx <arcosx@outlook.com> * add baidu bce bos as chunk storage backend Signed-off-by: arcosx <arcosx@outlook.com> * fix: some doc error && rewrite bad code Signed-off-by: arcosx <arcosx@outlook.com> * fix: add the `BceServiceError` source link Signed-off-by: arcosx <arcosx@outlook.com> * Update CHANGELOG.md Co-authored-by: Karen Miller <84039272+KMiller-Grafana@users.noreply.github.com> * Update docs/sources/configuration/_index.md Co-authored-by: Karen Miller <84039272+KMiller-Grafana@users.noreply.github.com> Signed-off-by: arcosx <arcosx@outlook.com> * Update docs/sources/configuration/_index.md Co-authored-by: Karen Miller <84039272+KMiller-Grafana@users.noreply.github.com> * Update docs/sources/configuration/_index.md Co-authored-by: Karen Miller <84039272+KMiller-Grafana@users.noreply.github.com> * Update docs/sources/configuration/_index.md Co-authored-by: Karen Miller <84039272+KMiller-Grafana@users.noreply.github.com> * Update docs/sources/configuration/_index.md Co-authored-by: Karen Miller <84039272+KMiller-Grafana@users.noreply.github.com> * Update pkg/storage/chunk/client/baidubce/bos_storage_client.go Co-authored-by: Karen Miller <84039272+KMiller-Grafana@users.noreply.github.com> * Update pkg/storage/chunk/client/baidubce/bos_storage_client.go Co-authored-by: Karen Miller <84039272+KMiller-Grafana@users.noreply.github.com> * Update pkg/storage/chunk/client/baidubce/bos_storage_client.go Co-authored-by: Karen Miller <84039272+KMiller-Grafana@users.noreply.github.com> Co-authored-by: Karen Miller <84039272+KMiller-Grafana@users.noreply.github.com>
3 years ago
AWSStorageConfig aws.StorageConfig `yaml:"aws"`
AzureStorageConfig azure.BlobStorageConfig `yaml:"azure"`
BOSStorageConfig baidubce.BOSStorageConfig `yaml:"bos"`
GCPStorageConfig gcp.Config `yaml:"bigtable"`
GCSConfig gcp.GCSConfig `yaml:"gcs"`
CassandraStorageConfig cassandra.Config `yaml:"cassandra"`
BoltDBConfig local.BoltDBConfig `yaml:"boltdb"`
FSConfig local.FSConfig `yaml:"filesystem"`
Swift openstack.SwiftConfig `yaml:"swift"`
GrpcConfig grpc.Config `yaml:"grpc_store"`
Hedging hedging.Config `yaml:"hedging"`
IndexCacheValidity time.Duration `yaml:"index_cache_validity"`
IndexQueriesCacheConfig cache.Config `yaml:"index_queries_cache_config"`
DisableBroadIndexQueries bool `yaml:"disable_broad_index_queries"`
MaxParallelGetChunk int `yaml:"max_parallel_get_chunk"`
TSDB shipper + WAL (#6049) * begins speccing out TSDB Head * auto incrementing series ref + mempostings * mintime/maxtime methods * tsdb head IndexReader impl * head correctly populates ref lookup * tsdb head tests * adds prometheus license to tsdb head * linting * [WIP] speccing out tsdb head wal * fix length check and adds tsdb wal encoding tests * exposes wal structs & removes closed semantics * logs start time in the tsdb wal * wal interface + testing * exports walrecord + returns ref when appending * specs out head manager * tsdb head manager wal initialization * tsdb wal rotation * wals dont use node name, but tsdb files do * cleans up fn signature * multi tsdb idx now just wraps Index interfaces * no longer sorts indices when creating multi-idx * tenantHeads & HeadManger index impls * head mgr tests * bugfixes & head manager tests * tsdb dir selection now helper fns * period utility * pulls out more code to helpers, fixes some var races * head recovery is more generic * tsdb manager builds from wals * pulls more helpers out of headmanager * lockedIdx, Close() on idx, tsdbManager update * removes mmap from index reader implementation * tsdb file * adds tsdb shipper config and refactors initStore * removes unused tsdbManager code * implements stores.Index and stores.ChunkWriter for tsdb * chunk.Data now supports an Entries() method * moves walreader to new util/wal pkg to avoid circular dep + tsdb storage alignment * tsdb store * passes indexWriter to chunkWriter * build a tsdb per index bucket in according with shipper conventions * dont open tsdb files until necessary for indexshipper * tsdbManager Index impl * tsdb defaults + initStore fix for invalid looping * fixes UsingTSDB helper * disables deleteRequestStore when using TSDB * pass limits to tsdb store * always start headmanager for tsdb Signed-off-by: Owen Diehl <ow.diehl@gmail.com> * fixes copy bug Signed-off-by: Owen Diehl <ow.diehl@gmail.com> * more logging Signed-off-by: Owen Diehl <ow.diehl@gmail.com> * fixes duplicate tenant label bug Signed-off-by: Owen Diehl <ow.diehl@gmail.com> * debug logs, uses label builder, removes __name__=logs for tsdb Signed-off-by: Owen Diehl <ow.diehl@gmail.com> * tsdb fixes labels at earlier pt Signed-off-by: Owen Diehl <ow.diehl@gmail.com> * account for setting tenant label in head manager test * changing tsdb dir names * identifier interface, builder to tsdb pkg * tsdb version path prefix * fixes buildfromwals identifier * fixes tsdb shipper paths * split buckets once per user set * refactors combining single and multi tenant tsdb indices on shipper reads * indexshipper ignores old gzip logic * method name refactor * remove unused record type * removes v1 prefix in tsdb paths and refactores indices method * ignores double optimization in tsdb looking for multitenant idx, shipper handles this * removes 5-ln requirement on shipper tablename regexp * groups identifiers, begins removing multitenant prefix in shipped files * passses open fn to indexshipper * exposes RealByteSlice * TSDBFile no longer needs a file descriptor, parses gzip extensions * method signature fixing * stop masquerading as compressed indices post-download in indexshipper * variable bucket regexp * removes accidental configs committed * label matcher handling for multitenancy and metricname in tsdb * explicitly require fingerprint when creating tsdb index * only add tenant label when creating multitenant tsdb write fingerprints without synthetic tenant label strip out tenant labels from queries * linting + unused removal * more linting :( * goimports * removes uploadername from indexshipper * maxuint32 for arm32 builds * tsdb chunk filterer support * always set ingester name when using object storage index Co-authored-by: Sandeep Sukhani <sandeep.d.sukhani@gmail.com>
3 years ago
MaxChunkBatchSize int `yaml:"max_chunk_batch_size"`
BoltDBShipperConfig shipper.Config `yaml:"boltdb_shipper"`
TSDBShipperConfig indexshipper.Config `yaml:"tsdb_shipper"`
// Config for using AsyncStore when using async index stores like `boltdb-shipper`.
// It is required for getting chunk ids of recently flushed chunks from the ingesters.
EnableAsyncStore bool `yaml:"-"`
AsyncStoreConfig AsyncStoreCfg `yaml:"-"`
}
// RegisterFlags adds the flags required to configure this flag set.
func (cfg *Config) RegisterFlags(f *flag.FlagSet) {
cfg.AWSStorageConfig.RegisterFlags(f)
cfg.AzureStorageConfig.RegisterFlags(f)
Feat: add Baidu Cloud BOS as storage backends for Loki #4788 (#5848) * feat: add baidu bce bos storage support Signed-off-by: arcosx <arcosx@outlook.com> * add baidu bce bos as chunk storage backend Signed-off-by: arcosx <arcosx@outlook.com> * fix: some doc error && rewrite bad code Signed-off-by: arcosx <arcosx@outlook.com> * fix: add the `BceServiceError` source link Signed-off-by: arcosx <arcosx@outlook.com> * Update CHANGELOG.md Co-authored-by: Karen Miller <84039272+KMiller-Grafana@users.noreply.github.com> * Update docs/sources/configuration/_index.md Co-authored-by: Karen Miller <84039272+KMiller-Grafana@users.noreply.github.com> Signed-off-by: arcosx <arcosx@outlook.com> * Update docs/sources/configuration/_index.md Co-authored-by: Karen Miller <84039272+KMiller-Grafana@users.noreply.github.com> * Update docs/sources/configuration/_index.md Co-authored-by: Karen Miller <84039272+KMiller-Grafana@users.noreply.github.com> * Update docs/sources/configuration/_index.md Co-authored-by: Karen Miller <84039272+KMiller-Grafana@users.noreply.github.com> * Update docs/sources/configuration/_index.md Co-authored-by: Karen Miller <84039272+KMiller-Grafana@users.noreply.github.com> * Update pkg/storage/chunk/client/baidubce/bos_storage_client.go Co-authored-by: Karen Miller <84039272+KMiller-Grafana@users.noreply.github.com> * Update pkg/storage/chunk/client/baidubce/bos_storage_client.go Co-authored-by: Karen Miller <84039272+KMiller-Grafana@users.noreply.github.com> * Update pkg/storage/chunk/client/baidubce/bos_storage_client.go Co-authored-by: Karen Miller <84039272+KMiller-Grafana@users.noreply.github.com> Co-authored-by: Karen Miller <84039272+KMiller-Grafana@users.noreply.github.com>
3 years ago
cfg.BOSStorageConfig.RegisterFlags(f)
cfg.GCPStorageConfig.RegisterFlags(f)
cfg.GCSConfig.RegisterFlags(f)
cfg.CassandraStorageConfig.RegisterFlags(f)
cfg.BoltDBConfig.RegisterFlags(f)
cfg.FSConfig.RegisterFlags(f)
cfg.Swift.RegisterFlags(f)
cfg.GrpcConfig.RegisterFlags(f)
cfg.Hedging.RegisterFlagsWithPrefix("store.", f)
cfg.IndexQueriesCacheConfig.RegisterFlagsWithPrefix("store.index-cache-read.", "Cache config for index entry reading.", f)
f.DurationVar(&cfg.IndexCacheValidity, "store.index-cache-validity", 5*time.Minute, "Cache validity for active index entries. Should be no higher than -ingester.max-chunk-idle.")
f.BoolVar(&cfg.DisableBroadIndexQueries, "store.disable-broad-index-queries", false, "Disable broad index queries which results in reduced cache usage and faster query performance at the expense of somewhat higher QPS on the index store.")
f.IntVar(&cfg.MaxParallelGetChunk, "store.max-parallel-get-chunk", 150, "Maximum number of parallel chunk reads.")
cfg.BoltDBShipperConfig.RegisterFlags(f)
f.IntVar(&cfg.MaxChunkBatchSize, "store.max-chunk-batch-size", 50, "The maximum number of chunks to fetch per batch.")
TSDB shipper + WAL (#6049) * begins speccing out TSDB Head * auto incrementing series ref + mempostings * mintime/maxtime methods * tsdb head IndexReader impl * head correctly populates ref lookup * tsdb head tests * adds prometheus license to tsdb head * linting * [WIP] speccing out tsdb head wal * fix length check and adds tsdb wal encoding tests * exposes wal structs & removes closed semantics * logs start time in the tsdb wal * wal interface + testing * exports walrecord + returns ref when appending * specs out head manager * tsdb head manager wal initialization * tsdb wal rotation * wals dont use node name, but tsdb files do * cleans up fn signature * multi tsdb idx now just wraps Index interfaces * no longer sorts indices when creating multi-idx * tenantHeads & HeadManger index impls * head mgr tests * bugfixes & head manager tests * tsdb dir selection now helper fns * period utility * pulls out more code to helpers, fixes some var races * head recovery is more generic * tsdb manager builds from wals * pulls more helpers out of headmanager * lockedIdx, Close() on idx, tsdbManager update * removes mmap from index reader implementation * tsdb file * adds tsdb shipper config and refactors initStore * removes unused tsdbManager code * implements stores.Index and stores.ChunkWriter for tsdb * chunk.Data now supports an Entries() method * moves walreader to new util/wal pkg to avoid circular dep + tsdb storage alignment * tsdb store * passes indexWriter to chunkWriter * build a tsdb per index bucket in according with shipper conventions * dont open tsdb files until necessary for indexshipper * tsdbManager Index impl * tsdb defaults + initStore fix for invalid looping * fixes UsingTSDB helper * disables deleteRequestStore when using TSDB * pass limits to tsdb store * always start headmanager for tsdb Signed-off-by: Owen Diehl <ow.diehl@gmail.com> * fixes copy bug Signed-off-by: Owen Diehl <ow.diehl@gmail.com> * more logging Signed-off-by: Owen Diehl <ow.diehl@gmail.com> * fixes duplicate tenant label bug Signed-off-by: Owen Diehl <ow.diehl@gmail.com> * debug logs, uses label builder, removes __name__=logs for tsdb Signed-off-by: Owen Diehl <ow.diehl@gmail.com> * tsdb fixes labels at earlier pt Signed-off-by: Owen Diehl <ow.diehl@gmail.com> * account for setting tenant label in head manager test * changing tsdb dir names * identifier interface, builder to tsdb pkg * tsdb version path prefix * fixes buildfromwals identifier * fixes tsdb shipper paths * split buckets once per user set * refactors combining single and multi tenant tsdb indices on shipper reads * indexshipper ignores old gzip logic * method name refactor * remove unused record type * removes v1 prefix in tsdb paths and refactores indices method * ignores double optimization in tsdb looking for multitenant idx, shipper handles this * removes 5-ln requirement on shipper tablename regexp * groups identifiers, begins removing multitenant prefix in shipped files * passses open fn to indexshipper * exposes RealByteSlice * TSDBFile no longer needs a file descriptor, parses gzip extensions * method signature fixing * stop masquerading as compressed indices post-download in indexshipper * variable bucket regexp * removes accidental configs committed * label matcher handling for multitenancy and metricname in tsdb * explicitly require fingerprint when creating tsdb index * only add tenant label when creating multitenant tsdb write fingerprints without synthetic tenant label strip out tenant labels from queries * linting + unused removal * more linting :( * goimports * removes uploadername from indexshipper * maxuint32 for arm32 builds * tsdb chunk filterer support * always set ingester name when using object storage index Co-authored-by: Sandeep Sukhani <sandeep.d.sukhani@gmail.com>
3 years ago
cfg.TSDBShipperConfig.RegisterFlagsWithPrefix("tsdb.", f)
}
// Validate config and returns error on failure
func (cfg *Config) Validate() error {
if err := cfg.CassandraStorageConfig.Validate(); err != nil {
return errors.Wrap(err, "invalid Cassandra Storage config")
}
if err := cfg.GCPStorageConfig.Validate(util_log.Logger); err != nil {
return errors.Wrap(err, "invalid GCP Storage Storage config")
}
if err := cfg.Swift.Validate(); err != nil {
return errors.Wrap(err, "invalid Swift Storage config")
}
if err := cfg.IndexQueriesCacheConfig.Validate(); err != nil {
return errors.Wrap(err, "invalid Index Queries Cache config")
}
if err := cfg.AzureStorageConfig.Validate(); err != nil {
return errors.Wrap(err, "invalid Azure Storage config")
}
if err := cfg.AWSStorageConfig.Validate(); err != nil {
return errors.Wrap(err, "invalid AWS Storage config")
}
if err := cfg.BoltDBShipperConfig.Validate(); err != nil {
return errors.Wrap(err, "invalid boltdb-shipper config")
}
if err := cfg.TSDBShipperConfig.Validate(); err != nil {
return errors.Wrap(err, "invalid tsdb config")
}
return nil
}
// NewIndexClient makes a new index client of the desired type.
Loki: Modifies TableManager to use IndexGateway ring (#5972) * Add two metrics to the IndexGateway. - Add a new `query_readiness_duration_seconds` metric, that reports query readiness duration of a tablemanager/index gateway instance. We should use it later to report performance against the ring mode - Add a new `usersToBeQueryReadyForTotal` metric, that reports number of users involved in the query readiness operation. We should use it later to correlate number of users with the query readiness duration. * Remove `usersToBeQueryReadyForTotal`. - It will report all users always for now, so it isn't too helpful the way it is. * Rename metric help text to not mislead people. * Log queryReadiness duration. * Fix where log message and duration and triggered. * Add two metrics to the IndexGateway. - Add a new `query_readiness_duration_seconds` metric, that reports query readiness duration of a tablemanager/index gateway instance. We should use it later to report performance against the ring mode - Add a new `usersToBeQueryReadyForTotal` metric, that reports number of users involved in the query readiness operation. We should use it later to correlate number of users with the query readiness duration. * Remove `usersToBeQueryReadyForTotal`. - It will report all users always for now, so it isn't too helpful the way it is. * Rename metric help text to not mislead people. * Log queryReadiness duration. * Fix where log message and duration and triggered. * Use boundaries to skip users in TableManager. - Separate the assigning of indexClient in the IndexGateway to allow initializing the shipper with the IndexGateway ring - Add new TenantBoundariesClient entity, that answers if a given tenantID should be ignored or not - Use the TenantBoundariesClient implemented by the IndexGateway in the downloads TableManager * Add IndexGateway configuration docs. * Add tests for pkg/util/ring TokenFor() and IsAssignedKey() Signed-off-by: JordanRushing <rushing.jordan@gmail.com> * Small whitespace fix in pkg/util/ring_test.go Signed-off-by: JordanRushing <rushing.jordan@gmail.com> * Apply suggestions from code review Rewrite QueryIndex error phrase. Co-authored-by: JordanRushing <rushing.jordan@gmail.com> * Join users list in a single string. - This is necessary since go-kit doesn't support array type. * Tweak queryReadiness log messages. - As suggested by Ed on https://github.com/grafana/loki/pull/5972#discussion_r859734129 and https://github.com/grafana/loki/pull/5972#discussion_r859736072 * Implement new RingManager. - Adds a new entity to indexgateway named RingManager, responsible for managing the ring and the lifecycler used by the indexgateway. The ringManager is guaranteed to be initiatiated before the Shipper and before the IndexGateway. - Remove the readiness logic from the IndexGateway. - Pass the RingManager as the TenantBoundaries implementation of the Shipper * Return non-empty ringmanager for all modes. * Fix lint regarding error not being check. * Fix lint due to wrong import order. * Implement support for client and server mode for the ring manager. * Fix ring manager services registration. * Add option to configure whether or not to log gateway requests. * Check if IndexGateway is enabled instead of active. * Tune RingManager behavior. - Instantiate ring buffers inside IsAssignedKey instead of reusing same buffer to avoid issues with concurrency. - Remove unnecessary details from IndexGateway mode docs. - Return error when wrongly instantiating a RingManager when IndexGateway is in simple mode. * Rewrite `TenantInBoundaries` as a func instead of interface. * Fix lint. - Fix YAML tag - Remove else clause * Use distributor instead of querier in test. - Since the querier needs to wait for the index gateway ring, it isn't suitable for this test anymore. * Add ring mode guard clause. * Return code erased by mistake. * Rename TenantInBoundaries to IndexGatewayOwnsTenant. * Log len of users instead of all of them to avoid log msgs. * Add docstrings to new public functions. * Apply suggestions from code review Document that tenant filtering is only applied during query readiness. Co-authored-by: Sandeep Sukhani <sandeep.d.sukhani@gmail.com> * Modify IsAssignedKey to expect address directly instead of lifecycler. - Also removes HasDedicatedAddress, since it isn't necessary anymore * Log error message when IsInReplicationSet fails. * Modify IsAssignedKey to return true by default. * Remove wrong assigning of Service for clientMode. * Log gateway requests before error checks. * Remove unnecessary endpoint registration. * Fix lint. * Update pkg/util/ring.go Co-authored-by: Sandeep Sukhani <sandeep.d.sukhani@gmail.com> Co-authored-by: JordanRushing <rushing.jordan@gmail.com> Co-authored-by: Sandeep Sukhani <sandeep.d.sukhani@gmail.com>
3 years ago
func NewIndexClient(name string, cfg Config, schemaCfg config.SchemaConfig, limits StoreLimits, cm ClientMetrics, ownsTenantFn downloads.IndexGatewayOwnsTenant, registerer prometheus.Registerer) (index.Client, error) {
switch name {
case config.StorageTypeInMemory:
store := testutils.NewMockStorage()
return store, nil
case config.StorageTypeAWS, config.StorageTypeAWSDynamo:
if cfg.AWSStorageConfig.DynamoDB.URL == nil {
return nil, fmt.Errorf("Must set -dynamodb.url in aws mode")
}
path := strings.TrimPrefix(cfg.AWSStorageConfig.DynamoDB.URL.Path, "/")
if len(path) > 0 {
level.Warn(util_log.Logger).Log("msg", "ignoring DynamoDB URL path", "path", path)
}
return aws.NewDynamoDBIndexClient(cfg.AWSStorageConfig.DynamoDBConfig, schemaCfg, registerer)
case config.StorageTypeGCP:
return gcp.NewStorageClientV1(context.Background(), cfg.GCPStorageConfig, schemaCfg)
case config.StorageTypeGCPColumnKey, config.StorageTypeBigTable:
return gcp.NewStorageClientColumnKey(context.Background(), cfg.GCPStorageConfig, schemaCfg)
case config.StorageTypeBigTableHashed:
cfg.GCPStorageConfig.DistributeKeys = true
return gcp.NewStorageClientColumnKey(context.Background(), cfg.GCPStorageConfig, schemaCfg)
case config.StorageTypeCassandra:
return cassandra.NewStorageClient(cfg.CassandraStorageConfig, schemaCfg, registerer)
case config.StorageTypeBoltDB:
return local.NewBoltDBIndexClient(cfg.BoltDBConfig)
case config.StorageTypeGrpc:
return grpc.NewStorageClient(cfg.GrpcConfig, schemaCfg)
case config.BoltDBShipperType:
if boltDBIndexClientWithShipper != nil {
return boltDBIndexClientWithShipper, nil
}
TSDB shipper + WAL (#6049) * begins speccing out TSDB Head * auto incrementing series ref + mempostings * mintime/maxtime methods * tsdb head IndexReader impl * head correctly populates ref lookup * tsdb head tests * adds prometheus license to tsdb head * linting * [WIP] speccing out tsdb head wal * fix length check and adds tsdb wal encoding tests * exposes wal structs & removes closed semantics * logs start time in the tsdb wal * wal interface + testing * exports walrecord + returns ref when appending * specs out head manager * tsdb head manager wal initialization * tsdb wal rotation * wals dont use node name, but tsdb files do * cleans up fn signature * multi tsdb idx now just wraps Index interfaces * no longer sorts indices when creating multi-idx * tenantHeads & HeadManger index impls * head mgr tests * bugfixes & head manager tests * tsdb dir selection now helper fns * period utility * pulls out more code to helpers, fixes some var races * head recovery is more generic * tsdb manager builds from wals * pulls more helpers out of headmanager * lockedIdx, Close() on idx, tsdbManager update * removes mmap from index reader implementation * tsdb file * adds tsdb shipper config and refactors initStore * removes unused tsdbManager code * implements stores.Index and stores.ChunkWriter for tsdb * chunk.Data now supports an Entries() method * moves walreader to new util/wal pkg to avoid circular dep + tsdb storage alignment * tsdb store * passes indexWriter to chunkWriter * build a tsdb per index bucket in according with shipper conventions * dont open tsdb files until necessary for indexshipper * tsdbManager Index impl * tsdb defaults + initStore fix for invalid looping * fixes UsingTSDB helper * disables deleteRequestStore when using TSDB * pass limits to tsdb store * always start headmanager for tsdb Signed-off-by: Owen Diehl <ow.diehl@gmail.com> * fixes copy bug Signed-off-by: Owen Diehl <ow.diehl@gmail.com> * more logging Signed-off-by: Owen Diehl <ow.diehl@gmail.com> * fixes duplicate tenant label bug Signed-off-by: Owen Diehl <ow.diehl@gmail.com> * debug logs, uses label builder, removes __name__=logs for tsdb Signed-off-by: Owen Diehl <ow.diehl@gmail.com> * tsdb fixes labels at earlier pt Signed-off-by: Owen Diehl <ow.diehl@gmail.com> * account for setting tenant label in head manager test * changing tsdb dir names * identifier interface, builder to tsdb pkg * tsdb version path prefix * fixes buildfromwals identifier * fixes tsdb shipper paths * split buckets once per user set * refactors combining single and multi tenant tsdb indices on shipper reads * indexshipper ignores old gzip logic * method name refactor * remove unused record type * removes v1 prefix in tsdb paths and refactores indices method * ignores double optimization in tsdb looking for multitenant idx, shipper handles this * removes 5-ln requirement on shipper tablename regexp * groups identifiers, begins removing multitenant prefix in shipped files * passses open fn to indexshipper * exposes RealByteSlice * TSDBFile no longer needs a file descriptor, parses gzip extensions * method signature fixing * stop masquerading as compressed indices post-download in indexshipper * variable bucket regexp * removes accidental configs committed * label matcher handling for multitenancy and metricname in tsdb * explicitly require fingerprint when creating tsdb index * only add tenant label when creating multitenant tsdb write fingerprints without synthetic tenant label strip out tenant labels from queries * linting + unused removal * more linting :( * goimports * removes uploadername from indexshipper * maxuint32 for arm32 builds * tsdb chunk filterer support * always set ingester name when using object storage index Co-authored-by: Sandeep Sukhani <sandeep.d.sukhani@gmail.com>
3 years ago
if shouldUseBoltDBIndexGatewayClient(cfg) {
gateway, err := gatewayclient.NewGatewayClient(cfg.BoltDBShipperConfig.IndexGatewayClientConfig, registerer, util_log.Logger)
if err != nil {
return nil, err
}
boltDBIndexClientWithShipper = gateway
return gateway, nil
}
objectClient, err := NewObjectClient(cfg.BoltDBShipperConfig.SharedStoreType, cfg, cm)
if err != nil {
return nil, err
}
Loki: Modifies TableManager to use IndexGateway ring (#5972) * Add two metrics to the IndexGateway. - Add a new `query_readiness_duration_seconds` metric, that reports query readiness duration of a tablemanager/index gateway instance. We should use it later to report performance against the ring mode - Add a new `usersToBeQueryReadyForTotal` metric, that reports number of users involved in the query readiness operation. We should use it later to correlate number of users with the query readiness duration. * Remove `usersToBeQueryReadyForTotal`. - It will report all users always for now, so it isn't too helpful the way it is. * Rename metric help text to not mislead people. * Log queryReadiness duration. * Fix where log message and duration and triggered. * Add two metrics to the IndexGateway. - Add a new `query_readiness_duration_seconds` metric, that reports query readiness duration of a tablemanager/index gateway instance. We should use it later to report performance against the ring mode - Add a new `usersToBeQueryReadyForTotal` metric, that reports number of users involved in the query readiness operation. We should use it later to correlate number of users with the query readiness duration. * Remove `usersToBeQueryReadyForTotal`. - It will report all users always for now, so it isn't too helpful the way it is. * Rename metric help text to not mislead people. * Log queryReadiness duration. * Fix where log message and duration and triggered. * Use boundaries to skip users in TableManager. - Separate the assigning of indexClient in the IndexGateway to allow initializing the shipper with the IndexGateway ring - Add new TenantBoundariesClient entity, that answers if a given tenantID should be ignored or not - Use the TenantBoundariesClient implemented by the IndexGateway in the downloads TableManager * Add IndexGateway configuration docs. * Add tests for pkg/util/ring TokenFor() and IsAssignedKey() Signed-off-by: JordanRushing <rushing.jordan@gmail.com> * Small whitespace fix in pkg/util/ring_test.go Signed-off-by: JordanRushing <rushing.jordan@gmail.com> * Apply suggestions from code review Rewrite QueryIndex error phrase. Co-authored-by: JordanRushing <rushing.jordan@gmail.com> * Join users list in a single string. - This is necessary since go-kit doesn't support array type. * Tweak queryReadiness log messages. - As suggested by Ed on https://github.com/grafana/loki/pull/5972#discussion_r859734129 and https://github.com/grafana/loki/pull/5972#discussion_r859736072 * Implement new RingManager. - Adds a new entity to indexgateway named RingManager, responsible for managing the ring and the lifecycler used by the indexgateway. The ringManager is guaranteed to be initiatiated before the Shipper and before the IndexGateway. - Remove the readiness logic from the IndexGateway. - Pass the RingManager as the TenantBoundaries implementation of the Shipper * Return non-empty ringmanager for all modes. * Fix lint regarding error not being check. * Fix lint due to wrong import order. * Implement support for client and server mode for the ring manager. * Fix ring manager services registration. * Add option to configure whether or not to log gateway requests. * Check if IndexGateway is enabled instead of active. * Tune RingManager behavior. - Instantiate ring buffers inside IsAssignedKey instead of reusing same buffer to avoid issues with concurrency. - Remove unnecessary details from IndexGateway mode docs. - Return error when wrongly instantiating a RingManager when IndexGateway is in simple mode. * Rewrite `TenantInBoundaries` as a func instead of interface. * Fix lint. - Fix YAML tag - Remove else clause * Use distributor instead of querier in test. - Since the querier needs to wait for the index gateway ring, it isn't suitable for this test anymore. * Add ring mode guard clause. * Return code erased by mistake. * Rename TenantInBoundaries to IndexGatewayOwnsTenant. * Log len of users instead of all of them to avoid log msgs. * Add docstrings to new public functions. * Apply suggestions from code review Document that tenant filtering is only applied during query readiness. Co-authored-by: Sandeep Sukhani <sandeep.d.sukhani@gmail.com> * Modify IsAssignedKey to expect address directly instead of lifecycler. - Also removes HasDedicatedAddress, since it isn't necessary anymore * Log error message when IsInReplicationSet fails. * Modify IsAssignedKey to return true by default. * Remove wrong assigning of Service for clientMode. * Log gateway requests before error checks. * Remove unnecessary endpoint registration. * Fix lint. * Update pkg/util/ring.go Co-authored-by: Sandeep Sukhani <sandeep.d.sukhani@gmail.com> Co-authored-by: JordanRushing <rushing.jordan@gmail.com> Co-authored-by: Sandeep Sukhani <sandeep.d.sukhani@gmail.com>
3 years ago
boltDBIndexClientWithShipper, err = shipper.NewShipper(cfg.BoltDBShipperConfig, objectClient, limits, ownsTenantFn, registerer)
return boltDBIndexClientWithShipper, err
default:
return nil, fmt.Errorf("Unrecognized storage client %v, choose one of: %v, %v, %v, %v, %v, %v", name, config.StorageTypeAWS, config.StorageTypeCassandra, config.StorageTypeInMemory, config.StorageTypeGCP, config.StorageTypeBigTable, config.StorageTypeBigTableHashed)
}
}
// NewChunkClient makes a new chunk.Client of the desired types.
func NewChunkClient(name string, cfg Config, schemaCfg config.SchemaConfig, clientMetrics ClientMetrics, registerer prometheus.Registerer) (client.Client, error) {
switch name {
case config.StorageTypeInMemory:
return testutils.NewMockStorage(), nil
case config.StorageTypeAWS, config.StorageTypeS3:
c, err := aws.NewS3ObjectClient(cfg.AWSStorageConfig.S3Config, cfg.Hedging)
if err != nil {
return nil, err
}
return client.NewClientWithMaxParallel(c, nil, cfg.MaxParallelGetChunk, schemaCfg), nil
case config.StorageTypeAWSDynamo:
if cfg.AWSStorageConfig.DynamoDB.URL == nil {
return nil, fmt.Errorf("Must set -dynamodb.url in aws mode")
}
path := strings.TrimPrefix(cfg.AWSStorageConfig.DynamoDB.URL.Path, "/")
if len(path) > 0 {
level.Warn(util_log.Logger).Log("msg", "ignoring DynamoDB URL path", "path", path)
}
return aws.NewDynamoDBChunkClient(cfg.AWSStorageConfig.DynamoDBConfig, schemaCfg, registerer)
case config.StorageTypeAzure:
c, err := azure.NewBlobStorage(&cfg.AzureStorageConfig, clientMetrics.AzureMetrics, cfg.Hedging)
if err != nil {
return nil, err
}
return client.NewClientWithMaxParallel(c, nil, cfg.MaxParallelGetChunk, schemaCfg), nil
Feat: add Baidu Cloud BOS as storage backends for Loki #4788 (#5848) * feat: add baidu bce bos storage support Signed-off-by: arcosx <arcosx@outlook.com> * add baidu bce bos as chunk storage backend Signed-off-by: arcosx <arcosx@outlook.com> * fix: some doc error && rewrite bad code Signed-off-by: arcosx <arcosx@outlook.com> * fix: add the `BceServiceError` source link Signed-off-by: arcosx <arcosx@outlook.com> * Update CHANGELOG.md Co-authored-by: Karen Miller <84039272+KMiller-Grafana@users.noreply.github.com> * Update docs/sources/configuration/_index.md Co-authored-by: Karen Miller <84039272+KMiller-Grafana@users.noreply.github.com> Signed-off-by: arcosx <arcosx@outlook.com> * Update docs/sources/configuration/_index.md Co-authored-by: Karen Miller <84039272+KMiller-Grafana@users.noreply.github.com> * Update docs/sources/configuration/_index.md Co-authored-by: Karen Miller <84039272+KMiller-Grafana@users.noreply.github.com> * Update docs/sources/configuration/_index.md Co-authored-by: Karen Miller <84039272+KMiller-Grafana@users.noreply.github.com> * Update docs/sources/configuration/_index.md Co-authored-by: Karen Miller <84039272+KMiller-Grafana@users.noreply.github.com> * Update pkg/storage/chunk/client/baidubce/bos_storage_client.go Co-authored-by: Karen Miller <84039272+KMiller-Grafana@users.noreply.github.com> * Update pkg/storage/chunk/client/baidubce/bos_storage_client.go Co-authored-by: Karen Miller <84039272+KMiller-Grafana@users.noreply.github.com> * Update pkg/storage/chunk/client/baidubce/bos_storage_client.go Co-authored-by: Karen Miller <84039272+KMiller-Grafana@users.noreply.github.com> Co-authored-by: Karen Miller <84039272+KMiller-Grafana@users.noreply.github.com>
3 years ago
case config.StorageTypeBOS:
c, err := baidubce.NewBOSObjectStorage(&cfg.BOSStorageConfig)
if err != nil {
return nil, err
}
return client.NewClientWithMaxParallel(c, nil, cfg.MaxChunkBatchSize, schemaCfg), nil
case config.StorageTypeGCP:
return gcp.NewBigtableObjectClient(context.Background(), cfg.GCPStorageConfig, schemaCfg)
case config.StorageTypeGCPColumnKey, config.StorageTypeBigTable, config.StorageTypeBigTableHashed:
return gcp.NewBigtableObjectClient(context.Background(), cfg.GCPStorageConfig, schemaCfg)
case config.StorageTypeGCS:
c, err := gcp.NewGCSObjectClient(context.Background(), cfg.GCSConfig, cfg.Hedging)
if err != nil {
return nil, err
}
return client.NewClientWithMaxParallel(c, nil, cfg.MaxParallelGetChunk, schemaCfg), nil
case config.StorageTypeSwift:
c, err := openstack.NewSwiftObjectClient(cfg.Swift, cfg.Hedging)
if err != nil {
return nil, err
}
return client.NewClientWithMaxParallel(c, nil, cfg.MaxParallelGetChunk, schemaCfg), nil
case config.StorageTypeCassandra:
return cassandra.NewObjectClient(cfg.CassandraStorageConfig, schemaCfg, registerer, cfg.MaxParallelGetChunk)
case config.StorageTypeFileSystem:
store, err := local.NewFSObjectClient(cfg.FSConfig)
if err != nil {
return nil, err
}
return client.NewClientWithMaxParallel(store, client.FSEncoder, cfg.MaxParallelGetChunk, schemaCfg), nil
case config.StorageTypeGrpc:
return grpc.NewStorageClient(cfg.GrpcConfig, schemaCfg)
default:
return nil, fmt.Errorf("Unrecognized storage client %v, choose one of: %v, %v, %v, %v, %v, %v, %v, %v", name, config.StorageTypeAWS, config.StorageTypeAzure, config.StorageTypeCassandra, config.StorageTypeInMemory, config.StorageTypeGCP, config.StorageTypeBigTable, config.StorageTypeBigTableHashed, config.StorageTypeGrpc)
}
}
// NewTableClient makes a new table client based on the configuration.
func NewTableClient(name string, cfg Config, cm ClientMetrics, registerer prometheus.Registerer) (index.TableClient, error) {
switch name {
case config.StorageTypeInMemory:
return testutils.NewMockStorage(), nil
case config.StorageTypeAWS, config.StorageTypeAWSDynamo:
if cfg.AWSStorageConfig.DynamoDB.URL == nil {
return nil, fmt.Errorf("Must set -dynamodb.url in aws mode")
}
path := strings.TrimPrefix(cfg.AWSStorageConfig.DynamoDB.URL.Path, "/")
if len(path) > 0 {
level.Warn(util_log.Logger).Log("msg", "ignoring DynamoDB URL path", "path", path)
}
return aws.NewDynamoDBTableClient(cfg.AWSStorageConfig.DynamoDBConfig, registerer)
case config.StorageTypeGCP, config.StorageTypeGCPColumnKey, config.StorageTypeBigTable, config.StorageTypeBigTableHashed:
return gcp.NewTableClient(context.Background(), cfg.GCPStorageConfig)
case config.StorageTypeCassandra:
return cassandra.NewTableClient(context.Background(), cfg.CassandraStorageConfig, registerer)
case config.StorageTypeBoltDB:
return local.NewTableClient(cfg.BoltDBConfig.Directory)
case config.StorageTypeGrpc:
return grpc.NewTableClient(cfg.GrpcConfig)
case config.BoltDBShipperType:
objectClient, err := NewObjectClient(cfg.BoltDBShipperConfig.SharedStoreType, cfg, cm)
if err != nil {
return nil, err
}
return shipper.NewBoltDBShipperTableClient(objectClient, cfg.BoltDBShipperConfig.SharedStoreKeyPrefix), nil
default:
return nil, fmt.Errorf("Unrecognized storage client %v, choose one of: %v, %v, %v, %v, %v, %v, %v", name, config.StorageTypeAWS, config.StorageTypeCassandra, config.StorageTypeInMemory, config.StorageTypeGCP, config.StorageTypeBigTable, config.StorageTypeBigTableHashed, config.StorageTypeGrpc)
}
}
// // 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 == shipper.BoltDBShipperType {
// name = "boltdb"
// cfg.FSConfig = chunk_local.FSConfig{Directory: cfg.BoltDBShipperConfig.ActiveIndexDirectory}
// }
// return storage.NewTableClient(name, cfg.Config, prometheus.DefaultRegisterer)
// }
// NewBucketClient makes a new bucket client based on the configuration.
func NewBucketClient(storageConfig Config) (index.BucketClient, error) {
if storageConfig.FSConfig.Directory != "" {
return local.NewFSObjectClient(storageConfig.FSConfig)
}
return nil, nil
}
type ClientMetrics struct {
AzureMetrics azure.BlobStorageMetrics
}
func NewClientMetrics() ClientMetrics {
return ClientMetrics{
AzureMetrics: azure.NewBlobStorageMetrics(),
}
}
func (c *ClientMetrics) Unregister() {
c.AzureMetrics.Unregister()
}
// NewObjectClient makes a new StorageClient of the desired types.
func NewObjectClient(name string, cfg Config, clientMetrics ClientMetrics) (client.ObjectClient, error) {
switch name {
case config.StorageTypeAWS, config.StorageTypeS3:
return aws.NewS3ObjectClient(cfg.AWSStorageConfig.S3Config, cfg.Hedging)
case config.StorageTypeGCS:
return gcp.NewGCSObjectClient(context.Background(), cfg.GCSConfig, cfg.Hedging)
case config.StorageTypeAzure:
return azure.NewBlobStorage(&cfg.AzureStorageConfig, clientMetrics.AzureMetrics, cfg.Hedging)
case config.StorageTypeSwift:
return openstack.NewSwiftObjectClient(cfg.Swift, cfg.Hedging)
case config.StorageTypeInMemory:
return testutils.NewMockStorage(), nil
case config.StorageTypeFileSystem:
return local.NewFSObjectClient(cfg.FSConfig)
Feat: add Baidu Cloud BOS as storage backends for Loki #4788 (#5848) * feat: add baidu bce bos storage support Signed-off-by: arcosx <arcosx@outlook.com> * add baidu bce bos as chunk storage backend Signed-off-by: arcosx <arcosx@outlook.com> * fix: some doc error && rewrite bad code Signed-off-by: arcosx <arcosx@outlook.com> * fix: add the `BceServiceError` source link Signed-off-by: arcosx <arcosx@outlook.com> * Update CHANGELOG.md Co-authored-by: Karen Miller <84039272+KMiller-Grafana@users.noreply.github.com> * Update docs/sources/configuration/_index.md Co-authored-by: Karen Miller <84039272+KMiller-Grafana@users.noreply.github.com> Signed-off-by: arcosx <arcosx@outlook.com> * Update docs/sources/configuration/_index.md Co-authored-by: Karen Miller <84039272+KMiller-Grafana@users.noreply.github.com> * Update docs/sources/configuration/_index.md Co-authored-by: Karen Miller <84039272+KMiller-Grafana@users.noreply.github.com> * Update docs/sources/configuration/_index.md Co-authored-by: Karen Miller <84039272+KMiller-Grafana@users.noreply.github.com> * Update docs/sources/configuration/_index.md Co-authored-by: Karen Miller <84039272+KMiller-Grafana@users.noreply.github.com> * Update pkg/storage/chunk/client/baidubce/bos_storage_client.go Co-authored-by: Karen Miller <84039272+KMiller-Grafana@users.noreply.github.com> * Update pkg/storage/chunk/client/baidubce/bos_storage_client.go Co-authored-by: Karen Miller <84039272+KMiller-Grafana@users.noreply.github.com> * Update pkg/storage/chunk/client/baidubce/bos_storage_client.go Co-authored-by: Karen Miller <84039272+KMiller-Grafana@users.noreply.github.com> Co-authored-by: Karen Miller <84039272+KMiller-Grafana@users.noreply.github.com>
3 years ago
case config.StorageTypeBOS:
return baidubce.NewBOSObjectStorage(&cfg.BOSStorageConfig)
default:
return nil, fmt.Errorf("Unrecognized storage client %v, choose one of: %v, %v, %v, %v, %v", name, config.StorageTypeAWS, config.StorageTypeS3, config.StorageTypeGCS, config.StorageTypeAzure, config.StorageTypeFileSystem)
}
}