2015-01-21 11:07:45 -08:00
|
|
|
// Copyright 2014 The Prometheus Authors
|
2014-09-19 09:18:44 -07:00
|
|
|
// 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.
|
|
|
|
|
2014-09-16 06:47:24 -07:00
|
|
|
// Package local contains the local time series storage used by Prometheus.
|
|
|
|
package local
|
2014-06-06 02:55:53 -07:00
|
|
|
|
|
|
|
import (
|
2014-11-13 11:50:25 -08:00
|
|
|
"container/list"
|
2016-03-03 04:15:02 -08:00
|
|
|
"errors"
|
2015-06-15 03:49:28 -07:00
|
|
|
"fmt"
|
2016-01-25 09:44:43 -08:00
|
|
|
"math"
|
storage: improve index lookups
tl;dr: This is not a fundamental solution to the indexing problem
(like tindex is) but it at least avoids utilizing the intersection
problem to the greatest possible amount.
In more detail:
Imagine the following query:
nicely:aggregating:rule{job="foo",env="prod"}
While it uses a nicely aggregating recording rule (which might have a
very low cardinality), Prometheus still intersects the low number of
fingerprints for `{__name__="nicely:aggregating:rule"}` with the many
thousands of fingerprints matching `{job="foo"}` and with the millions
of fingerprints matching `{env="prod"}`. This totally innocuous query
is dead slow if the Prometheus server has a lot of time series with
the `{env="prod"}` label. Ironically, if you make the query more
complicated, it becomes blazingly fast:
nicely:aggregating:rule{job=~"foo",env=~"prod"}
Why so? Because Prometheus only intersects with non-Equal matchers if
there are no Equal matchers. That's good in this case because it
retrieves the few fingerprints for
`{__name__="nicely:aggregating:rule"}` and then starts right ahead to
retrieve the metric for those FPs and checking individually if they
match the other matchers.
This change is generalizing the idea of when to stop intersecting FPs
and go into "retrieve metrics and check them individually against
remaining matchers" mode:
- First, sort all matchers by "expected cardinality". Matchers
matching the empty string are always worst (and never used for
intersections). Equal matchers are in general consider best, but by
using some crude heuristics, we declare some better than others
(instance labels or anything that looks like a recording rule).
- Then go through the matchers until we hit a threshold of remaining
FPs in the intersection. This threshold is higher if we are already
in the non-Equal matcher area as intersection is even more expensive
here.
- Once the threshold has been reached (or we have run out of matchers
that do not match the empty string), start with "retrieve metrics
and check them individually against remaining matchers".
A beefy server at SoundCloud was spending 67% of its CPU time in index
lookups (fingerprintsForLabelPairs), serving mostly a dashboard that
is exclusively built with recording rules. With this change, it spends
only 35% in fingerprintsForLabelPairs. The CPU usage dropped from 26
cores to 18 cores. The median latency for query_range dropped from 14s
to 50ms(!). As expected, higher percentile latency didn't improve that
much because the new approach is _occasionally_ running into the worst
case while the old one was _systematically_ doing so. The 99th
percentile latency is now about as high as the median before (14s)
while it was almost twice as high before (26s).
2016-06-28 11:18:32 -07:00
|
|
|
"sort"
|
2016-02-22 06:48:39 -08:00
|
|
|
"sync"
|
2014-10-23 06:18:32 -07:00
|
|
|
"sync/atomic"
|
2014-06-06 02:55:53 -07:00
|
|
|
"time"
|
|
|
|
|
2014-10-22 10:21:23 -07:00
|
|
|
"github.com/prometheus/client_golang/prometheus"
|
2015-10-03 01:21:43 -07:00
|
|
|
"github.com/prometheus/common/log"
|
2015-08-20 08:18:46 -07:00
|
|
|
"github.com/prometheus/common/model"
|
2016-09-15 15:58:06 -07:00
|
|
|
"golang.org/x/net/context"
|
2014-10-22 10:21:23 -07:00
|
|
|
|
2016-09-21 14:44:27 -07:00
|
|
|
"github.com/prometheus/prometheus/storage/local/chunk"
|
2014-06-06 02:55:53 -07:00
|
|
|
"github.com/prometheus/prometheus/storage/metric"
|
|
|
|
)
|
|
|
|
|
2014-11-05 11:02:45 -08:00
|
|
|
const (
|
Handle errors caused by data corruption more gracefully
This requires all the panic calls upon unexpected data to be converted
into errors returned. This pollute the function signatures quite
lot. Well, this is Go...
The ideas behind this are the following:
- panic only if it's a programming error. Data corruptions happen, and
they are not programming errors.
- If we detect a data corruption, we "quarantine" the series,
essentially removing it from the database and putting its data into
a separate directory for forensics.
- Failure during writing to a series file is not considered corruption
automatically. It will call setDirty, though, so that a
crashrecovery upon the next restart will commence and check for
that.
- Series quarantining and setDirty calls are logged and counted in
metrics, but are hidden from the user of the interfaces in
interface.go, whith the notable exception of Append(). The reasoning
is that we treat corruption by removing the corrupted series, i.e. a
query for it will return no results on its next call anyway, so
return no results right now. In the case of Append(), we want to
tell the user that no data has been appended, though.
Minor side effects:
- Now consistently using filepath.* instead of path.*.
- Introduced structured logging where I touched it. This makes things
less consistent, but a complete change to structured logging would
be out of scope for this PR.
2016-02-25 03:23:42 -08:00
|
|
|
evictRequestsCap = 1024
|
|
|
|
quarantineRequestsCap = 1024
|
2014-11-13 07:55:15 -08:00
|
|
|
|
|
|
|
// See waitForNextFP.
|
|
|
|
fpMaxSweepTime = 6 * time.Hour
|
2015-04-01 08:52:03 -07:00
|
|
|
fpMaxWaitDuration = 10 * time.Second
|
2014-11-13 11:50:25 -08:00
|
|
|
|
2015-04-01 08:52:03 -07:00
|
|
|
// See waitForNextFP.
|
2014-11-13 11:50:25 -08:00
|
|
|
maxEvictInterval = time.Minute
|
2015-03-19 07:41:50 -07:00
|
|
|
|
2016-01-25 09:44:43 -08:00
|
|
|
// Constants to control the hysteresis of entering and leaving "rushed
|
|
|
|
// mode". In rushed mode, the dirty series count is ignored for
|
2016-01-26 08:29:33 -08:00
|
|
|
// checkpointing, series are maintained as frequently as possible, and
|
|
|
|
// series files are not synced if the adaptive sync strategy is used.
|
|
|
|
persintenceUrgencyScoreForEnteringRushedMode = 0.8
|
|
|
|
persintenceUrgencyScoreForLeavingRushedMode = 0.7
|
|
|
|
|
|
|
|
// This factor times -storage.local.memory-chunks is the number of
|
2016-01-27 10:07:46 -08:00
|
|
|
// memory chunks we tolerate before throttling the storage. It is also a
|
|
|
|
// basis for calculating the persistenceUrgencyScore.
|
|
|
|
toleranceFactorMemChunks = 1.1
|
2016-01-26 08:29:33 -08:00
|
|
|
// This factor times -storage.local.max-chunks-to-persist is the minimum
|
|
|
|
// required number of chunks waiting for persistence before the number
|
|
|
|
// of chunks in memory may influence the persistenceUrgencyScore. (In
|
|
|
|
// other words: if there are no chunks to persist, it doesn't help chunk
|
|
|
|
// eviction if we speed up persistence.)
|
|
|
|
factorMinChunksToPersist = 0.2
|
storage: improve index lookups
tl;dr: This is not a fundamental solution to the indexing problem
(like tindex is) but it at least avoids utilizing the intersection
problem to the greatest possible amount.
In more detail:
Imagine the following query:
nicely:aggregating:rule{job="foo",env="prod"}
While it uses a nicely aggregating recording rule (which might have a
very low cardinality), Prometheus still intersects the low number of
fingerprints for `{__name__="nicely:aggregating:rule"}` with the many
thousands of fingerprints matching `{job="foo"}` and with the millions
of fingerprints matching `{env="prod"}`. This totally innocuous query
is dead slow if the Prometheus server has a lot of time series with
the `{env="prod"}` label. Ironically, if you make the query more
complicated, it becomes blazingly fast:
nicely:aggregating:rule{job=~"foo",env=~"prod"}
Why so? Because Prometheus only intersects with non-Equal matchers if
there are no Equal matchers. That's good in this case because it
retrieves the few fingerprints for
`{__name__="nicely:aggregating:rule"}` and then starts right ahead to
retrieve the metric for those FPs and checking individually if they
match the other matchers.
This change is generalizing the idea of when to stop intersecting FPs
and go into "retrieve metrics and check them individually against
remaining matchers" mode:
- First, sort all matchers by "expected cardinality". Matchers
matching the empty string are always worst (and never used for
intersections). Equal matchers are in general consider best, but by
using some crude heuristics, we declare some better than others
(instance labels or anything that looks like a recording rule).
- Then go through the matchers until we hit a threshold of remaining
FPs in the intersection. This threshold is higher if we are already
in the non-Equal matcher area as intersection is even more expensive
here.
- Once the threshold has been reached (or we have run out of matchers
that do not match the empty string), start with "retrieve metrics
and check them individually against remaining matchers".
A beefy server at SoundCloud was spending 67% of its CPU time in index
lookups (fingerprintsForLabelPairs), serving mostly a dashboard that
is exclusively built with recording rules. With this change, it spends
only 35% in fingerprintsForLabelPairs. The CPU usage dropped from 26
cores to 18 cores. The median latency for query_range dropped from 14s
to 50ms(!). As expected, higher percentile latency didn't improve that
much because the new approach is _occasionally_ running into the worst
case while the old one was _systematically_ doing so. The 99th
percentile latency is now about as high as the median before (14s)
while it was almost twice as high before (26s).
2016-06-28 11:18:32 -07:00
|
|
|
|
|
|
|
// Threshold for when to stop using LabelMatchers to retrieve and
|
|
|
|
// intersect fingerprints. The rationale here is that looking up more
|
|
|
|
// fingerprints has diminishing returns if we already have narrowed down
|
|
|
|
// the possible fingerprints significantly. It is then easier to simply
|
|
|
|
// lookup the metrics for all the fingerprints and directly compare them
|
|
|
|
// to the matchers. Since a fingerprint lookup for an Equal matcher is
|
|
|
|
// much less expensive, there is a lower threshold for that case.
|
|
|
|
// TODO(beorn7): These numbers need to be tweaked, probably a bit lower.
|
|
|
|
// 5x higher numbers have resulted in slightly worse performance in a
|
|
|
|
// real-life production scenario.
|
|
|
|
fpEqualMatchThreshold = 1000
|
|
|
|
fpOtherMatchThreshold = 10000
|
2014-11-05 11:02:45 -08:00
|
|
|
)
|
2014-06-06 02:55:53 -07:00
|
|
|
|
2015-03-08 18:33:10 -07:00
|
|
|
var (
|
2015-03-18 11:36:41 -07:00
|
|
|
numChunksToPersistDesc = prometheus.NewDesc(
|
|
|
|
prometheus.BuildFQName(namespace, subsystem, "chunks_to_persist"),
|
|
|
|
"The current number of chunks waiting for persistence.",
|
|
|
|
nil, nil,
|
|
|
|
)
|
|
|
|
maxChunksToPersistDesc = prometheus.NewDesc(
|
|
|
|
prometheus.BuildFQName(namespace, subsystem, "max_chunks_to_persist"),
|
|
|
|
"The maximum number of chunks that can be waiting for persistence before sample ingestion will stop.",
|
2015-03-08 18:33:10 -07:00
|
|
|
nil, nil,
|
|
|
|
)
|
2014-06-06 02:55:53 -07:00
|
|
|
)
|
|
|
|
|
Handle errors caused by data corruption more gracefully
This requires all the panic calls upon unexpected data to be converted
into errors returned. This pollute the function signatures quite
lot. Well, this is Go...
The ideas behind this are the following:
- panic only if it's a programming error. Data corruptions happen, and
they are not programming errors.
- If we detect a data corruption, we "quarantine" the series,
essentially removing it from the database and putting its data into
a separate directory for forensics.
- Failure during writing to a series file is not considered corruption
automatically. It will call setDirty, though, so that a
crashrecovery upon the next restart will commence and check for
that.
- Series quarantining and setDirty calls are logged and counted in
metrics, but are hidden from the user of the interfaces in
interface.go, whith the notable exception of Append(). The reasoning
is that we treat corruption by removing the corrupted series, i.e. a
query for it will return no results on its next call anyway, so
return no results right now. In the case of Append(), we want to
tell the user that no data has been appended, though.
Minor side effects:
- Now consistently using filepath.* instead of path.*.
- Introduced structured logging where I touched it. This makes things
less consistent, but a complete change to structured logging would
be out of scope for this PR.
2016-02-25 03:23:42 -08:00
|
|
|
type quarantineRequest struct {
|
|
|
|
fp model.Fingerprint
|
|
|
|
metric model.Metric
|
|
|
|
reason error
|
|
|
|
}
|
|
|
|
|
2015-03-19 07:41:50 -07:00
|
|
|
// SyncStrategy is an enum to select a sync strategy for series files.
|
|
|
|
type SyncStrategy int
|
|
|
|
|
2015-06-15 03:49:28 -07:00
|
|
|
// String implements flag.Value.
|
|
|
|
func (ss SyncStrategy) String() string {
|
|
|
|
switch ss {
|
|
|
|
case Adaptive:
|
|
|
|
return "adaptive"
|
|
|
|
case Always:
|
|
|
|
return "always"
|
|
|
|
case Never:
|
|
|
|
return "never"
|
|
|
|
}
|
|
|
|
return "<unknown>"
|
|
|
|
}
|
|
|
|
|
|
|
|
// Set implements flag.Value.
|
|
|
|
func (ss *SyncStrategy) Set(s string) error {
|
|
|
|
switch s {
|
|
|
|
case "adaptive":
|
|
|
|
*ss = Adaptive
|
|
|
|
case "always":
|
|
|
|
*ss = Always
|
|
|
|
case "never":
|
|
|
|
*ss = Never
|
|
|
|
default:
|
|
|
|
return fmt.Errorf("invalid sync strategy: %s", s)
|
|
|
|
}
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
2015-03-19 07:41:50 -07:00
|
|
|
// Possible values for SyncStrategy.
|
|
|
|
const (
|
|
|
|
_ SyncStrategy = iota
|
|
|
|
Never
|
|
|
|
Always
|
|
|
|
Adaptive
|
|
|
|
)
|
|
|
|
|
2015-03-19 09:54:59 -07:00
|
|
|
// A syncStrategy is a function that returns whether series files should be
|
|
|
|
// synced or not. It does not need to be goroutine safe.
|
2015-03-19 07:41:50 -07:00
|
|
|
type syncStrategy func() bool
|
|
|
|
|
2016-07-11 11:27:25 -07:00
|
|
|
// A MemorySeriesStorage manages series in memory over time, while also
|
|
|
|
// interfacing with a persistence layer to make time series data persistent
|
|
|
|
// across restarts and evictable from memory.
|
2016-06-28 23:14:23 -07:00
|
|
|
type MemorySeriesStorage struct {
|
2016-03-08 15:09:42 -08:00
|
|
|
// archiveHighWatermark and numChunksToPersist have to be aligned for atomic operations.
|
|
|
|
archiveHighWatermark model.Time // No archived series has samples after this time.
|
|
|
|
numChunksToPersist int64 // The number of chunks waiting for persistence.
|
|
|
|
maxChunksToPersist int // If numChunksToPersist reaches this threshold, ingestion will be throttled.
|
|
|
|
rushed bool // Whether the storage is in rushed mode.
|
|
|
|
rushedMtx sync.Mutex // Protects entering and exiting rushed mode.
|
|
|
|
throttled chan struct{} // This chan is sent to whenever NeedsThrottling() returns true (for logging).
|
2015-05-20 16:37:04 -07:00
|
|
|
|
2014-10-27 12:40:48 -07:00
|
|
|
fpLocker *fingerprintLocker
|
|
|
|
fpToSeries *seriesMap
|
2014-06-06 02:55:53 -07:00
|
|
|
|
2015-05-18 10:26:28 -07:00
|
|
|
options *MemorySeriesStorageOptions
|
|
|
|
|
2015-01-08 11:15:58 -08:00
|
|
|
loopStopping, loopStopped chan struct{}
|
2016-02-11 16:46:18 -08:00
|
|
|
logThrottlingStopped chan struct{}
|
2015-01-08 11:15:58 -08:00
|
|
|
maxMemoryChunks int
|
2015-02-26 06:19:44 -08:00
|
|
|
dropAfter time.Duration
|
2015-01-08 11:15:58 -08:00
|
|
|
checkpointInterval time.Duration
|
|
|
|
checkpointDirtySeriesLimit int
|
2014-10-24 11:27:27 -07:00
|
|
|
|
Improve persisting chunks to disk.
This is done by bucketing chunks by fingerprint. If the persisting to
disk falls behind, more and more chunks are in the queue. As soon as
there are "double hits", we will now persist both chunks in one go,
doubling the disk throughput (assuming it is limited by disk
seeks). Should even more pile up so that we end wit "triple hits", we
will persist those first, and so on.
Even if we have millions of time series, this will still help,
assuming not all of them are growing with the same speed. Series that
get many samples and/or are not very compressable will accumulate
chunks faster, and they will soon get double- or triple-writes.
To improve the chance of double writes,
-storage.local.persistence-queue-capacity could be set to a higher
value. However, that will slow down shutdown a lot (as the queue has
to be worked through). So we leave it to the user to set it to a
really high value. A more fundamental solution would be to checkpoint
not only head chunks, but also chunks still in the persist queue. That
would be quite complicated for a rather limited use-case (running many
time series with high ingestion rate on slow spinning disks).
2015-02-13 11:08:52 -08:00
|
|
|
persistence *persistence
|
2015-05-06 07:53:12 -07:00
|
|
|
mapper *fpMapper
|
2014-10-24 11:27:27 -07:00
|
|
|
|
2014-11-13 11:50:25 -08:00
|
|
|
evictList *list.List
|
2016-09-21 14:44:27 -07:00
|
|
|
evictRequests chan chunk.EvictRequest
|
2014-11-13 11:50:25 -08:00
|
|
|
evictStopping, evictStopped chan struct{}
|
|
|
|
|
Handle errors caused by data corruption more gracefully
This requires all the panic calls upon unexpected data to be converted
into errors returned. This pollute the function signatures quite
lot. Well, this is Go...
The ideas behind this are the following:
- panic only if it's a programming error. Data corruptions happen, and
they are not programming errors.
- If we detect a data corruption, we "quarantine" the series,
essentially removing it from the database and putting its data into
a separate directory for forensics.
- Failure during writing to a series file is not considered corruption
automatically. It will call setDirty, though, so that a
crashrecovery upon the next restart will commence and check for
that.
- Series quarantining and setDirty calls are logged and counted in
metrics, but are hidden from the user of the interfaces in
interface.go, whith the notable exception of Append(). The reasoning
is that we treat corruption by removing the corrupted series, i.e. a
query for it will return no results on its next call anyway, so
return no results right now. In the case of Append(), we want to
tell the user that no data has been appended, though.
Minor side effects:
- Now consistently using filepath.* instead of path.*.
- Introduced structured logging where I touched it. This makes things
less consistent, but a complete change to structured logging would
be out of scope for this PR.
2016-02-25 03:23:42 -08:00
|
|
|
quarantineRequests chan quarantineRequest
|
|
|
|
quarantineStopping, quarantineStopped chan struct{}
|
|
|
|
|
2016-03-13 03:54:24 -07:00
|
|
|
persistErrors prometheus.Counter
|
|
|
|
numSeries prometheus.Gauge
|
|
|
|
seriesOps *prometheus.CounterVec
|
|
|
|
ingestedSamplesCount prometheus.Counter
|
2016-04-25 07:43:52 -07:00
|
|
|
discardedSamplesCount *prometheus.CounterVec
|
2016-03-13 03:54:24 -07:00
|
|
|
nonExistentSeriesMatchesCount prometheus.Counter
|
|
|
|
maintainSeriesDuration *prometheus.SummaryVec
|
|
|
|
persistenceUrgencyScore prometheus.Gauge
|
|
|
|
rushedMode prometheus.Gauge
|
2014-06-06 02:55:53 -07:00
|
|
|
}
|
|
|
|
|
2014-09-16 06:47:24 -07:00
|
|
|
// MemorySeriesStorageOptions contains options needed by
|
|
|
|
// NewMemorySeriesStorage. It is not safe to leave any of those at their zero
|
|
|
|
// values.
|
2014-06-06 02:55:53 -07:00
|
|
|
type MemorySeriesStorageOptions struct {
|
2014-11-13 11:50:25 -08:00
|
|
|
MemoryChunks int // How many chunks to keep in memory.
|
2015-03-18 11:36:41 -07:00
|
|
|
MaxChunksToPersist int // Max number of chunks waiting to be persisted.
|
2014-10-07 10:11:24 -07:00
|
|
|
PersistenceStoragePath string // Location of persistence files.
|
2015-02-26 06:19:44 -08:00
|
|
|
PersistenceRetentionPeriod time.Duration // Chunks at least that old are dropped.
|
2014-10-24 11:27:27 -07:00
|
|
|
CheckpointInterval time.Duration // How often to checkpoint the series map and head chunks.
|
2015-01-08 11:15:58 -08:00
|
|
|
CheckpointDirtySeriesLimit int // How many dirty series will trigger an early checkpoint.
|
2014-11-05 11:02:45 -08:00
|
|
|
Dirty bool // Force the storage to consider itself dirty on startup.
|
2015-03-19 04:03:09 -07:00
|
|
|
PedanticChecks bool // If dirty, perform crash-recovery checks on each series file.
|
2015-03-19 07:41:50 -07:00
|
|
|
SyncStrategy SyncStrategy // Which sync strategy to apply to series files.
|
2016-01-11 07:42:10 -08:00
|
|
|
MinShrinkRatio float64 // Minimum ratio a series file has to shrink during truncation.
|
2016-06-02 10:18:00 -07:00
|
|
|
NumMutexes int // Number of mutexes used for stochastic fingerprint locking.
|
2014-06-06 02:55:53 -07:00
|
|
|
}
|
|
|
|
|
2014-09-16 06:47:24 -07:00
|
|
|
// NewMemorySeriesStorage returns a newly allocated Storage. Storage.Serve still
|
|
|
|
// has to be called to start the storage.
|
2016-06-28 23:14:23 -07:00
|
|
|
func NewMemorySeriesStorage(o *MemorySeriesStorageOptions) *MemorySeriesStorage {
|
|
|
|
s := &MemorySeriesStorage{
|
2016-06-02 10:18:00 -07:00
|
|
|
fpLocker: newFingerprintLocker(o.NumMutexes),
|
2014-06-06 02:55:53 -07:00
|
|
|
|
2015-05-18 10:26:28 -07:00
|
|
|
options: o,
|
|
|
|
|
2015-01-08 11:15:58 -08:00
|
|
|
loopStopping: make(chan struct{}),
|
|
|
|
loopStopped: make(chan struct{}),
|
2016-02-11 16:46:18 -08:00
|
|
|
logThrottlingStopped: make(chan struct{}),
|
2016-01-27 10:07:46 -08:00
|
|
|
throttled: make(chan struct{}, 1),
|
2015-01-08 11:15:58 -08:00
|
|
|
maxMemoryChunks: o.MemoryChunks,
|
2015-02-26 06:19:44 -08:00
|
|
|
dropAfter: o.PersistenceRetentionPeriod,
|
2015-01-08 11:15:58 -08:00
|
|
|
checkpointInterval: o.CheckpointInterval,
|
|
|
|
checkpointDirtySeriesLimit: o.CheckpointDirtySeriesLimit,
|
2016-03-08 15:09:42 -08:00
|
|
|
archiveHighWatermark: model.Now().Add(-headChunkTimeout),
|
2014-06-06 02:55:53 -07:00
|
|
|
|
2015-03-18 11:36:41 -07:00
|
|
|
maxChunksToPersist: o.MaxChunksToPersist,
|
2014-10-23 06:18:32 -07:00
|
|
|
|
2014-11-13 11:50:25 -08:00
|
|
|
evictList: list.New(),
|
2016-09-21 14:44:27 -07:00
|
|
|
evictRequests: make(chan chunk.EvictRequest, evictRequestsCap),
|
2014-11-13 11:50:25 -08:00
|
|
|
evictStopping: make(chan struct{}),
|
|
|
|
evictStopped: make(chan struct{}),
|
|
|
|
|
Handle errors caused by data corruption more gracefully
This requires all the panic calls upon unexpected data to be converted
into errors returned. This pollute the function signatures quite
lot. Well, this is Go...
The ideas behind this are the following:
- panic only if it's a programming error. Data corruptions happen, and
they are not programming errors.
- If we detect a data corruption, we "quarantine" the series,
essentially removing it from the database and putting its data into
a separate directory for forensics.
- Failure during writing to a series file is not considered corruption
automatically. It will call setDirty, though, so that a
crashrecovery upon the next restart will commence and check for
that.
- Series quarantining and setDirty calls are logged and counted in
metrics, but are hidden from the user of the interfaces in
interface.go, whith the notable exception of Append(). The reasoning
is that we treat corruption by removing the corrupted series, i.e. a
query for it will return no results on its next call anyway, so
return no results right now. In the case of Append(), we want to
tell the user that no data has been appended, though.
Minor side effects:
- Now consistently using filepath.* instead of path.*.
- Introduced structured logging where I touched it. This makes things
less consistent, but a complete change to structured logging would
be out of scope for this PR.
2016-02-25 03:23:42 -08:00
|
|
|
quarantineRequests: make(chan quarantineRequest, quarantineRequestsCap),
|
|
|
|
quarantineStopping: make(chan struct{}),
|
|
|
|
quarantineStopped: make(chan struct{}),
|
|
|
|
|
2015-02-01 03:47:51 -08:00
|
|
|
persistErrors: prometheus.NewCounter(prometheus.CounterOpts{
|
|
|
|
Namespace: namespace,
|
|
|
|
Subsystem: subsystem,
|
|
|
|
Name: "persist_errors_total",
|
|
|
|
Help: "The total number of errors while persisting chunks.",
|
|
|
|
}),
|
2015-03-19 07:41:50 -07:00
|
|
|
numSeries: prometheus.NewGauge(prometheus.GaugeOpts{
|
|
|
|
Namespace: namespace,
|
|
|
|
Subsystem: subsystem,
|
|
|
|
Name: "memory_series",
|
|
|
|
Help: "The current number of series in memory.",
|
|
|
|
}),
|
2014-10-23 06:18:32 -07:00
|
|
|
seriesOps: prometheus.NewCounterVec(
|
|
|
|
prometheus.CounterOpts{
|
|
|
|
Namespace: namespace,
|
|
|
|
Subsystem: subsystem,
|
|
|
|
Name: "series_ops_total",
|
|
|
|
Help: "The total number of series operations by their type.",
|
|
|
|
},
|
|
|
|
[]string{opTypeLabel},
|
|
|
|
),
|
|
|
|
ingestedSamplesCount: prometheus.NewCounter(prometheus.CounterOpts{
|
|
|
|
Namespace: namespace,
|
|
|
|
Subsystem: subsystem,
|
|
|
|
Name: "ingested_samples_total",
|
|
|
|
Help: "The total number of samples ingested.",
|
|
|
|
}),
|
2016-04-25 07:43:52 -07:00
|
|
|
discardedSamplesCount: prometheus.NewCounterVec(
|
|
|
|
prometheus.CounterOpts{
|
|
|
|
Namespace: namespace,
|
|
|
|
Subsystem: subsystem,
|
|
|
|
Name: "out_of_order_samples_total",
|
|
|
|
Help: "The total number of samples that were discarded because their timestamps were at or before the last received sample for a series.",
|
|
|
|
},
|
|
|
|
[]string{discardReasonLabel},
|
|
|
|
),
|
2016-03-13 03:54:24 -07:00
|
|
|
nonExistentSeriesMatchesCount: prometheus.NewCounter(prometheus.CounterOpts{
|
2014-11-05 11:02:45 -08:00
|
|
|
Namespace: namespace,
|
|
|
|
Subsystem: subsystem,
|
2016-03-13 03:54:24 -07:00
|
|
|
Name: "non_existent_series_matches_total",
|
|
|
|
Help: "How often a non-existent series was referred to during label matching or chunk preloading. This is an indication of outdated label indexes.",
|
2014-11-05 11:02:45 -08:00
|
|
|
}),
|
2015-03-19 09:06:16 -07:00
|
|
|
maintainSeriesDuration: prometheus.NewSummaryVec(
|
|
|
|
prometheus.SummaryOpts{
|
|
|
|
Namespace: namespace,
|
|
|
|
Subsystem: subsystem,
|
2016-06-23 08:50:06 -07:00
|
|
|
Name: "maintain_series_duration_seconds",
|
|
|
|
Help: "The duration in seconds it took to perform maintenance on a series.",
|
2015-03-19 09:06:16 -07:00
|
|
|
},
|
|
|
|
[]string{seriesLocationLabel},
|
|
|
|
),
|
2016-01-26 08:29:33 -08:00
|
|
|
persistenceUrgencyScore: prometheus.NewGauge(prometheus.GaugeOpts{
|
|
|
|
Namespace: namespace,
|
|
|
|
Subsystem: subsystem,
|
|
|
|
Name: "persistence_urgency_score",
|
|
|
|
Help: "A score of urgency to persist chunks, 0 is least urgent, 1 most.",
|
|
|
|
}),
|
|
|
|
rushedMode: prometheus.NewGauge(prometheus.GaugeOpts{
|
|
|
|
Namespace: namespace,
|
|
|
|
Subsystem: subsystem,
|
|
|
|
Name: "rushed_mode",
|
|
|
|
Help: "1 if the storage is in rushed mode, 0 otherwise. In rushed mode, the system behaves as if the persistence_urgency_score is 1.",
|
|
|
|
}),
|
2015-02-13 05:26:54 -08:00
|
|
|
}
|
2016-04-25 08:08:07 -07:00
|
|
|
|
|
|
|
// Initialize metric vectors.
|
2016-04-25 16:05:56 -07:00
|
|
|
// TODO(beorn7): Rework once we have a utility function for it in client_golang.
|
2016-04-25 08:08:07 -07:00
|
|
|
s.discardedSamplesCount.WithLabelValues(outOfOrderTimestamp)
|
|
|
|
s.discardedSamplesCount.WithLabelValues(duplicateSample)
|
|
|
|
s.maintainSeriesDuration.WithLabelValues(maintainInMemory)
|
|
|
|
s.maintainSeriesDuration.WithLabelValues(maintainArchived)
|
|
|
|
s.seriesOps.WithLabelValues(create)
|
|
|
|
s.seriesOps.WithLabelValues(archive)
|
|
|
|
s.seriesOps.WithLabelValues(unarchive)
|
|
|
|
s.seriesOps.WithLabelValues(memoryPurge)
|
|
|
|
s.seriesOps.WithLabelValues(archivePurge)
|
|
|
|
s.seriesOps.WithLabelValues(requestedPurge)
|
|
|
|
s.seriesOps.WithLabelValues(memoryMaintenance)
|
|
|
|
s.seriesOps.WithLabelValues(archiveMaintenance)
|
|
|
|
s.seriesOps.WithLabelValues(completedQurantine)
|
|
|
|
s.seriesOps.WithLabelValues(droppedQuarantine)
|
|
|
|
s.seriesOps.WithLabelValues(failedQuarantine)
|
|
|
|
|
2015-05-18 10:26:28 -07:00
|
|
|
return s
|
|
|
|
}
|
2015-02-13 05:26:54 -08:00
|
|
|
|
2015-05-18 10:26:28 -07:00
|
|
|
// Start implements Storage.
|
2016-06-28 23:14:23 -07:00
|
|
|
func (s *MemorySeriesStorage) Start() (err error) {
|
2015-03-19 07:41:50 -07:00
|
|
|
var syncStrategy syncStrategy
|
2015-05-18 10:26:28 -07:00
|
|
|
switch s.options.SyncStrategy {
|
2015-03-19 07:41:50 -07:00
|
|
|
case Never:
|
|
|
|
syncStrategy = func() bool { return false }
|
|
|
|
case Always:
|
|
|
|
syncStrategy = func() bool { return true }
|
|
|
|
case Adaptive:
|
2016-01-26 08:29:33 -08:00
|
|
|
syncStrategy = func() bool { return s.calculatePersistenceUrgencyScore() < 1 }
|
2015-03-19 07:41:50 -07:00
|
|
|
default:
|
|
|
|
panic("unknown sync strategy")
|
|
|
|
}
|
|
|
|
|
2015-05-23 03:03:14 -07:00
|
|
|
var p *persistence
|
2016-01-11 07:42:10 -08:00
|
|
|
p, err = newPersistence(
|
|
|
|
s.options.PersistenceStoragePath,
|
|
|
|
s.options.Dirty, s.options.PedanticChecks,
|
|
|
|
syncStrategy,
|
|
|
|
s.options.MinShrinkRatio,
|
|
|
|
)
|
2015-03-19 07:41:50 -07:00
|
|
|
if err != nil {
|
2015-05-18 10:26:28 -07:00
|
|
|
return err
|
2015-03-19 07:41:50 -07:00
|
|
|
}
|
|
|
|
s.persistence = p
|
2015-05-23 03:03:14 -07:00
|
|
|
// Persistence must start running before loadSeriesMapAndHeads() is called.
|
|
|
|
go s.persistence.run()
|
|
|
|
|
|
|
|
defer func() {
|
|
|
|
if err != nil {
|
|
|
|
if e := p.close(); e != nil {
|
|
|
|
log.Errorln("Error closing persistence:", e)
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}()
|
2015-03-19 07:41:50 -07:00
|
|
|
|
2015-05-20 09:10:29 -07:00
|
|
|
log.Info("Loading series map and head chunks...")
|
2015-03-19 07:41:50 -07:00
|
|
|
s.fpToSeries, s.numChunksToPersist, err = p.loadSeriesMapAndHeads()
|
|
|
|
if err != nil {
|
2015-05-18 10:26:28 -07:00
|
|
|
return err
|
2015-03-19 07:41:50 -07:00
|
|
|
}
|
2015-05-20 09:10:29 -07:00
|
|
|
log.Infof("%d series loaded.", s.fpToSeries.length())
|
2015-03-19 07:41:50 -07:00
|
|
|
s.numSeries.Set(float64(s.fpToSeries.length()))
|
|
|
|
|
2015-05-23 03:03:14 -07:00
|
|
|
s.mapper, err = newFPMapper(s.fpToSeries, p)
|
2015-05-06 07:53:12 -07:00
|
|
|
if err != nil {
|
2015-05-18 10:26:28 -07:00
|
|
|
return err
|
2015-05-06 07:53:12 -07:00
|
|
|
}
|
2014-06-06 02:55:53 -07:00
|
|
|
|
2014-11-13 11:50:25 -08:00
|
|
|
go s.handleEvictList()
|
Handle errors caused by data corruption more gracefully
This requires all the panic calls upon unexpected data to be converted
into errors returned. This pollute the function signatures quite
lot. Well, this is Go...
The ideas behind this are the following:
- panic only if it's a programming error. Data corruptions happen, and
they are not programming errors.
- If we detect a data corruption, we "quarantine" the series,
essentially removing it from the database and putting its data into
a separate directory for forensics.
- Failure during writing to a series file is not considered corruption
automatically. It will call setDirty, though, so that a
crashrecovery upon the next restart will commence and check for
that.
- Series quarantining and setDirty calls are logged and counted in
metrics, but are hidden from the user of the interfaces in
interface.go, whith the notable exception of Append(). The reasoning
is that we treat corruption by removing the corrupted series, i.e. a
query for it will return no results on its next call anyway, so
return no results right now. In the case of Append(), we want to
tell the user that no data has been appended, though.
Minor side effects:
- Now consistently using filepath.* instead of path.*.
- Introduced structured logging where I touched it. This makes things
less consistent, but a complete change to structured logging would
be out of scope for this PR.
2016-02-25 03:23:42 -08:00
|
|
|
go s.handleQuarantine()
|
2016-01-27 10:07:46 -08:00
|
|
|
go s.logThrottling()
|
2014-10-24 11:27:27 -07:00
|
|
|
go s.loop()
|
2015-05-18 10:26:28 -07:00
|
|
|
|
|
|
|
return nil
|
2014-10-24 11:27:27 -07:00
|
|
|
}
|
|
|
|
|
|
|
|
// Stop implements Storage.
|
2016-06-28 23:14:23 -07:00
|
|
|
func (s *MemorySeriesStorage) Stop() error {
|
2015-05-20 09:10:29 -07:00
|
|
|
log.Info("Stopping local storage...")
|
2014-11-13 11:50:25 -08:00
|
|
|
|
2015-05-20 09:10:29 -07:00
|
|
|
log.Info("Stopping maintenance loop...")
|
2014-10-24 11:27:27 -07:00
|
|
|
close(s.loopStopping)
|
|
|
|
<-s.loopStopped
|
|
|
|
|
Handle errors caused by data corruption more gracefully
This requires all the panic calls upon unexpected data to be converted
into errors returned. This pollute the function signatures quite
lot. Well, this is Go...
The ideas behind this are the following:
- panic only if it's a programming error. Data corruptions happen, and
they are not programming errors.
- If we detect a data corruption, we "quarantine" the series,
essentially removing it from the database and putting its data into
a separate directory for forensics.
- Failure during writing to a series file is not considered corruption
automatically. It will call setDirty, though, so that a
crashrecovery upon the next restart will commence and check for
that.
- Series quarantining and setDirty calls are logged and counted in
metrics, but are hidden from the user of the interfaces in
interface.go, whith the notable exception of Append(). The reasoning
is that we treat corruption by removing the corrupted series, i.e. a
query for it will return no results on its next call anyway, so
return no results right now. In the case of Append(), we want to
tell the user that no data has been appended, though.
Minor side effects:
- Now consistently using filepath.* instead of path.*.
- Introduced structured logging where I touched it. This makes things
less consistent, but a complete change to structured logging would
be out of scope for this PR.
2016-02-25 03:23:42 -08:00
|
|
|
log.Info("Stopping series quarantining...")
|
|
|
|
close(s.quarantineStopping)
|
|
|
|
<-s.quarantineStopped
|
|
|
|
|
2015-05-20 09:10:29 -07:00
|
|
|
log.Info("Stopping chunk eviction...")
|
2014-11-13 11:50:25 -08:00
|
|
|
close(s.evictStopping)
|
|
|
|
<-s.evictStopped
|
|
|
|
|
2014-10-24 11:27:27 -07:00
|
|
|
// One final checkpoint of the series map and the head chunks.
|
2014-10-27 12:40:48 -07:00
|
|
|
if err := s.persistence.checkpointSeriesMapAndHeads(s.fpToSeries, s.fpLocker); err != nil {
|
2014-10-24 11:27:27 -07:00
|
|
|
return err
|
|
|
|
}
|
Checkpoint fingerprint mappings only upon shutdown
Before, we checkpointed after every newly detected fingerprint
collision, which is not a problem as long as collisions are
rare. However, with a sufficient number of metrics or particular
nature of the data set, there might be a lot of collisions, all to be
detected upon the first set of scrapes, and then the checkpointing
after each detection will take a quite long time (it's O(n²),
essentially).
Since we are rebuilding the fingerprint mapping during crash recovery,
the previous, very conservative approach didn't even buy us
anything. We only ever read from the checkpoint file after a clean
shutdown, so the only time we need to write the checkpoint file is
during a clean shutdown.
2016-04-14 07:02:37 -07:00
|
|
|
if err := s.mapper.checkpoint(); err != nil {
|
|
|
|
return err
|
|
|
|
}
|
2014-10-24 11:27:27 -07:00
|
|
|
|
|
|
|
if err := s.persistence.close(); err != nil {
|
|
|
|
return err
|
|
|
|
}
|
2015-05-20 09:10:29 -07:00
|
|
|
log.Info("Local storage stopped.")
|
2014-10-24 11:27:27 -07:00
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
|
|
|
// WaitForIndexing implements Storage.
|
2016-06-28 23:14:23 -07:00
|
|
|
func (s *MemorySeriesStorage) WaitForIndexing() {
|
2014-10-24 11:27:27 -07:00
|
|
|
s.persistence.waitForIndexing()
|
|
|
|
}
|
|
|
|
|
2016-07-11 11:27:25 -07:00
|
|
|
// LastSampleForLabelMatchers implements Storage.
|
2016-09-15 15:58:06 -07:00
|
|
|
func (s *MemorySeriesStorage) LastSampleForLabelMatchers(_ context.Context, cutoff model.Time, matcherSets ...metric.LabelMatchers) (model.Vector, error) {
|
2016-09-18 04:20:46 -07:00
|
|
|
mergedFPs := map[model.Fingerprint]struct{}{}
|
2016-07-11 11:27:25 -07:00
|
|
|
for _, matchers := range matcherSets {
|
2016-09-18 04:20:46 -07:00
|
|
|
fps, err := s.fpsForLabelMatchers(cutoff, model.Latest, matchers...)
|
2016-07-11 11:27:25 -07:00
|
|
|
if err != nil {
|
|
|
|
return nil, err
|
|
|
|
}
|
2016-09-18 04:20:46 -07:00
|
|
|
for fp := range fps {
|
|
|
|
mergedFPs[fp] = struct{}{}
|
2016-07-11 11:27:25 -07:00
|
|
|
}
|
2016-03-08 15:09:42 -08:00
|
|
|
}
|
2016-07-11 11:27:25 -07:00
|
|
|
|
2016-09-18 04:20:46 -07:00
|
|
|
res := make(model.Vector, 0, len(mergedFPs))
|
|
|
|
for fp := range mergedFPs {
|
2016-07-11 11:27:25 -07:00
|
|
|
s.fpLocker.Lock(fp)
|
|
|
|
|
|
|
|
series, ok := s.fpToSeries.get(fp)
|
|
|
|
if !ok {
|
|
|
|
// A series could have disappeared between resolving label matchers and here.
|
|
|
|
s.fpLocker.Unlock(fp)
|
|
|
|
continue
|
|
|
|
}
|
|
|
|
sp := series.lastSamplePair()
|
|
|
|
res = append(res, &model.Sample{
|
|
|
|
Metric: series.metric,
|
|
|
|
Value: sp.Value,
|
|
|
|
Timestamp: sp.Timestamp,
|
|
|
|
})
|
|
|
|
s.fpLocker.Unlock(fp)
|
2015-06-22 13:50:47 -07:00
|
|
|
}
|
2016-07-11 11:27:25 -07:00
|
|
|
return res, nil
|
2015-06-22 13:50:47 -07:00
|
|
|
}
|
|
|
|
|
2015-05-27 02:24:56 -07:00
|
|
|
// boundedIterator wraps a SeriesIterator and does not allow fetching
|
|
|
|
// data from earlier than the configured start time.
|
|
|
|
type boundedIterator struct {
|
|
|
|
it SeriesIterator
|
2015-08-20 08:18:46 -07:00
|
|
|
start model.Time
|
2015-05-27 02:24:56 -07:00
|
|
|
}
|
|
|
|
|
Streamline series iterator creation
This will fix issue #1035 and will also help to make issue #1264 less
bad.
The fundamental problem in the current code:
In the preload phase, we quite accurately determine which chunks will
be used for the query being executed. However, in the subsequent step
of creating series iterators, the created iterators are referencing
_all_ in-memory chunks in their series, even the un-pinned ones. In
iterator creation, we copy a pointer to each in-memory chunk of a
series into the iterator. While this creates a certain amount of
allocation churn, the worst thing about it is that copying the chunk
pointer out of the chunkDesc requires a mutex acquisition. (Remember
that the iterator will also reference un-pinned chunks, so we need to
acquire the mutex to protect against concurrent eviction.) The worst
case happens if a series doesn't even contain any relevant samples for
the query time range. We notice that during preloading but then we
will still create a series iterator for it. But even for series that
do contain relevant samples, the overhead is quite bad for instant
queries that retrieve a single sample from each series, but still go
through all the effort of series iterator creation. All of that is
particularly bad if a series has many in-memory chunks.
This commit addresses the problem from two sides:
First, it merges preloading and iterator creation into one step,
i.e. the preload call returns an iterator for exactly the preloaded
chunks.
Second, the required mutex acquisition in chunkDesc has been greatly
reduced. That was enabled by a side effect of the first step, which is
that the iterator is only referencing pinned chunks, so there is no
risk of concurrent eviction anymore, and chunks can be accessed
without mutex acquisition.
To simplify the code changes for the above, the long-planned change of
ValueAtTime to ValueAtOrBefore time was performed at the same
time. (It should have been done first, but it kind of accidentally
happened while I was in the middle of writing the series iterator
changes. Sorry for that.) So far, we actively filtered the up to two
values that were returned by ValueAtTime, i.e. we invested work to
retrieve up to two values, and then we invested more work to throw one
of them away.
The SeriesIterator.BoundaryValues method can be removed once #1401 is
fixed. But I really didn't want to load even more changes into this
PR.
Benchmarks:
The BenchmarkFuzz.* benchmarks run 83% faster (i.e. about six times
faster) and allocate 95% fewer bytes. The reason for that is that the
benchmark reads one sample after another from the time series and
creates a new series iterator for each sample read.
To find out how much these improvements matter in practice, I have
mirrored a beefy Prometheus server at SoundCloud that suffers from
both issues #1035 and #1264. To reach steady state that would be
comparable, the server needs to run for 15d. So far, it has run for
1d. The test server currently has only half as many memory time series
and 60% of the memory chunks the main server has. The 90th percentile
rule evaluation cycle time is ~11s on the main server and only ~3s on
the test server. However, these numbers might get much closer over
time.
In addition to performance improvements, this commit removes about 150
LOC.
2016-02-16 09:47:50 -08:00
|
|
|
// ValueAtOrBeforeTime implements the SeriesIterator interface.
|
|
|
|
func (bit *boundedIterator) ValueAtOrBeforeTime(ts model.Time) model.SamplePair {
|
2015-05-27 02:24:56 -07:00
|
|
|
if ts < bit.start {
|
2016-09-28 14:40:26 -07:00
|
|
|
return model.ZeroSamplePair
|
2015-05-27 02:24:56 -07:00
|
|
|
}
|
Streamline series iterator creation
This will fix issue #1035 and will also help to make issue #1264 less
bad.
The fundamental problem in the current code:
In the preload phase, we quite accurately determine which chunks will
be used for the query being executed. However, in the subsequent step
of creating series iterators, the created iterators are referencing
_all_ in-memory chunks in their series, even the un-pinned ones. In
iterator creation, we copy a pointer to each in-memory chunk of a
series into the iterator. While this creates a certain amount of
allocation churn, the worst thing about it is that copying the chunk
pointer out of the chunkDesc requires a mutex acquisition. (Remember
that the iterator will also reference un-pinned chunks, so we need to
acquire the mutex to protect against concurrent eviction.) The worst
case happens if a series doesn't even contain any relevant samples for
the query time range. We notice that during preloading but then we
will still create a series iterator for it. But even for series that
do contain relevant samples, the overhead is quite bad for instant
queries that retrieve a single sample from each series, but still go
through all the effort of series iterator creation. All of that is
particularly bad if a series has many in-memory chunks.
This commit addresses the problem from two sides:
First, it merges preloading and iterator creation into one step,
i.e. the preload call returns an iterator for exactly the preloaded
chunks.
Second, the required mutex acquisition in chunkDesc has been greatly
reduced. That was enabled by a side effect of the first step, which is
that the iterator is only referencing pinned chunks, so there is no
risk of concurrent eviction anymore, and chunks can be accessed
without mutex acquisition.
To simplify the code changes for the above, the long-planned change of
ValueAtTime to ValueAtOrBefore time was performed at the same
time. (It should have been done first, but it kind of accidentally
happened while I was in the middle of writing the series iterator
changes. Sorry for that.) So far, we actively filtered the up to two
values that were returned by ValueAtTime, i.e. we invested work to
retrieve up to two values, and then we invested more work to throw one
of them away.
The SeriesIterator.BoundaryValues method can be removed once #1401 is
fixed. But I really didn't want to load even more changes into this
PR.
Benchmarks:
The BenchmarkFuzz.* benchmarks run 83% faster (i.e. about six times
faster) and allocate 95% fewer bytes. The reason for that is that the
benchmark reads one sample after another from the time series and
creates a new series iterator for each sample read.
To find out how much these improvements matter in practice, I have
mirrored a beefy Prometheus server at SoundCloud that suffers from
both issues #1035 and #1264. To reach steady state that would be
comparable, the server needs to run for 15d. So far, it has run for
1d. The test server currently has only half as many memory time series
and 60% of the memory chunks the main server has. The 90th percentile
rule evaluation cycle time is ~11s on the main server and only ~3s on
the test server. However, these numbers might get much closer over
time.
In addition to performance improvements, this commit removes about 150
LOC.
2016-02-16 09:47:50 -08:00
|
|
|
return bit.it.ValueAtOrBeforeTime(ts)
|
2015-05-27 02:24:56 -07:00
|
|
|
}
|
|
|
|
|
|
|
|
// RangeValues implements the SeriesIterator interface.
|
2015-08-22 05:52:35 -07:00
|
|
|
func (bit *boundedIterator) RangeValues(interval metric.Interval) []model.SamplePair {
|
2015-05-27 02:24:56 -07:00
|
|
|
if interval.NewestInclusive < bit.start {
|
2015-08-22 05:52:35 -07:00
|
|
|
return []model.SamplePair{}
|
2015-05-27 02:24:56 -07:00
|
|
|
}
|
|
|
|
if interval.OldestInclusive < bit.start {
|
|
|
|
interval.OldestInclusive = bit.start
|
|
|
|
}
|
|
|
|
return bit.it.RangeValues(interval)
|
2014-10-24 11:27:27 -07:00
|
|
|
}
|
|
|
|
|
2016-07-11 11:27:25 -07:00
|
|
|
// Metric implements SeriesIterator.
|
|
|
|
func (bit *boundedIterator) Metric() metric.Metric {
|
|
|
|
return bit.it.Metric()
|
|
|
|
}
|
|
|
|
|
|
|
|
// Close implements SeriesIterator.
|
|
|
|
func (bit *boundedIterator) Close() {
|
|
|
|
bit.it.Close()
|
|
|
|
}
|
|
|
|
|
|
|
|
// QueryRange implements Storage.
|
2016-09-15 15:58:06 -07:00
|
|
|
func (s *MemorySeriesStorage) QueryRange(_ context.Context, from, through model.Time, matchers ...*metric.LabelMatcher) ([]SeriesIterator, error) {
|
2016-09-18 04:20:46 -07:00
|
|
|
fpSeriesPairs, err := s.seriesForLabelMatchers(from, through, matchers...)
|
2016-07-11 11:27:25 -07:00
|
|
|
if err != nil {
|
|
|
|
return nil, err
|
|
|
|
}
|
2016-09-18 04:20:46 -07:00
|
|
|
iterators := make([]SeriesIterator, 0, len(fpSeriesPairs))
|
|
|
|
for _, pair := range fpSeriesPairs {
|
|
|
|
it := s.preloadChunksForRange(pair, from, through)
|
2016-07-11 11:27:25 -07:00
|
|
|
iterators = append(iterators, it)
|
2014-10-24 11:27:27 -07:00
|
|
|
}
|
2016-07-11 11:27:25 -07:00
|
|
|
return iterators, nil
|
|
|
|
}
|
|
|
|
|
|
|
|
// QueryInstant implements Storage.
|
2016-09-15 15:58:06 -07:00
|
|
|
func (s *MemorySeriesStorage) QueryInstant(_ context.Context, ts model.Time, stalenessDelta time.Duration, matchers ...*metric.LabelMatcher) ([]SeriesIterator, error) {
|
2016-07-11 11:27:25 -07:00
|
|
|
from := ts.Add(-stalenessDelta)
|
|
|
|
through := ts
|
|
|
|
|
2016-09-18 04:20:46 -07:00
|
|
|
fpSeriesPairs, err := s.seriesForLabelMatchers(from, through, matchers...)
|
2016-07-11 11:27:25 -07:00
|
|
|
if err != nil {
|
|
|
|
return nil, err
|
|
|
|
}
|
2016-09-18 04:20:46 -07:00
|
|
|
iterators := make([]SeriesIterator, 0, len(fpSeriesPairs))
|
|
|
|
for _, pair := range fpSeriesPairs {
|
|
|
|
it := s.preloadChunksForInstant(pair, from, through)
|
2016-07-11 11:27:25 -07:00
|
|
|
iterators = append(iterators, it)
|
|
|
|
}
|
|
|
|
return iterators, nil
|
2014-10-24 11:27:27 -07:00
|
|
|
}
|
|
|
|
|
storage: improve index lookups
tl;dr: This is not a fundamental solution to the indexing problem
(like tindex is) but it at least avoids utilizing the intersection
problem to the greatest possible amount.
In more detail:
Imagine the following query:
nicely:aggregating:rule{job="foo",env="prod"}
While it uses a nicely aggregating recording rule (which might have a
very low cardinality), Prometheus still intersects the low number of
fingerprints for `{__name__="nicely:aggregating:rule"}` with the many
thousands of fingerprints matching `{job="foo"}` and with the millions
of fingerprints matching `{env="prod"}`. This totally innocuous query
is dead slow if the Prometheus server has a lot of time series with
the `{env="prod"}` label. Ironically, if you make the query more
complicated, it becomes blazingly fast:
nicely:aggregating:rule{job=~"foo",env=~"prod"}
Why so? Because Prometheus only intersects with non-Equal matchers if
there are no Equal matchers. That's good in this case because it
retrieves the few fingerprints for
`{__name__="nicely:aggregating:rule"}` and then starts right ahead to
retrieve the metric for those FPs and checking individually if they
match the other matchers.
This change is generalizing the idea of when to stop intersecting FPs
and go into "retrieve metrics and check them individually against
remaining matchers" mode:
- First, sort all matchers by "expected cardinality". Matchers
matching the empty string are always worst (and never used for
intersections). Equal matchers are in general consider best, but by
using some crude heuristics, we declare some better than others
(instance labels or anything that looks like a recording rule).
- Then go through the matchers until we hit a threshold of remaining
FPs in the intersection. This threshold is higher if we are already
in the non-Equal matcher area as intersection is even more expensive
here.
- Once the threshold has been reached (or we have run out of matchers
that do not match the empty string), start with "retrieve metrics
and check them individually against remaining matchers".
A beefy server at SoundCloud was spending 67% of its CPU time in index
lookups (fingerprintsForLabelPairs), serving mostly a dashboard that
is exclusively built with recording rules. With this change, it spends
only 35% in fingerprintsForLabelPairs. The CPU usage dropped from 26
cores to 18 cores. The median latency for query_range dropped from 14s
to 50ms(!). As expected, higher percentile latency didn't improve that
much because the new approach is _occasionally_ running into the worst
case while the old one was _systematically_ doing so. The 99th
percentile latency is now about as high as the median before (14s)
while it was almost twice as high before (26s).
2016-06-28 11:18:32 -07:00
|
|
|
// fingerprintsForLabelPair returns the fingerprints with the given
|
|
|
|
// LabelPair. If intersectWith is non-nil, the method will only return
|
|
|
|
// fingerprints that are also contained in intersectsWith. If mergeWith is
|
|
|
|
// non-nil, the found fingerprints are added to the given map. The returned map
|
|
|
|
// is the same as the given one.
|
|
|
|
func (s *MemorySeriesStorage) fingerprintsForLabelPair(
|
|
|
|
pair model.LabelPair,
|
|
|
|
mergeWith map[model.Fingerprint]struct{},
|
|
|
|
intersectWith map[model.Fingerprint]struct{},
|
|
|
|
) map[model.Fingerprint]struct{} {
|
|
|
|
if mergeWith == nil {
|
|
|
|
mergeWith = map[model.Fingerprint]struct{}{}
|
|
|
|
}
|
|
|
|
for _, fp := range s.persistence.fingerprintsForLabelPair(pair) {
|
|
|
|
if intersectWith == nil {
|
|
|
|
mergeWith[fp] = struct{}{}
|
|
|
|
continue
|
2014-10-24 11:27:27 -07:00
|
|
|
}
|
storage: improve index lookups
tl;dr: This is not a fundamental solution to the indexing problem
(like tindex is) but it at least avoids utilizing the intersection
problem to the greatest possible amount.
In more detail:
Imagine the following query:
nicely:aggregating:rule{job="foo",env="prod"}
While it uses a nicely aggregating recording rule (which might have a
very low cardinality), Prometheus still intersects the low number of
fingerprints for `{__name__="nicely:aggregating:rule"}` with the many
thousands of fingerprints matching `{job="foo"}` and with the millions
of fingerprints matching `{env="prod"}`. This totally innocuous query
is dead slow if the Prometheus server has a lot of time series with
the `{env="prod"}` label. Ironically, if you make the query more
complicated, it becomes blazingly fast:
nicely:aggregating:rule{job=~"foo",env=~"prod"}
Why so? Because Prometheus only intersects with non-Equal matchers if
there are no Equal matchers. That's good in this case because it
retrieves the few fingerprints for
`{__name__="nicely:aggregating:rule"}` and then starts right ahead to
retrieve the metric for those FPs and checking individually if they
match the other matchers.
This change is generalizing the idea of when to stop intersecting FPs
and go into "retrieve metrics and check them individually against
remaining matchers" mode:
- First, sort all matchers by "expected cardinality". Matchers
matching the empty string are always worst (and never used for
intersections). Equal matchers are in general consider best, but by
using some crude heuristics, we declare some better than others
(instance labels or anything that looks like a recording rule).
- Then go through the matchers until we hit a threshold of remaining
FPs in the intersection. This threshold is higher if we are already
in the non-Equal matcher area as intersection is even more expensive
here.
- Once the threshold has been reached (or we have run out of matchers
that do not match the empty string), start with "retrieve metrics
and check them individually against remaining matchers".
A beefy server at SoundCloud was spending 67% of its CPU time in index
lookups (fingerprintsForLabelPairs), serving mostly a dashboard that
is exclusively built with recording rules. With this change, it spends
only 35% in fingerprintsForLabelPairs. The CPU usage dropped from 26
cores to 18 cores. The median latency for query_range dropped from 14s
to 50ms(!). As expected, higher percentile latency didn't improve that
much because the new approach is _occasionally_ running into the worst
case while the old one was _systematically_ doing so. The 99th
percentile latency is now about as high as the median before (14s)
while it was almost twice as high before (26s).
2016-06-28 11:18:32 -07:00
|
|
|
if _, ok := intersectWith[fp]; ok {
|
|
|
|
mergeWith[fp] = struct{}{}
|
2014-10-24 11:27:27 -07:00
|
|
|
}
|
|
|
|
}
|
storage: improve index lookups
tl;dr: This is not a fundamental solution to the indexing problem
(like tindex is) but it at least avoids utilizing the intersection
problem to the greatest possible amount.
In more detail:
Imagine the following query:
nicely:aggregating:rule{job="foo",env="prod"}
While it uses a nicely aggregating recording rule (which might have a
very low cardinality), Prometheus still intersects the low number of
fingerprints for `{__name__="nicely:aggregating:rule"}` with the many
thousands of fingerprints matching `{job="foo"}` and with the millions
of fingerprints matching `{env="prod"}`. This totally innocuous query
is dead slow if the Prometheus server has a lot of time series with
the `{env="prod"}` label. Ironically, if you make the query more
complicated, it becomes blazingly fast:
nicely:aggregating:rule{job=~"foo",env=~"prod"}
Why so? Because Prometheus only intersects with non-Equal matchers if
there are no Equal matchers. That's good in this case because it
retrieves the few fingerprints for
`{__name__="nicely:aggregating:rule"}` and then starts right ahead to
retrieve the metric for those FPs and checking individually if they
match the other matchers.
This change is generalizing the idea of when to stop intersecting FPs
and go into "retrieve metrics and check them individually against
remaining matchers" mode:
- First, sort all matchers by "expected cardinality". Matchers
matching the empty string are always worst (and never used for
intersections). Equal matchers are in general consider best, but by
using some crude heuristics, we declare some better than others
(instance labels or anything that looks like a recording rule).
- Then go through the matchers until we hit a threshold of remaining
FPs in the intersection. This threshold is higher if we are already
in the non-Equal matcher area as intersection is even more expensive
here.
- Once the threshold has been reached (or we have run out of matchers
that do not match the empty string), start with "retrieve metrics
and check them individually against remaining matchers".
A beefy server at SoundCloud was spending 67% of its CPU time in index
lookups (fingerprintsForLabelPairs), serving mostly a dashboard that
is exclusively built with recording rules. With this change, it spends
only 35% in fingerprintsForLabelPairs. The CPU usage dropped from 26
cores to 18 cores. The median latency for query_range dropped from 14s
to 50ms(!). As expected, higher percentile latency didn't improve that
much because the new approach is _occasionally_ running into the worst
case while the old one was _systematically_ doing so. The 99th
percentile latency is now about as high as the median before (14s)
while it was almost twice as high before (26s).
2016-06-28 11:18:32 -07:00
|
|
|
return mergeWith
|
2015-06-15 09:25:31 -07:00
|
|
|
}
|
2014-10-24 11:27:27 -07:00
|
|
|
|
2015-06-15 09:25:31 -07:00
|
|
|
// MetricsForLabelMatchers implements Storage.
|
2016-06-28 23:14:23 -07:00
|
|
|
func (s *MemorySeriesStorage) MetricsForLabelMatchers(
|
2016-09-15 15:58:06 -07:00
|
|
|
_ context.Context,
|
2016-07-11 11:27:25 -07:00
|
|
|
from, through model.Time,
|
|
|
|
matcherSets ...metric.LabelMatchers,
|
|
|
|
) ([]metric.Metric, error) {
|
|
|
|
fpToMetric := map[model.Fingerprint]metric.Metric{}
|
|
|
|
for _, matchers := range matcherSets {
|
|
|
|
metrics, err := s.metricsForLabelMatchers(from, through, matchers...)
|
|
|
|
if err != nil {
|
|
|
|
return nil, err
|
|
|
|
}
|
|
|
|
for fp, m := range metrics {
|
|
|
|
fpToMetric[fp] = m
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
metrics := make([]metric.Metric, 0, len(fpToMetric))
|
|
|
|
for _, m := range fpToMetric {
|
|
|
|
metrics = append(metrics, m)
|
|
|
|
}
|
|
|
|
return metrics, nil
|
|
|
|
}
|
|
|
|
|
2016-09-18 04:20:46 -07:00
|
|
|
// candidateFPsForLabelMatchers returns candidate FPs for given matchers and remaining matchers to be checked.
|
2016-09-18 03:37:17 -07:00
|
|
|
func (s *MemorySeriesStorage) candidateFPsForLabelMatchers(
|
2016-03-08 15:09:42 -08:00
|
|
|
matchers ...*metric.LabelMatcher,
|
2016-09-18 03:37:17 -07:00
|
|
|
) (map[model.Fingerprint]struct{}, []*metric.LabelMatcher, error) {
|
storage: improve index lookups
tl;dr: This is not a fundamental solution to the indexing problem
(like tindex is) but it at least avoids utilizing the intersection
problem to the greatest possible amount.
In more detail:
Imagine the following query:
nicely:aggregating:rule{job="foo",env="prod"}
While it uses a nicely aggregating recording rule (which might have a
very low cardinality), Prometheus still intersects the low number of
fingerprints for `{__name__="nicely:aggregating:rule"}` with the many
thousands of fingerprints matching `{job="foo"}` and with the millions
of fingerprints matching `{env="prod"}`. This totally innocuous query
is dead slow if the Prometheus server has a lot of time series with
the `{env="prod"}` label. Ironically, if you make the query more
complicated, it becomes blazingly fast:
nicely:aggregating:rule{job=~"foo",env=~"prod"}
Why so? Because Prometheus only intersects with non-Equal matchers if
there are no Equal matchers. That's good in this case because it
retrieves the few fingerprints for
`{__name__="nicely:aggregating:rule"}` and then starts right ahead to
retrieve the metric for those FPs and checking individually if they
match the other matchers.
This change is generalizing the idea of when to stop intersecting FPs
and go into "retrieve metrics and check them individually against
remaining matchers" mode:
- First, sort all matchers by "expected cardinality". Matchers
matching the empty string are always worst (and never used for
intersections). Equal matchers are in general consider best, but by
using some crude heuristics, we declare some better than others
(instance labels or anything that looks like a recording rule).
- Then go through the matchers until we hit a threshold of remaining
FPs in the intersection. This threshold is higher if we are already
in the non-Equal matcher area as intersection is even more expensive
here.
- Once the threshold has been reached (or we have run out of matchers
that do not match the empty string), start with "retrieve metrics
and check them individually against remaining matchers".
A beefy server at SoundCloud was spending 67% of its CPU time in index
lookups (fingerprintsForLabelPairs), serving mostly a dashboard that
is exclusively built with recording rules. With this change, it spends
only 35% in fingerprintsForLabelPairs. The CPU usage dropped from 26
cores to 18 cores. The median latency for query_range dropped from 14s
to 50ms(!). As expected, higher percentile latency didn't improve that
much because the new approach is _occasionally_ running into the worst
case while the old one was _systematically_ doing so. The 99th
percentile latency is now about as high as the median before (14s)
while it was almost twice as high before (26s).
2016-06-28 11:18:32 -07:00
|
|
|
sort.Sort(metric.LabelMatchers(matchers))
|
|
|
|
|
|
|
|
if len(matchers) == 0 || matchers[0].MatchesEmptyString() {
|
|
|
|
// No matchers at all or even the best matcher matches the empty string.
|
2016-09-18 03:37:17 -07:00
|
|
|
return nil, nil, nil
|
storage: improve index lookups
tl;dr: This is not a fundamental solution to the indexing problem
(like tindex is) but it at least avoids utilizing the intersection
problem to the greatest possible amount.
In more detail:
Imagine the following query:
nicely:aggregating:rule{job="foo",env="prod"}
While it uses a nicely aggregating recording rule (which might have a
very low cardinality), Prometheus still intersects the low number of
fingerprints for `{__name__="nicely:aggregating:rule"}` with the many
thousands of fingerprints matching `{job="foo"}` and with the millions
of fingerprints matching `{env="prod"}`. This totally innocuous query
is dead slow if the Prometheus server has a lot of time series with
the `{env="prod"}` label. Ironically, if you make the query more
complicated, it becomes blazingly fast:
nicely:aggregating:rule{job=~"foo",env=~"prod"}
Why so? Because Prometheus only intersects with non-Equal matchers if
there are no Equal matchers. That's good in this case because it
retrieves the few fingerprints for
`{__name__="nicely:aggregating:rule"}` and then starts right ahead to
retrieve the metric for those FPs and checking individually if they
match the other matchers.
This change is generalizing the idea of when to stop intersecting FPs
and go into "retrieve metrics and check them individually against
remaining matchers" mode:
- First, sort all matchers by "expected cardinality". Matchers
matching the empty string are always worst (and never used for
intersections). Equal matchers are in general consider best, but by
using some crude heuristics, we declare some better than others
(instance labels or anything that looks like a recording rule).
- Then go through the matchers until we hit a threshold of remaining
FPs in the intersection. This threshold is higher if we are already
in the non-Equal matcher area as intersection is even more expensive
here.
- Once the threshold has been reached (or we have run out of matchers
that do not match the empty string), start with "retrieve metrics
and check them individually against remaining matchers".
A beefy server at SoundCloud was spending 67% of its CPU time in index
lookups (fingerprintsForLabelPairs), serving mostly a dashboard that
is exclusively built with recording rules. With this change, it spends
only 35% in fingerprintsForLabelPairs. The CPU usage dropped from 26
cores to 18 cores. The median latency for query_range dropped from 14s
to 50ms(!). As expected, higher percentile latency didn't improve that
much because the new approach is _occasionally_ running into the worst
case while the old one was _systematically_ doing so. The 99th
percentile latency is now about as high as the median before (14s)
while it was almost twice as high before (26s).
2016-06-28 11:18:32 -07:00
|
|
|
}
|
|
|
|
|
2015-06-15 09:25:31 -07:00
|
|
|
var (
|
storage: improve index lookups
tl;dr: This is not a fundamental solution to the indexing problem
(like tindex is) but it at least avoids utilizing the intersection
problem to the greatest possible amount.
In more detail:
Imagine the following query:
nicely:aggregating:rule{job="foo",env="prod"}
While it uses a nicely aggregating recording rule (which might have a
very low cardinality), Prometheus still intersects the low number of
fingerprints for `{__name__="nicely:aggregating:rule"}` with the many
thousands of fingerprints matching `{job="foo"}` and with the millions
of fingerprints matching `{env="prod"}`. This totally innocuous query
is dead slow if the Prometheus server has a lot of time series with
the `{env="prod"}` label. Ironically, if you make the query more
complicated, it becomes blazingly fast:
nicely:aggregating:rule{job=~"foo",env=~"prod"}
Why so? Because Prometheus only intersects with non-Equal matchers if
there are no Equal matchers. That's good in this case because it
retrieves the few fingerprints for
`{__name__="nicely:aggregating:rule"}` and then starts right ahead to
retrieve the metric for those FPs and checking individually if they
match the other matchers.
This change is generalizing the idea of when to stop intersecting FPs
and go into "retrieve metrics and check them individually against
remaining matchers" mode:
- First, sort all matchers by "expected cardinality". Matchers
matching the empty string are always worst (and never used for
intersections). Equal matchers are in general consider best, but by
using some crude heuristics, we declare some better than others
(instance labels or anything that looks like a recording rule).
- Then go through the matchers until we hit a threshold of remaining
FPs in the intersection. This threshold is higher if we are already
in the non-Equal matcher area as intersection is even more expensive
here.
- Once the threshold has been reached (or we have run out of matchers
that do not match the empty string), start with "retrieve metrics
and check them individually against remaining matchers".
A beefy server at SoundCloud was spending 67% of its CPU time in index
lookups (fingerprintsForLabelPairs), serving mostly a dashboard that
is exclusively built with recording rules. With this change, it spends
only 35% in fingerprintsForLabelPairs. The CPU usage dropped from 26
cores to 18 cores. The median latency for query_range dropped from 14s
to 50ms(!). As expected, higher percentile latency didn't improve that
much because the new approach is _occasionally_ running into the worst
case while the old one was _systematically_ doing so. The 99th
percentile latency is now about as high as the median before (14s)
while it was almost twice as high before (26s).
2016-06-28 11:18:32 -07:00
|
|
|
matcherIdx int
|
2016-09-18 03:37:17 -07:00
|
|
|
candidateFPs map[model.Fingerprint]struct{}
|
2015-06-15 09:25:31 -07:00
|
|
|
)
|
storage: improve index lookups
tl;dr: This is not a fundamental solution to the indexing problem
(like tindex is) but it at least avoids utilizing the intersection
problem to the greatest possible amount.
In more detail:
Imagine the following query:
nicely:aggregating:rule{job="foo",env="prod"}
While it uses a nicely aggregating recording rule (which might have a
very low cardinality), Prometheus still intersects the low number of
fingerprints for `{__name__="nicely:aggregating:rule"}` with the many
thousands of fingerprints matching `{job="foo"}` and with the millions
of fingerprints matching `{env="prod"}`. This totally innocuous query
is dead slow if the Prometheus server has a lot of time series with
the `{env="prod"}` label. Ironically, if you make the query more
complicated, it becomes blazingly fast:
nicely:aggregating:rule{job=~"foo",env=~"prod"}
Why so? Because Prometheus only intersects with non-Equal matchers if
there are no Equal matchers. That's good in this case because it
retrieves the few fingerprints for
`{__name__="nicely:aggregating:rule"}` and then starts right ahead to
retrieve the metric for those FPs and checking individually if they
match the other matchers.
This change is generalizing the idea of when to stop intersecting FPs
and go into "retrieve metrics and check them individually against
remaining matchers" mode:
- First, sort all matchers by "expected cardinality". Matchers
matching the empty string are always worst (and never used for
intersections). Equal matchers are in general consider best, but by
using some crude heuristics, we declare some better than others
(instance labels or anything that looks like a recording rule).
- Then go through the matchers until we hit a threshold of remaining
FPs in the intersection. This threshold is higher if we are already
in the non-Equal matcher area as intersection is even more expensive
here.
- Once the threshold has been reached (or we have run out of matchers
that do not match the empty string), start with "retrieve metrics
and check them individually against remaining matchers".
A beefy server at SoundCloud was spending 67% of its CPU time in index
lookups (fingerprintsForLabelPairs), serving mostly a dashboard that
is exclusively built with recording rules. With this change, it spends
only 35% in fingerprintsForLabelPairs. The CPU usage dropped from 26
cores to 18 cores. The median latency for query_range dropped from 14s
to 50ms(!). As expected, higher percentile latency didn't improve that
much because the new approach is _occasionally_ running into the worst
case while the old one was _systematically_ doing so. The 99th
percentile latency is now about as high as the median before (14s)
while it was almost twice as high before (26s).
2016-06-28 11:18:32 -07:00
|
|
|
|
|
|
|
// Equal matchers.
|
2016-09-18 03:37:17 -07:00
|
|
|
for ; matcherIdx < len(matchers) && (candidateFPs == nil || len(candidateFPs) > fpEqualMatchThreshold); matcherIdx++ {
|
storage: improve index lookups
tl;dr: This is not a fundamental solution to the indexing problem
(like tindex is) but it at least avoids utilizing the intersection
problem to the greatest possible amount.
In more detail:
Imagine the following query:
nicely:aggregating:rule{job="foo",env="prod"}
While it uses a nicely aggregating recording rule (which might have a
very low cardinality), Prometheus still intersects the low number of
fingerprints for `{__name__="nicely:aggregating:rule"}` with the many
thousands of fingerprints matching `{job="foo"}` and with the millions
of fingerprints matching `{env="prod"}`. This totally innocuous query
is dead slow if the Prometheus server has a lot of time series with
the `{env="prod"}` label. Ironically, if you make the query more
complicated, it becomes blazingly fast:
nicely:aggregating:rule{job=~"foo",env=~"prod"}
Why so? Because Prometheus only intersects with non-Equal matchers if
there are no Equal matchers. That's good in this case because it
retrieves the few fingerprints for
`{__name__="nicely:aggregating:rule"}` and then starts right ahead to
retrieve the metric for those FPs and checking individually if they
match the other matchers.
This change is generalizing the idea of when to stop intersecting FPs
and go into "retrieve metrics and check them individually against
remaining matchers" mode:
- First, sort all matchers by "expected cardinality". Matchers
matching the empty string are always worst (and never used for
intersections). Equal matchers are in general consider best, but by
using some crude heuristics, we declare some better than others
(instance labels or anything that looks like a recording rule).
- Then go through the matchers until we hit a threshold of remaining
FPs in the intersection. This threshold is higher if we are already
in the non-Equal matcher area as intersection is even more expensive
here.
- Once the threshold has been reached (or we have run out of matchers
that do not match the empty string), start with "retrieve metrics
and check them individually against remaining matchers".
A beefy server at SoundCloud was spending 67% of its CPU time in index
lookups (fingerprintsForLabelPairs), serving mostly a dashboard that
is exclusively built with recording rules. With this change, it spends
only 35% in fingerprintsForLabelPairs. The CPU usage dropped from 26
cores to 18 cores. The median latency for query_range dropped from 14s
to 50ms(!). As expected, higher percentile latency didn't improve that
much because the new approach is _occasionally_ running into the worst
case while the old one was _systematically_ doing so. The 99th
percentile latency is now about as high as the median before (14s)
while it was almost twice as high before (26s).
2016-06-28 11:18:32 -07:00
|
|
|
m := matchers[matcherIdx]
|
|
|
|
if m.Type != metric.Equal || m.MatchesEmptyString() {
|
|
|
|
break
|
|
|
|
}
|
2016-09-18 03:37:17 -07:00
|
|
|
candidateFPs = s.fingerprintsForLabelPair(
|
storage: improve index lookups
tl;dr: This is not a fundamental solution to the indexing problem
(like tindex is) but it at least avoids utilizing the intersection
problem to the greatest possible amount.
In more detail:
Imagine the following query:
nicely:aggregating:rule{job="foo",env="prod"}
While it uses a nicely aggregating recording rule (which might have a
very low cardinality), Prometheus still intersects the low number of
fingerprints for `{__name__="nicely:aggregating:rule"}` with the many
thousands of fingerprints matching `{job="foo"}` and with the millions
of fingerprints matching `{env="prod"}`. This totally innocuous query
is dead slow if the Prometheus server has a lot of time series with
the `{env="prod"}` label. Ironically, if you make the query more
complicated, it becomes blazingly fast:
nicely:aggregating:rule{job=~"foo",env=~"prod"}
Why so? Because Prometheus only intersects with non-Equal matchers if
there are no Equal matchers. That's good in this case because it
retrieves the few fingerprints for
`{__name__="nicely:aggregating:rule"}` and then starts right ahead to
retrieve the metric for those FPs and checking individually if they
match the other matchers.
This change is generalizing the idea of when to stop intersecting FPs
and go into "retrieve metrics and check them individually against
remaining matchers" mode:
- First, sort all matchers by "expected cardinality". Matchers
matching the empty string are always worst (and never used for
intersections). Equal matchers are in general consider best, but by
using some crude heuristics, we declare some better than others
(instance labels or anything that looks like a recording rule).
- Then go through the matchers until we hit a threshold of remaining
FPs in the intersection. This threshold is higher if we are already
in the non-Equal matcher area as intersection is even more expensive
here.
- Once the threshold has been reached (or we have run out of matchers
that do not match the empty string), start with "retrieve metrics
and check them individually against remaining matchers".
A beefy server at SoundCloud was spending 67% of its CPU time in index
lookups (fingerprintsForLabelPairs), serving mostly a dashboard that
is exclusively built with recording rules. With this change, it spends
only 35% in fingerprintsForLabelPairs. The CPU usage dropped from 26
cores to 18 cores. The median latency for query_range dropped from 14s
to 50ms(!). As expected, higher percentile latency didn't improve that
much because the new approach is _occasionally_ running into the worst
case while the old one was _systematically_ doing so. The 99th
percentile latency is now about as high as the median before (14s)
while it was almost twice as high before (26s).
2016-06-28 11:18:32 -07:00
|
|
|
model.LabelPair{
|
|
|
|
Name: m.Name,
|
|
|
|
Value: m.Value,
|
|
|
|
},
|
|
|
|
nil,
|
2016-09-18 03:37:17 -07:00
|
|
|
candidateFPs,
|
storage: improve index lookups
tl;dr: This is not a fundamental solution to the indexing problem
(like tindex is) but it at least avoids utilizing the intersection
problem to the greatest possible amount.
In more detail:
Imagine the following query:
nicely:aggregating:rule{job="foo",env="prod"}
While it uses a nicely aggregating recording rule (which might have a
very low cardinality), Prometheus still intersects the low number of
fingerprints for `{__name__="nicely:aggregating:rule"}` with the many
thousands of fingerprints matching `{job="foo"}` and with the millions
of fingerprints matching `{env="prod"}`. This totally innocuous query
is dead slow if the Prometheus server has a lot of time series with
the `{env="prod"}` label. Ironically, if you make the query more
complicated, it becomes blazingly fast:
nicely:aggregating:rule{job=~"foo",env=~"prod"}
Why so? Because Prometheus only intersects with non-Equal matchers if
there are no Equal matchers. That's good in this case because it
retrieves the few fingerprints for
`{__name__="nicely:aggregating:rule"}` and then starts right ahead to
retrieve the metric for those FPs and checking individually if they
match the other matchers.
This change is generalizing the idea of when to stop intersecting FPs
and go into "retrieve metrics and check them individually against
remaining matchers" mode:
- First, sort all matchers by "expected cardinality". Matchers
matching the empty string are always worst (and never used for
intersections). Equal matchers are in general consider best, but by
using some crude heuristics, we declare some better than others
(instance labels or anything that looks like a recording rule).
- Then go through the matchers until we hit a threshold of remaining
FPs in the intersection. This threshold is higher if we are already
in the non-Equal matcher area as intersection is even more expensive
here.
- Once the threshold has been reached (or we have run out of matchers
that do not match the empty string), start with "retrieve metrics
and check them individually against remaining matchers".
A beefy server at SoundCloud was spending 67% of its CPU time in index
lookups (fingerprintsForLabelPairs), serving mostly a dashboard that
is exclusively built with recording rules. With this change, it spends
only 35% in fingerprintsForLabelPairs. The CPU usage dropped from 26
cores to 18 cores. The median latency for query_range dropped from 14s
to 50ms(!). As expected, higher percentile latency didn't improve that
much because the new approach is _occasionally_ running into the worst
case while the old one was _systematically_ doing so. The 99th
percentile latency is now about as high as the median before (14s)
while it was almost twice as high before (26s).
2016-06-28 11:18:32 -07:00
|
|
|
)
|
2016-09-18 03:37:17 -07:00
|
|
|
if len(candidateFPs) == 0 {
|
|
|
|
return nil, nil, nil
|
2015-06-15 09:25:31 -07:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
storage: improve index lookups
tl;dr: This is not a fundamental solution to the indexing problem
(like tindex is) but it at least avoids utilizing the intersection
problem to the greatest possible amount.
In more detail:
Imagine the following query:
nicely:aggregating:rule{job="foo",env="prod"}
While it uses a nicely aggregating recording rule (which might have a
very low cardinality), Prometheus still intersects the low number of
fingerprints for `{__name__="nicely:aggregating:rule"}` with the many
thousands of fingerprints matching `{job="foo"}` and with the millions
of fingerprints matching `{env="prod"}`. This totally innocuous query
is dead slow if the Prometheus server has a lot of time series with
the `{env="prod"}` label. Ironically, if you make the query more
complicated, it becomes blazingly fast:
nicely:aggregating:rule{job=~"foo",env=~"prod"}
Why so? Because Prometheus only intersects with non-Equal matchers if
there are no Equal matchers. That's good in this case because it
retrieves the few fingerprints for
`{__name__="nicely:aggregating:rule"}` and then starts right ahead to
retrieve the metric for those FPs and checking individually if they
match the other matchers.
This change is generalizing the idea of when to stop intersecting FPs
and go into "retrieve metrics and check them individually against
remaining matchers" mode:
- First, sort all matchers by "expected cardinality". Matchers
matching the empty string are always worst (and never used for
intersections). Equal matchers are in general consider best, but by
using some crude heuristics, we declare some better than others
(instance labels or anything that looks like a recording rule).
- Then go through the matchers until we hit a threshold of remaining
FPs in the intersection. This threshold is higher if we are already
in the non-Equal matcher area as intersection is even more expensive
here.
- Once the threshold has been reached (or we have run out of matchers
that do not match the empty string), start with "retrieve metrics
and check them individually against remaining matchers".
A beefy server at SoundCloud was spending 67% of its CPU time in index
lookups (fingerprintsForLabelPairs), serving mostly a dashboard that
is exclusively built with recording rules. With this change, it spends
only 35% in fingerprintsForLabelPairs. The CPU usage dropped from 26
cores to 18 cores. The median latency for query_range dropped from 14s
to 50ms(!). As expected, higher percentile latency didn't improve that
much because the new approach is _occasionally_ running into the worst
case while the old one was _systematically_ doing so. The 99th
percentile latency is now about as high as the median before (14s)
while it was almost twice as high before (26s).
2016-06-28 11:18:32 -07:00
|
|
|
// Other matchers.
|
2016-09-18 03:37:17 -07:00
|
|
|
for ; matcherIdx < len(matchers) && (candidateFPs == nil || len(candidateFPs) > fpOtherMatchThreshold); matcherIdx++ {
|
storage: improve index lookups
tl;dr: This is not a fundamental solution to the indexing problem
(like tindex is) but it at least avoids utilizing the intersection
problem to the greatest possible amount.
In more detail:
Imagine the following query:
nicely:aggregating:rule{job="foo",env="prod"}
While it uses a nicely aggregating recording rule (which might have a
very low cardinality), Prometheus still intersects the low number of
fingerprints for `{__name__="nicely:aggregating:rule"}` with the many
thousands of fingerprints matching `{job="foo"}` and with the millions
of fingerprints matching `{env="prod"}`. This totally innocuous query
is dead slow if the Prometheus server has a lot of time series with
the `{env="prod"}` label. Ironically, if you make the query more
complicated, it becomes blazingly fast:
nicely:aggregating:rule{job=~"foo",env=~"prod"}
Why so? Because Prometheus only intersects with non-Equal matchers if
there are no Equal matchers. That's good in this case because it
retrieves the few fingerprints for
`{__name__="nicely:aggregating:rule"}` and then starts right ahead to
retrieve the metric for those FPs and checking individually if they
match the other matchers.
This change is generalizing the idea of when to stop intersecting FPs
and go into "retrieve metrics and check them individually against
remaining matchers" mode:
- First, sort all matchers by "expected cardinality". Matchers
matching the empty string are always worst (and never used for
intersections). Equal matchers are in general consider best, but by
using some crude heuristics, we declare some better than others
(instance labels or anything that looks like a recording rule).
- Then go through the matchers until we hit a threshold of remaining
FPs in the intersection. This threshold is higher if we are already
in the non-Equal matcher area as intersection is even more expensive
here.
- Once the threshold has been reached (or we have run out of matchers
that do not match the empty string), start with "retrieve metrics
and check them individually against remaining matchers".
A beefy server at SoundCloud was spending 67% of its CPU time in index
lookups (fingerprintsForLabelPairs), serving mostly a dashboard that
is exclusively built with recording rules. With this change, it spends
only 35% in fingerprintsForLabelPairs. The CPU usage dropped from 26
cores to 18 cores. The median latency for query_range dropped from 14s
to 50ms(!). As expected, higher percentile latency didn't improve that
much because the new approach is _occasionally_ running into the worst
case while the old one was _systematically_ doing so. The 99th
percentile latency is now about as high as the median before (14s)
while it was almost twice as high before (26s).
2016-06-28 11:18:32 -07:00
|
|
|
m := matchers[matcherIdx]
|
|
|
|
if m.MatchesEmptyString() {
|
|
|
|
break
|
|
|
|
}
|
2016-07-11 11:27:25 -07:00
|
|
|
|
2016-09-15 15:58:06 -07:00
|
|
|
lvs, err := s.LabelValuesForLabelName(context.TODO(), m.Name)
|
2016-07-11 11:27:25 -07:00
|
|
|
if err != nil {
|
2016-09-18 03:37:17 -07:00
|
|
|
return nil, nil, err
|
2016-07-11 11:27:25 -07:00
|
|
|
}
|
|
|
|
lvs = m.Filter(lvs)
|
storage: improve index lookups
tl;dr: This is not a fundamental solution to the indexing problem
(like tindex is) but it at least avoids utilizing the intersection
problem to the greatest possible amount.
In more detail:
Imagine the following query:
nicely:aggregating:rule{job="foo",env="prod"}
While it uses a nicely aggregating recording rule (which might have a
very low cardinality), Prometheus still intersects the low number of
fingerprints for `{__name__="nicely:aggregating:rule"}` with the many
thousands of fingerprints matching `{job="foo"}` and with the millions
of fingerprints matching `{env="prod"}`. This totally innocuous query
is dead slow if the Prometheus server has a lot of time series with
the `{env="prod"}` label. Ironically, if you make the query more
complicated, it becomes blazingly fast:
nicely:aggregating:rule{job=~"foo",env=~"prod"}
Why so? Because Prometheus only intersects with non-Equal matchers if
there are no Equal matchers. That's good in this case because it
retrieves the few fingerprints for
`{__name__="nicely:aggregating:rule"}` and then starts right ahead to
retrieve the metric for those FPs and checking individually if they
match the other matchers.
This change is generalizing the idea of when to stop intersecting FPs
and go into "retrieve metrics and check them individually against
remaining matchers" mode:
- First, sort all matchers by "expected cardinality". Matchers
matching the empty string are always worst (and never used for
intersections). Equal matchers are in general consider best, but by
using some crude heuristics, we declare some better than others
(instance labels or anything that looks like a recording rule).
- Then go through the matchers until we hit a threshold of remaining
FPs in the intersection. This threshold is higher if we are already
in the non-Equal matcher area as intersection is even more expensive
here.
- Once the threshold has been reached (or we have run out of matchers
that do not match the empty string), start with "retrieve metrics
and check them individually against remaining matchers".
A beefy server at SoundCloud was spending 67% of its CPU time in index
lookups (fingerprintsForLabelPairs), serving mostly a dashboard that
is exclusively built with recording rules. With this change, it spends
only 35% in fingerprintsForLabelPairs. The CPU usage dropped from 26
cores to 18 cores. The median latency for query_range dropped from 14s
to 50ms(!). As expected, higher percentile latency didn't improve that
much because the new approach is _occasionally_ running into the worst
case while the old one was _systematically_ doing so. The 99th
percentile latency is now about as high as the median before (14s)
while it was almost twice as high before (26s).
2016-06-28 11:18:32 -07:00
|
|
|
if len(lvs) == 0 {
|
2016-09-18 03:37:17 -07:00
|
|
|
return nil, nil, nil
|
storage: improve index lookups
tl;dr: This is not a fundamental solution to the indexing problem
(like tindex is) but it at least avoids utilizing the intersection
problem to the greatest possible amount.
In more detail:
Imagine the following query:
nicely:aggregating:rule{job="foo",env="prod"}
While it uses a nicely aggregating recording rule (which might have a
very low cardinality), Prometheus still intersects the low number of
fingerprints for `{__name__="nicely:aggregating:rule"}` with the many
thousands of fingerprints matching `{job="foo"}` and with the millions
of fingerprints matching `{env="prod"}`. This totally innocuous query
is dead slow if the Prometheus server has a lot of time series with
the `{env="prod"}` label. Ironically, if you make the query more
complicated, it becomes blazingly fast:
nicely:aggregating:rule{job=~"foo",env=~"prod"}
Why so? Because Prometheus only intersects with non-Equal matchers if
there are no Equal matchers. That's good in this case because it
retrieves the few fingerprints for
`{__name__="nicely:aggregating:rule"}` and then starts right ahead to
retrieve the metric for those FPs and checking individually if they
match the other matchers.
This change is generalizing the idea of when to stop intersecting FPs
and go into "retrieve metrics and check them individually against
remaining matchers" mode:
- First, sort all matchers by "expected cardinality". Matchers
matching the empty string are always worst (and never used for
intersections). Equal matchers are in general consider best, but by
using some crude heuristics, we declare some better than others
(instance labels or anything that looks like a recording rule).
- Then go through the matchers until we hit a threshold of remaining
FPs in the intersection. This threshold is higher if we are already
in the non-Equal matcher area as intersection is even more expensive
here.
- Once the threshold has been reached (or we have run out of matchers
that do not match the empty string), start with "retrieve metrics
and check them individually against remaining matchers".
A beefy server at SoundCloud was spending 67% of its CPU time in index
lookups (fingerprintsForLabelPairs), serving mostly a dashboard that
is exclusively built with recording rules. With this change, it spends
only 35% in fingerprintsForLabelPairs. The CPU usage dropped from 26
cores to 18 cores. The median latency for query_range dropped from 14s
to 50ms(!). As expected, higher percentile latency didn't improve that
much because the new approach is _occasionally_ running into the worst
case while the old one was _systematically_ doing so. The 99th
percentile latency is now about as high as the median before (14s)
while it was almost twice as high before (26s).
2016-06-28 11:18:32 -07:00
|
|
|
}
|
|
|
|
fps := map[model.Fingerprint]struct{}{}
|
|
|
|
for _, lv := range lvs {
|
|
|
|
s.fingerprintsForLabelPair(
|
|
|
|
model.LabelPair{
|
|
|
|
Name: m.Name,
|
|
|
|
Value: lv,
|
|
|
|
},
|
|
|
|
fps,
|
2016-09-18 03:37:17 -07:00
|
|
|
candidateFPs,
|
storage: improve index lookups
tl;dr: This is not a fundamental solution to the indexing problem
(like tindex is) but it at least avoids utilizing the intersection
problem to the greatest possible amount.
In more detail:
Imagine the following query:
nicely:aggregating:rule{job="foo",env="prod"}
While it uses a nicely aggregating recording rule (which might have a
very low cardinality), Prometheus still intersects the low number of
fingerprints for `{__name__="nicely:aggregating:rule"}` with the many
thousands of fingerprints matching `{job="foo"}` and with the millions
of fingerprints matching `{env="prod"}`. This totally innocuous query
is dead slow if the Prometheus server has a lot of time series with
the `{env="prod"}` label. Ironically, if you make the query more
complicated, it becomes blazingly fast:
nicely:aggregating:rule{job=~"foo",env=~"prod"}
Why so? Because Prometheus only intersects with non-Equal matchers if
there are no Equal matchers. That's good in this case because it
retrieves the few fingerprints for
`{__name__="nicely:aggregating:rule"}` and then starts right ahead to
retrieve the metric for those FPs and checking individually if they
match the other matchers.
This change is generalizing the idea of when to stop intersecting FPs
and go into "retrieve metrics and check them individually against
remaining matchers" mode:
- First, sort all matchers by "expected cardinality". Matchers
matching the empty string are always worst (and never used for
intersections). Equal matchers are in general consider best, but by
using some crude heuristics, we declare some better than others
(instance labels or anything that looks like a recording rule).
- Then go through the matchers until we hit a threshold of remaining
FPs in the intersection. This threshold is higher if we are already
in the non-Equal matcher area as intersection is even more expensive
here.
- Once the threshold has been reached (or we have run out of matchers
that do not match the empty string), start with "retrieve metrics
and check them individually against remaining matchers".
A beefy server at SoundCloud was spending 67% of its CPU time in index
lookups (fingerprintsForLabelPairs), serving mostly a dashboard that
is exclusively built with recording rules. With this change, it spends
only 35% in fingerprintsForLabelPairs. The CPU usage dropped from 26
cores to 18 cores. The median latency for query_range dropped from 14s
to 50ms(!). As expected, higher percentile latency didn't improve that
much because the new approach is _occasionally_ running into the worst
case while the old one was _systematically_ doing so. The 99th
percentile latency is now about as high as the median before (14s)
while it was almost twice as high before (26s).
2016-06-28 11:18:32 -07:00
|
|
|
)
|
|
|
|
}
|
2016-09-18 03:37:17 -07:00
|
|
|
candidateFPs = fps
|
|
|
|
if len(candidateFPs) == 0 {
|
|
|
|
return nil, nil, nil
|
2015-06-15 09:25:31 -07:00
|
|
|
}
|
|
|
|
}
|
2016-09-18 03:37:17 -07:00
|
|
|
return candidateFPs, matchers[matcherIdx:], nil
|
|
|
|
}
|
2015-06-15 09:25:31 -07:00
|
|
|
|
2016-09-18 04:20:46 -07:00
|
|
|
func (s *MemorySeriesStorage) seriesForLabelMatchers(
|
|
|
|
from, through model.Time,
|
|
|
|
matchers ...*metric.LabelMatcher,
|
|
|
|
) ([]fingerprintSeriesPair, error) {
|
|
|
|
candidateFPs, matchersToCheck, err := s.candidateFPsForLabelMatchers(matchers...)
|
|
|
|
if err != nil {
|
|
|
|
return nil, err
|
|
|
|
}
|
|
|
|
|
|
|
|
result := []fingerprintSeriesPair{}
|
|
|
|
FPLoop:
|
|
|
|
for fp := range candidateFPs {
|
2016-03-09 12:56:15 -08:00
|
|
|
s.fpLocker.Lock(fp)
|
2016-09-18 04:20:46 -07:00
|
|
|
series := s.seriesForRange(fp, from, through)
|
|
|
|
s.fpLocker.Unlock(fp)
|
|
|
|
|
|
|
|
if series == nil {
|
|
|
|
continue FPLoop
|
|
|
|
}
|
|
|
|
|
|
|
|
for _, m := range matchersToCheck {
|
|
|
|
if !m.Match(series.metric[m.Name]) {
|
|
|
|
continue FPLoop
|
|
|
|
}
|
2016-03-08 15:09:42 -08:00
|
|
|
}
|
2016-09-18 04:20:46 -07:00
|
|
|
result = append(result, fingerprintSeriesPair{fp, series})
|
|
|
|
}
|
|
|
|
return result, nil
|
|
|
|
}
|
|
|
|
|
|
|
|
func (s *MemorySeriesStorage) fpsForLabelMatchers(
|
|
|
|
from, through model.Time,
|
|
|
|
matchers ...*metric.LabelMatcher,
|
|
|
|
) (map[model.Fingerprint]struct{}, error) {
|
|
|
|
candidateFPs, matchersToCheck, err := s.candidateFPsForLabelMatchers(matchers...)
|
|
|
|
if err != nil {
|
|
|
|
return nil, err
|
|
|
|
}
|
|
|
|
|
|
|
|
FPLoop:
|
|
|
|
for fp := range candidateFPs {
|
|
|
|
s.fpLocker.Lock(fp)
|
|
|
|
met, _, ok := s.metricForRange(fp, from, through)
|
2016-03-09 12:56:15 -08:00
|
|
|
s.fpLocker.Unlock(fp)
|
2016-09-18 04:20:46 -07:00
|
|
|
|
|
|
|
if !ok {
|
|
|
|
delete(candidateFPs, fp)
|
|
|
|
continue FPLoop
|
|
|
|
}
|
|
|
|
|
|
|
|
for _, m := range matchersToCheck {
|
|
|
|
if !m.Match(met[m.Name]) {
|
|
|
|
delete(candidateFPs, fp)
|
|
|
|
continue FPLoop
|
|
|
|
}
|
|
|
|
}
|
2015-06-15 09:25:31 -07:00
|
|
|
}
|
2016-09-18 04:20:46 -07:00
|
|
|
return candidateFPs, nil
|
|
|
|
}
|
|
|
|
|
2016-09-18 03:37:17 -07:00
|
|
|
func (s *MemorySeriesStorage) metricsForLabelMatchers(
|
|
|
|
from, through model.Time,
|
|
|
|
matchers ...*metric.LabelMatcher,
|
|
|
|
) (map[model.Fingerprint]metric.Metric, error) {
|
|
|
|
|
|
|
|
candidateFPs, matchersToCheck, err := s.candidateFPsForLabelMatchers(matchers...)
|
|
|
|
if err != nil {
|
|
|
|
return nil, err
|
|
|
|
}
|
2015-06-15 09:25:31 -07:00
|
|
|
|
2016-03-08 15:09:42 -08:00
|
|
|
result := map[model.Fingerprint]metric.Metric{}
|
2016-09-18 04:20:46 -07:00
|
|
|
FPLoop:
|
2016-09-18 03:37:17 -07:00
|
|
|
for fp := range candidateFPs {
|
2016-03-09 12:56:15 -08:00
|
|
|
s.fpLocker.Lock(fp)
|
2016-09-18 03:03:00 -07:00
|
|
|
met, _, ok := s.metricForRange(fp, from, through)
|
2016-03-09 12:56:15 -08:00
|
|
|
s.fpLocker.Unlock(fp)
|
2016-09-18 03:03:00 -07:00
|
|
|
|
|
|
|
if !ok {
|
2016-09-18 04:20:46 -07:00
|
|
|
continue FPLoop
|
2016-09-18 03:03:00 -07:00
|
|
|
}
|
|
|
|
|
2016-09-18 03:37:17 -07:00
|
|
|
for _, m := range matchersToCheck {
|
2016-09-18 03:03:00 -07:00
|
|
|
if !m.Match(met[m.Name]) {
|
2016-09-18 04:20:46 -07:00
|
|
|
continue FPLoop
|
2015-06-15 09:25:31 -07:00
|
|
|
}
|
|
|
|
}
|
2016-09-18 03:03:00 -07:00
|
|
|
result[fp] = metric.Metric{Metric: met}
|
2014-10-24 11:27:27 -07:00
|
|
|
}
|
2016-07-11 11:27:25 -07:00
|
|
|
return result, nil
|
2014-10-24 11:27:27 -07:00
|
|
|
}
|
|
|
|
|
2016-03-09 12:56:15 -08:00
|
|
|
// metricForRange returns the metric for the given fingerprint if the
|
|
|
|
// corresponding time series has samples between 'from' and 'through', together
|
|
|
|
// with a pointer to the series if it is in memory already. For a series that
|
|
|
|
// does not have samples between 'from' and 'through', the returned bool is
|
|
|
|
// false. For an archived series that does contain samples between 'from' and
|
|
|
|
// 'through', it returns (metric, nil, true).
|
|
|
|
//
|
|
|
|
// The caller must have locked the fp.
|
2016-06-28 23:14:23 -07:00
|
|
|
func (s *MemorySeriesStorage) metricForRange(
|
2016-03-08 15:09:42 -08:00
|
|
|
fp model.Fingerprint,
|
|
|
|
from, through model.Time,
|
2016-03-09 12:56:15 -08:00
|
|
|
) (model.Metric, *memorySeries, bool) {
|
2014-10-27 12:40:48 -07:00
|
|
|
series, ok := s.fpToSeries.get(fp)
|
2014-10-24 11:27:27 -07:00
|
|
|
if ok {
|
2016-09-26 04:06:06 -07:00
|
|
|
if series.lastTime.Before(from) || series.firstTime().After(through) {
|
2016-03-09 12:56:15 -08:00
|
|
|
return nil, nil, false
|
2016-03-08 15:09:42 -08:00
|
|
|
}
|
2016-03-09 12:56:15 -08:00
|
|
|
return series.metric, series, true
|
2016-03-08 15:09:42 -08:00
|
|
|
}
|
|
|
|
// From here on, we are only concerned with archived metrics.
|
|
|
|
// If the high watermark of archived series is before 'from', we are done.
|
2016-03-09 11:27:50 -08:00
|
|
|
watermark := model.Time(atomic.LoadInt64((*int64)(&s.archiveHighWatermark)))
|
2016-03-08 15:09:42 -08:00
|
|
|
if watermark < from {
|
2016-03-09 12:56:15 -08:00
|
|
|
return nil, nil, false
|
2016-03-08 15:09:42 -08:00
|
|
|
}
|
|
|
|
if from.After(model.Earliest) || through.Before(model.Latest) {
|
2016-03-09 11:27:50 -08:00
|
|
|
// The range lookup is relatively cheap, so let's do it first if
|
|
|
|
// we have a chance the archived metric is not in the range.
|
|
|
|
has, first, last := s.persistence.hasArchivedMetric(fp)
|
2016-03-13 03:54:24 -07:00
|
|
|
if !has {
|
|
|
|
s.nonExistentSeriesMatchesCount.Inc()
|
|
|
|
return nil, nil, false
|
|
|
|
}
|
|
|
|
if first.After(through) || last.Before(from) {
|
2016-03-09 12:56:15 -08:00
|
|
|
return nil, nil, false
|
2014-10-24 11:27:27 -07:00
|
|
|
}
|
|
|
|
}
|
2016-03-08 15:09:42 -08:00
|
|
|
|
2016-03-09 12:56:15 -08:00
|
|
|
metric, err := s.persistence.archivedMetric(fp)
|
|
|
|
if err != nil {
|
|
|
|
// archivedMetric has already flagged the storage as dirty in this case.
|
|
|
|
return nil, nil, false
|
2014-10-24 11:27:27 -07:00
|
|
|
}
|
2016-03-09 12:56:15 -08:00
|
|
|
return metric, nil, true
|
2016-03-08 15:09:42 -08:00
|
|
|
}
|
|
|
|
|
|
|
|
// LabelValuesForLabelName implements Storage.
|
2016-09-15 15:58:06 -07:00
|
|
|
func (s *MemorySeriesStorage) LabelValuesForLabelName(_ context.Context, labelName model.LabelName) (model.LabelValues, error) {
|
2016-03-09 09:56:30 -08:00
|
|
|
return s.persistence.labelValuesForLabelName(labelName)
|
2014-06-06 02:55:53 -07:00
|
|
|
}
|
|
|
|
|
2016-07-11 11:27:25 -07:00
|
|
|
// DropMetricsForLabelMatchers implements Storage.
|
2016-09-15 15:58:06 -07:00
|
|
|
func (s *MemorySeriesStorage) DropMetricsForLabelMatchers(_ context.Context, matchers ...*metric.LabelMatcher) (int, error) {
|
2016-09-18 04:20:46 -07:00
|
|
|
fps, err := s.fpsForLabelMatchers(model.Earliest, model.Latest, matchers...)
|
2016-07-11 11:27:25 -07:00
|
|
|
if err != nil {
|
|
|
|
return 0, err
|
|
|
|
}
|
2016-09-18 04:20:46 -07:00
|
|
|
for fp := range fps {
|
Handle errors caused by data corruption more gracefully
This requires all the panic calls upon unexpected data to be converted
into errors returned. This pollute the function signatures quite
lot. Well, this is Go...
The ideas behind this are the following:
- panic only if it's a programming error. Data corruptions happen, and
they are not programming errors.
- If we detect a data corruption, we "quarantine" the series,
essentially removing it from the database and putting its data into
a separate directory for forensics.
- Failure during writing to a series file is not considered corruption
automatically. It will call setDirty, though, so that a
crashrecovery upon the next restart will commence and check for
that.
- Series quarantining and setDirty calls are logged and counted in
metrics, but are hidden from the user of the interfaces in
interface.go, whith the notable exception of Append(). The reasoning
is that we treat corruption by removing the corrupted series, i.e. a
query for it will return no results on its next call anyway, so
return no results right now. In the case of Append(), we want to
tell the user that no data has been appended, though.
Minor side effects:
- Now consistently using filepath.* instead of path.*.
- Introduced structured logging where I touched it. This makes things
less consistent, but a complete change to structured logging would
be out of scope for this PR.
2016-02-25 03:23:42 -08:00
|
|
|
s.purgeSeries(fp, nil, nil)
|
2015-05-27 08:41:57 -07:00
|
|
|
}
|
2016-09-18 04:20:46 -07:00
|
|
|
return len(fps), nil
|
2015-05-27 08:41:57 -07:00
|
|
|
}
|
|
|
|
|
2016-04-25 07:43:52 -07:00
|
|
|
var (
|
|
|
|
// ErrOutOfOrderSample is returned if a sample has a timestamp before the latest
|
|
|
|
// timestamp in the series it is appended to.
|
|
|
|
ErrOutOfOrderSample = fmt.Errorf("sample timestamp out of order")
|
|
|
|
// ErrDuplicateSampleForTimestamp is returned if a sample has the same
|
|
|
|
// timestamp as the latest sample in the series it is appended to but a
|
2016-04-25 16:05:56 -07:00
|
|
|
// different value. (Appending an identical sample is a no-op and does
|
|
|
|
// not cause an error.)
|
2016-04-25 07:43:52 -07:00
|
|
|
ErrDuplicateSampleForTimestamp = fmt.Errorf("sample with repeated timestamp but different value")
|
|
|
|
)
|
2016-02-02 05:01:44 -08:00
|
|
|
|
2015-03-14 19:36:15 -07:00
|
|
|
// Append implements Storage.
|
2016-06-28 23:14:23 -07:00
|
|
|
func (s *MemorySeriesStorage) Append(sample *model.Sample) error {
|
2015-06-22 13:35:19 -07:00
|
|
|
for ln, lv := range sample.Metric {
|
|
|
|
if len(lv) == 0 {
|
|
|
|
delete(sample.Metric, ln)
|
|
|
|
}
|
|
|
|
}
|
2015-05-06 07:53:12 -07:00
|
|
|
rawFP := sample.Metric.FastFingerprint()
|
|
|
|
s.fpLocker.Lock(rawFP)
|
Checkpoint fingerprint mappings only upon shutdown
Before, we checkpointed after every newly detected fingerprint
collision, which is not a problem as long as collisions are
rare. However, with a sufficient number of metrics or particular
nature of the data set, there might be a lot of collisions, all to be
detected upon the first set of scrapes, and then the checkpointing
after each detection will take a quite long time (it's O(n²),
essentially).
Since we are rebuilding the fingerprint mapping during crash recovery,
the previous, very conservative approach didn't even buy us
anything. We only ever read from the checkpoint file after a clean
shutdown, so the only time we need to write the checkpoint file is
during a clean shutdown.
2016-04-14 07:02:37 -07:00
|
|
|
fp := s.mapper.mapFP(rawFP, sample.Metric)
|
Handle errors caused by data corruption more gracefully
This requires all the panic calls upon unexpected data to be converted
into errors returned. This pollute the function signatures quite
lot. Well, this is Go...
The ideas behind this are the following:
- panic only if it's a programming error. Data corruptions happen, and
they are not programming errors.
- If we detect a data corruption, we "quarantine" the series,
essentially removing it from the database and putting its data into
a separate directory for forensics.
- Failure during writing to a series file is not considered corruption
automatically. It will call setDirty, though, so that a
crashrecovery upon the next restart will commence and check for
that.
- Series quarantining and setDirty calls are logged and counted in
metrics, but are hidden from the user of the interfaces in
interface.go, whith the notable exception of Append(). The reasoning
is that we treat corruption by removing the corrupted series, i.e. a
query for it will return no results on its next call anyway, so
return no results right now. In the case of Append(), we want to
tell the user that no data has been appended, though.
Minor side effects:
- Now consistently using filepath.* instead of path.*.
- Introduced structured logging where I touched it. This makes things
less consistent, but a complete change to structured logging would
be out of scope for this PR.
2016-02-25 03:23:42 -08:00
|
|
|
defer func() {
|
|
|
|
s.fpLocker.Unlock(fp)
|
|
|
|
}() // Func wrapper because fp might change below.
|
2015-05-06 07:53:12 -07:00
|
|
|
if fp != rawFP {
|
|
|
|
// Switch locks.
|
|
|
|
s.fpLocker.Unlock(rawFP)
|
|
|
|
s.fpLocker.Lock(fp)
|
|
|
|
}
|
Handle errors caused by data corruption more gracefully
This requires all the panic calls upon unexpected data to be converted
into errors returned. This pollute the function signatures quite
lot. Well, this is Go...
The ideas behind this are the following:
- panic only if it's a programming error. Data corruptions happen, and
they are not programming errors.
- If we detect a data corruption, we "quarantine" the series,
essentially removing it from the database and putting its data into
a separate directory for forensics.
- Failure during writing to a series file is not considered corruption
automatically. It will call setDirty, though, so that a
crashrecovery upon the next restart will commence and check for
that.
- Series quarantining and setDirty calls are logged and counted in
metrics, but are hidden from the user of the interfaces in
interface.go, whith the notable exception of Append(). The reasoning
is that we treat corruption by removing the corrupted series, i.e. a
query for it will return no results on its next call anyway, so
return no results right now. In the case of Append(), we want to
tell the user that no data has been appended, though.
Minor side effects:
- Now consistently using filepath.* instead of path.*.
- Introduced structured logging where I touched it. This makes things
less consistent, but a complete change to structured logging would
be out of scope for this PR.
2016-02-25 03:23:42 -08:00
|
|
|
series, err := s.getOrCreateSeries(fp, sample.Metric)
|
|
|
|
if err != nil {
|
|
|
|
return err // getOrCreateSeries took care of quarantining already.
|
|
|
|
}
|
2015-07-13 12:12:27 -07:00
|
|
|
|
2016-04-25 07:43:52 -07:00
|
|
|
if sample.Timestamp == series.lastTime {
|
2016-02-19 09:16:41 -08:00
|
|
|
// Don't report "no-op appends", i.e. where timestamp and sample
|
|
|
|
// value are the same as for the last append, as they are a
|
|
|
|
// common occurrence when using client-side timestamps
|
|
|
|
// (e.g. Pushgateway or federation).
|
|
|
|
if sample.Timestamp == series.lastTime &&
|
|
|
|
series.lastSampleValueSet &&
|
2016-05-19 09:32:47 -07:00
|
|
|
sample.Value.Equal(series.lastSampleValue) {
|
2016-02-19 09:16:41 -08:00
|
|
|
return nil
|
2015-08-09 13:43:31 -07:00
|
|
|
}
|
2016-04-25 07:43:52 -07:00
|
|
|
s.discardedSamplesCount.WithLabelValues(duplicateSample).Inc()
|
|
|
|
return ErrDuplicateSampleForTimestamp // Caused by the caller.
|
|
|
|
}
|
|
|
|
if sample.Timestamp < series.lastTime {
|
|
|
|
s.discardedSamplesCount.WithLabelValues(outOfOrderTimestamp).Inc()
|
Handle errors caused by data corruption more gracefully
This requires all the panic calls upon unexpected data to be converted
into errors returned. This pollute the function signatures quite
lot. Well, this is Go...
The ideas behind this are the following:
- panic only if it's a programming error. Data corruptions happen, and
they are not programming errors.
- If we detect a data corruption, we "quarantine" the series,
essentially removing it from the database and putting its data into
a separate directory for forensics.
- Failure during writing to a series file is not considered corruption
automatically. It will call setDirty, though, so that a
crashrecovery upon the next restart will commence and check for
that.
- Series quarantining and setDirty calls are logged and counted in
metrics, but are hidden from the user of the interfaces in
interface.go, whith the notable exception of Append(). The reasoning
is that we treat corruption by removing the corrupted series, i.e. a
query for it will return no results on its next call anyway, so
return no results right now. In the case of Append(), we want to
tell the user that no data has been appended, though.
Minor side effects:
- Now consistently using filepath.* instead of path.*.
- Introduced structured logging where I touched it. This makes things
less consistent, but a complete change to structured logging would
be out of scope for this PR.
2016-02-25 03:23:42 -08:00
|
|
|
return ErrOutOfOrderSample // Caused by the caller.
|
2015-07-13 12:12:27 -07:00
|
|
|
}
|
2016-10-02 15:04:56 -07:00
|
|
|
completedChunksCount, err := series.add(model.SamplePair{
|
2014-06-06 02:55:53 -07:00
|
|
|
Value: sample.Value,
|
|
|
|
Timestamp: sample.Timestamp,
|
2014-10-27 12:40:48 -07:00
|
|
|
})
|
Handle errors caused by data corruption more gracefully
This requires all the panic calls upon unexpected data to be converted
into errors returned. This pollute the function signatures quite
lot. Well, this is Go...
The ideas behind this are the following:
- panic only if it's a programming error. Data corruptions happen, and
they are not programming errors.
- If we detect a data corruption, we "quarantine" the series,
essentially removing it from the database and putting its data into
a separate directory for forensics.
- Failure during writing to a series file is not considered corruption
automatically. It will call setDirty, though, so that a
crashrecovery upon the next restart will commence and check for
that.
- Series quarantining and setDirty calls are logged and counted in
metrics, but are hidden from the user of the interfaces in
interface.go, whith the notable exception of Append(). The reasoning
is that we treat corruption by removing the corrupted series, i.e. a
query for it will return no results on its next call anyway, so
return no results right now. In the case of Append(), we want to
tell the user that no data has been appended, though.
Minor side effects:
- Now consistently using filepath.* instead of path.*.
- Introduced structured logging where I touched it. This makes things
less consistent, but a complete change to structured logging would
be out of scope for this PR.
2016-02-25 03:23:42 -08:00
|
|
|
if err != nil {
|
|
|
|
s.quarantineSeries(fp, sample.Metric, err)
|
|
|
|
return err
|
|
|
|
}
|
2015-02-12 08:23:42 -08:00
|
|
|
s.ingestedSamplesCount.Inc()
|
2015-03-18 11:36:41 -07:00
|
|
|
s.incNumChunksToPersist(completedChunksCount)
|
2016-02-02 05:01:44 -08:00
|
|
|
|
|
|
|
return nil
|
2014-06-06 02:55:53 -07:00
|
|
|
}
|
|
|
|
|
2016-01-27 10:07:46 -08:00
|
|
|
// NeedsThrottling implements Storage.
|
2016-06-28 23:14:23 -07:00
|
|
|
func (s *MemorySeriesStorage) NeedsThrottling() bool {
|
2016-01-27 10:07:46 -08:00
|
|
|
if s.getNumChunksToPersist() > s.maxChunksToPersist ||
|
2016-09-21 14:44:27 -07:00
|
|
|
float64(atomic.LoadInt64(&chunk.NumMemChunks)) > float64(s.maxMemoryChunks)*toleranceFactorMemChunks {
|
2016-01-27 10:07:46 -08:00
|
|
|
select {
|
|
|
|
case s.throttled <- struct{}{}:
|
2016-02-09 18:47:00 -08:00
|
|
|
default: // Do nothing, signal already pending.
|
2016-01-27 10:07:46 -08:00
|
|
|
}
|
|
|
|
return true
|
|
|
|
}
|
|
|
|
return false
|
|
|
|
}
|
|
|
|
|
|
|
|
// logThrottling handles logging of throttled events and has to be started as a
|
|
|
|
// goroutine. It stops once s.loopStopping is closed.
|
|
|
|
//
|
|
|
|
// Logging strategy: Whenever Throttle() is called and returns true, an signal
|
|
|
|
// is sent to s.throttled. If that happens for the first time, an Error is
|
|
|
|
// logged that the storage is now throttled. As long as signals continues to be
|
|
|
|
// sent via s.throttled at least once per minute, nothing else is logged. Once
|
|
|
|
// no signal has arrived for a minute, an Info is logged that the storage is not
|
|
|
|
// throttled anymore. This resets things to the initial state, i.e. once a
|
|
|
|
// signal arrives again, the Error will be logged again.
|
2016-06-28 23:14:23 -07:00
|
|
|
func (s *MemorySeriesStorage) logThrottling() {
|
2016-01-27 10:07:46 -08:00
|
|
|
timer := time.NewTimer(time.Minute)
|
|
|
|
timer.Stop()
|
|
|
|
|
2016-02-11 16:46:18 -08:00
|
|
|
// Signal exit of the goroutine. Currently only needed by test code.
|
|
|
|
defer close(s.logThrottlingStopped)
|
|
|
|
|
2016-01-27 10:07:46 -08:00
|
|
|
for {
|
|
|
|
select {
|
|
|
|
case <-s.throttled:
|
|
|
|
if !timer.Reset(time.Minute) {
|
|
|
|
log.
|
|
|
|
With("chunksToPersist", s.getNumChunksToPersist()).
|
|
|
|
With("maxChunksToPersist", s.maxChunksToPersist).
|
2016-09-21 14:44:27 -07:00
|
|
|
With("memoryChunks", atomic.LoadInt64(&chunk.NumMemChunks)).
|
2016-01-27 10:07:46 -08:00
|
|
|
With("maxToleratedMemChunks", int(float64(s.maxMemoryChunks)*toleranceFactorMemChunks)).
|
|
|
|
Error("Storage needs throttling. Scrapes and rule evaluations will be skipped.")
|
|
|
|
}
|
|
|
|
case <-timer.C:
|
|
|
|
log.
|
|
|
|
With("chunksToPersist", s.getNumChunksToPersist()).
|
|
|
|
With("maxChunksToPersist", s.maxChunksToPersist).
|
2016-09-21 14:44:27 -07:00
|
|
|
With("memoryChunks", atomic.LoadInt64(&chunk.NumMemChunks)).
|
2016-01-27 10:07:46 -08:00
|
|
|
With("maxToleratedMemChunks", int(float64(s.maxMemoryChunks)*toleranceFactorMemChunks)).
|
|
|
|
Info("Storage does not need throttling anymore.")
|
|
|
|
case <-s.loopStopping:
|
|
|
|
return
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2016-06-28 23:14:23 -07:00
|
|
|
func (s *MemorySeriesStorage) getOrCreateSeries(fp model.Fingerprint, m model.Metric) (*memorySeries, error) {
|
2014-10-27 12:40:48 -07:00
|
|
|
series, ok := s.fpToSeries.get(fp)
|
2014-06-06 02:55:53 -07:00
|
|
|
if !ok {
|
2016-09-21 14:44:27 -07:00
|
|
|
var cds []*chunk.Desc
|
2015-07-13 12:12:27 -07:00
|
|
|
var modTime time.Time
|
|
|
|
unarchived, err := s.persistence.unarchiveMetric(fp)
|
2014-09-10 09:41:52 -07:00
|
|
|
if err != nil {
|
2015-07-13 12:12:27 -07:00
|
|
|
log.Errorf("Error unarchiving fingerprint %v (metric %v): %v", fp, m, err)
|
Handle errors caused by data corruption more gracefully
This requires all the panic calls upon unexpected data to be converted
into errors returned. This pollute the function signatures quite
lot. Well, this is Go...
The ideas behind this are the following:
- panic only if it's a programming error. Data corruptions happen, and
they are not programming errors.
- If we detect a data corruption, we "quarantine" the series,
essentially removing it from the database and putting its data into
a separate directory for forensics.
- Failure during writing to a series file is not considered corruption
automatically. It will call setDirty, though, so that a
crashrecovery upon the next restart will commence and check for
that.
- Series quarantining and setDirty calls are logged and counted in
metrics, but are hidden from the user of the interfaces in
interface.go, whith the notable exception of Append(). The reasoning
is that we treat corruption by removing the corrupted series, i.e. a
query for it will return no results on its next call anyway, so
return no results right now. In the case of Append(), we want to
tell the user that no data has been appended, though.
Minor side effects:
- Now consistently using filepath.* instead of path.*.
- Introduced structured logging where I touched it. This makes things
less consistent, but a complete change to structured logging would
be out of scope for this PR.
2016-02-25 03:23:42 -08:00
|
|
|
return nil, err
|
2014-09-10 09:41:52 -07:00
|
|
|
}
|
2014-10-22 10:21:23 -07:00
|
|
|
if unarchived {
|
2014-10-23 06:18:32 -07:00
|
|
|
s.seriesOps.WithLabelValues(unarchive).Inc()
|
2016-09-21 14:44:27 -07:00
|
|
|
// We have to load chunk.Descs anyway to do anything with
|
2015-07-13 12:12:27 -07:00
|
|
|
// the series, so let's do it right now so that we don't
|
2016-09-21 14:44:27 -07:00
|
|
|
// end up with a series without any chunk.Descs for a
|
2015-07-13 12:12:27 -07:00
|
|
|
// while (which is confusing as it makes the series
|
|
|
|
// appear as archived or purged).
|
2015-07-06 16:10:14 -07:00
|
|
|
cds, err = s.loadChunkDescs(fp, 0)
|
2015-07-13 12:12:27 -07:00
|
|
|
if err != nil {
|
Handle errors caused by data corruption more gracefully
This requires all the panic calls upon unexpected data to be converted
into errors returned. This pollute the function signatures quite
lot. Well, this is Go...
The ideas behind this are the following:
- panic only if it's a programming error. Data corruptions happen, and
they are not programming errors.
- If we detect a data corruption, we "quarantine" the series,
essentially removing it from the database and putting its data into
a separate directory for forensics.
- Failure during writing to a series file is not considered corruption
automatically. It will call setDirty, though, so that a
crashrecovery upon the next restart will commence and check for
that.
- Series quarantining and setDirty calls are logged and counted in
metrics, but are hidden from the user of the interfaces in
interface.go, whith the notable exception of Append(). The reasoning
is that we treat corruption by removing the corrupted series, i.e. a
query for it will return no results on its next call anyway, so
return no results right now. In the case of Append(), we want to
tell the user that no data has been appended, though.
Minor side effects:
- Now consistently using filepath.* instead of path.*.
- Introduced structured logging where I touched it. This makes things
less consistent, but a complete change to structured logging would
be out of scope for this PR.
2016-02-25 03:23:42 -08:00
|
|
|
s.quarantineSeries(fp, m, err)
|
|
|
|
return nil, err
|
2015-07-13 12:12:27 -07:00
|
|
|
}
|
|
|
|
modTime = s.persistence.seriesFileModTime(fp)
|
2014-10-22 10:21:23 -07:00
|
|
|
} else {
|
2014-09-10 09:41:52 -07:00
|
|
|
// This was a genuinely new series, so index the metric.
|
2014-10-28 11:01:41 -07:00
|
|
|
s.persistence.indexMetric(fp, m)
|
2014-10-23 06:18:32 -07:00
|
|
|
s.seriesOps.WithLabelValues(create).Inc()
|
2014-06-06 02:55:53 -07:00
|
|
|
}
|
Handle errors caused by data corruption more gracefully
This requires all the panic calls upon unexpected data to be converted
into errors returned. This pollute the function signatures quite
lot. Well, this is Go...
The ideas behind this are the following:
- panic only if it's a programming error. Data corruptions happen, and
they are not programming errors.
- If we detect a data corruption, we "quarantine" the series,
essentially removing it from the database and putting its data into
a separate directory for forensics.
- Failure during writing to a series file is not considered corruption
automatically. It will call setDirty, though, so that a
crashrecovery upon the next restart will commence and check for
that.
- Series quarantining and setDirty calls are logged and counted in
metrics, but are hidden from the user of the interfaces in
interface.go, whith the notable exception of Append(). The reasoning
is that we treat corruption by removing the corrupted series, i.e. a
query for it will return no results on its next call anyway, so
return no results right now. In the case of Append(), we want to
tell the user that no data has been appended, though.
Minor side effects:
- Now consistently using filepath.* instead of path.*.
- Introduced structured logging where I touched it. This makes things
less consistent, but a complete change to structured logging would
be out of scope for this PR.
2016-02-25 03:23:42 -08:00
|
|
|
series, err = newMemorySeries(m, cds, modTime)
|
|
|
|
if err != nil {
|
|
|
|
s.quarantineSeries(fp, m, err)
|
|
|
|
return nil, err
|
|
|
|
}
|
2014-10-27 12:40:48 -07:00
|
|
|
s.fpToSeries.put(fp, series)
|
2014-10-23 06:18:32 -07:00
|
|
|
s.numSeries.Inc()
|
2014-06-06 02:55:53 -07:00
|
|
|
}
|
Handle errors caused by data corruption more gracefully
This requires all the panic calls upon unexpected data to be converted
into errors returned. This pollute the function signatures quite
lot. Well, this is Go...
The ideas behind this are the following:
- panic only if it's a programming error. Data corruptions happen, and
they are not programming errors.
- If we detect a data corruption, we "quarantine" the series,
essentially removing it from the database and putting its data into
a separate directory for forensics.
- Failure during writing to a series file is not considered corruption
automatically. It will call setDirty, though, so that a
crashrecovery upon the next restart will commence and check for
that.
- Series quarantining and setDirty calls are logged and counted in
metrics, but are hidden from the user of the interfaces in
interface.go, whith the notable exception of Append(). The reasoning
is that we treat corruption by removing the corrupted series, i.e. a
query for it will return no results on its next call anyway, so
return no results right now. In the case of Append(), we want to
tell the user that no data has been appended, though.
Minor side effects:
- Now consistently using filepath.* instead of path.*.
- Introduced structured logging where I touched it. This makes things
less consistent, but a complete change to structured logging would
be out of scope for this PR.
2016-02-25 03:23:42 -08:00
|
|
|
return series, nil
|
2014-06-06 02:55:53 -07:00
|
|
|
}
|
|
|
|
|
2016-09-18 04:20:46 -07:00
|
|
|
// seriesForRange is a helper method for seriesForLabelMatchers.
|
2016-03-09 11:27:50 -08:00
|
|
|
//
|
|
|
|
// The caller must have locked the fp.
|
2016-06-28 23:14:23 -07:00
|
|
|
func (s *MemorySeriesStorage) seriesForRange(
|
2016-03-09 05:50:09 -08:00
|
|
|
fp model.Fingerprint,
|
|
|
|
from model.Time, through model.Time,
|
2016-03-09 08:13:16 -08:00
|
|
|
) *memorySeries {
|
2016-03-09 12:56:15 -08:00
|
|
|
metric, series, ok := s.metricForRange(fp, from, through)
|
|
|
|
if !ok {
|
2016-03-09 08:13:16 -08:00
|
|
|
return nil
|
|
|
|
}
|
2016-03-09 12:56:15 -08:00
|
|
|
if series == nil {
|
|
|
|
series, _ = s.getOrCreateSeries(fp, metric)
|
|
|
|
// getOrCreateSeries took care of quarantining already, so ignore the error.
|
2016-03-09 05:50:09 -08:00
|
|
|
}
|
2016-03-09 08:13:16 -08:00
|
|
|
return series
|
2016-03-09 05:50:09 -08:00
|
|
|
}
|
|
|
|
|
2016-06-28 23:14:23 -07:00
|
|
|
func (s *MemorySeriesStorage) preloadChunksForRange(
|
2016-09-18 04:20:46 -07:00
|
|
|
pair fingerprintSeriesPair,
|
2015-08-20 08:18:46 -07:00
|
|
|
from model.Time, through model.Time,
|
2016-07-11 11:27:25 -07:00
|
|
|
) SeriesIterator {
|
2016-09-18 04:20:46 -07:00
|
|
|
fp, series := pair.fp, pair.series
|
2016-03-09 08:13:16 -08:00
|
|
|
if series == nil {
|
2016-07-11 11:27:25 -07:00
|
|
|
return nopIter
|
2016-03-09 08:13:16 -08:00
|
|
|
}
|
2016-09-18 04:20:46 -07:00
|
|
|
|
|
|
|
s.fpLocker.Lock(fp)
|
|
|
|
defer s.fpLocker.Unlock(fp)
|
|
|
|
|
2016-07-11 11:27:25 -07:00
|
|
|
iter, err := series.preloadChunksForRange(fp, from, through, s)
|
2016-03-09 08:13:16 -08:00
|
|
|
if err != nil {
|
|
|
|
s.quarantineSeries(fp, series.metric, err)
|
2016-07-11 11:27:25 -07:00
|
|
|
return nopIter
|
2016-03-09 05:50:09 -08:00
|
|
|
}
|
2016-07-11 11:27:25 -07:00
|
|
|
return iter
|
2016-03-09 05:50:09 -08:00
|
|
|
}
|
|
|
|
|
2016-06-28 23:14:23 -07:00
|
|
|
func (s *MemorySeriesStorage) preloadChunksForInstant(
|
2016-09-18 04:20:46 -07:00
|
|
|
pair fingerprintSeriesPair,
|
2016-03-09 05:50:09 -08:00
|
|
|
from model.Time, through model.Time,
|
2016-07-11 11:27:25 -07:00
|
|
|
) SeriesIterator {
|
2016-09-18 04:20:46 -07:00
|
|
|
fp, series := pair.fp, pair.series
|
2016-03-09 08:13:16 -08:00
|
|
|
if series == nil {
|
2016-07-11 11:27:25 -07:00
|
|
|
return nopIter
|
2014-06-06 02:55:53 -07:00
|
|
|
}
|
2016-09-18 04:20:46 -07:00
|
|
|
|
|
|
|
s.fpLocker.Lock(fp)
|
|
|
|
defer s.fpLocker.Unlock(fp)
|
|
|
|
|
2016-07-11 11:27:25 -07:00
|
|
|
iter, err := series.preloadChunksForInstant(fp, from, through, s)
|
Handle errors caused by data corruption more gracefully
This requires all the panic calls upon unexpected data to be converted
into errors returned. This pollute the function signatures quite
lot. Well, this is Go...
The ideas behind this are the following:
- panic only if it's a programming error. Data corruptions happen, and
they are not programming errors.
- If we detect a data corruption, we "quarantine" the series,
essentially removing it from the database and putting its data into
a separate directory for forensics.
- Failure during writing to a series file is not considered corruption
automatically. It will call setDirty, though, so that a
crashrecovery upon the next restart will commence and check for
that.
- Series quarantining and setDirty calls are logged and counted in
metrics, but are hidden from the user of the interfaces in
interface.go, whith the notable exception of Append(). The reasoning
is that we treat corruption by removing the corrupted series, i.e. a
query for it will return no results on its next call anyway, so
return no results right now. In the case of Append(), we want to
tell the user that no data has been appended, though.
Minor side effects:
- Now consistently using filepath.* instead of path.*.
- Introduced structured logging where I touched it. This makes things
less consistent, but a complete change to structured logging would
be out of scope for this PR.
2016-02-25 03:23:42 -08:00
|
|
|
if err != nil {
|
|
|
|
s.quarantineSeries(fp, series.metric, err)
|
2016-07-11 11:27:25 -07:00
|
|
|
return nopIter
|
Handle errors caused by data corruption more gracefully
This requires all the panic calls upon unexpected data to be converted
into errors returned. This pollute the function signatures quite
lot. Well, this is Go...
The ideas behind this are the following:
- panic only if it's a programming error. Data corruptions happen, and
they are not programming errors.
- If we detect a data corruption, we "quarantine" the series,
essentially removing it from the database and putting its data into
a separate directory for forensics.
- Failure during writing to a series file is not considered corruption
automatically. It will call setDirty, though, so that a
crashrecovery upon the next restart will commence and check for
that.
- Series quarantining and setDirty calls are logged and counted in
metrics, but are hidden from the user of the interfaces in
interface.go, whith the notable exception of Append(). The reasoning
is that we treat corruption by removing the corrupted series, i.e. a
query for it will return no results on its next call anyway, so
return no results right now. In the case of Append(), we want to
tell the user that no data has been appended, though.
Minor side effects:
- Now consistently using filepath.* instead of path.*.
- Introduced structured logging where I touched it. This makes things
less consistent, but a complete change to structured logging would
be out of scope for this PR.
2016-02-25 03:23:42 -08:00
|
|
|
}
|
2016-07-11 11:27:25 -07:00
|
|
|
return iter
|
2014-11-13 11:50:25 -08:00
|
|
|
}
|
|
|
|
|
2016-06-28 23:14:23 -07:00
|
|
|
func (s *MemorySeriesStorage) handleEvictList() {
|
2014-11-13 11:50:25 -08:00
|
|
|
ticker := time.NewTicker(maxEvictInterval)
|
|
|
|
count := 0
|
2015-01-07 10:02:38 -08:00
|
|
|
|
2014-11-13 11:50:25 -08:00
|
|
|
for {
|
|
|
|
// To batch up evictions a bit, this tries evictions at least
|
|
|
|
// once per evict interval, but earlier if the number of evict
|
2014-11-20 12:03:51 -08:00
|
|
|
// requests with evict==true that have happened since the last
|
2014-11-13 11:50:25 -08:00
|
|
|
// evict run is more than maxMemoryChunks/1000.
|
|
|
|
select {
|
|
|
|
case req := <-s.evictRequests:
|
2016-09-21 14:44:27 -07:00
|
|
|
if req.Evict {
|
2016-09-28 14:33:34 -07:00
|
|
|
req.Desc.EvictListElement = s.evictList.PushBack(req.Desc)
|
2014-11-13 11:50:25 -08:00
|
|
|
count++
|
|
|
|
if count > s.maxMemoryChunks/1000 {
|
|
|
|
s.maybeEvict()
|
|
|
|
count = 0
|
|
|
|
}
|
|
|
|
} else {
|
2016-09-28 14:33:34 -07:00
|
|
|
if req.Desc.EvictListElement != nil {
|
|
|
|
s.evictList.Remove(req.Desc.EvictListElement)
|
|
|
|
req.Desc.EvictListElement = nil
|
2014-11-13 11:50:25 -08:00
|
|
|
}
|
|
|
|
}
|
|
|
|
case <-ticker.C:
|
|
|
|
if s.evictList.Len() > 0 {
|
|
|
|
s.maybeEvict()
|
|
|
|
}
|
|
|
|
case <-s.evictStopping:
|
2015-01-22 05:42:15 -08:00
|
|
|
// Drain evictRequests forever in a goroutine to not let
|
|
|
|
// requesters hang.
|
|
|
|
go func() {
|
|
|
|
for {
|
|
|
|
<-s.evictRequests
|
2015-01-07 10:02:38 -08:00
|
|
|
}
|
2015-01-22 05:42:15 -08:00
|
|
|
}()
|
|
|
|
ticker.Stop()
|
2015-05-20 09:10:29 -07:00
|
|
|
log.Info("Chunk eviction stopped.")
|
2015-01-22 05:42:15 -08:00
|
|
|
close(s.evictStopped)
|
|
|
|
return
|
2014-11-13 11:50:25 -08:00
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
// maybeEvict is a local helper method. Must only be called by handleEvictList.
|
2016-06-28 23:14:23 -07:00
|
|
|
func (s *MemorySeriesStorage) maybeEvict() {
|
2016-09-21 14:44:27 -07:00
|
|
|
numChunksToEvict := int(atomic.LoadInt64(&chunk.NumMemChunks)) - s.maxMemoryChunks
|
2014-11-13 11:50:25 -08:00
|
|
|
if numChunksToEvict <= 0 {
|
|
|
|
return
|
|
|
|
}
|
2016-09-21 14:44:27 -07:00
|
|
|
chunkDescsToEvict := make([]*chunk.Desc, numChunksToEvict)
|
2014-11-13 11:50:25 -08:00
|
|
|
for i := range chunkDescsToEvict {
|
|
|
|
e := s.evictList.Front()
|
|
|
|
if e == nil {
|
|
|
|
break
|
|
|
|
}
|
2016-09-21 14:44:27 -07:00
|
|
|
cd := e.Value.(*chunk.Desc)
|
|
|
|
cd.EvictListElement = nil
|
2014-11-13 11:50:25 -08:00
|
|
|
chunkDescsToEvict[i] = cd
|
|
|
|
s.evictList.Remove(e)
|
|
|
|
}
|
|
|
|
// Do the actual eviction in a goroutine as we might otherwise deadlock,
|
2016-09-21 14:44:27 -07:00
|
|
|
// in the following way: A chunk was Unpinned completely and therefore
|
2014-11-13 11:50:25 -08:00
|
|
|
// scheduled for eviction. At the time we actually try to evict it,
|
|
|
|
// another goroutine is pinning the chunk. The pinning goroutine has
|
|
|
|
// currently locked the chunk and tries to send the evict request (to
|
|
|
|
// remove the chunk from the evict list) to the evictRequests
|
|
|
|
// channel. The send blocks because evictRequests is full. However, the
|
2014-11-20 12:03:51 -08:00
|
|
|
// goroutine that is supposed to empty the channel is waiting for the
|
2016-09-21 14:44:27 -07:00
|
|
|
// Chunk.Desc lock to try to evict the chunk.
|
2014-11-13 11:50:25 -08:00
|
|
|
go func() {
|
|
|
|
for _, cd := range chunkDescsToEvict {
|
|
|
|
if cd == nil {
|
|
|
|
break
|
|
|
|
}
|
2016-09-21 14:44:27 -07:00
|
|
|
cd.MaybeEvict()
|
2014-11-13 11:50:25 -08:00
|
|
|
// We don't care if the eviction succeeds. If the chunk
|
|
|
|
// was pinned in the meantime, it will be added to the
|
2016-09-21 14:44:27 -07:00
|
|
|
// evict list once it gets Unpinned again.
|
2014-11-13 11:50:25 -08:00
|
|
|
}
|
|
|
|
}()
|
2014-06-06 02:55:53 -07:00
|
|
|
}
|
|
|
|
|
2014-11-10 13:26:07 -08:00
|
|
|
// waitForNextFP waits an estimated duration, after which we want to process
|
|
|
|
// another fingerprint so that we will process all fingerprints in a tenth of
|
2015-02-26 06:19:44 -08:00
|
|
|
// s.dropAfter assuming that the system is doing nothing else, e.g. if we want
|
|
|
|
// to drop chunks after 40h, we want to cycle through all fingerprints within
|
2015-04-01 08:52:03 -07:00
|
|
|
// 4h. The estimation is based on the total number of fingerprints as passed
|
|
|
|
// in. However, the maximum sweep time is capped at fpMaxSweepTime. Also, the
|
|
|
|
// method will never wait for longer than fpMaxWaitDuration.
|
|
|
|
//
|
|
|
|
// The maxWaitDurationFactor can be used to reduce the waiting time if a faster
|
|
|
|
// processing is required (for example because unpersisted chunks pile up too
|
|
|
|
// much).
|
|
|
|
//
|
|
|
|
// Normally, the method returns true once the wait duration has passed. However,
|
|
|
|
// if s.loopStopped is closed, it will return false immediately.
|
2016-06-28 23:14:23 -07:00
|
|
|
func (s *MemorySeriesStorage) waitForNextFP(numberOfFPs int, maxWaitDurationFactor float64) bool {
|
2014-11-13 07:55:15 -08:00
|
|
|
d := fpMaxWaitDuration
|
2014-11-10 13:26:07 -08:00
|
|
|
if numberOfFPs != 0 {
|
2015-02-26 06:19:44 -08:00
|
|
|
sweepTime := s.dropAfter / 10
|
2014-11-13 07:55:15 -08:00
|
|
|
if sweepTime > fpMaxSweepTime {
|
|
|
|
sweepTime = fpMaxSweepTime
|
|
|
|
}
|
2015-04-01 08:52:03 -07:00
|
|
|
calculatedWait := time.Duration(float64(sweepTime) / float64(numberOfFPs) * maxWaitDurationFactor)
|
|
|
|
if calculatedWait < d {
|
|
|
|
d = calculatedWait
|
2014-11-10 13:26:07 -08:00
|
|
|
}
|
|
|
|
}
|
2015-04-01 08:52:03 -07:00
|
|
|
if d == 0 {
|
|
|
|
return true
|
|
|
|
}
|
2014-11-10 13:26:07 -08:00
|
|
|
t := time.NewTimer(d)
|
|
|
|
select {
|
|
|
|
case <-t.C:
|
|
|
|
return true
|
|
|
|
case <-s.loopStopping:
|
|
|
|
return false
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2014-11-20 12:03:51 -08:00
|
|
|
// cycleThroughMemoryFingerprints returns a channel that emits fingerprints for
|
|
|
|
// series in memory in a throttled fashion. It continues to cycle through all
|
|
|
|
// fingerprints in memory until s.loopStopping is closed.
|
2016-06-28 23:14:23 -07:00
|
|
|
func (s *MemorySeriesStorage) cycleThroughMemoryFingerprints() chan model.Fingerprint {
|
2015-08-20 08:18:46 -07:00
|
|
|
memoryFingerprints := make(chan model.Fingerprint)
|
2014-11-10 13:26:07 -08:00
|
|
|
go func() {
|
2015-08-20 08:18:46 -07:00
|
|
|
var fpIter <-chan model.Fingerprint
|
2014-11-10 13:26:07 -08:00
|
|
|
|
|
|
|
defer func() {
|
|
|
|
if fpIter != nil {
|
2014-12-26 04:37:30 -08:00
|
|
|
for range fpIter {
|
2014-11-10 13:26:07 -08:00
|
|
|
// Consume the iterator.
|
|
|
|
}
|
|
|
|
}
|
|
|
|
close(memoryFingerprints)
|
|
|
|
}()
|
|
|
|
|
|
|
|
for {
|
|
|
|
// Initial wait, also important if there are no FPs yet.
|
2015-04-01 08:52:03 -07:00
|
|
|
if !s.waitForNextFP(s.fpToSeries.length(), 1) {
|
2014-11-10 13:26:07 -08:00
|
|
|
return
|
|
|
|
}
|
2014-11-20 12:03:51 -08:00
|
|
|
begin := time.Now()
|
2014-11-10 13:26:07 -08:00
|
|
|
fpIter = s.fpToSeries.fpIter()
|
2015-02-26 06:19:44 -08:00
|
|
|
count := 0
|
2014-11-10 13:26:07 -08:00
|
|
|
for fp := range fpIter {
|
|
|
|
select {
|
|
|
|
case memoryFingerprints <- fp:
|
|
|
|
case <-s.loopStopping:
|
|
|
|
return
|
|
|
|
}
|
2016-01-26 08:29:33 -08:00
|
|
|
// Reduce the wait time according to the urgency score.
|
|
|
|
s.waitForNextFP(s.fpToSeries.length(), 1-s.calculatePersistenceUrgencyScore())
|
2015-02-26 06:19:44 -08:00
|
|
|
count++
|
|
|
|
}
|
|
|
|
if count > 0 {
|
2015-05-20 09:10:29 -07:00
|
|
|
log.Infof(
|
2015-02-26 06:19:44 -08:00
|
|
|
"Completed maintenance sweep through %d in-memory fingerprints in %v.",
|
|
|
|
count, time.Since(begin),
|
|
|
|
)
|
2014-11-10 13:26:07 -08:00
|
|
|
}
|
|
|
|
}
|
|
|
|
}()
|
|
|
|
|
2014-11-20 12:03:51 -08:00
|
|
|
return memoryFingerprints
|
|
|
|
}
|
|
|
|
|
|
|
|
// cycleThroughArchivedFingerprints returns a channel that emits fingerprints
|
|
|
|
// for archived series in a throttled fashion. It continues to cycle through all
|
|
|
|
// archived fingerprints until s.loopStopping is closed.
|
2016-06-28 23:14:23 -07:00
|
|
|
func (s *MemorySeriesStorage) cycleThroughArchivedFingerprints() chan model.Fingerprint {
|
2015-08-20 08:18:46 -07:00
|
|
|
archivedFingerprints := make(chan model.Fingerprint)
|
2014-11-10 13:26:07 -08:00
|
|
|
go func() {
|
|
|
|
defer close(archivedFingerprints)
|
|
|
|
|
|
|
|
for {
|
2015-05-20 10:13:06 -07:00
|
|
|
archivedFPs, err := s.persistence.fingerprintsModifiedBefore(
|
2015-08-20 08:18:46 -07:00
|
|
|
model.Now().Add(-s.dropAfter),
|
2014-11-10 13:26:07 -08:00
|
|
|
)
|
|
|
|
if err != nil {
|
2015-05-20 09:10:29 -07:00
|
|
|
log.Error("Failed to lookup archived fingerprint ranges: ", err)
|
2015-04-01 08:52:03 -07:00
|
|
|
s.waitForNextFP(0, 1)
|
2014-11-10 13:26:07 -08:00
|
|
|
continue
|
|
|
|
}
|
|
|
|
// Initial wait, also important if there are no FPs yet.
|
2015-04-01 08:52:03 -07:00
|
|
|
if !s.waitForNextFP(len(archivedFPs), 1) {
|
2014-11-10 13:26:07 -08:00
|
|
|
return
|
|
|
|
}
|
2014-11-20 12:03:51 -08:00
|
|
|
begin := time.Now()
|
2014-11-10 13:26:07 -08:00
|
|
|
for _, fp := range archivedFPs {
|
|
|
|
select {
|
|
|
|
case archivedFingerprints <- fp:
|
|
|
|
case <-s.loopStopping:
|
|
|
|
return
|
|
|
|
}
|
2015-04-01 08:52:03 -07:00
|
|
|
// Never speed up maintenance of archived FPs.
|
|
|
|
s.waitForNextFP(len(archivedFPs), 1)
|
2014-11-10 13:26:07 -08:00
|
|
|
}
|
2015-02-26 06:19:44 -08:00
|
|
|
if len(archivedFPs) > 0 {
|
2015-05-20 09:10:29 -07:00
|
|
|
log.Infof(
|
2015-02-26 06:19:44 -08:00
|
|
|
"Completed maintenance sweep through %d archived fingerprints in %v.",
|
|
|
|
len(archivedFPs), time.Since(begin),
|
|
|
|
)
|
|
|
|
}
|
2014-11-10 13:26:07 -08:00
|
|
|
}
|
|
|
|
}()
|
2014-11-20 12:03:51 -08:00
|
|
|
return archivedFingerprints
|
|
|
|
}
|
|
|
|
|
2016-06-28 23:14:23 -07:00
|
|
|
func (s *MemorySeriesStorage) loop() {
|
2015-01-08 11:15:58 -08:00
|
|
|
checkpointTimer := time.NewTimer(s.checkpointInterval)
|
|
|
|
|
2015-03-08 18:33:10 -07:00
|
|
|
dirtySeriesCount := 0
|
2014-11-20 12:03:51 -08:00
|
|
|
|
|
|
|
defer func() {
|
2015-01-08 11:15:58 -08:00
|
|
|
checkpointTimer.Stop()
|
2015-05-20 09:10:29 -07:00
|
|
|
log.Info("Maintenance loop stopped.")
|
2014-11-20 12:03:51 -08:00
|
|
|
close(s.loopStopped)
|
|
|
|
}()
|
|
|
|
|
|
|
|
memoryFingerprints := s.cycleThroughMemoryFingerprints()
|
|
|
|
archivedFingerprints := s.cycleThroughArchivedFingerprints()
|
2014-11-10 13:26:07 -08:00
|
|
|
|
|
|
|
loop:
|
2014-06-06 02:55:53 -07:00
|
|
|
for {
|
|
|
|
select {
|
2014-10-24 11:27:27 -07:00
|
|
|
case <-s.loopStopping:
|
2014-11-10 13:26:07 -08:00
|
|
|
break loop
|
2015-01-08 11:15:58 -08:00
|
|
|
case <-checkpointTimer.C:
|
2015-09-07 09:08:23 -07:00
|
|
|
err := s.persistence.checkpointSeriesMapAndHeads(s.fpToSeries, s.fpLocker)
|
|
|
|
if err != nil {
|
|
|
|
log.Errorln("Error while checkpointing:", err)
|
|
|
|
} else {
|
|
|
|
dirtySeriesCount = 0
|
|
|
|
}
|
2016-02-23 06:30:56 -08:00
|
|
|
// If a checkpoint takes longer than checkpointInterval, unluckily timed
|
|
|
|
// combination with the Reset(0) call below can lead to a case where a
|
|
|
|
// time is lurking in C leading to repeated checkpointing without break.
|
|
|
|
select {
|
|
|
|
case <-checkpointTimer.C: // Get rid of the lurking time.
|
|
|
|
default:
|
|
|
|
}
|
2015-01-08 11:15:58 -08:00
|
|
|
checkpointTimer.Reset(s.checkpointInterval)
|
2014-11-10 13:26:07 -08:00
|
|
|
case fp := <-memoryFingerprints:
|
2015-08-20 08:18:46 -07:00
|
|
|
if s.maintainMemorySeries(fp, model.Now().Add(-s.dropAfter)) {
|
2015-03-08 18:33:10 -07:00
|
|
|
dirtySeriesCount++
|
2015-03-19 07:41:50 -07:00
|
|
|
// Check if we have enough "dirty" series so that we need an early checkpoint.
|
|
|
|
// However, if we are already behind persisting chunks, creating a checkpoint
|
|
|
|
// would be counterproductive, as it would slow down chunk persisting even more,
|
|
|
|
// while in a situation like that, where we are clearly lacking speed of disk
|
|
|
|
// maintenance, the best we can do for crash recovery is to persist chunks as
|
2016-01-26 08:29:33 -08:00
|
|
|
// quickly as possible. So only checkpoint if the urgency score is < 1.
|
|
|
|
if dirtySeriesCount >= s.checkpointDirtySeriesLimit &&
|
|
|
|
s.calculatePersistenceUrgencyScore() < 1 {
|
2015-03-08 18:33:10 -07:00
|
|
|
checkpointTimer.Reset(0)
|
|
|
|
}
|
|
|
|
}
|
2014-11-10 13:26:07 -08:00
|
|
|
case fp := <-archivedFingerprints:
|
2015-08-20 08:18:46 -07:00
|
|
|
s.maintainArchivedSeries(fp, model.Now().Add(-s.dropAfter))
|
2014-06-06 02:55:53 -07:00
|
|
|
}
|
|
|
|
}
|
2014-11-10 13:26:07 -08:00
|
|
|
// Wait until both channels are closed.
|
2014-12-26 04:37:30 -08:00
|
|
|
for range memoryFingerprints {
|
2014-11-10 13:26:07 -08:00
|
|
|
}
|
2014-12-26 04:37:30 -08:00
|
|
|
for range archivedFingerprints {
|
2014-11-10 13:26:07 -08:00
|
|
|
}
|
2014-06-06 02:55:53 -07:00
|
|
|
}
|
|
|
|
|
2015-03-08 18:33:10 -07:00
|
|
|
// maintainMemorySeries maintains a series that is in memory (i.e. not
|
|
|
|
// archived). It returns true if the method has changed from clean to dirty
|
|
|
|
// (i.e. it is inconsistent with the latest checkpoint now so that in case of a
|
|
|
|
// crash a recovery operation that requires a disk seek needed to be applied).
|
|
|
|
//
|
|
|
|
// The method first closes the head chunk if it was not touched for the duration
|
|
|
|
// of headChunkTimeout.
|
|
|
|
//
|
|
|
|
// Then it determines the chunks that need to be purged and the chunks that need
|
|
|
|
// to be persisted. Depending on the result, it does the following:
|
|
|
|
//
|
|
|
|
// - If all chunks of a series need to be purged, the whole series is deleted
|
|
|
|
// for good and the method returns false. (Detecting non-existence of a series
|
|
|
|
// file does not require a disk seek.)
|
|
|
|
//
|
|
|
|
// - If any chunks need to be purged (but not all of them), it purges those
|
|
|
|
// chunks from memory and rewrites the series file on disk, leaving out the
|
|
|
|
// purged chunks and appending all chunks not yet persisted (with the exception
|
|
|
|
// of a still open head chunk).
|
|
|
|
//
|
|
|
|
// - If no chunks on disk need to be purged, but chunks need to be persisted,
|
|
|
|
// those chunks are simply appended to the existing series file (or the file is
|
|
|
|
// created if it does not exist yet).
|
|
|
|
//
|
|
|
|
// - If no chunks need to be purged and no chunks need to be persisted, nothing
|
|
|
|
// happens in this step.
|
|
|
|
//
|
|
|
|
// Next, the method checks if all chunks in the series are evicted. In that
|
|
|
|
// case, it archives the series and returns true.
|
|
|
|
//
|
2016-09-21 14:44:27 -07:00
|
|
|
// Finally, it evicts chunk.Descs if there are too many.
|
2016-06-28 23:14:23 -07:00
|
|
|
func (s *MemorySeriesStorage) maintainMemorySeries(
|
2015-08-20 08:18:46 -07:00
|
|
|
fp model.Fingerprint, beforeTime model.Time,
|
2015-03-08 18:33:10 -07:00
|
|
|
) (becameDirty bool) {
|
2015-03-19 09:06:16 -07:00
|
|
|
defer func(begin time.Time) {
|
|
|
|
s.maintainSeriesDuration.WithLabelValues(maintainInMemory).Observe(
|
2016-07-07 06:24:35 -07:00
|
|
|
time.Since(begin).Seconds(),
|
2015-03-19 09:06:16 -07:00
|
|
|
)
|
|
|
|
}(time.Now())
|
|
|
|
|
2014-11-13 11:50:25 -08:00
|
|
|
s.fpLocker.Lock(fp)
|
2015-03-08 18:33:10 -07:00
|
|
|
defer s.fpLocker.Unlock(fp)
|
2014-11-13 11:50:25 -08:00
|
|
|
|
|
|
|
series, ok := s.fpToSeries.get(fp)
|
|
|
|
if !ok {
|
2015-02-26 06:19:44 -08:00
|
|
|
// Series is actually not in memory, perhaps archived or dropped in the meantime.
|
2015-03-08 18:33:10 -07:00
|
|
|
return false
|
2015-02-26 06:19:44 -08:00
|
|
|
}
|
|
|
|
|
|
|
|
defer s.seriesOps.WithLabelValues(memoryMaintenance).Inc()
|
|
|
|
|
2015-03-08 18:33:10 -07:00
|
|
|
if series.maybeCloseHeadChunk() {
|
2015-03-18 11:36:41 -07:00
|
|
|
s.incNumChunksToPersist(1)
|
2015-03-08 18:33:10 -07:00
|
|
|
}
|
|
|
|
|
|
|
|
seriesWasDirty := series.dirty
|
|
|
|
|
|
|
|
if s.writeMemorySeries(fp, series, beforeTime) {
|
2015-02-26 06:19:44 -08:00
|
|
|
// Series is gone now, we are done.
|
2015-03-08 18:33:10 -07:00
|
|
|
return false
|
2014-11-13 11:50:25 -08:00
|
|
|
}
|
2015-02-26 06:19:44 -08:00
|
|
|
|
2014-11-13 11:50:25 -08:00
|
|
|
iOldestNotEvicted := -1
|
|
|
|
for i, cd := range series.chunkDescs {
|
2016-09-21 14:44:27 -07:00
|
|
|
if !cd.IsEvicted() {
|
2014-11-13 11:50:25 -08:00
|
|
|
iOldestNotEvicted = i
|
|
|
|
break
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2016-03-08 15:09:42 -08:00
|
|
|
// Archive if all chunks are evicted. Also make sure the last sample has
|
|
|
|
// an age of at least headChunkTimeout (which is very likely anyway).
|
|
|
|
if iOldestNotEvicted == -1 && model.Now().Sub(series.lastTime) > headChunkTimeout {
|
2014-11-13 11:50:25 -08:00
|
|
|
s.fpToSeries.del(fp)
|
|
|
|
s.numSeries.Dec()
|
2016-09-26 04:06:06 -07:00
|
|
|
s.persistence.archiveMetric(fp, series.metric, series.firstTime(), series.lastTime)
|
Fix a bug handling freshly unarchived series.
Usually, if you unarchive a series, it is to add something to it,
which will create a new head chunk. However, if a series in
unarchived, and before anything is added to it, it is handled by the
maintenance loop, it will be archived again. In that case, we have to
load the chunkDescs to know the lastTime of the series to be
archived. Usually, this case will happen only rarely (as a race, has
never happened so far, possibly because the locking around unarchiving
and the subsequent sample append is smart enough). However, during
crash recovery, we sometimes treat series as "freshly unarchived"
without directly appending a sample. We might add more cases of that
type later, so better deal with archiving properly and load chunkDescs
if required.
2015-01-08 07:10:31 -08:00
|
|
|
s.seriesOps.WithLabelValues(archive).Inc()
|
2016-03-08 15:09:42 -08:00
|
|
|
oldWatermark := atomic.LoadInt64((*int64)(&s.archiveHighWatermark))
|
|
|
|
if oldWatermark < int64(series.lastTime) {
|
|
|
|
if !atomic.CompareAndSwapInt64(
|
|
|
|
(*int64)(&s.archiveHighWatermark),
|
|
|
|
oldWatermark, int64(series.lastTime),
|
|
|
|
) {
|
|
|
|
panic("s.archiveHighWatermark modified outside of maintainMemorySeries")
|
|
|
|
}
|
|
|
|
}
|
2014-11-13 11:50:25 -08:00
|
|
|
return
|
|
|
|
}
|
2016-09-21 14:44:27 -07:00
|
|
|
// If we are here, the series is not archived, so check for Chunk.Desc
|
2015-07-15 10:53:15 -07:00
|
|
|
// eviction next.
|
2014-11-13 11:50:25 -08:00
|
|
|
series.evictChunkDescs(iOldestNotEvicted)
|
2015-03-08 18:33:10 -07:00
|
|
|
|
|
|
|
return series.dirty && !seriesWasDirty
|
2014-11-13 11:50:25 -08:00
|
|
|
}
|
|
|
|
|
2015-03-08 18:33:10 -07:00
|
|
|
// writeMemorySeries (re-)writes a memory series file. While doing so, it drops
|
2015-03-18 11:09:07 -07:00
|
|
|
// chunks older than beforeTime from both the series file (if it exists) as well
|
|
|
|
// as from memory. The provided chunksToPersist are appended to the newly
|
2015-03-08 18:33:10 -07:00
|
|
|
// written series file. If no chunks need to be purged, but chunksToPersist is
|
|
|
|
// not empty, those chunks are simply appended to the series file. If the series
|
|
|
|
// contains no chunks after dropping old chunks, it is purged entirely. In that
|
|
|
|
// case, the method returns true.
|
|
|
|
//
|
|
|
|
// The caller must have locked the fp.
|
2016-06-28 23:14:23 -07:00
|
|
|
func (s *MemorySeriesStorage) writeMemorySeries(
|
2015-08-20 08:18:46 -07:00
|
|
|
fp model.Fingerprint, series *memorySeries, beforeTime model.Time,
|
2015-03-08 18:33:10 -07:00
|
|
|
) bool {
|
2016-03-03 04:15:02 -08:00
|
|
|
var (
|
|
|
|
persistErr error
|
|
|
|
cds = series.chunksToPersist()
|
|
|
|
)
|
|
|
|
|
2015-03-08 18:33:10 -07:00
|
|
|
defer func() {
|
2016-03-03 04:15:02 -08:00
|
|
|
if persistErr != nil {
|
|
|
|
s.quarantineSeries(fp, series.metric, persistErr)
|
|
|
|
s.persistErrors.Inc()
|
|
|
|
}
|
|
|
|
// The following is done even in case of an error to ensure
|
|
|
|
// correct counter bookkeeping and to not pin chunks in memory
|
|
|
|
// that belong to a series that is scheduled for quarantine
|
|
|
|
// anyway.
|
2015-03-08 18:33:10 -07:00
|
|
|
for _, cd := range cds {
|
2016-09-21 14:44:27 -07:00
|
|
|
cd.Unpin(s.evictRequests)
|
2015-03-08 18:33:10 -07:00
|
|
|
}
|
2015-03-18 11:36:41 -07:00
|
|
|
s.incNumChunksToPersist(-len(cds))
|
2016-09-28 14:33:34 -07:00
|
|
|
chunk.Ops.WithLabelValues(chunk.PersistAndUnpin).Add(float64(len(cds)))
|
2015-05-20 10:13:06 -07:00
|
|
|
series.modTime = s.persistence.seriesFileModTime(fp)
|
2015-03-08 18:33:10 -07:00
|
|
|
}()
|
|
|
|
|
2016-09-21 14:44:27 -07:00
|
|
|
// Get the actual chunks from underneath the chunk.Descs.
|
2015-05-20 10:13:06 -07:00
|
|
|
// No lock required as chunks still to persist cannot be evicted.
|
2016-09-21 14:44:27 -07:00
|
|
|
chunks := make([]chunk.Chunk, len(cds))
|
2015-03-08 18:33:10 -07:00
|
|
|
for i, cd := range cds {
|
2016-09-21 14:44:27 -07:00
|
|
|
chunks[i] = cd.C
|
2015-03-08 18:33:10 -07:00
|
|
|
}
|
|
|
|
|
2016-09-26 04:06:06 -07:00
|
|
|
if !series.firstTime().Before(beforeTime) {
|
2015-03-08 18:33:10 -07:00
|
|
|
// Oldest sample not old enough, just append chunks, if any.
|
|
|
|
if len(cds) == 0 {
|
|
|
|
return false
|
|
|
|
}
|
2016-03-03 04:15:02 -08:00
|
|
|
var offset int
|
|
|
|
offset, persistErr = s.persistence.persistChunks(fp, chunks)
|
|
|
|
if persistErr != nil {
|
2015-03-08 18:33:10 -07:00
|
|
|
return false
|
|
|
|
}
|
|
|
|
if series.chunkDescsOffset == -1 {
|
|
|
|
// This is the first chunk persisted for a newly created
|
|
|
|
// series that had prior chunks on disk. Finally, we can
|
|
|
|
// set the chunkDescsOffset.
|
|
|
|
series.chunkDescsOffset = offset
|
|
|
|
}
|
2015-02-26 06:19:44 -08:00
|
|
|
return false
|
|
|
|
}
|
2015-03-08 18:33:10 -07:00
|
|
|
|
2016-03-03 04:15:02 -08:00
|
|
|
newFirstTime, offset, numDroppedFromPersistence, allDroppedFromPersistence, persistErr :=
|
2015-03-08 18:33:10 -07:00
|
|
|
s.persistence.dropAndPersistChunks(fp, beforeTime, chunks)
|
2016-03-03 04:15:02 -08:00
|
|
|
if persistErr != nil {
|
2015-03-08 18:33:10 -07:00
|
|
|
return false
|
2015-02-26 06:19:44 -08:00
|
|
|
}
|
2016-03-03 04:15:02 -08:00
|
|
|
if persistErr = series.dropChunks(beforeTime); persistErr != nil {
|
Handle errors caused by data corruption more gracefully
This requires all the panic calls upon unexpected data to be converted
into errors returned. This pollute the function signatures quite
lot. Well, this is Go...
The ideas behind this are the following:
- panic only if it's a programming error. Data corruptions happen, and
they are not programming errors.
- If we detect a data corruption, we "quarantine" the series,
essentially removing it from the database and putting its data into
a separate directory for forensics.
- Failure during writing to a series file is not considered corruption
automatically. It will call setDirty, though, so that a
crashrecovery upon the next restart will commence and check for
that.
- Series quarantining and setDirty calls are logged and counted in
metrics, but are hidden from the user of the interfaces in
interface.go, whith the notable exception of Append(). The reasoning
is that we treat corruption by removing the corrupted series, i.e. a
query for it will return no results on its next call anyway, so
return no results right now. In the case of Append(), we want to
tell the user that no data has been appended, though.
Minor side effects:
- Now consistently using filepath.* instead of path.*.
- Introduced structured logging where I touched it. This makes things
less consistent, but a complete change to structured logging would
be out of scope for this PR.
2016-02-25 03:23:42 -08:00
|
|
|
return false
|
|
|
|
}
|
2015-04-09 06:57:11 -07:00
|
|
|
if len(series.chunkDescs) == 0 && allDroppedFromPersistence {
|
|
|
|
// All chunks dropped from both memory and persistence. Delete the series for good.
|
2015-02-26 06:19:44 -08:00
|
|
|
s.fpToSeries.del(fp)
|
|
|
|
s.numSeries.Dec()
|
|
|
|
s.seriesOps.WithLabelValues(memoryPurge).Inc()
|
|
|
|
s.persistence.unindexMetric(fp, series.metric)
|
|
|
|
return true
|
|
|
|
}
|
2015-03-08 18:33:10 -07:00
|
|
|
series.savedFirstTime = newFirstTime
|
|
|
|
if series.chunkDescsOffset == -1 {
|
|
|
|
series.chunkDescsOffset = offset
|
|
|
|
} else {
|
|
|
|
series.chunkDescsOffset -= numDroppedFromPersistence
|
2015-02-26 06:19:44 -08:00
|
|
|
if series.chunkDescsOffset < 0 {
|
2016-03-03 04:15:02 -08:00
|
|
|
persistErr = errors.New("dropped more chunks from persistence than from memory")
|
|
|
|
series.chunkDescsOffset = -1
|
2015-02-26 06:19:44 -08:00
|
|
|
}
|
|
|
|
}
|
|
|
|
return false
|
|
|
|
}
|
|
|
|
|
|
|
|
// maintainArchivedSeries drops chunks older than beforeTime from an archived
|
|
|
|
// series. If the series contains no chunks after that, it is purged entirely.
|
2016-06-28 23:14:23 -07:00
|
|
|
func (s *MemorySeriesStorage) maintainArchivedSeries(fp model.Fingerprint, beforeTime model.Time) {
|
2015-03-19 09:06:16 -07:00
|
|
|
defer func(begin time.Time) {
|
|
|
|
s.maintainSeriesDuration.WithLabelValues(maintainArchived).Observe(
|
2016-07-07 06:24:35 -07:00
|
|
|
time.Since(begin).Seconds(),
|
2015-03-19 09:06:16 -07:00
|
|
|
)
|
|
|
|
}(time.Now())
|
|
|
|
|
2014-10-07 10:11:24 -07:00
|
|
|
s.fpLocker.Lock(fp)
|
|
|
|
defer s.fpLocker.Unlock(fp)
|
2014-09-10 09:41:52 -07:00
|
|
|
|
2016-03-09 09:56:30 -08:00
|
|
|
has, firstTime, lastTime := s.persistence.hasArchivedMetric(fp)
|
2014-11-10 09:22:08 -08:00
|
|
|
if !has || !firstTime.Before(beforeTime) {
|
|
|
|
// Oldest sample not old enough, or metric purged or unarchived in the meantime.
|
|
|
|
return
|
|
|
|
}
|
|
|
|
|
2015-02-26 06:19:44 -08:00
|
|
|
defer s.seriesOps.WithLabelValues(archiveMaintenance).Inc()
|
|
|
|
|
2015-03-08 18:33:10 -07:00
|
|
|
newFirstTime, _, _, allDropped, err := s.persistence.dropAndPersistChunks(fp, beforeTime, nil)
|
2014-11-10 09:22:08 -08:00
|
|
|
if err != nil {
|
2015-05-20 09:10:29 -07:00
|
|
|
log.Error("Error dropping persisted chunks: ", err)
|
2014-11-10 09:22:08 -08:00
|
|
|
}
|
2014-10-15 06:53:05 -07:00
|
|
|
if allDropped {
|
2016-03-09 09:56:30 -08:00
|
|
|
s.persistence.purgeArchivedMetric(fp) // Ignoring error. Nothing we can do.
|
2014-11-10 09:22:08 -08:00
|
|
|
s.seriesOps.WithLabelValues(archivePurge).Inc()
|
|
|
|
return
|
2014-06-06 02:55:53 -07:00
|
|
|
}
|
2015-09-12 16:06:40 -07:00
|
|
|
if err := s.persistence.updateArchivedTimeRange(fp, newFirstTime, lastTime); err != nil {
|
|
|
|
log.Errorf("Error updating archived time range for fingerprint %v: %s", fp, err)
|
|
|
|
}
|
2014-06-06 02:55:53 -07:00
|
|
|
}
|
|
|
|
|
2014-11-27 11:46:45 -08:00
|
|
|
// See persistence.loadChunks for detailed explanation.
|
2016-09-21 14:44:27 -07:00
|
|
|
func (s *MemorySeriesStorage) loadChunks(fp model.Fingerprint, indexes []int, indexOffset int) ([]chunk.Chunk, error) {
|
2014-11-27 11:46:45 -08:00
|
|
|
return s.persistence.loadChunks(fp, indexes, indexOffset)
|
|
|
|
}
|
|
|
|
|
|
|
|
// See persistence.loadChunkDescs for detailed explanation.
|
2016-09-21 14:44:27 -07:00
|
|
|
func (s *MemorySeriesStorage) loadChunkDescs(fp model.Fingerprint, offsetFromEnd int) ([]*chunk.Desc, error) {
|
2015-07-06 16:10:14 -07:00
|
|
|
return s.persistence.loadChunkDescs(fp, offsetFromEnd)
|
2014-11-27 11:46:45 -08:00
|
|
|
}
|
|
|
|
|
2015-03-18 11:36:41 -07:00
|
|
|
// getNumChunksToPersist returns numChunksToPersist in a goroutine-safe way.
|
2016-06-28 23:14:23 -07:00
|
|
|
func (s *MemorySeriesStorage) getNumChunksToPersist() int {
|
2015-03-18 11:36:41 -07:00
|
|
|
return int(atomic.LoadInt64(&s.numChunksToPersist))
|
2015-03-08 18:33:10 -07:00
|
|
|
}
|
|
|
|
|
2015-03-18 11:36:41 -07:00
|
|
|
// incNumChunksToPersist increments numChunksToPersist in a goroutine-safe way. Use a
|
2015-03-08 18:33:10 -07:00
|
|
|
// negative 'by' to decrement.
|
2016-06-28 23:14:23 -07:00
|
|
|
func (s *MemorySeriesStorage) incNumChunksToPersist(by int) {
|
2015-03-18 11:36:41 -07:00
|
|
|
atomic.AddInt64(&s.numChunksToPersist, int64(by))
|
2015-03-08 18:33:10 -07:00
|
|
|
}
|
|
|
|
|
2016-01-26 08:29:33 -08:00
|
|
|
// calculatePersistenceUrgencyScore calculates and returns an urgency score for
|
|
|
|
// the speed of persisting chunks. The score is between 0 and 1, where 0 means
|
|
|
|
// no urgency at all and 1 means highest urgency.
|
|
|
|
//
|
|
|
|
// The score is the maximum of the two following sub-scores:
|
|
|
|
//
|
|
|
|
// (1) The first sub-score is the number of chunks waiting for persistence
|
|
|
|
// divided by the maximum number of chunks allowed to be waiting for
|
|
|
|
// persistence.
|
|
|
|
//
|
|
|
|
// (2) If there are more chunks in memory than allowed AND there are more chunks
|
|
|
|
// waiting for persistence than factorMinChunksToPersist times
|
|
|
|
// -storage.local.max-chunks-to-persist, then the second sub-score is the
|
|
|
|
// fraction the number of memory chunks has reached between
|
|
|
|
// -storage.local.memory-chunks and toleranceFactorForMemChunks times
|
|
|
|
// -storage.local.memory-chunks.
|
|
|
|
//
|
|
|
|
// Should the score ever hit persintenceUrgencyScoreForEnteringRushedMode, the
|
|
|
|
// storage locks into "rushed mode", in which the returned score is always
|
|
|
|
// bumped up to 1 until the non-bumped score is below
|
|
|
|
// persintenceUrgencyScoreForLeavingRushedMode.
|
|
|
|
//
|
|
|
|
// This method is not goroutine-safe, but it is only ever called by the single
|
|
|
|
// goroutine that is in charge of series maintenance. According to the returned
|
2016-02-09 18:47:00 -08:00
|
|
|
// score, series maintenance should be sped up. If a score of 1 is returned,
|
2016-01-26 08:29:33 -08:00
|
|
|
// checkpointing based on dirty-series count should be disabled, and series
|
|
|
|
// files should not by synced anymore provided the user has specified the
|
|
|
|
// adaptive sync strategy.
|
2016-06-28 23:14:23 -07:00
|
|
|
func (s *MemorySeriesStorage) calculatePersistenceUrgencyScore() float64 {
|
2016-02-22 06:48:39 -08:00
|
|
|
s.rushedMtx.Lock()
|
|
|
|
defer s.rushedMtx.Unlock()
|
|
|
|
|
2016-01-26 08:29:33 -08:00
|
|
|
var (
|
|
|
|
chunksToPersist = float64(s.getNumChunksToPersist())
|
|
|
|
maxChunksToPersist = float64(s.maxChunksToPersist)
|
2016-09-21 14:44:27 -07:00
|
|
|
memChunks = float64(atomic.LoadInt64(&chunk.NumMemChunks))
|
2016-01-26 08:29:33 -08:00
|
|
|
maxMemChunks = float64(s.maxMemoryChunks)
|
|
|
|
)
|
|
|
|
score := chunksToPersist / maxChunksToPersist
|
|
|
|
if chunksToPersist > maxChunksToPersist*factorMinChunksToPersist {
|
|
|
|
score = math.Max(
|
2016-01-25 09:44:43 -08:00
|
|
|
score,
|
2016-01-27 10:07:46 -08:00
|
|
|
(memChunks/maxMemChunks-1)/(toleranceFactorMemChunks-1),
|
2016-01-25 09:44:43 -08:00
|
|
|
)
|
|
|
|
}
|
|
|
|
if score > 1 {
|
2016-01-26 08:29:33 -08:00
|
|
|
score = 1
|
|
|
|
}
|
|
|
|
s.persistenceUrgencyScore.Set(score)
|
|
|
|
|
|
|
|
if s.rushed {
|
|
|
|
// We are already in rushed mode. If the score is still above
|
|
|
|
// persintenceUrgencyScoreForLeavingRushedMode, return 1 and
|
|
|
|
// leave things as they are.
|
|
|
|
if score > persintenceUrgencyScoreForLeavingRushedMode {
|
|
|
|
return 1
|
|
|
|
}
|
|
|
|
// We are out of rushed mode!
|
|
|
|
s.rushed = false
|
|
|
|
s.rushedMode.Set(0)
|
|
|
|
log.
|
|
|
|
With("urgencyScore", score).
|
2016-01-27 10:07:46 -08:00
|
|
|
With("chunksToPersist", int(chunksToPersist)).
|
|
|
|
With("maxChunksToPersist", int(maxChunksToPersist)).
|
|
|
|
With("memoryChunks", int(memChunks)).
|
|
|
|
With("maxMemoryChunks", int(maxMemChunks)).
|
|
|
|
Info("Storage has left rushed mode.")
|
2016-01-26 08:29:33 -08:00
|
|
|
return score
|
|
|
|
}
|
|
|
|
if score > persintenceUrgencyScoreForEnteringRushedMode {
|
|
|
|
// Enter rushed mode.
|
|
|
|
s.rushed = true
|
|
|
|
s.rushedMode.Set(1)
|
|
|
|
log.
|
|
|
|
With("urgencyScore", score).
|
2016-01-27 10:07:46 -08:00
|
|
|
With("chunksToPersist", int(chunksToPersist)).
|
|
|
|
With("maxChunksToPersist", int(maxChunksToPersist)).
|
|
|
|
With("memoryChunks", int(memChunks)).
|
|
|
|
With("maxMemoryChunks", int(maxMemChunks)).
|
2016-01-26 08:29:33 -08:00
|
|
|
Warn("Storage has entered rushed mode.")
|
2016-01-25 09:44:43 -08:00
|
|
|
return 1
|
|
|
|
}
|
2015-04-01 08:52:03 -07:00
|
|
|
return score
|
|
|
|
}
|
|
|
|
|
Handle errors caused by data corruption more gracefully
This requires all the panic calls upon unexpected data to be converted
into errors returned. This pollute the function signatures quite
lot. Well, this is Go...
The ideas behind this are the following:
- panic only if it's a programming error. Data corruptions happen, and
they are not programming errors.
- If we detect a data corruption, we "quarantine" the series,
essentially removing it from the database and putting its data into
a separate directory for forensics.
- Failure during writing to a series file is not considered corruption
automatically. It will call setDirty, though, so that a
crashrecovery upon the next restart will commence and check for
that.
- Series quarantining and setDirty calls are logged and counted in
metrics, but are hidden from the user of the interfaces in
interface.go, whith the notable exception of Append(). The reasoning
is that we treat corruption by removing the corrupted series, i.e. a
query for it will return no results on its next call anyway, so
return no results right now. In the case of Append(), we want to
tell the user that no data has been appended, though.
Minor side effects:
- Now consistently using filepath.* instead of path.*.
- Introduced structured logging where I touched it. This makes things
less consistent, but a complete change to structured logging would
be out of scope for this PR.
2016-02-25 03:23:42 -08:00
|
|
|
// quarantineSeries registers the provided fingerprint for quarantining. It
|
|
|
|
// always returns immediately. Quarantine requests are processed
|
|
|
|
// asynchronously. If there are too many requests queued, they are simply
|
|
|
|
// dropped.
|
|
|
|
//
|
|
|
|
// Quarantining means that the series file is moved to the orphaned directory,
|
|
|
|
// and all its traces are removed from indices. Call this method if an
|
|
|
|
// unrecoverable error is detected while dealing with a series, and pass in the
|
|
|
|
// encountered error. It will be saved as a hint in the orphaned directory.
|
2016-06-28 23:14:23 -07:00
|
|
|
func (s *MemorySeriesStorage) quarantineSeries(fp model.Fingerprint, metric model.Metric, err error) {
|
Handle errors caused by data corruption more gracefully
This requires all the panic calls upon unexpected data to be converted
into errors returned. This pollute the function signatures quite
lot. Well, this is Go...
The ideas behind this are the following:
- panic only if it's a programming error. Data corruptions happen, and
they are not programming errors.
- If we detect a data corruption, we "quarantine" the series,
essentially removing it from the database and putting its data into
a separate directory for forensics.
- Failure during writing to a series file is not considered corruption
automatically. It will call setDirty, though, so that a
crashrecovery upon the next restart will commence and check for
that.
- Series quarantining and setDirty calls are logged and counted in
metrics, but are hidden from the user of the interfaces in
interface.go, whith the notable exception of Append(). The reasoning
is that we treat corruption by removing the corrupted series, i.e. a
query for it will return no results on its next call anyway, so
return no results right now. In the case of Append(), we want to
tell the user that no data has been appended, though.
Minor side effects:
- Now consistently using filepath.* instead of path.*.
- Introduced structured logging where I touched it. This makes things
less consistent, but a complete change to structured logging would
be out of scope for this PR.
2016-02-25 03:23:42 -08:00
|
|
|
req := quarantineRequest{fp: fp, metric: metric, reason: err}
|
|
|
|
select {
|
|
|
|
case s.quarantineRequests <- req:
|
|
|
|
// Request submitted.
|
|
|
|
default:
|
|
|
|
log.
|
|
|
|
With("fingerprint", fp).
|
|
|
|
With("metric", metric).
|
|
|
|
With("reason", err).
|
|
|
|
Warn("Quarantine queue full. Dropped quarantine request.")
|
|
|
|
s.seriesOps.WithLabelValues(droppedQuarantine).Inc()
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2016-06-28 23:14:23 -07:00
|
|
|
func (s *MemorySeriesStorage) handleQuarantine() {
|
Handle errors caused by data corruption more gracefully
This requires all the panic calls upon unexpected data to be converted
into errors returned. This pollute the function signatures quite
lot. Well, this is Go...
The ideas behind this are the following:
- panic only if it's a programming error. Data corruptions happen, and
they are not programming errors.
- If we detect a data corruption, we "quarantine" the series,
essentially removing it from the database and putting its data into
a separate directory for forensics.
- Failure during writing to a series file is not considered corruption
automatically. It will call setDirty, though, so that a
crashrecovery upon the next restart will commence and check for
that.
- Series quarantining and setDirty calls are logged and counted in
metrics, but are hidden from the user of the interfaces in
interface.go, whith the notable exception of Append(). The reasoning
is that we treat corruption by removing the corrupted series, i.e. a
query for it will return no results on its next call anyway, so
return no results right now. In the case of Append(), we want to
tell the user that no data has been appended, though.
Minor side effects:
- Now consistently using filepath.* instead of path.*.
- Introduced structured logging where I touched it. This makes things
less consistent, but a complete change to structured logging would
be out of scope for this PR.
2016-02-25 03:23:42 -08:00
|
|
|
for {
|
|
|
|
select {
|
|
|
|
case req := <-s.quarantineRequests:
|
|
|
|
s.purgeSeries(req.fp, req.metric, req.reason)
|
|
|
|
log.
|
|
|
|
With("fingerprint", req.fp).
|
|
|
|
With("metric", req.metric).
|
|
|
|
With("reason", req.reason).
|
|
|
|
Warn("Series quarantined.")
|
|
|
|
case <-s.quarantineStopping:
|
|
|
|
log.Info("Series quarantining stopped.")
|
|
|
|
close(s.quarantineStopped)
|
|
|
|
return
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
}
|
|
|
|
|
|
|
|
// purgeSeries removes all traces of a series. If a non-nil quarantine reason is
|
|
|
|
// provided, the series file will not be deleted completely, but moved to the
|
|
|
|
// orphaned directory with the reason and the metric in a hint file. The
|
|
|
|
// provided metric might be nil if unknown.
|
2016-06-28 23:14:23 -07:00
|
|
|
func (s *MemorySeriesStorage) purgeSeries(fp model.Fingerprint, m model.Metric, quarantineReason error) {
|
Handle errors caused by data corruption more gracefully
This requires all the panic calls upon unexpected data to be converted
into errors returned. This pollute the function signatures quite
lot. Well, this is Go...
The ideas behind this are the following:
- panic only if it's a programming error. Data corruptions happen, and
they are not programming errors.
- If we detect a data corruption, we "quarantine" the series,
essentially removing it from the database and putting its data into
a separate directory for forensics.
- Failure during writing to a series file is not considered corruption
automatically. It will call setDirty, though, so that a
crashrecovery upon the next restart will commence and check for
that.
- Series quarantining and setDirty calls are logged and counted in
metrics, but are hidden from the user of the interfaces in
interface.go, whith the notable exception of Append(). The reasoning
is that we treat corruption by removing the corrupted series, i.e. a
query for it will return no results on its next call anyway, so
return no results right now. In the case of Append(), we want to
tell the user that no data has been appended, though.
Minor side effects:
- Now consistently using filepath.* instead of path.*.
- Introduced structured logging where I touched it. This makes things
less consistent, but a complete change to structured logging would
be out of scope for this PR.
2016-02-25 03:23:42 -08:00
|
|
|
s.fpLocker.Lock(fp)
|
|
|
|
|
|
|
|
var (
|
|
|
|
series *memorySeries
|
|
|
|
ok bool
|
|
|
|
)
|
|
|
|
|
|
|
|
if series, ok = s.fpToSeries.get(fp); ok {
|
|
|
|
s.fpToSeries.del(fp)
|
|
|
|
s.numSeries.Dec()
|
|
|
|
m = series.metric
|
|
|
|
|
2016-09-21 14:44:27 -07:00
|
|
|
// Adjust s.numChunksToPersist and chunk.NumMemChunks down by
|
Handle errors caused by data corruption more gracefully
This requires all the panic calls upon unexpected data to be converted
into errors returned. This pollute the function signatures quite
lot. Well, this is Go...
The ideas behind this are the following:
- panic only if it's a programming error. Data corruptions happen, and
they are not programming errors.
- If we detect a data corruption, we "quarantine" the series,
essentially removing it from the database and putting its data into
a separate directory for forensics.
- Failure during writing to a series file is not considered corruption
automatically. It will call setDirty, though, so that a
crashrecovery upon the next restart will commence and check for
that.
- Series quarantining and setDirty calls are logged and counted in
metrics, but are hidden from the user of the interfaces in
interface.go, whith the notable exception of Append(). The reasoning
is that we treat corruption by removing the corrupted series, i.e. a
query for it will return no results on its next call anyway, so
return no results right now. In the case of Append(), we want to
tell the user that no data has been appended, though.
Minor side effects:
- Now consistently using filepath.* instead of path.*.
- Introduced structured logging where I touched it. This makes things
less consistent, but a complete change to structured logging would
be out of scope for this PR.
2016-02-25 03:23:42 -08:00
|
|
|
// the number of chunks in this series that are not
|
|
|
|
// persisted yet. Persisted chunks will be deducted from
|
2016-09-21 14:44:27 -07:00
|
|
|
// chunk.NumMemChunks upon eviction.
|
Handle errors caused by data corruption more gracefully
This requires all the panic calls upon unexpected data to be converted
into errors returned. This pollute the function signatures quite
lot. Well, this is Go...
The ideas behind this are the following:
- panic only if it's a programming error. Data corruptions happen, and
they are not programming errors.
- If we detect a data corruption, we "quarantine" the series,
essentially removing it from the database and putting its data into
a separate directory for forensics.
- Failure during writing to a series file is not considered corruption
automatically. It will call setDirty, though, so that a
crashrecovery upon the next restart will commence and check for
that.
- Series quarantining and setDirty calls are logged and counted in
metrics, but are hidden from the user of the interfaces in
interface.go, whith the notable exception of Append(). The reasoning
is that we treat corruption by removing the corrupted series, i.e. a
query for it will return no results on its next call anyway, so
return no results right now. In the case of Append(), we want to
tell the user that no data has been appended, though.
Minor side effects:
- Now consistently using filepath.* instead of path.*.
- Introduced structured logging where I touched it. This makes things
less consistent, but a complete change to structured logging would
be out of scope for this PR.
2016-02-25 03:23:42 -08:00
|
|
|
numChunksNotYetPersisted := len(series.chunkDescs) - series.persistWatermark
|
2016-09-21 14:44:27 -07:00
|
|
|
atomic.AddInt64(&chunk.NumMemChunks, int64(-numChunksNotYetPersisted))
|
Handle errors caused by data corruption more gracefully
This requires all the panic calls upon unexpected data to be converted
into errors returned. This pollute the function signatures quite
lot. Well, this is Go...
The ideas behind this are the following:
- panic only if it's a programming error. Data corruptions happen, and
they are not programming errors.
- If we detect a data corruption, we "quarantine" the series,
essentially removing it from the database and putting its data into
a separate directory for forensics.
- Failure during writing to a series file is not considered corruption
automatically. It will call setDirty, though, so that a
crashrecovery upon the next restart will commence and check for
that.
- Series quarantining and setDirty calls are logged and counted in
metrics, but are hidden from the user of the interfaces in
interface.go, whith the notable exception of Append(). The reasoning
is that we treat corruption by removing the corrupted series, i.e. a
query for it will return no results on its next call anyway, so
return no results right now. In the case of Append(), we want to
tell the user that no data has been appended, though.
Minor side effects:
- Now consistently using filepath.* instead of path.*.
- Introduced structured logging where I touched it. This makes things
less consistent, but a complete change to structured logging would
be out of scope for this PR.
2016-02-25 03:23:42 -08:00
|
|
|
if !series.headChunkClosed {
|
|
|
|
// Head chunk wasn't counted as waiting for persistence yet.
|
|
|
|
// (But it was counted as a chunk in memory.)
|
|
|
|
numChunksNotYetPersisted--
|
|
|
|
}
|
|
|
|
s.incNumChunksToPersist(-numChunksNotYetPersisted)
|
|
|
|
|
|
|
|
} else {
|
2016-03-09 09:56:30 -08:00
|
|
|
s.persistence.purgeArchivedMetric(fp) // Ignoring error. There is nothing we can do.
|
Handle errors caused by data corruption more gracefully
This requires all the panic calls upon unexpected data to be converted
into errors returned. This pollute the function signatures quite
lot. Well, this is Go...
The ideas behind this are the following:
- panic only if it's a programming error. Data corruptions happen, and
they are not programming errors.
- If we detect a data corruption, we "quarantine" the series,
essentially removing it from the database and putting its data into
a separate directory for forensics.
- Failure during writing to a series file is not considered corruption
automatically. It will call setDirty, though, so that a
crashrecovery upon the next restart will commence and check for
that.
- Series quarantining and setDirty calls are logged and counted in
metrics, but are hidden from the user of the interfaces in
interface.go, whith the notable exception of Append(). The reasoning
is that we treat corruption by removing the corrupted series, i.e. a
query for it will return no results on its next call anyway, so
return no results right now. In the case of Append(), we want to
tell the user that no data has been appended, though.
Minor side effects:
- Now consistently using filepath.* instead of path.*.
- Introduced structured logging where I touched it. This makes things
less consistent, but a complete change to structured logging would
be out of scope for this PR.
2016-02-25 03:23:42 -08:00
|
|
|
}
|
|
|
|
if m != nil {
|
|
|
|
// If we know a metric now, unindex it in any case.
|
|
|
|
// purgeArchivedMetric might have done so already, but we cannot
|
|
|
|
// be sure. Unindexing in idempotent, though.
|
|
|
|
s.persistence.unindexMetric(fp, m)
|
|
|
|
}
|
|
|
|
// Attempt to delete/quarantine the series file in any case.
|
|
|
|
if quarantineReason == nil {
|
|
|
|
// No reason stated, simply delete the file.
|
|
|
|
if _, err := s.persistence.deleteSeriesFile(fp); err != nil {
|
|
|
|
log.
|
|
|
|
With("fingerprint", fp).
|
|
|
|
With("metric", m).
|
|
|
|
With("error", err).
|
|
|
|
Error("Error deleting series file.")
|
|
|
|
}
|
|
|
|
s.seriesOps.WithLabelValues(requestedPurge).Inc()
|
|
|
|
} else {
|
|
|
|
if err := s.persistence.quarantineSeriesFile(fp, quarantineReason, m); err == nil {
|
|
|
|
s.seriesOps.WithLabelValues(completedQurantine).Inc()
|
|
|
|
} else {
|
|
|
|
s.seriesOps.WithLabelValues(failedQuarantine).Inc()
|
|
|
|
log.
|
|
|
|
With("fingerprint", fp).
|
|
|
|
With("metric", m).
|
|
|
|
With("reason", quarantineReason).
|
|
|
|
With("error", err).
|
|
|
|
Error("Error quarantining series file.")
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
s.fpLocker.Unlock(fp)
|
|
|
|
}
|
|
|
|
|
2014-10-07 10:11:24 -07:00
|
|
|
// Describe implements prometheus.Collector.
|
2016-06-28 23:14:23 -07:00
|
|
|
func (s *MemorySeriesStorage) Describe(ch chan<- *prometheus.Desc) {
|
2014-10-07 10:11:24 -07:00
|
|
|
s.persistence.Describe(ch)
|
2015-05-21 08:50:06 -07:00
|
|
|
s.mapper.Describe(ch)
|
2014-10-23 06:18:32 -07:00
|
|
|
|
2015-02-01 03:47:51 -08:00
|
|
|
ch <- s.persistErrors.Desc()
|
2015-03-18 11:36:41 -07:00
|
|
|
ch <- maxChunksToPersistDesc
|
|
|
|
ch <- numChunksToPersistDesc
|
2014-10-23 06:18:32 -07:00
|
|
|
ch <- s.numSeries.Desc()
|
|
|
|
s.seriesOps.Describe(ch)
|
|
|
|
ch <- s.ingestedSamplesCount.Desc()
|
2016-04-25 07:43:52 -07:00
|
|
|
s.discardedSamplesCount.Describe(ch)
|
2016-03-13 03:54:24 -07:00
|
|
|
ch <- s.nonExistentSeriesMatchesCount.Desc()
|
2016-09-21 14:44:27 -07:00
|
|
|
ch <- chunk.NumMemChunksDesc
|
2015-03-19 09:06:16 -07:00
|
|
|
s.maintainSeriesDuration.Describe(ch)
|
2016-01-26 08:29:33 -08:00
|
|
|
ch <- s.persistenceUrgencyScore.Desc()
|
|
|
|
ch <- s.rushedMode.Desc()
|
2014-10-07 10:11:24 -07:00
|
|
|
}
|
|
|
|
|
|
|
|
// Collect implements prometheus.Collector.
|
2016-06-28 23:14:23 -07:00
|
|
|
func (s *MemorySeriesStorage) Collect(ch chan<- prometheus.Metric) {
|
2014-10-07 10:11:24 -07:00
|
|
|
s.persistence.Collect(ch)
|
2015-05-21 08:50:06 -07:00
|
|
|
s.mapper.Collect(ch)
|
2014-10-23 06:18:32 -07:00
|
|
|
|
2015-02-01 03:47:51 -08:00
|
|
|
ch <- s.persistErrors
|
2015-03-08 18:33:10 -07:00
|
|
|
ch <- prometheus.MustNewConstMetric(
|
2015-03-18 11:36:41 -07:00
|
|
|
maxChunksToPersistDesc,
|
|
|
|
prometheus.GaugeValue,
|
|
|
|
float64(s.maxChunksToPersist),
|
|
|
|
)
|
|
|
|
ch <- prometheus.MustNewConstMetric(
|
|
|
|
numChunksToPersistDesc,
|
2015-03-08 18:33:10 -07:00
|
|
|
prometheus.GaugeValue,
|
2015-03-18 11:36:41 -07:00
|
|
|
float64(s.getNumChunksToPersist()),
|
2015-03-08 18:33:10 -07:00
|
|
|
)
|
2014-10-23 06:18:32 -07:00
|
|
|
ch <- s.numSeries
|
|
|
|
s.seriesOps.Collect(ch)
|
|
|
|
ch <- s.ingestedSamplesCount
|
2016-04-25 07:43:52 -07:00
|
|
|
s.discardedSamplesCount.Collect(ch)
|
2016-03-13 03:54:24 -07:00
|
|
|
ch <- s.nonExistentSeriesMatchesCount
|
2015-02-26 15:06:16 -08:00
|
|
|
ch <- prometheus.MustNewConstMetric(
|
2016-09-21 14:44:27 -07:00
|
|
|
chunk.NumMemChunksDesc,
|
2015-02-26 15:06:16 -08:00
|
|
|
prometheus.GaugeValue,
|
2016-09-21 14:44:27 -07:00
|
|
|
float64(atomic.LoadInt64(&chunk.NumMemChunks)),
|
2015-03-08 18:33:10 -07:00
|
|
|
)
|
2015-03-19 09:06:16 -07:00
|
|
|
s.maintainSeriesDuration.Collect(ch)
|
2016-01-26 08:29:33 -08:00
|
|
|
ch <- s.persistenceUrgencyScore
|
|
|
|
ch <- s.rushedMode
|
Improve persisting chunks to disk.
This is done by bucketing chunks by fingerprint. If the persisting to
disk falls behind, more and more chunks are in the queue. As soon as
there are "double hits", we will now persist both chunks in one go,
doubling the disk throughput (assuming it is limited by disk
seeks). Should even more pile up so that we end wit "triple hits", we
will persist those first, and so on.
Even if we have millions of time series, this will still help,
assuming not all of them are growing with the same speed. Series that
get many samples and/or are not very compressable will accumulate
chunks faster, and they will soon get double- or triple-writes.
To improve the chance of double writes,
-storage.local.persistence-queue-capacity could be set to a higher
value. However, that will slow down shutdown a lot (as the queue has
to be worked through). So we leave it to the user to set it to a
really high value. A more fundamental solution would be to checkpoint
not only head chunks, but also chunks still in the persist queue. That
would be quite complicated for a rather limited use-case (running many
time series with high ingestion rate on slow spinning disks).
2015-02-13 11:08:52 -08:00
|
|
|
}
|