-
Notifications
You must be signed in to change notification settings - Fork 593
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
feat(metrics): add cdc consume lag metrics #13877
Changes from 10 commits
e9554d1
ac72afe
931564d
5b50d9a
96d4d11
b2808eb
55124f6
10a4ee8
f75f088
53e1fe2
1b9e109
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Large diffs are not rendered by default.
Large diffs are not rendered by default.
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -25,6 +25,7 @@ | |
import java.util.HashMap; | ||
import java.util.List; | ||
import java.util.concurrent.BlockingQueue; | ||
import org.apache.kafka.connect.data.Struct; | ||
import org.apache.kafka.connect.json.JsonConverter; | ||
import org.apache.kafka.connect.json.JsonConverterConfig; | ||
import org.apache.kafka.connect.source.SourceRecord; | ||
|
@@ -114,13 +115,19 @@ var record = event.value(); | |
committer.markProcessed(event); | ||
continue; | ||
} | ||
// get upstream event time from the "source" field | ||
var sourceStruct = ((Struct) record.value()).getStruct("source"); | ||
long sourceTsMs = | ||
sourceStruct == null | ||
? System.currentTimeMillis() | ||
: sourceStruct.getInt64("ts_ms"); | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. dumb question: Is There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I think yes, since |
||
byte[] payload = | ||
converter.fromConnectData( | ||
record.topic(), record.valueSchema(), record.value()); | ||
|
||
msgBuilder | ||
.setFullTableName(fullTableName) | ||
.setPayload(new String(payload, StandardCharsets.UTF_8)) | ||
.setSourceTsMs(sourceTsMs) | ||
.build(); | ||
var message = msgBuilder.build(); | ||
LOG.debug("record => {}", message.getPayload()); | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -16,7 +16,9 @@ use std::sync::{Arc, LazyLock}; | |
|
||
use prometheus::core::{AtomicI64, AtomicU64, GenericCounterVec, GenericGaugeVec}; | ||
use prometheus::{ | ||
register_int_counter_vec_with_registry, register_int_gauge_vec_with_registry, Registry, | ||
exponential_buckets, histogram_opts, register_histogram_vec_with_registry, | ||
register_int_counter_vec_with_registry, register_int_gauge_vec_with_registry, HistogramVec, | ||
Registry, | ||
}; | ||
use risingwave_common::monitor::GLOBAL_METRICS_REGISTRY; | ||
|
||
|
@@ -65,6 +67,8 @@ pub struct SourceMetrics { | |
pub rdkafka_native_metric: Arc<RdKafkaStats>, | ||
|
||
pub connector_source_rows_received: GenericCounterVec<AtomicU64>, | ||
|
||
pub direct_cdc_event_lag_latency: HistogramVec, | ||
} | ||
|
||
pub static GLOBAL_SOURCE_METRICS: LazyLock<SourceMetrics> = | ||
|
@@ -102,13 +106,22 @@ impl SourceMetrics { | |
) | ||
.unwrap(); | ||
|
||
let opts = histogram_opts!( | ||
"source_cdc_event_lag_duration_milliseconds", | ||
"source_cdc_lag_latency", | ||
exponential_buckets(1.0, 2.0, 20).unwrap(), // max 1048s | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. nits: the max will be 1*(2^(20-1)) ~= 524s There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Thanks for pointing out. |
||
); | ||
let direct_cdc_event_lag_latency = | ||
register_histogram_vec_with_registry!(opts, &["table_name"], registry).unwrap(); | ||
|
||
let rdkafka_native_metric = Arc::new(RdKafkaStats::new(registry.clone())); | ||
SourceMetrics { | ||
partition_input_count, | ||
partition_input_bytes, | ||
latest_message_id, | ||
rdkafka_native_metric, | ||
connector_source_rows_received, | ||
direct_cdc_event_lag_latency, | ||
} | ||
} | ||
} | ||
|
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Will there be any difference in
payload.source.ts_ms
andpayload.ts_ms
for direct cdc?There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
payload.ts_ms
is the time at which the connector processed the event, that is process time of the connector.