Compare commits

...

3 Commits

Author SHA1 Message Date
Shivanshu Raj Shrivastava
83f6dea2db Add support for trace_v3 schema in messaging queues (#6663)
feat: support trace v3 queries
2024-12-18 17:04:01 +05:30
Nityananda Gohain
7031c866e8 fix: add flags for using trace new schema (#6651) 2024-12-18 17:55:22 +07:00
Raj Kamal Singh
cd9f27ab08 Fix: QS: logs pipelines: better validation of pipelines being saved (#6652)
* chore: add test validating invalid field paths in pipeline operators are rejected

* chore: refactor posted pipelines validation to use a controller method

* fix: run a collector simulation to validate pipeline config being saved

* chore: minor cleanup
2024-12-18 10:42:14 +05:30
11 changed files with 164 additions and 99 deletions

View File

@@ -150,7 +150,8 @@ services:
command:
[
"-config=/root/config/prometheus.yml",
"--use-logs-new-schema=true"
"--use-logs-new-schema=true",
"--use-trace-new-schema=true"
]
# ports:
# - "6060:6060" # pprof port

View File

@@ -110,6 +110,7 @@ exporters:
clickhousetraces:
datasource: tcp://clickhouse:9000/signoz_traces
low_cardinal_exception_grouping: ${env:LOW_CARDINAL_EXCEPTION_GROUPING}
use_new_schema: true
clickhousemetricswrite:
endpoint: tcp://clickhouse:9000/signoz_metrics
resource_to_telemetry_conversion:

View File

@@ -25,7 +25,8 @@ services:
command:
[
"-config=/root/config/prometheus.yml",
"--use-logs-new-schema=true"
"--use-logs-new-schema=true",
"--use-trace-new-schema=true"
]
ports:
- "6060:6060"

View File

@@ -167,7 +167,8 @@ services:
command:
[
"-config=/root/config/prometheus.yml",
"--use-logs-new-schema=true"
"--use-logs-new-schema=true",
"--use-trace-new-schema=true"
]
# ports:
# - "6060:6060" # pprof port

View File

@@ -173,7 +173,8 @@ services:
[
"-config=/root/config/prometheus.yml",
"-gateway-url=https://api.staging.signoz.cloud",
"--use-logs-new-schema=true"
"--use-logs-new-schema=true",
"--use-trace-new-schema=true"
]
# ports:
# - "6060:6060" # pprof port

View File

@@ -119,6 +119,7 @@ exporters:
clickhousetraces:
datasource: tcp://clickhouse:9000/signoz_traces
low_cardinal_exception_grouping: ${env:LOW_CARDINAL_EXCEPTION_GROUPING}
use_new_schema: true
clickhousemetricswrite:
endpoint: tcp://clickhouse:9000/signoz_metrics
resource_to_telemetry_conversion:

View File

@@ -4075,10 +4075,9 @@ func (aH *APIHandler) CreateLogsPipeline(w http.ResponseWriter, r *http.Request)
zap.L().Warn("found no pipelines in the http request, this will delete all the pipelines")
}
for _, p := range postable {
if err := p.IsValid(); err != nil {
return nil, model.BadRequestStr(err.Error())
}
validationErr := aH.LogsParsingPipelineController.ValidatePipelines(ctx, postable)
if validationErr != nil {
return nil, validationErr
}
return aH.LogsParsingPipelineController.ApplyPipelines(ctx, postable)

View File

@@ -9,25 +9,25 @@ func generateConsumerSQL(start, end int64, topic, partition, consumerGroup, queu
query := fmt.Sprintf(`
WITH consumer_query AS (
SELECT
serviceName,
resource_string_service$$name,
quantile(0.99)(durationNano) / 1000000 AS p99,
COUNT(*) AS total_requests,
sumIf(1, statusCode = 2) AS error_count,
avg(CASE WHEN has(numberTagMap, 'messaging.message.body.size') THEN numberTagMap['messaging.message.body.size'] ELSE NULL END) AS avg_msg_size
FROM signoz_traces.distributed_signoz_index_v2
sumIf(1, status_code = 2) AS error_count,
avg(CASE WHEN has(attributes_number, 'messaging.message.body.size') THEN attributes_number['messaging.message.body.size'] ELSE NULL END) AS avg_msg_size
FROM signoz_traces.distributed_signoz_index_v3
WHERE
timestamp >= '%d'
AND timestamp <= '%d'
AND kind = 5
AND msgSystem = '%s'
AND stringTagMap['messaging.destination.name'] = '%s'
AND stringTagMap['messaging.destination.partition.id'] = '%s'
AND stringTagMap['messaging.kafka.consumer.group'] = '%s'
GROUP BY serviceName
AND attribute_string_messaging$$system = '%s'
AND attributes_string['messaging.destination.name'] = '%s'
AND attributes_string['messaging.destination.partition.id'] = '%s'
AND attributes_string['messaging.kafka.consumer.group'] = '%s'
GROUP BY resource_string_service$$name
)
SELECT
serviceName AS service_name,
resource_string_service$$name AS service_name,
p99,
COALESCE((error_count * 100.0) / total_requests, 0) AS error_rate,
COALESCE(total_requests / %d, 0) AS throughput,
@@ -35,7 +35,7 @@ SELECT
FROM
consumer_query
ORDER BY
serviceName;
resource_string_service$$name;
`, start, end, queueType, topic, partition, consumerGroup, timeRange)
return query
}
@@ -48,14 +48,14 @@ WITH partition_query AS (
SELECT
quantile(0.99)(durationNano) / 1000000 AS p99,
count(*) AS total_requests,
stringTagMap['messaging.destination.name'] AS topic,
stringTagMap['messaging.destination.partition.id'] AS partition
FROM signoz_traces.distributed_signoz_index_v2
attributes_string['messaging.destination.name'] AS topic,
attributes_string['messaging.destination.partition.id'] AS partition
FROM signoz_traces.distributed_signoz_index_v3
WHERE
timestamp >= '%d'
AND timestamp <= '%d'
AND kind = 4
AND msgSystem = '%s'
AND attribute_string_messaging$$system = '%s'
GROUP BY topic, partition
)
@@ -78,25 +78,25 @@ func generateConsumerPartitionLatencySQL(start, end int64, topic, partition, que
query := fmt.Sprintf(`
WITH consumer_pl AS (
SELECT
stringTagMap['messaging.kafka.consumer.group'] AS consumer_group,
serviceName,
attributes_string['messaging.kafka.consumer.group'] AS consumer_group,
resource_string_service$$name,
quantile(0.99)(durationNano) / 1000000 AS p99,
COUNT(*) AS total_requests,
sumIf(1, statusCode = 2) AS error_count
FROM signoz_traces.distributed_signoz_index_v2
sumIf(1, status_code = 2) AS error_count
FROM signoz_traces.distributed_signoz_index_v3
WHERE
timestamp >= '%d'
AND timestamp <= '%d'
AND kind = 5
AND msgSystem = '%s'
AND stringTagMap['messaging.destination.name'] = '%s'
AND stringTagMap['messaging.destination.partition.id'] = '%s'
GROUP BY consumer_group, serviceName
AND attribute_string_messaging$$system = '%s'
AND attributes_string['messaging.destination.name'] = '%s'
AND attributes_string['messaging.destination.partition.id'] = '%s'
GROUP BY consumer_group, resource_string_service$$name
)
SELECT
consumer_group,
serviceName AS service_name,
resource_string_service$$name AS service_name,
p99,
COALESCE((error_count * 100.0) / total_requests, 0) AS error_rate,
COALESCE(total_requests / %d, 0) AS throughput
@@ -115,23 +115,23 @@ func generateProducerPartitionThroughputSQL(start, end int64, queueType string)
query := fmt.Sprintf(`
WITH producer_latency AS (
SELECT
serviceName,
resource_string_service$$name,
quantile(0.99)(durationNano) / 1000000 AS p99,
stringTagMap['messaging.destination.name'] AS topic,
attributes_string['messaging.destination.name'] AS topic,
COUNT(*) AS total_requests,
sumIf(1, statusCode = 2) AS error_count
FROM signoz_traces.distributed_signoz_index_v2
sumIf(1, status_code = 2) AS error_count
FROM signoz_traces.distributed_signoz_index_v3
WHERE
timestamp >= '%d'
AND timestamp <= '%d'
AND kind = 4
AND msgSystem = '%s'
GROUP BY topic, serviceName
AND attribute_string_messaging$$system = '%s'
GROUP BY topic, resource_string_service$$name
)
SELECT
topic,
serviceName AS service_name,
resource_string_service$$name AS service_name,
p99,
COALESCE((error_count * 100.0) / total_requests, 0) AS error_rate,
COALESCE(total_requests / %d, 0) AS throughput
@@ -148,17 +148,17 @@ func generateProducerTopicLatencySQL(start, end int64, topic, service, queueType
WITH consumer_latency AS (
SELECT
quantile(0.99)(durationNano) / 1000000 AS p99,
stringTagMap['messaging.destination.partition.id'] AS partition,
attributes_string['messaging.destination.partition.id'] AS partition,
COUNT(*) AS total_requests,
sumIf(1, statusCode = 2) AS error_count
FROM signoz_traces.distributed_signoz_index_v2
sumIf(1, status_code = 2) AS error_count
FROM signoz_traces.distributed_signoz_index_v3
WHERE
timestamp >= '%d'
AND timestamp <= '%d'
AND kind = 4
AND serviceName = '%s'
AND msgSystem = '%s'
AND stringTagMap['messaging.destination.name'] = '%s'
AND resource_string_service$$name = '%s'
AND attribute_string_messaging$$system = '%s'
AND attributes_string['messaging.destination.name'] = '%s'
GROUP BY partition
)
@@ -179,24 +179,24 @@ func generateConsumerLatencySQL(start, end int64, queueType string) string {
query := fmt.Sprintf(`
WITH consumer_latency AS (
SELECT
serviceName,
stringTagMap['messaging.destination.name'] AS topic,
resource_string_service$$name,
attributes_string['messaging.destination.name'] AS topic,
quantile(0.99)(durationNano) / 1000000 AS p99,
COUNT(*) AS total_requests,
sumIf(1, statusCode = 2) AS error_count,
SUM(numberTagMap['messaging.message.body.size']) AS total_bytes
FROM signoz_traces.distributed_signoz_index_v2
sumIf(1, status_code = 2) AS error_count,
SUM(attributes_number['messaging.message.body.size']) AS total_bytes
FROM signoz_traces.distributed_signoz_index_v3
WHERE
timestamp >= '%d'
AND timestamp <= '%d'
AND kind = 5
AND msgSystem = '%s'
GROUP BY topic, serviceName
AND attribute_string_messaging$$system = '%s'
GROUP BY topic, resource_string_service$$name
)
SELECT
topic,
serviceName AS service_name,
resource_string_service$$name AS service_name,
p99,
COALESCE((error_count * 100.0) / total_requests, 0) AS error_rate,
COALESCE(total_requests / %d, 0) AS ingestion_rate,
@@ -216,17 +216,17 @@ func generateConsumerServiceLatencySQL(start, end int64, topic, service, queueTy
WITH consumer_latency AS (
SELECT
quantile(0.99)(durationNano) / 1000000 AS p99,
stringTagMap['messaging.destination.partition.id'] AS partition,
attributes_string['messaging.destination.partition.id'] AS partition,
COUNT(*) AS total_requests,
sumIf(1, statusCode = 2) AS error_count
FROM signoz_traces.distributed_signoz_index_v2
sumIf(1, status_code = 2) AS error_count
FROM signoz_traces.distributed_signoz_index_v3
WHERE
timestamp >= '%d'
AND timestamp <= '%d'
AND kind = 5
AND serviceName = '%s'
AND msgSystem = '%s'
AND stringTagMap['messaging.destination.name'] = '%s'
AND resource_string_service$$name = '%s'
AND attribute_string_messaging$$system = '%s'
AND attributes_string['messaging.destination.name'] = '%s'
GROUP BY partition
)
@@ -246,26 +246,26 @@ func generateProducerConsumerEvalSQL(start, end int64, queueType string, evalTim
query := fmt.Sprintf(`
WITH trace_data AS (
SELECT
p.serviceName AS producer_service,
c.serviceName AS consumer_service,
p.traceID,
p.resource_string_service$$name AS producer_service,
c.resource_string_service$$name AS consumer_service,
p.trace_id,
p.timestamp AS producer_timestamp,
c.timestamp AS consumer_timestamp,
p.durationNano AS durationNano,
(toUnixTimestamp64Nano(c.timestamp) - toUnixTimestamp64Nano(p.timestamp)) + p.durationNano AS time_difference
FROM
signoz_traces.distributed_signoz_index_v2 p
signoz_traces.distributed_signoz_index_v3 p
INNER JOIN
signoz_traces.distributed_signoz_index_v2 c
ON p.traceID = c.traceID
AND c.parentSpanID = p.spanID
signoz_traces.distributed_signoz_index_v3 c
ON p.trace_id = c.trace_id
AND c.parent_span_id = p.span_id
WHERE
p.kind = 4
AND c.kind = 5
AND toUnixTimestamp64Nano(p.timestamp) BETWEEN '%d' AND '%d'
AND toUnixTimestamp64Nano(c.timestamp) BETWEEN '%d' AND '%d'
AND c.msgSystem = '%s'
AND p.msgSystem = '%s'
AND c.attribute_string_messaging$$system = '%s'
AND p.attribute_string_messaging$$system = '%s'
)
SELECT
@@ -278,7 +278,7 @@ SELECT
arrayMap(x -> x.1,
arraySort(
x -> -x.2,
groupArrayIf((traceID, time_difference), time_difference > '%d')
groupArrayIf((trace_id, time_difference), time_difference > '%d')
)
),
1, 10
@@ -296,30 +296,30 @@ func generateProducerSQL(start, end int64, topic, partition, queueType string) s
query := fmt.Sprintf(`
WITH producer_query AS (
SELECT
serviceName,
resource_string_service$$name,
quantile(0.99)(durationNano) / 1000000 AS p99,
count(*) AS total_count,
sumIf(1, statusCode = 2) AS error_count
FROM signoz_traces.distributed_signoz_index_v2
sumIf(1, status_code = 2) AS error_count
FROM signoz_traces.distributed_signoz_index_v3
WHERE
timestamp >= '%d'
AND timestamp <= '%d'
AND kind = 4
AND msgSystem = '%s'
AND stringTagMap['messaging.destination.name'] = '%s'
AND stringTagMap['messaging.destination.partition.id'] = '%s'
GROUP BY serviceName
AND attribute_string_messaging$$system = '%s'
AND attributes_string['messaging.destination.name'] = '%s'
AND attributes_string['messaging.destination.partition.id'] = '%s'
GROUP BY resource_string_service$$name
)
SELECT
serviceName AS service_name,
resource_string_service$$name AS service_name,
p99,
COALESCE((error_count * 100.0) / total_count, 0) AS error_percentage,
COALESCE(total_count / %d, 0) AS throughput
FROM
producer_query
ORDER BY
serviceName;
resource_string_service$$name;
`, start, end, queueType, topic, partition, timeRange)
return query
}
@@ -328,18 +328,18 @@ func generateNetworkLatencyThroughputSQL(start, end int64, consumerGroup, partit
timeRange := (end - start) / 1000000000
query := fmt.Sprintf(`
SELECT
stringTagMap['messaging.client_id'] AS client_id,
stringTagMap['service.instance.id'] AS service_instance_id,
serviceName AS service_name,
attributes_string['messaging.client_id'] AS client_id,
attributes_string['service.instance.id'] AS service_instance_id,
resource_string_service$$name AS service_name,
count(*) / %d AS throughput
FROM signoz_traces.distributed_signoz_index_v2
FROM signoz_traces.distributed_signoz_index_v3
WHERE
timestamp >= '%d'
AND timestamp <= '%d'
AND kind = 5
AND msgSystem = '%s'
AND stringTagMap['messaging.kafka.consumer.group'] = '%s'
AND stringTagMap['messaging.destination.partition.id'] = '%s'
AND attribute_string_messaging$$system = '%s'
AND attributes_string['messaging.kafka.consumer.group'] = '%s'
AND attributes_string['messaging.destination.partition.id'] = '%s'
GROUP BY service_name, client_id, service_instance_id
ORDER BY throughput DESC
`, timeRange, start, end, queueType, consumerGroup, partitionID)
@@ -350,12 +350,12 @@ func onboardProducersSQL(start, end int64, queueType string) string {
query := fmt.Sprintf(`
SELECT
COUNT(*) = 0 AS entries,
COUNT(IF(msgSystem = '%s', 1, NULL)) = 0 AS queue,
COUNT(IF(attribute_string_messaging$$system = '%s', 1, NULL)) = 0 AS queue,
COUNT(IF(kind = 4, 1, NULL)) = 0 AS kind,
COUNT(IF(has(stringTagMap, 'messaging.destination.name'), 1, NULL)) = 0 AS destination,
COUNT(IF(has(stringTagMap, 'messaging.destination.partition.id'), 1, NULL)) = 0 AS partition
COUNT(IF(has(attributes_string, 'messaging.destination.name'), 1, NULL)) = 0 AS destination,
COUNT(IF(has(attributes_string, 'messaging.destination.partition.id'), 1, NULL)) = 0 AS partition
FROM
signoz_traces.distributed_signoz_index_v2
signoz_traces.distributed_signoz_index_v3
WHERE
timestamp >= '%d'
AND timestamp <= '%d';`, queueType, start, end)
@@ -366,16 +366,16 @@ func onboardConsumerSQL(start, end int64, queueType string) string {
query := fmt.Sprintf(`
SELECT
COUNT(*) = 0 AS entries,
COUNT(IF(msgSystem = '%s', 1, NULL)) = 0 AS queue,
COUNT(IF(attribute_string_messaging$$system = '%s', 1, NULL)) = 0 AS queue,
COUNT(IF(kind = 5, 1, NULL)) = 0 AS kind,
COUNT(serviceName) = 0 AS svc,
COUNT(IF(has(stringTagMap, 'messaging.destination.name'), 1, NULL)) = 0 AS destination,
COUNT(IF(has(stringTagMap, 'messaging.destination.partition.id'), 1, NULL)) = 0 AS partition,
COUNT(IF(has(stringTagMap, 'messaging.kafka.consumer.group'), 1, NULL)) = 0 AS cgroup,
COUNT(IF(has(numberTagMap, 'messaging.message.body.size'), 1, NULL)) = 0 AS bodysize,
COUNT(IF(has(stringTagMap, 'messaging.client_id'), 1, NULL)) = 0 AS clientid,
COUNT(IF(has(stringTagMap, 'service.instance.id'), 1, NULL)) = 0 AS instanceid
FROM signoz_traces.distributed_signoz_index_v2
COUNT(resource_string_service$$name) = 0 AS svc,
COUNT(IF(has(attributes_string, 'messaging.destination.name'), 1, NULL)) = 0 AS destination,
COUNT(IF(has(attributes_string, 'messaging.destination.partition.id'), 1, NULL)) = 0 AS partition,
COUNT(IF(has(attributes_string, 'messaging.kafka.consumer.group'), 1, NULL)) = 0 AS cgroup,
COUNT(IF(has(attributes_number, 'messaging.message.body.size'), 1, NULL)) = 0 AS bodysize,
COUNT(IF(has(attributes_string, 'messaging.client_id'), 1, NULL)) = 0 AS clientid,
COUNT(IF(has(attributes_string, 'service.instance.id'), 1, NULL)) = 0 AS instanceid
FROM signoz_traces.distributed_signoz_index_v3
WHERE
timestamp >= '%d'
AND timestamp <= '%d';`, queueType, start, end)

View File

@@ -94,6 +94,45 @@ func (ic *LogParsingPipelineController) ApplyPipelines(
return ic.GetPipelinesByVersion(ctx, cfg.Version)
}
func (ic *LogParsingPipelineController) ValidatePipelines(
ctx context.Context,
postedPipelines []PostablePipeline,
) *model.ApiError {
for _, p := range postedPipelines {
if err := p.IsValid(); err != nil {
return model.BadRequestStr(err.Error())
}
}
// Also run a collector simulation to ensure config is fit
// for e2e use with a collector
pipelines := []Pipeline{}
for _, pp := range postedPipelines {
pipelines = append(pipelines, Pipeline{
Id: uuid.New().String(),
OrderId: pp.OrderId,
Enabled: pp.Enabled,
Name: pp.Name,
Alias: pp.Alias,
Description: &pp.Description,
Filter: pp.Filter,
Config: pp.Config,
})
}
sampleLogs := []model.SignozLog{{Body: ""}}
_, _, simulationErr := SimulatePipelinesProcessing(
ctx, pipelines, sampleLogs,
)
if simulationErr != nil {
return model.BadRequest(fmt.Errorf(
"invalid pipelines config: %w", simulationErr.ToError(),
))
}
return nil
}
// Returns effective list of pipelines including user created
// pipelines and pipelines for installed integrations
func (ic *LogParsingPipelineController) getEffectivePipelinesByVersion(

View File

@@ -90,7 +90,7 @@ func EnableHostsInfraMonitoring() bool {
return GetOrDefaultEnv("ENABLE_INFRA_METRICS", "true") == "true"
}
var KafkaSpanEval = GetOrDefaultEnv("KAFKA_SPAN_EVAL", "false")
var KafkaSpanEval = GetOrDefaultEnv("KAFKA_SPAN_EVAL", "true")
func IsDurationSortFeatureEnabled() bool {
isDurationSortFeatureEnabledStr := DurationSortFeature

View File

@@ -350,6 +350,27 @@ func TestLogPipelinesValidation(t *testing.T) {
},
},
ExpectedResponseStatusCode: 400,
}, {
Name: "Invalid from field path",
Pipeline: logparsingpipeline.PostablePipeline{
OrderId: 1,
Name: "pipeline 1",
Alias: "pipeline1",
Enabled: true,
Filter: validPipelineFilterSet,
Config: []logparsingpipeline.PipelineOperator{
{
OrderId: 1,
ID: "move",
Type: "move",
From: `attributes.temp_parsed_body."@l"`,
To: "attributes.test",
Enabled: true,
Name: "test move",
},
},
},
ExpectedResponseStatusCode: 400,
},
}