refactor Points and Rows to dedicated packages
parent
6d4319d244
commit
d19a510ad2
client
cmd/influxd/restore
models
monitor
services
collectd
continuous_querier
udp
|
@ -14,7 +14,7 @@ import (
|
|||
"time"
|
||||
|
||||
"github.com/influxdb/influxdb/influxql"
|
||||
"github.com/influxdb/influxdb/tsdb"
|
||||
"github.com/influxdb/influxdb/models"
|
||||
)
|
||||
|
||||
const (
|
||||
|
@ -460,7 +460,7 @@ func (p *Point) MarshalJSON() ([]byte, error) {
|
|||
}
|
||||
|
||||
func (p *Point) MarshalString() string {
|
||||
pt := tsdb.NewPoint(p.Measurement, p.Tags, p.Fields, p.Time)
|
||||
pt := models.NewPoint(p.Measurement, p.Tags, p.Fields, p.Time)
|
||||
if p.Precision == "" || p.Precision == "ns" || p.Precision == "n" {
|
||||
return pt.String()
|
||||
}
|
||||
|
|
|
@ -12,6 +12,7 @@ import (
|
|||
|
||||
"github.com/influxdb/influxdb"
|
||||
"github.com/influxdb/influxdb/meta"
|
||||
"github.com/influxdb/influxdb/models"
|
||||
"github.com/influxdb/influxdb/tsdb"
|
||||
)
|
||||
|
||||
|
@ -94,15 +95,15 @@ type PointsWriter struct {
|
|||
|
||||
TSDBStore interface {
|
||||
CreateShard(database, retentionPolicy string, shardID uint64) error
|
||||
WriteToShard(shardID uint64, points []tsdb.Point) error
|
||||
WriteToShard(shardID uint64, points []models.Point) error
|
||||
}
|
||||
|
||||
ShardWriter interface {
|
||||
WriteShard(shardID, ownerID uint64, points []tsdb.Point) error
|
||||
WriteShard(shardID, ownerID uint64, points []models.Point) error
|
||||
}
|
||||
|
||||
HintedHandoff interface {
|
||||
WriteShard(shardID, ownerID uint64, points []tsdb.Point) error
|
||||
WriteShard(shardID, ownerID uint64, points []models.Point) error
|
||||
}
|
||||
|
||||
statMap *expvar.Map
|
||||
|
@ -120,23 +121,23 @@ func NewPointsWriter() *PointsWriter {
|
|||
|
||||
// ShardMapping contains a mapping of a shards to a points.
|
||||
type ShardMapping struct {
|
||||
Points map[uint64][]tsdb.Point // The points associated with a shard ID
|
||||
Points map[uint64][]models.Point // The points associated with a shard ID
|
||||
Shards map[uint64]*meta.ShardInfo // The shards that have been mapped, keyed by shard ID
|
||||
}
|
||||
|
||||
// NewShardMapping creates an empty ShardMapping
|
||||
func NewShardMapping() *ShardMapping {
|
||||
return &ShardMapping{
|
||||
Points: map[uint64][]tsdb.Point{},
|
||||
Points: map[uint64][]models.Point{},
|
||||
Shards: map[uint64]*meta.ShardInfo{},
|
||||
}
|
||||
}
|
||||
|
||||
// MapPoint maps a point to shard
|
||||
func (s *ShardMapping) MapPoint(shardInfo *meta.ShardInfo, p tsdb.Point) {
|
||||
func (s *ShardMapping) MapPoint(shardInfo *meta.ShardInfo, p models.Point) {
|
||||
points, ok := s.Points[shardInfo.ID]
|
||||
if !ok {
|
||||
s.Points[shardInfo.ID] = []tsdb.Point{p}
|
||||
s.Points[shardInfo.ID] = []models.Point{p}
|
||||
} else {
|
||||
s.Points[shardInfo.ID] = append(points, p)
|
||||
}
|
||||
|
@ -221,7 +222,7 @@ func (w *PointsWriter) WritePoints(p *WritePointsRequest) error {
|
|||
// as one fails.
|
||||
ch := make(chan error, len(shardMappings.Points))
|
||||
for shardID, points := range shardMappings.Points {
|
||||
go func(shard *meta.ShardInfo, database, retentionPolicy string, points []tsdb.Point) {
|
||||
go func(shard *meta.ShardInfo, database, retentionPolicy string, points []models.Point) {
|
||||
ch <- w.writeToShard(shard, p.Database, p.RetentionPolicy, p.ConsistencyLevel, points)
|
||||
}(shardMappings.Shards[shardID], p.Database, p.RetentionPolicy, points)
|
||||
}
|
||||
|
@ -242,7 +243,7 @@ func (w *PointsWriter) WritePoints(p *WritePointsRequest) error {
|
|||
// writeToShards writes points to a shard and ensures a write consistency level has been met. If the write
|
||||
// partially succeeds, ErrPartialWrite is returned.
|
||||
func (w *PointsWriter) writeToShard(shard *meta.ShardInfo, database, retentionPolicy string,
|
||||
consistency ConsistencyLevel, points []tsdb.Point) error {
|
||||
consistency ConsistencyLevel, points []models.Point) error {
|
||||
// The required number of writes to achieve the requested consistency level
|
||||
required := len(shard.Owners)
|
||||
switch consistency {
|
||||
|
@ -260,7 +261,7 @@ func (w *PointsWriter) writeToShard(shard *meta.ShardInfo, database, retentionPo
|
|||
ch := make(chan *AsyncWriteResult, len(shard.Owners))
|
||||
|
||||
for _, owner := range shard.Owners {
|
||||
go func(shardID uint64, owner meta.ShardOwner, points []tsdb.Point) {
|
||||
go func(shardID uint64, owner meta.ShardOwner, points []models.Point) {
|
||||
if w.MetaStore.NodeID() == owner.NodeID {
|
||||
w.statMap.Add(statPointWriteReqLocal, int64(len(points)))
|
||||
|
||||
|
|
|
@ -9,7 +9,7 @@ import (
|
|||
|
||||
"github.com/influxdb/influxdb/cluster"
|
||||
"github.com/influxdb/influxdb/meta"
|
||||
"github.com/influxdb/influxdb/tsdb"
|
||||
"github.com/influxdb/influxdb/models"
|
||||
)
|
||||
|
||||
// Ensures the points writer maps a single point to a single shard.
|
||||
|
@ -282,7 +282,7 @@ func TestPointsWriter_WritePoints(t *testing.T) {
|
|||
// lock on the write increment since these functions get called in parallel
|
||||
var mu sync.Mutex
|
||||
sw := &fakeShardWriter{
|
||||
ShardWriteFn: func(shardID, nodeID uint64, points []tsdb.Point) error {
|
||||
ShardWriteFn: func(shardID, nodeID uint64, points []models.Point) error {
|
||||
mu.Lock()
|
||||
defer mu.Unlock()
|
||||
return theTest.err[int(nodeID)-1]
|
||||
|
@ -290,7 +290,7 @@ func TestPointsWriter_WritePoints(t *testing.T) {
|
|||
}
|
||||
|
||||
store := &fakeStore{
|
||||
WriteFn: func(shardID uint64, points []tsdb.Point) error {
|
||||
WriteFn: func(shardID uint64, points []models.Point) error {
|
||||
mu.Lock()
|
||||
defer mu.Unlock()
|
||||
return theTest.err[0]
|
||||
|
@ -298,7 +298,7 @@ func TestPointsWriter_WritePoints(t *testing.T) {
|
|||
}
|
||||
|
||||
hh := &fakeShardWriter{
|
||||
ShardWriteFn: func(shardID, nodeID uint64, points []tsdb.Point) error {
|
||||
ShardWriteFn: func(shardID, nodeID uint64, points []models.Point) error {
|
||||
return nil
|
||||
},
|
||||
}
|
||||
|
@ -331,19 +331,19 @@ func TestPointsWriter_WritePoints(t *testing.T) {
|
|||
var shardID uint64
|
||||
|
||||
type fakeShardWriter struct {
|
||||
ShardWriteFn func(shardID, nodeID uint64, points []tsdb.Point) error
|
||||
ShardWriteFn func(shardID, nodeID uint64, points []models.Point) error
|
||||
}
|
||||
|
||||
func (f *fakeShardWriter) WriteShard(shardID, nodeID uint64, points []tsdb.Point) error {
|
||||
func (f *fakeShardWriter) WriteShard(shardID, nodeID uint64, points []models.Point) error {
|
||||
return f.ShardWriteFn(shardID, nodeID, points)
|
||||
}
|
||||
|
||||
type fakeStore struct {
|
||||
WriteFn func(shardID uint64, points []tsdb.Point) error
|
||||
WriteFn func(shardID uint64, points []models.Point) error
|
||||
CreateShardfn func(database, retentionPolicy string, shardID uint64) error
|
||||
}
|
||||
|
||||
func (f *fakeStore) WriteToShard(shardID uint64, points []tsdb.Point) error {
|
||||
func (f *fakeStore) WriteToShard(shardID uint64, points []models.Point) error {
|
||||
return f.WriteFn(shardID, points)
|
||||
}
|
||||
|
||||
|
|
|
@ -6,7 +6,7 @@ import (
|
|||
|
||||
"github.com/gogo/protobuf/proto"
|
||||
"github.com/influxdb/influxdb/cluster/internal"
|
||||
"github.com/influxdb/influxdb/tsdb"
|
||||
"github.com/influxdb/influxdb/models"
|
||||
)
|
||||
|
||||
//go:generate protoc --gogo_out=. internal/data.proto
|
||||
|
@ -79,12 +79,12 @@ type WritePointsRequest struct {
|
|||
Database string
|
||||
RetentionPolicy string
|
||||
ConsistencyLevel ConsistencyLevel
|
||||
Points []tsdb.Point
|
||||
Points []models.Point
|
||||
}
|
||||
|
||||
// AddPoint adds a point to the WritePointRequest with field name 'value'
|
||||
func (w *WritePointsRequest) AddPoint(name string, value interface{}, timestamp time.Time, tags map[string]string) {
|
||||
w.Points = append(w.Points, tsdb.NewPoint(
|
||||
w.Points = append(w.Points, models.NewPoint(
|
||||
name, tags, map[string]interface{}{"value": value}, timestamp,
|
||||
))
|
||||
}
|
||||
|
@ -102,15 +102,15 @@ type WriteShardResponse struct {
|
|||
func (w *WriteShardRequest) SetShardID(id uint64) { w.pb.ShardID = &id }
|
||||
func (w *WriteShardRequest) ShardID() uint64 { return w.pb.GetShardID() }
|
||||
|
||||
func (w *WriteShardRequest) Points() []tsdb.Point { return w.unmarshalPoints() }
|
||||
func (w *WriteShardRequest) Points() []models.Point { return w.unmarshalPoints() }
|
||||
|
||||
func (w *WriteShardRequest) AddPoint(name string, value interface{}, timestamp time.Time, tags map[string]string) {
|
||||
w.AddPoints([]tsdb.Point{tsdb.NewPoint(
|
||||
w.AddPoints([]models.Point{models.NewPoint(
|
||||
name, tags, map[string]interface{}{"value": value}, timestamp,
|
||||
)})
|
||||
}
|
||||
|
||||
func (w *WriteShardRequest) AddPoints(points []tsdb.Point) {
|
||||
func (w *WriteShardRequest) AddPoints(points []models.Point) {
|
||||
for _, p := range points {
|
||||
w.pb.Points = append(w.pb.Points, []byte(p.String()))
|
||||
}
|
||||
|
@ -129,10 +129,10 @@ func (w *WriteShardRequest) UnmarshalBinary(buf []byte) error {
|
|||
return nil
|
||||
}
|
||||
|
||||
func (w *WriteShardRequest) unmarshalPoints() []tsdb.Point {
|
||||
points := make([]tsdb.Point, len(w.pb.GetPoints()))
|
||||
func (w *WriteShardRequest) unmarshalPoints() []models.Point {
|
||||
points := make([]models.Point, len(w.pb.GetPoints()))
|
||||
for i, p := range w.pb.GetPoints() {
|
||||
pt, err := tsdb.ParsePoints(p)
|
||||
pt, err := models.ParsePoints(p)
|
||||
if err != nil {
|
||||
// A error here means that one node parsed the point correctly but sent an
|
||||
// unparseable version to another node. We could log and drop the point and allow
|
||||
|
|
|
@ -13,6 +13,7 @@ import (
|
|||
|
||||
"github.com/influxdb/influxdb/influxql"
|
||||
"github.com/influxdb/influxdb/meta"
|
||||
"github.com/influxdb/influxdb/models"
|
||||
"github.com/influxdb/influxdb/tsdb"
|
||||
)
|
||||
|
||||
|
@ -37,7 +38,7 @@ type Service struct {
|
|||
|
||||
TSDBStore interface {
|
||||
CreateShard(database, policy string, shardID uint64) error
|
||||
WriteToShard(shardID uint64, points []tsdb.Point) error
|
||||
WriteToShard(shardID uint64, points []models.Point) error
|
||||
CreateMapper(shardID uint64, stmt influxql.Statement, chunkSize int) (tsdb.Mapper, error)
|
||||
}
|
||||
|
||||
|
|
|
@ -8,6 +8,7 @@ import (
|
|||
"github.com/influxdb/influxdb/cluster"
|
||||
"github.com/influxdb/influxdb/influxql"
|
||||
"github.com/influxdb/influxdb/meta"
|
||||
"github.com/influxdb/influxdb/models"
|
||||
"github.com/influxdb/influxdb/tcp"
|
||||
"github.com/influxdb/influxdb/tsdb"
|
||||
)
|
||||
|
@ -27,12 +28,12 @@ type testService struct {
|
|||
nodeID uint64
|
||||
ln net.Listener
|
||||
muxln net.Listener
|
||||
writeShardFunc func(shardID uint64, points []tsdb.Point) error
|
||||
writeShardFunc func(shardID uint64, points []models.Point) error
|
||||
createShardFunc func(database, policy string, shardID uint64) error
|
||||
createMapperFunc func(shardID uint64, stmt influxql.Statement, chunkSize int) (tsdb.Mapper, error)
|
||||
}
|
||||
|
||||
func newTestWriteService(f func(shardID uint64, points []tsdb.Point) error) testService {
|
||||
func newTestWriteService(f func(shardID uint64, points []models.Point) error) testService {
|
||||
ln, err := net.Listen("tcp", "127.0.0.1:0")
|
||||
if err != nil {
|
||||
panic(err)
|
||||
|
@ -59,10 +60,10 @@ type serviceResponses []serviceResponse
|
|||
type serviceResponse struct {
|
||||
shardID uint64
|
||||
ownerID uint64
|
||||
points []tsdb.Point
|
||||
points []models.Point
|
||||
}
|
||||
|
||||
func (t testService) WriteToShard(shardID uint64, points []tsdb.Point) error {
|
||||
func (t testService) WriteToShard(shardID uint64, points []models.Point) error {
|
||||
return t.writeShardFunc(shardID, points)
|
||||
}
|
||||
|
||||
|
@ -74,7 +75,7 @@ func (t testService) CreateMapper(shardID uint64, stmt influxql.Statement, chunk
|
|||
return t.createMapperFunc(shardID, stmt, chunkSize)
|
||||
}
|
||||
|
||||
func writeShardSuccess(shardID uint64, points []tsdb.Point) error {
|
||||
func writeShardSuccess(shardID uint64, points []models.Point) error {
|
||||
responses <- &serviceResponse{
|
||||
shardID: shardID,
|
||||
points: points,
|
||||
|
@ -82,7 +83,7 @@ func writeShardSuccess(shardID uint64, points []tsdb.Point) error {
|
|||
return nil
|
||||
}
|
||||
|
||||
func writeShardFail(shardID uint64, points []tsdb.Point) error {
|
||||
func writeShardFail(shardID uint64, points []models.Point) error {
|
||||
return fmt.Errorf("failed to write")
|
||||
}
|
||||
|
||||
|
|
|
@ -6,7 +6,7 @@ import (
|
|||
"time"
|
||||
|
||||
"github.com/influxdb/influxdb/meta"
|
||||
"github.com/influxdb/influxdb/tsdb"
|
||||
"github.com/influxdb/influxdb/models"
|
||||
"gopkg.in/fatih/pool.v2"
|
||||
)
|
||||
|
||||
|
@ -35,7 +35,7 @@ func NewShardWriter(timeout time.Duration) *ShardWriter {
|
|||
}
|
||||
}
|
||||
|
||||
func (w *ShardWriter) WriteShard(shardID, ownerID uint64, points []tsdb.Point) error {
|
||||
func (w *ShardWriter) WriteShard(shardID, ownerID uint64, points []models.Point) error {
|
||||
c, err := w.dial(ownerID)
|
||||
if err != nil {
|
||||
return err
|
||||
|
|
|
@ -7,7 +7,7 @@ import (
|
|||
"time"
|
||||
|
||||
"github.com/influxdb/influxdb/cluster"
|
||||
"github.com/influxdb/influxdb/tsdb"
|
||||
"github.com/influxdb/influxdb/models"
|
||||
)
|
||||
|
||||
// Ensure the shard writer can successful write a single request.
|
||||
|
@ -27,8 +27,8 @@ func TestShardWriter_WriteShard_Success(t *testing.T) {
|
|||
|
||||
// Build a single point.
|
||||
now := time.Now()
|
||||
var points []tsdb.Point
|
||||
points = append(points, tsdb.NewPoint("cpu", tsdb.Tags{"host": "server01"}, map[string]interface{}{"value": int64(100)}, now))
|
||||
var points []models.Point
|
||||
points = append(points, models.NewPoint("cpu", models.Tags{"host": "server01"}, map[string]interface{}{"value": int64(100)}, now))
|
||||
|
||||
// Write to shard and close.
|
||||
if err := w.WriteShard(1, 2, points); err != nil {
|
||||
|
@ -74,8 +74,8 @@ func TestShardWriter_WriteShard_Multiple(t *testing.T) {
|
|||
|
||||
// Build a single point.
|
||||
now := time.Now()
|
||||
var points []tsdb.Point
|
||||
points = append(points, tsdb.NewPoint("cpu", tsdb.Tags{"host": "server01"}, map[string]interface{}{"value": int64(100)}, now))
|
||||
var points []models.Point
|
||||
points = append(points, models.NewPoint("cpu", models.Tags{"host": "server01"}, map[string]interface{}{"value": int64(100)}, now))
|
||||
|
||||
// Write to shard twice and close.
|
||||
if err := w.WriteShard(1, 2, points); err != nil {
|
||||
|
@ -124,9 +124,9 @@ func TestShardWriter_WriteShard_Error(t *testing.T) {
|
|||
|
||||
shardID := uint64(1)
|
||||
ownerID := uint64(2)
|
||||
var points []tsdb.Point
|
||||
points = append(points, tsdb.NewPoint(
|
||||
"cpu", tsdb.Tags{"host": "server01"}, map[string]interface{}{"value": int64(100)}, now,
|
||||
var points []models.Point
|
||||
points = append(points, models.NewPoint(
|
||||
"cpu", models.Tags{"host": "server01"}, map[string]interface{}{"value": int64(100)}, now,
|
||||
))
|
||||
|
||||
if err := w.WriteShard(shardID, ownerID, points); err == nil || err.Error() != "error code 1: write shard 1: failed to write" {
|
||||
|
@ -152,9 +152,9 @@ func TestShardWriter_Write_ErrDialTimeout(t *testing.T) {
|
|||
|
||||
shardID := uint64(1)
|
||||
ownerID := uint64(2)
|
||||
var points []tsdb.Point
|
||||
points = append(points, tsdb.NewPoint(
|
||||
"cpu", tsdb.Tags{"host": "server01"}, map[string]interface{}{"value": int64(100)}, now,
|
||||
var points []models.Point
|
||||
points = append(points, models.NewPoint(
|
||||
"cpu", models.Tags{"host": "server01"}, map[string]interface{}{"value": int64(100)}, now,
|
||||
))
|
||||
|
||||
if err, exp := w.WriteShard(shardID, ownerID, points), "i/o timeout"; err == nil || !strings.Contains(err.Error(), exp) {
|
||||
|
@ -175,9 +175,9 @@ func TestShardWriter_Write_ErrReadTimeout(t *testing.T) {
|
|||
|
||||
shardID := uint64(1)
|
||||
ownerID := uint64(2)
|
||||
var points []tsdb.Point
|
||||
points = append(points, tsdb.NewPoint(
|
||||
"cpu", tsdb.Tags{"host": "server01"}, map[string]interface{}{"value": int64(100)}, now,
|
||||
var points []models.Point
|
||||
points = append(points, models.NewPoint(
|
||||
"cpu", models.Tags{"host": "server01"}, map[string]interface{}{"value": int64(100)}, now,
|
||||
))
|
||||
|
||||
if err := w.WriteShard(shardID, ownerID, points); err == nil || !strings.Contains(err.Error(), "i/o timeout") {
|
||||
|
|
|
@ -55,7 +55,7 @@ func TestRestoreCommand(t *testing.T) {
|
|||
if err := s.CreateDatabase("db"); err != nil {
|
||||
t.Fatalf("cannot create database: %s", err)
|
||||
}
|
||||
if index, err := s.WriteSeries("db", "default", []tsdb.Point{tsdb.NewPoint("cpu", nil, map[string]interface{}{"value": float64(100)}, now)}); err != nil {
|
||||
if index, err := s.WriteSeries("db", "default", []models.Point{tsdb.NewPoint("cpu", nil, map[string]interface{}{"value": float64(100)}, now)}); err != nil {
|
||||
t.Fatalf("cannot write series: %s", err)
|
||||
} else if err = s.Sync(1, index); err != nil {
|
||||
t.Fatalf("shard sync: %s", err)
|
||||
|
@ -107,7 +107,7 @@ func TestRestoreCommand(t *testing.T) {
|
|||
if err := s.CreateDatabase("newdb"); err != nil {
|
||||
t.Fatalf("cannot create new database: %s", err)
|
||||
}
|
||||
if index, err := s.WriteSeries("newdb", "default", []tsdb.Point{tsdb.NewPoint("mem", nil, map[string]interface{}{"value": float64(1000)}, now)}); err != nil {
|
||||
if index, err := s.WriteSeries("newdb", "default", []models.Point{tsdb.NewPoint("mem", nil, map[string]interface{}{"value": float64(1000)}, now)}); err != nil {
|
||||
t.Fatalf("cannot write new series: %s", err)
|
||||
} else if err = s.Sync(2, index); err != nil {
|
||||
t.Fatalf("shard sync: %s", err)
|
||||
|
|
|
@ -1,14 +1,27 @@
|
|||
package tsdb
|
||||
package models
|
||||
|
||||
import (
|
||||
"bytes"
|
||||
"fmt"
|
||||
"hash/fnv"
|
||||
"regexp"
|
||||
"sort"
|
||||
"strconv"
|
||||
"strings"
|
||||
"time"
|
||||
|
||||
"github.com/influxdb/influxdb/pkg/escape"
|
||||
)
|
||||
|
||||
var (
|
||||
measurementEscapeCodes = map[byte][]byte{
|
||||
',': []byte(`\,`),
|
||||
' ': []byte(`\ `),
|
||||
}
|
||||
|
||||
tagEscapeCodes = map[byte][]byte{
|
||||
',': []byte(`\,`),
|
||||
' ': []byte(`\ `),
|
||||
'=': []byte(`\=`),
|
||||
}
|
||||
)
|
||||
|
||||
// Point defines the values that will be written to the database
|
||||
|
@ -92,36 +105,7 @@ const (
|
|||
minFloat64Digits = 27
|
||||
)
|
||||
|
||||
var (
|
||||
// Compile the regex that detects unquoted double quote sequences
|
||||
quoteReplacer = regexp.MustCompile(`([^\\])"`)
|
||||
|
||||
escapeCodes = map[byte][]byte{
|
||||
',': []byte(`\,`),
|
||||
'"': []byte(`\"`),
|
||||
' ': []byte(`\ `),
|
||||
'=': []byte(`\=`),
|
||||
}
|
||||
|
||||
escapeCodesStr = map[string]string{}
|
||||
|
||||
measurementEscapeCodes = map[byte][]byte{
|
||||
',': []byte(`\,`),
|
||||
' ': []byte(`\ `),
|
||||
}
|
||||
|
||||
tagEscapeCodes = map[byte][]byte{
|
||||
',': []byte(`\,`),
|
||||
' ': []byte(`\ `),
|
||||
'=': []byte(`\=`),
|
||||
}
|
||||
)
|
||||
|
||||
func init() {
|
||||
for k, v := range escapeCodes {
|
||||
escapeCodesStr[string(k)] = string(v)
|
||||
}
|
||||
}
|
||||
var ()
|
||||
|
||||
func ParsePointsString(buf string) ([]Point, error) {
|
||||
return ParsePoints([]byte(buf))
|
||||
|
@ -421,7 +405,7 @@ func less(buf []byte, indices []int, i, j int) bool {
|
|||
}
|
||||
|
||||
func isFieldEscapeChar(b byte) bool {
|
||||
for c := range escapeCodes {
|
||||
for c := range escape.Codes {
|
||||
if c == b {
|
||||
return true
|
||||
}
|
||||
|
@ -903,62 +887,6 @@ func unescapeTag(in []byte) []byte {
|
|||
return in
|
||||
}
|
||||
|
||||
func escape(in []byte) []byte {
|
||||
for b, esc := range escapeCodes {
|
||||
in = bytes.Replace(in, []byte{b}, esc, -1)
|
||||
}
|
||||
return in
|
||||
}
|
||||
|
||||
func escapeString(in string) string {
|
||||
for b, esc := range escapeCodesStr {
|
||||
in = strings.Replace(in, b, esc, -1)
|
||||
}
|
||||
return in
|
||||
}
|
||||
|
||||
func unescape(in []byte) []byte {
|
||||
i := 0
|
||||
inLen := len(in)
|
||||
var out []byte
|
||||
|
||||
for {
|
||||
if i >= inLen {
|
||||
break
|
||||
}
|
||||
if in[i] == '\\' && i+1 < inLen {
|
||||
switch in[i+1] {
|
||||
case ',':
|
||||
out = append(out, ',')
|
||||
i += 2
|
||||
continue
|
||||
case '"':
|
||||
out = append(out, '"')
|
||||
i += 2
|
||||
continue
|
||||
case ' ':
|
||||
out = append(out, ' ')
|
||||
i += 2
|
||||
continue
|
||||
case '=':
|
||||
out = append(out, '=')
|
||||
i += 2
|
||||
continue
|
||||
}
|
||||
}
|
||||
out = append(out, in[i])
|
||||
i += 1
|
||||
}
|
||||
return out
|
||||
}
|
||||
|
||||
func unescapeString(in string) string {
|
||||
for b, esc := range escapeCodesStr {
|
||||
in = strings.Replace(in, esc, b, -1)
|
||||
}
|
||||
return in
|
||||
}
|
||||
|
||||
// escapeStringField returns a copy of in with any double quotes or
|
||||
// backslashes with escaped values
|
||||
func escapeStringField(in string) string {
|
||||
|
@ -1048,7 +976,7 @@ func (p *point) Name() string {
|
|||
if p.cachedName != "" {
|
||||
return p.cachedName
|
||||
}
|
||||
p.cachedName = string(unescape(p.name()))
|
||||
p.cachedName = string(escape.Unescape(p.name()))
|
||||
return p.cachedName
|
||||
}
|
||||
|
||||
|
@ -1277,7 +1205,7 @@ func newFieldsFromBinary(buf []byte) Fields {
|
|||
if len(name) == 0 {
|
||||
continue
|
||||
}
|
||||
name = unescape(name)
|
||||
name = escape.Unescape(name)
|
||||
|
||||
i, valueBuf = scanFieldValue(buf, i+1)
|
||||
if len(valueBuf) == 0 {
|
||||
|
@ -1327,7 +1255,7 @@ func (p Fields) MarshalBinary() []byte {
|
|||
|
||||
for _, k := range keys {
|
||||
v := p[k]
|
||||
b = append(b, []byte(escapeString(k))...)
|
||||
b = append(b, []byte(escape.String(k))...)
|
||||
b = append(b, '=')
|
||||
switch t := v.(type) {
|
||||
case int:
|
File diff suppressed because it is too large
Load Diff
|
@ -13,7 +13,7 @@ import (
|
|||
|
||||
"github.com/influxdb/influxdb/cluster"
|
||||
"github.com/influxdb/influxdb/meta"
|
||||
"github.com/influxdb/influxdb/tsdb"
|
||||
"github.com/influxdb/influxdb/models"
|
||||
)
|
||||
|
||||
const leaderWaitTimeout = 30 * time.Second
|
||||
|
@ -338,9 +338,9 @@ func (m *Monitor) storeStatistics() {
|
|||
continue
|
||||
}
|
||||
|
||||
points := make(tsdb.Points, 0, len(stats))
|
||||
points := make(models.Points, 0, len(stats))
|
||||
for _, s := range stats {
|
||||
points = append(points, tsdb.NewPoint(s.Name, s.Tags, s.Values, time.Now()))
|
||||
points = append(points, models.NewPoint(s.Name, s.Tags, s.Values, time.Now()))
|
||||
}
|
||||
|
||||
err = m.PointsWriter.WritePoints(&cluster.WritePointsRequest{
|
||||
|
|
|
@ -13,6 +13,7 @@ import (
|
|||
"github.com/influxdb/influxdb"
|
||||
"github.com/influxdb/influxdb/cluster"
|
||||
"github.com/influxdb/influxdb/meta"
|
||||
"github.com/influxdb/influxdb/models"
|
||||
"github.com/influxdb/influxdb/tsdb"
|
||||
"github.com/kimor79/gollectd"
|
||||
)
|
||||
|
@ -257,7 +258,7 @@ func (s *Service) writePoints() {
|
|||
}
|
||||
|
||||
// Unmarshal translates a collectd packet into InfluxDB data points.
|
||||
func Unmarshal(packet *gollectd.Packet) []tsdb.Point {
|
||||
func Unmarshal(packet *gollectd.Packet) []models.Point {
|
||||
// Prefer high resolution timestamp.
|
||||
var timestamp time.Time
|
||||
if packet.TimeHR > 0 {
|
||||
|
@ -272,7 +273,7 @@ func Unmarshal(packet *gollectd.Packet) []tsdb.Point {
|
|||
timestamp = time.Unix(int64(packet.Time), 0).UTC()
|
||||
}
|
||||
|
||||
var points []tsdb.Point
|
||||
var points []models.Point
|
||||
for i := range packet.Values {
|
||||
name := fmt.Sprintf("%s_%s", packet.Plugin, packet.Values[i].Name)
|
||||
tags := make(map[string]string)
|
||||
|
@ -292,7 +293,7 @@ func Unmarshal(packet *gollectd.Packet) []tsdb.Point {
|
|||
if packet.TypeInstance != "" {
|
||||
tags["type_instance"] = packet.TypeInstance
|
||||
}
|
||||
p := tsdb.NewPoint(name, tags, fields, timestamp)
|
||||
p := models.NewPoint(name, tags, fields, timestamp)
|
||||
|
||||
points = append(points, p)
|
||||
}
|
||||
|
|
|
@ -11,8 +11,8 @@ import (
|
|||
|
||||
"github.com/influxdb/influxdb/cluster"
|
||||
"github.com/influxdb/influxdb/meta"
|
||||
"github.com/influxdb/influxdb/models"
|
||||
"github.com/influxdb/influxdb/toml"
|
||||
"github.com/influxdb/influxdb/tsdb"
|
||||
)
|
||||
|
||||
// Test that the service checks / creates the target database on startup.
|
||||
|
@ -54,7 +54,7 @@ func TestService_BatchSize(t *testing.T) {
|
|||
func() {
|
||||
s := newTestService(batchSize, time.Second)
|
||||
|
||||
pointCh := make(chan tsdb.Point)
|
||||
pointCh := make(chan models.Point)
|
||||
s.MetaStore.CreateDatabaseIfNotExistsFn = func(name string) (*meta.DatabaseInfo, error) { return nil, nil }
|
||||
s.PointsWriter.WritePointsFn = func(req *cluster.WritePointsRequest) error {
|
||||
if len(req.Points) != batchSize {
|
||||
|
@ -86,7 +86,7 @@ func TestService_BatchSize(t *testing.T) {
|
|||
t.Fatalf("only sent %d of %d bytes", n, len(testData))
|
||||
}
|
||||
|
||||
points := []tsdb.Point{}
|
||||
points := []models.Point{}
|
||||
Loop:
|
||||
for {
|
||||
select {
|
||||
|
@ -123,7 +123,7 @@ func TestService_BatchDuration(t *testing.T) {
|
|||
|
||||
s := newTestService(5000, 250*time.Millisecond)
|
||||
|
||||
pointCh := make(chan tsdb.Point, 1000)
|
||||
pointCh := make(chan models.Point, 1000)
|
||||
s.MetaStore.CreateDatabaseIfNotExistsFn = func(name string) (*meta.DatabaseInfo, error) { return nil, nil }
|
||||
s.PointsWriter.WritePointsFn = func(req *cluster.WritePointsRequest) error {
|
||||
for _, p := range req.Points {
|
||||
|
@ -151,7 +151,7 @@ func TestService_BatchDuration(t *testing.T) {
|
|||
t.Fatalf("only sent %d of %d bytes", n, len(testData))
|
||||
}
|
||||
|
||||
points := []tsdb.Point{}
|
||||
points := []models.Point{}
|
||||
Loop:
|
||||
for {
|
||||
select {
|
||||
|
|
|
@ -14,6 +14,7 @@ import (
|
|||
"github.com/influxdb/influxdb/cluster"
|
||||
"github.com/influxdb/influxdb/influxql"
|
||||
"github.com/influxdb/influxdb/meta"
|
||||
"github.com/influxdb/influxdb/models"
|
||||
"github.com/influxdb/influxdb/tsdb"
|
||||
)
|
||||
|
||||
|
@ -332,7 +333,7 @@ func (s *Service) runContinuousQueryAndWriteResult(cq *ContinuousQuery) error {
|
|||
}
|
||||
|
||||
// Read all rows from the result channel.
|
||||
points := make([]tsdb.Point, 0, 100)
|
||||
points := make([]models.Point, 0, 100)
|
||||
for result := range ch {
|
||||
if result.Err != nil {
|
||||
return result.Err
|
||||
|
@ -397,7 +398,7 @@ func (s *Service) runContinuousQueryAndWriteResult(cq *ContinuousQuery) error {
|
|||
|
||||
// convertRowToPoints will convert a query result Row into Points that can be written back in.
|
||||
// Used for continuous and INTO queries
|
||||
func (s *Service) convertRowToPoints(measurementName string, row *influxql.Row) ([]tsdb.Point, error) {
|
||||
func (s *Service) convertRowToPoints(measurementName string, row *influxql.Row) ([]models.Point, error) {
|
||||
// figure out which parts of the result are the time and which are the fields
|
||||
timeIndex := -1
|
||||
fieldIndexes := make(map[string]int)
|
||||
|
@ -413,14 +414,14 @@ func (s *Service) convertRowToPoints(measurementName string, row *influxql.Row)
|
|||
return nil, errors.New("error finding time index in result")
|
||||
}
|
||||
|
||||
points := make([]tsdb.Point, 0, len(row.Values))
|
||||
points := make([]models.Point, 0, len(row.Values))
|
||||
for _, v := range row.Values {
|
||||
vals := make(map[string]interface{})
|
||||
for fieldName, fieldIndex := range fieldIndexes {
|
||||
vals[fieldName] = v[fieldIndex]
|
||||
}
|
||||
|
||||
p := tsdb.NewPoint(measurementName, row.Tags, vals, v[timeIndex].(time.Time))
|
||||
p := models.NewPoint(measurementName, row.Tags, vals, v[timeIndex].(time.Time))
|
||||
|
||||
points = append(points, p)
|
||||
}
|
||||
|
|
|
@ -5,8 +5,8 @@ import (
|
|||
"strings"
|
||||
"time"
|
||||
|
||||
"github.com/influxdb/influxdb/models"
|
||||
"github.com/influxdb/influxdb/toml"
|
||||
"github.com/influxdb/influxdb/tsdb"
|
||||
)
|
||||
|
||||
const (
|
||||
|
@ -82,8 +82,8 @@ func (c *Config) WithDefaults() *Config {
|
|||
return &d
|
||||
}
|
||||
|
||||
func (c *Config) DefaultTags() tsdb.Tags {
|
||||
tags := tsdb.Tags{}
|
||||
func (c *Config) DefaultTags() models.Tags {
|
||||
tags := models.Tags{}
|
||||
for _, t := range c.Tags {
|
||||
parts := strings.Split(t, "=")
|
||||
tags[parts[0]] = parts[1]
|
||||
|
|
|
@ -8,7 +8,7 @@ import (
|
|||
"strings"
|
||||
"time"
|
||||
|
||||
"github.com/influxdb/influxdb/tsdb"
|
||||
"github.com/influxdb/influxdb/models"
|
||||
)
|
||||
|
||||
var (
|
||||
|
@ -28,14 +28,14 @@ func init() {
|
|||
// Parser encapsulates a Graphite Parser.
|
||||
type Parser struct {
|
||||
matcher *matcher
|
||||
tags tsdb.Tags
|
||||
tags models.Tags
|
||||
}
|
||||
|
||||
// Options are configurable values that can be provided to a Parser
|
||||
type Options struct {
|
||||
Separator string
|
||||
Templates []string
|
||||
DefaultTags tsdb.Tags
|
||||
DefaultTags models.Tags
|
||||
}
|
||||
|
||||
// NewParserWithOptions returns a graphite parser using the given options
|
||||
|
@ -60,7 +60,7 @@ func NewParserWithOptions(options Options) (*Parser, error) {
|
|||
}
|
||||
|
||||
// Parse out the default tags specific to this template
|
||||
tags := tsdb.Tags{}
|
||||
tags := models.Tags{}
|
||||
if strings.Contains(parts[len(parts)-1], "=") {
|
||||
tagStrs := strings.Split(parts[len(parts)-1], ",")
|
||||
for _, kv := range tagStrs {
|
||||
|
@ -79,7 +79,7 @@ func NewParserWithOptions(options Options) (*Parser, error) {
|
|||
}
|
||||
|
||||
// NewParser returns a GraphiteParser instance.
|
||||
func NewParser(templates []string, defaultTags tsdb.Tags) (*Parser, error) {
|
||||
func NewParser(templates []string, defaultTags models.Tags) (*Parser, error) {
|
||||
return NewParserWithOptions(
|
||||
Options{
|
||||
Templates: templates,
|
||||
|
@ -89,7 +89,7 @@ func NewParser(templates []string, defaultTags tsdb.Tags) (*Parser, error) {
|
|||
}
|
||||
|
||||
// Parse performs Graphite parsing of a single line.
|
||||
func (p *Parser) Parse(line string) (tsdb.Point, error) {
|
||||
func (p *Parser) Parse(line string) (models.Point, error) {
|
||||
// Break into 3 fields (name, value, timestamp).
|
||||
fields := strings.Fields(line)
|
||||
if len(fields) != 2 && len(fields) != 3 {
|
||||
|
@ -140,7 +140,7 @@ func (p *Parser) Parse(line string) (tsdb.Point, error) {
|
|||
tags[k] = v
|
||||
}
|
||||
}
|
||||
point := tsdb.NewPoint(measurement, tags, fieldValues, timestamp)
|
||||
point := models.NewPoint(measurement, tags, fieldValues, timestamp)
|
||||
|
||||
return point, nil
|
||||
}
|
||||
|
@ -148,12 +148,12 @@ func (p *Parser) Parse(line string) (tsdb.Point, error) {
|
|||
// template represents a pattern and tags to map a graphite metric string to a influxdb Point
|
||||
type template struct {
|
||||
tags []string
|
||||
defaultTags tsdb.Tags
|
||||
defaultTags models.Tags
|
||||
greedyMeasurement bool
|
||||
separator string
|
||||
}
|
||||
|
||||
func NewTemplate(pattern string, defaultTags tsdb.Tags, separator string) (*template, error) {
|
||||
func NewTemplate(pattern string, defaultTags models.Tags, separator string) (*template, error) {
|
||||
tags := strings.Split(pattern, ".")
|
||||
hasMeasurement := false
|
||||
template := &template{tags: tags, defaultTags: defaultTags, separator: separator}
|
||||
|
|
|
@ -6,8 +6,8 @@ import (
|
|||
"testing"
|
||||
"time"
|
||||
|
||||
"github.com/influxdb/influxdb/models"
|
||||
"github.com/influxdb/influxdb/services/graphite"
|
||||
"github.com/influxdb/influxdb/tsdb"
|
||||
)
|
||||
|
||||
func BenchmarkParse(b *testing.B) {
|
||||
|
@ -229,9 +229,9 @@ func TestParseNaN(t *testing.T) {
|
|||
t.Fatalf("parse error: %v", err)
|
||||
}
|
||||
|
||||
exp := tsdb.NewPoint("servers.localhost.cpu_load",
|
||||
tsdb.Tags{},
|
||||
tsdb.Fields{"value": math.NaN()},
|
||||
exp := models.NewPoint("servers.localhost.cpu_load",
|
||||
models.Tags{},
|
||||
models.Fields{"value": math.NaN()},
|
||||
time.Unix(1435077219, 0))
|
||||
|
||||
if exp.String() != pt.String() {
|
||||
|
@ -249,9 +249,9 @@ func TestFilterMatchDefault(t *testing.T) {
|
|||
t.Fatalf("unexpected error creating parser, got %v", err)
|
||||
}
|
||||
|
||||
exp := tsdb.NewPoint("miss.servers.localhost.cpu_load",
|
||||
tsdb.Tags{},
|
||||
tsdb.Fields{"value": float64(11)},
|
||||
exp := models.NewPoint("miss.servers.localhost.cpu_load",
|
||||
models.Tags{},
|
||||
models.Fields{"value": float64(11)},
|
||||
time.Unix(1435077219, 0))
|
||||
|
||||
pt, err := p.Parse("miss.servers.localhost.cpu_load 11 1435077219")
|
||||
|
@ -270,9 +270,9 @@ func TestFilterMatchMultipleMeasurement(t *testing.T) {
|
|||
t.Fatalf("unexpected error creating parser, got %v", err)
|
||||
}
|
||||
|
||||
exp := tsdb.NewPoint("cpu.cpu_load.10",
|
||||
tsdb.Tags{"host": "localhost"},
|
||||
tsdb.Fields{"value": float64(11)},
|
||||
exp := models.NewPoint("cpu.cpu_load.10",
|
||||
models.Tags{"host": "localhost"},
|
||||
models.Fields{"value": float64(11)},
|
||||
time.Unix(1435077219, 0))
|
||||
|
||||
pt, err := p.Parse("servers.localhost.cpu.cpu_load.10 11 1435077219")
|
||||
|
@ -294,9 +294,9 @@ func TestFilterMatchMultipleMeasurementSeparator(t *testing.T) {
|
|||
t.Fatalf("unexpected error creating parser, got %v", err)
|
||||
}
|
||||
|
||||
exp := tsdb.NewPoint("cpu_cpu_load_10",
|
||||
tsdb.Tags{"host": "localhost"},
|
||||
tsdb.Fields{"value": float64(11)},
|
||||
exp := models.NewPoint("cpu_cpu_load_10",
|
||||
models.Tags{"host": "localhost"},
|
||||
models.Fields{"value": float64(11)},
|
||||
time.Unix(1435077219, 0))
|
||||
|
||||
pt, err := p.Parse("servers.localhost.cpu.cpu_load.10 11 1435077219")
|
||||
|
@ -315,9 +315,9 @@ func TestFilterMatchSingle(t *testing.T) {
|
|||
t.Fatalf("unexpected error creating parser, got %v", err)
|
||||
}
|
||||
|
||||
exp := tsdb.NewPoint("cpu_load",
|
||||
tsdb.Tags{"host": "localhost"},
|
||||
tsdb.Fields{"value": float64(11)},
|
||||
exp := models.NewPoint("cpu_load",
|
||||
models.Tags{"host": "localhost"},
|
||||
models.Fields{"value": float64(11)},
|
||||
time.Unix(1435077219, 0))
|
||||
|
||||
pt, err := p.Parse("servers.localhost.cpu_load 11 1435077219")
|
||||
|
@ -336,9 +336,9 @@ func TestParseNoMatch(t *testing.T) {
|
|||
t.Fatalf("unexpected error creating parser, got %v", err)
|
||||
}
|
||||
|
||||
exp := tsdb.NewPoint("servers.localhost.memory.VmallocChunk",
|
||||
tsdb.Tags{},
|
||||
tsdb.Fields{"value": float64(11)},
|
||||
exp := models.NewPoint("servers.localhost.memory.VmallocChunk",
|
||||
models.Tags{},
|
||||
models.Fields{"value": float64(11)},
|
||||
time.Unix(1435077219, 0))
|
||||
|
||||
pt, err := p.Parse("servers.localhost.memory.VmallocChunk 11 1435077219")
|
||||
|
@ -357,9 +357,9 @@ func TestFilterMatchWildcard(t *testing.T) {
|
|||
t.Fatalf("unexpected error creating parser, got %v", err)
|
||||
}
|
||||
|
||||
exp := tsdb.NewPoint("cpu_load",
|
||||
tsdb.Tags{"host": "localhost"},
|
||||
tsdb.Fields{"value": float64(11)},
|
||||
exp := models.NewPoint("cpu_load",
|
||||
models.Tags{"host": "localhost"},
|
||||
models.Fields{"value": float64(11)},
|
||||
time.Unix(1435077219, 0))
|
||||
|
||||
pt, err := p.Parse("servers.localhost.cpu_load 11 1435077219")
|
||||
|
@ -380,9 +380,9 @@ func TestFilterMatchExactBeforeWildcard(t *testing.T) {
|
|||
t.Fatalf("unexpected error creating parser, got %v", err)
|
||||
}
|
||||
|
||||
exp := tsdb.NewPoint("cpu_load",
|
||||
tsdb.Tags{"host": "localhost"},
|
||||
tsdb.Fields{"value": float64(11)},
|
||||
exp := models.NewPoint("cpu_load",
|
||||
models.Tags{"host": "localhost"},
|
||||
models.Fields{"value": float64(11)},
|
||||
time.Unix(1435077219, 0))
|
||||
|
||||
pt, err := p.Parse("servers.localhost.cpu_load 11 1435077219")
|
||||
|
@ -408,9 +408,9 @@ func TestFilterMatchMostLongestFilter(t *testing.T) {
|
|||
t.Fatalf("unexpected error creating parser, got %v", err)
|
||||
}
|
||||
|
||||
exp := tsdb.NewPoint("cpu_load",
|
||||
tsdb.Tags{"host": "localhost", "resource": "cpu"},
|
||||
tsdb.Fields{"value": float64(11)},
|
||||
exp := models.NewPoint("cpu_load",
|
||||
models.Tags{"host": "localhost", "resource": "cpu"},
|
||||
models.Fields{"value": float64(11)},
|
||||
time.Unix(1435077219, 0))
|
||||
|
||||
pt, err := p.Parse("servers.localhost.cpu.cpu_load 11 1435077219")
|
||||
|
@ -435,9 +435,9 @@ func TestFilterMatchMultipleWildcards(t *testing.T) {
|
|||
t.Fatalf("unexpected error creating parser, got %v", err)
|
||||
}
|
||||
|
||||
exp := tsdb.NewPoint("cpu_load",
|
||||
tsdb.Tags{"host": "server01"},
|
||||
tsdb.Fields{"value": float64(11)},
|
||||
exp := models.NewPoint("cpu_load",
|
||||
models.Tags{"host": "server01"},
|
||||
models.Fields{"value": float64(11)},
|
||||
time.Unix(1435077219, 0))
|
||||
|
||||
pt, err := p.Parse("servers.server01.cpu_load 11 1435077219")
|
||||
|
@ -451,7 +451,7 @@ func TestFilterMatchMultipleWildcards(t *testing.T) {
|
|||
}
|
||||
|
||||
func TestParseDefaultTags(t *testing.T) {
|
||||
p, err := graphite.NewParser([]string{"servers.localhost .host.measurement*"}, tsdb.Tags{
|
||||
p, err := graphite.NewParser([]string{"servers.localhost .host.measurement*"}, models.Tags{
|
||||
"region": "us-east",
|
||||
"zone": "1c",
|
||||
"host": "should not set",
|
||||
|
@ -460,9 +460,9 @@ func TestParseDefaultTags(t *testing.T) {
|
|||
t.Fatalf("unexpected error creating parser, got %v", err)
|
||||
}
|
||||
|
||||
exp := tsdb.NewPoint("cpu_load",
|
||||
tsdb.Tags{"host": "localhost", "region": "us-east", "zone": "1c"},
|
||||
tsdb.Fields{"value": float64(11)},
|
||||
exp := models.NewPoint("cpu_load",
|
||||
models.Tags{"host": "localhost", "region": "us-east", "zone": "1c"},
|
||||
models.Fields{"value": float64(11)},
|
||||
time.Unix(1435077219, 0))
|
||||
|
||||
pt, err := p.Parse("servers.localhost.cpu_load 11 1435077219")
|
||||
|
@ -476,7 +476,7 @@ func TestParseDefaultTags(t *testing.T) {
|
|||
}
|
||||
|
||||
func TestParseDefaultTemplateTags(t *testing.T) {
|
||||
p, err := graphite.NewParser([]string{"servers.localhost .host.measurement* zone=1c"}, tsdb.Tags{
|
||||
p, err := graphite.NewParser([]string{"servers.localhost .host.measurement* zone=1c"}, models.Tags{
|
||||
"region": "us-east",
|
||||
"host": "should not set",
|
||||
})
|
||||
|
@ -484,9 +484,9 @@ func TestParseDefaultTemplateTags(t *testing.T) {
|
|||
t.Fatalf("unexpected error creating parser, got %v", err)
|
||||
}
|
||||
|
||||
exp := tsdb.NewPoint("cpu_load",
|
||||
tsdb.Tags{"host": "localhost", "region": "us-east", "zone": "1c"},
|
||||
tsdb.Fields{"value": float64(11)},
|
||||
exp := models.NewPoint("cpu_load",
|
||||
models.Tags{"host": "localhost", "region": "us-east", "zone": "1c"},
|
||||
models.Fields{"value": float64(11)},
|
||||
time.Unix(1435077219, 0))
|
||||
|
||||
pt, err := p.Parse("servers.localhost.cpu_load 11 1435077219")
|
||||
|
@ -500,7 +500,7 @@ func TestParseDefaultTemplateTags(t *testing.T) {
|
|||
}
|
||||
|
||||
func TestParseDefaultTemplateTagsOverridGlobal(t *testing.T) {
|
||||
p, err := graphite.NewParser([]string{"servers.localhost .host.measurement* zone=1c,region=us-east"}, tsdb.Tags{
|
||||
p, err := graphite.NewParser([]string{"servers.localhost .host.measurement* zone=1c,region=us-east"}, models.Tags{
|
||||
"region": "shot not be set",
|
||||
"host": "should not set",
|
||||
})
|
||||
|
@ -508,9 +508,9 @@ func TestParseDefaultTemplateTagsOverridGlobal(t *testing.T) {
|
|||
t.Fatalf("unexpected error creating parser, got %v", err)
|
||||
}
|
||||
|
||||
exp := tsdb.NewPoint("cpu_load",
|
||||
tsdb.Tags{"host": "localhost", "region": "us-east", "zone": "1c"},
|
||||
tsdb.Fields{"value": float64(11)},
|
||||
exp := models.NewPoint("cpu_load",
|
||||
models.Tags{"host": "localhost", "region": "us-east", "zone": "1c"},
|
||||
models.Fields{"value": float64(11)},
|
||||
time.Unix(1435077219, 0))
|
||||
|
||||
pt, err := p.Parse("servers.localhost.cpu_load 11 1435077219")
|
||||
|
@ -524,7 +524,7 @@ func TestParseDefaultTemplateTagsOverridGlobal(t *testing.T) {
|
|||
}
|
||||
|
||||
func TestParseTemplateWhitespace(t *testing.T) {
|
||||
p, err := graphite.NewParser([]string{"servers.localhost .host.measurement* zone=1c"}, tsdb.Tags{
|
||||
p, err := graphite.NewParser([]string{"servers.localhost .host.measurement* zone=1c"}, models.Tags{
|
||||
"region": "us-east",
|
||||
"host": "should not set",
|
||||
})
|
||||
|
@ -532,9 +532,9 @@ func TestParseTemplateWhitespace(t *testing.T) {
|
|||
t.Fatalf("unexpected error creating parser, got %v", err)
|
||||
}
|
||||
|
||||
exp := tsdb.NewPoint("cpu_load",
|
||||
tsdb.Tags{"host": "localhost", "region": "us-east", "zone": "1c"},
|
||||
tsdb.Fields{"value": float64(11)},
|
||||
exp := models.NewPoint("cpu_load",
|
||||
models.Tags{"host": "localhost", "region": "us-east", "zone": "1c"},
|
||||
models.Fields{"value": float64(11)},
|
||||
time.Unix(1435077219, 0))
|
||||
|
||||
pt, err := p.Parse("servers.localhost.cpu_load 11 1435077219")
|
||||
|
|
|
@ -9,9 +9,9 @@ import (
|
|||
|
||||
"github.com/influxdb/influxdb/cluster"
|
||||
"github.com/influxdb/influxdb/meta"
|
||||
"github.com/influxdb/influxdb/models"
|
||||
"github.com/influxdb/influxdb/services/graphite"
|
||||
"github.com/influxdb/influxdb/toml"
|
||||
"github.com/influxdb/influxdb/tsdb"
|
||||
)
|
||||
|
||||
func Test_ServerGraphiteTCP(t *testing.T) {
|
||||
|
@ -43,7 +43,7 @@ func Test_ServerGraphiteTCP(t *testing.T) {
|
|||
} else if req.RetentionPolicy != "" {
|
||||
t.Fatalf("unexpected retention policy: %s", req.RetentionPolicy)
|
||||
} else if req.Points[0].String() !=
|
||||
tsdb.NewPoint(
|
||||
models.NewPoint(
|
||||
"cpu",
|
||||
map[string]string{},
|
||||
map[string]interface{}{"value": 23.456},
|
||||
|
@ -112,7 +112,7 @@ func Test_ServerGraphiteUDP(t *testing.T) {
|
|||
} else if req.RetentionPolicy != "" {
|
||||
t.Fatalf("unexpected retention policy: %s", req.RetentionPolicy)
|
||||
} else if req.Points[0].String() !=
|
||||
tsdb.NewPoint(
|
||||
models.NewPoint(
|
||||
"cpu",
|
||||
map[string]string{},
|
||||
map[string]interface{}{"value": 23.456},
|
||||
|
|
|
@ -11,6 +11,7 @@ import (
|
|||
"sync"
|
||||
"time"
|
||||
|
||||
"github.com/influxdb/influxdb/models"
|
||||
"github.com/influxdb/influxdb/tsdb"
|
||||
)
|
||||
|
||||
|
@ -103,7 +104,7 @@ func (p *Processor) addQueue(nodeID uint64) (*queue, error) {
|
|||
return queue, nil
|
||||
}
|
||||
|
||||
func (p *Processor) WriteShard(shardID, ownerID uint64, points []tsdb.Point) error {
|
||||
func (p *Processor) WriteShard(shardID, ownerID uint64, points []models.Point) error {
|
||||
queue, ok := p.queues[ownerID]
|
||||
if !ok {
|
||||
var err error
|
||||
|
@ -186,7 +187,7 @@ func (p *Processor) Process() error {
|
|||
return nil
|
||||
}
|
||||
|
||||
func (p *Processor) marshalWrite(shardID uint64, points []tsdb.Point) []byte {
|
||||
func (p *Processor) marshalWrite(shardID uint64, points []models.Point) []byte {
|
||||
b := make([]byte, 8)
|
||||
binary.BigEndian.PutUint64(b, shardID)
|
||||
for _, p := range points {
|
||||
|
@ -196,12 +197,12 @@ func (p *Processor) marshalWrite(shardID uint64, points []tsdb.Point) []byte {
|
|||
return b
|
||||
}
|
||||
|
||||
func (p *Processor) unmarshalWrite(b []byte) (uint64, []tsdb.Point, error) {
|
||||
func (p *Processor) unmarshalWrite(b []byte) (uint64, []models.Point, error) {
|
||||
if len(b) < 8 {
|
||||
return 0, nil, fmt.Errorf("too short: len = %d", len(b))
|
||||
}
|
||||
ownerID := binary.BigEndian.Uint64(b[:8])
|
||||
points, err := tsdb.ParsePoints(b[8:])
|
||||
points, err := models.ParsePoints(b[8:])
|
||||
return ownerID, points, err
|
||||
}
|
||||
|
||||
|
|
|
@ -5,14 +5,14 @@ import (
|
|||
"testing"
|
||||
"time"
|
||||
|
||||
"github.com/influxdb/influxdb/tsdb"
|
||||
"github.com/influxdb/influxdb/models"
|
||||
)
|
||||
|
||||
type fakeShardWriter struct {
|
||||
ShardWriteFn func(shardID, nodeID uint64, points []tsdb.Point) error
|
||||
ShardWriteFn func(shardID, nodeID uint64, points []models.Point) error
|
||||
}
|
||||
|
||||
func (f *fakeShardWriter) WriteShard(shardID, nodeID uint64, points []tsdb.Point) error {
|
||||
func (f *fakeShardWriter) WriteShard(shardID, nodeID uint64, points []models.Point) error {
|
||||
return f.ShardWriteFn(shardID, nodeID, points)
|
||||
}
|
||||
|
||||
|
@ -24,10 +24,10 @@ func TestProcessorProcess(t *testing.T) {
|
|||
|
||||
// expected data to be queue and sent to the shardWriter
|
||||
var expShardID, expNodeID, count = uint64(100), uint64(200), 0
|
||||
pt := tsdb.NewPoint("cpu", tsdb.Tags{"foo": "bar"}, tsdb.Fields{"value": 1.0}, time.Unix(0, 0))
|
||||
pt := models.NewPoint("cpu", models.Tags{"foo": "bar"}, models.Fields{"value": 1.0}, time.Unix(0, 0))
|
||||
|
||||
sh := &fakeShardWriter{
|
||||
ShardWriteFn: func(shardID, nodeID uint64, points []tsdb.Point) error {
|
||||
ShardWriteFn: func(shardID, nodeID uint64, points []models.Point) error {
|
||||
count += 1
|
||||
if shardID != expShardID {
|
||||
t.Errorf("Process() shardID mismatch: got %v, exp %v", shardID, expShardID)
|
||||
|
@ -54,7 +54,7 @@ func TestProcessorProcess(t *testing.T) {
|
|||
}
|
||||
|
||||
// This should queue the writes
|
||||
if err := p.WriteShard(expShardID, expNodeID, []tsdb.Point{pt}); err != nil {
|
||||
if err := p.WriteShard(expShardID, expNodeID, []models.Point{pt}); err != nil {
|
||||
t.Fatalf("Process() failed to write points: %v", err)
|
||||
}
|
||||
|
||||
|
|
|
@ -8,7 +8,7 @@ import (
|
|||
"sync"
|
||||
"time"
|
||||
|
||||
"github.com/influxdb/influxdb/tsdb"
|
||||
"github.com/influxdb/influxdb/models"
|
||||
)
|
||||
|
||||
var ErrHintedHandoffDisabled = fmt.Errorf("hinted handoff disabled")
|
||||
|
@ -24,14 +24,14 @@ type Service struct {
|
|||
ShardWriter shardWriter
|
||||
|
||||
HintedHandoff interface {
|
||||
WriteShard(shardID, ownerID uint64, points []tsdb.Point) error
|
||||
WriteShard(shardID, ownerID uint64, points []models.Point) error
|
||||
Process() error
|
||||
PurgeOlderThan(when time.Duration) error
|
||||
}
|
||||
}
|
||||
|
||||
type shardWriter interface {
|
||||
WriteShard(shardID, ownerID uint64, points []tsdb.Point) error
|
||||
WriteShard(shardID, ownerID uint64, points []models.Point) error
|
||||
}
|
||||
|
||||
// NewService returns a new instance of Service.
|
||||
|
@ -87,7 +87,7 @@ func (s *Service) SetLogger(l *log.Logger) {
|
|||
}
|
||||
|
||||
// WriteShard queues the points write for shardID to node ownerID to handoff queue
|
||||
func (s *Service) WriteShard(shardID, ownerID uint64, points []tsdb.Point) error {
|
||||
func (s *Service) WriteShard(shardID, ownerID uint64, points []models.Point) error {
|
||||
if !s.cfg.Enabled {
|
||||
return ErrHintedHandoffDisabled
|
||||
}
|
||||
|
|
|
@ -23,8 +23,8 @@ import (
|
|||
"github.com/influxdb/influxdb/cluster"
|
||||
"github.com/influxdb/influxdb/influxql"
|
||||
"github.com/influxdb/influxdb/meta"
|
||||
"github.com/influxdb/influxdb/models"
|
||||
"github.com/influxdb/influxdb/services/continuous_querier"
|
||||
"github.com/influxdb/influxdb/tsdb"
|
||||
"github.com/influxdb/influxdb/uuid"
|
||||
)
|
||||
|
||||
|
@ -473,7 +473,7 @@ func (h *Handler) serveWriteLine(w http.ResponseWriter, r *http.Request, body []
|
|||
precision = "n"
|
||||
}
|
||||
|
||||
points, err := tsdb.ParsePointsWithPrecision(body, time.Now().UTC(), precision)
|
||||
points, err := models.ParsePointsWithPrecision(body, time.Now().UTC(), precision)
|
||||
if err != nil {
|
||||
if err.Error() == "EOF" {
|
||||
w.WriteHeader(http.StatusOK)
|
||||
|
@ -868,8 +868,8 @@ func (r *Response) Error() error {
|
|||
// NormalizeBatchPoints returns a slice of Points, created by populating individual
|
||||
// points within the batch, which do not have times or tags, with the top-level
|
||||
// values.
|
||||
func NormalizeBatchPoints(bp client.BatchPoints) ([]tsdb.Point, error) {
|
||||
points := []tsdb.Point{}
|
||||
func NormalizeBatchPoints(bp client.BatchPoints) ([]models.Point, error) {
|
||||
points := []models.Point{}
|
||||
for _, p := range bp.Points {
|
||||
if p.Time.IsZero() {
|
||||
if bp.Time.IsZero() {
|
||||
|
@ -901,7 +901,7 @@ func NormalizeBatchPoints(bp client.BatchPoints) ([]tsdb.Point, error) {
|
|||
return points, fmt.Errorf("missing fields")
|
||||
}
|
||||
// Need to convert from a client.Point to a influxdb.Point
|
||||
points = append(points, tsdb.NewPoint(p.Measurement, p.Tags, p.Fields, p.Time))
|
||||
points = append(points, models.NewPoint(p.Measurement, p.Tags, p.Fields, p.Time))
|
||||
}
|
||||
|
||||
return points, nil
|
||||
|
|
|
@ -16,6 +16,7 @@ import (
|
|||
"github.com/influxdb/influxdb/client"
|
||||
"github.com/influxdb/influxdb/influxql"
|
||||
"github.com/influxdb/influxdb/meta"
|
||||
"github.com/influxdb/influxdb/models"
|
||||
"github.com/influxdb/influxdb/services/httpd"
|
||||
"github.com/influxdb/influxdb/tsdb"
|
||||
)
|
||||
|
@ -296,7 +297,7 @@ func TestNormalizeBatchPoints(t *testing.T) {
|
|||
tests := []struct {
|
||||
name string
|
||||
bp client.BatchPoints
|
||||
p []tsdb.Point
|
||||
p []models.Point
|
||||
err string
|
||||
}{
|
||||
{
|
||||
|
@ -306,8 +307,8 @@ func TestNormalizeBatchPoints(t *testing.T) {
|
|||
{Measurement: "cpu", Tags: map[string]string{"region": "useast"}, Time: now, Fields: map[string]interface{}{"value": 1.0}},
|
||||
},
|
||||
},
|
||||
p: []tsdb.Point{
|
||||
tsdb.NewPoint("cpu", map[string]string{"region": "useast"}, map[string]interface{}{"value": 1.0}, now),
|
||||
p: []models.Point{
|
||||
models.NewPoint("cpu", map[string]string{"region": "useast"}, map[string]interface{}{"value": 1.0}, now),
|
||||
},
|
||||
},
|
||||
{
|
||||
|
@ -318,8 +319,8 @@ func TestNormalizeBatchPoints(t *testing.T) {
|
|||
{Measurement: "cpu", Tags: map[string]string{"region": "useast"}, Fields: map[string]interface{}{"value": 1.0}},
|
||||
},
|
||||
},
|
||||
p: []tsdb.Point{
|
||||
tsdb.NewPoint("cpu", map[string]string{"region": "useast"}, map[string]interface{}{"value": 1.0}, now),
|
||||
p: []models.Point{
|
||||
models.NewPoint("cpu", map[string]string{"region": "useast"}, map[string]interface{}{"value": 1.0}, now),
|
||||
},
|
||||
},
|
||||
{
|
||||
|
@ -331,9 +332,9 @@ func TestNormalizeBatchPoints(t *testing.T) {
|
|||
{Measurement: "memory", Time: now, Fields: map[string]interface{}{"value": 2.0}},
|
||||
},
|
||||
},
|
||||
p: []tsdb.Point{
|
||||
tsdb.NewPoint("cpu", map[string]string{"day": "monday", "region": "useast"}, map[string]interface{}{"value": 1.0}, now),
|
||||
tsdb.NewPoint("memory", map[string]string{"day": "monday"}, map[string]interface{}{"value": 2.0}, now),
|
||||
p: []models.Point{
|
||||
models.NewPoint("cpu", map[string]string{"day": "monday", "region": "useast"}, map[string]interface{}{"value": 1.0}, now),
|
||||
models.NewPoint("memory", map[string]string{"day": "monday"}, map[string]interface{}{"value": 2.0}, now),
|
||||
},
|
||||
},
|
||||
}
|
||||
|
|
|
@ -13,7 +13,7 @@ import (
|
|||
|
||||
"github.com/influxdb/influxdb"
|
||||
"github.com/influxdb/influxdb/cluster"
|
||||
"github.com/influxdb/influxdb/tsdb"
|
||||
"github.com/influxdb/influxdb/models"
|
||||
)
|
||||
|
||||
type Handler struct {
|
||||
|
@ -96,7 +96,7 @@ func (h *Handler) servePut(w http.ResponseWriter, r *http.Request) {
|
|||
}
|
||||
|
||||
// Convert points into TSDB points.
|
||||
points := make([]tsdb.Point, 0, len(dps))
|
||||
points := make([]models.Point, 0, len(dps))
|
||||
for i := range dps {
|
||||
p := dps[i]
|
||||
|
||||
|
@ -109,7 +109,7 @@ func (h *Handler) servePut(w http.ResponseWriter, r *http.Request) {
|
|||
ts = time.Unix(p.Time/1000, (p.Time%1000)*1000)
|
||||
}
|
||||
|
||||
points = append(points, tsdb.NewPoint(p.Metric, p.Tags, map[string]interface{}{"value": p.Value}, ts))
|
||||
points = append(points, models.NewPoint(p.Metric, p.Tags, map[string]interface{}{"value": p.Value}, ts))
|
||||
}
|
||||
|
||||
// Write points.
|
||||
|
|
|
@ -19,6 +19,7 @@ import (
|
|||
"github.com/influxdb/influxdb"
|
||||
"github.com/influxdb/influxdb/cluster"
|
||||
"github.com/influxdb/influxdb/meta"
|
||||
"github.com/influxdb/influxdb/models"
|
||||
"github.com/influxdb/influxdb/tsdb"
|
||||
)
|
||||
|
||||
|
@ -324,7 +325,7 @@ func (s *Service) handleTelnetConn(conn net.Conn) {
|
|||
continue
|
||||
}
|
||||
|
||||
s.batcher.In() <- tsdb.NewPoint(measurement, tags, fields, t)
|
||||
s.batcher.In() <- models.NewPoint(measurement, tags, fields, t)
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -14,8 +14,8 @@ import (
|
|||
"github.com/davecgh/go-spew/spew"
|
||||
"github.com/influxdb/influxdb/cluster"
|
||||
"github.com/influxdb/influxdb/meta"
|
||||
"github.com/influxdb/influxdb/models"
|
||||
"github.com/influxdb/influxdb/services/opentsdb"
|
||||
"github.com/influxdb/influxdb/tsdb"
|
||||
)
|
||||
|
||||
// Ensure a point can be written via the telnet protocol.
|
||||
|
@ -37,8 +37,8 @@ func TestService_Telnet(t *testing.T) {
|
|||
t.Fatalf("unexpected database: %s", req.Database)
|
||||
} else if req.RetentionPolicy != "" {
|
||||
t.Fatalf("unexpected retention policy: %s", req.RetentionPolicy)
|
||||
} else if !reflect.DeepEqual(req.Points, []tsdb.Point{
|
||||
tsdb.NewPoint(
|
||||
} else if !reflect.DeepEqual(req.Points, []models.Point{
|
||||
models.NewPoint(
|
||||
"sys.cpu.user",
|
||||
map[string]string{"host": "webserver01", "cpu": "0"},
|
||||
map[string]interface{}{"value": 42.5},
|
||||
|
@ -91,8 +91,8 @@ func TestService_HTTP(t *testing.T) {
|
|||
t.Fatalf("unexpected database: %s", req.Database)
|
||||
} else if req.RetentionPolicy != "" {
|
||||
t.Fatalf("unexpected retention policy: %s", req.RetentionPolicy)
|
||||
} else if !reflect.DeepEqual(req.Points, []tsdb.Point{
|
||||
tsdb.NewPoint(
|
||||
} else if !reflect.DeepEqual(req.Points, []models.Point{
|
||||
models.NewPoint(
|
||||
"sys.cpu.nice",
|
||||
map[string]string{"dc": "lga", "host": "web01"},
|
||||
map[string]interface{}{"value": 18.0},
|
||||
|
|
|
@ -12,6 +12,7 @@ import (
|
|||
|
||||
"github.com/influxdb/influxdb"
|
||||
"github.com/influxdb/influxdb/cluster"
|
||||
"github.com/influxdb/influxdb/models"
|
||||
"github.com/influxdb/influxdb/tsdb"
|
||||
)
|
||||
|
||||
|
@ -145,7 +146,7 @@ func (s *Service) serve() {
|
|||
}
|
||||
s.statMap.Add(statBytesReceived, int64(n))
|
||||
|
||||
points, err := tsdb.ParsePoints(buf[:n])
|
||||
points, err := models.ParsePoints(buf[:n])
|
||||
if err != nil {
|
||||
s.statMap.Add(statPointsParseFail, 1)
|
||||
s.Logger.Printf("Failed to parse points: %s", err)
|
||||
|
|
|
@ -4,6 +4,8 @@ import (
|
|||
"sync"
|
||||
"sync/atomic"
|
||||
"time"
|
||||
|
||||
"github.com/influxdb/influxdb/models"
|
||||
)
|
||||
|
||||
// PointBatcher accepts Points and will emit a batch of those points when either
|
||||
|
@ -15,8 +17,8 @@ type PointBatcher struct {
|
|||
duration time.Duration
|
||||
|
||||
stop chan struct{}
|
||||
in chan Point
|
||||
out chan []Point
|
||||
in chan models.Point
|
||||
out chan []models.Point
|
||||
flush chan struct{}
|
||||
|
||||
wg *sync.WaitGroup
|
||||
|
@ -31,8 +33,8 @@ func NewPointBatcher(sz int, bp int, d time.Duration) *PointBatcher {
|
|||
size: sz,
|
||||
duration: d,
|
||||
stop: make(chan struct{}),
|
||||
in: make(chan Point, bp*sz),
|
||||
out: make(chan []Point),
|
||||
in: make(chan models.Point, bp*sz),
|
||||
out: make(chan []models.Point),
|
||||
flush: make(chan struct{}),
|
||||
}
|
||||
}
|
||||
|
@ -54,7 +56,7 @@ func (b *PointBatcher) Start() {
|
|||
}
|
||||
|
||||
var timer *time.Timer
|
||||
var batch []Point
|
||||
var batch []models.Point
|
||||
var timerCh <-chan time.Time
|
||||
|
||||
emit := func() {
|
||||
|
@ -79,7 +81,7 @@ func (b *PointBatcher) Start() {
|
|||
case p := <-b.in:
|
||||
atomic.AddUint64(&b.stats.PointTotal, 1)
|
||||
if batch == nil {
|
||||
batch = make([]Point, 0, b.size)
|
||||
batch = make([]models.Point, 0, b.size)
|
||||
if b.duration > 0 {
|
||||
timer = time.NewTimer(b.duration)
|
||||
timerCh = timer.C
|
||||
|
@ -118,12 +120,12 @@ func (b *PointBatcher) Stop() {
|
|||
}
|
||||
|
||||
// In returns the channel to which points should be written.
|
||||
func (b *PointBatcher) In() chan<- Point {
|
||||
func (b *PointBatcher) In() chan<- models.Point {
|
||||
return b.in
|
||||
}
|
||||
|
||||
// Out returns the channel from which batches should be read.
|
||||
func (b *PointBatcher) Out() <-chan []Point {
|
||||
func (b *PointBatcher) Out() <-chan []models.Point {
|
||||
return b.out
|
||||
}
|
||||
|
||||
|
|
|
@ -4,6 +4,7 @@ import (
|
|||
"testing"
|
||||
"time"
|
||||
|
||||
"github.com/influxdb/influxdb/models"
|
||||
"github.com/influxdb/influxdb/tsdb"
|
||||
)
|
||||
|
||||
|
@ -17,7 +18,7 @@ func TestBatch_Size(t *testing.T) {
|
|||
|
||||
batcher.Start()
|
||||
|
||||
var p tsdb.Point
|
||||
var p models.Point
|
||||
go func() {
|
||||
for i := 0; i < batchSize; i++ {
|
||||
batcher.In() <- p
|
||||
|
@ -40,7 +41,7 @@ func TestBatch_SizeBuffered(t *testing.T) {
|
|||
|
||||
batcher.Start()
|
||||
|
||||
var p tsdb.Point
|
||||
var p models.Point
|
||||
go func() {
|
||||
for i := 0; i < batchSize; i++ {
|
||||
batcher.In() <- p
|
||||
|
@ -63,7 +64,7 @@ func TestBatch_Timeout(t *testing.T) {
|
|||
|
||||
batcher.Start()
|
||||
|
||||
var p tsdb.Point
|
||||
var p models.Point
|
||||
go func() {
|
||||
for i := 0; i < batchSize; i++ {
|
||||
batcher.In() <- p
|
||||
|
@ -86,7 +87,7 @@ func TestBatch_Flush(t *testing.T) {
|
|||
|
||||
batcher.Start()
|
||||
|
||||
var p tsdb.Point
|
||||
var p models.Point
|
||||
go func() {
|
||||
batcher.In() <- p
|
||||
batcher.Flush()
|
||||
|
@ -108,8 +109,8 @@ func TestBatch_MultipleBatches(t *testing.T) {
|
|||
|
||||
batcher.Start()
|
||||
|
||||
var p tsdb.Point
|
||||
var b []tsdb.Point
|
||||
var p models.Point
|
||||
var b []models.Point
|
||||
|
||||
batcher.In() <- p
|
||||
batcher.In() <- p
|
||||
|
|
|
@ -10,6 +10,7 @@ import (
|
|||
"time"
|
||||
|
||||
"github.com/boltdb/bolt"
|
||||
"github.com/influxdb/influxdb/models"
|
||||
)
|
||||
|
||||
var (
|
||||
|
@ -29,7 +30,7 @@ type Engine interface {
|
|||
LoadMetadataIndex(index *DatabaseIndex, measurementFields map[string]*MeasurementFields) error
|
||||
|
||||
Begin(writable bool) (Tx, error)
|
||||
WritePoints(points []Point, measurementFieldsToSave map[string]*MeasurementFields, seriesToCreate []*SeriesCreate) error
|
||||
WritePoints(points []models.Point, measurementFieldsToSave map[string]*MeasurementFields, seriesToCreate []*SeriesCreate) error
|
||||
DeleteSeries(keys []string) error
|
||||
DeleteMeasurement(name string, seriesKeys []string) error
|
||||
SeriesCount() (n int, err error)
|
||||
|
|
|
@ -14,6 +14,7 @@ import (
|
|||
"time"
|
||||
|
||||
"github.com/boltdb/bolt"
|
||||
"github.com/influxdb/influxdb/models"
|
||||
"github.com/influxdb/influxdb/tsdb"
|
||||
)
|
||||
|
||||
|
@ -206,7 +207,7 @@ func (e *Engine) LoadMetadataIndex(index *tsdb.DatabaseIndex, measurementFields
|
|||
}
|
||||
|
||||
// WritePoints will write the raw data points and any new metadata to the index in the shard
|
||||
func (e *Engine) WritePoints(points []tsdb.Point, measurementFieldsToSave map[string]*tsdb.MeasurementFields, seriesToCreate []*tsdb.SeriesCreate) error {
|
||||
func (e *Engine) WritePoints(points []models.Point, measurementFieldsToSave map[string]*tsdb.MeasurementFields, seriesToCreate []*tsdb.SeriesCreate) error {
|
||||
// save to the underlying bolt instance
|
||||
if err := e.db.Update(func(tx *bolt.Tx) error {
|
||||
// save any new metadata
|
||||
|
|
|
@ -10,6 +10,7 @@ import (
|
|||
"time"
|
||||
|
||||
"github.com/influxdb/influxdb/influxql"
|
||||
"github.com/influxdb/influxdb/models"
|
||||
"github.com/influxdb/influxdb/tsdb"
|
||||
"github.com/influxdb/influxdb/tsdb/engine/b1"
|
||||
)
|
||||
|
@ -23,11 +24,11 @@ func TestEngine_WritePoints(t *testing.T) {
|
|||
mf := &tsdb.MeasurementFields{Fields: make(map[string]*tsdb.Field)}
|
||||
mf.CreateFieldIfNotExists("value", influxql.Float)
|
||||
seriesToCreate := []*tsdb.SeriesCreate{
|
||||
{Series: tsdb.NewSeries(string(tsdb.MakeKey([]byte("temperature"), nil)), nil)},
|
||||
{Series: tsdb.NewSeries(string(models.MakeKey([]byte("temperature"), nil)), nil)},
|
||||
}
|
||||
|
||||
// Parse point.
|
||||
points, err := tsdb.ParsePointsWithPrecision([]byte("temperature value=100 1434059627"), time.Now().UTC(), "s")
|
||||
points, err := models.ParsePointsWithPrecision([]byte("temperature value=100 1434059627"), time.Now().UTC(), "s")
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
} else if data, err := mf.Codec.EncodeFields(points[0].Fields()); err != nil {
|
||||
|
@ -47,7 +48,7 @@ func TestEngine_WritePoints(t *testing.T) {
|
|||
}
|
||||
|
||||
// Parse new point.
|
||||
points, err = tsdb.ParsePointsWithPrecision([]byte("temperature value=200 1434059627"), time.Now().UTC(), "s")
|
||||
points, err = models.ParsePointsWithPrecision([]byte("temperature value=200 1434059627"), time.Now().UTC(), "s")
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
} else if data, err := mf.Codec.EncodeFields(points[0].Fields()); err != nil {
|
||||
|
@ -88,11 +89,11 @@ func TestEngine_WritePoints_Reverse(t *testing.T) {
|
|||
mf := &tsdb.MeasurementFields{Fields: make(map[string]*tsdb.Field)}
|
||||
mf.CreateFieldIfNotExists("value", influxql.Float)
|
||||
seriesToCreate := []*tsdb.SeriesCreate{
|
||||
{Series: tsdb.NewSeries(string(tsdb.MakeKey([]byte("temperature"), nil)), nil)},
|
||||
{Series: tsdb.NewSeries(string(models.MakeKey([]byte("temperature"), nil)), nil)},
|
||||
}
|
||||
|
||||
// Parse point.
|
||||
points, err := tsdb.ParsePointsWithPrecision([]byte("temperature value=100 0"), time.Now().UTC(), "s")
|
||||
points, err := models.ParsePointsWithPrecision([]byte("temperature value=100 0"), time.Now().UTC(), "s")
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
} else if data, err := mf.Codec.EncodeFields(points[0].Fields()); err != nil {
|
||||
|
@ -112,7 +113,7 @@ func TestEngine_WritePoints_Reverse(t *testing.T) {
|
|||
}
|
||||
|
||||
// Parse new point.
|
||||
points, err = tsdb.ParsePointsWithPrecision([]byte("temperature value=200 1"), time.Now().UTC(), "s")
|
||||
points, err = models.ParsePointsWithPrecision([]byte("temperature value=200 1"), time.Now().UTC(), "s")
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
} else if data, err := mf.Codec.EncodeFields(points[0].Fields()); err != nil {
|
||||
|
|
|
@ -17,6 +17,7 @@ import (
|
|||
"github.com/boltdb/bolt"
|
||||
"github.com/golang/snappy"
|
||||
"github.com/influxdb/influxdb"
|
||||
"github.com/influxdb/influxdb/models"
|
||||
"github.com/influxdb/influxdb/tsdb"
|
||||
"github.com/influxdb/influxdb/tsdb/engine/wal"
|
||||
)
|
||||
|
@ -70,7 +71,7 @@ type Engine struct {
|
|||
|
||||
// WAL represents a write ahead log that can be queried
|
||||
type WAL interface {
|
||||
WritePoints(points []tsdb.Point, measurementFieldsToSave map[string]*tsdb.MeasurementFields, seriesToCreate []*tsdb.SeriesCreate) error
|
||||
WritePoints(points []models.Point, measurementFieldsToSave map[string]*tsdb.MeasurementFields, seriesToCreate []*tsdb.SeriesCreate) error
|
||||
LoadMetadataIndex(index *tsdb.DatabaseIndex, measurementFields map[string]*tsdb.MeasurementFields) error
|
||||
DeleteSeries(keys []string) error
|
||||
Cursor(key string, direction tsdb.Direction) tsdb.Cursor
|
||||
|
@ -225,7 +226,7 @@ func (e *Engine) LoadMetadataIndex(index *tsdb.DatabaseIndex, measurementFields
|
|||
|
||||
// WritePoints writes metadata and point data into the engine.
|
||||
// Returns an error if new points are added to an existing key.
|
||||
func (e *Engine) WritePoints(points []tsdb.Point, measurementFieldsToSave map[string]*tsdb.MeasurementFields, seriesToCreate []*tsdb.SeriesCreate) error {
|
||||
func (e *Engine) WritePoints(points []models.Point, measurementFieldsToSave map[string]*tsdb.MeasurementFields, seriesToCreate []*tsdb.SeriesCreate) error {
|
||||
// Write points to the WAL.
|
||||
if err := e.WAL.WritePoints(points, measurementFieldsToSave, seriesToCreate); err != nil {
|
||||
return fmt.Errorf("write points: %s", err)
|
||||
|
|
|
@ -18,6 +18,7 @@ import (
|
|||
"time"
|
||||
|
||||
"github.com/influxdb/influxdb/influxql"
|
||||
"github.com/influxdb/influxdb/models"
|
||||
"github.com/influxdb/influxdb/tsdb"
|
||||
"github.com/influxdb/influxdb/tsdb/engine/bz1"
|
||||
"github.com/influxdb/influxdb/tsdb/engine/wal"
|
||||
|
@ -40,11 +41,11 @@ func TestEngine_LoadMetadataIndex_Series(t *testing.T) {
|
|||
|
||||
// Setup mock that writes the index
|
||||
seriesToCreate := []*tsdb.SeriesCreate{
|
||||
{Series: tsdb.NewSeries(string(tsdb.MakeKey([]byte("cpu"), map[string]string{"host": "server0"})), map[string]string{"host": "server0"})},
|
||||
{Series: tsdb.NewSeries(string(tsdb.MakeKey([]byte("cpu"), map[string]string{"host": "server1"})), map[string]string{"host": "server1"})},
|
||||
{Series: tsdb.NewSeries(string(models.MakeKey([]byte("cpu"), map[string]string{"host": "server0"})), map[string]string{"host": "server0"})},
|
||||
{Series: tsdb.NewSeries(string(models.MakeKey([]byte("cpu"), map[string]string{"host": "server1"})), map[string]string{"host": "server1"})},
|
||||
{Series: tsdb.NewSeries("series with spaces", nil)},
|
||||
}
|
||||
e.PointsWriter.WritePointsFn = func(a []tsdb.Point) error { return e.WriteIndex(nil, nil, seriesToCreate) }
|
||||
e.PointsWriter.WritePointsFn = func(a []models.Point) error { return e.WriteIndex(nil, nil, seriesToCreate) }
|
||||
|
||||
// Write series metadata.
|
||||
if err := e.WritePoints(nil, nil, seriesToCreate); err != nil {
|
||||
|
@ -86,7 +87,7 @@ func TestEngine_LoadMetadataIndex_Fields(t *testing.T) {
|
|||
},
|
||||
},
|
||||
}
|
||||
e.PointsWriter.WritePointsFn = func(a []tsdb.Point) error { return e.WriteIndex(nil, fields, nil) }
|
||||
e.PointsWriter.WritePointsFn = func(a []models.Point) error { return e.WriteIndex(nil, fields, nil) }
|
||||
|
||||
// Write series metadata.
|
||||
if err := e.WritePoints(nil, fields, nil); err != nil {
|
||||
|
@ -113,17 +114,17 @@ func TestEngine_WritePoints_PointsWriter(t *testing.T) {
|
|||
defer e.Close()
|
||||
|
||||
// Points to be inserted.
|
||||
points := []tsdb.Point{
|
||||
tsdb.NewPoint("cpu", tsdb.Tags{}, tsdb.Fields{}, time.Unix(0, 1)),
|
||||
tsdb.NewPoint("cpu", tsdb.Tags{}, tsdb.Fields{}, time.Unix(0, 0)),
|
||||
tsdb.NewPoint("cpu", tsdb.Tags{}, tsdb.Fields{}, time.Unix(1, 0)),
|
||||
points := []models.Point{
|
||||
models.NewPoint("cpu", models.Tags{}, models.Fields{}, time.Unix(0, 1)),
|
||||
models.NewPoint("cpu", models.Tags{}, models.Fields{}, time.Unix(0, 0)),
|
||||
models.NewPoint("cpu", models.Tags{}, models.Fields{}, time.Unix(1, 0)),
|
||||
|
||||
tsdb.NewPoint("cpu", tsdb.Tags{"host": "serverA"}, tsdb.Fields{}, time.Unix(0, 0)),
|
||||
models.NewPoint("cpu", models.Tags{"host": "serverA"}, models.Fields{}, time.Unix(0, 0)),
|
||||
}
|
||||
|
||||
// Mock points writer to ensure points are passed through.
|
||||
var invoked bool
|
||||
e.PointsWriter.WritePointsFn = func(a []tsdb.Point) error {
|
||||
e.PointsWriter.WritePointsFn = func(a []models.Point) error {
|
||||
invoked = true
|
||||
if !reflect.DeepEqual(points, a) {
|
||||
t.Fatalf("unexpected points: %#v", a)
|
||||
|
@ -145,7 +146,7 @@ func TestEngine_WritePoints_ErrPointsWriter(t *testing.T) {
|
|||
defer e.Close()
|
||||
|
||||
// Ensure points writer returns an error.
|
||||
e.PointsWriter.WritePointsFn = func(a []tsdb.Point) error { return errors.New("marker") }
|
||||
e.PointsWriter.WritePointsFn = func(a []models.Point) error { return errors.New("marker") }
|
||||
|
||||
// Write to engine.
|
||||
if err := e.WritePoints(nil, nil, nil); err == nil || err.Error() != `write points: marker` {
|
||||
|
@ -490,7 +491,7 @@ func benchmarkEngine_WriteIndex(b *testing.B, blockSize int) {
|
|||
a := make(map[string][][]byte)
|
||||
a["cpu"] = make([][]byte, b.N)
|
||||
for i := 0; i < b.N; i++ {
|
||||
a["cpu"][i] = wal.MarshalEntry(int64(i), MustEncodeFields(codec, tsdb.Fields{"value": float64(i)}))
|
||||
a["cpu"][i] = wal.MarshalEntry(int64(i), MustEncodeFields(codec, models.Fields{"value": float64(i)}))
|
||||
}
|
||||
|
||||
b.ResetTimer()
|
||||
|
@ -565,10 +566,10 @@ func (e *Engine) MustBegin(writable bool) tsdb.Tx {
|
|||
|
||||
// EnginePointsWriter represents a mock that implements Engine.PointsWriter.
|
||||
type EnginePointsWriter struct {
|
||||
WritePointsFn func(points []tsdb.Point) error
|
||||
WritePointsFn func(points []models.Point) error
|
||||
}
|
||||
|
||||
func (w *EnginePointsWriter) WritePoints(points []tsdb.Point, measurementFieldsToSave map[string]*tsdb.MeasurementFields, seriesToCreate []*tsdb.SeriesCreate) error {
|
||||
func (w *EnginePointsWriter) WritePoints(points []models.Point, measurementFieldsToSave map[string]*tsdb.MeasurementFields, seriesToCreate []*tsdb.SeriesCreate) error {
|
||||
return w.WritePointsFn(points)
|
||||
}
|
||||
|
||||
|
@ -674,7 +675,7 @@ func MergePoints(a []Points) Points {
|
|||
}
|
||||
|
||||
// MustEncodeFields encodes fields with codec. Panic on error.
|
||||
func MustEncodeFields(codec *tsdb.FieldCodec, fields tsdb.Fields) []byte {
|
||||
func MustEncodeFields(codec *tsdb.FieldCodec, fields models.Fields) []byte {
|
||||
b, err := codec.EncodeFields(fields)
|
||||
if err != nil {
|
||||
panic(err)
|
||||
|
|
|
@ -39,6 +39,7 @@ import (
|
|||
|
||||
"github.com/golang/snappy"
|
||||
"github.com/influxdb/influxdb"
|
||||
"github.com/influxdb/influxdb/models"
|
||||
"github.com/influxdb/influxdb/tsdb"
|
||||
)
|
||||
|
||||
|
@ -261,7 +262,7 @@ func (l *Log) Cursor(key string, direction tsdb.Direction) tsdb.Cursor {
|
|||
return l.partition.cursor(key, direction)
|
||||
}
|
||||
|
||||
func (l *Log) WritePoints(points []tsdb.Point, fields map[string]*tsdb.MeasurementFields, series []*tsdb.SeriesCreate) error {
|
||||
func (l *Log) WritePoints(points []models.Point, fields map[string]*tsdb.MeasurementFields, series []*tsdb.SeriesCreate) error {
|
||||
l.statMap.Add(statPointsWriteReq, 1)
|
||||
l.statMap.Add(statPointsWrite, int64(len(points)))
|
||||
|
||||
|
@ -799,7 +800,7 @@ func (p *Partition) Close() error {
|
|||
// Write will write a compressed block of the points to the current segment file. If the segment
|
||||
// file is larger than the max size, it will roll over to a new file before performing the write.
|
||||
// This method will also add the points to the in memory cache
|
||||
func (p *Partition) Write(points []tsdb.Point) error {
|
||||
func (p *Partition) Write(points []models.Point) error {
|
||||
|
||||
// Check if we should compact due to memory pressure and if we should fail the write if
|
||||
// we're way too far over the threshold.
|
||||
|
@ -1523,7 +1524,7 @@ func marshalWALEntry(buf *bytes.Buffer, key []byte, timestamp int64, data []byte
|
|||
buf.Write(data)
|
||||
}
|
||||
|
||||
func walEntryLength(p tsdb.Point) int {
|
||||
func walEntryLength(p models.Point) int {
|
||||
return 8 + 4 + 4 + len(p.Key()) + len(p.Data())
|
||||
}
|
||||
|
||||
|
|
|
@ -12,6 +12,7 @@ import (
|
|||
"time"
|
||||
|
||||
"github.com/influxdb/influxdb/influxql"
|
||||
"github.com/influxdb/influxdb/models"
|
||||
"github.com/influxdb/influxdb/tsdb"
|
||||
)
|
||||
|
||||
|
@ -36,7 +37,7 @@ func TestWAL_WritePoints(t *testing.T) {
|
|||
p1 := parsePoint("cpu,host=A value=23.2 1", codec)
|
||||
p2 := parsePoint("cpu,host=A value=25.3 4", codec)
|
||||
p3 := parsePoint("cpu,host=B value=1.0 1", codec)
|
||||
if err := log.WritePoints([]tsdb.Point{p1, p2, p3}, nil, nil); err != nil {
|
||||
if err := log.WritePoints([]models.Point{p1, p2, p3}, nil, nil); err != nil {
|
||||
t.Fatalf("failed to write points: %s", err.Error())
|
||||
}
|
||||
|
||||
|
@ -93,7 +94,7 @@ func TestWAL_WritePoints(t *testing.T) {
|
|||
p6 := parsePoint("cpu,host=A value=1.3 2", codec)
|
||||
// // ensure we can write to a new partition
|
||||
// p7 := parsePoint("cpu,region=west value=2.2", codec)
|
||||
if err := log.WritePoints([]tsdb.Point{p4, p5, p6}, nil, nil); err != nil {
|
||||
if err := log.WritePoints([]models.Point{p4, p5, p6}, nil, nil); err != nil {
|
||||
t.Fatalf("failed to write points: %s", err.Error())
|
||||
}
|
||||
|
||||
|
@ -151,7 +152,7 @@ func TestWAL_CorruptDataLengthSize(t *testing.T) {
|
|||
// test that we can write to two different series
|
||||
p1 := parsePoint("cpu,host=A value=23.2 1", codec)
|
||||
p2 := parsePoint("cpu,host=A value=25.3 4", codec)
|
||||
if err := log.WritePoints([]tsdb.Point{p1, p2}, nil, nil); err != nil {
|
||||
if err := log.WritePoints([]models.Point{p1, p2}, nil, nil); err != nil {
|
||||
t.Fatalf("failed to write points: %s", err.Error())
|
||||
}
|
||||
|
||||
|
@ -190,7 +191,7 @@ func TestWAL_CorruptDataLengthSize(t *testing.T) {
|
|||
|
||||
// now write new data and ensure it's all good
|
||||
p3 := parsePoint("cpu,host=A value=29.2 6", codec)
|
||||
if err := log.WritePoints([]tsdb.Point{p3}, nil, nil); err != nil {
|
||||
if err := log.WritePoints([]models.Point{p3}, nil, nil); err != nil {
|
||||
t.Fatalf("failed to write point: %s", err.Error())
|
||||
}
|
||||
|
||||
|
@ -230,7 +231,7 @@ func TestWAL_CorruptDataBlock(t *testing.T) {
|
|||
// test that we can write to two different series
|
||||
p1 := parsePoint("cpu,host=A value=23.2 1", codec)
|
||||
p2 := parsePoint("cpu,host=A value=25.3 4", codec)
|
||||
if err := log.WritePoints([]tsdb.Point{p1, p2}, nil, nil); err != nil {
|
||||
if err := log.WritePoints([]models.Point{p1, p2}, nil, nil); err != nil {
|
||||
t.Fatalf("failed to write points: %s", err.Error())
|
||||
}
|
||||
|
||||
|
@ -275,7 +276,7 @@ func TestWAL_CorruptDataBlock(t *testing.T) {
|
|||
|
||||
// now write new data and ensure it's all good
|
||||
p3 := parsePoint("cpu,host=A value=29.2 6", codec)
|
||||
if err := log.WritePoints([]tsdb.Point{p3}, nil, nil); err != nil {
|
||||
if err := log.WritePoints([]models.Point{p3}, nil, nil); err != nil {
|
||||
t.Fatalf("failed to write point: %s", err.Error())
|
||||
}
|
||||
|
||||
|
@ -394,8 +395,8 @@ func TestWAL_SeriesAndFieldsGetPersisted(t *testing.T) {
|
|||
p3 := parsePoint("cpu,host=B value=1.0 1", codec)
|
||||
|
||||
seriesToCreate := []*tsdb.SeriesCreate{
|
||||
{Series: tsdb.NewSeries(string(tsdb.MakeKey([]byte("cpu"), map[string]string{"host": "A"})), map[string]string{"host": "A"})},
|
||||
{Series: tsdb.NewSeries(string(tsdb.MakeKey([]byte("cpu"), map[string]string{"host": "B"})), map[string]string{"host": "B"})},
|
||||
{Series: tsdb.NewSeries(string(models.MakeKey([]byte("cpu"), map[string]string{"host": "A"})), map[string]string{"host": "A"})},
|
||||
{Series: tsdb.NewSeries(string(models.MakeKey([]byte("cpu"), map[string]string{"host": "B"})), map[string]string{"host": "B"})},
|
||||
}
|
||||
|
||||
measaurementsToCreate := map[string]*tsdb.MeasurementFields{
|
||||
|
@ -406,7 +407,7 @@ func TestWAL_SeriesAndFieldsGetPersisted(t *testing.T) {
|
|||
},
|
||||
}
|
||||
|
||||
if err := log.WritePoints([]tsdb.Point{p1, p2, p3}, measaurementsToCreate, seriesToCreate); err != nil {
|
||||
if err := log.WritePoints([]models.Point{p1, p2, p3}, measaurementsToCreate, seriesToCreate); err != nil {
|
||||
t.Fatalf("failed to write points: %s", err.Error())
|
||||
}
|
||||
|
||||
|
@ -505,8 +506,8 @@ func TestWAL_DeleteSeries(t *testing.T) {
|
|||
}}
|
||||
|
||||
seriesToCreate := []*tsdb.SeriesCreate{
|
||||
{Series: tsdb.NewSeries(string(tsdb.MakeKey([]byte("cpu"), map[string]string{"host": "A"})), map[string]string{"host": "A"})},
|
||||
{Series: tsdb.NewSeries(string(tsdb.MakeKey([]byte("cpu"), map[string]string{"host": "B"})), map[string]string{"host": "B"})},
|
||||
{Series: tsdb.NewSeries(string(models.MakeKey([]byte("cpu"), map[string]string{"host": "A"})), map[string]string{"host": "A"})},
|
||||
{Series: tsdb.NewSeries(string(models.MakeKey([]byte("cpu"), map[string]string{"host": "B"})), map[string]string{"host": "B"})},
|
||||
}
|
||||
|
||||
if err := log.Open(); err != nil {
|
||||
|
@ -518,7 +519,7 @@ func TestWAL_DeleteSeries(t *testing.T) {
|
|||
p2 := parsePoint("cpu,host=B value=0.9 2", codec)
|
||||
p3 := parsePoint("cpu,host=A value=25.3 4", codec)
|
||||
p4 := parsePoint("cpu,host=B value=1.0 3", codec)
|
||||
if err := log.WritePoints([]tsdb.Point{p1, p2, p3, p4}, nil, seriesToCreate); err != nil {
|
||||
if err := log.WritePoints([]models.Point{p1, p2, p3, p4}, nil, seriesToCreate); err != nil {
|
||||
t.Fatalf("failed to write points: %s", err.Error())
|
||||
}
|
||||
|
||||
|
@ -612,7 +613,7 @@ func TestWAL_QueryDuringCompaction(t *testing.T) {
|
|||
|
||||
// test that we can write to two different series
|
||||
p1 := parsePoint("cpu,host=A value=23.2 1", codec)
|
||||
if err := log.WritePoints([]tsdb.Point{p1}, nil, nil); err != nil {
|
||||
if err := log.WritePoints([]models.Point{p1}, nil, nil); err != nil {
|
||||
t.Fatalf("failed to write points: %s", err.Error())
|
||||
}
|
||||
|
||||
|
@ -659,7 +660,7 @@ func TestWAL_PointsSorted(t *testing.T) {
|
|||
p2 := parsePoint("cpu,host=A value=4.4 4", codec)
|
||||
p3 := parsePoint("cpu,host=A value=2.2 2", codec)
|
||||
p4 := parsePoint("cpu,host=A value=6.6 6", codec)
|
||||
if err := log.WritePoints([]tsdb.Point{p1, p2, p3, p4}, nil, nil); err != nil {
|
||||
if err := log.WritePoints([]models.Point{p1, p2, p3, p4}, nil, nil); err != nil {
|
||||
t.Fatalf("failed to write points: %s", err.Error())
|
||||
}
|
||||
|
||||
|
@ -704,7 +705,7 @@ func TestWAL_Cursor_Reverse(t *testing.T) {
|
|||
p2 := parsePoint("cpu,host=A value=4.4 4", codec)
|
||||
p3 := parsePoint("cpu,host=A value=2.2 2", codec)
|
||||
p4 := parsePoint("cpu,host=A value=6.6 6", codec)
|
||||
if err := log.WritePoints([]tsdb.Point{p1, p2, p3, p4}, nil, nil); err != nil {
|
||||
if err := log.WritePoints([]models.Point{p1, p2, p3, p4}, nil, nil); err != nil {
|
||||
t.Fatalf("failed to write points: %s", err.Error())
|
||||
}
|
||||
|
||||
|
@ -743,8 +744,8 @@ func openTestWAL() *Log {
|
|||
return NewLog(dir)
|
||||
}
|
||||
|
||||
func parsePoints(buf string, codec *tsdb.FieldCodec) []tsdb.Point {
|
||||
points, err := tsdb.ParsePointsString(buf)
|
||||
func parsePoints(buf string, codec *tsdb.FieldCodec) []models.Point {
|
||||
points, err := models.ParsePointsString(buf)
|
||||
if err != nil {
|
||||
panic(fmt.Sprintf("couldn't parse points: %s", err.Error()))
|
||||
}
|
||||
|
@ -758,7 +759,7 @@ func parsePoints(buf string, codec *tsdb.FieldCodec) []tsdb.Point {
|
|||
return points
|
||||
}
|
||||
|
||||
func parsePoint(buf string, codec *tsdb.FieldCodec) tsdb.Point {
|
||||
func parsePoint(buf string, codec *tsdb.FieldCodec) models.Point {
|
||||
return parsePoints(buf, codec)[0]
|
||||
}
|
||||
|
||||
|
|
|
@ -11,6 +11,7 @@ import (
|
|||
|
||||
"github.com/influxdb/influxdb/influxql"
|
||||
"github.com/influxdb/influxdb/meta"
|
||||
"github.com/influxdb/influxdb/models"
|
||||
"github.com/influxdb/influxdb/tsdb"
|
||||
)
|
||||
|
||||
|
@ -56,7 +57,7 @@ func TestWritePointsAndExecuteTwoShards(t *testing.T) {
|
|||
|
||||
// Write two points across shards.
|
||||
pt1time := time.Unix(1, 0).UTC()
|
||||
if err := store.WriteToShard(sID0, []tsdb.Point{tsdb.NewPoint(
|
||||
if err := store.WriteToShard(sID0, []models.Point{models.NewPoint(
|
||||
"cpu",
|
||||
map[string]string{"host": "serverA", "region": "us-east"},
|
||||
map[string]interface{}{"value": 100},
|
||||
|
@ -65,7 +66,7 @@ func TestWritePointsAndExecuteTwoShards(t *testing.T) {
|
|||
t.Fatalf(err.Error())
|
||||
}
|
||||
pt2time := time.Unix(2, 0).UTC()
|
||||
if err := store.WriteToShard(sID1, []tsdb.Point{tsdb.NewPoint(
|
||||
if err := store.WriteToShard(sID1, []models.Point{models.NewPoint(
|
||||
"cpu",
|
||||
map[string]string{"host": "serverB", "region": "us-east"},
|
||||
map[string]interface{}{"value": 200},
|
||||
|
@ -185,7 +186,7 @@ func TestWritePointsAndExecuteTwoShardsAlign(t *testing.T) {
|
|||
}
|
||||
|
||||
// Write interleaving, by time, chunks to the shards.
|
||||
if err := store.WriteToShard(sID0, []tsdb.Point{tsdb.NewPoint(
|
||||
if err := store.WriteToShard(sID0, []models.Point{models.NewPoint(
|
||||
"cpu",
|
||||
map[string]string{"host": "serverA"},
|
||||
map[string]interface{}{"value": 100},
|
||||
|
@ -193,7 +194,7 @@ func TestWritePointsAndExecuteTwoShardsAlign(t *testing.T) {
|
|||
)}); err != nil {
|
||||
t.Fatalf(err.Error())
|
||||
}
|
||||
if err := store.WriteToShard(sID1, []tsdb.Point{tsdb.NewPoint(
|
||||
if err := store.WriteToShard(sID1, []models.Point{models.NewPoint(
|
||||
"cpu",
|
||||
map[string]string{"host": "serverB"},
|
||||
map[string]interface{}{"value": 200},
|
||||
|
@ -201,7 +202,7 @@ func TestWritePointsAndExecuteTwoShardsAlign(t *testing.T) {
|
|||
)}); err != nil {
|
||||
t.Fatalf(err.Error())
|
||||
}
|
||||
if err := store.WriteToShard(sID1, []tsdb.Point{tsdb.NewPoint(
|
||||
if err := store.WriteToShard(sID1, []models.Point{models.NewPoint(
|
||||
"cpu",
|
||||
map[string]string{"host": "serverA"},
|
||||
map[string]interface{}{"value": 300},
|
||||
|
@ -265,7 +266,7 @@ func TestWritePointsAndExecuteTwoShardsQueryRewrite(t *testing.T) {
|
|||
|
||||
// Write two points across shards.
|
||||
pt1time := time.Unix(1, 0).UTC()
|
||||
if err := store0.WriteToShard(sID0, []tsdb.Point{tsdb.NewPoint(
|
||||
if err := store0.WriteToShard(sID0, []models.Point{models.NewPoint(
|
||||
"cpu",
|
||||
map[string]string{"host": "serverA"},
|
||||
map[string]interface{}{"value1": 100},
|
||||
|
@ -274,7 +275,7 @@ func TestWritePointsAndExecuteTwoShardsQueryRewrite(t *testing.T) {
|
|||
t.Fatalf(err.Error())
|
||||
}
|
||||
pt2time := time.Unix(2, 0).UTC()
|
||||
if err := store1.WriteToShard(sID1, []tsdb.Point{tsdb.NewPoint(
|
||||
if err := store1.WriteToShard(sID1, []models.Point{models.NewPoint(
|
||||
"cpu",
|
||||
map[string]string{"host": "serverB"},
|
||||
map[string]interface{}{"value2": 200},
|
||||
|
@ -357,7 +358,7 @@ func TestWritePointsAndExecuteTwoShardsTagSetOrdering(t *testing.T) {
|
|||
}
|
||||
|
||||
// Write tagsets "y" and "z" to first shard.
|
||||
if err := store.WriteToShard(sID0, []tsdb.Point{tsdb.NewPoint(
|
||||
if err := store.WriteToShard(sID0, []models.Point{models.NewPoint(
|
||||
"cpu",
|
||||
map[string]string{"host": "y"},
|
||||
map[string]interface{}{"value": 100},
|
||||
|
@ -365,7 +366,7 @@ func TestWritePointsAndExecuteTwoShardsTagSetOrdering(t *testing.T) {
|
|||
)}); err != nil {
|
||||
t.Fatalf(err.Error())
|
||||
}
|
||||
if err := store.WriteToShard(sID0, []tsdb.Point{tsdb.NewPoint(
|
||||
if err := store.WriteToShard(sID0, []models.Point{models.NewPoint(
|
||||
"cpu",
|
||||
map[string]string{"host": "z"},
|
||||
map[string]interface{}{"value": 200},
|
||||
|
@ -375,7 +376,7 @@ func TestWritePointsAndExecuteTwoShardsTagSetOrdering(t *testing.T) {
|
|||
}
|
||||
|
||||
// Write tagsets "x", y" and "z" to second shard.
|
||||
if err := store.WriteToShard(sID1, []tsdb.Point{tsdb.NewPoint(
|
||||
if err := store.WriteToShard(sID1, []models.Point{models.NewPoint(
|
||||
"cpu",
|
||||
map[string]string{"host": "x"},
|
||||
map[string]interface{}{"value": 300},
|
||||
|
@ -383,7 +384,7 @@ func TestWritePointsAndExecuteTwoShardsTagSetOrdering(t *testing.T) {
|
|||
)}); err != nil {
|
||||
t.Fatalf(err.Error())
|
||||
}
|
||||
if err := store.WriteToShard(sID1, []tsdb.Point{tsdb.NewPoint(
|
||||
if err := store.WriteToShard(sID1, []models.Point{models.NewPoint(
|
||||
"cpu",
|
||||
map[string]string{"host": "y"},
|
||||
map[string]interface{}{"value": 400},
|
||||
|
@ -391,7 +392,7 @@ func TestWritePointsAndExecuteTwoShardsTagSetOrdering(t *testing.T) {
|
|||
)}); err != nil {
|
||||
t.Fatalf(err.Error())
|
||||
}
|
||||
if err := store.WriteToShard(sID1, []tsdb.Point{tsdb.NewPoint(
|
||||
if err := store.WriteToShard(sID1, []models.Point{models.NewPoint(
|
||||
"cpu",
|
||||
map[string]string{"host": "z"},
|
||||
map[string]interface{}{"value": 500},
|
||||
|
@ -448,7 +449,7 @@ func TestWritePointsAndExecuteTwoShardsShowMeasurements(t *testing.T) {
|
|||
|
||||
// Write two points across shards.
|
||||
pt1time := time.Unix(1, 0).UTC()
|
||||
if err := store0.WriteToShard(sID0, []tsdb.Point{tsdb.NewPoint(
|
||||
if err := store0.WriteToShard(sID0, []models.Point{models.NewPoint(
|
||||
"cpu",
|
||||
map[string]string{"host": "serverA"},
|
||||
map[string]interface{}{"value1": 100},
|
||||
|
@ -457,7 +458,7 @@ func TestWritePointsAndExecuteTwoShardsShowMeasurements(t *testing.T) {
|
|||
t.Fatalf(err.Error())
|
||||
}
|
||||
pt2time := time.Unix(2, 0).UTC()
|
||||
if err := store1.WriteToShard(sID1, []tsdb.Point{tsdb.NewPoint(
|
||||
if err := store1.WriteToShard(sID1, []models.Point{models.NewPoint(
|
||||
"mem",
|
||||
map[string]string{"host": "serverB"},
|
||||
map[string]interface{}{"value2": 200},
|
||||
|
|
|
@ -13,6 +13,7 @@ import (
|
|||
"time"
|
||||
|
||||
"github.com/influxdb/influxdb/influxql"
|
||||
"github.com/influxdb/influxdb/models"
|
||||
"github.com/influxdb/influxdb/tsdb"
|
||||
)
|
||||
|
||||
|
@ -22,20 +23,20 @@ func TestShardMapper_RawMapperTagSetsFields(t *testing.T) {
|
|||
shard := mustCreateShard(tmpDir)
|
||||
|
||||
pt1time := time.Unix(1, 0).UTC()
|
||||
pt1 := tsdb.NewPoint(
|
||||
pt1 := models.NewPoint(
|
||||
"cpu",
|
||||
map[string]string{"host": "serverA", "region": "us-east"},
|
||||
map[string]interface{}{"idle": 60},
|
||||
pt1time,
|
||||
)
|
||||
pt2time := time.Unix(2, 0).UTC()
|
||||
pt2 := tsdb.NewPoint(
|
||||
pt2 := models.NewPoint(
|
||||
"cpu",
|
||||
map[string]string{"host": "serverB", "region": "us-east"},
|
||||
map[string]interface{}{"load": 60},
|
||||
pt2time,
|
||||
)
|
||||
err := shard.WritePoints([]tsdb.Point{pt1, pt2})
|
||||
err := shard.WritePoints([]models.Point{pt1, pt2})
|
||||
if err != nil {
|
||||
t.Fatalf(err.Error())
|
||||
}
|
||||
|
@ -112,20 +113,20 @@ func TestShardMapper_WriteAndSingleMapperRawQuerySingleValue(t *testing.T) {
|
|||
shard := mustCreateShard(tmpDir)
|
||||
|
||||
pt1time := time.Unix(1, 0).UTC()
|
||||
pt1 := tsdb.NewPoint(
|
||||
pt1 := models.NewPoint(
|
||||
"cpu",
|
||||
map[string]string{"host": "serverA", "region": "us-east"},
|
||||
map[string]interface{}{"load": 42},
|
||||
pt1time,
|
||||
)
|
||||
pt2time := time.Unix(2, 0).UTC()
|
||||
pt2 := tsdb.NewPoint(
|
||||
pt2 := models.NewPoint(
|
||||
"cpu",
|
||||
map[string]string{"host": "serverB", "region": "us-east"},
|
||||
map[string]interface{}{"load": 60},
|
||||
pt2time,
|
||||
)
|
||||
err := shard.WritePoints([]tsdb.Point{pt1, pt2})
|
||||
err := shard.WritePoints([]models.Point{pt1, pt2})
|
||||
if err != nil {
|
||||
t.Fatalf(err.Error())
|
||||
}
|
||||
|
@ -219,20 +220,20 @@ func TestShardMapper_WriteAndSingleMapperRawQueryMultiValue(t *testing.T) {
|
|||
shard := mustCreateShard(tmpDir)
|
||||
|
||||
pt1time := time.Unix(1, 0).UTC()
|
||||
pt1 := tsdb.NewPoint(
|
||||
pt1 := models.NewPoint(
|
||||
"cpu",
|
||||
map[string]string{"host": "serverA", "region": "us-east"},
|
||||
map[string]interface{}{"foo": 42, "bar": 43},
|
||||
pt1time,
|
||||
)
|
||||
pt2time := time.Unix(2, 0).UTC()
|
||||
pt2 := tsdb.NewPoint(
|
||||
pt2 := models.NewPoint(
|
||||
"cpu",
|
||||
map[string]string{"host": "serverB", "region": "us-east"},
|
||||
map[string]interface{}{"foo": 60, "bar": 61},
|
||||
pt2time,
|
||||
)
|
||||
err := shard.WritePoints([]tsdb.Point{pt1, pt2})
|
||||
err := shard.WritePoints([]models.Point{pt1, pt2})
|
||||
if err != nil {
|
||||
t.Fatalf(err.Error())
|
||||
}
|
||||
|
@ -272,20 +273,20 @@ func TestShardMapper_WriteAndSingleMapperRawQueryMultiSource(t *testing.T) {
|
|||
shard := mustCreateShard(tmpDir)
|
||||
|
||||
pt1time := time.Unix(1, 0).UTC()
|
||||
pt1 := tsdb.NewPoint(
|
||||
pt1 := models.NewPoint(
|
||||
"cpu0",
|
||||
map[string]string{"host": "serverA", "region": "us-east"},
|
||||
map[string]interface{}{"foo": 42},
|
||||
pt1time,
|
||||
)
|
||||
pt2time := time.Unix(2, 0).UTC()
|
||||
pt2 := tsdb.NewPoint(
|
||||
pt2 := models.NewPoint(
|
||||
"cpu1",
|
||||
map[string]string{"host": "serverB", "region": "us-east"},
|
||||
map[string]interface{}{"bar": 60},
|
||||
pt2time,
|
||||
)
|
||||
err := shard.WritePoints([]tsdb.Point{pt1, pt2})
|
||||
err := shard.WritePoints([]models.Point{pt1, pt2})
|
||||
if err != nil {
|
||||
t.Fatalf(err.Error())
|
||||
}
|
||||
|
@ -337,20 +338,20 @@ func TestShardMapper_WriteAndSingleMapperAggregateQuery(t *testing.T) {
|
|||
shard := mustCreateShard(tmpDir)
|
||||
|
||||
pt1time := time.Unix(10, 0).UTC()
|
||||
pt1 := tsdb.NewPoint(
|
||||
pt1 := models.NewPoint(
|
||||
"cpu",
|
||||
map[string]string{"host": "serverA", "region": "us-east"},
|
||||
map[string]interface{}{"value": 1},
|
||||
pt1time,
|
||||
)
|
||||
pt2time := time.Unix(20, 0).UTC()
|
||||
pt2 := tsdb.NewPoint(
|
||||
pt2 := models.NewPoint(
|
||||
"cpu",
|
||||
map[string]string{"host": "serverB", "region": "us-east"},
|
||||
map[string]interface{}{"value": 60},
|
||||
pt2time,
|
||||
)
|
||||
err := shard.WritePoints([]tsdb.Point{pt1, pt2})
|
||||
err := shard.WritePoints([]models.Point{pt1, pt2})
|
||||
if err != nil {
|
||||
t.Fatalf(err.Error())
|
||||
}
|
||||
|
@ -433,20 +434,20 @@ func TestShardMapper_SelectMapperTagSetsFields(t *testing.T) {
|
|||
shard := mustCreateShard(tmpDir)
|
||||
|
||||
pt1time := time.Unix(1, 0).UTC()
|
||||
pt1 := tsdb.NewPoint(
|
||||
pt1 := models.NewPoint(
|
||||
"cpu",
|
||||
map[string]string{"host": "serverA", "region": "us-east"},
|
||||
map[string]interface{}{"value": 42},
|
||||
pt1time,
|
||||
)
|
||||
pt2time := time.Unix(2, 0).UTC()
|
||||
pt2 := tsdb.NewPoint(
|
||||
pt2 := models.NewPoint(
|
||||
"cpu",
|
||||
map[string]string{"host": "serverB", "region": "us-east"},
|
||||
map[string]interface{}{"value": 60},
|
||||
pt2time,
|
||||
)
|
||||
err := shard.WritePoints([]tsdb.Point{pt1, pt2})
|
||||
err := shard.WritePoints([]models.Point{pt1, pt2})
|
||||
if err != nil {
|
||||
t.Fatalf(err.Error())
|
||||
}
|
||||
|
|
|
@ -9,6 +9,7 @@ import (
|
|||
"time"
|
||||
|
||||
"github.com/influxdb/influxdb/influxql"
|
||||
"github.com/influxdb/influxdb/pkg/escape"
|
||||
"github.com/influxdb/influxdb/tsdb/internal"
|
||||
|
||||
"github.com/gogo/protobuf/proto"
|
||||
|
@ -92,7 +93,7 @@ func (s *DatabaseIndex) CreateSeriesIndexIfNotExists(measurementName string, ser
|
|||
|
||||
// CreateMeasurementIndexIfNotExists creates or retrieves an in memory index object for the measurement
|
||||
func (s *DatabaseIndex) CreateMeasurementIndexIfNotExists(name string) *Measurement {
|
||||
name = unescapeString(name)
|
||||
name = escape.UnescapeString(name)
|
||||
m := s.measurements[name]
|
||||
if m == nil {
|
||||
m = NewMeasurement(name, s)
|
||||
|
|
|
@ -11,6 +11,7 @@ import (
|
|||
|
||||
"github.com/influxdb/influxdb/influxql"
|
||||
"github.com/influxdb/influxdb/meta"
|
||||
"github.com/influxdb/influxdb/models"
|
||||
"github.com/influxdb/influxdb/tsdb"
|
||||
)
|
||||
|
||||
|
@ -22,7 +23,7 @@ func TestWritePointsAndExecuteQuery(t *testing.T) {
|
|||
defer os.RemoveAll(store.Path())
|
||||
|
||||
// Write first point.
|
||||
if err := store.WriteToShard(shardID, []tsdb.Point{tsdb.NewPoint(
|
||||
if err := store.WriteToShard(shardID, []models.Point{models.NewPoint(
|
||||
"cpu",
|
||||
map[string]string{"host": "server"},
|
||||
map[string]interface{}{"value": 1.0},
|
||||
|
@ -32,7 +33,7 @@ func TestWritePointsAndExecuteQuery(t *testing.T) {
|
|||
}
|
||||
|
||||
// Write second point.
|
||||
if err := store.WriteToShard(shardID, []tsdb.Point{tsdb.NewPoint(
|
||||
if err := store.WriteToShard(shardID, []models.Point{models.NewPoint(
|
||||
"cpu",
|
||||
map[string]string{"host": "server"},
|
||||
map[string]interface{}{"value": 1.0},
|
||||
|
@ -75,7 +76,7 @@ func TestWritePointsAndExecuteQuery_Update(t *testing.T) {
|
|||
defer os.RemoveAll(store.Path())
|
||||
|
||||
// Write original point.
|
||||
if err := store.WriteToShard(1, []tsdb.Point{tsdb.NewPoint(
|
||||
if err := store.WriteToShard(1, []models.Point{models.NewPoint(
|
||||
"temperature",
|
||||
map[string]string{},
|
||||
map[string]interface{}{"value": 100.0},
|
||||
|
@ -96,7 +97,7 @@ func TestWritePointsAndExecuteQuery_Update(t *testing.T) {
|
|||
executor.ShardMapper = &testShardMapper{store: store}
|
||||
|
||||
// Rewrite point with new value.
|
||||
if err := store.WriteToShard(1, []tsdb.Point{tsdb.NewPoint(
|
||||
if err := store.WriteToShard(1, []models.Point{models.NewPoint(
|
||||
"temperature",
|
||||
map[string]string{},
|
||||
map[string]interface{}{"value": 200.0},
|
||||
|
@ -116,14 +117,14 @@ func TestDropSeriesStatement(t *testing.T) {
|
|||
store, executor := testStoreAndExecutor("")
|
||||
defer os.RemoveAll(store.Path())
|
||||
|
||||
pt := tsdb.NewPoint(
|
||||
pt := models.NewPoint(
|
||||
"cpu",
|
||||
map[string]string{"host": "server"},
|
||||
map[string]interface{}{"value": 1.0},
|
||||
time.Unix(1, 2),
|
||||
)
|
||||
|
||||
err := store.WriteToShard(shardID, []tsdb.Point{pt})
|
||||
err := store.WriteToShard(shardID, []models.Point{pt})
|
||||
if err != nil {
|
||||
t.Fatalf(err.Error())
|
||||
}
|
||||
|
@ -172,20 +173,20 @@ func TestDropMeasurementStatement(t *testing.T) {
|
|||
store, executor := testStoreAndExecutor("")
|
||||
defer os.RemoveAll(store.Path())
|
||||
|
||||
pt := tsdb.NewPoint(
|
||||
pt := models.NewPoint(
|
||||
"cpu",
|
||||
map[string]string{"host": "server"},
|
||||
map[string]interface{}{"value": 1.0},
|
||||
time.Unix(1, 2),
|
||||
)
|
||||
pt2 := tsdb.NewPoint(
|
||||
pt2 := models.NewPoint(
|
||||
"memory",
|
||||
map[string]string{"host": "server"},
|
||||
map[string]interface{}{"value": 1.0},
|
||||
time.Unix(1, 2),
|
||||
)
|
||||
|
||||
if err := store.WriteToShard(shardID, []tsdb.Point{pt, pt2}); err != nil {
|
||||
if err := store.WriteToShard(shardID, []models.Point{pt, pt2}); err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
|
||||
|
@ -238,14 +239,14 @@ func TestDropDatabase(t *testing.T) {
|
|||
store, executor := testStoreAndExecutor("")
|
||||
defer os.RemoveAll(store.Path())
|
||||
|
||||
pt := tsdb.NewPoint(
|
||||
pt := models.NewPoint(
|
||||
"cpu",
|
||||
map[string]string{"host": "server"},
|
||||
map[string]interface{}{"value": 1.0},
|
||||
time.Unix(1, 2),
|
||||
)
|
||||
|
||||
if err := store.WriteToShard(shardID, []tsdb.Point{pt}); err != nil {
|
||||
if err := store.WriteToShard(shardID, []models.Point{pt}); err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
|
||||
|
@ -290,7 +291,7 @@ func TestDropDatabase(t *testing.T) {
|
|||
executor.Store = store
|
||||
executor.ShardMapper = &testShardMapper{store: store}
|
||||
|
||||
if err := store.WriteToShard(shardID, []tsdb.Point{pt}); err == nil || err.Error() != "shard not found" {
|
||||
if err := store.WriteToShard(shardID, []models.Point{pt}); err == nil || err.Error() != "shard not found" {
|
||||
t.Fatalf("expected shard to not be found")
|
||||
}
|
||||
}
|
||||
|
|
|
@ -13,6 +13,7 @@ import (
|
|||
|
||||
"github.com/influxdb/influxdb"
|
||||
"github.com/influxdb/influxdb/influxql"
|
||||
"github.com/influxdb/influxdb/models"
|
||||
"github.com/influxdb/influxdb/tsdb/internal"
|
||||
|
||||
"github.com/boltdb/bolt"
|
||||
|
@ -191,7 +192,7 @@ type SeriesCreate struct {
|
|||
}
|
||||
|
||||
// WritePoints will write the raw data points and any new metadata to the index in the shard
|
||||
func (s *Shard) WritePoints(points []Point) error {
|
||||
func (s *Shard) WritePoints(points []models.Point) error {
|
||||
s.statMap.Add(statWriteReq, 1)
|
||||
|
||||
seriesToCreate, fieldsToCreate, seriesToAddShardTo, err := s.validateSeriesAndFields(points)
|
||||
|
@ -372,7 +373,7 @@ func (s *Shard) createFieldsAndMeasurements(fieldsToCreate []*FieldCreate) (map[
|
|||
}
|
||||
|
||||
// validateSeriesAndFields checks which series and fields are new and whose metadata should be saved and indexed
|
||||
func (s *Shard) validateSeriesAndFields(points []Point) ([]*SeriesCreate, []*FieldCreate, []string, error) {
|
||||
func (s *Shard) validateSeriesAndFields(points []models.Point) ([]*SeriesCreate, []*FieldCreate, []string, error) {
|
||||
var seriesToCreate []*SeriesCreate
|
||||
var fieldsToCreate []*FieldCreate
|
||||
var seriesToAddShardTo []string
|
||||
|
|
|
@ -10,6 +10,7 @@ import (
|
|||
"testing"
|
||||
"time"
|
||||
|
||||
"github.com/influxdb/influxdb/models"
|
||||
"github.com/influxdb/influxdb/tsdb"
|
||||
"github.com/influxdb/influxdb/tsdb/engine/b1"
|
||||
)
|
||||
|
@ -29,20 +30,20 @@ func TestShardWriteAndIndex(t *testing.T) {
|
|||
t.Fatalf("error openeing shard: %s", err.Error())
|
||||
}
|
||||
|
||||
pt := tsdb.NewPoint(
|
||||
pt := models.NewPoint(
|
||||
"cpu",
|
||||
map[string]string{"host": "server"},
|
||||
map[string]interface{}{"value": 1.0},
|
||||
time.Unix(1, 2),
|
||||
)
|
||||
|
||||
err := sh.WritePoints([]tsdb.Point{pt})
|
||||
err := sh.WritePoints([]models.Point{pt})
|
||||
if err != nil {
|
||||
t.Fatalf(err.Error())
|
||||
}
|
||||
|
||||
pt.SetTime(time.Unix(2, 3))
|
||||
err = sh.WritePoints([]tsdb.Point{pt})
|
||||
err = sh.WritePoints([]models.Point{pt})
|
||||
if err != nil {
|
||||
t.Fatalf(err.Error())
|
||||
}
|
||||
|
@ -76,7 +77,7 @@ func TestShardWriteAndIndex(t *testing.T) {
|
|||
|
||||
// and ensure that we can still write data
|
||||
pt.SetTime(time.Unix(2, 6))
|
||||
err = sh.WritePoints([]tsdb.Point{pt})
|
||||
err = sh.WritePoints([]models.Point{pt})
|
||||
if err != nil {
|
||||
t.Fatalf(err.Error())
|
||||
}
|
||||
|
@ -98,26 +99,26 @@ func TestShardWriteAddNewField(t *testing.T) {
|
|||
}
|
||||
defer sh.Close()
|
||||
|
||||
pt := tsdb.NewPoint(
|
||||
pt := models.NewPoint(
|
||||
"cpu",
|
||||
map[string]string{"host": "server"},
|
||||
map[string]interface{}{"value": 1.0},
|
||||
time.Unix(1, 2),
|
||||
)
|
||||
|
||||
err := sh.WritePoints([]tsdb.Point{pt})
|
||||
err := sh.WritePoints([]models.Point{pt})
|
||||
if err != nil {
|
||||
t.Fatalf(err.Error())
|
||||
}
|
||||
|
||||
pt = tsdb.NewPoint(
|
||||
pt = models.NewPoint(
|
||||
"cpu",
|
||||
map[string]string{"host": "server"},
|
||||
map[string]interface{}{"value": 1.0, "value2": 2.0},
|
||||
time.Unix(1, 2),
|
||||
)
|
||||
|
||||
err = sh.WritePoints([]tsdb.Point{pt})
|
||||
err = sh.WritePoints([]models.Point{pt})
|
||||
if err != nil {
|
||||
t.Fatalf(err.Error())
|
||||
}
|
||||
|
@ -158,7 +159,7 @@ func TestShard_Autoflush(t *testing.T) {
|
|||
|
||||
// Write a bunch of points.
|
||||
for i := 0; i < 100; i++ {
|
||||
if err := sh.WritePoints([]tsdb.Point{tsdb.NewPoint(
|
||||
if err := sh.WritePoints([]models.Point{models.NewPoint(
|
||||
fmt.Sprintf("cpu%d", i),
|
||||
map[string]string{"host": "server"},
|
||||
map[string]interface{}{"value": 1.0},
|
||||
|
@ -198,7 +199,7 @@ func TestShard_Autoflush_FlushInterval(t *testing.T) {
|
|||
|
||||
// Write some points.
|
||||
for i := 0; i < 100; i++ {
|
||||
if err := sh.WritePoints([]tsdb.Point{tsdb.NewPoint(
|
||||
if err := sh.WritePoints([]models.Point{models.NewPoint(
|
||||
fmt.Sprintf("cpu%d", i),
|
||||
map[string]string{"host": "server"},
|
||||
map[string]interface{}{"value": 1.0},
|
||||
|
@ -252,10 +253,10 @@ func benchmarkWritePoints(b *testing.B, mCnt, tkCnt, tvCnt, pntCnt int) {
|
|||
// Create index for the shard to use.
|
||||
index := tsdb.NewDatabaseIndex()
|
||||
// Generate point data to write to the shard.
|
||||
points := []tsdb.Point{}
|
||||
points := []models.Point{}
|
||||
for _, s := range series {
|
||||
for val := 0.0; val < float64(pntCnt); val++ {
|
||||
p := tsdb.NewPoint(s.Measurement, s.Series.Tags, map[string]interface{}{"value": val}, time.Now())
|
||||
p := models.NewPoint(s.Measurement, s.Series.Tags, map[string]interface{}{"value": val}, time.Now())
|
||||
points = append(points, p)
|
||||
}
|
||||
}
|
||||
|
@ -293,10 +294,10 @@ func benchmarkWritePointsExistingSeries(b *testing.B, mCnt, tkCnt, tvCnt, pntCnt
|
|||
// Create index for the shard to use.
|
||||
index := tsdb.NewDatabaseIndex()
|
||||
// Generate point data to write to the shard.
|
||||
points := []tsdb.Point{}
|
||||
points := []models.Point{}
|
||||
for _, s := range series {
|
||||
for val := 0.0; val < float64(pntCnt); val++ {
|
||||
p := tsdb.NewPoint(s.Measurement, s.Series.Tags, map[string]interface{}{"value": val}, time.Now())
|
||||
p := models.NewPoint(s.Measurement, s.Series.Tags, map[string]interface{}{"value": val}, time.Now())
|
||||
points = append(points, p)
|
||||
}
|
||||
}
|
||||
|
@ -326,7 +327,7 @@ func benchmarkWritePointsExistingSeries(b *testing.B, mCnt, tkCnt, tvCnt, pntCnt
|
|||
}
|
||||
}
|
||||
|
||||
func chunkedWrite(shard *tsdb.Shard, points []tsdb.Point) {
|
||||
func chunkedWrite(shard *tsdb.Shard, points []models.Point) {
|
||||
nPts := len(points)
|
||||
chunkSz := 10000
|
||||
start := 0
|
||||
|
|
|
@ -11,6 +11,7 @@ import (
|
|||
"sync"
|
||||
|
||||
"github.com/influxdb/influxdb/influxql"
|
||||
"github.com/influxdb/influxdb/models"
|
||||
)
|
||||
|
||||
func NewStore(path string) *Store {
|
||||
|
@ -328,7 +329,7 @@ func (s *Store) Open() error {
|
|||
return nil
|
||||
}
|
||||
|
||||
func (s *Store) WriteToShard(shardID uint64, points []Point) error {
|
||||
func (s *Store) WriteToShard(shardID uint64, points []models.Point) error {
|
||||
s.mu.RLock()
|
||||
defer s.mu.RUnlock()
|
||||
sh, ok := s.shards[shardID]
|
||||
|
|
|
@ -7,6 +7,7 @@ import (
|
|||
"testing"
|
||||
"time"
|
||||
|
||||
"github.com/influxdb/influxdb/models"
|
||||
"github.com/influxdb/influxdb/tsdb"
|
||||
)
|
||||
|
||||
|
@ -236,7 +237,7 @@ func TestStoreEnsureSeriesPersistedInNewShards(t *testing.T) {
|
|||
t.Fatalf("error creating shard: %v", err)
|
||||
}
|
||||
|
||||
p, _ := tsdb.ParsePoints([]byte("cpu val=1"))
|
||||
p, _ := models.ParsePoints([]byte("cpu val=1"))
|
||||
if err := s.WriteToShard(1, p); err != nil {
|
||||
t.Fatalf("error writing to shard: %v", err)
|
||||
}
|
||||
|
@ -282,10 +283,10 @@ func benchmarkStoreOpen(b *testing.B, mCnt, tkCnt, tvCnt, pntCnt, shardCnt int)
|
|||
// Generate test series (measurements + unique tag sets).
|
||||
series := genTestSeries(mCnt, tkCnt, tvCnt)
|
||||
// Generate point data to write to the shards.
|
||||
points := []tsdb.Point{}
|
||||
points := []models.Point{}
|
||||
for _, s := range series {
|
||||
for val := 0.0; val < float64(pntCnt); val++ {
|
||||
p := tsdb.NewPoint(s.Measurement, s.Series.Tags, map[string]interface{}{"value": val}, time.Now())
|
||||
p := models.NewPoint(s.Measurement, s.Series.Tags, map[string]interface{}{"value": val}, time.Now())
|
||||
points = append(points, p)
|
||||
}
|
||||
}
|
||||
|
@ -324,7 +325,7 @@ func benchmarkStoreOpen(b *testing.B, mCnt, tkCnt, tvCnt, pntCnt, shardCnt int)
|
|||
}
|
||||
}
|
||||
|
||||
func chunkedWriteStoreShard(store *tsdb.Store, shardID int, points []tsdb.Point) {
|
||||
func chunkedWriteStoreShard(store *tsdb.Store, shardID int, points []models.Point) {
|
||||
nPts := len(points)
|
||||
chunkSz := 10000
|
||||
start := 0
|
||||
|
|
Loading…
Reference in New Issue