Add empty series sketches back to tsi1 index
This commit adds initial empty sketches back to the tsi1 index, as well as ensuring that ephemeral sketches in the index `LogFile` are updated accordingly. The commit also adds a test that verifies that the merged sketches at the store level produce the correct results under writes, deletions and re-opening of the store. This commit does not provide working sketches for post-compaction on the tsi1 index.pull/9412/head
parent
c8f30da88a
commit
544329380f
|
@ -163,6 +163,10 @@ func (h *Plus) Add(v []byte) {
|
||||||
|
|
||||||
// Count returns a cardinality estimate.
|
// Count returns a cardinality estimate.
|
||||||
func (h *Plus) Count() uint64 {
|
func (h *Plus) Count() uint64 {
|
||||||
|
if h == nil {
|
||||||
|
return 0 // Nothing to do.
|
||||||
|
}
|
||||||
|
|
||||||
if h.sparse {
|
if h.sparse {
|
||||||
h.mergeSparse()
|
h.mergeSparse()
|
||||||
return uint64(h.linearCount(h.mp, h.mp-uint32(h.sparseList.count)))
|
return uint64(h.linearCount(h.mp, h.mp-uint32(h.sparseList.count)))
|
||||||
|
|
|
@ -57,6 +57,7 @@ type Engine interface {
|
||||||
DeleteSeriesRange(itr SeriesIterator, min, max int64) error
|
DeleteSeriesRange(itr SeriesIterator, min, max int64) error
|
||||||
|
|
||||||
MeasurementsSketches() (estimator.Sketch, estimator.Sketch, error)
|
MeasurementsSketches() (estimator.Sketch, estimator.Sketch, error)
|
||||||
|
SeriesSketches() (estimator.Sketch, estimator.Sketch, error)
|
||||||
SeriesN() int64
|
SeriesN() int64
|
||||||
|
|
||||||
MeasurementExists(name []byte) (bool, error)
|
MeasurementExists(name []byte) (bool, error)
|
||||||
|
|
|
@ -532,10 +532,20 @@ func (e *Engine) SeriesN() int64 {
|
||||||
return e.index.SeriesN()
|
return e.index.SeriesN()
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// MeasurementsSketches returns sketches that describe the cardinality of the
|
||||||
|
// measurements in this shard and measurements that were in this shard, but have
|
||||||
|
// been tombstoned.
|
||||||
func (e *Engine) MeasurementsSketches() (estimator.Sketch, estimator.Sketch, error) {
|
func (e *Engine) MeasurementsSketches() (estimator.Sketch, estimator.Sketch, error) {
|
||||||
return e.index.MeasurementsSketches()
|
return e.index.MeasurementsSketches()
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// SeriesSketches returns sketches that describe the cardinality of the
|
||||||
|
// series in this shard and series that were in this shard, but have
|
||||||
|
// been tombstoned.
|
||||||
|
func (e *Engine) SeriesSketches() (estimator.Sketch, estimator.Sketch, error) {
|
||||||
|
return e.index.SeriesSketches()
|
||||||
|
}
|
||||||
|
|
||||||
// LastModified returns the time when this shard was last modified.
|
// LastModified returns the time when this shard was last modified.
|
||||||
func (e *Engine) LastModified() time.Time {
|
func (e *Engine) LastModified() time.Time {
|
||||||
walTime := e.WAL.LastWriteTime()
|
walTime := e.WAL.LastWriteTime()
|
||||||
|
|
|
@ -36,6 +36,7 @@ type Index interface {
|
||||||
|
|
||||||
MeasurementsSketches() (estimator.Sketch, estimator.Sketch, error)
|
MeasurementsSketches() (estimator.Sketch, estimator.Sketch, error)
|
||||||
SeriesN() int64
|
SeriesN() int64
|
||||||
|
SeriesSketches() (estimator.Sketch, estimator.Sketch, error)
|
||||||
|
|
||||||
HasTagKey(name, key []byte) (bool, error)
|
HasTagKey(name, key []byte) (bool, error)
|
||||||
HasTagValue(name, key, value []byte) (bool, error)
|
HasTagValue(name, key, value []byte) (bool, error)
|
||||||
|
|
|
@ -408,6 +408,19 @@ func (fs *FileSet) MeasurementsSketches() (estimator.Sketch, estimator.Sketch, e
|
||||||
return sketch, tsketch, nil
|
return sketch, tsketch, nil
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// SeriesSketches returns the merged measurement sketches for the FileSet.
|
||||||
|
func (fs *FileSet) SeriesSketches() (estimator.Sketch, estimator.Sketch, error) {
|
||||||
|
sketch, tsketch := hll.NewDefaultPlus(), hll.NewDefaultPlus()
|
||||||
|
|
||||||
|
// Iterate over all the files and merge the sketches into the result.
|
||||||
|
for _, f := range fs.files {
|
||||||
|
if err := f.MergeSeriesSketches(sketch, tsketch); err != nil {
|
||||||
|
return nil, nil, err
|
||||||
|
}
|
||||||
|
}
|
||||||
|
return sketch, tsketch, nil
|
||||||
|
}
|
||||||
|
|
||||||
// File represents a log or index file.
|
// File represents a log or index file.
|
||||||
type File interface {
|
type File interface {
|
||||||
Close() error
|
Close() error
|
||||||
|
@ -433,6 +446,7 @@ type File interface {
|
||||||
|
|
||||||
// Sketches for cardinality estimation
|
// Sketches for cardinality estimation
|
||||||
MergeMeasurementsSketches(s, t estimator.Sketch) error
|
MergeMeasurementsSketches(s, t estimator.Sketch) error
|
||||||
|
MergeSeriesSketches(s, t estimator.Sketch) error
|
||||||
|
|
||||||
// Bitmap series existance.
|
// Bitmap series existance.
|
||||||
SeriesIDSet() (*tsdb.SeriesIDSet, error)
|
SeriesIDSet() (*tsdb.SeriesIDSet, error)
|
||||||
|
|
|
@ -594,7 +594,7 @@ func (i *Index) DropMeasurementIfSeriesNotExist(name []byte) error {
|
||||||
}
|
}
|
||||||
|
|
||||||
// MeasurementsSketches returns the two sketches for the index by merging all
|
// MeasurementsSketches returns the two sketches for the index by merging all
|
||||||
// instances of the type sketch types in all the index files.
|
// instances of the type sketch types in all the partitions.
|
||||||
func (i *Index) MeasurementsSketches() (estimator.Sketch, estimator.Sketch, error) {
|
func (i *Index) MeasurementsSketches() (estimator.Sketch, estimator.Sketch, error) {
|
||||||
s, ts := hll.NewDefaultPlus(), hll.NewDefaultPlus()
|
s, ts := hll.NewDefaultPlus(), hll.NewDefaultPlus()
|
||||||
for _, p := range i.partitions {
|
for _, p := range i.partitions {
|
||||||
|
@ -614,8 +614,27 @@ func (i *Index) MeasurementsSketches() (estimator.Sketch, estimator.Sketch, erro
|
||||||
return s, ts, nil
|
return s, ts, nil
|
||||||
}
|
}
|
||||||
|
|
||||||
// SeriesN returns the number of unique non-tombstoned series in this index.
|
// SeriesSketches returns the two sketches for the index by merging all
|
||||||
//
|
// instances of the type sketch types in all the partitions.
|
||||||
|
func (i *Index) SeriesSketches() (estimator.Sketch, estimator.Sketch, error) {
|
||||||
|
s, ts := hll.NewDefaultPlus(), hll.NewDefaultPlus()
|
||||||
|
for _, p := range i.partitions {
|
||||||
|
// Get partition's measurement sketches and merge.
|
||||||
|
ps, pts, err := p.SeriesSketches()
|
||||||
|
if err != nil {
|
||||||
|
return nil, nil, err
|
||||||
|
}
|
||||||
|
|
||||||
|
if err := s.Merge(ps); err != nil {
|
||||||
|
return nil, nil, err
|
||||||
|
} else if err := ts.Merge(pts); err != nil {
|
||||||
|
return nil, nil, err
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
return s, ts, nil
|
||||||
|
}
|
||||||
|
|
||||||
// Since indexes are not shared across shards, the count returned by SeriesN
|
// Since indexes are not shared across shards, the count returned by SeriesN
|
||||||
// cannot be combined with other shard's results. If you need to count series
|
// cannot be combined with other shard's results. If you need to count series
|
||||||
// across indexes then use either the database-wide series file, or merge the
|
// across indexes then use either the database-wide series file, or merge the
|
||||||
|
|
|
@ -52,6 +52,9 @@ type IndexFile struct {
|
||||||
seriesIDSetData []byte
|
seriesIDSetData []byte
|
||||||
tombstoneSeriesIDSetData []byte
|
tombstoneSeriesIDSetData []byte
|
||||||
|
|
||||||
|
// Series sketches
|
||||||
|
sketch, tSketch estimator.Sketch
|
||||||
|
|
||||||
// Sortable identifier & filepath to the log file.
|
// Sortable identifier & filepath to the log file.
|
||||||
level int
|
level int
|
||||||
id int
|
id int
|
||||||
|
@ -340,8 +343,8 @@ func (f *IndexFile) MeasurementSeriesIDIterator(name []byte) tsdb.SeriesIDIterat
|
||||||
return f.mblk.SeriesIDIterator(name)
|
return f.mblk.SeriesIDIterator(name)
|
||||||
}
|
}
|
||||||
|
|
||||||
// MergeMeasurementsSketches merges the index file's series sketches into the provided
|
// MergeMeasurementsSketches merges the index file's measurements sketches into
|
||||||
// sketches.
|
// the provided sketches.
|
||||||
func (f *IndexFile) MergeMeasurementsSketches(s, t estimator.Sketch) error {
|
func (f *IndexFile) MergeMeasurementsSketches(s, t estimator.Sketch) error {
|
||||||
if err := s.Merge(f.mblk.sketch); err != nil {
|
if err := s.Merge(f.mblk.sketch); err != nil {
|
||||||
return err
|
return err
|
||||||
|
@ -349,6 +352,15 @@ func (f *IndexFile) MergeMeasurementsSketches(s, t estimator.Sketch) error {
|
||||||
return t.Merge(f.mblk.tSketch)
|
return t.Merge(f.mblk.tSketch)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// MergeSeriesSketches merges the index file's series sketches into the provided
|
||||||
|
// sketches.
|
||||||
|
func (f *IndexFile) MergeSeriesSketches(s, t estimator.Sketch) error {
|
||||||
|
if err := s.Merge(f.sketch); err != nil {
|
||||||
|
return err
|
||||||
|
}
|
||||||
|
return t.Merge(f.tSketch)
|
||||||
|
}
|
||||||
|
|
||||||
// ReadIndexFileTrailer returns the index file trailer from data.
|
// ReadIndexFileTrailer returns the index file trailer from data.
|
||||||
func ReadIndexFileTrailer(data []byte) (IndexFileTrailer, error) {
|
func ReadIndexFileTrailer(data []byte) (IndexFileTrailer, error) {
|
||||||
var t IndexFileTrailer
|
var t IndexFileTrailer
|
||||||
|
|
|
@ -162,7 +162,6 @@ func (f *LogFile) Close() error {
|
||||||
}
|
}
|
||||||
|
|
||||||
f.mms = make(logMeasurements)
|
f.mms = make(logMeasurements)
|
||||||
|
|
||||||
return nil
|
return nil
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -658,15 +657,13 @@ func (f *LogFile) execSeriesEntry(e *LogEntry) {
|
||||||
mm.tagSet[string(k)] = ts
|
mm.tagSet[string(k)] = ts
|
||||||
}
|
}
|
||||||
|
|
||||||
// TODO(edd) increment series count....
|
|
||||||
f.sSketch.Add(seriesKey) // Add series to sketch.
|
|
||||||
f.mSketch.Add(name) // Add measurement to sketch as this may be the fist series for the measurement.
|
|
||||||
|
|
||||||
// Add/remove from appropriate series id sets.
|
// Add/remove from appropriate series id sets.
|
||||||
if !deleted {
|
if !deleted {
|
||||||
|
f.sSketch.Add(seriesKey) // Add series to sketch - key in series file format.
|
||||||
f.seriesIDSet.Add(e.SeriesID)
|
f.seriesIDSet.Add(e.SeriesID)
|
||||||
f.tombstoneSeriesIDSet.Remove(e.SeriesID)
|
f.tombstoneSeriesIDSet.Remove(e.SeriesID)
|
||||||
} else {
|
} else {
|
||||||
|
f.sTSketch.Add(seriesKey) // Add series to tombstone sketch - key in series file format.
|
||||||
f.seriesIDSet.Remove(e.SeriesID)
|
f.seriesIDSet.Remove(e.SeriesID)
|
||||||
f.tombstoneSeriesIDSet.Add(e.SeriesID)
|
f.tombstoneSeriesIDSet.Add(e.SeriesID)
|
||||||
}
|
}
|
||||||
|
@ -732,6 +729,9 @@ func (f *LogFile) createMeasurementIfNotExists(name []byte) *logMeasurement {
|
||||||
series: make(map[uint64]struct{}),
|
series: make(map[uint64]struct{}),
|
||||||
}
|
}
|
||||||
f.mms[string(name)] = mm
|
f.mms[string(name)] = mm
|
||||||
|
|
||||||
|
// Add measurement to sketch.
|
||||||
|
f.mSketch.Add(name)
|
||||||
}
|
}
|
||||||
return mm
|
return mm
|
||||||
}
|
}
|
||||||
|
@ -968,6 +968,20 @@ func (f *LogFile) MergeMeasurementsSketches(sketch, tsketch estimator.Sketch) er
|
||||||
return tsketch.Merge(f.mTSketch)
|
return tsketch.Merge(f.mTSketch)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// MergeSeriesSketches merges the series sketches belonging to this
|
||||||
|
// LogFile into the provided sketches.
|
||||||
|
//
|
||||||
|
// MergeSeriesSketches is safe for concurrent use by multiple goroutines.
|
||||||
|
func (f *LogFile) MergeSeriesSketches(sketch, tsketch estimator.Sketch) error {
|
||||||
|
f.mu.RLock()
|
||||||
|
defer f.mu.RUnlock()
|
||||||
|
|
||||||
|
if err := sketch.Merge(f.sSketch); err != nil {
|
||||||
|
return err
|
||||||
|
}
|
||||||
|
return tsketch.Merge(f.sTSketch)
|
||||||
|
}
|
||||||
|
|
||||||
// LogEntry represents a single log entry in the write-ahead log.
|
// LogEntry represents a single log entry in the write-ahead log.
|
||||||
type LogEntry struct {
|
type LogEntry struct {
|
||||||
Flag byte // flag
|
Flag byte // flag
|
||||||
|
|
|
@ -622,7 +622,7 @@ func (i *Partition) DropSeries(seriesID uint64) error {
|
||||||
return i.CheckLogFile()
|
return i.CheckLogFile()
|
||||||
}
|
}
|
||||||
|
|
||||||
// MeasurementsSketches returns the two sketches for the index by merging all
|
// MeasurementsSketches returns the two sketches for the partition by merging all
|
||||||
// instances of the type sketch types in all the index files.
|
// instances of the type sketch types in all the index files.
|
||||||
func (i *Partition) MeasurementsSketches() (estimator.Sketch, estimator.Sketch, error) {
|
func (i *Partition) MeasurementsSketches() (estimator.Sketch, estimator.Sketch, error) {
|
||||||
fs, err := i.RetainFileSet()
|
fs, err := i.RetainFileSet()
|
||||||
|
@ -633,6 +633,17 @@ func (i *Partition) MeasurementsSketches() (estimator.Sketch, estimator.Sketch,
|
||||||
return fs.MeasurementsSketches()
|
return fs.MeasurementsSketches()
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// SeriesSketches returns the two sketches for the partition by merging all
|
||||||
|
// instances of the type sketch types in all the index files.
|
||||||
|
func (i *Partition) SeriesSketches() (estimator.Sketch, estimator.Sketch, error) {
|
||||||
|
fs, err := i.RetainFileSet()
|
||||||
|
if err != nil {
|
||||||
|
return nil, nil, err
|
||||||
|
}
|
||||||
|
defer fs.Release()
|
||||||
|
return fs.SeriesSketches()
|
||||||
|
}
|
||||||
|
|
||||||
// HasTagKey returns true if tag key exists.
|
// HasTagKey returns true if tag key exists.
|
||||||
func (i *Partition) HasTagKey(name, key []byte) (bool, error) {
|
func (i *Partition) HasTagKey(name, key []byte) (bool, error) {
|
||||||
fs, err := i.RetainFileSet()
|
fs, err := i.RetainFileSet()
|
||||||
|
|
|
@ -705,6 +705,15 @@ func (s *Shard) SeriesN() int64 {
|
||||||
return engine.SeriesN()
|
return engine.SeriesN()
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// SeriesSketches returns the measurement sketches for the shard.
|
||||||
|
func (s *Shard) SeriesSketches() (estimator.Sketch, estimator.Sketch, error) {
|
||||||
|
engine, err := s.engine()
|
||||||
|
if err != nil {
|
||||||
|
return nil, nil, err
|
||||||
|
}
|
||||||
|
return engine.SeriesSketches()
|
||||||
|
}
|
||||||
|
|
||||||
// MeasurementsSketches returns the measurement sketches for the shard.
|
// MeasurementsSketches returns the measurement sketches for the shard.
|
||||||
func (s *Shard) MeasurementsSketches() (estimator.Sketch, estimator.Sketch, error) {
|
func (s *Shard) MeasurementsSketches() (estimator.Sketch, estimator.Sketch, error) {
|
||||||
engine, err := s.engine()
|
engine, err := s.engine()
|
||||||
|
|
|
@ -849,7 +849,9 @@ func (s *Store) DiskSize() (int64, error) {
|
||||||
return size, nil
|
return size, nil
|
||||||
}
|
}
|
||||||
|
|
||||||
func (s *Store) estimateCardinality(dbName string, getSketches func(*Shard) (estimator.Sketch, estimator.Sketch, error)) (int64, error) {
|
// sketchesForDatabase returns merged sketches for the provided database, by
|
||||||
|
// walking each shard in the database and merging the sketches found there.
|
||||||
|
func (s *Store) sketchesForDatabase(dbName string, getSketches func(*Shard) (estimator.Sketch, estimator.Sketch, error)) (estimator.Sketch, estimator.Sketch, error) {
|
||||||
var (
|
var (
|
||||||
ss estimator.Sketch // Sketch estimating number of items.
|
ss estimator.Sketch // Sketch estimating number of items.
|
||||||
ts estimator.Sketch // Sketch estimating number of tombstoned items.
|
ts estimator.Sketch // Sketch estimating number of tombstoned items.
|
||||||
|
@ -863,27 +865,26 @@ func (s *Store) estimateCardinality(dbName string, getSketches func(*Shard) (est
|
||||||
for _, shard := range shards {
|
for _, shard := range shards {
|
||||||
s, t, err := getSketches(shard)
|
s, t, err := getSketches(shard)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return 0, err
|
return nil, nil, err
|
||||||
}
|
}
|
||||||
|
|
||||||
if ss == nil {
|
if ss == nil {
|
||||||
ss, ts = s, t
|
ss, ts = s, t
|
||||||
} else if err = ss.Merge(s); err != nil {
|
} else if err = ss.Merge(s); err != nil {
|
||||||
return 0, err
|
return nil, nil, err
|
||||||
} else if err = ts.Merge(t); err != nil {
|
} else if err = ts.Merge(t); err != nil {
|
||||||
return 0, err
|
return nil, nil, err
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
return ss, ts, nil
|
||||||
if ss != nil {
|
|
||||||
return int64(ss.Count() - ts.Count()), nil
|
|
||||||
}
|
|
||||||
return 0, nil
|
|
||||||
}
|
}
|
||||||
|
|
||||||
// SeriesCardinality returns the series cardinality for the provided database.
|
// SeriesCardinality returns the exact series cardinality for the provided
|
||||||
|
// database.
|
||||||
|
//
|
||||||
|
// Cardinality is calculated exactly by unioning all shards' bitsets of series
|
||||||
|
// IDs. The result of this method cannot be combined with any other results.
|
||||||
//
|
//
|
||||||
// Cardinality is calculated exactly by unioning all shards' bitsets of series IDs.
|
|
||||||
func (s *Store) SeriesCardinality(database string) (int64, error) {
|
func (s *Store) SeriesCardinality(database string) (int64, error) {
|
||||||
s.mu.RLock()
|
s.mu.RLock()
|
||||||
shards := s.filterShards(byDatabase(database))
|
shards := s.filterShards(byDatabase(database))
|
||||||
|
@ -911,12 +912,46 @@ func (s *Store) SeriesCardinality(database string) (int64, error) {
|
||||||
return int64(ss.Cardinality()), nil
|
return int64(ss.Cardinality()), nil
|
||||||
}
|
}
|
||||||
|
|
||||||
// MeasurementsCardinality returns the measurement cardinality for the provided
|
// SeriesSketches returns the sketches associated with the series data in all
|
||||||
// database.
|
// the shards in the provided database.
|
||||||
//
|
//
|
||||||
// Cardinality is calculated using a sketch-based estimation.
|
// The returned sketches can be combined with other sketches to provide an
|
||||||
|
// estimation across distributed databases.
|
||||||
|
func (s *Store) SeriesSketches(database string) (estimator.Sketch, estimator.Sketch, error) {
|
||||||
|
return s.sketchesForDatabase(database, func(sh *Shard) (estimator.Sketch, estimator.Sketch, error) {
|
||||||
|
if sh == nil {
|
||||||
|
return nil, nil, errors.New("shard nil, can't get cardinality")
|
||||||
|
}
|
||||||
|
return sh.SeriesSketches()
|
||||||
|
})
|
||||||
|
}
|
||||||
|
|
||||||
|
// MeasurementsCardinality returns an estimation of the measurement cardinality
|
||||||
|
// for the provided database.
|
||||||
|
//
|
||||||
|
// Cardinality is calculated using a sketch-based estimation. The result of this
|
||||||
|
// method cannot be combined with any other results.
|
||||||
func (s *Store) MeasurementsCardinality(database string) (int64, error) {
|
func (s *Store) MeasurementsCardinality(database string) (int64, error) {
|
||||||
return s.estimateCardinality(database, func(sh *Shard) (estimator.Sketch, estimator.Sketch, error) {
|
ss, ts, err := s.sketchesForDatabase(database, func(sh *Shard) (estimator.Sketch, estimator.Sketch, error) {
|
||||||
|
if sh == nil {
|
||||||
|
return nil, nil, errors.New("shard nil, can't get cardinality")
|
||||||
|
}
|
||||||
|
return sh.MeasurementsSketches()
|
||||||
|
})
|
||||||
|
|
||||||
|
if err != nil {
|
||||||
|
return 0, err
|
||||||
|
}
|
||||||
|
return int64(ss.Count() - ts.Count()), nil
|
||||||
|
}
|
||||||
|
|
||||||
|
// MeasurementsSketches returns the sketches associated with the measurement
|
||||||
|
// data in all the shards in the provided database.
|
||||||
|
//
|
||||||
|
// The returned sketches can be combined with other sketches to provide an
|
||||||
|
// estimation across distributed databases.
|
||||||
|
func (s *Store) MeasurementsSketches(database string) (estimator.Sketch, estimator.Sketch, error) {
|
||||||
|
return s.sketchesForDatabase(database, func(sh *Shard) (estimator.Sketch, estimator.Sketch, error) {
|
||||||
if sh == nil {
|
if sh == nil {
|
||||||
return nil, nil, errors.New("shard nil, can't get cardinality")
|
return nil, nil, errors.New("shard nil, can't get cardinality")
|
||||||
}
|
}
|
||||||
|
|
|
@ -16,6 +16,8 @@ import (
|
||||||
"testing"
|
"testing"
|
||||||
"time"
|
"time"
|
||||||
|
|
||||||
|
"github.com/influxdata/influxdb/tsdb/index/inmem"
|
||||||
|
|
||||||
"github.com/davecgh/go-spew/spew"
|
"github.com/davecgh/go-spew/spew"
|
||||||
"github.com/influxdata/influxdb/internal"
|
"github.com/influxdata/influxdb/internal"
|
||||||
"github.com/influxdata/influxdb/logger"
|
"github.com/influxdata/influxdb/logger"
|
||||||
|
@ -941,6 +943,138 @@ func TestStore_Cardinality_Compactions(t *testing.T) {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
func TestStore_Sketches(t *testing.T) {
|
||||||
|
t.Parallel()
|
||||||
|
|
||||||
|
checkCardinalities := func(store *tsdb.Store, series, tseries, measurements, tmeasurements int) error {
|
||||||
|
// Get sketches and check cardinality...
|
||||||
|
sketch, tsketch, err := store.SeriesSketches("db")
|
||||||
|
if err != nil {
|
||||||
|
return err
|
||||||
|
}
|
||||||
|
|
||||||
|
// delta calculates a rough 10% delta. If i is small then a minimum value
|
||||||
|
// of 2 is used.
|
||||||
|
delta := func(i int) int {
|
||||||
|
v := i / 10
|
||||||
|
if v == 0 {
|
||||||
|
v = 2
|
||||||
|
}
|
||||||
|
return v
|
||||||
|
}
|
||||||
|
|
||||||
|
// series cardinality should be well within 10%.
|
||||||
|
if got, exp := int(sketch.Count()), series; got-exp < -delta(series) || got-exp > delta(series) {
|
||||||
|
return fmt.Errorf("got series cardinality %d, expected ~%d", got, exp)
|
||||||
|
}
|
||||||
|
|
||||||
|
// check series tombstones
|
||||||
|
if got, exp := int(tsketch.Count()), tseries; got-exp < -delta(tseries) || got-exp > delta(tseries) {
|
||||||
|
return fmt.Errorf("got series tombstone cardinality %d, expected ~%d", got, exp)
|
||||||
|
}
|
||||||
|
|
||||||
|
// Check measurement cardinality.
|
||||||
|
if sketch, tsketch, err = store.MeasurementsSketches("db"); err != nil {
|
||||||
|
return err
|
||||||
|
}
|
||||||
|
|
||||||
|
if got, exp := int(sketch.Count()), measurements; got-exp < -delta(measurements) || got-exp > delta(measurements) {
|
||||||
|
return fmt.Errorf("got measurement cardinality %d, expected ~%d", got, exp)
|
||||||
|
}
|
||||||
|
|
||||||
|
if got, exp := int(tsketch.Count()), tmeasurements; got-exp < -delta(tmeasurements) || got-exp > delta(tmeasurements) {
|
||||||
|
return fmt.Errorf("got measurement tombstone cardinality %d, expected ~%d", got, exp)
|
||||||
|
}
|
||||||
|
return nil
|
||||||
|
}
|
||||||
|
|
||||||
|
test := func(index string) error {
|
||||||
|
store := MustOpenStore(index)
|
||||||
|
defer store.Close()
|
||||||
|
|
||||||
|
// Generate point data to write to the shards.
|
||||||
|
series := genTestSeries(10, 2, 4) // 160 series
|
||||||
|
|
||||||
|
points := make([]models.Point, 0, len(series))
|
||||||
|
for _, s := range series {
|
||||||
|
points = append(points, models.MustNewPoint(s.Measurement, s.Tags, map[string]interface{}{"value": 1.0}, time.Now()))
|
||||||
|
}
|
||||||
|
|
||||||
|
// Create requested number of shards in the store & write points across
|
||||||
|
// shards such that we never write the same series to multiple shards.
|
||||||
|
for shardID := 0; shardID < 4; shardID++ {
|
||||||
|
if err := store.CreateShard("db", "rp", uint64(shardID), true); err != nil {
|
||||||
|
return fmt.Errorf("create shard: %s", err)
|
||||||
|
}
|
||||||
|
|
||||||
|
if err := store.BatchWrite(shardID, points[shardID*40:(shardID+1)*40]); err != nil {
|
||||||
|
return fmt.Errorf("batch write: %s", err)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
// Check cardinalities
|
||||||
|
if err := checkCardinalities(store.Store, 160, 0, 10, 0); err != nil {
|
||||||
|
return fmt.Errorf("[initial] %v", err)
|
||||||
|
}
|
||||||
|
|
||||||
|
// Reopen the store.
|
||||||
|
if err := store.Reopen(); err != nil {
|
||||||
|
return err
|
||||||
|
}
|
||||||
|
|
||||||
|
// Check cardinalities
|
||||||
|
if err := checkCardinalities(store.Store, 160, 0, 10, 0); err != nil {
|
||||||
|
return fmt.Errorf("[initial|re-open] %v", err)
|
||||||
|
}
|
||||||
|
|
||||||
|
// Delete half the the measurements data
|
||||||
|
mnames, err := store.MeasurementNames(nil, "db", nil)
|
||||||
|
if err != nil {
|
||||||
|
return err
|
||||||
|
}
|
||||||
|
|
||||||
|
for _, name := range mnames[:len(mnames)/2] {
|
||||||
|
if err := store.DeleteSeries("db", []influxql.Source{&influxql.Measurement{Name: string(name)}}, nil); err != nil {
|
||||||
|
return err
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
// Check cardinalities - tombstones should be in
|
||||||
|
if err := checkCardinalities(store.Store, 160, 80, 10, 5); err != nil {
|
||||||
|
return fmt.Errorf("[initial|re-open|delete] %v", err)
|
||||||
|
}
|
||||||
|
|
||||||
|
// Reopen the store.
|
||||||
|
if err := store.Reopen(); err != nil {
|
||||||
|
return err
|
||||||
|
}
|
||||||
|
|
||||||
|
// Check cardinalities. In this case, the indexes behave differently.
|
||||||
|
//
|
||||||
|
// - The inmem index will report that there are 80 series and no tombstones.
|
||||||
|
// - The tsi1 index will report that there are 160 series and 80 tombstones.
|
||||||
|
//
|
||||||
|
// The result is the same, but the implementation differs.
|
||||||
|
expS, expTS, expM, expTM := 160, 80, 10, 5
|
||||||
|
if index == inmem.IndexName {
|
||||||
|
expS, expTS, expM, expTM = 80, 0, 5, 0
|
||||||
|
}
|
||||||
|
|
||||||
|
if err := checkCardinalities(store.Store, expS, expTS, expM, expTM); err != nil {
|
||||||
|
return fmt.Errorf("[initial|re-open|delete|re-open] %v", err)
|
||||||
|
}
|
||||||
|
return nil
|
||||||
|
}
|
||||||
|
|
||||||
|
for _, index := range tsdb.RegisteredIndexes() {
|
||||||
|
t.Run(index, func(t *testing.T) {
|
||||||
|
if err := test(index); err != nil {
|
||||||
|
t.Fatal(err)
|
||||||
|
}
|
||||||
|
})
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
func TestStore_TagValues(t *testing.T) {
|
func TestStore_TagValues(t *testing.T) {
|
||||||
t.Parallel()
|
t.Parallel()
|
||||||
|
|
||||||
|
|
Loading…
Reference in New Issue