diff --git a/main.go b/main.go index 01cb8bfb69..576fda825c 100644 --- a/main.go +++ b/main.go @@ -23,15 +23,15 @@ import ( "github.com/golang/glog" "github.com/prometheus/client_golang/extraction" - registry "github.com/prometheus/client_golang/prometheus" clientmodel "github.com/prometheus/client_golang/model" + registry "github.com/prometheus/client_golang/prometheus" "github.com/prometheus/prometheus/config" "github.com/prometheus/prometheus/notification" "github.com/prometheus/prometheus/retrieval" "github.com/prometheus/prometheus/rules/manager" - "github.com/prometheus/prometheus/storage/metric/tiered" + "github.com/prometheus/prometheus/storage/local" "github.com/prometheus/prometheus/storage/remote" "github.com/prometheus/prometheus/storage/remote/opentsdb" "github.com/prometheus/prometheus/web" @@ -54,41 +54,30 @@ var ( diskAppendQueueCapacity = flag.Int("storage.queue.diskAppendCapacity", 1000000, "The size of the queue for items that are pending writing to disk.") memoryAppendQueueCapacity = flag.Int("storage.queue.memoryAppendCapacity", 10000, "The size of the queue for items that are pending writing to memory.") - compactInterval = flag.Duration("compact.interval", 3*time.Hour, "The amount of time between compactions.") - compactGroupSize = flag.Int("compact.groupSize", 500, "The minimum group size for compacted samples.") - compactAgeInclusiveness = flag.Duration("compact.ageInclusiveness", 5*time.Minute, "The age beyond which samples should be compacted.") - deleteInterval = flag.Duration("delete.interval", 11*time.Hour, "The amount of time between deletion of old values.") deleteAge = flag.Duration("delete.ageMaximum", 15*24*time.Hour, "The relative maximum age for values before they are deleted.") - arenaFlushInterval = flag.Duration("arena.flushInterval", 15*time.Minute, "The period at which the in-memory arena is flushed to disk.") - arenaTTL = flag.Duration("arena.ttl", 10*time.Minute, "The relative age of values to purge to disk from memory.") + memoryEvictionInterval = flag.Duration("storage.memory.evictionInterval", 15*time.Minute, "The period at which old data is evicted from memory.") + memoryRetentionPeriod = flag.Duration("storage.memory.retentionPeriod", time.Hour, "The period of time to retain in memory during evictions.") + + storagePurgeInterval = flag.Duration("storage.purgeInterval", time.Hour, "How frequently to purge old data from the storage.") + storageRetentionPeriod = flag.Duration("storage.retentionPeriod", 15*24*time.Hour, "The period of time to retain in storage.") notificationQueueCapacity = flag.Int("alertmanager.notificationQueueCapacity", 100, "The size of the queue for pending alert manager notifications.") printVersion = flag.Bool("version", false, "print version information") - - shutdownTimeout = flag.Duration("shutdownGracePeriod", 0*time.Second, "The amount of time Prometheus gives background services to finish running when shutdown is requested.") ) type prometheus struct { - compactionTimer *time.Ticker - deletionTimer *time.Ticker - - curationSema chan struct{} - stopBackgroundOperations chan struct{} - unwrittenSamples chan *extraction.Result ruleManager manager.RuleManager targetManager retrieval.TargetManager notifications chan notification.NotificationReqs - storage *tiered.TieredStorage + storage storage_ng.Storage remoteTSDBQueue *remote.TSDBQueueManager - curationState tiered.CurationStateUpdater - closeOnce sync.Once } @@ -105,73 +94,6 @@ func (p *prometheus) interruptHandler() { os.Exit(0) } -func (p *prometheus) compact(olderThan time.Duration, groupSize int) error { - select { - case s, ok := <-p.curationSema: - if !ok { - glog.Warning("Prometheus is shutting down; no more curation runs are allowed.") - return nil - } - - defer func() { - p.curationSema <- s - }() - - default: - glog.Warningf("Deferred compaction for %s and %s due to existing operation.", olderThan, groupSize) - - return nil - } - - processor := tiered.NewCompactionProcessor(&tiered.CompactionProcessorOptions{ - MaximumMutationPoolBatch: groupSize * 3, - MinimumGroupSize: groupSize, - }) - defer processor.Close() - - curator := tiered.NewCurator(&tiered.CuratorOptions{ - Stop: p.stopBackgroundOperations, - - ViewQueue: p.storage.ViewQueue, - }) - defer curator.Close() - - return curator.Run(olderThan, clientmodel.Now(), processor, p.storage.DiskStorage.CurationRemarks, p.storage.DiskStorage.MetricSamples, p.storage.DiskStorage.MetricHighWatermarks, p.curationState) -} - -func (p *prometheus) delete(olderThan time.Duration, batchSize int) error { - select { - case s, ok := <-p.curationSema: - if !ok { - glog.Warning("Prometheus is shutting down; no more curation runs are allowed.") - return nil - } - - defer func() { - p.curationSema <- s - }() - - default: - glog.Warningf("Deferred deletion for %s due to existing operation.", olderThan) - - return nil - } - - processor := tiered.NewDeletionProcessor(&tiered.DeletionProcessorOptions{ - MaximumMutationPoolBatch: batchSize, - }) - defer processor.Close() - - curator := tiered.NewCurator(&tiered.CuratorOptions{ - Stop: p.stopBackgroundOperations, - - ViewQueue: p.storage.ViewQueue, - }) - defer curator.Close() - - return curator.Run(olderThan, clientmodel.Now(), processor, p.storage.DiskStorage.CurationRemarks, p.storage.DiskStorage.MetricSamples, p.storage.DiskStorage.MetricHighWatermarks, p.curationState) -} - func (p *prometheus) Close() { p.closeOnce.Do(p.close) } @@ -185,31 +107,11 @@ func (p *prometheus) close() { p.ruleManager.Stop() glog.Info("Rule Executor: Done") - // Stop any currently active curation (deletion or compaction). - close(p.stopBackgroundOperations) - glog.Info("Current Curation Workers: Requested") - - // Disallow further curation work. - close(p.curationSema) - - // Stop curation timers. - if p.compactionTimer != nil { - p.compactionTimer.Stop() - } - if p.deletionTimer != nil { - p.deletionTimer.Stop() - } - glog.Info("Future Curation Workers: Done") - - glog.Infof("Waiting %s for background systems to exit and flush before finalizing (DO NOT INTERRUPT THE PROCESS) ...", *shutdownTimeout) - - // Wart: We should have a concrete form of synchronization for this, not a - // hokey sleep statement. - time.Sleep(*shutdownTimeout) - close(p.unwrittenSamples) - p.storage.Close() + if err := p.storage.Close(); err != nil { + glog.Error("Error closing local storage: ", err) + } glog.Info("Local Storage: Done") if p.remoteTSDBQueue != nil { @@ -239,11 +141,23 @@ func main() { glog.Fatalf("Error loading configuration from %s: %v", *configFile, err) } - ts, err := tiered.NewTieredStorage(uint(*diskAppendQueueCapacity), 100, *arenaFlushInterval, *arenaTTL, *metricsStoragePath) + persistence, err := storage_ng.NewDiskPersistence(*metricsStoragePath, 1024) if err != nil { - glog.Fatal("Error opening storage: ", err) + glog.Fatal("Error opening disk persistence: ", err) } - registry.MustRegister(ts) + + o := &storage_ng.MemorySeriesStorageOptions{ + Persistence: persistence, + MemoryEvictionInterval: *memoryEvictionInterval, + MemoryRetentionPeriod: *memoryRetentionPeriod, + PersistencePurgeInterval: *storagePurgeInterval, + PersistenceRetentionPeriod: *storageRetentionPeriod, + } + memStorage, err := storage_ng.NewMemorySeriesStorage(o) + if err != nil { + glog.Fatal("Error opening memory series storage: ", err) + } + //registry.MustRegister(memStorage) var remoteTSDBQueue *remote.TSDBQueueManager if *remoteTSDBUrl == "" { @@ -263,9 +177,6 @@ func main() { Ingester: retrieval.ChannelIngester(unwrittenSamples), } - compactionTimer := time.NewTicker(*compactInterval) - deletionTimer := time.NewTicker(*deleteInterval) - // Queue depth will need to be exposed targetManager := retrieval.NewTargetManager(ingester) targetManager.AddTargetsFromConfig(conf) @@ -277,7 +188,7 @@ func main() { Results: unwrittenSamples, Notifications: notifications, EvaluationInterval: conf.EvaluationInterval(), - Storage: ts, + Storage: memStorage, PrometheusUrl: web.MustBuildServerUrl(), }) if err := ruleManager.AddRulesFromConfig(conf); err != nil { @@ -309,82 +220,41 @@ func main() { } consolesHandler := &web.ConsolesHandler{ - Storage: ts, - } - - databasesHandler := &web.DatabasesHandler{ - Provider: ts.DiskStorage, - RefreshInterval: 5 * time.Minute, + Storage: memStorage, } metricsService := &api.MetricsService{ Config: &conf, TargetManager: targetManager, - Storage: ts, + Storage: memStorage, } prometheus := &prometheus{ - compactionTimer: compactionTimer, - - deletionTimer: deletionTimer, - - curationState: prometheusStatus, - curationSema: make(chan struct{}, 1), - unwrittenSamples: unwrittenSamples, - stopBackgroundOperations: make(chan struct{}), - ruleManager: ruleManager, targetManager: targetManager, notifications: notifications, - storage: ts, + storage: memStorage, remoteTSDBQueue: remoteTSDBQueue, } defer prometheus.Close() webService := &web.WebService{ - StatusHandler: prometheusStatus, - MetricsHandler: metricsService, - DatabasesHandler: databasesHandler, - ConsolesHandler: consolesHandler, - AlertsHandler: alertsHandler, + StatusHandler: prometheusStatus, + MetricsHandler: metricsService, + ConsolesHandler: consolesHandler, + AlertsHandler: alertsHandler, QuitDelegate: prometheus.Close, } - prometheus.curationSema <- struct{}{} - storageStarted := make(chan bool) - go ts.Serve(storageStarted) + go memStorage.Serve(storageStarted) <-storageStarted go prometheus.interruptHandler() - go func() { - for _ = range prometheus.compactionTimer.C { - glog.Info("Starting compaction...") - err := prometheus.compact(*compactAgeInclusiveness, *compactGroupSize) - - if err != nil { - glog.Error("could not compact: ", err) - } - glog.Info("Done") - } - }() - - go func() { - for _ = range prometheus.deletionTimer.C { - glog.Info("Starting deletion of stale values...") - err := prometheus.delete(*deleteAge, deletionBatchSize) - - if err != nil { - glog.Error("could not delete: ", err) - } - glog.Info("Done") - } - }() - go func() { err := webService.ServeForever() if err != nil { @@ -395,7 +265,7 @@ func main() { // TODO(all): Migrate this into prometheus.serve(). for block := range unwrittenSamples { if block.Err == nil && len(block.Samples) > 0 { - ts.AppendSamples(block.Samples) + memStorage.AppendSamples(block.Samples) if remoteTSDBQueue != nil { remoteTSDBQueue.Queue(block.Samples) } diff --git a/rules/alerting.go b/rules/alerting.go index 1872c948b6..062f0d258e 100644 --- a/rules/alerting.go +++ b/rules/alerting.go @@ -23,7 +23,7 @@ import ( "github.com/prometheus/prometheus/rules/ast" "github.com/prometheus/prometheus/stats" - "github.com/prometheus/prometheus/storage/metric" + "github.com/prometheus/prometheus/storage/local" "github.com/prometheus/prometheus/utility" ) @@ -118,11 +118,11 @@ func (rule *AlertingRule) Name() string { return rule.name } -func (rule *AlertingRule) EvalRaw(timestamp clientmodel.Timestamp, storage metric.PreloadingPersistence) (ast.Vector, error) { +func (rule *AlertingRule) EvalRaw(timestamp clientmodel.Timestamp, storage storage_ng.Storage) (ast.Vector, error) { return ast.EvalVectorInstant(rule.Vector, timestamp, storage, stats.NewTimerGroup()) } -func (rule *AlertingRule) Eval(timestamp clientmodel.Timestamp, storage metric.PreloadingPersistence) (ast.Vector, error) { +func (rule *AlertingRule) Eval(timestamp clientmodel.Timestamp, storage storage_ng.Storage) (ast.Vector, error) { // Get the raw value of the rule expression. exprResult, err := rule.EvalRaw(timestamp, storage) if err != nil { diff --git a/rules/ast/ast.go b/rules/ast/ast.go index c03602db6e..87359c36fc 100644 --- a/rules/ast/ast.go +++ b/rules/ast/ast.go @@ -15,20 +15,22 @@ package ast import ( "errors" + "flag" "fmt" "hash/fnv" "math" "sort" "time" - "github.com/golang/glog" - clientmodel "github.com/prometheus/client_golang/model" "github.com/prometheus/prometheus/stats" "github.com/prometheus/prometheus/storage/metric" + "github.com/prometheus/prometheus/storage/local" ) +var defaultStalenessDelta = flag.Duration("defaultStalenessDelta", 300*time.Second, "Default staleness delta allowance in seconds during expression evaluations.") + // ---------------------------------------------------------------------------- // Raw data value types. @@ -114,7 +116,7 @@ type Node interface { type ScalarNode interface { Node // Eval evaluates and returns the value of the scalar represented by this node. - Eval(timestamp clientmodel.Timestamp, view *viewAdapter) clientmodel.SampleValue + Eval(timestamp clientmodel.Timestamp) clientmodel.SampleValue } // VectorNode is a Node for vector values. @@ -123,17 +125,17 @@ type VectorNode interface { // Eval evaluates the node recursively and returns the result // as a Vector (i.e. a slice of Samples all at the given // Timestamp). - Eval(timestamp clientmodel.Timestamp, view *viewAdapter) Vector + Eval(timestamp clientmodel.Timestamp) Vector } // MatrixNode is a Node for matrix values. type MatrixNode interface { Node // Eval evaluates the node recursively and returns the result as a Matrix. - Eval(timestamp clientmodel.Timestamp, view *viewAdapter) Matrix + Eval(timestamp clientmodel.Timestamp) Matrix // Eval evaluates the node recursively and returns the result // as a Matrix that only contains the boundary values. - EvalBoundaries(timestamp clientmodel.Timestamp, view *viewAdapter) Matrix + EvalBoundaries(timestamp clientmodel.Timestamp) Matrix } // StringNode is a Node for string values. @@ -141,7 +143,7 @@ type StringNode interface { Node // Eval evaluates and returns the value of the string // represented by this node. - Eval(timestamp clientmodel.Timestamp, view *viewAdapter) string + Eval(timestamp clientmodel.Timestamp) string } // ---------------------------------------------------------------------------- @@ -176,7 +178,11 @@ type ( // A VectorSelector represents a metric name plus labelset. VectorSelector struct { labelMatchers metric.LabelMatchers + // The series iterators are populated at query analysis time. + iterators map[clientmodel.Fingerprint]storage_ng.SeriesIterator + metrics map[clientmodel.Fingerprint]clientmodel.Metric // Fingerprints are populated from label matchers at query analysis time. + // TODO: do we still need these? fingerprints clientmodel.Fingerprints } @@ -213,8 +219,11 @@ type ( // timerange. MatrixSelector struct { labelMatchers metric.LabelMatchers - // Fingerprints are populated from label matchers at query - // analysis time. + // The series iterators are populated at query analysis time. + iterators map[clientmodel.Fingerprint]storage_ng.SeriesIterator + metrics map[clientmodel.Fingerprint]clientmodel.Metric + // Fingerprints are populated from label matchers at query analysis time. + // TODO: do we still need these? fingerprints clientmodel.Fingerprints interval time.Duration } @@ -308,22 +317,22 @@ func (node StringFunctionCall) Children() Nodes { return node.args } // Eval implements the ScalarNode interface and returns the selector // value. -func (node *ScalarLiteral) Eval(timestamp clientmodel.Timestamp, view *viewAdapter) clientmodel.SampleValue { +func (node *ScalarLiteral) Eval(timestamp clientmodel.Timestamp) clientmodel.SampleValue { return node.value } // Eval implements the ScalarNode interface and returns the result of // the expression. -func (node *ScalarArithExpr) Eval(timestamp clientmodel.Timestamp, view *viewAdapter) clientmodel.SampleValue { - lhs := node.lhs.Eval(timestamp, view) - rhs := node.rhs.Eval(timestamp, view) +func (node *ScalarArithExpr) Eval(timestamp clientmodel.Timestamp) clientmodel.SampleValue { + lhs := node.lhs.Eval(timestamp) + rhs := node.rhs.Eval(timestamp) return evalScalarBinop(node.opType, lhs, rhs) } // Eval implements the ScalarNode interface and returns the result of // the function call. -func (node *ScalarFunctionCall) Eval(timestamp clientmodel.Timestamp, view *viewAdapter) clientmodel.SampleValue { - return node.function.callFn(timestamp, view, node.args).(clientmodel.SampleValue) +func (node *ScalarFunctionCall) Eval(timestamp clientmodel.Timestamp) clientmodel.SampleValue { + return node.function.callFn(timestamp, node.args).(clientmodel.SampleValue) } func (node *VectorAggregation) labelsToGroupingKey(labels clientmodel.Metric) uint64 { @@ -357,33 +366,34 @@ func labelsToKey(labels clientmodel.Metric) uint64 { } // EvalVectorInstant evaluates a VectorNode with an instant query. -func EvalVectorInstant(node VectorNode, timestamp clientmodel.Timestamp, storage metric.PreloadingPersistence, queryStats *stats.TimerGroup) (vector Vector, err error) { - viewAdapter, err := viewAdapterForInstantQuery(node, timestamp, storage, queryStats) +func EvalVectorInstant(node VectorNode, timestamp clientmodel.Timestamp, storage storage_ng.Storage, queryStats *stats.TimerGroup) (Vector, error) { + closer, err := prepareInstantQuery(node, timestamp, storage, queryStats) if err != nil { - return + return nil, err } - vector = node.Eval(timestamp, viewAdapter) - return + defer closer.Close() + return node.Eval(timestamp), nil } // EvalVectorRange evaluates a VectorNode with a range query. -func EvalVectorRange(node VectorNode, start clientmodel.Timestamp, end clientmodel.Timestamp, interval time.Duration, storage metric.PreloadingPersistence, queryStats *stats.TimerGroup) (Matrix, error) { +func EvalVectorRange(node VectorNode, start clientmodel.Timestamp, end clientmodel.Timestamp, interval time.Duration, storage storage_ng.Storage, queryStats *stats.TimerGroup) (Matrix, error) { // Explicitly initialize to an empty matrix since a nil Matrix encodes to // null in JSON. matrix := Matrix{} - viewTimer := queryStats.GetTimer(stats.TotalViewBuildingTime).Start() - viewAdapter, err := viewAdapterForRangeQuery(node, start, end, interval, storage, queryStats) - viewTimer.Stop() + prepareTimer := queryStats.GetTimer(stats.TotalQueryPreparationTime).Start() + closer, err := prepareRangeQuery(node, start, end, interval, storage, queryStats) + prepareTimer.Stop() if err != nil { return nil, err } + defer closer.Close() // TODO implement watchdog timer for long-running queries. evalTimer := queryStats.GetTimer(stats.InnerEvalTime).Start() sampleSets := map[uint64]*metric.SampleSet{} for t := start; t.Before(end); t = t.Add(interval) { - vector := node.Eval(t, viewAdapter) + vector := node.Eval(t) for _, sample := range vector { samplePair := metric.SamplePair{ Value: sample.Value, @@ -444,8 +454,8 @@ func (node *VectorAggregation) groupedAggregationsToVector(aggregations map[uint // Eval implements the VectorNode interface and returns the aggregated // Vector. -func (node *VectorAggregation) Eval(timestamp clientmodel.Timestamp, view *viewAdapter) Vector { - vector := node.vector.Eval(timestamp, view) +func (node *VectorAggregation) Eval(timestamp clientmodel.Timestamp) Vector { + vector := node.vector.Eval(timestamp) result := map[uint64]*groupedAggregation{} for _, sample := range vector { groupingKey := node.labelsToGroupingKey(sample.Metric) @@ -498,19 +508,91 @@ func (node *VectorAggregation) Eval(timestamp clientmodel.Timestamp, view *viewA // Eval implements the VectorNode interface and returns the value of // the selector. -func (node *VectorSelector) Eval(timestamp clientmodel.Timestamp, view *viewAdapter) Vector { - values, err := view.GetValueAtTime(node.fingerprints, timestamp) - if err != nil { - glog.Error("Unable to get vector values: ", err) - return Vector{} +func (node *VectorSelector) Eval(timestamp clientmodel.Timestamp) Vector { + //// timer := v.stats.GetTimer(stats.GetValueAtTimeTime).Start() + samples := Vector{} + for fp, it := range node.iterators { + sampleCandidates := it.GetValueAtTime(timestamp) + samplePair := chooseClosestSample(sampleCandidates, timestamp) + if samplePair != nil { + samples = append(samples, &clientmodel.Sample{ + Metric: node.metrics[fp], // TODO: need copy here because downstream can modify! + Value: samplePair.Value, + Timestamp: timestamp, + }) + } + } + //// timer.Stop() + return samples +} + +// chooseClosestSample chooses the closest sample of a list of samples +// surrounding a given target time. If samples are found both before and after +// the target time, the sample value is interpolated between these. Otherwise, +// the single closest sample is returned verbatim. +func chooseClosestSample(samples metric.Values, timestamp clientmodel.Timestamp) *metric.SamplePair { + var closestBefore *metric.SamplePair + var closestAfter *metric.SamplePair + for _, candidate := range samples { + delta := candidate.Timestamp.Sub(timestamp) + // Samples before target time. + if delta < 0 { + // Ignore samples outside of staleness policy window. + if -delta > *defaultStalenessDelta { + continue + } + // Ignore samples that are farther away than what we've seen before. + if closestBefore != nil && candidate.Timestamp.Before(closestBefore.Timestamp) { + continue + } + sample := candidate + closestBefore = &sample + } + + // Samples after target time. + if delta >= 0 { + // Ignore samples outside of staleness policy window. + if delta > *defaultStalenessDelta { + continue + } + // Ignore samples that are farther away than samples we've seen before. + if closestAfter != nil && candidate.Timestamp.After(closestAfter.Timestamp) { + continue + } + sample := candidate + closestAfter = &sample + } + } + + switch { + case closestBefore != nil && closestAfter != nil: + return interpolateSamples(closestBefore, closestAfter, timestamp) + case closestBefore != nil: + return closestBefore + default: + return closestAfter + } +} + +// interpolateSamples interpolates a value at a target time between two +// provided sample pairs. +func interpolateSamples(first, second *metric.SamplePair, timestamp clientmodel.Timestamp) *metric.SamplePair { + dv := second.Value - first.Value + dt := second.Timestamp.Sub(first.Timestamp) + + dDt := dv / clientmodel.SampleValue(dt) + offset := clientmodel.SampleValue(timestamp.Sub(first.Timestamp)) + + return &metric.SamplePair{ + Value: first.Value + (offset * dDt), + Timestamp: timestamp, } - return values } // Eval implements the VectorNode interface and returns the result of // the function call. -func (node *VectorFunctionCall) Eval(timestamp clientmodel.Timestamp, view *viewAdapter) Vector { - return node.function.callFn(timestamp, view, node.args).(Vector) +func (node *VectorFunctionCall) Eval(timestamp clientmodel.Timestamp) Vector { + return node.function.callFn(timestamp, node.args).(Vector) } func evalScalarBinop(opType BinOpType, @@ -639,11 +721,11 @@ func labelsEqual(labels1, labels2 clientmodel.Metric) bool { // Eval implements the VectorNode interface and returns the result of // the expression. -func (node *VectorArithExpr) Eval(timestamp clientmodel.Timestamp, view *viewAdapter) Vector { +func (node *VectorArithExpr) Eval(timestamp clientmodel.Timestamp) Vector { result := Vector{} if node.lhs.Type() == SCALAR && node.rhs.Type() == VECTOR { - lhs := node.lhs.(ScalarNode).Eval(timestamp, view) - rhs := node.rhs.(VectorNode).Eval(timestamp, view) + lhs := node.lhs.(ScalarNode).Eval(timestamp) + rhs := node.rhs.(VectorNode).Eval(timestamp) for _, rhsSample := range rhs { value, keep := evalVectorBinop(node.opType, lhs, rhsSample.Value) if keep { @@ -653,8 +735,8 @@ func (node *VectorArithExpr) Eval(timestamp clientmodel.Timestamp, view *viewAda } return result } else if node.lhs.Type() == VECTOR && node.rhs.Type() == SCALAR { - lhs := node.lhs.(VectorNode).Eval(timestamp, view) - rhs := node.rhs.(ScalarNode).Eval(timestamp, view) + lhs := node.lhs.(VectorNode).Eval(timestamp) + rhs := node.rhs.(ScalarNode).Eval(timestamp) for _, lhsSample := range lhs { value, keep := evalVectorBinop(node.opType, lhsSample.Value, rhs) if keep { @@ -664,8 +746,8 @@ func (node *VectorArithExpr) Eval(timestamp clientmodel.Timestamp, view *viewAda } return result } else if node.lhs.Type() == VECTOR && node.rhs.Type() == VECTOR { - lhs := node.lhs.(VectorNode).Eval(timestamp, view) - rhs := node.rhs.(VectorNode).Eval(timestamp, view) + lhs := node.lhs.(VectorNode).Eval(timestamp) + rhs := node.rhs.(VectorNode).Eval(timestamp) for _, lhsSample := range lhs { for _, rhsSample := range rhs { if labelsEqual(lhsSample.Metric, rhsSample.Metric) { @@ -684,32 +766,54 @@ func (node *VectorArithExpr) Eval(timestamp clientmodel.Timestamp, view *viewAda // Eval implements the MatrixNode interface and returns the value of // the selector. -func (node *MatrixSelector) Eval(timestamp clientmodel.Timestamp, view *viewAdapter) Matrix { +func (node *MatrixSelector) Eval(timestamp clientmodel.Timestamp) Matrix { interval := &metric.Interval{ OldestInclusive: timestamp.Add(-node.interval), NewestInclusive: timestamp, } - values, err := view.GetRangeValues(node.fingerprints, interval) - if err != nil { - glog.Error("Unable to get values for vector interval: ", err) - return Matrix{} + + //// timer := v.stats.GetTimer(stats.GetRangeValuesTime).Start() + sampleSets := []metric.SampleSet{} + for fp, it := range node.iterators { + samplePairs := it.GetRangeValues(*interval) + if len(samplePairs) == 0 { + continue + } + + sampleSet := metric.SampleSet{ + Metric: node.metrics[fp], // TODO: need copy here because downstream can modify! + Values: samplePairs, + } + sampleSets = append(sampleSets, sampleSet) } - return values + //// timer.Stop() + return sampleSets } // EvalBoundaries implements the MatrixNode interface and returns the // boundary values of the selector. -func (node *MatrixSelector) EvalBoundaries(timestamp clientmodel.Timestamp, view *viewAdapter) Matrix { +func (node *MatrixSelector) EvalBoundaries(timestamp clientmodel.Timestamp) Matrix { interval := &metric.Interval{ OldestInclusive: timestamp.Add(-node.interval), NewestInclusive: timestamp, } - values, err := view.GetBoundaryValues(node.fingerprints, interval) - if err != nil { - glog.Error("Unable to get boundary values for vector interval: ", err) - return Matrix{} + + //// timer := v.stats.GetTimer(stats.GetBoundaryValuesTime).Start() + sampleSets := []metric.SampleSet{} + for fp, it := range node.iterators { + samplePairs := it.GetBoundaryValues(*interval) + if len(samplePairs) == 0 { + continue + } + + sampleSet := metric.SampleSet{ + Metric: node.metrics[fp], // TODO: make copy of metric. + Values: samplePairs, + } + sampleSets = append(sampleSets, sampleSet) } - return values + //// timer.Stop() + return sampleSets } // Len implements sort.Interface. @@ -729,14 +833,14 @@ func (matrix Matrix) Swap(i, j int) { // Eval implements the StringNode interface and returns the value of // the selector. -func (node *StringLiteral) Eval(timestamp clientmodel.Timestamp, view *viewAdapter) string { +func (node *StringLiteral) Eval(timestamp clientmodel.Timestamp) string { return node.str } // Eval implements the StringNode interface and returns the result of // the function call. -func (node *StringFunctionCall) Eval(timestamp clientmodel.Timestamp, view *viewAdapter) string { - return node.function.callFn(timestamp, view, node.args).(string) +func (node *StringFunctionCall) Eval(timestamp clientmodel.Timestamp) string { + return node.function.callFn(timestamp, node.args).(string) } // ---------------------------------------------------------------------------- @@ -754,6 +858,8 @@ func NewScalarLiteral(value clientmodel.SampleValue) *ScalarLiteral { func NewVectorSelector(m metric.LabelMatchers) *VectorSelector { return &VectorSelector{ labelMatchers: m, + iterators: map[clientmodel.Fingerprint]storage_ng.SeriesIterator{}, + metrics: map[clientmodel.Fingerprint]clientmodel.Metric{}, } } @@ -845,6 +951,8 @@ func NewMatrixSelector(vector *VectorSelector, interval time.Duration) *MatrixSe return &MatrixSelector{ labelMatchers: vector.labelMatchers, interval: interval, + iterators: map[clientmodel.Fingerprint]storage_ng.SeriesIterator{}, + metrics: map[clientmodel.Fingerprint]clientmodel.Metric{}, } } diff --git a/rules/ast/functions.go b/rules/ast/functions.go index ccfe3e271b..562703f74d 100644 --- a/rules/ast/functions.go +++ b/rules/ast/functions.go @@ -31,7 +31,7 @@ type Function struct { name string argTypes []ExprType returnType ExprType - callFn func(timestamp clientmodel.Timestamp, view *viewAdapter, args []Node) interface{} + callFn func(timestamp clientmodel.Timestamp, args []Node) interface{} } // CheckArgTypes returns a non-nil error if the number or types of @@ -74,14 +74,14 @@ func (function *Function) CheckArgTypes(args []Node) error { } // === time() clientmodel.SampleValue === -func timeImpl(timestamp clientmodel.Timestamp, view *viewAdapter, args []Node) interface{} { +func timeImpl(timestamp clientmodel.Timestamp, args []Node) interface{} { return clientmodel.SampleValue(timestamp.Unix()) } // === delta(matrix MatrixNode, isCounter ScalarNode) Vector === -func deltaImpl(timestamp clientmodel.Timestamp, view *viewAdapter, args []Node) interface{} { +func deltaImpl(timestamp clientmodel.Timestamp, args []Node) interface{} { matrixNode := args[0].(MatrixNode) - isCounter := args[1].(ScalarNode).Eval(timestamp, view) > 0 + isCounter := args[1].(ScalarNode).Eval(timestamp) > 0 resultVector := Vector{} // If we treat these metrics as counters, we need to fetch all values @@ -89,9 +89,9 @@ func deltaImpl(timestamp clientmodel.Timestamp, view *viewAdapter, args []Node) // I.e. if a counter resets, we want to ignore that reset. var matrixValue Matrix if isCounter { - matrixValue = matrixNode.Eval(timestamp, view) + matrixValue = matrixNode.Eval(timestamp) } else { - matrixValue = matrixNode.EvalBoundaries(timestamp, view) + matrixValue = matrixNode.EvalBoundaries(timestamp) } for _, samples := range matrixValue { // No sense in trying to compute a delta without at least two points. Drop @@ -139,9 +139,9 @@ func deltaImpl(timestamp clientmodel.Timestamp, view *viewAdapter, args []Node) } // === rate(node MatrixNode) Vector === -func rateImpl(timestamp clientmodel.Timestamp, view *viewAdapter, args []Node) interface{} { +func rateImpl(timestamp clientmodel.Timestamp, args []Node) interface{} { args = append(args, &ScalarLiteral{value: 1}) - vector := deltaImpl(timestamp, view, args).(Vector) + vector := deltaImpl(timestamp, args).(Vector) // TODO: could be other type of MatrixNode in the future (right now, only // MatrixSelector exists). Find a better way of getting the duration of a @@ -188,28 +188,28 @@ func (s reverseHeap) Less(i, j int) bool { } // === sort(node VectorNode) Vector === -func sortImpl(timestamp clientmodel.Timestamp, view *viewAdapter, args []Node) interface{} { - byValueSorter := vectorByValueHeap(args[0].(VectorNode).Eval(timestamp, view)) +func sortImpl(timestamp clientmodel.Timestamp, args []Node) interface{} { + byValueSorter := vectorByValueHeap(args[0].(VectorNode).Eval(timestamp)) sort.Sort(byValueSorter) return Vector(byValueSorter) } // === sortDesc(node VectorNode) Vector === -func sortDescImpl(timestamp clientmodel.Timestamp, view *viewAdapter, args []Node) interface{} { - byValueSorter := vectorByValueHeap(args[0].(VectorNode).Eval(timestamp, view)) +func sortDescImpl(timestamp clientmodel.Timestamp, args []Node) interface{} { + byValueSorter := vectorByValueHeap(args[0].(VectorNode).Eval(timestamp)) sort.Sort(sort.Reverse(byValueSorter)) return Vector(byValueSorter) } // === topk(k ScalarNode, node VectorNode) Vector === -func topkImpl(timestamp clientmodel.Timestamp, view *viewAdapter, args []Node) interface{} { - k := int(args[0].(ScalarNode).Eval(timestamp, view)) +func topkImpl(timestamp clientmodel.Timestamp, args []Node) interface{} { + k := int(args[0].(ScalarNode).Eval(timestamp)) if k < 1 { return Vector{} } topk := make(vectorByValueHeap, 0, k) - vector := args[1].(VectorNode).Eval(timestamp, view) + vector := args[1].(VectorNode).Eval(timestamp) for _, el := range vector { if len(topk) < k || topk[0].Value < el.Value { @@ -224,15 +224,15 @@ func topkImpl(timestamp clientmodel.Timestamp, view *viewAdapter, args []Node) i } // === bottomk(k ScalarNode, node VectorNode) Vector === -func bottomkImpl(timestamp clientmodel.Timestamp, view *viewAdapter, args []Node) interface{} { - k := int(args[0].(ScalarNode).Eval(timestamp, view)) +func bottomkImpl(timestamp clientmodel.Timestamp, args []Node) interface{} { + k := int(args[0].(ScalarNode).Eval(timestamp)) if k < 1 { return Vector{} } bottomk := make(vectorByValueHeap, 0, k) bkHeap := reverseHeap{Interface: &bottomk} - vector := args[1].(VectorNode).Eval(timestamp, view) + vector := args[1].(VectorNode).Eval(timestamp) for _, el := range vector { if len(bottomk) < k || bottomk[0].Value > el.Value { @@ -247,8 +247,8 @@ func bottomkImpl(timestamp clientmodel.Timestamp, view *viewAdapter, args []Node } // === drop_common_labels(node VectorNode) Vector === -func dropCommonLabelsImpl(timestamp clientmodel.Timestamp, view *viewAdapter, args []Node) interface{} { - vector := args[0].(VectorNode).Eval(timestamp, view) +func dropCommonLabelsImpl(timestamp clientmodel.Timestamp, args []Node) interface{} { + vector := args[0].(VectorNode).Eval(timestamp) if len(vector) < 1 { return Vector{} } @@ -285,7 +285,7 @@ func dropCommonLabelsImpl(timestamp clientmodel.Timestamp, view *viewAdapter, ar } // === sampleVectorImpl() Vector === -func sampleVectorImpl(timestamp clientmodel.Timestamp, view *viewAdapter, args []Node) interface{} { +func sampleVectorImpl(timestamp clientmodel.Timestamp, args []Node) interface{} { return Vector{ &clientmodel.Sample{ Metric: clientmodel.Metric{ @@ -358,8 +358,8 @@ func sampleVectorImpl(timestamp clientmodel.Timestamp, view *viewAdapter, args [ } // === scalar(node VectorNode) Scalar === -func scalarImpl(timestamp clientmodel.Timestamp, view *viewAdapter, args []Node) interface{} { - v := args[0].(VectorNode).Eval(timestamp, view) +func scalarImpl(timestamp clientmodel.Timestamp, args []Node) interface{} { + v := args[0].(VectorNode).Eval(timestamp) if len(v) != 1 { return clientmodel.SampleValue(math.NaN()) } @@ -367,13 +367,13 @@ func scalarImpl(timestamp clientmodel.Timestamp, view *viewAdapter, args []Node) } // === count_scalar(vector VectorNode) model.SampleValue === -func countScalarImpl(timestamp clientmodel.Timestamp, view *viewAdapter, args []Node) interface{} { - return clientmodel.SampleValue(len(args[0].(VectorNode).Eval(timestamp, view))) +func countScalarImpl(timestamp clientmodel.Timestamp, args []Node) interface{} { + return clientmodel.SampleValue(len(args[0].(VectorNode).Eval(timestamp))) } -func aggrOverTime(timestamp clientmodel.Timestamp, view *viewAdapter, args []Node, aggrFn func(metric.Values) clientmodel.SampleValue) interface{} { +func aggrOverTime(timestamp clientmodel.Timestamp, args []Node, aggrFn func(metric.Values) clientmodel.SampleValue) interface{} { n := args[0].(MatrixNode) - matrixVal := n.Eval(timestamp, view) + matrixVal := n.Eval(timestamp) resultVector := Vector{} for _, el := range matrixVal { @@ -391,8 +391,8 @@ func aggrOverTime(timestamp clientmodel.Timestamp, view *viewAdapter, args []Nod } // === avg_over_time(matrix MatrixNode) Vector === -func avgOverTimeImpl(timestamp clientmodel.Timestamp, view *viewAdapter, args []Node) interface{} { - return aggrOverTime(timestamp, view, args, func(values metric.Values) clientmodel.SampleValue { +func avgOverTimeImpl(timestamp clientmodel.Timestamp, args []Node) interface{} { + return aggrOverTime(timestamp, args, func(values metric.Values) clientmodel.SampleValue { var sum clientmodel.SampleValue for _, v := range values { sum += v.Value @@ -402,15 +402,15 @@ func avgOverTimeImpl(timestamp clientmodel.Timestamp, view *viewAdapter, args [] } // === count_over_time(matrix MatrixNode) Vector === -func countOverTimeImpl(timestamp clientmodel.Timestamp, view *viewAdapter, args []Node) interface{} { - return aggrOverTime(timestamp, view, args, func(values metric.Values) clientmodel.SampleValue { +func countOverTimeImpl(timestamp clientmodel.Timestamp, args []Node) interface{} { + return aggrOverTime(timestamp, args, func(values metric.Values) clientmodel.SampleValue { return clientmodel.SampleValue(len(values)) }) } // === max_over_time(matrix MatrixNode) Vector === -func maxOverTimeImpl(timestamp clientmodel.Timestamp, view *viewAdapter, args []Node) interface{} { - return aggrOverTime(timestamp, view, args, func(values metric.Values) clientmodel.SampleValue { +func maxOverTimeImpl(timestamp clientmodel.Timestamp, args []Node) interface{} { + return aggrOverTime(timestamp, args, func(values metric.Values) clientmodel.SampleValue { max := math.Inf(-1) for _, v := range values { max = math.Max(max, float64(v.Value)) @@ -420,8 +420,8 @@ func maxOverTimeImpl(timestamp clientmodel.Timestamp, view *viewAdapter, args [] } // === min_over_time(matrix MatrixNode) Vector === -func minOverTimeImpl(timestamp clientmodel.Timestamp, view *viewAdapter, args []Node) interface{} { - return aggrOverTime(timestamp, view, args, func(values metric.Values) clientmodel.SampleValue { +func minOverTimeImpl(timestamp clientmodel.Timestamp, args []Node) interface{} { + return aggrOverTime(timestamp, args, func(values metric.Values) clientmodel.SampleValue { min := math.Inf(1) for _, v := range values { min = math.Min(min, float64(v.Value)) @@ -431,8 +431,8 @@ func minOverTimeImpl(timestamp clientmodel.Timestamp, view *viewAdapter, args [] } // === sum_over_time(matrix MatrixNode) Vector === -func sumOverTimeImpl(timestamp clientmodel.Timestamp, view *viewAdapter, args []Node) interface{} { - return aggrOverTime(timestamp, view, args, func(values metric.Values) clientmodel.SampleValue { +func sumOverTimeImpl(timestamp clientmodel.Timestamp, args []Node) interface{} { + return aggrOverTime(timestamp, args, func(values metric.Values) clientmodel.SampleValue { var sum clientmodel.SampleValue for _, v := range values { sum += v.Value @@ -442,9 +442,9 @@ func sumOverTimeImpl(timestamp clientmodel.Timestamp, view *viewAdapter, args [] } // === abs(vector VectorNode) Vector === -func absImpl(timestamp clientmodel.Timestamp, view *viewAdapter, args []Node) interface{} { +func absImpl(timestamp clientmodel.Timestamp, args []Node) interface{} { n := args[0].(VectorNode) - vector := n.Eval(timestamp, view) + vector := n.Eval(timestamp) for _, el := range vector { el.Value = clientmodel.SampleValue(math.Abs(float64(el.Value))) } diff --git a/rules/ast/functions_test.go b/rules/ast/functions_test.go index 3392f63b75..fa4d1da717 100644 --- a/rules/ast/functions_test.go +++ b/rules/ast/functions_test.go @@ -28,7 +28,7 @@ func (node emptyRangeNode) NodeTreeToDotGraph() string { return "" } func (node emptyRangeNode) String() string { return "" } func (node emptyRangeNode) Children() Nodes { return Nodes{} } -func (node emptyRangeNode) Eval(timestamp clientmodel.Timestamp, view *viewAdapter) Matrix { +func (node emptyRangeNode) Eval(timestamp clientmodel.Timestamp) Matrix { return Matrix{ metric.SampleSet{ Metric: clientmodel.Metric{clientmodel.MetricNameLabel: "empty_metric"}, @@ -37,7 +37,7 @@ func (node emptyRangeNode) Eval(timestamp clientmodel.Timestamp, view *viewAdapt } } -func (node emptyRangeNode) EvalBoundaries(timestamp clientmodel.Timestamp, view *viewAdapter) Matrix { +func (node emptyRangeNode) EvalBoundaries(timestamp clientmodel.Timestamp) Matrix { return Matrix{ metric.SampleSet{ Metric: clientmodel.Metric{clientmodel.MetricNameLabel: "empty_metric"}, @@ -48,11 +48,11 @@ func (node emptyRangeNode) EvalBoundaries(timestamp clientmodel.Timestamp, view func TestDeltaWithEmptyElementDoesNotCrash(t *testing.T) { now := clientmodel.Now() - vector := deltaImpl(now, nil, []Node{emptyRangeNode{}, &ScalarLiteral{value: 0}}).(Vector) + vector := deltaImpl(now, []Node{emptyRangeNode{}, &ScalarLiteral{value: 0}}).(Vector) if len(vector) != 0 { t.Fatalf("Expected empty result vector, got: %v", vector) } - vector = deltaImpl(now, nil, []Node{emptyRangeNode{}, &ScalarLiteral{value: 1}}).(Vector) + vector = deltaImpl(now, []Node{emptyRangeNode{}, &ScalarLiteral{value: 1}}).(Vector) if len(vector) != 0 { t.Fatalf("Expected empty result vector, got: %v", vector) } diff --git a/rules/ast/printer.go b/rules/ast/printer.go index 0fefcdd1cc..f90fc245aa 100644 --- a/rules/ast/printer.go +++ b/rules/ast/printer.go @@ -23,7 +23,7 @@ import ( clientmodel "github.com/prometheus/client_golang/model" "github.com/prometheus/prometheus/stats" - "github.com/prometheus/prometheus/storage/metric" + "github.com/prometheus/prometheus/storage/local" "github.com/prometheus/prometheus/utility" ) @@ -151,18 +151,19 @@ func TypedValueToJSON(data interface{}, typeStr string) string { } // EvalToString evaluates the given node into a string of the given format. -func EvalToString(node Node, timestamp clientmodel.Timestamp, format OutputFormat, storage metric.PreloadingPersistence, queryStats *stats.TimerGroup) string { - viewTimer := queryStats.GetTimer(stats.TotalViewBuildingTime).Start() - viewAdapter, err := viewAdapterForInstantQuery(node, timestamp, storage, queryStats) - viewTimer.Stop() +func EvalToString(node Node, timestamp clientmodel.Timestamp, format OutputFormat, storage storage_ng.Storage, queryStats *stats.TimerGroup) string { + prepareTimer := queryStats.GetTimer(stats.TotalQueryPreparationTime).Start() + closer, err := prepareInstantQuery(node, timestamp, storage, queryStats) + prepareTimer.Stop() if err != nil { panic(err) } + defer closer.Close() evalTimer := queryStats.GetTimer(stats.InnerEvalTime).Start() switch node.Type() { case SCALAR: - scalar := node.(ScalarNode).Eval(timestamp, viewAdapter) + scalar := node.(ScalarNode).Eval(timestamp) evalTimer.Stop() switch format { case TEXT: @@ -171,7 +172,7 @@ func EvalToString(node Node, timestamp clientmodel.Timestamp, format OutputForma return TypedValueToJSON(scalar, "scalar") } case VECTOR: - vector := node.(VectorNode).Eval(timestamp, viewAdapter) + vector := node.(VectorNode).Eval(timestamp) evalTimer.Stop() switch format { case TEXT: @@ -180,7 +181,7 @@ func EvalToString(node Node, timestamp clientmodel.Timestamp, format OutputForma return TypedValueToJSON(vector, "vector") } case MATRIX: - matrix := node.(MatrixNode).Eval(timestamp, viewAdapter) + matrix := node.(MatrixNode).Eval(timestamp) evalTimer.Stop() switch format { case TEXT: @@ -189,7 +190,7 @@ func EvalToString(node Node, timestamp clientmodel.Timestamp, format OutputForma return TypedValueToJSON(matrix, "matrix") } case STRING: - str := node.(StringNode).Eval(timestamp, viewAdapter) + str := node.(StringNode).Eval(timestamp) evalTimer.Stop() switch format { case TEXT: @@ -202,28 +203,29 @@ func EvalToString(node Node, timestamp clientmodel.Timestamp, format OutputForma } // EvalToVector evaluates the given node into a Vector. Matrices aren't supported. -func EvalToVector(node Node, timestamp clientmodel.Timestamp, storage metric.PreloadingPersistence, queryStats *stats.TimerGroup) (Vector, error) { - viewTimer := queryStats.GetTimer(stats.TotalViewBuildingTime).Start() - viewAdapter, err := viewAdapterForInstantQuery(node, timestamp, storage, queryStats) - viewTimer.Stop() +func EvalToVector(node Node, timestamp clientmodel.Timestamp, storage storage_ng.Storage, queryStats *stats.TimerGroup) (Vector, error) { + prepareTimer := queryStats.GetTimer(stats.TotalQueryPreparationTime).Start() + closer, err := prepareInstantQuery(node, timestamp, storage, queryStats) + prepareTimer.Stop() if err != nil { panic(err) } + defer closer.Close() evalTimer := queryStats.GetTimer(stats.InnerEvalTime).Start() switch node.Type() { case SCALAR: - scalar := node.(ScalarNode).Eval(timestamp, viewAdapter) + scalar := node.(ScalarNode).Eval(timestamp) evalTimer.Stop() return Vector{&clientmodel.Sample{Value: scalar}}, nil case VECTOR: - vector := node.(VectorNode).Eval(timestamp, viewAdapter) + vector := node.(VectorNode).Eval(timestamp) evalTimer.Stop() return vector, nil case MATRIX: return nil, errors.New("Matrices not supported by EvalToVector") case STRING: - str := node.(StringNode).Eval(timestamp, viewAdapter) + str := node.(StringNode).Eval(timestamp) evalTimer.Stop() return Vector{&clientmodel.Sample{ Metric: clientmodel.Metric{"__value__": clientmodel.LabelValue(str)}}}, nil diff --git a/rules/ast/query_analyzer.go b/rules/ast/query_analyzer.go index 8d909ba595..bab00c5008 100644 --- a/rules/ast/query_analyzer.go +++ b/rules/ast/query_analyzer.go @@ -16,12 +16,10 @@ package ast import ( "time" - "github.com/golang/glog" - clientmodel "github.com/prometheus/client_golang/model" "github.com/prometheus/prometheus/stats" - "github.com/prometheus/prometheus/storage/metric" + "github.com/prometheus/prometheus/storage/local" ) // FullRangeMap maps the fingerprint of a full range to the duration @@ -48,13 +46,13 @@ type QueryAnalyzer struct { IntervalRanges IntervalRangeMap // The underlying storage to which the query will be applied. Needed for // extracting timeseries fingerprint information during query analysis. - storage metric.Persistence + storage storage_ng.Storage } // NewQueryAnalyzer returns a pointer to a newly instantiated // QueryAnalyzer. The storage is needed to extract timeseries // fingerprint information during query analysis. -func NewQueryAnalyzer(storage metric.Persistence) *QueryAnalyzer { +func NewQueryAnalyzer(storage storage_ng.Storage) *QueryAnalyzer { return &QueryAnalyzer{ FullRanges: FullRangeMap{}, IntervalRanges: IntervalRangeMap{}, @@ -66,94 +64,122 @@ func NewQueryAnalyzer(storage metric.Persistence) *QueryAnalyzer { func (analyzer *QueryAnalyzer) Visit(node Node) { switch n := node.(type) { case *VectorSelector: - fingerprints, err := analyzer.storage.GetFingerprintsForLabelMatchers(n.labelMatchers) - if err != nil { - glog.Errorf("Error getting fingerprints for label matchers %v: %v", n.labelMatchers, err) - return - } + fingerprints := analyzer.storage.GetFingerprintsForLabelMatchers(n.labelMatchers) n.fingerprints = fingerprints - for _, fingerprint := range fingerprints { + for _, fp := range fingerprints { // Only add the fingerprint to IntervalRanges if not yet present in FullRanges. // Full ranges always contain more points and span more time than interval ranges. - if _, alreadyInFullRanges := analyzer.FullRanges[*fingerprint]; !alreadyInFullRanges { - analyzer.IntervalRanges[*fingerprint] = true + if _, alreadyInFullRanges := analyzer.FullRanges[fp]; !alreadyInFullRanges { + analyzer.IntervalRanges[fp] = true } + + n.metrics[fp] = analyzer.storage.GetMetricForFingerprint(fp) } case *MatrixSelector: - fingerprints, err := analyzer.storage.GetFingerprintsForLabelMatchers(n.labelMatchers) - if err != nil { - glog.Errorf("Error getting fingerprints for label matchers %v: %v", n.labelMatchers, err) - return - } + fingerprints := analyzer.storage.GetFingerprintsForLabelMatchers(n.labelMatchers) n.fingerprints = fingerprints - for _, fingerprint := range fingerprints { - if analyzer.FullRanges[*fingerprint] < n.interval { - analyzer.FullRanges[*fingerprint] = n.interval + for _, fp := range fingerprints { + if analyzer.FullRanges[fp] < n.interval { + analyzer.FullRanges[fp] = n.interval // Delete the fingerprint from IntervalRanges. Full ranges always contain // more points and span more time than interval ranges, so we don't need // an interval range for the same fingerprint, should we have one. - delete(analyzer.IntervalRanges, *fingerprint) + delete(analyzer.IntervalRanges, fp) } + + n.metrics[fp] = analyzer.storage.GetMetricForFingerprint(fp) } } } -// AnalyzeQueries walks the AST, starting at node, calling Visit on -// each node to collect fingerprints. -func (analyzer *QueryAnalyzer) AnalyzeQueries(node Node) { +type iteratorInitializer struct { + storage storage_ng.Storage +} + +func (i *iteratorInitializer) Visit(node Node) { + switch n := node.(type) { + case *VectorSelector: + for _, fp := range n.fingerprints { + n.iterators[fp] = i.storage.NewIterator(fp) + } + case *MatrixSelector: + for _, fp := range n.fingerprints { + n.iterators[fp] = i.storage.NewIterator(fp) + } + } +} + +func prepareInstantQuery(node Node, timestamp clientmodel.Timestamp, storage storage_ng.Storage, queryStats *stats.TimerGroup) (storage_ng.Closer, error) { + analyzeTimer := queryStats.GetTimer(stats.QueryAnalysisTime).Start() + analyzer := NewQueryAnalyzer(storage) Walk(analyzer, node) -} - -func viewAdapterForInstantQuery(node Node, timestamp clientmodel.Timestamp, storage metric.PreloadingPersistence, queryStats *stats.TimerGroup) (*viewAdapter, error) { - analyzeTimer := queryStats.GetTimer(stats.QueryAnalysisTime).Start() - analyzer := NewQueryAnalyzer(storage) - analyzer.AnalyzeQueries(node) analyzeTimer.Stop() - requestBuildTimer := queryStats.GetTimer(stats.ViewRequestBuildTime).Start() - viewBuilder := storage.NewViewRequestBuilder() - for fingerprint, rangeDuration := range analyzer.FullRanges { - viewBuilder.GetMetricRange(&fingerprint, timestamp.Add(-rangeDuration), timestamp) - } - for fingerprint := range analyzer.IntervalRanges { - viewBuilder.GetMetricAtTime(&fingerprint, timestamp) - } - requestBuildTimer.Stop() - - buildTimer := queryStats.GetTimer(stats.InnerViewBuildingTime).Start() - view, err := viewBuilder.Execute(60*time.Second, queryStats) - buildTimer.Stop() - if err != nil { - return nil, err - } - return NewViewAdapter(view, storage, queryStats), nil -} - -func viewAdapterForRangeQuery(node Node, start clientmodel.Timestamp, end clientmodel.Timestamp, interval time.Duration, storage metric.PreloadingPersistence, queryStats *stats.TimerGroup) (*viewAdapter, error) { - analyzeTimer := queryStats.GetTimer(stats.QueryAnalysisTime).Start() - analyzer := NewQueryAnalyzer(storage) - analyzer.AnalyzeQueries(node) - analyzeTimer.Stop() - - requestBuildTimer := queryStats.GetTimer(stats.ViewRequestBuildTime).Start() - viewBuilder := storage.NewViewRequestBuilder() - for fingerprint, rangeDuration := range analyzer.FullRanges { - if interval < rangeDuration { - viewBuilder.GetMetricRange(&fingerprint, start.Add(-rangeDuration), end) - } else { - viewBuilder.GetMetricRangeAtInterval(&fingerprint, start.Add(-rangeDuration), end, interval, rangeDuration) + // TODO: Preloading should time out after a given duration. + preloadTimer := queryStats.GetTimer(stats.PreloadTime).Start() + p := storage.NewPreloader() + for fp, rangeDuration := range analyzer.FullRanges { + if err := p.PreloadRange(fp, timestamp.Add(-rangeDuration), timestamp); err != nil { + p.Close() + return nil, err } } - for fingerprint := range analyzer.IntervalRanges { - viewBuilder.GetMetricAtInterval(&fingerprint, start, end, interval) + for fp := range analyzer.IntervalRanges { + if err := p.PreloadRange(fp, timestamp, timestamp); err != nil { + p.Close() + return nil, err + } } - requestBuildTimer.Stop() + preloadTimer.Stop() - buildTimer := queryStats.GetTimer(stats.InnerViewBuildingTime).Start() - view, err := viewBuilder.Execute(time.Duration(60)*time.Second, queryStats) - buildTimer.Stop() - if err != nil { - return nil, err + ii := &iteratorInitializer{ + storage: storage, } - return NewViewAdapter(view, storage, queryStats), nil + Walk(ii, node) + + return p, nil +} + +func prepareRangeQuery(node Node, start clientmodel.Timestamp, end clientmodel.Timestamp, interval time.Duration, storage storage_ng.Storage, queryStats *stats.TimerGroup) (storage_ng.Closer, error) { + analyzeTimer := queryStats.GetTimer(stats.QueryAnalysisTime).Start() + analyzer := NewQueryAnalyzer(storage) + Walk(analyzer, node) + analyzeTimer.Stop() + + // TODO: Preloading should time out after a given duration. + preloadTimer := queryStats.GetTimer(stats.PreloadTime).Start() + p := storage.NewPreloader() + for fp, rangeDuration := range analyzer.FullRanges { + if err := p.PreloadRange(fp, start.Add(-rangeDuration), end); err != nil { + p.Close() + return nil, err + } + /* + if interval < rangeDuration { + if err := p.GetMetricRange(fp, end, end.Sub(start)+rangeDuration); err != nil { + p.Close() + return nil, err + } + } else { + if err := p.GetMetricRangeAtInterval(fp, start, end, interval, rangeDuration); err != nil { + p.Close() + return nil, err + } + } + */ + } + for fp := range analyzer.IntervalRanges { + if err := p.PreloadRange(fp, start, end); err != nil { + p.Close() + return nil, err + } + } + preloadTimer.Stop() + + ii := &iteratorInitializer{ + storage: storage, + } + Walk(ii, node) + + return p, nil } diff --git a/rules/ast/view_adapter.go b/rules/ast/view_adapter.go index ca16c490a3..dc18611e8a 100644 --- a/rules/ast/view_adapter.go +++ b/rules/ast/view_adapter.go @@ -13,186 +13,4 @@ package ast -import ( - "flag" - "time" - - clientmodel "github.com/prometheus/client_golang/model" - - "github.com/prometheus/prometheus/stats" - "github.com/prometheus/prometheus/storage/metric" -) - -var defaultStalenessDelta = flag.Int("defaultStalenessDelta", 300, "Default staleness delta allowance in seconds during expression evaluations.") - -// StalenessPolicy describes the lenience limits to apply to values -// from the materialized view. -type StalenessPolicy struct { - // Describes the inclusive limit at which individual points if requested will - // be matched and subject to interpolation. - DeltaAllowance time.Duration -} - -type viewAdapter struct { - // Policy that dictates when sample values around an evaluation time are to - // be interpreted as stale. - stalenessPolicy StalenessPolicy - // AST-global storage to use for operations that are not supported by views - // (i.e. fingerprint->metric lookups). - storage metric.Persistence - // The materialized view which contains all timeseries data required for - // executing a query. - view metric.View - // The TimerGroup object in which to capture query timing statistics. - stats *stats.TimerGroup -} - -// interpolateSamples interpolates a value at a target time between two -// provided sample pairs. -func interpolateSamples(first, second *metric.SamplePair, timestamp clientmodel.Timestamp) *metric.SamplePair { - dv := second.Value - first.Value - dt := second.Timestamp.Sub(first.Timestamp) - - dDt := dv / clientmodel.SampleValue(dt) - offset := clientmodel.SampleValue(timestamp.Sub(first.Timestamp)) - - return &metric.SamplePair{ - Value: first.Value + (offset * dDt), - Timestamp: timestamp, - } -} - -// chooseClosestSample chooses the closest sample of a list of samples -// surrounding a given target time. If samples are found both before and after -// the target time, the sample value is interpolated between these. Otherwise, -// the single closest sample is returned verbatim. -func (v *viewAdapter) chooseClosestSample(samples metric.Values, timestamp clientmodel.Timestamp) *metric.SamplePair { - var closestBefore *metric.SamplePair - var closestAfter *metric.SamplePair - for _, candidate := range samples { - delta := candidate.Timestamp.Sub(timestamp) - // Samples before target time. - if delta < 0 { - // Ignore samples outside of staleness policy window. - if -delta > v.stalenessPolicy.DeltaAllowance { - continue - } - // Ignore samples that are farther away than what we've seen before. - if closestBefore != nil && candidate.Timestamp.Before(closestBefore.Timestamp) { - continue - } - sample := candidate - closestBefore = &sample - } - - // Samples after target time. - if delta >= 0 { - // Ignore samples outside of staleness policy window. - if delta > v.stalenessPolicy.DeltaAllowance { - continue - } - // Ignore samples that are farther away than samples we've seen before. - if closestAfter != nil && candidate.Timestamp.After(closestAfter.Timestamp) { - continue - } - sample := candidate - closestAfter = &sample - } - } - - switch { - case closestBefore != nil && closestAfter != nil: - return interpolateSamples(closestBefore, closestAfter, timestamp) - case closestBefore != nil: - return closestBefore - default: - return closestAfter - } -} - -func (v *viewAdapter) GetValueAtTime(fingerprints clientmodel.Fingerprints, timestamp clientmodel.Timestamp) (Vector, error) { - timer := v.stats.GetTimer(stats.GetValueAtTimeTime).Start() - samples := Vector{} - for _, fingerprint := range fingerprints { - sampleCandidates := v.view.GetValueAtTime(fingerprint, timestamp) - samplePair := v.chooseClosestSample(sampleCandidates, timestamp) - m, err := v.storage.GetMetricForFingerprint(fingerprint) - if err != nil { - return nil, err - } - if samplePair != nil { - samples = append(samples, &clientmodel.Sample{ - Metric: m, - Value: samplePair.Value, - Timestamp: timestamp, - }) - } - } - timer.Stop() - return samples, nil -} - -func (v *viewAdapter) GetBoundaryValues(fingerprints clientmodel.Fingerprints, interval *metric.Interval) ([]metric.SampleSet, error) { - timer := v.stats.GetTimer(stats.GetBoundaryValuesTime).Start() - sampleSets := []metric.SampleSet{} - for _, fingerprint := range fingerprints { - samplePairs := v.view.GetBoundaryValues(fingerprint, *interval) - if len(samplePairs) == 0 { - continue - } - - // TODO: memoize/cache this. - m, err := v.storage.GetMetricForFingerprint(fingerprint) - if err != nil { - return nil, err - } - - sampleSet := metric.SampleSet{ - Metric: m, - Values: samplePairs, - } - sampleSets = append(sampleSets, sampleSet) - } - timer.Stop() - return sampleSets, nil -} - -func (v *viewAdapter) GetRangeValues(fingerprints clientmodel.Fingerprints, interval *metric.Interval) ([]metric.SampleSet, error) { - timer := v.stats.GetTimer(stats.GetRangeValuesTime).Start() - sampleSets := []metric.SampleSet{} - for _, fingerprint := range fingerprints { - samplePairs := v.view.GetRangeValues(fingerprint, *interval) - if len(samplePairs) == 0 { - continue - } - - // TODO: memoize/cache this. - m, err := v.storage.GetMetricForFingerprint(fingerprint) - if err != nil { - return nil, err - } - - sampleSet := metric.SampleSet{ - Metric: m, - Values: samplePairs, - } - sampleSets = append(sampleSets, sampleSet) - } - timer.Stop() - return sampleSets, nil -} - -// NewViewAdapter returns an initialized view adapter with a default -// staleness policy (based on the --defaultStalenessDelta flag). -func NewViewAdapter(view metric.View, storage metric.Persistence, queryStats *stats.TimerGroup) *viewAdapter { - stalenessPolicy := StalenessPolicy{ - DeltaAllowance: time.Duration(*defaultStalenessDelta) * time.Second, - } - - return &viewAdapter{ - stalenessPolicy: stalenessPolicy, - storage: storage, - view: view, - stats: queryStats, - } -} +// TODO: remove file. diff --git a/rules/helpers_test.go b/rules/helpers_test.go index 02d7aca9a0..69836978d3 100644 --- a/rules/helpers_test.go +++ b/rules/helpers_test.go @@ -19,6 +19,7 @@ import ( clientmodel "github.com/prometheus/client_golang/model" "github.com/prometheus/prometheus/rules/ast" + "github.com/prometheus/prometheus/storage/local" "github.com/prometheus/prometheus/storage/metric" ) @@ -51,7 +52,7 @@ func getTestVectorFromTestMatrix(matrix ast.Matrix) ast.Vector { return vector } -func storeMatrix(storage metric.Persistence, matrix ast.Matrix) (err error) { +func storeMatrix(storage storage_ng.Storage, matrix ast.Matrix) { pendingSamples := clientmodel.Samples{} for _, sampleSet := range matrix { for _, sample := range sampleSet.Values { @@ -62,8 +63,7 @@ func storeMatrix(storage metric.Persistence, matrix ast.Matrix) (err error) { }) } } - err = storage.AppendSamples(pendingSamples) - return + storage.AppendSamples(pendingSamples) } var testMatrix = ast.Matrix{ diff --git a/rules/manager/manager.go b/rules/manager/manager.go index e1917454d7..a7b22f0781 100644 --- a/rules/manager/manager.go +++ b/rules/manager/manager.go @@ -27,7 +27,7 @@ import ( "github.com/prometheus/prometheus/config" "github.com/prometheus/prometheus/notification" "github.com/prometheus/prometheus/rules" - "github.com/prometheus/prometheus/storage/metric" + "github.com/prometheus/prometheus/storage/local" "github.com/prometheus/prometheus/templates" ) @@ -83,7 +83,7 @@ type ruleManager struct { done chan bool interval time.Duration - storage metric.PreloadingPersistence + storage storage_ng.Storage results chan<- *extraction.Result notifications chan<- notification.NotificationReqs @@ -93,7 +93,7 @@ type ruleManager struct { type RuleManagerOptions struct { EvaluationInterval time.Duration - Storage metric.PreloadingPersistence + Storage storage_ng.Storage Notifications chan<- notification.NotificationReqs Results chan<- *extraction.Result @@ -126,17 +126,14 @@ func (m *ruleManager) Run() { m.runIteration(m.results) iterationDuration.Observe(float64(time.Since(start) / time.Millisecond)) case <-m.done: - glog.Info("rules.Rule manager exiting...") + glog.Info("Rule manager exiting...") return } } } func (m *ruleManager) Stop() { - select { - case m.done <- true: - default: - } + m.done <- true } func (m *ruleManager) queueAlertNotifications(rule *rules.AlertingRule, timestamp clientmodel.Timestamp) { diff --git a/rules/recording.go b/rules/recording.go index 9f5695892c..8a259b30c5 100644 --- a/rules/recording.go +++ b/rules/recording.go @@ -21,7 +21,7 @@ import ( "github.com/prometheus/prometheus/rules/ast" "github.com/prometheus/prometheus/stats" - "github.com/prometheus/prometheus/storage/metric" + "github.com/prometheus/prometheus/storage/local" ) // A RecordingRule records its vector expression into new timeseries. @@ -34,11 +34,11 @@ type RecordingRule struct { func (rule RecordingRule) Name() string { return rule.name } -func (rule RecordingRule) EvalRaw(timestamp clientmodel.Timestamp, storage metric.PreloadingPersistence) (ast.Vector, error) { +func (rule RecordingRule) EvalRaw(timestamp clientmodel.Timestamp, storage storage_ng.Storage) (ast.Vector, error) { return ast.EvalVectorInstant(rule.vector, timestamp, storage, stats.NewTimerGroup()) } -func (rule RecordingRule) Eval(timestamp clientmodel.Timestamp, storage metric.PreloadingPersistence) (ast.Vector, error) { +func (rule RecordingRule) Eval(timestamp clientmodel.Timestamp, storage storage_ng.Storage) (ast.Vector, error) { // Get the raw value of the rule expression. vector, err := rule.EvalRaw(timestamp, storage) if err != nil { diff --git a/rules/rules.go b/rules/rules.go index 0c624df30c..b959b9a334 100644 --- a/rules/rules.go +++ b/rules/rules.go @@ -19,7 +19,7 @@ import ( clientmodel "github.com/prometheus/client_golang/model" "github.com/prometheus/prometheus/rules/ast" - "github.com/prometheus/prometheus/storage/metric" + "github.com/prometheus/prometheus/storage/local" ) // A Rule encapsulates a vector expression which is evaluated at a specified @@ -29,9 +29,9 @@ type Rule interface { Name() string // EvalRaw evaluates the rule's vector expression without triggering any // other actions, like recording or alerting. - EvalRaw(timestamp clientmodel.Timestamp, storage metric.PreloadingPersistence) (ast.Vector, error) + EvalRaw(timestamp clientmodel.Timestamp, storage storage_ng.Storage) (ast.Vector, error) // Eval evaluates the rule, including any associated recording or alerting actions. - Eval(timestamp clientmodel.Timestamp, storage metric.PreloadingPersistence) (ast.Vector, error) + Eval(timestamp clientmodel.Timestamp, storage storage_ng.Storage) (ast.Vector, error) // ToDotGraph returns a Graphviz dot graph of the rule. ToDotGraph() string // String returns a human-readable string representation of the rule. diff --git a/rules/rules_test.go b/rules/rules_test.go index d05919d2d4..083f6dd094 100644 --- a/rules/rules_test.go +++ b/rules/rules_test.go @@ -24,7 +24,7 @@ import ( "github.com/prometheus/prometheus/rules/ast" "github.com/prometheus/prometheus/stats" - "github.com/prometheus/prometheus/storage/metric/tiered" + "github.com/prometheus/prometheus/storage/local" "github.com/prometheus/prometheus/utility/test" ) @@ -52,23 +52,10 @@ func vectorComparisonString(expected []string, actual []string) string { separator) } -type testTieredStorageCloser struct { - storage *tiered.TieredStorage - directory test.Closer -} - -func (t testTieredStorageCloser) Close() { - t.storage.Close() - t.directory.Close() -} - -func newTestStorage(t testing.TB) (storage *tiered.TieredStorage, closer test.Closer) { - storage, closer = tiered.NewTestTieredStorage(t) - if storage == nil { - t.Fatal("storage == nil") - } +func newTestStorage(t testing.TB) (storage storage_ng.Storage, closer test.Closer) { + storage, closer = storage_ng.NewTestStorage(t) storeMatrix(storage, testMatrix) - return + return storage, closer } func TestExpressions(t *testing.T) { @@ -551,9 +538,8 @@ func TestExpressions(t *testing.T) { }, } - tieredStorage, closer := newTestStorage(t) + storage, closer := newTestStorage(t) defer closer.Close() - tieredStorage.Flush() for i, exprTest := range expressionTests { expectedLines := annotateWithTime(exprTest.output, testEvalTime) @@ -571,7 +557,7 @@ func TestExpressions(t *testing.T) { t.Errorf("%d. Test should fail, but didn't", i) } failed := false - resultStr := ast.EvalToString(testExpr, testEvalTime, ast.TEXT, tieredStorage, stats.NewTimerGroup()) + resultStr := ast.EvalToString(testExpr, testEvalTime, ast.TEXT, storage, stats.NewTimerGroup()) resultLines := strings.Split(resultStr, "\n") if len(exprTest.output) != len(resultLines) { @@ -601,8 +587,8 @@ func TestExpressions(t *testing.T) { } } - analyzer := ast.NewQueryAnalyzer(tieredStorage) - analyzer.AnalyzeQueries(testExpr) + analyzer := ast.NewQueryAnalyzer(storage) + ast.Walk(analyzer, testExpr) if exprTest.fullRanges != len(analyzer.FullRanges) { t.Errorf("%d. Count of full ranges didn't match: %v vs %v", i, exprTest.fullRanges, len(analyzer.FullRanges)) failed = true @@ -711,9 +697,8 @@ func TestAlertingRule(t *testing.T) { }, } - tieredStorage, closer := newTestStorage(t) + storage, closer := newTestStorage(t) defer closer.Close() - tieredStorage.Flush() alertExpr, err := LoadExprFromString(`http_requests{group="canary", job="app-server"} < 100`) if err != nil { @@ -727,7 +712,7 @@ func TestAlertingRule(t *testing.T) { for i, expected := range evalOutputs { evalTime := testStartTime.Add(testSampleInterval * time.Duration(i)) - actual, err := rule.Eval(evalTime, tieredStorage) + actual, err := rule.Eval(evalTime, storage) if err != nil { t.Fatalf("Error during alerting rule evaluation: %s", err) } diff --git a/stats/query_stats.go b/stats/query_stats.go index 4fa73cac09..b838a2b7d7 100644 --- a/stats/query_stats.go +++ b/stats/query_stats.go @@ -22,8 +22,8 @@ const ( TotalEvalTime QueryTiming = iota ResultSortTime JsonEncodeTime - TotalViewBuildingTime - ViewRequestBuildTime + PreloadTime + TotalQueryPreparationTime InnerViewBuildingTime InnerEvalTime ResultAppendTime @@ -46,10 +46,10 @@ func (s QueryTiming) String() string { return "Result sorting time" case JsonEncodeTime: return "JSON encoding time" - case TotalViewBuildingTime: - return "Total view building time" - case ViewRequestBuildTime: - return "View request building time" + case PreloadTime: + return "Query preloading time" + case TotalQueryPreparationTime: + return "Total query preparation time" case InnerViewBuildingTime: return "Inner view building time" case InnerEvalTime: diff --git a/storage/interface.go b/storage/interface.go deleted file mode 100644 index 36dca39dab..0000000000 --- a/storage/interface.go +++ /dev/null @@ -1,76 +0,0 @@ -// Copyright 2013 Prometheus Team -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -package storage - -// RecordDecoder decodes each key-value pair in the database. The protocol -// around it makes the assumption that the underlying implementation is -// concurrency safe. -type RecordDecoder interface { - DecodeKey(in interface{}) (out interface{}, err error) - DecodeValue(in interface{}) (out interface{}, err error) -} - -// FilterResult describes the record matching and scanning behavior for the -// database. -type FilterResult int - -const ( - // Stop scanning the database. - Stop FilterResult = iota - // Skip this record but continue scanning. - Skip - // Accept this record for the Operator. - Accept -) - -func (f FilterResult) String() string { - switch f { - case Stop: - return "STOP" - case Skip: - return "SKIP" - case Accept: - return "ACCEPT" - } - - panic("unknown") -} - -// OperatorError is used for storage operations upon errors that may or may not -// be continuable. -type OperatorError struct { - Error error - Continuable bool -} - -// RecordFilter is responsible for controlling the behavior of the database scan -// process and determines the disposition of various records. -// -// The protocol around it makes the assumption that the underlying -// implementation is concurrency safe. -type RecordFilter interface { - // Filter receives the key and value as decoded from the RecordDecoder type. - Filter(key, value interface{}) (filterResult FilterResult) -} - -// RecordOperator is responsible for taking action upon each entity that is -// passed to it. -// -// The protocol around it makes the assumption that the underlying -// implementation is concurrency safe. -type RecordOperator interface { - // Take action on a given record. If the action returns an error, the entire - // scan process stops. - Operate(key, value interface{}) (err *OperatorError) -} diff --git a/storage/local/chunk.go b/storage/local/chunk.go new file mode 100644 index 0000000000..5b590db03a --- /dev/null +++ b/storage/local/chunk.go @@ -0,0 +1,67 @@ +package storage_ng + +import ( + "io" + + clientmodel "github.com/prometheus/client_golang/model" + + "github.com/prometheus/prometheus/storage/metric" +) + +type chunks []chunk + +type chunk interface { + add(*metric.SamplePair) chunks + clone() chunk + firstTime() clientmodel.Timestamp + lastTime() clientmodel.Timestamp + newIterator() chunkIterator + marshal(io.Writer) error + unmarshal(io.Reader) error + close() + + // TODO: remove? + values() <-chan *metric.SamplePair +} + +type chunkIterator interface { + getValueAtTime(clientmodel.Timestamp) metric.Values + getBoundaryValues(metric.Interval) metric.Values + getRangeValues(metric.Interval) metric.Values + contains(clientmodel.Timestamp) bool +} + +func transcodeAndAdd(dst chunk, src chunk, s *metric.SamplePair) chunks { + numTranscodes.Inc() + defer src.close() + + head := dst + body := chunks{} + for v := range src.values() { + newChunks := head.add(v) + body = append(body, newChunks[:len(newChunks)-1]...) + head = newChunks[len(newChunks)-1] + } + newChunks := head.add(s) + body = append(body, newChunks[:len(newChunks)-1]...) + head = newChunks[len(newChunks)-1] + return append(body, head) +} + +func chunkType(c chunk) byte { + switch c.(type) { + case *deltaEncodedChunk: + return 0 + default: + panic("unknown chunk type") + } +} + +func chunkForType(chunkType byte) chunk { + switch chunkType { + case 0: + return newDeltaEncodedChunk(1, 1, false) + default: + panic("unknown chunk type") + } +} diff --git a/storage/local/delta.go b/storage/local/delta.go new file mode 100644 index 0000000000..b706d6b49a --- /dev/null +++ b/storage/local/delta.go @@ -0,0 +1,399 @@ +package storage_ng + +import ( + "encoding/binary" + "fmt" + "io" + "math" + "sort" + + clientmodel "github.com/prometheus/client_golang/model" + + "github.com/prometheus/prometheus/storage/metric" +) + +type deltaBytes int + +const ( + d0 deltaBytes = 0 + d1 = 1 + d2 = 2 + d4 = 4 + d8 = 8 +) + +// The 21-byte header of a delta-encoded chunk looks like: +// +// - time delta bytes: 1 bytes +// - value delta bytes: 1 bytes +// - is integer: 1 byte +// - base time: 8 bytes +// - base value: 8 bytes +// - used buf bytes: 2 bytes +const ( + deltaHeaderBytes = 21 + + deltaHeaderTimeBytesOffset = 0 + deltaHeaderValueBytesOffset = 1 + deltaHeaderIsIntOffset = 2 + deltaHeaderBaseTimeOffset = 3 + deltaHeaderBaseValueOffset = 11 + deltaHeaderBufLenOffset = 19 +) + +type deltaEncodedChunk struct { + buf []byte +} + +func newDeltaEncodedChunk(tb, vb deltaBytes, isInt bool) *deltaEncodedChunk { + buf := chunkBufs.Get() + buf = buf[:deltaHeaderIsIntOffset+1] + + buf[deltaHeaderTimeBytesOffset] = byte(tb) + buf[deltaHeaderValueBytesOffset] = byte(vb) + if isInt { + buf[deltaHeaderIsIntOffset] = 1 + } else { + buf[deltaHeaderIsIntOffset] = 0 + } + + return &deltaEncodedChunk{ + buf: buf, + } +} + +func (c *deltaEncodedChunk) newFollowupChunk() chunk { + return newDeltaEncodedChunk(d1, d1, true) + //return newDeltaEncodedChunk(c.timeBytes(), c.valueBytes(), c.isInt()) +} + +func (c *deltaEncodedChunk) clone() chunk { + buf := chunkBufs.Get() + buf = buf[:len(c.buf)] + copy(buf, c.buf) + return &deltaEncodedChunk{ + buf: buf, + } +} + +func neededDeltaBytes(deltaT clientmodel.Timestamp, deltaV clientmodel.SampleValue, isInt bool) (dtb, dvb deltaBytes) { + dtb = 1 + if deltaT >= 256 { + dtb = 2 + } + if deltaT >= 256*256 { + dtb = 4 + } + if deltaT >= 256*256*256*256 { + dtb = 8 + } + + if isInt { + dvb = 0 + if deltaV != 0 { + dvb = 1 + } + if deltaV < -(256/2) || deltaV > (256/2)-1 { + dvb = 2 + } + if deltaV < -(256*256/2) || deltaV > (256*256/2)-1 { + dvb = 4 + } + if deltaV < -(256*256*256*256/2) || deltaV > (256*256*256*256/2)-1 { + dvb = 8 + } + } else { + dvb = 4 + if clientmodel.SampleValue(float32(deltaV)) != deltaV { + dvb = 8 + } + } + return dtb, dvb +} + +func max(a, b deltaBytes) deltaBytes { + if a > b { + return a + } + return b +} + +func (c *deltaEncodedChunk) timeBytes() deltaBytes { + return deltaBytes(c.buf[deltaHeaderTimeBytesOffset]) +} + +func (c *deltaEncodedChunk) valueBytes() deltaBytes { + return deltaBytes(c.buf[deltaHeaderValueBytesOffset]) +} + +func (c *deltaEncodedChunk) isInt() bool { + return c.buf[deltaHeaderIsIntOffset] == 1 +} + +func (c *deltaEncodedChunk) baseTime() clientmodel.Timestamp { + return clientmodel.Timestamp(binary.LittleEndian.Uint64(c.buf[deltaHeaderBaseTimeOffset:])) +} + +func (c *deltaEncodedChunk) baseValue() clientmodel.SampleValue { + return clientmodel.SampleValue(math.Float64frombits(binary.LittleEndian.Uint64(c.buf[deltaHeaderBaseValueOffset:]))) +} + +func (c *deltaEncodedChunk) add(s *metric.SamplePair) chunks { + if len(c.buf) < deltaHeaderBytes { + c.buf = c.buf[:deltaHeaderBytes] + binary.LittleEndian.PutUint64(c.buf[deltaHeaderBaseTimeOffset:], uint64(s.Timestamp)) + binary.LittleEndian.PutUint64(c.buf[deltaHeaderBaseValueOffset:], math.Float64bits(float64(s.Value))) + } + + remainingBytes := cap(c.buf) - len(c.buf) + sampleSize := c.sampleSize() + + // Do we generally have space for another sample in this chunk? If not, + // overflow into a new one. We assume that if we have seen floating point + // values once, the series will most likely contain floats in the future. + if remainingBytes < sampleSize { + //fmt.Println("overflow") + overflowChunks := c.newFollowupChunk().add(s) + return chunks{c, overflowChunks[0]} + } + + dt := s.Timestamp - c.baseTime() + dv := s.Value - c.baseValue() + + // If the new sample is incompatible with the current encoding, reencode the + // existing chunk data into new chunk(s). + // + // int->float. + // TODO: compare speed with Math.Modf. + if c.isInt() && clientmodel.SampleValue(int64(dv)) != dv { + //fmt.Println("int->float", len(c.buf), cap(c.buf)) + return transcodeAndAdd(newDeltaEncodedChunk(c.timeBytes(), d4, false), c, s) + } + // float32->float64. + if !c.isInt() && c.valueBytes() == d4 && clientmodel.SampleValue(float32(dv)) != dv { + //fmt.Println("float32->float64", float32(dv), dv, len(c.buf), cap(c.buf)) + return transcodeAndAdd(newDeltaEncodedChunk(c.timeBytes(), d8, false), c, s) + } + // More bytes per sample. + if dtb, dvb := neededDeltaBytes(dt, dv, c.isInt()); dtb > c.timeBytes() || dvb > c.valueBytes() { + //fmt.Printf("transcoding T: %v->%v, V: %v->%v, I: %v; len %v, cap %v\n", c.timeBytes(), dtb, c.valueBytes(), dvb, c.isInt(), len(c.buf), cap(c.buf)) + dtb = max(dtb, c.timeBytes()) + dvb = max(dvb, c.valueBytes()) + return transcodeAndAdd(newDeltaEncodedChunk(dtb, dvb, c.isInt()), c, s) + } + + offset := len(c.buf) + c.buf = c.buf[:offset+sampleSize] + + switch c.timeBytes() { + case 1: + c.buf[offset] = byte(dt) + case 2: + binary.LittleEndian.PutUint16(c.buf[offset:], uint16(dt)) + case 4: + binary.LittleEndian.PutUint32(c.buf[offset:], uint32(dt)) + case 8: + binary.LittleEndian.PutUint64(c.buf[offset:], uint64(dt)) + } + + offset += int(c.timeBytes()) + + if c.isInt() { + switch c.valueBytes() { + case 0: + // No-op. Constant value is stored as base value. + case 1: + c.buf[offset] = byte(dv) + case 2: + binary.LittleEndian.PutUint16(c.buf[offset:], uint16(dv)) + case 4: + binary.LittleEndian.PutUint32(c.buf[offset:], uint32(dv)) + case 8: + binary.LittleEndian.PutUint64(c.buf[offset:], uint64(dv)) + default: + panic("Invalid number of bytes for integer delta") + } + } else { + switch c.valueBytes() { + case 4: + binary.LittleEndian.PutUint32(c.buf[offset:], math.Float32bits(float32(dv))) + case 8: + binary.LittleEndian.PutUint64(c.buf[offset:], math.Float64bits(float64(dv))) + default: + panic("Invalid number of bytes for floating point delta") + } + } + return chunks{c} +} + +func (c *deltaEncodedChunk) close() { + //fmt.Println("returning chunk") + chunkBufs.Give(c.buf) +} + +func (c *deltaEncodedChunk) sampleSize() int { + return int(c.timeBytes() + c.valueBytes()) +} + +func (c *deltaEncodedChunk) len() int { + if len(c.buf) < deltaHeaderBytes { + return 0 + } + return (len(c.buf) - deltaHeaderBytes) / c.sampleSize() +} + +// TODO: remove? +func (c *deltaEncodedChunk) values() <-chan *metric.SamplePair { + n := c.len() + valuesChan := make(chan *metric.SamplePair) + go func() { + for i := 0; i < n; i++ { + valuesChan <- c.valueAtIndex(i) + } + close(valuesChan) + }() + return valuesChan +} + +func (c *deltaEncodedChunk) valueAtIndex(idx int) *metric.SamplePair { + offset := deltaHeaderBytes + idx*c.sampleSize() + + var dt uint64 + switch c.timeBytes() { + case 1: + dt = uint64(uint8(c.buf[offset])) + case 2: + dt = uint64(binary.LittleEndian.Uint16(c.buf[offset:])) + case 4: + dt = uint64(binary.LittleEndian.Uint32(c.buf[offset:])) + case 8: + dt = uint64(binary.LittleEndian.Uint64(c.buf[offset:])) + } + + offset += int(c.timeBytes()) + + var dv clientmodel.SampleValue + if c.isInt() { + switch c.valueBytes() { + case 0: + dv = clientmodel.SampleValue(0) + case 1: + dv = clientmodel.SampleValue(int8(c.buf[offset])) + case 2: + dv = clientmodel.SampleValue(int16(binary.LittleEndian.Uint16(c.buf[offset:]))) + case 4: + dv = clientmodel.SampleValue(int32(binary.LittleEndian.Uint32(c.buf[offset:]))) + case 8: + dv = clientmodel.SampleValue(int64(binary.LittleEndian.Uint64(c.buf[offset:]))) + default: + panic("Invalid number of bytes for integer delta") + } + } else { + switch c.valueBytes() { + case 4: + dv = clientmodel.SampleValue(math.Float32frombits(binary.LittleEndian.Uint32(c.buf[offset:]))) + case 8: + dv = clientmodel.SampleValue(math.Float64frombits(binary.LittleEndian.Uint64(c.buf[offset:]))) + default: + panic("Invalid number of bytes for floating point delta") + } + } + return &metric.SamplePair{ + Timestamp: c.baseTime() + clientmodel.Timestamp(dt), + Value: c.baseValue() + dv, + } +} + +func (c *deltaEncodedChunk) firstTime() clientmodel.Timestamp { + return c.valueAtIndex(0).Timestamp +} + +func (c *deltaEncodedChunk) lastTime() clientmodel.Timestamp { + return c.valueAtIndex(c.len() - 1).Timestamp +} + +func (c *deltaEncodedChunk) marshal(w io.Writer) error { + // TODO: check somewhere that configured buf len cannot overflow 16 bit. + binary.LittleEndian.PutUint16(c.buf[deltaHeaderBufLenOffset:], uint16(len(c.buf))) + + n, err := w.Write(c.buf[:cap(c.buf)]) + if err != nil { + return err + } + if n != cap(c.buf) { + return fmt.Errorf("wanted to write %d bytes, wrote %d", len(c.buf), n) + } + return nil +} + +func (c *deltaEncodedChunk) unmarshal(r io.Reader) error { + c.buf = c.buf[:cap(c.buf)] + readBytes := 0 + for readBytes < len(c.buf) { + n, err := r.Read(c.buf[readBytes:]) + if err != nil { + return err + } + readBytes += n + } + c.buf = c.buf[:binary.LittleEndian.Uint16(c.buf[deltaHeaderBufLenOffset:])] + return nil +} + +type deltaEncodedChunkIterator struct { + chunk *deltaEncodedChunk + // TODO: add more fields here to keep track of last position. +} + +func (c *deltaEncodedChunk) newIterator() chunkIterator { + return &deltaEncodedChunkIterator{ + chunk: c, + } +} + +func (it *deltaEncodedChunkIterator) getValueAtTime(t clientmodel.Timestamp) metric.Values { + i := sort.Search(it.chunk.len(), func(i int) bool { + return !it.chunk.valueAtIndex(i).Timestamp.Before(t) + }) + + switch i { + case 0: + return metric.Values{*it.chunk.valueAtIndex(0)} + case it.chunk.len(): + return metric.Values{*it.chunk.valueAtIndex(it.chunk.len() - 1)} + default: + if v := it.chunk.valueAtIndex(i); v.Timestamp.Equal(t) { + return metric.Values{*v} + } else { + return metric.Values{*it.chunk.valueAtIndex(i - 1), *v} + } + } +} + +func (it *deltaEncodedChunkIterator) getBoundaryValues(in metric.Interval) metric.Values { + return nil +} + +func (it *deltaEncodedChunkIterator) getRangeValues(in metric.Interval) metric.Values { + oldest := sort.Search(it.chunk.len(), func(i int) bool { + return !it.chunk.valueAtIndex(i).Timestamp.Before(in.OldestInclusive) + }) + + newest := sort.Search(it.chunk.len(), func(i int) bool { + return it.chunk.valueAtIndex(i).Timestamp.After(in.NewestInclusive) + }) + + if oldest == it.chunk.len() { + return nil + } + + result := make(metric.Values, 0, newest-oldest) + for i := oldest; i < newest; i++ { + result = append(result, *it.chunk.valueAtIndex(i)) + } + return result +} + +func (it *deltaEncodedChunkIterator) contains(t clientmodel.Timestamp) bool { + return !t.Before(it.chunk.firstTime()) && !t.After(it.chunk.lastTime()) +} diff --git a/storage/local/freelist.go b/storage/local/freelist.go new file mode 100644 index 0000000000..609c3e7aac --- /dev/null +++ b/storage/local/freelist.go @@ -0,0 +1,44 @@ +package storage_ng + +import ( + "github.com/prometheus/prometheus/utility" +) + +var chunkBufs = newChunkBufList(10000, 10000) + +type chunkBufList struct { + l utility.FreeList +} + +func newChunkBuf() []byte { + return make([]byte, 0, 1024) // TODO: This value somehow needs to be set in coordination with the one passed into the disk persistence. +} + +func newChunkBufList(length, capacity int) *chunkBufList { + l := &chunkBufList{ + l: utility.NewFreeList(capacity), + } + for i := 0; i < length; i++ { + l.l.Give(newChunkBuf()) + } + return l +} + +func (l *chunkBufList) Get() []byte { + numChunkGets.Inc() + if v, ok := l.l.Get(); ok { + return v.([]byte) + } + + return newChunkBuf() +} + +func (l *chunkBufList) Give(v []byte) bool { + numChunkGives.Inc() + v = v[:0] + return l.l.Give(v) +} + +func (l *chunkBufList) Close() { + l.l.Close() +} diff --git a/storage/local/instrumentation.go b/storage/local/instrumentation.go new file mode 100644 index 0000000000..a3fe2cb0a2 --- /dev/null +++ b/storage/local/instrumentation.go @@ -0,0 +1,88 @@ +// Copyright 2013 Prometheus Team +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package storage_ng + +import ( + "github.com/prometheus/client_golang/prometheus" +) + +const ( + address = "instance" + alive = "alive" + failure = "failure" + outcome = "outcome" + state = "state" + success = "success" + unreachable = "unreachable" +) + +var ( + numSeries = prometheus.NewGauge(prometheus.GaugeOpts{ + Name: "prometheus_stored_series_count", + Help: "The number of currently stored series.", + }) + numSamples = prometheus.NewCounter(prometheus.CounterOpts{ + Name: "prometheus_stored_samples_total", + Help: "The total number of stored samples.", + }) + + numChunks = prometheus.NewGauge(prometheus.GaugeOpts{ + Name: "prometheus_used_chunks_count", + Help: "The current number of chunks in use.", + }) + numChunkGives = prometheus.NewCounter(prometheus.CounterOpts{ + Name: "prometheus_chunk_gives_total", + Help: "The total number of returned (freed) chunks.", + }) + numChunkGets = prometheus.NewCounter(prometheus.CounterOpts{ + Name: "prometheus_chunk_gets_total", + Help: "The total number of retrieved chunks.", + }) + numTranscodes = prometheus.NewCounter(prometheus.CounterOpts{ + Name: "prometheus_chunk_transcodes_total", + Help: "The total number of chunk transcodes.", + }) + numPinnedChunks = prometheus.NewGauge(prometheus.GaugeOpts{ + Name: "prometheus_pinned_chunks_count", + Help: "The current number of pinned chunks.", + }) + + persistLatencies = prometheus.NewSummaryVec(prometheus.SummaryOpts{ + Name: "prometheus_persist_latency_ms", + Help: "A summary of latencies for persisting each chunk.", + }, []string{outcome}) + persistQueueLength = prometheus.NewGauge(prometheus.GaugeOpts{ + Name: "prometheus_persist_queue_length", + Help: "The current number of chunks waiting in the persist queue.", + }) + persistQueueCapacity = prometheus.NewGauge(prometheus.GaugeOpts{ + Name: "prometheus_persist_queue_capacity", + Help: "The total capacity of the persist queue.", + }) +) + +func init() { + prometheus.MustRegister(numSeries) + prometheus.MustRegister(numSamples) + prometheus.MustRegister(numChunks) + prometheus.MustRegister(numChunkGives) + prometheus.MustRegister(numChunkGets) + prometheus.MustRegister(numTranscodes) + prometheus.MustRegister(numPinnedChunks) + prometheus.MustRegister(persistLatencies) + prometheus.MustRegister(persistQueueLength) + prometheus.MustRegister(persistQueueCapacity) + + persistQueueCapacity.Set(float64(persistQueueCap)) +} diff --git a/storage/local/interface.go b/storage/local/interface.go new file mode 100644 index 0000000000..3e79b3984f --- /dev/null +++ b/storage/local/interface.go @@ -0,0 +1,98 @@ +package storage_ng + +import ( + clientmodel "github.com/prometheus/client_golang/model" + + "github.com/prometheus/prometheus/storage/metric" + "github.com/prometheus/prometheus/utility" +) + +type Storage interface { + // AppendSamples stores a group of new samples. Multiple samples for the same + // fingerprint need to be submitted in chronological order, from oldest to + // newest (both in the same call to AppendSamples and across multiple calls). + AppendSamples(clientmodel.Samples) + // NewPreloader returns a new Preloader which allows preloading and pinning + // series data into memory for use within a query. + NewPreloader() Preloader + // Get all of the metric fingerprints that are associated with the + // provided label matchers. + GetFingerprintsForLabelMatchers(metric.LabelMatchers) clientmodel.Fingerprints + // Get all of the label values that are associated with a given label name. + GetLabelValuesForLabelName(clientmodel.LabelName) clientmodel.LabelValues + // Get the metric associated with the provided fingerprint. + GetMetricForFingerprint(clientmodel.Fingerprint) clientmodel.Metric + // Get all label values that are associated with a given label name. + GetAllValuesForLabel(clientmodel.LabelName) clientmodel.LabelValues + // Construct an iterator for a given fingerprint. + NewIterator(clientmodel.Fingerprint) SeriesIterator + // Run the request-serving and maintenance loop. + Serve(started chan<- bool) + // Close the MetricsStorage and releases all resources. + Close() error +} + +type SeriesIterator interface { + // Get the two values that are immediately adjacent to a given time. + GetValueAtTime(clientmodel.Timestamp) metric.Values + // Get the boundary values of an interval: the first value older than + // the interval start, and the first value younger than the interval + // end. + GetBoundaryValues(metric.Interval) metric.Values + // Get all values contained within a provided interval. + GetRangeValues(metric.Interval) metric.Values +} + +// A Persistence stores samples persistently across restarts. +type Persistence interface { + // PersistChunk persists a single chunk of a series. + PersistChunk(clientmodel.Fingerprint, chunk) error + // PersistIndexes persists a Prometheus server's timeseries indexes. + PersistIndexes(i *Indexes) error + // PersistHeads persists all open (non-full) head chunks. + PersistHeads(map[clientmodel.Fingerprint]*memorySeries) error + + // DropChunks deletes all chunks from a timeseries whose last sample time is + // before beforeTime. + DropChunks(fp clientmodel.Fingerprint, beforeTime clientmodel.Timestamp) error + + // LoadChunks loads a group of chunks of a timeseries by their index. The + // chunk with the earliest time will have index 0, the following ones will + // have incrementally larger indexes. + LoadChunks(fp clientmodel.Fingerprint, indexes []int) (chunks, error) + // LoadChunkDescs loads chunkDescs for a series up until a given time. + LoadChunkDescs(fp clientmodel.Fingerprint, beforeTime clientmodel.Timestamp) (chunkDescs, error) + // LoadHeads loads all open (non-full) head chunks. + LoadHeads(map[clientmodel.Fingerprint]*memorySeries) error + // LoadIndexes loads and returns all timeseries indexes. + LoadIndexes() (*Indexes, error) +} + +// A Preloader preloads series data necessary for a query into memory and pins +// them until released via Close(). +type Preloader interface { + PreloadRange(fp clientmodel.Fingerprint, from clientmodel.Timestamp, through clientmodel.Timestamp) error + /* + // GetMetricAtTime loads and pins samples around a given time. + GetMetricAtTime(clientmodel.Fingerprint, clientmodel.Timestamp) error + // GetMetricAtInterval loads and pins samples at intervals. + GetMetricAtInterval(fp clientmodel.Fingerprint, from, through clientmodel.Timestamp, interval time.Duration) error + // GetMetricRange loads and pins a given range of samples. + GetMetricRange(fp clientmodel.Fingerprint, from, through clientmodel.Timestamp) error + // GetMetricRangeAtInterval loads and pins sample ranges at intervals. + GetMetricRangeAtInterval(fp clientmodel.Fingerprint, from, through clientmodel.Timestamp, interval, rangeDuration time.Duration) error + */ + // Close unpins any previously requested series data from memory. + Close() +} + +type Closer interface { + // Close cleans up any used resources. + Close() +} + +type Indexes struct { + FingerprintToSeries map[clientmodel.Fingerprint]*memorySeries + LabelPairToFingerprints map[metric.LabelPair]utility.Set + LabelNameToLabelValues map[clientmodel.LabelName]utility.Set +} diff --git a/storage/local/persistence.go b/storage/local/persistence.go new file mode 100644 index 0000000000..e154431d9c --- /dev/null +++ b/storage/local/persistence.go @@ -0,0 +1,388 @@ +package storage_ng + +import ( + "bufio" + "encoding/binary" + "encoding/gob" + "fmt" + "io" + "os" + "path" + + "github.com/golang/glog" + + //"github.com/prometheus/prometheus/storage/metric" + + //"github.com/prometheus/client_golang/prometheus" + + clientmodel "github.com/prometheus/client_golang/model" + + "github.com/prometheus/prometheus/storage/metric" + "github.com/prometheus/prometheus/utility" +) + +const ( + seriesFileName = "series.db" + seriesTempFileName = "series.db.tmp" + indexFileName = "index.db" + headsFileName = "heads.db" + + chunkHeaderLen = 17 + chunkHeaderTypeOffset = 0 + chunkHeaderFirstTimeOffset = 1 + chunkHeaderLastTimeOffset = 9 + + headsHeaderLen = 9 + headsHeaderFingerprintOffset = 0 + headsHeaderTypeOffset = 8 +) + +type diskPersistence struct { + basePath string + chunkLen int +} + +func NewDiskPersistence(basePath string, chunkLen int) (Persistence, error) { + gob.Register(clientmodel.Fingerprint(0)) + gob.Register(clientmodel.LabelValue("")) + + err := os.MkdirAll(basePath, 0700) + if err != nil { + return nil, err + } + return &diskPersistence{ + basePath: basePath, + chunkLen: chunkLen, + }, nil +} + +func (p *diskPersistence) dirForFingerprint(fp clientmodel.Fingerprint) string { + fpStr := fp.String() + return fmt.Sprintf("%s/%c%c/%s", p.basePath, fpStr[0], fpStr[1], fpStr[2:]) +} + +// exists returns true when the given file or directory exists. +func exists(path string) (bool, error) { + _, err := os.Stat(path) + if err == nil { + return true, nil + } + if os.IsNotExist(err) { + return false, nil + } + + return false, err +} + +func (p *diskPersistence) openChunkFileForWriting(fp clientmodel.Fingerprint) (*os.File, error) { + dirname := p.dirForFingerprint(fp) + ex, err := exists(dirname) + if err != nil { + return nil, err + } + if !ex { + if err := os.MkdirAll(dirname, 0700); err != nil { + return nil, err + } + } + return os.OpenFile(path.Join(dirname, seriesFileName), os.O_WRONLY|os.O_APPEND|os.O_CREATE, 0640) +} + +func (p *diskPersistence) openChunkFileForReading(fp clientmodel.Fingerprint) (*os.File, error) { + dirname := p.dirForFingerprint(fp) + return os.Open(path.Join(dirname, seriesFileName)) +} + +func writeChunkHeader(w io.Writer, c chunk) error { + header := make([]byte, chunkHeaderLen) + header[chunkHeaderTypeOffset] = chunkType(c) + binary.LittleEndian.PutUint64(header[chunkHeaderFirstTimeOffset:], uint64(c.firstTime())) + binary.LittleEndian.PutUint64(header[chunkHeaderLastTimeOffset:], uint64(c.lastTime())) + _, err := w.Write(header) + return err +} + +func (p *diskPersistence) PersistChunk(fp clientmodel.Fingerprint, c chunk) error { + // 1. Open chunk file. + f, err := p.openChunkFileForWriting(fp) + if err != nil { + return err + } + defer f.Close() + + b := bufio.NewWriterSize(f, chunkHeaderLen+p.chunkLen) + defer b.Flush() + + // 2. Write the header (chunk type and first/last times). + err = writeChunkHeader(b, c) + if err != nil { + return err + } + + // 3. Write chunk into file. + return c.marshal(b) +} + +func (p *diskPersistence) offsetForChunkIndex(i int) int64 { + return int64(i * (chunkHeaderLen + p.chunkLen)) +} + +func (p *diskPersistence) LoadChunks(fp clientmodel.Fingerprint, indexes []int) (chunks, error) { + // TODO: we need to verify at some point that file length is a multiple of + // the chunk size. When is the best time to do this, and where to remember + // it? Right now, we only do it when loading chunkDescs. + f, err := p.openChunkFileForReading(fp) + if err != nil { + return nil, err + } + defer f.Close() + + chunks := make(chunks, 0, len(indexes)) + defer func() { + if err == nil { + return + } + for _, c := range chunks { + c.close() + } + }() + + typeBuf := make([]byte, 1) + for _, idx := range indexes { + _, err := f.Seek(p.offsetForChunkIndex(idx), os.SEEK_SET) + if err != nil { + return nil, err + } + // TODO: check seek offset too? + + n, err := f.Read(typeBuf) + if err != nil { + return nil, err + } + if n != 1 { + // Shouldn't happen? + panic("read returned != 1 bytes") + } + + _, err = f.Seek(chunkHeaderLen-1, os.SEEK_CUR) + if err != nil { + return nil, err + } + chunk := chunkForType(typeBuf[0]) + chunk.unmarshal(f) + chunks = append(chunks, chunk) + } + return chunks, nil +} + +func (p *diskPersistence) LoadChunkDescs(fp clientmodel.Fingerprint, beforeTime clientmodel.Timestamp) (chunkDescs, error) { + f, err := p.openChunkFileForReading(fp) + if os.IsNotExist(err) { + return nil, nil + } + if err != nil { + return nil, err + } + defer f.Close() + + fi, err := f.Stat() + if err != nil { + return nil, err + } + totalChunkLen := chunkHeaderLen + p.chunkLen + if fi.Size()%int64(totalChunkLen) != 0 { + // TODO: record number of encountered corrupt series files in a metric? + + // Truncate the file size to the nearest multiple of chunkLen. + truncateTo := fi.Size() - fi.Size()%int64(totalChunkLen) + glog.Infof("Bad series file size for %s: %d bytes (no multiple of %d). Truncating to %d bytes.", fp, fi.Size(), totalChunkLen, truncateTo) + // TODO: this doesn't work, as this is a read-only file handle. + if err := f.Truncate(truncateTo); err != nil { + return nil, err + } + } + + numChunks := int(fi.Size()) / totalChunkLen + cds := make(chunkDescs, 0, numChunks) + for i := 0; i < numChunks; i++ { + _, err := f.Seek(p.offsetForChunkIndex(i)+chunkHeaderFirstTimeOffset, os.SEEK_SET) + if err != nil { + return nil, err + } + + chunkTimesBuf := make([]byte, 16) + _, err = io.ReadAtLeast(f, chunkTimesBuf, 16) + if err != nil { + return nil, err + } + cd := &chunkDesc{ + firstTimeField: clientmodel.Timestamp(binary.LittleEndian.Uint64(chunkTimesBuf)), + lastTimeField: clientmodel.Timestamp(binary.LittleEndian.Uint64(chunkTimesBuf[8:])), + } + if !cd.firstTime().Before(beforeTime) { + // From here on, we have chunkDescs in memory already. + break + } + cds = append(cds, cd) + } + return cds, nil +} + +func (p *diskPersistence) indexPath() string { + return path.Join(p.basePath, indexFileName) +} + +func (p *diskPersistence) PersistIndexes(i *Indexes) error { + f, err := os.OpenFile(p.indexPath(), os.O_WRONLY|os.O_TRUNC|os.O_CREATE, 0640) + if err != nil { + return err + } + defer f.Close() + + enc := gob.NewEncoder(f) + if err := enc.Encode(i); err != nil { + return err + } + + return nil +} + +func (p *diskPersistence) LoadIndexes() (*Indexes, error) { + f, err := os.Open(p.indexPath()) + if os.IsNotExist(err) { + return &Indexes{ + FingerprintToSeries: make(map[clientmodel.Fingerprint]*memorySeries), + LabelPairToFingerprints: make(map[metric.LabelPair]utility.Set), + LabelNameToLabelValues: make(map[clientmodel.LabelName]utility.Set), + }, nil + } + if err != nil { + return nil, err + } + defer f.Close() + + dec := gob.NewDecoder(f) + var i Indexes + if err := dec.Decode(&i); err != nil { + return nil, err + } + + return &i, nil +} + +func (p *diskPersistence) headsPath() string { + return path.Join(p.basePath, headsFileName) +} + +func (p *diskPersistence) PersistHeads(fpToSeries map[clientmodel.Fingerprint]*memorySeries) error { + f, err := os.OpenFile(p.headsPath(), os.O_WRONLY|os.O_TRUNC|os.O_CREATE, 0640) + if err != nil { + return err + } + header := make([]byte, 9) + for fp, series := range fpToSeries { + head := series.head().chunk + + binary.LittleEndian.PutUint64(header[headsHeaderFingerprintOffset:], uint64(fp)) + header[headsHeaderTypeOffset] = chunkType(head) + _, err := f.Write(header) + if err != nil { + return err + } + err = head.marshal(f) + if err != nil { + return err + } + } + return nil +} + +func (p *diskPersistence) DropChunks(fp clientmodel.Fingerprint, beforeTime clientmodel.Timestamp) error { + f, err := p.openChunkFileForReading(fp) + if os.IsNotExist(err) { + return nil + } + if err != nil { + return err + } + defer f.Close() + + // Find the first chunk that should be kept. + for i := 0; ; i++ { + _, err := f.Seek(p.offsetForChunkIndex(i)+chunkHeaderLastTimeOffset, os.SEEK_SET) + if err != nil { + return err + } + lastTimeBuf := make([]byte, 8) + _, err = io.ReadAtLeast(f, lastTimeBuf, 8) + if err == io.EOF { + // We ran into the end of the file without finding any chunks that should + // be kept. Remove the whole file. + if err := os.Remove(f.Name()); err != nil { + return err + } + return nil + } + if err != nil { + return err + } + lastTime := clientmodel.Timestamp(binary.LittleEndian.Uint64(lastTimeBuf)) + if !lastTime.Before(beforeTime) { + break + } + } + + // We've found the first chunk that should be kept. Seek backwards to the + // beginning of its header and start copying everything from there into a new + // file. + _, err = f.Seek(-(chunkHeaderLastTimeOffset + 8), os.SEEK_CUR) + if err != nil { + return err + } + + dirname := p.dirForFingerprint(fp) + temp, err := os.OpenFile(path.Join(dirname, seriesTempFileName), os.O_WRONLY|os.O_CREATE, 0640) + if err != nil { + return err + } + defer temp.Close() + + if _, err := io.Copy(temp, f); err != nil { + return err + } + + os.Rename(path.Join(dirname, seriesTempFileName), path.Join(dirname, seriesFileName)) + return nil +} + +func (p *diskPersistence) LoadHeads(fpToSeries map[clientmodel.Fingerprint]*memorySeries) error { + f, err := os.Open(p.headsPath()) + if os.IsNotExist(err) { + // TODO: this should only happen if there never was a shutdown before. In + // that case, all heads should be in order already, since the series got + // created during this process' runtime. + // Still, make this more robust. + return nil + } + + header := make([]byte, headsHeaderLen) + for { + _, err := io.ReadAtLeast(f, header, headsHeaderLen) + if err == io.ErrUnexpectedEOF { + // TODO: this should only be ok if n is 0. + break + } + if err != nil { + return nil + } + // TODO: this relies on the implementation (uint64) of Fingerprint. + fp := clientmodel.Fingerprint(binary.LittleEndian.Uint64(header[headsHeaderFingerprintOffset:])) + chunk := chunkForType(header[headsHeaderTypeOffset]) + chunk.unmarshal(f) + fpToSeries[fp].chunkDescs = append(fpToSeries[fp].chunkDescs, &chunkDesc{ + chunk: chunk, + refCount: 1, + }) + } + return nil +} diff --git a/storage/local/persistence_test.go b/storage/local/persistence_test.go new file mode 100644 index 0000000000..5941fe76ee --- /dev/null +++ b/storage/local/persistence_test.go @@ -0,0 +1,107 @@ +package storage_ng + +import ( + "io/ioutil" + "testing" + + clientmodel "github.com/prometheus/client_golang/model" + + "github.com/prometheus/prometheus/storage/metric" + "github.com/prometheus/prometheus/utility" +) + +func TestIndexPersistence(t *testing.T) { + expected := Indexes{ + FingerprintToSeries: map[clientmodel.Fingerprint]*memorySeries{ + 0: { + metric: clientmodel.Metric{ + clientmodel.MetricNameLabel: "metric_0", + "label_1": "value_1", + }, + }, + 1: { + metric: clientmodel.Metric{ + clientmodel.MetricNameLabel: "metric_0", + "label_2": "value_2", + }, + }, + }, + LabelPairToFingerprints: map[metric.LabelPair]utility.Set{ + metric.LabelPair{ + Name: clientmodel.MetricNameLabel, + Value: "metric_0", + }: { + 0: struct{}{}, + 1: struct{}{}, + }, + metric.LabelPair{ + Name: "label_1", + Value: "value_1", + }: { + 0: struct{}{}, + }, + metric.LabelPair{ + Name: "label_2", + Value: "value_2", + }: { + 1: struct{}{}, + }, + }, + LabelNameToLabelValues: map[clientmodel.LabelName]utility.Set{ + clientmodel.MetricNameLabel: { + clientmodel.LabelValue("metric_0"): struct{}{}, + }, + "label_1": { + clientmodel.LabelValue("value_1"): struct{}{}, + }, + "label_2": { + clientmodel.LabelValue("value_2"): struct{}{}, + }, + }, + } + + basePath, err := ioutil.TempDir("", "test_index_persistence") + if err != nil { + t.Fatal(err) + } + p, err := NewDiskPersistence(basePath, 1024) + if err != nil { + t.Fatal(err) + } + p.PersistIndexes(&expected) + + actual, err := p.LoadIndexes() + if err != nil { + t.Fatal(err) + } + + if len(actual.FingerprintToSeries) != len(expected.FingerprintToSeries) { + t.Fatalf("Count mismatch: Got %d; want %d", len(actual.FingerprintToSeries), len(expected.FingerprintToSeries)) + } + for fp, actualSeries := range actual.FingerprintToSeries { + expectedSeries := expected.FingerprintToSeries[fp] + if !expectedSeries.metric.Equal(actualSeries.metric) { + t.Fatalf("%s: Got %s; want %s", fp, actualSeries.metric, expectedSeries.metric) + } + } + + if len(actual.LabelPairToFingerprints) != len(expected.LabelPairToFingerprints) { + t.Fatalf("Count mismatch: Got %d; want %d", len(actual.LabelPairToFingerprints), len(expected.LabelPairToFingerprints)) + } + for lp, actualFps := range actual.LabelPairToFingerprints { + expectedFps := expected.LabelPairToFingerprints[lp] + if len(actualFps) != len(actualFps.Intersection(expectedFps)) { + t.Fatalf("%s: Got %s; want %s", lp, actualFps, expectedFps) + } + } + + if len(actual.LabelNameToLabelValues) != len(expected.LabelNameToLabelValues) { + t.Fatalf("Count mismatch: Got %d; want %d", len(actual.LabelNameToLabelValues), len(expected.LabelNameToLabelValues)) + } + for name, actualVals := range actual.LabelNameToLabelValues { + expectedVals := expected.LabelNameToLabelValues[name] + if len(actualVals) != len(actualVals.Intersection(expectedVals)) { + t.Fatalf("%s: Got %s; want %s", name, actualVals, expectedVals) + } + } +} diff --git a/storage/local/preload.go b/storage/local/preload.go new file mode 100644 index 0000000000..550746e9e5 --- /dev/null +++ b/storage/local/preload.go @@ -0,0 +1,102 @@ +// Copyright 2013 Prometheus Team +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package storage_ng + +import ( + clientmodel "github.com/prometheus/client_golang/model" +) + +// memorySeriesPreloader is a Preloader for the memorySeriesStorage. +type memorySeriesPreloader struct { + storage *memorySeriesStorage + pinnedChunkDescs chunkDescs +} + +func (p *memorySeriesPreloader) PreloadRange(fp clientmodel.Fingerprint, from clientmodel.Timestamp, through clientmodel.Timestamp) error { + cds, err := p.storage.preloadChunksForRange(fp, from, through) + if err != nil { + return err + } + p.pinnedChunkDescs = append(p.pinnedChunkDescs, cds...) + return nil +} + +/* +// GetMetricAtTime implements Preloader. +func (p *memorySeriesPreloader) GetMetricAtTime(fp clientmodel.Fingerprint, t clientmodel.Timestamp) error { + cds, err := p.storage.preloadChunks(fp, &timeSelector{ + from: t, + through: t, + }) + if err != nil { + return err + } + p.pinnedChunkDescs = append(p.pinnedChunkDescs, cds...) + return nil +} + +// GetMetricAtInterval implements Preloader. +func (p *memorySeriesPreloader) GetMetricAtInterval(fp clientmodel.Fingerprint, from, through clientmodel.Timestamp, interval time.Duration) error { + cds, err := p.storage.preloadChunks(fp, &timeSelector{ + from: from, + through: through, + interval: interval, + }) + if err != nil { + return err + } + p.pinnedChunkDescs = append(p.pinnedChunkDescs, cds...) + return +} + +// GetMetricRange implements Preloader. +func (p *memorySeriesPreloader) GetMetricRange(fp clientmodel.Fingerprint, t clientmodel.Timestamp, rangeDuration time.Duration) error { + cds, err := p.storage.preloadChunks(fp, &timeSelector{ + from: t, + through: t, + rangeDuration: through.Sub(from), + }) + if err != nil { + return err + } + p.pinnedChunkDescs = append(p.pinnedChunkDescs, cds...) + return +} + +// GetMetricRangeAtInterval implements Preloader. +func (p *memorySeriesPreloader) GetMetricRangeAtInterval(fp clientmodel.Fingerprint, from, through clientmodel.Timestamp, interval, rangeDuration time.Duration) error { + cds, err := p.storage.preloadChunks(fp, &timeSelector{ + from: from, + through: through, + interval: interval, + rangeDuration: rangeDuration, + }) + if err != nil { + return err + } + p.pinnedChunkDescs = append(p.pinnedChunkDescs, cds...) + return +} +*/ + +// Close implements Preloader. +func (p *memorySeriesPreloader) Close() { + for _, cd := range p.pinnedChunkDescs { + // TODO: unpinning may synchronously cause closing of chunks if they have + // been marked to be evicted. This could interfere with other parts of the + // storage that check whether a chunk is swapped in or not. Is it a good + // idea / sufficient to take the storage lock here? + cd.unpin() + } +} diff --git a/storage/local/series.go b/storage/local/series.go new file mode 100644 index 0000000000..3c3abc8a5d --- /dev/null +++ b/storage/local/series.go @@ -0,0 +1,453 @@ +package storage_ng + +import ( + "bytes" + "encoding/gob" + "sort" + "sync" + + clientmodel "github.com/prometheus/client_golang/model" + + "github.com/prometheus/prometheus/storage/metric" +) + +type chunkDescs []*chunkDesc + +type chunkDesc struct { + sync.Mutex + chunk chunk + refCount int + evict bool + firstTimeField clientmodel.Timestamp // TODO: stupid name, reorganize. + lastTimeField clientmodel.Timestamp +} + +func (cd *chunkDesc) add(s *metric.SamplePair) chunks { + cd.Lock() + defer cd.Unlock() + + return cd.chunk.add(s) +} + +func (cd *chunkDesc) pin() { + cd.Lock() + defer cd.Unlock() + + numPinnedChunks.Inc() + cd.refCount++ +} + +func (cd *chunkDesc) unpin() { + cd.Lock() + defer cd.Unlock() + + if cd.refCount == 0 { + panic("cannot unpin already unpinned chunk") + } + numPinnedChunks.Dec() + cd.refCount-- + if cd.refCount == 0 && cd.evict { + cd.evictNow() + } +} + +func (cd *chunkDesc) firstTime() clientmodel.Timestamp { + if cd.chunk == nil { + return cd.firstTimeField + } + return cd.chunk.firstTime() +} + +func (cd *chunkDesc) lastTime() clientmodel.Timestamp { + if cd.chunk == nil { + return cd.lastTimeField + } + return cd.chunk.lastTime() +} + +func (cd *chunkDesc) contains(t clientmodel.Timestamp) bool { + return !t.Before(cd.firstTime()) && !t.After(cd.lastTime()) +} + +func (cd *chunkDesc) open(c chunk) { + cd.Lock() + defer cd.Unlock() + + if cd.refCount != 0 || cd.chunk != nil { + panic("cannot open already pinned chunk") + } + cd.evict = false + cd.chunk = c + numPinnedChunks.Inc() + cd.refCount++ +} + +func (cd *chunkDesc) evictOnUnpin() { + cd.Lock() + defer cd.Unlock() + + if cd.refCount == 0 { + cd.evictNow() + } + cd.evict = true +} + +func (cd *chunkDesc) evictNow() { + cd.firstTimeField = cd.chunk.firstTime() + cd.lastTimeField = cd.chunk.lastTime() + cd.chunk.close() + cd.chunk = nil +} + +type memorySeries struct { + mtx sync.Mutex + + metric clientmodel.Metric + // Sorted by start time, no overlapping chunk ranges allowed. + chunkDescs chunkDescs + chunkDescsLoaded bool +} + +func newMemorySeries(m clientmodel.Metric) *memorySeries { + return &memorySeries{ + metric: m, + // TODO: should we set this to nil initially and only create a chunk when + // adding? But right now, we also only call newMemorySeries when adding, so + // it turns out to be the same. + chunkDescs: chunkDescs{ + // TODO: should there be a newChunkDesc() function? + &chunkDesc{ + chunk: newDeltaEncodedChunk(d1, d0, true), + refCount: 1, + }, + }, + chunkDescsLoaded: true, + } +} + +func (s *memorySeries) add(v *metric.SamplePair, persistQueue chan *persistRequest) { + s.mtx.Lock() + defer s.mtx.Unlock() + + chunks := s.head().add(v) + + s.head().chunk = chunks[0] + if len(chunks) > 1 { + fp := s.metric.Fingerprint() + + queuePersist := func(cd *chunkDesc) { + persistQueue <- &persistRequest{ + fingerprint: fp, + chunkDesc: cd, + } + } + + queuePersist(s.head()) + + for i, c := range chunks[1:] { + cd := &chunkDesc{ + chunk: c, + refCount: 1, + } + s.chunkDescs = append(s.chunkDescs, cd) + // The last chunk is still growing. + if i < len(chunks[1:])-1 { + queuePersist(cd) + } + } + } +} + +func (s *memorySeries) evictOlderThan(t clientmodel.Timestamp) { + s.mtx.Lock() + defer s.mtx.Unlock() + + // For now, always drop the entire range from oldest to t. + for _, cd := range s.chunkDescs { + if !cd.lastTime().Before(t) { + break + } + if cd.chunk == nil { + continue + } + cd.evictOnUnpin() + } +} + +func (s *memorySeries) purgeOlderThan(t clientmodel.Timestamp, p Persistence) (dropSeries bool, err error) { + s.mtx.Lock() + defer s.mtx.Unlock() + + if err := p.DropChunks(s.metric.Fingerprint(), t); err != nil { + return false, err + } + + keepIdx := len(s.chunkDescs) + for i, cd := range s.chunkDescs { + if !cd.lastTime().Before(t) { + keepIdx = i + break + } + } + + for i := 0; i < keepIdx; i++ { + if s.chunkDescs[i].chunk != nil { + s.chunkDescs[i].evictOnUnpin() + } + } + s.chunkDescs = s.chunkDescs[keepIdx:] + + return len(s.chunkDescs) == 0, nil +} + +func (s *memorySeries) close() { + for _, cd := range s.chunkDescs { + if cd.chunk != nil { + cd.evictNow() + } + // TODO: need to handle unwritten heads here. + } +} + +// TODO: in this method (and other places), we just fudge around with chunkDesc +// internals without grabbing the chunkDesc lock. Study how this needs to be +// protected against other accesses that don't hold the series lock. +func (s *memorySeries) preloadChunks(indexes []int, p Persistence) (chunkDescs, error) { + loadIndexes := []int{} + pinnedChunkDescs := make(chunkDescs, 0, len(indexes)) + for _, idx := range indexes { + pinnedChunkDescs = append(pinnedChunkDescs, s.chunkDescs[idx]) + if s.chunkDescs[idx].chunk == nil { + loadIndexes = append(loadIndexes, idx) + } else { + s.chunkDescs[idx].pin() + } + } + + if len(loadIndexes) > 0 { + fp := s.metric.Fingerprint() + chunks, err := p.LoadChunks(fp, loadIndexes) + if err != nil { + // TODO: unpin all chunks that were already loaded before. + return nil, err + } + for i, c := range chunks { + cd := s.chunkDescs[loadIndexes[i]] + cd.open(c) + } + } + + return pinnedChunkDescs, nil +} + +/* +func (s *memorySeries) preloadChunksAtTime(t clientmodel.Timestamp, p Persistence) (chunkDescs, error) { + s.mtx.Lock() + defer s.mtx.Unlock() + + if len(s.chunkDescs) == 0 { + return nil, nil + } + + var pinIndexes []int + // Find first chunk where lastTime() is after or equal to t. + i := sort.Search(len(s.chunkDescs), func(i int) bool { + return !s.chunkDescs[i].lastTime().Before(t) + }) + switch i { + case 0: + pinIndexes = []int{0} + case len(s.chunkDescs): + pinIndexes = []int{i - 1} + default: + if s.chunkDescs[i].contains(t) { + pinIndexes = []int{i} + } else { + pinIndexes = []int{i - 1, i} + } + } + + return s.preloadChunks(pinIndexes, p) +} +*/ + +func (s *memorySeries) loadChunkDescs(p Persistence) error { + cds, err := p.LoadChunkDescs(s.metric.Fingerprint(), s.chunkDescs[0].firstTime()) + if err != nil { + return err + } + s.chunkDescs = append(cds, s.chunkDescs...) + s.chunkDescsLoaded = true + return nil +} + +func (s *memorySeries) preloadChunksForRange(from clientmodel.Timestamp, through clientmodel.Timestamp, p Persistence) (chunkDescs, error) { + s.mtx.Lock() + defer s.mtx.Unlock() + + if !s.chunkDescsLoaded && from.Before(s.chunkDescs[0].firstTime()) { + if err := s.loadChunkDescs(p); err != nil { + return nil, err + } + } + + if len(s.chunkDescs) == 0 { + return nil, nil + } + + // Find first chunk with start time after "from". + fromIdx := sort.Search(len(s.chunkDescs), func(i int) bool { + return s.chunkDescs[i].firstTime().After(from) + }) + // Find first chunk with start time after "through". + throughIdx := sort.Search(len(s.chunkDescs), func(i int) bool { + return s.chunkDescs[i].firstTime().After(through) + }) + if fromIdx > 0 { + fromIdx-- + } + if throughIdx == len(s.chunkDescs) { + throughIdx-- + } + + pinIndexes := make([]int, 0, throughIdx-fromIdx+1) + for i := fromIdx; i <= throughIdx; i++ { + pinIndexes = append(pinIndexes, i) + } + return s.preloadChunks(pinIndexes, p) +} + +type memorySeriesIterator struct { + mtx *sync.Mutex + chunkIt chunkIterator + chunks chunks +} + +func (s *memorySeries) newIterator() SeriesIterator { + chunks := make(chunks, 0, len(s.chunkDescs)) + for i, cd := range s.chunkDescs { + if cd.chunk != nil { + if i == len(s.chunkDescs)-1 { + chunks = append(chunks, cd.chunk.clone()) + } else { + chunks = append(chunks, cd.chunk) + } + } + } + + return &memorySeriesIterator{ + mtx: &s.mtx, + chunks: chunks, + } +} + +func (s *memorySeries) head() *chunkDesc { + return s.chunkDescs[len(s.chunkDescs)-1] +} + +func (s *memorySeries) values() metric.Values { + var values metric.Values + for _, cd := range s.chunkDescs { + for sample := range cd.chunk.values() { + values = append(values, *sample) + } + } + return values +} + +var gobWriter bytes.Buffer +var seriesEncoder *gob.Encoder + +func (s *memorySeries) GobEncode() ([]byte, error) { + gobWriter.Reset() + if seriesEncoder == nil { + seriesEncoder = gob.NewEncoder(&gobWriter) + } + err := seriesEncoder.Encode(s.metric) + return gobWriter.Bytes(), err +} + +var gobReader bytes.Reader +var seriesDecoder *gob.Decoder + +func (s *memorySeries) GobDecode(buf []byte) error { + gobReader = *bytes.NewReader(buf) + if seriesDecoder == nil { + seriesDecoder = gob.NewDecoder(&gobReader) + } + err := seriesDecoder.Decode(&s.metric) + return err +} + +func (it *memorySeriesIterator) GetValueAtTime(t clientmodel.Timestamp) metric.Values { + it.mtx.Lock() + defer it.mtx.Unlock() + + // The most common case. We are iterating through a chunk. + if it.chunkIt != nil && it.chunkIt.contains(t) { + return it.chunkIt.getValueAtTime(t) + } + + it.chunkIt = nil + + if len(it.chunks) == 0 { + return nil + } + + // Before or exactly on the first sample of the series. + if !t.After(it.chunks[0].firstTime()) { + // return first value of first chunk + return it.chunks[0].newIterator().getValueAtTime(t) + } + // After or exactly on the last sample of the series. + if !t.Before(it.chunks[len(it.chunks)-1].lastTime()) { + // return last value of last chunk + return it.chunks[len(it.chunks)-1].newIterator().getValueAtTime(t) + } + + // Find first chunk where lastTime() is after or equal to t. + i := sort.Search(len(it.chunks), func(i int) bool { + return !it.chunks[i].lastTime().Before(t) + }) + if i == len(it.chunks) { + panic("out of bounds") + } + + if t.Before(it.chunks[i].firstTime()) { + // We ended up between two chunks. + return metric.Values{ + it.chunks[i-1].newIterator().getValueAtTime(t)[0], + it.chunks[i].newIterator().getValueAtTime(t)[0], + } + } else { + // We ended up in the middle of a chunk. We might stay there for a while, + // so save it as the current chunk iterator. + it.chunkIt = it.chunks[i].newIterator() + return it.chunkIt.getValueAtTime(t) + } +} + +func (it *memorySeriesIterator) GetBoundaryValues(in metric.Interval) metric.Values { + // TODO: implement real GetBoundaryValues here. + return it.GetRangeValues(in) +} + +func (it *memorySeriesIterator) GetRangeValues(in metric.Interval) metric.Values { + it.mtx.Lock() + defer it.mtx.Unlock() + + // Find the first relevant chunk. + i := sort.Search(len(it.chunks), func(i int) bool { + return !it.chunks[i].lastTime().Before(in.OldestInclusive) + }) + values := metric.Values{} + for _, c := range it.chunks[i:] { + if c.firstTime().After(in.NewestInclusive) { + break + } + // TODO: actually reuse an iterator between calls if we get multiple ranges + // from the same chunk. + values = append(values, c.newIterator().getRangeValues(in)...) + } + return values +} diff --git a/storage/local/storage.go b/storage/local/storage.go new file mode 100644 index 0000000000..5eaf46d0d1 --- /dev/null +++ b/storage/local/storage.go @@ -0,0 +1,503 @@ +package storage_ng + +import ( + "sync" + "time" + + "github.com/golang/glog" + + clientmodel "github.com/prometheus/client_golang/model" + + "github.com/prometheus/prometheus/storage/metric" + "github.com/prometheus/prometheus/utility" +) + +const persistQueueCap = 1024 + +type storageState uint + +const ( + storageStarting storageState = iota + storageServing + storageStopping +) + +type memorySeriesStorage struct { + mtx sync.RWMutex + + state storageState + persistDone chan bool + stopServing chan chan<- bool + + fingerprintToSeries map[clientmodel.Fingerprint]*memorySeries + labelPairToFingerprints map[metric.LabelPair]utility.Set + labelNameToLabelValues map[clientmodel.LabelName]utility.Set + + memoryEvictionInterval time.Duration + memoryRetentionPeriod time.Duration + + persistencePurgeInterval time.Duration + persistenceRetentionPeriod time.Duration + + persistQueue chan *persistRequest + persistence Persistence +} + +type MemorySeriesStorageOptions struct { + Persistence Persistence + MemoryEvictionInterval time.Duration + MemoryRetentionPeriod time.Duration + PersistencePurgeInterval time.Duration + PersistenceRetentionPeriod time.Duration +} + +func NewMemorySeriesStorage(o *MemorySeriesStorageOptions) (*memorySeriesStorage, error) { // TODO: change to return Storage? + glog.Info("Loading indexes...") + i, err := o.Persistence.LoadIndexes() + if err != nil { + return nil, err + } + glog.Info("Loading series head chunks...") + if err := o.Persistence.LoadHeads(i.FingerprintToSeries); err != nil { + return nil, err + } + numSeries.Set(float64(len(i.FingerprintToSeries))) + return &memorySeriesStorage{ + fingerprintToSeries: i.FingerprintToSeries, + labelPairToFingerprints: i.LabelPairToFingerprints, + labelNameToLabelValues: i.LabelNameToLabelValues, + + persistDone: make(chan bool), + stopServing: make(chan chan<- bool), + + memoryEvictionInterval: o.MemoryEvictionInterval, + memoryRetentionPeriod: o.MemoryRetentionPeriod, + + persistencePurgeInterval: o.PersistencePurgeInterval, + persistenceRetentionPeriod: o.PersistenceRetentionPeriod, + + persistQueue: make(chan *persistRequest, persistQueueCap), + persistence: o.Persistence, + }, nil +} + +type persistRequest struct { + fingerprint clientmodel.Fingerprint + chunkDesc *chunkDesc +} + +func (s *memorySeriesStorage) AppendSamples(samples clientmodel.Samples) { + /* + s.mtx.Lock() + defer s.mtx.Unlock() + if s.state != storageServing { + panic("storage is not serving") + } + s.mtx.Unlock() + */ + + for _, sample := range samples { + s.appendSample(sample) + } + + numSamples.Add(float64(len(samples))) +} + +func (s *memorySeriesStorage) appendSample(sample *clientmodel.Sample) { + s.mtx.Lock() + defer s.mtx.Unlock() + + series := s.getOrCreateSeries(sample.Metric) + series.add(&metric.SamplePair{ + Value: sample.Value, + Timestamp: sample.Timestamp, + }, s.persistQueue) +} + +func (s *memorySeriesStorage) getOrCreateSeries(m clientmodel.Metric) *memorySeries { + fp := m.Fingerprint() + series, ok := s.fingerprintToSeries[fp] + + if !ok { + series = newMemorySeries(m) + s.fingerprintToSeries[fp] = series + numSeries.Set(float64(len(s.fingerprintToSeries))) + + for k, v := range m { + labelPair := metric.LabelPair{ + Name: k, + Value: v, + } + + fps, ok := s.labelPairToFingerprints[labelPair] + if !ok { + fps = utility.Set{} + s.labelPairToFingerprints[labelPair] = fps + } + fps.Add(fp) + + values, ok := s.labelNameToLabelValues[k] + if !ok { + values = utility.Set{} + s.labelNameToLabelValues[k] = values + } + values.Add(v) + } + } + return series +} + +/* +func (s *memorySeriesStorage) preloadChunksAtTime(fp clientmodel.Fingerprint, ts clientmodel.Timestamp) (chunkDescs, error) { + series, ok := s.fingerprintToSeries[fp] + if !ok { + panic("requested preload for non-existent series") + } + return series.preloadChunksAtTime(ts, s.persistence) +} +*/ + +func (s *memorySeriesStorage) preloadChunksForRange(fp clientmodel.Fingerprint, from clientmodel.Timestamp, through clientmodel.Timestamp) (chunkDescs, error) { + s.mtx.RLock() + series, ok := s.fingerprintToSeries[fp] + s.mtx.RUnlock() + + if !ok { + panic("requested preload for non-existent series") + } + return series.preloadChunksForRange(from, through, s.persistence) +} + +func (s *memorySeriesStorage) NewIterator(fp clientmodel.Fingerprint) SeriesIterator { + s.mtx.RLock() + series, ok := s.fingerprintToSeries[fp] + s.mtx.RUnlock() + + if !ok { + panic("requested iterator for non-existent series") + } + return series.newIterator() +} + +func (s *memorySeriesStorage) evictMemoryChunks(ttl time.Duration) { + s.mtx.RLock() + defer s.mtx.RUnlock() + + for _, series := range s.fingerprintToSeries { + series.evictOlderThan(clientmodel.TimestampFromTime(time.Now()).Add(-1 * ttl)) + } +} + +func recordPersist(start time.Time, err error) { + outcome := success + if err != nil { + outcome = failure + } + persistLatencies.WithLabelValues(outcome).Observe(float64(time.Since(start) / time.Millisecond)) +} + +func (s *memorySeriesStorage) handlePersistQueue() { + for req := range s.persistQueue { + // TODO: Make this thread-safe? + persistQueueLength.Set(float64(len(s.persistQueue))) + + //glog.Info("Persist request: ", *req.fingerprint) + start := time.Now() + err := s.persistence.PersistChunk(req.fingerprint, req.chunkDesc.chunk) + recordPersist(start, err) + if err != nil { + glog.Error("Error persisting chunk, requeuing: ", err) + s.persistQueue <- req + continue + } + req.chunkDesc.unpin() + } + s.persistDone <- true +} + +// Close stops serving, flushes all pending operations, and frees all resources. +func (s *memorySeriesStorage) Close() error { + s.mtx.Lock() + defer s.mtx.Unlock() + + if s.state == storageStopping { + panic("Illegal State: Attempted to restop memorySeriesStorage.") + } + + stopped := make(chan bool) + glog.Info("Waiting for storage to stop serving...") + s.stopServing <- (stopped) + glog.Info("Serving stopped.") + <-stopped + + glog.Info("Stopping persist loop...") + close(s.persistQueue) + <-s.persistDone + glog.Info("Persist loop stopped.") + + glog.Info("Persisting head chunks...") + if err := s.persistHeads(); err != nil { + return err + } + glog.Info("Done persisting head chunks.") + + glog.Info("Persisting indexes...") + if err := s.persistIndexes(); err != nil { + return err + } + glog.Info("Done persisting indexes.") + + for _, series := range s.fingerprintToSeries { + series.close() + } + s.fingerprintToSeries = nil + + s.state = storageStopping + + return nil +} + +func (s *memorySeriesStorage) persistHeads() error { + return s.persistence.PersistHeads(s.fingerprintToSeries) +} + +func (s *memorySeriesStorage) persistIndexes() error { + err := s.persistence.PersistIndexes(&Indexes{ + FingerprintToSeries: s.fingerprintToSeries, + LabelPairToFingerprints: s.labelPairToFingerprints, + LabelNameToLabelValues: s.labelNameToLabelValues, + }) + return err +} + +func (s *memorySeriesStorage) purgePeriodically(stop <-chan bool) { + purgeTicker := time.NewTicker(s.persistencePurgeInterval) + defer purgeTicker.Stop() + + for { + select { + case <-stop: + return + case <-purgeTicker.C: + glog.Info("Purging old series data...") + s.mtx.RLock() + fps := make([]clientmodel.Fingerprint, 0, len(s.fingerprintToSeries)) + for fp, _ := range s.fingerprintToSeries { + fps = append(fps, fp) + } + s.mtx.RUnlock() + + for _, fp := range fps { + select { + case <-stop: + glog.Info("Interrupted running series purge.") + return + default: + s.purgeSeries(&fp) + } + } + glog.Info("Done purging old series data.") + } + } +} + +func (s *memorySeriesStorage) purgeSeries(fp *clientmodel.Fingerprint) { + s.mtx.RLock() + series, ok := s.fingerprintToSeries[*fp] + if !ok { + return + } + s.mtx.RUnlock() + + drop, err := series.purgeOlderThan(clientmodel.TimestampFromTime(time.Now()).Add(-1*s.persistenceRetentionPeriod), s.persistence) + if err != nil { + glog.Error("Error purging series data: ", err) + } + if drop { + s.dropSeries(fp) + } +} + +// Drop a label value from the label names to label values index. +func (s *memorySeriesStorage) dropLabelValue(l clientmodel.LabelName, v clientmodel.LabelValue) { + if set, ok := s.labelNameToLabelValues[l]; ok { + set.Remove(v) + if len(set) == 0 { + delete(s.labelNameToLabelValues, l) + } + } +} + +// Drop all references to a series, including any samples. +func (s *memorySeriesStorage) dropSeries(fp *clientmodel.Fingerprint) { + s.mtx.Lock() + defer s.mtx.Unlock() + + series, ok := s.fingerprintToSeries[*fp] + if !ok { + return + } + + for k, v := range series.metric { + labelPair := metric.LabelPair{ + Name: k, + Value: v, + } + if set, ok := s.labelPairToFingerprints[labelPair]; ok { + set.Remove(*fp) + if len(set) == 0 { + delete(s.labelPairToFingerprints, labelPair) + s.dropLabelValue(k, v) + } + } + } + delete(s.fingerprintToSeries, *fp) +} + +func (s *memorySeriesStorage) Serve(started chan<- bool) { + s.mtx.Lock() + if s.state != storageStarting { + panic("Illegal State: Attempted to restart memorySeriesStorage.") + } + s.state = storageServing + s.mtx.Unlock() + + evictMemoryTicker := time.NewTicker(s.memoryEvictionInterval) + defer evictMemoryTicker.Stop() + + go s.handlePersistQueue() + + stopPurge := make(chan bool) + go s.purgePeriodically(stopPurge) + + started <- true + for { + select { + case <-evictMemoryTicker.C: + s.evictMemoryChunks(s.memoryRetentionPeriod) + case stopped := <-s.stopServing: + stopPurge <- true + stopped <- true + return + } + } +} + +func (s *memorySeriesStorage) NewPreloader() Preloader { + return &memorySeriesPreloader{ + storage: s, + } +} + +func (s *memorySeriesStorage) GetFingerprintsForLabelMatchers(labelMatchers metric.LabelMatchers) clientmodel.Fingerprints { + s.mtx.RLock() + defer s.mtx.RUnlock() + + sets := []utility.Set{} + for _, matcher := range labelMatchers { + switch matcher.Type { + case metric.Equal: + set, ok := s.labelPairToFingerprints[metric.LabelPair{ + Name: matcher.Name, + Value: matcher.Value, + }] + if !ok { + return nil + } + sets = append(sets, set) + default: + values := s.getLabelValuesForLabelName(matcher.Name) + matches := matcher.Filter(values) + if len(matches) == 0 { + return nil + } + set := utility.Set{} + for _, v := range matches { + subset, ok := s.labelPairToFingerprints[metric.LabelPair{ + Name: matcher.Name, + Value: v, + }] + if !ok { + return nil + } + for fp := range subset { + set.Add(fp) + } + } + sets = append(sets, set) + } + } + + setCount := len(sets) + if setCount == 0 { + return nil + } + + base := sets[0] + for i := 1; i < setCount; i++ { + base = base.Intersection(sets[i]) + } + + fingerprints := clientmodel.Fingerprints{} + for _, e := range base.Elements() { + fingerprints = append(fingerprints, e.(clientmodel.Fingerprint)) + } + + return fingerprints +} + +func (s *memorySeriesStorage) GetLabelValuesForLabelName(labelName clientmodel.LabelName) clientmodel.LabelValues { + s.mtx.RLock() + defer s.mtx.RUnlock() + + return s.getLabelValuesForLabelName(labelName) +} + +func (s *memorySeriesStorage) getLabelValuesForLabelName(labelName clientmodel.LabelName) clientmodel.LabelValues { + set, ok := s.labelNameToLabelValues[labelName] + if !ok { + return nil + } + + values := make(clientmodel.LabelValues, 0, len(set)) + for e := range set { + val := e.(clientmodel.LabelValue) + values = append(values, val) + } + return values +} + +func (s *memorySeriesStorage) GetMetricForFingerprint(fp clientmodel.Fingerprint) clientmodel.Metric { + s.mtx.RLock() + defer s.mtx.RUnlock() + + series, ok := s.fingerprintToSeries[fp] + if !ok { + return nil + } + + metric := clientmodel.Metric{} + for label, value := range series.metric { + metric[label] = value + } + + return metric +} + +func (s *memorySeriesStorage) GetAllValuesForLabel(labelName clientmodel.LabelName) clientmodel.LabelValues { + s.mtx.RLock() + defer s.mtx.RUnlock() + + var values clientmodel.LabelValues + valueSet := map[clientmodel.LabelValue]struct{}{} + for _, series := range s.fingerprintToSeries { + if value, ok := series.metric[labelName]; ok { + if _, ok := valueSet[value]; !ok { + values = append(values, value) + valueSet[value] = struct{}{} + } + } + } + + return values +} diff --git a/storage/local/storage_test.go b/storage/local/storage_test.go new file mode 100644 index 0000000000..f91754ddf5 --- /dev/null +++ b/storage/local/storage_test.go @@ -0,0 +1,112 @@ +package storage_ng + +import ( + "fmt" + "testing" + "time" + + clientmodel "github.com/prometheus/client_golang/model" +) + +func TestChunk(t *testing.T) { + samples := make(clientmodel.Samples, 500000) + for i := range samples { + samples[i] = &clientmodel.Sample{ + Timestamp: clientmodel.Timestamp(i), + Value: clientmodel.SampleValue(float64(i) * 0.2), + } + } + s, closer := NewTestStorage(t) + defer closer.Close() + + s.AppendSamples(samples) + + for _, s := range s.(*memorySeriesStorage).fingerprintToSeries { + for i, v := range s.values() { + if samples[i].Timestamp != v.Timestamp { + t.Fatalf("%d. Got %v; want %v", i, v.Timestamp, samples[i].Timestamp) + } + if samples[i].Value != v.Value { + t.Fatalf("%d. Got %v; want %v", i, v.Value, samples[i].Value) + } + } + } +} + +func TestGetValueAtTime(t *testing.T) { + samples := make(clientmodel.Samples, 50000) + for i := range samples { + samples[i] = &clientmodel.Sample{ + Timestamp: clientmodel.Timestamp(time.Duration(i) * time.Second), + Value: clientmodel.SampleValue(float64(i) * 0.2), + } + } + s, closer := NewTestStorage(t) + defer closer.Close() + + s.AppendSamples(samples) + + fp := clientmodel.Metric{}.Fingerprint() + + it := s.NewIterator(fp) + + for i, expected := range samples { + actual := it.GetValueAtTime(samples[i].Timestamp) + + if expected.Timestamp != actual[0].Timestamp { + t.Fatalf("%d. Got %v; want %v", i, actual[0].Timestamp, expected.Timestamp) + } + if expected.Value != actual[0].Value { + t.Fatalf("%d. Got %v; want %v", i, actual[0].Value, expected.Value) + } + } +} + +func TestGetRangeValues(t *testing.T) { + samples := make(clientmodel.Samples, 50000) + for i := range samples { + samples[i] = &clientmodel.Sample{ + Timestamp: clientmodel.Timestamp(time.Duration(i) * time.Second), + Value: clientmodel.SampleValue(float64(i) * 0.2), + } + } + s, closer := NewTestStorage(t) + defer closer.Close() + + s.AppendSamples(samples) + + fp := clientmodel.Metric{}.Fingerprint() + + it := s.NewIterator(fp) + + for i, expected := range samples { + actual := it.GetValueAtTime(samples[i].Timestamp) + + if expected.Timestamp != actual[0].Timestamp { + t.Fatalf("%d. Got %v; want %v", i, actual[0].Timestamp, expected.Timestamp) + } + if expected.Value != actual[0].Value { + t.Fatalf("%d. Got %v; want %v", i, actual[0].Value, expected.Value) + } + } +} + +func BenchmarkAppend(b *testing.B) { + samples := make(clientmodel.Samples, b.N) + for i := range samples { + samples[i] = &clientmodel.Sample{ + Metric: clientmodel.Metric{ + clientmodel.MetricNameLabel: clientmodel.LabelValue(fmt.Sprintf("test_metric_%d", i%10)), + "label1": clientmodel.LabelValue(fmt.Sprintf("test_metric_%d", i%10)), + "label2": clientmodel.LabelValue(fmt.Sprintf("test_metric_%d", i%10)), + }, + Timestamp: clientmodel.Timestamp(i), + Value: clientmodel.SampleValue(i), + } + } + b.ResetTimer() + s, closer := NewTestStorage(b) + defer closer.Close() + + s.AppendSamples(samples) +} diff --git a/storage/local/test_helpers.go b/storage/local/test_helpers.go new file mode 100644 index 0000000000..703974266b --- /dev/null +++ b/storage/local/test_helpers.go @@ -0,0 +1,47 @@ +package storage_ng + +import ( + "testing" + "time" + + "github.com/prometheus/prometheus/utility/test" +) + +type testStorageCloser struct { + storage Storage + directory test.Closer +} + +func (t *testStorageCloser) Close() { + t.storage.Close() + t.directory.Close() +} + +func NewTestStorage(t testing.TB) (Storage, test.Closer) { + directory := test.NewTemporaryDirectory("test_storage", t) + persistence, err := NewDiskPersistence(directory.Path(), 1024) + if err != nil { + t.Fatal("Error opening disk persistence: ", err) + } + o := &MemorySeriesStorageOptions{ + Persistence: persistence, + MemoryEvictionInterval: time.Minute, + MemoryRetentionPeriod: time.Hour, + } + storage, err := NewMemorySeriesStorage(o) + if err != nil { + directory.Close() + t.Fatalf("Error creating storage: %s", err) + } + + storageStarted := make(chan bool) + go storage.Serve(storageStarted) + <-storageStarted + + closer := &testStorageCloser{ + storage: storage, + directory: directory, + } + + return storage, closer +} diff --git a/storage/metric/interface.go b/storage/metric/interface.go deleted file mode 100644 index 1bdcece9f8..0000000000 --- a/storage/metric/interface.go +++ /dev/null @@ -1,142 +0,0 @@ -// Copyright 2013 Prometheus Team -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -package metric - -import ( - "time" - - clientmodel "github.com/prometheus/client_golang/model" - - "github.com/prometheus/prometheus/stats" -) - -// Persistence is a system for storing metric samples in a persistence -// layer. -type Persistence interface { - // A storage system may rely on external resources and thusly should be - // closed when finished. - Close() - - // Record a group of new samples in the storage layer. Multiple samples for - // the same fingerprint need to be submitted in chronological order, from - // oldest to newest (both in the same call to AppendSamples and across - // multiple calls). - AppendSamples(clientmodel.Samples) error - - // Get all of the metric fingerprints that are associated with the - // provided label matchers. - GetFingerprintsForLabelMatchers(LabelMatchers) (clientmodel.Fingerprints, error) - - // Get all of the label values that are associated with a given label name. - GetLabelValuesForLabelName(clientmodel.LabelName) (clientmodel.LabelValues, error) - - // Get the metric associated with the provided fingerprint. - GetMetricForFingerprint(*clientmodel.Fingerprint) (clientmodel.Metric, error) - - // Get all label values that are associated with a given label name. - GetAllValuesForLabel(clientmodel.LabelName) (clientmodel.LabelValues, error) -} - -// PreloadingPersistence is a Persistence which supports building -// preloaded views. -type PreloadingPersistence interface { - Persistence - // NewViewRequestBuilder furnishes a ViewRequestBuilder for remarking what - // types of queries to perform. - NewViewRequestBuilder() ViewRequestBuilder -} - -// View provides a view of the values in the datastore subject to the request -// of a preloading operation. -type View interface { - // Get the two values that are immediately adjacent to a given time. - GetValueAtTime(*clientmodel.Fingerprint, clientmodel.Timestamp) Values - // Get the boundary values of an interval: the first value older than - // the interval start, and the first value younger than the interval - // end. - GetBoundaryValues(*clientmodel.Fingerprint, Interval) Values - // Get all values contained within a provided interval. - GetRangeValues(*clientmodel.Fingerprint, Interval) Values -} - -// ViewablePersistence is a Persistence that is able to present the -// samples it has stored as a View. -type ViewablePersistence interface { - Persistence - View -} - -// ViewRequestBuilder represents the summation of all datastore queries that -// shall be performed to extract values. Call the Get... methods to record the -// queries. Once done, use HasOp and PopOp to retrieve the resulting -// operations. The operations are sorted by their fingerprint (and, for equal -// fingerprints, by the StartsAt timestamp of their operation). -type ViewRequestBuilder interface { - // GetMetricAtTime records a query to get, for the given Fingerprint, - // either the value at that time if there is a match or the one or two - // values adjacent thereto. - GetMetricAtTime(fingerprint *clientmodel.Fingerprint, time clientmodel.Timestamp) - // GetMetricAtInterval records a query to get, for the given - // Fingerprint, either the value at that interval from From through - // Through if there is a match or the one or two values adjacent for - // each point. - GetMetricAtInterval(fingerprint *clientmodel.Fingerprint, from, through clientmodel.Timestamp, interval time.Duration) - // GetMetricRange records a query to get, for the given Fingerprint, the - // values that occur inclusively from From through Through. - GetMetricRange(fingerprint *clientmodel.Fingerprint, from, through clientmodel.Timestamp) - // GetMetricRangeAtInterval records a query to get value ranges at - // intervals for the given Fingerprint: - // - // |----| |----| |----| |----| - // ^ ^ ^ ^ ^ ^ - // | \------------/ \----/ | - // from interval rangeDuration through - GetMetricRangeAtInterval(fp *clientmodel.Fingerprint, from, through clientmodel.Timestamp, interval, rangeDuration time.Duration) - // Execute materializes a View, subject to a timeout. - Execute(deadline time.Duration, queryStats *stats.TimerGroup) (View, error) - // PopOp emits the next operation in the queue (sorted by - // fingerprint). If called while HasOps returns false, the - // behavior is undefined. - PopOp() Op - // HasOp returns true if there is at least one more operation in the - // queue. - HasOp() bool -} - -// Op encapsulates a primitive query operation. -type Op interface { - // Fingerprint returns the fingerprint of the metric this operation - // operates on. - Fingerprint() *clientmodel.Fingerprint - // ExtractSamples extracts samples from a stream of values and advances - // the operation time. - ExtractSamples(Values) Values - // Consumed returns whether the operator has consumed all data it needs. - Consumed() bool - // CurrentTime gets the current operation time. In a newly created op, - // this is the starting time of the operation. During ongoing execution - // of the op, the current time is advanced accordingly. Once no - // subsequent work associated with the operation remains, nil is - // returned. - CurrentTime() clientmodel.Timestamp -} - -// CurationState contains high-level curation state information for the -// heads-up-display. -type CurationState struct { - Active bool - Name string - Limit time.Duration - Fingerprint *clientmodel.Fingerprint -} diff --git a/storage/metric/sample.go b/storage/metric/sample.go index 4061cb1d0c..d5f2ee9ff1 100644 --- a/storage/metric/sample.go +++ b/storage/metric/sample.go @@ -14,9 +14,7 @@ package metric import ( - "bytes" "fmt" - "sort" clientmodel "github.com/prometheus/client_golang/model" ) @@ -32,6 +30,7 @@ type SamplePair struct { Value clientmodel.SampleValue } +// TODO: can this method be deleted, or is it used in tests? // Equal returns true if this SamplePair and o have equal Values and equal // Timestamps. func (s *SamplePair) Equal(o *SamplePair) bool { @@ -46,96 +45,9 @@ func (s *SamplePair) String() string { return fmt.Sprintf("SamplePair at %s of %s", s.Timestamp, s.Value) } -// Values is a sortable slice of SamplePairs (as in: it implements -// sort.Interface). Sorting happens by Timestamp. +// Values is a slice of SamplePairs. type Values []SamplePair -// Len implements sort.Interface. -func (v Values) Len() int { - return len(v) -} - -// Less implements sort.Interface. -func (v Values) Less(i, j int) bool { - return v[i].Timestamp.Before(v[j].Timestamp) -} - -// Swap implements sort.Interface. -func (v Values) Swap(i, j int) { - v[i], v[j] = v[j], v[i] -} - -// Equal returns true if these Values are of the same length as o, and each -// value is equal to the corresponding value in o (i.e. at the same index). -func (v Values) Equal(o Values) bool { - if len(v) != len(o) { - return false - } - - for i, expected := range v { - if !expected.Equal(&o[i]) { - return false - } - } - - return true -} - -// FirstTimeAfter indicates whether the first sample of a set is after a given -// timestamp. -func (v Values) FirstTimeAfter(t clientmodel.Timestamp) bool { - return v[0].Timestamp.After(t) -} - -// LastTimeBefore indicates whether the last sample of a set is before a given -// timestamp. -func (v Values) LastTimeBefore(t clientmodel.Timestamp) bool { - return v[len(v)-1].Timestamp.Before(t) -} - -// InsideInterval indicates whether a given range of sorted values could contain -// a value for a given time. -func (v Values) InsideInterval(t clientmodel.Timestamp) bool { - switch { - case v.Len() == 0: - return false - case t.Before(v[0].Timestamp): - return false - case !v[v.Len()-1].Timestamp.Before(t): - return false - default: - return true - } -} - -// TruncateBefore returns a subslice of the original such that extraneous -// samples in the collection that occur before the provided time are -// dropped. The original slice is not mutated. -func (v Values) TruncateBefore(t clientmodel.Timestamp) Values { - index := sort.Search(len(v), func(i int) bool { - timestamp := v[i].Timestamp - - return !timestamp.Before(t) - }) - - return v[index:] -} - -func (v Values) String() string { - buffer := bytes.Buffer{} - - fmt.Fprintf(&buffer, "[") - for i, value := range v { - fmt.Fprintf(&buffer, "%d. %s", i, value) - if i != len(v)-1 { - fmt.Fprintf(&buffer, "\n") - } - } - fmt.Fprintf(&buffer, "]") - - return buffer.String() -} - // SampleSet is Values with a Metric attached. type SampleSet struct { Metric clientmodel.Metric diff --git a/storage/metric/tiered/compaction_regression_test.go b/storage/metric/tiered/compaction_regression_test.go deleted file mode 100644 index 1a78f34ed2..0000000000 --- a/storage/metric/tiered/compaction_regression_test.go +++ /dev/null @@ -1,267 +0,0 @@ -// Copyright 2013 Prometheus Team -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -package tiered - -import ( - "flag" - "fmt" - "sort" - "testing" - "time" - - "github.com/prometheus/prometheus/storage" - "github.com/prometheus/prometheus/storage/metric" - - clientmodel "github.com/prometheus/client_golang/model" -) - -type nopCurationStateUpdater struct{} - -func (n *nopCurationStateUpdater) UpdateCurationState(*metric.CurationState) {} - -func generateTestSamples(endTime clientmodel.Timestamp, numTs int, samplesPerTs int, interval time.Duration) clientmodel.Samples { - samples := make(clientmodel.Samples, 0, numTs*samplesPerTs) - - startTime := endTime.Add(-interval * time.Duration(samplesPerTs-1)) - for ts := 0; ts < numTs; ts++ { - metric := clientmodel.Metric{} - metric[clientmodel.MetricNameLabel] = clientmodel.LabelValue(fmt.Sprintf("metric_%d", ts)) - for i := 0; i < samplesPerTs; i++ { - sample := &clientmodel.Sample{ - Metric: metric, - Value: clientmodel.SampleValue(ts + 1000*i), - Timestamp: startTime.Add(interval * time.Duration(i)), - } - samples = append(samples, sample) - } - } - sort.Sort(samples) - return samples -} - -type compactionChecker struct { - t *testing.T - sampleIdx int - numChunks int - expectedSamples clientmodel.Samples -} - -func (c *compactionChecker) Operate(key, value interface{}) *storage.OperatorError { - c.numChunks++ - sampleKey := key.(*SampleKey) - if sampleKey.FirstTimestamp.After(sampleKey.LastTimestamp) { - c.t.Fatalf("Chunk FirstTimestamp (%v) is after LastTimestamp (%v): %v", sampleKey.FirstTimestamp.Unix(), sampleKey.LastTimestamp.Unix(), sampleKey) - } - fp := &clientmodel.Fingerprint{} - for _, sample := range value.(metric.Values) { - if sample.Timestamp.Before(sampleKey.FirstTimestamp) || sample.Timestamp.After(sampleKey.LastTimestamp) { - c.t.Fatalf("Sample not within chunk boundaries: chunk FirstTimestamp (%v), chunk LastTimestamp (%v) vs. sample Timestamp (%v)", sampleKey.FirstTimestamp.Unix(), sampleKey.LastTimestamp.Unix(), sample.Timestamp) - } - - expected := c.expectedSamples[c.sampleIdx] - - fp.LoadFromMetric(expected.Metric) - if !sampleKey.Fingerprint.Equal(fp) { - c.t.Fatalf("%d. Expected fingerprint %s, got %s", c.sampleIdx, fp, sampleKey.Fingerprint) - } - - sp := &metric.SamplePair{ - Value: expected.Value, - Timestamp: expected.Timestamp, - } - if !sample.Equal(sp) { - c.t.Fatalf("%d. Expected sample %s, got %s", c.sampleIdx, sp, sample) - } - c.sampleIdx++ - } - return nil -} - -func checkStorageSaneAndEquivalent(t *testing.T, name string, ts *TieredStorage, samples clientmodel.Samples, expectedNumChunks int) { - cc := &compactionChecker{ - expectedSamples: samples, - t: t, - } - entire, err := ts.DiskStorage.MetricSamples.ForEach(&MetricSamplesDecoder{}, &AcceptAllFilter{}, cc) - if err != nil { - t.Fatalf("%s: Error checking samples: %s", name, err) - } - if !entire { - t.Fatalf("%s: Didn't scan entire corpus", name) - } - if cc.numChunks != expectedNumChunks { - t.Fatalf("%s: Expected %d chunks, got %d", name, expectedNumChunks, cc.numChunks) - } -} - -type compactionTestScenario struct { - leveldbChunkSize int - numTimeseries int - samplesPerTs int - - ignoreYoungerThan time.Duration - maximumMutationPoolBatch int - minimumGroupSize int - - uncompactedChunks int - compactedChunks int -} - -func (s compactionTestScenario) test(t *testing.T) { - defer flag.Set("leveldbChunkSize", flag.Lookup("leveldbChunkSize").Value.String()) - flag.Set("leveldbChunkSize", fmt.Sprintf("%d", s.leveldbChunkSize)) - - ts, closer := NewTestTieredStorage(t) - defer closer.Close() - - // 1. Store test values. - samples := generateTestSamples(testInstant, s.numTimeseries, s.samplesPerTs, time.Minute) - ts.AppendSamples(samples) - ts.Flush() - - // 2. Check sanity of uncompacted values. - checkStorageSaneAndEquivalent(t, "Before compaction", ts, samples, s.uncompactedChunks) - - // 3. Compact test storage. - processor := NewCompactionProcessor(&CompactionProcessorOptions{ - MaximumMutationPoolBatch: s.maximumMutationPoolBatch, - MinimumGroupSize: s.minimumGroupSize, - }) - defer processor.Close() - - curator := NewCurator(&CuratorOptions{ - Stop: make(chan struct{}), - ViewQueue: ts.ViewQueue, - }) - defer curator.Close() - - err := curator.Run(s.ignoreYoungerThan, testInstant, processor, ts.DiskStorage.CurationRemarks, ts.DiskStorage.MetricSamples, ts.DiskStorage.MetricHighWatermarks, &nopCurationStateUpdater{}) - if err != nil { - t.Fatalf("Failed to run curator: %s", err) - } - - // 4. Check sanity of compacted values. - checkStorageSaneAndEquivalent(t, "After compaction", ts, samples, s.compactedChunks) -} - -func TestCompaction(t *testing.T) { - scenarios := []compactionTestScenario{ - // BEFORE COMPACTION: - // - // Chunk size | Fingerprint | Samples - // 5 | A | 1 .. 5 - // 5 | A | 6 .. 10 - // 5 | A | 11 .. 15 - // 5 | B | 1 .. 5 - // 5 | B | 6 .. 10 - // 5 | B | 11 .. 15 - // 5 | C | 1 .. 5 - // 5 | C | 6 .. 10 - // 5 | C | 11 .. 15 - // - // AFTER COMPACTION: - // - // Chunk size | Fingerprint | Samples - // 10 | A | 1 .. 10 - // 5 | A | 11 .. 15 - // 10 | B | 1 .. 10 - // 5 | B | 11 .. 15 - // 10 | C | 1 .. 10 - // 5 | C | 11 .. 15 - { - leveldbChunkSize: 5, - numTimeseries: 3, - samplesPerTs: 15, - - ignoreYoungerThan: time.Minute, - maximumMutationPoolBatch: 30, - minimumGroupSize: 10, - - uncompactedChunks: 9, - compactedChunks: 6, - }, - // BEFORE COMPACTION: - // - // Chunk size | Fingerprint | Samples - // 5 | A | 1 .. 5 - // 5 | A | 6 .. 10 - // 5 | A | 11 .. 15 - // 5 | B | 1 .. 5 - // 5 | B | 6 .. 10 - // 5 | B | 11 .. 15 - // 5 | C | 1 .. 5 - // 5 | C | 6 .. 10 - // 5 | C | 11 .. 15 - // - // AFTER COMPACTION: - // - // Chunk size | Fingerprint | Samples - // 10 | A | 1 .. 15 - // 10 | B | 1 .. 15 - // 10 | C | 1 .. 15 - { - leveldbChunkSize: 5, - numTimeseries: 3, - samplesPerTs: 15, - - ignoreYoungerThan: time.Minute, - maximumMutationPoolBatch: 30, - minimumGroupSize: 30, - - uncompactedChunks: 9, - compactedChunks: 3, - }, - // BEFORE COMPACTION: - // - // Chunk size | Fingerprint | Samples - // 5 | A | 1 .. 5 - // 5 | A | 6 .. 10 - // 5 | A | 11 .. 15 - // 5 | A | 16 .. 20 - // 5 | B | 1 .. 5 - // 5 | B | 6 .. 10 - // 5 | B | 11 .. 15 - // 5 | B | 16 .. 20 - // 5 | C | 1 .. 5 - // 5 | C | 6 .. 10 - // 5 | C | 11 .. 15 - // 5 | C | 16 .. 20 - // - // AFTER COMPACTION: - // - // Chunk size | Fingerprint | Samples - // 10 | A | 1 .. 15 - // 10 | A | 16 .. 20 - // 10 | B | 1 .. 15 - // 10 | B | 16 .. 20 - // 10 | C | 1 .. 15 - // 10 | C | 16 .. 20 - { - leveldbChunkSize: 5, - numTimeseries: 3, - samplesPerTs: 20, - - ignoreYoungerThan: time.Minute, - maximumMutationPoolBatch: 30, - minimumGroupSize: 10, - - uncompactedChunks: 12, - compactedChunks: 6, - }, - } - - for _, s := range scenarios { - s.test(t) - } -} diff --git a/storage/metric/tiered/curator.go b/storage/metric/tiered/curator.go deleted file mode 100644 index ace1d58eb5..0000000000 --- a/storage/metric/tiered/curator.go +++ /dev/null @@ -1,509 +0,0 @@ -// Copyright 2013 Prometheus Team -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -package tiered - -import ( - "bytes" - "errors" - "fmt" - "strings" - "time" - - "code.google.com/p/goprotobuf/proto" - "github.com/golang/glog" - "github.com/prometheus/client_golang/prometheus" - - clientmodel "github.com/prometheus/client_golang/model" - - "github.com/prometheus/prometheus/storage" - "github.com/prometheus/prometheus/storage/metric" - "github.com/prometheus/prometheus/storage/raw" - "github.com/prometheus/prometheus/storage/raw/leveldb" - - dto "github.com/prometheus/prometheus/model/generated" -) - -const curationYieldPeriod = 250 * time.Millisecond - -var errIllegalIterator = errors.New("iterator invalid") - -// Constants for instrumentation. -const ( - cutOff = "recency_threshold" - processorName = "processor" -) - -var ( - curationDurations = prometheus.NewSummaryVec( - prometheus.SummaryOpts{ - Namespace: namespace, - Name: "curation_durations_milliseconds", - Help: "Histogram of time spent in curation.", - Objectives: []float64{0.01, 0.05, 0.5, 0.90, 0.99}, - }, - []string{cutOff, processorName, result}, - ) - curationFilterOperations = prometheus.NewCounterVec( - prometheus.CounterOpts{ - Namespace: namespace, - Name: "curation_filter_operations_total", - Help: "The number of curation filter operations completed.", - }, - []string{cutOff, processorName, result}, - ) -) - -func init() { - prometheus.MustRegister(curationDurations) - prometheus.MustRegister(curationFilterOperations) -} - -// CurationStateUpdater receives updates about the curation state. -type CurationStateUpdater interface { - UpdateCurationState(*metric.CurationState) -} - -// CuratorOptions bundles the parameters needed to create a Curator. -type CuratorOptions struct { - Stop chan struct{} - - ViewQueue chan viewJob -} - -// Curator is responsible for effectuating a given curation policy across the -// stored samples on-disk. This is useful to compact sparse sample values into -// single sample entities to reduce keyspace load on the datastore. -type Curator struct { - stop chan struct{} - - viewQueue chan viewJob - - dtoSampleKeys *dtoSampleKeyList - sampleKeys *sampleKeyList -} - -// NewCurator returns an initialized Curator. -func NewCurator(o *CuratorOptions) *Curator { - return &Curator{ - stop: o.Stop, - - viewQueue: o.ViewQueue, - - dtoSampleKeys: newDtoSampleKeyList(10), - sampleKeys: newSampleKeyList(10), - } -} - -// watermarkScanner converts (dto.Fingerprint, dto.MetricHighWatermark) doubles -// into (model.Fingerprint, model.Watermark) doubles. -// -// watermarkScanner determines whether to include or exclude candidate -// values from the curation process by virtue of how old the high watermark is. -// -// watermarkScanner scans over the curator.samples table for metrics whose -// high watermark has been determined to be allowable for curation. This type -// is individually responsible for compaction. -// -// The scanning starts from CurationRemark.LastCompletionTimestamp and goes -// forward until the stop point or end of the series is reached. -type watermarkScanner struct { - // curationState is the data store for curation remarks. - curationState CurationRemarker - // ignoreYoungerThan is passed into the curation remark for the given series. - ignoreYoungerThan time.Duration - // processor is responsible for executing a given stategy on the - // to-be-operated-on series. - processor Processor - // sampleIterator is a snapshotted iterator for the time series. - sampleIterator leveldb.Iterator - // samples - samples raw.Persistence - // stopAt is a cue for when to stop mutating a given series. - stopAt clientmodel.Timestamp - - // stop functions as the global stop channel for all future operations. - stop chan struct{} - // status is the outbound channel for notifying the status page of its state. - status CurationStateUpdater - - firstBlock, lastBlock *SampleKey - - ViewQueue chan viewJob - - dtoSampleKeys *dtoSampleKeyList - sampleKeys *sampleKeyList -} - -// Run facilitates the curation lifecycle. -// -// recencyThreshold represents the most recent time up to which values will be -// curated. -// curationState is the on-disk store where the curation remarks are made for -// how much progress has been made. -func (c *Curator) Run(ignoreYoungerThan time.Duration, instant clientmodel.Timestamp, processor Processor, curationState CurationRemarker, samples *leveldb.LevelDBPersistence, watermarks HighWatermarker, status CurationStateUpdater) (err error) { - defer func(t time.Time) { - duration := float64(time.Since(t) / time.Millisecond) - - labels := prometheus.Labels{ - cutOff: fmt.Sprint(ignoreYoungerThan), - processorName: processor.Name(), - result: success, - } - if err != nil { - labels[result] = failure - } - - curationDurations.With(labels).Observe(duration) - }(time.Now()) - - defer status.UpdateCurationState(&metric.CurationState{Active: false}) - - iterator, err := samples.NewIterator(true) - if err != nil { - return err - } - defer iterator.Close() - - if !iterator.SeekToLast() { - glog.Info("Empty database; skipping curation.") - - return - } - - keyDto, _ := c.dtoSampleKeys.Get() - defer c.dtoSampleKeys.Give(keyDto) - - lastBlock, _ := c.sampleKeys.Get() - defer c.sampleKeys.Give(lastBlock) - - if err := iterator.Key(keyDto); err != nil { - panic(err) - } - - lastBlock.Load(keyDto) - - if !iterator.SeekToFirst() { - glog.Info("Empty database; skipping curation.") - - return - } - - firstBlock, _ := c.sampleKeys.Get() - defer c.sampleKeys.Give(firstBlock) - - if err := iterator.Key(keyDto); err != nil { - panic(err) - } - - firstBlock.Load(keyDto) - - scanner := &watermarkScanner{ - curationState: curationState, - ignoreYoungerThan: ignoreYoungerThan, - processor: processor, - status: status, - stop: c.stop, - stopAt: instant.Add(-1 * ignoreYoungerThan), - - sampleIterator: iterator, - samples: samples, - - firstBlock: firstBlock, - lastBlock: lastBlock, - - ViewQueue: c.viewQueue, - - dtoSampleKeys: c.dtoSampleKeys, - sampleKeys: c.sampleKeys, - } - - // Right now, the ability to stop a curation is limited to the beginning of - // each fingerprint cycle. It is impractical to cease the work once it has - // begun for a given series. - _, err = watermarks.ForEach(scanner, scanner, scanner) - - return -} - -// Close needs to be called to cleanly dispose of a curator. -func (c *Curator) Close() { - c.dtoSampleKeys.Close() - c.sampleKeys.Close() -} - -func (w *watermarkScanner) DecodeKey(in interface{}) (interface{}, error) { - key := &dto.Fingerprint{} - bytes := in.([]byte) - - if err := proto.Unmarshal(bytes, key); err != nil { - return nil, err - } - - fingerprint := &clientmodel.Fingerprint{} - loadFingerprint(fingerprint, key) - - return fingerprint, nil -} - -func (w *watermarkScanner) DecodeValue(in interface{}) (interface{}, error) { - value := &dto.MetricHighWatermark{} - bytes := in.([]byte) - - if err := proto.Unmarshal(bytes, value); err != nil { - return nil, err - } - - watermark := &watermarks{} - watermark.load(value) - - return watermark, nil -} - -func (w *watermarkScanner) shouldStop() bool { - select { - case _, ok := <-w.stop: - if ok { - panic("channel should be closed only") - } - return true - default: - return false - } -} - -func (w *watermarkScanner) Filter(key, value interface{}) (r storage.FilterResult) { - fingerprint := key.(*clientmodel.Fingerprint) - - defer func() { - labels := prometheus.Labels{ - cutOff: fmt.Sprint(w.ignoreYoungerThan), - result: strings.ToLower(r.String()), - processorName: w.processor.Name(), - } - - curationFilterOperations.With(labels).Inc() - - w.status.UpdateCurationState(&metric.CurationState{ - Active: true, - Name: w.processor.Name(), - Limit: w.ignoreYoungerThan, - Fingerprint: fingerprint, - }) - }() - - if w.shouldStop() { - return storage.Stop - } - - k := &curationKey{ - Fingerprint: fingerprint, - ProcessorMessageRaw: w.processor.Signature(), - ProcessorMessageTypeName: w.processor.Name(), - IgnoreYoungerThan: w.ignoreYoungerThan, - } - - curationRemark, present, err := w.curationState.Get(k) - if err != nil { - return - } - if !present { - return storage.Accept - } - if !curationRemark.Before(w.stopAt) { - return storage.Skip - } - watermark := value.(*watermarks) - if !curationRemark.Before(watermark.High) { - return storage.Skip - } - curationConsistent, err := w.curationConsistent(fingerprint, watermark) - if err != nil { - return - } - if curationConsistent { - return storage.Skip - } - - return storage.Accept -} - -// curationConsistent determines whether the given metric is in a dirty state -// and needs curation. -func (w *watermarkScanner) curationConsistent(f *clientmodel.Fingerprint, watermark *watermarks) (bool, error) { - k := &curationKey{ - Fingerprint: f, - ProcessorMessageRaw: w.processor.Signature(), - ProcessorMessageTypeName: w.processor.Name(), - IgnoreYoungerThan: w.ignoreYoungerThan, - } - curationRemark, present, err := w.curationState.Get(k) - if err != nil { - return false, err - } - if !present { - return false, nil - } - if !curationRemark.Before(watermark.High) { - return true, nil - } - - return false, nil -} - -func (w *watermarkScanner) Operate(key, _ interface{}) (oErr *storage.OperatorError) { - fingerprint := key.(*clientmodel.Fingerprint) - - glog.Infof("Curating %s...", fingerprint) - - if len(w.ViewQueue) > 0 { - glog.Warning("Deferred due to view queue.") - time.Sleep(curationYieldPeriod) - } - - if fingerprint.Less(w.firstBlock.Fingerprint) { - glog.Warning("Skipped since before keyspace.") - return nil - } - if w.lastBlock.Fingerprint.Less(fingerprint) { - glog.Warning("Skipped since after keyspace.") - return nil - } - - curationState, _, err := w.curationState.Get(&curationKey{ - Fingerprint: fingerprint, - ProcessorMessageRaw: w.processor.Signature(), - ProcessorMessageTypeName: w.processor.Name(), - IgnoreYoungerThan: w.ignoreYoungerThan, - }) - if err != nil { - glog.Warning("Unable to get curation state: %s", err) - // An anomaly with the curation remark is likely not fatal in the sense that - // there was a decoding error with the entity and shouldn't be cause to stop - // work. The process will simply start from a pessimistic work time and - // work forward. With an idempotent processor, this is safe. - return &storage.OperatorError{Error: err, Continuable: true} - } - - keySet, _ := w.sampleKeys.Get() - defer w.sampleKeys.Give(keySet) - - keySet.Fingerprint = fingerprint - keySet.FirstTimestamp = curationState - - // Invariant: The fingerprint tests above ensure that we have the same - // fingerprint. - keySet.Constrain(w.firstBlock, w.lastBlock) - - seeker := &iteratorSeekerState{ - i: w.sampleIterator, - - obj: keySet, - - first: w.firstBlock, - last: w.lastBlock, - - dtoSampleKeys: w.dtoSampleKeys, - sampleKeys: w.sampleKeys, - } - - for state := seeker.initialize; state != nil; state = state() { - } - - if seeker.err != nil { - glog.Warningf("Got error in state machine: %s", seeker.err) - - return &storage.OperatorError{Error: seeker.err, Continuable: !seeker.iteratorInvalid} - } - - if seeker.iteratorInvalid { - glog.Warningf("Got illegal iterator in state machine: %s", err) - - return &storage.OperatorError{Error: errIllegalIterator, Continuable: false} - } - - if !seeker.seriesOperable { - return - } - - lastTime, err := w.processor.Apply(w.sampleIterator, w.samples, w.stopAt, fingerprint) - if err != nil { - // We can't divine the severity of a processor error without refactoring the - // interface. - return &storage.OperatorError{Error: err, Continuable: false} - } - - if err = w.curationState.Update(&curationKey{ - Fingerprint: fingerprint, - ProcessorMessageRaw: w.processor.Signature(), - ProcessorMessageTypeName: w.processor.Name(), - IgnoreYoungerThan: w.ignoreYoungerThan, - }, lastTime); err != nil { - // Under the assumption that the processors are idempotent, they can be - // re-run; thusly, the commitment of the curation remark is no cause - // to cease further progress. - return &storage.OperatorError{Error: err, Continuable: true} - } - - return nil -} - -// curationKey provides a representation of dto.CurationKey with associated -// business logic methods attached to it to enhance code readability. -type curationKey struct { - Fingerprint *clientmodel.Fingerprint - ProcessorMessageRaw []byte - ProcessorMessageTypeName string - IgnoreYoungerThan time.Duration -} - -// Equal answers whether the two curationKeys are equivalent. -func (c *curationKey) Equal(o *curationKey) bool { - switch { - case !c.Fingerprint.Equal(o.Fingerprint): - return false - case bytes.Compare(c.ProcessorMessageRaw, o.ProcessorMessageRaw) != 0: - return false - case c.ProcessorMessageTypeName != o.ProcessorMessageTypeName: - return false - case c.IgnoreYoungerThan != o.IgnoreYoungerThan: - return false - } - - return true -} - -func (c *curationKey) dump(d *dto.CurationKey) { - d.Reset() - - // BUG(matt): Avenue for simplification. - fingerprintDTO := &dto.Fingerprint{} - - dumpFingerprint(fingerprintDTO, c.Fingerprint) - - d.Fingerprint = fingerprintDTO - d.ProcessorMessageRaw = c.ProcessorMessageRaw - d.ProcessorMessageTypeName = proto.String(c.ProcessorMessageTypeName) - d.IgnoreYoungerThan = proto.Int64(int64(c.IgnoreYoungerThan)) -} - -func (c *curationKey) load(d *dto.CurationKey) { - // BUG(matt): Avenue for simplification. - c.Fingerprint = &clientmodel.Fingerprint{} - - loadFingerprint(c.Fingerprint, d.Fingerprint) - - c.ProcessorMessageRaw = d.ProcessorMessageRaw - c.ProcessorMessageTypeName = d.GetProcessorMessageTypeName() - c.IgnoreYoungerThan = time.Duration(d.GetIgnoreYoungerThan()) -} diff --git a/storage/metric/tiered/dto.go b/storage/metric/tiered/dto.go deleted file mode 100644 index c96b69f0be..0000000000 --- a/storage/metric/tiered/dto.go +++ /dev/null @@ -1,68 +0,0 @@ -// Copyright 2013 Prometheus Team -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -package tiered - -import ( - "sort" - - "code.google.com/p/goprotobuf/proto" - - clientmodel "github.com/prometheus/client_golang/model" - - dto "github.com/prometheus/prometheus/model/generated" -) - -func dumpFingerprint(d *dto.Fingerprint, f *clientmodel.Fingerprint) { - d.Reset() - - d.Signature = proto.String(f.String()) -} - -func loadFingerprint(f *clientmodel.Fingerprint, d *dto.Fingerprint) { - f.LoadFromString(d.GetSignature()) -} - -func dumpMetric(d *dto.Metric, m clientmodel.Metric) { - d.Reset() - - metricLength := len(m) - labelNames := make([]string, 0, metricLength) - - for labelName := range m { - labelNames = append(labelNames, string(labelName)) - } - - sort.Strings(labelNames) - - pairs := make([]*dto.LabelPair, 0, metricLength) - - for _, labelName := range labelNames { - l := clientmodel.LabelName(labelName) - labelValue := m[l] - labelPair := &dto.LabelPair{ - Name: proto.String(string(labelName)), - Value: proto.String(string(labelValue)), - } - - pairs = append(pairs, labelPair) - } - - d.LabelPair = pairs -} - -func dumpLabelName(d *dto.LabelName, l clientmodel.LabelName) { - d.Reset() - - d.Name = proto.String(string(l)) -} diff --git a/storage/metric/tiered/end_to_end_test.go b/storage/metric/tiered/end_to_end_test.go deleted file mode 100644 index a321d08834..0000000000 --- a/storage/metric/tiered/end_to_end_test.go +++ /dev/null @@ -1,548 +0,0 @@ -// Copyright 2013 Prometheus Team -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -package tiered - -import ( - "sort" - "testing" - "time" - - clientmodel "github.com/prometheus/client_golang/model" - - "github.com/prometheus/prometheus/storage/metric" -) - -func GetFingerprintsForLabelSetTests(p metric.Persistence, t testing.TB) { - metrics := []clientmodel.Metric{ - { - clientmodel.MetricNameLabel: "test_metric", - "method": "get", - "result": "success", - }, - { - clientmodel.MetricNameLabel: "test_metric", - "method": "get", - "result": "failure", - }, - { - clientmodel.MetricNameLabel: "test_metric", - "method": "post", - "result": "success", - }, - { - clientmodel.MetricNameLabel: "test_metric", - "method": "post", - "result": "failure", - }, - } - - newTestLabelMatcher := func(matchType metric.MatchType, name clientmodel.LabelName, value clientmodel.LabelValue) *metric.LabelMatcher { - m, err := metric.NewLabelMatcher(matchType, name, value) - if err != nil { - t.Fatalf("Couldn't create label matcher: %v", err) - } - return m - } - - scenarios := []struct { - in metric.LabelMatchers - outIndexes []int - }{ - { - in: metric.LabelMatchers{ - newTestLabelMatcher(metric.Equal, clientmodel.MetricNameLabel, "test_metric"), - }, - outIndexes: []int{0, 1, 2, 3}, - }, - { - in: metric.LabelMatchers{ - newTestLabelMatcher(metric.Equal, clientmodel.MetricNameLabel, "non_existent_metric"), - }, - outIndexes: []int{}, - }, - { - in: metric.LabelMatchers{ - newTestLabelMatcher(metric.Equal, clientmodel.MetricNameLabel, "non_existent_metric"), - newTestLabelMatcher(metric.Equal, "result", "success"), - }, - outIndexes: []int{}, - }, - { - in: metric.LabelMatchers{ - newTestLabelMatcher(metric.Equal, clientmodel.MetricNameLabel, "test_metric"), - newTestLabelMatcher(metric.Equal, "result", "success"), - }, - outIndexes: []int{0, 2}, - }, - { - in: metric.LabelMatchers{ - newTestLabelMatcher(metric.Equal, clientmodel.MetricNameLabel, "test_metric"), - newTestLabelMatcher(metric.NotEqual, "result", "success"), - }, - outIndexes: []int{1, 3}, - }, - { - in: metric.LabelMatchers{ - newTestLabelMatcher(metric.Equal, clientmodel.MetricNameLabel, "test_metric"), - newTestLabelMatcher(metric.RegexMatch, "result", "foo|success|bar"), - }, - outIndexes: []int{0, 2}, - }, - { - in: metric.LabelMatchers{ - newTestLabelMatcher(metric.Equal, clientmodel.MetricNameLabel, "test_metric"), - newTestLabelMatcher(metric.RegexNoMatch, "result", "foo|success|bar"), - }, - outIndexes: []int{1, 3}, - }, - { - in: metric.LabelMatchers{ - newTestLabelMatcher(metric.Equal, clientmodel.MetricNameLabel, "test_metric"), - newTestLabelMatcher(metric.RegexNoMatch, "result", "foo|success|bar"), - newTestLabelMatcher(metric.RegexMatch, "method", "os"), - }, - outIndexes: []int{3}, - }, - } - - for _, m := range metrics { - testAppendSamples(p, &clientmodel.Sample{ - Value: 0, - Timestamp: 0, - Metric: m, - }, t) - } - - for i, s := range scenarios { - actualFps, err := p.GetFingerprintsForLabelMatchers(s.in) - if err != nil { - t.Fatalf("%d. Couldn't get fingerprints for label matchers: %v", i, err) - } - - expectedFps := clientmodel.Fingerprints{} - for _, i := range s.outIndexes { - fp := &clientmodel.Fingerprint{} - fp.LoadFromMetric(metrics[i]) - expectedFps = append(expectedFps, fp) - } - - sort.Sort(actualFps) - sort.Sort(expectedFps) - - if len(actualFps) != len(expectedFps) { - t.Fatalf("%d. Got %d fingerprints; want %d", i, len(actualFps), len(expectedFps)) - } - - for j, actualFp := range actualFps { - if !actualFp.Equal(expectedFps[j]) { - t.Fatalf("%d.%d. Got fingerprint %v; want %v", i, j, actualFp, expectedFps[j]) - } - } - } -} - -func GetLabelValuesForLabelNameTests(p metric.Persistence, t testing.TB) { - testAppendSamples(p, &clientmodel.Sample{ - Value: 0, - Timestamp: 0, - Metric: clientmodel.Metric{ - clientmodel.MetricNameLabel: "my_metric", - "request_type": "create", - "result": "success", - }, - }, t) - - testAppendSamples(p, &clientmodel.Sample{ - Value: 0, - Timestamp: 0, - Metric: clientmodel.Metric{ - clientmodel.MetricNameLabel: "my_metric", - "request_type": "delete", - "outcome": "failure", - }, - }, t) - - expectedIndex := map[clientmodel.LabelName]clientmodel.LabelValues{ - clientmodel.MetricNameLabel: {"my_metric"}, - "request_type": {"create", "delete"}, - "result": {"success"}, - "outcome": {"failure"}, - } - - for name, expected := range expectedIndex { - actual, err := p.GetLabelValuesForLabelName(name) - if err != nil { - t.Fatalf("Error getting values for label %s: %v", name, err) - } - if len(actual) != len(expected) { - t.Fatalf("Number of values don't match for label %s: got %d; want %d", name, len(actual), len(expected)) - } - for i := range expected { - inActual := false - for _, a := range actual { - if expected[i] == a { - inActual = true - break - } - } - if !inActual { - t.Fatalf("%d. Expected label value %s not in output", i, expected[i]) - } - } - } -} - -func GetMetricForFingerprintTests(p metric.Persistence, t testing.TB) { - testAppendSamples(p, &clientmodel.Sample{ - Value: 0, - Timestamp: 0, - Metric: clientmodel.Metric{ - "request_type": "your_mom", - }, - }, t) - - testAppendSamples(p, &clientmodel.Sample{ - Value: 0, - Timestamp: 0, - Metric: clientmodel.Metric{ - "request_type": "your_dad", - "one-off": "value", - }, - }, t) - - result, err := p.GetFingerprintsForLabelMatchers(metric.LabelMatchers{{ - Type: metric.Equal, - Name: "request_type", - Value: "your_mom", - }}) - - if err != nil { - t.Error(err) - } - - if len(result) != 1 { - t.Errorf("Expected one element.") - } - - m, err := p.GetMetricForFingerprint(result[0]) - if err != nil { - t.Error(err) - } - - if m == nil { - t.Fatal("Did not expect nil.") - } - - if len(m) != 1 { - t.Errorf("Expected one-dimensional metric.") - } - - if m["request_type"] != "your_mom" { - t.Errorf("Expected metric to match.") - } - - result, err = p.GetFingerprintsForLabelMatchers(metric.LabelMatchers{{ - Type: metric.Equal, - Name: "request_type", - Value: "your_dad", - }}) - - if err != nil { - t.Error(err) - } - - if len(result) != 1 { - t.Errorf("Expected one element.") - } - - m, err = p.GetMetricForFingerprint(result[0]) - - if m == nil { - t.Fatal("Did not expect nil.") - } - - if err != nil { - t.Error(err) - } - - if len(m) != 2 { - t.Errorf("Expected two-dimensional metric.") - } - - if m["request_type"] != "your_dad" { - t.Errorf("Expected metric to match.") - } - - if m["one-off"] != "value" { - t.Errorf("Expected metric to match.") - } - - // Verify that mutating a returned metric does not result in the mutated - // metric to be returned at the next GetMetricForFingerprint() call. - m["one-off"] = "new value" - m, err = p.GetMetricForFingerprint(result[0]) - - if m == nil { - t.Fatal("Did not expect nil.") - } - - if err != nil { - t.Error(err) - } - - if len(m) != 2 { - t.Errorf("Expected two-dimensional metric.") - } - - if m["request_type"] != "your_dad" { - t.Errorf("Expected metric to match.") - } - - if m["one-off"] != "value" { - t.Errorf("Expected metric to match.") - } -} - -func AppendRepeatingValuesTests(p metric.Persistence, t testing.TB) { - m := clientmodel.Metric{ - clientmodel.MetricNameLabel: "errors_total", - "controller": "foo", - "operation": "bar", - } - - increments := 10 - repetitions := 500 - - for i := 0; i < increments; i++ { - for j := 0; j < repetitions; j++ { - time := clientmodel.Timestamp(0).Add(time.Duration(i) * time.Hour).Add(time.Duration(j) * time.Second) - testAppendSamples(p, &clientmodel.Sample{ - Value: clientmodel.SampleValue(i), - Timestamp: time, - Metric: m, - }, t) - } - } - - v, ok := p.(metric.View) - if !ok { - // It's purely a benchmark for a Persistence that is not viewable. - return - } - - matchers := labelMatchersFromLabelSet(clientmodel.LabelSet{ - clientmodel.MetricNameLabel: "errors_total", - "controller": "foo", - "operation": "bar", - }) - - for i := 0; i < increments; i++ { - for j := 0; j < repetitions; j++ { - fingerprints, err := p.GetFingerprintsForLabelMatchers(matchers) - if err != nil { - t.Fatal(err) - } - if len(fingerprints) != 1 { - t.Fatalf("expected %d fingerprints, got %d", 1, len(fingerprints)) - } - - time := clientmodel.Timestamp(0).Add(time.Duration(i) * time.Hour).Add(time.Duration(j) * time.Second) - samples := v.GetValueAtTime(fingerprints[0], time) - if len(samples) == 0 { - t.Fatal("expected at least one sample.") - } - - expected := clientmodel.SampleValue(i) - - for _, sample := range samples { - if sample.Value != expected { - t.Fatalf("expected %v value, got %v", expected, sample.Value) - } - } - } - } -} - -func AppendsRepeatingValuesTests(p metric.Persistence, t testing.TB) { - m := clientmodel.Metric{ - clientmodel.MetricNameLabel: "errors_total", - "controller": "foo", - "operation": "bar", - } - - increments := 10 - repetitions := 500 - - s := clientmodel.Samples{} - for i := 0; i < increments; i++ { - for j := 0; j < repetitions; j++ { - time := clientmodel.Timestamp(0).Add(time.Duration(i) * time.Hour).Add(time.Duration(j) * time.Second) - s = append(s, &clientmodel.Sample{ - Value: clientmodel.SampleValue(i), - Timestamp: time, - Metric: m, - }) - } - } - - p.AppendSamples(s) - - v, ok := p.(metric.View) - if !ok { - // It's purely a benchmark for a MetricPersistance that is not viewable. - return - } - - matchers := labelMatchersFromLabelSet(clientmodel.LabelSet{ - clientmodel.MetricNameLabel: "errors_total", - "controller": "foo", - "operation": "bar", - }) - - for i := 0; i < increments; i++ { - for j := 0; j < repetitions; j++ { - fingerprints, err := p.GetFingerprintsForLabelMatchers(matchers) - if err != nil { - t.Fatal(err) - } - if len(fingerprints) != 1 { - t.Fatalf("expected %d fingerprints, got %d", 1, len(fingerprints)) - } - - time := clientmodel.Timestamp(0).Add(time.Duration(i) * time.Hour).Add(time.Duration(j) * time.Second) - samples := v.GetValueAtTime(fingerprints[0], time) - if len(samples) == 0 { - t.Fatal("expected at least one sample.") - } - - expected := clientmodel.SampleValue(i) - - for _, sample := range samples { - if sample.Value != expected { - t.Fatalf("expected %v value, got %v", expected, sample.Value) - } - } - } - } -} - -// Test Definitions Below - -var testLevelDBGetFingerprintsForLabelSet = buildLevelDBTestPersistence("get_fingerprints_for_labelset", GetFingerprintsForLabelSetTests) - -func TestLevelDBGetFingerprintsForLabelSet(t *testing.T) { - testLevelDBGetFingerprintsForLabelSet(t) -} - -func BenchmarkLevelDBGetFingerprintsForLabelSet(b *testing.B) { - for i := 0; i < b.N; i++ { - testLevelDBGetFingerprintsForLabelSet(b) - } -} - -var testLevelDBGetLabelValuesForLabelName = buildLevelDBTestPersistence("get_label_values_for_labelname", GetLabelValuesForLabelNameTests) - -func TestLevelDBGetFingerprintsForLabelName(t *testing.T) { - testLevelDBGetLabelValuesForLabelName(t) -} - -func BenchmarkLevelDBGetLabelValuesForLabelName(b *testing.B) { - for i := 0; i < b.N; i++ { - testLevelDBGetLabelValuesForLabelName(b) - } -} - -var testLevelDBGetMetricForFingerprint = buildLevelDBTestPersistence("get_metric_for_fingerprint", GetMetricForFingerprintTests) - -func TestLevelDBGetMetricForFingerprint(t *testing.T) { - testLevelDBGetMetricForFingerprint(t) -} - -func BenchmarkLevelDBGetMetricForFingerprint(b *testing.B) { - for i := 0; i < b.N; i++ { - testLevelDBGetMetricForFingerprint(b) - } -} - -var testLevelDBAppendRepeatingValues = buildLevelDBTestPersistence("append_repeating_values", AppendRepeatingValuesTests) - -func TestLevelDBAppendRepeatingValues(t *testing.T) { - testLevelDBAppendRepeatingValues(t) -} - -func BenchmarkLevelDBAppendRepeatingValues(b *testing.B) { - for i := 0; i < b.N; i++ { - testLevelDBAppendRepeatingValues(b) - } -} - -var testLevelDBAppendsRepeatingValues = buildLevelDBTestPersistence("appends_repeating_values", AppendsRepeatingValuesTests) - -func TestLevelDBAppendsRepeatingValues(t *testing.T) { - testLevelDBAppendsRepeatingValues(t) -} - -func BenchmarkLevelDBAppendsRepeatingValues(b *testing.B) { - for i := 0; i < b.N; i++ { - testLevelDBAppendsRepeatingValues(b) - } -} - -var testMemoryGetFingerprintsForLabelSet = buildMemoryTestPersistence(GetFingerprintsForLabelSetTests) - -func TestMemoryGetFingerprintsForLabelSet(t *testing.T) { - testMemoryGetFingerprintsForLabelSet(t) -} - -func BenchmarkMemoryGetFingerprintsForLabelSet(b *testing.B) { - for i := 0; i < b.N; i++ { - testMemoryGetFingerprintsForLabelSet(b) - } -} - -var testMemoryGetLabelValuesForLabelName = buildMemoryTestPersistence(GetLabelValuesForLabelNameTests) - -func TestMemoryGetLabelValuesForLabelName(t *testing.T) { - testMemoryGetLabelValuesForLabelName(t) -} - -func BenchmarkMemoryGetLabelValuesForLabelName(b *testing.B) { - for i := 0; i < b.N; i++ { - testMemoryGetLabelValuesForLabelName(b) - } -} - -var testMemoryGetMetricForFingerprint = buildMemoryTestPersistence(GetMetricForFingerprintTests) - -func TestMemoryGetMetricForFingerprint(t *testing.T) { - testMemoryGetMetricForFingerprint(t) -} - -func BenchmarkMemoryGetMetricForFingerprint(b *testing.B) { - for i := 0; i < b.N; i++ { - testMemoryGetMetricForFingerprint(b) - } -} - -var testMemoryAppendRepeatingValues = buildMemoryTestPersistence(AppendRepeatingValuesTests) - -func TestMemoryAppendRepeatingValues(t *testing.T) { - testMemoryAppendRepeatingValues(t) -} - -func BenchmarkMemoryAppendRepeatingValues(b *testing.B) { - for i := 0; i < b.N; i++ { - testMemoryAppendRepeatingValues(b) - } -} diff --git a/storage/metric/tiered/freelist.go b/storage/metric/tiered/freelist.go deleted file mode 100644 index 18e2c02127..0000000000 --- a/storage/metric/tiered/freelist.go +++ /dev/null @@ -1,212 +0,0 @@ -// Copyright 2013 Prometheus Team -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -package tiered - -import ( - "time" - - "github.com/prometheus/prometheus/utility" - - clientmodel "github.com/prometheus/client_golang/model" - dto "github.com/prometheus/prometheus/model/generated" -) - -type dtoSampleKeyList struct { - l utility.FreeList -} - -func newDtoSampleKeyList(cap int) *dtoSampleKeyList { - return &dtoSampleKeyList{ - l: utility.NewFreeList(cap), - } -} - -func (l *dtoSampleKeyList) Get() (*dto.SampleKey, bool) { - if v, ok := l.l.Get(); ok { - return v.(*dto.SampleKey), ok - } - - return &dto.SampleKey{}, false -} - -func (l *dtoSampleKeyList) Give(v *dto.SampleKey) bool { - v.Reset() - - return l.l.Give(v) -} - -func (l *dtoSampleKeyList) Close() { - l.l.Close() -} - -type sampleKeyList struct { - l utility.FreeList -} - -var defaultSampleKey = &SampleKey{} - -func newSampleKeyList(cap int) *sampleKeyList { - return &sampleKeyList{ - l: utility.NewFreeList(cap), - } -} - -func (l *sampleKeyList) Get() (*SampleKey, bool) { - if v, ok := l.l.Get(); ok { - return v.(*SampleKey), ok - } - - return &SampleKey{}, false -} - -func (l *sampleKeyList) Give(v *SampleKey) bool { - *v = *defaultSampleKey - - return l.l.Give(v) -} - -func (l *sampleKeyList) Close() { - l.l.Close() -} - -type valueAtTimeList struct { - l utility.FreeList -} - -func (l *valueAtTimeList) Get(fp *clientmodel.Fingerprint, time clientmodel.Timestamp) *getValuesAtTimeOp { - var op *getValuesAtTimeOp - v, ok := l.l.Get() - if ok { - op = v.(*getValuesAtTimeOp) - } else { - op = &getValuesAtTimeOp{} - } - op.fp = *fp - op.current = time - return op -} - -var pGetValuesAtTimeOp = &getValuesAtTimeOp{} - -func (l *valueAtTimeList) Give(v *getValuesAtTimeOp) bool { - *v = *pGetValuesAtTimeOp - - return l.l.Give(v) -} - -func newValueAtTimeList(cap int) *valueAtTimeList { - return &valueAtTimeList{ - l: utility.NewFreeList(cap), - } -} - -type valueAtIntervalList struct { - l utility.FreeList -} - -func (l *valueAtIntervalList) Get(fp *clientmodel.Fingerprint, from, through clientmodel.Timestamp, interval time.Duration) *getValuesAtIntervalOp { - var op *getValuesAtIntervalOp - v, ok := l.l.Get() - if ok { - op = v.(*getValuesAtIntervalOp) - } else { - op = &getValuesAtIntervalOp{} - } - op.fp = *fp - op.current = from - op.through = through - op.interval = interval - return op -} - -var pGetValuesAtIntervalOp = &getValuesAtIntervalOp{} - -func (l *valueAtIntervalList) Give(v *getValuesAtIntervalOp) bool { - *v = *pGetValuesAtIntervalOp - - return l.l.Give(v) -} - -func newValueAtIntervalList(cap int) *valueAtIntervalList { - return &valueAtIntervalList{ - l: utility.NewFreeList(cap), - } -} - -type valueAlongRangeList struct { - l utility.FreeList -} - -func (l *valueAlongRangeList) Get(fp *clientmodel.Fingerprint, from, through clientmodel.Timestamp) *getValuesAlongRangeOp { - var op *getValuesAlongRangeOp - v, ok := l.l.Get() - if ok { - op = v.(*getValuesAlongRangeOp) - } else { - op = &getValuesAlongRangeOp{} - } - op.fp = *fp - op.current = from - op.through = through - return op -} - -var pGetValuesAlongRangeOp = &getValuesAlongRangeOp{} - -func (l *valueAlongRangeList) Give(v *getValuesAlongRangeOp) bool { - *v = *pGetValuesAlongRangeOp - - return l.l.Give(v) -} - -func newValueAlongRangeList(cap int) *valueAlongRangeList { - return &valueAlongRangeList{ - l: utility.NewFreeList(cap), - } -} - -type valueAtIntervalAlongRangeList struct { - l utility.FreeList -} - -func (l *valueAtIntervalAlongRangeList) Get(fp *clientmodel.Fingerprint, from, through clientmodel.Timestamp, interval, rangeDuration time.Duration) *getValueRangeAtIntervalOp { - var op *getValueRangeAtIntervalOp - v, ok := l.l.Get() - if ok { - op = v.(*getValueRangeAtIntervalOp) - } else { - op = &getValueRangeAtIntervalOp{} - } - op.fp = *fp - op.current = from - op.rangeThrough = from.Add(rangeDuration) - op.rangeDuration = rangeDuration - op.interval = interval - op.through = through - return op -} - -var pGetValueRangeAtIntervalOp = &getValueRangeAtIntervalOp{} - -func (l *valueAtIntervalAlongRangeList) Give(v *getValueRangeAtIntervalOp) bool { - *v = *pGetValueRangeAtIntervalOp - - return l.l.Give(v) -} - -func newValueAtIntervalAlongRangeList(cap int) *valueAtIntervalAlongRangeList { - return &valueAtIntervalAlongRangeList{ - l: utility.NewFreeList(cap), - } -} diff --git a/storage/metric/tiered/freelist_test.go b/storage/metric/tiered/freelist_test.go deleted file mode 100644 index c575583cc2..0000000000 --- a/storage/metric/tiered/freelist_test.go +++ /dev/null @@ -1,37 +0,0 @@ -// Copyright 2014 Prometheus Team -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -package tiered - -import ( - "testing" - - "github.com/prometheus/client_golang/model" -) - -// TestValueAtTimeListGet tests if the timestamp is set properly in the op -// retrieved from the free list and if the 'consumed' member is zeroed properly. -func TestValueAtTimeListGet(t *testing.T) { - l := newValueAtTimeList(1) - op := l.Get(&model.Fingerprint{}, 42) - op.consumed = true - l.Give(op) - - op2 := l.Get(&model.Fingerprint{}, 4711) - if op2.Consumed() { - t.Error("Op retrieved from freelist is already consumed.") - } - if got, expected := op2.CurrentTime(), model.Timestamp(4711); got != expected { - t.Errorf("op2.CurrentTime() = %d; want %d.", got, expected) - } -} diff --git a/storage/metric/tiered/index.go b/storage/metric/tiered/index.go deleted file mode 100644 index cc3eb45bf7..0000000000 --- a/storage/metric/tiered/index.go +++ /dev/null @@ -1,689 +0,0 @@ -// Copyright 2013 Prometheus Team -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -package tiered - -import ( - "io" - "sort" - "sync" - - "code.google.com/p/goprotobuf/proto" - - clientmodel "github.com/prometheus/client_golang/model" - "github.com/prometheus/prometheus/storage/metric" - "github.com/prometheus/prometheus/storage/raw" - "github.com/prometheus/prometheus/storage/raw/leveldb" - "github.com/prometheus/prometheus/utility" - - dto "github.com/prometheus/prometheus/model/generated" -) - -// FingerprintMetricMapping is an in-memory map of Fingerprints to Metrics. -type FingerprintMetricMapping map[clientmodel.Fingerprint]clientmodel.Metric - -// FingerprintMetricIndex models a database mapping Fingerprints to Metrics. -type FingerprintMetricIndex interface { - raw.Database - raw.Pruner - - IndexBatch(FingerprintMetricMapping) error - Lookup(*clientmodel.Fingerprint) (m clientmodel.Metric, ok bool, err error) -} - -// LevelDBFingerprintMetricIndex implements FingerprintMetricIndex using -// leveldb. -type LevelDBFingerprintMetricIndex struct { - *leveldb.LevelDBPersistence -} - -// IndexBatch implements FingerprintMetricIndex. -func (i *LevelDBFingerprintMetricIndex) IndexBatch(mapping FingerprintMetricMapping) error { - b := leveldb.NewBatch() - defer b.Close() - - for f, m := range mapping { - k := &dto.Fingerprint{} - dumpFingerprint(k, &f) - v := &dto.Metric{} - dumpMetric(v, m) - - b.Put(k, v) - } - - return i.LevelDBPersistence.Commit(b) -} - -// Lookup implements FingerprintMetricIndex. -func (i *LevelDBFingerprintMetricIndex) Lookup(f *clientmodel.Fingerprint) (m clientmodel.Metric, ok bool, err error) { - k := &dto.Fingerprint{} - dumpFingerprint(k, f) - v := &dto.Metric{} - if ok, err := i.LevelDBPersistence.Get(k, v); !ok { - return nil, false, nil - } else if err != nil { - return nil, false, err - } - - m = clientmodel.Metric{} - - for _, pair := range v.LabelPair { - m[clientmodel.LabelName(pair.GetName())] = clientmodel.LabelValue(pair.GetValue()) - } - - return m, true, nil -} - -// NewLevelDBFingerprintMetricIndex returns a LevelDBFingerprintMetricIndex -// object ready to use. -func NewLevelDBFingerprintMetricIndex(o leveldb.LevelDBOptions) (*LevelDBFingerprintMetricIndex, error) { - s, err := leveldb.NewLevelDBPersistence(o) - if err != nil { - return nil, err - } - - return &LevelDBFingerprintMetricIndex{ - LevelDBPersistence: s, - }, nil -} - -// LabelNameLabelValuesMapping is an in-memory map of LabelNames to -// LabelValues. -type LabelNameLabelValuesMapping map[clientmodel.LabelName]clientmodel.LabelValues - -// LabelNameLabelValuesIndex models a database mapping LabelNames to -// LabelValues. -type LabelNameLabelValuesIndex interface { - raw.Database - raw.Pruner - - IndexBatch(LabelNameLabelValuesMapping) error - Lookup(clientmodel.LabelName) (values clientmodel.LabelValues, ok bool, err error) - Has(clientmodel.LabelName) (ok bool, err error) -} - -// LevelDBLabelNameLabelValuesIndex implements LabelNameLabelValuesIndex using -// leveldb. -type LevelDBLabelNameLabelValuesIndex struct { - *leveldb.LevelDBPersistence -} - -// IndexBatch implements LabelNameLabelValuesIndex. -func (i *LevelDBLabelNameLabelValuesIndex) IndexBatch(b LabelNameLabelValuesMapping) error { - batch := leveldb.NewBatch() - defer batch.Close() - - for labelName, labelValues := range b { - sort.Sort(labelValues) - - key := &dto.LabelName{ - Name: proto.String(string(labelName)), - } - value := &dto.LabelValueCollection{} - value.Member = make([]string, 0, len(labelValues)) - for _, labelValue := range labelValues { - value.Member = append(value.Member, string(labelValue)) - } - - batch.Put(key, value) - } - - return i.LevelDBPersistence.Commit(batch) -} - -// Lookup implements LabelNameLabelValuesIndex. -func (i *LevelDBLabelNameLabelValuesIndex) Lookup(l clientmodel.LabelName) (values clientmodel.LabelValues, ok bool, err error) { - k := &dto.LabelName{} - dumpLabelName(k, l) - v := &dto.LabelValueCollection{} - ok, err = i.LevelDBPersistence.Get(k, v) - if err != nil { - return nil, false, err - } - if !ok { - return nil, false, nil - } - - for _, m := range v.Member { - values = append(values, clientmodel.LabelValue(m)) - } - - return values, true, nil -} - -// Has implements LabelNameLabelValuesIndex. -func (i *LevelDBLabelNameLabelValuesIndex) Has(l clientmodel.LabelName) (ok bool, err error) { - return i.LevelDBPersistence.Has(&dto.LabelName{ - Name: proto.String(string(l)), - }) -} - -// NewLevelDBLabelNameLabelValuesIndex returns a LevelDBLabelNameLabelValuesIndex -// ready to use. -func NewLevelDBLabelNameLabelValuesIndex(o leveldb.LevelDBOptions) (*LevelDBLabelNameLabelValuesIndex, error) { - s, err := leveldb.NewLevelDBPersistence(o) - if err != nil { - return nil, err - } - - return &LevelDBLabelNameLabelValuesIndex{ - LevelDBPersistence: s, - }, nil -} - -// LabelPairFingerprintMapping is an in-memory map of LabelPairs to -// Fingerprints. -type LabelPairFingerprintMapping map[metric.LabelPair]clientmodel.Fingerprints - -// LabelPairFingerprintIndex models a database mapping LabelPairs to -// Fingerprints. -type LabelPairFingerprintIndex interface { - raw.Database - raw.ForEacher - raw.Pruner - - IndexBatch(LabelPairFingerprintMapping) error - Lookup(*metric.LabelPair) (m clientmodel.Fingerprints, ok bool, err error) - Has(*metric.LabelPair) (ok bool, err error) -} - -// LevelDBLabelPairFingerprintIndex implements LabelPairFingerprintIndex using -// leveldb. -type LevelDBLabelPairFingerprintIndex struct { - *leveldb.LevelDBPersistence -} - -// IndexBatch implements LabelPairFingerprintMapping. -func (i *LevelDBLabelPairFingerprintIndex) IndexBatch(m LabelPairFingerprintMapping) error { - batch := leveldb.NewBatch() - defer batch.Close() - - for pair, fps := range m { - sort.Sort(fps) - - key := &dto.LabelPair{ - Name: proto.String(string(pair.Name)), - Value: proto.String(string(pair.Value)), - } - value := &dto.FingerprintCollection{} - for _, fp := range fps { - f := &dto.Fingerprint{} - dumpFingerprint(f, fp) - value.Member = append(value.Member, f) - } - - batch.Put(key, value) - } - - return i.LevelDBPersistence.Commit(batch) -} - -// Lookup implements LabelPairFingerprintMapping. -func (i *LevelDBLabelPairFingerprintIndex) Lookup(p *metric.LabelPair) (m clientmodel.Fingerprints, ok bool, err error) { - k := &dto.LabelPair{ - Name: proto.String(string(p.Name)), - Value: proto.String(string(p.Value)), - } - v := &dto.FingerprintCollection{} - - ok, err = i.LevelDBPersistence.Get(k, v) - - if !ok { - return nil, false, nil - } - if err != nil { - return nil, false, err - } - - for _, pair := range v.Member { - fp := &clientmodel.Fingerprint{} - loadFingerprint(fp, pair) - m = append(m, fp) - } - - return m, true, nil -} - -// Has implements LabelPairFingerprintMapping. -func (i *LevelDBLabelPairFingerprintIndex) Has(p *metric.LabelPair) (ok bool, err error) { - k := &dto.LabelPair{ - Name: proto.String(string(p.Name)), - Value: proto.String(string(p.Value)), - } - - return i.LevelDBPersistence.Has(k) -} - -// NewLevelDBLabelSetFingerprintIndex returns a LevelDBLabelPairFingerprintIndex -// object ready to use. -func NewLevelDBLabelSetFingerprintIndex(o leveldb.LevelDBOptions) (*LevelDBLabelPairFingerprintIndex, error) { - s, err := leveldb.NewLevelDBPersistence(o) - if err != nil { - return nil, err - } - - return &LevelDBLabelPairFingerprintIndex{ - LevelDBPersistence: s, - }, nil -} - -// MetricMembershipIndex models a database tracking the existence of Metrics. -type MetricMembershipIndex interface { - raw.Database - raw.Pruner - - IndexBatch(FingerprintMetricMapping) error - Has(clientmodel.Metric) (ok bool, err error) -} - -// LevelDBMetricMembershipIndex implements MetricMembershipIndex using leveldb. -type LevelDBMetricMembershipIndex struct { - *leveldb.LevelDBPersistence -} - -var existenceIdentity = &dto.MembershipIndexValue{} - -// IndexBatch implements MetricMembershipIndex. -func (i *LevelDBMetricMembershipIndex) IndexBatch(b FingerprintMetricMapping) error { - batch := leveldb.NewBatch() - defer batch.Close() - - for _, m := range b { - k := &dto.Metric{} - dumpMetric(k, m) - batch.Put(k, existenceIdentity) - } - - return i.LevelDBPersistence.Commit(batch) -} - -// Has implements MetricMembershipIndex. -func (i *LevelDBMetricMembershipIndex) Has(m clientmodel.Metric) (ok bool, err error) { - k := &dto.Metric{} - dumpMetric(k, m) - - return i.LevelDBPersistence.Has(k) -} - -// NewLevelDBMetricMembershipIndex returns a LevelDBMetricMembershipIndex object -// ready to use. -func NewLevelDBMetricMembershipIndex(o leveldb.LevelDBOptions) (*LevelDBMetricMembershipIndex, error) { - s, err := leveldb.NewLevelDBPersistence(o) - if err != nil { - return nil, err - } - - return &LevelDBMetricMembershipIndex{ - LevelDBPersistence: s, - }, nil -} - -// MetricIndexer indexes facets of a clientmodel.Metric. -type MetricIndexer interface { - // IndexMetric makes no assumptions about the concurrency safety of the - // underlying implementer. - IndexMetrics(FingerprintMetricMapping) error -} - -// IndexerObserver listens and receives changes to a given -// FingerprintMetricMapping. -type IndexerObserver interface { - Observe(FingerprintMetricMapping) error -} - -// IndexerProxy receives IndexMetric requests and proxies them to the underlying -// MetricIndexer. Upon success of the underlying receiver, the registered -// IndexObservers are called serially. -// -// If an error occurs in the underlying MetricIndexer or any of the observers, -// this proxy will not work any further and return the offending error in this -// call or any subsequent ones. -type IndexerProxy struct { - err error - - i MetricIndexer - observers []IndexerObserver -} - -// IndexMetrics proxies the given FingerprintMetricMapping to the underlying -// MetricIndexer and calls all registered observers with it. -func (p *IndexerProxy) IndexMetrics(b FingerprintMetricMapping) error { - if p.err != nil { - return p.err - } - if p.err = p.i.IndexMetrics(b); p.err != nil { - return p.err - } - - for _, o := range p.observers { - if p.err = o.Observe(b); p.err != nil { - return p.err - } - } - - return nil -} - -// Close closes the underlying indexer. -func (p *IndexerProxy) Close() error { - if p.err != nil { - return p.err - } - if closer, ok := p.i.(io.Closer); ok { - p.err = closer.Close() - return p.err - } - return nil -} - -// Flush flushes the underlying index requests before closing. -func (p *IndexerProxy) Flush() error { - if p.err != nil { - return p.err - } - if flusher, ok := p.i.(flusher); ok { - p.err = flusher.Flush() - return p.err - } - return nil -} - -// NewIndexerProxy builds an IndexerProxy for the given configuration. -func NewIndexerProxy(i MetricIndexer, o ...IndexerObserver) *IndexerProxy { - return &IndexerProxy{ - i: i, - observers: o, - } -} - -// SynchronizedIndexer provides naive locking for any MetricIndexer. -type SynchronizedIndexer struct { - mu sync.Mutex - i MetricIndexer -} - -// IndexMetrics calls IndexMetrics of the wrapped MetricIndexer after acquiring -// a lock. -func (i *SynchronizedIndexer) IndexMetrics(b FingerprintMetricMapping) error { - i.mu.Lock() - defer i.mu.Unlock() - - return i.i.IndexMetrics(b) -} - -type flusher interface { - Flush() error -} - -// Flush calls Flush of the wrapped MetricIndexer after acquiring a lock. If the -// wrapped MetricIndexer has no Flush method, this is a no-op. -func (i *SynchronizedIndexer) Flush() error { - if flusher, ok := i.i.(flusher); ok { - i.mu.Lock() - defer i.mu.Unlock() - - return flusher.Flush() - } - - return nil -} - -// Close calls Close of the wrapped MetricIndexer after acquiring a lock. If the -// wrapped MetricIndexer has no Close method, this is a no-op. -func (i *SynchronizedIndexer) Close() error { - if closer, ok := i.i.(io.Closer); ok { - i.mu.Lock() - defer i.mu.Unlock() - - return closer.Close() - } - - return nil -} - -// NewSynchronizedIndexer returns a SynchronizedIndexer wrapping the given -// MetricIndexer. -func NewSynchronizedIndexer(i MetricIndexer) *SynchronizedIndexer { - return &SynchronizedIndexer{ - i: i, - } -} - -// BufferedIndexer provides unsynchronized index buffering. -// -// If an error occurs in the underlying MetricIndexer or any of the observers, -// this proxy will not work any further and return the offending error. -type BufferedIndexer struct { - i MetricIndexer - - limit int - - buf []FingerprintMetricMapping - - err error -} - -// IndexMetrics writes the entries in the given FingerprintMetricMapping to the -// index. -func (i *BufferedIndexer) IndexMetrics(b FingerprintMetricMapping) error { - if i.err != nil { - return i.err - } - - if len(i.buf) < i.limit { - i.buf = append(i.buf, b) - - return nil - } - - i.err = i.Flush() - - return i.err -} - -// Flush writes all pending entries to the index. -func (i *BufferedIndexer) Flush() error { - if i.err != nil { - return i.err - } - - if len(i.buf) == 0 { - return nil - } - - superset := FingerprintMetricMapping{} - for _, b := range i.buf { - for fp, m := range b { - if _, ok := superset[fp]; ok { - continue - } - - superset[fp] = m - } - } - - i.buf = make([]FingerprintMetricMapping, 0, i.limit) - - i.err = i.i.IndexMetrics(superset) - - return i.err -} - -// Close flushes and closes the underlying buffer. -func (i *BufferedIndexer) Close() error { - if err := i.Flush(); err != nil { - return err - } - - if closer, ok := i.i.(io.Closer); ok { - return closer.Close() - } - - return nil -} - -// NewBufferedIndexer returns a BufferedIndexer ready to use. -func NewBufferedIndexer(i MetricIndexer, limit int) *BufferedIndexer { - return &BufferedIndexer{ - i: i, - limit: limit, - buf: make([]FingerprintMetricMapping, 0, limit), - } -} - -// TotalIndexer is a MetricIndexer that indexes all standard facets of a metric -// that a user or the Prometheus subsystem would want to query against: -// -// "