2014-02-02 23:18:19 +00:00
|
|
|
package datastore
|
|
|
|
|
|
|
|
import (
|
2014-02-06 18:17:59 +00:00
|
|
|
"bytes"
|
|
|
|
"encoding/binary"
|
|
|
|
"errors"
|
|
|
|
"fmt"
|
|
|
|
"math"
|
2014-02-08 21:37:13 +00:00
|
|
|
"regexp"
|
|
|
|
"time"
|
2014-03-28 16:38:35 +00:00
|
|
|
|
|
|
|
"code.google.com/p/goprotobuf/proto"
|
|
|
|
log "code.google.com/p/log4go"
|
2014-06-27 16:57:06 +00:00
|
|
|
"github.com/influxdb/influxdb/cluster"
|
|
|
|
"github.com/influxdb/influxdb/common"
|
|
|
|
"github.com/influxdb/influxdb/datastore/storage"
|
2014-07-14 22:37:22 +00:00
|
|
|
"github.com/influxdb/influxdb/metastore"
|
2014-06-27 16:57:06 +00:00
|
|
|
"github.com/influxdb/influxdb/parser"
|
|
|
|
"github.com/influxdb/influxdb/protocol"
|
2014-02-02 23:18:19 +00:00
|
|
|
)
|
|
|
|
|
2014-06-13 22:43:58 +00:00
|
|
|
type Shard struct {
|
2014-06-04 18:31:39 +00:00
|
|
|
db storage.Engine
|
2014-03-07 19:13:15 +00:00
|
|
|
closed bool
|
|
|
|
pointBatchSize int
|
2014-05-14 16:36:56 +00:00
|
|
|
writeBatchSize int
|
2014-07-10 21:59:45 +00:00
|
|
|
metaStore *metastore.Store
|
2014-02-02 23:18:19 +00:00
|
|
|
}
|
|
|
|
|
2014-07-10 21:59:45 +00:00
|
|
|
func NewShard(db storage.Engine, pointBatchSize, writeBatchSize int, metaStore *metastore.Store) (*Shard, error) {
|
2014-06-13 22:43:58 +00:00
|
|
|
return &Shard{
|
2014-03-07 19:13:15 +00:00
|
|
|
db: db,
|
|
|
|
pointBatchSize: pointBatchSize,
|
2014-05-14 16:36:56 +00:00
|
|
|
writeBatchSize: writeBatchSize,
|
2014-07-10 21:59:45 +00:00
|
|
|
metaStore: metaStore,
|
2014-02-06 18:17:59 +00:00
|
|
|
}, nil
|
2014-02-02 23:18:19 +00:00
|
|
|
}
|
|
|
|
|
2014-06-13 22:43:58 +00:00
|
|
|
func (self *Shard) Write(database string, series []*protocol.Series) error {
|
2014-06-13 18:26:45 +00:00
|
|
|
wb := make([]storage.Write, 0)
|
2014-02-06 18:17:59 +00:00
|
|
|
|
2014-05-14 17:58:01 +00:00
|
|
|
for _, s := range series {
|
|
|
|
if len(s.Points) == 0 {
|
|
|
|
return errors.New("Unable to write no data. Series was nil or had no points.")
|
2014-02-06 18:17:59 +00:00
|
|
|
}
|
2014-07-10 21:59:45 +00:00
|
|
|
if len(s.FieldIds) == 0 {
|
|
|
|
return errors.New("Unable to write points without fields")
|
|
|
|
}
|
2014-02-06 18:17:59 +00:00
|
|
|
|
2014-05-14 17:58:01 +00:00
|
|
|
count := 0
|
2014-07-10 21:59:45 +00:00
|
|
|
for fieldIndex, id := range s.FieldIds {
|
2014-05-14 17:58:01 +00:00
|
|
|
for _, point := range s.Points {
|
2014-07-10 21:59:45 +00:00
|
|
|
// keyBuffer and dataBuffer have to be recreated since we are
|
|
|
|
// batching the writes, otherwise new writes will override the
|
|
|
|
// old writes that are still in memory
|
2014-06-04 18:31:39 +00:00
|
|
|
keyBuffer := bytes.NewBuffer(make([]byte, 0, 24))
|
|
|
|
dataBuffer := proto.NewBuffer(nil)
|
2014-07-10 21:59:45 +00:00
|
|
|
var err error
|
2014-05-14 17:58:01 +00:00
|
|
|
|
2014-07-10 21:59:45 +00:00
|
|
|
binary.Write(keyBuffer, binary.BigEndian, &id)
|
2014-05-14 17:58:01 +00:00
|
|
|
timestamp := self.convertTimestampToUint(point.GetTimestampInMicroseconds())
|
|
|
|
// pass the uint64 by reference so binary.Write() doesn't create a new buffer
|
|
|
|
// see the source code for intDataSize() in binary.go
|
|
|
|
binary.Write(keyBuffer, binary.BigEndian, ×tamp)
|
|
|
|
binary.Write(keyBuffer, binary.BigEndian, point.SequenceNumber)
|
|
|
|
pointKey := keyBuffer.Bytes()
|
|
|
|
|
|
|
|
if point.Values[fieldIndex].GetIsNull() {
|
2014-06-23 18:01:53 +00:00
|
|
|
wb = append(wb, storage.Write{Key: pointKey, Value: nil})
|
2014-05-14 17:58:01 +00:00
|
|
|
goto check
|
|
|
|
}
|
|
|
|
|
|
|
|
err = dataBuffer.Marshal(point.Values[fieldIndex])
|
2014-04-24 21:42:01 +00:00
|
|
|
if err != nil {
|
|
|
|
return err
|
|
|
|
}
|
2014-06-23 18:01:53 +00:00
|
|
|
wb = append(wb, storage.Write{Key: pointKey, Value: dataBuffer.Bytes()})
|
2014-05-14 17:58:01 +00:00
|
|
|
check:
|
|
|
|
count++
|
|
|
|
if count >= self.writeBatchSize {
|
2014-06-04 18:31:39 +00:00
|
|
|
err = self.db.BatchPut(wb)
|
2014-05-14 17:58:01 +00:00
|
|
|
if err != nil {
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
count = 0
|
2014-06-04 18:31:39 +00:00
|
|
|
wb = make([]storage.Write, 0, self.writeBatchSize)
|
2014-05-14 17:58:01 +00:00
|
|
|
}
|
2014-04-24 21:42:01 +00:00
|
|
|
}
|
2014-02-06 18:17:59 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2014-06-04 18:31:39 +00:00
|
|
|
return self.db.BatchPut(wb)
|
2014-02-02 23:18:19 +00:00
|
|
|
}
|
|
|
|
|
2014-06-13 22:43:58 +00:00
|
|
|
func (self *Shard) Query(querySpec *parser.QuerySpec, processor cluster.QueryProcessor) error {
|
2014-02-17 00:21:28 +00:00
|
|
|
if querySpec.IsListSeriesQuery() {
|
2014-08-15 17:18:43 +00:00
|
|
|
return fmt.Errorf("List series queries should never come to the shard")
|
2014-02-17 15:37:48 +00:00
|
|
|
} else if querySpec.IsDeleteFromSeriesQuery() {
|
|
|
|
return self.executeDeleteQuery(querySpec, processor)
|
2014-02-17 00:21:28 +00:00
|
|
|
}
|
|
|
|
|
2014-02-08 21:37:13 +00:00
|
|
|
seriesAndColumns := querySpec.SelectQuery().GetReferencedColumns()
|
|
|
|
|
|
|
|
if !self.hasReadAccess(querySpec) {
|
|
|
|
return errors.New("User does not have access to one or more of the series requested.")
|
|
|
|
}
|
|
|
|
|
|
|
|
for series, columns := range seriesAndColumns {
|
|
|
|
if regex, ok := series.GetCompiledRegex(); ok {
|
2014-07-10 21:59:45 +00:00
|
|
|
seriesNames := self.metaStore.GetSeriesForDatabaseAndRegex(querySpec.Database(), regex)
|
2014-02-08 21:37:13 +00:00
|
|
|
for _, name := range seriesNames {
|
|
|
|
if !querySpec.HasReadAccess(name) {
|
|
|
|
continue
|
|
|
|
}
|
|
|
|
err := self.executeQueryForSeries(querySpec, name, columns, processor)
|
|
|
|
if err != nil {
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
}
|
|
|
|
} else {
|
|
|
|
err := self.executeQueryForSeries(querySpec, series.Name, columns, processor)
|
|
|
|
if err != nil {
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
2014-06-13 22:43:58 +00:00
|
|
|
func (self *Shard) IsClosed() bool {
|
2014-03-05 22:09:01 +00:00
|
|
|
return self.closed
|
|
|
|
}
|
|
|
|
|
2014-06-13 22:43:58 +00:00
|
|
|
func (self *Shard) executeQueryForSeries(querySpec *parser.QuerySpec, seriesName string, columns []string, processor cluster.QueryProcessor) error {
|
2014-02-08 21:37:13 +00:00
|
|
|
startTimeBytes := self.byteArrayForTime(querySpec.GetStartTime())
|
|
|
|
endTimeBytes := self.byteArrayForTime(querySpec.GetEndTime())
|
|
|
|
|
|
|
|
fields, err := self.getFieldsForSeries(querySpec.Database(), seriesName, columns)
|
|
|
|
if err != nil {
|
2014-07-10 21:59:45 +00:00
|
|
|
log.Error("Error looking up fields for %s: %s", seriesName, err)
|
|
|
|
return err
|
2014-02-08 21:37:13 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
fieldCount := len(fields)
|
2014-04-24 23:55:20 +00:00
|
|
|
rawColumnValues := make([]rawColumnValue, fieldCount, fieldCount)
|
2014-02-08 21:37:13 +00:00
|
|
|
query := querySpec.SelectQuery()
|
|
|
|
|
2014-02-19 22:30:05 +00:00
|
|
|
aliases := query.GetTableAliases(seriesName)
|
2014-02-08 21:37:13 +00:00
|
|
|
if querySpec.IsSinglePointQuery() {
|
|
|
|
series, err := self.fetchSinglePoint(querySpec, seriesName, fields)
|
|
|
|
if err != nil {
|
2014-05-20 17:14:28 +00:00
|
|
|
log.Error("Error reading a single point: %s", err)
|
2014-02-08 21:37:13 +00:00
|
|
|
return err
|
|
|
|
}
|
|
|
|
if len(series.Points) > 0 {
|
|
|
|
processor.YieldPoint(series.Name, series.Fields, series.Points[0])
|
|
|
|
}
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
|
|
|
fieldNames, iterators := self.getIterators(fields, startTimeBytes, endTimeBytes, query.Ascending)
|
2014-03-03 21:15:36 +00:00
|
|
|
defer func() {
|
|
|
|
for _, it := range iterators {
|
|
|
|
it.Close()
|
|
|
|
}
|
|
|
|
}()
|
2014-02-08 21:37:13 +00:00
|
|
|
|
2014-03-07 19:13:15 +00:00
|
|
|
seriesOutgoing := &protocol.Series{Name: protocol.String(seriesName), Fields: fieldNames, Points: make([]*protocol.Point, 0, self.pointBatchSize)}
|
|
|
|
|
2014-02-08 21:37:13 +00:00
|
|
|
// TODO: clean up, this is super gnarly
|
|
|
|
// optimize for the case where we're pulling back only a single column or aggregate
|
2014-04-24 23:55:20 +00:00
|
|
|
buffer := bytes.NewBuffer(nil)
|
|
|
|
valueBuffer := proto.NewBuffer(nil)
|
2014-02-08 21:37:13 +00:00
|
|
|
for {
|
|
|
|
isValid := false
|
|
|
|
point := &protocol.Point{Values: make([]*protocol.FieldValue, fieldCount, fieldCount)}
|
|
|
|
|
|
|
|
for i, it := range iterators {
|
2014-04-24 23:55:20 +00:00
|
|
|
if rawColumnValues[i].value != nil || !it.Valid() {
|
2014-06-04 18:31:39 +00:00
|
|
|
if err := it.Error(); err != nil {
|
|
|
|
return err
|
|
|
|
}
|
2014-02-08 21:37:13 +00:00
|
|
|
continue
|
|
|
|
}
|
|
|
|
|
|
|
|
key := it.Key()
|
|
|
|
if len(key) < 16 {
|
|
|
|
continue
|
|
|
|
}
|
|
|
|
|
2014-07-10 21:59:45 +00:00
|
|
|
if !isPointInRange(fields[i].IdAsBytes(), startTimeBytes, endTimeBytes, key) {
|
2014-02-08 21:37:13 +00:00
|
|
|
continue
|
|
|
|
}
|
|
|
|
|
|
|
|
value := it.Value()
|
|
|
|
sequenceNumber := key[16:]
|
|
|
|
|
|
|
|
rawTime := key[8:16]
|
2014-04-24 23:55:20 +00:00
|
|
|
rawColumnValues[i] = rawColumnValue{time: rawTime, sequence: sequenceNumber, value: value}
|
2014-02-08 21:37:13 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
var pointTimeRaw []byte
|
|
|
|
var pointSequenceRaw []byte
|
|
|
|
// choose the highest (or lowest in case of ascending queries) timestamp
|
|
|
|
// and sequence number. that will become the timestamp and sequence of
|
|
|
|
// the next point.
|
|
|
|
for _, value := range rawColumnValues {
|
2014-04-24 23:55:20 +00:00
|
|
|
if value.value == nil {
|
2014-02-08 21:37:13 +00:00
|
|
|
continue
|
|
|
|
}
|
|
|
|
|
|
|
|
pointTimeRaw, pointSequenceRaw = value.updatePointTimeAndSequence(pointTimeRaw,
|
|
|
|
pointSequenceRaw, query.Ascending)
|
|
|
|
}
|
|
|
|
|
|
|
|
for i, iterator := range iterators {
|
|
|
|
// if the value is nil or doesn't match the point's timestamp and sequence number
|
|
|
|
// then skip it
|
2014-04-24 23:55:20 +00:00
|
|
|
if rawColumnValues[i].value == nil ||
|
2014-02-08 21:37:13 +00:00
|
|
|
!bytes.Equal(rawColumnValues[i].time, pointTimeRaw) ||
|
|
|
|
!bytes.Equal(rawColumnValues[i].sequence, pointSequenceRaw) {
|
|
|
|
|
|
|
|
point.Values[i] = &protocol.FieldValue{IsNull: &TRUE}
|
|
|
|
continue
|
|
|
|
}
|
|
|
|
|
|
|
|
// if we emitted at lease one column, then we should keep
|
|
|
|
// trying to get more points
|
|
|
|
isValid = true
|
|
|
|
|
|
|
|
// advance the iterator to read a new value in the next iteration
|
|
|
|
if query.Ascending {
|
|
|
|
iterator.Next()
|
|
|
|
} else {
|
|
|
|
iterator.Prev()
|
|
|
|
}
|
|
|
|
|
|
|
|
fv := &protocol.FieldValue{}
|
2014-04-24 23:55:20 +00:00
|
|
|
valueBuffer.SetBuf(rawColumnValues[i].value)
|
|
|
|
err := valueBuffer.Unmarshal(fv)
|
2014-02-08 21:37:13 +00:00
|
|
|
if err != nil {
|
2014-05-20 17:14:28 +00:00
|
|
|
log.Error("Error while running query: %s", err)
|
2014-02-08 21:37:13 +00:00
|
|
|
return err
|
|
|
|
}
|
|
|
|
point.Values[i] = fv
|
2014-04-24 23:55:20 +00:00
|
|
|
rawColumnValues[i].value = nil
|
2014-02-08 21:37:13 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
var sequence uint64
|
|
|
|
var t uint64
|
2014-04-24 23:55:20 +00:00
|
|
|
|
|
|
|
// set the point sequence number and timestamp
|
|
|
|
buffer.Reset()
|
|
|
|
buffer.Write(pointSequenceRaw)
|
|
|
|
binary.Read(buffer, binary.BigEndian, &sequence)
|
|
|
|
buffer.Reset()
|
|
|
|
buffer.Write(pointTimeRaw)
|
|
|
|
binary.Read(buffer, binary.BigEndian, &t)
|
|
|
|
|
2014-02-08 21:37:13 +00:00
|
|
|
time := self.convertUintTimestampToInt64(&t)
|
|
|
|
point.SetTimestampInMicroseconds(time)
|
|
|
|
point.SequenceNumber = &sequence
|
|
|
|
|
|
|
|
// stop the loop if we ran out of points
|
|
|
|
if !isValid {
|
|
|
|
break
|
|
|
|
}
|
|
|
|
|
2014-02-27 17:03:54 +00:00
|
|
|
shouldContinue := true
|
2014-03-07 19:13:15 +00:00
|
|
|
|
|
|
|
seriesOutgoing.Points = append(seriesOutgoing.Points, point)
|
|
|
|
|
|
|
|
if len(seriesOutgoing.Points) >= self.pointBatchSize {
|
|
|
|
for _, alias := range aliases {
|
2014-03-12 23:37:16 +00:00
|
|
|
series := &protocol.Series{
|
|
|
|
Name: proto.String(alias),
|
|
|
|
Fields: fieldNames,
|
|
|
|
Points: seriesOutgoing.Points,
|
|
|
|
}
|
|
|
|
if !processor.YieldSeries(series) {
|
2014-08-15 17:18:43 +00:00
|
|
|
log.Debug("Stopping processing")
|
2014-03-07 19:13:15 +00:00
|
|
|
shouldContinue = false
|
|
|
|
}
|
2014-02-19 22:30:05 +00:00
|
|
|
}
|
2014-03-07 19:13:15 +00:00
|
|
|
seriesOutgoing = &protocol.Series{Name: protocol.String(seriesName), Fields: fieldNames, Points: make([]*protocol.Point, 0, self.pointBatchSize)}
|
2014-02-08 21:37:13 +00:00
|
|
|
}
|
2014-02-27 17:03:54 +00:00
|
|
|
|
|
|
|
if !shouldContinue {
|
|
|
|
break
|
|
|
|
}
|
2014-02-08 21:37:13 +00:00
|
|
|
}
|
|
|
|
|
2014-03-07 19:13:15 +00:00
|
|
|
//Yield remaining data
|
|
|
|
for _, alias := range aliases {
|
2014-03-12 23:37:16 +00:00
|
|
|
log.Debug("Final Flush %s", alias)
|
|
|
|
series := &protocol.Series{Name: protocol.String(alias), Fields: seriesOutgoing.Fields, Points: seriesOutgoing.Points}
|
|
|
|
if !processor.YieldSeries(series) {
|
2014-03-07 19:13:15 +00:00
|
|
|
log.Debug("Cancelled...")
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2014-05-30 22:52:31 +00:00
|
|
|
log.Debug("Finished running query %s", query.GetQueryString())
|
2014-02-02 23:18:19 +00:00
|
|
|
return nil
|
|
|
|
}
|
2014-02-06 18:17:59 +00:00
|
|
|
|
2014-06-13 22:43:58 +00:00
|
|
|
func (self *Shard) executeDeleteQuery(querySpec *parser.QuerySpec, processor cluster.QueryProcessor) error {
|
2014-02-17 15:37:48 +00:00
|
|
|
query := querySpec.DeleteQuery()
|
|
|
|
series := query.GetFromClause()
|
|
|
|
database := querySpec.Database()
|
|
|
|
if series.Type != parser.FromClauseArray {
|
2014-06-23 18:01:53 +00:00
|
|
|
return fmt.Errorf("Merge and Inner joins can't be used with a delete query: %v", series.Type)
|
2014-02-17 15:37:48 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
for _, name := range series.Names {
|
|
|
|
var err error
|
|
|
|
if regex, ok := name.Name.GetCompiledRegex(); ok {
|
|
|
|
err = self.deleteRangeOfRegex(database, regex, query.GetStartTime(), query.GetEndTime())
|
|
|
|
} else {
|
|
|
|
err = self.deleteRangeOfSeries(database, name.Name.Name, query.GetStartTime(), query.GetEndTime())
|
|
|
|
}
|
|
|
|
|
|
|
|
if err != nil {
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
}
|
2014-06-04 18:31:39 +00:00
|
|
|
self.db.Compact()
|
2014-02-17 15:37:48 +00:00
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
2014-07-10 21:59:45 +00:00
|
|
|
func (self *Shard) DropFields(fields []*metastore.Field) error {
|
2014-02-17 20:55:39 +00:00
|
|
|
startTimeBytes := []byte{0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00}
|
|
|
|
endTimeBytes := []byte{0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF}
|
2014-07-10 21:59:45 +00:00
|
|
|
return self.deleteRangeOfFields(fields, startTimeBytes, endTimeBytes)
|
2014-02-17 20:55:39 +00:00
|
|
|
}
|
|
|
|
|
2014-06-13 22:43:58 +00:00
|
|
|
func (self *Shard) byteArrayForTimeInt(time int64) []byte {
|
2014-02-17 15:37:48 +00:00
|
|
|
timeBuffer := bytes.NewBuffer(make([]byte, 0, 8))
|
|
|
|
binary.Write(timeBuffer, binary.BigEndian, self.convertTimestampToUint(&time))
|
|
|
|
bytes := timeBuffer.Bytes()
|
|
|
|
return bytes
|
|
|
|
}
|
|
|
|
|
2014-06-13 22:43:58 +00:00
|
|
|
func (self *Shard) byteArraysForStartAndEndTimes(startTime, endTime int64) ([]byte, []byte) {
|
2014-02-17 15:37:48 +00:00
|
|
|
return self.byteArrayForTimeInt(startTime), self.byteArrayForTimeInt(endTime)
|
|
|
|
}
|
|
|
|
|
2014-06-13 22:43:58 +00:00
|
|
|
func (self *Shard) deleteRangeOfSeriesCommon(database, series string, startTimeBytes, endTimeBytes []byte) error {
|
2014-07-10 21:59:45 +00:00
|
|
|
fields := self.metaStore.GetFieldsForSeries(database, series)
|
|
|
|
return self.deleteRangeOfFields(fields, startTimeBytes, endTimeBytes)
|
|
|
|
}
|
|
|
|
|
|
|
|
func (self *Shard) deleteRangeOfFields(fields []*metastore.Field, startTimeBytes, endTimeBytes []byte) error {
|
2014-06-04 18:31:39 +00:00
|
|
|
startKey := bytes.NewBuffer(nil)
|
|
|
|
endKey := bytes.NewBuffer(nil)
|
2014-02-17 15:37:48 +00:00
|
|
|
for _, field := range fields {
|
2014-07-10 21:59:45 +00:00
|
|
|
idBytes := field.IdAsBytes()
|
2014-06-04 18:31:39 +00:00
|
|
|
startKey.Reset()
|
2014-07-10 21:59:45 +00:00
|
|
|
startKey.Write(idBytes)
|
2014-06-04 18:31:39 +00:00
|
|
|
startKey.Write(startTimeBytes)
|
|
|
|
startKey.Write([]byte{0, 0, 0, 0, 0, 0, 0, 0})
|
|
|
|
endKey.Reset()
|
2014-07-10 21:59:45 +00:00
|
|
|
endKey.Write(idBytes)
|
2014-06-04 18:31:39 +00:00
|
|
|
endKey.Write(endTimeBytes)
|
|
|
|
endKey.Write([]byte{0xff, 0xff, 0xff, 0xff, 0xff, 0xff, 0xff, 0xff})
|
|
|
|
|
|
|
|
err := self.db.Del(startKey.Bytes(), endKey.Bytes())
|
|
|
|
if err != nil {
|
|
|
|
return err
|
2014-02-17 15:37:48 +00:00
|
|
|
}
|
|
|
|
}
|
2014-06-04 18:31:39 +00:00
|
|
|
return nil
|
2014-02-17 15:37:48 +00:00
|
|
|
}
|
|
|
|
|
2014-06-13 22:43:58 +00:00
|
|
|
// func (self *Shard) compact() {
|
2014-06-04 18:31:39 +00:00
|
|
|
// log.Info("Compacting shard")
|
|
|
|
// self.db.CompactRange(levigo.Range{})
|
|
|
|
// log.Info("Shard compaction is done")
|
|
|
|
// }
|
2014-04-07 23:01:27 +00:00
|
|
|
|
2014-06-13 22:43:58 +00:00
|
|
|
func (self *Shard) deleteRangeOfSeries(database, series string, startTime, endTime time.Time) error {
|
2014-02-17 15:37:48 +00:00
|
|
|
startTimeBytes, endTimeBytes := self.byteArraysForStartAndEndTimes(common.TimeToMicroseconds(startTime), common.TimeToMicroseconds(endTime))
|
|
|
|
return self.deleteRangeOfSeriesCommon(database, series, startTimeBytes, endTimeBytes)
|
|
|
|
}
|
|
|
|
|
2014-06-13 22:43:58 +00:00
|
|
|
func (self *Shard) deleteRangeOfRegex(database string, regex *regexp.Regexp, startTime, endTime time.Time) error {
|
2014-07-10 21:59:45 +00:00
|
|
|
series := self.metaStore.GetSeriesForDatabaseAndRegex(database, regex)
|
2014-02-17 15:37:48 +00:00
|
|
|
for _, name := range series {
|
|
|
|
err := self.deleteRangeOfSeries(database, name, startTime, endTime)
|
|
|
|
if err != nil {
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
}
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
2014-06-13 22:43:58 +00:00
|
|
|
func (self *Shard) hasReadAccess(querySpec *parser.QuerySpec) bool {
|
2014-06-23 17:14:00 +00:00
|
|
|
for series := range querySpec.SeriesValuesAndColumns() {
|
2014-02-08 21:37:13 +00:00
|
|
|
if _, isRegex := series.GetCompiledRegex(); !isRegex {
|
|
|
|
if !querySpec.HasReadAccess(series.Name) {
|
|
|
|
return false
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
return true
|
|
|
|
}
|
|
|
|
|
2014-06-13 22:43:58 +00:00
|
|
|
func (self *Shard) byteArrayForTime(t time.Time) []byte {
|
2014-02-08 21:37:13 +00:00
|
|
|
timeBuffer := bytes.NewBuffer(make([]byte, 0, 8))
|
|
|
|
timeMicro := common.TimeToMicroseconds(t)
|
|
|
|
binary.Write(timeBuffer, binary.BigEndian, self.convertTimestampToUint(&timeMicro))
|
|
|
|
return timeBuffer.Bytes()
|
|
|
|
}
|
|
|
|
|
2014-06-13 22:43:58 +00:00
|
|
|
func (self *Shard) close() {
|
2014-03-05 22:09:01 +00:00
|
|
|
self.closed = true
|
2014-02-06 18:17:59 +00:00
|
|
|
self.db.Close()
|
2014-06-04 18:31:39 +00:00
|
|
|
self.db = nil
|
2014-02-06 18:17:59 +00:00
|
|
|
}
|
|
|
|
|
2014-06-13 22:43:58 +00:00
|
|
|
func (self *Shard) convertTimestampToUint(t *int64) uint64 {
|
2014-02-06 18:17:59 +00:00
|
|
|
if *t < 0 {
|
|
|
|
return uint64(math.MaxInt64 + *t + 1)
|
|
|
|
}
|
|
|
|
return uint64(*t) + uint64(math.MaxInt64) + uint64(1)
|
|
|
|
}
|
2014-02-08 21:37:13 +00:00
|
|
|
|
2014-07-10 21:59:45 +00:00
|
|
|
func (self *Shard) fetchSinglePoint(querySpec *parser.QuerySpec, series string, fields []*metastore.Field) (*protocol.Series, error) {
|
2014-02-08 21:37:13 +00:00
|
|
|
query := querySpec.SelectQuery()
|
|
|
|
fieldCount := len(fields)
|
|
|
|
fieldNames := make([]string, 0, fieldCount)
|
|
|
|
point := &protocol.Point{Values: make([]*protocol.FieldValue, 0, fieldCount)}
|
|
|
|
timestamp := common.TimeToMicroseconds(query.GetStartTime())
|
|
|
|
sequenceNumber, err := query.GetSinglePointQuerySequenceNumber()
|
|
|
|
if err != nil {
|
|
|
|
return nil, err
|
|
|
|
}
|
|
|
|
|
|
|
|
timeAndSequenceBuffer := bytes.NewBuffer(make([]byte, 0, 16))
|
|
|
|
binary.Write(timeAndSequenceBuffer, binary.BigEndian, self.convertTimestampToUint(×tamp))
|
|
|
|
binary.Write(timeAndSequenceBuffer, binary.BigEndian, sequenceNumber)
|
|
|
|
sequenceNumber_uint64 := uint64(sequenceNumber)
|
|
|
|
point.SequenceNumber = &sequenceNumber_uint64
|
|
|
|
point.SetTimestampInMicroseconds(timestamp)
|
|
|
|
|
|
|
|
timeAndSequenceBytes := timeAndSequenceBuffer.Bytes()
|
|
|
|
for _, field := range fields {
|
2014-07-10 21:59:45 +00:00
|
|
|
pointKeyBuff := bytes.NewBuffer(make([]byte, 0, 24))
|
|
|
|
pointKeyBuff.Write(field.IdAsBytes())
|
|
|
|
pointKeyBuff.Write(timeAndSequenceBytes)
|
2014-02-08 21:37:13 +00:00
|
|
|
|
2014-07-10 21:59:45 +00:00
|
|
|
if data, err := self.db.Get(pointKeyBuff.Bytes()); err != nil {
|
2014-02-08 21:37:13 +00:00
|
|
|
return nil, err
|
|
|
|
} else {
|
|
|
|
fieldValue := &protocol.FieldValue{}
|
|
|
|
err := proto.Unmarshal(data, fieldValue)
|
|
|
|
if err != nil {
|
|
|
|
return nil, err
|
|
|
|
}
|
|
|
|
if data != nil {
|
|
|
|
fieldNames = append(fieldNames, field.Name)
|
|
|
|
point.Values = append(point.Values, fieldValue)
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
result := &protocol.Series{Name: &series, Fields: fieldNames, Points: []*protocol.Point{point}}
|
|
|
|
|
|
|
|
return result, nil
|
|
|
|
}
|
|
|
|
|
2014-07-10 21:59:45 +00:00
|
|
|
func (self *Shard) getIterators(fields []*metastore.Field, start, end []byte, isAscendingQuery bool) (fieldNames []string, iterators []storage.Iterator) {
|
2014-06-04 18:31:39 +00:00
|
|
|
iterators = make([]storage.Iterator, len(fields))
|
2014-02-08 21:37:13 +00:00
|
|
|
fieldNames = make([]string, len(fields))
|
|
|
|
|
|
|
|
// start the iterators to go through the series data
|
|
|
|
for i, field := range fields {
|
2014-07-10 21:59:45 +00:00
|
|
|
idBytes := field.IdAsBytes()
|
2014-02-08 21:37:13 +00:00
|
|
|
fieldNames[i] = field.Name
|
2014-06-04 18:31:39 +00:00
|
|
|
iterators[i] = self.db.Iterator()
|
|
|
|
|
2014-02-08 21:37:13 +00:00
|
|
|
if isAscendingQuery {
|
2014-07-10 21:59:45 +00:00
|
|
|
iterators[i].Seek(append(idBytes, start...))
|
2014-02-08 21:37:13 +00:00
|
|
|
} else {
|
2014-07-10 21:59:45 +00:00
|
|
|
iterators[i].Seek(append(append(idBytes, end...), MAX_SEQUENCE...))
|
2014-02-08 21:37:13 +00:00
|
|
|
if iterators[i].Valid() {
|
|
|
|
iterators[i].Prev()
|
|
|
|
}
|
|
|
|
}
|
2014-06-04 18:31:39 +00:00
|
|
|
|
|
|
|
if err := iterators[i].Error(); err != nil {
|
|
|
|
log.Error("Error while getting iterators: %s", err)
|
|
|
|
return nil, nil
|
|
|
|
}
|
2014-02-08 21:37:13 +00:00
|
|
|
}
|
|
|
|
return
|
|
|
|
}
|
|
|
|
|
2014-06-13 22:43:58 +00:00
|
|
|
func (self *Shard) convertUintTimestampToInt64(t *uint64) int64 {
|
2014-02-08 21:37:13 +00:00
|
|
|
if *t > uint64(math.MaxInt64) {
|
|
|
|
return int64(*t-math.MaxInt64) - int64(1)
|
|
|
|
}
|
|
|
|
return int64(*t) - math.MaxInt64 - int64(1)
|
|
|
|
}
|
2014-07-10 21:59:45 +00:00
|
|
|
|
|
|
|
func (self *Shard) getFieldsForSeries(db, series string, columns []string) ([]*metastore.Field, error) {
|
|
|
|
allFields := self.metaStore.GetFieldsForSeries(db, series)
|
|
|
|
if len(allFields) == 0 {
|
|
|
|
return nil, FieldLookupError{"Couldn't look up columns for series: " + series}
|
|
|
|
}
|
|
|
|
if len(columns) > 0 && columns[0] == "*" {
|
|
|
|
return allFields, nil
|
|
|
|
}
|
|
|
|
|
|
|
|
fields := make([]*metastore.Field, len(columns), len(columns))
|
|
|
|
|
|
|
|
for i, name := range columns {
|
|
|
|
hasField := false
|
|
|
|
for _, f := range allFields {
|
|
|
|
if f.Name == name {
|
|
|
|
field := f
|
|
|
|
hasField = true
|
|
|
|
fields[i] = field
|
|
|
|
break
|
|
|
|
}
|
|
|
|
}
|
|
|
|
if !hasField {
|
|
|
|
return nil, FieldLookupError{"Field " + name + " doesn't exist in series " + series}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
return fields, nil
|
|
|
|
}
|