func (self *CoordinatorImpl) shouldQuerySequentially(shards []*cluster.ShardData, querySpec *parser.QuerySpec) bool { // if the query isn't a select, then it doesn't matter if querySpec.SelectQuery != nil { return false } // if the query is a regex, we can't predic the number of responses // we get back if querySpec.IsRegex() { return true } groupByClause := querySpec.SelectQuery().GetGroupByClause() // if there's no group by clause, then we're returning raw points // with some math done on them, thus we can't predict the number of // points if groupByClause == nil { return true } // if there's a group by clause but no group by interval, we can't // predict the cardinality of the columns used in the group by // interval, thus we can't predict the number of responses returned // from the shard if querySpec.GetGroupByInterval() == nil { return true } // if there's a group by time and other columns, then the previous // logic holds if len(groupByClause.Elems) > 1 { return true } // parallel querying only if we're querying a single series, with // group by time only return false }
func (self *Shard) Query(querySpec *parser.QuerySpec, processor cluster.QueryProcessor) error { if querySpec.IsListSeriesQuery() { return self.executeListSeriesQuery(querySpec, processor) } else if querySpec.IsDeleteFromSeriesQuery() { return self.executeDeleteQuery(querySpec, processor) } 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 { seriesNames := self.metaStore.GetSeriesForDatabaseAndRegex(querySpec.Database(), regex) 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 }
func (self *CoordinatorImpl) getShardsAndProcessor(querySpec *parser.QuerySpec, writer SeriesWriter) ([]*cluster.ShardData, cluster.QueryProcessor, chan bool, error) { shards := self.clusterConfiguration.GetShards(querySpec) shouldAggregateLocally := self.shouldAggregateLocally(shards, querySpec) var err error var processor cluster.QueryProcessor responseChan := make(chan *protocol.Response) seriesClosed := make(chan bool) selectQuery := querySpec.SelectQuery() if selectQuery != nil { if !shouldAggregateLocally { // if we should aggregate in the coordinator (i.e. aggregation // isn't happening locally at the shard level), create an engine processor, err = engine.NewQueryEngine(querySpec.SelectQuery(), responseChan) } else { // if we have a query with limit, then create an engine, or we can // make the passthrough limit aware processor = engine.NewPassthroughEngineWithLimit(responseChan, 100, selectQuery.Limit) } } else if !shouldAggregateLocally { processor = engine.NewPassthroughEngine(responseChan, 100) } if err != nil { return nil, nil, nil, err } if processor == nil { return shards, nil, nil, nil } go func() { for { response := <-responseChan if *response.Type == endStreamResponse || *response.Type == accessDeniedResponse { writer.Close() seriesClosed <- true return } if !(*response.Type == queryResponse && querySpec.IsExplainQuery()) { if response.Series != nil && len(response.Series.Points) > 0 { writer.Write(response.Series) } } } }() return shards, processor, seriesClosed, nil }
func (self *CoordinatorImpl) checkPermission(user common.User, querySpec *parser.QuerySpec) error { // if this isn't a regex query do the permission check here fromClause := querySpec.SelectQuery().GetFromClause() for _, n := range fromClause.Names { if _, ok := n.Name.GetCompiledRegex(); ok { break } else if name := n.Name.Name; !user.HasReadAccess(name) { return fmt.Errorf("User doesn't have read access to %s", name) } } return nil }
func (self *ShardData) Query(querySpec *parser.QuerySpec, response chan *protocol.Response) error { // This is only for queries that are deletes or drops. They need to be sent everywhere as opposed to just the local or one of the remote shards. // But this boolean should only be set to true on the server that receives the initial query. if querySpec.RunAgainstAllServersInShard { if querySpec.IsDeleteFromSeriesQuery() { return self.logAndHandleDeleteQuery(querySpec, response) } else if querySpec.IsDropSeriesQuery() { return self.logAndHandleDropSeriesQuery(querySpec, response) } } if self.localShard != nil { var processor QueryProcessor if querySpec.IsListSeriesQuery() { processor = engine.NewListSeriesEngine(response) } else if querySpec.IsDeleteFromSeriesQuery() || querySpec.IsDropSeriesQuery() || querySpec.IsSinglePointQuery() { maxDeleteResults := 10000 processor = engine.NewPassthroughEngine(response, maxDeleteResults) } else { if self.ShouldAggregateLocally(querySpec) { processor = engine.NewQueryEngine(querySpec.SelectQuery(), response) } else { maxPointsToBufferBeforeSending := 1000 processor = engine.NewPassthroughEngine(response, maxPointsToBufferBeforeSending) } } err := self.localShard.Query(querySpec, processor) processor.Close() return err } healthyServers := make([]*ClusterServer, 0, len(self.clusterServers)) for _, s := range self.clusterServers { if !s.IsUp() { continue } healthyServers = append(healthyServers, s) } healthyCount := len(healthyServers) if healthyCount == 0 { message := fmt.Sprintf("No servers up to query shard %d", self.id) response <- &protocol.Response{Type: &endStreamResponse, ErrorMessage: &message} return errors.New(message) } randServerIndex := int(time.Now().UnixNano() % int64(healthyCount)) server := healthyServers[randServerIndex] request := self.createRequest(querySpec) return server.MakeRequest(request, response) }
func (self *CoordinatorImpl) shouldQuerySequentially(shards []*cluster.ShardData, querySpec *parser.QuerySpec) bool { // if the query isn't a select, then it doesn't matter if querySpec.SelectQuery() == nil { return false } // if the query is a regex, we can't predic the number of responses // we get back if querySpec.IsRegex() { return true } groupByClause := querySpec.SelectQuery().GetGroupByClause() // if there's no group by clause, then we're returning raw points // with some math done on them, thus we can't predict the number of // points if groupByClause == nil { return true } // if there's a group by clause but no group by interval, we can't // predict the cardinality of the columns used in the group by // interval, thus we can't predict the number of responses returned // from the shard if querySpec.GetGroupByInterval() == nil { return true } // if there's a group by time and other columns, then the previous // logic holds if len(groupByClause.Elems) > 1 { return true } if !self.shouldAggregateLocally(shards, querySpec) { return true } for _, shard := range shards { bufferSize := shard.QueryResponseBufferSize(querySpec, self.config.LevelDbPointBatchSize) // if the number of repsonses is too big, do a sequential querying if bufferSize > self.config.ClusterMaxResponseBufferSize { return true } } // parallel querying only if we're querying a single series, with // group by time only return false }
func (self *Shard) fetchSinglePoint(querySpec *parser.QuerySpec, series string, fields []*metastore.Field) (*protocol.Series, error) { 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 { pointKeyBuff := bytes.NewBuffer(make([]byte, 0, 24)) pointKeyBuff.Write(field.IdAsBytes()) pointKeyBuff.Write(timeAndSequenceBytes) if data, err := self.db.Get(pointKeyBuff.Bytes()); err != nil { 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 }
func (self *Shard) executeQueryForSeries(querySpec *parser.QuerySpec, seriesName string, columns []string, processor cluster.QueryProcessor) error { startTimeBytes := self.byteArrayForTime(querySpec.GetStartTime()) endTimeBytes := self.byteArrayForTime(querySpec.GetEndTime()) fields, err := self.getFieldsForSeries(querySpec.Database(), seriesName, columns) if err != nil { // because a db is distributed across the cluster, it's possible we don't have the series indexed here. ignore switch err := err.(type) { case FieldLookupError: log.Debug("Cannot find fields %v", columns) return nil default: log.Error("Error looking up fields for %s: %s", seriesName, err) return fmt.Errorf("Error looking up fields for %s: %s", seriesName, err) } } fieldCount := len(fields) rawColumnValues := make([]rawColumnValue, fieldCount, fieldCount) query := querySpec.SelectQuery() aliases := query.GetTableAliases(seriesName) if querySpec.IsSinglePointQuery() { series, err := self.fetchSinglePoint(querySpec, seriesName, fields) if err != nil { log.Error("Error reading a single point: %s", err) 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) defer func() { for _, it := range iterators { it.Close() } }() seriesOutgoing := &protocol.Series{Name: protocol.String(seriesName), Fields: fieldNames, Points: make([]*protocol.Point, 0, self.pointBatchSize)} // TODO: clean up, this is super gnarly // optimize for the case where we're pulling back only a single column or aggregate buffer := bytes.NewBuffer(nil) valueBuffer := proto.NewBuffer(nil) for { isValid := false point := &protocol.Point{Values: make([]*protocol.FieldValue, fieldCount, fieldCount)} for i, it := range iterators { if rawColumnValues[i].value != nil || !it.Valid() { if err := it.Error(); err != nil { return err } continue } key := it.Key() if len(key) < 16 { continue } if !isPointInRange(fields[i].Id, startTimeBytes, endTimeBytes, key) { continue } value := it.Value() sequenceNumber := key[16:] rawTime := key[8:16] rawColumnValues[i] = rawColumnValue{time: rawTime, sequence: sequenceNumber, value: value} } 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 { if value.value == nil { 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 if rawColumnValues[i].value == nil || !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{} valueBuffer.SetBuf(rawColumnValues[i].value) err := valueBuffer.Unmarshal(fv) if err != nil { log.Error("Error while running query: %s", err) return err } point.Values[i] = fv rawColumnValues[i].value = nil } var sequence uint64 var t uint64 // 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) time := self.convertUintTimestampToInt64(&t) point.SetTimestampInMicroseconds(time) point.SequenceNumber = &sequence // stop the loop if we ran out of points if !isValid { break } shouldContinue := true seriesOutgoing.Points = append(seriesOutgoing.Points, point) if len(seriesOutgoing.Points) >= self.pointBatchSize { for _, alias := range aliases { series := &protocol.Series{ Name: proto.String(alias), Fields: fieldNames, Points: seriesOutgoing.Points, } if !processor.YieldSeries(series) { log.Info("Stopping processing") shouldContinue = false } } seriesOutgoing = &protocol.Series{Name: protocol.String(seriesName), Fields: fieldNames, Points: make([]*protocol.Point, 0, self.pointBatchSize)} } if !shouldContinue { break } } //Yield remaining data for _, alias := range aliases { log.Debug("Final Flush %s", alias) series := &protocol.Series{Name: protocol.String(alias), Fields: seriesOutgoing.Fields, Points: seriesOutgoing.Points} if !processor.YieldSeries(series) { log.Debug("Cancelled...") } } log.Debug("Finished running query %s", query.GetQueryString()) return nil }
func (self *ShardData) Query(querySpec *parser.QuerySpec, response chan *p.Response) { log.Debug("QUERY: shard %d, query '%s'", self.Id(), querySpec.GetQueryString()) defer common.RecoverFunc(querySpec.Database(), querySpec.GetQueryString(), func(err interface{}) { response <- &p.Response{Type: &endStreamResponse, ErrorMessage: p.String(fmt.Sprintf("%s", err))} }) // This is only for queries that are deletes or drops. They need to be sent everywhere as opposed to just the local or one of the remote shards. // But this boolean should only be set to true on the server that receives the initial query. if querySpec.RunAgainstAllServersInShard { if querySpec.IsDeleteFromSeriesQuery() { self.logAndHandleDeleteQuery(querySpec, response) } else if querySpec.IsDropSeriesQuery() { self.logAndHandleDropSeriesQuery(querySpec, response) } } if self.IsLocal { var processor QueryProcessor var err error if querySpec.IsListSeriesQuery() { processor = engine.NewListSeriesEngine(response) } else if querySpec.IsDeleteFromSeriesQuery() || querySpec.IsDropSeriesQuery() || querySpec.IsSinglePointQuery() { maxDeleteResults := 10000 processor = engine.NewPassthroughEngine(response, maxDeleteResults) } else { query := querySpec.SelectQuery() if self.ShouldAggregateLocally(querySpec) { log.Debug("creating a query engine") processor, err = engine.NewQueryEngine(query, response) if err != nil { response <- &p.Response{Type: &endStreamResponse, ErrorMessage: p.String(err.Error())} log.Error("Error while creating engine: %s", err) return } processor.SetShardInfo(int(self.Id()), self.IsLocal) } else if query.HasAggregates() { maxPointsToBufferBeforeSending := 1000 log.Debug("creating a passthrough engine") processor = engine.NewPassthroughEngine(response, maxPointsToBufferBeforeSending) } else { maxPointsToBufferBeforeSending := 1000 log.Debug("creating a passthrough engine with limit") processor = engine.NewPassthroughEngineWithLimit(response, maxPointsToBufferBeforeSending, query.Limit) } if query.GetFromClause().Type != parser.FromClauseInnerJoin { // Joins do their own filtering since we need to get all // points before filtering. This is due to the fact that some // where expressions will be difficult to compute before the // points are joined together, think where clause with // left.column = 'something' or right.column = // 'something_else'. We can't filter the individual series // separately. The filtering happens in merge.go:55 processor = engine.NewFilteringEngine(query, processor) } } shard, err := self.store.GetOrCreateShard(self.id) if err != nil { response <- &p.Response{Type: &endStreamResponse, ErrorMessage: p.String(err.Error())} log.Error("Error while getting shards: %s", err) return } defer self.store.ReturnShard(self.id) err = shard.Query(querySpec, processor) // if we call Close() in case of an error it will mask the error if err != nil { response <- &p.Response{Type: &endStreamResponse, ErrorMessage: p.String(err.Error())} return } processor.Close() response <- &p.Response{Type: &endStreamResponse} return } if server := self.randomHealthyServer(); server != nil { log.Debug("Querying server %d for shard %d", server.GetId(), self.Id()) request := self.createRequest(querySpec) server.MakeRequest(request, response) return } message := fmt.Sprintf("No servers up to query shard %d", self.id) response <- &p.Response{Type: &endStreamResponse, ErrorMessage: &message} log.Error(message) }
func (self *ShardData) Query(querySpec *parser.QuerySpec, response chan *p.Response) { // This is only for queries that are deletes or drops. They need to be sent everywhere as opposed to just the local or one of the remote shards. // But this boolean should only be set to true on the server that receives the initial query. if querySpec.RunAgainstAllServersInShard { if querySpec.IsDeleteFromSeriesQuery() { self.logAndHandleDeleteQuery(querySpec, response) } else if querySpec.IsDropSeriesQuery() { self.logAndHandleDropSeriesQuery(querySpec, response) } } if self.IsLocal { var processor QueryProcessor var err error if querySpec.IsListSeriesQuery() { processor = engine.NewListSeriesEngine(response) } else if querySpec.IsDeleteFromSeriesQuery() || querySpec.IsDropSeriesQuery() || querySpec.IsSinglePointQuery() { maxDeleteResults := 10000 processor = engine.NewPassthroughEngine(response, maxDeleteResults) } else { query := querySpec.SelectQuery() if self.ShouldAggregateLocally(querySpec) { log.Debug("creating a query engine\n") processor, err = engine.NewQueryEngine(query, response) if err != nil { response <- &p.Response{Type: &endStreamResponse, ErrorMessage: p.String(err.Error())} log.Error("Error while creating engine: %s", err) return } processor.SetShardInfo(int(self.Id()), self.IsLocal) } else if query.HasAggregates() { maxPointsToBufferBeforeSending := 1000 log.Debug("creating a passthrough engine\n") processor = engine.NewPassthroughEngine(response, maxPointsToBufferBeforeSending) } else { maxPointsToBufferBeforeSending := 1000 log.Debug("creating a passthrough engine with limit\n") processor = engine.NewPassthroughEngineWithLimit(response, maxPointsToBufferBeforeSending, query.Limit) } processor = engine.NewFilteringEngine(query, processor) } shard, err := self.store.GetOrCreateShard(self.id) if err != nil { response <- &p.Response{Type: &endStreamResponse, ErrorMessage: p.String(err.Error())} log.Error("Error while getting shards: %s", err) return } defer self.store.ReturnShard(self.id) err = shard.Query(querySpec, processor) processor.Close() if err != nil { response <- &p.Response{Type: &endStreamResponse, ErrorMessage: p.String(err.Error())} } response <- &p.Response{Type: &endStreamResponse} return } healthyServers := make([]*ClusterServer, 0, len(self.clusterServers)) for _, s := range self.clusterServers { if !s.IsUp() { continue } healthyServers = append(healthyServers, s) } healthyCount := len(healthyServers) if healthyCount == 0 { message := fmt.Sprintf("No servers up to query shard %d", self.id) response <- &p.Response{Type: &endStreamResponse, ErrorMessage: &message} log.Error(message) return } randServerIndex := int(time.Now().UnixNano() % int64(healthyCount)) server := healthyServers[randServerIndex] request := self.createRequest(querySpec) server.MakeRequest(request, response) }
func (self *CoordinatorImpl) runQuerySpec(querySpec *parser.QuerySpec, seriesWriter SeriesWriter) error { shards := self.clusterConfiguration.GetShards(querySpec) shouldAggregateLocally := true var processor cluster.QueryProcessor var responseChan chan *protocol.Response var seriesClosed chan bool for _, s := range shards { // If the aggregation is done at the shard level, we don't need to // do it here at the coordinator level. if !s.ShouldAggregateLocally(querySpec) { seriesClosed = make(chan bool) shouldAggregateLocally = false responseChan = make(chan *protocol.Response) if querySpec.SelectQuery() != nil { processor = engine.NewQueryEngine(querySpec.SelectQuery(), responseChan) } else { bufferSize := 100 processor = engine.NewPassthroughEngine(responseChan, bufferSize) } go func() { for { res := <-responseChan if *res.Type == endStreamResponse || *res.Type == accessDeniedResponse { seriesWriter.Close() seriesClosed <- true return } if res.Series != nil && len(res.Series.Points) > 0 { seriesWriter.Write(res.Series) } } }() break } } responses := make([]chan *protocol.Response, 0) for _, shard := range shards { responseChan := make(chan *protocol.Response, self.config.QueryShardBufferSize) go shard.Query(querySpec, responseChan) responses = append(responses, responseChan) } for i, responseChan := range responses { log.Debug("READING: shard: ", shards[i].String()) for { response := <-responseChan log.Debug("GOT RESPONSE: ", response.Type, response.Series) if *response.Type == endStreamResponse || *response.Type == accessDeniedResponse { break } if shouldAggregateLocally { log.Debug("WRITING: ", len(response.Series.Points)) seriesWriter.Write(response.Series) log.Debug("WRITING (done)") continue } // if the data wasn't aggregated at the shard level, aggregate // the data here log.Debug("YIELDING: ", len(response.Series.Points)) if response.Series != nil { for _, p := range response.Series.Points { processor.YieldPoint(response.Series.Name, response.Series.Fields, p) } } } log.Debug("DONE: shard: ", shards[i].String()) } if !shouldAggregateLocally { processor.Close() <-seriesClosed return nil } seriesWriter.Close() return nil }