forked from influxdata/influxdb
-
Notifications
You must be signed in to change notification settings - Fork 0
/
shard.go
1108 lines (928 loc) · 27.7 KB
/
shard.go
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864
865
866
867
868
869
870
871
872
873
874
875
876
877
878
879
880
881
882
883
884
885
886
887
888
889
890
891
892
893
894
895
896
897
898
899
900
901
902
903
904
905
906
907
908
909
910
911
912
913
914
915
916
917
918
919
920
921
922
923
924
925
926
927
928
929
930
931
932
933
934
935
936
937
938
939
940
941
942
943
944
945
946
947
948
949
950
951
952
953
954
955
956
957
958
959
960
961
962
963
964
965
966
967
968
969
970
971
972
973
974
975
976
977
978
979
980
981
982
983
984
985
986
987
988
989
990
991
992
993
994
995
996
997
998
999
1000
package tsdb
import (
"errors"
"fmt"
"io"
"log"
"os"
"path/filepath"
"sort"
"sync"
"sync/atomic"
"time"
"github.com/gogo/protobuf/proto"
"github.com/influxdata/influxdb/influxql"
"github.com/influxdata/influxdb/models"
internal "github.com/influxdata/influxdb/tsdb/internal"
)
// monitorStatInterval is the interval at which the shard is inspected
// for the purpose of determining certain monitoring statistics.
const monitorStatInterval = 30 * time.Second
const (
statWriteReq = "writeReq"
statWriteReqOK = "writeReqOk"
statWriteReqErr = "writeReqErr"
statSeriesCreate = "seriesCreate"
statFieldsCreate = "fieldsCreate"
statWritePointsErr = "writePointsErr"
statWritePointsOK = "writePointsOk"
statWriteBytes = "writeBytes"
statDiskBytes = "diskBytes"
)
var (
// ErrFieldOverflow is returned when too many fields are created on a measurement.
ErrFieldOverflow = errors.New("field overflow")
// ErrFieldTypeConflict is returned when a new field already exists with a different type.
ErrFieldTypeConflict = errors.New("field type conflict")
// ErrFieldNotFound is returned when a field cannot be found.
ErrFieldNotFound = errors.New("field not found")
// ErrFieldUnmappedID is returned when the system is presented, during decode, with a field ID
// there is no mapping for.
ErrFieldUnmappedID = errors.New("field ID not mapped")
// ErrEngineClosed is returned when a caller attempts indirectly to
// access the shard's underlying engine.
ErrEngineClosed = errors.New("engine is closed")
// ErrShardDisabled is returned when a the shard is not available for
// queries or writes.
ErrShardDisabled = errors.New("shard is disabled")
)
// A ShardError implements the error interface, and contains extra
// context about the shard that generated the error.
type ShardError struct {
id uint64
Err error
}
// NewShardError returns a new ShardError.
func NewShardError(id uint64, err error) error {
if err == nil {
return nil
}
return ShardError{id: id, Err: err}
}
func (e ShardError) Error() string {
return fmt.Sprintf("[shard %d] %s", e.id, e.Err)
}
// Shard represents a self-contained time series database. An inverted index of
// the measurement and tag data is kept along with the raw time series data.
// Data can be split across many shards. The query engine in TSDB is responsible
// for combining the output of many shards into a single query result.
type Shard struct {
index *DatabaseIndex
path string
walPath string
id uint64
database string
retentionPolicy string
options EngineOptions
mu sync.RWMutex
engine Engine
closing chan struct{}
enabled bool
// expvar-based stats.
stats *ShardStatistics
statTags models.Tags
logger *log.Logger
// used by logger. Referenced so it can be passed down to new caches.
logOutput io.Writer
EnableOnOpen bool
}
// NewShard returns a new initialized Shard. walPath doesn't apply to the b1 type index
func NewShard(id uint64, index *DatabaseIndex, path string, walPath string, options EngineOptions) *Shard {
db, rp := DecodeStorePath(path)
s := &Shard{
index: index,
id: id,
path: path,
walPath: walPath,
options: options,
closing: make(chan struct{}),
stats: &ShardStatistics{},
statTags: map[string]string{
"path": path,
"id": fmt.Sprintf("%d", id),
"database": db,
"retentionPolicy": rp,
},
database: db,
retentionPolicy: rp,
logger: log.New(os.Stderr, "[shard] ", log.LstdFlags),
logOutput: os.Stderr,
EnableOnOpen: true,
}
return s
}
// SetLogOutput sets the writer to which log output will be written. It is safe
// for concurrent use.
func (s *Shard) SetLogOutput(w io.Writer) {
s.logger.SetOutput(w)
if err := s.ready(); err == nil {
s.engine.SetLogOutput(w)
}
s.mu.Lock()
s.logOutput = w
s.mu.Unlock()
}
// SetEnabled enables the shard for queries and write. When disabled, all
// writes and queries return an error and compactions are stopped for the shard.
func (s *Shard) SetEnabled(enabled bool) {
s.mu.Lock()
// Prevent writes and queries
s.enabled = enabled
if s.engine != nil {
// Disable background compactions and snapshotting
s.engine.SetEnabled(enabled)
}
s.mu.Unlock()
}
// ShardStatistics maintains statistics for a shard.
type ShardStatistics struct {
WriteReq int64
WriteReqOK int64
WriteReqErr int64
SeriesCreated int64
FieldsCreated int64
WritePointsErr int64
WritePointsOK int64
BytesWritten int64
DiskBytes int64
}
// Statistics returns statistics for periodic monitoring.
func (s *Shard) Statistics(tags map[string]string) []models.Statistic {
if err := s.ready(); err != nil {
return nil
}
tags = s.statTags.Merge(tags)
seriesN, _ := s.engine.SeriesCount()
statistics := []models.Statistic{{
Name: "shard",
Tags: models.Tags(tags).Merge(map[string]string{"engine": s.options.EngineVersion}),
Values: map[string]interface{}{
statWriteReq: atomic.LoadInt64(&s.stats.WriteReq),
statWriteReqOK: atomic.LoadInt64(&s.stats.WriteReqOK),
statWriteReqErr: atomic.LoadInt64(&s.stats.WriteReqErr),
statSeriesCreate: seriesN,
statWritePointsErr: atomic.LoadInt64(&s.stats.WritePointsErr),
statWritePointsOK: atomic.LoadInt64(&s.stats.WritePointsOK),
statWriteBytes: atomic.LoadInt64(&s.stats.BytesWritten),
statDiskBytes: atomic.LoadInt64(&s.stats.DiskBytes),
},
}}
statistics = append(statistics, s.engine.Statistics(tags)...)
return statistics
}
// Path returns the path set on the shard when it was created.
func (s *Shard) Path() string { return s.path }
// Open initializes and opens the shard's store.
func (s *Shard) Open() error {
if err := func() error {
s.mu.Lock()
defer s.mu.Unlock()
// Return if the shard is already open
if s.engine != nil {
return nil
}
// Initialize underlying engine.
e, err := NewEngine(s.path, s.walPath, s.options)
if err != nil {
return err
}
// Set log output on the engine.
e.SetLogOutput(s.logOutput)
// Disable compactions while loading the index
e.SetEnabled(false)
// Open engine.
if err := e.Open(); err != nil {
return err
}
// Load metadata index.
start := time.Now()
if err := e.LoadMetadataIndex(s.id, s.index); err != nil {
return err
}
count := s.index.SeriesShardN(s.id)
atomic.AddInt64(&s.stats.SeriesCreated, int64(count))
s.engine = e
s.logger.Printf("%s database index loaded in %s", s.path, time.Now().Sub(start))
go s.monitorSize()
return nil
}(); err != nil {
s.close()
return NewShardError(s.id, err)
}
if s.EnableOnOpen {
// enable writes, queries and compactions
s.SetEnabled(true)
}
return nil
}
// Close shuts down the shard's store.
func (s *Shard) Close() error {
s.mu.Lock()
defer s.mu.Unlock()
return s.close()
}
func (s *Shard) close() error {
if s.engine == nil {
return nil
}
// Close the closing channel at most once.
select {
case <-s.closing:
default:
close(s.closing)
}
// Don't leak our shard ID and series keys in the index
s.index.RemoveShard(s.id)
err := s.engine.Close()
if err == nil {
s.engine = nil
}
return err
}
// ready determines if the Shard is ready for queries or writes.
// It returns nil if ready, otherwise ErrShardClosed or ErrShardDiabled
func (s *Shard) ready() error {
var err error
s.mu.RLock()
if s.engine == nil {
err = ErrEngineClosed
} else if !s.enabled {
err = ErrShardDisabled
}
s.mu.RUnlock()
return err
}
// DiskSize returns the size on disk of this shard
func (s *Shard) DiskSize() (int64, error) {
var size int64
err := filepath.Walk(s.path, func(_ string, fi os.FileInfo, err error) error {
if err != nil {
return err
}
if !fi.IsDir() {
size += fi.Size()
}
return err
})
if err != nil {
return 0, err
}
err = filepath.Walk(s.walPath, func(_ string, fi os.FileInfo, err error) error {
if err != nil {
return err
}
if !fi.IsDir() {
size += fi.Size()
}
return err
})
return size, err
}
// FieldCreate holds information for a field to create on a measurement
type FieldCreate struct {
Measurement string
Field *Field
}
// SeriesCreate holds information for a series to create
type SeriesCreate struct {
Measurement string
Series *Series
}
// WritePoints will write the raw data points and any new metadata to the index in the shard
func (s *Shard) WritePoints(points []models.Point) error {
if err := s.ready(); err != nil {
return err
}
s.mu.RLock()
defer s.mu.RUnlock()
atomic.AddInt64(&s.stats.WriteReq, 1)
fieldsToCreate, err := s.validateSeriesAndFields(points)
if err != nil {
return err
}
atomic.AddInt64(&s.stats.FieldsCreated, int64(len(fieldsToCreate)))
// add any new fields and keep track of what needs to be saved
if err := s.createFieldsAndMeasurements(fieldsToCreate); err != nil {
return err
}
// Write to the engine.
if err := s.engine.WritePoints(points); err != nil {
atomic.AddInt64(&s.stats.WritePointsErr, int64(len(points)))
atomic.AddInt64(&s.stats.WriteReqErr, 1)
return fmt.Errorf("engine: %s", err)
}
atomic.AddInt64(&s.stats.WritePointsOK, int64(len(points)))
atomic.AddInt64(&s.stats.WriteReqOK, 1)
return nil
}
func (s *Shard) ContainsSeries(seriesKeys []string) (map[string]bool, error) {
if err := s.ready(); err != nil {
return nil, err
}
return s.engine.ContainsSeries(seriesKeys)
}
// DeleteSeries deletes a list of series.
func (s *Shard) DeleteSeries(seriesKeys []string) error {
if err := s.ready(); err != nil {
return err
}
if err := s.engine.DeleteSeries(seriesKeys); err != nil {
return err
}
return nil
}
// DeleteSeriesRange deletes all values from for seriesKeys between min and max (inclusive)
func (s *Shard) DeleteSeriesRange(seriesKeys []string, min, max int64) error {
if err := s.ready(); err != nil {
return err
}
if err := s.engine.DeleteSeriesRange(seriesKeys, min, max); err != nil {
return err
}
return nil
}
// DeleteMeasurement deletes a measurement and all underlying series.
func (s *Shard) DeleteMeasurement(name string, seriesKeys []string) error {
if err := s.ready(); err != nil {
return err
}
if err := s.engine.DeleteMeasurement(name, seriesKeys); err != nil {
return err
}
return nil
}
func (s *Shard) createFieldsAndMeasurements(fieldsToCreate []*FieldCreate) error {
if len(fieldsToCreate) == 0 {
return nil
}
// add fields
for _, f := range fieldsToCreate {
m := s.engine.MeasurementFields(f.Measurement)
// Add the field to the in memory index
if err := m.CreateFieldIfNotExists(f.Field.Name, f.Field.Type, false); err != nil {
return err
}
// ensure the measurement is in the index and the field is there
measurement := s.index.CreateMeasurementIndexIfNotExists(f.Measurement)
measurement.SetFieldName(f.Field.Name)
}
return nil
}
// validateSeriesAndFields checks which series and fields are new and whose metadata should be saved and indexed
func (s *Shard) validateSeriesAndFields(points []models.Point) ([]*FieldCreate, error) {
var fieldsToCreate []*FieldCreate
// get the shard mutex for locally defined fields
for _, p := range points {
// see if the series should be added to the index
key := string(p.Key())
ss := s.index.Series(key)
if ss == nil {
if s.options.Config.MaxSeriesPerDatabase > 0 && len(s.index.series)+1 > s.options.Config.MaxSeriesPerDatabase {
return nil, fmt.Errorf("max series per database exceeded: %s", key)
}
ss = NewSeries(key, p.Tags())
atomic.AddInt64(&s.stats.SeriesCreated, 1)
}
ss = s.index.CreateSeriesIndexIfNotExists(p.Name(), ss)
s.index.AssignShard(ss.Key, s.id)
// see if the field definitions need to be saved to the shard
mf := s.engine.MeasurementFields(p.Name())
if mf == nil {
for name, value := range p.Fields() {
fieldsToCreate = append(fieldsToCreate, &FieldCreate{p.Name(), &Field{Name: name, Type: influxql.InspectDataType(value)}})
}
continue // skip validation since all fields are new
}
// validate field types and encode data
for name, value := range p.Fields() {
if f := mf.Field(name); f != nil {
// Field present in shard metadata, make sure there is no type conflict.
if f.Type != influxql.InspectDataType(value) {
return nil, fmt.Errorf("field type conflict: input field \"%s\" on measurement \"%s\" is type %T, already exists as type %s", name, p.Name(), value, f.Type)
}
continue // Field is present, and it's of the same type. Nothing more to do.
}
fieldsToCreate = append(fieldsToCreate, &FieldCreate{p.Name(), &Field{Name: name, Type: influxql.InspectDataType(value)}})
}
}
return fieldsToCreate, nil
}
// SeriesCount returns the number of series buckets on the shard.
func (s *Shard) SeriesCount() (int, error) {
if err := s.ready(); err != nil {
return 0, err
}
return s.engine.SeriesCount()
}
// WriteTo writes the shard's data to w.
func (s *Shard) WriteTo(w io.Writer) (int64, error) {
if err := s.ready(); err != nil {
return 0, err
}
n, err := s.engine.WriteTo(w)
atomic.AddInt64(&s.stats.BytesWritten, int64(n))
return n, err
}
// CreateIterator returns an iterator for the data in the shard.
func (s *Shard) CreateIterator(opt influxql.IteratorOptions) (influxql.Iterator, error) {
if err := s.ready(); err != nil {
return nil, err
}
if influxql.Sources(opt.Sources).HasSystemSource() {
return s.createSystemIterator(opt)
}
opt.Sources = influxql.Sources(opt.Sources).Filter(s.database, s.retentionPolicy)
return s.engine.CreateIterator(opt)
}
// createSystemIterator returns an iterator for a system source.
func (s *Shard) createSystemIterator(opt influxql.IteratorOptions) (influxql.Iterator, error) {
// Only support a single system source.
if len(opt.Sources) > 1 {
return nil, errors.New("cannot select from multiple system sources")
}
m := opt.Sources[0].(*influxql.Measurement)
switch m.Name {
case "_fieldKeys":
return NewFieldKeysIterator(s, opt)
case "_series":
return NewSeriesIterator(s, opt)
case "_tagKeys":
return NewTagKeysIterator(s, opt)
default:
return nil, fmt.Errorf("unknown system source: %s", m.Name)
}
}
// FieldDimensions returns unique sets of fields and dimensions across a list of sources.
func (s *Shard) FieldDimensions(sources influxql.Sources) (fields map[string]influxql.DataType, dimensions map[string]struct{}, err error) {
if err := s.ready(); err != nil {
return nil, nil, err
}
if sources.HasSystemSource() {
// Only support a single system source.
if len(sources) > 1 {
return nil, nil, errors.New("cannot select from multiple system sources")
}
switch m := sources[0].(type) {
case *influxql.Measurement:
switch m.Name {
case "_fieldKeys":
return map[string]influxql.DataType{
"fieldKey": influxql.String,
"fieldType": influxql.String,
}, nil, nil
case "_series":
return map[string]influxql.DataType{
"key": influxql.String,
}, nil, nil
case "_tagKeys":
return map[string]influxql.DataType{
"tagKey": influxql.String,
}, nil, nil
}
}
return nil, nil, nil
}
fields = make(map[string]influxql.DataType)
dimensions = make(map[string]struct{})
for _, src := range sources {
switch m := src.(type) {
case *influxql.Measurement:
// Retrieve measurement.
mm := s.index.Measurement(m.Name)
if mm == nil {
continue
}
// Append fields and dimensions.
mf := s.engine.MeasurementFields(m.Name)
if mf != nil {
for name, typ := range mf.FieldSet() {
fields[name] = typ
}
}
for _, key := range mm.TagKeys() {
dimensions[key] = struct{}{}
}
}
}
return
}
// ExpandSources expands regex sources and removes duplicates.
// NOTE: sources must be normalized (db and rp set) before calling this function.
func (s *Shard) ExpandSources(sources influxql.Sources) (influxql.Sources, error) {
// Use a map as a set to prevent duplicates.
set := map[string]influxql.Source{}
// Iterate all sources, expanding regexes when they're found.
for _, source := range sources {
switch src := source.(type) {
case *influxql.Measurement:
// Add non-regex measurements directly to the set.
if src.Regex == nil {
set[src.String()] = src
continue
}
// Loop over matching measurements.
for _, m := range s.index.MeasurementsByRegex(src.Regex.Val) {
other := &influxql.Measurement{
Database: src.Database,
RetentionPolicy: src.RetentionPolicy,
Name: m.Name,
}
set[other.String()] = other
}
default:
return nil, fmt.Errorf("expandSources: unsupported source type: %T", source)
}
}
// Convert set to sorted slice.
names := make([]string, 0, len(set))
for name := range set {
names = append(names, name)
}
sort.Strings(names)
// Convert set to a list of Sources.
expanded := make(influxql.Sources, 0, len(set))
for _, name := range names {
expanded = append(expanded, set[name])
}
return expanded, nil
}
// Restore restores data to the underlying engine for the shard.
// The shard is reopened after restore.
func (s *Shard) Restore(r io.Reader, basePath string) error {
s.mu.Lock()
// Restore to engine.
if err := s.engine.Restore(r, basePath); err != nil {
s.mu.Unlock()
return err
}
s.mu.Unlock()
// Close shard.
if err := s.Close(); err != nil {
return err
}
// Reopen engine.
return s.Open()
}
// CreateSnapshot will return a path to a temp directory
// containing hard links to the underlying shard files
func (s *Shard) CreateSnapshot() (string, error) {
s.mu.RLock()
defer s.mu.RUnlock()
return s.engine.CreateSnapshot()
}
func (s *Shard) monitorSize() {
t := time.NewTicker(monitorStatInterval)
defer t.Stop()
for {
select {
case <-s.closing:
return
case <-t.C:
size, err := s.DiskSize()
if err != nil {
s.logger.Printf("Error collecting shard size: %v", err)
continue
}
atomic.StoreInt64(&s.stats.DiskBytes, size)
}
}
}
// Shards represents a sortable list of shards.
type Shards []*Shard
func (a Shards) Len() int { return len(a) }
func (a Shards) Less(i, j int) bool { return a[i].id < a[j].id }
func (a Shards) Swap(i, j int) { a[i], a[j] = a[j], a[i] }
// MeasurementFields holds the fields of a measurement and their codec.
type MeasurementFields struct {
mu sync.RWMutex
fields map[string]*Field
}
func NewMeasurementFields() *MeasurementFields {
return &MeasurementFields{fields: make(map[string]*Field)}
}
// MarshalBinary encodes the object to a binary format.
func (m *MeasurementFields) MarshalBinary() ([]byte, error) {
m.mu.RLock()
defer m.mu.RUnlock()
var pb internal.MeasurementFields
for _, f := range m.fields {
id := int32(f.ID)
name := f.Name
t := int32(f.Type)
pb.Fields = append(pb.Fields, &internal.Field{ID: &id, Name: &name, Type: &t})
}
return proto.Marshal(&pb)
}
// UnmarshalBinary decodes the object from a binary format.
func (m *MeasurementFields) UnmarshalBinary(buf []byte) error {
m.mu.Lock()
defer m.mu.Unlock()
var pb internal.MeasurementFields
if err := proto.Unmarshal(buf, &pb); err != nil {
return err
}
m.fields = make(map[string]*Field, len(pb.Fields))
for _, f := range pb.Fields {
m.fields[f.GetName()] = &Field{ID: uint8(f.GetID()), Name: f.GetName(), Type: influxql.DataType(f.GetType())}
}
return nil
}
// CreateFieldIfNotExists creates a new field with an autoincrementing ID.
// Returns an error if 255 fields have already been created on the measurement or
// the fields already exists with a different type.
func (m *MeasurementFields) CreateFieldIfNotExists(name string, typ influxql.DataType, limitCount bool) error {
m.mu.RLock()
// Ignore if the field already exists.
if f := m.fields[name]; f != nil {
if f.Type != typ {
m.mu.RUnlock()
return ErrFieldTypeConflict
}
m.mu.RUnlock()
return nil
}
m.mu.RUnlock()
m.mu.Lock()
defer m.mu.Unlock()
if f := m.fields[name]; f != nil {
return nil
}
// Create and append a new field.
f := &Field{
ID: uint8(len(m.fields) + 1),
Name: name,
Type: typ,
}
m.fields[name] = f
return nil
}
func (m *MeasurementFields) Field(name string) *Field {
m.mu.RLock()
f := m.fields[name]
m.mu.RUnlock()
return f
}
func (m *MeasurementFields) FieldSet() map[string]influxql.DataType {
m.mu.RLock()
defer m.mu.RUnlock()
fields := make(map[string]influxql.DataType)
for name, f := range m.fields {
fields[name] = f.Type
}
return fields
}
// Field represents a series field.
type Field struct {
ID uint8 `json:"id,omitempty"`
Name string `json:"name,omitempty"`
Type influxql.DataType `json:"type,omitempty"`
}
// shardIteratorCreator creates iterators for a local shard.
// This simply wraps the shard so that Close() does not close the underlying shard.
type shardIteratorCreator struct {
sh *Shard
maxSeriesN int
}
func (ic *shardIteratorCreator) Close() error { return nil }
func (ic *shardIteratorCreator) CreateIterator(opt influxql.IteratorOptions) (influxql.Iterator, error) {
itr, err := ic.sh.CreateIterator(opt)
if err != nil {
return nil, err
} else if itr == nil {
return nil, nil
}
// Enforce series limit at creation time.
if ic.maxSeriesN > 0 {
stats := itr.Stats()
if stats.SeriesN > ic.maxSeriesN {
itr.Close()
return nil, fmt.Errorf("max select series count exceeded: %d series", stats.SeriesN)
}
}
return itr, nil
}
func (ic *shardIteratorCreator) FieldDimensions(sources influxql.Sources) (fields map[string]influxql.DataType, dimensions map[string]struct{}, err error) {
return ic.sh.FieldDimensions(sources)
}
func (ic *shardIteratorCreator) ExpandSources(sources influxql.Sources) (influxql.Sources, error) {
return ic.sh.ExpandSources(sources)
}
func NewFieldKeysIterator(sh *Shard, opt influxql.IteratorOptions) (influxql.Iterator, error) {
itr := &fieldKeysIterator{sh: sh}
// Retrieve measurements from shard. Filter if condition specified.
if opt.Condition == nil {
itr.mms = sh.index.Measurements()
} else {
mms, _, err := sh.index.measurementsByExpr(opt.Condition)
if err != nil {
return nil, err
}
itr.mms = mms
}
// Sort measurements by name.
sort.Sort(itr.mms)
return itr, nil
}
// fieldKeysIterator iterates over measurements and gets field keys from each measurement.
type fieldKeysIterator struct {
sh *Shard
mms Measurements // remaining measurements
buf struct {
mm *Measurement // current measurement
fields []Field // current measurement's fields
}
}
// Stats returns stats about the points processed.
func (itr *fieldKeysIterator) Stats() influxql.IteratorStats { return influxql.IteratorStats{} }
// Close closes the iterator.
func (itr *fieldKeysIterator) Close() error { return nil }
// Next emits the next tag key name.
func (itr *fieldKeysIterator) Next() (*influxql.FloatPoint, error) {
for {
// If there are no more keys then move to the next measurements.
if len(itr.buf.fields) == 0 {
if len(itr.mms) == 0 {
return nil, nil
}
itr.buf.mm = itr.mms[0]
mf := itr.sh.engine.MeasurementFields(itr.buf.mm.Name)
if mf != nil {
fset := mf.FieldSet()
if len(fset) == 0 {
itr.mms = itr.mms[1:]
continue
}
keys := make([]string, 0, len(fset))
for k := range fset {
keys = append(keys, k)
}
sort.Strings(keys)
itr.buf.fields = make([]Field, len(keys))
for i, name := range keys {
itr.buf.fields[i] = Field{Name: name, Type: fset[name]}
}
}
itr.mms = itr.mms[1:]
continue
}
// Return next key.
field := itr.buf.fields[0]
p := &influxql.FloatPoint{
Name: itr.buf.mm.Name,
Aux: []interface{}{field.Name, field.Type.String()},
}
itr.buf.fields = itr.buf.fields[1:]
return p, nil
}
}
// seriesIterator emits series ids.
type seriesIterator struct {
mms Measurements
keys struct {
buf []string
i int
}
point influxql.FloatPoint // reusable point
opt influxql.IteratorOptions
}
// NewSeriesIterator returns a new instance of SeriesIterator.
func NewSeriesIterator(sh *Shard, opt influxql.IteratorOptions) (influxql.Iterator, error) {
// Only equality operators are allowed.
var err error
influxql.WalkFunc(opt.Condition, func(n influxql.Node) {
switch n := n.(type) {
case *influxql.BinaryExpr:
switch n.Op {
case influxql.EQ, influxql.NEQ, influxql.EQREGEX, influxql.NEQREGEX,
influxql.OR, influxql.AND:
default:
err = errors.New("invalid tag comparison operator")
}
}
})
if err != nil {
return nil, err
}
// Read and sort all measurements.
mms := sh.index.Measurements()
sort.Sort(mms)
return &seriesIterator{
mms: mms,
point: influxql.FloatPoint{
Aux: make([]interface{}, len(opt.Aux)),
},
opt: opt,
}, nil
}
// Stats returns stats about the points processed.
func (itr *seriesIterator) Stats() influxql.IteratorStats { return influxql.IteratorStats{} }
// Close closes the iterator.
func (itr *seriesIterator) Close() error { return nil }
// Next emits the next point in the iterator.
func (itr *seriesIterator) Next() (*influxql.FloatPoint, error) {
for {
// Load next measurement's keys if there are no more remaining.
if itr.keys.i >= len(itr.keys.buf) {
if err := itr.nextKeys(); err != nil {
return nil, err
}
if len(itr.keys.buf) == 0 {
return nil, nil
}
}
// Read the next key.
key := itr.keys.buf[itr.keys.i]
itr.keys.i++
// Write auxiliary fields.
for i, f := range itr.opt.Aux {