2025-06-02 13:09:43 +00:00

6460 lines
226 KiB
Go

package clickhouseReader
import (
"context"
"database/sql"
"encoding/json"
"fmt"
"math"
"math/rand"
"reflect"
"regexp"
"sort"
"strconv"
"strings"
"sync"
"time"
"github.com/SigNoz/signoz/pkg/prometheus"
"github.com/SigNoz/signoz/pkg/query-service/model/metrics_explorer"
"github.com/SigNoz/signoz/pkg/sqlstore"
"github.com/SigNoz/signoz/pkg/telemetrystore"
"github.com/SigNoz/signoz/pkg/types"
"github.com/SigNoz/signoz/pkg/valuer"
"github.com/uptrace/bun"
errorsV2 "github.com/SigNoz/signoz/pkg/errors"
"github.com/google/uuid"
"github.com/pkg/errors"
"github.com/prometheus/prometheus/promql"
"github.com/prometheus/prometheus/util/stats"
"github.com/ClickHouse/clickhouse-go/v2"
"github.com/ClickHouse/clickhouse-go/v2/lib/driver"
"github.com/SigNoz/signoz/pkg/cache"
"github.com/SigNoz/signoz/pkg/types/authtypes"
"go.uber.org/zap"
queryprogress "github.com/SigNoz/signoz/pkg/query-service/app/clickhouseReader/query_progress"
"github.com/SigNoz/signoz/pkg/query-service/app/logs"
"github.com/SigNoz/signoz/pkg/query-service/app/resource"
"github.com/SigNoz/signoz/pkg/query-service/app/services"
"github.com/SigNoz/signoz/pkg/query-service/app/traces/smart"
"github.com/SigNoz/signoz/pkg/query-service/app/traces/tracedetail"
"github.com/SigNoz/signoz/pkg/query-service/common"
"github.com/SigNoz/signoz/pkg/query-service/constants"
chErrors "github.com/SigNoz/signoz/pkg/query-service/errors"
"github.com/SigNoz/signoz/pkg/query-service/metrics"
"github.com/SigNoz/signoz/pkg/query-service/model"
v3 "github.com/SigNoz/signoz/pkg/query-service/model/v3"
"github.com/SigNoz/signoz/pkg/query-service/telemetry"
"github.com/SigNoz/signoz/pkg/query-service/utils"
)
const (
primaryNamespace = "clickhouse"
archiveNamespace = "clickhouse-archive"
signozTraceDBName = "signoz_traces"
signozHistoryDBName = "signoz_analytics"
ruleStateHistoryTableName = "distributed_rule_state_history_v0"
signozDurationMVTable = "distributed_durationSort"
signozUsageExplorerTable = "distributed_usage_explorer"
signozSpansTable = "distributed_signoz_spans"
signozErrorIndexTable = "distributed_signoz_error_index_v2"
signozTraceTableName = "distributed_signoz_index_v2"
signozTraceLocalTableName = "signoz_index_v2"
signozMetricDBName = "signoz_metrics"
signozMetadataDbName = "signoz_metadata"
signozSampleLocalTableName = "samples_v4"
signozSampleTableName = "distributed_samples_v4"
signozSamplesAgg5mLocalTableName = "samples_v4_agg_5m"
signozSamplesAgg5mTableName = "distributed_samples_v4_agg_5m"
signozSamplesAgg30mLocalTableName = "samples_v4_agg_30m"
signozSamplesAgg30mTableName = "distributed_samples_v4_agg_30m"
signozExpHistLocalTableName = "exp_hist"
signozExpHistTableName = "distributed_exp_hist"
signozTSLocalTableNameV4 = "time_series_v4"
signozTSTableNameV4 = "distributed_time_series_v4"
signozTSLocalTableNameV46Hrs = "time_series_v4_6hrs"
signozTSTableNameV46Hrs = "distributed_time_series_v4_6hrs"
signozTSLocalTableNameV41Day = "time_series_v4_1day"
signozTSTableNameV41Day = "distributed_time_series_v4_1day"
signozTSLocalTableNameV41Week = "time_series_v4_1week"
signozTSTableNameV41Week = "distributed_time_series_v4_1week"
signozTableAttributesMetadata = "distributed_attributes_metadata"
signozLocalTableAttributesMetadata = "attributes_metadata"
signozUpdatedMetricsMetadataLocalTable = "updated_metadata"
signozUpdatedMetricsMetadataTable = "distributed_updated_metadata"
minTimespanForProgressiveSearch = time.Hour
minTimespanForProgressiveSearchMargin = time.Minute
maxProgressiveSteps = 4
charset = "abcdefghijklmnopqrstuvwxyz" +
"ABCDEFGHIJKLMNOPQRSTUVWXYZ0123456789"
NANOSECOND = 1000000000
)
var (
ErrNoOperationsTable = errors.New("no operations table supplied")
ErrNoIndexTable = errors.New("no index table supplied")
ErrStartTimeRequired = errors.New("start time is required for search queries")
seededRand *rand.Rand = rand.New(
rand.NewSource(time.Now().UnixNano()))
)
// SpanWriter for reading spans from ClickHouse
type ClickHouseReader struct {
db clickhouse.Conn
prometheus prometheus.Prometheus
sqlDB sqlstore.SQLStore
TraceDB string
operationsTable string
durationTable string
indexTable string
errorTable string
usageExplorerTable string
SpansTable string
spanAttributeTableV2 string
spanAttributesKeysTable string
dependencyGraphTable string
topLevelOperationsTable string
logsDB string
logsTable string
logsLocalTable string
logsAttributeKeys string
logsResourceKeys string
logsTagAttributeTableV2 string
queryProgressTracker queryprogress.QueryProgressTracker
logsTableV2 string
logsLocalTableV2 string
logsResourceTableV2 string
logsResourceLocalTableV2 string
liveTailRefreshSeconds int
cluster string
logsTableName string
logsLocalTableName string
traceTableName string
traceLocalTableName string
traceResourceTableV3 string
traceSummaryTable string
fluxIntervalForTraceDetail time.Duration
cache cache.Cache
metadataDB string
metadataTable string
}
// NewTraceReader returns a TraceReader for the database
func NewReader(
sqlDB sqlstore.SQLStore,
telemetryStore telemetrystore.TelemetryStore,
prometheus prometheus.Prometheus,
cluster string,
fluxIntervalForTraceDetail time.Duration,
cache cache.Cache,
) *ClickHouseReader {
options := NewOptions(primaryNamespace, archiveNamespace)
return NewReaderFromClickhouseConnection(options, sqlDB, telemetryStore, prometheus, cluster, fluxIntervalForTraceDetail, cache)
}
func NewReaderFromClickhouseConnection(
options *Options,
sqlDB sqlstore.SQLStore,
telemetryStore telemetrystore.TelemetryStore,
prometheus prometheus.Prometheus,
cluster string,
fluxIntervalForTraceDetail time.Duration,
cache cache.Cache,
) *ClickHouseReader {
logsTableName := options.primary.LogsTableV2
logsLocalTableName := options.primary.LogsLocalTableV2
traceTableName := options.primary.TraceIndexTableV3
traceLocalTableName := options.primary.TraceLocalTableNameV3
return &ClickHouseReader{
db: telemetryStore.ClickhouseDB(),
prometheus: prometheus,
sqlDB: sqlDB,
TraceDB: options.primary.TraceDB,
operationsTable: options.primary.OperationsTable,
indexTable: options.primary.IndexTable,
errorTable: options.primary.ErrorTable,
usageExplorerTable: options.primary.UsageExplorerTable,
durationTable: options.primary.DurationTable,
SpansTable: options.primary.SpansTable,
spanAttributeTableV2: options.primary.SpanAttributeTableV2,
spanAttributesKeysTable: options.primary.SpanAttributeKeysTable,
dependencyGraphTable: options.primary.DependencyGraphTable,
topLevelOperationsTable: options.primary.TopLevelOperationsTable,
logsDB: options.primary.LogsDB,
logsTable: options.primary.LogsTable,
logsLocalTable: options.primary.LogsLocalTable,
logsAttributeKeys: options.primary.LogsAttributeKeysTable,
logsResourceKeys: options.primary.LogsResourceKeysTable,
logsTagAttributeTableV2: options.primary.LogsTagAttributeTableV2,
liveTailRefreshSeconds: options.primary.LiveTailRefreshSeconds,
cluster: cluster,
queryProgressTracker: queryprogress.NewQueryProgressTracker(),
logsTableV2: options.primary.LogsTableV2,
logsLocalTableV2: options.primary.LogsLocalTableV2,
logsResourceTableV2: options.primary.LogsResourceTableV2,
logsResourceLocalTableV2: options.primary.LogsResourceLocalTableV2,
logsTableName: logsTableName,
logsLocalTableName: logsLocalTableName,
traceLocalTableName: traceLocalTableName,
traceTableName: traceTableName,
traceResourceTableV3: options.primary.TraceResourceTableV3,
traceSummaryTable: options.primary.TraceSummaryTable,
fluxIntervalForTraceDetail: fluxIntervalForTraceDetail,
cache: cache,
metadataDB: options.primary.MetadataDB,
metadataTable: options.primary.MetadataTable,
}
}
func (r *ClickHouseReader) GetInstantQueryMetricsResult(ctx context.Context, queryParams *model.InstantQueryMetricsParams) (*promql.Result, *stats.QueryStats, *model.ApiError) {
qry, err := r.prometheus.Engine().NewInstantQuery(ctx, r.prometheus.Storage(), nil, queryParams.Query, queryParams.Time)
if err != nil {
return nil, nil, &model.ApiError{Typ: model.ErrorBadData, Err: err}
}
res := qry.Exec(ctx)
// Optional stats field in response if parameter "stats" is not empty.
var qs stats.QueryStats
if queryParams.Stats != "" {
qs = stats.NewQueryStats(qry.Stats())
}
qry.Close()
return res, &qs, nil
}
func (r *ClickHouseReader) GetQueryRangeResult(ctx context.Context, query *model.QueryRangeParams) (*promql.Result, *stats.QueryStats, *model.ApiError) {
qry, err := r.prometheus.Engine().NewRangeQuery(ctx, r.prometheus.Storage(), nil, query.Query, query.Start, query.End, query.Step)
if err != nil {
return nil, nil, &model.ApiError{Typ: model.ErrorBadData, Err: err}
}
res := qry.Exec(ctx)
// Optional stats field in response if parameter "stats" is not empty.
var qs stats.QueryStats
if query.Stats != "" {
qs = stats.NewQueryStats(qry.Stats())
}
qry.Close()
return res, &qs, nil
}
func (r *ClickHouseReader) GetServicesList(ctx context.Context) (*[]string, error) {
services := []string{}
rows, err := r.db.Query(ctx, fmt.Sprintf(`SELECT DISTINCT serviceName FROM %s.%s WHERE ts_bucket_start > (toUnixTimestamp(now() - INTERVAL 1 DAY) - 1800) AND toDate(timestamp) > now() - INTERVAL 1 DAY`, r.TraceDB, r.traceTableName))
if err != nil {
return nil, fmt.Errorf("error in processing sql query")
}
defer rows.Close()
for rows.Next() {
var serviceName string
if err := rows.Scan(&serviceName); err != nil {
return &services, err
}
services = append(services, serviceName)
}
return &services, nil
}
func (r *ClickHouseReader) GetTopLevelOperations(ctx context.Context, start, end time.Time, services []string) (*map[string][]string, *model.ApiError) {
start = start.In(time.UTC)
// The `top_level_operations` that have `time` >= start
operations := map[string][]string{}
// We can't use the `end` because the `top_level_operations` table has the most recent instances of the operations
// We can only use the `start` time to filter the operations
query := fmt.Sprintf(`SELECT name, serviceName, max(time) as ts FROM %s.%s WHERE time >= @start`, r.TraceDB, r.topLevelOperationsTable)
if len(services) > 0 {
query += ` AND serviceName IN @services`
}
query += ` GROUP BY name, serviceName ORDER BY ts DESC LIMIT 5000`
rows, err := r.db.Query(ctx, query, clickhouse.Named("start", start), clickhouse.Named("services", services))
if err != nil {
zap.L().Error("Error in processing sql query", zap.Error(err))
return nil, &model.ApiError{Typ: model.ErrorExec, Err: fmt.Errorf("error in processing sql query")}
}
defer rows.Close()
for rows.Next() {
var name, serviceName string
var t time.Time
if err := rows.Scan(&name, &serviceName, &t); err != nil {
return nil, &model.ApiError{Typ: model.ErrorInternal, Err: fmt.Errorf("error in reading data")}
}
if _, ok := operations[serviceName]; !ok {
operations[serviceName] = []string{"overflow_operation"}
}
operations[serviceName] = append(operations[serviceName], name)
}
return &operations, nil
}
func (r *ClickHouseReader) buildResourceSubQuery(tags []model.TagQueryParam, svc string, start, end time.Time) (string, error) {
// assuming all will be resource attributes.
// and resource attributes are string for traces
filterSet := v3.FilterSet{}
for _, tag := range tags {
// skip the collector id as we don't add it to traces
if tag.Key == "signoz.collector.id" {
continue
}
key := v3.AttributeKey{
Key: tag.Key,
DataType: v3.AttributeKeyDataTypeString,
Type: v3.AttributeKeyTypeResource,
}
it := v3.FilterItem{
Key: key,
}
// as of now only in and not in are supported
switch tag.Operator {
case model.NotInOperator:
it.Operator = v3.FilterOperatorNotIn
it.Value = tag.StringValues
case model.InOperator:
it.Operator = v3.FilterOperatorIn
it.Value = tag.StringValues
default:
return "", fmt.Errorf("operator %s not supported", tag.Operator)
}
filterSet.Items = append(filterSet.Items, it)
}
filterSet.Items = append(filterSet.Items, v3.FilterItem{
Key: v3.AttributeKey{
Key: "service.name",
DataType: v3.AttributeKeyDataTypeString,
Type: v3.AttributeKeyTypeResource,
},
Operator: v3.FilterOperatorEqual,
Value: svc,
})
resourceSubQuery, err := resource.BuildResourceSubQuery(
r.TraceDB,
r.traceResourceTableV3,
start.Unix()-1800,
end.Unix(),
&filterSet,
[]v3.AttributeKey{},
v3.AttributeKey{},
false)
if err != nil {
zap.L().Error("Error in processing sql query", zap.Error(err))
return "", err
}
return resourceSubQuery, nil
}
func (r *ClickHouseReader) GetServices(ctx context.Context, queryParams *model.GetServicesParams) (*[]model.ServiceItem, *model.ApiError) {
if r.indexTable == "" {
return nil, &model.ApiError{Typ: model.ErrorExec, Err: ErrNoIndexTable}
}
topLevelOps, apiErr := r.GetTopLevelOperations(ctx, *queryParams.Start, *queryParams.End, nil)
if apiErr != nil {
return nil, apiErr
}
serviceItems := []model.ServiceItem{}
var wg sync.WaitGroup
// limit the number of concurrent queries to not overload the clickhouse server
sem := make(chan struct{}, 10)
var mtx sync.RWMutex
for svc, ops := range *topLevelOps {
sem <- struct{}{}
wg.Add(1)
go func(svc string, ops []string) {
defer wg.Done()
defer func() { <-sem }()
var serviceItem model.ServiceItem
var numErrors uint64
// Even if the total number of operations within the time range is less and the all
// the top level operations are high, we want to warn to let user know the issue
// with the instrumentation
serviceItem.DataWarning = model.DataWarning{
TopLevelOps: (*topLevelOps)[svc],
}
// default max_query_size = 262144
// Let's assume the average size of the item in `ops` is 50 bytes
// We can have 262144/50 = 5242 items in the `ops` array
// Although we have make it as big as 5k, We cap the number of items
// in the `ops` array to 1500
ops = ops[:int(math.Min(1500, float64(len(ops))))]
query := fmt.Sprintf(
`SELECT
quantile(0.99)(durationNano) as p99,
avg(durationNano) as avgDuration,
count(*) as numCalls
FROM %s.%s
WHERE serviceName = @serviceName AND name In @names AND timestamp>= @start AND timestamp<= @end`,
r.TraceDB, r.traceTableName,
)
errorQuery := fmt.Sprintf(
`SELECT
count(*) as numErrors
FROM %s.%s
WHERE serviceName = @serviceName AND name In @names AND timestamp>= @start AND timestamp<= @end AND statusCode=2`,
r.TraceDB, r.traceTableName,
)
args := []interface{}{}
args = append(args,
clickhouse.Named("start", strconv.FormatInt(queryParams.Start.UnixNano(), 10)),
clickhouse.Named("end", strconv.FormatInt(queryParams.End.UnixNano(), 10)),
clickhouse.Named("serviceName", svc),
clickhouse.Named("names", ops),
)
resourceSubQuery, err := r.buildResourceSubQuery(queryParams.Tags, svc, *queryParams.Start, *queryParams.End)
if err != nil {
zap.L().Error("Error in processing sql query", zap.Error(err))
return
}
query += `
AND (
resource_fingerprint GLOBAL IN ` +
resourceSubQuery +
`) AND ts_bucket_start >= @start_bucket AND ts_bucket_start <= @end_bucket`
args = append(args,
clickhouse.Named("start_bucket", strconv.FormatInt(queryParams.Start.Unix()-1800, 10)),
clickhouse.Named("end_bucket", strconv.FormatInt(queryParams.End.Unix(), 10)),
)
err = r.db.QueryRow(
ctx,
query,
args...,
).ScanStruct(&serviceItem)
if serviceItem.NumCalls == 0 {
return
}
if err != nil {
zap.L().Error("Error in processing sql query", zap.Error(err))
return
}
errorQuery += `
AND (
resource_fingerprint GLOBAL IN ` +
resourceSubQuery +
`) AND ts_bucket_start >= @start_bucket AND ts_bucket_start <= @end_bucket`
err = r.db.QueryRow(ctx, errorQuery, args...).Scan(&numErrors)
if err != nil {
zap.L().Error("Error in processing sql query", zap.Error(err))
return
}
serviceItem.ServiceName = svc
serviceItem.NumErrors = numErrors
mtx.Lock()
serviceItems = append(serviceItems, serviceItem)
mtx.Unlock()
}(svc, ops)
}
wg.Wait()
for idx := range serviceItems {
serviceItems[idx].CallRate = float64(serviceItems[idx].NumCalls) / float64(queryParams.Period)
serviceItems[idx].ErrorRate = float64(serviceItems[idx].NumErrors) * 100 / float64(serviceItems[idx].NumCalls)
}
return &serviceItems, nil
}
func getStatusFilters(query string, statusParams []string, excludeMap map[string]struct{}) string {
// status can only be two and if both are selected than they are equivalent to none selected
if _, ok := excludeMap["status"]; ok {
if len(statusParams) == 1 {
if statusParams[0] == "error" {
query += " AND hasError = false"
} else if statusParams[0] == "ok" {
query += " AND hasError = true"
}
}
} else if len(statusParams) == 1 {
if statusParams[0] == "error" {
query += " AND hasError = true"
} else if statusParams[0] == "ok" {
query += " AND hasError = false"
}
}
return query
}
func createTagQueryFromTagQueryParams(queryParams []model.TagQueryParam) []model.TagQuery {
tags := []model.TagQuery{}
for _, tag := range queryParams {
if len(tag.StringValues) > 0 {
tags = append(tags, model.NewTagQueryString(tag))
}
if len(tag.NumberValues) > 0 {
tags = append(tags, model.NewTagQueryNumber(tag))
}
if len(tag.BoolValues) > 0 {
tags = append(tags, model.NewTagQueryBool(tag))
}
}
return tags
}
func StringWithCharset(length int, charset string) string {
b := make([]byte, length)
for i := range b {
b[i] = charset[seededRand.Intn(len(charset))]
}
return string(b)
}
func String(length int) string {
return StringWithCharset(length, charset)
}
func buildQueryWithTagParams(_ context.Context, tags []model.TagQuery) (string, []interface{}, *model.ApiError) {
query := ""
var args []interface{}
for _, item := range tags {
var subQuery string
var argsSubQuery []interface{}
tagMapType := item.GetTagMapColumn()
switch item.GetOperator() {
case model.EqualOperator:
subQuery, argsSubQuery = addArithmeticOperator(item, tagMapType, "=")
case model.NotEqualOperator:
subQuery, argsSubQuery = addArithmeticOperator(item, tagMapType, "!=")
case model.LessThanOperator:
subQuery, argsSubQuery = addArithmeticOperator(item, tagMapType, "<")
case model.GreaterThanOperator:
subQuery, argsSubQuery = addArithmeticOperator(item, tagMapType, ">")
case model.InOperator:
subQuery, argsSubQuery = addInOperator(item, tagMapType, false)
case model.NotInOperator:
subQuery, argsSubQuery = addInOperator(item, tagMapType, true)
case model.LessThanEqualOperator:
subQuery, argsSubQuery = addArithmeticOperator(item, tagMapType, "<=")
case model.GreaterThanEqualOperator:
subQuery, argsSubQuery = addArithmeticOperator(item, tagMapType, ">=")
case model.ContainsOperator:
subQuery, argsSubQuery = addContainsOperator(item, tagMapType, false)
case model.NotContainsOperator:
subQuery, argsSubQuery = addContainsOperator(item, tagMapType, true)
case model.StartsWithOperator:
subQuery, argsSubQuery = addStartsWithOperator(item, tagMapType, false)
case model.NotStartsWithOperator:
subQuery, argsSubQuery = addStartsWithOperator(item, tagMapType, true)
case model.ExistsOperator:
subQuery, argsSubQuery = addExistsOperator(item, tagMapType, false)
case model.NotExistsOperator:
subQuery, argsSubQuery = addExistsOperator(item, tagMapType, true)
default:
return "", nil, &model.ApiError{Typ: model.ErrorExec, Err: fmt.Errorf("filter operator %s not supported", item.GetOperator())}
}
query += subQuery
args = append(args, argsSubQuery...)
}
return query, args, nil
}
func addInOperator(item model.TagQuery, tagMapType string, not bool) (string, []interface{}) {
values := item.GetValues()
args := []interface{}{}
notStr := ""
if not {
notStr = "NOT"
}
tagValuePair := []string{}
for _, value := range values {
tagKey := "inTagKey" + String(5)
tagValue := "inTagValue" + String(5)
tagValuePair = append(tagValuePair, fmt.Sprintf("%s[@%s] = @%s", tagMapType, tagKey, tagValue))
args = append(args, clickhouse.Named(tagKey, item.GetKey()))
args = append(args, clickhouse.Named(tagValue, value))
}
return fmt.Sprintf(" AND %s (%s)", notStr, strings.Join(tagValuePair, " OR ")), args
}
func addContainsOperator(item model.TagQuery, tagMapType string, not bool) (string, []interface{}) {
values := item.GetValues()
args := []interface{}{}
notStr := ""
if not {
notStr = "NOT"
}
tagValuePair := []string{}
for _, value := range values {
tagKey := "containsTagKey" + String(5)
tagValue := "containsTagValue" + String(5)
tagValuePair = append(tagValuePair, fmt.Sprintf("%s[@%s] ILIKE @%s", tagMapType, tagKey, tagValue))
args = append(args, clickhouse.Named(tagKey, item.GetKey()))
args = append(args, clickhouse.Named(tagValue, "%"+fmt.Sprintf("%v", value)+"%"))
}
return fmt.Sprintf(" AND %s (%s)", notStr, strings.Join(tagValuePair, " OR ")), args
}
func addStartsWithOperator(item model.TagQuery, tagMapType string, not bool) (string, []interface{}) {
values := item.GetValues()
args := []interface{}{}
notStr := ""
if not {
notStr = "NOT"
}
tagValuePair := []string{}
for _, value := range values {
tagKey := "startsWithTagKey" + String(5)
tagValue := "startsWithTagValue" + String(5)
tagValuePair = append(tagValuePair, fmt.Sprintf("%s[@%s] ILIKE @%s", tagMapType, tagKey, tagValue))
args = append(args, clickhouse.Named(tagKey, item.GetKey()))
args = append(args, clickhouse.Named(tagValue, "%"+fmt.Sprintf("%v", value)+"%"))
}
return fmt.Sprintf(" AND %s (%s)", notStr, strings.Join(tagValuePair, " OR ")), args
}
func addArithmeticOperator(item model.TagQuery, tagMapType string, operator string) (string, []interface{}) {
values := item.GetValues()
args := []interface{}{}
tagValuePair := []string{}
for _, value := range values {
tagKey := "arithmeticTagKey" + String(5)
tagValue := "arithmeticTagValue" + String(5)
tagValuePair = append(tagValuePair, fmt.Sprintf("%s[@%s] %s @%s", tagMapType, tagKey, operator, tagValue))
args = append(args, clickhouse.Named(tagKey, item.GetKey()))
args = append(args, clickhouse.Named(tagValue, value))
}
return fmt.Sprintf(" AND (%s)", strings.Join(tagValuePair, " OR ")), args
}
func addExistsOperator(item model.TagQuery, tagMapType string, not bool) (string, []interface{}) {
values := item.GetValues()
notStr := ""
if not {
notStr = "NOT"
}
args := []interface{}{}
tagOperatorPair := []string{}
for range values {
tagKey := "existsTagKey" + String(5)
tagOperatorPair = append(tagOperatorPair, fmt.Sprintf("mapContains(%s, @%s)", tagMapType, tagKey))
args = append(args, clickhouse.Named(tagKey, item.GetKey()))
}
return fmt.Sprintf(" AND %s (%s)", notStr, strings.Join(tagOperatorPair, " OR ")), args
}
func (r *ClickHouseReader) GetEntryPointOperations(ctx context.Context, queryParams *model.GetTopOperationsParams) (*[]model.TopOperationsItem, error) {
// Step 1: Get top operations for the given service
topOps, err := r.GetTopOperations(ctx, queryParams)
if err != nil {
return nil, errorsV2.Wrapf(err, errorsV2.TypeInternal, errorsV2.CodeInternal, "Error in getting Top Operations")
}
if topOps == nil {
return nil, errorsV2.Newf(errorsV2.TypeNotFound, errorsV2.CodeNotFound, "no top operations found")
}
// Step 2: Get entry point operation names for the given service using GetTopLevelOperations
// instead of running a separate query
serviceName := []string{queryParams.ServiceName}
var startTime, endTime time.Time
if queryParams.Start != nil {
startTime = *queryParams.Start
}
if queryParams.End != nil {
endTime = *queryParams.End
}
topLevelOpsResult, apiErr := r.GetTopLevelOperations(ctx, startTime, endTime, serviceName)
if apiErr != nil {
return nil, errorsV2.Wrapf(apiErr.Err, errorsV2.TypeInternal, errorsV2.CodeInternal, "failed to get top level operations")
}
// Create a set of entry point operations
entryPointSet := map[string]struct{}{}
// Extract operations for the requested service from topLevelOpsResult
if serviceOperations, ok := (*topLevelOpsResult)[queryParams.ServiceName]; ok {
// Skip the first "overflow_operation" if present
startIdx := 0
if len(serviceOperations) > 0 && serviceOperations[0] == "overflow_operation" {
startIdx = 1
}
// Add each operation name to the entry point set
for i := startIdx; i < len(serviceOperations); i++ {
entryPointSet[serviceOperations[i]] = struct{}{}
}
}
// Step 3: Filter topOps based on entryPointSet (same as original)
var filtered []model.TopOperationsItem
for _, op := range *topOps {
if _, ok := entryPointSet[op.Name]; ok {
filtered = append(filtered, op)
}
}
return &filtered, nil
}
func (r *ClickHouseReader) GetTopOperations(ctx context.Context, queryParams *model.GetTopOperationsParams) (*[]model.TopOperationsItem, *model.ApiError) {
namedArgs := []interface{}{
clickhouse.Named("start", strconv.FormatInt(queryParams.Start.UnixNano(), 10)),
clickhouse.Named("end", strconv.FormatInt(queryParams.End.UnixNano(), 10)),
clickhouse.Named("serviceName", queryParams.ServiceName),
clickhouse.Named("start_bucket", strconv.FormatInt(queryParams.Start.Unix()-1800, 10)),
clickhouse.Named("end_bucket", strconv.FormatInt(queryParams.End.Unix(), 10)),
}
var topOperationsItems []model.TopOperationsItem
query := fmt.Sprintf(`
SELECT
quantile(0.5)(durationNano) as p50,
quantile(0.95)(durationNano) as p95,
quantile(0.99)(durationNano) as p99,
COUNT(*) as numCalls,
countIf(statusCode=2) as errorCount,
name
FROM %s.%s
WHERE serviceName = @serviceName AND timestamp>= @start AND timestamp<= @end`,
r.TraceDB, r.traceTableName,
)
resourceSubQuery, err := r.buildResourceSubQuery(queryParams.Tags, queryParams.ServiceName, *queryParams.Start, *queryParams.End)
if err != nil {
zap.L().Error("Error in processing sql query", zap.Error(err))
return nil, &model.ApiError{Typ: model.ErrorExec, Err: fmt.Errorf("error in processing sql query")}
}
query += `
AND (
resource_fingerprint GLOBAL IN ` +
resourceSubQuery +
`) AND ts_bucket_start >= @start_bucket AND ts_bucket_start <= @end_bucket`
query += " GROUP BY name ORDER BY p99 DESC"
if queryParams.Limit > 0 {
query += " LIMIT @limit"
namedArgs = append(namedArgs, clickhouse.Named("limit", queryParams.Limit))
}
err = r.db.Select(ctx, &topOperationsItems, query, namedArgs...)
if err != nil {
zap.L().Error("Error in processing sql query", zap.Error(err))
return nil, &model.ApiError{Typ: model.ErrorExec, Err: fmt.Errorf("error in processing sql query")}
}
if topOperationsItems == nil {
topOperationsItems = []model.TopOperationsItem{}
}
return &topOperationsItems, nil
}
func (r *ClickHouseReader) GetUsage(ctx context.Context, queryParams *model.GetUsageParams) (*[]model.UsageItem, error) {
var usageItems []model.UsageItem
namedArgs := []interface{}{
clickhouse.Named("interval", queryParams.StepHour),
clickhouse.Named("start", strconv.FormatInt(queryParams.Start.UnixNano(), 10)),
clickhouse.Named("end", strconv.FormatInt(queryParams.End.UnixNano(), 10)),
}
var query string
if len(queryParams.ServiceName) != 0 {
namedArgs = append(namedArgs, clickhouse.Named("serviceName", queryParams.ServiceName))
query = fmt.Sprintf("SELECT toStartOfInterval(timestamp, INTERVAL @interval HOUR) as time, sum(count) as count FROM %s.%s WHERE service_name=@serviceName AND timestamp>=@start AND timestamp<=@end GROUP BY time ORDER BY time ASC", r.TraceDB, r.usageExplorerTable)
} else {
query = fmt.Sprintf("SELECT toStartOfInterval(timestamp, INTERVAL @interval HOUR) as time, sum(count) as count FROM %s.%s WHERE timestamp>=@start AND timestamp<=@end GROUP BY time ORDER BY time ASC", r.TraceDB, r.usageExplorerTable)
}
err := r.db.Select(ctx, &usageItems, query, namedArgs...)
zap.L().Info(query)
if err != nil {
zap.L().Error("Error in processing sql query", zap.Error(err))
return nil, fmt.Errorf("error in processing sql query")
}
for i := range usageItems {
usageItems[i].Timestamp = uint64(usageItems[i].Time.UnixNano())
}
if usageItems == nil {
usageItems = []model.UsageItem{}
}
return &usageItems, nil
}
func (r *ClickHouseReader) GetSpansForTrace(ctx context.Context, traceID string, traceDetailsQuery string) ([]model.SpanItemV2, *model.ApiError) {
var traceSummary model.TraceSummary
summaryQuery := fmt.Sprintf("SELECT * from %s.%s WHERE trace_id=$1", r.TraceDB, r.traceSummaryTable)
err := r.db.QueryRow(ctx, summaryQuery, traceID).Scan(&traceSummary.TraceID, &traceSummary.Start, &traceSummary.End, &traceSummary.NumSpans)
if err != nil {
if err == sql.ErrNoRows {
return []model.SpanItemV2{}, nil
}
zap.L().Error("Error in processing trace summary sql query", zap.Error(err))
return nil, model.ExecutionError(fmt.Errorf("error in processing trace summary sql query: %w", err))
}
var searchScanResponses []model.SpanItemV2
queryStartTime := time.Now()
err = r.db.Select(ctx, &searchScanResponses, traceDetailsQuery, traceID, strconv.FormatInt(traceSummary.Start.Unix()-1800, 10), strconv.FormatInt(traceSummary.End.Unix(), 10))
zap.L().Info(traceDetailsQuery)
if err != nil {
zap.L().Error("Error in processing sql query", zap.Error(err))
return nil, model.ExecutionError(fmt.Errorf("error in processing trace data sql query: %w", err))
}
zap.L().Info("trace details query took: ", zap.Duration("duration", time.Since(queryStartTime)), zap.String("traceID", traceID))
return searchScanResponses, nil
}
func (r *ClickHouseReader) GetWaterfallSpansForTraceWithMetadataCache(ctx context.Context, orgID valuer.UUID, traceID string) (*model.GetWaterfallSpansForTraceWithMetadataCache, error) {
cachedTraceData := new(model.GetWaterfallSpansForTraceWithMetadataCache)
err := r.cache.Get(ctx, orgID, strings.Join([]string{"getWaterfallSpansForTraceWithMetadata", traceID}, "-"), cachedTraceData, false)
if err != nil {
zap.L().Debug("error in retrieving getWaterfallSpansForTraceWithMetadata cache", zap.Error(err), zap.String("traceID", traceID))
return nil, err
}
if time.Since(time.UnixMilli(int64(cachedTraceData.EndTime))) < r.fluxIntervalForTraceDetail {
zap.L().Info("the trace end time falls under the flux interval, skipping getWaterfallSpansForTraceWithMetadata cache", zap.String("traceID", traceID))
return nil, errors.Errorf("the trace end time falls under the flux interval, skipping getWaterfallSpansForTraceWithMetadata cache, traceID: %s", traceID)
}
zap.L().Info("cache is successfully hit, applying cache for getWaterfallSpansForTraceWithMetadata", zap.String("traceID", traceID))
return cachedTraceData, nil
}
func (r *ClickHouseReader) GetWaterfallSpansForTraceWithMetadata(ctx context.Context, orgID valuer.UUID, traceID string, req *model.GetWaterfallSpansForTraceWithMetadataParams) (*model.GetWaterfallSpansForTraceWithMetadataResponse, *model.ApiError) {
response := new(model.GetWaterfallSpansForTraceWithMetadataResponse)
var startTime, endTime, durationNano, totalErrorSpans, totalSpans uint64
var spanIdToSpanNodeMap = map[string]*model.Span{}
var traceRoots []*model.Span
var serviceNameToTotalDurationMap = map[string]uint64{}
var serviceNameIntervalMap = map[string][]tracedetail.Interval{}
var hasMissingSpans bool
claims, errv2 := authtypes.ClaimsFromContext(ctx)
cachedTraceData, err := r.GetWaterfallSpansForTraceWithMetadataCache(ctx, orgID, traceID)
if err == nil {
startTime = cachedTraceData.StartTime
endTime = cachedTraceData.EndTime
durationNano = cachedTraceData.DurationNano
spanIdToSpanNodeMap = cachedTraceData.SpanIdToSpanNodeMap
serviceNameToTotalDurationMap = cachedTraceData.ServiceNameToTotalDurationMap
traceRoots = cachedTraceData.TraceRoots
totalSpans = cachedTraceData.TotalSpans
totalErrorSpans = cachedTraceData.TotalErrorSpans
hasMissingSpans = cachedTraceData.HasMissingSpans
if errv2 == nil {
telemetry.GetInstance().SendEvent(telemetry.TELEMETRY_EVENT_TRACE_DETAIL_API, map[string]interface{}{"traceSize": totalSpans}, claims.Email, true, false)
}
}
if err != nil {
zap.L().Info("cache miss for getWaterfallSpansForTraceWithMetadata", zap.String("traceID", traceID))
searchScanResponses, err := r.GetSpansForTrace(ctx, traceID, fmt.Sprintf("SELECT DISTINCT ON (span_id) timestamp, duration_nano, span_id, trace_id, has_error, kind, resource_string_service$$name, name, references, attributes_string, attributes_number, attributes_bool, resources_string, events, status_message, status_code_string, kind_string FROM %s.%s WHERE trace_id=$1 and ts_bucket_start>=$2 and ts_bucket_start<=$3 ORDER BY timestamp ASC, name ASC", r.TraceDB, r.traceTableName))
if err != nil {
return nil, err
}
if len(searchScanResponses) == 0 {
return response, nil
}
totalSpans = uint64(len(searchScanResponses))
if errv2 == nil {
telemetry.GetInstance().SendEvent(telemetry.TELEMETRY_EVENT_TRACE_DETAIL_API, map[string]interface{}{"traceSize": totalSpans}, claims.Email, true, false)
}
processingBeforeCache := time.Now()
for _, item := range searchScanResponses {
ref := []model.OtelSpanRef{}
err := json.Unmarshal([]byte(item.References), &ref)
if err != nil {
zap.L().Error("getWaterfallSpansForTraceWithMetadata: error unmarshalling references", zap.Error(err), zap.String("traceID", traceID))
return nil, model.BadRequest(fmt.Errorf("getWaterfallSpansForTraceWithMetadata: error unmarshalling references %w", err))
}
// merge attributes_number and attributes_bool to attributes_string
for k, v := range item.Attributes_bool {
item.Attributes_string[k] = fmt.Sprintf("%v", v)
}
for k, v := range item.Attributes_number {
item.Attributes_string[k] = strconv.FormatFloat(v, 'f', -1, 64)
}
for k, v := range item.Resources_string {
item.Attributes_string[k] = v
}
jsonItem := model.Span{
SpanID: item.SpanID,
TraceID: item.TraceID,
ServiceName: item.ServiceName,
Name: item.Name,
Kind: int32(item.Kind),
DurationNano: item.DurationNano,
HasError: item.HasError,
StatusMessage: item.StatusMessage,
StatusCodeString: item.StatusCodeString,
SpanKind: item.SpanKind,
References: ref,
Events: item.Events,
TagMap: item.Attributes_string,
Children: make([]*model.Span, 0),
}
// metadata calculation
startTimeUnixNano := uint64(item.TimeUnixNano.UnixNano())
if startTime == 0 || startTimeUnixNano < startTime {
startTime = startTimeUnixNano
}
if endTime == 0 || (startTimeUnixNano+jsonItem.DurationNano) > endTime {
endTime = (startTimeUnixNano + jsonItem.DurationNano)
}
if durationNano == 0 || jsonItem.DurationNano > durationNano {
durationNano = jsonItem.DurationNano
}
if jsonItem.HasError {
totalErrorSpans = totalErrorSpans + 1
}
// convert start timestamp to millis because right now frontend is expecting it in millis
jsonItem.TimeUnixNano = uint64(item.TimeUnixNano.UnixNano() / 1000000)
// collect the intervals for service for execution time calculation
serviceNameIntervalMap[jsonItem.ServiceName] =
append(serviceNameIntervalMap[jsonItem.ServiceName], tracedetail.Interval{StartTime: jsonItem.TimeUnixNano, Duration: jsonItem.DurationNano / 1000000, Service: jsonItem.ServiceName})
// append to the span node map
spanIdToSpanNodeMap[jsonItem.SpanID] = &jsonItem
}
// traverse through the map and append each node to the children array of the parent node
// and add the missing spans
for _, spanNode := range spanIdToSpanNodeMap {
hasParentSpanNode := false
for _, reference := range spanNode.References {
if reference.RefType == "CHILD_OF" && reference.SpanId != "" {
hasParentSpanNode = true
if parentNode, exists := spanIdToSpanNodeMap[reference.SpanId]; exists {
parentNode.Children = append(parentNode.Children, spanNode)
} else {
// insert the missing span
missingSpan := model.Span{
SpanID: reference.SpanId,
TraceID: spanNode.TraceID,
ServiceName: "",
Name: "Missing Span",
TimeUnixNano: spanNode.TimeUnixNano,
Kind: 0,
DurationNano: spanNode.DurationNano,
HasError: false,
StatusMessage: "",
StatusCodeString: "",
SpanKind: "",
Children: make([]*model.Span, 0),
}
missingSpan.Children = append(missingSpan.Children, spanNode)
spanIdToSpanNodeMap[missingSpan.SpanID] = &missingSpan
traceRoots = append(traceRoots, &missingSpan)
hasMissingSpans = true
}
}
}
if !hasParentSpanNode && !tracedetail.ContainsWaterfallSpan(traceRoots, spanNode) {
traceRoots = append(traceRoots, spanNode)
}
}
// sort the trace roots to add missing spans at the right order
sort.Slice(traceRoots, func(i, j int) bool {
if traceRoots[i].TimeUnixNano == traceRoots[j].TimeUnixNano {
return traceRoots[i].Name < traceRoots[j].Name
}
return traceRoots[i].TimeUnixNano < traceRoots[j].TimeUnixNano
})
serviceNameToTotalDurationMap = tracedetail.CalculateServiceTime(serviceNameIntervalMap)
traceCache := model.GetWaterfallSpansForTraceWithMetadataCache{
StartTime: startTime,
EndTime: endTime,
DurationNano: durationNano,
TotalSpans: totalSpans,
TotalErrorSpans: totalErrorSpans,
SpanIdToSpanNodeMap: spanIdToSpanNodeMap,
ServiceNameToTotalDurationMap: serviceNameToTotalDurationMap,
TraceRoots: traceRoots,
HasMissingSpans: hasMissingSpans,
}
zap.L().Info("getWaterfallSpansForTraceWithMetadata: processing pre cache", zap.Duration("duration", time.Since(processingBeforeCache)), zap.String("traceID", traceID))
cacheErr := r.cache.Set(ctx, orgID, strings.Join([]string{"getWaterfallSpansForTraceWithMetadata", traceID}, "-"), &traceCache, time.Minute*5)
if cacheErr != nil {
zap.L().Debug("failed to store cache for getWaterfallSpansForTraceWithMetadata", zap.String("traceID", traceID), zap.Error(err))
}
}
processingPostCache := time.Now()
selectedSpans, uncollapsedSpans, rootServiceName, rootServiceEntryPoint := tracedetail.GetSelectedSpans(req.UncollapsedSpans, req.SelectedSpanID, traceRoots, spanIdToSpanNodeMap, req.IsSelectedSpanIDUnCollapsed)
zap.L().Info("getWaterfallSpansForTraceWithMetadata: processing post cache", zap.Duration("duration", time.Since(processingPostCache)), zap.String("traceID", traceID))
response.Spans = selectedSpans
response.UncollapsedSpans = uncollapsedSpans
response.StartTimestampMillis = startTime / 1000000
response.EndTimestampMillis = endTime / 1000000
response.TotalSpansCount = totalSpans
response.TotalErrorSpansCount = totalErrorSpans
response.RootServiceName = rootServiceName
response.RootServiceEntryPoint = rootServiceEntryPoint
response.ServiceNameToTotalDurationMap = serviceNameToTotalDurationMap
response.HasMissingSpans = hasMissingSpans
return response, nil
}
func (r *ClickHouseReader) GetFlamegraphSpansForTraceCache(ctx context.Context, orgID valuer.UUID, traceID string) (*model.GetFlamegraphSpansForTraceCache, error) {
cachedTraceData := new(model.GetFlamegraphSpansForTraceCache)
err := r.cache.Get(ctx, orgID, strings.Join([]string{"getFlamegraphSpansForTrace", traceID}, "-"), cachedTraceData, false)
if err != nil {
zap.L().Debug("error in retrieving getFlamegraphSpansForTrace cache", zap.Error(err), zap.String("traceID", traceID))
return nil, err
}
if time.Since(time.UnixMilli(int64(cachedTraceData.EndTime))) < r.fluxIntervalForTraceDetail {
zap.L().Info("the trace end time falls under the flux interval, skipping getFlamegraphSpansForTrace cache", zap.String("traceID", traceID))
return nil, errors.Errorf("the trace end time falls under the flux interval, skipping getFlamegraphSpansForTrace cache, traceID: %s", traceID)
}
zap.L().Info("cache is successfully hit, applying cache for getFlamegraphSpansForTrace", zap.String("traceID", traceID))
return cachedTraceData, nil
}
func (r *ClickHouseReader) GetFlamegraphSpansForTrace(ctx context.Context, orgID valuer.UUID, traceID string, req *model.GetFlamegraphSpansForTraceParams) (*model.GetFlamegraphSpansForTraceResponse, *model.ApiError) {
trace := new(model.GetFlamegraphSpansForTraceResponse)
var startTime, endTime, durationNano uint64
var spanIdToSpanNodeMap = map[string]*model.FlamegraphSpan{}
// map[traceID][level]span
var selectedSpans = [][]*model.FlamegraphSpan{}
var traceRoots []*model.FlamegraphSpan
// get the trace tree from cache!
cachedTraceData, err := r.GetFlamegraphSpansForTraceCache(ctx, orgID, traceID)
if err == nil {
startTime = cachedTraceData.StartTime
endTime = cachedTraceData.EndTime
durationNano = cachedTraceData.DurationNano
selectedSpans = cachedTraceData.SelectedSpans
traceRoots = cachedTraceData.TraceRoots
}
if err != nil {
zap.L().Info("cache miss for getFlamegraphSpansForTrace", zap.String("traceID", traceID))
searchScanResponses, err := r.GetSpansForTrace(ctx, traceID, fmt.Sprintf("SELECT timestamp, duration_nano, span_id, trace_id, has_error,references, resource_string_service$$name, name FROM %s.%s WHERE trace_id=$1 and ts_bucket_start>=$2 and ts_bucket_start<=$3 ORDER BY timestamp ASC, name ASC", r.TraceDB, r.traceTableName))
if err != nil {
return nil, err
}
if len(searchScanResponses) == 0 {
return trace, nil
}
processingBeforeCache := time.Now()
for _, item := range searchScanResponses {
ref := []model.OtelSpanRef{}
err := json.Unmarshal([]byte(item.References), &ref)
if err != nil {
zap.L().Error("Error unmarshalling references", zap.Error(err))
return nil, &model.ApiError{Typ: model.ErrorBadData, Err: fmt.Errorf("error in unmarshalling references: %w", err)}
}
jsonItem := model.FlamegraphSpan{
SpanID: item.SpanID,
TraceID: item.TraceID,
ServiceName: item.ServiceName,
Name: item.Name,
DurationNano: item.DurationNano,
HasError: item.HasError,
References: ref,
Children: make([]*model.FlamegraphSpan, 0),
}
// metadata calculation
startTimeUnixNano := uint64(item.TimeUnixNano.UnixNano())
if startTime == 0 || startTimeUnixNano < startTime {
startTime = startTimeUnixNano
}
if endTime == 0 || (startTimeUnixNano+jsonItem.DurationNano) > endTime {
endTime = (startTimeUnixNano + jsonItem.DurationNano)
}
if durationNano == 0 || jsonItem.DurationNano > durationNano {
durationNano = jsonItem.DurationNano
}
jsonItem.TimeUnixNano = uint64(item.TimeUnixNano.UnixNano() / 1000000)
spanIdToSpanNodeMap[jsonItem.SpanID] = &jsonItem
}
// traverse through the map and append each node to the children array of the parent node
// and add missing spans
for _, spanNode := range spanIdToSpanNodeMap {
hasParentSpanNode := false
for _, reference := range spanNode.References {
if reference.RefType == "CHILD_OF" && reference.SpanId != "" {
hasParentSpanNode = true
if parentNode, exists := spanIdToSpanNodeMap[reference.SpanId]; exists {
parentNode.Children = append(parentNode.Children, spanNode)
} else {
// insert the missing spans
missingSpan := model.FlamegraphSpan{
SpanID: reference.SpanId,
TraceID: spanNode.TraceID,
ServiceName: "",
Name: "Missing Span",
TimeUnixNano: spanNode.TimeUnixNano,
DurationNano: spanNode.DurationNano,
HasError: false,
Children: make([]*model.FlamegraphSpan, 0),
}
missingSpan.Children = append(missingSpan.Children, spanNode)
spanIdToSpanNodeMap[missingSpan.SpanID] = &missingSpan
traceRoots = append(traceRoots, &missingSpan)
}
}
}
if !hasParentSpanNode && !tracedetail.ContainsFlamegraphSpan(traceRoots, spanNode) {
traceRoots = append(traceRoots, spanNode)
}
}
selectedSpans = tracedetail.GetSelectedSpansForFlamegraph(traceRoots, spanIdToSpanNodeMap)
traceCache := model.GetFlamegraphSpansForTraceCache{
StartTime: startTime,
EndTime: endTime,
DurationNano: durationNano,
SelectedSpans: selectedSpans,
TraceRoots: traceRoots,
}
zap.L().Info("getFlamegraphSpansForTrace: processing pre cache", zap.Duration("duration", time.Since(processingBeforeCache)), zap.String("traceID", traceID))
cacheErr := r.cache.Set(ctx, orgID, strings.Join([]string{"getFlamegraphSpansForTrace", traceID}, "-"), &traceCache, time.Minute*5)
if cacheErr != nil {
zap.L().Debug("failed to store cache for getFlamegraphSpansForTrace", zap.String("traceID", traceID), zap.Error(err))
}
}
processingPostCache := time.Now()
selectedSpansForRequest := tracedetail.GetSelectedSpansForFlamegraphForRequest(req.SelectedSpanID, selectedSpans, startTime, endTime)
zap.L().Info("getFlamegraphSpansForTrace: processing post cache", zap.Duration("duration", time.Since(processingPostCache)), zap.String("traceID", traceID))
trace.Spans = selectedSpansForRequest
trace.StartTimestampMillis = startTime / 1000000
trace.EndTimestampMillis = endTime / 1000000
return trace, nil
}
func (r *ClickHouseReader) GetDependencyGraph(ctx context.Context, queryParams *model.GetServicesParams) (*[]model.ServiceMapDependencyResponseItem, error) {
response := []model.ServiceMapDependencyResponseItem{}
args := []interface{}{}
args = append(args,
clickhouse.Named("start", uint64(queryParams.Start.Unix())),
clickhouse.Named("end", uint64(queryParams.End.Unix())),
clickhouse.Named("duration", uint64(queryParams.End.Unix()-queryParams.Start.Unix())),
)
query := fmt.Sprintf(`
WITH
quantilesMergeState(0.5, 0.75, 0.9, 0.95, 0.99)(duration_quantiles_state) AS duration_quantiles_state,
finalizeAggregation(duration_quantiles_state) AS result
SELECT
src as parent,
dest as child,
result[1] AS p50,
result[2] AS p75,
result[3] AS p90,
result[4] AS p95,
result[5] AS p99,
sum(total_count) as callCount,
sum(total_count)/ @duration AS callRate,
sum(error_count)/sum(total_count) * 100 as errorRate
FROM %s.%s
WHERE toUInt64(toDateTime(timestamp)) >= @start AND toUInt64(toDateTime(timestamp)) <= @end`,
r.TraceDB, r.dependencyGraphTable,
)
tags := createTagQueryFromTagQueryParams(queryParams.Tags)
filterQuery, filterArgs := services.BuildServiceMapQuery(tags)
query += filterQuery + " GROUP BY src, dest;"
args = append(args, filterArgs...)
zap.L().Debug("GetDependencyGraph query", zap.String("query", query), zap.Any("args", args))
err := r.db.Select(ctx, &response, query, args...)
if err != nil {
zap.L().Error("Error in processing sql query", zap.Error(err))
return nil, fmt.Errorf("error in processing sql query %w", err)
}
return &response, nil
}
func getLocalTableName(tableName string) string {
tableNameSplit := strings.Split(tableName, ".")
return tableNameSplit[0] + "." + strings.Split(tableNameSplit[1], "distributed_")[1]
}
func (r *ClickHouseReader) setTTLLogs(ctx context.Context, orgID string, params *model.TTLParams) (*model.SetTTLResponseItem, *model.ApiError) {
// uuid is used as transaction id
uuidWithHyphen := uuid.New()
uuid := strings.Replace(uuidWithHyphen.String(), "-", "", -1)
coldStorageDuration := -1
if len(params.ColdStorageVolume) > 0 {
coldStorageDuration = int(params.ToColdStorageDuration)
}
tableNameArray := []string{r.logsDB + "." + r.logsLocalTableV2, r.logsDB + "." + r.logsResourceLocalTableV2}
// check if there is existing things to be done
for _, tableName := range tableNameArray {
statusItem, err := r.checkTTLStatusItem(ctx, orgID, tableName)
if err != nil {
return nil, &model.ApiError{Typ: model.ErrorExec, Err: fmt.Errorf("error in processing ttl_status check sql query")}
}
if statusItem.Status == constants.StatusPending {
return nil, &model.ApiError{Typ: model.ErrorConflict, Err: fmt.Errorf("TTL is already running")}
}
}
// TTL query for logs_v2 table
ttlLogsV2 := fmt.Sprintf(
"ALTER TABLE %v ON CLUSTER %s MODIFY TTL toDateTime(timestamp / 1000000000) + "+
"INTERVAL %v SECOND DELETE", tableNameArray[0], r.cluster, params.DelDuration)
if len(params.ColdStorageVolume) > 0 {
ttlLogsV2 += fmt.Sprintf(", toDateTime(timestamp / 1000000000)"+
" + INTERVAL %v SECOND TO VOLUME '%s'",
params.ToColdStorageDuration, params.ColdStorageVolume)
}
// TTL query for logs_v2_resource table
// adding 1800 as our bucket size is 1800 seconds
ttlLogsV2Resource := fmt.Sprintf(
"ALTER TABLE %v ON CLUSTER %s MODIFY TTL toDateTime(seen_at_ts_bucket_start) + toIntervalSecond(1800) + "+
"INTERVAL %v SECOND DELETE", tableNameArray[1], r.cluster, params.DelDuration)
if len(params.ColdStorageVolume) > 0 {
ttlLogsV2Resource += fmt.Sprintf(", toDateTime(seen_at_ts_bucket_start) + toIntervalSecond(1800) + "+
"INTERVAL %v SECOND TO VOLUME '%s'",
params.ToColdStorageDuration, params.ColdStorageVolume)
}
ttlPayload := map[string]string{
tableNameArray[0]: ttlLogsV2,
tableNameArray[1]: ttlLogsV2Resource,
}
// set the ttl if nothing is pending/ no errors
go func(ttlPayload map[string]string) {
for tableName, query := range ttlPayload {
// https://github.com/SigNoz/signoz/issues/5470
// we will change ttl for only the new parts and not the old ones
query += " SETTINGS materialize_ttl_after_modify=0"
ttl := types.TTLSetting{
Identifiable: types.Identifiable{
ID: valuer.GenerateUUID(),
},
TimeAuditable: types.TimeAuditable{
CreatedAt: time.Now(),
UpdatedAt: time.Now(),
},
TransactionID: uuid,
TableName: tableName,
TTL: int(params.DelDuration),
Status: constants.StatusPending,
ColdStorageTTL: coldStorageDuration,
OrgID: orgID,
}
_, dbErr := r.
sqlDB.
BunDB().
NewInsert().
Model(&ttl).
Exec(ctx)
if dbErr != nil {
zap.L().Error("error in inserting to ttl_status table", zap.Error(dbErr))
return
}
err := r.setColdStorage(context.Background(), tableName, params.ColdStorageVolume)
if err != nil {
zap.L().Error("error in setting cold storage", zap.Error(err))
statusItem, err := r.checkTTLStatusItem(ctx, orgID, tableName)
if err == nil {
_, dbErr := r.
sqlDB.
BunDB().
NewUpdate().
Model(new(types.TTLSetting)).
Set("updated_at = ?", time.Now()).
Set("status = ?", constants.StatusFailed).
Where("id = ?", statusItem.ID.StringValue()).
Exec(ctx)
if dbErr != nil {
zap.L().Error("Error in processing ttl_status update sql query", zap.Error(dbErr))
return
}
}
return
}
zap.L().Info("Executing TTL request: ", zap.String("request", query))
statusItem, _ := r.checkTTLStatusItem(ctx, orgID, tableName)
if err := r.db.Exec(ctx, query); err != nil {
zap.L().Error("error while setting ttl", zap.Error(err))
_, dbErr := r.
sqlDB.
BunDB().
NewUpdate().
Model(new(types.TTLSetting)).
Set("updated_at = ?", time.Now()).
Set("status = ?", constants.StatusFailed).
Where("id = ?", statusItem.ID.StringValue()).
Exec(ctx)
if dbErr != nil {
zap.L().Error("Error in processing ttl_status update sql query", zap.Error(dbErr))
return
}
return
}
_, dbErr = r.
sqlDB.
BunDB().
NewUpdate().
Model(new(types.TTLSetting)).
Set("updated_at = ?", time.Now()).
Set("status = ?", constants.StatusSuccess).
Where("id = ?", statusItem.ID.StringValue()).
Exec(ctx)
if dbErr != nil {
zap.L().Error("Error in processing ttl_status update sql query", zap.Error(dbErr))
return
}
}
}(ttlPayload)
return &model.SetTTLResponseItem{Message: "move ttl has been successfully set up"}, nil
}
func (r *ClickHouseReader) setTTLTraces(ctx context.Context, orgID string, params *model.TTLParams) (*model.SetTTLResponseItem, *model.ApiError) {
// uuid is used as transaction id
uuidWithHyphen := uuid.New()
uuid := strings.Replace(uuidWithHyphen.String(), "-", "", -1)
tableNames := []string{
r.TraceDB + "." + r.traceTableName,
r.TraceDB + "." + r.traceResourceTableV3,
r.TraceDB + "." + signozErrorIndexTable,
r.TraceDB + "." + signozUsageExplorerTable,
r.TraceDB + "." + defaultDependencyGraphTable,
r.TraceDB + "." + r.traceSummaryTable,
}
coldStorageDuration := -1
if len(params.ColdStorageVolume) > 0 {
coldStorageDuration = int(params.ToColdStorageDuration)
}
// check if there is existing things to be done
for _, tableName := range tableNames {
statusItem, err := r.checkTTLStatusItem(ctx, orgID, tableName)
if err != nil {
return nil, &model.ApiError{Typ: model.ErrorExec, Err: fmt.Errorf("error in processing ttl_status check sql query")}
}
if statusItem.Status == constants.StatusPending {
return nil, &model.ApiError{Typ: model.ErrorConflict, Err: fmt.Errorf("TTL is already running")}
}
}
// TTL query
ttlV2 := "ALTER TABLE %s ON CLUSTER %s MODIFY TTL toDateTime(%s) + INTERVAL %v SECOND DELETE"
ttlV2ColdStorage := ", toDateTime(%s) + INTERVAL %v SECOND TO VOLUME '%s'"
// TTL query for resource table
ttlV2Resource := "ALTER TABLE %s ON CLUSTER %s MODIFY TTL toDateTime(seen_at_ts_bucket_start) + toIntervalSecond(1800) + INTERVAL %v SECOND DELETE"
ttlTracesV2ResourceColdStorage := ", toDateTime(seen_at_ts_bucket_start) + toIntervalSecond(1800) + INTERVAL %v SECOND TO VOLUME '%s'"
for _, distributedTableName := range tableNames {
go func(distributedTableName string) {
tableName := getLocalTableName(distributedTableName)
// for trace summary table, we need to use end instead of timestamp
timestamp := "timestamp"
if strings.HasSuffix(distributedTableName, r.traceSummaryTable) {
timestamp = "end"
}
ttl := types.TTLSetting{
Identifiable: types.Identifiable{
ID: valuer.GenerateUUID(),
},
TimeAuditable: types.TimeAuditable{
CreatedAt: time.Now(),
UpdatedAt: time.Now(),
},
TransactionID: uuid,
TableName: tableName,
TTL: int(params.DelDuration),
Status: constants.StatusPending,
ColdStorageTTL: coldStorageDuration,
OrgID: orgID,
}
_, dbErr := r.
sqlDB.
BunDB().
NewInsert().
Model(&ttl).
Exec(ctx)
if dbErr != nil {
zap.L().Error("error in inserting to ttl_status table", zap.Error(dbErr))
return
}
req := fmt.Sprintf(ttlV2, tableName, r.cluster, timestamp, params.DelDuration)
if strings.HasSuffix(distributedTableName, r.traceResourceTableV3) {
req = fmt.Sprintf(ttlV2Resource, tableName, r.cluster, params.DelDuration)
}
if len(params.ColdStorageVolume) > 0 {
if strings.HasSuffix(distributedTableName, r.traceResourceTableV3) {
req += fmt.Sprintf(ttlTracesV2ResourceColdStorage, params.ToColdStorageDuration, params.ColdStorageVolume)
} else {
req += fmt.Sprintf(ttlV2ColdStorage, timestamp, params.ToColdStorageDuration, params.ColdStorageVolume)
}
}
err := r.setColdStorage(context.Background(), tableName, params.ColdStorageVolume)
if err != nil {
zap.L().Error("Error in setting cold storage", zap.Error(err))
statusItem, err := r.checkTTLStatusItem(ctx, orgID, tableName)
if err == nil {
_, dbErr := r.
sqlDB.
BunDB().
NewUpdate().
Model(new(types.TTLSetting)).
Set("updated_at = ?", time.Now()).
Set("status = ?", constants.StatusFailed).
Where("id = ?", statusItem.ID.StringValue()).
Exec(ctx)
if dbErr != nil {
zap.L().Error("Error in processing ttl_status update sql query", zap.Error(dbErr))
return
}
}
return
}
req += " SETTINGS materialize_ttl_after_modify=0;"
zap.L().Error(" ExecutingTTL request: ", zap.String("request", req))
statusItem, _ := r.checkTTLStatusItem(ctx, orgID, tableName)
if err := r.db.Exec(ctx, req); err != nil {
zap.L().Error("Error in executing set TTL query", zap.Error(err))
_, dbErr := r.
sqlDB.
BunDB().
NewUpdate().
Model(new(types.TTLSetting)).
Set("updated_at = ?", time.Now()).
Set("status = ?", constants.StatusFailed).
Where("id = ?", statusItem.ID.StringValue()).
Exec(ctx)
if dbErr != nil {
zap.L().Error("Error in processing ttl_status update sql query", zap.Error(dbErr))
return
}
return
}
_, dbErr = r.
sqlDB.
BunDB().
NewUpdate().
Model(new(types.TTLSetting)).
Set("updated_at = ?", time.Now()).
Set("status = ?", constants.StatusSuccess).
Where("id = ?", statusItem.ID.StringValue()).
Exec(ctx)
if dbErr != nil {
zap.L().Error("Error in processing ttl_status update sql query", zap.Error(dbErr))
return
}
}(distributedTableName)
}
return &model.SetTTLResponseItem{Message: "move ttl has been successfully set up"}, nil
}
// SetTTL sets the TTL for traces or metrics or logs tables.
// This is an async API which creates goroutines to set TTL.
// Status of TTL update is tracked with ttl_status table in sqlite db.
func (r *ClickHouseReader) SetTTL(ctx context.Context, orgID string, params *model.TTLParams) (*model.SetTTLResponseItem, *model.ApiError) {
// Keep only latest 100 transactions/requests
r.deleteTtlTransactions(ctx, orgID, 100)
// uuid is used as transaction id
uuidWithHyphen := uuid.New()
uuid := strings.Replace(uuidWithHyphen.String(), "-", "", -1)
coldStorageDuration := -1
if len(params.ColdStorageVolume) > 0 {
coldStorageDuration = int(params.ToColdStorageDuration)
}
switch params.Type {
case constants.TraceTTL:
return r.setTTLTraces(ctx, orgID, params)
case constants.MetricsTTL:
tableNames := []string{
signozMetricDBName + "." + signozSampleLocalTableName,
signozMetricDBName + "." + signozSamplesAgg5mLocalTableName,
signozMetricDBName + "." + signozSamplesAgg30mLocalTableName,
signozMetricDBName + "." + signozExpHistLocalTableName,
signozMetricDBName + "." + signozTSLocalTableNameV4,
signozMetricDBName + "." + signozTSLocalTableNameV46Hrs,
signozMetricDBName + "." + signozTSLocalTableNameV41Day,
signozMetricDBName + "." + signozTSLocalTableNameV41Week,
}
for _, tableName := range tableNames {
statusItem, err := r.checkTTLStatusItem(ctx, orgID, tableName)
if err != nil {
return nil, &model.ApiError{Typ: model.ErrorExec, Err: fmt.Errorf("error in processing ttl_status check sql query")}
}
if statusItem.Status == constants.StatusPending {
return nil, &model.ApiError{Typ: model.ErrorConflict, Err: fmt.Errorf("TTL is already running")}
}
}
metricTTL := func(tableName string) {
ttl := types.TTLSetting{
Identifiable: types.Identifiable{
ID: valuer.GenerateUUID(),
},
TimeAuditable: types.TimeAuditable{
CreatedAt: time.Now(),
UpdatedAt: time.Now(),
},
TransactionID: uuid,
TableName: tableName,
TTL: int(params.DelDuration),
Status: constants.StatusPending,
ColdStorageTTL: coldStorageDuration,
OrgID: orgID,
}
_, dbErr := r.
sqlDB.
BunDB().
NewInsert().
Model(&ttl).
Exec(ctx)
if dbErr != nil {
zap.L().Error("error in inserting to ttl_status table", zap.Error(dbErr))
return
}
timeColumn := "timestamp_ms"
if strings.Contains(tableName, "v4") || strings.Contains(tableName, "exp_hist") {
timeColumn = "unix_milli"
}
req := fmt.Sprintf(
"ALTER TABLE %v ON CLUSTER %s MODIFY TTL toDateTime(toUInt32(%s / 1000), 'UTC') + "+
"INTERVAL %v SECOND DELETE", tableName, r.cluster, timeColumn, params.DelDuration)
if len(params.ColdStorageVolume) > 0 {
req += fmt.Sprintf(", toDateTime(toUInt32(%s / 1000), 'UTC')"+
" + INTERVAL %v SECOND TO VOLUME '%s'",
timeColumn, params.ToColdStorageDuration, params.ColdStorageVolume)
}
err := r.setColdStorage(context.Background(), tableName, params.ColdStorageVolume)
if err != nil {
zap.L().Error("Error in setting cold storage", zap.Error(err))
statusItem, err := r.checkTTLStatusItem(ctx, orgID, tableName)
if err == nil {
_, dbErr := r.
sqlDB.
BunDB().
NewUpdate().
Model(new(types.TTLSetting)).
Set("updated_at = ?", time.Now()).
Set("status = ?", constants.StatusFailed).
Where("id = ?", statusItem.ID.StringValue()).
Exec(ctx)
if dbErr != nil {
zap.L().Error("Error in processing ttl_status update sql query", zap.Error(dbErr))
return
}
}
return
}
req += " SETTINGS materialize_ttl_after_modify=0"
zap.L().Info("Executing TTL request: ", zap.String("request", req))
statusItem, _ := r.checkTTLStatusItem(ctx, orgID, tableName)
if err := r.db.Exec(ctx, req); err != nil {
zap.L().Error("error while setting ttl.", zap.Error(err))
_, dbErr := r.
sqlDB.
BunDB().
NewUpdate().
Model(new(types.TTLSetting)).
Set("updated_at = ?", time.Now()).
Set("status = ?", constants.StatusFailed).
Where("id = ?", statusItem.ID.StringValue()).
Exec(ctx)
if dbErr != nil {
zap.L().Error("Error in processing ttl_status update sql query", zap.Error(dbErr))
return
}
return
}
_, dbErr = r.
sqlDB.
BunDB().
NewUpdate().
Model(new(types.TTLSetting)).
Set("updated_at = ?", time.Now()).
Set("status = ?", constants.StatusSuccess).
Where("id = ?", statusItem.ID.StringValue()).
Exec(ctx)
if dbErr != nil {
zap.L().Error("Error in processing ttl_status update sql query", zap.Error(dbErr))
return
}
}
for _, tableName := range tableNames {
go metricTTL(tableName)
}
case constants.LogsTTL:
return r.setTTLLogs(ctx, orgID, params)
default:
return nil, &model.ApiError{Typ: model.ErrorExec, Err: fmt.Errorf("error while setting ttl. ttl type should be <metrics|traces>, got %v", params.Type)}
}
return &model.SetTTLResponseItem{Message: "move ttl has been successfully set up"}, nil
}
func (r *ClickHouseReader) deleteTtlTransactions(ctx context.Context, orgID string, numberOfTransactionsStore int) {
limitTransactions := []string{}
err := r.
sqlDB.
BunDB().
NewSelect().
ColumnExpr("distinct(transaction_id)").
Model(new(types.TTLSetting)).
Where("org_id = ?", orgID).
OrderExpr("created_at DESC").
Limit(numberOfTransactionsStore).
Scan(ctx, &limitTransactions)
if err != nil {
zap.L().Error("Error in processing ttl_status delete sql query", zap.Error(err))
}
_, err = r.
sqlDB.
BunDB().
NewDelete().
Model(new(types.TTLSetting)).
Where("transaction_id NOT IN (?)", bun.In(limitTransactions)).
Exec(ctx)
if err != nil {
zap.L().Error("Error in processing ttl_status delete sql query", zap.Error(err))
}
}
// checkTTLStatusItem checks if ttl_status table has an entry for the given table name
func (r *ClickHouseReader) checkTTLStatusItem(ctx context.Context, orgID string, tableName string) (*types.TTLSetting, *model.ApiError) {
zap.L().Info("checkTTLStatusItem query", zap.String("tableName", tableName))
ttl := new(types.TTLSetting)
err := r.
sqlDB.
BunDB().
NewSelect().
Model(ttl).
Where("table_name = ?", tableName).
Where("org_id = ?", orgID).
OrderExpr("created_at DESC").
Limit(1).
Scan(ctx)
if err != nil && err != sql.ErrNoRows {
zap.L().Error("Error in processing sql query", zap.Error(err))
return ttl, &model.ApiError{Typ: model.ErrorExec, Err: fmt.Errorf("error in processing ttl_status check sql query")}
}
return ttl, nil
}
// setTTLQueryStatus fetches ttl_status table status from DB
func (r *ClickHouseReader) setTTLQueryStatus(ctx context.Context, orgID string, tableNameArray []string) (string, *model.ApiError) {
failFlag := false
status := constants.StatusSuccess
for _, tableName := range tableNameArray {
statusItem, err := r.checkTTLStatusItem(ctx, orgID, tableName)
emptyStatusStruct := new(types.TTLSetting)
if statusItem == emptyStatusStruct {
return "", nil
}
if err != nil {
return "", &model.ApiError{Typ: model.ErrorExec, Err: fmt.Errorf("error in processing ttl_status check sql query")}
}
if statusItem.Status == constants.StatusPending && statusItem.UpdatedAt.Unix()-time.Now().Unix() < 3600 {
status = constants.StatusPending
return status, nil
}
if statusItem.Status == constants.StatusFailed {
failFlag = true
}
}
if failFlag {
status = constants.StatusFailed
}
return status, nil
}
func (r *ClickHouseReader) setColdStorage(ctx context.Context, tableName string, coldStorageVolume string) *model.ApiError {
// Set the storage policy for the required table. If it is already set, then setting it again
// will not a problem.
if len(coldStorageVolume) > 0 {
policyReq := fmt.Sprintf("ALTER TABLE %s ON CLUSTER %s MODIFY SETTING storage_policy='tiered'", tableName, r.cluster)
zap.L().Info("Executing Storage policy request: ", zap.String("request", policyReq))
if err := r.db.Exec(ctx, policyReq); err != nil {
zap.L().Error("error while setting storage policy", zap.Error(err))
return &model.ApiError{Typ: model.ErrorExec, Err: fmt.Errorf("error while setting storage policy. Err=%v", err)}
}
}
return nil
}
// GetDisks returns a list of disks {name, type} configured in clickhouse DB.
func (r *ClickHouseReader) GetDisks(ctx context.Context) (*[]model.DiskItem, *model.ApiError) {
diskItems := []model.DiskItem{}
query := "SELECT name,type FROM system.disks"
if err := r.db.Select(ctx, &diskItems, query); err != nil {
zap.L().Error("Error in processing sql query", zap.Error(err))
return nil, &model.ApiError{Typ: model.ErrorExec, Err: fmt.Errorf("error while getting disks. Err=%v", err)}
}
return &diskItems, nil
}
func getLocalTableNameArray(tableNames []string) []string {
var localTableNames []string
for _, name := range tableNames {
tableNameSplit := strings.Split(name, ".")
localTableNames = append(localTableNames, tableNameSplit[0]+"."+strings.Split(tableNameSplit[1], "distributed_")[1])
}
return localTableNames
}
// GetTTL returns current ttl, expected ttl and past setTTL status for metrics/traces.
func (r *ClickHouseReader) GetTTL(ctx context.Context, orgID string, ttlParams *model.GetTTLParams) (*model.GetTTLResponseItem, *model.ApiError) {
parseTTL := func(queryResp string) (int, int) {
zap.L().Info("Parsing TTL from: ", zap.String("queryResp", queryResp))
deleteTTLExp := regexp.MustCompile(`toIntervalSecond\(([0-9]*)\)`)
moveTTLExp := regexp.MustCompile(`toIntervalSecond\(([0-9]*)\) TO VOLUME`)
var delTTL, moveTTL int = -1, -1
m := deleteTTLExp.FindStringSubmatch(queryResp)
if len(m) > 1 {
seconds_int, err := strconv.Atoi(m[1])
if err != nil {
return -1, -1
}
delTTL = seconds_int / 3600
}
m = moveTTLExp.FindStringSubmatch(queryResp)
if len(m) > 1 {
seconds_int, err := strconv.Atoi(m[1])
if err != nil {
return -1, -1
}
moveTTL = seconds_int / 3600
}
return delTTL, moveTTL
}
getMetricsTTL := func() (*model.DBResponseTTL, *model.ApiError) {
var dbResp []model.DBResponseTTL
query := fmt.Sprintf("SELECT engine_full FROM system.tables WHERE name='%v'", signozSampleLocalTableName)
err := r.db.Select(ctx, &dbResp, query)
if err != nil {
zap.L().Error("error while getting ttl", zap.Error(err))
return nil, &model.ApiError{Typ: model.ErrorExec, Err: fmt.Errorf("error while getting ttl. Err=%v", err)}
}
if len(dbResp) == 0 {
return nil, nil
} else {
return &dbResp[0], nil
}
}
getTracesTTL := func() (*model.DBResponseTTL, *model.ApiError) {
var dbResp []model.DBResponseTTL
query := fmt.Sprintf("SELECT engine_full FROM system.tables WHERE name='%v' AND database='%v'", r.traceLocalTableName, signozTraceDBName)
err := r.db.Select(ctx, &dbResp, query)
if err != nil {
zap.L().Error("error while getting ttl", zap.Error(err))
return nil, &model.ApiError{Typ: model.ErrorExec, Err: fmt.Errorf("error while getting ttl. Err=%v", err)}
}
if len(dbResp) == 0 {
return nil, nil
} else {
return &dbResp[0], nil
}
}
getLogsTTL := func() (*model.DBResponseTTL, *model.ApiError) {
var dbResp []model.DBResponseTTL
query := fmt.Sprintf("SELECT engine_full FROM system.tables WHERE name='%v' AND database='%v'", r.logsLocalTableName, r.logsDB)
err := r.db.Select(ctx, &dbResp, query)
if err != nil {
zap.L().Error("error while getting ttl", zap.Error(err))
return nil, &model.ApiError{Typ: model.ErrorExec, Err: fmt.Errorf("error while getting ttl. Err=%v", err)}
}
if len(dbResp) == 0 {
return nil, nil
} else {
return &dbResp[0], nil
}
}
switch ttlParams.Type {
case constants.TraceTTL:
tableNameArray := []string{signozTraceDBName + "." + signozTraceTableName, signozTraceDBName + "." + signozDurationMVTable, signozTraceDBName + "." + signozSpansTable, signozTraceDBName + "." + signozErrorIndexTable, signozTraceDBName + "." + signozUsageExplorerTable, signozTraceDBName + "." + defaultDependencyGraphTable}
tableNameArray = getLocalTableNameArray(tableNameArray)
status, err := r.setTTLQueryStatus(ctx, orgID, tableNameArray)
if err != nil {
return nil, err
}
dbResp, err := getTracesTTL()
if err != nil {
return nil, err
}
ttlQuery, err := r.checkTTLStatusItem(ctx, orgID, tableNameArray[0])
if err != nil {
return nil, err
}
ttlQuery.TTL = ttlQuery.TTL / 3600 // convert to hours
if ttlQuery.ColdStorageTTL != -1 {
ttlQuery.ColdStorageTTL = ttlQuery.ColdStorageTTL / 3600 // convert to hours
}
delTTL, moveTTL := parseTTL(dbResp.EngineFull)
return &model.GetTTLResponseItem{TracesTime: delTTL, TracesMoveTime: moveTTL, ExpectedTracesTime: ttlQuery.TTL, ExpectedTracesMoveTime: ttlQuery.ColdStorageTTL, Status: status}, nil
case constants.MetricsTTL:
tableNameArray := []string{signozMetricDBName + "." + signozSampleTableName}
tableNameArray = getLocalTableNameArray(tableNameArray)
status, err := r.setTTLQueryStatus(ctx, orgID, tableNameArray)
if err != nil {
return nil, err
}
dbResp, err := getMetricsTTL()
if err != nil {
return nil, err
}
ttlQuery, err := r.checkTTLStatusItem(ctx, orgID, tableNameArray[0])
if err != nil {
return nil, err
}
ttlQuery.TTL = ttlQuery.TTL / 3600 // convert to hours
if ttlQuery.ColdStorageTTL != -1 {
ttlQuery.ColdStorageTTL = ttlQuery.ColdStorageTTL / 3600 // convert to hours
}
delTTL, moveTTL := parseTTL(dbResp.EngineFull)
return &model.GetTTLResponseItem{MetricsTime: delTTL, MetricsMoveTime: moveTTL, ExpectedMetricsTime: ttlQuery.TTL, ExpectedMetricsMoveTime: ttlQuery.ColdStorageTTL, Status: status}, nil
case constants.LogsTTL:
tableNameArray := []string{r.logsDB + "." + r.logsTableName}
tableNameArray = getLocalTableNameArray(tableNameArray)
status, err := r.setTTLQueryStatus(ctx, orgID, tableNameArray)
if err != nil {
return nil, err
}
dbResp, err := getLogsTTL()
if err != nil {
return nil, err
}
ttlQuery, err := r.checkTTLStatusItem(ctx, orgID, tableNameArray[0])
if err != nil {
return nil, err
}
ttlQuery.TTL = ttlQuery.TTL / 3600 // convert to hours
if ttlQuery.ColdStorageTTL != -1 {
ttlQuery.ColdStorageTTL = ttlQuery.ColdStorageTTL / 3600 // convert to hours
}
delTTL, moveTTL := parseTTL(dbResp.EngineFull)
return &model.GetTTLResponseItem{LogsTime: delTTL, LogsMoveTime: moveTTL, ExpectedLogsTime: ttlQuery.TTL, ExpectedLogsMoveTime: ttlQuery.ColdStorageTTL, Status: status}, nil
default:
return nil, &model.ApiError{Typ: model.ErrorExec, Err: fmt.Errorf("error while getting ttl. ttl type should be metrics|traces, got %v",
ttlParams.Type)}
}
}
func (r *ClickHouseReader) ListErrors(ctx context.Context, queryParams *model.ListErrorsParams) (*[]model.Error, *model.ApiError) {
var getErrorResponses []model.Error
query := "SELECT any(exceptionMessage) as exceptionMessage, count() AS exceptionCount, min(timestamp) as firstSeen, max(timestamp) as lastSeen, groupID"
if len(queryParams.ServiceName) != 0 {
query = query + ", serviceName"
} else {
query = query + ", any(serviceName) as serviceName"
}
if len(queryParams.ExceptionType) != 0 {
query = query + ", exceptionType"
} else {
query = query + ", any(exceptionType) as exceptionType"
}
query += fmt.Sprintf(" FROM %s.%s WHERE timestamp >= @timestampL AND timestamp <= @timestampU", r.TraceDB, r.errorTable)
args := []interface{}{clickhouse.Named("timestampL", strconv.FormatInt(queryParams.Start.UnixNano(), 10)), clickhouse.Named("timestampU", strconv.FormatInt(queryParams.End.UnixNano(), 10))}
if len(queryParams.ServiceName) != 0 {
query = query + " AND serviceName ilike @serviceName"
args = append(args, clickhouse.Named("serviceName", "%"+queryParams.ServiceName+"%"))
}
if len(queryParams.ExceptionType) != 0 {
query = query + " AND exceptionType ilike @exceptionType"
args = append(args, clickhouse.Named("exceptionType", "%"+queryParams.ExceptionType+"%"))
}
// create TagQuery from TagQueryParams
tags := createTagQueryFromTagQueryParams(queryParams.Tags)
subQuery, argsSubQuery, errStatus := buildQueryWithTagParams(ctx, tags)
query += subQuery
args = append(args, argsSubQuery...)
if errStatus != nil {
zap.L().Error("Error in processing tags", zap.Error(errStatus))
return nil, errStatus
}
query = query + " GROUP BY groupID"
if len(queryParams.ServiceName) != 0 {
query = query + ", serviceName"
}
if len(queryParams.ExceptionType) != 0 {
query = query + ", exceptionType"
}
if len(queryParams.OrderParam) != 0 {
if queryParams.Order == constants.Descending {
query = query + " ORDER BY " + queryParams.OrderParam + " DESC"
} else if queryParams.Order == constants.Ascending {
query = query + " ORDER BY " + queryParams.OrderParam + " ASC"
}
}
if queryParams.Limit > 0 {
query = query + " LIMIT @limit"
args = append(args, clickhouse.Named("limit", queryParams.Limit))
}
if queryParams.Offset > 0 {
query = query + " OFFSET @offset"
args = append(args, clickhouse.Named("offset", queryParams.Offset))
}
err := r.db.Select(ctx, &getErrorResponses, query, args...)
zap.L().Info(query)
if err != nil {
zap.L().Error("Error in processing sql query", zap.Error(err))
return nil, &model.ApiError{Typ: model.ErrorExec, Err: fmt.Errorf("error in processing sql query")}
}
return &getErrorResponses, nil
}
func (r *ClickHouseReader) CountErrors(ctx context.Context, queryParams *model.CountErrorsParams) (uint64, *model.ApiError) {
var errorCount uint64
query := fmt.Sprintf("SELECT count(distinct(groupID)) FROM %s.%s WHERE timestamp >= @timestampL AND timestamp <= @timestampU", r.TraceDB, r.errorTable)
args := []interface{}{clickhouse.Named("timestampL", strconv.FormatInt(queryParams.Start.UnixNano(), 10)), clickhouse.Named("timestampU", strconv.FormatInt(queryParams.End.UnixNano(), 10))}
if len(queryParams.ServiceName) != 0 {
query = query + " AND serviceName ilike @serviceName"
args = append(args, clickhouse.Named("serviceName", "%"+queryParams.ServiceName+"%"))
}
if len(queryParams.ExceptionType) != 0 {
query = query + " AND exceptionType ilike @exceptionType"
args = append(args, clickhouse.Named("exceptionType", "%"+queryParams.ExceptionType+"%"))
}
// create TagQuery from TagQueryParams
tags := createTagQueryFromTagQueryParams(queryParams.Tags)
subQuery, argsSubQuery, errStatus := buildQueryWithTagParams(ctx, tags)
query += subQuery
args = append(args, argsSubQuery...)
if errStatus != nil {
zap.L().Error("Error in processing tags", zap.Error(errStatus))
return 0, errStatus
}
err := r.db.QueryRow(ctx, query, args...).Scan(&errorCount)
zap.L().Info(query)
if err != nil {
zap.L().Error("Error in processing sql query", zap.Error(err))
return 0, &model.ApiError{Typ: model.ErrorExec, Err: fmt.Errorf("error in processing sql query")}
}
return errorCount, nil
}
func (r *ClickHouseReader) GetErrorFromErrorID(ctx context.Context, queryParams *model.GetErrorParams) (*model.ErrorWithSpan, *model.ApiError) {
if queryParams.ErrorID == "" {
zap.L().Error("errorId missing from params")
return nil, &model.ApiError{Typ: model.ErrorBadData, Err: fmt.Errorf("ErrorID missing from params")}
}
var getErrorWithSpanReponse []model.ErrorWithSpan
query := fmt.Sprintf("SELECT errorID, exceptionType, exceptionStacktrace, exceptionEscaped, exceptionMessage, timestamp, spanID, traceID, serviceName, groupID FROM %s.%s WHERE timestamp = @timestamp AND groupID = @groupID AND errorID = @errorID LIMIT 1", r.TraceDB, r.errorTable)
args := []interface{}{clickhouse.Named("errorID", queryParams.ErrorID), clickhouse.Named("groupID", queryParams.GroupID), clickhouse.Named("timestamp", strconv.FormatInt(queryParams.Timestamp.UnixNano(), 10))}
err := r.db.Select(ctx, &getErrorWithSpanReponse, query, args...)
zap.L().Info(query)
if err != nil {
zap.L().Error("Error in processing sql query", zap.Error(err))
return nil, &model.ApiError{Typ: model.ErrorExec, Err: fmt.Errorf("error in processing sql query")}
}
if len(getErrorWithSpanReponse) > 0 {
return &getErrorWithSpanReponse[0], nil
} else {
return nil, &model.ApiError{Typ: model.ErrorNotFound, Err: fmt.Errorf("Error/Exception not found")}
}
}
func (r *ClickHouseReader) GetErrorFromGroupID(ctx context.Context, queryParams *model.GetErrorParams) (*model.ErrorWithSpan, *model.ApiError) {
var getErrorWithSpanReponse []model.ErrorWithSpan
query := fmt.Sprintf("SELECT errorID, exceptionType, exceptionStacktrace, exceptionEscaped, exceptionMessage, timestamp, spanID, traceID, serviceName, groupID FROM %s.%s WHERE timestamp = @timestamp AND groupID = @groupID LIMIT 1", r.TraceDB, r.errorTable)
args := []interface{}{clickhouse.Named("groupID", queryParams.GroupID), clickhouse.Named("timestamp", strconv.FormatInt(queryParams.Timestamp.UnixNano(), 10))}
err := r.db.Select(ctx, &getErrorWithSpanReponse, query, args...)
zap.L().Info(query)
if err != nil {
zap.L().Error("Error in processing sql query", zap.Error(err))
return nil, &model.ApiError{Typ: model.ErrorExec, Err: fmt.Errorf("error in processing sql query")}
}
if len(getErrorWithSpanReponse) > 0 {
return &getErrorWithSpanReponse[0], nil
} else {
return nil, &model.ApiError{Typ: model.ErrorNotFound, Err: fmt.Errorf("Error/Exception not found")}
}
}
func (r *ClickHouseReader) GetNextPrevErrorIDs(ctx context.Context, queryParams *model.GetErrorParams) (*model.NextPrevErrorIDs, *model.ApiError) {
if queryParams.ErrorID == "" {
zap.L().Error("errorId missing from params")
return nil, &model.ApiError{Typ: model.ErrorBadData, Err: fmt.Errorf("ErrorID missing from params")}
}
var err *model.ApiError
getNextPrevErrorIDsResponse := model.NextPrevErrorIDs{
GroupID: queryParams.GroupID,
}
getNextPrevErrorIDsResponse.NextErrorID, getNextPrevErrorIDsResponse.NextTimestamp, err = r.getNextErrorID(ctx, queryParams)
if err != nil {
zap.L().Error("Unable to get next error ID due to err: ", zap.Error(err))
return nil, err
}
getNextPrevErrorIDsResponse.PrevErrorID, getNextPrevErrorIDsResponse.PrevTimestamp, err = r.getPrevErrorID(ctx, queryParams)
if err != nil {
zap.L().Error("Unable to get prev error ID due to err: ", zap.Error(err))
return nil, err
}
return &getNextPrevErrorIDsResponse, nil
}
func (r *ClickHouseReader) getNextErrorID(ctx context.Context, queryParams *model.GetErrorParams) (string, time.Time, *model.ApiError) {
var getNextErrorIDReponse []model.NextPrevErrorIDsDBResponse
query := fmt.Sprintf("SELECT errorID as nextErrorID, timestamp as nextTimestamp FROM %s.%s WHERE groupID = @groupID AND timestamp >= @timestamp AND errorID != @errorID ORDER BY timestamp ASC LIMIT 2", r.TraceDB, r.errorTable)
args := []interface{}{clickhouse.Named("errorID", queryParams.ErrorID), clickhouse.Named("groupID", queryParams.GroupID), clickhouse.Named("timestamp", strconv.FormatInt(queryParams.Timestamp.UnixNano(), 10))}
err := r.db.Select(ctx, &getNextErrorIDReponse, query, args...)
zap.L().Info(query)
if err != nil {
zap.L().Error("Error in processing sql query", zap.Error(err))
return "", time.Time{}, &model.ApiError{Typ: model.ErrorExec, Err: fmt.Errorf("error in processing sql query")}
}
if len(getNextErrorIDReponse) == 0 {
zap.L().Info("NextErrorID not found")
return "", time.Time{}, nil
} else if len(getNextErrorIDReponse) == 1 {
zap.L().Info("NextErrorID found")
return getNextErrorIDReponse[0].NextErrorID, getNextErrorIDReponse[0].NextTimestamp, nil
} else {
if getNextErrorIDReponse[0].Timestamp.UnixNano() == getNextErrorIDReponse[1].Timestamp.UnixNano() {
var getNextErrorIDReponse []model.NextPrevErrorIDsDBResponse
query := fmt.Sprintf("SELECT errorID as nextErrorID, timestamp as nextTimestamp FROM %s.%s WHERE groupID = @groupID AND timestamp = @timestamp AND errorID > @errorID ORDER BY errorID ASC LIMIT 1", r.TraceDB, r.errorTable)
args := []interface{}{clickhouse.Named("errorID", queryParams.ErrorID), clickhouse.Named("groupID", queryParams.GroupID), clickhouse.Named("timestamp", strconv.FormatInt(queryParams.Timestamp.UnixNano(), 10))}
err := r.db.Select(ctx, &getNextErrorIDReponse, query, args...)
zap.L().Info(query)
if err != nil {
zap.L().Error("Error in processing sql query", zap.Error(err))
return "", time.Time{}, &model.ApiError{Typ: model.ErrorExec, Err: fmt.Errorf("error in processing sql query")}
}
if len(getNextErrorIDReponse) == 0 {
var getNextErrorIDReponse []model.NextPrevErrorIDsDBResponse
query := fmt.Sprintf("SELECT errorID as nextErrorID, timestamp as nextTimestamp FROM %s.%s WHERE groupID = @groupID AND timestamp > @timestamp ORDER BY timestamp ASC LIMIT 1", r.TraceDB, r.errorTable)
args := []interface{}{clickhouse.Named("errorID", queryParams.ErrorID), clickhouse.Named("groupID", queryParams.GroupID), clickhouse.Named("timestamp", strconv.FormatInt(queryParams.Timestamp.UnixNano(), 10))}
err := r.db.Select(ctx, &getNextErrorIDReponse, query, args...)
zap.L().Info(query)
if err != nil {
zap.L().Error("Error in processing sql query", zap.Error(err))
return "", time.Time{}, &model.ApiError{Typ: model.ErrorExec, Err: fmt.Errorf("error in processing sql query")}
}
if len(getNextErrorIDReponse) == 0 {
zap.L().Info("NextErrorID not found")
return "", time.Time{}, nil
} else {
zap.L().Info("NextErrorID found")
return getNextErrorIDReponse[0].NextErrorID, getNextErrorIDReponse[0].NextTimestamp, nil
}
} else {
zap.L().Info("NextErrorID found")
return getNextErrorIDReponse[0].NextErrorID, getNextErrorIDReponse[0].NextTimestamp, nil
}
} else {
zap.L().Info("NextErrorID found")
return getNextErrorIDReponse[0].NextErrorID, getNextErrorIDReponse[0].NextTimestamp, nil
}
}
}
func (r *ClickHouseReader) getPrevErrorID(ctx context.Context, queryParams *model.GetErrorParams) (string, time.Time, *model.ApiError) {
var getPrevErrorIDReponse []model.NextPrevErrorIDsDBResponse
query := fmt.Sprintf("SELECT errorID as prevErrorID, timestamp as prevTimestamp FROM %s.%s WHERE groupID = @groupID AND timestamp <= @timestamp AND errorID != @errorID ORDER BY timestamp DESC LIMIT 2", r.TraceDB, r.errorTable)
args := []interface{}{clickhouse.Named("errorID", queryParams.ErrorID), clickhouse.Named("groupID", queryParams.GroupID), clickhouse.Named("timestamp", strconv.FormatInt(queryParams.Timestamp.UnixNano(), 10))}
err := r.db.Select(ctx, &getPrevErrorIDReponse, query, args...)
zap.L().Info(query)
if err != nil {
zap.L().Error("Error in processing sql query", zap.Error(err))
return "", time.Time{}, &model.ApiError{Typ: model.ErrorExec, Err: fmt.Errorf("error in processing sql query")}
}
if len(getPrevErrorIDReponse) == 0 {
zap.L().Info("PrevErrorID not found")
return "", time.Time{}, nil
} else if len(getPrevErrorIDReponse) == 1 {
zap.L().Info("PrevErrorID found")
return getPrevErrorIDReponse[0].PrevErrorID, getPrevErrorIDReponse[0].PrevTimestamp, nil
} else {
if getPrevErrorIDReponse[0].Timestamp.UnixNano() == getPrevErrorIDReponse[1].Timestamp.UnixNano() {
var getPrevErrorIDReponse []model.NextPrevErrorIDsDBResponse
query := fmt.Sprintf("SELECT errorID as prevErrorID, timestamp as prevTimestamp FROM %s.%s WHERE groupID = @groupID AND timestamp = @timestamp AND errorID < @errorID ORDER BY errorID DESC LIMIT 1", r.TraceDB, r.errorTable)
args := []interface{}{clickhouse.Named("errorID", queryParams.ErrorID), clickhouse.Named("groupID", queryParams.GroupID), clickhouse.Named("timestamp", strconv.FormatInt(queryParams.Timestamp.UnixNano(), 10))}
err := r.db.Select(ctx, &getPrevErrorIDReponse, query, args...)
zap.L().Info(query)
if err != nil {
zap.L().Error("Error in processing sql query", zap.Error(err))
return "", time.Time{}, &model.ApiError{Typ: model.ErrorExec, Err: fmt.Errorf("error in processing sql query")}
}
if len(getPrevErrorIDReponse) == 0 {
var getPrevErrorIDReponse []model.NextPrevErrorIDsDBResponse
query := fmt.Sprintf("SELECT errorID as prevErrorID, timestamp as prevTimestamp FROM %s.%s WHERE groupID = @groupID AND timestamp < @timestamp ORDER BY timestamp DESC LIMIT 1", r.TraceDB, r.errorTable)
args := []interface{}{clickhouse.Named("errorID", queryParams.ErrorID), clickhouse.Named("groupID", queryParams.GroupID), clickhouse.Named("timestamp", strconv.FormatInt(queryParams.Timestamp.UnixNano(), 10))}
err := r.db.Select(ctx, &getPrevErrorIDReponse, query, args...)
zap.L().Info(query)
if err != nil {
zap.L().Error("Error in processing sql query", zap.Error(err))
return "", time.Time{}, &model.ApiError{Typ: model.ErrorExec, Err: fmt.Errorf("error in processing sql query")}
}
if len(getPrevErrorIDReponse) == 0 {
zap.L().Info("PrevErrorID not found")
return "", time.Time{}, nil
} else {
zap.L().Info("PrevErrorID found")
return getPrevErrorIDReponse[0].PrevErrorID, getPrevErrorIDReponse[0].PrevTimestamp, nil
}
} else {
zap.L().Info("PrevErrorID found")
return getPrevErrorIDReponse[0].PrevErrorID, getPrevErrorIDReponse[0].PrevTimestamp, nil
}
} else {
zap.L().Info("PrevErrorID found")
return getPrevErrorIDReponse[0].PrevErrorID, getPrevErrorIDReponse[0].PrevTimestamp, nil
}
}
}
func (r *ClickHouseReader) GetTotalSpans(ctx context.Context) (uint64, error) {
var totalSpans uint64
queryStr := fmt.Sprintf("SELECT count() from %s.%s;", signozTraceDBName, r.traceTableName)
r.db.QueryRow(ctx, queryStr).Scan(&totalSpans)
return totalSpans, nil
}
func (r *ClickHouseReader) GetSpansInLastHeartBeatInterval(ctx context.Context, interval time.Duration) (uint64, error) {
var spansInLastHeartBeatInterval uint64
r.db.QueryRow(ctx, fmt.Sprintf("SELECT count() from %s.%s where ts_bucket_start >= toUInt64(toUnixTimestamp(now() - toIntervalMinute(%d))) - 1800 and timestamp > toUnixTimestamp(now()-toIntervalMinute(%d));", signozTraceDBName, r.traceTableName, int(interval.Minutes()), int(interval.Minutes()))).Scan(&spansInLastHeartBeatInterval)
return spansInLastHeartBeatInterval, nil
}
func (r *ClickHouseReader) GetTotalLogs(ctx context.Context) (uint64, error) {
var totalLogs uint64
queryStr := fmt.Sprintf("SELECT count() from %s.%s;", r.logsDB, r.logsTableName)
r.db.QueryRow(ctx, queryStr).Scan(&totalLogs)
return totalLogs, nil
}
func (r *ClickHouseReader) FetchTemporality(ctx context.Context, orgID valuer.UUID, metricNames []string) (map[string]map[v3.Temporality]bool, error) {
metricNameToTemporality := make(map[string]map[v3.Temporality]bool)
var metricNamesToQuery []string
for _, metricName := range metricNames {
updatedMetadata, cacheErr := r.GetUpdatedMetricsMetadata(ctx, orgID, metricName)
if cacheErr != nil {
zap.L().Info("Error in getting metrics cached metadata", zap.Error(cacheErr))
}
if metadata, exist := updatedMetadata[metricName]; exist {
if _, exists := metricNameToTemporality[metricName]; !exists {
metricNameToTemporality[metricName] = make(map[v3.Temporality]bool)
}
metricNameToTemporality[metricName][metadata.Temporality] = true
} else {
metricNamesToQuery = append(metricNamesToQuery, metricName)
}
}
query := fmt.Sprintf(`SELECT DISTINCT metric_name, temporality FROM %s.%s WHERE metric_name IN $1`, signozMetricDBName, signozTSTableNameV41Day)
rows, err := r.db.Query(ctx, query, metricNames)
if err != nil {
return nil, err
}
defer rows.Close()
for rows.Next() {
var metricName, temporality string
err := rows.Scan(&metricName, &temporality)
if err != nil {
return nil, err
}
if _, ok := metricNameToTemporality[metricName]; !ok {
metricNameToTemporality[metricName] = make(map[v3.Temporality]bool)
}
metricNameToTemporality[metricName][v3.Temporality(temporality)] = true
}
return metricNameToTemporality, nil
}
func (r *ClickHouseReader) GetTimeSeriesInfo(ctx context.Context) (map[string]interface{}, error) {
queryStr := fmt.Sprintf("SELECT countDistinct(fingerprint) as count from %s.%s where metric_name not like 'signoz_%%' group by metric_name order by count desc;", signozMetricDBName, signozTSTableNameV41Day)
rows, _ := r.db.Query(ctx, queryStr)
var totalTS uint64
totalTS = 0
var maxTS uint64
maxTS = 0
count := 0
for rows.Next() {
var value uint64
rows.Scan(&value)
totalTS += value
if count == 0 {
maxTS = value
}
count += 1
}
timeSeriesData := map[string]interface{}{}
timeSeriesData["totalTS"] = totalTS
timeSeriesData["maxTS"] = maxTS
return timeSeriesData, nil
}
func (r *ClickHouseReader) GetSamplesInfoInLastHeartBeatInterval(ctx context.Context, interval time.Duration) (uint64, error) {
var totalSamples uint64
queryStr := fmt.Sprintf("select count() from %s.%s where metric_name not like 'signoz_%%' and unix_milli > toUnixTimestamp(now()-toIntervalMinute(%d))*1000;", signozMetricDBName, signozSampleTableName, int(interval.Minutes()))
r.db.QueryRow(ctx, queryStr).Scan(&totalSamples)
return totalSamples, nil
}
func (r *ClickHouseReader) GetTotalSamples(ctx context.Context) (uint64, error) {
var totalSamples uint64
queryStr := fmt.Sprintf("select count() from %s.%s where metric_name not like 'signoz_%%';", signozMetricDBName, signozSampleTableName)
r.db.QueryRow(ctx, queryStr).Scan(&totalSamples)
return totalSamples, nil
}
func (r *ClickHouseReader) GetDistributedInfoInLastHeartBeatInterval(ctx context.Context) (map[string]interface{}, error) {
clusterInfo := []model.ClusterInfo{}
queryStr := `SELECT shard_num, shard_weight, replica_num, errors_count, slowdowns_count, estimated_recovery_time FROM system.clusters where cluster='cluster';`
r.db.Select(ctx, &clusterInfo, queryStr)
if len(clusterInfo) == 1 {
return clusterInfo[0].GetMapFromStruct(), nil
}
return nil, nil
}
func (r *ClickHouseReader) GetLogsInfoInLastHeartBeatInterval(ctx context.Context, interval time.Duration) (uint64, error) {
var totalLogLines uint64
queryStr := fmt.Sprintf("select count() from %s.%s where timestamp > toUnixTimestamp(now()-toIntervalMinute(%d))*1000000000;", r.logsDB, r.logsTableV2, int(interval.Minutes()))
err := r.db.QueryRow(ctx, queryStr).Scan(&totalLogLines)
return totalLogLines, err
}
func (r *ClickHouseReader) GetTagsInfoInLastHeartBeatInterval(ctx context.Context, interval time.Duration) (*model.TagsInfo, error) {
queryStr := fmt.Sprintf(`select serviceName, resources_string['deployment.environment'] as env,
resources_string['telemetry.sdk.language'] as language from %s.%s
where timestamp > toUnixTimestamp(now()-toIntervalMinute(%d))
group by serviceName, env, language;`, r.TraceDB, r.traceTableName, int(interval.Minutes()))
tagTelemetryDataList := []model.TagTelemetryData{}
err := r.db.Select(ctx, &tagTelemetryDataList, queryStr)
if err != nil {
zap.L().Error("Error in processing sql query: ", zap.Error(err))
return nil, err
}
tagsInfo := model.TagsInfo{
Languages: make(map[string]interface{}),
Services: make(map[string]interface{}),
}
for _, tagTelemetryData := range tagTelemetryDataList {
if len(tagTelemetryData.ServiceName) != 0 && strings.Contains(tagTelemetryData.ServiceName, "prod") {
tagsInfo.Env = tagTelemetryData.ServiceName
}
if len(tagTelemetryData.Env) != 0 && strings.Contains(tagTelemetryData.Env, "prod") {
tagsInfo.Env = tagTelemetryData.Env
}
if len(tagTelemetryData.Language) != 0 {
tagsInfo.Languages[tagTelemetryData.Language] = struct{}{}
}
if len(tagTelemetryData.ServiceName) != 0 {
tagsInfo.Services[tagTelemetryData.ServiceName] = struct{}{}
}
}
return &tagsInfo, nil
}
// remove this after sometime
func removeUnderscoreDuplicateFields(fields []model.Field) []model.Field {
lookup := map[string]model.Field{}
for _, v := range fields {
lookup[v.Name+v.DataType] = v
}
for k := range lookup {
if strings.Contains(k, ".") {
delete(lookup, strings.ReplaceAll(k, ".", "_"))
}
}
updatedFields := []model.Field{}
for _, v := range lookup {
updatedFields = append(updatedFields, v)
}
return updatedFields
}
func (r *ClickHouseReader) GetLogFields(ctx context.Context) (*model.GetFieldsResponse, *model.ApiError) {
// response will contain top level fields from the otel log model
response := model.GetFieldsResponse{
Selected: constants.StaticSelectedLogFields,
Interesting: []model.Field{},
}
// get attribute keys
attributes := []model.Field{}
query := fmt.Sprintf("SELECT DISTINCT name, datatype from %s.%s group by name, datatype", r.logsDB, r.logsAttributeKeys)
err := r.db.Select(ctx, &attributes, query)
if err != nil {
return nil, &model.ApiError{Err: err, Typ: model.ErrorInternal}
}
// get resource keys
resources := []model.Field{}
query = fmt.Sprintf("SELECT DISTINCT name, datatype from %s.%s group by name, datatype", r.logsDB, r.logsResourceKeys)
err = r.db.Select(ctx, &resources, query)
if err != nil {
return nil, &model.ApiError{Err: err, Typ: model.ErrorInternal}
}
//remove this code after sometime
attributes = removeUnderscoreDuplicateFields(attributes)
resources = removeUnderscoreDuplicateFields(resources)
statements := []model.ShowCreateTableStatement{}
query = fmt.Sprintf("SHOW CREATE TABLE %s.%s", r.logsDB, r.logsLocalTableName)
err = r.db.Select(ctx, &statements, query)
if err != nil {
return nil, &model.ApiError{Err: err, Typ: model.ErrorInternal}
}
r.extractSelectedAndInterestingFields(statements[0].Statement, constants.Attributes, &attributes, &response)
r.extractSelectedAndInterestingFields(statements[0].Statement, constants.Resources, &resources, &response)
return &response, nil
}
func (r *ClickHouseReader) extractSelectedAndInterestingFields(tableStatement string, overrideFieldType string, fields *[]model.Field, response *model.GetFieldsResponse) {
for _, field := range *fields {
if overrideFieldType != "" {
field.Type = overrideFieldType
}
// all static fields are assumed to be selected as we don't allow changing them
if isColumn(tableStatement, field.Type, field.Name, field.DataType) {
response.Selected = append(response.Selected, field)
} else {
response.Interesting = append(response.Interesting, field)
}
}
}
func (r *ClickHouseReader) UpdateLogField(ctx context.Context, field *model.UpdateField) *model.ApiError {
if !field.Selected {
return model.ForbiddenError(errors.New("removing a selected field is not allowed, please reach out to support."))
}
colname := utils.GetClickhouseColumnNameV2(field.Type, field.DataType, field.Name)
field.DataType = strings.ToLower(field.DataType)
dataType := constants.MaterializedDataTypeMap[field.DataType]
chDataType := constants.ChDataTypeMap[field.DataType]
attrColName := fmt.Sprintf("%s_%s", field.Type, dataType)
for _, table := range []string{r.logsLocalTableV2, r.logsTableV2} {
q := "ALTER TABLE %s.%s ON CLUSTER %s ADD COLUMN IF NOT EXISTS `%s` %s DEFAULT %s['%s'] CODEC(ZSTD(1))"
query := fmt.Sprintf(q,
r.logsDB, table,
r.cluster,
colname, chDataType,
attrColName,
field.Name,
)
err := r.db.Exec(ctx, query)
if err != nil {
return &model.ApiError{Err: err, Typ: model.ErrorInternal}
}
query = fmt.Sprintf("ALTER TABLE %s.%s ON CLUSTER %s ADD COLUMN IF NOT EXISTS `%s_exists` bool DEFAULT if(mapContains(%s, '%s') != 0, true, false) CODEC(ZSTD(1))",
r.logsDB, table,
r.cluster,
colname,
attrColName,
field.Name,
)
err = r.db.Exec(ctx, query)
if err != nil {
return &model.ApiError{Err: err, Typ: model.ErrorInternal}
}
}
// create the index
if strings.ToLower(field.DataType) == "bool" {
// there is no point in creating index for bool attributes as the cardinality is just 2
return nil
}
if field.IndexType == "" {
field.IndexType = constants.DefaultLogSkipIndexType
}
if field.IndexGranularity == 0 {
field.IndexGranularity = constants.DefaultLogSkipIndexGranularity
}
query := fmt.Sprintf("ALTER TABLE %s.%s ON CLUSTER %s ADD INDEX IF NOT EXISTS `%s_idx` (`%s`) TYPE %s GRANULARITY %d",
r.logsDB, r.logsLocalTableV2,
r.cluster,
colname,
colname,
field.IndexType,
field.IndexGranularity,
)
err := r.db.Exec(ctx, query)
if err != nil {
return &model.ApiError{Err: err, Typ: model.ErrorInternal}
}
return nil
}
func (r *ClickHouseReader) GetTraceFields(ctx context.Context) (*model.GetFieldsResponse, *model.ApiError) {
// response will contain top level fields from the otel trace model
response := model.GetFieldsResponse{
Selected: []model.Field{},
Interesting: []model.Field{},
}
// get the top level selected fields
for _, field := range constants.NewStaticFieldsTraces {
if (v3.AttributeKey{} == field) {
continue
}
response.Selected = append(response.Selected, model.Field{
Name: field.Key,
DataType: field.DataType.String(),
Type: constants.Static,
})
}
// get attribute keys
attributes := []model.Field{}
query := fmt.Sprintf("SELECT tagKey, tagType, dataType from %s.%s group by tagKey, tagType, dataType", r.TraceDB, r.spanAttributesKeysTable)
rows, err := r.db.Query(ctx, query)
if err != nil {
return nil, &model.ApiError{Err: err, Typ: model.ErrorInternal}
}
defer rows.Close()
var tagKey string
var dataType string
var tagType string
for rows.Next() {
if err := rows.Scan(&tagKey, &tagType, &dataType); err != nil {
return nil, &model.ApiError{Err: err, Typ: model.ErrorInternal}
}
attributes = append(attributes, model.Field{
Name: tagKey,
DataType: dataType,
Type: tagType,
})
}
statements := []model.ShowCreateTableStatement{}
query = fmt.Sprintf("SHOW CREATE TABLE %s.%s", r.TraceDB, r.traceLocalTableName)
err = r.db.Select(ctx, &statements, query)
if err != nil {
return nil, &model.ApiError{Err: err, Typ: model.ErrorInternal}
}
r.extractSelectedAndInterestingFields(statements[0].Statement, "", &attributes, &response)
return &response, nil
}
func (r *ClickHouseReader) UpdateTraceField(ctx context.Context, field *model.UpdateField) *model.ApiError {
if !field.Selected {
return model.ForbiddenError(errors.New("removing a selected field is not allowed, please reach out to support."))
}
// name of the materialized column
colname := utils.GetClickhouseColumnNameV2(field.Type, field.DataType, field.Name)
field.DataType = strings.ToLower(field.DataType)
// dataType and chDataType of the materialized column
chDataType := constants.ChDataTypeMap[field.DataType]
dataType := constants.MaterializedDataTypeMap[field.DataType]
// typeName: tag => attributes, resource => resources
typeName := field.Type
if field.Type == string(v3.AttributeKeyTypeTag) {
typeName = constants.Attributes
} else if field.Type == string(v3.AttributeKeyTypeResource) {
typeName = constants.Resources
}
attrColName := fmt.Sprintf("%s_%s", typeName, dataType)
for _, table := range []string{r.traceLocalTableName, r.traceTableName} {
q := "ALTER TABLE %s.%s ON CLUSTER %s ADD COLUMN IF NOT EXISTS `%s` %s DEFAULT %s['%s'] CODEC(ZSTD(1))"
query := fmt.Sprintf(q,
r.TraceDB, table,
r.cluster,
colname, chDataType,
attrColName,
field.Name,
)
err := r.db.Exec(ctx, query)
if err != nil {
return &model.ApiError{Err: err, Typ: model.ErrorInternal}
}
query = fmt.Sprintf("ALTER TABLE %s.%s ON CLUSTER %s ADD COLUMN IF NOT EXISTS `%s_exists` bool DEFAULT if(mapContains(%s, '%s') != 0, true, false) CODEC(ZSTD(1))",
r.TraceDB, table,
r.cluster,
colname,
attrColName,
field.Name,
)
err = r.db.Exec(ctx, query)
if err != nil {
return &model.ApiError{Err: err, Typ: model.ErrorInternal}
}
}
// create the index
if strings.ToLower(field.DataType) == "bool" {
// there is no point in creating index for bool attributes as the cardinality is just 2
return nil
}
if field.IndexType == "" {
field.IndexType = constants.DefaultLogSkipIndexType
}
if field.IndexGranularity == 0 {
field.IndexGranularity = constants.DefaultLogSkipIndexGranularity
}
query := fmt.Sprintf("ALTER TABLE %s.%s ON CLUSTER %s ADD INDEX IF NOT EXISTS `%s_idx` (`%s`) TYPE %s GRANULARITY %d",
r.TraceDB, r.traceLocalTableName,
r.cluster,
colname,
colname,
field.IndexType,
field.IndexGranularity,
)
err := r.db.Exec(ctx, query)
if err != nil {
return &model.ApiError{Err: err, Typ: model.ErrorInternal}
}
// add a default minmax index for numbers
if dataType == "number" {
query = fmt.Sprintf("ALTER TABLE %s.%s ON CLUSTER %s ADD INDEX IF NOT EXISTS `%s_minmax_idx` (`%s`) TYPE minmax GRANULARITY 1",
r.TraceDB, r.traceLocalTableName,
r.cluster,
colname,
colname,
)
err = r.db.Exec(ctx, query)
if err != nil {
return &model.ApiError{Err: err, Typ: model.ErrorInternal}
}
}
return nil
}
func (r *ClickHouseReader) GetLogs(ctx context.Context, params *model.LogsFilterParams) (*[]model.SignozLog, *model.ApiError) {
response := []model.SignozLog{}
fields, apiErr := r.GetLogFields(ctx)
if apiErr != nil {
return nil, apiErr
}
isPaginatePrev := logs.CheckIfPrevousPaginateAndModifyOrder(params)
filterSql, lenFilters, err := logs.GenerateSQLWhere(fields, params)
if err != nil {
return nil, &model.ApiError{Err: err, Typ: model.ErrorBadData}
}
data := map[string]interface{}{
"lenFilters": lenFilters,
}
if lenFilters != 0 {
claims, errv2 := authtypes.ClaimsFromContext(ctx)
if errv2 == nil {
telemetry.GetInstance().SendEvent(telemetry.TELEMETRY_EVENT_LOGS_FILTERS, data, claims.Email, true, false)
}
}
query := fmt.Sprintf("%s from %s.%s", constants.LogsSQLSelect, r.logsDB, r.logsTable)
if filterSql != "" {
query = fmt.Sprintf("%s where %s", query, filterSql)
}
query = fmt.Sprintf("%s order by %s %s limit %d", query, params.OrderBy, params.Order, params.Limit)
err = r.db.Select(ctx, &response, query)
if err != nil {
return nil, &model.ApiError{Err: err, Typ: model.ErrorInternal}
}
if isPaginatePrev {
// rever the results from db
for i, j := 0, len(response)-1; i < j; i, j = i+1, j-1 {
response[i], response[j] = response[j], response[i]
}
}
return &response, nil
}
func (r *ClickHouseReader) TailLogs(ctx context.Context, client *model.LogsTailClient) {
fields, apiErr := r.GetLogFields(ctx)
if apiErr != nil {
client.Error <- apiErr.Err
return
}
filterSql, lenFilters, err := logs.GenerateSQLWhere(fields, &model.LogsFilterParams{
Query: client.Filter.Query,
})
data := map[string]interface{}{
"lenFilters": lenFilters,
}
if lenFilters != 0 {
claims, errv2 := authtypes.ClaimsFromContext(ctx)
if errv2 == nil {
telemetry.GetInstance().SendEvent(telemetry.TELEMETRY_EVENT_LOGS_FILTERS, data, claims.Email, true, false)
}
}
if err != nil {
client.Error <- err
return
}
query := fmt.Sprintf("%s from %s.%s", constants.LogsSQLSelect, r.logsDB, r.logsTable)
tsStart := uint64(time.Now().UnixNano())
if client.Filter.TimestampStart != 0 {
tsStart = client.Filter.TimestampStart
}
var idStart string
if client.Filter.IdGt != "" {
idStart = client.Filter.IdGt
}
ticker := time.NewTicker(time.Duration(r.liveTailRefreshSeconds) * time.Second)
defer ticker.Stop()
for {
select {
case <-ctx.Done():
done := true
client.Done <- &done
zap.L().Debug("closing go routine : " + client.Name)
return
case <-ticker.C:
// get the new 100 logs as anything more older won't make sense
tmpQuery := fmt.Sprintf("%s where timestamp >='%d'", query, tsStart)
if filterSql != "" {
tmpQuery = fmt.Sprintf("%s and %s", tmpQuery, filterSql)
}
if idStart != "" {
tmpQuery = fmt.Sprintf("%s and id > '%s'", tmpQuery, idStart)
}
tmpQuery = fmt.Sprintf("%s order by timestamp desc, id desc limit 100", tmpQuery)
response := []model.SignozLog{}
err := r.db.Select(ctx, &response, tmpQuery)
if err != nil {
zap.L().Error("Error while getting logs", zap.Error(err))
client.Error <- err
return
}
for i := len(response) - 1; i >= 0; i-- {
select {
case <-ctx.Done():
done := true
client.Done <- &done
zap.L().Debug("closing go routine while sending logs : " + client.Name)
return
default:
client.Logs <- &response[i]
if i == 0 {
tsStart = response[i].Timestamp
idStart = response[i].ID
}
}
}
}
}
}
func (r *ClickHouseReader) AggregateLogs(ctx context.Context, params *model.LogsAggregateParams) (*model.GetLogsAggregatesResponse, *model.ApiError) {
logAggregatesDBResponseItems := []model.LogsAggregatesDBResponseItem{}
function := "toFloat64(count()) as value"
if params.Function != "" {
function = fmt.Sprintf("toFloat64(%s) as value", params.Function)
}
fields, apiErr := r.GetLogFields(ctx)
if apiErr != nil {
return nil, apiErr
}
filterSql, lenFilters, err := logs.GenerateSQLWhere(fields, &model.LogsFilterParams{
Query: params.Query,
})
if err != nil {
return nil, &model.ApiError{Err: err, Typ: model.ErrorBadData}
}
data := map[string]interface{}{
"lenFilters": lenFilters,
}
if lenFilters != 0 {
claims, errv2 := authtypes.ClaimsFromContext(ctx)
if errv2 == nil {
telemetry.GetInstance().SendEvent(telemetry.TELEMETRY_EVENT_LOGS_FILTERS, data, claims.Email, true, false)
}
}
query := ""
if params.GroupBy != "" {
query = fmt.Sprintf("SELECT toInt64(toUnixTimestamp(toStartOfInterval(toDateTime(timestamp/1000000000), INTERVAL %d minute))*1000000000) as ts_start_interval, toString(%s) as groupBy, "+
"%s "+
"FROM %s.%s WHERE (timestamp >= '%d' AND timestamp <= '%d' )",
params.StepSeconds/60, params.GroupBy, function, r.logsDB, r.logsTable, params.TimestampStart, params.TimestampEnd)
} else {
query = fmt.Sprintf("SELECT toInt64(toUnixTimestamp(toStartOfInterval(toDateTime(timestamp/1000000000), INTERVAL %d minute))*1000000000) as ts_start_interval, "+
"%s "+
"FROM %s.%s WHERE (timestamp >= '%d' AND timestamp <= '%d' )",
params.StepSeconds/60, function, r.logsDB, r.logsTable, params.TimestampStart, params.TimestampEnd)
}
if filterSql != "" {
query = fmt.Sprintf("%s AND ( %s ) ", query, filterSql)
}
if params.GroupBy != "" {
query = fmt.Sprintf("%s GROUP BY ts_start_interval, toString(%s) as groupBy ORDER BY ts_start_interval", query, params.GroupBy)
} else {
query = fmt.Sprintf("%s GROUP BY ts_start_interval ORDER BY ts_start_interval", query)
}
err = r.db.Select(ctx, &logAggregatesDBResponseItems, query)
if err != nil {
return nil, &model.ApiError{Err: err, Typ: model.ErrorInternal}
}
aggregateResponse := model.GetLogsAggregatesResponse{
Items: make(map[int64]model.LogsAggregatesResponseItem),
}
for i := range logAggregatesDBResponseItems {
if elem, ok := aggregateResponse.Items[int64(logAggregatesDBResponseItems[i].Timestamp)]; ok {
if params.GroupBy != "" && logAggregatesDBResponseItems[i].GroupBy != "" {
elem.GroupBy[logAggregatesDBResponseItems[i].GroupBy] = logAggregatesDBResponseItems[i].Value
}
aggregateResponse.Items[logAggregatesDBResponseItems[i].Timestamp] = elem
} else {
if params.GroupBy != "" && logAggregatesDBResponseItems[i].GroupBy != "" {
aggregateResponse.Items[logAggregatesDBResponseItems[i].Timestamp] = model.LogsAggregatesResponseItem{
Timestamp: logAggregatesDBResponseItems[i].Timestamp,
GroupBy: map[string]interface{}{logAggregatesDBResponseItems[i].GroupBy: logAggregatesDBResponseItems[i].Value},
}
} else if params.GroupBy == "" {
aggregateResponse.Items[logAggregatesDBResponseItems[i].Timestamp] = model.LogsAggregatesResponseItem{
Timestamp: logAggregatesDBResponseItems[i].Timestamp,
Value: logAggregatesDBResponseItems[i].Value,
}
}
}
}
return &aggregateResponse, nil
}
func (r *ClickHouseReader) QueryDashboardVars(ctx context.Context, query string) (*model.DashboardVar, error) {
var result = model.DashboardVar{VariableValues: make([]interface{}, 0)}
rows, err := r.db.Query(ctx, query)
zap.L().Info(query)
if err != nil {
zap.L().Error("Error in processing sql query", zap.Error(err))
return nil, err
}
var (
columnTypes = rows.ColumnTypes()
vars = make([]interface{}, len(columnTypes))
)
for i := range columnTypes {
vars[i] = reflect.New(columnTypes[i].ScanType()).Interface()
}
defer rows.Close()
for rows.Next() {
if err := rows.Scan(vars...); err != nil {
return nil, err
}
for _, v := range vars {
switch v := v.(type) {
case *string, *int8, *int16, *int32, *int64, *uint8, *uint16, *uint32, *uint64, *float32, *float64, *time.Time, *bool:
result.VariableValues = append(result.VariableValues, reflect.ValueOf(v).Elem().Interface())
default:
return nil, fmt.Errorf("unsupported value type encountered")
}
}
}
return &result, nil
}
func (r *ClickHouseReader) GetMetricAggregateAttributes(ctx context.Context, orgID valuer.UUID, req *v3.AggregateAttributeRequest, skipSignozMetrics bool) (*v3.AggregateAttributeResponse, error) {
var query string
var err error
var rows driver.Rows
var response v3.AggregateAttributeResponse
normalized := true
if constants.IsDotMetricsEnabled {
normalized = false
}
query = fmt.Sprintf("SELECT metric_name, type, is_monotonic, temporality FROM %s.%s WHERE metric_name ILIKE $1 and __normalized = $2 GROUP BY metric_name, type, is_monotonic, temporality", signozMetricDBName, signozTSTableNameV41Day)
if req.Limit != 0 {
query = query + fmt.Sprintf(" LIMIT %d;", req.Limit)
}
rows, err = r.db.Query(ctx, query, fmt.Sprintf("%%%s%%", req.SearchText), normalized)
if err != nil {
zap.L().Error("Error while executing query", zap.Error(err))
return nil, fmt.Errorf("error while executing query: %s", err.Error())
}
defer rows.Close()
seen := make(map[string]struct{})
var metricName, typ, temporality string
var isMonotonic bool
for rows.Next() {
if err := rows.Scan(&metricName, &typ, &isMonotonic, &temporality); err != nil {
return nil, fmt.Errorf("error while scanning rows: %s", err.Error())
}
if skipSignozMetrics && strings.HasPrefix(metricName, "signoz") {
continue
}
metadata, apiError := r.GetUpdatedMetricsMetadata(ctx, orgID, metricName)
if apiError != nil {
zap.L().Error("Error in getting metrics cached metadata", zap.Error(apiError))
}
if updatedMetadata, exist := metadata[metricName]; exist {
typ = string(updatedMetadata.MetricType)
isMonotonic = updatedMetadata.IsMonotonic
temporality = string(updatedMetadata.Temporality)
}
// Non-monotonic cumulative sums are treated as gauges
if typ == "Sum" && !isMonotonic && temporality == string(v3.Cumulative) {
typ = "Gauge"
}
// unlike traces/logs `tag`/`resource` type, the `Type` will be metric type
key := v3.AttributeKey{
Key: metricName,
DataType: v3.AttributeKeyDataTypeFloat64,
Type: v3.AttributeKeyType(typ),
IsColumn: true,
}
// remove duplicates
if _, ok := seen[metricName+typ]; ok {
continue
}
seen[metricName+typ] = struct{}{}
response.AttributeKeys = append(response.AttributeKeys, key)
}
return &response, nil
}
func (r *ClickHouseReader) GetMetricAttributeKeys(ctx context.Context, req *v3.FilterAttributeKeyRequest) (*v3.FilterAttributeKeyResponse, error) {
var query string
var err error
var rows driver.Rows
var response v3.FilterAttributeKeyResponse
normalized := true
if constants.IsDotMetricsEnabled {
normalized = false
}
// skips the internal attributes i.e attributes starting with __
query = fmt.Sprintf("SELECT arrayJoin(tagKeys) AS distinctTagKey FROM (SELECT JSONExtractKeys(labels) AS tagKeys FROM %s.%s WHERE metric_name=$1 AND unix_milli >= $2 AND __normalized = $3 GROUP BY tagKeys) WHERE distinctTagKey ILIKE $4 AND distinctTagKey NOT LIKE '\\_\\_%%' GROUP BY distinctTagKey", signozMetricDBName, signozTSTableNameV41Day)
if req.Limit != 0 {
query = query + fmt.Sprintf(" LIMIT %d;", req.Limit)
}
rows, err = r.db.Query(ctx, query, req.AggregateAttribute, common.PastDayRoundOff(), normalized, fmt.Sprintf("%%%s%%", req.SearchText))
if err != nil {
zap.L().Error("Error while executing query", zap.Error(err))
return nil, fmt.Errorf("error while executing query: %s", err.Error())
}
defer rows.Close()
var attributeKey string
for rows.Next() {
if err := rows.Scan(&attributeKey); err != nil {
return nil, fmt.Errorf("error while scanning rows: %s", err.Error())
}
key := v3.AttributeKey{
Key: attributeKey,
DataType: v3.AttributeKeyDataTypeString, // https://github.com/OpenObservability/OpenMetrics/blob/main/proto/openmetrics_data_model.proto#L64-L72.
Type: v3.AttributeKeyTypeTag,
IsColumn: false,
}
response.AttributeKeys = append(response.AttributeKeys, key)
}
return &response, nil
}
func (r *ClickHouseReader) GetMetricAttributeValues(ctx context.Context, req *v3.FilterAttributeValueRequest) (*v3.FilterAttributeValueResponse, error) {
var query string
var err error
var rows driver.Rows
var attributeValues v3.FilterAttributeValueResponse
normalized := true
if constants.IsDotMetricsEnabled {
normalized = false
}
query = fmt.Sprintf("SELECT JSONExtractString(labels, $1) AS tagValue FROM %s.%s WHERE metric_name IN $2 AND JSONExtractString(labels, $3) ILIKE $4 AND unix_milli >= $5 AND __normalized=$6 GROUP BY tagValue", signozMetricDBName, signozTSTableNameV41Day)
if req.Limit != 0 {
query = query + fmt.Sprintf(" LIMIT %d;", req.Limit)
}
names := []string{req.AggregateAttribute}
names = append(names, metrics.GetTransitionedMetric(req.AggregateAttribute, normalized))
rows, err = r.db.Query(ctx, query, req.FilterAttributeKey, names, req.FilterAttributeKey, fmt.Sprintf("%%%s%%", req.SearchText), common.PastDayRoundOff(), normalized)
if err != nil {
zap.L().Error("Error while executing query", zap.Error(err))
return nil, fmt.Errorf("error while executing query: %s", err.Error())
}
defer rows.Close()
var atrributeValue string
for rows.Next() {
if err := rows.Scan(&atrributeValue); err != nil {
return nil, fmt.Errorf("error while scanning rows: %s", err.Error())
}
// https://github.com/OpenObservability/OpenMetrics/blob/main/proto/openmetrics_data_model.proto#L64-L72
// this may change in future if we use OTLP as the data model
attributeValues.StringAttributeValues = append(attributeValues.StringAttributeValues, atrributeValue)
}
return &attributeValues, nil
}
func (r *ClickHouseReader) GetMetricMetadata(ctx context.Context, orgID valuer.UUID, metricName, serviceName string) (*v3.MetricMetadataResponse, error) {
unixMilli := common.PastDayRoundOff()
// Note: metric metadata should be accessible regardless of the time range selection
// our standard retention period is 30 days, so we are querying the table v4_1_day to reduce the
// amount of data scanned
query := fmt.Sprintf("SELECT temporality, description, type, unit, is_monotonic from %s.%s WHERE metric_name=$1 AND unix_milli >= $2 GROUP BY temporality, description, type, unit, is_monotonic", signozMetricDBName, signozTSTableNameV41Day)
rows, err := r.db.Query(ctx, query, metricName, unixMilli)
if err != nil {
zap.L().Error("Error while fetching metric metadata", zap.Error(err))
return nil, fmt.Errorf("error while fetching metric metadata: %s", err.Error())
}
defer rows.Close()
var deltaExists, isMonotonic bool
var temporality, description, metricType, unit string
for rows.Next() {
if err := rows.Scan(&temporality, &description, &metricType, &unit, &isMonotonic); err != nil {
return nil, fmt.Errorf("error while scanning rows: %s", err.Error())
}
if temporality == string(v3.Delta) {
deltaExists = true
}
}
metadata, apiError := r.GetUpdatedMetricsMetadata(ctx, orgID, metricName)
if apiError != nil {
zap.L().Error("Error in getting metric cached metadata", zap.Error(apiError))
}
if updatedMetadata, exist := metadata[metricName]; exist {
metricType = string(updatedMetadata.MetricType)
temporality = string(updatedMetadata.Temporality)
if temporality == string(v3.Delta) {
deltaExists = true
}
isMonotonic = updatedMetadata.IsMonotonic
if updatedMetadata.Description != "" {
description = updatedMetadata.Description
}
if updatedMetadata.Unit != "" {
unit = updatedMetadata.Unit
}
}
query = fmt.Sprintf("SELECT JSONExtractString(labels, 'le') as le from %s.%s WHERE metric_name=$1 AND unix_milli >= $2 AND type = 'Histogram' AND JSONExtractString(labels, 'service_name') = $3 GROUP BY le ORDER BY le", signozMetricDBName, signozTSTableNameV41Day)
rows, err = r.db.Query(ctx, query, metricName, unixMilli, serviceName)
if err != nil {
zap.L().Error("Error while executing query", zap.Error(err))
return nil, fmt.Errorf("error while executing query: %s", err.Error())
}
defer rows.Close()
var leFloat64 []float64
for rows.Next() {
var leStr string
if err := rows.Scan(&leStr); err != nil {
return nil, fmt.Errorf("error while scanning rows: %s", err.Error())
}
le, err := strconv.ParseFloat(leStr, 64)
// ignore the error and continue if the value is not a float
// ideally this should not happen but we have seen ClickHouse
// returning empty string for some values
if err != nil {
zap.L().Error("error while parsing le value", zap.Error(err))
continue
}
if math.IsInf(le, 0) {
continue
}
leFloat64 = append(leFloat64, le)
}
return &v3.MetricMetadataResponse{
Delta: deltaExists,
Le: leFloat64,
Description: description,
Unit: unit,
Type: metricType,
IsMonotonic: isMonotonic,
Temporality: temporality,
}, nil
}
// GetCountOfThings returns the count of things in the query
// This is a generic function that can be used to check if any data exists for a given query
func (r *ClickHouseReader) GetCountOfThings(ctx context.Context, query string) (uint64, error) {
var count uint64
err := r.db.QueryRow(ctx, query).Scan(&count)
if err != nil {
return 0, err
}
return count, nil
}
func (r *ClickHouseReader) GetLatestReceivedMetric(
ctx context.Context, metricNames []string, labelValues map[string]string,
) (*model.MetricStatus, *model.ApiError) {
// at least 1 metric name must be specified.
// this query can be too slow otherwise.
if len(metricNames) < 1 {
return nil, model.BadRequest(fmt.Errorf("atleast 1 metric name must be specified"))
}
quotedMetricNames := []string{}
for _, m := range metricNames {
quotedMetricNames = append(quotedMetricNames, utils.ClickHouseFormattedValue(m))
}
commaSeparatedMetricNames := strings.Join(quotedMetricNames, ", ")
whereClauseParts := []string{
fmt.Sprintf(`metric_name in (%s)`, commaSeparatedMetricNames),
}
if labelValues != nil {
for label, val := range labelValues {
whereClauseParts = append(
whereClauseParts,
fmt.Sprintf(`JSONExtractString(labels, '%s') = '%s'`, label, val),
)
}
}
if len(whereClauseParts) < 1 {
return nil, nil
}
whereClause := strings.Join(whereClauseParts, " AND ")
query := fmt.Sprintf(`
SELECT metric_name, anyLast(labels), max(unix_milli)
from %s.%s
where %s
group by metric_name
limit 1
`, signozMetricDBName, signozTSTableNameV4, whereClause,
)
rows, err := r.db.Query(ctx, query)
if err != nil {
return nil, model.InternalError(fmt.Errorf(
"couldn't query clickhouse for received metrics status: %w", err,
))
}
defer rows.Close()
var result *model.MetricStatus
if rows.Next() {
result = &model.MetricStatus{}
var labelsJson string
err := rows.Scan(
&result.MetricName,
&labelsJson,
&result.LastReceivedTsMillis,
)
if err != nil {
return nil, model.InternalError(fmt.Errorf(
"couldn't scan metric status row: %w", err,
))
}
err = json.Unmarshal([]byte(labelsJson), &result.LastReceivedLabels)
if err != nil {
return nil, model.InternalError(fmt.Errorf(
"couldn't unmarshal metric labels json: %w", err,
))
}
}
return result, nil
}
func isColumn(tableStatement, attrType, field, datType string) bool {
name := fmt.Sprintf("`%s`", utils.GetClickhouseColumnNameV2(attrType, datType, field))
return strings.Contains(tableStatement, fmt.Sprintf("%s ", name))
}
func (r *ClickHouseReader) GetLogAggregateAttributes(ctx context.Context, req *v3.AggregateAttributeRequest) (*v3.AggregateAttributeResponse, error) {
var query string
var err error
var rows driver.Rows
var response v3.AggregateAttributeResponse
var stringAllowed bool
where := ""
switch req.Operator {
case
v3.AggregateOperatorCountDistinct,
v3.AggregateOperatorCount:
where = "tag_key ILIKE $1"
stringAllowed = true
case
v3.AggregateOperatorRateSum,
v3.AggregateOperatorRateMax,
v3.AggregateOperatorRateAvg,
v3.AggregateOperatorRate,
v3.AggregateOperatorRateMin,
v3.AggregateOperatorP05,
v3.AggregateOperatorP10,
v3.AggregateOperatorP20,
v3.AggregateOperatorP25,
v3.AggregateOperatorP50,
v3.AggregateOperatorP75,
v3.AggregateOperatorP90,
v3.AggregateOperatorP95,
v3.AggregateOperatorP99,
v3.AggregateOperatorAvg,
v3.AggregateOperatorSum,
v3.AggregateOperatorMin,
v3.AggregateOperatorMax:
where = "tag_key ILIKE $1 AND (tag_data_type='int64' or tag_data_type='float64')"
stringAllowed = false
case
v3.AggregateOperatorNoOp:
return &v3.AggregateAttributeResponse{}, nil
default:
return nil, fmt.Errorf("unsupported aggregate operator")
}
query = fmt.Sprintf("SELECT DISTINCT(tag_key), tag_type, tag_data_type from %s.%s WHERE %s and tag_type != 'logfield' limit $2", r.logsDB, r.logsTagAttributeTableV2, where)
rows, err = r.db.Query(ctx, query, fmt.Sprintf("%%%s%%", req.SearchText), req.Limit)
if err != nil {
zap.L().Error("Error while executing query", zap.Error(err))
return nil, fmt.Errorf("error while executing query: %s", err.Error())
}
defer rows.Close()
statements := []model.ShowCreateTableStatement{}
query = fmt.Sprintf("SHOW CREATE TABLE %s.%s", r.logsDB, r.logsLocalTableName)
err = r.db.Select(ctx, &statements, query)
if err != nil {
return nil, fmt.Errorf("error while fetching logs schema: %s", err.Error())
}
var tagKey string
var dataType string
var attType string
for rows.Next() {
if err := rows.Scan(&tagKey, &attType, &dataType); err != nil {
return nil, fmt.Errorf("error while scanning rows: %s", err.Error())
}
key := v3.AttributeKey{
Key: tagKey,
DataType: v3.AttributeKeyDataType(dataType),
Type: v3.AttributeKeyType(attType),
IsColumn: isColumn(statements[0].Statement, attType, tagKey, dataType),
}
response.AttributeKeys = append(response.AttributeKeys, key)
}
// add other attributes
for _, field := range constants.StaticFieldsLogsV3 {
if (!stringAllowed && field.DataType == v3.AttributeKeyDataTypeString) || (v3.AttributeKey{} == field) {
continue
} else if len(req.SearchText) == 0 || strings.Contains(field.Key, req.SearchText) {
response.AttributeKeys = append(response.AttributeKeys, field)
}
}
return &response, nil
}
func (r *ClickHouseReader) GetLogAttributeKeys(ctx context.Context, req *v3.FilterAttributeKeyRequest) (*v3.FilterAttributeKeyResponse, error) {
var query string
var err error
var rows driver.Rows
var response v3.FilterAttributeKeyResponse
tagTypeFilter := `tag_type != 'logfield'`
if req.TagType != "" {
tagTypeFilter = fmt.Sprintf(`tag_type != 'logfield' and tag_type = '%s'`, req.TagType)
}
if len(req.SearchText) != 0 {
query = fmt.Sprintf("select distinct tag_key, tag_type, tag_data_type from %s.%s where %s and tag_key ILIKE $1 limit $2", r.logsDB, r.logsTagAttributeTableV2, tagTypeFilter)
rows, err = r.db.Query(ctx, query, fmt.Sprintf("%%%s%%", req.SearchText), req.Limit)
} else {
query = fmt.Sprintf("select distinct tag_key, tag_type, tag_data_type from %s.%s where %s limit $1", r.logsDB, r.logsTagAttributeTableV2, tagTypeFilter)
rows, err = r.db.Query(ctx, query, req.Limit)
}
if err != nil {
zap.L().Error("Error while executing query", zap.Error(err))
return nil, fmt.Errorf("error while executing query: %s", err.Error())
}
defer rows.Close()
statements := []model.ShowCreateTableStatement{}
query = fmt.Sprintf("SHOW CREATE TABLE %s.%s", r.logsDB, r.logsLocalTableName)
err = r.db.Select(ctx, &statements, query)
if err != nil {
return nil, fmt.Errorf("error while fetching logs schema: %s", err.Error())
}
var attributeKey string
var attributeDataType string
var tagType string
for rows.Next() {
if err := rows.Scan(&attributeKey, &tagType, &attributeDataType); err != nil {
return nil, fmt.Errorf("error while scanning rows: %s", err.Error())
}
key := v3.AttributeKey{
Key: attributeKey,
DataType: v3.AttributeKeyDataType(attributeDataType),
Type: v3.AttributeKeyType(tagType),
IsColumn: isColumn(statements[0].Statement, tagType, attributeKey, attributeDataType),
}
response.AttributeKeys = append(response.AttributeKeys, key)
}
// add other attributes only when the tagType is not specified
// i.e retrieve all attributes
if req.TagType == "" {
for _, f := range constants.StaticFieldsLogsV3 {
if (v3.AttributeKey{} == f) {
continue
}
if len(req.SearchText) == 0 || strings.Contains(f.Key, req.SearchText) {
response.AttributeKeys = append(response.AttributeKeys, f)
}
}
}
return &response, nil
}
func (r *ClickHouseReader) FetchRelatedValues(ctx context.Context, req *v3.FilterAttributeValueRequest) ([]string, error) {
var andConditions []string
andConditions = append(andConditions, fmt.Sprintf("unix_milli >= %d", req.StartTimeMillis))
andConditions = append(andConditions, fmt.Sprintf("unix_milli <= %d", req.EndTimeMillis))
if len(req.ExistingFilterItems) != 0 {
for _, item := range req.ExistingFilterItems {
// we only support string for related values
if item.Key.DataType != v3.AttributeKeyDataTypeString {
continue
}
var colName string
switch item.Key.Type {
case v3.AttributeKeyTypeResource:
colName = "resource_attributes"
case v3.AttributeKeyTypeTag:
colName = "attributes"
default:
// we only support resource and tag for related values as of now
continue
}
// IN doesn't make use of map value index, we convert it to = or !=
operator := item.Operator
if v3.FilterOperator(strings.ToLower(string(item.Operator))) == v3.FilterOperatorIn {
operator = "="
} else if v3.FilterOperator(strings.ToLower(string(item.Operator))) == v3.FilterOperatorNotIn {
operator = "!="
}
addCondition := func(val string) {
andConditions = append(andConditions, fmt.Sprintf("mapContains(%s, '%s') AND %s['%s'] %s %s", colName, item.Key.Key, colName, item.Key.Key, operator, val))
}
switch v := item.Value.(type) {
case string:
fmtVal := utils.ClickHouseFormattedValue(v)
addCondition(fmtVal)
case []string:
for _, val := range v {
fmtVal := utils.ClickHouseFormattedValue(val)
addCondition(fmtVal)
}
case []interface{}:
for _, val := range v {
fmtVal := utils.ClickHouseFormattedValue(val)
addCondition(fmtVal)
}
}
}
}
whereClause := strings.Join(andConditions, " AND ")
var selectColumn string
switch req.TagType {
case v3.TagTypeResource:
selectColumn = "resource_attributes" + "['" + req.FilterAttributeKey + "']"
case v3.TagTypeTag:
selectColumn = "attributes" + "['" + req.FilterAttributeKey + "']"
default:
selectColumn = "attributes" + "['" + req.FilterAttributeKey + "']"
}
filterSubQuery := fmt.Sprintf(
"SELECT DISTINCT %s FROM %s.%s WHERE %s LIMIT 100",
selectColumn,
r.metadataDB,
r.metadataTable,
whereClause,
)
zap.L().Debug("filterSubQuery for related values", zap.String("query", filterSubQuery))
rows, err := r.db.Query(ctx, filterSubQuery)
if err != nil {
return nil, fmt.Errorf("error while executing query: %s", err.Error())
}
defer rows.Close()
var attributeValues []string
for rows.Next() {
var value string
if err := rows.Scan(&value); err != nil {
return nil, fmt.Errorf("error while scanning rows: %s", err.Error())
}
if value != "" {
attributeValues = append(attributeValues, value)
}
}
return attributeValues, nil
}
func (r *ClickHouseReader) GetLogAttributeValues(ctx context.Context, req *v3.FilterAttributeValueRequest) (*v3.FilterAttributeValueResponse, error) {
var err error
var filterValueColumn string
var rows driver.Rows
var attributeValues v3.FilterAttributeValueResponse
// if dataType or tagType is not present return empty response
if len(req.FilterAttributeKeyDataType) == 0 || len(req.TagType) == 0 {
// also check if it is not a top level key
if _, ok := constants.StaticFieldsLogsV3[req.FilterAttributeKey]; !ok {
return &v3.FilterAttributeValueResponse{}, nil
}
}
// ignore autocomplete request for body
if req.FilterAttributeKey == "body" || req.FilterAttributeKey == "__attrs" {
return &v3.FilterAttributeValueResponse{}, nil
}
// if data type is bool, return true and false
if req.FilterAttributeKeyDataType == v3.AttributeKeyDataTypeBool {
return &v3.FilterAttributeValueResponse{
BoolAttributeValues: []bool{true, false},
}, nil
}
query := "select distinct"
switch req.FilterAttributeKeyDataType {
case v3.AttributeKeyDataTypeInt64:
filterValueColumn = "number_value"
case v3.AttributeKeyDataTypeFloat64:
filterValueColumn = "number_value"
case v3.AttributeKeyDataTypeString:
filterValueColumn = "string_value"
}
searchText := fmt.Sprintf("%%%s%%", req.SearchText)
// check if the tagKey is a topLevelColumn
if _, ok := constants.StaticFieldsLogsV3[req.FilterAttributeKey]; ok {
// query the column for the last 48 hours
filterValueColumnWhere := req.FilterAttributeKey
selectKey := req.FilterAttributeKey
if req.FilterAttributeKeyDataType != v3.AttributeKeyDataTypeString {
filterValueColumnWhere = fmt.Sprintf("toString(%s)", req.FilterAttributeKey)
selectKey = fmt.Sprintf("toInt64(%s)", req.FilterAttributeKey)
}
// prepare the query and run
if len(req.SearchText) != 0 {
query = fmt.Sprintf("select distinct %s from %s.%s where timestamp >= toInt64(toUnixTimestamp(now() - INTERVAL 48 HOUR)*1000000000) and %s ILIKE $1 limit $2", selectKey, r.logsDB, r.logsLocalTableName, filterValueColumnWhere)
rows, err = r.db.Query(ctx, query, searchText, req.Limit)
} else {
query = fmt.Sprintf("select distinct %s from %s.%s where timestamp >= toInt64(toUnixTimestamp(now() - INTERVAL 48 HOUR)*1000000000) limit $1", selectKey, r.logsDB, r.logsLocalTableName)
rows, err = r.db.Query(ctx, query, req.Limit)
}
} else if len(req.SearchText) != 0 {
filterValueColumnWhere := filterValueColumn
if req.FilterAttributeKeyDataType != v3.AttributeKeyDataTypeString {
filterValueColumnWhere = fmt.Sprintf("toString(%s)", filterValueColumn)
}
query = fmt.Sprintf("SELECT DISTINCT %s FROM %s.%s WHERE tag_key=$1 AND %s ILIKE $2 AND tag_type=$3 LIMIT $4", filterValueColumn, r.logsDB, r.logsTagAttributeTableV2, filterValueColumnWhere)
rows, err = r.db.Query(ctx, query, req.FilterAttributeKey, searchText, req.TagType, req.Limit)
} else {
query = fmt.Sprintf("SELECT DISTINCT %s FROM %s.%s WHERE tag_key=$1 AND tag_type=$2 LIMIT $3", filterValueColumn, r.logsDB, r.logsTagAttributeTableV2)
rows, err = r.db.Query(ctx, query, req.FilterAttributeKey, req.TagType, req.Limit)
}
if err != nil {
zap.L().Error("Error while executing query", zap.Error(err))
return nil, fmt.Errorf("error while executing query: %s", err.Error())
}
defer rows.Close()
var strAttributeValue string
var float64AttributeValue sql.NullFloat64
var int64AttributeValue sql.NullInt64
for rows.Next() {
switch req.FilterAttributeKeyDataType {
case v3.AttributeKeyDataTypeInt64:
if err := rows.Scan(&int64AttributeValue); err != nil {
return nil, fmt.Errorf("error while scanning rows: %s", err.Error())
}
if int64AttributeValue.Valid {
attributeValues.NumberAttributeValues = append(attributeValues.NumberAttributeValues, int64AttributeValue.Int64)
}
case v3.AttributeKeyDataTypeFloat64:
if err := rows.Scan(&float64AttributeValue); err != nil {
return nil, fmt.Errorf("error while scanning rows: %s", err.Error())
}
if float64AttributeValue.Valid {
attributeValues.NumberAttributeValues = append(attributeValues.NumberAttributeValues, float64AttributeValue.Float64)
}
case v3.AttributeKeyDataTypeString:
if err := rows.Scan(&strAttributeValue); err != nil {
return nil, fmt.Errorf("error while scanning rows: %s", err.Error())
}
attributeValues.StringAttributeValues = append(attributeValues.StringAttributeValues, strAttributeValue)
}
}
if req.IncludeRelated {
relatedValues, _ := r.FetchRelatedValues(ctx, req)
attributeValues.RelatedValues = &v3.FilterAttributeValueResponse{
StringAttributeValues: relatedValues,
}
}
return &attributeValues, nil
}
func readRow(vars []interface{}, columnNames []string, countOfNumberCols int) ([]string, map[string]string, []map[string]string, *v3.Point) {
// Each row will have a value and a timestamp, and an optional list of label values
// example: {Timestamp: ..., Value: ...}
// The timestamp may also not present in some cases where the time series is reduced to single value
var point v3.Point
// groupBy is a container to hold label values for the current point
// example: ["frontend", "/fetch"]
var groupBy []string
var groupAttributesArray []map[string]string
// groupAttributes is a container to hold the key-value pairs for the current
// metric point.
// example: {"serviceName": "frontend", "operation": "/fetch"}
groupAttributes := make(map[string]string)
isValidPoint := false
for idx, v := range vars {
colName := columnNames[idx]
switch v := v.(type) {
case *string:
// special case for returning all labels in metrics datasource
if colName == "fullLabels" {
var metric map[string]string
err := json.Unmarshal([]byte(*v), &metric)
if err != nil {
zap.L().Error("unexpected error encountered", zap.Error(err))
}
for key, val := range metric {
groupBy = append(groupBy, val)
if _, ok := groupAttributes[key]; !ok {
groupAttributesArray = append(groupAttributesArray, map[string]string{key: val})
}
groupAttributes[key] = val
}
} else {
groupBy = append(groupBy, *v)
if _, ok := groupAttributes[colName]; !ok {
groupAttributesArray = append(groupAttributesArray, map[string]string{colName: *v})
}
groupAttributes[colName] = *v
}
case *time.Time:
point.Timestamp = v.UnixMilli()
case *float64, *float32:
if _, ok := constants.ReservedColumnTargetAliases[colName]; ok || countOfNumberCols == 1 {
isValidPoint = true
point.Value = float64(reflect.ValueOf(v).Elem().Float())
} else {
val := strconv.FormatFloat(reflect.ValueOf(v).Elem().Float(), 'f', -1, 64)
groupBy = append(groupBy, val)
if _, ok := groupAttributes[colName]; !ok {
groupAttributesArray = append(groupAttributesArray, map[string]string{colName: val})
}
groupAttributes[colName] = val
}
case **float64, **float32:
val := reflect.ValueOf(v)
if val.IsValid() && !val.IsNil() && !val.Elem().IsNil() {
value := reflect.ValueOf(v).Elem().Elem().Float()
if _, ok := constants.ReservedColumnTargetAliases[colName]; ok || countOfNumberCols == 1 {
isValidPoint = true
point.Value = value
} else {
val := strconv.FormatFloat(value, 'f', -1, 64)
groupBy = append(groupBy, val)
if _, ok := groupAttributes[colName]; !ok {
groupAttributesArray = append(groupAttributesArray, map[string]string{colName: val})
}
groupAttributes[colName] = val
}
}
case *uint, *uint8, *uint64, *uint16, *uint32:
if _, ok := constants.ReservedColumnTargetAliases[colName]; ok || countOfNumberCols == 1 {
isValidPoint = true
point.Value = float64(reflect.ValueOf(v).Elem().Uint())
} else {
groupBy = append(groupBy, fmt.Sprintf("%v", reflect.ValueOf(v).Elem().Uint()))
if _, ok := groupAttributes[colName]; !ok {
groupAttributesArray = append(groupAttributesArray, map[string]string{colName: fmt.Sprintf("%v", reflect.ValueOf(v).Elem().Uint())})
}
groupAttributes[colName] = fmt.Sprintf("%v", reflect.ValueOf(v).Elem().Uint())
}
case **uint, **uint8, **uint64, **uint16, **uint32:
val := reflect.ValueOf(v)
if val.IsValid() && !val.IsNil() && !val.Elem().IsNil() {
value := reflect.ValueOf(v).Elem().Elem().Uint()
if _, ok := constants.ReservedColumnTargetAliases[colName]; ok || countOfNumberCols == 1 {
isValidPoint = true
point.Value = float64(value)
} else {
groupBy = append(groupBy, fmt.Sprintf("%v", value))
if _, ok := groupAttributes[colName]; !ok {
groupAttributesArray = append(groupAttributesArray, map[string]string{colName: fmt.Sprintf("%v", value)})
}
groupAttributes[colName] = fmt.Sprintf("%v", value)
}
}
case *int, *int8, *int16, *int32, *int64:
if _, ok := constants.ReservedColumnTargetAliases[colName]; ok || countOfNumberCols == 1 {
isValidPoint = true
point.Value = float64(reflect.ValueOf(v).Elem().Int())
} else {
groupBy = append(groupBy, fmt.Sprintf("%v", reflect.ValueOf(v).Elem().Int()))
if _, ok := groupAttributes[colName]; !ok {
groupAttributesArray = append(groupAttributesArray, map[string]string{colName: fmt.Sprintf("%v", reflect.ValueOf(v).Elem().Int())})
}
groupAttributes[colName] = fmt.Sprintf("%v", reflect.ValueOf(v).Elem().Int())
}
case **int, **int8, **int16, **int32, **int64:
val := reflect.ValueOf(v)
if val.IsValid() && !val.IsNil() && !val.Elem().IsNil() {
value := reflect.ValueOf(v).Elem().Elem().Int()
if _, ok := constants.ReservedColumnTargetAliases[colName]; ok || countOfNumberCols == 1 {
isValidPoint = true
point.Value = float64(value)
} else {
groupBy = append(groupBy, fmt.Sprintf("%v", value))
if _, ok := groupAttributes[colName]; !ok {
groupAttributesArray = append(groupAttributesArray, map[string]string{colName: fmt.Sprintf("%v", value)})
}
groupAttributes[colName] = fmt.Sprintf("%v", value)
}
}
case *bool:
groupBy = append(groupBy, fmt.Sprintf("%v", *v))
if _, ok := groupAttributes[colName]; !ok {
groupAttributesArray = append(groupAttributesArray, map[string]string{colName: fmt.Sprintf("%v", *v)})
}
groupAttributes[colName] = fmt.Sprintf("%v", *v)
default:
zap.L().Error("unsupported var type found in query builder query result", zap.Any("v", v), zap.String("colName", colName))
}
}
if isValidPoint {
return groupBy, groupAttributes, groupAttributesArray, &point
}
return groupBy, groupAttributes, groupAttributesArray, nil
}
func readRowsForTimeSeriesResult(rows driver.Rows, vars []interface{}, columnNames []string, countOfNumberCols int) ([]*v3.Series, error) {
// when groupBy is applied, each combination of cartesian product
// of attribute values is a separate series. Each item in seriesToPoints
// represent a unique series where the key is sorted attribute values joined
// by "," and the value is the list of points for that series
// For instance, group by (serviceName, operation)
// with two services and three operations in each will result in (maximum of) 6 series
// ("frontend", "order") x ("/fetch", "/fetch/{Id}", "/order")
//
// ("frontend", "/fetch")
// ("frontend", "/fetch/{Id}")
// ("frontend", "/order")
// ("order", "/fetch")
// ("order", "/fetch/{Id}")
// ("order", "/order")
seriesToPoints := make(map[string][]v3.Point)
var keys []string
// seriesToAttrs is a mapping of key to a map of attribute key to attribute value
// for each series. This is used to populate the series' attributes
// For instance, for the above example, the seriesToAttrs will be
// {
// "frontend,/fetch": {"serviceName": "frontend", "operation": "/fetch"},
// "frontend,/fetch/{Id}": {"serviceName": "frontend", "operation": "/fetch/{Id}"},
// "frontend,/order": {"serviceName": "frontend", "operation": "/order"},
// "order,/fetch": {"serviceName": "order", "operation": "/fetch"},
// "order,/fetch/{Id}": {"serviceName": "order", "operation": "/fetch/{Id}"},
// "order,/order": {"serviceName": "order", "operation": "/order"},
// }
seriesToAttrs := make(map[string]map[string]string)
labelsArray := make(map[string][]map[string]string)
for rows.Next() {
if err := rows.Scan(vars...); err != nil {
return nil, err
}
groupBy, groupAttributes, groupAttributesArray, metricPoint := readRow(vars, columnNames, countOfNumberCols)
// skip the point if the value is NaN or Inf
// are they ever useful enough to be returned?
if metricPoint != nil && (math.IsNaN(metricPoint.Value) || math.IsInf(metricPoint.Value, 0)) {
continue
}
sort.Strings(groupBy)
key := strings.Join(groupBy, "")
if _, exists := seriesToAttrs[key]; !exists {
keys = append(keys, key)
}
seriesToAttrs[key] = groupAttributes
labelsArray[key] = groupAttributesArray
if metricPoint != nil {
seriesToPoints[key] = append(seriesToPoints[key], *metricPoint)
}
}
var seriesList []*v3.Series
for _, key := range keys {
points := seriesToPoints[key]
series := v3.Series{Labels: seriesToAttrs[key], Points: points, LabelsArray: labelsArray[key]}
seriesList = append(seriesList, &series)
}
return seriesList, getPersonalisedError(rows.Err())
}
func logCommentKVs(ctx context.Context) map[string]string {
kv := ctx.Value(common.LogCommentKey)
if kv == nil {
return nil
}
logCommentKVs, ok := kv.(map[string]string)
if !ok {
return nil
}
return logCommentKVs
}
// GetTimeSeriesResultV3 runs the query and returns list of time series
func (r *ClickHouseReader) GetTimeSeriesResultV3(ctx context.Context, query string) ([]*v3.Series, error) {
ctxArgs := map[string]interface{}{"query": query}
for k, v := range logCommentKVs(ctx) {
ctxArgs[k] = v
}
defer utils.Elapsed("GetTimeSeriesResultV3", ctxArgs)()
// Hook up query progress reporting if requested.
queryId := ctx.Value("queryId")
if queryId != nil {
qid, ok := queryId.(string)
if !ok {
zap.L().Error("GetTimeSeriesResultV3: queryId in ctx not a string as expected", zap.Any("queryId", queryId))
} else {
ctx = clickhouse.Context(ctx, clickhouse.WithProgress(
func(p *clickhouse.Progress) {
go func() {
err := r.queryProgressTracker.ReportQueryProgress(qid, p)
if err != nil {
zap.L().Error(
"Couldn't report query progress",
zap.String("queryId", qid), zap.Error(err),
)
}
}()
},
))
}
}
rows, err := r.db.Query(ctx, query)
if err != nil {
zap.L().Error("error while reading time series result", zap.Error(err))
return nil, errors.New(err.Error())
}
defer rows.Close()
var (
columnTypes = rows.ColumnTypes()
columnNames = rows.Columns()
vars = make([]interface{}, len(columnTypes))
)
var countOfNumberCols int
for i := range columnTypes {
vars[i] = reflect.New(columnTypes[i].ScanType()).Interface()
switch columnTypes[i].ScanType().Kind() {
case reflect.Float32,
reflect.Float64,
reflect.Uint,
reflect.Uint8,
reflect.Uint16,
reflect.Uint32,
reflect.Uint64,
reflect.Int,
reflect.Int8,
reflect.Int16,
reflect.Int32,
reflect.Int64:
countOfNumberCols++
}
}
return readRowsForTimeSeriesResult(rows, vars, columnNames, countOfNumberCols)
}
// GetListResultV3 runs the query and returns list of rows
func (r *ClickHouseReader) GetListResultV3(ctx context.Context, query string) ([]*v3.Row, error) {
ctxArgs := map[string]interface{}{"query": query}
for k, v := range logCommentKVs(ctx) {
ctxArgs[k] = v
}
defer utils.Elapsed("GetListResultV3", ctxArgs)()
rows, err := r.db.Query(ctx, query)
if err != nil {
zap.L().Error("error while reading time series result", zap.Error(err))
return nil, errors.New(err.Error())
}
defer rows.Close()
var (
columnTypes = rows.ColumnTypes()
columnNames = rows.Columns()
)
var rowList []*v3.Row
for rows.Next() {
var vars = make([]interface{}, len(columnTypes))
for i := range columnTypes {
vars[i] = reflect.New(columnTypes[i].ScanType()).Interface()
}
if err := rows.Scan(vars...); err != nil {
return nil, err
}
row := map[string]interface{}{}
var t time.Time
for idx, v := range vars {
if columnNames[idx] == "timestamp" {
switch v := v.(type) {
case *uint64:
t = time.Unix(0, int64(*v))
case *time.Time:
t = *v
}
} else if columnNames[idx] == "timestamp_datetime" {
t = *v.(*time.Time)
} else if columnNames[idx] == "events" {
var events []map[string]interface{}
eventsFromDB, ok := v.(*[]string)
if !ok {
continue
}
for _, event := range *eventsFromDB {
var eventMap map[string]interface{}
json.Unmarshal([]byte(event), &eventMap)
events = append(events, eventMap)
}
row[columnNames[idx]] = events
} else {
row[columnNames[idx]] = v
}
}
// remove duplicate _ attributes for logs.
// remove this function after a month
removeDuplicateUnderscoreAttributes(row)
rowList = append(rowList, &v3.Row{Timestamp: t, Data: row})
}
return rowList, getPersonalisedError(rows.Err())
}
func getPersonalisedError(err error) error {
if err == nil {
return nil
}
zap.L().Error("error while reading result", zap.Error(err))
if strings.Contains(err.Error(), "code: 307") {
return chErrors.ErrResourceBytesLimitExceeded
}
if strings.Contains(err.Error(), "code: 159") {
return chErrors.ErrResourceTimeLimitExceeded
}
return err
}
func removeDuplicateUnderscoreAttributes(row map[string]interface{}) {
if val, ok := row["attributes_int64"]; ok {
attributes := val.(*map[string]int64)
for key := range *attributes {
if strings.Contains(key, ".") {
uKey := strings.ReplaceAll(key, ".", "_")
delete(*attributes, uKey)
}
}
}
if val, ok := row["attributes_float64"]; ok {
attributes := val.(*map[string]float64)
for key := range *attributes {
if strings.Contains(key, ".") {
uKey := strings.ReplaceAll(key, ".", "_")
delete(*attributes, uKey)
}
}
}
if val, ok := row["attributes_bool"]; ok {
attributes := val.(*map[string]bool)
for key := range *attributes {
if strings.Contains(key, ".") {
uKey := strings.ReplaceAll(key, ".", "_")
delete(*attributes, uKey)
}
}
}
for _, k := range []string{"attributes_string", "resources_string"} {
if val, ok := row[k]; ok {
attributes := val.(*map[string]string)
for key := range *attributes {
if strings.Contains(key, ".") {
uKey := strings.ReplaceAll(key, ".", "_")
delete(*attributes, uKey)
}
}
}
}
}
func (r *ClickHouseReader) CheckClickHouse(ctx context.Context) error {
rows, err := r.db.Query(ctx, "SELECT 1")
if err != nil {
return err
}
defer rows.Close()
return nil
}
func (r *ClickHouseReader) GetTraceAggregateAttributes(ctx context.Context, req *v3.AggregateAttributeRequest) (*v3.AggregateAttributeResponse, error) {
var query string
var err error
var rows driver.Rows
var response v3.AggregateAttributeResponse
var stringAllowed bool
where := ""
switch req.Operator {
case
v3.AggregateOperatorCountDistinct,
v3.AggregateOperatorCount:
where = "tag_key ILIKE $1"
stringAllowed = true
case
v3.AggregateOperatorRateSum,
v3.AggregateOperatorRateMax,
v3.AggregateOperatorRateAvg,
v3.AggregateOperatorRate,
v3.AggregateOperatorRateMin,
v3.AggregateOperatorP05,
v3.AggregateOperatorP10,
v3.AggregateOperatorP20,
v3.AggregateOperatorP25,
v3.AggregateOperatorP50,
v3.AggregateOperatorP75,
v3.AggregateOperatorP90,
v3.AggregateOperatorP95,
v3.AggregateOperatorP99,
v3.AggregateOperatorAvg,
v3.AggregateOperatorSum,
v3.AggregateOperatorMin,
v3.AggregateOperatorMax:
where = "tag_key ILIKE $1 AND tag_data_type='float64'"
stringAllowed = false
case
v3.AggregateOperatorNoOp:
return &v3.AggregateAttributeResponse{}, nil
default:
return nil, fmt.Errorf("unsupported aggregate operator")
}
query = fmt.Sprintf("SELECT DISTINCT(tag_key), tag_type, tag_data_type FROM %s.%s WHERE %s and tag_type != 'spanfield'", r.TraceDB, r.spanAttributeTableV2, where)
if req.Limit != 0 {
query = query + fmt.Sprintf(" LIMIT %d;", req.Limit)
}
rows, err = r.db.Query(ctx, query, fmt.Sprintf("%%%s%%", req.SearchText))
if err != nil {
zap.L().Error("Error while executing query", zap.Error(err))
return nil, fmt.Errorf("error while executing query: %s", err.Error())
}
defer rows.Close()
statements := []model.ShowCreateTableStatement{}
query = fmt.Sprintf("SHOW CREATE TABLE %s.%s", r.TraceDB, r.traceLocalTableName)
err = r.db.Select(ctx, &statements, query)
if err != nil {
return nil, fmt.Errorf("error while fetching trace schema: %s", err.Error())
}
var tagKey string
var dataType string
var tagType string
for rows.Next() {
if err := rows.Scan(&tagKey, &tagType, &dataType); err != nil {
return nil, fmt.Errorf("error while scanning rows: %s", err.Error())
}
key := v3.AttributeKey{
Key: tagKey,
DataType: v3.AttributeKeyDataType(dataType),
Type: v3.AttributeKeyType(tagType),
IsColumn: isColumn(statements[0].Statement, tagType, tagKey, dataType),
}
if _, ok := constants.DeprecatedStaticFieldsTraces[tagKey]; !ok {
response.AttributeKeys = append(response.AttributeKeys, key)
}
}
fields := constants.NewStaticFieldsTraces
// add the new static fields
for _, field := range fields {
if (!stringAllowed && field.DataType == v3.AttributeKeyDataTypeString) || (v3.AttributeKey{} == field) {
continue
} else if len(req.SearchText) == 0 || strings.Contains(field.Key, req.SearchText) {
response.AttributeKeys = append(response.AttributeKeys, field)
}
}
return &response, nil
}
func (r *ClickHouseReader) GetTraceAttributeKeys(ctx context.Context, req *v3.FilterAttributeKeyRequest) (*v3.FilterAttributeKeyResponse, error) {
var query string
var err error
var rows driver.Rows
var response v3.FilterAttributeKeyResponse
tagTypeFilter := `tag_type != 'spanfield'`
if req.TagType != "" {
tagTypeFilter = fmt.Sprintf(`tag_type != 'spanfield' and tag_type = '%s'`, req.TagType)
}
query = fmt.Sprintf("SELECT DISTINCT(tag_key), tag_type, tag_data_type FROM %s.%s WHERE tag_key ILIKE $1 and %s LIMIT $2", r.TraceDB, r.spanAttributeTableV2, tagTypeFilter)
rows, err = r.db.Query(ctx, query, fmt.Sprintf("%%%s%%", req.SearchText), req.Limit)
if err != nil {
zap.L().Error("Error while executing query", zap.Error(err))
return nil, fmt.Errorf("error while executing query: %s", err.Error())
}
defer rows.Close()
statements := []model.ShowCreateTableStatement{}
query = fmt.Sprintf("SHOW CREATE TABLE %s.%s", r.TraceDB, r.traceLocalTableName)
err = r.db.Select(ctx, &statements, query)
if err != nil {
return nil, fmt.Errorf("error while fetching trace schema: %s", err.Error())
}
var tagKey string
var dataType string
var tagType string
for rows.Next() {
if err := rows.Scan(&tagKey, &tagType, &dataType); err != nil {
return nil, fmt.Errorf("error while scanning rows: %s", err.Error())
}
key := v3.AttributeKey{
Key: tagKey,
DataType: v3.AttributeKeyDataType(dataType),
Type: v3.AttributeKeyType(tagType),
IsColumn: isColumn(statements[0].Statement, tagType, tagKey, dataType),
}
// don't send deprecated static fields
// this is added so that once the old tenants are moved to new schema,
// they old attributes are not sent to the frontend autocomplete
if _, ok := constants.DeprecatedStaticFieldsTraces[tagKey]; !ok {
response.AttributeKeys = append(response.AttributeKeys, key)
}
}
// remove this later just to have NewStaticFieldsTraces in the response
fields := constants.NewStaticFieldsTraces
// add the new static fields only when the tagType is not specified
// i.e retrieve all attributes
if req.TagType == "" {
for _, f := range fields {
if (v3.AttributeKey{} == f) {
continue
}
if len(req.SearchText) == 0 || strings.Contains(f.Key, req.SearchText) {
response.AttributeKeys = append(response.AttributeKeys, f)
}
}
}
return &response, nil
}
func (r *ClickHouseReader) GetTraceAttributeValues(ctx context.Context, req *v3.FilterAttributeValueRequest) (*v3.FilterAttributeValueResponse, error) {
var query string
var filterValueColumn string
var err error
var rows driver.Rows
var attributeValues v3.FilterAttributeValueResponse
// if dataType or tagType is not present return empty response
if len(req.FilterAttributeKeyDataType) == 0 || len(req.TagType) == 0 {
// add data type if it's a top level key
if k, ok := constants.StaticFieldsTraces[req.FilterAttributeKey]; ok {
req.FilterAttributeKeyDataType = k.DataType
} else {
return &v3.FilterAttributeValueResponse{}, nil
}
}
// if data type is bool, return true and false
if req.FilterAttributeKeyDataType == v3.AttributeKeyDataTypeBool {
return &v3.FilterAttributeValueResponse{
BoolAttributeValues: []bool{true, false},
}, nil
}
query = "SELECT DISTINCT"
switch req.FilterAttributeKeyDataType {
case v3.AttributeKeyDataTypeFloat64:
filterValueColumn = "number_value"
case v3.AttributeKeyDataTypeString:
filterValueColumn = "string_value"
}
searchText := fmt.Sprintf("%%%s%%", req.SearchText)
// check if the tagKey is a topLevelColumn
// here we are using StaticFieldsTraces instead of NewStaticFieldsTraces as we want to consider old columns as well.
if _, ok := constants.StaticFieldsTraces[req.FilterAttributeKey]; ok {
// query the column for the last 48 hours
filterValueColumnWhere := req.FilterAttributeKey
selectKey := req.FilterAttributeKey
if req.FilterAttributeKeyDataType != v3.AttributeKeyDataTypeString {
filterValueColumnWhere = fmt.Sprintf("toString(%s)", req.FilterAttributeKey)
selectKey = fmt.Sprintf("toInt64(%s)", req.FilterAttributeKey)
}
// TODO(nitya): remove 24 hour limit in future after checking the perf/resource implications
where := "timestamp >= toDateTime64(now() - INTERVAL 48 HOUR, 9) AND ts_bucket_start >= toUInt64(toUnixTimestamp(now() - INTERVAL 48 HOUR))"
query = fmt.Sprintf("SELECT DISTINCT %s FROM %s.%s WHERE %s AND %s ILIKE $1 LIMIT $2", selectKey, r.TraceDB, r.traceTableName, where, filterValueColumnWhere)
rows, err = r.db.Query(ctx, query, searchText, req.Limit)
} else {
filterValueColumnWhere := filterValueColumn
if req.FilterAttributeKeyDataType != v3.AttributeKeyDataTypeString {
filterValueColumnWhere = fmt.Sprintf("toString(%s)", filterValueColumn)
}
query = fmt.Sprintf("SELECT DISTINCT %s FROM %s.%s WHERE tag_key=$1 AND %s ILIKE $2 AND tag_type=$3 LIMIT $4", filterValueColumn, r.TraceDB, r.spanAttributeTableV2, filterValueColumnWhere)
rows, err = r.db.Query(ctx, query, req.FilterAttributeKey, searchText, req.TagType, req.Limit)
}
if err != nil {
zap.L().Error("Error while executing query", zap.Error(err))
return nil, fmt.Errorf("error while executing query: %s", err.Error())
}
defer rows.Close()
var strAttributeValue string
var float64AttributeValue sql.NullFloat64
for rows.Next() {
switch req.FilterAttributeKeyDataType {
case v3.AttributeKeyDataTypeFloat64:
if err := rows.Scan(&float64AttributeValue); err != nil {
return nil, fmt.Errorf("error while scanning rows: %s", err.Error())
}
if float64AttributeValue.Valid {
attributeValues.NumberAttributeValues = append(attributeValues.NumberAttributeValues, float64AttributeValue.Float64)
}
case v3.AttributeKeyDataTypeString:
if err := rows.Scan(&strAttributeValue); err != nil {
return nil, fmt.Errorf("error while scanning rows: %s", err.Error())
}
attributeValues.StringAttributeValues = append(attributeValues.StringAttributeValues, strAttributeValue)
}
}
if req.IncludeRelated {
relatedValues, _ := r.FetchRelatedValues(ctx, req)
attributeValues.RelatedValues = &v3.FilterAttributeValueResponse{
StringAttributeValues: relatedValues,
}
}
return &attributeValues, nil
}
func (r *ClickHouseReader) GetSpanAttributeKeys(ctx context.Context) (map[string]v3.AttributeKey, error) {
var query string
var err error
var rows driver.Rows
response := map[string]v3.AttributeKey{}
query = fmt.Sprintf("SELECT DISTINCT(tagKey), tagType, dataType FROM %s.%s", r.TraceDB, r.spanAttributesKeysTable)
rows, err = r.db.Query(ctx, query)
if err != nil {
zap.L().Error("Error while executing query", zap.Error(err))
return nil, fmt.Errorf("error while executing query: %s", err.Error())
}
defer rows.Close()
statements := []model.ShowCreateTableStatement{}
query = fmt.Sprintf("SHOW CREATE TABLE %s.%s", r.TraceDB, r.traceTableName)
err = r.db.Select(ctx, &statements, query)
if err != nil {
return nil, fmt.Errorf("error while fetching trace schema: %s", err.Error())
}
var tagKey string
var dataType string
var tagType string
for rows.Next() {
if err := rows.Scan(&tagKey, &tagType, &dataType); err != nil {
return nil, fmt.Errorf("error while scanning rows: %s", err.Error())
}
key := v3.AttributeKey{
Key: tagKey,
DataType: v3.AttributeKeyDataType(dataType),
Type: v3.AttributeKeyType(tagType),
IsColumn: isColumn(statements[0].Statement, tagType, tagKey, dataType),
}
name := tagKey + "##" + tagType + "##" + strings.ToLower(dataType)
response[name] = key
}
for _, key := range constants.StaticFieldsTraces {
name := key.Key + "##" + key.Type.String() + "##" + strings.ToLower(key.DataType.String())
response[name] = key
}
return response, nil
}
func (r *ClickHouseReader) LiveTailLogsV4(ctx context.Context, query string, timestampStart uint64, idStart string, client *model.LogsLiveTailClientV2) {
if timestampStart == 0 {
timestampStart = uint64(time.Now().UnixNano())
} else {
timestampStart = uint64(utils.GetEpochNanoSecs(int64(timestampStart)))
}
ticker := time.NewTicker(time.Duration(r.liveTailRefreshSeconds) * time.Second)
defer ticker.Stop()
for {
select {
case <-ctx.Done():
done := true
client.Done <- &done
zap.L().Debug("closing go routine : " + client.Name)
return
case <-ticker.C:
// get the new 100 logs as anything more older won't make sense
var tmpQuery string
bucketStart := (timestampStart / NANOSECOND) - 1800
// we have to form the query differently if the resource filters are used
if strings.Contains(query, r.logsResourceTableV2) {
tmpQuery = fmt.Sprintf("seen_at_ts_bucket_start >=%d)) AND ts_bucket_start >=%d AND timestamp >=%d", bucketStart, bucketStart, timestampStart)
} else {
tmpQuery = fmt.Sprintf("ts_bucket_start >=%d AND timestamp >=%d", bucketStart, timestampStart)
}
if idStart != "" {
tmpQuery = fmt.Sprintf("%s AND id > '%s'", tmpQuery, idStart)
}
// the reason we are doing desc is that we need the latest logs first
tmpQuery = query + tmpQuery + " order by timestamp desc, id desc limit 100"
// using the old structure since we can directly read it to the struct as use it.
response := []model.SignozLogV2{}
err := r.db.Select(ctx, &response, tmpQuery)
if err != nil {
zap.L().Error("Error while getting logs", zap.Error(err))
client.Error <- err
return
}
for i := len(response) - 1; i >= 0; i-- {
client.Logs <- &response[i]
if i == 0 {
timestampStart = response[i].Timestamp
idStart = response[i].ID
}
}
}
}
}
func (r *ClickHouseReader) LiveTailLogsV3(ctx context.Context, query string, timestampStart uint64, idStart string, client *model.LogsLiveTailClient) {
if timestampStart == 0 {
timestampStart = uint64(time.Now().UnixNano())
} else {
timestampStart = uint64(utils.GetEpochNanoSecs(int64(timestampStart)))
}
ticker := time.NewTicker(time.Duration(r.liveTailRefreshSeconds) * time.Second)
defer ticker.Stop()
for {
select {
case <-ctx.Done():
done := true
client.Done <- &done
zap.L().Debug("closing go routine : " + client.Name)
return
case <-ticker.C:
// get the new 100 logs as anything more older won't make sense
tmpQuery := fmt.Sprintf("timestamp >='%d'", timestampStart)
if idStart != "" {
tmpQuery = fmt.Sprintf("%s AND id > '%s'", tmpQuery, idStart)
}
// the reason we are doing desc is that we need the latest logs first
tmpQuery = query + tmpQuery + " order by timestamp desc, id desc limit 100"
// using the old structure since we can directly read it to the struct as use it.
response := []model.SignozLog{}
err := r.db.Select(ctx, &response, tmpQuery)
if err != nil {
zap.L().Error("Error while getting logs", zap.Error(err))
client.Error <- err
return
}
for i := len(response) - 1; i >= 0; i-- {
client.Logs <- &response[i]
if i == 0 {
timestampStart = response[i].Timestamp
idStart = response[i].ID
}
}
}
}
}
func (r *ClickHouseReader) AddRuleStateHistory(ctx context.Context, ruleStateHistory []model.RuleStateHistory) error {
var statement driver.Batch
var err error
defer func() {
if statement != nil {
statement.Abort()
}
}()
statement, err = r.db.PrepareBatch(ctx, fmt.Sprintf("INSERT INTO %s.%s (rule_id, rule_name, overall_state, overall_state_changed, state, state_changed, unix_milli, labels, fingerprint, value) VALUES ($1, $2, $3, $4, $5, $6, $7, $8, $9, $10)",
signozHistoryDBName, ruleStateHistoryTableName))
if err != nil {
return err
}
for _, history := range ruleStateHistory {
err = statement.Append(history.RuleID, history.RuleName, history.OverallState, history.OverallStateChanged, history.State, history.StateChanged, history.UnixMilli, history.Labels, history.Fingerprint, history.Value)
if err != nil {
return err
}
}
err = statement.Send()
if err != nil {
return err
}
return nil
}
func (r *ClickHouseReader) GetLastSavedRuleStateHistory(ctx context.Context, ruleID string) ([]model.RuleStateHistory, error) {
query := fmt.Sprintf("SELECT * FROM %s.%s WHERE rule_id = '%s' AND state_changed = true ORDER BY unix_milli DESC LIMIT 1 BY fingerprint",
signozHistoryDBName, ruleStateHistoryTableName, ruleID)
history := []model.RuleStateHistory{}
err := r.db.Select(ctx, &history, query)
if err != nil {
return nil, err
}
return history, nil
}
func (r *ClickHouseReader) ReadRuleStateHistoryByRuleID(
ctx context.Context, ruleID string, params *model.QueryRuleStateHistory) (*model.RuleStateTimeline, error) {
var conditions []string
conditions = append(conditions, fmt.Sprintf("rule_id = '%s'", ruleID))
conditions = append(conditions, fmt.Sprintf("unix_milli >= %d AND unix_milli < %d", params.Start, params.End))
if params.State != "" {
conditions = append(conditions, fmt.Sprintf("state = '%s'", params.State))
}
if params.Filters != nil && len(params.Filters.Items) != 0 {
for _, item := range params.Filters.Items {
toFormat := item.Value
op := v3.FilterOperator(strings.ToLower(strings.TrimSpace(string(item.Operator))))
if op == v3.FilterOperatorContains || op == v3.FilterOperatorNotContains {
toFormat = fmt.Sprintf("%%%s%%", toFormat)
}
fmtVal := utils.ClickHouseFormattedValue(toFormat)
switch op {
case v3.FilterOperatorEqual:
conditions = append(conditions, fmt.Sprintf("JSONExtractString(labels, '%s') = %s", item.Key.Key, fmtVal))
case v3.FilterOperatorNotEqual:
conditions = append(conditions, fmt.Sprintf("JSONExtractString(labels, '%s') != %s", item.Key.Key, fmtVal))
case v3.FilterOperatorIn:
conditions = append(conditions, fmt.Sprintf("JSONExtractString(labels, '%s') IN %s", item.Key.Key, fmtVal))
case v3.FilterOperatorNotIn:
conditions = append(conditions, fmt.Sprintf("JSONExtractString(labels, '%s') NOT IN %s", item.Key.Key, fmtVal))
case v3.FilterOperatorLike:
conditions = append(conditions, fmt.Sprintf("like(JSONExtractString(labels, '%s'), %s)", item.Key.Key, fmtVal))
case v3.FilterOperatorNotLike:
conditions = append(conditions, fmt.Sprintf("notLike(JSONExtractString(labels, '%s'), %s)", item.Key.Key, fmtVal))
case v3.FilterOperatorRegex:
conditions = append(conditions, fmt.Sprintf("match(JSONExtractString(labels, '%s'), %s)", item.Key.Key, fmtVal))
case v3.FilterOperatorNotRegex:
conditions = append(conditions, fmt.Sprintf("not match(JSONExtractString(labels, '%s'), %s)", item.Key.Key, fmtVal))
case v3.FilterOperatorGreaterThan:
conditions = append(conditions, fmt.Sprintf("JSONExtractString(labels, '%s') > %s", item.Key.Key, fmtVal))
case v3.FilterOperatorGreaterThanOrEq:
conditions = append(conditions, fmt.Sprintf("JSONExtractString(labels, '%s') >= %s", item.Key.Key, fmtVal))
case v3.FilterOperatorLessThan:
conditions = append(conditions, fmt.Sprintf("JSONExtractString(labels, '%s') < %s", item.Key.Key, fmtVal))
case v3.FilterOperatorLessThanOrEq:
conditions = append(conditions, fmt.Sprintf("JSONExtractString(labels, '%s') <= %s", item.Key.Key, fmtVal))
case v3.FilterOperatorContains:
conditions = append(conditions, fmt.Sprintf("like(JSONExtractString(labels, '%s'), %s)", item.Key.Key, fmtVal))
case v3.FilterOperatorNotContains:
conditions = append(conditions, fmt.Sprintf("notLike(JSONExtractString(labels, '%s'), %s)", item.Key.Key, fmtVal))
case v3.FilterOperatorExists:
conditions = append(conditions, fmt.Sprintf("has(JSONExtractKeys(labels), '%s')", item.Key.Key))
case v3.FilterOperatorNotExists:
conditions = append(conditions, fmt.Sprintf("not has(JSONExtractKeys(labels), '%s')", item.Key.Key))
default:
return nil, fmt.Errorf("unsupported filter operator")
}
}
}
whereClause := strings.Join(conditions, " AND ")
query := fmt.Sprintf("SELECT * FROM %s.%s WHERE %s ORDER BY unix_milli %s LIMIT %d OFFSET %d",
signozHistoryDBName, ruleStateHistoryTableName, whereClause, params.Order, params.Limit, params.Offset)
history := []model.RuleStateHistory{}
zap.L().Debug("rule state history query", zap.String("query", query))
err := r.db.Select(ctx, &history, query)
if err != nil {
zap.L().Error("Error while reading rule state history", zap.Error(err))
return nil, err
}
var total uint64
zap.L().Debug("rule state history total query", zap.String("query", fmt.Sprintf("SELECT count(*) FROM %s.%s WHERE %s",
signozHistoryDBName, ruleStateHistoryTableName, whereClause)))
err = r.db.QueryRow(ctx, fmt.Sprintf("SELECT count(*) FROM %s.%s WHERE %s",
signozHistoryDBName, ruleStateHistoryTableName, whereClause)).Scan(&total)
if err != nil {
return nil, err
}
labelsQuery := fmt.Sprintf("SELECT DISTINCT labels FROM %s.%s WHERE rule_id = $1",
signozHistoryDBName, ruleStateHistoryTableName)
rows, err := r.db.Query(ctx, labelsQuery, ruleID)
if err != nil {
return nil, err
}
defer rows.Close()
labelsMap := make(map[string][]string)
for rows.Next() {
var rawLabel string
err = rows.Scan(&rawLabel)
if err != nil {
return nil, err
}
label := map[string]string{}
err = json.Unmarshal([]byte(rawLabel), &label)
if err != nil {
return nil, err
}
for k, v := range label {
labelsMap[k] = append(labelsMap[k], v)
}
}
timeline := &model.RuleStateTimeline{
Items: history,
Total: total,
Labels: labelsMap,
}
return timeline, nil
}
func (r *ClickHouseReader) ReadRuleStateHistoryTopContributorsByRuleID(
ctx context.Context, ruleID string, params *model.QueryRuleStateHistory) ([]model.RuleStateHistoryContributor, error) {
query := fmt.Sprintf(`SELECT
fingerprint,
any(labels) as labels,
count(*) as count
FROM %s.%s
WHERE rule_id = '%s' AND (state_changed = true) AND (state = '%s') AND unix_milli >= %d AND unix_milli <= %d
GROUP BY fingerprint
HAVING labels != '{}'
ORDER BY count DESC`,
signozHistoryDBName, ruleStateHistoryTableName, ruleID, model.StateFiring.String(), params.Start, params.End)
zap.L().Debug("rule state history top contributors query", zap.String("query", query))
contributors := []model.RuleStateHistoryContributor{}
err := r.db.Select(ctx, &contributors, query)
if err != nil {
zap.L().Error("Error while reading rule state history", zap.Error(err))
return nil, err
}
return contributors, nil
}
func (r *ClickHouseReader) GetOverallStateTransitions(ctx context.Context, ruleID string, params *model.QueryRuleStateHistory) ([]model.ReleStateItem, error) {
tmpl := `WITH firing_events AS (
SELECT
rule_id,
state,
unix_milli AS firing_time
FROM %s.%s
WHERE overall_state = '` + model.StateFiring.String() + `'
AND overall_state_changed = true
AND rule_id IN ('%s')
AND unix_milli >= %d AND unix_milli <= %d
),
resolution_events AS (
SELECT
rule_id,
state,
unix_milli AS resolution_time
FROM %s.%s
WHERE overall_state = '` + model.StateInactive.String() + `'
AND overall_state_changed = true
AND rule_id IN ('%s')
AND unix_milli >= %d AND unix_milli <= %d
),
matched_events AS (
SELECT
f.rule_id,
f.state,
f.firing_time,
MIN(r.resolution_time) AS resolution_time
FROM firing_events f
LEFT JOIN resolution_events r
ON f.rule_id = r.rule_id
WHERE r.resolution_time > f.firing_time
GROUP BY f.rule_id, f.state, f.firing_time
)
SELECT *
FROM matched_events
ORDER BY firing_time ASC;`
query := fmt.Sprintf(tmpl,
signozHistoryDBName, ruleStateHistoryTableName, ruleID, params.Start, params.End,
signozHistoryDBName, ruleStateHistoryTableName, ruleID, params.Start, params.End)
zap.L().Debug("overall state transitions query", zap.String("query", query))
transitions := []model.RuleStateTransition{}
err := r.db.Select(ctx, &transitions, query)
if err != nil {
return nil, err
}
stateItems := []model.ReleStateItem{}
for idx, item := range transitions {
start := item.FiringTime
end := item.ResolutionTime
stateItems = append(stateItems, model.ReleStateItem{
State: item.State,
Start: start,
End: end,
})
if idx < len(transitions)-1 {
nextStart := transitions[idx+1].FiringTime
if nextStart > end {
stateItems = append(stateItems, model.ReleStateItem{
State: model.StateInactive,
Start: end,
End: nextStart,
})
}
}
}
// fetch the most recent overall_state from the table
var state model.AlertState
stateQuery := fmt.Sprintf("SELECT state FROM %s.%s WHERE rule_id = '%s' AND unix_milli <= %d ORDER BY unix_milli DESC LIMIT 1",
signozHistoryDBName, ruleStateHistoryTableName, ruleID, params.End)
if err := r.db.QueryRow(ctx, stateQuery).Scan(&state); err != nil {
if err != sql.ErrNoRows {
return nil, err
}
state = model.StateInactive
}
if len(transitions) == 0 {
// no transitions found, it is either firing or inactive for whole time range
stateItems = append(stateItems, model.ReleStateItem{
State: state,
Start: params.Start,
End: params.End,
})
} else {
// there were some transitions, we need to add the last state at the end
if state == model.StateInactive {
stateItems = append(stateItems, model.ReleStateItem{
State: model.StateInactive,
Start: transitions[len(transitions)-1].ResolutionTime,
End: params.End,
})
} else {
// fetch the most recent firing event from the table in the given time range
var firingTime int64
firingQuery := fmt.Sprintf(`
SELECT
unix_milli
FROM %s.%s
WHERE rule_id = '%s' AND overall_state_changed = true AND overall_state = '%s' AND unix_milli <= %d
ORDER BY unix_milli DESC LIMIT 1`, signozHistoryDBName, ruleStateHistoryTableName, ruleID, model.StateFiring.String(), params.End)
if err := r.db.QueryRow(ctx, firingQuery).Scan(&firingTime); err != nil {
return nil, err
}
stateItems = append(stateItems, model.ReleStateItem{
State: model.StateInactive,
Start: transitions[len(transitions)-1].ResolutionTime,
End: firingTime,
})
stateItems = append(stateItems, model.ReleStateItem{
State: model.StateFiring,
Start: firingTime,
End: params.End,
})
}
}
return stateItems, nil
}
func (r *ClickHouseReader) GetAvgResolutionTime(ctx context.Context, ruleID string, params *model.QueryRuleStateHistory) (float64, error) {
tmpl := `
WITH firing_events AS (
SELECT
rule_id,
state,
unix_milli AS firing_time
FROM %s.%s
WHERE overall_state = '` + model.StateFiring.String() + `'
AND overall_state_changed = true
AND rule_id IN ('%s')
AND unix_milli >= %d AND unix_milli <= %d
),
resolution_events AS (
SELECT
rule_id,
state,
unix_milli AS resolution_time
FROM %s.%s
WHERE overall_state = '` + model.StateInactive.String() + `'
AND overall_state_changed = true
AND rule_id IN ('%s')
AND unix_milli >= %d AND unix_milli <= %d
),
matched_events AS (
SELECT
f.rule_id,
f.state,
f.firing_time,
MIN(r.resolution_time) AS resolution_time
FROM firing_events f
LEFT JOIN resolution_events r
ON f.rule_id = r.rule_id
WHERE r.resolution_time > f.firing_time
GROUP BY f.rule_id, f.state, f.firing_time
)
SELECT AVG(resolution_time - firing_time) / 1000 AS avg_resolution_time
FROM matched_events;
`
query := fmt.Sprintf(tmpl,
signozHistoryDBName, ruleStateHistoryTableName, ruleID, params.Start, params.End,
signozHistoryDBName, ruleStateHistoryTableName, ruleID, params.Start, params.End)
zap.L().Debug("avg resolution time query", zap.String("query", query))
var avgResolutionTime float64
err := r.db.QueryRow(ctx, query).Scan(&avgResolutionTime)
if err != nil {
return 0, err
}
return avgResolutionTime, nil
}
func (r *ClickHouseReader) GetAvgResolutionTimeByInterval(ctx context.Context, ruleID string, params *model.QueryRuleStateHistory) (*v3.Series, error) {
step := common.MinAllowedStepInterval(params.Start, params.End)
tmpl := `
WITH firing_events AS (
SELECT
rule_id,
state,
unix_milli AS firing_time
FROM %s.%s
WHERE overall_state = '` + model.StateFiring.String() + `'
AND overall_state_changed = true
AND rule_id IN ('%s')
AND unix_milli >= %d AND unix_milli <= %d
),
resolution_events AS (
SELECT
rule_id,
state,
unix_milli AS resolution_time
FROM %s.%s
WHERE overall_state = '` + model.StateInactive.String() + `'
AND overall_state_changed = true
AND rule_id IN ('%s')
AND unix_milli >= %d AND unix_milli <= %d
),
matched_events AS (
SELECT
f.rule_id,
f.state,
f.firing_time,
MIN(r.resolution_time) AS resolution_time
FROM firing_events f
LEFT JOIN resolution_events r
ON f.rule_id = r.rule_id
WHERE r.resolution_time > f.firing_time
GROUP BY f.rule_id, f.state, f.firing_time
)
SELECT toStartOfInterval(toDateTime(firing_time / 1000), INTERVAL %d SECOND) AS ts, AVG(resolution_time - firing_time) / 1000 AS avg_resolution_time
FROM matched_events
GROUP BY ts
ORDER BY ts ASC;`
query := fmt.Sprintf(tmpl,
signozHistoryDBName, ruleStateHistoryTableName, ruleID, params.Start, params.End,
signozHistoryDBName, ruleStateHistoryTableName, ruleID, params.Start, params.End, step)
zap.L().Debug("avg resolution time by interval query", zap.String("query", query))
result, err := r.GetTimeSeriesResultV3(ctx, query)
if err != nil || len(result) == 0 {
return nil, err
}
return result[0], nil
}
func (r *ClickHouseReader) GetTotalTriggers(ctx context.Context, ruleID string, params *model.QueryRuleStateHistory) (uint64, error) {
query := fmt.Sprintf("SELECT count(*) FROM %s.%s WHERE rule_id = '%s' AND (state_changed = true) AND (state = '%s') AND unix_milli >= %d AND unix_milli <= %d",
signozHistoryDBName, ruleStateHistoryTableName, ruleID, model.StateFiring.String(), params.Start, params.End)
var totalTriggers uint64
err := r.db.QueryRow(ctx, query).Scan(&totalTriggers)
if err != nil {
return 0, err
}
return totalTriggers, nil
}
func (r *ClickHouseReader) GetTriggersByInterval(ctx context.Context, ruleID string, params *model.QueryRuleStateHistory) (*v3.Series, error) {
step := common.MinAllowedStepInterval(params.Start, params.End)
query := fmt.Sprintf("SELECT count(*), toStartOfInterval(toDateTime(intDiv(unix_milli, 1000)), INTERVAL %d SECOND) as ts FROM %s.%s WHERE rule_id = '%s' AND (state_changed = true) AND (state = '%s') AND unix_milli >= %d AND unix_milli <= %d GROUP BY ts ORDER BY ts ASC",
step, signozHistoryDBName, ruleStateHistoryTableName, ruleID, model.StateFiring.String(), params.Start, params.End)
result, err := r.GetTimeSeriesResultV3(ctx, query)
if err != nil || len(result) == 0 {
return nil, err
}
return result[0], nil
}
func (r *ClickHouseReader) GetMinAndMaxTimestampForTraceID(ctx context.Context, traceID []string) (int64, int64, error) {
var minTime, maxTime time.Time
query := fmt.Sprintf("SELECT min(timestamp), max(timestamp) FROM %s.%s WHERE traceID IN ('%s')",
r.TraceDB, r.SpansTable, strings.Join(traceID, "','"))
zap.L().Debug("GetMinAndMaxTimestampForTraceID", zap.String("query", query))
err := r.db.QueryRow(ctx, query).Scan(&minTime, &maxTime)
if err != nil {
zap.L().Error("Error while executing query", zap.Error(err))
return 0, 0, err
}
// return current time if traceID not found
if minTime.IsZero() || maxTime.IsZero() {
zap.L().Debug("minTime or maxTime is zero, traceID not found")
return time.Now().UnixNano(), time.Now().UnixNano(), nil
}
zap.L().Debug("GetMinAndMaxTimestampForTraceID", zap.Any("minTime", minTime), zap.Any("maxTime", maxTime))
return minTime.UnixNano(), maxTime.UnixNano(), nil
}
func (r *ClickHouseReader) ReportQueryStartForProgressTracking(
queryId string,
) (func(), *model.ApiError) {
return r.queryProgressTracker.ReportQueryStarted(queryId)
}
func (r *ClickHouseReader) SubscribeToQueryProgress(
queryId string,
) (<-chan model.QueryProgress, func(), *model.ApiError) {
return r.queryProgressTracker.SubscribeToQueryProgress(queryId)
}
func (r *ClickHouseReader) GetAllMetricFilterAttributeKeys(ctx context.Context, req *metrics_explorer.FilterKeyRequest) (*[]v3.AttributeKey, *model.ApiError) {
var rows driver.Rows
var response []v3.AttributeKey
normalized := true
if constants.IsDotMetricsEnabled {
normalized = false
}
query := fmt.Sprintf("SELECT arrayJoin(tagKeys) AS distinctTagKey FROM (SELECT JSONExtractKeys(labels) AS tagKeys FROM %s.%s WHERE unix_milli >= $1 and __normalized = $2 GROUP BY tagKeys) WHERE distinctTagKey ILIKE $3 AND distinctTagKey NOT LIKE '\\_\\_%%' GROUP BY distinctTagKey", signozMetricDBName, signozTSTableNameV41Day)
if req.Limit != 0 {
query = query + fmt.Sprintf(" LIMIT %d;", req.Limit)
}
valueCtx := context.WithValue(ctx, "clickhouse_max_threads", constants.MetricsExplorerClickhouseThreads)
rows, err := r.db.Query(valueCtx, query, common.PastDayRoundOff(), normalized, fmt.Sprintf("%%%s%%", req.SearchText)) //only showing past day data
if err != nil {
zap.L().Error("Error while executing query", zap.Error(err))
return nil, &model.ApiError{Typ: "ClickHouseError", Err: err}
}
var attributeKey string
for rows.Next() {
if err := rows.Scan(&attributeKey); err != nil {
return nil, &model.ApiError{Typ: "ClickHouseError", Err: err}
}
key := v3.AttributeKey{
Key: attributeKey,
DataType: v3.AttributeKeyDataTypeString, // https://github.com/OpenObservability/OpenMetrics/blob/main/proto/openmetrics_data_model.proto#L64-L72.
Type: v3.AttributeKeyTypeTag,
IsColumn: false,
}
response = append(response, key)
}
if err := rows.Err(); err != nil {
return nil, &model.ApiError{Typ: "ClickHouseError", Err: err}
}
return &response, nil
}
func (r *ClickHouseReader) GetAllMetricFilterAttributeValues(ctx context.Context, req *metrics_explorer.FilterValueRequest) ([]string, *model.ApiError) {
var query string
var err error
var rows driver.Rows
var attributeValues []string
normalized := true
if constants.IsDotMetricsEnabled {
normalized = false
}
query = fmt.Sprintf("SELECT JSONExtractString(labels, $1) AS tagValue FROM %s.%s WHERE JSONExtractString(labels, $2) ILIKE $3 AND unix_milli >= $4 AND __normalized = $5 GROUP BY tagValue", signozMetricDBName, signozTSTableNameV41Day)
if req.Limit != 0 {
query = query + fmt.Sprintf(" LIMIT %d;", req.Limit)
}
valueCtx := context.WithValue(ctx, "clickhouse_max_threads", constants.MetricsExplorerClickhouseThreads)
rows, err = r.db.Query(valueCtx, query, req.FilterKey, req.FilterKey, fmt.Sprintf("%%%s%%", req.SearchText), common.PastDayRoundOff(), normalized) //only showing past day data
if err != nil {
zap.L().Error("Error while executing query", zap.Error(err))
return nil, &model.ApiError{Typ: "ClickHouseError", Err: err}
}
defer rows.Close()
var atrributeValue string
for rows.Next() {
if err := rows.Scan(&atrributeValue); err != nil {
return nil, &model.ApiError{Typ: "ClickHouseError", Err: err}
}
attributeValues = append(attributeValues, atrributeValue)
}
if err := rows.Err(); err != nil {
return nil, &model.ApiError{Typ: "ClickHouseError", Err: err}
}
return attributeValues, nil
}
func (r *ClickHouseReader) GetAllMetricFilterUnits(ctx context.Context, req *metrics_explorer.FilterValueRequest) ([]string, *model.ApiError) {
var rows driver.Rows
var response []string
query := fmt.Sprintf("SELECT DISTINCT unit FROM %s.%s WHERE unit ILIKE $1 AND unit IS NOT NULL ORDER BY unit", signozMetricDBName, signozTSTableNameV41Day)
if req.Limit != 0 {
query = query + fmt.Sprintf(" LIMIT %d;", req.Limit)
}
valueCtx := context.WithValue(ctx, "clickhouse_max_threads", constants.MetricsExplorerClickhouseThreads)
rows, err := r.db.Query(valueCtx, query, fmt.Sprintf("%%%s%%", req.SearchText))
if err != nil {
zap.L().Error("Error while executing query", zap.Error(err))
return nil, &model.ApiError{Typ: "ClickHouseError", Err: err}
}
var attributeKey string
for rows.Next() {
if err := rows.Scan(&attributeKey); err != nil {
return nil, &model.ApiError{Typ: "ClickHouseError", Err: err}
}
response = append(response, attributeKey)
}
if err := rows.Err(); err != nil {
return nil, &model.ApiError{Typ: "ClickHouseError", Err: err}
}
return response, nil
}
func (r *ClickHouseReader) GetAllMetricFilterTypes(ctx context.Context, req *metrics_explorer.FilterValueRequest) ([]string, *model.ApiError) {
var rows driver.Rows
var response []string
query := fmt.Sprintf("SELECT DISTINCT type FROM %s.%s WHERE type ILIKE $1 AND type IS NOT NULL ORDER BY type", signozMetricDBName, signozTSTableNameV41Day)
if req.Limit != 0 {
query = query + fmt.Sprintf(" LIMIT %d;", req.Limit)
}
valueCtx := context.WithValue(ctx, "clickhouse_max_threads", constants.MetricsExplorerClickhouseThreads)
rows, err := r.db.Query(valueCtx, query, fmt.Sprintf("%%%s%%", req.SearchText))
if err != nil {
zap.L().Error("Error while executing query", zap.Error(err))
return nil, &model.ApiError{Typ: "ClickHouseError", Err: err}
}
var attributeKey string
for rows.Next() {
if err := rows.Scan(&attributeKey); err != nil {
return nil, &model.ApiError{Typ: "ClickHouseError", Err: err}
}
response = append(response, attributeKey)
}
if err := rows.Err(); err != nil {
return nil, &model.ApiError{Typ: "ClickHouseError", Err: err}
}
return response, nil
}
func (r *ClickHouseReader) GetMetricsDataPoints(ctx context.Context, metricName string) (uint64, *model.ApiError) {
query := fmt.Sprintf(`SELECT
sum(count) as data_points
FROM %s.%s
WHERE metric_name = ?
`, signozMetricDBName, constants.SIGNOZ_SAMPLES_V4_AGG_30M_TABLENAME)
var dataPoints uint64
valueCtx := context.WithValue(ctx, "clickhouse_max_threads", constants.MetricsExplorerClickhouseThreads)
err := r.db.QueryRow(valueCtx, query, metricName).Scan(&dataPoints)
if err != nil {
return 0, &model.ApiError{Typ: "ClickHouseError", Err: err}
}
return dataPoints, nil // Convert to uint64 before returning
}
func (r *ClickHouseReader) GetMetricsLastReceived(ctx context.Context, metricName string) (int64, *model.ApiError) {
query := fmt.Sprintf(`SELECT
MAX(unix_milli) AS last_received_time
FROM %s.%s
WHERE metric_name = ?
`, signozMetricDBName, signozSamplesAgg30mLocalTableName)
var lastReceived int64
valueCtx := context.WithValue(ctx, "clickhouse_max_threads", constants.MetricsExplorerClickhouseThreads)
err := r.db.QueryRow(valueCtx, query, metricName).Scan(&lastReceived)
if err != nil {
return 0, &model.ApiError{Typ: "ClickHouseError", Err: err}
}
query = fmt.Sprintf(`SELECT
MAX(unix_milli) AS last_received_time
FROM %s.%s
WHERE metric_name = ? and unix_milli > ?
`, signozMetricDBName, signozSampleTableName)
var finalLastReceived int64
err = r.db.QueryRow(valueCtx, query, metricName, lastReceived).Scan(&finalLastReceived)
if err != nil {
return 0, &model.ApiError{Typ: "ClickHouseError", Err: err}
}
return finalLastReceived, nil // Convert to uint64 before returning
}
func (r *ClickHouseReader) GetTotalTimeSeriesForMetricName(ctx context.Context, metricName string) (uint64, *model.ApiError) {
query := fmt.Sprintf(`SELECT
uniq(fingerprint) AS timeSeriesCount
FROM %s.%s
WHERE metric_name = ?;`, signozMetricDBName, signozTSTableNameV41Week)
var timeSeriesCount uint64
valueCtx := context.WithValue(ctx, "clickhouse_max_threads", constants.MetricsExplorerClickhouseThreads)
err := r.db.QueryRow(valueCtx, query, metricName).Scan(&timeSeriesCount)
if err != nil {
return 0, &model.ApiError{Typ: "ClickHouseError", Err: err}
}
return timeSeriesCount, nil
}
func (r *ClickHouseReader) GetAttributesForMetricName(ctx context.Context, metricName string, start, end *int64, filters *v3.FilterSet) (*[]metrics_explorer.Attribute, *model.ApiError) {
whereClause := ""
if filters != nil {
conditions, _ := utils.BuildFilterConditions(filters, "t")
if conditions != nil {
whereClause = "AND " + strings.Join(conditions, " AND ")
}
}
normalized := true
if constants.IsDotMetricsEnabled {
normalized = false
}
const baseQueryTemplate = `
SELECT
kv.1 AS key,
arrayMap(x -> trim(BOTH '"' FROM x), groupUniqArray(1000)(kv.2)) AS values,
length(groupUniqArray(10000)(kv.2)) AS valueCount
FROM %s.%s
ARRAY JOIN arrayFilter(x -> NOT startsWith(x.1, '__'), JSONExtractKeysAndValuesRaw(labels)) AS kv
WHERE metric_name = ? AND __normalized=? %s`
var args []interface{}
args = append(args, metricName)
tableName := signozTSTableNameV41Week
args = append(args, normalized)
if start != nil && end != nil {
st, en, tsTable, _ := utils.WhichTSTableToUse(*start, *end)
*start, *end, tableName = st, en, tsTable
args = append(args, *start, *end)
} else if start == nil && end == nil {
tableName = signozTSTableNameV41Week
}
query := fmt.Sprintf(baseQueryTemplate, signozMetricDBName, tableName, whereClause)
if start != nil && end != nil {
query += " AND unix_milli BETWEEN ? AND ?"
}
query += "\nGROUP BY kv.1\nORDER BY valueCount DESC;"
valueCtx := context.WithValue(ctx, "clickhouse_max_threads", constants.MetricsExplorerClickhouseThreads)
rows, err := r.db.Query(valueCtx, query, args...)
if err != nil {
return nil, &model.ApiError{Typ: "ClickHouseError", Err: err}
}
defer rows.Close()
var attributesList []metrics_explorer.Attribute
for rows.Next() {
var attr metrics_explorer.Attribute
if err := rows.Scan(&attr.Key, &attr.Value, &attr.ValueCount); err != nil {
return nil, &model.ApiError{Typ: "ClickHouseError", Err: err}
}
attributesList = append(attributesList, attr)
}
if err := rows.Err(); err != nil {
return nil, &model.ApiError{Typ: "ClickHouseError", Err: err}
}
return &attributesList, nil
}
func (r *ClickHouseReader) GetActiveTimeSeriesForMetricName(ctx context.Context, metricName string, duration time.Duration) (uint64, *model.ApiError) {
milli := time.Now().Add(-duration).UnixMilli()
query := fmt.Sprintf("SELECT uniq(fingerprint) FROM %s.%s WHERE metric_name = '%s' and unix_milli >= ?", signozMetricDBName, signozTSTableNameV4, metricName)
var timeSeries uint64
// Using QueryRow instead of Select since we're only expecting a single value
valueCtx := context.WithValue(ctx, "clickhouse_max_threads", constants.MetricsExplorerClickhouseThreads)
err := r.db.QueryRow(valueCtx, query, milli).Scan(&timeSeries)
if err != nil {
return 0, &model.ApiError{Typ: "ClickHouseError", Err: err}
}
return timeSeries, nil
}
func (r *ClickHouseReader) ListSummaryMetrics(ctx context.Context, orgID valuer.UUID, req *metrics_explorer.SummaryListMetricsRequest) (*metrics_explorer.SummaryListMetricsResponse, *model.ApiError) {
var args []interface{}
// Build filter conditions (if any)
conditions, _ := utils.BuildFilterConditions(&req.Filters, "t")
whereClause := ""
if conditions != nil {
whereClause = "AND " + strings.Join(conditions, " AND ")
}
firstQueryLimit := req.Limit
samplesOrder := false
var orderByClauseFirstQuery string
if req.OrderBy.ColumnName == "samples" {
samplesOrder = true
orderByClauseFirstQuery = fmt.Sprintf("ORDER BY timeseries %s", req.OrderBy.Order)
if req.Limit < 50 {
firstQueryLimit = 50
}
} else {
orderByClauseFirstQuery = fmt.Sprintf("ORDER BY %s %s", req.OrderBy.ColumnName, req.OrderBy.Order)
}
normalized := true
if constants.IsDotMetricsEnabled {
normalized = false
}
// Determine which tables to use
start, end, tsTable, localTsTable := utils.WhichTSTableToUse(req.Start, req.End)
sampleTable, countExp := utils.WhichSampleTableToUse(req.Start, req.End)
metricsQuery := fmt.Sprintf(
`SELECT
t.metric_name AS metric_name,
ANY_VALUE(t.description) AS description,
ANY_VALUE(t.type) AS metric_type,
ANY_VALUE(t.unit) AS metric_unit,
uniq(t.fingerprint) AS timeseries,
uniq(metric_name) OVER() AS total
FROM %s.%s AS t
WHERE unix_milli BETWEEN ? AND ?
AND NOT startsWith(metric_name, 'signoz')
AND __normalized = ?
%s
GROUP BY t.metric_name
%s
LIMIT %d OFFSET %d;`,
signozMetricDBName, tsTable, whereClause, orderByClauseFirstQuery, firstQueryLimit, req.Offset)
args = append(args, start, end)
args = append(args, normalized)
valueCtx := context.WithValue(ctx, "clickhouse_max_threads", constants.MetricsExplorerClickhouseThreads)
begin := time.Now()
rows, err := r.db.Query(valueCtx, metricsQuery, args...)
queryDuration := time.Since(begin)
zap.L().Info("Time taken to execute metrics query to fetch metrics with high time series", zap.String("query", metricsQuery), zap.Any("args", args), zap.Duration("duration", queryDuration))
if err != nil {
zap.L().Error("Error executing metrics query", zap.Error(err))
return &metrics_explorer.SummaryListMetricsResponse{}, &model.ApiError{Typ: "ClickHouseError", Err: err}
}
defer rows.Close()
var response metrics_explorer.SummaryListMetricsResponse
var metricNames []string
for rows.Next() {
var metric metrics_explorer.MetricDetail
if err := rows.Scan(&metric.MetricName, &metric.Description, &metric.MetricType, &metric.MetricUnit, &metric.TimeSeries, &response.Total); err != nil {
zap.L().Error("Error scanning metric row", zap.Error(err))
return &response, &model.ApiError{Typ: "ClickHouseError", Err: err}
}
metricNames = append(metricNames, metric.MetricName)
response.Metrics = append(response.Metrics, metric)
}
if err := rows.Err(); err != nil {
zap.L().Error("Error iterating over metric rows", zap.Error(err))
return &response, &model.ApiError{Typ: "ClickHouseError", Err: err}
}
// If no metrics were found, return early.
if len(metricNames) == 0 {
return &response, nil
}
// Build a comma-separated list of quoted metric names.
metricsList := "'" + strings.Join(metricNames, "', '") + "'"
// If samples are being sorted by datapoints, update the ORDER clause.
if samplesOrder {
orderByClauseFirstQuery = fmt.Sprintf("ORDER BY s.samples %s", req.OrderBy.Order)
} else {
orderByClauseFirstQuery = ""
}
args = make([]interface{}, 0)
var sampleQuery string
var sb strings.Builder
if whereClause != "" {
sb.WriteString(fmt.Sprintf(
`SELECT
s.samples,
s.metric_name
FROM (
SELECT
dm.metric_name,
%s AS samples
FROM %s.%s AS dm
WHERE dm.metric_name IN (%s)
AND dm.fingerprint IN (
SELECT fingerprint
FROM %s.%s
WHERE metric_name IN (%s)
AND __normalized = ?
AND unix_milli BETWEEN ? AND ?
%s
GROUP BY fingerprint
)
AND dm.unix_milli BETWEEN ? AND ?
GROUP BY dm.metric_name
) AS s `,
countExp,
signozMetricDBName, sampleTable,
metricsList,
signozMetricDBName, localTsTable,
metricsList,
whereClause,
))
args = append(args, normalized)
args = append(args, start, end)
args = append(args, req.Start, req.End)
} else {
// If no filters, it is a simpler query.
sb.WriteString(fmt.Sprintf(
`SELECT
s.samples,
s.metric_name
FROM (
SELECT
metric_name,
%s AS samples
FROM %s.%s
WHERE metric_name IN (%s)
AND unix_milli BETWEEN ? AND ?
GROUP BY metric_name
) AS s `,
countExp,
signozMetricDBName, sampleTable,
metricsList))
args = append(args, req.Start, req.End)
}
// Append ORDER BY clause if provided.
if orderByClauseFirstQuery != "" {
sb.WriteString(orderByClauseFirstQuery + " ")
}
// Append LIMIT clause.
sb.WriteString(fmt.Sprintf("LIMIT %d;", req.Limit))
sampleQuery = sb.String()
begin = time.Now()
rows, err = r.db.Query(valueCtx, sampleQuery, args...)
queryDuration = time.Since(begin)
zap.L().Info("Time taken to execute list summary query", zap.String("query", sampleQuery), zap.Any("args", args), zap.Duration("duration", queryDuration))
if err != nil {
zap.L().Error("Error executing samples query", zap.Error(err))
return &response, &model.ApiError{Typ: "ClickHouseError", Err: err}
}
defer rows.Close()
samplesMap := make(map[string]uint64)
for rows.Next() {
var samples uint64
var metricName string
if err := rows.Scan(&samples, &metricName); err != nil {
zap.L().Error("Error scanning sample row", zap.Error(err))
return &response, &model.ApiError{Typ: "ClickHouseError", Err: err}
}
samplesMap[metricName] = samples
}
if err := rows.Err(); err != nil {
zap.L().Error("Error iterating over sample rows", zap.Error(err))
return &response, &model.ApiError{Typ: "ClickHouseError", Err: err}
}
//get updated metrics data
batch, apiError := r.GetUpdatedMetricsMetadata(ctx, orgID, metricNames...)
if apiError != nil {
zap.L().Error("Error in getting metrics cached metadata", zap.Error(apiError))
}
var filteredMetrics []metrics_explorer.MetricDetail
for i := range response.Metrics {
if updatedMetrics, exists := batch[response.Metrics[i].MetricName]; exists {
response.Metrics[i].MetricType = string(updatedMetrics.MetricType)
if updatedMetrics.Unit != "" {
response.Metrics[i].MetricUnit = updatedMetrics.Unit
}
if updatedMetrics.Description != "" {
response.Metrics[i].Description = updatedMetrics.Description
}
}
if samples, exists := samplesMap[response.Metrics[i].MetricName]; exists {
response.Metrics[i].Samples = samples
filteredMetrics = append(filteredMetrics, response.Metrics[i])
}
}
response.Metrics = filteredMetrics
// If ordering by samples, sort in-memory.
if samplesOrder {
sort.Slice(response.Metrics, func(i, j int) bool {
return response.Metrics[i].Samples > response.Metrics[j].Samples
})
}
return &response, nil
}
func (r *ClickHouseReader) GetMetricsTimeSeriesPercentage(ctx context.Context, req *metrics_explorer.TreeMapMetricsRequest) (*[]metrics_explorer.TreeMapResponseItem, *model.ApiError) {
var args []interface{}
normalized := true
if constants.IsDotMetricsEnabled {
normalized = false
}
// Build filters dynamically
conditions, _ := utils.BuildFilterConditions(&req.Filters, "")
whereClause := ""
if len(conditions) > 0 {
whereClause = "AND " + strings.Join(conditions, " AND ")
}
start, end, tsTable, _ := utils.WhichTSTableToUse(req.Start, req.End)
// Construct the query without backticks
query := fmt.Sprintf(`
SELECT
metric_name,
total_value,
(total_value * 100.0 / total_time_series) AS percentage
FROM (
SELECT
metric_name,
uniq(fingerprint) AS total_value,
(SELECT uniq(fingerprint)
FROM %s.%s
WHERE unix_milli BETWEEN ? AND ? AND __normalized = ?) AS total_time_series
FROM %s.%s
WHERE unix_milli BETWEEN ? AND ? AND NOT startsWith(metric_name, 'signoz') AND __normalized = ? %s
GROUP BY metric_name
)
ORDER BY percentage DESC
LIMIT %d;`,
signozMetricDBName,
tsTable,
signozMetricDBName,
tsTable,
whereClause,
req.Limit,
)
args = append(args,
start, end,
normalized, // For total_time_series subquery
start, end, // For main query
normalized,
)
valueCtx := context.WithValue(ctx, "clickhouse_max_threads", constants.MetricsExplorerClickhouseThreads)
begin := time.Now()
rows, err := r.db.Query(valueCtx, query, args...)
duration := time.Since(begin)
zap.L().Info("Time taken to execute time series percentage query", zap.String("query", query), zap.Any("args", args), zap.Duration("duration", duration))
if err != nil {
zap.L().Error("Error executing time series percentage query", zap.Error(err), zap.String("query", query))
return nil, &model.ApiError{Typ: "ClickHouseError", Err: err}
}
defer rows.Close()
var treemap []metrics_explorer.TreeMapResponseItem
for rows.Next() {
var item metrics_explorer.TreeMapResponseItem
if err := rows.Scan(&item.MetricName, &item.TotalValue, &item.Percentage); err != nil {
zap.L().Error("Error scanning row", zap.Error(err))
return nil, &model.ApiError{Typ: "ClickHouseError", Err: err}
}
treemap = append(treemap, item)
}
if err := rows.Err(); err != nil {
zap.L().Error("Error iterating over rows", zap.Error(err))
return nil, &model.ApiError{Typ: "ClickHouseError", Err: err}
}
return &treemap, nil
}
func (r *ClickHouseReader) GetMetricsSamplesPercentage(ctx context.Context, req *metrics_explorer.TreeMapMetricsRequest) (*[]metrics_explorer.TreeMapResponseItem, *model.ApiError) {
conditions, _ := utils.BuildFilterConditions(&req.Filters, "ts")
whereClause := ""
if conditions != nil {
whereClause = "AND " + strings.Join(conditions, " AND ")
}
normalized := true
if constants.IsDotMetricsEnabled {
normalized = false
}
// Determine time range and tables to use
start, end, tsTable, localTsTable := utils.WhichTSTableToUse(req.Start, req.End)
sampleTable, countExp := utils.WhichSampleTableToUse(req.Start, req.End)
queryLimit := 50 + req.Limit
metricsQuery := fmt.Sprintf(
`SELECT
ts.metric_name AS metric_name,
uniq(ts.fingerprint) AS timeSeries
FROM %s.%s AS ts
WHERE NOT startsWith(ts.metric_name, 'signoz_')
AND __normalized = ?
AND unix_milli BETWEEN ? AND ?
%s
GROUP BY ts.metric_name
ORDER BY timeSeries DESC
LIMIT %d;`,
signozMetricDBName, tsTable, whereClause, queryLimit,
)
valueCtx := context.WithValue(ctx, "clickhouse_max_threads", constants.MetricsExplorerClickhouseThreads)
begin := time.Now()
rows, err := r.db.Query(valueCtx, metricsQuery, normalized, start, end)
duration := time.Since(begin)
zap.L().Info("Time taken to execute samples percentage metric name query to reduce search space", zap.String("query", metricsQuery), zap.Any("start", start), zap.Any("end", end), zap.Duration("duration", duration))
if err != nil {
zap.L().Error("Error executing samples percentage query", zap.Error(err))
return nil, &model.ApiError{Typ: "ClickHouseError", Err: err}
}
defer rows.Close()
// Process the query results
var metricNames []string
for rows.Next() {
var metricName string
var timeSeries uint64
if err := rows.Scan(&metricName, &timeSeries); err != nil {
zap.L().Error("Error scanning metric row", zap.Error(err))
return nil, &model.ApiError{Typ: "ClickHouseError", Err: err}
}
metricNames = append(metricNames, metricName)
}
if err := rows.Err(); err != nil {
zap.L().Error("Error iterating over metric rows", zap.Error(err))
return nil, &model.ApiError{Typ: "ClickHouseError", Err: err}
}
// If no metrics found, return early
if len(metricNames) == 0 {
return nil, nil
}
// Format metric names for query
metricsList := "'" + strings.Join(metricNames, "', '") + "'"
// Build optimized query with JOIN but `unix_milli` filter only on the sample table
var sb strings.Builder
sb.WriteString(fmt.Sprintf(
`WITH TotalSamples AS (
SELECT %s AS total_samples
FROM %s.%s
WHERE unix_milli BETWEEN ? AND ?
)
SELECT
s.samples,
s.metric_name,
COALESCE((s.samples * 100.0 / t.total_samples), 0) AS percentage
FROM
(
SELECT
dm.metric_name,
%s AS samples
FROM %s.%s AS dm`,
countExp, signozMetricDBName, sampleTable, // Total samples
countExp, signozMetricDBName, sampleTable, // Inner select samples
))
var args []interface{}
args = append(args,
req.Start, req.End, // For total_samples subquery
)
// Apply `unix_milli` filter **only** on the sample table (`dm`)
sb.WriteString(` WHERE dm.unix_milli BETWEEN ? AND ?`)
args = append(args, req.Start, req.End)
// Use JOIN instead of IN (subquery) when additional filters exist
if whereClause != "" {
sb.WriteString(fmt.Sprintf(
` AND dm.fingerprint IN (
SELECT ts.fingerprint
FROM %s.%s AS ts
WHERE ts.metric_name IN (%s)
AND unix_milli BETWEEN ? AND ?
AND __normalized = ?
%s
GROUP BY ts.fingerprint
)`,
signozMetricDBName, localTsTable, metricsList, whereClause,
))
args = append(args, start, end, normalized)
}
// Apply metric filtering after all conditions
sb.WriteString(fmt.Sprintf(
` AND dm.metric_name IN (%s)
GROUP BY dm.metric_name
) AS s
JOIN TotalSamples t ON 1 = 1
ORDER BY percentage DESC
LIMIT ?;`,
metricsList,
))
args = append(args, req.Limit)
sampleQuery := sb.String()
// Add start and end time to args (only for sample table)
begin = time.Now()
// Execute the sample percentage query
rows, err = r.db.Query(valueCtx, sampleQuery, args...)
duration = time.Since(begin)
zap.L().Info("Time taken to execute samples percentage query", zap.String("query", sampleQuery), zap.Any("args", args), zap.Duration("duration", duration))
if err != nil {
zap.L().Error("Error executing samples query", zap.Error(err))
return nil, &model.ApiError{Typ: "ClickHouseError", Err: err}
}
defer rows.Close()
// Process the results into a response slice
var treemap []metrics_explorer.TreeMapResponseItem
for rows.Next() {
var item metrics_explorer.TreeMapResponseItem
if err := rows.Scan(&item.TotalValue, &item.MetricName, &item.Percentage); err != nil {
zap.L().Error("Error scanning row", zap.Error(err))
return nil, &model.ApiError{Typ: "ClickHouseError", Err: err}
}
treemap = append(treemap, item)
}
if err := rows.Err(); err != nil {
zap.L().Error("Error iterating over sample rows", zap.Error(err))
return nil, &model.ApiError{Typ: "ClickHouseError", Err: err}
}
return &treemap, nil
}
func (r *ClickHouseReader) GetNameSimilarity(ctx context.Context, req *metrics_explorer.RelatedMetricsRequest) (map[string]metrics_explorer.RelatedMetricsScore, *model.ApiError) {
start, end, tsTable, _ := utils.WhichTSTableToUse(req.Start, req.End)
normalized := true
if constants.IsDotMetricsEnabled {
normalized = false
}
query := fmt.Sprintf(`
SELECT
metric_name,
any(type) as type,
any(temporality) as temporality,
any(is_monotonic) as monotonic,
1 - (levenshteinDistance(?, metric_name) / greatest(NULLIF(length(?), 0), NULLIF(length(metric_name), 0))) AS name_similarity
FROM %s.%s
WHERE metric_name != ?
AND unix_milli BETWEEN ? AND ?
AND NOT startsWith(metric_name, 'signoz')
AND __normalized = ?
GROUP BY metric_name
ORDER BY name_similarity DESC
LIMIT 30;`,
signozMetricDBName, tsTable)
valueCtx := context.WithValue(ctx, "clickhouse_max_threads", constants.MetricsExplorerClickhouseThreads)
rows, err := r.db.Query(valueCtx, query, req.CurrentMetricName, req.CurrentMetricName, req.CurrentMetricName, start, end, normalized)
if err != nil {
return nil, &model.ApiError{Typ: "ClickHouseError", Err: err}
}
defer rows.Close()
result := make(map[string]metrics_explorer.RelatedMetricsScore)
for rows.Next() {
var metric string
var sim float64
var metricType v3.MetricType
var temporality v3.Temporality
var isMonotonic bool
if err := rows.Scan(&metric, &metricType, &temporality, &isMonotonic, &sim); err != nil {
return nil, &model.ApiError{Typ: "ClickHouseError", Err: err}
}
result[metric] = metrics_explorer.RelatedMetricsScore{
NameSimilarity: sim,
MetricType: metricType,
Temporality: temporality,
IsMonotonic: isMonotonic,
}
}
return result, nil
}
func (r *ClickHouseReader) GetAttributeSimilarity(ctx context.Context, req *metrics_explorer.RelatedMetricsRequest) (map[string]metrics_explorer.RelatedMetricsScore, *model.ApiError) {
start, end, tsTable, _ := utils.WhichTSTableToUse(req.Start, req.End)
normalized := true
if constants.IsDotMetricsEnabled {
normalized = false
}
// Get target labels
extractedLabelsQuery := fmt.Sprintf(`
SELECT
kv.1 AS label_key,
topK(10)(JSONExtractString(kv.2)) AS label_values
FROM %s.%s
ARRAY JOIN JSONExtractKeysAndValuesRaw(labels) AS kv
WHERE metric_name = ?
AND unix_milli between ? and ?
AND NOT startsWith(kv.1, '__')
AND NOT startsWith(metric_name, 'signoz_')
AND __normalized = ?
GROUP BY label_key
LIMIT 50`, signozMetricDBName, tsTable)
valueCtx := context.WithValue(ctx, "clickhouse_max_threads", constants.MetricsExplorerClickhouseThreads)
rows, err := r.db.Query(valueCtx, extractedLabelsQuery, req.CurrentMetricName, start, end, normalized)
if err != nil {
return nil, &model.ApiError{Typ: "ClickHouseError", Err: err}
}
defer rows.Close()
var targetKeys []string
var targetValues []string
for rows.Next() {
var key string
var value []string
if err := rows.Scan(&key, &value); err != nil {
return nil, &model.ApiError{Typ: "ClickHouseError", Err: err}
}
targetKeys = append(targetKeys, key)
targetValues = append(targetValues, value...)
}
targetKeysList := "'" + strings.Join(targetKeys, "', '") + "'"
targetValuesList := "'" + strings.Join(targetValues, "', '") + "'"
var priorityList []string
for _, f := range req.Filters.Items {
if f.Operator == v3.FilterOperatorEqual {
priorityList = append(priorityList, fmt.Sprintf("tuple('%s', '%s')", f.Key.Key, f.Value))
}
}
priorityListString := strings.Join(priorityList, ", ")
candidateLabelsQuery := fmt.Sprintf(`
WITH
arrayDistinct([%s]) AS filter_keys,
arrayDistinct([%s]) AS filter_values,
[%s] AS priority_pairs_input,
%d AS priority_multiplier
SELECT
metric_name,
any(type) as type,
any(temporality) as temporality,
any(is_monotonic) as monotonic,
SUM(
arraySum(
kv -> if(has(filter_keys, kv.1) AND has(filter_values, kv.2), 1, 0),
JSONExtractKeysAndValues(labels, 'String')
)
)::UInt64 AS raw_match_count,
SUM(
arraySum(
kv ->
if(
arrayExists(pr -> pr.1 = kv.1 AND pr.2 = kv.2, priority_pairs_input),
priority_multiplier,
0
),
JSONExtractKeysAndValues(labels, 'String')
)
)::UInt64 AS weighted_match_count,
toJSONString(
arrayDistinct(
arrayFlatten(
groupArray(
arrayFilter(
kv -> arrayExists(pr -> pr.1 = kv.1 AND pr.2 = kv.2, priority_pairs_input),
JSONExtractKeysAndValues(labels, 'String')
)
)
)
)
) AS priority_pairs
FROM %s.%s
WHERE rand() %% 100 < 10
AND unix_milli between ? and ?
AND NOT startsWith(metric_name, 'signoz_')
AND __normalized = ?
GROUP BY metric_name
ORDER BY weighted_match_count DESC, raw_match_count DESC
LIMIT 30
`,
targetKeysList, targetValuesList, priorityListString, 2,
signozMetricDBName, tsTable)
rows, err = r.db.Query(valueCtx, candidateLabelsQuery, start, end, normalized)
if err != nil {
return nil, &model.ApiError{Typ: "ClickHouseError", Err: err}
}
defer rows.Close()
result := make(map[string]metrics_explorer.RelatedMetricsScore)
attributeMap := make(map[string]uint64)
for rows.Next() {
var metric string
var metricType v3.MetricType
var temporality v3.Temporality
var isMonotonic bool
var weightedMatchCount, rawMatchCount uint64
var priorityPairsJSON string
if err := rows.Scan(&metric, &metricType, &temporality, &isMonotonic, &rawMatchCount, &weightedMatchCount, &priorityPairsJSON); err != nil {
return nil, &model.ApiError{Typ: "ClickHouseError", Err: err}
}
attributeMap[metric] = weightedMatchCount + (rawMatchCount)/10
var priorityPairs [][]string
if err := json.Unmarshal([]byte(priorityPairsJSON), &priorityPairs); err != nil {
priorityPairs = [][]string{}
}
result[metric] = metrics_explorer.RelatedMetricsScore{
AttributeSimilarity: float64(attributeMap[metric]), // Will be normalized later
Filters: priorityPairs,
MetricType: metricType,
Temporality: temporality,
IsMonotonic: isMonotonic,
}
}
if err := rows.Err(); err != nil {
return nil, &model.ApiError{Typ: "ClickHouseError", Err: err}
}
// Normalize the attribute similarity scores
normalizeMap := utils.NormalizeMap(attributeMap)
for metric := range result {
if score, exists := normalizeMap[metric]; exists {
metricScore := result[metric]
metricScore.AttributeSimilarity = score
result[metric] = metricScore
}
}
return result, nil
}
func (r *ClickHouseReader) GetMetricsAllResourceAttributes(ctx context.Context, start int64, end int64) (map[string]uint64, *model.ApiError) {
start, end, attTable, _ := utils.WhichAttributesTableToUse(start, end)
query := fmt.Sprintf(`SELECT
key,
count(distinct value) AS distinct_value_count
FROM (
SELECT key, value
FROM %s.%s
ARRAY JOIN
arrayConcat(mapKeys(resource_attributes)) AS key,
arrayConcat(mapValues(resource_attributes)) AS value
WHERE unix_milli between ? and ?
)
GROUP BY key
ORDER BY distinct_value_count DESC;`, signozMetadataDbName, attTable)
valueCtx := context.WithValue(ctx, "clickhouse_max_threads", constants.MetricsExplorerClickhouseThreads)
rows, err := r.db.Query(valueCtx, query, start, end)
if err != nil {
return nil, &model.ApiError{Typ: "ClickHouseError", Err: err}
}
attributes := make(map[string]uint64)
for rows.Next() {
var attrs string
var uniqCount uint64
if err := rows.Scan(&attrs, &uniqCount); err != nil {
return nil, &model.ApiError{Typ: "ClickHouseError", Err: err}
}
attributes[attrs] = uniqCount
}
if err := rows.Err(); err != nil {
return nil, &model.ApiError{Typ: "ClickHouseError", Err: err}
}
return attributes, nil
}
func (r *ClickHouseReader) GetInspectMetrics(ctx context.Context, req *metrics_explorer.InspectMetricsRequest, fingerprints []string) (*metrics_explorer.InspectMetricsResponse, *model.ApiError) {
start, end, _, localTsTable := utils.WhichTSTableToUse(req.Start, req.End)
fingerprintsString := strings.Join(fingerprints, ",")
query := fmt.Sprintf(`SELECT
fingerprint,
labels,
unix_milli,
value as per_series_value
FROM
signoz_metrics.distributed_samples_v4
INNER JOIN (
SELECT DISTINCT
fingerprint,
labels
FROM
%s.%s
WHERE
fingerprint in (%s)
AND unix_milli >= ?
AND unix_milli < ?) as filtered_time_series
USING fingerprint
WHERE
metric_name = ?
AND unix_milli >= ?
AND unix_milli < ?
ORDER BY fingerprint DESC, unix_milli DESC`, signozMetricDBName, localTsTable, fingerprintsString)
valueCtx := context.WithValue(ctx, "clickhouse_max_threads", constants.MetricsExplorerClickhouseThreads)
rows, err := r.db.Query(valueCtx, query, start, end, req.MetricName, start, end)
if err != nil {
return nil, &model.ApiError{Typ: "ClickHouseError", Err: err}
}
defer rows.Close()
seriesMap := make(map[uint64]*v3.Series)
for rows.Next() {
var fingerprint uint64
var labelsJSON string
var unixMilli int64
var perSeriesValue float64
if err := rows.Scan(&fingerprint, &labelsJSON, &unixMilli, &perSeriesValue); err != nil {
return nil, &model.ApiError{Typ: "ClickHouseError", Err: err}
}
var labelsMap map[string]string
if err := json.Unmarshal([]byte(labelsJSON), &labelsMap); err != nil {
return nil, &model.ApiError{Typ: "JsonUnmarshalError", Err: err}
}
// Filter out keys starting with "__"
filteredLabelsMap := make(map[string]string)
for k, v := range labelsMap {
if !strings.HasPrefix(k, "__") {
filteredLabelsMap[k] = v
}
}
var labelsArray []map[string]string
for k, v := range filteredLabelsMap {
labelsArray = append(labelsArray, map[string]string{k: v})
}
// Check if we already have a Series for this fingerprint.
series, exists := seriesMap[fingerprint]
if !exists {
series = &v3.Series{
Labels: filteredLabelsMap,
LabelsArray: labelsArray,
Points: []v3.Point{},
}
seriesMap[fingerprint] = series
}
series.Points = append(series.Points, v3.Point{
Timestamp: unixMilli,
Value: perSeriesValue,
})
}
if err = rows.Err(); err != nil {
return nil, &model.ApiError{Typ: "ClickHouseError", Err: err}
}
var seriesList []v3.Series
for _, s := range seriesMap {
seriesList = append(seriesList, *s)
}
return &metrics_explorer.InspectMetricsResponse{
Series: &seriesList,
}, nil
}
func (r *ClickHouseReader) GetInspectMetricsFingerprints(ctx context.Context, attributes []string, req *metrics_explorer.InspectMetricsRequest) ([]string, *model.ApiError) {
// Build dynamic key selections and JSON extracts
var jsonExtracts []string
var groupBys []string
for i, attr := range attributes {
keyAlias := fmt.Sprintf("key%d", i+1)
jsonExtracts = append(jsonExtracts, fmt.Sprintf("JSONExtractString(labels, '%s') AS %s", attr, keyAlias))
groupBys = append(groupBys, keyAlias)
}
conditions, _ := utils.BuildFilterConditions(&req.Filters, "")
whereClause := ""
if len(conditions) > 0 {
whereClause = "AND " + strings.Join(conditions, " AND ")
}
start, end, tsTable, _ := utils.WhichTSTableToUse(req.Start, req.End)
query := fmt.Sprintf(`
SELECT
arrayDistinct(groupArray(toString(fingerprint))) AS fingerprints
FROM
(
SELECT
metric_name, labels, fingerprint,
%s
FROM %s.%s
WHERE metric_name = ?
AND unix_milli BETWEEN ? AND ?
%s
)
GROUP BY %s
ORDER BY length(fingerprints) DESC, rand()
LIMIT 40`, // added rand to get diff value every time we run this query
strings.Join(jsonExtracts, ", "),
signozMetricDBName, tsTable,
whereClause,
strings.Join(groupBys, ", "))
valueCtx := context.WithValue(ctx, "clickhouse_max_threads", constants.MetricsExplorerClickhouseThreads)
rows, err := r.db.Query(valueCtx, query,
req.MetricName,
start,
end,
)
if err != nil {
return nil, &model.ApiError{Typ: model.ErrorExec, Err: err}
}
defer rows.Close()
var fingerprints []string
for rows.Next() {
// Create dynamic scanning based on number of attributes
var batch []string
if err := rows.Scan(&batch); err != nil {
return nil, &model.ApiError{Typ: model.ErrorExec, Err: err}
}
remaining := 40 - len(fingerprints)
if remaining <= 0 {
break
}
// if this batch would overshoot, only take as many as we need
if len(batch) > remaining {
fingerprints = append(fingerprints, batch[:remaining]...)
break
}
// otherwise take the whole batch and keep going
fingerprints = append(fingerprints, batch...)
}
if err := rows.Err(); err != nil {
return nil, &model.ApiError{Typ: model.ErrorExec, Err: err}
}
return fingerprints, nil
}
func (r *ClickHouseReader) DeleteMetricsMetadata(ctx context.Context, orgID valuer.UUID, metricName string) *model.ApiError {
delQuery := fmt.Sprintf(`ALTER TABLE %s.%s DELETE WHERE metric_name = ?;`, signozMetricDBName, signozUpdatedMetricsMetadataLocalTable)
valueCtx := context.WithValue(ctx, "clickhouse_max_threads", constants.MetricsExplorerClickhouseThreads)
err := r.db.Exec(valueCtx, delQuery, metricName)
if err != nil {
return &model.ApiError{Typ: "ClickHouseError", Err: err}
}
r.cache.Delete(ctx, orgID, constants.UpdatedMetricsMetadataCachePrefix+metricName)
return nil
}
func (r *ClickHouseReader) UpdateMetricsMetadata(ctx context.Context, orgID valuer.UUID, req *model.UpdateMetricsMetadata) *model.ApiError {
if req.MetricType == v3.MetricTypeHistogram {
labels := []string{"le"}
hasLabels, apiError := r.CheckForLabelsInMetric(ctx, req.MetricName, labels)
if apiError != nil {
return apiError
}
if !hasLabels {
return &model.ApiError{
Typ: model.ErrorBadData,
Err: fmt.Errorf("metric '%s' cannot be set as histogram type", req.MetricName),
}
}
}
if req.MetricType == v3.MetricTypeSummary {
labels := []string{"quantile"}
hasLabels, apiError := r.CheckForLabelsInMetric(ctx, req.MetricName, labels)
if apiError != nil {
return apiError
}
if !hasLabels {
return &model.ApiError{
Typ: model.ErrorBadData,
Err: fmt.Errorf("metric '%s' cannot be set as summary type", req.MetricName),
}
}
}
apiErr := r.DeleteMetricsMetadata(ctx, orgID, req.MetricName)
if apiErr != nil {
return apiErr
}
insertQuery := fmt.Sprintf(`INSERT INTO %s.%s (metric_name, temporality, is_monotonic, type, description, unit, created_at)
VALUES ( ?, ?, ?, ?, ?, ?, ?);`, signozMetricDBName, signozUpdatedMetricsMetadataTable)
valueCtx := context.WithValue(ctx, "clickhouse_max_threads", constants.MetricsExplorerClickhouseThreads)
err := r.db.Exec(valueCtx, insertQuery, req.MetricName, req.Temporality, req.IsMonotonic, req.MetricType, req.Description, req.Unit, req.CreatedAt.UnixMilli())
if err != nil {
return &model.ApiError{Typ: "ClickHouseError", Err: err}
}
err = r.cache.Set(ctx, orgID, constants.UpdatedMetricsMetadataCachePrefix+req.MetricName, req, -1)
if err != nil {
return &model.ApiError{Typ: "CachingErr", Err: err}
}
return nil
}
func (r *ClickHouseReader) CheckForLabelsInMetric(ctx context.Context, metricName string, labels []string) (bool, *model.ApiError) {
if len(labels) == 0 {
return true, nil
}
conditions := "metric_name = ?"
for range labels {
conditions += " AND JSONHas(labels, ?) = 1"
}
query := fmt.Sprintf(`
SELECT count(*) > 0 as has_le
FROM %s.%s
WHERE %s
LIMIT 1`, signozMetricDBName, signozTSTableNameV41Day, conditions)
args := make([]interface{}, 0, len(labels)+1)
args = append(args, metricName)
for _, label := range labels {
args = append(args, label)
}
var hasLE bool
valueCtx := context.WithValue(ctx, "clickhouse_max_threads", constants.MetricsExplorerClickhouseThreads)
err := r.db.QueryRow(valueCtx, query, args...).Scan(&hasLE)
if err != nil {
return false, &model.ApiError{
Typ: "ClickHouseError",
Err: fmt.Errorf("error checking summary labels: %v", err),
}
}
return hasLE, nil
}
func (r *ClickHouseReader) PreloadMetricsMetadata(ctx context.Context, orgID valuer.UUID) []error {
var allMetricsMetadata []model.UpdateMetricsMetadata
var errorList []error
// Fetch all rows from ClickHouse
query := fmt.Sprintf(`SELECT metric_name, type, description , temporality, is_monotonic, unit
FROM %s.%s;`, signozMetricDBName, signozUpdatedMetricsMetadataTable)
valueCtx := context.WithValue(ctx, "clickhouse_max_threads", constants.MetricsExplorerClickhouseThreads)
err := r.db.Select(valueCtx, &allMetricsMetadata, query)
if err != nil {
errorList = append(errorList, err)
return errorList
}
for _, m := range allMetricsMetadata {
err := r.cache.Set(ctx, orgID, constants.UpdatedMetricsMetadataCachePrefix+m.MetricName, &m, -1)
if err != nil {
errorList = append(errorList, err)
}
}
return errorList
}
func (r *ClickHouseReader) GetUpdatedMetricsMetadata(ctx context.Context, orgID valuer.UUID, metricNames ...string) (map[string]*model.UpdateMetricsMetadata, *model.ApiError) {
cachedMetadata := make(map[string]*model.UpdateMetricsMetadata)
var missingMetrics []string
// First, try retrieving each metric from cache.
for _, metricName := range metricNames {
metadata := new(model.UpdateMetricsMetadata)
cacheKey := constants.UpdatedMetricsMetadataCachePrefix + metricName
err := r.cache.Get(ctx, orgID, cacheKey, metadata, true)
if err == nil {
cachedMetadata[metricName] = metadata
} else {
missingMetrics = append(missingMetrics, metricName)
}
}
// If there are any metrics missing in the cache, query them from the database.
if len(missingMetrics) > 0 {
// Join the missing metric names; ensure proper quoting if needed.
metricList := "'" + strings.Join(metricNames, "', '") + "'"
query := fmt.Sprintf(`SELECT metric_name, type, description, temporality, is_monotonic, unit
FROM %s.%s
WHERE metric_name IN (%s);`, signozMetricDBName, signozUpdatedMetricsMetadataTable, metricList)
valueCtx := context.WithValue(ctx, "clickhouse_max_threads", constants.MetricsExplorerClickhouseThreads)
rows, err := r.db.Query(valueCtx, query)
if err != nil {
return cachedMetadata, &model.ApiError{Typ: "ClickhouseErr", Err: fmt.Errorf("error querying metrics metadata: %v", err)}
}
defer rows.Close()
for rows.Next() {
metadata := new(model.UpdateMetricsMetadata)
if err := rows.Scan(
&metadata.MetricName,
&metadata.MetricType,
&metadata.Description,
&metadata.Temporality,
&metadata.IsMonotonic,
&metadata.Unit,
); err != nil {
return cachedMetadata, &model.ApiError{Typ: "ClickhouseErr", Err: fmt.Errorf("error scanning metrics metadata: %v", err)}
}
// Cache the result for future requests.
cacheKey := constants.UpdatedMetricsMetadataCachePrefix + metadata.MetricName
if cacheErr := r.cache.Set(ctx, orgID, cacheKey, metadata, -1); cacheErr != nil {
zap.L().Error("Failed to store metrics metadata in cache", zap.String("metric_name", metadata.MetricName), zap.Error(cacheErr))
}
cachedMetadata[metadata.MetricName] = metadata
}
}
return cachedMetadata, nil
}
func (r *ClickHouseReader) SearchTraces(ctx context.Context, params *model.SearchTracesParams) (*[]model.SearchSpansResult, error) {
searchSpansResult := []model.SearchSpansResult{
{
Columns: []string{"__time", "SpanId", "TraceId", "ServiceName", "Name", "Kind", "DurationNano", "TagsKeys", "TagsValues", "References", "Events", "HasError", "StatusMessage", "StatusCodeString", "SpanKind"},
IsSubTree: false,
Events: make([][]interface{}, 0),
},
}
var traceSummary model.TraceSummary
summaryQuery := fmt.Sprintf("SELECT * from %s.%s WHERE trace_id=$1", r.TraceDB, r.traceSummaryTable)
err := r.db.QueryRow(ctx, summaryQuery, params.TraceID).Scan(&traceSummary.TraceID, &traceSummary.Start, &traceSummary.End, &traceSummary.NumSpans)
if err != nil {
if err == sql.ErrNoRows {
return &searchSpansResult, nil
}
zap.L().Error("Error in processing sql query", zap.Error(err))
return nil, fmt.Errorf("error in processing sql query")
}
if traceSummary.NumSpans > uint64(params.MaxSpansInTrace) {
zap.L().Error("Max spans allowed in a trace limit reached", zap.Int("MaxSpansInTrace", params.MaxSpansInTrace),
zap.Uint64("Count", traceSummary.NumSpans))
claims, errv2 := authtypes.ClaimsFromContext(ctx)
if errv2 == nil {
data := map[string]interface{}{
"traceSize": traceSummary.NumSpans,
"maxSpansInTraceLimit": params.MaxSpansInTrace,
"algo": "smart",
}
telemetry.GetInstance().SendEvent(telemetry.TELEMETRY_EVENT_MAX_SPANS_ALLOWED_LIMIT_REACHED, data, claims.Email, true, false)
}
return nil, fmt.Errorf("max spans allowed in trace limit reached, please contact support for more details")
}
claims, errv2 := authtypes.ClaimsFromContext(ctx)
if errv2 == nil {
data := map[string]interface{}{
"traceSize": traceSummary.NumSpans,
"algo": "smart",
}
telemetry.GetInstance().SendEvent(telemetry.TELEMETRY_EVENT_TRACE_DETAIL_API, data, claims.Email, true, false)
}
var startTime, endTime, durationNano uint64
var searchScanResponses []model.SpanItemV2
query := fmt.Sprintf("SELECT timestamp, duration_nano, span_id, trace_id, has_error, kind, resource_string_service$$name, name, references, attributes_string, attributes_number, attributes_bool, resources_string, events, status_message, status_code_string, kind_string FROM %s.%s WHERE trace_id=$1 and ts_bucket_start>=$2 and ts_bucket_start<=$3", r.TraceDB, r.traceTableName)
start := time.Now()
err = r.db.Select(ctx, &searchScanResponses, query, params.TraceID, strconv.FormatInt(traceSummary.Start.Unix()-1800, 10), strconv.FormatInt(traceSummary.End.Unix(), 10))
zap.L().Info(query)
if err != nil {
zap.L().Error("Error in processing sql query", zap.Error(err))
return nil, fmt.Errorf("error in processing sql query")
}
end := time.Now()
zap.L().Debug("getTraceSQLQuery took: ", zap.Duration("duration", end.Sub(start)))
searchSpansResult[0].Events = make([][]interface{}, len(searchScanResponses))
searchSpanResponses := []model.SearchSpanResponseItem{}
start = time.Now()
for _, item := range searchScanResponses {
ref := []model.OtelSpanRef{}
err := json.Unmarshal([]byte(item.References), &ref)
if err != nil {
zap.L().Error("Error unmarshalling references", zap.Error(err))
return nil, err
}
// merge attributes_number and attributes_bool to attributes_string
for k, v := range item.Attributes_bool {
item.Attributes_string[k] = fmt.Sprintf("%v", v)
}
for k, v := range item.Attributes_number {
item.Attributes_string[k] = strconv.FormatFloat(v, 'f', -1, 64)
}
for k, v := range item.Resources_string {
item.Attributes_string[k] = v
}
jsonItem := model.SearchSpanResponseItem{
SpanID: item.SpanID,
TraceID: item.TraceID,
ServiceName: item.ServiceName,
Name: item.Name,
Kind: int32(item.Kind),
DurationNano: int64(item.DurationNano),
HasError: item.HasError,
StatusMessage: item.StatusMessage,
StatusCodeString: item.StatusCodeString,
SpanKind: item.SpanKind,
References: ref,
Events: item.Events,
TagMap: item.Attributes_string,
}
jsonItem.TimeUnixNano = uint64(item.TimeUnixNano.UnixNano() / 1000000)
searchSpanResponses = append(searchSpanResponses, jsonItem)
if startTime == 0 || jsonItem.TimeUnixNano < startTime {
startTime = jsonItem.TimeUnixNano
}
if endTime == 0 || jsonItem.TimeUnixNano > endTime {
endTime = jsonItem.TimeUnixNano
}
if durationNano == 0 || uint64(jsonItem.DurationNano) > durationNano {
durationNano = uint64(jsonItem.DurationNano)
}
}
end = time.Now()
zap.L().Debug("getTraceSQLQuery unmarshal took: ", zap.Duration("duration", end.Sub(start)))
if len(searchScanResponses) > params.SpansRenderLimit {
start = time.Now()
searchSpansResult, err = smart.SmartTraceAlgorithm(searchSpanResponses, params.SpanID, params.LevelUp, params.LevelDown, params.SpansRenderLimit)
if err != nil {
return nil, err
}
end = time.Now()
zap.L().Debug("smartTraceAlgo took: ", zap.Duration("duration", end.Sub(start)))
claims, errv2 := authtypes.ClaimsFromContext(ctx)
if errv2 == nil {
data := map[string]interface{}{
"traceSize": len(searchScanResponses),
"spansRenderLimit": params.SpansRenderLimit,
"algo": "smart",
}
telemetry.GetInstance().SendEvent(telemetry.TELEMETRY_EVENT_LARGE_TRACE_OPENED, data, claims.Email, true, false)
}
} else {
for i, item := range searchSpanResponses {
spanEvents := item.GetValues()
searchSpansResult[0].Events[i] = spanEvents
}
}
searchSpansResult[0].StartTimestampMillis = startTime - (durationNano / 1000000)
searchSpansResult[0].EndTimestampMillis = endTime + (durationNano / 1000000)
return &searchSpansResult, nil
}