2018-09-07 21:26:04 +00:00
// Copyright 2018 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 remote
import (
"fmt"
2019-01-18 20:31:36 +00:00
"io"
"math"
2018-09-07 21:26:04 +00:00
"os"
"path"
2019-02-13 14:47:35 +00:00
"sort"
2018-09-07 21:26:04 +00:00
"strconv"
"strings"
"time"
"github.com/go-kit/kit/log"
"github.com/go-kit/kit/log/level"
"github.com/pkg/errors"
"github.com/prometheus/client_golang/prometheus"
"github.com/prometheus/tsdb"
"github.com/prometheus/tsdb/fileutil"
"github.com/prometheus/tsdb/wal"
)
2019-01-18 12:48:16 +00:00
const (
readPeriod = 10 * time . Millisecond
checkpointPeriod = 5 * time . Second
segmentCheckPeriod = 100 * time . Millisecond
)
2018-09-07 21:26:04 +00:00
var (
watcherSamplesRecordsRead = prometheus . NewCounterVec (
prometheus . CounterOpts {
Namespace : "prometheus" ,
Subsystem : "wal_watcher" ,
Name : "samples_records_read_total" ,
Help : "Number of samples records read by the WAL watcher from the WAL." ,
} ,
[ ] string { queue } ,
)
watcherSeriesRecordsRead = prometheus . NewCounterVec (
prometheus . CounterOpts {
Namespace : "prometheus" ,
Subsystem : "wal_watcher" ,
Name : "series_records_read_total" ,
Help : "Number of series records read by the WAL watcher from the WAL." ,
} ,
[ ] string { queue } ,
)
watcherTombstoneRecordsRead = prometheus . NewCounterVec (
prometheus . CounterOpts {
Namespace : "prometheus" ,
Subsystem : "wal_watcher" ,
Name : "tombstone_records_read_total" ,
Help : "Number of tombstone records read by the WAL watcher from the WAL." ,
} ,
[ ] string { queue } ,
)
watcherInvalidRecordsRead = prometheus . NewCounterVec (
prometheus . CounterOpts {
Namespace : "prometheus" ,
Subsystem : "wal_watcher" ,
Name : "invalid_records_read_total" ,
Help : "Number of invalid records read by the WAL watcher from the WAL." ,
} ,
[ ] string { queue } ,
)
watcherUnknownTypeRecordsRead = prometheus . NewCounterVec (
prometheus . CounterOpts {
Namespace : "prometheus" ,
Subsystem : "wal_watcher" ,
Name : "unknown_records_read_total" ,
Help : "Number of records read by the WAL watcher from the WAL of an unknown record type." ,
} ,
[ ] string { queue } ,
)
watcherRecordDecodeFails = prometheus . NewCounterVec (
prometheus . CounterOpts {
Namespace : "prometheus" ,
Subsystem : "wal_watcher" ,
Name : "record_decode_failures_total" ,
Help : "Number of records read by the WAL watcher that resulted in an error when decoding." ,
} ,
[ ] string { queue } ,
)
watcherSamplesSentPreTailing = prometheus . NewCounterVec (
prometheus . CounterOpts {
Namespace : "prometheus" ,
Subsystem : "wal_watcher" ,
Name : "samples_sent_pre_tailing_total" ,
Help : "Number of sample records read by the WAL watcher and sent to remote write during replay of existing WAL." ,
} ,
[ ] string { queue } ,
)
watcherCurrentSegment = prometheus . NewGaugeVec (
prometheus . GaugeOpts {
Namespace : "prometheus" ,
Subsystem : "wal_watcher" ,
Name : "current_segment" ,
Help : "Current segment the WAL watcher is reading records from." ,
} ,
[ ] string { queue } ,
)
)
func init ( ) {
prometheus . MustRegister ( watcherSamplesRecordsRead )
prometheus . MustRegister ( watcherSeriesRecordsRead )
prometheus . MustRegister ( watcherTombstoneRecordsRead )
prometheus . MustRegister ( watcherInvalidRecordsRead )
prometheus . MustRegister ( watcherUnknownTypeRecordsRead )
prometheus . MustRegister ( watcherRecordDecodeFails )
prometheus . MustRegister ( watcherSamplesSentPreTailing )
prometheus . MustRegister ( watcherCurrentSegment )
}
type writeTo interface {
Append ( [ ] tsdb . RefSample ) bool
StoreSeries ( [ ] tsdb . RefSeries , int )
SeriesReset ( int )
}
// WALWatcher watches the TSDB WAL for a given WriteTo.
type WALWatcher struct {
2019-01-18 12:48:16 +00:00
name string
writer writeTo
logger log . Logger
walDir string
2018-09-07 21:26:04 +00:00
currentSegment int
lastCheckpoint string
startTime int64
samplesReadMetric prometheus . Counter
seriesReadMetric prometheus . Counter
tombstonesReadMetric prometheus . Counter
invalidReadMetric prometheus . Counter
unknownReadMetric prometheus . Counter
recordDecodeFailsMetric prometheus . Counter
samplesSentPreTailing prometheus . Counter
currentSegmentMetric prometheus . Gauge
2019-01-18 12:48:16 +00:00
quit chan struct { }
2018-09-07 21:26:04 +00:00
}
// NewWALWatcher creates a new WAL watcher for a given WriteTo.
2019-01-18 12:48:16 +00:00
func NewWALWatcher ( logger log . Logger , name string , writer writeTo , walDir string , startTime int64 ) * WALWatcher {
2018-09-07 21:26:04 +00:00
if logger == nil {
logger = log . NewNopLogger ( )
}
w := & WALWatcher {
logger : logger ,
writer : writer ,
walDir : path . Join ( walDir , "wal" ) ,
startTime : startTime ,
2019-01-18 20:31:36 +00:00
name : name ,
2018-09-07 21:26:04 +00:00
quit : make ( chan struct { } ) ,
}
2019-01-18 20:31:36 +00:00
w . samplesReadMetric = watcherSamplesRecordsRead . WithLabelValues ( w . name )
w . seriesReadMetric = watcherSeriesRecordsRead . WithLabelValues ( w . name )
w . tombstonesReadMetric = watcherTombstoneRecordsRead . WithLabelValues ( w . name )
w . unknownReadMetric = watcherUnknownTypeRecordsRead . WithLabelValues ( w . name )
w . invalidReadMetric = watcherInvalidRecordsRead . WithLabelValues ( w . name )
w . recordDecodeFailsMetric = watcherRecordDecodeFails . WithLabelValues ( w . name )
w . samplesSentPreTailing = watcherSamplesSentPreTailing . WithLabelValues ( w . name )
w . currentSegmentMetric = watcherCurrentSegment . WithLabelValues ( w . name )
2018-09-07 21:26:04 +00:00
return w
}
func ( w * WALWatcher ) Start ( ) {
2019-01-18 12:48:16 +00:00
level . Info ( w . logger ) . Log ( "msg" , "starting WAL watcher" , "queue" , w . name )
2019-02-13 14:47:35 +00:00
go w . loop ( )
2018-09-07 21:26:04 +00:00
}
func ( w * WALWatcher ) Stop ( ) {
2019-01-18 12:48:16 +00:00
level . Info ( w . logger ) . Log ( "msg" , "stopping WAL watcher" , "queue" , w . name )
2018-09-07 21:26:04 +00:00
close ( w . quit )
}
2019-02-13 14:47:35 +00:00
func ( w * WALWatcher ) loop ( ) {
// We may encourter failures processing the WAL; we should wait and retry.
2019-01-18 12:48:16 +00:00
for {
2019-02-13 14:47:35 +00:00
if err := w . run ( ) ; err != nil {
level . Error ( w . logger ) . Log ( "msg" , "error tailing WAL" , "err" , err )
2018-09-07 21:26:04 +00:00
}
2019-02-13 14:47:35 +00:00
select {
case <- w . quit :
return
case <- time . After ( 5 * time . Second ) :
}
2018-09-07 21:26:04 +00:00
}
2019-02-13 14:47:35 +00:00
}
2018-09-07 21:26:04 +00:00
2019-02-13 14:47:35 +00:00
func ( w * WALWatcher ) run ( ) error {
nw , err := wal . New ( nil , nil , w . walDir )
2019-01-18 12:48:16 +00:00
if err != nil {
2019-02-13 14:47:35 +00:00
return errors . Wrap ( err , "wal.New" )
2018-09-07 21:26:04 +00:00
}
2019-01-18 20:31:36 +00:00
// Backfill from the checkpoint first if it exists.
2019-02-13 14:47:35 +00:00
var nextIndex int
w . lastCheckpoint , nextIndex , err = tsdb . LastCheckpoint ( w . walDir )
2019-01-18 20:31:36 +00:00
if err != nil && err != tsdb . ErrNotFound {
2019-02-13 14:47:35 +00:00
return err
2019-01-18 20:31:36 +00:00
}
2019-02-13 14:47:35 +00:00
level . Debug ( w . logger ) . Log ( "msg" , "reading checkpoint" , "dir" , w . lastCheckpoint , "startFrom" , nextIndex )
2019-01-18 20:31:36 +00:00
if err == nil {
2019-02-13 14:47:35 +00:00
if err = w . readCheckpoint ( w . lastCheckpoint ) ; err != nil {
return err
2019-01-18 20:31:36 +00:00
}
}
2019-02-13 14:47:35 +00:00
w . currentSegment , err = w . findSegmentForIndex ( nextIndex )
if err != nil {
return err
}
2019-01-18 12:48:16 +00:00
2019-02-13 14:47:35 +00:00
level . Debug ( w . logger ) . Log ( "msg" , "starting from" , "currentSegment" , w . currentSegment )
2019-02-12 14:12:37 +00:00
2019-02-13 14:47:35 +00:00
for {
2019-02-12 14:12:37 +00:00
w . currentSegmentMetric . Set ( float64 ( w . currentSegment ) )
2019-02-13 14:47:35 +00:00
level . Info ( w . logger ) . Log ( "msg" , "process segment" , "segment" , w . currentSegment )
2019-01-18 12:48:16 +00:00
// On start, after reading the existing WAL for series records, we have a pointer to what is the latest segment.
// On subsequent calls to this function, currentSegment will have been incremented and we should open that segment.
2019-02-13 14:47:35 +00:00
if err := w . watch ( nw , w . currentSegment , true ) ; err != nil {
2019-01-18 12:48:16 +00:00
level . Error ( w . logger ) . Log ( "msg" , "runWatcher is ending" , "err" , err )
2019-02-13 14:47:35 +00:00
return err
2019-01-18 12:48:16 +00:00
}
w . currentSegment ++
}
2018-09-07 21:26:04 +00:00
}
2019-02-13 14:47:35 +00:00
func ( w * WALWatcher ) findSegmentForIndex ( index int ) ( int , error ) {
files , err := fileutil . ReadDir ( w . walDir )
if err != nil {
return - 1 , err
}
var refs [ ] int
var last int
for _ , fn := range files {
k , err := strconv . Atoi ( fn )
if err != nil {
continue
}
if len ( refs ) > 0 && k > last + 1 {
return - 1 , errors . New ( "segments are not sequential" )
}
refs = append ( refs , k )
last = k
}
sort . Sort ( sort . IntSlice ( refs ) )
for _ , r := range refs {
if r >= index {
return r , nil
}
}
return - 1 , errors . New ( "failed to find segment for index" )
}
// Use tail true to indicate thatreader is currently on a segment that is
2019-01-18 20:31:36 +00:00
// actively being written to. If false, assume it's a full segment and we're
// replaying it on start to cache the series records.
2019-02-12 14:12:37 +00:00
func ( w * WALWatcher ) watch ( wl * wal . WAL , segmentNum int , tail bool ) error {
segment , err := wal . OpenReadSegment ( wal . SegmentName ( w . walDir , segmentNum ) )
if err != nil {
return err
}
defer segment . Close ( )
reader := wal . NewLiveReader ( segment )
2019-01-18 12:48:16 +00:00
readTicker := time . NewTicker ( readPeriod )
2018-09-07 21:26:04 +00:00
defer readTicker . Stop ( )
2019-01-18 12:48:16 +00:00
checkpointTicker := time . NewTicker ( checkpointPeriod )
2018-09-07 21:26:04 +00:00
defer checkpointTicker . Stop ( )
2019-01-18 12:48:16 +00:00
segmentTicker := time . NewTicker ( segmentCheckPeriod )
defer segmentTicker . Stop ( )
2019-02-12 14:12:37 +00:00
2019-01-18 20:31:36 +00:00
// If we're replaying the segment we need to know the size of the file to know
// when to return from watch and move on to the next segment.
size := int64 ( math . MaxInt64 )
if ! tail {
segmentTicker . Stop ( )
checkpointTicker . Stop ( )
var err error
size , err = getSegmentSize ( w . walDir , w . currentSegment )
if err != nil {
level . Error ( w . logger ) . Log ( "msg" , "error getting segment size" , "segment" , w . currentSegment )
return errors . Wrap ( err , "get segment size" )
}
}
2018-09-07 21:26:04 +00:00
for {
select {
case <- w . quit :
level . Info ( w . logger ) . Log ( "msg" , "quitting WAL watcher watch loop" )
return errors . New ( "quit channel" )
2019-01-18 12:48:16 +00:00
2018-09-07 21:26:04 +00:00
case <- checkpointTicker . C :
2019-01-18 12:48:16 +00:00
// Periodically check if there is a new checkpoint.
// As this is considered an optimisation, we ignore errors during
// checkpoint processing.
2018-09-07 21:26:04 +00:00
dir , _ , err := tsdb . LastCheckpoint ( w . walDir )
if err != nil && err != tsdb . ErrNotFound {
2019-01-18 12:48:16 +00:00
level . Error ( w . logger ) . Log ( "msg" , "error getting last checkpoint" , "err" , err )
2018-09-07 21:26:04 +00:00
continue
}
2019-01-18 12:48:16 +00:00
if dir == w . lastCheckpoint {
continue
}
level . Info ( w . logger ) . Log ( "msg" , "new checkpoint detected" , "last" , w . lastCheckpoint , "new" , dir )
d , err := checkpointNum ( dir )
2018-09-07 21:26:04 +00:00
if err != nil {
2019-01-18 12:48:16 +00:00
level . Error ( w . logger ) . Log ( "msg" , "error parsing checkpoint" , "err" , err )
2018-09-07 21:26:04 +00:00
continue
}
2019-01-18 12:48:16 +00:00
if d >= w . currentSegment {
level . Info ( w . logger ) . Log ( "msg" , "current segment is behind the checkpoint, skipping reading of checkpoint" , "current" , fmt . Sprintf ( "%08d" , w . currentSegment ) , "checkpoint" , dir )
continue
2018-09-07 21:26:04 +00:00
}
2019-01-18 12:48:16 +00:00
w . lastCheckpoint = dir
// This potentially takes a long time, should we run it in another go routine?
err = w . readCheckpoint ( w . lastCheckpoint )
if err != nil {
level . Error ( w . logger ) . Log ( "err" , err )
}
2019-01-18 20:31:36 +00:00
// Clear series with a checkpoint or segment index # lower than the checkpoint we just read.
w . writer . SeriesReset ( d )
2019-01-18 12:48:16 +00:00
2018-09-07 21:26:04 +00:00
case <- segmentTicker . C :
_ , last , err := wl . Segments ( )
if err != nil {
2019-01-18 12:48:16 +00:00
return errors . Wrap ( err , "segments" )
}
2019-01-18 20:31:36 +00:00
// Check if new segments exists.
2019-01-18 12:48:16 +00:00
if last <= w . currentSegment {
2018-09-07 21:26:04 +00:00
continue
}
2019-01-18 12:48:16 +00:00
if err := w . readSegment ( reader ) ; err != nil {
// Ignore errors reading to end of segment, as we're going to move to
// next segment now.
level . Error ( w . logger ) . Log ( "msg" , "error reading to end of segment" , "err" , err )
2018-09-07 21:26:04 +00:00
}
2019-01-18 12:48:16 +00:00
level . Info ( w . logger ) . Log ( "msg" , "a new segment exists, we should start reading it" , "current" , fmt . Sprintf ( "%08d" , w . currentSegment ) , "new" , fmt . Sprintf ( "%08d" , last ) )
return nil
2018-09-07 21:26:04 +00:00
case <- readTicker . C :
2019-01-18 20:31:36 +00:00
if err := w . readSegment ( reader ) ; err != nil && err != io . EOF {
2019-01-18 12:48:16 +00:00
level . Error ( w . logger ) . Log ( "err" , err )
return err
}
2019-01-18 20:31:36 +00:00
if reader . TotalRead ( ) >= size && ! tail {
level . Info ( w . logger ) . Log ( "msg" , "done replaying segment" , "segment" , w . currentSegment , "size" , size , "read" , reader . TotalRead ( ) )
return nil
}
2018-09-07 21:26:04 +00:00
}
}
}
2019-01-18 12:48:16 +00:00
func ( w * WALWatcher ) readSegment ( r * wal . LiveReader ) error {
for r . Next ( ) && ! isClosed ( w . quit ) {
err := w . decodeRecord ( r . Record ( ) )
// Intentionally skip over record decode errors.
if err != nil {
level . Error ( w . logger ) . Log ( "err" , err )
2018-09-07 21:26:04 +00:00
}
}
2019-01-18 12:48:16 +00:00
return r . Err ( )
}
2018-09-07 21:26:04 +00:00
2019-01-18 12:48:16 +00:00
func ( w * WALWatcher ) decodeRecord ( rec [ ] byte ) error {
var (
dec tsdb . RecordDecoder
series [ ] tsdb . RefSeries
samples [ ] tsdb . RefSample
)
switch dec . Type ( rec ) {
case tsdb . RecordSeries :
series , err := dec . Series ( rec , series [ : 0 ] )
if err != nil {
w . recordDecodeFailsMetric . Inc ( )
return err
}
w . seriesReadMetric . Add ( float64 ( len ( series ) ) )
w . writer . StoreSeries ( series , w . currentSegment )
case tsdb . RecordSamples :
samples , err := dec . Samples ( rec , samples [ : 0 ] )
if err != nil {
w . recordDecodeFailsMetric . Inc ( )
return err
}
2019-01-18 20:31:36 +00:00
var send [ ] tsdb . RefSample
for _ , s := range samples {
if s . T > w . startTime {
send = append ( send , s )
}
}
if len ( send ) > 0 {
// We don't want to count samples read prior to the starting timestamp
// so that we can compare samples in vs samples read and succeeded samples.
w . samplesReadMetric . Add ( float64 ( len ( samples ) ) )
// Blocks until the sample is sent to all remote write endpoints or closed (because enqueue blocks).
w . writer . Append ( send )
}
2019-01-18 12:48:16 +00:00
case tsdb . RecordTombstones :
w . tombstonesReadMetric . Add ( float64 ( len ( samples ) ) )
case tsdb . RecordInvalid :
w . invalidReadMetric . Add ( float64 ( len ( samples ) ) )
return errors . New ( "invalid record" )
default :
w . recordDecodeFailsMetric . Inc ( )
return errors . New ( "unknown TSDB record type" )
2018-09-07 21:26:04 +00:00
}
2019-01-18 12:48:16 +00:00
return nil
}
2018-09-07 21:26:04 +00:00
2019-01-18 12:48:16 +00:00
// Read all the series records from a Checkpoint directory.
func ( w * WALWatcher ) readCheckpoint ( checkpointDir string ) error {
2019-01-18 20:31:36 +00:00
level . Info ( w . logger ) . Log ( "msg" , "reading checkpoint" , "dir" , checkpointDir )
2019-01-18 12:48:16 +00:00
sr , err := wal . NewSegmentsReader ( checkpointDir )
2018-09-07 21:26:04 +00:00
if err != nil {
2019-01-18 12:48:16 +00:00
return errors . Wrap ( err , "open checkpoint" )
2018-09-07 21:26:04 +00:00
}
2019-01-18 12:48:16 +00:00
defer sr . Close ( )
2018-09-07 21:26:04 +00:00
2019-01-18 12:48:16 +00:00
size , err := getCheckpointSize ( checkpointDir )
2018-09-07 21:26:04 +00:00
if err != nil {
2019-01-18 12:48:16 +00:00
level . Error ( w . logger ) . Log ( "msg" , "error getting checkpoint size" , "checkpoint" , checkpointDir )
return errors . Wrap ( err , "get checkpoint size" )
2018-09-07 21:26:04 +00:00
}
2019-01-18 20:31:36 +00:00
// w.readSeriesRecords(wal.NewLiveReader(sr), i, size)
r := wal . NewLiveReader ( sr )
2019-02-13 14:47:35 +00:00
if err := w . readSegment ( r ) ; err != nil {
return errors . Wrap ( err , "readSegment" )
}
2019-01-18 20:31:36 +00:00
if r . TotalRead ( ) != size {
level . Warn ( w . logger ) . Log ( "msg" , "may not have read all data from checkpoint" )
}
2019-01-18 12:48:16 +00:00
level . Debug ( w . logger ) . Log ( "msg" , "read series references from checkpoint" , "checkpoint" , checkpointDir )
2019-02-13 14:47:35 +00:00
2019-01-18 12:48:16 +00:00
return nil
2018-09-07 21:26:04 +00:00
}
2019-01-18 12:48:16 +00:00
func checkpointNum ( dir string ) ( int , error ) {
2018-09-07 21:26:04 +00:00
// Checkpoint dir names are in the format checkpoint.000001
chunks := strings . Split ( dir , "." )
if len ( chunks ) != 2 {
2019-01-18 12:48:16 +00:00
return 0 , errors . Errorf ( "invalid checkpoint dir string: %s" , dir )
2018-09-07 21:26:04 +00:00
}
2019-01-18 12:48:16 +00:00
result , err := strconv . Atoi ( chunks [ 1 ] )
if err != nil {
return 0 , errors . Errorf ( "invalid checkpoint dir string: %s" , dir )
}
return result , nil
2018-09-07 21:26:04 +00:00
}
func getCheckpointSize ( dir string ) ( int64 , error ) {
i := int64 ( 0 )
segs , err := fileutil . ReadDir ( dir )
if err != nil {
return 0 , err
}
for _ , fn := range segs {
num , err := strconv . Atoi ( fn )
if err != nil {
return i , err
}
sz , err := getSegmentSize ( dir , num )
if err != nil {
return i , err
}
i += sz
}
return i , nil
}
// Get size of segment.
func getSegmentSize ( dir string , index int ) ( int64 , error ) {
i := int64 ( - 1 )
fi , err := os . Stat ( wal . SegmentName ( dir , index ) )
if err == nil {
i = fi . Size ( )
}
return i , err
}
func isClosed ( c chan struct { } ) bool {
select {
case <- c :
return true
default :
return false
}
}