github.com/thanos-io/thanos@v0.32.5/pkg/compact/downsample/downsample.go (about)

     1  // Copyright (c) The Thanos Authors.
     2  // Licensed under the Apache License 2.0.
     3  
     4  package downsample
     5  
     6  import (
     7  	"context"
     8  	"fmt"
     9  	"math"
    10  	"math/rand"
    11  	"os"
    12  	"path/filepath"
    13  	"sync"
    14  	"time"
    15  
    16  	"github.com/go-kit/log"
    17  	"github.com/go-kit/log/level"
    18  	"github.com/oklog/ulid"
    19  	"github.com/pkg/errors"
    20  	"github.com/prometheus/prometheus/model/histogram"
    21  	"github.com/prometheus/prometheus/model/labels"
    22  	"github.com/prometheus/prometheus/model/value"
    23  	"github.com/prometheus/prometheus/tsdb"
    24  	"github.com/prometheus/prometheus/tsdb/chunkenc"
    25  	"github.com/prometheus/prometheus/tsdb/chunks"
    26  	"github.com/prometheus/prometheus/tsdb/index"
    27  	"github.com/prometheus/prometheus/tsdb/tsdbutil"
    28  	"golang.org/x/sync/errgroup"
    29  
    30  	"github.com/thanos-io/objstore"
    31  
    32  	"github.com/thanos-io/thanos/pkg/block"
    33  	"github.com/thanos-io/thanos/pkg/block/metadata"
    34  	"github.com/thanos-io/thanos/pkg/errutil"
    35  	"github.com/thanos-io/thanos/pkg/runutil"
    36  )
    37  
    38  // Standard downsampling resolution levels in Thanos.
    39  const (
    40  	ResLevel0 = int64(0)              // Raw data.
    41  	ResLevel1 = int64(5 * 60 * 1000)  // 5 minutes in milliseconds.
    42  	ResLevel2 = int64(60 * 60 * 1000) // 1 hour in milliseconds.
    43  )
    44  
    45  // Downsampling ranges i.e. minimum block size after which we start to downsample blocks (in seconds).
    46  const (
    47  	ResLevel1DownsampleRange = 40 * 60 * 60 * 1000      // 40 hours.
    48  	ResLevel2DownsampleRange = 10 * 24 * 60 * 60 * 1000 // 10 days.
    49  )
    50  
    51  // Downsample downsamples the given block. It writes a new block into dir and returns its ID.
    52  func Downsample(
    53  	logger log.Logger,
    54  	origMeta *metadata.Meta,
    55  	b tsdb.BlockReader,
    56  	dir string,
    57  	resolution int64,
    58  ) (id ulid.ULID, err error) {
    59  	if origMeta.Thanos.Downsample.Resolution >= resolution {
    60  		return id, errors.New("target resolution not lower than existing one")
    61  	}
    62  
    63  	indexr, err := b.Index()
    64  	if err != nil {
    65  		return id, errors.Wrap(err, "open index reader")
    66  	}
    67  	defer runutil.CloseWithErrCapture(&err, indexr, "downsample index reader")
    68  
    69  	chunkr, err := b.Chunks()
    70  	if err != nil {
    71  		return id, errors.Wrap(err, "open chunk reader")
    72  	}
    73  	defer runutil.CloseWithErrCapture(&err, chunkr, "downsample chunk reader")
    74  
    75  	// Generate new block id.
    76  	uid := ulid.MustNew(ulid.Now(), rand.New(rand.NewSource(time.Now().UnixNano())))
    77  
    78  	// Create block directory to populate with chunks, meta and index files into.
    79  	blockDir := filepath.Join(dir, uid.String())
    80  	if err := os.MkdirAll(blockDir, 0750); err != nil {
    81  		return id, errors.Wrap(err, "mkdir block dir")
    82  	}
    83  
    84  	// Remove blockDir in case of errors.
    85  	defer func() {
    86  		if err != nil {
    87  			var merr errutil.MultiError
    88  			merr.Add(err)
    89  			merr.Add(os.RemoveAll(blockDir))
    90  			err = merr.Err()
    91  		}
    92  	}()
    93  
    94  	// Copy original meta to the new one. Update downsampling resolution and ULID for a new block.
    95  	newMeta := *origMeta
    96  	newMeta.Thanos.Downsample.Resolution = resolution
    97  	newMeta.ULID = uid
    98  
    99  	// Writes downsampled chunks right into the files, avoiding excess memory allocation.
   100  	// Flushes index and meta data after aggregations.
   101  	streamedBlockWriter, err := NewStreamedBlockWriter(blockDir, indexr, logger, newMeta)
   102  	if err != nil {
   103  		return id, errors.Wrap(err, "get streamed block writer")
   104  	}
   105  	defer runutil.CloseWithErrCapture(&err, streamedBlockWriter, "close stream block writer")
   106  
   107  	postings, err := indexr.Postings(index.AllPostingsKey())
   108  	if err != nil {
   109  		return id, errors.Wrap(err, "get all postings list")
   110  	}
   111  
   112  	var (
   113  		aggrChunks []*AggrChunk
   114  		all        []sample
   115  		chks       []chunks.Meta
   116  		builder    labels.ScratchBuilder
   117  		reuseIt    chunkenc.Iterator
   118  	)
   119  	for postings.Next() {
   120  		chks = chks[:0]
   121  		all = all[:0]
   122  		aggrChunks = aggrChunks[:0]
   123  
   124  		// Get series labels and chunks. Downsampled data is sensitive to chunk boundaries
   125  		// and we need to preserve them to properly downsample previously downsampled data.
   126  		if err := indexr.Series(postings.At(), &builder, &chks); err != nil {
   127  			return id, errors.Wrapf(err, "get series %d", postings.At())
   128  		}
   129  		lset := builder.Labels()
   130  
   131  		for i, c := range chks[1:] {
   132  			if chks[i].MaxTime >= c.MinTime {
   133  				return id, errors.Errorf("found overlapping chunks within series %d. Chunks expected to be ordered by min time and non-overlapping, got: %v", postings.At(), chks)
   134  			}
   135  		}
   136  
   137  		// While #183 exists, we sanitize the chunks we retrieved from the block
   138  		// before retrieving their samples.
   139  		for i, c := range chks {
   140  			chk, err := chunkr.Chunk(c)
   141  			if err != nil {
   142  				return id, errors.Wrapf(err, "get chunk %d, series %d", c.Ref, postings.At())
   143  			}
   144  			chks[i].Chunk = chk
   145  		}
   146  
   147  		// Raw and already downsampled data need different processing.
   148  		if origMeta.Thanos.Downsample.Resolution == 0 {
   149  			for _, c := range chks {
   150  				// TODO(bwplotka): We can optimze this further by using in WriteSeries iterators of each chunk instead of
   151  				// samples. Also ensure 120 sample limit, otherwise we have gigantic chunks.
   152  				// https://github.com/thanos-io/thanos/issues/2542.
   153  				if err := expandChunkIterator(c.Chunk.Iterator(reuseIt), &all); err != nil {
   154  					return id, errors.Wrapf(err, "expand chunk %d, series %d", c.Ref, postings.At())
   155  				}
   156  			}
   157  			if err := streamedBlockWriter.WriteSeries(lset, DownsampleRaw(all, resolution)); err != nil {
   158  				return id, errors.Wrapf(err, "downsample raw data, series: %d", postings.At())
   159  			}
   160  		} else {
   161  			// Downsample a block that contains aggregated chunks already.
   162  			for _, c := range chks {
   163  				ac, ok := c.Chunk.(*AggrChunk)
   164  				if !ok {
   165  					if c.Chunk.NumSamples() == 0 {
   166  						// Downsampled block can erroneously contain empty XOR chunks, skip those
   167  						// https://github.com/thanos-io/thanos/issues/5272
   168  						level.Warn(logger).Log("msg", fmt.Sprintf("expected downsampled chunk (*downsample.AggrChunk) got an empty %T instead for series: %d", c.Chunk, postings.At()))
   169  						continue
   170  					} else {
   171  						if err := expandChunkIterator(c.Chunk.Iterator(reuseIt), &all); err != nil {
   172  							return id, errors.Wrapf(err, "expand chunk %d, series %d", c.Ref, postings.At())
   173  						}
   174  						aggrDataChunks := DownsampleRaw(all, ResLevel1)
   175  						for _, cn := range aggrDataChunks {
   176  							ac, ok = cn.Chunk.(*AggrChunk)
   177  							if !ok {
   178  								return id, errors.New("Not able to convert non-empty XOR chunks to 5m downsampled aggregated chunks.")
   179  							}
   180  						}
   181  					}
   182  
   183  				}
   184  				aggrChunks = append(aggrChunks, ac)
   185  
   186  			}
   187  			downsampledChunks, err := downsampleAggr(
   188  				aggrChunks,
   189  				&all,
   190  				chks[0].MinTime,
   191  				chks[len(chks)-1].MaxTime,
   192  				origMeta.Thanos.Downsample.Resolution,
   193  				resolution,
   194  			)
   195  			if err != nil {
   196  				return id, errors.Wrapf(err, "downsample aggregate block, series: %d", postings.At())
   197  			}
   198  			if err := streamedBlockWriter.WriteSeries(lset, downsampledChunks); err != nil {
   199  				return id, errors.Wrapf(err, "write series: %d", postings.At())
   200  			}
   201  		}
   202  	}
   203  	if postings.Err() != nil {
   204  		return id, errors.Wrap(postings.Err(), "iterate series set")
   205  	}
   206  
   207  	id = uid
   208  	return
   209  }
   210  
   211  // currentWindow returns the end timestamp of the window that t falls into.
   212  func currentWindow(t, r int64) int64 {
   213  	// The next timestamp is the next number after s.t that's aligned with window.
   214  	// We subtract 1 because block ranges are [from, to) and the last sample would
   215  	// go out of bounds otherwise.
   216  	return t - (t % r) + r - 1
   217  }
   218  
   219  // rangeFullness returns the fraction of how the range [mint, maxt] covered
   220  // with count samples at the given step size.
   221  // It return value is bounded to [0, 1].
   222  func rangeFullness(mint, maxt, step int64, count int) float64 {
   223  	f := float64(count) / (float64(maxt-mint) / float64(step))
   224  	if f > 1 {
   225  		return 1
   226  	}
   227  	return f
   228  }
   229  
   230  // targetChunkCount calculates how many chunks should be produced when downsampling a series.
   231  // It consider the total time range, the number of input sample, the input and output resolution.
   232  func targetChunkCount(mint, maxt, inRes, outRes int64, count int) (x int) {
   233  	// We compute how many samples we could produce for the given time range and adjust
   234  	// it by how densely the range is actually filled given the number of input samples and their
   235  	// resolution.
   236  	maxSamples := float64((maxt - mint) / outRes)
   237  	expSamples := int(maxSamples*rangeFullness(mint, maxt, inRes, count)) + 1
   238  
   239  	// Increase the number of target chunks until each chunk will have less than
   240  	// 140 samples on average.
   241  	for x = 1; expSamples/x > 140; x++ {
   242  	}
   243  	return x
   244  }
   245  
   246  // aggregator collects cumulative stats for a stream of values.
   247  type aggregator struct {
   248  	total   int     // Total samples processed.
   249  	count   int     // Samples in current window.
   250  	sum     float64 // Value sum of current window.
   251  	min     float64 // Min of current window.
   252  	max     float64 // Max of current window.
   253  	counter float64 // Total counter state since beginning.
   254  	resets  int     // Number of counter resets since beginning.
   255  	last    float64 // Last added value.
   256  }
   257  
   258  // reset the stats to start a new aggregation window.
   259  func (a *aggregator) reset() {
   260  	a.count = 0
   261  	a.sum = 0
   262  	a.min = math.MaxFloat64
   263  	a.max = -math.MaxFloat64
   264  }
   265  
   266  func (a *aggregator) add(v float64) {
   267  	if a.total > 0 {
   268  		if v < a.last {
   269  			// Counter reset, correct the value.
   270  			a.counter += v
   271  			a.resets++
   272  		} else {
   273  			// Add delta with last value to the counter.
   274  			a.counter += v - a.last
   275  		}
   276  	} else {
   277  		// First sample sets the counter.
   278  		a.counter = v
   279  	}
   280  	a.last = v
   281  
   282  	a.sum += v
   283  	a.count++
   284  	a.total++
   285  
   286  	if v < a.min {
   287  		a.min = v
   288  	}
   289  	if v > a.max {
   290  		a.max = v
   291  	}
   292  }
   293  
   294  // aggrChunkBuilder builds chunks for multiple different aggregates.
   295  type aggrChunkBuilder struct {
   296  	mint, maxt int64
   297  	added      int
   298  
   299  	chunks [5]chunkenc.Chunk
   300  	apps   [5]chunkenc.Appender
   301  }
   302  
   303  func newAggrChunkBuilder() *aggrChunkBuilder {
   304  	b := &aggrChunkBuilder{
   305  		mint: math.MaxInt64,
   306  		maxt: math.MinInt64,
   307  	}
   308  	b.chunks[AggrCount] = chunkenc.NewXORChunk()
   309  	b.chunks[AggrSum] = chunkenc.NewXORChunk()
   310  	b.chunks[AggrMin] = chunkenc.NewXORChunk()
   311  	b.chunks[AggrMax] = chunkenc.NewXORChunk()
   312  	b.chunks[AggrCounter] = chunkenc.NewXORChunk()
   313  
   314  	for i, c := range b.chunks {
   315  		if c != nil {
   316  			b.apps[i], _ = c.Appender()
   317  		}
   318  	}
   319  	return b
   320  }
   321  
   322  func (b *aggrChunkBuilder) add(t int64, aggr *aggregator) {
   323  	if t < b.mint {
   324  		b.mint = t
   325  	}
   326  	if t > b.maxt {
   327  		b.maxt = t
   328  	}
   329  	b.apps[AggrSum].Append(t, aggr.sum)
   330  	b.apps[AggrMin].Append(t, aggr.min)
   331  	b.apps[AggrMax].Append(t, aggr.max)
   332  	b.apps[AggrCount].Append(t, float64(aggr.count))
   333  	b.apps[AggrCounter].Append(t, aggr.counter)
   334  
   335  	b.added++
   336  }
   337  
   338  func (b *aggrChunkBuilder) encode() chunks.Meta {
   339  	return chunks.Meta{
   340  		MinTime: b.mint,
   341  		MaxTime: b.maxt,
   342  		Chunk:   EncodeAggrChunk(b.chunks),
   343  	}
   344  }
   345  
   346  // DownsampleRaw create a series of aggregation chunks for the given sample data.
   347  func DownsampleRaw(data []sample, resolution int64) []chunks.Meta {
   348  	if len(data) == 0 {
   349  		return nil
   350  	}
   351  
   352  	mint, maxt := data[0].t, data[len(data)-1].t
   353  	// We assume a raw resolution of 1 minute. In practice it will often be lower
   354  	// but this is sufficient for our heuristic to produce well-sized chunks.
   355  	numChunks := targetChunkCount(mint, maxt, 1*60*1000, resolution, len(data))
   356  	return downsampleRawLoop(data, resolution, numChunks)
   357  }
   358  
   359  func downsampleRawLoop(data []sample, resolution int64, numChunks int) []chunks.Meta {
   360  	batchSize := (len(data) / numChunks) + 1
   361  	chks := make([]chunks.Meta, 0, numChunks)
   362  
   363  	for len(data) > 0 {
   364  		j := batchSize
   365  		if j > len(data) {
   366  			j = len(data)
   367  		}
   368  		curW := currentWindow(data[j-1].t, resolution)
   369  
   370  		// The batch we took might end in the middle of a downsampling window. We additionally grab
   371  		// all further samples in the window to keep our samples regular.
   372  		for ; j < len(data) && data[j].t <= curW; j++ {
   373  		}
   374  
   375  		batch := data[:j]
   376  		data = data[j:]
   377  
   378  		ab := newAggrChunkBuilder()
   379  
   380  		// Encode first raw value; see ApplyCounterResetsSeriesIterator.
   381  		ab.apps[AggrCounter].Append(batch[0].t, batch[0].v)
   382  
   383  		lastT := downsampleBatch(batch, resolution, ab.add)
   384  
   385  		// Encode last raw value; see ApplyCounterResetsSeriesIterator.
   386  		ab.apps[AggrCounter].Append(lastT, batch[len(batch)-1].v)
   387  
   388  		chks = append(chks, ab.encode())
   389  	}
   390  
   391  	return chks
   392  }
   393  
   394  // downsampleBatch aggregates the data over the given resolution and calls add each time
   395  // the end of a resolution was reached.
   396  func downsampleBatch(data []sample, resolution int64, add func(int64, *aggregator)) int64 {
   397  	var (
   398  		aggr  aggregator
   399  		nextT = int64(-1)
   400  		lastT = data[len(data)-1].t
   401  	)
   402  	// Fill up one aggregate chunk with up to m samples.
   403  	for _, s := range data {
   404  		if value.IsStaleNaN(s.v) {
   405  			continue
   406  		}
   407  		if s.t > nextT {
   408  			if nextT != -1 {
   409  				add(nextT, &aggr)
   410  			}
   411  			aggr.reset()
   412  			nextT = currentWindow(s.t, resolution)
   413  			// Limit next timestamp to not go beyond the batch. A subsequent batch
   414  			// may overlap in time range otherwise.
   415  			// We have aligned batches for raw downsamplings but subsequent downsamples
   416  			// are forced to be chunk-boundary aligned and cannot guarantee this.
   417  			if nextT > lastT {
   418  				nextT = lastT
   419  			}
   420  		}
   421  		aggr.add(s.v)
   422  	}
   423  	// Add the last sample.
   424  	add(nextT, &aggr)
   425  
   426  	return nextT
   427  }
   428  
   429  // downsampleAggr downsamples a sequence of aggregation chunks to the given resolution.
   430  func downsampleAggr(chks []*AggrChunk, buf *[]sample, mint, maxt, inRes, outRes int64) ([]chunks.Meta, error) {
   431  	var numSamples int
   432  	for _, c := range chks {
   433  		numSamples += c.NumSamples()
   434  	}
   435  	numChunks := targetChunkCount(mint, maxt, inRes, outRes, numSamples)
   436  	return downsampleAggrLoop(chks, buf, outRes, numChunks)
   437  }
   438  
   439  func downsampleAggrLoop(chks []*AggrChunk, buf *[]sample, resolution int64, numChunks int) ([]chunks.Meta, error) {
   440  	// We downsample aggregates only along chunk boundaries. This is required
   441  	// for counters to be downsampled correctly since a chunk's first and last
   442  	// counter values are the true values of the original series. We need
   443  	// to preserve them even across multiple aggregation iterations.
   444  	res := make([]chunks.Meta, 0, numChunks)
   445  	batchSize := len(chks) / numChunks
   446  
   447  	for len(chks) > 0 {
   448  		j := batchSize
   449  		if j > len(chks) {
   450  			j = len(chks)
   451  		}
   452  		part := chks[:j]
   453  		chks = chks[j:]
   454  
   455  		chk, err := downsampleAggrBatch(part, buf, resolution)
   456  		if err != nil {
   457  			return nil, err
   458  		}
   459  		res = append(res, chk)
   460  	}
   461  
   462  	return res, nil
   463  }
   464  
   465  // expandChunkIterator reads all samples from the iterator and appends them to buf.
   466  // Stale markers and out of order samples are skipped.
   467  func expandChunkIterator(it chunkenc.Iterator, buf *[]sample) error {
   468  	// For safety reasons, we check for each sample that it does not go back in time.
   469  	// If it does, we skip it.
   470  	lastT := int64(0)
   471  
   472  	for it.Next() != chunkenc.ValNone {
   473  		t, v := it.At()
   474  		if value.IsStaleNaN(v) {
   475  			continue
   476  		}
   477  		if t >= lastT {
   478  			*buf = append(*buf, sample{t, v})
   479  			lastT = t
   480  		}
   481  	}
   482  	return it.Err()
   483  }
   484  
   485  func downsampleAggrBatch(chks []*AggrChunk, buf *[]sample, resolution int64) (chk chunks.Meta, err error) {
   486  	ab := &aggrChunkBuilder{}
   487  	mint, maxt := int64(math.MaxInt64), int64(math.MinInt64)
   488  	var reuseIt chunkenc.Iterator
   489  
   490  	// do does a generic aggregation for count, sum, min, and max aggregates.
   491  	// Counters need special treatment.
   492  	do := func(at AggrType, f func(a *aggregator) float64) error {
   493  		*buf = (*buf)[:0]
   494  		// Expand all samples for the aggregate type.
   495  		for _, chk := range chks {
   496  			c, err := chk.Get(at)
   497  			if err == ErrAggrNotExist {
   498  				continue
   499  			} else if err != nil {
   500  				return err
   501  			}
   502  			if err := expandChunkIterator(c.Iterator(reuseIt), buf); err != nil {
   503  				return err
   504  			}
   505  		}
   506  		if len(*buf) == 0 {
   507  			return nil
   508  		}
   509  		ab.chunks[at] = chunkenc.NewXORChunk()
   510  		ab.apps[at], _ = ab.chunks[at].Appender()
   511  
   512  		downsampleBatch(*buf, resolution, func(t int64, a *aggregator) {
   513  			if t < mint {
   514  				mint = t
   515  			} else if t > maxt {
   516  				maxt = t
   517  			}
   518  			ab.apps[at].Append(t, f(a))
   519  		})
   520  		return nil
   521  	}
   522  	if err := do(AggrCount, func(a *aggregator) float64 {
   523  		// To get correct count of elements from already downsampled count chunk
   524  		// we have to sum those values.
   525  		return a.sum
   526  	}); err != nil {
   527  		return chk, err
   528  	}
   529  	if err = do(AggrSum, func(a *aggregator) float64 {
   530  		return a.sum
   531  	}); err != nil {
   532  		return chk, err
   533  	}
   534  	if err := do(AggrMin, func(a *aggregator) float64 {
   535  		return a.min
   536  	}); err != nil {
   537  		return chk, err
   538  	}
   539  	if err := do(AggrMax, func(a *aggregator) float64 {
   540  		return a.max
   541  	}); err != nil {
   542  		return chk, err
   543  	}
   544  
   545  	// Handle counters by applying resets directly.
   546  	acs := make([]chunkenc.Iterator, 0, len(chks))
   547  	for _, achk := range chks {
   548  		c, err := achk.Get(AggrCounter)
   549  		if err == ErrAggrNotExist {
   550  			continue
   551  		} else if err != nil {
   552  			return chk, err
   553  		}
   554  		acs = append(acs, c.Iterator(reuseIt))
   555  	}
   556  	*buf = (*buf)[:0]
   557  	it := NewApplyCounterResetsIterator(acs...)
   558  
   559  	if err := expandChunkIterator(it, buf); err != nil {
   560  		return chk, err
   561  	}
   562  	if len(*buf) == 0 {
   563  		ab.mint = mint
   564  		ab.maxt = maxt
   565  		return ab.encode(), nil
   566  	}
   567  	ab.chunks[AggrCounter] = chunkenc.NewXORChunk()
   568  	ab.apps[AggrCounter], _ = ab.chunks[AggrCounter].Appender()
   569  
   570  	// Retain first raw value; see ApplyCounterResetsSeriesIterator.
   571  	ab.apps[AggrCounter].Append((*buf)[0].t, (*buf)[0].v)
   572  
   573  	lastT := downsampleBatch(*buf, resolution, func(t int64, a *aggregator) {
   574  		if t < mint {
   575  			mint = t
   576  		} else if t > maxt {
   577  			maxt = t
   578  		}
   579  		ab.apps[AggrCounter].Append(t, a.counter)
   580  	})
   581  
   582  	// Retain last raw value; see ApplyCounterResetsSeriesIterator.
   583  	ab.apps[AggrCounter].Append(lastT, it.lastV)
   584  
   585  	ab.mint = mint
   586  	ab.maxt = maxt
   587  	return ab.encode(), nil
   588  }
   589  
   590  type sample struct {
   591  	t int64
   592  	v float64
   593  }
   594  
   595  // ApplyCounterResetsSeriesIterator generates monotonically increasing values by iterating
   596  // over an ordered sequence of chunks, which should be raw or aggregated chunks
   597  // of counter values. The generated samples can be used by PromQL functions
   598  // like 'rate' that calculate differences between counter values. Stale Markers
   599  // are removed as well.
   600  //
   601  // Counter aggregation chunks must have the first and last values from their
   602  // original raw series: the first raw value should be the first value encoded
   603  // in the chunk, and the last raw value is encoded by the duplication of the
   604  // previous sample's timestamp. As iteration occurs between chunks, the
   605  // comparison between the last raw value of the earlier chunk and the first raw
   606  // value of the later chunk ensures that counter resets between chunks are
   607  // recognized and that the correct value delta is calculated.
   608  //
   609  // It handles overlapped chunks (removes overlaps).
   610  // NOTE: It is important to deduplicate with care ensuring that you don't hit
   611  // issue https://github.com/thanos-io/thanos/issues/2401#issuecomment-621958839.
   612  // NOTE(bwplotka): This hides resets from PromQL engine. This means it will not work for PromQL resets function.
   613  type ApplyCounterResetsSeriesIterator struct {
   614  	chks        []chunkenc.Iterator
   615  	i           int     // Current chunk.
   616  	total       int     // Total number of processed samples.
   617  	lastT       int64   // Timestamp of the last sample.
   618  	lastV       float64 // Value of the last sample.
   619  	totalV      float64 // Total counter state since beginning of series.
   620  	lastValType chunkenc.ValueType
   621  }
   622  
   623  func NewApplyCounterResetsIterator(chks ...chunkenc.Iterator) *ApplyCounterResetsSeriesIterator {
   624  	return &ApplyCounterResetsSeriesIterator{chks: chks}
   625  }
   626  
   627  // TODO(rabenhorst): Native histogram support needs to be added, float type is hardcoded.
   628  func (it *ApplyCounterResetsSeriesIterator) Next() chunkenc.ValueType {
   629  	for {
   630  		if it.i >= len(it.chks) {
   631  			return chunkenc.ValNone
   632  		}
   633  		it.lastValType = it.chks[it.i].Next()
   634  		if it.lastValType == chunkenc.ValNone {
   635  			it.i++
   636  			// While iterators are ordered, they are not generally guaranteed to be
   637  			// non-overlapping. Ensure that the series does not go back in time by seeking at least
   638  			// to the next timestamp.
   639  			return it.Seek(it.lastT + 1)
   640  		}
   641  		// Counter resets do not need to be handled for non-float sample types.
   642  		if it.lastValType != chunkenc.ValFloat {
   643  			it.lastT = it.chks[it.i].AtT()
   644  			return it.lastValType
   645  		}
   646  
   647  		t, v := it.chks[it.i].At()
   648  		if math.IsNaN(v) {
   649  			return it.Next()
   650  		}
   651  		// First sample sets the initial counter state.
   652  		if it.total == 0 {
   653  			it.total++
   654  			it.lastT, it.lastV = t, v
   655  			it.totalV = v
   656  			return chunkenc.ValFloat
   657  		}
   658  		// If the timestamp increased, it is not the special last sample.
   659  		if t > it.lastT {
   660  			if v >= it.lastV {
   661  				it.totalV += v - it.lastV
   662  			} else {
   663  				it.totalV += v
   664  			}
   665  			it.lastT, it.lastV = t, v
   666  			it.total++
   667  			return chunkenc.ValFloat
   668  		}
   669  		// We hit a sample that indicates what the true last value was. For the
   670  		// next chunk we use it to determine whether there was a counter reset between them.
   671  		if t == it.lastT {
   672  			it.lastV = v
   673  		}
   674  		// Otherwise the series went back in time and we just keep moving forward.
   675  	}
   676  }
   677  
   678  func (it *ApplyCounterResetsSeriesIterator) At() (t int64, v float64) {
   679  	return it.lastT, it.totalV
   680  }
   681  
   682  func (it *ApplyCounterResetsSeriesIterator) AtHistogram() (int64, *histogram.Histogram) {
   683  	return it.chks[it.i].AtHistogram()
   684  }
   685  
   686  func (it *ApplyCounterResetsSeriesIterator) AtFloatHistogram() (int64, *histogram.FloatHistogram) {
   687  	return it.chks[it.i].AtFloatHistogram()
   688  }
   689  
   690  func (it *ApplyCounterResetsSeriesIterator) AtT() int64 {
   691  	return it.lastT
   692  }
   693  
   694  func (it *ApplyCounterResetsSeriesIterator) Seek(x int64) chunkenc.ValueType {
   695  	// Don't use underlying Seek, but iterate over next to not miss counter resets.
   696  	for {
   697  		if t := it.AtT(); t >= x {
   698  			return it.lastValType
   699  		}
   700  
   701  		if it.Next() == chunkenc.ValNone {
   702  			return chunkenc.ValNone
   703  		}
   704  	}
   705  }
   706  
   707  func (it *ApplyCounterResetsSeriesIterator) Err() error {
   708  	if it.i >= len(it.chks) {
   709  		return nil
   710  	}
   711  	return it.chks[it.i].Err()
   712  }
   713  
   714  // AverageChunkIterator emits an artificial series of average samples based in aggregate
   715  // chunks with sum and count aggregates.
   716  type AverageChunkIterator struct {
   717  	cntIt chunkenc.Iterator
   718  	sumIt chunkenc.Iterator
   719  	t     int64
   720  	v     float64
   721  	err   error
   722  }
   723  
   724  func NewAverageChunkIterator(cnt, sum chunkenc.Iterator) *AverageChunkIterator {
   725  	return &AverageChunkIterator{cntIt: cnt, sumIt: sum}
   726  }
   727  
   728  // TODO(rabenhorst): Native histogram support needs to be added, float type is hardcoded.
   729  func (it *AverageChunkIterator) Next() chunkenc.ValueType {
   730  	cok, sok := it.cntIt.Next(), it.sumIt.Next()
   731  	if cok != sok {
   732  		it.err = errors.New("sum and count iterator not aligned")
   733  		return chunkenc.ValNone
   734  	}
   735  	if cok == chunkenc.ValNone {
   736  		return chunkenc.ValNone
   737  	}
   738  
   739  	cntT, cntV := it.cntIt.At()
   740  	sumT, sumV := it.sumIt.At()
   741  	if cntT != sumT {
   742  		it.err = errors.New("sum and count timestamps not aligned")
   743  		return chunkenc.ValNone
   744  	}
   745  	it.t, it.v = cntT, sumV/cntV
   746  	return chunkenc.ValFloat
   747  }
   748  
   749  func (it *AverageChunkIterator) Seek(t int64) chunkenc.ValueType {
   750  	it.err = errors.New("seek used, but not implemented")
   751  	return chunkenc.ValNone
   752  }
   753  
   754  func (it *AverageChunkIterator) At() (int64, float64) {
   755  	return it.t, it.v
   756  }
   757  
   758  // TODO(rabenhorst): Needs to be implemented for native histogram support.
   759  func (it *AverageChunkIterator) AtHistogram() (int64, *histogram.Histogram) {
   760  	panic("not implemented")
   761  }
   762  
   763  func (it *AverageChunkIterator) AtFloatHistogram() (int64, *histogram.FloatHistogram) {
   764  	panic("not implemented")
   765  }
   766  
   767  func (it *AverageChunkIterator) AtT() int64 {
   768  	return it.t
   769  }
   770  
   771  func (it *AverageChunkIterator) Err() error {
   772  	if it.cntIt.Err() != nil {
   773  		return it.cntIt.Err()
   774  	}
   775  	if it.sumIt.Err() != nil {
   776  		return it.sumIt.Err()
   777  	}
   778  	return it.err
   779  }
   780  
   781  // SamplesFromTSDBSamples converts tsdbutil.Sample slice to samples.
   782  func SamplesFromTSDBSamples(samples []tsdbutil.Sample) []sample {
   783  	res := make([]sample, len(samples))
   784  	for i, s := range samples {
   785  		res[i] = sample{t: s.T(), v: s.F()}
   786  	}
   787  	return res
   788  }
   789  
   790  // GatherNoDownsampleMarkFilter is a block.Fetcher filter that passes all metas.
   791  // While doing it, it gathers all no-downsample-mark.json markers.
   792  type GatherNoDownsampleMarkFilter struct {
   793  	logger                log.Logger
   794  	bkt                   objstore.InstrumentedBucketReader
   795  	noDownsampleMarkedMap map[ulid.ULID]*metadata.NoDownsampleMark
   796  	concurrency           int
   797  	mtx                   sync.Mutex
   798  }
   799  
   800  // NewGatherNoDownsampleMarkFilter creates GatherNoDownsampleMarkFilter.
   801  func NewGatherNoDownsampleMarkFilter(logger log.Logger, bkt objstore.InstrumentedBucketReader) *GatherNoDownsampleMarkFilter {
   802  	return &GatherNoDownsampleMarkFilter{
   803  		logger:      logger,
   804  		bkt:         bkt,
   805  		concurrency: 1,
   806  	}
   807  }
   808  
   809  // NoDownsampleMarkedBlocks returns block ids that were marked for no downsample.
   810  func (f *GatherNoDownsampleMarkFilter) NoDownsampleMarkedBlocks() map[ulid.ULID]*metadata.NoDownsampleMark {
   811  	f.mtx.Lock()
   812  	copiedNoDownsampleMarked := make(map[ulid.ULID]*metadata.NoDownsampleMark, len(f.noDownsampleMarkedMap))
   813  	for k, v := range f.noDownsampleMarkedMap {
   814  		copiedNoDownsampleMarked[k] = v
   815  	}
   816  	f.mtx.Unlock()
   817  
   818  	return copiedNoDownsampleMarked
   819  }
   820  
   821  // TODO (@rohitkochhar): reduce code duplication here by combining
   822  // this code with that of GatherNoCompactionMarkFilter
   823  // Filter passes all metas, while gathering no downsample markers.
   824  func (f *GatherNoDownsampleMarkFilter) Filter(ctx context.Context, metas map[ulid.ULID]*metadata.Meta, synced block.GaugeVec, modified block.GaugeVec) error {
   825  	f.mtx.Lock()
   826  	f.noDownsampleMarkedMap = make(map[ulid.ULID]*metadata.NoDownsampleMark)
   827  	f.mtx.Unlock()
   828  
   829  	// Make a copy of block IDs to check, in order to avoid concurrency issues
   830  	// between the scheduler and workers.
   831  	blockIDs := make([]ulid.ULID, 0, len(metas))
   832  	for id := range metas {
   833  		blockIDs = append(blockIDs, id)
   834  	}
   835  
   836  	var (
   837  		eg errgroup.Group
   838  		ch = make(chan ulid.ULID, f.concurrency)
   839  	)
   840  
   841  	for i := 0; i < f.concurrency; i++ {
   842  		eg.Go(func() error {
   843  			var lastErr error
   844  			for id := range ch {
   845  				m := &metadata.NoDownsampleMark{}
   846  
   847  				if err := metadata.ReadMarker(ctx, f.logger, f.bkt, id.String(), m); err != nil {
   848  					if errors.Cause(err) == metadata.ErrorMarkerNotFound {
   849  						continue
   850  					}
   851  					if errors.Cause(err) == metadata.ErrorUnmarshalMarker {
   852  						level.Warn(f.logger).Log("msg", "found partial no-downsample-mark.json; if we will see it happening often for the same block, consider manually deleting no-downsample-mark.json from the object storage", "block", id, "err", err)
   853  						continue
   854  					}
   855  					// Remember the last error and continue draining the channel.
   856  					lastErr = err
   857  					continue
   858  				}
   859  
   860  				f.mtx.Lock()
   861  				f.noDownsampleMarkedMap[id] = m
   862  				f.mtx.Unlock()
   863  				synced.WithLabelValues(block.MarkedForNoDownsampleMeta).Inc()
   864  			}
   865  
   866  			return lastErr
   867  		})
   868  	}
   869  
   870  	// Workers scheduled, distribute blocks.
   871  	eg.Go(func() error {
   872  		defer close(ch)
   873  
   874  		for _, id := range blockIDs {
   875  			select {
   876  			case ch <- id:
   877  				// Nothing to do.
   878  			case <-ctx.Done():
   879  				return ctx.Err()
   880  			}
   881  		}
   882  
   883  		return nil
   884  	})
   885  
   886  	if err := eg.Wait(); err != nil {
   887  		return errors.Wrap(err, "filter blocks marked for no downsample")
   888  	}
   889  
   890  	return nil
   891  }