prometheus/tsdb/head_wal.go

1484 lines
40 KiB
Go
Raw Normal View History

// Copyright 2021 The Prometheus Authors
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
package tsdb
import (
"fmt"
"math"
Sync sparsehistogram branch with main (#9189) * Fix `kuma_sd` targetgroup reporting (#9157) * Bundle all xDS targets into a single group Signed-off-by: austin ce <austin.cawley@gmail.com> * Snapshot in-memory chunks on shutdown for faster restarts (#7229) Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Rename links Signed-off-by: Levi Harrison <git@leviharrison.dev> * Remove Individual Data Type Caps in Per-shard Buffering for Remote Write (#8921) * Moved everything to nPending buffer Signed-off-by: Levi Harrison <git@leviharrison.dev> * Simplify exemplar capacity addition Signed-off-by: Levi Harrison <git@leviharrison.dev> * Added pre-allocation Signed-off-by: Levi Harrison <git@leviharrison.dev> * Don't allocate if not sending exemplars Signed-off-by: Levi Harrison <git@leviharrison.dev> * Avoid deadlock when processing duplicate series record (#9170) * Avoid deadlock when processing duplicate series record `processWALSamples()` needs to be able to send on its output channel before it can read the input channel, so reads to allow this in case the output channel is full. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * processWALSamples: update comment Previous text seems to relate to an earlier implementation. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Optimise WAL loading by removing extra map and caching min-time (#9160) * BenchmarkLoadWAL: close WAL after use So that goroutines are stopped and resources released Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * BenchmarkLoadWAL: make series IDs co-prime with #workers Series are distributed across workers by taking the modulus of the ID with the number of workers, so multiples of 100 are a poor choice. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * BenchmarkLoadWAL: simulate mmapped chunks Real Prometheus cuts chunks every 120 samples, then skips those samples when re-reading the WAL. Simulate this by creating a single mapped chunk for each series, since the max time is all the reader looks at. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Fix comment Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Remove series map from processWALSamples() The locks that is commented to reduce contention in are now sharded 32,000 ways, so won't be contended. Removing the map saves memory and goes just as fast. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * loadWAL: Cache the last mmapped chunk time So we can skip calling append() for samples it will reject. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Improvements from code review Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Full stops and capitals on comments Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Cache max time in both places mmappedChunks is updated Including refactor to extract function `setMMappedChunks`, to reduce code duplication. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Update head min/max time when mmapped chunks added This ensures we have the correct values if no WAL samples are added for that series. Note that `mSeries.maxTime()` was always `math.MinInt64` before, since that function doesn't consider mmapped chunks. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Split Go and React Tests (#8897) * Added go-ci and react-ci Co-authored-by: Julien Pivotto <roidelapluie@inuits.eu> Signed-off-by: Levi Harrison <git@leviharrison.dev> * Remove search keymap from new expression editor (#9184) Signed-off-by: Julius Volz <julius.volz@gmail.com> Co-authored-by: Austin Cawley-Edwards <austin.cawley@gmail.com> Co-authored-by: Levi Harrison <git@leviharrison.dev> Co-authored-by: Julien Pivotto <roidelapluie@inuits.eu> Co-authored-by: Bryan Boreham <bjboreham@gmail.com> Co-authored-by: Julius Volz <julius.volz@gmail.com>
2021-08-11 03:13:17 -07:00
"os"
"path/filepath"
"runtime"
Sync sparsehistogram branch with main (#9189) * Fix `kuma_sd` targetgroup reporting (#9157) * Bundle all xDS targets into a single group Signed-off-by: austin ce <austin.cawley@gmail.com> * Snapshot in-memory chunks on shutdown for faster restarts (#7229) Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Rename links Signed-off-by: Levi Harrison <git@leviharrison.dev> * Remove Individual Data Type Caps in Per-shard Buffering for Remote Write (#8921) * Moved everything to nPending buffer Signed-off-by: Levi Harrison <git@leviharrison.dev> * Simplify exemplar capacity addition Signed-off-by: Levi Harrison <git@leviharrison.dev> * Added pre-allocation Signed-off-by: Levi Harrison <git@leviharrison.dev> * Don't allocate if not sending exemplars Signed-off-by: Levi Harrison <git@leviharrison.dev> * Avoid deadlock when processing duplicate series record (#9170) * Avoid deadlock when processing duplicate series record `processWALSamples()` needs to be able to send on its output channel before it can read the input channel, so reads to allow this in case the output channel is full. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * processWALSamples: update comment Previous text seems to relate to an earlier implementation. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Optimise WAL loading by removing extra map and caching min-time (#9160) * BenchmarkLoadWAL: close WAL after use So that goroutines are stopped and resources released Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * BenchmarkLoadWAL: make series IDs co-prime with #workers Series are distributed across workers by taking the modulus of the ID with the number of workers, so multiples of 100 are a poor choice. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * BenchmarkLoadWAL: simulate mmapped chunks Real Prometheus cuts chunks every 120 samples, then skips those samples when re-reading the WAL. Simulate this by creating a single mapped chunk for each series, since the max time is all the reader looks at. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Fix comment Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Remove series map from processWALSamples() The locks that is commented to reduce contention in are now sharded 32,000 ways, so won't be contended. Removing the map saves memory and goes just as fast. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * loadWAL: Cache the last mmapped chunk time So we can skip calling append() for samples it will reject. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Improvements from code review Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Full stops and capitals on comments Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Cache max time in both places mmappedChunks is updated Including refactor to extract function `setMMappedChunks`, to reduce code duplication. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Update head min/max time when mmapped chunks added This ensures we have the correct values if no WAL samples are added for that series. Note that `mSeries.maxTime()` was always `math.MinInt64` before, since that function doesn't consider mmapped chunks. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Split Go and React Tests (#8897) * Added go-ci and react-ci Co-authored-by: Julien Pivotto <roidelapluie@inuits.eu> Signed-off-by: Levi Harrison <git@leviharrison.dev> * Remove search keymap from new expression editor (#9184) Signed-off-by: Julius Volz <julius.volz@gmail.com> Co-authored-by: Austin Cawley-Edwards <austin.cawley@gmail.com> Co-authored-by: Levi Harrison <git@leviharrison.dev> Co-authored-by: Julien Pivotto <roidelapluie@inuits.eu> Co-authored-by: Bryan Boreham <bjboreham@gmail.com> Co-authored-by: Julius Volz <julius.volz@gmail.com>
2021-08-11 03:13:17 -07:00
"strconv"
"strings"
"sync"
"time"
"github.com/go-kit/log/level"
"github.com/pkg/errors"
"go.uber.org/atomic"
"github.com/prometheus/prometheus/model/exemplar"
"github.com/prometheus/prometheus/model/labels"
Introduce TSDB changes for appending metadata to the WAL (#10972) * Append metadata to the WAL Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Remove extra whitespace; Reword some docstrings and comments Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Use RLock() for hasNewMetadata check Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Use single byte for metric type in RefMetadata Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Update proposed WAL format for single-byte type metadata Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Implementa MetadataAppender interface for the Agent Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Address first round of review comments Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Amend description of metadata in wal.md Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Correct key used to retrieve metadata from cache When we're setting metadata entries in the scrapeCace, we're using the p.Help(), p.Unit(), p.Type() helpers, which retrieve the series name and use it as the cache key. When checking for cache entries though, we used p.Series() as the key, which included the metric name _with_ its labels. That meant that we were never actually hitting the cache. We're fixing this by utiling the __name__ internal label for correctly getting the cache entries after they've been set by setHelp(), setType() or setUnit(). Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Put feature behind a feature flag Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Fix AppendMetadata docstring Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Reorder WAL format document Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Change error message of AppendMetadata; Fix access of s.meta in AppendMetadata Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Reuse temporary buffer in Metadata encoder Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Only keep latest metadata for each refID during checkpointing Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Fix test that's referencing decoding metadata Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Avoid creating metadata block if no new metadata are present Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Add tests for corrupt metadata block and relevant record type Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Fix CR comments Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Extract logic about changing metadata in an anonymous function Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Implement new proposed WAL format and amend relevant tests Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Use 'const' for metadata field names Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Apply metadata to head memSeries in Commit, not in AppendMetadata Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Add docstring and rename extracted helper in scrape.go Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Add tests for tsdb-related cases Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Fix linter issues vol1 Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Fix linter issues vol2 Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Fix Windows test by closing WAL reader files Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Use switch instead of two if statements in metadata decoding Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Fix review comments around TestMetadata* tests Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Add code for replaying WAL; test correctness of in-memory data after a replay Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Remove scrape-loop related code from PR Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Address first round of comments Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Simplify tests by sorting slices before comparison Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Fix test to use separate transactions Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Empty out buffer and record slices after encoding latest metadata Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Fix linting issue Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Update calculation for DroppedMetadata metric Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Rename MetadataAppender interface and AppendMetadata method to MetadataUpdater/UpdateMetadata Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Reuse buffer when encoding latest metadata for each series Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Fix review comments; Check all returned error values using two helpers Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Simplify use of helpers Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Satisfy linter Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com>
2022-07-19 01:58:52 -07:00
"github.com/prometheus/prometheus/model/metadata"
"github.com/prometheus/prometheus/storage"
"github.com/prometheus/prometheus/tsdb/chunkenc"
"github.com/prometheus/prometheus/tsdb/chunks"
"github.com/prometheus/prometheus/tsdb/encoding"
tsdb_errors "github.com/prometheus/prometheus/tsdb/errors"
"github.com/prometheus/prometheus/tsdb/fileutil"
"github.com/prometheus/prometheus/tsdb/record"
"github.com/prometheus/prometheus/tsdb/tombstones"
"github.com/prometheus/prometheus/tsdb/wlog"
)
func (h *Head) loadWAL(r *wlog.Reader, multiRef map[chunks.HeadSeriesRef]chunks.HeadSeriesRef, mmappedChunks, oooMmappedChunks map[chunks.HeadSeriesRef][]*mmappedChunk) (err error) {
// Track number of samples that referenced a series we don't know about
// for error reporting.
var unknownRefs atomic.Uint64
var unknownExemplarRefs atomic.Uint64
var unknownHistogramRefs atomic.Uint64
Introduce TSDB changes for appending metadata to the WAL (#10972) * Append metadata to the WAL Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Remove extra whitespace; Reword some docstrings and comments Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Use RLock() for hasNewMetadata check Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Use single byte for metric type in RefMetadata Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Update proposed WAL format for single-byte type metadata Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Implementa MetadataAppender interface for the Agent Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Address first round of review comments Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Amend description of metadata in wal.md Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Correct key used to retrieve metadata from cache When we're setting metadata entries in the scrapeCace, we're using the p.Help(), p.Unit(), p.Type() helpers, which retrieve the series name and use it as the cache key. When checking for cache entries though, we used p.Series() as the key, which included the metric name _with_ its labels. That meant that we were never actually hitting the cache. We're fixing this by utiling the __name__ internal label for correctly getting the cache entries after they've been set by setHelp(), setType() or setUnit(). Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Put feature behind a feature flag Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Fix AppendMetadata docstring Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Reorder WAL format document Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Change error message of AppendMetadata; Fix access of s.meta in AppendMetadata Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Reuse temporary buffer in Metadata encoder Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Only keep latest metadata for each refID during checkpointing Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Fix test that's referencing decoding metadata Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Avoid creating metadata block if no new metadata are present Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Add tests for corrupt metadata block and relevant record type Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Fix CR comments Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Extract logic about changing metadata in an anonymous function Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Implement new proposed WAL format and amend relevant tests Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Use 'const' for metadata field names Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Apply metadata to head memSeries in Commit, not in AppendMetadata Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Add docstring and rename extracted helper in scrape.go Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Add tests for tsdb-related cases Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Fix linter issues vol1 Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Fix linter issues vol2 Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Fix Windows test by closing WAL reader files Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Use switch instead of two if statements in metadata decoding Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Fix review comments around TestMetadata* tests Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Add code for replaying WAL; test correctness of in-memory data after a replay Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Remove scrape-loop related code from PR Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Address first round of comments Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Simplify tests by sorting slices before comparison Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Fix test to use separate transactions Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Empty out buffer and record slices after encoding latest metadata Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Fix linting issue Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Update calculation for DroppedMetadata metric Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Rename MetadataAppender interface and AppendMetadata method to MetadataUpdater/UpdateMetadata Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Reuse buffer when encoding latest metadata for each series Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Fix review comments; Check all returned error values using two helpers Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Simplify use of helpers Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Satisfy linter Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com>
2022-07-19 01:58:52 -07:00
var unknownMetadataRefs atomic.Uint64
// Track number of series records that had overlapping m-map chunks.
var mmapOverlappingChunks atomic.Uint64
// Start workers that each process samples for a partition of the series ID space.
var (
wg sync.WaitGroup
n = runtime.GOMAXPROCS(0)
processors = make([]walSubsetProcessor, n)
exemplarsInput chan record.RefExemplar
2021-11-30 08:19:06 -08:00
dec record.Decoder
shards = make([][]record.RefSample, n)
histogramShards = make([][]record.RefHistogramSample, n)
decoded = make(chan interface{}, 10)
decodeErr, seriesCreationErr error
seriesPool = sync.Pool{
New: func() interface{} {
return []record.RefSeries{}
},
}
samplesPool = sync.Pool{
New: func() interface{} {
return []record.RefSample{}
},
}
tstonesPool = sync.Pool{
New: func() interface{} {
return []tombstones.Stone{}
},
}
exemplarsPool = sync.Pool{
New: func() interface{} {
return []record.RefExemplar{}
},
}
histogramsPool = sync.Pool{
New: func() interface{} {
return []record.RefHistogramSample{}
},
}
Introduce TSDB changes for appending metadata to the WAL (#10972) * Append metadata to the WAL Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Remove extra whitespace; Reword some docstrings and comments Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Use RLock() for hasNewMetadata check Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Use single byte for metric type in RefMetadata Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Update proposed WAL format for single-byte type metadata Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Implementa MetadataAppender interface for the Agent Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Address first round of review comments Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Amend description of metadata in wal.md Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Correct key used to retrieve metadata from cache When we're setting metadata entries in the scrapeCace, we're using the p.Help(), p.Unit(), p.Type() helpers, which retrieve the series name and use it as the cache key. When checking for cache entries though, we used p.Series() as the key, which included the metric name _with_ its labels. That meant that we were never actually hitting the cache. We're fixing this by utiling the __name__ internal label for correctly getting the cache entries after they've been set by setHelp(), setType() or setUnit(). Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Put feature behind a feature flag Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Fix AppendMetadata docstring Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Reorder WAL format document Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Change error message of AppendMetadata; Fix access of s.meta in AppendMetadata Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Reuse temporary buffer in Metadata encoder Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Only keep latest metadata for each refID during checkpointing Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Fix test that's referencing decoding metadata Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Avoid creating metadata block if no new metadata are present Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Add tests for corrupt metadata block and relevant record type Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Fix CR comments Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Extract logic about changing metadata in an anonymous function Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Implement new proposed WAL format and amend relevant tests Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Use 'const' for metadata field names Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Apply metadata to head memSeries in Commit, not in AppendMetadata Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Add docstring and rename extracted helper in scrape.go Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Add tests for tsdb-related cases Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Fix linter issues vol1 Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Fix linter issues vol2 Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Fix Windows test by closing WAL reader files Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Use switch instead of two if statements in metadata decoding Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Fix review comments around TestMetadata* tests Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Add code for replaying WAL; test correctness of in-memory data after a replay Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Remove scrape-loop related code from PR Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Address first round of comments Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Simplify tests by sorting slices before comparison Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Fix test to use separate transactions Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Empty out buffer and record slices after encoding latest metadata Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Fix linting issue Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Update calculation for DroppedMetadata metric Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Rename MetadataAppender interface and AppendMetadata method to MetadataUpdater/UpdateMetadata Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Reuse buffer when encoding latest metadata for each series Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Fix review comments; Check all returned error values using two helpers Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Simplify use of helpers Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Satisfy linter Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com>
2022-07-19 01:58:52 -07:00
metadataPool = sync.Pool{
New: func() interface{} {
return []record.RefMetadata{}
},
}
)
defer func() {
// For CorruptionErr ensure to terminate all workers before exiting.
_, ok := err.(*wlog.CorruptionErr)
if ok || seriesCreationErr != nil {
for i := 0; i < n; i++ {
processors[i].closeAndDrain()
}
close(exemplarsInput)
wg.Wait()
}
}()
wg.Add(n)
for i := 0; i < n; i++ {
processors[i].setup()
go func(wp *walSubsetProcessor) {
unknown, unknownHistograms, overlapping := wp.processWALSamples(h, mmappedChunks, oooMmappedChunks)
unknownRefs.Add(unknown)
mmapOverlappingChunks.Add(overlapping)
2021-11-30 08:19:06 -08:00
unknownHistogramRefs.Add(unknownHistograms)
wg.Done()
}(&processors[i])
}
wg.Add(1)
exemplarsInput = make(chan record.RefExemplar, 300)
go func(input <-chan record.RefExemplar) {
var err error
defer wg.Done()
for e := range input {
ms := h.series.getByID(e.Ref)
if ms == nil {
unknownExemplarRefs.Inc()
continue
}
if e.T < h.minValidTime.Load() {
continue
}
// At the moment the only possible error here is out of order exemplars, which we shouldn't see when
// replaying the WAL, so lets just log the error if it's not that type.
err = h.exemplars.AddExemplar(ms.lset, exemplar.Exemplar{Ts: e.T, Value: e.V, Labels: e.Labels})
if err != nil && err == storage.ErrOutOfOrderExemplar {
level.Warn(h.logger).Log("msg", "Unexpected error when replaying WAL on exemplar record", "err", err)
}
}
}(exemplarsInput)
go func() {
defer close(decoded)
var err error
for r.Next() {
rec := r.Record()
switch dec.Type(rec) {
case record.Series:
series := seriesPool.Get().([]record.RefSeries)[:0]
series, err = dec.Series(rec, series)
if err != nil {
decodeErr = &wlog.CorruptionErr{
Err: errors.Wrap(err, "decode series"),
Segment: r.Segment(),
Offset: r.Offset(),
}
return
}
decoded <- series
case record.Samples:
samples := samplesPool.Get().([]record.RefSample)[:0]
samples, err = dec.Samples(rec, samples)
if err != nil {
decodeErr = &wlog.CorruptionErr{
Err: errors.Wrap(err, "decode samples"),
Segment: r.Segment(),
Offset: r.Offset(),
}
return
}
decoded <- samples
case record.Tombstones:
tstones := tstonesPool.Get().([]tombstones.Stone)[:0]
tstones, err = dec.Tombstones(rec, tstones)
if err != nil {
decodeErr = &wlog.CorruptionErr{
Err: errors.Wrap(err, "decode tombstones"),
Segment: r.Segment(),
Offset: r.Offset(),
}
return
}
decoded <- tstones
case record.Exemplars:
exemplars := exemplarsPool.Get().([]record.RefExemplar)[:0]
exemplars, err = dec.Exemplars(rec, exemplars)
if err != nil {
decodeErr = &wlog.CorruptionErr{
Err: errors.Wrap(err, "decode exemplars"),
Segment: r.Segment(),
Offset: r.Offset(),
}
return
}
decoded <- exemplars
case record.HistogramSamples:
hists := histogramsPool.Get().([]record.RefHistogramSample)[:0]
hists, err = dec.HistogramSamples(rec, hists)
if err != nil {
decodeErr = &wlog.CorruptionErr{
Err: errors.Wrap(err, "decode histograms"),
Segment: r.Segment(),
Offset: r.Offset(),
}
return
}
decoded <- hists
Introduce TSDB changes for appending metadata to the WAL (#10972) * Append metadata to the WAL Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Remove extra whitespace; Reword some docstrings and comments Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Use RLock() for hasNewMetadata check Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Use single byte for metric type in RefMetadata Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Update proposed WAL format for single-byte type metadata Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Implementa MetadataAppender interface for the Agent Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Address first round of review comments Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Amend description of metadata in wal.md Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Correct key used to retrieve metadata from cache When we're setting metadata entries in the scrapeCace, we're using the p.Help(), p.Unit(), p.Type() helpers, which retrieve the series name and use it as the cache key. When checking for cache entries though, we used p.Series() as the key, which included the metric name _with_ its labels. That meant that we were never actually hitting the cache. We're fixing this by utiling the __name__ internal label for correctly getting the cache entries after they've been set by setHelp(), setType() or setUnit(). Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Put feature behind a feature flag Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Fix AppendMetadata docstring Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Reorder WAL format document Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Change error message of AppendMetadata; Fix access of s.meta in AppendMetadata Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Reuse temporary buffer in Metadata encoder Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Only keep latest metadata for each refID during checkpointing Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Fix test that's referencing decoding metadata Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Avoid creating metadata block if no new metadata are present Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Add tests for corrupt metadata block and relevant record type Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Fix CR comments Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Extract logic about changing metadata in an anonymous function Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Implement new proposed WAL format and amend relevant tests Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Use 'const' for metadata field names Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Apply metadata to head memSeries in Commit, not in AppendMetadata Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Add docstring and rename extracted helper in scrape.go Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Add tests for tsdb-related cases Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Fix linter issues vol1 Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Fix linter issues vol2 Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Fix Windows test by closing WAL reader files Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Use switch instead of two if statements in metadata decoding Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Fix review comments around TestMetadata* tests Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Add code for replaying WAL; test correctness of in-memory data after a replay Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Remove scrape-loop related code from PR Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Address first round of comments Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Simplify tests by sorting slices before comparison Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Fix test to use separate transactions Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Empty out buffer and record slices after encoding latest metadata Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Fix linting issue Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Update calculation for DroppedMetadata metric Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Rename MetadataAppender interface and AppendMetadata method to MetadataUpdater/UpdateMetadata Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Reuse buffer when encoding latest metadata for each series Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Fix review comments; Check all returned error values using two helpers Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Simplify use of helpers Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Satisfy linter Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com>
2022-07-19 01:58:52 -07:00
case record.Metadata:
meta := metadataPool.Get().([]record.RefMetadata)[:0]
meta, err := dec.Metadata(rec, meta)
if err != nil {
decodeErr = &wlog.CorruptionErr{
Introduce TSDB changes for appending metadata to the WAL (#10972) * Append metadata to the WAL Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Remove extra whitespace; Reword some docstrings and comments Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Use RLock() for hasNewMetadata check Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Use single byte for metric type in RefMetadata Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Update proposed WAL format for single-byte type metadata Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Implementa MetadataAppender interface for the Agent Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Address first round of review comments Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Amend description of metadata in wal.md Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Correct key used to retrieve metadata from cache When we're setting metadata entries in the scrapeCace, we're using the p.Help(), p.Unit(), p.Type() helpers, which retrieve the series name and use it as the cache key. When checking for cache entries though, we used p.Series() as the key, which included the metric name _with_ its labels. That meant that we were never actually hitting the cache. We're fixing this by utiling the __name__ internal label for correctly getting the cache entries after they've been set by setHelp(), setType() or setUnit(). Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Put feature behind a feature flag Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Fix AppendMetadata docstring Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Reorder WAL format document Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Change error message of AppendMetadata; Fix access of s.meta in AppendMetadata Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Reuse temporary buffer in Metadata encoder Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Only keep latest metadata for each refID during checkpointing Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Fix test that's referencing decoding metadata Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Avoid creating metadata block if no new metadata are present Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Add tests for corrupt metadata block and relevant record type Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Fix CR comments Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Extract logic about changing metadata in an anonymous function Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Implement new proposed WAL format and amend relevant tests Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Use 'const' for metadata field names Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Apply metadata to head memSeries in Commit, not in AppendMetadata Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Add docstring and rename extracted helper in scrape.go Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Add tests for tsdb-related cases Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Fix linter issues vol1 Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Fix linter issues vol2 Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Fix Windows test by closing WAL reader files Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Use switch instead of two if statements in metadata decoding Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Fix review comments around TestMetadata* tests Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Add code for replaying WAL; test correctness of in-memory data after a replay Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Remove scrape-loop related code from PR Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Address first round of comments Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Simplify tests by sorting slices before comparison Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Fix test to use separate transactions Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Empty out buffer and record slices after encoding latest metadata Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Fix linting issue Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Update calculation for DroppedMetadata metric Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Rename MetadataAppender interface and AppendMetadata method to MetadataUpdater/UpdateMetadata Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Reuse buffer when encoding latest metadata for each series Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Fix review comments; Check all returned error values using two helpers Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Simplify use of helpers Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Satisfy linter Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com>
2022-07-19 01:58:52 -07:00
Err: errors.Wrap(err, "decode metadata"),
Segment: r.Segment(),
Offset: r.Offset(),
}
return
}
decoded <- meta
default:
// Noop.
}
}
}()
// The records are always replayed from the oldest to the newest.
Outer:
for d := range decoded {
switch v := d.(type) {
case []record.RefSeries:
for _, walSeries := range v {
mSeries, created, err := h.getOrCreateWithID(walSeries.Ref, walSeries.Labels.Hash(), walSeries.Labels)
if err != nil {
seriesCreationErr = err
break Outer
}
if chunks.HeadSeriesRef(h.lastSeriesID.Load()) < walSeries.Ref {
h.lastSeriesID.Store(uint64(walSeries.Ref))
}
if !created {
multiRef[walSeries.Ref] = mSeries.ref
}
idx := uint64(mSeries.ref) % uint64(n)
processors[idx].input <- walSubsetProcessorInputItem{walSeriesRef: walSeries.Ref, existingSeries: mSeries}
}
//nolint:staticcheck // Ignore SA6002 relax staticcheck verification.
seriesPool.Put(v)
case []record.RefSample:
samples := v
minValidTime := h.minValidTime.Load()
// We split up the samples into chunks of 5000 samples or less.
// With O(300 * #cores) in-flight sample batches, large scrapes could otherwise
// cause thousands of very large in flight buffers occupying large amounts
// of unused memory.
for len(samples) > 0 {
m := 5000
if len(samples) < m {
m = len(samples)
}
for i := 0; i < n; i++ {
if shards[i] == nil {
shards[i] = processors[i].reuseBuf()
}
}
for _, sam := range samples[:m] {
if sam.T < minValidTime {
continue // Before minValidTime: discard.
}
if r, ok := multiRef[sam.Ref]; ok {
sam.Ref = r
}
mod := uint64(sam.Ref) % uint64(n)
shards[mod] = append(shards[mod], sam)
}
for i := 0; i < n; i++ {
if len(shards[i]) > 0 {
processors[i].input <- walSubsetProcessorInputItem{samples: shards[i]}
shards[i] = nil
}
}
samples = samples[m:]
}
//nolint:staticcheck // Ignore SA6002 relax staticcheck verification.
samplesPool.Put(v)
case []tombstones.Stone:
for _, s := range v {
for _, itv := range s.Intervals {
if itv.Maxt < h.minValidTime.Load() {
continue
}
if m := h.series.getByID(chunks.HeadSeriesRef(s.Ref)); m == nil {
unknownRefs.Inc()
continue
}
h.tombstones.AddInterval(storage.SeriesRef(s.Ref), itv)
}
}
//nolint:staticcheck // Ignore SA6002 relax staticcheck verification.
tstonesPool.Put(v)
case []record.RefExemplar:
for _, e := range v {
exemplarsInput <- e
}
//nolint:staticcheck // Ignore SA6002 relax staticcheck verification.
exemplarsPool.Put(v)
case []record.RefHistogramSample:
2021-11-30 08:19:06 -08:00
samples := v
minValidTime := h.minValidTime.Load()
2021-11-30 08:19:06 -08:00
// We split up the samples into chunks of 5000 samples or less.
// With O(300 * #cores) in-flight sample batches, large scrapes could otherwise
// cause thousands of very large in flight buffers occupying large amounts
// of unused memory.
for len(samples) > 0 {
m := 5000
if len(samples) < m {
m = len(samples)
}
for i := 0; i < n; i++ {
if histogramShards[i] == nil {
histogramShards[i] = processors[i].reuseHistogramBuf()
}
2021-11-30 08:19:06 -08:00
}
for _, sam := range samples[:m] {
if sam.T < minValidTime {
continue // Before minValidTime: discard.
}
2021-11-30 08:19:06 -08:00
if r, ok := multiRef[sam.Ref]; ok {
sam.Ref = r
}
mod := uint64(sam.Ref) % uint64(n)
histogramShards[mod] = append(histogramShards[mod], sam)
}
for i := 0; i < n; i++ {
if len(histogramShards[i]) > 0 {
processors[i].input <- walSubsetProcessorInputItem{histogramSamples: histogramShards[i]}
histogramShards[i] = nil
}
2021-11-30 08:19:06 -08:00
}
samples = samples[m:]
}
//nolint:staticcheck // Ignore SA6002 relax staticcheck verification.
histogramsPool.Put(v)
Introduce TSDB changes for appending metadata to the WAL (#10972) * Append metadata to the WAL Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Remove extra whitespace; Reword some docstrings and comments Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Use RLock() for hasNewMetadata check Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Use single byte for metric type in RefMetadata Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Update proposed WAL format for single-byte type metadata Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Implementa MetadataAppender interface for the Agent Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Address first round of review comments Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Amend description of metadata in wal.md Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Correct key used to retrieve metadata from cache When we're setting metadata entries in the scrapeCace, we're using the p.Help(), p.Unit(), p.Type() helpers, which retrieve the series name and use it as the cache key. When checking for cache entries though, we used p.Series() as the key, which included the metric name _with_ its labels. That meant that we were never actually hitting the cache. We're fixing this by utiling the __name__ internal label for correctly getting the cache entries after they've been set by setHelp(), setType() or setUnit(). Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Put feature behind a feature flag Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Fix AppendMetadata docstring Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Reorder WAL format document Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Change error message of AppendMetadata; Fix access of s.meta in AppendMetadata Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Reuse temporary buffer in Metadata encoder Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Only keep latest metadata for each refID during checkpointing Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Fix test that's referencing decoding metadata Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Avoid creating metadata block if no new metadata are present Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Add tests for corrupt metadata block and relevant record type Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Fix CR comments Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Extract logic about changing metadata in an anonymous function Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Implement new proposed WAL format and amend relevant tests Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Use 'const' for metadata field names Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Apply metadata to head memSeries in Commit, not in AppendMetadata Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Add docstring and rename extracted helper in scrape.go Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Add tests for tsdb-related cases Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Fix linter issues vol1 Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Fix linter issues vol2 Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Fix Windows test by closing WAL reader files Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Use switch instead of two if statements in metadata decoding Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Fix review comments around TestMetadata* tests Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Add code for replaying WAL; test correctness of in-memory data after a replay Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Remove scrape-loop related code from PR Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Address first round of comments Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Simplify tests by sorting slices before comparison Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Fix test to use separate transactions Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Empty out buffer and record slices after encoding latest metadata Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Fix linting issue Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Update calculation for DroppedMetadata metric Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Rename MetadataAppender interface and AppendMetadata method to MetadataUpdater/UpdateMetadata Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Reuse buffer when encoding latest metadata for each series Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Fix review comments; Check all returned error values using two helpers Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Simplify use of helpers Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Satisfy linter Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com>
2022-07-19 01:58:52 -07:00
case []record.RefMetadata:
for _, m := range v {
s := h.series.getByID(chunks.HeadSeriesRef(m.Ref))
if s == nil {
unknownMetadataRefs.Inc()
continue
}
s.meta = &metadata.Metadata{
Introduce TSDB changes for appending metadata to the WAL (#10972) * Append metadata to the WAL Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Remove extra whitespace; Reword some docstrings and comments Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Use RLock() for hasNewMetadata check Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Use single byte for metric type in RefMetadata Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Update proposed WAL format for single-byte type metadata Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Implementa MetadataAppender interface for the Agent Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Address first round of review comments Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Amend description of metadata in wal.md Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Correct key used to retrieve metadata from cache When we're setting metadata entries in the scrapeCace, we're using the p.Help(), p.Unit(), p.Type() helpers, which retrieve the series name and use it as the cache key. When checking for cache entries though, we used p.Series() as the key, which included the metric name _with_ its labels. That meant that we were never actually hitting the cache. We're fixing this by utiling the __name__ internal label for correctly getting the cache entries after they've been set by setHelp(), setType() or setUnit(). Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Put feature behind a feature flag Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Fix AppendMetadata docstring Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Reorder WAL format document Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Change error message of AppendMetadata; Fix access of s.meta in AppendMetadata Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Reuse temporary buffer in Metadata encoder Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Only keep latest metadata for each refID during checkpointing Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Fix test that's referencing decoding metadata Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Avoid creating metadata block if no new metadata are present Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Add tests for corrupt metadata block and relevant record type Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Fix CR comments Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Extract logic about changing metadata in an anonymous function Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Implement new proposed WAL format and amend relevant tests Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Use 'const' for metadata field names Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Apply metadata to head memSeries in Commit, not in AppendMetadata Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Add docstring and rename extracted helper in scrape.go Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Add tests for tsdb-related cases Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Fix linter issues vol1 Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Fix linter issues vol2 Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Fix Windows test by closing WAL reader files Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Use switch instead of two if statements in metadata decoding Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Fix review comments around TestMetadata* tests Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Add code for replaying WAL; test correctness of in-memory data after a replay Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Remove scrape-loop related code from PR Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Address first round of comments Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Simplify tests by sorting slices before comparison Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Fix test to use separate transactions Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Empty out buffer and record slices after encoding latest metadata Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Fix linting issue Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Update calculation for DroppedMetadata metric Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Rename MetadataAppender interface and AppendMetadata method to MetadataUpdater/UpdateMetadata Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Reuse buffer when encoding latest metadata for each series Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Fix review comments; Check all returned error values using two helpers Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Simplify use of helpers Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Satisfy linter Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com>
2022-07-19 01:58:52 -07:00
Type: record.ToTextparseMetricType(m.Type),
Unit: m.Unit,
Help: m.Help,
}
}
//nolint:staticcheck // Ignore SA6002 relax staticcheck verification.
metadataPool.Put(v)
default:
panic(fmt.Errorf("unexpected decoded type: %T", d))
}
}
if decodeErr != nil {
return decodeErr
}
if seriesCreationErr != nil {
// Drain the channel to unblock the goroutine.
for range decoded {
}
return seriesCreationErr
}
// Signal termination to each worker and wait for it to close its output channel.
for i := 0; i < n; i++ {
processors[i].closeAndDrain()
}
close(exemplarsInput)
wg.Wait()
if r.Err() != nil {
return errors.Wrap(r.Err(), "read records")
}
if unknownRefs.Load()+unknownExemplarRefs.Load()+unknownHistogramRefs.Load()+unknownMetadataRefs.Load() > 0 {
level.Warn(h.logger).Log(
"msg", "Unknown series references",
"samples", unknownRefs.Load(),
"exemplars", unknownExemplarRefs.Load(),
"histograms", unknownHistogramRefs.Load(),
"metadata", unknownMetadataRefs.Load(),
)
}
if count := mmapOverlappingChunks.Load(); count > 0 {
level.Info(h.logger).Log("msg", "Overlapping m-map chunks on duplicate series records", "count", count)
}
return nil
}
// resetSeriesWithMMappedChunks is only used during the WAL replay.
Add out-of-order sample support to the TSDB (#11075) * Introduce out-of-order TSDB support This implementation is based on this design doc: https://docs.google.com/document/d/1Kppm7qL9C-BJB1j6yb6-9ObG3AbdZnFUBYPNNWwDBYM/edit?usp=sharing This commit adds support to accept out-of-order ("OOO") sample into the TSDB up to a configurable time allowance. If OOO is enabled, overlapping querying are automatically enabled. Most of the additions have been borrowed from https://github.com/grafana/mimir-prometheus/ Here is the list ist of the original commits cherry picked from mimir-prometheus into this branch: - 4b2198d7ec47d50989b7c2df66b7b207c32f7f6e - 2836e5513f1bc591535a859f5d41154a75e7c6bc - 00b379c3a5b1ec3799699b6242f300a2b3ea30f0 - ff0dc757587cada63ca948d2d5eb00bf090d63e0 - a632c73352a7e39d60b445700beb47d691549c3e - c6f3d4ab339ab80bbbce74c9946237ced01f0509 - 5e8406a1d4a50d0052bbee83e28ca3b3371408aa - abde1e0ba128936b9eb0224ee1551e56216ebd4a - e70e7698897bb03860bee0467c733fa44e14c9bd - df59320886e03a555d379ac4b0b3130f661407e0 Co-authored-by: Jesus Vazquez <jesus.vazquez@grafana.com> Co-authored-by: Ganesh Vernekar <ganeshvern@gmail.com> Co-authored-by: Dieter Plaetinck <dieter@grafana.com> Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * gofumpt files Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Add license header to missing files Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Fix OOO tests due to existing chunk disk mapper implementation Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Fix truncate int overflow Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Add Sync method to the WAL and update tests Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * remove useless sync Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Update minOOOTime after truncating Head * Update minOOOTime after truncating Head Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Fix lint Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Add a unit test Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Load OutOfOrderTimeWindow only once per appender Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Fix OOO Head LabelValues and PostingsForMatchers Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Fix replay of OOO mmap chunks Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Remove unnecessary err check Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Prevent panic with ApplyConfig Signed-off-by: Ganesh Vernekar 15064823+codesome@users.noreply.github.com Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Run OOO compaction after restart if there is OOO data from WBL Signed-off-by: Ganesh Vernekar 15064823+codesome@users.noreply.github.com Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Apply Bartek's suggestions Co-authored-by: Bartlomiej Plotka <bwplotka@gmail.com> Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Refactor OOO compaction Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Address comments and TODOs - Added a comment explaining why we need the allow overlapping compaction toggle - Clarified TSDBConfig OutOfOrderTimeWindow doc - Added an owner to all the TODOs in the code Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Run go format Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Fix remaining review comments Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Fix tests Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Change wbl reference when truncating ooo in TestHeadMinOOOTimeUpdate Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Fix TestWBLAndMmapReplay test failure on windows Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Address most of the feedback Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Refactor the block meta for out of order Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Fix windows error Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Fix review comments Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> Signed-off-by: Ganesh Vernekar 15064823+codesome@users.noreply.github.com Co-authored-by: Ganesh Vernekar <15064823+codesome@users.noreply.github.com> Co-authored-by: Ganesh Vernekar <ganeshvern@gmail.com> Co-authored-by: Dieter Plaetinck <dieter@grafana.com> Co-authored-by: Oleg Zaytsev <mail@olegzaytsev.com> Co-authored-by: Bartlomiej Plotka <bwplotka@gmail.com>
2022-09-20 10:05:50 -07:00
func (h *Head) resetSeriesWithMMappedChunks(mSeries *memSeries, mmc, oooMmc []*mmappedChunk, walSeriesRef chunks.HeadSeriesRef) (overlapped bool) {
if mSeries.ref != walSeriesRef {
// Checking if the new m-mapped chunks overlap with the already existing ones.
if len(mSeries.mmappedChunks) > 0 && len(mmc) > 0 {
if overlapsClosedInterval(
mSeries.mmappedChunks[0].minTime,
mSeries.mmappedChunks[len(mSeries.mmappedChunks)-1].maxTime,
mmc[0].minTime,
mmc[len(mmc)-1].maxTime,
) {
level.Debug(h.logger).Log(
"msg", "M-mapped chunks overlap on a duplicate series record",
"series", mSeries.lset.String(),
"oldref", mSeries.ref,
"oldmint", mSeries.mmappedChunks[0].minTime,
"oldmaxt", mSeries.mmappedChunks[len(mSeries.mmappedChunks)-1].maxTime,
"newref", walSeriesRef,
"newmint", mmc[0].minTime,
"newmaxt", mmc[len(mmc)-1].maxTime,
)
overlapped = true
}
}
}
Add out-of-order sample support to the TSDB (#11075) * Introduce out-of-order TSDB support This implementation is based on this design doc: https://docs.google.com/document/d/1Kppm7qL9C-BJB1j6yb6-9ObG3AbdZnFUBYPNNWwDBYM/edit?usp=sharing This commit adds support to accept out-of-order ("OOO") sample into the TSDB up to a configurable time allowance. If OOO is enabled, overlapping querying are automatically enabled. Most of the additions have been borrowed from https://github.com/grafana/mimir-prometheus/ Here is the list ist of the original commits cherry picked from mimir-prometheus into this branch: - 4b2198d7ec47d50989b7c2df66b7b207c32f7f6e - 2836e5513f1bc591535a859f5d41154a75e7c6bc - 00b379c3a5b1ec3799699b6242f300a2b3ea30f0 - ff0dc757587cada63ca948d2d5eb00bf090d63e0 - a632c73352a7e39d60b445700beb47d691549c3e - c6f3d4ab339ab80bbbce74c9946237ced01f0509 - 5e8406a1d4a50d0052bbee83e28ca3b3371408aa - abde1e0ba128936b9eb0224ee1551e56216ebd4a - e70e7698897bb03860bee0467c733fa44e14c9bd - df59320886e03a555d379ac4b0b3130f661407e0 Co-authored-by: Jesus Vazquez <jesus.vazquez@grafana.com> Co-authored-by: Ganesh Vernekar <ganeshvern@gmail.com> Co-authored-by: Dieter Plaetinck <dieter@grafana.com> Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * gofumpt files Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Add license header to missing files Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Fix OOO tests due to existing chunk disk mapper implementation Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Fix truncate int overflow Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Add Sync method to the WAL and update tests Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * remove useless sync Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Update minOOOTime after truncating Head * Update minOOOTime after truncating Head Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Fix lint Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Add a unit test Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Load OutOfOrderTimeWindow only once per appender Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Fix OOO Head LabelValues and PostingsForMatchers Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Fix replay of OOO mmap chunks Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Remove unnecessary err check Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Prevent panic with ApplyConfig Signed-off-by: Ganesh Vernekar 15064823+codesome@users.noreply.github.com Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Run OOO compaction after restart if there is OOO data from WBL Signed-off-by: Ganesh Vernekar 15064823+codesome@users.noreply.github.com Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Apply Bartek's suggestions Co-authored-by: Bartlomiej Plotka <bwplotka@gmail.com> Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Refactor OOO compaction Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Address comments and TODOs - Added a comment explaining why we need the allow overlapping compaction toggle - Clarified TSDBConfig OutOfOrderTimeWindow doc - Added an owner to all the TODOs in the code Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Run go format Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Fix remaining review comments Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Fix tests Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Change wbl reference when truncating ooo in TestHeadMinOOOTimeUpdate Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Fix TestWBLAndMmapReplay test failure on windows Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Address most of the feedback Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Refactor the block meta for out of order Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Fix windows error Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Fix review comments Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> Signed-off-by: Ganesh Vernekar 15064823+codesome@users.noreply.github.com Co-authored-by: Ganesh Vernekar <15064823+codesome@users.noreply.github.com> Co-authored-by: Ganesh Vernekar <ganeshvern@gmail.com> Co-authored-by: Dieter Plaetinck <dieter@grafana.com> Co-authored-by: Oleg Zaytsev <mail@olegzaytsev.com> Co-authored-by: Bartlomiej Plotka <bwplotka@gmail.com>
2022-09-20 10:05:50 -07:00
h.metrics.chunksCreated.Add(float64(len(mmc) + len(oooMmc)))
Sync sparsehistogram branch with main (#9189) * Fix `kuma_sd` targetgroup reporting (#9157) * Bundle all xDS targets into a single group Signed-off-by: austin ce <austin.cawley@gmail.com> * Snapshot in-memory chunks on shutdown for faster restarts (#7229) Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Rename links Signed-off-by: Levi Harrison <git@leviharrison.dev> * Remove Individual Data Type Caps in Per-shard Buffering for Remote Write (#8921) * Moved everything to nPending buffer Signed-off-by: Levi Harrison <git@leviharrison.dev> * Simplify exemplar capacity addition Signed-off-by: Levi Harrison <git@leviharrison.dev> * Added pre-allocation Signed-off-by: Levi Harrison <git@leviharrison.dev> * Don't allocate if not sending exemplars Signed-off-by: Levi Harrison <git@leviharrison.dev> * Avoid deadlock when processing duplicate series record (#9170) * Avoid deadlock when processing duplicate series record `processWALSamples()` needs to be able to send on its output channel before it can read the input channel, so reads to allow this in case the output channel is full. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * processWALSamples: update comment Previous text seems to relate to an earlier implementation. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Optimise WAL loading by removing extra map and caching min-time (#9160) * BenchmarkLoadWAL: close WAL after use So that goroutines are stopped and resources released Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * BenchmarkLoadWAL: make series IDs co-prime with #workers Series are distributed across workers by taking the modulus of the ID with the number of workers, so multiples of 100 are a poor choice. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * BenchmarkLoadWAL: simulate mmapped chunks Real Prometheus cuts chunks every 120 samples, then skips those samples when re-reading the WAL. Simulate this by creating a single mapped chunk for each series, since the max time is all the reader looks at. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Fix comment Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Remove series map from processWALSamples() The locks that is commented to reduce contention in are now sharded 32,000 ways, so won't be contended. Removing the map saves memory and goes just as fast. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * loadWAL: Cache the last mmapped chunk time So we can skip calling append() for samples it will reject. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Improvements from code review Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Full stops and capitals on comments Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Cache max time in both places mmappedChunks is updated Including refactor to extract function `setMMappedChunks`, to reduce code duplication. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Update head min/max time when mmapped chunks added This ensures we have the correct values if no WAL samples are added for that series. Note that `mSeries.maxTime()` was always `math.MinInt64` before, since that function doesn't consider mmapped chunks. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Split Go and React Tests (#8897) * Added go-ci and react-ci Co-authored-by: Julien Pivotto <roidelapluie@inuits.eu> Signed-off-by: Levi Harrison <git@leviharrison.dev> * Remove search keymap from new expression editor (#9184) Signed-off-by: Julius Volz <julius.volz@gmail.com> Co-authored-by: Austin Cawley-Edwards <austin.cawley@gmail.com> Co-authored-by: Levi Harrison <git@leviharrison.dev> Co-authored-by: Julien Pivotto <roidelapluie@inuits.eu> Co-authored-by: Bryan Boreham <bjboreham@gmail.com> Co-authored-by: Julius Volz <julius.volz@gmail.com>
2021-08-11 03:13:17 -07:00
h.metrics.chunksRemoved.Add(float64(len(mSeries.mmappedChunks)))
Add out-of-order sample support to the TSDB (#11075) * Introduce out-of-order TSDB support This implementation is based on this design doc: https://docs.google.com/document/d/1Kppm7qL9C-BJB1j6yb6-9ObG3AbdZnFUBYPNNWwDBYM/edit?usp=sharing This commit adds support to accept out-of-order ("OOO") sample into the TSDB up to a configurable time allowance. If OOO is enabled, overlapping querying are automatically enabled. Most of the additions have been borrowed from https://github.com/grafana/mimir-prometheus/ Here is the list ist of the original commits cherry picked from mimir-prometheus into this branch: - 4b2198d7ec47d50989b7c2df66b7b207c32f7f6e - 2836e5513f1bc591535a859f5d41154a75e7c6bc - 00b379c3a5b1ec3799699b6242f300a2b3ea30f0 - ff0dc757587cada63ca948d2d5eb00bf090d63e0 - a632c73352a7e39d60b445700beb47d691549c3e - c6f3d4ab339ab80bbbce74c9946237ced01f0509 - 5e8406a1d4a50d0052bbee83e28ca3b3371408aa - abde1e0ba128936b9eb0224ee1551e56216ebd4a - e70e7698897bb03860bee0467c733fa44e14c9bd - df59320886e03a555d379ac4b0b3130f661407e0 Co-authored-by: Jesus Vazquez <jesus.vazquez@grafana.com> Co-authored-by: Ganesh Vernekar <ganeshvern@gmail.com> Co-authored-by: Dieter Plaetinck <dieter@grafana.com> Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * gofumpt files Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Add license header to missing files Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Fix OOO tests due to existing chunk disk mapper implementation Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Fix truncate int overflow Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Add Sync method to the WAL and update tests Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * remove useless sync Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Update minOOOTime after truncating Head * Update minOOOTime after truncating Head Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Fix lint Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Add a unit test Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Load OutOfOrderTimeWindow only once per appender Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Fix OOO Head LabelValues and PostingsForMatchers Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Fix replay of OOO mmap chunks Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Remove unnecessary err check Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Prevent panic with ApplyConfig Signed-off-by: Ganesh Vernekar 15064823+codesome@users.noreply.github.com Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Run OOO compaction after restart if there is OOO data from WBL Signed-off-by: Ganesh Vernekar 15064823+codesome@users.noreply.github.com Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Apply Bartek's suggestions Co-authored-by: Bartlomiej Plotka <bwplotka@gmail.com> Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Refactor OOO compaction Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Address comments and TODOs - Added a comment explaining why we need the allow overlapping compaction toggle - Clarified TSDBConfig OutOfOrderTimeWindow doc - Added an owner to all the TODOs in the code Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Run go format Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Fix remaining review comments Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Fix tests Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Change wbl reference when truncating ooo in TestHeadMinOOOTimeUpdate Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Fix TestWBLAndMmapReplay test failure on windows Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Address most of the feedback Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Refactor the block meta for out of order Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Fix windows error Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Fix review comments Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> Signed-off-by: Ganesh Vernekar 15064823+codesome@users.noreply.github.com Co-authored-by: Ganesh Vernekar <15064823+codesome@users.noreply.github.com> Co-authored-by: Ganesh Vernekar <ganeshvern@gmail.com> Co-authored-by: Dieter Plaetinck <dieter@grafana.com> Co-authored-by: Oleg Zaytsev <mail@olegzaytsev.com> Co-authored-by: Bartlomiej Plotka <bwplotka@gmail.com>
2022-09-20 10:05:50 -07:00
h.metrics.chunks.Add(float64(len(mmc) + len(oooMmc) - len(mSeries.mmappedChunks)))
Sync sparsehistogram branch with main (#9189) * Fix `kuma_sd` targetgroup reporting (#9157) * Bundle all xDS targets into a single group Signed-off-by: austin ce <austin.cawley@gmail.com> * Snapshot in-memory chunks on shutdown for faster restarts (#7229) Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Rename links Signed-off-by: Levi Harrison <git@leviharrison.dev> * Remove Individual Data Type Caps in Per-shard Buffering for Remote Write (#8921) * Moved everything to nPending buffer Signed-off-by: Levi Harrison <git@leviharrison.dev> * Simplify exemplar capacity addition Signed-off-by: Levi Harrison <git@leviharrison.dev> * Added pre-allocation Signed-off-by: Levi Harrison <git@leviharrison.dev> * Don't allocate if not sending exemplars Signed-off-by: Levi Harrison <git@leviharrison.dev> * Avoid deadlock when processing duplicate series record (#9170) * Avoid deadlock when processing duplicate series record `processWALSamples()` needs to be able to send on its output channel before it can read the input channel, so reads to allow this in case the output channel is full. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * processWALSamples: update comment Previous text seems to relate to an earlier implementation. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Optimise WAL loading by removing extra map and caching min-time (#9160) * BenchmarkLoadWAL: close WAL after use So that goroutines are stopped and resources released Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * BenchmarkLoadWAL: make series IDs co-prime with #workers Series are distributed across workers by taking the modulus of the ID with the number of workers, so multiples of 100 are a poor choice. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * BenchmarkLoadWAL: simulate mmapped chunks Real Prometheus cuts chunks every 120 samples, then skips those samples when re-reading the WAL. Simulate this by creating a single mapped chunk for each series, since the max time is all the reader looks at. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Fix comment Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Remove series map from processWALSamples() The locks that is commented to reduce contention in are now sharded 32,000 ways, so won't be contended. Removing the map saves memory and goes just as fast. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * loadWAL: Cache the last mmapped chunk time So we can skip calling append() for samples it will reject. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Improvements from code review Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Full stops and capitals on comments Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Cache max time in both places mmappedChunks is updated Including refactor to extract function `setMMappedChunks`, to reduce code duplication. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Update head min/max time when mmapped chunks added This ensures we have the correct values if no WAL samples are added for that series. Note that `mSeries.maxTime()` was always `math.MinInt64` before, since that function doesn't consider mmapped chunks. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Split Go and React Tests (#8897) * Added go-ci and react-ci Co-authored-by: Julien Pivotto <roidelapluie@inuits.eu> Signed-off-by: Levi Harrison <git@leviharrison.dev> * Remove search keymap from new expression editor (#9184) Signed-off-by: Julius Volz <julius.volz@gmail.com> Co-authored-by: Austin Cawley-Edwards <austin.cawley@gmail.com> Co-authored-by: Levi Harrison <git@leviharrison.dev> Co-authored-by: Julien Pivotto <roidelapluie@inuits.eu> Co-authored-by: Bryan Boreham <bjboreham@gmail.com> Co-authored-by: Julius Volz <julius.volz@gmail.com>
2021-08-11 03:13:17 -07:00
mSeries.mmappedChunks = mmc
Add out-of-order sample support to the TSDB (#11075) * Introduce out-of-order TSDB support This implementation is based on this design doc: https://docs.google.com/document/d/1Kppm7qL9C-BJB1j6yb6-9ObG3AbdZnFUBYPNNWwDBYM/edit?usp=sharing This commit adds support to accept out-of-order ("OOO") sample into the TSDB up to a configurable time allowance. If OOO is enabled, overlapping querying are automatically enabled. Most of the additions have been borrowed from https://github.com/grafana/mimir-prometheus/ Here is the list ist of the original commits cherry picked from mimir-prometheus into this branch: - 4b2198d7ec47d50989b7c2df66b7b207c32f7f6e - 2836e5513f1bc591535a859f5d41154a75e7c6bc - 00b379c3a5b1ec3799699b6242f300a2b3ea30f0 - ff0dc757587cada63ca948d2d5eb00bf090d63e0 - a632c73352a7e39d60b445700beb47d691549c3e - c6f3d4ab339ab80bbbce74c9946237ced01f0509 - 5e8406a1d4a50d0052bbee83e28ca3b3371408aa - abde1e0ba128936b9eb0224ee1551e56216ebd4a - e70e7698897bb03860bee0467c733fa44e14c9bd - df59320886e03a555d379ac4b0b3130f661407e0 Co-authored-by: Jesus Vazquez <jesus.vazquez@grafana.com> Co-authored-by: Ganesh Vernekar <ganeshvern@gmail.com> Co-authored-by: Dieter Plaetinck <dieter@grafana.com> Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * gofumpt files Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Add license header to missing files Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Fix OOO tests due to existing chunk disk mapper implementation Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Fix truncate int overflow Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Add Sync method to the WAL and update tests Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * remove useless sync Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Update minOOOTime after truncating Head * Update minOOOTime after truncating Head Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Fix lint Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Add a unit test Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Load OutOfOrderTimeWindow only once per appender Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Fix OOO Head LabelValues and PostingsForMatchers Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Fix replay of OOO mmap chunks Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Remove unnecessary err check Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Prevent panic with ApplyConfig Signed-off-by: Ganesh Vernekar 15064823+codesome@users.noreply.github.com Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Run OOO compaction after restart if there is OOO data from WBL Signed-off-by: Ganesh Vernekar 15064823+codesome@users.noreply.github.com Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Apply Bartek's suggestions Co-authored-by: Bartlomiej Plotka <bwplotka@gmail.com> Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Refactor OOO compaction Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Address comments and TODOs - Added a comment explaining why we need the allow overlapping compaction toggle - Clarified TSDBConfig OutOfOrderTimeWindow doc - Added an owner to all the TODOs in the code Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Run go format Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Fix remaining review comments Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Fix tests Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Change wbl reference when truncating ooo in TestHeadMinOOOTimeUpdate Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Fix TestWBLAndMmapReplay test failure on windows Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Address most of the feedback Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Refactor the block meta for out of order Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Fix windows error Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Fix review comments Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> Signed-off-by: Ganesh Vernekar 15064823+codesome@users.noreply.github.com Co-authored-by: Ganesh Vernekar <15064823+codesome@users.noreply.github.com> Co-authored-by: Ganesh Vernekar <ganeshvern@gmail.com> Co-authored-by: Dieter Plaetinck <dieter@grafana.com> Co-authored-by: Oleg Zaytsev <mail@olegzaytsev.com> Co-authored-by: Bartlomiej Plotka <bwplotka@gmail.com>
2022-09-20 10:05:50 -07:00
mSeries.oooMmappedChunks = oooMmc
Sync sparsehistogram branch with main (#9189) * Fix `kuma_sd` targetgroup reporting (#9157) * Bundle all xDS targets into a single group Signed-off-by: austin ce <austin.cawley@gmail.com> * Snapshot in-memory chunks on shutdown for faster restarts (#7229) Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Rename links Signed-off-by: Levi Harrison <git@leviharrison.dev> * Remove Individual Data Type Caps in Per-shard Buffering for Remote Write (#8921) * Moved everything to nPending buffer Signed-off-by: Levi Harrison <git@leviharrison.dev> * Simplify exemplar capacity addition Signed-off-by: Levi Harrison <git@leviharrison.dev> * Added pre-allocation Signed-off-by: Levi Harrison <git@leviharrison.dev> * Don't allocate if not sending exemplars Signed-off-by: Levi Harrison <git@leviharrison.dev> * Avoid deadlock when processing duplicate series record (#9170) * Avoid deadlock when processing duplicate series record `processWALSamples()` needs to be able to send on its output channel before it can read the input channel, so reads to allow this in case the output channel is full. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * processWALSamples: update comment Previous text seems to relate to an earlier implementation. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Optimise WAL loading by removing extra map and caching min-time (#9160) * BenchmarkLoadWAL: close WAL after use So that goroutines are stopped and resources released Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * BenchmarkLoadWAL: make series IDs co-prime with #workers Series are distributed across workers by taking the modulus of the ID with the number of workers, so multiples of 100 are a poor choice. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * BenchmarkLoadWAL: simulate mmapped chunks Real Prometheus cuts chunks every 120 samples, then skips those samples when re-reading the WAL. Simulate this by creating a single mapped chunk for each series, since the max time is all the reader looks at. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Fix comment Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Remove series map from processWALSamples() The locks that is commented to reduce contention in are now sharded 32,000 ways, so won't be contended. Removing the map saves memory and goes just as fast. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * loadWAL: Cache the last mmapped chunk time So we can skip calling append() for samples it will reject. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Improvements from code review Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Full stops and capitals on comments Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Cache max time in both places mmappedChunks is updated Including refactor to extract function `setMMappedChunks`, to reduce code duplication. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Update head min/max time when mmapped chunks added This ensures we have the correct values if no WAL samples are added for that series. Note that `mSeries.maxTime()` was always `math.MinInt64` before, since that function doesn't consider mmapped chunks. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Split Go and React Tests (#8897) * Added go-ci and react-ci Co-authored-by: Julien Pivotto <roidelapluie@inuits.eu> Signed-off-by: Levi Harrison <git@leviharrison.dev> * Remove search keymap from new expression editor (#9184) Signed-off-by: Julius Volz <julius.volz@gmail.com> Co-authored-by: Austin Cawley-Edwards <austin.cawley@gmail.com> Co-authored-by: Levi Harrison <git@leviharrison.dev> Co-authored-by: Julien Pivotto <roidelapluie@inuits.eu> Co-authored-by: Bryan Boreham <bjboreham@gmail.com> Co-authored-by: Julius Volz <julius.volz@gmail.com>
2021-08-11 03:13:17 -07:00
// Cache the last mmapped chunk time, so we can skip calling append() for samples it will reject.
if len(mmc) == 0 {
mSeries.mmMaxTime = math.MinInt64
} else {
mSeries.mmMaxTime = mmc[len(mmc)-1].maxTime
h.updateMinMaxTime(mmc[0].minTime, mSeries.mmMaxTime)
}
Add out-of-order sample support to the TSDB (#11075) * Introduce out-of-order TSDB support This implementation is based on this design doc: https://docs.google.com/document/d/1Kppm7qL9C-BJB1j6yb6-9ObG3AbdZnFUBYPNNWwDBYM/edit?usp=sharing This commit adds support to accept out-of-order ("OOO") sample into the TSDB up to a configurable time allowance. If OOO is enabled, overlapping querying are automatically enabled. Most of the additions have been borrowed from https://github.com/grafana/mimir-prometheus/ Here is the list ist of the original commits cherry picked from mimir-prometheus into this branch: - 4b2198d7ec47d50989b7c2df66b7b207c32f7f6e - 2836e5513f1bc591535a859f5d41154a75e7c6bc - 00b379c3a5b1ec3799699b6242f300a2b3ea30f0 - ff0dc757587cada63ca948d2d5eb00bf090d63e0 - a632c73352a7e39d60b445700beb47d691549c3e - c6f3d4ab339ab80bbbce74c9946237ced01f0509 - 5e8406a1d4a50d0052bbee83e28ca3b3371408aa - abde1e0ba128936b9eb0224ee1551e56216ebd4a - e70e7698897bb03860bee0467c733fa44e14c9bd - df59320886e03a555d379ac4b0b3130f661407e0 Co-authored-by: Jesus Vazquez <jesus.vazquez@grafana.com> Co-authored-by: Ganesh Vernekar <ganeshvern@gmail.com> Co-authored-by: Dieter Plaetinck <dieter@grafana.com> Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * gofumpt files Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Add license header to missing files Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Fix OOO tests due to existing chunk disk mapper implementation Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Fix truncate int overflow Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Add Sync method to the WAL and update tests Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * remove useless sync Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Update minOOOTime after truncating Head * Update minOOOTime after truncating Head Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Fix lint Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Add a unit test Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Load OutOfOrderTimeWindow only once per appender Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Fix OOO Head LabelValues and PostingsForMatchers Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Fix replay of OOO mmap chunks Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Remove unnecessary err check Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Prevent panic with ApplyConfig Signed-off-by: Ganesh Vernekar 15064823+codesome@users.noreply.github.com Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Run OOO compaction after restart if there is OOO data from WBL Signed-off-by: Ganesh Vernekar 15064823+codesome@users.noreply.github.com Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Apply Bartek's suggestions Co-authored-by: Bartlomiej Plotka <bwplotka@gmail.com> Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Refactor OOO compaction Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Address comments and TODOs - Added a comment explaining why we need the allow overlapping compaction toggle - Clarified TSDBConfig OutOfOrderTimeWindow doc - Added an owner to all the TODOs in the code Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Run go format Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Fix remaining review comments Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Fix tests Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Change wbl reference when truncating ooo in TestHeadMinOOOTimeUpdate Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Fix TestWBLAndMmapReplay test failure on windows Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Address most of the feedback Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Refactor the block meta for out of order Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Fix windows error Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Fix review comments Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> Signed-off-by: Ganesh Vernekar 15064823+codesome@users.noreply.github.com Co-authored-by: Ganesh Vernekar <15064823+codesome@users.noreply.github.com> Co-authored-by: Ganesh Vernekar <ganeshvern@gmail.com> Co-authored-by: Dieter Plaetinck <dieter@grafana.com> Co-authored-by: Oleg Zaytsev <mail@olegzaytsev.com> Co-authored-by: Bartlomiej Plotka <bwplotka@gmail.com>
2022-09-20 10:05:50 -07:00
if len(oooMmc) != 0 {
// Mint and maxt can be in any chunk, they are not sorted.
mint, maxt := int64(math.MaxInt64), int64(math.MinInt64)
for _, ch := range oooMmc {
if ch.minTime < mint {
mint = ch.minTime
}
if ch.maxTime > maxt {
maxt = ch.maxTime
}
}
h.updateMinOOOMaxOOOTime(mint, maxt)
}
// Any samples replayed till now would already be compacted. Resetting the head chunk.
mSeries.nextAt = 0
mSeries.headChunk = nil
mSeries.app = nil
return
}
type walSubsetProcessor struct {
input chan walSubsetProcessorInputItem
2021-11-30 08:19:06 -08:00
output chan []record.RefSample
histogramsOutput chan []record.RefHistogramSample
}
type walSubsetProcessorInputItem struct {
samples []record.RefSample
histogramSamples []record.RefHistogramSample
existingSeries *memSeries
walSeriesRef chunks.HeadSeriesRef
}
func (wp *walSubsetProcessor) setup() {
wp.input = make(chan walSubsetProcessorInputItem, 300)
wp.output = make(chan []record.RefSample, 300)
wp.histogramsOutput = make(chan []record.RefHistogramSample, 300)
}
func (wp *walSubsetProcessor) closeAndDrain() {
close(wp.input)
for range wp.output {
}
2021-11-30 08:19:06 -08:00
for range wp.histogramsOutput {
}
}
// If there is a buffer in the output chan, return it for reuse, otherwise return nil.
func (wp *walSubsetProcessor) reuseBuf() []record.RefSample {
select {
case buf := <-wp.output:
return buf[:0]
default:
}
return nil
Optimise WAL loading by removing extra map and caching min-time (#9160) * BenchmarkLoadWAL: close WAL after use So that goroutines are stopped and resources released Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * BenchmarkLoadWAL: make series IDs co-prime with #workers Series are distributed across workers by taking the modulus of the ID with the number of workers, so multiples of 100 are a poor choice. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * BenchmarkLoadWAL: simulate mmapped chunks Real Prometheus cuts chunks every 120 samples, then skips those samples when re-reading the WAL. Simulate this by creating a single mapped chunk for each series, since the max time is all the reader looks at. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Fix comment Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Remove series map from processWALSamples() The locks that is commented to reduce contention in are now sharded 32,000 ways, so won't be contended. Removing the map saves memory and goes just as fast. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * loadWAL: Cache the last mmapped chunk time So we can skip calling append() for samples it will reject. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Improvements from code review Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Full stops and capitals on comments Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Cache max time in both places mmappedChunks is updated Including refactor to extract function `setMMappedChunks`, to reduce code duplication. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Update head min/max time when mmapped chunks added This ensures we have the correct values if no WAL samples are added for that series. Note that `mSeries.maxTime()` was always `math.MinInt64` before, since that function doesn't consider mmapped chunks. Signed-off-by: Bryan Boreham <bjboreham@gmail.com>
2021-08-10 02:23:31 -07:00
}
2021-11-30 08:19:06 -08:00
// If there is a buffer in the output chan, return it for reuse, otherwise return nil.
func (wp *walSubsetProcessor) reuseHistogramBuf() []record.RefHistogramSample {
2021-11-30 08:19:06 -08:00
select {
case buf := <-wp.histogramsOutput:
return buf[:0]
default:
}
return nil
Sync sparsehistogram branch with main (#9189) * Fix `kuma_sd` targetgroup reporting (#9157) * Bundle all xDS targets into a single group Signed-off-by: austin ce <austin.cawley@gmail.com> * Snapshot in-memory chunks on shutdown for faster restarts (#7229) Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Rename links Signed-off-by: Levi Harrison <git@leviharrison.dev> * Remove Individual Data Type Caps in Per-shard Buffering for Remote Write (#8921) * Moved everything to nPending buffer Signed-off-by: Levi Harrison <git@leviharrison.dev> * Simplify exemplar capacity addition Signed-off-by: Levi Harrison <git@leviharrison.dev> * Added pre-allocation Signed-off-by: Levi Harrison <git@leviharrison.dev> * Don't allocate if not sending exemplars Signed-off-by: Levi Harrison <git@leviharrison.dev> * Avoid deadlock when processing duplicate series record (#9170) * Avoid deadlock when processing duplicate series record `processWALSamples()` needs to be able to send on its output channel before it can read the input channel, so reads to allow this in case the output channel is full. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * processWALSamples: update comment Previous text seems to relate to an earlier implementation. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Optimise WAL loading by removing extra map and caching min-time (#9160) * BenchmarkLoadWAL: close WAL after use So that goroutines are stopped and resources released Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * BenchmarkLoadWAL: make series IDs co-prime with #workers Series are distributed across workers by taking the modulus of the ID with the number of workers, so multiples of 100 are a poor choice. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * BenchmarkLoadWAL: simulate mmapped chunks Real Prometheus cuts chunks every 120 samples, then skips those samples when re-reading the WAL. Simulate this by creating a single mapped chunk for each series, since the max time is all the reader looks at. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Fix comment Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Remove series map from processWALSamples() The locks that is commented to reduce contention in are now sharded 32,000 ways, so won't be contended. Removing the map saves memory and goes just as fast. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * loadWAL: Cache the last mmapped chunk time So we can skip calling append() for samples it will reject. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Improvements from code review Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Full stops and capitals on comments Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Cache max time in both places mmappedChunks is updated Including refactor to extract function `setMMappedChunks`, to reduce code duplication. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Update head min/max time when mmapped chunks added This ensures we have the correct values if no WAL samples are added for that series. Note that `mSeries.maxTime()` was always `math.MinInt64` before, since that function doesn't consider mmapped chunks. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Split Go and React Tests (#8897) * Added go-ci and react-ci Co-authored-by: Julien Pivotto <roidelapluie@inuits.eu> Signed-off-by: Levi Harrison <git@leviharrison.dev> * Remove search keymap from new expression editor (#9184) Signed-off-by: Julius Volz <julius.volz@gmail.com> Co-authored-by: Austin Cawley-Edwards <austin.cawley@gmail.com> Co-authored-by: Levi Harrison <git@leviharrison.dev> Co-authored-by: Julien Pivotto <roidelapluie@inuits.eu> Co-authored-by: Bryan Boreham <bjboreham@gmail.com> Co-authored-by: Julius Volz <julius.volz@gmail.com>
2021-08-11 03:13:17 -07:00
}
// processWALSamples adds the samples it receives to the head and passes
// the buffer received to an output channel for reuse.
// Samples before the minValidTime timestamp are discarded.
func (wp *walSubsetProcessor) processWALSamples(h *Head, mmappedChunks, oooMmappedChunks map[chunks.HeadSeriesRef][]*mmappedChunk) (unknownRefs, unknownHistogramRefs, mmapOverlappingChunks uint64) {
defer close(wp.output)
2021-11-30 08:19:06 -08:00
defer close(wp.histogramsOutput)
minValidTime := h.minValidTime.Load()
mint, maxt := int64(math.MaxInt64), int64(math.MinInt64)
chunkRange := h.chunkRange.Load()
for in := range wp.input {
if in.existingSeries != nil {
mmc := mmappedChunks[in.walSeriesRef]
Add out-of-order sample support to the TSDB (#11075) * Introduce out-of-order TSDB support This implementation is based on this design doc: https://docs.google.com/document/d/1Kppm7qL9C-BJB1j6yb6-9ObG3AbdZnFUBYPNNWwDBYM/edit?usp=sharing This commit adds support to accept out-of-order ("OOO") sample into the TSDB up to a configurable time allowance. If OOO is enabled, overlapping querying are automatically enabled. Most of the additions have been borrowed from https://github.com/grafana/mimir-prometheus/ Here is the list ist of the original commits cherry picked from mimir-prometheus into this branch: - 4b2198d7ec47d50989b7c2df66b7b207c32f7f6e - 2836e5513f1bc591535a859f5d41154a75e7c6bc - 00b379c3a5b1ec3799699b6242f300a2b3ea30f0 - ff0dc757587cada63ca948d2d5eb00bf090d63e0 - a632c73352a7e39d60b445700beb47d691549c3e - c6f3d4ab339ab80bbbce74c9946237ced01f0509 - 5e8406a1d4a50d0052bbee83e28ca3b3371408aa - abde1e0ba128936b9eb0224ee1551e56216ebd4a - e70e7698897bb03860bee0467c733fa44e14c9bd - df59320886e03a555d379ac4b0b3130f661407e0 Co-authored-by: Jesus Vazquez <jesus.vazquez@grafana.com> Co-authored-by: Ganesh Vernekar <ganeshvern@gmail.com> Co-authored-by: Dieter Plaetinck <dieter@grafana.com> Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * gofumpt files Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Add license header to missing files Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Fix OOO tests due to existing chunk disk mapper implementation Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Fix truncate int overflow Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Add Sync method to the WAL and update tests Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * remove useless sync Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Update minOOOTime after truncating Head * Update minOOOTime after truncating Head Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Fix lint Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Add a unit test Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Load OutOfOrderTimeWindow only once per appender Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Fix OOO Head LabelValues and PostingsForMatchers Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Fix replay of OOO mmap chunks Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Remove unnecessary err check Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Prevent panic with ApplyConfig Signed-off-by: Ganesh Vernekar 15064823+codesome@users.noreply.github.com Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Run OOO compaction after restart if there is OOO data from WBL Signed-off-by: Ganesh Vernekar 15064823+codesome@users.noreply.github.com Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Apply Bartek's suggestions Co-authored-by: Bartlomiej Plotka <bwplotka@gmail.com> Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Refactor OOO compaction Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Address comments and TODOs - Added a comment explaining why we need the allow overlapping compaction toggle - Clarified TSDBConfig OutOfOrderTimeWindow doc - Added an owner to all the TODOs in the code Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Run go format Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Fix remaining review comments Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Fix tests Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Change wbl reference when truncating ooo in TestHeadMinOOOTimeUpdate Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Fix TestWBLAndMmapReplay test failure on windows Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Address most of the feedback Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Refactor the block meta for out of order Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Fix windows error Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Fix review comments Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> Signed-off-by: Ganesh Vernekar 15064823+codesome@users.noreply.github.com Co-authored-by: Ganesh Vernekar <15064823+codesome@users.noreply.github.com> Co-authored-by: Ganesh Vernekar <ganeshvern@gmail.com> Co-authored-by: Dieter Plaetinck <dieter@grafana.com> Co-authored-by: Oleg Zaytsev <mail@olegzaytsev.com> Co-authored-by: Bartlomiej Plotka <bwplotka@gmail.com>
2022-09-20 10:05:50 -07:00
oooMmc := oooMmappedChunks[in.walSeriesRef]
if h.resetSeriesWithMMappedChunks(in.existingSeries, mmc, oooMmc, in.walSeriesRef) {
mmapOverlappingChunks++
}
continue
}
for _, s := range in.samples {
Optimise WAL loading by removing extra map and caching min-time (#9160) * BenchmarkLoadWAL: close WAL after use So that goroutines are stopped and resources released Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * BenchmarkLoadWAL: make series IDs co-prime with #workers Series are distributed across workers by taking the modulus of the ID with the number of workers, so multiples of 100 are a poor choice. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * BenchmarkLoadWAL: simulate mmapped chunks Real Prometheus cuts chunks every 120 samples, then skips those samples when re-reading the WAL. Simulate this by creating a single mapped chunk for each series, since the max time is all the reader looks at. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Fix comment Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Remove series map from processWALSamples() The locks that is commented to reduce contention in are now sharded 32,000 ways, so won't be contended. Removing the map saves memory and goes just as fast. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * loadWAL: Cache the last mmapped chunk time So we can skip calling append() for samples it will reject. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Improvements from code review Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Full stops and capitals on comments Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Cache max time in both places mmappedChunks is updated Including refactor to extract function `setMMappedChunks`, to reduce code duplication. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Update head min/max time when mmapped chunks added This ensures we have the correct values if no WAL samples are added for that series. Note that `mSeries.maxTime()` was always `math.MinInt64` before, since that function doesn't consider mmapped chunks. Signed-off-by: Bryan Boreham <bjboreham@gmail.com>
2021-08-10 02:23:31 -07:00
ms := h.series.getByID(s.Ref)
if ms == nil {
Optimise WAL loading by removing extra map and caching min-time (#9160) * BenchmarkLoadWAL: close WAL after use So that goroutines are stopped and resources released Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * BenchmarkLoadWAL: make series IDs co-prime with #workers Series are distributed across workers by taking the modulus of the ID with the number of workers, so multiples of 100 are a poor choice. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * BenchmarkLoadWAL: simulate mmapped chunks Real Prometheus cuts chunks every 120 samples, then skips those samples when re-reading the WAL. Simulate this by creating a single mapped chunk for each series, since the max time is all the reader looks at. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Fix comment Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Remove series map from processWALSamples() The locks that is commented to reduce contention in are now sharded 32,000 ways, so won't be contended. Removing the map saves memory and goes just as fast. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * loadWAL: Cache the last mmapped chunk time So we can skip calling append() for samples it will reject. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Improvements from code review Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Full stops and capitals on comments Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Cache max time in both places mmappedChunks is updated Including refactor to extract function `setMMappedChunks`, to reduce code duplication. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Update head min/max time when mmapped chunks added This ensures we have the correct values if no WAL samples are added for that series. Note that `mSeries.maxTime()` was always `math.MinInt64` before, since that function doesn't consider mmapped chunks. Signed-off-by: Bryan Boreham <bjboreham@gmail.com>
2021-08-10 02:23:31 -07:00
unknownRefs++
continue
}
if s.T <= ms.mmMaxTime {
continue
}
ms.isHistogramSeries = false
if s.T <= ms.mmMaxTime {
continue
}
if _, chunkCreated := ms.append(s.T, s.V, 0, h.chunkDiskMapper, chunkRange); chunkCreated {
h.metrics.chunksCreated.Inc()
h.metrics.chunks.Inc()
}
if s.T > maxt {
maxt = s.T
}
if s.T < mint {
mint = s.T
}
}
select {
case wp.output <- in.samples:
default:
}
for _, s := range in.histogramSamples {
if s.T < minValidTime {
continue
}
ms := h.series.getByID(s.Ref)
if ms == nil {
unknownHistogramRefs++
continue
}
ms.isHistogramSeries = true
if s.T <= ms.mmMaxTime {
continue
}
if _, chunkCreated := ms.appendHistogram(s.T, s.H, 0, h.chunkDiskMapper, chunkRange); chunkCreated {
h.metrics.chunksCreated.Inc()
h.metrics.chunks.Inc()
}
if s.T > maxt {
maxt = s.T
}
if s.T < mint {
mint = s.T
}
}
select {
case wp.histogramsOutput <- in.histogramSamples:
default:
}
}
h.updateMinMaxTime(mint, maxt)
return unknownRefs, unknownHistogramRefs, mmapOverlappingChunks
}
func (h *Head) loadWBL(r *wlog.Reader, multiRef map[chunks.HeadSeriesRef]chunks.HeadSeriesRef, lastMmapRef chunks.ChunkDiskMapperRef) (err error) {
Add out-of-order sample support to the TSDB (#11075) * Introduce out-of-order TSDB support This implementation is based on this design doc: https://docs.google.com/document/d/1Kppm7qL9C-BJB1j6yb6-9ObG3AbdZnFUBYPNNWwDBYM/edit?usp=sharing This commit adds support to accept out-of-order ("OOO") sample into the TSDB up to a configurable time allowance. If OOO is enabled, overlapping querying are automatically enabled. Most of the additions have been borrowed from https://github.com/grafana/mimir-prometheus/ Here is the list ist of the original commits cherry picked from mimir-prometheus into this branch: - 4b2198d7ec47d50989b7c2df66b7b207c32f7f6e - 2836e5513f1bc591535a859f5d41154a75e7c6bc - 00b379c3a5b1ec3799699b6242f300a2b3ea30f0 - ff0dc757587cada63ca948d2d5eb00bf090d63e0 - a632c73352a7e39d60b445700beb47d691549c3e - c6f3d4ab339ab80bbbce74c9946237ced01f0509 - 5e8406a1d4a50d0052bbee83e28ca3b3371408aa - abde1e0ba128936b9eb0224ee1551e56216ebd4a - e70e7698897bb03860bee0467c733fa44e14c9bd - df59320886e03a555d379ac4b0b3130f661407e0 Co-authored-by: Jesus Vazquez <jesus.vazquez@grafana.com> Co-authored-by: Ganesh Vernekar <ganeshvern@gmail.com> Co-authored-by: Dieter Plaetinck <dieter@grafana.com> Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * gofumpt files Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Add license header to missing files Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Fix OOO tests due to existing chunk disk mapper implementation Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Fix truncate int overflow Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Add Sync method to the WAL and update tests Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * remove useless sync Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Update minOOOTime after truncating Head * Update minOOOTime after truncating Head Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Fix lint Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Add a unit test Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Load OutOfOrderTimeWindow only once per appender Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Fix OOO Head LabelValues and PostingsForMatchers Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Fix replay of OOO mmap chunks Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Remove unnecessary err check Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Prevent panic with ApplyConfig Signed-off-by: Ganesh Vernekar 15064823+codesome@users.noreply.github.com Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Run OOO compaction after restart if there is OOO data from WBL Signed-off-by: Ganesh Vernekar 15064823+codesome@users.noreply.github.com Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Apply Bartek's suggestions Co-authored-by: Bartlomiej Plotka <bwplotka@gmail.com> Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Refactor OOO compaction Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Address comments and TODOs - Added a comment explaining why we need the allow overlapping compaction toggle - Clarified TSDBConfig OutOfOrderTimeWindow doc - Added an owner to all the TODOs in the code Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Run go format Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Fix remaining review comments Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Fix tests Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Change wbl reference when truncating ooo in TestHeadMinOOOTimeUpdate Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Fix TestWBLAndMmapReplay test failure on windows Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Address most of the feedback Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Refactor the block meta for out of order Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Fix windows error Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Fix review comments Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> Signed-off-by: Ganesh Vernekar 15064823+codesome@users.noreply.github.com Co-authored-by: Ganesh Vernekar <15064823+codesome@users.noreply.github.com> Co-authored-by: Ganesh Vernekar <ganeshvern@gmail.com> Co-authored-by: Dieter Plaetinck <dieter@grafana.com> Co-authored-by: Oleg Zaytsev <mail@olegzaytsev.com> Co-authored-by: Bartlomiej Plotka <bwplotka@gmail.com>
2022-09-20 10:05:50 -07:00
// Track number of samples, m-map markers, that referenced a series we don't know about
// for error reporting.
var unknownRefs, mmapMarkerUnknownRefs atomic.Uint64
lastSeq, lastOff := lastMmapRef.Unpack()
// Start workers that each process samples for a partition of the series ID space.
var (
wg sync.WaitGroup
n = runtime.GOMAXPROCS(0)
processors = make([]wblSubsetProcessor, n)
dec record.Decoder
shards = make([][]record.RefSample, n)
decodedCh = make(chan interface{}, 10)
decodeErr error
samplesPool = sync.Pool{
New: func() interface{} {
return []record.RefSample{}
},
}
markersPool = sync.Pool{
New: func() interface{} {
return []record.RefMmapMarker{}
},
}
)
defer func() {
// For CorruptionErr ensure to terminate all workers before exiting.
// We also wrap it to identify OOO WBL corruption.
_, ok := err.(*wlog.CorruptionErr)
Add out-of-order sample support to the TSDB (#11075) * Introduce out-of-order TSDB support This implementation is based on this design doc: https://docs.google.com/document/d/1Kppm7qL9C-BJB1j6yb6-9ObG3AbdZnFUBYPNNWwDBYM/edit?usp=sharing This commit adds support to accept out-of-order ("OOO") sample into the TSDB up to a configurable time allowance. If OOO is enabled, overlapping querying are automatically enabled. Most of the additions have been borrowed from https://github.com/grafana/mimir-prometheus/ Here is the list ist of the original commits cherry picked from mimir-prometheus into this branch: - 4b2198d7ec47d50989b7c2df66b7b207c32f7f6e - 2836e5513f1bc591535a859f5d41154a75e7c6bc - 00b379c3a5b1ec3799699b6242f300a2b3ea30f0 - ff0dc757587cada63ca948d2d5eb00bf090d63e0 - a632c73352a7e39d60b445700beb47d691549c3e - c6f3d4ab339ab80bbbce74c9946237ced01f0509 - 5e8406a1d4a50d0052bbee83e28ca3b3371408aa - abde1e0ba128936b9eb0224ee1551e56216ebd4a - e70e7698897bb03860bee0467c733fa44e14c9bd - df59320886e03a555d379ac4b0b3130f661407e0 Co-authored-by: Jesus Vazquez <jesus.vazquez@grafana.com> Co-authored-by: Ganesh Vernekar <ganeshvern@gmail.com> Co-authored-by: Dieter Plaetinck <dieter@grafana.com> Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * gofumpt files Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Add license header to missing files Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Fix OOO tests due to existing chunk disk mapper implementation Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Fix truncate int overflow Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Add Sync method to the WAL and update tests Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * remove useless sync Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Update minOOOTime after truncating Head * Update minOOOTime after truncating Head Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Fix lint Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Add a unit test Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Load OutOfOrderTimeWindow only once per appender Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Fix OOO Head LabelValues and PostingsForMatchers Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Fix replay of OOO mmap chunks Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Remove unnecessary err check Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Prevent panic with ApplyConfig Signed-off-by: Ganesh Vernekar 15064823+codesome@users.noreply.github.com Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Run OOO compaction after restart if there is OOO data from WBL Signed-off-by: Ganesh Vernekar 15064823+codesome@users.noreply.github.com Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Apply Bartek's suggestions Co-authored-by: Bartlomiej Plotka <bwplotka@gmail.com> Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Refactor OOO compaction Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Address comments and TODOs - Added a comment explaining why we need the allow overlapping compaction toggle - Clarified TSDBConfig OutOfOrderTimeWindow doc - Added an owner to all the TODOs in the code Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Run go format Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Fix remaining review comments Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Fix tests Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Change wbl reference when truncating ooo in TestHeadMinOOOTimeUpdate Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Fix TestWBLAndMmapReplay test failure on windows Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Address most of the feedback Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Refactor the block meta for out of order Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Fix windows error Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Fix review comments Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> Signed-off-by: Ganesh Vernekar 15064823+codesome@users.noreply.github.com Co-authored-by: Ganesh Vernekar <15064823+codesome@users.noreply.github.com> Co-authored-by: Ganesh Vernekar <ganeshvern@gmail.com> Co-authored-by: Dieter Plaetinck <dieter@grafana.com> Co-authored-by: Oleg Zaytsev <mail@olegzaytsev.com> Co-authored-by: Bartlomiej Plotka <bwplotka@gmail.com>
2022-09-20 10:05:50 -07:00
if ok {
err = &errLoadWbl{err: err}
for i := 0; i < n; i++ {
processors[i].closeAndDrain()
}
wg.Wait()
}
}()
wg.Add(n)
for i := 0; i < n; i++ {
processors[i].setup()
go func(wp *wblSubsetProcessor) {
unknown := wp.processWBLSamples(h)
unknownRefs.Add(unknown)
wg.Done()
}(&processors[i])
}
go func() {
defer close(decodedCh)
for r.Next() {
rec := r.Record()
switch dec.Type(rec) {
case record.Samples:
samples := samplesPool.Get().([]record.RefSample)[:0]
samples, err = dec.Samples(rec, samples)
if err != nil {
decodeErr = &wlog.CorruptionErr{
Add out-of-order sample support to the TSDB (#11075) * Introduce out-of-order TSDB support This implementation is based on this design doc: https://docs.google.com/document/d/1Kppm7qL9C-BJB1j6yb6-9ObG3AbdZnFUBYPNNWwDBYM/edit?usp=sharing This commit adds support to accept out-of-order ("OOO") sample into the TSDB up to a configurable time allowance. If OOO is enabled, overlapping querying are automatically enabled. Most of the additions have been borrowed from https://github.com/grafana/mimir-prometheus/ Here is the list ist of the original commits cherry picked from mimir-prometheus into this branch: - 4b2198d7ec47d50989b7c2df66b7b207c32f7f6e - 2836e5513f1bc591535a859f5d41154a75e7c6bc - 00b379c3a5b1ec3799699b6242f300a2b3ea30f0 - ff0dc757587cada63ca948d2d5eb00bf090d63e0 - a632c73352a7e39d60b445700beb47d691549c3e - c6f3d4ab339ab80bbbce74c9946237ced01f0509 - 5e8406a1d4a50d0052bbee83e28ca3b3371408aa - abde1e0ba128936b9eb0224ee1551e56216ebd4a - e70e7698897bb03860bee0467c733fa44e14c9bd - df59320886e03a555d379ac4b0b3130f661407e0 Co-authored-by: Jesus Vazquez <jesus.vazquez@grafana.com> Co-authored-by: Ganesh Vernekar <ganeshvern@gmail.com> Co-authored-by: Dieter Plaetinck <dieter@grafana.com> Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * gofumpt files Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Add license header to missing files Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Fix OOO tests due to existing chunk disk mapper implementation Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Fix truncate int overflow Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Add Sync method to the WAL and update tests Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * remove useless sync Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Update minOOOTime after truncating Head * Update minOOOTime after truncating Head Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Fix lint Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Add a unit test Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Load OutOfOrderTimeWindow only once per appender Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Fix OOO Head LabelValues and PostingsForMatchers Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Fix replay of OOO mmap chunks Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Remove unnecessary err check Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Prevent panic with ApplyConfig Signed-off-by: Ganesh Vernekar 15064823+codesome@users.noreply.github.com Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Run OOO compaction after restart if there is OOO data from WBL Signed-off-by: Ganesh Vernekar 15064823+codesome@users.noreply.github.com Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Apply Bartek's suggestions Co-authored-by: Bartlomiej Plotka <bwplotka@gmail.com> Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Refactor OOO compaction Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Address comments and TODOs - Added a comment explaining why we need the allow overlapping compaction toggle - Clarified TSDBConfig OutOfOrderTimeWindow doc - Added an owner to all the TODOs in the code Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Run go format Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Fix remaining review comments Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Fix tests Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Change wbl reference when truncating ooo in TestHeadMinOOOTimeUpdate Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Fix TestWBLAndMmapReplay test failure on windows Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Address most of the feedback Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Refactor the block meta for out of order Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Fix windows error Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Fix review comments Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> Signed-off-by: Ganesh Vernekar 15064823+codesome@users.noreply.github.com Co-authored-by: Ganesh Vernekar <15064823+codesome@users.noreply.github.com> Co-authored-by: Ganesh Vernekar <ganeshvern@gmail.com> Co-authored-by: Dieter Plaetinck <dieter@grafana.com> Co-authored-by: Oleg Zaytsev <mail@olegzaytsev.com> Co-authored-by: Bartlomiej Plotka <bwplotka@gmail.com>
2022-09-20 10:05:50 -07:00
Err: errors.Wrap(err, "decode samples"),
Segment: r.Segment(),
Offset: r.Offset(),
}
return
2021-11-30 08:19:06 -08:00
}
Add out-of-order sample support to the TSDB (#11075) * Introduce out-of-order TSDB support This implementation is based on this design doc: https://docs.google.com/document/d/1Kppm7qL9C-BJB1j6yb6-9ObG3AbdZnFUBYPNNWwDBYM/edit?usp=sharing This commit adds support to accept out-of-order ("OOO") sample into the TSDB up to a configurable time allowance. If OOO is enabled, overlapping querying are automatically enabled. Most of the additions have been borrowed from https://github.com/grafana/mimir-prometheus/ Here is the list ist of the original commits cherry picked from mimir-prometheus into this branch: - 4b2198d7ec47d50989b7c2df66b7b207c32f7f6e - 2836e5513f1bc591535a859f5d41154a75e7c6bc - 00b379c3a5b1ec3799699b6242f300a2b3ea30f0 - ff0dc757587cada63ca948d2d5eb00bf090d63e0 - a632c73352a7e39d60b445700beb47d691549c3e - c6f3d4ab339ab80bbbce74c9946237ced01f0509 - 5e8406a1d4a50d0052bbee83e28ca3b3371408aa - abde1e0ba128936b9eb0224ee1551e56216ebd4a - e70e7698897bb03860bee0467c733fa44e14c9bd - df59320886e03a555d379ac4b0b3130f661407e0 Co-authored-by: Jesus Vazquez <jesus.vazquez@grafana.com> Co-authored-by: Ganesh Vernekar <ganeshvern@gmail.com> Co-authored-by: Dieter Plaetinck <dieter@grafana.com> Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * gofumpt files Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Add license header to missing files Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Fix OOO tests due to existing chunk disk mapper implementation Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Fix truncate int overflow Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Add Sync method to the WAL and update tests Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * remove useless sync Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Update minOOOTime after truncating Head * Update minOOOTime after truncating Head Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Fix lint Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Add a unit test Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Load OutOfOrderTimeWindow only once per appender Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Fix OOO Head LabelValues and PostingsForMatchers Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Fix replay of OOO mmap chunks Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Remove unnecessary err check Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Prevent panic with ApplyConfig Signed-off-by: Ganesh Vernekar 15064823+codesome@users.noreply.github.com Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Run OOO compaction after restart if there is OOO data from WBL Signed-off-by: Ganesh Vernekar 15064823+codesome@users.noreply.github.com Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Apply Bartek's suggestions Co-authored-by: Bartlomiej Plotka <bwplotka@gmail.com> Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Refactor OOO compaction Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Address comments and TODOs - Added a comment explaining why we need the allow overlapping compaction toggle - Clarified TSDBConfig OutOfOrderTimeWindow doc - Added an owner to all the TODOs in the code Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Run go format Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Fix remaining review comments Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Fix tests Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Change wbl reference when truncating ooo in TestHeadMinOOOTimeUpdate Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Fix TestWBLAndMmapReplay test failure on windows Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Address most of the feedback Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Refactor the block meta for out of order Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Fix windows error Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Fix review comments Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> Signed-off-by: Ganesh Vernekar 15064823+codesome@users.noreply.github.com Co-authored-by: Ganesh Vernekar <15064823+codesome@users.noreply.github.com> Co-authored-by: Ganesh Vernekar <ganeshvern@gmail.com> Co-authored-by: Dieter Plaetinck <dieter@grafana.com> Co-authored-by: Oleg Zaytsev <mail@olegzaytsev.com> Co-authored-by: Bartlomiej Plotka <bwplotka@gmail.com>
2022-09-20 10:05:50 -07:00
decodedCh <- samples
case record.MmapMarkers:
markers := markersPool.Get().([]record.RefMmapMarker)[:0]
markers, err = dec.MmapMarkers(rec, markers)
if err != nil {
decodeErr = &wlog.CorruptionErr{
Add out-of-order sample support to the TSDB (#11075) * Introduce out-of-order TSDB support This implementation is based on this design doc: https://docs.google.com/document/d/1Kppm7qL9C-BJB1j6yb6-9ObG3AbdZnFUBYPNNWwDBYM/edit?usp=sharing This commit adds support to accept out-of-order ("OOO") sample into the TSDB up to a configurable time allowance. If OOO is enabled, overlapping querying are automatically enabled. Most of the additions have been borrowed from https://github.com/grafana/mimir-prometheus/ Here is the list ist of the original commits cherry picked from mimir-prometheus into this branch: - 4b2198d7ec47d50989b7c2df66b7b207c32f7f6e - 2836e5513f1bc591535a859f5d41154a75e7c6bc - 00b379c3a5b1ec3799699b6242f300a2b3ea30f0 - ff0dc757587cada63ca948d2d5eb00bf090d63e0 - a632c73352a7e39d60b445700beb47d691549c3e - c6f3d4ab339ab80bbbce74c9946237ced01f0509 - 5e8406a1d4a50d0052bbee83e28ca3b3371408aa - abde1e0ba128936b9eb0224ee1551e56216ebd4a - e70e7698897bb03860bee0467c733fa44e14c9bd - df59320886e03a555d379ac4b0b3130f661407e0 Co-authored-by: Jesus Vazquez <jesus.vazquez@grafana.com> Co-authored-by: Ganesh Vernekar <ganeshvern@gmail.com> Co-authored-by: Dieter Plaetinck <dieter@grafana.com> Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * gofumpt files Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Add license header to missing files Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Fix OOO tests due to existing chunk disk mapper implementation Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Fix truncate int overflow Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Add Sync method to the WAL and update tests Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * remove useless sync Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Update minOOOTime after truncating Head * Update minOOOTime after truncating Head Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Fix lint Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Add a unit test Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Load OutOfOrderTimeWindow only once per appender Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Fix OOO Head LabelValues and PostingsForMatchers Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Fix replay of OOO mmap chunks Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Remove unnecessary err check Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Prevent panic with ApplyConfig Signed-off-by: Ganesh Vernekar 15064823+codesome@users.noreply.github.com Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Run OOO compaction after restart if there is OOO data from WBL Signed-off-by: Ganesh Vernekar 15064823+codesome@users.noreply.github.com Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Apply Bartek's suggestions Co-authored-by: Bartlomiej Plotka <bwplotka@gmail.com> Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Refactor OOO compaction Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Address comments and TODOs - Added a comment explaining why we need the allow overlapping compaction toggle - Clarified TSDBConfig OutOfOrderTimeWindow doc - Added an owner to all the TODOs in the code Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Run go format Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Fix remaining review comments Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Fix tests Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Change wbl reference when truncating ooo in TestHeadMinOOOTimeUpdate Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Fix TestWBLAndMmapReplay test failure on windows Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Address most of the feedback Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Refactor the block meta for out of order Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Fix windows error Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Fix review comments Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> Signed-off-by: Ganesh Vernekar 15064823+codesome@users.noreply.github.com Co-authored-by: Ganesh Vernekar <15064823+codesome@users.noreply.github.com> Co-authored-by: Ganesh Vernekar <ganeshvern@gmail.com> Co-authored-by: Dieter Plaetinck <dieter@grafana.com> Co-authored-by: Oleg Zaytsev <mail@olegzaytsev.com> Co-authored-by: Bartlomiej Plotka <bwplotka@gmail.com>
2022-09-20 10:05:50 -07:00
Err: errors.Wrap(err, "decode mmap markers"),
Segment: r.Segment(),
Offset: r.Offset(),
}
return
2021-11-30 08:19:06 -08:00
}
Add out-of-order sample support to the TSDB (#11075) * Introduce out-of-order TSDB support This implementation is based on this design doc: https://docs.google.com/document/d/1Kppm7qL9C-BJB1j6yb6-9ObG3AbdZnFUBYPNNWwDBYM/edit?usp=sharing This commit adds support to accept out-of-order ("OOO") sample into the TSDB up to a configurable time allowance. If OOO is enabled, overlapping querying are automatically enabled. Most of the additions have been borrowed from https://github.com/grafana/mimir-prometheus/ Here is the list ist of the original commits cherry picked from mimir-prometheus into this branch: - 4b2198d7ec47d50989b7c2df66b7b207c32f7f6e - 2836e5513f1bc591535a859f5d41154a75e7c6bc - 00b379c3a5b1ec3799699b6242f300a2b3ea30f0 - ff0dc757587cada63ca948d2d5eb00bf090d63e0 - a632c73352a7e39d60b445700beb47d691549c3e - c6f3d4ab339ab80bbbce74c9946237ced01f0509 - 5e8406a1d4a50d0052bbee83e28ca3b3371408aa - abde1e0ba128936b9eb0224ee1551e56216ebd4a - e70e7698897bb03860bee0467c733fa44e14c9bd - df59320886e03a555d379ac4b0b3130f661407e0 Co-authored-by: Jesus Vazquez <jesus.vazquez@grafana.com> Co-authored-by: Ganesh Vernekar <ganeshvern@gmail.com> Co-authored-by: Dieter Plaetinck <dieter@grafana.com> Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * gofumpt files Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Add license header to missing files Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Fix OOO tests due to existing chunk disk mapper implementation Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Fix truncate int overflow Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Add Sync method to the WAL and update tests Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * remove useless sync Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Update minOOOTime after truncating Head * Update minOOOTime after truncating Head Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Fix lint Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Add a unit test Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Load OutOfOrderTimeWindow only once per appender Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Fix OOO Head LabelValues and PostingsForMatchers Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Fix replay of OOO mmap chunks Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Remove unnecessary err check Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Prevent panic with ApplyConfig Signed-off-by: Ganesh Vernekar 15064823+codesome@users.noreply.github.com Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Run OOO compaction after restart if there is OOO data from WBL Signed-off-by: Ganesh Vernekar 15064823+codesome@users.noreply.github.com Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Apply Bartek's suggestions Co-authored-by: Bartlomiej Plotka <bwplotka@gmail.com> Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Refactor OOO compaction Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Address comments and TODOs - Added a comment explaining why we need the allow overlapping compaction toggle - Clarified TSDBConfig OutOfOrderTimeWindow doc - Added an owner to all the TODOs in the code Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Run go format Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Fix remaining review comments Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Fix tests Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Change wbl reference when truncating ooo in TestHeadMinOOOTimeUpdate Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Fix TestWBLAndMmapReplay test failure on windows Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Address most of the feedback Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Refactor the block meta for out of order Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Fix windows error Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Fix review comments Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> Signed-off-by: Ganesh Vernekar 15064823+codesome@users.noreply.github.com Co-authored-by: Ganesh Vernekar <15064823+codesome@users.noreply.github.com> Co-authored-by: Ganesh Vernekar <ganeshvern@gmail.com> Co-authored-by: Dieter Plaetinck <dieter@grafana.com> Co-authored-by: Oleg Zaytsev <mail@olegzaytsev.com> Co-authored-by: Bartlomiej Plotka <bwplotka@gmail.com>
2022-09-20 10:05:50 -07:00
decodedCh <- markers
default:
// Noop.
}
}
}()
// The records are always replayed from the oldest to the newest.
for d := range decodedCh {
switch v := d.(type) {
case []record.RefSample:
samples := v
// We split up the samples into parts of 5000 samples or less.
// With O(300 * #cores) in-flight sample batches, large scrapes could otherwise
// cause thousands of very large in flight buffers occupying large amounts
// of unused memory.
for len(samples) > 0 {
m := 5000
if len(samples) < m {
m = len(samples)
2021-11-30 08:19:06 -08:00
}
Add out-of-order sample support to the TSDB (#11075) * Introduce out-of-order TSDB support This implementation is based on this design doc: https://docs.google.com/document/d/1Kppm7qL9C-BJB1j6yb6-9ObG3AbdZnFUBYPNNWwDBYM/edit?usp=sharing This commit adds support to accept out-of-order ("OOO") sample into the TSDB up to a configurable time allowance. If OOO is enabled, overlapping querying are automatically enabled. Most of the additions have been borrowed from https://github.com/grafana/mimir-prometheus/ Here is the list ist of the original commits cherry picked from mimir-prometheus into this branch: - 4b2198d7ec47d50989b7c2df66b7b207c32f7f6e - 2836e5513f1bc591535a859f5d41154a75e7c6bc - 00b379c3a5b1ec3799699b6242f300a2b3ea30f0 - ff0dc757587cada63ca948d2d5eb00bf090d63e0 - a632c73352a7e39d60b445700beb47d691549c3e - c6f3d4ab339ab80bbbce74c9946237ced01f0509 - 5e8406a1d4a50d0052bbee83e28ca3b3371408aa - abde1e0ba128936b9eb0224ee1551e56216ebd4a - e70e7698897bb03860bee0467c733fa44e14c9bd - df59320886e03a555d379ac4b0b3130f661407e0 Co-authored-by: Jesus Vazquez <jesus.vazquez@grafana.com> Co-authored-by: Ganesh Vernekar <ganeshvern@gmail.com> Co-authored-by: Dieter Plaetinck <dieter@grafana.com> Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * gofumpt files Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Add license header to missing files Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Fix OOO tests due to existing chunk disk mapper implementation Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Fix truncate int overflow Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Add Sync method to the WAL and update tests Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * remove useless sync Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Update minOOOTime after truncating Head * Update minOOOTime after truncating Head Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Fix lint Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Add a unit test Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Load OutOfOrderTimeWindow only once per appender Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Fix OOO Head LabelValues and PostingsForMatchers Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Fix replay of OOO mmap chunks Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Remove unnecessary err check Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Prevent panic with ApplyConfig Signed-off-by: Ganesh Vernekar 15064823+codesome@users.noreply.github.com Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Run OOO compaction after restart if there is OOO data from WBL Signed-off-by: Ganesh Vernekar 15064823+codesome@users.noreply.github.com Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Apply Bartek's suggestions Co-authored-by: Bartlomiej Plotka <bwplotka@gmail.com> Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Refactor OOO compaction Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Address comments and TODOs - Added a comment explaining why we need the allow overlapping compaction toggle - Clarified TSDBConfig OutOfOrderTimeWindow doc - Added an owner to all the TODOs in the code Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Run go format Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Fix remaining review comments Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Fix tests Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Change wbl reference when truncating ooo in TestHeadMinOOOTimeUpdate Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Fix TestWBLAndMmapReplay test failure on windows Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Address most of the feedback Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Refactor the block meta for out of order Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Fix windows error Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Fix review comments Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> Signed-off-by: Ganesh Vernekar 15064823+codesome@users.noreply.github.com Co-authored-by: Ganesh Vernekar <15064823+codesome@users.noreply.github.com> Co-authored-by: Ganesh Vernekar <ganeshvern@gmail.com> Co-authored-by: Dieter Plaetinck <dieter@grafana.com> Co-authored-by: Oleg Zaytsev <mail@olegzaytsev.com> Co-authored-by: Bartlomiej Plotka <bwplotka@gmail.com>
2022-09-20 10:05:50 -07:00
for i := 0; i < n; i++ {
shards[i] = processors[i].reuseBuf()
2021-11-30 08:19:06 -08:00
}
Add out-of-order sample support to the TSDB (#11075) * Introduce out-of-order TSDB support This implementation is based on this design doc: https://docs.google.com/document/d/1Kppm7qL9C-BJB1j6yb6-9ObG3AbdZnFUBYPNNWwDBYM/edit?usp=sharing This commit adds support to accept out-of-order ("OOO") sample into the TSDB up to a configurable time allowance. If OOO is enabled, overlapping querying are automatically enabled. Most of the additions have been borrowed from https://github.com/grafana/mimir-prometheus/ Here is the list ist of the original commits cherry picked from mimir-prometheus into this branch: - 4b2198d7ec47d50989b7c2df66b7b207c32f7f6e - 2836e5513f1bc591535a859f5d41154a75e7c6bc - 00b379c3a5b1ec3799699b6242f300a2b3ea30f0 - ff0dc757587cada63ca948d2d5eb00bf090d63e0 - a632c73352a7e39d60b445700beb47d691549c3e - c6f3d4ab339ab80bbbce74c9946237ced01f0509 - 5e8406a1d4a50d0052bbee83e28ca3b3371408aa - abde1e0ba128936b9eb0224ee1551e56216ebd4a - e70e7698897bb03860bee0467c733fa44e14c9bd - df59320886e03a555d379ac4b0b3130f661407e0 Co-authored-by: Jesus Vazquez <jesus.vazquez@grafana.com> Co-authored-by: Ganesh Vernekar <ganeshvern@gmail.com> Co-authored-by: Dieter Plaetinck <dieter@grafana.com> Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * gofumpt files Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Add license header to missing files Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Fix OOO tests due to existing chunk disk mapper implementation Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Fix truncate int overflow Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Add Sync method to the WAL and update tests Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * remove useless sync Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Update minOOOTime after truncating Head * Update minOOOTime after truncating Head Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Fix lint Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Add a unit test Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Load OutOfOrderTimeWindow only once per appender Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Fix OOO Head LabelValues and PostingsForMatchers Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Fix replay of OOO mmap chunks Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Remove unnecessary err check Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Prevent panic with ApplyConfig Signed-off-by: Ganesh Vernekar 15064823+codesome@users.noreply.github.com Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Run OOO compaction after restart if there is OOO data from WBL Signed-off-by: Ganesh Vernekar 15064823+codesome@users.noreply.github.com Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Apply Bartek's suggestions Co-authored-by: Bartlomiej Plotka <bwplotka@gmail.com> Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Refactor OOO compaction Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Address comments and TODOs - Added a comment explaining why we need the allow overlapping compaction toggle - Clarified TSDBConfig OutOfOrderTimeWindow doc - Added an owner to all the TODOs in the code Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Run go format Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Fix remaining review comments Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Fix tests Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Change wbl reference when truncating ooo in TestHeadMinOOOTimeUpdate Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Fix TestWBLAndMmapReplay test failure on windows Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Address most of the feedback Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Refactor the block meta for out of order Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Fix windows error Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Fix review comments Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> Signed-off-by: Ganesh Vernekar 15064823+codesome@users.noreply.github.com Co-authored-by: Ganesh Vernekar <15064823+codesome@users.noreply.github.com> Co-authored-by: Ganesh Vernekar <ganeshvern@gmail.com> Co-authored-by: Dieter Plaetinck <dieter@grafana.com> Co-authored-by: Oleg Zaytsev <mail@olegzaytsev.com> Co-authored-by: Bartlomiej Plotka <bwplotka@gmail.com>
2022-09-20 10:05:50 -07:00
for _, sam := range samples[:m] {
if r, ok := multiRef[sam.Ref]; ok {
sam.Ref = r
}
mod := uint64(sam.Ref) % uint64(n)
shards[mod] = append(shards[mod], sam)
2021-11-30 08:19:06 -08:00
}
Add out-of-order sample support to the TSDB (#11075) * Introduce out-of-order TSDB support This implementation is based on this design doc: https://docs.google.com/document/d/1Kppm7qL9C-BJB1j6yb6-9ObG3AbdZnFUBYPNNWwDBYM/edit?usp=sharing This commit adds support to accept out-of-order ("OOO") sample into the TSDB up to a configurable time allowance. If OOO is enabled, overlapping querying are automatically enabled. Most of the additions have been borrowed from https://github.com/grafana/mimir-prometheus/ Here is the list ist of the original commits cherry picked from mimir-prometheus into this branch: - 4b2198d7ec47d50989b7c2df66b7b207c32f7f6e - 2836e5513f1bc591535a859f5d41154a75e7c6bc - 00b379c3a5b1ec3799699b6242f300a2b3ea30f0 - ff0dc757587cada63ca948d2d5eb00bf090d63e0 - a632c73352a7e39d60b445700beb47d691549c3e - c6f3d4ab339ab80bbbce74c9946237ced01f0509 - 5e8406a1d4a50d0052bbee83e28ca3b3371408aa - abde1e0ba128936b9eb0224ee1551e56216ebd4a - e70e7698897bb03860bee0467c733fa44e14c9bd - df59320886e03a555d379ac4b0b3130f661407e0 Co-authored-by: Jesus Vazquez <jesus.vazquez@grafana.com> Co-authored-by: Ganesh Vernekar <ganeshvern@gmail.com> Co-authored-by: Dieter Plaetinck <dieter@grafana.com> Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * gofumpt files Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Add license header to missing files Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Fix OOO tests due to existing chunk disk mapper implementation Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Fix truncate int overflow Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Add Sync method to the WAL and update tests Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * remove useless sync Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Update minOOOTime after truncating Head * Update minOOOTime after truncating Head Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Fix lint Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Add a unit test Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Load OutOfOrderTimeWindow only once per appender Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Fix OOO Head LabelValues and PostingsForMatchers Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Fix replay of OOO mmap chunks Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Remove unnecessary err check Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Prevent panic with ApplyConfig Signed-off-by: Ganesh Vernekar 15064823+codesome@users.noreply.github.com Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Run OOO compaction after restart if there is OOO data from WBL Signed-off-by: Ganesh Vernekar 15064823+codesome@users.noreply.github.com Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Apply Bartek's suggestions Co-authored-by: Bartlomiej Plotka <bwplotka@gmail.com> Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Refactor OOO compaction Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Address comments and TODOs - Added a comment explaining why we need the allow overlapping compaction toggle - Clarified TSDBConfig OutOfOrderTimeWindow doc - Added an owner to all the TODOs in the code Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Run go format Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Fix remaining review comments Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Fix tests Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Change wbl reference when truncating ooo in TestHeadMinOOOTimeUpdate Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Fix TestWBLAndMmapReplay test failure on windows Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Address most of the feedback Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Refactor the block meta for out of order Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Fix windows error Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Fix review comments Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> Signed-off-by: Ganesh Vernekar 15064823+codesome@users.noreply.github.com Co-authored-by: Ganesh Vernekar <15064823+codesome@users.noreply.github.com> Co-authored-by: Ganesh Vernekar <ganeshvern@gmail.com> Co-authored-by: Dieter Plaetinck <dieter@grafana.com> Co-authored-by: Oleg Zaytsev <mail@olegzaytsev.com> Co-authored-by: Bartlomiej Plotka <bwplotka@gmail.com>
2022-09-20 10:05:50 -07:00
for i := 0; i < n; i++ {
processors[i].input <- shards[i]
2021-11-30 08:19:06 -08:00
}
Add out-of-order sample support to the TSDB (#11075) * Introduce out-of-order TSDB support This implementation is based on this design doc: https://docs.google.com/document/d/1Kppm7qL9C-BJB1j6yb6-9ObG3AbdZnFUBYPNNWwDBYM/edit?usp=sharing This commit adds support to accept out-of-order ("OOO") sample into the TSDB up to a configurable time allowance. If OOO is enabled, overlapping querying are automatically enabled. Most of the additions have been borrowed from https://github.com/grafana/mimir-prometheus/ Here is the list ist of the original commits cherry picked from mimir-prometheus into this branch: - 4b2198d7ec47d50989b7c2df66b7b207c32f7f6e - 2836e5513f1bc591535a859f5d41154a75e7c6bc - 00b379c3a5b1ec3799699b6242f300a2b3ea30f0 - ff0dc757587cada63ca948d2d5eb00bf090d63e0 - a632c73352a7e39d60b445700beb47d691549c3e - c6f3d4ab339ab80bbbce74c9946237ced01f0509 - 5e8406a1d4a50d0052bbee83e28ca3b3371408aa - abde1e0ba128936b9eb0224ee1551e56216ebd4a - e70e7698897bb03860bee0467c733fa44e14c9bd - df59320886e03a555d379ac4b0b3130f661407e0 Co-authored-by: Jesus Vazquez <jesus.vazquez@grafana.com> Co-authored-by: Ganesh Vernekar <ganeshvern@gmail.com> Co-authored-by: Dieter Plaetinck <dieter@grafana.com> Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * gofumpt files Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Add license header to missing files Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Fix OOO tests due to existing chunk disk mapper implementation Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Fix truncate int overflow Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Add Sync method to the WAL and update tests Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * remove useless sync Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Update minOOOTime after truncating Head * Update minOOOTime after truncating Head Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Fix lint Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Add a unit test Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Load OutOfOrderTimeWindow only once per appender Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Fix OOO Head LabelValues and PostingsForMatchers Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Fix replay of OOO mmap chunks Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Remove unnecessary err check Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Prevent panic with ApplyConfig Signed-off-by: Ganesh Vernekar 15064823+codesome@users.noreply.github.com Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Run OOO compaction after restart if there is OOO data from WBL Signed-off-by: Ganesh Vernekar 15064823+codesome@users.noreply.github.com Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Apply Bartek's suggestions Co-authored-by: Bartlomiej Plotka <bwplotka@gmail.com> Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Refactor OOO compaction Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Address comments and TODOs - Added a comment explaining why we need the allow overlapping compaction toggle - Clarified TSDBConfig OutOfOrderTimeWindow doc - Added an owner to all the TODOs in the code Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Run go format Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Fix remaining review comments Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Fix tests Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Change wbl reference when truncating ooo in TestHeadMinOOOTimeUpdate Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Fix TestWBLAndMmapReplay test failure on windows Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Address most of the feedback Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Refactor the block meta for out of order Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Fix windows error Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Fix review comments Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> Signed-off-by: Ganesh Vernekar 15064823+codesome@users.noreply.github.com Co-authored-by: Ganesh Vernekar <15064823+codesome@users.noreply.github.com> Co-authored-by: Ganesh Vernekar <ganeshvern@gmail.com> Co-authored-by: Dieter Plaetinck <dieter@grafana.com> Co-authored-by: Oleg Zaytsev <mail@olegzaytsev.com> Co-authored-by: Bartlomiej Plotka <bwplotka@gmail.com>
2022-09-20 10:05:50 -07:00
samples = samples[m:]
}
Add out-of-order sample support to the TSDB (#11075) * Introduce out-of-order TSDB support This implementation is based on this design doc: https://docs.google.com/document/d/1Kppm7qL9C-BJB1j6yb6-9ObG3AbdZnFUBYPNNWwDBYM/edit?usp=sharing This commit adds support to accept out-of-order ("OOO") sample into the TSDB up to a configurable time allowance. If OOO is enabled, overlapping querying are automatically enabled. Most of the additions have been borrowed from https://github.com/grafana/mimir-prometheus/ Here is the list ist of the original commits cherry picked from mimir-prometheus into this branch: - 4b2198d7ec47d50989b7c2df66b7b207c32f7f6e - 2836e5513f1bc591535a859f5d41154a75e7c6bc - 00b379c3a5b1ec3799699b6242f300a2b3ea30f0 - ff0dc757587cada63ca948d2d5eb00bf090d63e0 - a632c73352a7e39d60b445700beb47d691549c3e - c6f3d4ab339ab80bbbce74c9946237ced01f0509 - 5e8406a1d4a50d0052bbee83e28ca3b3371408aa - abde1e0ba128936b9eb0224ee1551e56216ebd4a - e70e7698897bb03860bee0467c733fa44e14c9bd - df59320886e03a555d379ac4b0b3130f661407e0 Co-authored-by: Jesus Vazquez <jesus.vazquez@grafana.com> Co-authored-by: Ganesh Vernekar <ganeshvern@gmail.com> Co-authored-by: Dieter Plaetinck <dieter@grafana.com> Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * gofumpt files Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Add license header to missing files Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Fix OOO tests due to existing chunk disk mapper implementation Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Fix truncate int overflow Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Add Sync method to the WAL and update tests Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * remove useless sync Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Update minOOOTime after truncating Head * Update minOOOTime after truncating Head Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Fix lint Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Add a unit test Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Load OutOfOrderTimeWindow only once per appender Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Fix OOO Head LabelValues and PostingsForMatchers Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Fix replay of OOO mmap chunks Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Remove unnecessary err check Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Prevent panic with ApplyConfig Signed-off-by: Ganesh Vernekar 15064823+codesome@users.noreply.github.com Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Run OOO compaction after restart if there is OOO data from WBL Signed-off-by: Ganesh Vernekar 15064823+codesome@users.noreply.github.com Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Apply Bartek's suggestions Co-authored-by: Bartlomiej Plotka <bwplotka@gmail.com> Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Refactor OOO compaction Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Address comments and TODOs - Added a comment explaining why we need the allow overlapping compaction toggle - Clarified TSDBConfig OutOfOrderTimeWindow doc - Added an owner to all the TODOs in the code Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Run go format Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Fix remaining review comments Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Fix tests Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Change wbl reference when truncating ooo in TestHeadMinOOOTimeUpdate Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Fix TestWBLAndMmapReplay test failure on windows Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Address most of the feedback Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Refactor the block meta for out of order Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Fix windows error Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Fix review comments Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> Signed-off-by: Ganesh Vernekar 15064823+codesome@users.noreply.github.com Co-authored-by: Ganesh Vernekar <15064823+codesome@users.noreply.github.com> Co-authored-by: Ganesh Vernekar <ganeshvern@gmail.com> Co-authored-by: Dieter Plaetinck <dieter@grafana.com> Co-authored-by: Oleg Zaytsev <mail@olegzaytsev.com> Co-authored-by: Bartlomiej Plotka <bwplotka@gmail.com>
2022-09-20 10:05:50 -07:00
//nolint:staticcheck // Ignore SA6002 relax staticcheck verification.
samplesPool.Put(d)
case []record.RefMmapMarker:
markers := v
for _, rm := range markers {
seq, off := rm.MmapRef.Unpack()
if seq > lastSeq || (seq == lastSeq && off > lastOff) {
// This m-map chunk from markers was not present during
// the load of mmapped chunks that happened in the head
// initialization.
2021-11-30 08:19:06 -08:00
continue
}
Add out-of-order sample support to the TSDB (#11075) * Introduce out-of-order TSDB support This implementation is based on this design doc: https://docs.google.com/document/d/1Kppm7qL9C-BJB1j6yb6-9ObG3AbdZnFUBYPNNWwDBYM/edit?usp=sharing This commit adds support to accept out-of-order ("OOO") sample into the TSDB up to a configurable time allowance. If OOO is enabled, overlapping querying are automatically enabled. Most of the additions have been borrowed from https://github.com/grafana/mimir-prometheus/ Here is the list ist of the original commits cherry picked from mimir-prometheus into this branch: - 4b2198d7ec47d50989b7c2df66b7b207c32f7f6e - 2836e5513f1bc591535a859f5d41154a75e7c6bc - 00b379c3a5b1ec3799699b6242f300a2b3ea30f0 - ff0dc757587cada63ca948d2d5eb00bf090d63e0 - a632c73352a7e39d60b445700beb47d691549c3e - c6f3d4ab339ab80bbbce74c9946237ced01f0509 - 5e8406a1d4a50d0052bbee83e28ca3b3371408aa - abde1e0ba128936b9eb0224ee1551e56216ebd4a - e70e7698897bb03860bee0467c733fa44e14c9bd - df59320886e03a555d379ac4b0b3130f661407e0 Co-authored-by: Jesus Vazquez <jesus.vazquez@grafana.com> Co-authored-by: Ganesh Vernekar <ganeshvern@gmail.com> Co-authored-by: Dieter Plaetinck <dieter@grafana.com> Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * gofumpt files Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Add license header to missing files Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Fix OOO tests due to existing chunk disk mapper implementation Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Fix truncate int overflow Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Add Sync method to the WAL and update tests Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * remove useless sync Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Update minOOOTime after truncating Head * Update minOOOTime after truncating Head Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Fix lint Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Add a unit test Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Load OutOfOrderTimeWindow only once per appender Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Fix OOO Head LabelValues and PostingsForMatchers Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Fix replay of OOO mmap chunks Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Remove unnecessary err check Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Prevent panic with ApplyConfig Signed-off-by: Ganesh Vernekar 15064823+codesome@users.noreply.github.com Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Run OOO compaction after restart if there is OOO data from WBL Signed-off-by: Ganesh Vernekar 15064823+codesome@users.noreply.github.com Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Apply Bartek's suggestions Co-authored-by: Bartlomiej Plotka <bwplotka@gmail.com> Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Refactor OOO compaction Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Address comments and TODOs - Added a comment explaining why we need the allow overlapping compaction toggle - Clarified TSDBConfig OutOfOrderTimeWindow doc - Added an owner to all the TODOs in the code Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Run go format Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Fix remaining review comments Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Fix tests Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Change wbl reference when truncating ooo in TestHeadMinOOOTimeUpdate Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Fix TestWBLAndMmapReplay test failure on windows Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Address most of the feedback Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Refactor the block meta for out of order Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Fix windows error Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Fix review comments Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> Signed-off-by: Ganesh Vernekar 15064823+codesome@users.noreply.github.com Co-authored-by: Ganesh Vernekar <15064823+codesome@users.noreply.github.com> Co-authored-by: Ganesh Vernekar <ganeshvern@gmail.com> Co-authored-by: Dieter Plaetinck <dieter@grafana.com> Co-authored-by: Oleg Zaytsev <mail@olegzaytsev.com> Co-authored-by: Bartlomiej Plotka <bwplotka@gmail.com>
2022-09-20 10:05:50 -07:00
if r, ok := multiRef[rm.Ref]; ok {
rm.Ref = r
2021-11-30 08:19:06 -08:00
}
Add out-of-order sample support to the TSDB (#11075) * Introduce out-of-order TSDB support This implementation is based on this design doc: https://docs.google.com/document/d/1Kppm7qL9C-BJB1j6yb6-9ObG3AbdZnFUBYPNNWwDBYM/edit?usp=sharing This commit adds support to accept out-of-order ("OOO") sample into the TSDB up to a configurable time allowance. If OOO is enabled, overlapping querying are automatically enabled. Most of the additions have been borrowed from https://github.com/grafana/mimir-prometheus/ Here is the list ist of the original commits cherry picked from mimir-prometheus into this branch: - 4b2198d7ec47d50989b7c2df66b7b207c32f7f6e - 2836e5513f1bc591535a859f5d41154a75e7c6bc - 00b379c3a5b1ec3799699b6242f300a2b3ea30f0 - ff0dc757587cada63ca948d2d5eb00bf090d63e0 - a632c73352a7e39d60b445700beb47d691549c3e - c6f3d4ab339ab80bbbce74c9946237ced01f0509 - 5e8406a1d4a50d0052bbee83e28ca3b3371408aa - abde1e0ba128936b9eb0224ee1551e56216ebd4a - e70e7698897bb03860bee0467c733fa44e14c9bd - df59320886e03a555d379ac4b0b3130f661407e0 Co-authored-by: Jesus Vazquez <jesus.vazquez@grafana.com> Co-authored-by: Ganesh Vernekar <ganeshvern@gmail.com> Co-authored-by: Dieter Plaetinck <dieter@grafana.com> Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * gofumpt files Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Add license header to missing files Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Fix OOO tests due to existing chunk disk mapper implementation Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Fix truncate int overflow Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Add Sync method to the WAL and update tests Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * remove useless sync Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Update minOOOTime after truncating Head * Update minOOOTime after truncating Head Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Fix lint Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Add a unit test Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Load OutOfOrderTimeWindow only once per appender Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Fix OOO Head LabelValues and PostingsForMatchers Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Fix replay of OOO mmap chunks Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Remove unnecessary err check Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Prevent panic with ApplyConfig Signed-off-by: Ganesh Vernekar 15064823+codesome@users.noreply.github.com Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Run OOO compaction after restart if there is OOO data from WBL Signed-off-by: Ganesh Vernekar 15064823+codesome@users.noreply.github.com Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Apply Bartek's suggestions Co-authored-by: Bartlomiej Plotka <bwplotka@gmail.com> Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Refactor OOO compaction Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Address comments and TODOs - Added a comment explaining why we need the allow overlapping compaction toggle - Clarified TSDBConfig OutOfOrderTimeWindow doc - Added an owner to all the TODOs in the code Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Run go format Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Fix remaining review comments Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Fix tests Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Change wbl reference when truncating ooo in TestHeadMinOOOTimeUpdate Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Fix TestWBLAndMmapReplay test failure on windows Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Address most of the feedback Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Refactor the block meta for out of order Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Fix windows error Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Fix review comments Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> Signed-off-by: Ganesh Vernekar 15064823+codesome@users.noreply.github.com Co-authored-by: Ganesh Vernekar <15064823+codesome@users.noreply.github.com> Co-authored-by: Ganesh Vernekar <ganeshvern@gmail.com> Co-authored-by: Dieter Plaetinck <dieter@grafana.com> Co-authored-by: Oleg Zaytsev <mail@olegzaytsev.com> Co-authored-by: Bartlomiej Plotka <bwplotka@gmail.com>
2022-09-20 10:05:50 -07:00
ms := h.series.getByID(rm.Ref)
if ms == nil {
mmapMarkerUnknownRefs.Inc()
2021-11-30 08:19:06 -08:00
continue
}
Add out-of-order sample support to the TSDB (#11075) * Introduce out-of-order TSDB support This implementation is based on this design doc: https://docs.google.com/document/d/1Kppm7qL9C-BJB1j6yb6-9ObG3AbdZnFUBYPNNWwDBYM/edit?usp=sharing This commit adds support to accept out-of-order ("OOO") sample into the TSDB up to a configurable time allowance. If OOO is enabled, overlapping querying are automatically enabled. Most of the additions have been borrowed from https://github.com/grafana/mimir-prometheus/ Here is the list ist of the original commits cherry picked from mimir-prometheus into this branch: - 4b2198d7ec47d50989b7c2df66b7b207c32f7f6e - 2836e5513f1bc591535a859f5d41154a75e7c6bc - 00b379c3a5b1ec3799699b6242f300a2b3ea30f0 - ff0dc757587cada63ca948d2d5eb00bf090d63e0 - a632c73352a7e39d60b445700beb47d691549c3e - c6f3d4ab339ab80bbbce74c9946237ced01f0509 - 5e8406a1d4a50d0052bbee83e28ca3b3371408aa - abde1e0ba128936b9eb0224ee1551e56216ebd4a - e70e7698897bb03860bee0467c733fa44e14c9bd - df59320886e03a555d379ac4b0b3130f661407e0 Co-authored-by: Jesus Vazquez <jesus.vazquez@grafana.com> Co-authored-by: Ganesh Vernekar <ganeshvern@gmail.com> Co-authored-by: Dieter Plaetinck <dieter@grafana.com> Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * gofumpt files Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Add license header to missing files Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Fix OOO tests due to existing chunk disk mapper implementation Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Fix truncate int overflow Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Add Sync method to the WAL and update tests Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * remove useless sync Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Update minOOOTime after truncating Head * Update minOOOTime after truncating Head Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Fix lint Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Add a unit test Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Load OutOfOrderTimeWindow only once per appender Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Fix OOO Head LabelValues and PostingsForMatchers Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Fix replay of OOO mmap chunks Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Remove unnecessary err check Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Prevent panic with ApplyConfig Signed-off-by: Ganesh Vernekar 15064823+codesome@users.noreply.github.com Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Run OOO compaction after restart if there is OOO data from WBL Signed-off-by: Ganesh Vernekar 15064823+codesome@users.noreply.github.com Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Apply Bartek's suggestions Co-authored-by: Bartlomiej Plotka <bwplotka@gmail.com> Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Refactor OOO compaction Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Address comments and TODOs - Added a comment explaining why we need the allow overlapping compaction toggle - Clarified TSDBConfig OutOfOrderTimeWindow doc - Added an owner to all the TODOs in the code Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Run go format Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Fix remaining review comments Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Fix tests Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Change wbl reference when truncating ooo in TestHeadMinOOOTimeUpdate Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Fix TestWBLAndMmapReplay test failure on windows Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Address most of the feedback Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Refactor the block meta for out of order Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Fix windows error Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Fix review comments Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> Signed-off-by: Ganesh Vernekar 15064823+codesome@users.noreply.github.com Co-authored-by: Ganesh Vernekar <15064823+codesome@users.noreply.github.com> Co-authored-by: Ganesh Vernekar <ganeshvern@gmail.com> Co-authored-by: Dieter Plaetinck <dieter@grafana.com> Co-authored-by: Oleg Zaytsev <mail@olegzaytsev.com> Co-authored-by: Bartlomiej Plotka <bwplotka@gmail.com>
2022-09-20 10:05:50 -07:00
idx := uint64(ms.ref) % uint64(n)
// It is possible that some old sample is being processed in processWALSamples that
// could cause race below. So we wait for the goroutine to empty input the buffer and finish
// processing all old samples after emptying the buffer.
processors[idx].waitUntilIdle()
// Lock the subset so we can modify the series object
processors[idx].mx.Lock()
// All samples till now have been m-mapped. Hence clear out the headChunk.
// In case some samples slipped through and went into m-map chunks because of changed
// chunk size parameters, we are not taking care of that here.
// TODO(codesome): see if there is a way to avoid duplicate m-map chunks if
// the size of ooo chunk was reduced between restart.
ms.oooHeadChunk = nil
processors[idx].mx.Unlock()
}
default:
panic(fmt.Errorf("unexpected decodedCh type: %T", d))
}
}
if decodeErr != nil {
return decodeErr
}
// Signal termination to each worker and wait for it to close its output channel.
for i := 0; i < n; i++ {
processors[i].closeAndDrain()
}
wg.Wait()
if r.Err() != nil {
return errors.Wrap(r.Err(), "read records")
}
if unknownRefs.Load() > 0 || mmapMarkerUnknownRefs.Load() > 0 {
level.Warn(h.logger).Log("msg", "Unknown series references for ooo WAL replay", "samples", unknownRefs.Load(), "mmap_markers", mmapMarkerUnknownRefs.Load())
}
return nil
}
type errLoadWbl struct {
err error
}
func (e errLoadWbl) Error() string {
return e.err.Error()
}
// To support errors.Cause().
func (e errLoadWbl) Cause() error {
return e.err
}
// To support errors.Unwrap().
func (e errLoadWbl) Unwrap() error {
return e.err
}
// isErrLoadOOOWal returns a boolean if the error is errLoadWbl.
func isErrLoadOOOWal(err error) bool {
_, ok := err.(*errLoadWbl)
return ok
}
type wblSubsetProcessor struct {
mx sync.Mutex // Take this lock while modifying series in the subset.
input chan []record.RefSample
output chan []record.RefSample
}
func (wp *wblSubsetProcessor) setup() {
wp.output = make(chan []record.RefSample, 300)
wp.input = make(chan []record.RefSample, 300)
}
func (wp *wblSubsetProcessor) closeAndDrain() {
close(wp.input)
for range wp.output {
}
}
// If there is a buffer in the output chan, return it for reuse, otherwise return nil.
func (wp *wblSubsetProcessor) reuseBuf() []record.RefSample {
select {
case buf := <-wp.output:
return buf[:0]
default:
}
return nil
}
// processWBLSamples adds the samples it receives to the head and passes
// the buffer received to an output channel for reuse.
// Samples before the minValidTime timestamp are discarded.
func (wp *wblSubsetProcessor) processWBLSamples(h *Head) (unknownRefs uint64) {
defer close(wp.output)
oooCapMax := h.opts.OutOfOrderCapMax.Load()
Add out-of-order sample support to the TSDB (#11075) * Introduce out-of-order TSDB support This implementation is based on this design doc: https://docs.google.com/document/d/1Kppm7qL9C-BJB1j6yb6-9ObG3AbdZnFUBYPNNWwDBYM/edit?usp=sharing This commit adds support to accept out-of-order ("OOO") sample into the TSDB up to a configurable time allowance. If OOO is enabled, overlapping querying are automatically enabled. Most of the additions have been borrowed from https://github.com/grafana/mimir-prometheus/ Here is the list ist of the original commits cherry picked from mimir-prometheus into this branch: - 4b2198d7ec47d50989b7c2df66b7b207c32f7f6e - 2836e5513f1bc591535a859f5d41154a75e7c6bc - 00b379c3a5b1ec3799699b6242f300a2b3ea30f0 - ff0dc757587cada63ca948d2d5eb00bf090d63e0 - a632c73352a7e39d60b445700beb47d691549c3e - c6f3d4ab339ab80bbbce74c9946237ced01f0509 - 5e8406a1d4a50d0052bbee83e28ca3b3371408aa - abde1e0ba128936b9eb0224ee1551e56216ebd4a - e70e7698897bb03860bee0467c733fa44e14c9bd - df59320886e03a555d379ac4b0b3130f661407e0 Co-authored-by: Jesus Vazquez <jesus.vazquez@grafana.com> Co-authored-by: Ganesh Vernekar <ganeshvern@gmail.com> Co-authored-by: Dieter Plaetinck <dieter@grafana.com> Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * gofumpt files Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Add license header to missing files Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Fix OOO tests due to existing chunk disk mapper implementation Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Fix truncate int overflow Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Add Sync method to the WAL and update tests Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * remove useless sync Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Update minOOOTime after truncating Head * Update minOOOTime after truncating Head Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Fix lint Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Add a unit test Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Load OutOfOrderTimeWindow only once per appender Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Fix OOO Head LabelValues and PostingsForMatchers Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Fix replay of OOO mmap chunks Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Remove unnecessary err check Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Prevent panic with ApplyConfig Signed-off-by: Ganesh Vernekar 15064823+codesome@users.noreply.github.com Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Run OOO compaction after restart if there is OOO data from WBL Signed-off-by: Ganesh Vernekar 15064823+codesome@users.noreply.github.com Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Apply Bartek's suggestions Co-authored-by: Bartlomiej Plotka <bwplotka@gmail.com> Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Refactor OOO compaction Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Address comments and TODOs - Added a comment explaining why we need the allow overlapping compaction toggle - Clarified TSDBConfig OutOfOrderTimeWindow doc - Added an owner to all the TODOs in the code Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Run go format Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Fix remaining review comments Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Fix tests Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Change wbl reference when truncating ooo in TestHeadMinOOOTimeUpdate Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Fix TestWBLAndMmapReplay test failure on windows Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Address most of the feedback Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Refactor the block meta for out of order Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Fix windows error Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Fix review comments Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> Signed-off-by: Ganesh Vernekar 15064823+codesome@users.noreply.github.com Co-authored-by: Ganesh Vernekar <15064823+codesome@users.noreply.github.com> Co-authored-by: Ganesh Vernekar <ganeshvern@gmail.com> Co-authored-by: Dieter Plaetinck <dieter@grafana.com> Co-authored-by: Oleg Zaytsev <mail@olegzaytsev.com> Co-authored-by: Bartlomiej Plotka <bwplotka@gmail.com>
2022-09-20 10:05:50 -07:00
// We don't check for minValidTime for ooo samples.
mint, maxt := int64(math.MaxInt64), int64(math.MinInt64)
for samples := range wp.input {
wp.mx.Lock()
for _, s := range samples {
ms := h.series.getByID(s.Ref)
if ms == nil {
unknownRefs++
continue
}
ok, chunkCreated, _ := ms.insert(s.T, s.V, h.chunkDiskMapper, oooCapMax)
Add out-of-order sample support to the TSDB (#11075) * Introduce out-of-order TSDB support This implementation is based on this design doc: https://docs.google.com/document/d/1Kppm7qL9C-BJB1j6yb6-9ObG3AbdZnFUBYPNNWwDBYM/edit?usp=sharing This commit adds support to accept out-of-order ("OOO") sample into the TSDB up to a configurable time allowance. If OOO is enabled, overlapping querying are automatically enabled. Most of the additions have been borrowed from https://github.com/grafana/mimir-prometheus/ Here is the list ist of the original commits cherry picked from mimir-prometheus into this branch: - 4b2198d7ec47d50989b7c2df66b7b207c32f7f6e - 2836e5513f1bc591535a859f5d41154a75e7c6bc - 00b379c3a5b1ec3799699b6242f300a2b3ea30f0 - ff0dc757587cada63ca948d2d5eb00bf090d63e0 - a632c73352a7e39d60b445700beb47d691549c3e - c6f3d4ab339ab80bbbce74c9946237ced01f0509 - 5e8406a1d4a50d0052bbee83e28ca3b3371408aa - abde1e0ba128936b9eb0224ee1551e56216ebd4a - e70e7698897bb03860bee0467c733fa44e14c9bd - df59320886e03a555d379ac4b0b3130f661407e0 Co-authored-by: Jesus Vazquez <jesus.vazquez@grafana.com> Co-authored-by: Ganesh Vernekar <ganeshvern@gmail.com> Co-authored-by: Dieter Plaetinck <dieter@grafana.com> Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * gofumpt files Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Add license header to missing files Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Fix OOO tests due to existing chunk disk mapper implementation Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Fix truncate int overflow Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Add Sync method to the WAL and update tests Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * remove useless sync Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Update minOOOTime after truncating Head * Update minOOOTime after truncating Head Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Fix lint Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Add a unit test Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Load OutOfOrderTimeWindow only once per appender Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Fix OOO Head LabelValues and PostingsForMatchers Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Fix replay of OOO mmap chunks Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Remove unnecessary err check Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Prevent panic with ApplyConfig Signed-off-by: Ganesh Vernekar 15064823+codesome@users.noreply.github.com Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Run OOO compaction after restart if there is OOO data from WBL Signed-off-by: Ganesh Vernekar 15064823+codesome@users.noreply.github.com Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Apply Bartek's suggestions Co-authored-by: Bartlomiej Plotka <bwplotka@gmail.com> Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Refactor OOO compaction Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Address comments and TODOs - Added a comment explaining why we need the allow overlapping compaction toggle - Clarified TSDBConfig OutOfOrderTimeWindow doc - Added an owner to all the TODOs in the code Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Run go format Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Fix remaining review comments Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Fix tests Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Change wbl reference when truncating ooo in TestHeadMinOOOTimeUpdate Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Fix TestWBLAndMmapReplay test failure on windows Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Address most of the feedback Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Refactor the block meta for out of order Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Fix windows error Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Fix review comments Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> Signed-off-by: Ganesh Vernekar 15064823+codesome@users.noreply.github.com Co-authored-by: Ganesh Vernekar <15064823+codesome@users.noreply.github.com> Co-authored-by: Ganesh Vernekar <ganeshvern@gmail.com> Co-authored-by: Dieter Plaetinck <dieter@grafana.com> Co-authored-by: Oleg Zaytsev <mail@olegzaytsev.com> Co-authored-by: Bartlomiej Plotka <bwplotka@gmail.com>
2022-09-20 10:05:50 -07:00
if chunkCreated {
h.metrics.chunksCreated.Inc()
h.metrics.chunks.Inc()
}
if ok {
if s.T < mint {
mint = s.T
2021-11-30 08:19:06 -08:00
}
if s.T > maxt {
maxt = s.T
}
}
}
wp.mx.Unlock()
2021-11-30 08:19:06 -08:00
}
Add out-of-order sample support to the TSDB (#11075) * Introduce out-of-order TSDB support This implementation is based on this design doc: https://docs.google.com/document/d/1Kppm7qL9C-BJB1j6yb6-9ObG3AbdZnFUBYPNNWwDBYM/edit?usp=sharing This commit adds support to accept out-of-order ("OOO") sample into the TSDB up to a configurable time allowance. If OOO is enabled, overlapping querying are automatically enabled. Most of the additions have been borrowed from https://github.com/grafana/mimir-prometheus/ Here is the list ist of the original commits cherry picked from mimir-prometheus into this branch: - 4b2198d7ec47d50989b7c2df66b7b207c32f7f6e - 2836e5513f1bc591535a859f5d41154a75e7c6bc - 00b379c3a5b1ec3799699b6242f300a2b3ea30f0 - ff0dc757587cada63ca948d2d5eb00bf090d63e0 - a632c73352a7e39d60b445700beb47d691549c3e - c6f3d4ab339ab80bbbce74c9946237ced01f0509 - 5e8406a1d4a50d0052bbee83e28ca3b3371408aa - abde1e0ba128936b9eb0224ee1551e56216ebd4a - e70e7698897bb03860bee0467c733fa44e14c9bd - df59320886e03a555d379ac4b0b3130f661407e0 Co-authored-by: Jesus Vazquez <jesus.vazquez@grafana.com> Co-authored-by: Ganesh Vernekar <ganeshvern@gmail.com> Co-authored-by: Dieter Plaetinck <dieter@grafana.com> Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * gofumpt files Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Add license header to missing files Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Fix OOO tests due to existing chunk disk mapper implementation Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Fix truncate int overflow Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Add Sync method to the WAL and update tests Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * remove useless sync Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Update minOOOTime after truncating Head * Update minOOOTime after truncating Head Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Fix lint Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Add a unit test Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Load OutOfOrderTimeWindow only once per appender Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Fix OOO Head LabelValues and PostingsForMatchers Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Fix replay of OOO mmap chunks Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Remove unnecessary err check Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Prevent panic with ApplyConfig Signed-off-by: Ganesh Vernekar 15064823+codesome@users.noreply.github.com Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Run OOO compaction after restart if there is OOO data from WBL Signed-off-by: Ganesh Vernekar 15064823+codesome@users.noreply.github.com Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Apply Bartek's suggestions Co-authored-by: Bartlomiej Plotka <bwplotka@gmail.com> Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Refactor OOO compaction Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Address comments and TODOs - Added a comment explaining why we need the allow overlapping compaction toggle - Clarified TSDBConfig OutOfOrderTimeWindow doc - Added an owner to all the TODOs in the code Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Run go format Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Fix remaining review comments Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Fix tests Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Change wbl reference when truncating ooo in TestHeadMinOOOTimeUpdate Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Fix TestWBLAndMmapReplay test failure on windows Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Address most of the feedback Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Refactor the block meta for out of order Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Fix windows error Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Fix review comments Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> Signed-off-by: Ganesh Vernekar 15064823+codesome@users.noreply.github.com Co-authored-by: Ganesh Vernekar <15064823+codesome@users.noreply.github.com> Co-authored-by: Ganesh Vernekar <ganeshvern@gmail.com> Co-authored-by: Dieter Plaetinck <dieter@grafana.com> Co-authored-by: Oleg Zaytsev <mail@olegzaytsev.com> Co-authored-by: Bartlomiej Plotka <bwplotka@gmail.com>
2022-09-20 10:05:50 -07:00
h.updateMinOOOMaxOOOTime(mint, maxt)
return unknownRefs
}
Add out-of-order sample support to the TSDB (#11075) * Introduce out-of-order TSDB support This implementation is based on this design doc: https://docs.google.com/document/d/1Kppm7qL9C-BJB1j6yb6-9ObG3AbdZnFUBYPNNWwDBYM/edit?usp=sharing This commit adds support to accept out-of-order ("OOO") sample into the TSDB up to a configurable time allowance. If OOO is enabled, overlapping querying are automatically enabled. Most of the additions have been borrowed from https://github.com/grafana/mimir-prometheus/ Here is the list ist of the original commits cherry picked from mimir-prometheus into this branch: - 4b2198d7ec47d50989b7c2df66b7b207c32f7f6e - 2836e5513f1bc591535a859f5d41154a75e7c6bc - 00b379c3a5b1ec3799699b6242f300a2b3ea30f0 - ff0dc757587cada63ca948d2d5eb00bf090d63e0 - a632c73352a7e39d60b445700beb47d691549c3e - c6f3d4ab339ab80bbbce74c9946237ced01f0509 - 5e8406a1d4a50d0052bbee83e28ca3b3371408aa - abde1e0ba128936b9eb0224ee1551e56216ebd4a - e70e7698897bb03860bee0467c733fa44e14c9bd - df59320886e03a555d379ac4b0b3130f661407e0 Co-authored-by: Jesus Vazquez <jesus.vazquez@grafana.com> Co-authored-by: Ganesh Vernekar <ganeshvern@gmail.com> Co-authored-by: Dieter Plaetinck <dieter@grafana.com> Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * gofumpt files Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Add license header to missing files Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Fix OOO tests due to existing chunk disk mapper implementation Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Fix truncate int overflow Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Add Sync method to the WAL and update tests Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * remove useless sync Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Update minOOOTime after truncating Head * Update minOOOTime after truncating Head Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Fix lint Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Add a unit test Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Load OutOfOrderTimeWindow only once per appender Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Fix OOO Head LabelValues and PostingsForMatchers Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Fix replay of OOO mmap chunks Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Remove unnecessary err check Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Prevent panic with ApplyConfig Signed-off-by: Ganesh Vernekar 15064823+codesome@users.noreply.github.com Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Run OOO compaction after restart if there is OOO data from WBL Signed-off-by: Ganesh Vernekar 15064823+codesome@users.noreply.github.com Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Apply Bartek's suggestions Co-authored-by: Bartlomiej Plotka <bwplotka@gmail.com> Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Refactor OOO compaction Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Address comments and TODOs - Added a comment explaining why we need the allow overlapping compaction toggle - Clarified TSDBConfig OutOfOrderTimeWindow doc - Added an owner to all the TODOs in the code Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Run go format Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Fix remaining review comments Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Fix tests Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Change wbl reference when truncating ooo in TestHeadMinOOOTimeUpdate Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Fix TestWBLAndMmapReplay test failure on windows Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Address most of the feedback Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Refactor the block meta for out of order Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Fix windows error Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Fix review comments Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> Signed-off-by: Ganesh Vernekar 15064823+codesome@users.noreply.github.com Co-authored-by: Ganesh Vernekar <15064823+codesome@users.noreply.github.com> Co-authored-by: Ganesh Vernekar <ganeshvern@gmail.com> Co-authored-by: Dieter Plaetinck <dieter@grafana.com> Co-authored-by: Oleg Zaytsev <mail@olegzaytsev.com> Co-authored-by: Bartlomiej Plotka <bwplotka@gmail.com>
2022-09-20 10:05:50 -07:00
func (wp *wblSubsetProcessor) waitUntilIdle() {
select {
case <-wp.output: // Allow output side to drain to avoid deadlock.
default:
}
wp.input <- []record.RefSample{}
for len(wp.input) != 0 {
Add out-of-order sample support to the TSDB (#11075) * Introduce out-of-order TSDB support This implementation is based on this design doc: https://docs.google.com/document/d/1Kppm7qL9C-BJB1j6yb6-9ObG3AbdZnFUBYPNNWwDBYM/edit?usp=sharing This commit adds support to accept out-of-order ("OOO") sample into the TSDB up to a configurable time allowance. If OOO is enabled, overlapping querying are automatically enabled. Most of the additions have been borrowed from https://github.com/grafana/mimir-prometheus/ Here is the list ist of the original commits cherry picked from mimir-prometheus into this branch: - 4b2198d7ec47d50989b7c2df66b7b207c32f7f6e - 2836e5513f1bc591535a859f5d41154a75e7c6bc - 00b379c3a5b1ec3799699b6242f300a2b3ea30f0 - ff0dc757587cada63ca948d2d5eb00bf090d63e0 - a632c73352a7e39d60b445700beb47d691549c3e - c6f3d4ab339ab80bbbce74c9946237ced01f0509 - 5e8406a1d4a50d0052bbee83e28ca3b3371408aa - abde1e0ba128936b9eb0224ee1551e56216ebd4a - e70e7698897bb03860bee0467c733fa44e14c9bd - df59320886e03a555d379ac4b0b3130f661407e0 Co-authored-by: Jesus Vazquez <jesus.vazquez@grafana.com> Co-authored-by: Ganesh Vernekar <ganeshvern@gmail.com> Co-authored-by: Dieter Plaetinck <dieter@grafana.com> Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * gofumpt files Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Add license header to missing files Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Fix OOO tests due to existing chunk disk mapper implementation Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Fix truncate int overflow Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Add Sync method to the WAL and update tests Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * remove useless sync Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Update minOOOTime after truncating Head * Update minOOOTime after truncating Head Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Fix lint Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Add a unit test Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Load OutOfOrderTimeWindow only once per appender Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Fix OOO Head LabelValues and PostingsForMatchers Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Fix replay of OOO mmap chunks Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Remove unnecessary err check Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Prevent panic with ApplyConfig Signed-off-by: Ganesh Vernekar 15064823+codesome@users.noreply.github.com Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Run OOO compaction after restart if there is OOO data from WBL Signed-off-by: Ganesh Vernekar 15064823+codesome@users.noreply.github.com Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Apply Bartek's suggestions Co-authored-by: Bartlomiej Plotka <bwplotka@gmail.com> Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Refactor OOO compaction Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Address comments and TODOs - Added a comment explaining why we need the allow overlapping compaction toggle - Clarified TSDBConfig OutOfOrderTimeWindow doc - Added an owner to all the TODOs in the code Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Run go format Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Fix remaining review comments Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Fix tests Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Change wbl reference when truncating ooo in TestHeadMinOOOTimeUpdate Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> * Fix TestWBLAndMmapReplay test failure on windows Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Address most of the feedback Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Refactor the block meta for out of order Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Fix windows error Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Fix review comments Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> Signed-off-by: Jesus Vazquez <jesus.vazquez@grafana.com> Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> Signed-off-by: Ganesh Vernekar 15064823+codesome@users.noreply.github.com Co-authored-by: Ganesh Vernekar <15064823+codesome@users.noreply.github.com> Co-authored-by: Ganesh Vernekar <ganeshvern@gmail.com> Co-authored-by: Dieter Plaetinck <dieter@grafana.com> Co-authored-by: Oleg Zaytsev <mail@olegzaytsev.com> Co-authored-by: Bartlomiej Plotka <bwplotka@gmail.com>
2022-09-20 10:05:50 -07:00
time.Sleep(10 * time.Microsecond)
select {
case <-wp.output: // Allow output side to drain to avoid deadlock.
2021-11-30 08:19:06 -08:00
default:
}
}
}
Sync sparsehistogram branch with main (#9189) * Fix `kuma_sd` targetgroup reporting (#9157) * Bundle all xDS targets into a single group Signed-off-by: austin ce <austin.cawley@gmail.com> * Snapshot in-memory chunks on shutdown for faster restarts (#7229) Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Rename links Signed-off-by: Levi Harrison <git@leviharrison.dev> * Remove Individual Data Type Caps in Per-shard Buffering for Remote Write (#8921) * Moved everything to nPending buffer Signed-off-by: Levi Harrison <git@leviharrison.dev> * Simplify exemplar capacity addition Signed-off-by: Levi Harrison <git@leviharrison.dev> * Added pre-allocation Signed-off-by: Levi Harrison <git@leviharrison.dev> * Don't allocate if not sending exemplars Signed-off-by: Levi Harrison <git@leviharrison.dev> * Avoid deadlock when processing duplicate series record (#9170) * Avoid deadlock when processing duplicate series record `processWALSamples()` needs to be able to send on its output channel before it can read the input channel, so reads to allow this in case the output channel is full. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * processWALSamples: update comment Previous text seems to relate to an earlier implementation. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Optimise WAL loading by removing extra map and caching min-time (#9160) * BenchmarkLoadWAL: close WAL after use So that goroutines are stopped and resources released Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * BenchmarkLoadWAL: make series IDs co-prime with #workers Series are distributed across workers by taking the modulus of the ID with the number of workers, so multiples of 100 are a poor choice. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * BenchmarkLoadWAL: simulate mmapped chunks Real Prometheus cuts chunks every 120 samples, then skips those samples when re-reading the WAL. Simulate this by creating a single mapped chunk for each series, since the max time is all the reader looks at. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Fix comment Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Remove series map from processWALSamples() The locks that is commented to reduce contention in are now sharded 32,000 ways, so won't be contended. Removing the map saves memory and goes just as fast. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * loadWAL: Cache the last mmapped chunk time So we can skip calling append() for samples it will reject. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Improvements from code review Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Full stops and capitals on comments Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Cache max time in both places mmappedChunks is updated Including refactor to extract function `setMMappedChunks`, to reduce code duplication. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Update head min/max time when mmapped chunks added This ensures we have the correct values if no WAL samples are added for that series. Note that `mSeries.maxTime()` was always `math.MinInt64` before, since that function doesn't consider mmapped chunks. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Split Go and React Tests (#8897) * Added go-ci and react-ci Co-authored-by: Julien Pivotto <roidelapluie@inuits.eu> Signed-off-by: Levi Harrison <git@leviharrison.dev> * Remove search keymap from new expression editor (#9184) Signed-off-by: Julius Volz <julius.volz@gmail.com> Co-authored-by: Austin Cawley-Edwards <austin.cawley@gmail.com> Co-authored-by: Levi Harrison <git@leviharrison.dev> Co-authored-by: Julien Pivotto <roidelapluie@inuits.eu> Co-authored-by: Bryan Boreham <bjboreham@gmail.com> Co-authored-by: Julius Volz <julius.volz@gmail.com>
2021-08-11 03:13:17 -07:00
const (
chunkSnapshotRecordTypeSeries uint8 = 1
chunkSnapshotRecordTypeTombstones uint8 = 2
chunkSnapshotRecordTypeExemplars uint8 = 3
Sync sparsehistogram branch with main (#9189) * Fix `kuma_sd` targetgroup reporting (#9157) * Bundle all xDS targets into a single group Signed-off-by: austin ce <austin.cawley@gmail.com> * Snapshot in-memory chunks on shutdown for faster restarts (#7229) Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Rename links Signed-off-by: Levi Harrison <git@leviharrison.dev> * Remove Individual Data Type Caps in Per-shard Buffering for Remote Write (#8921) * Moved everything to nPending buffer Signed-off-by: Levi Harrison <git@leviharrison.dev> * Simplify exemplar capacity addition Signed-off-by: Levi Harrison <git@leviharrison.dev> * Added pre-allocation Signed-off-by: Levi Harrison <git@leviharrison.dev> * Don't allocate if not sending exemplars Signed-off-by: Levi Harrison <git@leviharrison.dev> * Avoid deadlock when processing duplicate series record (#9170) * Avoid deadlock when processing duplicate series record `processWALSamples()` needs to be able to send on its output channel before it can read the input channel, so reads to allow this in case the output channel is full. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * processWALSamples: update comment Previous text seems to relate to an earlier implementation. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Optimise WAL loading by removing extra map and caching min-time (#9160) * BenchmarkLoadWAL: close WAL after use So that goroutines are stopped and resources released Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * BenchmarkLoadWAL: make series IDs co-prime with #workers Series are distributed across workers by taking the modulus of the ID with the number of workers, so multiples of 100 are a poor choice. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * BenchmarkLoadWAL: simulate mmapped chunks Real Prometheus cuts chunks every 120 samples, then skips those samples when re-reading the WAL. Simulate this by creating a single mapped chunk for each series, since the max time is all the reader looks at. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Fix comment Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Remove series map from processWALSamples() The locks that is commented to reduce contention in are now sharded 32,000 ways, so won't be contended. Removing the map saves memory and goes just as fast. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * loadWAL: Cache the last mmapped chunk time So we can skip calling append() for samples it will reject. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Improvements from code review Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Full stops and capitals on comments Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Cache max time in both places mmappedChunks is updated Including refactor to extract function `setMMappedChunks`, to reduce code duplication. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Update head min/max time when mmapped chunks added This ensures we have the correct values if no WAL samples are added for that series. Note that `mSeries.maxTime()` was always `math.MinInt64` before, since that function doesn't consider mmapped chunks. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Split Go and React Tests (#8897) * Added go-ci and react-ci Co-authored-by: Julien Pivotto <roidelapluie@inuits.eu> Signed-off-by: Levi Harrison <git@leviharrison.dev> * Remove search keymap from new expression editor (#9184) Signed-off-by: Julius Volz <julius.volz@gmail.com> Co-authored-by: Austin Cawley-Edwards <austin.cawley@gmail.com> Co-authored-by: Levi Harrison <git@leviharrison.dev> Co-authored-by: Julien Pivotto <roidelapluie@inuits.eu> Co-authored-by: Bryan Boreham <bjboreham@gmail.com> Co-authored-by: Julius Volz <julius.volz@gmail.com>
2021-08-11 03:13:17 -07:00
)
type chunkSnapshotRecord struct {
ref chunks.HeadSeriesRef
lset labels.Labels
mc *memChunk
tsdb: stop saving a copy of last 4 samples in memSeries (#11296) * TSDB chunks: remove race between writing and reading Because the data is stored as a bit-stream, the last byte in the stream could change if the stream is appended to after an Iterator is obtained. Copy the last byte when the Iterator is created, so we don't have to read it later. Clarify in comments that concurrent Iterator and Appender are allowed, but the chunk must not be modified while an Iterator is created. (This was already the case, in order to copy the bstream slice header.) * TSDB: stop saving last 4 samples in memSeries This extra copy of the last 4 samples was introduced to avoid a race condition between reading the last byte of the chunk and writing to it. But now we have fixed that by having `bstreamReader` copy the last byte, we don't need to copy the last 4 samples. This change saves 56 bytes per series, which is very worthwhile when you have millions or tens of millions of series. * TSDB: tidy up stopIterator re-use Previous changes have left this code duplicating some lines; pull them out to a separate function and tidy up. * TSDB head_test: stop checking when iterators are wrapped The behaviour has changed so chunk iterators are only wrapped when transaction isolation requires them to stop short of the end. This makes tests fail which are checking the type. Tests should check the observable behaviour, not the type. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> Co-authored-by: Ganesh Vernekar <ganeshvern@gmail.com>
2022-09-27 07:02:05 -07:00
lastValue float64
Sync sparsehistogram branch with main (#9189) * Fix `kuma_sd` targetgroup reporting (#9157) * Bundle all xDS targets into a single group Signed-off-by: austin ce <austin.cawley@gmail.com> * Snapshot in-memory chunks on shutdown for faster restarts (#7229) Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Rename links Signed-off-by: Levi Harrison <git@leviharrison.dev> * Remove Individual Data Type Caps in Per-shard Buffering for Remote Write (#8921) * Moved everything to nPending buffer Signed-off-by: Levi Harrison <git@leviharrison.dev> * Simplify exemplar capacity addition Signed-off-by: Levi Harrison <git@leviharrison.dev> * Added pre-allocation Signed-off-by: Levi Harrison <git@leviharrison.dev> * Don't allocate if not sending exemplars Signed-off-by: Levi Harrison <git@leviharrison.dev> * Avoid deadlock when processing duplicate series record (#9170) * Avoid deadlock when processing duplicate series record `processWALSamples()` needs to be able to send on its output channel before it can read the input channel, so reads to allow this in case the output channel is full. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * processWALSamples: update comment Previous text seems to relate to an earlier implementation. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Optimise WAL loading by removing extra map and caching min-time (#9160) * BenchmarkLoadWAL: close WAL after use So that goroutines are stopped and resources released Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * BenchmarkLoadWAL: make series IDs co-prime with #workers Series are distributed across workers by taking the modulus of the ID with the number of workers, so multiples of 100 are a poor choice. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * BenchmarkLoadWAL: simulate mmapped chunks Real Prometheus cuts chunks every 120 samples, then skips those samples when re-reading the WAL. Simulate this by creating a single mapped chunk for each series, since the max time is all the reader looks at. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Fix comment Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Remove series map from processWALSamples() The locks that is commented to reduce contention in are now sharded 32,000 ways, so won't be contended. Removing the map saves memory and goes just as fast. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * loadWAL: Cache the last mmapped chunk time So we can skip calling append() for samples it will reject. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Improvements from code review Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Full stops and capitals on comments Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Cache max time in both places mmappedChunks is updated Including refactor to extract function `setMMappedChunks`, to reduce code duplication. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Update head min/max time when mmapped chunks added This ensures we have the correct values if no WAL samples are added for that series. Note that `mSeries.maxTime()` was always `math.MinInt64` before, since that function doesn't consider mmapped chunks. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Split Go and React Tests (#8897) * Added go-ci and react-ci Co-authored-by: Julien Pivotto <roidelapluie@inuits.eu> Signed-off-by: Levi Harrison <git@leviharrison.dev> * Remove search keymap from new expression editor (#9184) Signed-off-by: Julius Volz <julius.volz@gmail.com> Co-authored-by: Austin Cawley-Edwards <austin.cawley@gmail.com> Co-authored-by: Levi Harrison <git@leviharrison.dev> Co-authored-by: Julien Pivotto <roidelapluie@inuits.eu> Co-authored-by: Bryan Boreham <bjboreham@gmail.com> Co-authored-by: Julius Volz <julius.volz@gmail.com>
2021-08-11 03:13:17 -07:00
}
func (s *memSeries) encodeToSnapshotRecord(b []byte) []byte {
buf := encoding.Encbuf{B: b}
buf.PutByte(chunkSnapshotRecordTypeSeries)
buf.PutBE64(uint64(s.ref))
record.EncodeLabels(&buf, s.lset)
buf.PutBE64int64(0) // Backwards-compatibility; was chunkRange but now unused.
Sync sparsehistogram branch with main (#9189) * Fix `kuma_sd` targetgroup reporting (#9157) * Bundle all xDS targets into a single group Signed-off-by: austin ce <austin.cawley@gmail.com> * Snapshot in-memory chunks on shutdown for faster restarts (#7229) Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Rename links Signed-off-by: Levi Harrison <git@leviharrison.dev> * Remove Individual Data Type Caps in Per-shard Buffering for Remote Write (#8921) * Moved everything to nPending buffer Signed-off-by: Levi Harrison <git@leviharrison.dev> * Simplify exemplar capacity addition Signed-off-by: Levi Harrison <git@leviharrison.dev> * Added pre-allocation Signed-off-by: Levi Harrison <git@leviharrison.dev> * Don't allocate if not sending exemplars Signed-off-by: Levi Harrison <git@leviharrison.dev> * Avoid deadlock when processing duplicate series record (#9170) * Avoid deadlock when processing duplicate series record `processWALSamples()` needs to be able to send on its output channel before it can read the input channel, so reads to allow this in case the output channel is full. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * processWALSamples: update comment Previous text seems to relate to an earlier implementation. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Optimise WAL loading by removing extra map and caching min-time (#9160) * BenchmarkLoadWAL: close WAL after use So that goroutines are stopped and resources released Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * BenchmarkLoadWAL: make series IDs co-prime with #workers Series are distributed across workers by taking the modulus of the ID with the number of workers, so multiples of 100 are a poor choice. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * BenchmarkLoadWAL: simulate mmapped chunks Real Prometheus cuts chunks every 120 samples, then skips those samples when re-reading the WAL. Simulate this by creating a single mapped chunk for each series, since the max time is all the reader looks at. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Fix comment Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Remove series map from processWALSamples() The locks that is commented to reduce contention in are now sharded 32,000 ways, so won't be contended. Removing the map saves memory and goes just as fast. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * loadWAL: Cache the last mmapped chunk time So we can skip calling append() for samples it will reject. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Improvements from code review Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Full stops and capitals on comments Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Cache max time in both places mmappedChunks is updated Including refactor to extract function `setMMappedChunks`, to reduce code duplication. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Update head min/max time when mmapped chunks added This ensures we have the correct values if no WAL samples are added for that series. Note that `mSeries.maxTime()` was always `math.MinInt64` before, since that function doesn't consider mmapped chunks. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Split Go and React Tests (#8897) * Added go-ci and react-ci Co-authored-by: Julien Pivotto <roidelapluie@inuits.eu> Signed-off-by: Levi Harrison <git@leviharrison.dev> * Remove search keymap from new expression editor (#9184) Signed-off-by: Julius Volz <julius.volz@gmail.com> Co-authored-by: Austin Cawley-Edwards <austin.cawley@gmail.com> Co-authored-by: Levi Harrison <git@leviharrison.dev> Co-authored-by: Julien Pivotto <roidelapluie@inuits.eu> Co-authored-by: Bryan Boreham <bjboreham@gmail.com> Co-authored-by: Julius Volz <julius.volz@gmail.com>
2021-08-11 03:13:17 -07:00
s.Lock()
if s.headChunk == nil {
buf.PutUvarint(0)
} else {
buf.PutUvarint(1)
buf.PutBE64int64(s.headChunk.minTime)
buf.PutBE64int64(s.headChunk.maxTime)
buf.PutByte(byte(s.headChunk.chunk.Encoding()))
buf.PutUvarintBytes(s.headChunk.chunk.Bytes())
tsdb: stop saving a copy of last 4 samples in memSeries (#11296) * TSDB chunks: remove race between writing and reading Because the data is stored as a bit-stream, the last byte in the stream could change if the stream is appended to after an Iterator is obtained. Copy the last byte when the Iterator is created, so we don't have to read it later. Clarify in comments that concurrent Iterator and Appender are allowed, but the chunk must not be modified while an Iterator is created. (This was already the case, in order to copy the bstream slice header.) * TSDB: stop saving last 4 samples in memSeries This extra copy of the last 4 samples was introduced to avoid a race condition between reading the last byte of the chunk and writing to it. But now we have fixed that by having `bstreamReader` copy the last byte, we don't need to copy the last 4 samples. This change saves 56 bytes per series, which is very worthwhile when you have millions or tens of millions of series. * TSDB: tidy up stopIterator re-use Previous changes have left this code duplicating some lines; pull them out to a separate function and tidy up. * TSDB head_test: stop checking when iterators are wrapped The behaviour has changed so chunk iterators are only wrapped when transaction isolation requires them to stop short of the end. This makes tests fail which are checking the type. Tests should check the observable behaviour, not the type. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> Co-authored-by: Ganesh Vernekar <ganeshvern@gmail.com>
2022-09-27 07:02:05 -07:00
// Backwards compatibility for old sampleBuf which had last 4 samples.
for i := 0; i < 3; i++ {
buf.PutBE64int64(0)
buf.PutBEFloat64(0)
Sync sparsehistogram branch with main (#9189) * Fix `kuma_sd` targetgroup reporting (#9157) * Bundle all xDS targets into a single group Signed-off-by: austin ce <austin.cawley@gmail.com> * Snapshot in-memory chunks on shutdown for faster restarts (#7229) Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Rename links Signed-off-by: Levi Harrison <git@leviharrison.dev> * Remove Individual Data Type Caps in Per-shard Buffering for Remote Write (#8921) * Moved everything to nPending buffer Signed-off-by: Levi Harrison <git@leviharrison.dev> * Simplify exemplar capacity addition Signed-off-by: Levi Harrison <git@leviharrison.dev> * Added pre-allocation Signed-off-by: Levi Harrison <git@leviharrison.dev> * Don't allocate if not sending exemplars Signed-off-by: Levi Harrison <git@leviharrison.dev> * Avoid deadlock when processing duplicate series record (#9170) * Avoid deadlock when processing duplicate series record `processWALSamples()` needs to be able to send on its output channel before it can read the input channel, so reads to allow this in case the output channel is full. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * processWALSamples: update comment Previous text seems to relate to an earlier implementation. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Optimise WAL loading by removing extra map and caching min-time (#9160) * BenchmarkLoadWAL: close WAL after use So that goroutines are stopped and resources released Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * BenchmarkLoadWAL: make series IDs co-prime with #workers Series are distributed across workers by taking the modulus of the ID with the number of workers, so multiples of 100 are a poor choice. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * BenchmarkLoadWAL: simulate mmapped chunks Real Prometheus cuts chunks every 120 samples, then skips those samples when re-reading the WAL. Simulate this by creating a single mapped chunk for each series, since the max time is all the reader looks at. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Fix comment Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Remove series map from processWALSamples() The locks that is commented to reduce contention in are now sharded 32,000 ways, so won't be contended. Removing the map saves memory and goes just as fast. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * loadWAL: Cache the last mmapped chunk time So we can skip calling append() for samples it will reject. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Improvements from code review Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Full stops and capitals on comments Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Cache max time in both places mmappedChunks is updated Including refactor to extract function `setMMappedChunks`, to reduce code duplication. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Update head min/max time when mmapped chunks added This ensures we have the correct values if no WAL samples are added for that series. Note that `mSeries.maxTime()` was always `math.MinInt64` before, since that function doesn't consider mmapped chunks. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Split Go and React Tests (#8897) * Added go-ci and react-ci Co-authored-by: Julien Pivotto <roidelapluie@inuits.eu> Signed-off-by: Levi Harrison <git@leviharrison.dev> * Remove search keymap from new expression editor (#9184) Signed-off-by: Julius Volz <julius.volz@gmail.com> Co-authored-by: Austin Cawley-Edwards <austin.cawley@gmail.com> Co-authored-by: Levi Harrison <git@leviharrison.dev> Co-authored-by: Julien Pivotto <roidelapluie@inuits.eu> Co-authored-by: Bryan Boreham <bjboreham@gmail.com> Co-authored-by: Julius Volz <julius.volz@gmail.com>
2021-08-11 03:13:17 -07:00
}
tsdb: stop saving a copy of last 4 samples in memSeries (#11296) * TSDB chunks: remove race between writing and reading Because the data is stored as a bit-stream, the last byte in the stream could change if the stream is appended to after an Iterator is obtained. Copy the last byte when the Iterator is created, so we don't have to read it later. Clarify in comments that concurrent Iterator and Appender are allowed, but the chunk must not be modified while an Iterator is created. (This was already the case, in order to copy the bstream slice header.) * TSDB: stop saving last 4 samples in memSeries This extra copy of the last 4 samples was introduced to avoid a race condition between reading the last byte of the chunk and writing to it. But now we have fixed that by having `bstreamReader` copy the last byte, we don't need to copy the last 4 samples. This change saves 56 bytes per series, which is very worthwhile when you have millions or tens of millions of series. * TSDB: tidy up stopIterator re-use Previous changes have left this code duplicating some lines; pull them out to a separate function and tidy up. * TSDB head_test: stop checking when iterators are wrapped The behaviour has changed so chunk iterators are only wrapped when transaction isolation requires them to stop short of the end. This makes tests fail which are checking the type. Tests should check the observable behaviour, not the type. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> Co-authored-by: Ganesh Vernekar <ganeshvern@gmail.com>
2022-09-27 07:02:05 -07:00
buf.PutBE64int64(0)
buf.PutBEFloat64(s.lastValue)
Sync sparsehistogram branch with main (#9189) * Fix `kuma_sd` targetgroup reporting (#9157) * Bundle all xDS targets into a single group Signed-off-by: austin ce <austin.cawley@gmail.com> * Snapshot in-memory chunks on shutdown for faster restarts (#7229) Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Rename links Signed-off-by: Levi Harrison <git@leviharrison.dev> * Remove Individual Data Type Caps in Per-shard Buffering for Remote Write (#8921) * Moved everything to nPending buffer Signed-off-by: Levi Harrison <git@leviharrison.dev> * Simplify exemplar capacity addition Signed-off-by: Levi Harrison <git@leviharrison.dev> * Added pre-allocation Signed-off-by: Levi Harrison <git@leviharrison.dev> * Don't allocate if not sending exemplars Signed-off-by: Levi Harrison <git@leviharrison.dev> * Avoid deadlock when processing duplicate series record (#9170) * Avoid deadlock when processing duplicate series record `processWALSamples()` needs to be able to send on its output channel before it can read the input channel, so reads to allow this in case the output channel is full. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * processWALSamples: update comment Previous text seems to relate to an earlier implementation. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Optimise WAL loading by removing extra map and caching min-time (#9160) * BenchmarkLoadWAL: close WAL after use So that goroutines are stopped and resources released Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * BenchmarkLoadWAL: make series IDs co-prime with #workers Series are distributed across workers by taking the modulus of the ID with the number of workers, so multiples of 100 are a poor choice. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * BenchmarkLoadWAL: simulate mmapped chunks Real Prometheus cuts chunks every 120 samples, then skips those samples when re-reading the WAL. Simulate this by creating a single mapped chunk for each series, since the max time is all the reader looks at. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Fix comment Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Remove series map from processWALSamples() The locks that is commented to reduce contention in are now sharded 32,000 ways, so won't be contended. Removing the map saves memory and goes just as fast. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * loadWAL: Cache the last mmapped chunk time So we can skip calling append() for samples it will reject. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Improvements from code review Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Full stops and capitals on comments Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Cache max time in both places mmappedChunks is updated Including refactor to extract function `setMMappedChunks`, to reduce code duplication. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Update head min/max time when mmapped chunks added This ensures we have the correct values if no WAL samples are added for that series. Note that `mSeries.maxTime()` was always `math.MinInt64` before, since that function doesn't consider mmapped chunks. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Split Go and React Tests (#8897) * Added go-ci and react-ci Co-authored-by: Julien Pivotto <roidelapluie@inuits.eu> Signed-off-by: Levi Harrison <git@leviharrison.dev> * Remove search keymap from new expression editor (#9184) Signed-off-by: Julius Volz <julius.volz@gmail.com> Co-authored-by: Austin Cawley-Edwards <austin.cawley@gmail.com> Co-authored-by: Levi Harrison <git@leviharrison.dev> Co-authored-by: Julien Pivotto <roidelapluie@inuits.eu> Co-authored-by: Bryan Boreham <bjboreham@gmail.com> Co-authored-by: Julius Volz <julius.volz@gmail.com>
2021-08-11 03:13:17 -07:00
}
s.Unlock()
return buf.Get()
}
func decodeSeriesFromChunkSnapshot(d *record.Decoder, b []byte) (csr chunkSnapshotRecord, err error) {
Sync sparsehistogram branch with main (#9189) * Fix `kuma_sd` targetgroup reporting (#9157) * Bundle all xDS targets into a single group Signed-off-by: austin ce <austin.cawley@gmail.com> * Snapshot in-memory chunks on shutdown for faster restarts (#7229) Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Rename links Signed-off-by: Levi Harrison <git@leviharrison.dev> * Remove Individual Data Type Caps in Per-shard Buffering for Remote Write (#8921) * Moved everything to nPending buffer Signed-off-by: Levi Harrison <git@leviharrison.dev> * Simplify exemplar capacity addition Signed-off-by: Levi Harrison <git@leviharrison.dev> * Added pre-allocation Signed-off-by: Levi Harrison <git@leviharrison.dev> * Don't allocate if not sending exemplars Signed-off-by: Levi Harrison <git@leviharrison.dev> * Avoid deadlock when processing duplicate series record (#9170) * Avoid deadlock when processing duplicate series record `processWALSamples()` needs to be able to send on its output channel before it can read the input channel, so reads to allow this in case the output channel is full. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * processWALSamples: update comment Previous text seems to relate to an earlier implementation. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Optimise WAL loading by removing extra map and caching min-time (#9160) * BenchmarkLoadWAL: close WAL after use So that goroutines are stopped and resources released Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * BenchmarkLoadWAL: make series IDs co-prime with #workers Series are distributed across workers by taking the modulus of the ID with the number of workers, so multiples of 100 are a poor choice. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * BenchmarkLoadWAL: simulate mmapped chunks Real Prometheus cuts chunks every 120 samples, then skips those samples when re-reading the WAL. Simulate this by creating a single mapped chunk for each series, since the max time is all the reader looks at. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Fix comment Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Remove series map from processWALSamples() The locks that is commented to reduce contention in are now sharded 32,000 ways, so won't be contended. Removing the map saves memory and goes just as fast. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * loadWAL: Cache the last mmapped chunk time So we can skip calling append() for samples it will reject. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Improvements from code review Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Full stops and capitals on comments Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Cache max time in both places mmappedChunks is updated Including refactor to extract function `setMMappedChunks`, to reduce code duplication. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Update head min/max time when mmapped chunks added This ensures we have the correct values if no WAL samples are added for that series. Note that `mSeries.maxTime()` was always `math.MinInt64` before, since that function doesn't consider mmapped chunks. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Split Go and React Tests (#8897) * Added go-ci and react-ci Co-authored-by: Julien Pivotto <roidelapluie@inuits.eu> Signed-off-by: Levi Harrison <git@leviharrison.dev> * Remove search keymap from new expression editor (#9184) Signed-off-by: Julius Volz <julius.volz@gmail.com> Co-authored-by: Austin Cawley-Edwards <austin.cawley@gmail.com> Co-authored-by: Levi Harrison <git@leviharrison.dev> Co-authored-by: Julien Pivotto <roidelapluie@inuits.eu> Co-authored-by: Bryan Boreham <bjboreham@gmail.com> Co-authored-by: Julius Volz <julius.volz@gmail.com>
2021-08-11 03:13:17 -07:00
dec := encoding.Decbuf{B: b}
if flag := dec.Byte(); flag != chunkSnapshotRecordTypeSeries {
return csr, errors.Errorf("invalid record type %x", flag)
}
csr.ref = chunks.HeadSeriesRef(dec.Be64())
Sync sparsehistogram branch with main (#9189) * Fix `kuma_sd` targetgroup reporting (#9157) * Bundle all xDS targets into a single group Signed-off-by: austin ce <austin.cawley@gmail.com> * Snapshot in-memory chunks on shutdown for faster restarts (#7229) Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Rename links Signed-off-by: Levi Harrison <git@leviharrison.dev> * Remove Individual Data Type Caps in Per-shard Buffering for Remote Write (#8921) * Moved everything to nPending buffer Signed-off-by: Levi Harrison <git@leviharrison.dev> * Simplify exemplar capacity addition Signed-off-by: Levi Harrison <git@leviharrison.dev> * Added pre-allocation Signed-off-by: Levi Harrison <git@leviharrison.dev> * Don't allocate if not sending exemplars Signed-off-by: Levi Harrison <git@leviharrison.dev> * Avoid deadlock when processing duplicate series record (#9170) * Avoid deadlock when processing duplicate series record `processWALSamples()` needs to be able to send on its output channel before it can read the input channel, so reads to allow this in case the output channel is full. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * processWALSamples: update comment Previous text seems to relate to an earlier implementation. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Optimise WAL loading by removing extra map and caching min-time (#9160) * BenchmarkLoadWAL: close WAL after use So that goroutines are stopped and resources released Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * BenchmarkLoadWAL: make series IDs co-prime with #workers Series are distributed across workers by taking the modulus of the ID with the number of workers, so multiples of 100 are a poor choice. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * BenchmarkLoadWAL: simulate mmapped chunks Real Prometheus cuts chunks every 120 samples, then skips those samples when re-reading the WAL. Simulate this by creating a single mapped chunk for each series, since the max time is all the reader looks at. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Fix comment Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Remove series map from processWALSamples() The locks that is commented to reduce contention in are now sharded 32,000 ways, so won't be contended. Removing the map saves memory and goes just as fast. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * loadWAL: Cache the last mmapped chunk time So we can skip calling append() for samples it will reject. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Improvements from code review Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Full stops and capitals on comments Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Cache max time in both places mmappedChunks is updated Including refactor to extract function `setMMappedChunks`, to reduce code duplication. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Update head min/max time when mmapped chunks added This ensures we have the correct values if no WAL samples are added for that series. Note that `mSeries.maxTime()` was always `math.MinInt64` before, since that function doesn't consider mmapped chunks. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Split Go and React Tests (#8897) * Added go-ci and react-ci Co-authored-by: Julien Pivotto <roidelapluie@inuits.eu> Signed-off-by: Levi Harrison <git@leviharrison.dev> * Remove search keymap from new expression editor (#9184) Signed-off-by: Julius Volz <julius.volz@gmail.com> Co-authored-by: Austin Cawley-Edwards <austin.cawley@gmail.com> Co-authored-by: Levi Harrison <git@leviharrison.dev> Co-authored-by: Julien Pivotto <roidelapluie@inuits.eu> Co-authored-by: Bryan Boreham <bjboreham@gmail.com> Co-authored-by: Julius Volz <julius.volz@gmail.com>
2021-08-11 03:13:17 -07:00
// The label set written to the disk is already sorted.
// TODO: figure out why DecodeLabels calls Sort(), and perhaps remove it.
csr.lset = d.DecodeLabels(&dec)
Sync sparsehistogram branch with main (#9189) * Fix `kuma_sd` targetgroup reporting (#9157) * Bundle all xDS targets into a single group Signed-off-by: austin ce <austin.cawley@gmail.com> * Snapshot in-memory chunks on shutdown for faster restarts (#7229) Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Rename links Signed-off-by: Levi Harrison <git@leviharrison.dev> * Remove Individual Data Type Caps in Per-shard Buffering for Remote Write (#8921) * Moved everything to nPending buffer Signed-off-by: Levi Harrison <git@leviharrison.dev> * Simplify exemplar capacity addition Signed-off-by: Levi Harrison <git@leviharrison.dev> * Added pre-allocation Signed-off-by: Levi Harrison <git@leviharrison.dev> * Don't allocate if not sending exemplars Signed-off-by: Levi Harrison <git@leviharrison.dev> * Avoid deadlock when processing duplicate series record (#9170) * Avoid deadlock when processing duplicate series record `processWALSamples()` needs to be able to send on its output channel before it can read the input channel, so reads to allow this in case the output channel is full. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * processWALSamples: update comment Previous text seems to relate to an earlier implementation. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Optimise WAL loading by removing extra map and caching min-time (#9160) * BenchmarkLoadWAL: close WAL after use So that goroutines are stopped and resources released Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * BenchmarkLoadWAL: make series IDs co-prime with #workers Series are distributed across workers by taking the modulus of the ID with the number of workers, so multiples of 100 are a poor choice. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * BenchmarkLoadWAL: simulate mmapped chunks Real Prometheus cuts chunks every 120 samples, then skips those samples when re-reading the WAL. Simulate this by creating a single mapped chunk for each series, since the max time is all the reader looks at. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Fix comment Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Remove series map from processWALSamples() The locks that is commented to reduce contention in are now sharded 32,000 ways, so won't be contended. Removing the map saves memory and goes just as fast. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * loadWAL: Cache the last mmapped chunk time So we can skip calling append() for samples it will reject. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Improvements from code review Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Full stops and capitals on comments Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Cache max time in both places mmappedChunks is updated Including refactor to extract function `setMMappedChunks`, to reduce code duplication. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Update head min/max time when mmapped chunks added This ensures we have the correct values if no WAL samples are added for that series. Note that `mSeries.maxTime()` was always `math.MinInt64` before, since that function doesn't consider mmapped chunks. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Split Go and React Tests (#8897) * Added go-ci and react-ci Co-authored-by: Julien Pivotto <roidelapluie@inuits.eu> Signed-off-by: Levi Harrison <git@leviharrison.dev> * Remove search keymap from new expression editor (#9184) Signed-off-by: Julius Volz <julius.volz@gmail.com> Co-authored-by: Austin Cawley-Edwards <austin.cawley@gmail.com> Co-authored-by: Levi Harrison <git@leviharrison.dev> Co-authored-by: Julien Pivotto <roidelapluie@inuits.eu> Co-authored-by: Bryan Boreham <bjboreham@gmail.com> Co-authored-by: Julius Volz <julius.volz@gmail.com>
2021-08-11 03:13:17 -07:00
_ = dec.Be64int64() // Was chunkRange but now unused.
Sync sparsehistogram branch with main (#9189) * Fix `kuma_sd` targetgroup reporting (#9157) * Bundle all xDS targets into a single group Signed-off-by: austin ce <austin.cawley@gmail.com> * Snapshot in-memory chunks on shutdown for faster restarts (#7229) Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Rename links Signed-off-by: Levi Harrison <git@leviharrison.dev> * Remove Individual Data Type Caps in Per-shard Buffering for Remote Write (#8921) * Moved everything to nPending buffer Signed-off-by: Levi Harrison <git@leviharrison.dev> * Simplify exemplar capacity addition Signed-off-by: Levi Harrison <git@leviharrison.dev> * Added pre-allocation Signed-off-by: Levi Harrison <git@leviharrison.dev> * Don't allocate if not sending exemplars Signed-off-by: Levi Harrison <git@leviharrison.dev> * Avoid deadlock when processing duplicate series record (#9170) * Avoid deadlock when processing duplicate series record `processWALSamples()` needs to be able to send on its output channel before it can read the input channel, so reads to allow this in case the output channel is full. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * processWALSamples: update comment Previous text seems to relate to an earlier implementation. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Optimise WAL loading by removing extra map and caching min-time (#9160) * BenchmarkLoadWAL: close WAL after use So that goroutines are stopped and resources released Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * BenchmarkLoadWAL: make series IDs co-prime with #workers Series are distributed across workers by taking the modulus of the ID with the number of workers, so multiples of 100 are a poor choice. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * BenchmarkLoadWAL: simulate mmapped chunks Real Prometheus cuts chunks every 120 samples, then skips those samples when re-reading the WAL. Simulate this by creating a single mapped chunk for each series, since the max time is all the reader looks at. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Fix comment Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Remove series map from processWALSamples() The locks that is commented to reduce contention in are now sharded 32,000 ways, so won't be contended. Removing the map saves memory and goes just as fast. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * loadWAL: Cache the last mmapped chunk time So we can skip calling append() for samples it will reject. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Improvements from code review Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Full stops and capitals on comments Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Cache max time in both places mmappedChunks is updated Including refactor to extract function `setMMappedChunks`, to reduce code duplication. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Update head min/max time when mmapped chunks added This ensures we have the correct values if no WAL samples are added for that series. Note that `mSeries.maxTime()` was always `math.MinInt64` before, since that function doesn't consider mmapped chunks. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Split Go and React Tests (#8897) * Added go-ci and react-ci Co-authored-by: Julien Pivotto <roidelapluie@inuits.eu> Signed-off-by: Levi Harrison <git@leviharrison.dev> * Remove search keymap from new expression editor (#9184) Signed-off-by: Julius Volz <julius.volz@gmail.com> Co-authored-by: Austin Cawley-Edwards <austin.cawley@gmail.com> Co-authored-by: Levi Harrison <git@leviharrison.dev> Co-authored-by: Julien Pivotto <roidelapluie@inuits.eu> Co-authored-by: Bryan Boreham <bjboreham@gmail.com> Co-authored-by: Julius Volz <julius.volz@gmail.com>
2021-08-11 03:13:17 -07:00
if dec.Uvarint() == 0 {
return
}
csr.mc = &memChunk{}
csr.mc.minTime = dec.Be64int64()
csr.mc.maxTime = dec.Be64int64()
enc := chunkenc.Encoding(dec.Byte())
// The underlying bytes gets re-used later, so make a copy.
chunkBytes := dec.UvarintBytes()
chunkBytesCopy := make([]byte, len(chunkBytes))
copy(chunkBytesCopy, chunkBytes)
chk, err := chunkenc.FromData(enc, chunkBytesCopy)
if err != nil {
return csr, errors.Wrap(err, "chunk from data")
}
csr.mc.chunk = chk
tsdb: stop saving a copy of last 4 samples in memSeries (#11296) * TSDB chunks: remove race between writing and reading Because the data is stored as a bit-stream, the last byte in the stream could change if the stream is appended to after an Iterator is obtained. Copy the last byte when the Iterator is created, so we don't have to read it later. Clarify in comments that concurrent Iterator and Appender are allowed, but the chunk must not be modified while an Iterator is created. (This was already the case, in order to copy the bstream slice header.) * TSDB: stop saving last 4 samples in memSeries This extra copy of the last 4 samples was introduced to avoid a race condition between reading the last byte of the chunk and writing to it. But now we have fixed that by having `bstreamReader` copy the last byte, we don't need to copy the last 4 samples. This change saves 56 bytes per series, which is very worthwhile when you have millions or tens of millions of series. * TSDB: tidy up stopIterator re-use Previous changes have left this code duplicating some lines; pull them out to a separate function and tidy up. * TSDB head_test: stop checking when iterators are wrapped The behaviour has changed so chunk iterators are only wrapped when transaction isolation requires them to stop short of the end. This makes tests fail which are checking the type. Tests should check the observable behaviour, not the type. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> Co-authored-by: Ganesh Vernekar <ganeshvern@gmail.com>
2022-09-27 07:02:05 -07:00
// Backwards-compatibility for old sampleBuf which had last 4 samples.
for i := 0; i < 3; i++ {
_ = dec.Be64int64()
_ = dec.Be64Float64()
Sync sparsehistogram branch with main (#9189) * Fix `kuma_sd` targetgroup reporting (#9157) * Bundle all xDS targets into a single group Signed-off-by: austin ce <austin.cawley@gmail.com> * Snapshot in-memory chunks on shutdown for faster restarts (#7229) Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Rename links Signed-off-by: Levi Harrison <git@leviharrison.dev> * Remove Individual Data Type Caps in Per-shard Buffering for Remote Write (#8921) * Moved everything to nPending buffer Signed-off-by: Levi Harrison <git@leviharrison.dev> * Simplify exemplar capacity addition Signed-off-by: Levi Harrison <git@leviharrison.dev> * Added pre-allocation Signed-off-by: Levi Harrison <git@leviharrison.dev> * Don't allocate if not sending exemplars Signed-off-by: Levi Harrison <git@leviharrison.dev> * Avoid deadlock when processing duplicate series record (#9170) * Avoid deadlock when processing duplicate series record `processWALSamples()` needs to be able to send on its output channel before it can read the input channel, so reads to allow this in case the output channel is full. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * processWALSamples: update comment Previous text seems to relate to an earlier implementation. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Optimise WAL loading by removing extra map and caching min-time (#9160) * BenchmarkLoadWAL: close WAL after use So that goroutines are stopped and resources released Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * BenchmarkLoadWAL: make series IDs co-prime with #workers Series are distributed across workers by taking the modulus of the ID with the number of workers, so multiples of 100 are a poor choice. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * BenchmarkLoadWAL: simulate mmapped chunks Real Prometheus cuts chunks every 120 samples, then skips those samples when re-reading the WAL. Simulate this by creating a single mapped chunk for each series, since the max time is all the reader looks at. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Fix comment Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Remove series map from processWALSamples() The locks that is commented to reduce contention in are now sharded 32,000 ways, so won't be contended. Removing the map saves memory and goes just as fast. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * loadWAL: Cache the last mmapped chunk time So we can skip calling append() for samples it will reject. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Improvements from code review Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Full stops and capitals on comments Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Cache max time in both places mmappedChunks is updated Including refactor to extract function `setMMappedChunks`, to reduce code duplication. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Update head min/max time when mmapped chunks added This ensures we have the correct values if no WAL samples are added for that series. Note that `mSeries.maxTime()` was always `math.MinInt64` before, since that function doesn't consider mmapped chunks. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Split Go and React Tests (#8897) * Added go-ci and react-ci Co-authored-by: Julien Pivotto <roidelapluie@inuits.eu> Signed-off-by: Levi Harrison <git@leviharrison.dev> * Remove search keymap from new expression editor (#9184) Signed-off-by: Julius Volz <julius.volz@gmail.com> Co-authored-by: Austin Cawley-Edwards <austin.cawley@gmail.com> Co-authored-by: Levi Harrison <git@leviharrison.dev> Co-authored-by: Julien Pivotto <roidelapluie@inuits.eu> Co-authored-by: Bryan Boreham <bjboreham@gmail.com> Co-authored-by: Julius Volz <julius.volz@gmail.com>
2021-08-11 03:13:17 -07:00
}
tsdb: stop saving a copy of last 4 samples in memSeries (#11296) * TSDB chunks: remove race between writing and reading Because the data is stored as a bit-stream, the last byte in the stream could change if the stream is appended to after an Iterator is obtained. Copy the last byte when the Iterator is created, so we don't have to read it later. Clarify in comments that concurrent Iterator and Appender are allowed, but the chunk must not be modified while an Iterator is created. (This was already the case, in order to copy the bstream slice header.) * TSDB: stop saving last 4 samples in memSeries This extra copy of the last 4 samples was introduced to avoid a race condition between reading the last byte of the chunk and writing to it. But now we have fixed that by having `bstreamReader` copy the last byte, we don't need to copy the last 4 samples. This change saves 56 bytes per series, which is very worthwhile when you have millions or tens of millions of series. * TSDB: tidy up stopIterator re-use Previous changes have left this code duplicating some lines; pull them out to a separate function and tidy up. * TSDB head_test: stop checking when iterators are wrapped The behaviour has changed so chunk iterators are only wrapped when transaction isolation requires them to stop short of the end. This makes tests fail which are checking the type. Tests should check the observable behaviour, not the type. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> Co-authored-by: Ganesh Vernekar <ganeshvern@gmail.com>
2022-09-27 07:02:05 -07:00
_ = dec.Be64int64()
csr.lastValue = dec.Be64Float64()
Sync sparsehistogram branch with main (#9189) * Fix `kuma_sd` targetgroup reporting (#9157) * Bundle all xDS targets into a single group Signed-off-by: austin ce <austin.cawley@gmail.com> * Snapshot in-memory chunks on shutdown for faster restarts (#7229) Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Rename links Signed-off-by: Levi Harrison <git@leviharrison.dev> * Remove Individual Data Type Caps in Per-shard Buffering for Remote Write (#8921) * Moved everything to nPending buffer Signed-off-by: Levi Harrison <git@leviharrison.dev> * Simplify exemplar capacity addition Signed-off-by: Levi Harrison <git@leviharrison.dev> * Added pre-allocation Signed-off-by: Levi Harrison <git@leviharrison.dev> * Don't allocate if not sending exemplars Signed-off-by: Levi Harrison <git@leviharrison.dev> * Avoid deadlock when processing duplicate series record (#9170) * Avoid deadlock when processing duplicate series record `processWALSamples()` needs to be able to send on its output channel before it can read the input channel, so reads to allow this in case the output channel is full. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * processWALSamples: update comment Previous text seems to relate to an earlier implementation. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Optimise WAL loading by removing extra map and caching min-time (#9160) * BenchmarkLoadWAL: close WAL after use So that goroutines are stopped and resources released Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * BenchmarkLoadWAL: make series IDs co-prime with #workers Series are distributed across workers by taking the modulus of the ID with the number of workers, so multiples of 100 are a poor choice. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * BenchmarkLoadWAL: simulate mmapped chunks Real Prometheus cuts chunks every 120 samples, then skips those samples when re-reading the WAL. Simulate this by creating a single mapped chunk for each series, since the max time is all the reader looks at. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Fix comment Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Remove series map from processWALSamples() The locks that is commented to reduce contention in are now sharded 32,000 ways, so won't be contended. Removing the map saves memory and goes just as fast. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * loadWAL: Cache the last mmapped chunk time So we can skip calling append() for samples it will reject. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Improvements from code review Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Full stops and capitals on comments Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Cache max time in both places mmappedChunks is updated Including refactor to extract function `setMMappedChunks`, to reduce code duplication. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Update head min/max time when mmapped chunks added This ensures we have the correct values if no WAL samples are added for that series. Note that `mSeries.maxTime()` was always `math.MinInt64` before, since that function doesn't consider mmapped chunks. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Split Go and React Tests (#8897) * Added go-ci and react-ci Co-authored-by: Julien Pivotto <roidelapluie@inuits.eu> Signed-off-by: Levi Harrison <git@leviharrison.dev> * Remove search keymap from new expression editor (#9184) Signed-off-by: Julius Volz <julius.volz@gmail.com> Co-authored-by: Austin Cawley-Edwards <austin.cawley@gmail.com> Co-authored-by: Levi Harrison <git@leviharrison.dev> Co-authored-by: Julien Pivotto <roidelapluie@inuits.eu> Co-authored-by: Bryan Boreham <bjboreham@gmail.com> Co-authored-by: Julius Volz <julius.volz@gmail.com>
2021-08-11 03:13:17 -07:00
err = dec.Err()
if err != nil && len(dec.B) > 0 {
err = errors.Errorf("unexpected %d bytes left in entry", len(dec.B))
}
return
}
func encodeTombstonesToSnapshotRecord(tr tombstones.Reader) ([]byte, error) {
buf := encoding.Encbuf{}
buf.PutByte(chunkSnapshotRecordTypeTombstones)
b, err := tombstones.Encode(tr)
if err != nil {
return nil, errors.Wrap(err, "encode tombstones")
}
buf.PutUvarintBytes(b)
return buf.Get(), nil
}
func decodeTombstonesSnapshotRecord(b []byte) (tombstones.Reader, error) {
dec := encoding.Decbuf{B: b}
if flag := dec.Byte(); flag != chunkSnapshotRecordTypeTombstones {
return nil, errors.Errorf("invalid record type %x", flag)
}
tr, err := tombstones.Decode(dec.UvarintBytes())
return tr, errors.Wrap(err, "decode tombstones")
}
const chunkSnapshotPrefix = "chunk_snapshot."
// ChunkSnapshot creates a snapshot of all the series and tombstones in the head.
// It deletes the old chunk snapshots if the chunk snapshot creation is successful.
//
// The chunk snapshot is stored in a directory named chunk_snapshot.N.M and is written
// using the WAL package. N is the last WAL segment present during snapshotting and
// M is the offset in segment N upto which data was written.
//
// The snapshot first contains all series (each in individual records and not sorted), followed by
// tombstones (a single record), and finally exemplars (>= 1 record). Exemplars are in the order they
// were written to the circular buffer.
Sync sparsehistogram branch with main (#9189) * Fix `kuma_sd` targetgroup reporting (#9157) * Bundle all xDS targets into a single group Signed-off-by: austin ce <austin.cawley@gmail.com> * Snapshot in-memory chunks on shutdown for faster restarts (#7229) Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Rename links Signed-off-by: Levi Harrison <git@leviharrison.dev> * Remove Individual Data Type Caps in Per-shard Buffering for Remote Write (#8921) * Moved everything to nPending buffer Signed-off-by: Levi Harrison <git@leviharrison.dev> * Simplify exemplar capacity addition Signed-off-by: Levi Harrison <git@leviharrison.dev> * Added pre-allocation Signed-off-by: Levi Harrison <git@leviharrison.dev> * Don't allocate if not sending exemplars Signed-off-by: Levi Harrison <git@leviharrison.dev> * Avoid deadlock when processing duplicate series record (#9170) * Avoid deadlock when processing duplicate series record `processWALSamples()` needs to be able to send on its output channel before it can read the input channel, so reads to allow this in case the output channel is full. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * processWALSamples: update comment Previous text seems to relate to an earlier implementation. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Optimise WAL loading by removing extra map and caching min-time (#9160) * BenchmarkLoadWAL: close WAL after use So that goroutines are stopped and resources released Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * BenchmarkLoadWAL: make series IDs co-prime with #workers Series are distributed across workers by taking the modulus of the ID with the number of workers, so multiples of 100 are a poor choice. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * BenchmarkLoadWAL: simulate mmapped chunks Real Prometheus cuts chunks every 120 samples, then skips those samples when re-reading the WAL. Simulate this by creating a single mapped chunk for each series, since the max time is all the reader looks at. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Fix comment Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Remove series map from processWALSamples() The locks that is commented to reduce contention in are now sharded 32,000 ways, so won't be contended. Removing the map saves memory and goes just as fast. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * loadWAL: Cache the last mmapped chunk time So we can skip calling append() for samples it will reject. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Improvements from code review Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Full stops and capitals on comments Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Cache max time in both places mmappedChunks is updated Including refactor to extract function `setMMappedChunks`, to reduce code duplication. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Update head min/max time when mmapped chunks added This ensures we have the correct values if no WAL samples are added for that series. Note that `mSeries.maxTime()` was always `math.MinInt64` before, since that function doesn't consider mmapped chunks. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Split Go and React Tests (#8897) * Added go-ci and react-ci Co-authored-by: Julien Pivotto <roidelapluie@inuits.eu> Signed-off-by: Levi Harrison <git@leviharrison.dev> * Remove search keymap from new expression editor (#9184) Signed-off-by: Julius Volz <julius.volz@gmail.com> Co-authored-by: Austin Cawley-Edwards <austin.cawley@gmail.com> Co-authored-by: Levi Harrison <git@leviharrison.dev> Co-authored-by: Julien Pivotto <roidelapluie@inuits.eu> Co-authored-by: Bryan Boreham <bjboreham@gmail.com> Co-authored-by: Julius Volz <julius.volz@gmail.com>
2021-08-11 03:13:17 -07:00
func (h *Head) ChunkSnapshot() (*ChunkSnapshotStats, error) {
if h.wal == nil {
// If we are not storing any WAL, does not make sense to take a snapshot too.
level.Warn(h.logger).Log("msg", "skipping chunk snapshotting as WAL is disabled")
return &ChunkSnapshotStats{}, nil
}
h.chunkSnapshotMtx.Lock()
defer h.chunkSnapshotMtx.Unlock()
stats := &ChunkSnapshotStats{}
wlast, woffset, err := h.wal.LastSegmentAndOffset()
if err != nil && err != record.ErrNotFound {
return stats, errors.Wrap(err, "get last wal segment and offset")
}
_, cslast, csoffset, err := LastChunkSnapshot(h.opts.ChunkDirRoot)
if err != nil && err != record.ErrNotFound {
return stats, errors.Wrap(err, "find last chunk snapshot")
}
if wlast == cslast && woffset == csoffset {
// Nothing has been written to the WAL/Head since the last snapshot.
return stats, nil
}
snapshotName := chunkSnapshotDir(wlast, woffset)
Sync sparsehistogram branch with main (#9189) * Fix `kuma_sd` targetgroup reporting (#9157) * Bundle all xDS targets into a single group Signed-off-by: austin ce <austin.cawley@gmail.com> * Snapshot in-memory chunks on shutdown for faster restarts (#7229) Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Rename links Signed-off-by: Levi Harrison <git@leviharrison.dev> * Remove Individual Data Type Caps in Per-shard Buffering for Remote Write (#8921) * Moved everything to nPending buffer Signed-off-by: Levi Harrison <git@leviharrison.dev> * Simplify exemplar capacity addition Signed-off-by: Levi Harrison <git@leviharrison.dev> * Added pre-allocation Signed-off-by: Levi Harrison <git@leviharrison.dev> * Don't allocate if not sending exemplars Signed-off-by: Levi Harrison <git@leviharrison.dev> * Avoid deadlock when processing duplicate series record (#9170) * Avoid deadlock when processing duplicate series record `processWALSamples()` needs to be able to send on its output channel before it can read the input channel, so reads to allow this in case the output channel is full. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * processWALSamples: update comment Previous text seems to relate to an earlier implementation. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Optimise WAL loading by removing extra map and caching min-time (#9160) * BenchmarkLoadWAL: close WAL after use So that goroutines are stopped and resources released Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * BenchmarkLoadWAL: make series IDs co-prime with #workers Series are distributed across workers by taking the modulus of the ID with the number of workers, so multiples of 100 are a poor choice. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * BenchmarkLoadWAL: simulate mmapped chunks Real Prometheus cuts chunks every 120 samples, then skips those samples when re-reading the WAL. Simulate this by creating a single mapped chunk for each series, since the max time is all the reader looks at. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Fix comment Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Remove series map from processWALSamples() The locks that is commented to reduce contention in are now sharded 32,000 ways, so won't be contended. Removing the map saves memory and goes just as fast. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * loadWAL: Cache the last mmapped chunk time So we can skip calling append() for samples it will reject. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Improvements from code review Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Full stops and capitals on comments Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Cache max time in both places mmappedChunks is updated Including refactor to extract function `setMMappedChunks`, to reduce code duplication. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Update head min/max time when mmapped chunks added This ensures we have the correct values if no WAL samples are added for that series. Note that `mSeries.maxTime()` was always `math.MinInt64` before, since that function doesn't consider mmapped chunks. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Split Go and React Tests (#8897) * Added go-ci and react-ci Co-authored-by: Julien Pivotto <roidelapluie@inuits.eu> Signed-off-by: Levi Harrison <git@leviharrison.dev> * Remove search keymap from new expression editor (#9184) Signed-off-by: Julius Volz <julius.volz@gmail.com> Co-authored-by: Austin Cawley-Edwards <austin.cawley@gmail.com> Co-authored-by: Levi Harrison <git@leviharrison.dev> Co-authored-by: Julien Pivotto <roidelapluie@inuits.eu> Co-authored-by: Bryan Boreham <bjboreham@gmail.com> Co-authored-by: Julius Volz <julius.volz@gmail.com>
2021-08-11 03:13:17 -07:00
cpdir := filepath.Join(h.opts.ChunkDirRoot, snapshotName)
cpdirtmp := cpdir + ".tmp"
stats.Dir = cpdir
if err := os.MkdirAll(cpdirtmp, 0o777); err != nil {
Sync sparsehistogram branch with main (#9189) * Fix `kuma_sd` targetgroup reporting (#9157) * Bundle all xDS targets into a single group Signed-off-by: austin ce <austin.cawley@gmail.com> * Snapshot in-memory chunks on shutdown for faster restarts (#7229) Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Rename links Signed-off-by: Levi Harrison <git@leviharrison.dev> * Remove Individual Data Type Caps in Per-shard Buffering for Remote Write (#8921) * Moved everything to nPending buffer Signed-off-by: Levi Harrison <git@leviharrison.dev> * Simplify exemplar capacity addition Signed-off-by: Levi Harrison <git@leviharrison.dev> * Added pre-allocation Signed-off-by: Levi Harrison <git@leviharrison.dev> * Don't allocate if not sending exemplars Signed-off-by: Levi Harrison <git@leviharrison.dev> * Avoid deadlock when processing duplicate series record (#9170) * Avoid deadlock when processing duplicate series record `processWALSamples()` needs to be able to send on its output channel before it can read the input channel, so reads to allow this in case the output channel is full. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * processWALSamples: update comment Previous text seems to relate to an earlier implementation. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Optimise WAL loading by removing extra map and caching min-time (#9160) * BenchmarkLoadWAL: close WAL after use So that goroutines are stopped and resources released Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * BenchmarkLoadWAL: make series IDs co-prime with #workers Series are distributed across workers by taking the modulus of the ID with the number of workers, so multiples of 100 are a poor choice. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * BenchmarkLoadWAL: simulate mmapped chunks Real Prometheus cuts chunks every 120 samples, then skips those samples when re-reading the WAL. Simulate this by creating a single mapped chunk for each series, since the max time is all the reader looks at. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Fix comment Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Remove series map from processWALSamples() The locks that is commented to reduce contention in are now sharded 32,000 ways, so won't be contended. Removing the map saves memory and goes just as fast. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * loadWAL: Cache the last mmapped chunk time So we can skip calling append() for samples it will reject. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Improvements from code review Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Full stops and capitals on comments Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Cache max time in both places mmappedChunks is updated Including refactor to extract function `setMMappedChunks`, to reduce code duplication. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Update head min/max time when mmapped chunks added This ensures we have the correct values if no WAL samples are added for that series. Note that `mSeries.maxTime()` was always `math.MinInt64` before, since that function doesn't consider mmapped chunks. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Split Go and React Tests (#8897) * Added go-ci and react-ci Co-authored-by: Julien Pivotto <roidelapluie@inuits.eu> Signed-off-by: Levi Harrison <git@leviharrison.dev> * Remove search keymap from new expression editor (#9184) Signed-off-by: Julius Volz <julius.volz@gmail.com> Co-authored-by: Austin Cawley-Edwards <austin.cawley@gmail.com> Co-authored-by: Levi Harrison <git@leviharrison.dev> Co-authored-by: Julien Pivotto <roidelapluie@inuits.eu> Co-authored-by: Bryan Boreham <bjboreham@gmail.com> Co-authored-by: Julius Volz <julius.volz@gmail.com>
2021-08-11 03:13:17 -07:00
return stats, errors.Wrap(err, "create chunk snapshot dir")
}
cp, err := wlog.New(nil, nil, cpdirtmp, h.wal.CompressionEnabled())
Sync sparsehistogram branch with main (#9189) * Fix `kuma_sd` targetgroup reporting (#9157) * Bundle all xDS targets into a single group Signed-off-by: austin ce <austin.cawley@gmail.com> * Snapshot in-memory chunks on shutdown for faster restarts (#7229) Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Rename links Signed-off-by: Levi Harrison <git@leviharrison.dev> * Remove Individual Data Type Caps in Per-shard Buffering for Remote Write (#8921) * Moved everything to nPending buffer Signed-off-by: Levi Harrison <git@leviharrison.dev> * Simplify exemplar capacity addition Signed-off-by: Levi Harrison <git@leviharrison.dev> * Added pre-allocation Signed-off-by: Levi Harrison <git@leviharrison.dev> * Don't allocate if not sending exemplars Signed-off-by: Levi Harrison <git@leviharrison.dev> * Avoid deadlock when processing duplicate series record (#9170) * Avoid deadlock when processing duplicate series record `processWALSamples()` needs to be able to send on its output channel before it can read the input channel, so reads to allow this in case the output channel is full. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * processWALSamples: update comment Previous text seems to relate to an earlier implementation. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Optimise WAL loading by removing extra map and caching min-time (#9160) * BenchmarkLoadWAL: close WAL after use So that goroutines are stopped and resources released Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * BenchmarkLoadWAL: make series IDs co-prime with #workers Series are distributed across workers by taking the modulus of the ID with the number of workers, so multiples of 100 are a poor choice. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * BenchmarkLoadWAL: simulate mmapped chunks Real Prometheus cuts chunks every 120 samples, then skips those samples when re-reading the WAL. Simulate this by creating a single mapped chunk for each series, since the max time is all the reader looks at. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Fix comment Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Remove series map from processWALSamples() The locks that is commented to reduce contention in are now sharded 32,000 ways, so won't be contended. Removing the map saves memory and goes just as fast. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * loadWAL: Cache the last mmapped chunk time So we can skip calling append() for samples it will reject. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Improvements from code review Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Full stops and capitals on comments Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Cache max time in both places mmappedChunks is updated Including refactor to extract function `setMMappedChunks`, to reduce code duplication. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Update head min/max time when mmapped chunks added This ensures we have the correct values if no WAL samples are added for that series. Note that `mSeries.maxTime()` was always `math.MinInt64` before, since that function doesn't consider mmapped chunks. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Split Go and React Tests (#8897) * Added go-ci and react-ci Co-authored-by: Julien Pivotto <roidelapluie@inuits.eu> Signed-off-by: Levi Harrison <git@leviharrison.dev> * Remove search keymap from new expression editor (#9184) Signed-off-by: Julius Volz <julius.volz@gmail.com> Co-authored-by: Austin Cawley-Edwards <austin.cawley@gmail.com> Co-authored-by: Levi Harrison <git@leviharrison.dev> Co-authored-by: Julien Pivotto <roidelapluie@inuits.eu> Co-authored-by: Bryan Boreham <bjboreham@gmail.com> Co-authored-by: Julius Volz <julius.volz@gmail.com>
2021-08-11 03:13:17 -07:00
if err != nil {
return stats, errors.Wrap(err, "open chunk snapshot")
}
// Ensures that an early return caused by an error doesn't leave any tmp files.
defer func() {
cp.Close()
os.RemoveAll(cpdirtmp)
}()
var (
buf []byte
recs [][]byte
)
// Add all series to the snapshot.
Sync sparsehistogram branch with main (#9189) * Fix `kuma_sd` targetgroup reporting (#9157) * Bundle all xDS targets into a single group Signed-off-by: austin ce <austin.cawley@gmail.com> * Snapshot in-memory chunks on shutdown for faster restarts (#7229) Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Rename links Signed-off-by: Levi Harrison <git@leviharrison.dev> * Remove Individual Data Type Caps in Per-shard Buffering for Remote Write (#8921) * Moved everything to nPending buffer Signed-off-by: Levi Harrison <git@leviharrison.dev> * Simplify exemplar capacity addition Signed-off-by: Levi Harrison <git@leviharrison.dev> * Added pre-allocation Signed-off-by: Levi Harrison <git@leviharrison.dev> * Don't allocate if not sending exemplars Signed-off-by: Levi Harrison <git@leviharrison.dev> * Avoid deadlock when processing duplicate series record (#9170) * Avoid deadlock when processing duplicate series record `processWALSamples()` needs to be able to send on its output channel before it can read the input channel, so reads to allow this in case the output channel is full. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * processWALSamples: update comment Previous text seems to relate to an earlier implementation. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Optimise WAL loading by removing extra map and caching min-time (#9160) * BenchmarkLoadWAL: close WAL after use So that goroutines are stopped and resources released Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * BenchmarkLoadWAL: make series IDs co-prime with #workers Series are distributed across workers by taking the modulus of the ID with the number of workers, so multiples of 100 are a poor choice. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * BenchmarkLoadWAL: simulate mmapped chunks Real Prometheus cuts chunks every 120 samples, then skips those samples when re-reading the WAL. Simulate this by creating a single mapped chunk for each series, since the max time is all the reader looks at. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Fix comment Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Remove series map from processWALSamples() The locks that is commented to reduce contention in are now sharded 32,000 ways, so won't be contended. Removing the map saves memory and goes just as fast. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * loadWAL: Cache the last mmapped chunk time So we can skip calling append() for samples it will reject. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Improvements from code review Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Full stops and capitals on comments Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Cache max time in both places mmappedChunks is updated Including refactor to extract function `setMMappedChunks`, to reduce code duplication. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Update head min/max time when mmapped chunks added This ensures we have the correct values if no WAL samples are added for that series. Note that `mSeries.maxTime()` was always `math.MinInt64` before, since that function doesn't consider mmapped chunks. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Split Go and React Tests (#8897) * Added go-ci and react-ci Co-authored-by: Julien Pivotto <roidelapluie@inuits.eu> Signed-off-by: Levi Harrison <git@leviharrison.dev> * Remove search keymap from new expression editor (#9184) Signed-off-by: Julius Volz <julius.volz@gmail.com> Co-authored-by: Austin Cawley-Edwards <austin.cawley@gmail.com> Co-authored-by: Levi Harrison <git@leviharrison.dev> Co-authored-by: Julien Pivotto <roidelapluie@inuits.eu> Co-authored-by: Bryan Boreham <bjboreham@gmail.com> Co-authored-by: Julius Volz <julius.volz@gmail.com>
2021-08-11 03:13:17 -07:00
stripeSize := h.series.size
for i := 0; i < stripeSize; i++ {
h.series.locks[i].RLock()
for _, s := range h.series.series[i] {
start := len(buf)
buf = s.encodeToSnapshotRecord(buf)
if len(buf[start:]) == 0 {
continue // All contents discarded.
}
recs = append(recs, buf[start:])
// Flush records in 10 MB increments.
if len(buf) > 10*1024*1024 {
if err := cp.Log(recs...); err != nil {
h.series.locks[i].RUnlock()
return stats, errors.Wrap(err, "flush records")
}
buf, recs = buf[:0], recs[:0]
}
}
stats.TotalSeries += len(h.series.series[i])
h.series.locks[i].RUnlock()
}
// Add tombstones to the snapshot.
tombstonesReader, err := h.Tombstones()
if err != nil {
return stats, errors.Wrap(err, "get tombstones")
}
rec, err := encodeTombstonesToSnapshotRecord(tombstonesReader)
if err != nil {
return stats, errors.Wrap(err, "encode tombstones")
}
recs = append(recs, rec)
// Flush remaining series records and tombstones.
Sync sparsehistogram branch with main (#9189) * Fix `kuma_sd` targetgroup reporting (#9157) * Bundle all xDS targets into a single group Signed-off-by: austin ce <austin.cawley@gmail.com> * Snapshot in-memory chunks on shutdown for faster restarts (#7229) Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Rename links Signed-off-by: Levi Harrison <git@leviharrison.dev> * Remove Individual Data Type Caps in Per-shard Buffering for Remote Write (#8921) * Moved everything to nPending buffer Signed-off-by: Levi Harrison <git@leviharrison.dev> * Simplify exemplar capacity addition Signed-off-by: Levi Harrison <git@leviharrison.dev> * Added pre-allocation Signed-off-by: Levi Harrison <git@leviharrison.dev> * Don't allocate if not sending exemplars Signed-off-by: Levi Harrison <git@leviharrison.dev> * Avoid deadlock when processing duplicate series record (#9170) * Avoid deadlock when processing duplicate series record `processWALSamples()` needs to be able to send on its output channel before it can read the input channel, so reads to allow this in case the output channel is full. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * processWALSamples: update comment Previous text seems to relate to an earlier implementation. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Optimise WAL loading by removing extra map and caching min-time (#9160) * BenchmarkLoadWAL: close WAL after use So that goroutines are stopped and resources released Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * BenchmarkLoadWAL: make series IDs co-prime with #workers Series are distributed across workers by taking the modulus of the ID with the number of workers, so multiples of 100 are a poor choice. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * BenchmarkLoadWAL: simulate mmapped chunks Real Prometheus cuts chunks every 120 samples, then skips those samples when re-reading the WAL. Simulate this by creating a single mapped chunk for each series, since the max time is all the reader looks at. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Fix comment Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Remove series map from processWALSamples() The locks that is commented to reduce contention in are now sharded 32,000 ways, so won't be contended. Removing the map saves memory and goes just as fast. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * loadWAL: Cache the last mmapped chunk time So we can skip calling append() for samples it will reject. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Improvements from code review Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Full stops and capitals on comments Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Cache max time in both places mmappedChunks is updated Including refactor to extract function `setMMappedChunks`, to reduce code duplication. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Update head min/max time when mmapped chunks added This ensures we have the correct values if no WAL samples are added for that series. Note that `mSeries.maxTime()` was always `math.MinInt64` before, since that function doesn't consider mmapped chunks. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Split Go and React Tests (#8897) * Added go-ci and react-ci Co-authored-by: Julien Pivotto <roidelapluie@inuits.eu> Signed-off-by: Levi Harrison <git@leviharrison.dev> * Remove search keymap from new expression editor (#9184) Signed-off-by: Julius Volz <julius.volz@gmail.com> Co-authored-by: Austin Cawley-Edwards <austin.cawley@gmail.com> Co-authored-by: Levi Harrison <git@leviharrison.dev> Co-authored-by: Julien Pivotto <roidelapluie@inuits.eu> Co-authored-by: Bryan Boreham <bjboreham@gmail.com> Co-authored-by: Julius Volz <julius.volz@gmail.com>
2021-08-11 03:13:17 -07:00
if err := cp.Log(recs...); err != nil {
return stats, errors.Wrap(err, "flush records")
}
buf = buf[:0]
// Add exemplars in the snapshot.
// We log in batches, with each record having upto 10000 exemplars.
// Assuming 100 bytes (overestimate) per exemplar, that's ~1MB.
maxExemplarsPerRecord := 10000
batch := make([]record.RefExemplar, 0, maxExemplarsPerRecord)
enc := record.Encoder{}
flushExemplars := func() error {
if len(batch) == 0 {
return nil
}
buf = buf[:0]
encbuf := encoding.Encbuf{B: buf}
encbuf.PutByte(chunkSnapshotRecordTypeExemplars)
enc.EncodeExemplarsIntoBuffer(batch, &encbuf)
if err := cp.Log(encbuf.Get()); err != nil {
return errors.Wrap(err, "log exemplars")
}
buf, batch = buf[:0], batch[:0]
return nil
}
err = h.exemplars.IterateExemplars(func(seriesLabels labels.Labels, e exemplar.Exemplar) error {
if len(batch) >= maxExemplarsPerRecord {
if err := flushExemplars(); err != nil {
return errors.Wrap(err, "flush exemplars")
}
}
ms := h.series.getByHash(seriesLabels.Hash(), seriesLabels)
if ms == nil {
// It is possible that exemplar refers to some old series. We discard such exemplars.
return nil
}
batch = append(batch, record.RefExemplar{
Ref: ms.ref,
T: e.Ts,
V: e.Value,
Labels: e.Labels,
})
return nil
})
if err != nil {
return stats, errors.Wrap(err, "iterate exemplars")
}
// Flush remaining exemplars.
if err := flushExemplars(); err != nil {
return stats, errors.Wrap(err, "flush exemplars at the end")
}
Sync sparsehistogram branch with main (#9189) * Fix `kuma_sd` targetgroup reporting (#9157) * Bundle all xDS targets into a single group Signed-off-by: austin ce <austin.cawley@gmail.com> * Snapshot in-memory chunks on shutdown for faster restarts (#7229) Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Rename links Signed-off-by: Levi Harrison <git@leviharrison.dev> * Remove Individual Data Type Caps in Per-shard Buffering for Remote Write (#8921) * Moved everything to nPending buffer Signed-off-by: Levi Harrison <git@leviharrison.dev> * Simplify exemplar capacity addition Signed-off-by: Levi Harrison <git@leviharrison.dev> * Added pre-allocation Signed-off-by: Levi Harrison <git@leviharrison.dev> * Don't allocate if not sending exemplars Signed-off-by: Levi Harrison <git@leviharrison.dev> * Avoid deadlock when processing duplicate series record (#9170) * Avoid deadlock when processing duplicate series record `processWALSamples()` needs to be able to send on its output channel before it can read the input channel, so reads to allow this in case the output channel is full. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * processWALSamples: update comment Previous text seems to relate to an earlier implementation. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Optimise WAL loading by removing extra map and caching min-time (#9160) * BenchmarkLoadWAL: close WAL after use So that goroutines are stopped and resources released Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * BenchmarkLoadWAL: make series IDs co-prime with #workers Series are distributed across workers by taking the modulus of the ID with the number of workers, so multiples of 100 are a poor choice. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * BenchmarkLoadWAL: simulate mmapped chunks Real Prometheus cuts chunks every 120 samples, then skips those samples when re-reading the WAL. Simulate this by creating a single mapped chunk for each series, since the max time is all the reader looks at. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Fix comment Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Remove series map from processWALSamples() The locks that is commented to reduce contention in are now sharded 32,000 ways, so won't be contended. Removing the map saves memory and goes just as fast. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * loadWAL: Cache the last mmapped chunk time So we can skip calling append() for samples it will reject. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Improvements from code review Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Full stops and capitals on comments Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Cache max time in both places mmappedChunks is updated Including refactor to extract function `setMMappedChunks`, to reduce code duplication. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Update head min/max time when mmapped chunks added This ensures we have the correct values if no WAL samples are added for that series. Note that `mSeries.maxTime()` was always `math.MinInt64` before, since that function doesn't consider mmapped chunks. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Split Go and React Tests (#8897) * Added go-ci and react-ci Co-authored-by: Julien Pivotto <roidelapluie@inuits.eu> Signed-off-by: Levi Harrison <git@leviharrison.dev> * Remove search keymap from new expression editor (#9184) Signed-off-by: Julius Volz <julius.volz@gmail.com> Co-authored-by: Austin Cawley-Edwards <austin.cawley@gmail.com> Co-authored-by: Levi Harrison <git@leviharrison.dev> Co-authored-by: Julien Pivotto <roidelapluie@inuits.eu> Co-authored-by: Bryan Boreham <bjboreham@gmail.com> Co-authored-by: Julius Volz <julius.volz@gmail.com>
2021-08-11 03:13:17 -07:00
if err := cp.Close(); err != nil {
return stats, errors.Wrap(err, "close chunk snapshot")
}
if err := fileutil.Replace(cpdirtmp, cpdir); err != nil {
return stats, errors.Wrap(err, "rename chunk snapshot directory")
}
if err := DeleteChunkSnapshots(h.opts.ChunkDirRoot, wlast, woffset); err != nil {
Sync sparsehistogram branch with main (#9189) * Fix `kuma_sd` targetgroup reporting (#9157) * Bundle all xDS targets into a single group Signed-off-by: austin ce <austin.cawley@gmail.com> * Snapshot in-memory chunks on shutdown for faster restarts (#7229) Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Rename links Signed-off-by: Levi Harrison <git@leviharrison.dev> * Remove Individual Data Type Caps in Per-shard Buffering for Remote Write (#8921) * Moved everything to nPending buffer Signed-off-by: Levi Harrison <git@leviharrison.dev> * Simplify exemplar capacity addition Signed-off-by: Levi Harrison <git@leviharrison.dev> * Added pre-allocation Signed-off-by: Levi Harrison <git@leviharrison.dev> * Don't allocate if not sending exemplars Signed-off-by: Levi Harrison <git@leviharrison.dev> * Avoid deadlock when processing duplicate series record (#9170) * Avoid deadlock when processing duplicate series record `processWALSamples()` needs to be able to send on its output channel before it can read the input channel, so reads to allow this in case the output channel is full. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * processWALSamples: update comment Previous text seems to relate to an earlier implementation. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Optimise WAL loading by removing extra map and caching min-time (#9160) * BenchmarkLoadWAL: close WAL after use So that goroutines are stopped and resources released Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * BenchmarkLoadWAL: make series IDs co-prime with #workers Series are distributed across workers by taking the modulus of the ID with the number of workers, so multiples of 100 are a poor choice. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * BenchmarkLoadWAL: simulate mmapped chunks Real Prometheus cuts chunks every 120 samples, then skips those samples when re-reading the WAL. Simulate this by creating a single mapped chunk for each series, since the max time is all the reader looks at. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Fix comment Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Remove series map from processWALSamples() The locks that is commented to reduce contention in are now sharded 32,000 ways, so won't be contended. Removing the map saves memory and goes just as fast. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * loadWAL: Cache the last mmapped chunk time So we can skip calling append() for samples it will reject. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Improvements from code review Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Full stops and capitals on comments Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Cache max time in both places mmappedChunks is updated Including refactor to extract function `setMMappedChunks`, to reduce code duplication. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Update head min/max time when mmapped chunks added This ensures we have the correct values if no WAL samples are added for that series. Note that `mSeries.maxTime()` was always `math.MinInt64` before, since that function doesn't consider mmapped chunks. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Split Go and React Tests (#8897) * Added go-ci and react-ci Co-authored-by: Julien Pivotto <roidelapluie@inuits.eu> Signed-off-by: Levi Harrison <git@leviharrison.dev> * Remove search keymap from new expression editor (#9184) Signed-off-by: Julius Volz <julius.volz@gmail.com> Co-authored-by: Austin Cawley-Edwards <austin.cawley@gmail.com> Co-authored-by: Levi Harrison <git@leviharrison.dev> Co-authored-by: Julien Pivotto <roidelapluie@inuits.eu> Co-authored-by: Bryan Boreham <bjboreham@gmail.com> Co-authored-by: Julius Volz <julius.volz@gmail.com>
2021-08-11 03:13:17 -07:00
// Leftover old chunk snapshots do not cause problems down the line beyond
// occupying disk space.
// They will just be ignored since a higher chunk snapshot exists.
level.Error(h.logger).Log("msg", "delete old chunk snapshots", "err", err)
}
return stats, nil
}
func chunkSnapshotDir(wlast, woffset int) string {
return fmt.Sprintf(chunkSnapshotPrefix+"%06d.%010d", wlast, woffset)
}
Sync sparsehistogram branch with main (#9189) * Fix `kuma_sd` targetgroup reporting (#9157) * Bundle all xDS targets into a single group Signed-off-by: austin ce <austin.cawley@gmail.com> * Snapshot in-memory chunks on shutdown for faster restarts (#7229) Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Rename links Signed-off-by: Levi Harrison <git@leviharrison.dev> * Remove Individual Data Type Caps in Per-shard Buffering for Remote Write (#8921) * Moved everything to nPending buffer Signed-off-by: Levi Harrison <git@leviharrison.dev> * Simplify exemplar capacity addition Signed-off-by: Levi Harrison <git@leviharrison.dev> * Added pre-allocation Signed-off-by: Levi Harrison <git@leviharrison.dev> * Don't allocate if not sending exemplars Signed-off-by: Levi Harrison <git@leviharrison.dev> * Avoid deadlock when processing duplicate series record (#9170) * Avoid deadlock when processing duplicate series record `processWALSamples()` needs to be able to send on its output channel before it can read the input channel, so reads to allow this in case the output channel is full. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * processWALSamples: update comment Previous text seems to relate to an earlier implementation. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Optimise WAL loading by removing extra map and caching min-time (#9160) * BenchmarkLoadWAL: close WAL after use So that goroutines are stopped and resources released Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * BenchmarkLoadWAL: make series IDs co-prime with #workers Series are distributed across workers by taking the modulus of the ID with the number of workers, so multiples of 100 are a poor choice. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * BenchmarkLoadWAL: simulate mmapped chunks Real Prometheus cuts chunks every 120 samples, then skips those samples when re-reading the WAL. Simulate this by creating a single mapped chunk for each series, since the max time is all the reader looks at. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Fix comment Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Remove series map from processWALSamples() The locks that is commented to reduce contention in are now sharded 32,000 ways, so won't be contended. Removing the map saves memory and goes just as fast. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * loadWAL: Cache the last mmapped chunk time So we can skip calling append() for samples it will reject. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Improvements from code review Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Full stops and capitals on comments Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Cache max time in both places mmappedChunks is updated Including refactor to extract function `setMMappedChunks`, to reduce code duplication. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Update head min/max time when mmapped chunks added This ensures we have the correct values if no WAL samples are added for that series. Note that `mSeries.maxTime()` was always `math.MinInt64` before, since that function doesn't consider mmapped chunks. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Split Go and React Tests (#8897) * Added go-ci and react-ci Co-authored-by: Julien Pivotto <roidelapluie@inuits.eu> Signed-off-by: Levi Harrison <git@leviharrison.dev> * Remove search keymap from new expression editor (#9184) Signed-off-by: Julius Volz <julius.volz@gmail.com> Co-authored-by: Austin Cawley-Edwards <austin.cawley@gmail.com> Co-authored-by: Levi Harrison <git@leviharrison.dev> Co-authored-by: Julien Pivotto <roidelapluie@inuits.eu> Co-authored-by: Bryan Boreham <bjboreham@gmail.com> Co-authored-by: Julius Volz <julius.volz@gmail.com>
2021-08-11 03:13:17 -07:00
func (h *Head) performChunkSnapshot() error {
level.Info(h.logger).Log("msg", "creating chunk snapshot")
startTime := time.Now()
stats, err := h.ChunkSnapshot()
elapsed := time.Since(startTime)
if err == nil {
level.Info(h.logger).Log("msg", "chunk snapshot complete", "duration", elapsed.String(), "num_series", stats.TotalSeries, "dir", stats.Dir)
}
return errors.Wrap(err, "chunk snapshot")
}
// ChunkSnapshotStats returns stats about a created chunk snapshot.
type ChunkSnapshotStats struct {
TotalSeries int
Dir string
}
// LastChunkSnapshot returns the directory name and index of the most recent chunk snapshot.
// If dir does not contain any chunk snapshots, ErrNotFound is returned.
func LastChunkSnapshot(dir string) (string, int, int, error) {
files, err := os.ReadDir(dir)
Sync sparsehistogram branch with main (#9189) * Fix `kuma_sd` targetgroup reporting (#9157) * Bundle all xDS targets into a single group Signed-off-by: austin ce <austin.cawley@gmail.com> * Snapshot in-memory chunks on shutdown for faster restarts (#7229) Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Rename links Signed-off-by: Levi Harrison <git@leviharrison.dev> * Remove Individual Data Type Caps in Per-shard Buffering for Remote Write (#8921) * Moved everything to nPending buffer Signed-off-by: Levi Harrison <git@leviharrison.dev> * Simplify exemplar capacity addition Signed-off-by: Levi Harrison <git@leviharrison.dev> * Added pre-allocation Signed-off-by: Levi Harrison <git@leviharrison.dev> * Don't allocate if not sending exemplars Signed-off-by: Levi Harrison <git@leviharrison.dev> * Avoid deadlock when processing duplicate series record (#9170) * Avoid deadlock when processing duplicate series record `processWALSamples()` needs to be able to send on its output channel before it can read the input channel, so reads to allow this in case the output channel is full. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * processWALSamples: update comment Previous text seems to relate to an earlier implementation. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Optimise WAL loading by removing extra map and caching min-time (#9160) * BenchmarkLoadWAL: close WAL after use So that goroutines are stopped and resources released Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * BenchmarkLoadWAL: make series IDs co-prime with #workers Series are distributed across workers by taking the modulus of the ID with the number of workers, so multiples of 100 are a poor choice. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * BenchmarkLoadWAL: simulate mmapped chunks Real Prometheus cuts chunks every 120 samples, then skips those samples when re-reading the WAL. Simulate this by creating a single mapped chunk for each series, since the max time is all the reader looks at. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Fix comment Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Remove series map from processWALSamples() The locks that is commented to reduce contention in are now sharded 32,000 ways, so won't be contended. Removing the map saves memory and goes just as fast. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * loadWAL: Cache the last mmapped chunk time So we can skip calling append() for samples it will reject. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Improvements from code review Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Full stops and capitals on comments Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Cache max time in both places mmappedChunks is updated Including refactor to extract function `setMMappedChunks`, to reduce code duplication. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Update head min/max time when mmapped chunks added This ensures we have the correct values if no WAL samples are added for that series. Note that `mSeries.maxTime()` was always `math.MinInt64` before, since that function doesn't consider mmapped chunks. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Split Go and React Tests (#8897) * Added go-ci and react-ci Co-authored-by: Julien Pivotto <roidelapluie@inuits.eu> Signed-off-by: Levi Harrison <git@leviharrison.dev> * Remove search keymap from new expression editor (#9184) Signed-off-by: Julius Volz <julius.volz@gmail.com> Co-authored-by: Austin Cawley-Edwards <austin.cawley@gmail.com> Co-authored-by: Levi Harrison <git@leviharrison.dev> Co-authored-by: Julien Pivotto <roidelapluie@inuits.eu> Co-authored-by: Bryan Boreham <bjboreham@gmail.com> Co-authored-by: Julius Volz <julius.volz@gmail.com>
2021-08-11 03:13:17 -07:00
if err != nil {
return "", 0, 0, err
}
maxIdx, maxOffset := -1, -1
maxFileName := ""
for i := 0; i < len(files); i++ {
Sync sparsehistogram branch with main (#9189) * Fix `kuma_sd` targetgroup reporting (#9157) * Bundle all xDS targets into a single group Signed-off-by: austin ce <austin.cawley@gmail.com> * Snapshot in-memory chunks on shutdown for faster restarts (#7229) Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Rename links Signed-off-by: Levi Harrison <git@leviharrison.dev> * Remove Individual Data Type Caps in Per-shard Buffering for Remote Write (#8921) * Moved everything to nPending buffer Signed-off-by: Levi Harrison <git@leviharrison.dev> * Simplify exemplar capacity addition Signed-off-by: Levi Harrison <git@leviharrison.dev> * Added pre-allocation Signed-off-by: Levi Harrison <git@leviharrison.dev> * Don't allocate if not sending exemplars Signed-off-by: Levi Harrison <git@leviharrison.dev> * Avoid deadlock when processing duplicate series record (#9170) * Avoid deadlock when processing duplicate series record `processWALSamples()` needs to be able to send on its output channel before it can read the input channel, so reads to allow this in case the output channel is full. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * processWALSamples: update comment Previous text seems to relate to an earlier implementation. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Optimise WAL loading by removing extra map and caching min-time (#9160) * BenchmarkLoadWAL: close WAL after use So that goroutines are stopped and resources released Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * BenchmarkLoadWAL: make series IDs co-prime with #workers Series are distributed across workers by taking the modulus of the ID with the number of workers, so multiples of 100 are a poor choice. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * BenchmarkLoadWAL: simulate mmapped chunks Real Prometheus cuts chunks every 120 samples, then skips those samples when re-reading the WAL. Simulate this by creating a single mapped chunk for each series, since the max time is all the reader looks at. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Fix comment Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Remove series map from processWALSamples() The locks that is commented to reduce contention in are now sharded 32,000 ways, so won't be contended. Removing the map saves memory and goes just as fast. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * loadWAL: Cache the last mmapped chunk time So we can skip calling append() for samples it will reject. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Improvements from code review Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Full stops and capitals on comments Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Cache max time in both places mmappedChunks is updated Including refactor to extract function `setMMappedChunks`, to reduce code duplication. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Update head min/max time when mmapped chunks added This ensures we have the correct values if no WAL samples are added for that series. Note that `mSeries.maxTime()` was always `math.MinInt64` before, since that function doesn't consider mmapped chunks. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Split Go and React Tests (#8897) * Added go-ci and react-ci Co-authored-by: Julien Pivotto <roidelapluie@inuits.eu> Signed-off-by: Levi Harrison <git@leviharrison.dev> * Remove search keymap from new expression editor (#9184) Signed-off-by: Julius Volz <julius.volz@gmail.com> Co-authored-by: Austin Cawley-Edwards <austin.cawley@gmail.com> Co-authored-by: Levi Harrison <git@leviharrison.dev> Co-authored-by: Julien Pivotto <roidelapluie@inuits.eu> Co-authored-by: Bryan Boreham <bjboreham@gmail.com> Co-authored-by: Julius Volz <julius.volz@gmail.com>
2021-08-11 03:13:17 -07:00
fi := files[i]
if !strings.HasPrefix(fi.Name(), chunkSnapshotPrefix) {
continue
}
if !fi.IsDir() {
return "", 0, 0, errors.Errorf("chunk snapshot %s is not a directory", fi.Name())
}
splits := strings.Split(fi.Name()[len(chunkSnapshotPrefix):], ".")
if len(splits) != 2 {
// Chunk snapshots is not in the right format, we do not care about it.
continue
Sync sparsehistogram branch with main (#9189) * Fix `kuma_sd` targetgroup reporting (#9157) * Bundle all xDS targets into a single group Signed-off-by: austin ce <austin.cawley@gmail.com> * Snapshot in-memory chunks on shutdown for faster restarts (#7229) Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Rename links Signed-off-by: Levi Harrison <git@leviharrison.dev> * Remove Individual Data Type Caps in Per-shard Buffering for Remote Write (#8921) * Moved everything to nPending buffer Signed-off-by: Levi Harrison <git@leviharrison.dev> * Simplify exemplar capacity addition Signed-off-by: Levi Harrison <git@leviharrison.dev> * Added pre-allocation Signed-off-by: Levi Harrison <git@leviharrison.dev> * Don't allocate if not sending exemplars Signed-off-by: Levi Harrison <git@leviharrison.dev> * Avoid deadlock when processing duplicate series record (#9170) * Avoid deadlock when processing duplicate series record `processWALSamples()` needs to be able to send on its output channel before it can read the input channel, so reads to allow this in case the output channel is full. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * processWALSamples: update comment Previous text seems to relate to an earlier implementation. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Optimise WAL loading by removing extra map and caching min-time (#9160) * BenchmarkLoadWAL: close WAL after use So that goroutines are stopped and resources released Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * BenchmarkLoadWAL: make series IDs co-prime with #workers Series are distributed across workers by taking the modulus of the ID with the number of workers, so multiples of 100 are a poor choice. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * BenchmarkLoadWAL: simulate mmapped chunks Real Prometheus cuts chunks every 120 samples, then skips those samples when re-reading the WAL. Simulate this by creating a single mapped chunk for each series, since the max time is all the reader looks at. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Fix comment Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Remove series map from processWALSamples() The locks that is commented to reduce contention in are now sharded 32,000 ways, so won't be contended. Removing the map saves memory and goes just as fast. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * loadWAL: Cache the last mmapped chunk time So we can skip calling append() for samples it will reject. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Improvements from code review Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Full stops and capitals on comments Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Cache max time in both places mmappedChunks is updated Including refactor to extract function `setMMappedChunks`, to reduce code duplication. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Update head min/max time when mmapped chunks added This ensures we have the correct values if no WAL samples are added for that series. Note that `mSeries.maxTime()` was always `math.MinInt64` before, since that function doesn't consider mmapped chunks. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Split Go and React Tests (#8897) * Added go-ci and react-ci Co-authored-by: Julien Pivotto <roidelapluie@inuits.eu> Signed-off-by: Levi Harrison <git@leviharrison.dev> * Remove search keymap from new expression editor (#9184) Signed-off-by: Julius Volz <julius.volz@gmail.com> Co-authored-by: Austin Cawley-Edwards <austin.cawley@gmail.com> Co-authored-by: Levi Harrison <git@leviharrison.dev> Co-authored-by: Julien Pivotto <roidelapluie@inuits.eu> Co-authored-by: Bryan Boreham <bjboreham@gmail.com> Co-authored-by: Julius Volz <julius.volz@gmail.com>
2021-08-11 03:13:17 -07:00
}
idx, err := strconv.Atoi(splits[0])
if err != nil {
continue
}
offset, err := strconv.Atoi(splits[1])
if err != nil {
continue
}
if idx > maxIdx || (idx == maxIdx && offset > maxOffset) {
maxIdx, maxOffset = idx, offset
maxFileName = filepath.Join(dir, fi.Name())
}
Sync sparsehistogram branch with main (#9189) * Fix `kuma_sd` targetgroup reporting (#9157) * Bundle all xDS targets into a single group Signed-off-by: austin ce <austin.cawley@gmail.com> * Snapshot in-memory chunks on shutdown for faster restarts (#7229) Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Rename links Signed-off-by: Levi Harrison <git@leviharrison.dev> * Remove Individual Data Type Caps in Per-shard Buffering for Remote Write (#8921) * Moved everything to nPending buffer Signed-off-by: Levi Harrison <git@leviharrison.dev> * Simplify exemplar capacity addition Signed-off-by: Levi Harrison <git@leviharrison.dev> * Added pre-allocation Signed-off-by: Levi Harrison <git@leviharrison.dev> * Don't allocate if not sending exemplars Signed-off-by: Levi Harrison <git@leviharrison.dev> * Avoid deadlock when processing duplicate series record (#9170) * Avoid deadlock when processing duplicate series record `processWALSamples()` needs to be able to send on its output channel before it can read the input channel, so reads to allow this in case the output channel is full. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * processWALSamples: update comment Previous text seems to relate to an earlier implementation. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Optimise WAL loading by removing extra map and caching min-time (#9160) * BenchmarkLoadWAL: close WAL after use So that goroutines are stopped and resources released Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * BenchmarkLoadWAL: make series IDs co-prime with #workers Series are distributed across workers by taking the modulus of the ID with the number of workers, so multiples of 100 are a poor choice. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * BenchmarkLoadWAL: simulate mmapped chunks Real Prometheus cuts chunks every 120 samples, then skips those samples when re-reading the WAL. Simulate this by creating a single mapped chunk for each series, since the max time is all the reader looks at. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Fix comment Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Remove series map from processWALSamples() The locks that is commented to reduce contention in are now sharded 32,000 ways, so won't be contended. Removing the map saves memory and goes just as fast. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * loadWAL: Cache the last mmapped chunk time So we can skip calling append() for samples it will reject. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Improvements from code review Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Full stops and capitals on comments Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Cache max time in both places mmappedChunks is updated Including refactor to extract function `setMMappedChunks`, to reduce code duplication. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Update head min/max time when mmapped chunks added This ensures we have the correct values if no WAL samples are added for that series. Note that `mSeries.maxTime()` was always `math.MinInt64` before, since that function doesn't consider mmapped chunks. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Split Go and React Tests (#8897) * Added go-ci and react-ci Co-authored-by: Julien Pivotto <roidelapluie@inuits.eu> Signed-off-by: Levi Harrison <git@leviharrison.dev> * Remove search keymap from new expression editor (#9184) Signed-off-by: Julius Volz <julius.volz@gmail.com> Co-authored-by: Austin Cawley-Edwards <austin.cawley@gmail.com> Co-authored-by: Levi Harrison <git@leviharrison.dev> Co-authored-by: Julien Pivotto <roidelapluie@inuits.eu> Co-authored-by: Bryan Boreham <bjboreham@gmail.com> Co-authored-by: Julius Volz <julius.volz@gmail.com>
2021-08-11 03:13:17 -07:00
}
if maxFileName == "" {
return "", 0, 0, record.ErrNotFound
}
return maxFileName, maxIdx, maxOffset, nil
Sync sparsehistogram branch with main (#9189) * Fix `kuma_sd` targetgroup reporting (#9157) * Bundle all xDS targets into a single group Signed-off-by: austin ce <austin.cawley@gmail.com> * Snapshot in-memory chunks on shutdown for faster restarts (#7229) Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Rename links Signed-off-by: Levi Harrison <git@leviharrison.dev> * Remove Individual Data Type Caps in Per-shard Buffering for Remote Write (#8921) * Moved everything to nPending buffer Signed-off-by: Levi Harrison <git@leviharrison.dev> * Simplify exemplar capacity addition Signed-off-by: Levi Harrison <git@leviharrison.dev> * Added pre-allocation Signed-off-by: Levi Harrison <git@leviharrison.dev> * Don't allocate if not sending exemplars Signed-off-by: Levi Harrison <git@leviharrison.dev> * Avoid deadlock when processing duplicate series record (#9170) * Avoid deadlock when processing duplicate series record `processWALSamples()` needs to be able to send on its output channel before it can read the input channel, so reads to allow this in case the output channel is full. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * processWALSamples: update comment Previous text seems to relate to an earlier implementation. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Optimise WAL loading by removing extra map and caching min-time (#9160) * BenchmarkLoadWAL: close WAL after use So that goroutines are stopped and resources released Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * BenchmarkLoadWAL: make series IDs co-prime with #workers Series are distributed across workers by taking the modulus of the ID with the number of workers, so multiples of 100 are a poor choice. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * BenchmarkLoadWAL: simulate mmapped chunks Real Prometheus cuts chunks every 120 samples, then skips those samples when re-reading the WAL. Simulate this by creating a single mapped chunk for each series, since the max time is all the reader looks at. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Fix comment Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Remove series map from processWALSamples() The locks that is commented to reduce contention in are now sharded 32,000 ways, so won't be contended. Removing the map saves memory and goes just as fast. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * loadWAL: Cache the last mmapped chunk time So we can skip calling append() for samples it will reject. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Improvements from code review Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Full stops and capitals on comments Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Cache max time in both places mmappedChunks is updated Including refactor to extract function `setMMappedChunks`, to reduce code duplication. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Update head min/max time when mmapped chunks added This ensures we have the correct values if no WAL samples are added for that series. Note that `mSeries.maxTime()` was always `math.MinInt64` before, since that function doesn't consider mmapped chunks. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Split Go and React Tests (#8897) * Added go-ci and react-ci Co-authored-by: Julien Pivotto <roidelapluie@inuits.eu> Signed-off-by: Levi Harrison <git@leviharrison.dev> * Remove search keymap from new expression editor (#9184) Signed-off-by: Julius Volz <julius.volz@gmail.com> Co-authored-by: Austin Cawley-Edwards <austin.cawley@gmail.com> Co-authored-by: Levi Harrison <git@leviharrison.dev> Co-authored-by: Julien Pivotto <roidelapluie@inuits.eu> Co-authored-by: Bryan Boreham <bjboreham@gmail.com> Co-authored-by: Julius Volz <julius.volz@gmail.com>
2021-08-11 03:13:17 -07:00
}
// DeleteChunkSnapshots deletes all chunk snapshots in a directory below a given index.
func DeleteChunkSnapshots(dir string, maxIndex, maxOffset int) error {
files, err := os.ReadDir(dir)
Sync sparsehistogram branch with main (#9189) * Fix `kuma_sd` targetgroup reporting (#9157) * Bundle all xDS targets into a single group Signed-off-by: austin ce <austin.cawley@gmail.com> * Snapshot in-memory chunks on shutdown for faster restarts (#7229) Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Rename links Signed-off-by: Levi Harrison <git@leviharrison.dev> * Remove Individual Data Type Caps in Per-shard Buffering for Remote Write (#8921) * Moved everything to nPending buffer Signed-off-by: Levi Harrison <git@leviharrison.dev> * Simplify exemplar capacity addition Signed-off-by: Levi Harrison <git@leviharrison.dev> * Added pre-allocation Signed-off-by: Levi Harrison <git@leviharrison.dev> * Don't allocate if not sending exemplars Signed-off-by: Levi Harrison <git@leviharrison.dev> * Avoid deadlock when processing duplicate series record (#9170) * Avoid deadlock when processing duplicate series record `processWALSamples()` needs to be able to send on its output channel before it can read the input channel, so reads to allow this in case the output channel is full. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * processWALSamples: update comment Previous text seems to relate to an earlier implementation. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Optimise WAL loading by removing extra map and caching min-time (#9160) * BenchmarkLoadWAL: close WAL after use So that goroutines are stopped and resources released Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * BenchmarkLoadWAL: make series IDs co-prime with #workers Series are distributed across workers by taking the modulus of the ID with the number of workers, so multiples of 100 are a poor choice. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * BenchmarkLoadWAL: simulate mmapped chunks Real Prometheus cuts chunks every 120 samples, then skips those samples when re-reading the WAL. Simulate this by creating a single mapped chunk for each series, since the max time is all the reader looks at. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Fix comment Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Remove series map from processWALSamples() The locks that is commented to reduce contention in are now sharded 32,000 ways, so won't be contended. Removing the map saves memory and goes just as fast. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * loadWAL: Cache the last mmapped chunk time So we can skip calling append() for samples it will reject. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Improvements from code review Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Full stops and capitals on comments Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Cache max time in both places mmappedChunks is updated Including refactor to extract function `setMMappedChunks`, to reduce code duplication. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Update head min/max time when mmapped chunks added This ensures we have the correct values if no WAL samples are added for that series. Note that `mSeries.maxTime()` was always `math.MinInt64` before, since that function doesn't consider mmapped chunks. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Split Go and React Tests (#8897) * Added go-ci and react-ci Co-authored-by: Julien Pivotto <roidelapluie@inuits.eu> Signed-off-by: Levi Harrison <git@leviharrison.dev> * Remove search keymap from new expression editor (#9184) Signed-off-by: Julius Volz <julius.volz@gmail.com> Co-authored-by: Austin Cawley-Edwards <austin.cawley@gmail.com> Co-authored-by: Levi Harrison <git@leviharrison.dev> Co-authored-by: Julien Pivotto <roidelapluie@inuits.eu> Co-authored-by: Bryan Boreham <bjboreham@gmail.com> Co-authored-by: Julius Volz <julius.volz@gmail.com>
2021-08-11 03:13:17 -07:00
if err != nil {
return err
}
errs := tsdb_errors.NewMulti()
for _, fi := range files {
if !strings.HasPrefix(fi.Name(), chunkSnapshotPrefix) {
continue
}
splits := strings.Split(fi.Name()[len(chunkSnapshotPrefix):], ".")
if len(splits) != 2 {
continue
}
idx, err := strconv.Atoi(splits[0])
if err != nil {
continue
}
offset, err := strconv.Atoi(splits[1])
if err != nil {
continue
}
if idx < maxIndex || (idx == maxIndex && offset < maxOffset) {
Sync sparsehistogram branch with main (#9189) * Fix `kuma_sd` targetgroup reporting (#9157) * Bundle all xDS targets into a single group Signed-off-by: austin ce <austin.cawley@gmail.com> * Snapshot in-memory chunks on shutdown for faster restarts (#7229) Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Rename links Signed-off-by: Levi Harrison <git@leviharrison.dev> * Remove Individual Data Type Caps in Per-shard Buffering for Remote Write (#8921) * Moved everything to nPending buffer Signed-off-by: Levi Harrison <git@leviharrison.dev> * Simplify exemplar capacity addition Signed-off-by: Levi Harrison <git@leviharrison.dev> * Added pre-allocation Signed-off-by: Levi Harrison <git@leviharrison.dev> * Don't allocate if not sending exemplars Signed-off-by: Levi Harrison <git@leviharrison.dev> * Avoid deadlock when processing duplicate series record (#9170) * Avoid deadlock when processing duplicate series record `processWALSamples()` needs to be able to send on its output channel before it can read the input channel, so reads to allow this in case the output channel is full. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * processWALSamples: update comment Previous text seems to relate to an earlier implementation. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Optimise WAL loading by removing extra map and caching min-time (#9160) * BenchmarkLoadWAL: close WAL after use So that goroutines are stopped and resources released Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * BenchmarkLoadWAL: make series IDs co-prime with #workers Series are distributed across workers by taking the modulus of the ID with the number of workers, so multiples of 100 are a poor choice. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * BenchmarkLoadWAL: simulate mmapped chunks Real Prometheus cuts chunks every 120 samples, then skips those samples when re-reading the WAL. Simulate this by creating a single mapped chunk for each series, since the max time is all the reader looks at. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Fix comment Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Remove series map from processWALSamples() The locks that is commented to reduce contention in are now sharded 32,000 ways, so won't be contended. Removing the map saves memory and goes just as fast. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * loadWAL: Cache the last mmapped chunk time So we can skip calling append() for samples it will reject. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Improvements from code review Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Full stops and capitals on comments Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Cache max time in both places mmappedChunks is updated Including refactor to extract function `setMMappedChunks`, to reduce code duplication. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Update head min/max time when mmapped chunks added This ensures we have the correct values if no WAL samples are added for that series. Note that `mSeries.maxTime()` was always `math.MinInt64` before, since that function doesn't consider mmapped chunks. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Split Go and React Tests (#8897) * Added go-ci and react-ci Co-authored-by: Julien Pivotto <roidelapluie@inuits.eu> Signed-off-by: Levi Harrison <git@leviharrison.dev> * Remove search keymap from new expression editor (#9184) Signed-off-by: Julius Volz <julius.volz@gmail.com> Co-authored-by: Austin Cawley-Edwards <austin.cawley@gmail.com> Co-authored-by: Levi Harrison <git@leviharrison.dev> Co-authored-by: Julien Pivotto <roidelapluie@inuits.eu> Co-authored-by: Bryan Boreham <bjboreham@gmail.com> Co-authored-by: Julius Volz <julius.volz@gmail.com>
2021-08-11 03:13:17 -07:00
if err := os.RemoveAll(filepath.Join(dir, fi.Name())); err != nil {
errs.Add(err)
}
}
}
return errs.Err()
}
// loadChunkSnapshot replays the chunk snapshot and restores the Head state from it. If there was any error returned,
// it is the responsibility of the caller to clear the contents of the Head.
func (h *Head) loadChunkSnapshot() (int, int, map[chunks.HeadSeriesRef]*memSeries, error) {
Sync sparsehistogram branch with main (#9189) * Fix `kuma_sd` targetgroup reporting (#9157) * Bundle all xDS targets into a single group Signed-off-by: austin ce <austin.cawley@gmail.com> * Snapshot in-memory chunks on shutdown for faster restarts (#7229) Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Rename links Signed-off-by: Levi Harrison <git@leviharrison.dev> * Remove Individual Data Type Caps in Per-shard Buffering for Remote Write (#8921) * Moved everything to nPending buffer Signed-off-by: Levi Harrison <git@leviharrison.dev> * Simplify exemplar capacity addition Signed-off-by: Levi Harrison <git@leviharrison.dev> * Added pre-allocation Signed-off-by: Levi Harrison <git@leviharrison.dev> * Don't allocate if not sending exemplars Signed-off-by: Levi Harrison <git@leviharrison.dev> * Avoid deadlock when processing duplicate series record (#9170) * Avoid deadlock when processing duplicate series record `processWALSamples()` needs to be able to send on its output channel before it can read the input channel, so reads to allow this in case the output channel is full. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * processWALSamples: update comment Previous text seems to relate to an earlier implementation. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Optimise WAL loading by removing extra map and caching min-time (#9160) * BenchmarkLoadWAL: close WAL after use So that goroutines are stopped and resources released Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * BenchmarkLoadWAL: make series IDs co-prime with #workers Series are distributed across workers by taking the modulus of the ID with the number of workers, so multiples of 100 are a poor choice. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * BenchmarkLoadWAL: simulate mmapped chunks Real Prometheus cuts chunks every 120 samples, then skips those samples when re-reading the WAL. Simulate this by creating a single mapped chunk for each series, since the max time is all the reader looks at. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Fix comment Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Remove series map from processWALSamples() The locks that is commented to reduce contention in are now sharded 32,000 ways, so won't be contended. Removing the map saves memory and goes just as fast. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * loadWAL: Cache the last mmapped chunk time So we can skip calling append() for samples it will reject. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Improvements from code review Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Full stops and capitals on comments Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Cache max time in both places mmappedChunks is updated Including refactor to extract function `setMMappedChunks`, to reduce code duplication. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Update head min/max time when mmapped chunks added This ensures we have the correct values if no WAL samples are added for that series. Note that `mSeries.maxTime()` was always `math.MinInt64` before, since that function doesn't consider mmapped chunks. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Split Go and React Tests (#8897) * Added go-ci and react-ci Co-authored-by: Julien Pivotto <roidelapluie@inuits.eu> Signed-off-by: Levi Harrison <git@leviharrison.dev> * Remove search keymap from new expression editor (#9184) Signed-off-by: Julius Volz <julius.volz@gmail.com> Co-authored-by: Austin Cawley-Edwards <austin.cawley@gmail.com> Co-authored-by: Levi Harrison <git@leviharrison.dev> Co-authored-by: Julien Pivotto <roidelapluie@inuits.eu> Co-authored-by: Bryan Boreham <bjboreham@gmail.com> Co-authored-by: Julius Volz <julius.volz@gmail.com>
2021-08-11 03:13:17 -07:00
dir, snapIdx, snapOffset, err := LastChunkSnapshot(h.opts.ChunkDirRoot)
if err != nil {
if err == record.ErrNotFound {
return snapIdx, snapOffset, nil, nil
}
return snapIdx, snapOffset, nil, errors.Wrap(err, "find last chunk snapshot")
}
start := time.Now()
sr, err := wlog.NewSegmentsReader(dir)
Sync sparsehistogram branch with main (#9189) * Fix `kuma_sd` targetgroup reporting (#9157) * Bundle all xDS targets into a single group Signed-off-by: austin ce <austin.cawley@gmail.com> * Snapshot in-memory chunks on shutdown for faster restarts (#7229) Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Rename links Signed-off-by: Levi Harrison <git@leviharrison.dev> * Remove Individual Data Type Caps in Per-shard Buffering for Remote Write (#8921) * Moved everything to nPending buffer Signed-off-by: Levi Harrison <git@leviharrison.dev> * Simplify exemplar capacity addition Signed-off-by: Levi Harrison <git@leviharrison.dev> * Added pre-allocation Signed-off-by: Levi Harrison <git@leviharrison.dev> * Don't allocate if not sending exemplars Signed-off-by: Levi Harrison <git@leviharrison.dev> * Avoid deadlock when processing duplicate series record (#9170) * Avoid deadlock when processing duplicate series record `processWALSamples()` needs to be able to send on its output channel before it can read the input channel, so reads to allow this in case the output channel is full. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * processWALSamples: update comment Previous text seems to relate to an earlier implementation. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Optimise WAL loading by removing extra map and caching min-time (#9160) * BenchmarkLoadWAL: close WAL after use So that goroutines are stopped and resources released Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * BenchmarkLoadWAL: make series IDs co-prime with #workers Series are distributed across workers by taking the modulus of the ID with the number of workers, so multiples of 100 are a poor choice. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * BenchmarkLoadWAL: simulate mmapped chunks Real Prometheus cuts chunks every 120 samples, then skips those samples when re-reading the WAL. Simulate this by creating a single mapped chunk for each series, since the max time is all the reader looks at. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Fix comment Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Remove series map from processWALSamples() The locks that is commented to reduce contention in are now sharded 32,000 ways, so won't be contended. Removing the map saves memory and goes just as fast. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * loadWAL: Cache the last mmapped chunk time So we can skip calling append() for samples it will reject. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Improvements from code review Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Full stops and capitals on comments Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Cache max time in both places mmappedChunks is updated Including refactor to extract function `setMMappedChunks`, to reduce code duplication. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Update head min/max time when mmapped chunks added This ensures we have the correct values if no WAL samples are added for that series. Note that `mSeries.maxTime()` was always `math.MinInt64` before, since that function doesn't consider mmapped chunks. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Split Go and React Tests (#8897) * Added go-ci and react-ci Co-authored-by: Julien Pivotto <roidelapluie@inuits.eu> Signed-off-by: Levi Harrison <git@leviharrison.dev> * Remove search keymap from new expression editor (#9184) Signed-off-by: Julius Volz <julius.volz@gmail.com> Co-authored-by: Austin Cawley-Edwards <austin.cawley@gmail.com> Co-authored-by: Levi Harrison <git@leviharrison.dev> Co-authored-by: Julien Pivotto <roidelapluie@inuits.eu> Co-authored-by: Bryan Boreham <bjboreham@gmail.com> Co-authored-by: Julius Volz <julius.volz@gmail.com>
2021-08-11 03:13:17 -07:00
if err != nil {
return snapIdx, snapOffset, nil, errors.Wrap(err, "open chunk snapshot")
}
defer func() {
if err := sr.Close(); err != nil {
level.Warn(h.logger).Log("msg", "error while closing the wal segments reader", "err", err)
}
}()
var (
numSeries = 0
unknownRefs = int64(0)
n = runtime.GOMAXPROCS(0)
wg sync.WaitGroup
recordChan = make(chan chunkSnapshotRecord, 5*n)
shardedRefSeries = make([]map[chunks.HeadSeriesRef]*memSeries, n)
Sync sparsehistogram branch with main (#9189) * Fix `kuma_sd` targetgroup reporting (#9157) * Bundle all xDS targets into a single group Signed-off-by: austin ce <austin.cawley@gmail.com> * Snapshot in-memory chunks on shutdown for faster restarts (#7229) Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Rename links Signed-off-by: Levi Harrison <git@leviharrison.dev> * Remove Individual Data Type Caps in Per-shard Buffering for Remote Write (#8921) * Moved everything to nPending buffer Signed-off-by: Levi Harrison <git@leviharrison.dev> * Simplify exemplar capacity addition Signed-off-by: Levi Harrison <git@leviharrison.dev> * Added pre-allocation Signed-off-by: Levi Harrison <git@leviharrison.dev> * Don't allocate if not sending exemplars Signed-off-by: Levi Harrison <git@leviharrison.dev> * Avoid deadlock when processing duplicate series record (#9170) * Avoid deadlock when processing duplicate series record `processWALSamples()` needs to be able to send on its output channel before it can read the input channel, so reads to allow this in case the output channel is full. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * processWALSamples: update comment Previous text seems to relate to an earlier implementation. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Optimise WAL loading by removing extra map and caching min-time (#9160) * BenchmarkLoadWAL: close WAL after use So that goroutines are stopped and resources released Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * BenchmarkLoadWAL: make series IDs co-prime with #workers Series are distributed across workers by taking the modulus of the ID with the number of workers, so multiples of 100 are a poor choice. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * BenchmarkLoadWAL: simulate mmapped chunks Real Prometheus cuts chunks every 120 samples, then skips those samples when re-reading the WAL. Simulate this by creating a single mapped chunk for each series, since the max time is all the reader looks at. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Fix comment Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Remove series map from processWALSamples() The locks that is commented to reduce contention in are now sharded 32,000 ways, so won't be contended. Removing the map saves memory and goes just as fast. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * loadWAL: Cache the last mmapped chunk time So we can skip calling append() for samples it will reject. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Improvements from code review Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Full stops and capitals on comments Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Cache max time in both places mmappedChunks is updated Including refactor to extract function `setMMappedChunks`, to reduce code duplication. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Update head min/max time when mmapped chunks added This ensures we have the correct values if no WAL samples are added for that series. Note that `mSeries.maxTime()` was always `math.MinInt64` before, since that function doesn't consider mmapped chunks. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Split Go and React Tests (#8897) * Added go-ci and react-ci Co-authored-by: Julien Pivotto <roidelapluie@inuits.eu> Signed-off-by: Levi Harrison <git@leviharrison.dev> * Remove search keymap from new expression editor (#9184) Signed-off-by: Julius Volz <julius.volz@gmail.com> Co-authored-by: Austin Cawley-Edwards <austin.cawley@gmail.com> Co-authored-by: Levi Harrison <git@leviharrison.dev> Co-authored-by: Julien Pivotto <roidelapluie@inuits.eu> Co-authored-by: Bryan Boreham <bjboreham@gmail.com> Co-authored-by: Julius Volz <julius.volz@gmail.com>
2021-08-11 03:13:17 -07:00
errChan = make(chan error, n)
refSeries map[chunks.HeadSeriesRef]*memSeries
exemplarBuf []record.RefExemplar
dec record.Decoder
Sync sparsehistogram branch with main (#9189) * Fix `kuma_sd` targetgroup reporting (#9157) * Bundle all xDS targets into a single group Signed-off-by: austin ce <austin.cawley@gmail.com> * Snapshot in-memory chunks on shutdown for faster restarts (#7229) Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Rename links Signed-off-by: Levi Harrison <git@leviharrison.dev> * Remove Individual Data Type Caps in Per-shard Buffering for Remote Write (#8921) * Moved everything to nPending buffer Signed-off-by: Levi Harrison <git@leviharrison.dev> * Simplify exemplar capacity addition Signed-off-by: Levi Harrison <git@leviharrison.dev> * Added pre-allocation Signed-off-by: Levi Harrison <git@leviharrison.dev> * Don't allocate if not sending exemplars Signed-off-by: Levi Harrison <git@leviharrison.dev> * Avoid deadlock when processing duplicate series record (#9170) * Avoid deadlock when processing duplicate series record `processWALSamples()` needs to be able to send on its output channel before it can read the input channel, so reads to allow this in case the output channel is full. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * processWALSamples: update comment Previous text seems to relate to an earlier implementation. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Optimise WAL loading by removing extra map and caching min-time (#9160) * BenchmarkLoadWAL: close WAL after use So that goroutines are stopped and resources released Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * BenchmarkLoadWAL: make series IDs co-prime with #workers Series are distributed across workers by taking the modulus of the ID with the number of workers, so multiples of 100 are a poor choice. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * BenchmarkLoadWAL: simulate mmapped chunks Real Prometheus cuts chunks every 120 samples, then skips those samples when re-reading the WAL. Simulate this by creating a single mapped chunk for each series, since the max time is all the reader looks at. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Fix comment Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Remove series map from processWALSamples() The locks that is commented to reduce contention in are now sharded 32,000 ways, so won't be contended. Removing the map saves memory and goes just as fast. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * loadWAL: Cache the last mmapped chunk time So we can skip calling append() for samples it will reject. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Improvements from code review Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Full stops and capitals on comments Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Cache max time in both places mmappedChunks is updated Including refactor to extract function `setMMappedChunks`, to reduce code duplication. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Update head min/max time when mmapped chunks added This ensures we have the correct values if no WAL samples are added for that series. Note that `mSeries.maxTime()` was always `math.MinInt64` before, since that function doesn't consider mmapped chunks. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Split Go and React Tests (#8897) * Added go-ci and react-ci Co-authored-by: Julien Pivotto <roidelapluie@inuits.eu> Signed-off-by: Levi Harrison <git@leviharrison.dev> * Remove search keymap from new expression editor (#9184) Signed-off-by: Julius Volz <julius.volz@gmail.com> Co-authored-by: Austin Cawley-Edwards <austin.cawley@gmail.com> Co-authored-by: Levi Harrison <git@leviharrison.dev> Co-authored-by: Julien Pivotto <roidelapluie@inuits.eu> Co-authored-by: Bryan Boreham <bjboreham@gmail.com> Co-authored-by: Julius Volz <julius.volz@gmail.com>
2021-08-11 03:13:17 -07:00
)
wg.Add(n)
for i := 0; i < n; i++ {
go func(idx int, rc <-chan chunkSnapshotRecord) {
defer wg.Done()
defer func() {
// If there was an error, drain the channel
// to unblock the main thread.
for range rc {
}
}()
shardedRefSeries[idx] = make(map[chunks.HeadSeriesRef]*memSeries)
Sync sparsehistogram branch with main (#9189) * Fix `kuma_sd` targetgroup reporting (#9157) * Bundle all xDS targets into a single group Signed-off-by: austin ce <austin.cawley@gmail.com> * Snapshot in-memory chunks on shutdown for faster restarts (#7229) Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Rename links Signed-off-by: Levi Harrison <git@leviharrison.dev> * Remove Individual Data Type Caps in Per-shard Buffering for Remote Write (#8921) * Moved everything to nPending buffer Signed-off-by: Levi Harrison <git@leviharrison.dev> * Simplify exemplar capacity addition Signed-off-by: Levi Harrison <git@leviharrison.dev> * Added pre-allocation Signed-off-by: Levi Harrison <git@leviharrison.dev> * Don't allocate if not sending exemplars Signed-off-by: Levi Harrison <git@leviharrison.dev> * Avoid deadlock when processing duplicate series record (#9170) * Avoid deadlock when processing duplicate series record `processWALSamples()` needs to be able to send on its output channel before it can read the input channel, so reads to allow this in case the output channel is full. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * processWALSamples: update comment Previous text seems to relate to an earlier implementation. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Optimise WAL loading by removing extra map and caching min-time (#9160) * BenchmarkLoadWAL: close WAL after use So that goroutines are stopped and resources released Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * BenchmarkLoadWAL: make series IDs co-prime with #workers Series are distributed across workers by taking the modulus of the ID with the number of workers, so multiples of 100 are a poor choice. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * BenchmarkLoadWAL: simulate mmapped chunks Real Prometheus cuts chunks every 120 samples, then skips those samples when re-reading the WAL. Simulate this by creating a single mapped chunk for each series, since the max time is all the reader looks at. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Fix comment Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Remove series map from processWALSamples() The locks that is commented to reduce contention in are now sharded 32,000 ways, so won't be contended. Removing the map saves memory and goes just as fast. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * loadWAL: Cache the last mmapped chunk time So we can skip calling append() for samples it will reject. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Improvements from code review Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Full stops and capitals on comments Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Cache max time in both places mmappedChunks is updated Including refactor to extract function `setMMappedChunks`, to reduce code duplication. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Update head min/max time when mmapped chunks added This ensures we have the correct values if no WAL samples are added for that series. Note that `mSeries.maxTime()` was always `math.MinInt64` before, since that function doesn't consider mmapped chunks. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Split Go and React Tests (#8897) * Added go-ci and react-ci Co-authored-by: Julien Pivotto <roidelapluie@inuits.eu> Signed-off-by: Levi Harrison <git@leviharrison.dev> * Remove search keymap from new expression editor (#9184) Signed-off-by: Julius Volz <julius.volz@gmail.com> Co-authored-by: Austin Cawley-Edwards <austin.cawley@gmail.com> Co-authored-by: Levi Harrison <git@leviharrison.dev> Co-authored-by: Julien Pivotto <roidelapluie@inuits.eu> Co-authored-by: Bryan Boreham <bjboreham@gmail.com> Co-authored-by: Julius Volz <julius.volz@gmail.com>
2021-08-11 03:13:17 -07:00
localRefSeries := shardedRefSeries[idx]
for csr := range rc {
series, _, err := h.getOrCreateWithID(csr.ref, csr.lset.Hash(), csr.lset)
if err != nil {
errChan <- err
return
}
localRefSeries[csr.ref] = series
for {
seriesID := uint64(series.ref)
lastSeriesID := h.lastSeriesID.Load()
if lastSeriesID >= seriesID || h.lastSeriesID.CompareAndSwap(lastSeriesID, seriesID) {
break
}
Sync sparsehistogram branch with main (#9189) * Fix `kuma_sd` targetgroup reporting (#9157) * Bundle all xDS targets into a single group Signed-off-by: austin ce <austin.cawley@gmail.com> * Snapshot in-memory chunks on shutdown for faster restarts (#7229) Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Rename links Signed-off-by: Levi Harrison <git@leviharrison.dev> * Remove Individual Data Type Caps in Per-shard Buffering for Remote Write (#8921) * Moved everything to nPending buffer Signed-off-by: Levi Harrison <git@leviharrison.dev> * Simplify exemplar capacity addition Signed-off-by: Levi Harrison <git@leviharrison.dev> * Added pre-allocation Signed-off-by: Levi Harrison <git@leviharrison.dev> * Don't allocate if not sending exemplars Signed-off-by: Levi Harrison <git@leviharrison.dev> * Avoid deadlock when processing duplicate series record (#9170) * Avoid deadlock when processing duplicate series record `processWALSamples()` needs to be able to send on its output channel before it can read the input channel, so reads to allow this in case the output channel is full. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * processWALSamples: update comment Previous text seems to relate to an earlier implementation. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Optimise WAL loading by removing extra map and caching min-time (#9160) * BenchmarkLoadWAL: close WAL after use So that goroutines are stopped and resources released Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * BenchmarkLoadWAL: make series IDs co-prime with #workers Series are distributed across workers by taking the modulus of the ID with the number of workers, so multiples of 100 are a poor choice. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * BenchmarkLoadWAL: simulate mmapped chunks Real Prometheus cuts chunks every 120 samples, then skips those samples when re-reading the WAL. Simulate this by creating a single mapped chunk for each series, since the max time is all the reader looks at. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Fix comment Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Remove series map from processWALSamples() The locks that is commented to reduce contention in are now sharded 32,000 ways, so won't be contended. Removing the map saves memory and goes just as fast. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * loadWAL: Cache the last mmapped chunk time So we can skip calling append() for samples it will reject. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Improvements from code review Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Full stops and capitals on comments Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Cache max time in both places mmappedChunks is updated Including refactor to extract function `setMMappedChunks`, to reduce code duplication. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Update head min/max time when mmapped chunks added This ensures we have the correct values if no WAL samples are added for that series. Note that `mSeries.maxTime()` was always `math.MinInt64` before, since that function doesn't consider mmapped chunks. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Split Go and React Tests (#8897) * Added go-ci and react-ci Co-authored-by: Julien Pivotto <roidelapluie@inuits.eu> Signed-off-by: Levi Harrison <git@leviharrison.dev> * Remove search keymap from new expression editor (#9184) Signed-off-by: Julius Volz <julius.volz@gmail.com> Co-authored-by: Austin Cawley-Edwards <austin.cawley@gmail.com> Co-authored-by: Levi Harrison <git@leviharrison.dev> Co-authored-by: Julien Pivotto <roidelapluie@inuits.eu> Co-authored-by: Bryan Boreham <bjboreham@gmail.com> Co-authored-by: Julius Volz <julius.volz@gmail.com>
2021-08-11 03:13:17 -07:00
}
if csr.mc == nil {
continue
}
series.nextAt = csr.mc.maxTime // This will create a new chunk on append.
series.headChunk = csr.mc
tsdb: stop saving a copy of last 4 samples in memSeries (#11296) * TSDB chunks: remove race between writing and reading Because the data is stored as a bit-stream, the last byte in the stream could change if the stream is appended to after an Iterator is obtained. Copy the last byte when the Iterator is created, so we don't have to read it later. Clarify in comments that concurrent Iterator and Appender are allowed, but the chunk must not be modified while an Iterator is created. (This was already the case, in order to copy the bstream slice header.) * TSDB: stop saving last 4 samples in memSeries This extra copy of the last 4 samples was introduced to avoid a race condition between reading the last byte of the chunk and writing to it. But now we have fixed that by having `bstreamReader` copy the last byte, we don't need to copy the last 4 samples. This change saves 56 bytes per series, which is very worthwhile when you have millions or tens of millions of series. * TSDB: tidy up stopIterator re-use Previous changes have left this code duplicating some lines; pull them out to a separate function and tidy up. * TSDB head_test: stop checking when iterators are wrapped The behaviour has changed so chunk iterators are only wrapped when transaction isolation requires them to stop short of the end. This makes tests fail which are checking the type. Tests should check the observable behaviour, not the type. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> Co-authored-by: Ganesh Vernekar <ganeshvern@gmail.com>
2022-09-27 07:02:05 -07:00
series.lastValue = csr.lastValue
Sync sparsehistogram branch with main (#9189) * Fix `kuma_sd` targetgroup reporting (#9157) * Bundle all xDS targets into a single group Signed-off-by: austin ce <austin.cawley@gmail.com> * Snapshot in-memory chunks on shutdown for faster restarts (#7229) Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Rename links Signed-off-by: Levi Harrison <git@leviharrison.dev> * Remove Individual Data Type Caps in Per-shard Buffering for Remote Write (#8921) * Moved everything to nPending buffer Signed-off-by: Levi Harrison <git@leviharrison.dev> * Simplify exemplar capacity addition Signed-off-by: Levi Harrison <git@leviharrison.dev> * Added pre-allocation Signed-off-by: Levi Harrison <git@leviharrison.dev> * Don't allocate if not sending exemplars Signed-off-by: Levi Harrison <git@leviharrison.dev> * Avoid deadlock when processing duplicate series record (#9170) * Avoid deadlock when processing duplicate series record `processWALSamples()` needs to be able to send on its output channel before it can read the input channel, so reads to allow this in case the output channel is full. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * processWALSamples: update comment Previous text seems to relate to an earlier implementation. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Optimise WAL loading by removing extra map and caching min-time (#9160) * BenchmarkLoadWAL: close WAL after use So that goroutines are stopped and resources released Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * BenchmarkLoadWAL: make series IDs co-prime with #workers Series are distributed across workers by taking the modulus of the ID with the number of workers, so multiples of 100 are a poor choice. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * BenchmarkLoadWAL: simulate mmapped chunks Real Prometheus cuts chunks every 120 samples, then skips those samples when re-reading the WAL. Simulate this by creating a single mapped chunk for each series, since the max time is all the reader looks at. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Fix comment Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Remove series map from processWALSamples() The locks that is commented to reduce contention in are now sharded 32,000 ways, so won't be contended. Removing the map saves memory and goes just as fast. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * loadWAL: Cache the last mmapped chunk time So we can skip calling append() for samples it will reject. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Improvements from code review Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Full stops and capitals on comments Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Cache max time in both places mmappedChunks is updated Including refactor to extract function `setMMappedChunks`, to reduce code duplication. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Update head min/max time when mmapped chunks added This ensures we have the correct values if no WAL samples are added for that series. Note that `mSeries.maxTime()` was always `math.MinInt64` before, since that function doesn't consider mmapped chunks. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Split Go and React Tests (#8897) * Added go-ci and react-ci Co-authored-by: Julien Pivotto <roidelapluie@inuits.eu> Signed-off-by: Levi Harrison <git@leviharrison.dev> * Remove search keymap from new expression editor (#9184) Signed-off-by: Julius Volz <julius.volz@gmail.com> Co-authored-by: Austin Cawley-Edwards <austin.cawley@gmail.com> Co-authored-by: Levi Harrison <git@leviharrison.dev> Co-authored-by: Julien Pivotto <roidelapluie@inuits.eu> Co-authored-by: Bryan Boreham <bjboreham@gmail.com> Co-authored-by: Julius Volz <julius.volz@gmail.com>
2021-08-11 03:13:17 -07:00
app, err := series.headChunk.chunk.Appender()
if err != nil {
errChan <- err
return
}
series.app = app
h.updateMinMaxTime(csr.mc.minTime, csr.mc.maxTime)
}
}(i, recordChan)
}
r := wlog.NewReader(sr)
Sync sparsehistogram branch with main (#9189) * Fix `kuma_sd` targetgroup reporting (#9157) * Bundle all xDS targets into a single group Signed-off-by: austin ce <austin.cawley@gmail.com> * Snapshot in-memory chunks on shutdown for faster restarts (#7229) Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Rename links Signed-off-by: Levi Harrison <git@leviharrison.dev> * Remove Individual Data Type Caps in Per-shard Buffering for Remote Write (#8921) * Moved everything to nPending buffer Signed-off-by: Levi Harrison <git@leviharrison.dev> * Simplify exemplar capacity addition Signed-off-by: Levi Harrison <git@leviharrison.dev> * Added pre-allocation Signed-off-by: Levi Harrison <git@leviharrison.dev> * Don't allocate if not sending exemplars Signed-off-by: Levi Harrison <git@leviharrison.dev> * Avoid deadlock when processing duplicate series record (#9170) * Avoid deadlock when processing duplicate series record `processWALSamples()` needs to be able to send on its output channel before it can read the input channel, so reads to allow this in case the output channel is full. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * processWALSamples: update comment Previous text seems to relate to an earlier implementation. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Optimise WAL loading by removing extra map and caching min-time (#9160) * BenchmarkLoadWAL: close WAL after use So that goroutines are stopped and resources released Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * BenchmarkLoadWAL: make series IDs co-prime with #workers Series are distributed across workers by taking the modulus of the ID with the number of workers, so multiples of 100 are a poor choice. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * BenchmarkLoadWAL: simulate mmapped chunks Real Prometheus cuts chunks every 120 samples, then skips those samples when re-reading the WAL. Simulate this by creating a single mapped chunk for each series, since the max time is all the reader looks at. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Fix comment Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Remove series map from processWALSamples() The locks that is commented to reduce contention in are now sharded 32,000 ways, so won't be contended. Removing the map saves memory and goes just as fast. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * loadWAL: Cache the last mmapped chunk time So we can skip calling append() for samples it will reject. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Improvements from code review Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Full stops and capitals on comments Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Cache max time in both places mmappedChunks is updated Including refactor to extract function `setMMappedChunks`, to reduce code duplication. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Update head min/max time when mmapped chunks added This ensures we have the correct values if no WAL samples are added for that series. Note that `mSeries.maxTime()` was always `math.MinInt64` before, since that function doesn't consider mmapped chunks. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Split Go and React Tests (#8897) * Added go-ci and react-ci Co-authored-by: Julien Pivotto <roidelapluie@inuits.eu> Signed-off-by: Levi Harrison <git@leviharrison.dev> * Remove search keymap from new expression editor (#9184) Signed-off-by: Julius Volz <julius.volz@gmail.com> Co-authored-by: Austin Cawley-Edwards <austin.cawley@gmail.com> Co-authored-by: Levi Harrison <git@leviharrison.dev> Co-authored-by: Julien Pivotto <roidelapluie@inuits.eu> Co-authored-by: Bryan Boreham <bjboreham@gmail.com> Co-authored-by: Julius Volz <julius.volz@gmail.com>
2021-08-11 03:13:17 -07:00
var loopErr error
Outer:
for r.Next() {
select {
case err := <-errChan:
errChan <- err
break Outer
default:
}
rec := r.Record()
switch rec[0] {
case chunkSnapshotRecordTypeSeries:
numSeries++
csr, err := decodeSeriesFromChunkSnapshot(&dec, rec)
Sync sparsehistogram branch with main (#9189) * Fix `kuma_sd` targetgroup reporting (#9157) * Bundle all xDS targets into a single group Signed-off-by: austin ce <austin.cawley@gmail.com> * Snapshot in-memory chunks on shutdown for faster restarts (#7229) Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Rename links Signed-off-by: Levi Harrison <git@leviharrison.dev> * Remove Individual Data Type Caps in Per-shard Buffering for Remote Write (#8921) * Moved everything to nPending buffer Signed-off-by: Levi Harrison <git@leviharrison.dev> * Simplify exemplar capacity addition Signed-off-by: Levi Harrison <git@leviharrison.dev> * Added pre-allocation Signed-off-by: Levi Harrison <git@leviharrison.dev> * Don't allocate if not sending exemplars Signed-off-by: Levi Harrison <git@leviharrison.dev> * Avoid deadlock when processing duplicate series record (#9170) * Avoid deadlock when processing duplicate series record `processWALSamples()` needs to be able to send on its output channel before it can read the input channel, so reads to allow this in case the output channel is full. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * processWALSamples: update comment Previous text seems to relate to an earlier implementation. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Optimise WAL loading by removing extra map and caching min-time (#9160) * BenchmarkLoadWAL: close WAL after use So that goroutines are stopped and resources released Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * BenchmarkLoadWAL: make series IDs co-prime with #workers Series are distributed across workers by taking the modulus of the ID with the number of workers, so multiples of 100 are a poor choice. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * BenchmarkLoadWAL: simulate mmapped chunks Real Prometheus cuts chunks every 120 samples, then skips those samples when re-reading the WAL. Simulate this by creating a single mapped chunk for each series, since the max time is all the reader looks at. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Fix comment Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Remove series map from processWALSamples() The locks that is commented to reduce contention in are now sharded 32,000 ways, so won't be contended. Removing the map saves memory and goes just as fast. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * loadWAL: Cache the last mmapped chunk time So we can skip calling append() for samples it will reject. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Improvements from code review Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Full stops and capitals on comments Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Cache max time in both places mmappedChunks is updated Including refactor to extract function `setMMappedChunks`, to reduce code duplication. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Update head min/max time when mmapped chunks added This ensures we have the correct values if no WAL samples are added for that series. Note that `mSeries.maxTime()` was always `math.MinInt64` before, since that function doesn't consider mmapped chunks. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Split Go and React Tests (#8897) * Added go-ci and react-ci Co-authored-by: Julien Pivotto <roidelapluie@inuits.eu> Signed-off-by: Levi Harrison <git@leviharrison.dev> * Remove search keymap from new expression editor (#9184) Signed-off-by: Julius Volz <julius.volz@gmail.com> Co-authored-by: Austin Cawley-Edwards <austin.cawley@gmail.com> Co-authored-by: Levi Harrison <git@leviharrison.dev> Co-authored-by: Julien Pivotto <roidelapluie@inuits.eu> Co-authored-by: Bryan Boreham <bjboreham@gmail.com> Co-authored-by: Julius Volz <julius.volz@gmail.com>
2021-08-11 03:13:17 -07:00
if err != nil {
loopErr = errors.Wrap(err, "decode series record")
break Outer
}
recordChan <- csr
case chunkSnapshotRecordTypeTombstones:
tr, err := decodeTombstonesSnapshotRecord(rec)
if err != nil {
loopErr = errors.Wrap(err, "decode tombstones")
break Outer
}
if err = tr.Iter(func(ref storage.SeriesRef, ivs tombstones.Intervals) error {
Sync sparsehistogram branch with main (#9189) * Fix `kuma_sd` targetgroup reporting (#9157) * Bundle all xDS targets into a single group Signed-off-by: austin ce <austin.cawley@gmail.com> * Snapshot in-memory chunks on shutdown for faster restarts (#7229) Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Rename links Signed-off-by: Levi Harrison <git@leviharrison.dev> * Remove Individual Data Type Caps in Per-shard Buffering for Remote Write (#8921) * Moved everything to nPending buffer Signed-off-by: Levi Harrison <git@leviharrison.dev> * Simplify exemplar capacity addition Signed-off-by: Levi Harrison <git@leviharrison.dev> * Added pre-allocation Signed-off-by: Levi Harrison <git@leviharrison.dev> * Don't allocate if not sending exemplars Signed-off-by: Levi Harrison <git@leviharrison.dev> * Avoid deadlock when processing duplicate series record (#9170) * Avoid deadlock when processing duplicate series record `processWALSamples()` needs to be able to send on its output channel before it can read the input channel, so reads to allow this in case the output channel is full. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * processWALSamples: update comment Previous text seems to relate to an earlier implementation. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Optimise WAL loading by removing extra map and caching min-time (#9160) * BenchmarkLoadWAL: close WAL after use So that goroutines are stopped and resources released Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * BenchmarkLoadWAL: make series IDs co-prime with #workers Series are distributed across workers by taking the modulus of the ID with the number of workers, so multiples of 100 are a poor choice. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * BenchmarkLoadWAL: simulate mmapped chunks Real Prometheus cuts chunks every 120 samples, then skips those samples when re-reading the WAL. Simulate this by creating a single mapped chunk for each series, since the max time is all the reader looks at. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Fix comment Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Remove series map from processWALSamples() The locks that is commented to reduce contention in are now sharded 32,000 ways, so won't be contended. Removing the map saves memory and goes just as fast. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * loadWAL: Cache the last mmapped chunk time So we can skip calling append() for samples it will reject. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Improvements from code review Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Full stops and capitals on comments Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Cache max time in both places mmappedChunks is updated Including refactor to extract function `setMMappedChunks`, to reduce code duplication. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Update head min/max time when mmapped chunks added This ensures we have the correct values if no WAL samples are added for that series. Note that `mSeries.maxTime()` was always `math.MinInt64` before, since that function doesn't consider mmapped chunks. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Split Go and React Tests (#8897) * Added go-ci and react-ci Co-authored-by: Julien Pivotto <roidelapluie@inuits.eu> Signed-off-by: Levi Harrison <git@leviharrison.dev> * Remove search keymap from new expression editor (#9184) Signed-off-by: Julius Volz <julius.volz@gmail.com> Co-authored-by: Austin Cawley-Edwards <austin.cawley@gmail.com> Co-authored-by: Levi Harrison <git@leviharrison.dev> Co-authored-by: Julien Pivotto <roidelapluie@inuits.eu> Co-authored-by: Bryan Boreham <bjboreham@gmail.com> Co-authored-by: Julius Volz <julius.volz@gmail.com>
2021-08-11 03:13:17 -07:00
h.tombstones.AddInterval(ref, ivs...)
return nil
}); err != nil {
loopErr = errors.Wrap(err, "iterate tombstones")
break Outer
}
case chunkSnapshotRecordTypeExemplars:
// Exemplars are at the end of snapshot. So all series are loaded at this point.
if len(refSeries) == 0 {
close(recordChan)
wg.Wait()
refSeries = make(map[chunks.HeadSeriesRef]*memSeries, numSeries)
for _, shard := range shardedRefSeries {
for k, v := range shard {
refSeries[k] = v
}
}
}
if !h.opts.EnableExemplarStorage || h.opts.MaxExemplars.Load() <= 0 {
// Exemplar storage is disabled.
continue Outer
}
decbuf := encoding.Decbuf{B: rec[1:]}
exemplarBuf = exemplarBuf[:0]
exemplarBuf, err = dec.ExemplarsFromBuffer(&decbuf, exemplarBuf)
if err != nil {
loopErr = errors.Wrap(err, "exemplars from buffer")
break Outer
}
for _, e := range exemplarBuf {
ms, ok := refSeries[e.Ref]
if !ok {
unknownRefs++
continue
}
if err := h.exemplars.AddExemplar(ms.lset, exemplar.Exemplar{
Labels: e.Labels,
Value: e.V,
Ts: e.T,
}); err != nil {
loopErr = errors.Wrap(err, "add exemplar")
break Outer
}
}
default:
// This is a record type we don't understand. It is either and old format from earlier versions,
// or a new format and the code was rolled back to old version.
loopErr = errors.Errorf("unsuported snapshot record type 0b%b", rec[0])
break Outer
}
}
if len(refSeries) == 0 {
close(recordChan)
wg.Wait()
Sync sparsehistogram branch with main (#9189) * Fix `kuma_sd` targetgroup reporting (#9157) * Bundle all xDS targets into a single group Signed-off-by: austin ce <austin.cawley@gmail.com> * Snapshot in-memory chunks on shutdown for faster restarts (#7229) Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Rename links Signed-off-by: Levi Harrison <git@leviharrison.dev> * Remove Individual Data Type Caps in Per-shard Buffering for Remote Write (#8921) * Moved everything to nPending buffer Signed-off-by: Levi Harrison <git@leviharrison.dev> * Simplify exemplar capacity addition Signed-off-by: Levi Harrison <git@leviharrison.dev> * Added pre-allocation Signed-off-by: Levi Harrison <git@leviharrison.dev> * Don't allocate if not sending exemplars Signed-off-by: Levi Harrison <git@leviharrison.dev> * Avoid deadlock when processing duplicate series record (#9170) * Avoid deadlock when processing duplicate series record `processWALSamples()` needs to be able to send on its output channel before it can read the input channel, so reads to allow this in case the output channel is full. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * processWALSamples: update comment Previous text seems to relate to an earlier implementation. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Optimise WAL loading by removing extra map and caching min-time (#9160) * BenchmarkLoadWAL: close WAL after use So that goroutines are stopped and resources released Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * BenchmarkLoadWAL: make series IDs co-prime with #workers Series are distributed across workers by taking the modulus of the ID with the number of workers, so multiples of 100 are a poor choice. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * BenchmarkLoadWAL: simulate mmapped chunks Real Prometheus cuts chunks every 120 samples, then skips those samples when re-reading the WAL. Simulate this by creating a single mapped chunk for each series, since the max time is all the reader looks at. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Fix comment Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Remove series map from processWALSamples() The locks that is commented to reduce contention in are now sharded 32,000 ways, so won't be contended. Removing the map saves memory and goes just as fast. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * loadWAL: Cache the last mmapped chunk time So we can skip calling append() for samples it will reject. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Improvements from code review Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Full stops and capitals on comments Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Cache max time in both places mmappedChunks is updated Including refactor to extract function `setMMappedChunks`, to reduce code duplication. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Update head min/max time when mmapped chunks added This ensures we have the correct values if no WAL samples are added for that series. Note that `mSeries.maxTime()` was always `math.MinInt64` before, since that function doesn't consider mmapped chunks. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Split Go and React Tests (#8897) * Added go-ci and react-ci Co-authored-by: Julien Pivotto <roidelapluie@inuits.eu> Signed-off-by: Levi Harrison <git@leviharrison.dev> * Remove search keymap from new expression editor (#9184) Signed-off-by: Julius Volz <julius.volz@gmail.com> Co-authored-by: Austin Cawley-Edwards <austin.cawley@gmail.com> Co-authored-by: Levi Harrison <git@leviharrison.dev> Co-authored-by: Julien Pivotto <roidelapluie@inuits.eu> Co-authored-by: Bryan Boreham <bjboreham@gmail.com> Co-authored-by: Julius Volz <julius.volz@gmail.com>
2021-08-11 03:13:17 -07:00
}
close(errChan)
merr := tsdb_errors.NewMulti(errors.Wrap(loopErr, "decode loop"))
for err := range errChan {
merr.Add(errors.Wrap(err, "record processing"))
}
if err := merr.Err(); err != nil {
return -1, -1, nil, err
}
if r.Err() != nil {
return -1, -1, nil, errors.Wrap(r.Err(), "read records")
}
if len(refSeries) == 0 {
// We had no exemplar record, so we have to build the map here.
refSeries = make(map[chunks.HeadSeriesRef]*memSeries, numSeries)
for _, shard := range shardedRefSeries {
for k, v := range shard {
refSeries[k] = v
}
Sync sparsehistogram branch with main (#9189) * Fix `kuma_sd` targetgroup reporting (#9157) * Bundle all xDS targets into a single group Signed-off-by: austin ce <austin.cawley@gmail.com> * Snapshot in-memory chunks on shutdown for faster restarts (#7229) Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com> * Rename links Signed-off-by: Levi Harrison <git@leviharrison.dev> * Remove Individual Data Type Caps in Per-shard Buffering for Remote Write (#8921) * Moved everything to nPending buffer Signed-off-by: Levi Harrison <git@leviharrison.dev> * Simplify exemplar capacity addition Signed-off-by: Levi Harrison <git@leviharrison.dev> * Added pre-allocation Signed-off-by: Levi Harrison <git@leviharrison.dev> * Don't allocate if not sending exemplars Signed-off-by: Levi Harrison <git@leviharrison.dev> * Avoid deadlock when processing duplicate series record (#9170) * Avoid deadlock when processing duplicate series record `processWALSamples()` needs to be able to send on its output channel before it can read the input channel, so reads to allow this in case the output channel is full. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * processWALSamples: update comment Previous text seems to relate to an earlier implementation. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Optimise WAL loading by removing extra map and caching min-time (#9160) * BenchmarkLoadWAL: close WAL after use So that goroutines are stopped and resources released Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * BenchmarkLoadWAL: make series IDs co-prime with #workers Series are distributed across workers by taking the modulus of the ID with the number of workers, so multiples of 100 are a poor choice. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * BenchmarkLoadWAL: simulate mmapped chunks Real Prometheus cuts chunks every 120 samples, then skips those samples when re-reading the WAL. Simulate this by creating a single mapped chunk for each series, since the max time is all the reader looks at. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Fix comment Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Remove series map from processWALSamples() The locks that is commented to reduce contention in are now sharded 32,000 ways, so won't be contended. Removing the map saves memory and goes just as fast. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * loadWAL: Cache the last mmapped chunk time So we can skip calling append() for samples it will reject. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Improvements from code review Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Full stops and capitals on comments Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Cache max time in both places mmappedChunks is updated Including refactor to extract function `setMMappedChunks`, to reduce code duplication. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Update head min/max time when mmapped chunks added This ensures we have the correct values if no WAL samples are added for that series. Note that `mSeries.maxTime()` was always `math.MinInt64` before, since that function doesn't consider mmapped chunks. Signed-off-by: Bryan Boreham <bjboreham@gmail.com> * Split Go and React Tests (#8897) * Added go-ci and react-ci Co-authored-by: Julien Pivotto <roidelapluie@inuits.eu> Signed-off-by: Levi Harrison <git@leviharrison.dev> * Remove search keymap from new expression editor (#9184) Signed-off-by: Julius Volz <julius.volz@gmail.com> Co-authored-by: Austin Cawley-Edwards <austin.cawley@gmail.com> Co-authored-by: Levi Harrison <git@leviharrison.dev> Co-authored-by: Julien Pivotto <roidelapluie@inuits.eu> Co-authored-by: Bryan Boreham <bjboreham@gmail.com> Co-authored-by: Julius Volz <julius.volz@gmail.com>
2021-08-11 03:13:17 -07:00
}
}
elapsed := time.Since(start)
level.Info(h.logger).Log("msg", "chunk snapshot loaded", "dir", dir, "num_series", numSeries, "duration", elapsed.String())
if unknownRefs > 0 {
level.Warn(h.logger).Log("msg", "unknown series references during chunk snapshot replay", "count", unknownRefs)
}
return snapIdx, snapOffset, refSeries, nil
}