influxdb/tsdb/store.go

1043 lines
25 KiB
Go
Raw Normal View History

2016-02-10 18:30:52 +00:00
package tsdb // import "github.com/influxdata/influxdb/tsdb"
import (
"errors"
"fmt"
"io"
2015-05-28 22:02:12 +00:00
"io/ioutil"
"os"
"path/filepath"
"runtime"
2015-11-04 21:06:06 +00:00
"sort"
2015-05-28 22:02:12 +00:00
"strconv"
"strings"
2015-05-28 22:02:12 +00:00
"sync"
"time"
2015-05-28 22:02:12 +00:00
"github.com/influxdata/influxdb/influxql"
"github.com/influxdata/influxdb/models"
2016-09-21 15:04:37 +00:00
"github.com/influxdata/influxdb/pkg/estimator"
2016-07-18 18:00:58 +00:00
"github.com/influxdata/influxdb/pkg/limiter"
"go.uber.org/zap"
)
var (
2016-12-31 05:12:37 +00:00
// ErrShardNotFound is returned when trying to get a non existing shard.
ErrShardNotFound = fmt.Errorf("shard not found")
2016-12-31 05:12:37 +00:00
// ErrStoreClosed is returned when trying to use a closed Store.
2016-02-10 20:04:18 +00:00
ErrStoreClosed = fmt.Errorf("store is closed")
)
2016-02-10 20:04:18 +00:00
// Store manages shards and indexes for databases.
type Store struct {
2016-09-01 12:40:16 +00:00
mu sync.RWMutex
// databases keeps track of the number of databases being managed by the
// store.
databases map[string]struct{}
2016-09-01 12:40:16 +00:00
path string
// shards is a map of shard IDs to the associated Shard.
shards map[uint64]*Shard
2015-07-22 14:53:20 +00:00
EngineOptions EngineOptions
baseLogger zap.Logger
Logger zap.Logger
closing chan struct{}
wg sync.WaitGroup
opened bool
}
2016-02-10 20:04:18 +00:00
// NewStore returns a new store with the given path and a default configuration.
// The returned store must be initialized by calling Open before using it.
2015-11-04 21:06:06 +00:00
func NewStore(path string) *Store {
opts := NewEngineOptions()
logger := zap.New(zap.NullEncoder())
2015-11-04 21:06:06 +00:00
return &Store{
2016-09-01 12:40:16 +00:00
databases: make(map[string]struct{}),
2015-11-04 21:06:06 +00:00
path: path,
EngineOptions: opts,
Logger: logger,
baseLogger: logger,
}
}
2016-12-31 05:12:37 +00:00
// WithLogger sets the logger for the store.
func (s *Store) WithLogger(log zap.Logger) {
s.baseLogger = log
s.Logger = log.With(zap.String("service", "store"))
for _, sh := range s.shards {
sh.WithLogger(s.baseLogger)
2015-11-04 21:06:06 +00:00
}
}
2016-12-31 05:12:37 +00:00
// Statistics returns statistics for period monitoring.
func (s *Store) Statistics(tags map[string]string) []models.Statistic {
s.mu.RLock()
shards := s.shardsSlice()
s.mu.RUnlock()
2016-09-01 12:40:16 +00:00
// Gather all statistics for all shards.
var statistics []models.Statistic
for _, shard := range shards {
statistics = append(statistics, shard.Statistics(tags)...)
}
return statistics
}
// Path returns the store's root path.
func (s *Store) Path() string { return s.path }
2016-02-10 20:04:18 +00:00
// Open initializes the store, creating all necessary directories, loading all
2016-09-01 12:40:16 +00:00
// shards as well as initializing periodic maintenance of them.
2015-11-04 21:06:06 +00:00
func (s *Store) Open() error {
s.mu.Lock()
defer s.mu.Unlock()
s.closing = make(chan struct{})
s.shards = map[uint64]*Shard{}
2015-11-04 21:06:06 +00:00
s.Logger.Info(fmt.Sprintf("Using data dir: %v", s.Path()))
2015-11-04 21:06:06 +00:00
// Create directory.
if err := os.MkdirAll(s.path, 0777); err != nil {
return err
}
if err := s.loadShards(); err != nil {
return err
}
s.opened = true
return nil
}
func (s *Store) loadShards() error {
2016-09-01 12:40:16 +00:00
// res holds the result from opening each shard in a goroutine
2016-01-11 18:00:25 +00:00
type res struct {
s *Shard
err error
}
2016-07-18 18:00:58 +00:00
t := limiter.NewFixed(runtime.GOMAXPROCS(0))
2016-01-11 18:00:25 +00:00
resC := make(chan *res)
var n int
2016-09-01 12:40:16 +00:00
// Determine how many shards we need to open by checking the store path.
dbDirs, err := ioutil.ReadDir(s.path)
if err != nil {
return err
}
for _, db := range dbDirs {
if !db.IsDir() {
s.Logger.Printf("Not loading %s. Not a database directory.", db.Name())
continue
}
// Load each retention policy within the database directory.
rpDirs, err := ioutil.ReadDir(filepath.Join(s.path, db.Name()))
2015-11-04 21:06:06 +00:00
if err != nil {
return err
}
2016-09-01 12:40:16 +00:00
for _, rp := range rpDirs {
2015-11-04 21:06:06 +00:00
if !rp.IsDir() {
s.Logger.Info(fmt.Sprintf("Skipping retention policy dir: %s. Not a directory", rp.Name()))
2015-11-04 21:06:06 +00:00
continue
}
2016-09-01 12:40:16 +00:00
shardDirs, err := ioutil.ReadDir(filepath.Join(s.path, db.Name(), rp.Name()))
2015-11-04 21:06:06 +00:00
if err != nil {
return err
}
2016-09-01 12:40:16 +00:00
for _, sh := range shardDirs {
2016-01-11 18:00:25 +00:00
n++
2016-09-01 12:40:16 +00:00
go func(db, rp, sh string) {
t.Take()
defer t.Release()
2016-01-11 18:00:25 +00:00
start := time.Now()
path := filepath.Join(s.path, db, rp, sh)
walPath := filepath.Join(s.EngineOptions.Config.WALDir, db, rp, sh)
// Shard file names are numeric shardIDs
shardID, err := strconv.ParseUint(sh, 10, 64)
if err != nil {
resC <- &res{err: fmt.Errorf("%s is not a valid ID. Skipping shard.", sh)}
return
}
2016-09-01 12:40:16 +00:00
shard := NewShard(shardID, path, walPath, s.EngineOptions)
shard.WithLogger(s.baseLogger)
2016-01-11 18:00:25 +00:00
err = shard.Open()
if err != nil {
resC <- &res{err: fmt.Errorf("Failed to open shard: %d: %s", shardID, err)}
return
}
resC <- &res{s: shard}
s.Logger.Info(fmt.Sprintf("%s opened in %s", path, time.Now().Sub(start)))
2016-09-01 12:40:16 +00:00
}(db, rp.Name(), sh.Name())
2015-11-04 21:06:06 +00:00
}
}
}
2016-09-01 12:40:16 +00:00
// Gather results of opening shards concurrently, keeping track of how
// many databases we are managing.
2016-01-11 18:00:25 +00:00
for i := 0; i < n; i++ {
res := <-resC
if res.err != nil {
s.Logger.Info(res.err.Error())
2016-01-11 18:00:25 +00:00
continue
}
s.shards[res.s.id] = res.s
2016-09-01 12:40:16 +00:00
s.databases[res.s.database] = struct{}{}
2016-01-11 18:00:25 +00:00
}
close(resC)
2015-11-04 21:06:06 +00:00
return nil
}
2016-02-10 20:04:18 +00:00
// Close closes the store and all associated shards. After calling Close accessing
// shards through the Store will result in ErrStoreClosed being returned.
2015-11-04 21:06:06 +00:00
func (s *Store) Close() error {
s.mu.Lock()
defer s.mu.Unlock()
if s.opened {
close(s.closing)
}
s.wg.Wait()
// Close all the shards in parallel.
if err := s.walkShards(s.shardsSlice(), func(sh *Shard) error {
return sh.Close()
}); err != nil {
return err
2015-11-04 21:06:06 +00:00
}
2015-11-04 21:06:06 +00:00
s.opened = false
s.shards = nil
2015-11-04 21:06:06 +00:00
return nil
}
2015-07-20 19:59:46 +00:00
// Shard returns a shard by id.
func (s *Store) Shard(id uint64) *Shard {
s.mu.RLock()
defer s.mu.RUnlock()
sh, ok := s.shards[id]
if !ok {
return nil
}
return sh
2015-07-20 19:59:46 +00:00
}
2015-11-04 21:06:06 +00:00
// Shards returns a list of shards by id.
func (s *Store) Shards(ids []uint64) []*Shard {
s.mu.RLock()
defer s.mu.RUnlock()
a := make([]*Shard, 0, len(ids))
for _, id := range ids {
sh, ok := s.shards[id]
if !ok {
continue
}
a = append(a, sh)
2015-11-04 21:06:06 +00:00
}
return a
}
// ShardN returns the number of shards in the store.
2015-07-20 19:59:46 +00:00
func (s *Store) ShardN() int {
s.mu.RLock()
defer s.mu.RUnlock()
return len(s.shards)
2015-07-20 19:59:46 +00:00
}
2016-02-10 20:04:18 +00:00
// CreateShard creates a shard with the given id and retention policy on a database.
func (s *Store) CreateShard(database, retentionPolicy string, shardID uint64, enabled bool) error {
s.mu.Lock()
defer s.mu.Unlock()
select {
case <-s.closing:
return ErrStoreClosed
default:
}
2016-09-01 12:40:16 +00:00
// Shard already exists.
if _, ok := s.shards[shardID]; ok {
return nil
}
2016-09-01 12:40:16 +00:00
// Create the db and retention policy directories if they don't exist.
if err := os.MkdirAll(filepath.Join(s.path, database, retentionPolicy), 0700); err != nil {
return err
}
2016-09-01 12:40:16 +00:00
// Create the WAL directory.
walPath := filepath.Join(s.EngineOptions.Config.WALDir, database, retentionPolicy, fmt.Sprintf("%d", shardID))
if err := os.MkdirAll(walPath, 0700); err != nil {
return err
}
path := filepath.Join(s.path, database, retentionPolicy, strconv.FormatUint(shardID, 10))
2016-09-01 12:40:16 +00:00
shard := NewShard(shardID, path, walPath, s.EngineOptions)
shard.WithLogger(s.baseLogger)
shard.EnableOnOpen = enabled
if err := shard.Open(); err != nil {
return err
}
s.shards[shardID] = shard
2016-09-01 12:40:16 +00:00
s.databases[database] = struct{}{} // Ensure we are tracking any new db.
return nil
}
2016-12-31 05:12:37 +00:00
// CreateShardSnapShot will create a hard link to the underlying shard and return a path.
// The caller is responsible for cleaning up (removing) the file path returned.
2016-05-09 15:53:34 +00:00
func (s *Store) CreateShardSnapshot(id uint64) (string, error) {
sh := s.Shard(id)
if sh == nil {
return "", ErrShardNotFound
}
return sh.CreateSnapshot()
}
2016-12-31 05:12:37 +00:00
// SetShardEnabled enables or disables a shard for read and writes.
func (s *Store) SetShardEnabled(shardID uint64, enabled bool) error {
sh := s.Shard(shardID)
if sh == nil {
return ErrShardNotFound
}
sh.SetEnabled(enabled)
return nil
}
2015-06-04 01:02:49 +00:00
// DeleteShard removes a shard from disk.
func (s *Store) DeleteShard(shardID uint64) error {
sh := s.Shard(shardID)
if sh == nil {
2015-06-04 01:02:49 +00:00
return nil
}
if err := sh.Close(); err != nil {
2015-06-04 01:02:49 +00:00
return err
}
if err := os.RemoveAll(sh.path); err != nil {
2015-06-04 01:02:49 +00:00
return err
}
if err := os.RemoveAll(sh.walPath); err != nil {
return err
}
s.mu.Lock()
delete(s.shards, shardID)
s.mu.Unlock()
2015-06-04 01:02:49 +00:00
return nil
}
// ShardIteratorCreator returns an iterator creator for a shard.
func (s *Store) ShardIteratorCreator(id uint64, opt *influxql.SelectOptions) influxql.IteratorCreator {
sh := s.Shard(id)
if sh == nil {
return nil
}
return &shardIteratorCreator{
sh: sh,
maxSeriesN: opt.MaxSeriesN,
}
}
// DeleteDatabase will close all shards associated with a database and remove the directory and files from disk.
func (s *Store) DeleteDatabase(name string) error {
s.mu.RLock()
2016-09-01 12:40:16 +00:00
shards := s.filterShards(byDatabase(name))
s.mu.RUnlock()
if err := s.walkShards(shards, func(sh *Shard) error {
if sh.database != name {
return nil
2015-06-05 16:31:04 +00:00
}
return sh.Close()
}); err != nil {
return err
2015-06-05 16:31:04 +00:00
}
if err := os.RemoveAll(filepath.Join(s.path, name)); err != nil {
return err
}
if err := os.RemoveAll(filepath.Join(s.EngineOptions.Config.WALDir, name)); err != nil {
return err
}
2015-11-04 21:06:06 +00:00
s.mu.Lock()
for _, sh := range shards {
delete(s.shards, sh.id)
}
2016-09-01 12:40:16 +00:00
// Remove database from store list of databases
delete(s.databases, name)
s.mu.Unlock()
2015-11-04 21:06:06 +00:00
return nil
}
// DeleteRetentionPolicy will close all shards associated with the
// provided retention policy, remove the retention policy directories on
// both the DB and WAL, and remove all shard files from disk.
func (s *Store) DeleteRetentionPolicy(database, name string) error {
s.mu.RLock()
shards := s.filterShards(func(sh *Shard) bool {
return sh.database == database && sh.retentionPolicy == name
})
s.mu.RUnlock()
// Close and delete all shards under the retention policy on the
// database.
if err := s.walkShards(shards, func(sh *Shard) error {
if sh.database != database || sh.retentionPolicy != name {
return nil
}
return sh.Close()
}); err != nil {
return err
}
2016-09-01 12:40:16 +00:00
// Remove the retention policy folder.
if err := os.RemoveAll(filepath.Join(s.path, database, name)); err != nil {
return err
}
// Remove the retention policy folder from the the WAL.
if err := os.RemoveAll(filepath.Join(s.EngineOptions.Config.WALDir, database, name)); err != nil {
return err
}
s.mu.Lock()
for _, sh := range shards {
delete(s.shards, sh.id)
}
s.mu.Unlock()
return nil
}
2015-11-04 21:06:06 +00:00
// DeleteMeasurement removes a measurement and all associated series from a database.
func (s *Store) DeleteMeasurement(database, name string) error {
s.mu.RLock()
2016-09-01 12:40:16 +00:00
shards := s.filterShards(byDatabase(database))
s.mu.RUnlock()
2016-09-01 12:40:16 +00:00
return s.walkShards(shards, func(sh *Shard) error {
if err := sh.DeleteMeasurement(name); err != nil {
return err
}
return nil
2016-09-01 12:40:16 +00:00
})
}
// filterShards returns a slice of shards where fn returns true
2016-09-14 13:55:44 +00:00
// for the shard. If the provided predicate is nil then all shards are returned.
func (s *Store) filterShards(fn func(sh *Shard) bool) []*Shard {
2016-09-14 13:55:44 +00:00
var shards []*Shard
if fn == nil {
shards = make([]*Shard, 0, len(s.shards))
fn = func(*Shard) bool { return true }
} else {
shards = make([]*Shard, 0)
}
for _, sh := range s.shards {
if fn(sh) {
shards = append(shards, sh)
2016-02-05 17:23:35 +00:00
}
}
return shards
}
2016-09-01 12:40:16 +00:00
// byDatabase provides a predicate for filterShards that matches on the name of
// the database passed in.
var byDatabase = func(name string) func(sh *Shard) bool {
return func(sh *Shard) bool {
return sh.database == name
}
}
// walkShards apply a function to each shard in parallel. If any of the
// functions return an error, the first error is returned.
func (s *Store) walkShards(shards []*Shard, fn func(sh *Shard) error) error {
// struct to hold the result of opening each reader in a goroutine
type res struct {
err error
2015-11-04 21:06:06 +00:00
}
2016-07-15 18:14:25 +00:00
resC := make(chan res)
var n int
for _, sh := range shards {
n++
go func(sh *Shard) {
if err := fn(sh); err != nil {
2016-07-15 18:14:25 +00:00
resC <- res{err: fmt.Errorf("shard %d: %s", sh.id, err)}
return
}
2016-07-15 18:14:25 +00:00
resC <- res{}
}(sh)
}
var err error
for i := 0; i < n; i++ {
res := <-resC
if res.err != nil {
err = res.err
}
}
close(resC)
return err
2015-06-05 16:31:04 +00:00
}
2015-11-04 21:06:06 +00:00
// shardsSlice returns an ordered list of shards.
func (s *Store) shardsSlice() []*Shard {
a := make([]*Shard, 0, len(s.shards))
for _, sh := range s.shards {
a = append(a, sh)
2015-05-28 22:02:12 +00:00
}
2015-11-04 21:06:06 +00:00
sort.Sort(Shards(a))
return a
2015-05-28 22:02:12 +00:00
}
2016-09-01 12:40:16 +00:00
// Databases returns the names of all databases managed by the store.
func (s *Store) Databases() []string {
s.mu.RLock()
defer s.mu.RUnlock()
2016-09-01 12:40:16 +00:00
databases := make([]string, 0, len(s.databases))
for k, _ := range s.databases {
databases = append(databases, k)
2015-05-28 22:02:12 +00:00
}
2016-09-01 12:40:16 +00:00
return databases
2015-05-28 22:02:12 +00:00
}
2016-09-01 12:40:16 +00:00
// DiskSize returns the size of all the shard files in bytes.
// This size does not include the WAL size.
func (s *Store) DiskSize() (int64, error) {
var size int64
2016-09-14 13:55:44 +00:00
s.mu.RLock()
allShards := s.filterShards(nil)
s.mu.RUnlock()
for _, sh := range allShards {
sz, err := sh.DiskSize()
if err != nil {
return 0, err
}
size += sz
}
return size, nil
}
2016-09-14 13:55:44 +00:00
// SeriesCardinality returns the series cardinality for the provided database.
func (s *Store) SeriesCardinality(database string) (int64, error) {
2016-09-21 15:04:37 +00:00
s.mu.RLock()
shards := s.filterShards(byDatabase(database))
s.mu.RUnlock()
var sketch estimator.Sketch
// Iterate over all shards for the database and combine all of the series
// sketches.
for _, shard := range shards {
other, err := shard.engine.SeriesSketch()
if err != nil {
return 0, err
}
if sketch == nil {
sketch = other
} else if err = sketch.Merge(other); err != nil {
return 0, err
}
}
if sketch != nil {
cnt, err := sketch.Count()
return int64(cnt), err
}
return 0, nil
2016-09-14 13:55:44 +00:00
}
// MeasurementsCardinality returns the measurement cardinality for the provided
// database.
func (s *Store) MeasurementsCardinality(database string) (int64, error) {
panic("TODO: edd")
}
2016-09-01 12:40:16 +00:00
// BackupShard will get the shard and have the engine backup since the passed in
// time to the writer.
func (s *Store) BackupShard(id uint64, since time.Time, w io.Writer) error {
shard := s.Shard(id)
if shard == nil {
return fmt.Errorf("shard %d doesn't exist on this server", id)
}
path, err := relativePath(s.path, shard.path)
if err != nil {
return err
}
return shard.engine.Backup(w, path, since)
}
2016-04-29 00:29:09 +00:00
// RestoreShard restores a backup from r to a given shard.
// This will only overwrite files included in the backup.
func (s *Store) RestoreShard(id uint64, r io.Reader) error {
shard := s.Shard(id)
if shard == nil {
return fmt.Errorf("shard %d doesn't exist on this server", id)
}
path, err := relativePath(s.path, shard.path)
if err != nil {
return err
}
return shard.Restore(r, path)
}
2016-09-01 12:40:16 +00:00
// ShardRelativePath will return the relative path to the shard, i.e.,
// <database>/<retention>/<id>.
func (s *Store) ShardRelativePath(id uint64) (string, error) {
shard := s.Shard(id)
if shard == nil {
return "", fmt.Errorf("shard %d doesn't exist on this server", id)
}
return relativePath(s.path, shard.path)
}
2016-09-01 12:40:16 +00:00
// DeleteSeries loops through the local shards and deletes the series data for
// the passed in series keys.
func (s *Store) DeleteSeries(database string, sources []influxql.Source, condition influxql.Expr) error {
// Expand regex expressions in the FROM clause.
a, err := s.ExpandSources(sources)
if err != nil {
return err
} else if sources != nil && len(sources) != 0 && len(a) == 0 {
return nil
}
sources = a
// Determine deletion time range.
min, max, err := influxql.TimeRangeAsEpochNano(condition)
if err != nil {
return err
}
s.mu.RLock()
2016-09-01 12:40:16 +00:00
shards := s.filterShards(byDatabase(database))
s.mu.RUnlock()
2016-09-01 12:40:16 +00:00
mMap := make(map[string]*Measurement)
for _, shard := range shards {
shardMeasures := shard.Measurements()
for _, m := range shardMeasures {
mMap[m.Name] = m
}
}
2016-09-01 12:40:16 +00:00
s.mu.RLock()
defer s.mu.RUnlock()
measurements, err := measurementsFromSourcesOrDB(mMap, sources...)
if err != nil {
return err
}
var seriesKeys []string
for _, m := range measurements {
var ids SeriesIDs
var filters FilterExprs
if condition != nil {
// Get series IDs that match the WHERE clause.
ids, filters, err = m.walkWhereForSeriesIds(condition)
if err != nil {
return err
}
// Delete boolean literal true filter expressions.
// These are returned for `WHERE tagKey = 'tagVal'` type expressions and are okay.
filters.DeleteBoolLiteralTrues()
// Check for unsupported field filters.
// Any remaining filters means there were fields (e.g., `WHERE value = 1.2`).
if filters.Len() > 0 {
return errors.New("fields not supported in WHERE clause during deletion")
}
} else {
// No WHERE clause so get all series IDs for this measurement.
ids = m.seriesIDs
}
for _, id := range ids {
seriesKeys = append(seriesKeys, m.seriesByID[id].Key)
}
}
2016-09-01 12:40:16 +00:00
// delete the raw series data.
return s.walkShards(shards, func(sh *Shard) error {
if err := sh.DeleteSeriesRange(seriesKeys, min, max); err != nil {
return err
}
return nil
})
}
// ExpandSources expands sources against all local shards.
2015-11-04 21:06:06 +00:00
func (s *Store) ExpandSources(sources influxql.Sources) (influxql.Sources, error) {
return s.IteratorCreators().ExpandSources(sources)
}
// IteratorCreators returns a set of all local shards as iterator creators.
func (s *Store) IteratorCreators() influxql.IteratorCreators {
s.mu.RLock()
defer s.mu.RUnlock()
a := make(influxql.IteratorCreators, 0, len(s.shards))
for _, sh := range s.shards {
a = append(a, sh)
2015-11-04 21:06:06 +00:00
}
return a
}
2016-12-31 05:12:37 +00:00
// IteratorCreator returns an iterator creator for all shards in the given shard IDs.
func (s *Store) IteratorCreator(shards []uint64, opt *influxql.SelectOptions) (influxql.IteratorCreator, error) {
Cleanup QueryExecutor and split statement execution code The QueryExecutor had a lot of dead code made obsolete by the query engine refactor that has now been removed. The TSDBStore interface has also been cleaned up so we can have multiple implementations of this (such as a local and remote version). A StatementExecutor interface has been created for adding custom functionality to the QueryExecutor that may not be available in the open source version. The QueryExecutor delegate all statement execution to the StatementExecutor and the QueryExecutor will only keep track of housekeeping. Implementing additional queries is as simple as wrapping the cluster.StatementExecutor struct or replacing it with something completely different. The PointsWriter in the QueryExecutor has been changed to a simple interface that implements the one method needed by the query executor. This is to allow different PointsWriter implementations to be used by the QueryExecutor. It has also been moved into the StatementExecutor instead. The TSDBStore interface has now been modified to contain the code for creating an IteratorCreator. This is so the underlying TSDBStore can implement different ways of accessing the underlying shards rather than always having to access each shard individually (such as batch requests). Remove the show servers handling. This isn't a valid command in the open source version of InfluxDB anymore. The QueryManager interface is now built into QueryExecutor and is no longer necessary. The StatementExecutor and QueryExecutor split allows task management to much more easily be built into QueryExecutor rather than as a separate struct.
2016-03-31 22:12:29 +00:00
// Generate iterators for each node.
ics := make([]influxql.IteratorCreator, 0)
if err := func() error {
for _, id := range shards {
ic := s.ShardIteratorCreator(id, opt)
Cleanup QueryExecutor and split statement execution code The QueryExecutor had a lot of dead code made obsolete by the query engine refactor that has now been removed. The TSDBStore interface has also been cleaned up so we can have multiple implementations of this (such as a local and remote version). A StatementExecutor interface has been created for adding custom functionality to the QueryExecutor that may not be available in the open source version. The QueryExecutor delegate all statement execution to the StatementExecutor and the QueryExecutor will only keep track of housekeeping. Implementing additional queries is as simple as wrapping the cluster.StatementExecutor struct or replacing it with something completely different. The PointsWriter in the QueryExecutor has been changed to a simple interface that implements the one method needed by the query executor. This is to allow different PointsWriter implementations to be used by the QueryExecutor. It has also been moved into the StatementExecutor instead. The TSDBStore interface has now been modified to contain the code for creating an IteratorCreator. This is so the underlying TSDBStore can implement different ways of accessing the underlying shards rather than always having to access each shard individually (such as batch requests). Remove the show servers handling. This isn't a valid command in the open source version of InfluxDB anymore. The QueryManager interface is now built into QueryExecutor and is no longer necessary. The StatementExecutor and QueryExecutor split allows task management to much more easily be built into QueryExecutor rather than as a separate struct.
2016-03-31 22:12:29 +00:00
if ic == nil {
continue
}
ics = append(ics, ic)
}
return nil
}(); err != nil {
influxql.IteratorCreators(ics).Close()
return nil, err
}
return influxql.IteratorCreators(ics), nil
Cleanup QueryExecutor and split statement execution code The QueryExecutor had a lot of dead code made obsolete by the query engine refactor that has now been removed. The TSDBStore interface has also been cleaned up so we can have multiple implementations of this (such as a local and remote version). A StatementExecutor interface has been created for adding custom functionality to the QueryExecutor that may not be available in the open source version. The QueryExecutor delegate all statement execution to the StatementExecutor and the QueryExecutor will only keep track of housekeeping. Implementing additional queries is as simple as wrapping the cluster.StatementExecutor struct or replacing it with something completely different. The PointsWriter in the QueryExecutor has been changed to a simple interface that implements the one method needed by the query executor. This is to allow different PointsWriter implementations to be used by the QueryExecutor. It has also been moved into the StatementExecutor instead. The TSDBStore interface has now been modified to contain the code for creating an IteratorCreator. This is so the underlying TSDBStore can implement different ways of accessing the underlying shards rather than always having to access each shard individually (such as batch requests). Remove the show servers handling. This isn't a valid command in the open source version of InfluxDB anymore. The QueryManager interface is now built into QueryExecutor and is no longer necessary. The StatementExecutor and QueryExecutor split allows task management to much more easily be built into QueryExecutor rather than as a separate struct.
2016-03-31 22:12:29 +00:00
}
2016-02-10 20:04:18 +00:00
// WriteToShard writes a list of points to a shard identified by its ID.
func (s *Store) WriteToShard(shardID uint64, points []models.Point) error {
2015-06-03 17:46:18 +00:00
s.mu.RLock()
select {
case <-s.closing:
s.mu.RUnlock()
return ErrStoreClosed
default:
}
sh := s.shards[shardID]
if sh == nil {
s.mu.RUnlock()
return ErrShardNotFound
}
s.mu.RUnlock()
return sh.WritePoints(points)
}
2016-09-01 12:40:16 +00:00
// Measurements returns a slice of all measurements. Measurements accepts an
// optional condition expression. If cond is nil, then all measurements for the
// database will be returned.
func (s *Store) Measurements(database string, cond influxql.Expr) ([]string, error) {
2016-09-01 12:40:16 +00:00
s.mu.RLock()
shards := s.filterShards(byDatabase(database))
s.mu.RUnlock()
2016-09-01 12:40:16 +00:00
var m Measurements
for _, sh := range shards {
var mms Measurements
// Retrieve measurements from database index. Filter if condition specified.
if cond == nil {
mms = sh.Measurements()
} else {
var err error
mms, _, err = sh.MeasurementsByExpr(cond)
if err != nil {
return nil, err
}
}
2016-09-01 12:40:16 +00:00
m = append(m, mms...)
}
// Sort measurements by name.
2016-09-01 12:40:16 +00:00
sort.Sort(m)
2016-09-01 12:40:16 +00:00
measurements := make([]string, 0, len(m))
for _, m := range m {
measurements = append(measurements, m.Name)
}
return measurements, nil
}
2016-09-01 12:40:16 +00:00
// MeasurementSeriesCounts returns the number of measurements and series in all
// the shards' indices.
func (s *Store) MeasurementSeriesCounts(database string) (measuments int, series int) {
// TODO: implement me
return 0, 0
}
type TagValues struct {
Measurement string
Values []KeyValue
}
2016-12-31 05:12:37 +00:00
// TagValues returns the tag keys and values in the given database, matching the condition.
func (s *Store) TagValues(database string, cond influxql.Expr) ([]TagValues, error) {
if cond == nil {
return nil, errors.New("a condition is required")
}
measurementExpr := influxql.CloneExpr(cond)
measurementExpr = influxql.Reduce(influxql.RewriteExpr(measurementExpr, func(e influxql.Expr) influxql.Expr {
switch e := e.(type) {
case *influxql.BinaryExpr:
switch e.Op {
case influxql.EQ, influxql.NEQ, influxql.EQREGEX, influxql.NEQREGEX:
tag, ok := e.LHS.(*influxql.VarRef)
if !ok || tag.Val != "_name" {
return nil
}
}
}
return e
}), nil)
2016-09-01 12:40:16 +00:00
// Get all measurements for the shards we're interested in.
s.mu.RLock()
shards := s.filterShards(byDatabase(database))
s.mu.RUnlock()
var measures Measurements
for _, sh := range shards {
mms, ok, err := sh.MeasurementsByExpr(measurementExpr)
if err != nil {
return nil, err
} else if !ok {
// TODO(edd): can we simplify this so we don't have to check the
// ok value, and we can call sh.measurements with a shard filter
// instead?
mms = sh.Measurements()
}
measures = append(measures, mms...)
}
// If there are no measurements, return immediately.
2016-09-01 12:40:16 +00:00
if len(measures) == 0 {
return nil, nil
}
2016-09-01 12:40:16 +00:00
sort.Sort(measures)
filterExpr := influxql.CloneExpr(cond)
filterExpr = influxql.Reduce(influxql.RewriteExpr(filterExpr, func(e influxql.Expr) influxql.Expr {
switch e := e.(type) {
case *influxql.BinaryExpr:
switch e.Op {
case influxql.EQ, influxql.NEQ, influxql.EQREGEX, influxql.NEQREGEX:
tag, ok := e.LHS.(*influxql.VarRef)
if !ok || strings.HasPrefix(tag.Val, "_") {
return nil
}
}
}
return e
}), nil)
2016-09-01 12:40:16 +00:00
tagValues := make([]TagValues, len(measures))
for i, mm := range measures {
tagValues[i].Measurement = mm.Name
ids, err := mm.SeriesIDsAllOrByExpr(filterExpr)
if err != nil {
return nil, err
}
ss := mm.SeriesByIDSlice(ids)
// Determine a list of keys from condition.
keySet, ok, err := mm.TagKeysByExpr(cond)
if err != nil {
return nil, err
}
// Loop over all keys for each series.
m := make(map[KeyValue]struct{}, len(ss))
for _, series := range ss {
for _, t := range series.Tags {
if !ok {
// nop
} else if _, exists := keySet[string(t.Key)]; !exists {
continue
}
m[KeyValue{string(t.Key), string(t.Value)}] = struct{}{}
}
}
// Return an empty slice if there are no key/value matches.
if len(m) == 0 {
continue
}
// Sort key/value set.
a := make([]KeyValue, 0, len(m))
for kv := range m {
a = append(a, kv)
}
sort.Sort(KeyValues(a))
tagValues[i].Values = a
}
return tagValues, nil
}
2016-12-31 05:12:37 +00:00
// KeyValue holds a string key and a string value.
type KeyValue struct {
Key, Value string
}
2016-12-31 05:12:37 +00:00
// KeyValues is a sortable slice of KeyValue.
type KeyValues []KeyValue
2016-12-31 05:12:37 +00:00
// Len implements sort.Interface.
func (a KeyValues) Len() int { return len(a) }
// Swap implements sort.Interface.
func (a KeyValues) Swap(i, j int) { a[i], a[j] = a[j], a[i] }
2016-12-31 05:12:37 +00:00
// Less implements sort.Interface. Keys are compared before values.
func (a KeyValues) Less(i, j int) bool {
ki, kj := a[i].Key, a[j].Key
if ki == kj {
return a[i].Value < a[j].Value
}
return ki < kj
}
// filterShowSeriesResult will limit the number of series returned based on the limit and the offset.
// Unlike limit and offset on SELECT statements, the limit and offset don't apply to the number of Rows, but
// to the number of total Values returned, since each Value represents a unique series.
func (e *Store) filterShowSeriesResult(limit, offset int, rows models.Rows) models.Rows {
var filteredSeries models.Rows
seriesCount := 0
for _, r := range rows {
var currentSeries [][]interface{}
// filter the values
for _, v := range r.Values {
if seriesCount >= offset && seriesCount-offset < limit {
currentSeries = append(currentSeries, v)
}
seriesCount++
}
// only add the row back in if there are some values in it
if len(currentSeries) > 0 {
r.Values = currentSeries
filteredSeries = append(filteredSeries, r)
if seriesCount > limit+offset {
return filteredSeries
}
}
}
return filteredSeries
}
2016-09-14 13:55:44 +00:00
// decodeStorePath extracts the database and retention policy names
// from a given shard or WAL path.
2016-09-14 13:55:44 +00:00
func decodeStorePath(shardOrWALPath string) (database, retentionPolicy string) {
// shardOrWALPath format: /maybe/absolute/base/then/:database/:retentionPolicy/:nameOfShardOrWAL
// Discard the last part of the path (the shard name or the wal name).
path, _ := filepath.Split(filepath.Clean(shardOrWALPath))
// Extract the database and retention policy.
path, rp := filepath.Split(filepath.Clean(path))
_, db := filepath.Split(filepath.Clean(path))
return db, rp
}
// relativePath will expand out the full paths passed in and return
// the relative shard path from the store
func relativePath(storePath, shardPath string) (string, error) {
path, err := filepath.Abs(storePath)
if err != nil {
return "", fmt.Errorf("store abs path: %s", err)
}
fp, err := filepath.Abs(shardPath)
if err != nil {
return "", fmt.Errorf("file abs path: %s", err)
}
name, err := filepath.Rel(path, fp)
if err != nil {
return "", fmt.Errorf("file rel path: %s", err)
}
return name, nil
}
// measurementsFromSourcesOrDB returns a list of measurements from the
// sources passed in or, if sources is empty, a list of all
2016-09-01 12:40:16 +00:00
// measurement names from the measurement map passed in.
func measurementsFromSourcesOrDB(measurements map[string]*Measurement, sources ...influxql.Source) (Measurements, error) {
var all Measurements
if len(sources) > 0 {
for _, source := range sources {
if m, ok := source.(*influxql.Measurement); ok {
2016-09-01 12:40:16 +00:00
measurement := measurements[m.Name]
if measurement == nil {
continue
}
2016-09-01 12:40:16 +00:00
all = append(all, measurement)
} else {
return nil, errors.New("identifiers in FROM clause must be measurement names")
}
}
} else {
// No measurements specified in FROM clause so get all measurements that have series.
2016-09-01 12:40:16 +00:00
for _, m := range measurements {
if m.HasSeries() {
2016-09-01 12:40:16 +00:00
all = append(all, m)
}
}
}
2016-09-01 12:40:16 +00:00
sort.Sort(all)
2016-09-01 12:40:16 +00:00
return all, nil
}