Integrate WAL into engine
parent
e85999ed45
commit
81e0fbabeb
|
@ -3,7 +3,6 @@ package main
|
||||||
import (
|
import (
|
||||||
"context"
|
"context"
|
||||||
"fmt"
|
"fmt"
|
||||||
"io"
|
|
||||||
nethttp "net/http"
|
nethttp "net/http"
|
||||||
_ "net/http/pprof"
|
_ "net/http/pprof"
|
||||||
"os"
|
"os"
|
||||||
|
@ -57,7 +56,7 @@ var (
|
||||||
httpBindAddress string
|
httpBindAddress string
|
||||||
authorizationPath string
|
authorizationPath string
|
||||||
boltPath string
|
boltPath string
|
||||||
walPath string
|
natsPath string
|
||||||
developerMode bool
|
developerMode bool
|
||||||
enginePath string
|
enginePath string
|
||||||
)
|
)
|
||||||
|
@ -115,10 +114,11 @@ func init() {
|
||||||
os.Exit(1)
|
os.Exit(1)
|
||||||
}
|
}
|
||||||
|
|
||||||
platformCmd.Flags().StringVar(&walPath, "wal-path", filepath.Join(dir, "wal"), "path to persistent WAL files")
|
// TODO(edd): do we need NATS for anything?
|
||||||
viper.BindEnv("WAL_PATH")
|
platformCmd.Flags().StringVar(&natsPath, "nats-path", filepath.Join(dir, "nats"), "path to persistent NATS files")
|
||||||
if h := viper.GetString("WAL_PATH"); h != "" {
|
viper.BindEnv("NATS_PATH")
|
||||||
walPath = h
|
if h := viper.GetString("NATS_PATH"); h != "" {
|
||||||
|
natsPath = h
|
||||||
}
|
}
|
||||||
|
|
||||||
platformCmd.Flags().StringVar(&enginePath, "engine-path", filepath.Join(dir, "engine"), "path to persistent engine files")
|
platformCmd.Flags().StringVar(&enginePath, "engine-path", filepath.Join(dir, "engine"), "path to persistent engine files")
|
||||||
|
@ -211,14 +211,11 @@ func platformF(cmd *cobra.Command, args []string) {
|
||||||
|
|
||||||
// TODO(jeff): this block is hacky support for a storage engine. it is not intended to
|
// TODO(jeff): this block is hacky support for a storage engine. it is not intended to
|
||||||
// be a long term solution.
|
// be a long term solution.
|
||||||
var (
|
var storageQueryService query.ProxyQueryService
|
||||||
natsHandler nats.Handler
|
var pointsWriter storage.PointsWriter
|
||||||
storageQueryService query.ProxyQueryService
|
|
||||||
)
|
|
||||||
{
|
{
|
||||||
config := storage.NewConfig()
|
config := storage.NewConfig()
|
||||||
config.CacheSnapshotMemorySize = 0
|
config.EngineOptions.WALEnabled = true // Enable a disk-based WAL.
|
||||||
config.TraceLoggingEnabled = true
|
|
||||||
config.EngineOptions.Config = config.Config
|
config.EngineOptions.Config = config.Config
|
||||||
|
|
||||||
engine := storage.NewEngine(enginePath, config)
|
engine := storage.NewEngine(enginePath, config)
|
||||||
|
@ -229,12 +226,7 @@ func platformF(cmd *cobra.Command, args []string) {
|
||||||
os.Exit(1)
|
os.Exit(1)
|
||||||
}
|
}
|
||||||
|
|
||||||
engineHandler := nats.NewEngineHandler()
|
pointsWriter = engine
|
||||||
engineHandler.Logger = logger.With(zap.String("handler", "engine"))
|
|
||||||
engineHandler.Engine = engine
|
|
||||||
|
|
||||||
natsHandler = engineHandler
|
|
||||||
|
|
||||||
storageQueryService = query.ProxyQueryServiceBridge{
|
storageQueryService = query.ProxyQueryServiceBridge{
|
||||||
QueryService: query.QueryServiceBridge{
|
QueryService: query.QueryServiceBridge{
|
||||||
AsyncQueryService: &queryAdapter{
|
AsyncQueryService: &queryAdapter{
|
||||||
|
@ -297,7 +289,7 @@ func platformF(cmd *cobra.Command, args []string) {
|
||||||
signal.Notify(sigs, syscall.SIGTERM, os.Interrupt)
|
signal.Notify(sigs, syscall.SIGTERM, os.Interrupt)
|
||||||
|
|
||||||
// NATS streaming server
|
// NATS streaming server
|
||||||
natsServer := nats.NewServer(nats.Config{FilestoreDir: walPath})
|
natsServer := nats.NewServer(nats.Config{FilestoreDir: natsPath})
|
||||||
if err := natsServer.Open(); err != nil {
|
if err := natsServer.Open(); err != nil {
|
||||||
logger.Error("failed to start nats streaming server", zap.Error(err))
|
logger.Error("failed to start nats streaming server", zap.Error(err))
|
||||||
os.Exit(1)
|
os.Exit(1)
|
||||||
|
@ -316,11 +308,6 @@ func platformF(cmd *cobra.Command, args []string) {
|
||||||
os.Exit(1)
|
os.Exit(1)
|
||||||
}
|
}
|
||||||
|
|
||||||
if err := subscriber.Subscribe(IngressSubject, IngressGroup, natsHandler); err != nil {
|
|
||||||
logger.Error("failed to create nats subscriber", zap.Error(err))
|
|
||||||
os.Exit(1)
|
|
||||||
}
|
|
||||||
|
|
||||||
scraperScheduler, err := gather.NewScheduler(10, logger, scraperTargetSvc, publisher, subscriber, 0, 0)
|
scraperScheduler, err := gather.NewScheduler(10, logger, scraperTargetSvc, publisher, subscriber, 0, 0)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
logger.Error("failed to create scraper subscriber", zap.Error(err))
|
logger.Error("failed to create scraper subscriber", zap.Error(err))
|
||||||
|
@ -338,9 +325,7 @@ func platformF(cmd *cobra.Command, args []string) {
|
||||||
Logger: logger,
|
Logger: logger,
|
||||||
NewBucketService: source.NewBucketService,
|
NewBucketService: source.NewBucketService,
|
||||||
NewQueryService: source.NewQueryService,
|
NewQueryService: source.NewQueryService,
|
||||||
PublisherFn: func(r io.Reader) error {
|
PointsWriter: pointsWriter,
|
||||||
return publisher.Publish(IngressSubject, r)
|
|
||||||
},
|
|
||||||
AuthorizationService: authSvc,
|
AuthorizationService: authSvc,
|
||||||
BucketService: bucketSvc,
|
BucketService: bucketSvc,
|
||||||
SessionService: sessionSvc,
|
SessionService: sessionSvc,
|
||||||
|
|
|
@ -1,13 +1,13 @@
|
||||||
package http
|
package http
|
||||||
|
|
||||||
import (
|
import (
|
||||||
"io"
|
|
||||||
http "net/http"
|
http "net/http"
|
||||||
"strings"
|
"strings"
|
||||||
|
|
||||||
"github.com/influxdata/platform"
|
"github.com/influxdata/platform"
|
||||||
"github.com/influxdata/platform/chronograf/server"
|
"github.com/influxdata/platform/chronograf/server"
|
||||||
"github.com/influxdata/platform/query"
|
"github.com/influxdata/platform/query"
|
||||||
|
"github.com/influxdata/platform/storage"
|
||||||
"go.uber.org/zap"
|
"go.uber.org/zap"
|
||||||
)
|
)
|
||||||
|
|
||||||
|
@ -39,8 +39,8 @@ type APIBackend struct {
|
||||||
NewBucketService func(*platform.Source) (platform.BucketService, error)
|
NewBucketService func(*platform.Source) (platform.BucketService, error)
|
||||||
NewQueryService func(*platform.Source) (query.ProxyQueryService, error)
|
NewQueryService func(*platform.Source) (query.ProxyQueryService, error)
|
||||||
|
|
||||||
PublisherFn func(r io.Reader) error
|
|
||||||
|
PointsWriter storage.PointsWriter
|
||||||
AuthorizationService platform.AuthorizationService
|
AuthorizationService platform.AuthorizationService
|
||||||
BucketService platform.BucketService
|
BucketService platform.BucketService
|
||||||
SessionService platform.SessionService
|
SessionService platform.SessionService
|
||||||
|
@ -105,7 +105,7 @@ func NewAPIHandler(b *APIBackend) *APIHandler {
|
||||||
h.TaskHandler = NewTaskHandler(b.Logger)
|
h.TaskHandler = NewTaskHandler(b.Logger)
|
||||||
h.TaskHandler.TaskService = b.TaskService
|
h.TaskHandler.TaskService = b.TaskService
|
||||||
|
|
||||||
h.WriteHandler = NewWriteHandler(b.PublisherFn)
|
h.WriteHandler = NewWriteHandler(b.PointsWriter)
|
||||||
h.WriteHandler.AuthorizationService = b.AuthorizationService
|
h.WriteHandler.AuthorizationService = b.AuthorizationService
|
||||||
h.WriteHandler.OrganizationService = b.OrganizationService
|
h.WriteHandler.OrganizationService = b.OrganizationService
|
||||||
h.WriteHandler.BucketService = b.BucketService
|
h.WriteHandler.BucketService = b.BucketService
|
||||||
|
|
|
@ -1,11 +1,9 @@
|
||||||
package http
|
package http
|
||||||
|
|
||||||
import (
|
import (
|
||||||
"bytes"
|
|
||||||
"compress/gzip"
|
"compress/gzip"
|
||||||
"context"
|
"context"
|
||||||
"fmt"
|
"fmt"
|
||||||
"io"
|
|
||||||
"io/ioutil"
|
"io/ioutil"
|
||||||
"net/http"
|
"net/http"
|
||||||
|
|
||||||
|
@ -13,6 +11,7 @@ import (
|
||||||
pcontext "github.com/influxdata/platform/context"
|
pcontext "github.com/influxdata/platform/context"
|
||||||
"github.com/influxdata/platform/kit/errors"
|
"github.com/influxdata/platform/kit/errors"
|
||||||
"github.com/influxdata/platform/models"
|
"github.com/influxdata/platform/models"
|
||||||
|
"github.com/influxdata/platform/storage"
|
||||||
"github.com/influxdata/platform/tsdb"
|
"github.com/influxdata/platform/tsdb"
|
||||||
"github.com/julienschmidt/httprouter"
|
"github.com/julienschmidt/httprouter"
|
||||||
"go.uber.org/zap"
|
"go.uber.org/zap"
|
||||||
|
@ -28,15 +27,15 @@ type WriteHandler struct {
|
||||||
BucketService platform.BucketService
|
BucketService platform.BucketService
|
||||||
OrganizationService platform.OrganizationService
|
OrganizationService platform.OrganizationService
|
||||||
|
|
||||||
Publish func(io.Reader) error
|
PointsWriter storage.PointsWriter
|
||||||
}
|
}
|
||||||
|
|
||||||
// NewWriteHandler creates a new handler at /api/v2/write to receive line protocol.
|
// NewWriteHandler creates a new handler at /api/v2/write to receive line protocol.
|
||||||
func NewWriteHandler(publishFn func(io.Reader) error) *WriteHandler {
|
func NewWriteHandler(writer storage.PointsWriter) *WriteHandler {
|
||||||
h := &WriteHandler{
|
h := &WriteHandler{
|
||||||
Router: httprouter.New(),
|
Router: httprouter.New(),
|
||||||
Logger: zap.NewNop(),
|
Logger: zap.NewNop(),
|
||||||
Publish: publishFn,
|
PointsWriter: writer,
|
||||||
}
|
}
|
||||||
|
|
||||||
h.HandlerFunc("POST", "/api/v2/write", h.handleWrite)
|
h.HandlerFunc("POST", "/api/v2/write", h.handleWrite)
|
||||||
|
@ -137,7 +136,6 @@ func (h *WriteHandler) handleWrite(w http.ResponseWriter, r *http.Request) {
|
||||||
// TODO(jeff): we should be publishing with the org and bucket instead of
|
// TODO(jeff): we should be publishing with the org and bucket instead of
|
||||||
// parsing, rewriting, and publishing, but the interface isn't quite there yet.
|
// parsing, rewriting, and publishing, but the interface isn't quite there yet.
|
||||||
// be sure to remove this when it is there!
|
// be sure to remove this when it is there!
|
||||||
{
|
|
||||||
data, err := ioutil.ReadAll(in)
|
data, err := ioutil.ReadAll(in)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
logger.Info("Error reading body", zap.Error(err))
|
logger.Info("Error reading body", zap.Error(err))
|
||||||
|
@ -159,19 +157,11 @@ func (h *WriteHandler) handleWrite(w http.ResponseWriter, r *http.Request) {
|
||||||
return
|
return
|
||||||
}
|
}
|
||||||
|
|
||||||
var buf []byte
|
if err := h.PointsWriter.WritePoints(exploded); err != nil {
|
||||||
for _, pt := range exploded {
|
EncodeError(ctx, errors.BadRequestError(err.Error()), w)
|
||||||
buf = pt.AppendString(buf)
|
return
|
||||||
buf = append(buf, '\n')
|
|
||||||
}
|
}
|
||||||
|
|
||||||
in = ioutil.NopCloser(bytes.NewReader(buf))
|
|
||||||
}
|
|
||||||
|
|
||||||
if err := h.Publish(in); err != nil {
|
|
||||||
EncodeError(ctx, errors.BadRequestError(err.Error()), w)
|
|
||||||
return
|
|
||||||
}
|
|
||||||
|
|
||||||
w.WriteHeader(http.StatusNoContent)
|
w.WriteHeader(http.StatusNoContent)
|
||||||
}
|
}
|
||||||
|
|
|
@ -1,45 +0,0 @@
|
||||||
package nats
|
|
||||||
|
|
||||||
import (
|
|
||||||
"github.com/influxdata/platform/models"
|
|
||||||
"go.uber.org/zap"
|
|
||||||
)
|
|
||||||
|
|
||||||
type Engine interface {
|
|
||||||
WritePoints(points []models.Point) error
|
|
||||||
}
|
|
||||||
|
|
||||||
type EngineHandler struct {
|
|
||||||
Logger *zap.Logger
|
|
||||||
Engine Engine
|
|
||||||
}
|
|
||||||
|
|
||||||
func NewEngineHandler() *EngineHandler {
|
|
||||||
return &EngineHandler{}
|
|
||||||
}
|
|
||||||
|
|
||||||
func (lh *EngineHandler) Process(s Subscription, m Message) {
|
|
||||||
points, err := models.ParsePoints(m.Data())
|
|
||||||
if err != nil {
|
|
||||||
lh.Logger.Info("error parsing points", zap.Error(err))
|
|
||||||
m.Ack()
|
|
||||||
return
|
|
||||||
}
|
|
||||||
|
|
||||||
// TODO(jeff): This is super problematic. We need to only ack after the engine has flushed
|
|
||||||
// the cache. There's no real good way to either force that or to wait for it. It's also
|
|
||||||
// unclear what the semantics of Ack are with respect to previous messages. Oh well, for
|
|
||||||
// now just ack and if the process dies, you lose data!
|
|
||||||
|
|
||||||
if err := lh.Engine.WritePoints(points); err != nil {
|
|
||||||
// TODO(jeff): we need some idea of if this is a permanent or temporary error.
|
|
||||||
// For example, some sorts of PartialWriteErrors should not be retried. For
|
|
||||||
// now, just Ack.
|
|
||||||
lh.Logger.Info("error writing points", zap.Error(err))
|
|
||||||
m.Ack()
|
|
||||||
return
|
|
||||||
}
|
|
||||||
|
|
||||||
m.Ack()
|
|
||||||
return
|
|
||||||
}
|
|
|
@ -1,7 +1,9 @@
|
||||||
package storage
|
package storage
|
||||||
|
|
||||||
import (
|
import (
|
||||||
|
"bytes"
|
||||||
"context"
|
"context"
|
||||||
|
"errors"
|
||||||
"fmt"
|
"fmt"
|
||||||
"path/filepath"
|
"path/filepath"
|
||||||
"sync"
|
"sync"
|
||||||
|
@ -17,11 +19,16 @@ import (
|
||||||
// Static objects to prevent small allocs.
|
// Static objects to prevent small allocs.
|
||||||
var timeBytes = []byte("time")
|
var timeBytes = []byte("time")
|
||||||
|
|
||||||
|
// ErrEngineClosed is returned when a caller attempts to use the engine while
|
||||||
|
// it's closed.
|
||||||
|
var ErrEngineClosed = errors.New("engine is closed")
|
||||||
|
|
||||||
type Engine struct {
|
type Engine struct {
|
||||||
config Config
|
config Config
|
||||||
path string
|
path string
|
||||||
|
|
||||||
mu sync.RWMutex
|
mu sync.RWMutex
|
||||||
|
open bool
|
||||||
index *tsi1.Index
|
index *tsi1.Index
|
||||||
sfile *tsdb.SeriesFile
|
sfile *tsdb.SeriesFile
|
||||||
engine *tsm1.Engine
|
engine *tsm1.Engine
|
||||||
|
@ -56,7 +63,9 @@ func NewEngine(path string, c Config, options ...Option) *Engine {
|
||||||
e.index = index
|
e.index = index
|
||||||
|
|
||||||
// Initialise Engine
|
// Initialise Engine
|
||||||
engine := tsm1.NewEngine(0, tsdb.Index(e.index), filepath.Join(path, "data"), "remove-me-wal", e.sfile, c.EngineOptions)
|
// TODO(edd): should just be able to use the config values for data/wal.
|
||||||
|
engine := tsm1.NewEngine(0, tsdb.Index(e.index), filepath.Join(path, "data"), filepath.Join(path, "wal"), e.sfile, c.EngineOptions)
|
||||||
|
|
||||||
// TODO(edd): Once the tsdb.Engine abstraction is gone, this won't be needed.
|
// TODO(edd): Once the tsdb.Engine abstraction is gone, this won't be needed.
|
||||||
e.engine = engine.(*tsm1.Engine)
|
e.engine = engine.(*tsm1.Engine)
|
||||||
|
|
||||||
|
@ -81,6 +90,10 @@ func (e *Engine) Open() error {
|
||||||
e.mu.Lock()
|
e.mu.Lock()
|
||||||
defer e.mu.Unlock()
|
defer e.mu.Unlock()
|
||||||
|
|
||||||
|
if e.open {
|
||||||
|
return nil // no-op
|
||||||
|
}
|
||||||
|
|
||||||
if err := e.sfile.Open(); err != nil {
|
if err := e.sfile.Open(); err != nil {
|
||||||
return err
|
return err
|
||||||
}
|
}
|
||||||
|
@ -93,6 +106,7 @@ func (e *Engine) Open() error {
|
||||||
return err
|
return err
|
||||||
}
|
}
|
||||||
e.engine.SetCompactionsEnabled(true) // TODO(edd):is this needed?
|
e.engine.SetCompactionsEnabled(true) // TODO(edd):is this needed?
|
||||||
|
e.open = true
|
||||||
return nil
|
return nil
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -102,6 +116,10 @@ func (e *Engine) Close() error {
|
||||||
e.mu.Lock()
|
e.mu.Lock()
|
||||||
defer e.mu.Unlock()
|
defer e.mu.Unlock()
|
||||||
|
|
||||||
|
if !e.open {
|
||||||
|
return nil // no-op
|
||||||
|
}
|
||||||
|
e.open = false
|
||||||
if err := e.sfile.Close(); err != nil {
|
if err := e.sfile.Close(); err != nil {
|
||||||
return err
|
return err
|
||||||
}
|
}
|
||||||
|
@ -116,6 +134,9 @@ func (e *Engine) Close() error {
|
||||||
func (e *Engine) CreateSeriesCursor(ctx context.Context, req SeriesCursorRequest, cond influxql.Expr) (SeriesCursor, error) {
|
func (e *Engine) CreateSeriesCursor(ctx context.Context, req SeriesCursorRequest, cond influxql.Expr) (SeriesCursor, error) {
|
||||||
e.mu.RLock()
|
e.mu.RLock()
|
||||||
defer e.mu.RUnlock()
|
defer e.mu.RUnlock()
|
||||||
|
if !e.open {
|
||||||
|
return nil, ErrEngineClosed
|
||||||
|
}
|
||||||
// TODO(edd): remove IndexSet
|
// TODO(edd): remove IndexSet
|
||||||
return newSeriesCursor(req, tsdb.IndexSet{Indexes: []tsdb.Index{e.index}, SeriesFile: e.sfile}, cond)
|
return newSeriesCursor(req, tsdb.IndexSet{Indexes: []tsdb.Index{e.index}, SeriesFile: e.sfile}, cond)
|
||||||
}
|
}
|
||||||
|
@ -123,6 +144,9 @@ func (e *Engine) CreateSeriesCursor(ctx context.Context, req SeriesCursorRequest
|
||||||
func (e *Engine) CreateCursorIterator(ctx context.Context) (tsdb.CursorIterator, error) {
|
func (e *Engine) CreateCursorIterator(ctx context.Context) (tsdb.CursorIterator, error) {
|
||||||
e.mu.RLock()
|
e.mu.RLock()
|
||||||
defer e.mu.RUnlock()
|
defer e.mu.RUnlock()
|
||||||
|
if !e.open {
|
||||||
|
return nil, ErrEngineClosed
|
||||||
|
}
|
||||||
return e.engine.CreateCursorIterator(ctx)
|
return e.engine.CreateCursorIterator(ctx)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -138,12 +162,19 @@ func (e *Engine) WritePoints(points []models.Point) error {
|
||||||
for iter := collection.Iterator(); iter.Next(); {
|
for iter := collection.Iterator(); iter.Next(); {
|
||||||
tags := iter.Tags()
|
tags := iter.Tags()
|
||||||
|
|
||||||
|
if tags.Len() > 0 && bytes.Equal(tags[0].Key, tsdb.FieldKeyTagKeyBytes) && bytes.Equal(tags[0].Value, timeBytes) {
|
||||||
|
// Field key "time" is invalid
|
||||||
|
if collection.Reason == "" {
|
||||||
|
collection.Reason = fmt.Sprintf("invalid field key: input field %q is invalid", timeBytes)
|
||||||
|
}
|
||||||
|
collection.Dropped++
|
||||||
|
collection.DroppedKeys = append(collection.DroppedKeys, iter.Key())
|
||||||
|
}
|
||||||
|
|
||||||
// Filter out any tags with key equal to "time": they are invalid.
|
// Filter out any tags with key equal to "time": they are invalid.
|
||||||
if tags.Get(timeBytes) != nil {
|
if tags.Get(timeBytes) != nil {
|
||||||
if collection.Reason == "" {
|
if collection.Reason == "" {
|
||||||
collection.Reason = fmt.Sprintf(
|
collection.Reason = fmt.Sprintf("invalid tag key: input tag %q on measurement %q is invalid", timeBytes, iter.Name())
|
||||||
"invalid tag key: input tag %q on measurement %q is invalid",
|
|
||||||
timeBytes, iter.Name())
|
|
||||||
}
|
}
|
||||||
collection.Dropped++
|
collection.Dropped++
|
||||||
collection.DroppedKeys = append(collection.DroppedKeys, iter.Key())
|
collection.DroppedKeys = append(collection.DroppedKeys, iter.Key())
|
||||||
|
@ -168,6 +199,10 @@ func (e *Engine) WritePoints(points []models.Point) error {
|
||||||
e.mu.RLock()
|
e.mu.RLock()
|
||||||
defer e.mu.RUnlock()
|
defer e.mu.RUnlock()
|
||||||
|
|
||||||
|
if !e.open {
|
||||||
|
return ErrEngineClosed
|
||||||
|
}
|
||||||
|
|
||||||
// Add new series to the index and series file. Check for partial writes.
|
// Add new series to the index and series file. Check for partial writes.
|
||||||
if err := e.index.CreateSeriesListIfNotExists(collection); err != nil {
|
if err := e.index.CreateSeriesListIfNotExists(collection); err != nil {
|
||||||
// ignore PartialWriteErrors. The collection captures it.
|
// ignore PartialWriteErrors. The collection captures it.
|
||||||
|
@ -189,6 +224,9 @@ func (e *Engine) WritePoints(points []models.Point) error {
|
||||||
func (e *Engine) DeleteSeriesRangeWithPredicate(itr tsdb.SeriesIterator, fn func([]byte, models.Tags) (int64, int64, bool)) error {
|
func (e *Engine) DeleteSeriesRangeWithPredicate(itr tsdb.SeriesIterator, fn func([]byte, models.Tags) (int64, int64, bool)) error {
|
||||||
e.mu.RLock()
|
e.mu.RLock()
|
||||||
defer e.mu.RUnlock()
|
defer e.mu.RUnlock()
|
||||||
|
if !e.open {
|
||||||
|
return ErrEngineClosed
|
||||||
|
}
|
||||||
return e.engine.DeleteSeriesRangeWithPredicate(itr, fn)
|
return e.engine.DeleteSeriesRangeWithPredicate(itr, fn)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -196,6 +234,9 @@ func (e *Engine) DeleteSeriesRangeWithPredicate(itr tsdb.SeriesIterator, fn func
|
||||||
func (e *Engine) SeriesCardinality() int64 {
|
func (e *Engine) SeriesCardinality() int64 {
|
||||||
e.mu.RLock()
|
e.mu.RLock()
|
||||||
defer e.mu.RUnlock()
|
defer e.mu.RUnlock()
|
||||||
|
if !e.open {
|
||||||
|
return 0
|
||||||
|
}
|
||||||
return e.index.SeriesN()
|
return e.index.SeriesN()
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -209,5 +250,8 @@ func (e *Engine) Path() string {
|
||||||
func (e *Engine) ApplyFnToSeriesIDSet(fn func(*tsdb.SeriesIDSet)) {
|
func (e *Engine) ApplyFnToSeriesIDSet(fn func(*tsdb.SeriesIDSet)) {
|
||||||
e.mu.RLock()
|
e.mu.RLock()
|
||||||
defer e.mu.RUnlock()
|
defer e.mu.RUnlock()
|
||||||
|
if !e.open {
|
||||||
|
return
|
||||||
|
}
|
||||||
fn(e.index.SeriesIDSet())
|
fn(e.index.SeriesIDSet())
|
||||||
}
|
}
|
||||||
|
|
|
@ -0,0 +1,8 @@
|
||||||
|
package storage
|
||||||
|
|
||||||
|
import "github.com/influxdata/platform/models"
|
||||||
|
|
||||||
|
// PointsWriter describes the ability to write points into a storage engine.
|
||||||
|
type PointsWriter interface {
|
||||||
|
WritePoints([]models.Point) error
|
||||||
|
}
|
|
@ -9,10 +9,6 @@ var (
|
||||||
// ErrFieldTypeConflict is returned when a new field already exists with a different type.
|
// ErrFieldTypeConflict is returned when a new field already exists with a different type.
|
||||||
ErrFieldTypeConflict = errors.New("field type conflict")
|
ErrFieldTypeConflict = errors.New("field type conflict")
|
||||||
|
|
||||||
// ErrEngineClosed is returned when a caller attempts indirectly to
|
|
||||||
// access the shard's underlying engine.
|
|
||||||
ErrEngineClosed = errors.New("engine is closed")
|
|
||||||
|
|
||||||
// ErrShardDisabled is returned when a the shard is not available for
|
// ErrShardDisabled is returned when a the shard is not available for
|
||||||
// queries or writes.
|
// queries or writes.
|
||||||
ErrShardDisabled = errors.New("shard is disabled")
|
ErrShardDisabled = errors.New("shard is disabled")
|
||||||
|
|
|
@ -825,12 +825,14 @@ func (i *Index) SeriesSketches() (estimator.Sketch, estimator.Sketch, error) {
|
||||||
return i.sSketch, i.sTSketch, nil
|
return i.sSketch, i.sTSketch, nil
|
||||||
}
|
}
|
||||||
|
|
||||||
// Since indexes are not shared across shards, the count returned by SeriesN
|
// SeriesN returns the series cardinality in the index. It is the sum of all
|
||||||
// cannot be combined with other shard's results. If you need to count series
|
// partition cardinalities.
|
||||||
// across indexes then use either the database-wide series file, or merge the
|
|
||||||
// index-level bitsets or sketches.
|
|
||||||
func (i *Index) SeriesN() int64 {
|
func (i *Index) SeriesN() int64 {
|
||||||
return int64(i.SeriesIDSet().Cardinality())
|
var total int64
|
||||||
|
for _, p := range i.partitions {
|
||||||
|
total += int64(p.seriesIDSet.Cardinality())
|
||||||
|
}
|
||||||
|
return total
|
||||||
}
|
}
|
||||||
|
|
||||||
// HasTagKey returns true if tag key exists. It returns the first error
|
// HasTagKey returns true if tag key exists. It returns the first error
|
||||||
|
|
|
@ -186,13 +186,14 @@ func (s *Shard) close() error {
|
||||||
// ready determines if the Shard is ready for queries or writes.
|
// ready determines if the Shard is ready for queries or writes.
|
||||||
// It returns nil if ready, otherwise ErrShardClosed or ErrShardDisabled
|
// It returns nil if ready, otherwise ErrShardClosed or ErrShardDisabled
|
||||||
func (s *Shard) ready() error {
|
func (s *Shard) ready() error {
|
||||||
var err error
|
return nil // TODO(edd)remove
|
||||||
if s._engine == nil {
|
// var err error
|
||||||
err = ErrEngineClosed
|
// if s._engine == nil {
|
||||||
} else if !s.enabled {
|
// err = ErrEngineClosed
|
||||||
err = ErrShardDisabled
|
// } else if !s.enabled {
|
||||||
}
|
// err = ErrShardDisabled
|
||||||
return err
|
// }
|
||||||
|
// return err
|
||||||
}
|
}
|
||||||
|
|
||||||
// Engine returns a reference to the currently loaded engine.
|
// Engine returns a reference to the currently loaded engine.
|
||||||
|
|
|
@ -225,7 +225,7 @@ func NewEngine(id uint64, idx tsdb.Index, path string, walPath string, sfile *ts
|
||||||
traceLogger: logger,
|
traceLogger: logger,
|
||||||
traceLogging: opt.Config.TraceLoggingEnabled,
|
traceLogging: opt.Config.TraceLoggingEnabled,
|
||||||
|
|
||||||
WAL: NopWAL{},
|
WAL: NopWAL{},
|
||||||
Cache: cache,
|
Cache: cache,
|
||||||
|
|
||||||
FileStore: fs,
|
FileStore: fs,
|
||||||
|
@ -243,7 +243,7 @@ func NewEngine(id uint64, idx tsdb.Index, path string, walPath string, sfile *ts
|
||||||
}
|
}
|
||||||
|
|
||||||
if opt.WALEnabled {
|
if opt.WALEnabled {
|
||||||
wal := NewWAL(walPath)
|
wal := NewWAL(walPath)
|
||||||
wal.syncDelay = time.Duration(opt.Config.WALFsyncDelay)
|
wal.syncDelay = time.Duration(opt.Config.WALFsyncDelay)
|
||||||
e.WAL = wal
|
e.WAL = wal
|
||||||
}
|
}
|
||||||
|
@ -749,7 +749,7 @@ func (e *Engine) Close() error {
|
||||||
if err := e.FileStore.Close(); err != nil {
|
if err := e.FileStore.Close(); err != nil {
|
||||||
return err
|
return err
|
||||||
}
|
}
|
||||||
return e.WAL.Close()
|
return e.WAL.Close()
|
||||||
}
|
}
|
||||||
|
|
||||||
// WithLogger sets the logger for the engine.
|
// WithLogger sets the logger for the engine.
|
||||||
|
@ -1260,11 +1260,6 @@ func (e *Engine) WritePoints(points []models.Point) error {
|
||||||
iter := p.FieldIterator()
|
iter := p.FieldIterator()
|
||||||
t := p.Time().UnixNano()
|
t := p.Time().UnixNano()
|
||||||
for iter.Next() {
|
for iter.Next() {
|
||||||
// Skip fields name "time", they are illegal
|
|
||||||
if bytes.Equal(iter.FieldKey(), timeBytes) {
|
|
||||||
continue
|
|
||||||
}
|
|
||||||
|
|
||||||
keyBuf = append(keyBuf[:baseLen], iter.FieldKey()...)
|
keyBuf = append(keyBuf[:baseLen], iter.FieldKey()...)
|
||||||
|
|
||||||
if e.seriesTypeMap != nil {
|
if e.seriesTypeMap != nil {
|
||||||
|
@ -1553,9 +1548,9 @@ func (e *Engine) deleteSeriesRange(seriesKeys [][]byte, min, max int64) error {
|
||||||
e.Cache.DeleteRange(deleteKeys, min, max)
|
e.Cache.DeleteRange(deleteKeys, min, max)
|
||||||
|
|
||||||
// delete from the WAL
|
// delete from the WAL
|
||||||
if _, err := e.WAL.DeleteRange(deleteKeys, min, max); err != nil {
|
if _, err := e.WAL.DeleteRange(deleteKeys, min, max); err != nil {
|
||||||
return err
|
return err
|
||||||
}
|
}
|
||||||
|
|
||||||
// The series are deleted on disk, but the index may still say they exist.
|
// The series are deleted on disk, but the index may still say they exist.
|
||||||
// Depending on the the min,max time passed in, the series may or not actually
|
// Depending on the the min,max time passed in, the series may or not actually
|
||||||
|
@ -2203,7 +2198,7 @@ func (e *Engine) reloadCache() error {
|
||||||
return err
|
return err
|
||||||
}
|
}
|
||||||
|
|
||||||
e.traceLogger.Info("Reloaded WAL cache",zap.String("path", e.WAL.Path()), zap.Duration("duration", time.Since(now)))
|
e.traceLogger.Info("Reloaded WAL cache", zap.String("path", e.WAL.Path()), zap.Duration("duration", time.Since(now)))
|
||||||
return nil
|
return nil
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
Loading…
Reference in New Issue