Skip to content
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: new crate trace_metric for collecting metrics in read procedure #714

Merged
merged 23 commits into from
Mar 14, 2023
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension


Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
27 changes: 27 additions & 0 deletions Cargo.lock

Some generated files are not rendered by default. Learn more about how customized files appear on GitHub.

6 changes: 6 additions & 0 deletions Cargo.toml
Original file line number Diff line number Diff line change
Expand Up @@ -29,6 +29,9 @@ members = [
"components/profile",
"components/skiplist",
"components/table_kv",
"components/trace_metric",
"components/trace_metric_derive",
"components/trace_metric_derive_tests",
"components/tracing_util",
"df_operator",
"integration_tests",
Expand Down Expand Up @@ -112,6 +115,9 @@ table_engine = { path = "table_engine" }
table_kv = { path = "components/table_kv" }
tempfile = "3.1.0"
tracing_util = { path = "components/tracing_util" }
trace_metric = { path = "components/trace_metric" }
trace_metric_derive = { path = "components/trace_metric_derive" }
trace_metric_derive_tests = { path = "components/trace_metric_derive_tests" }
tonic = "0.8.1"
tokio = { version = "1.25", features = ["full"] }
wal = { path = "wal" }
Expand Down
1 change: 1 addition & 0 deletions analytic_engine/Cargo.toml
Original file line number Diff line number Diff line change
Expand Up @@ -46,6 +46,7 @@ table_engine = { workspace = true }
table_kv = { workspace = true }
tempfile = { workspace = true, optional = true }
tokio = { workspace = true }
trace_metric = { workspace = true }
wal = { workspace = true }
xorfilter-rs = { workspace = true }

Expand Down
2 changes: 2 additions & 0 deletions analytic_engine/src/instance/flush_compaction.rs
Original file line number Diff line number Diff line change
Expand Up @@ -855,6 +855,7 @@ impl SpaceStore {
let sequence = table_data.last_sequence();
let mut builder = MergeBuilder::new(MergeConfig {
request_id,
metrics_collector: None,
// no need to set deadline for compaction
deadline: None,
space_id,
Expand Down Expand Up @@ -1053,6 +1054,7 @@ fn build_mem_table_iter(memtable: MemTableRef, table_data: &TableData) -> Result
projected_schema: ProjectedSchema::no_projection(table_data.schema()),
need_dedup: table_data.dedup(),
reverse: false,
metrics_collector: None,
};
memtable
.scan(scan_ctx, scan_req)
Expand Down
34 changes: 26 additions & 8 deletions analytic_engine/src/instance/read.rs
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,7 @@ use table_engine::{
table::ReadRequest,
};
use tokio::sync::mpsc::{self, Receiver};
use trace_metric::Metric;

use crate::{
instance::Instance,
Expand Down Expand Up @@ -66,6 +67,9 @@ pub enum Error {
define_result!(Error);

const RECORD_BATCH_READ_BUF_SIZE: usize = 1000;
const MERGE_SORT_METRIC_NAME: &str = "do_merge_sort";
const ITER_NUM_METRIC_NAME: &str = "iter_num";
const MERGE_ITER_METRICS_COLLECTOR_NAME_PREFIX: &str = "merge_iter";

/// Check whether it needs to apply merge sorting when reading the table with
/// the `table_options` by the `read_request`.
Expand All @@ -91,13 +95,18 @@ impl Instance {

let table_data = space_table.table_data();

// Collect metrics.
table_data.metrics.on_read_request_begin();

let iter_options = self.iter_options.clone();
let table_options = table_data.table_options();

if need_merge_sort_streams(&table_data.table_options(), &request) {
// Collect metrics.
table_data.metrics.on_read_request_begin();
let need_merge_sort = need_merge_sort_streams(&table_options, &request);
request.metrics_collector.collect(Metric::boolean(
MERGE_SORT_METRIC_NAME.to_string(),
need_merge_sort,
));

if need_merge_sort {
let merge_iters = self
.build_merge_iters(table_data, &request, iter_options, &table_options)
.await?;
Expand All @@ -123,16 +132,16 @@ impl Instance {
};

// Split iterators into `read_parallelism` groups.
let mut splited_iters: Vec<_> = std::iter::repeat_with(Vec::new)
let mut splitted_iters: Vec<_> = std::iter::repeat_with(Vec::new)
.take(read_parallelism)
.collect();

for (i, time_aligned_iter) in partitioned_iters.into_iter().enumerate() {
splited_iters[i % read_parallelism].push(time_aligned_iter);
splitted_iters[i % read_parallelism].push(time_aligned_iter);
}

let mut streams = Vec::with_capacity(read_parallelism);
for iters in splited_iters {
for iters in splitted_iters {
let stream = iters_to_stream(iters, self.read_runtime(), &request.projected_schema);
streams.push(stream);
}
Expand Down Expand Up @@ -169,9 +178,13 @@ impl Instance {
let read_views = self.partition_ssts_and_memtables(time_range, version, table_options);

let mut iters = Vec::with_capacity(read_views.len());
for read_view in read_views {
for (idx, read_view) in read_views.into_iter().enumerate() {
let metrics_collector = request
.metrics_collector
.span(format!("{MERGE_ITER_METRICS_COLLECTOR_NAME_PREFIX}_{idx}"));
let merge_config = MergeConfig {
request_id: request.request_id,
metrics_collector: Some(metrics_collector),
deadline: request.opts.deadline,
space_id: table_data.space_id,
table_id: table_data.id,
Expand Down Expand Up @@ -201,6 +214,11 @@ impl Instance {
iters.push(dedup_iter);
}

request.metrics_collector.collect(Metric::number(
ITER_NUM_METRIC_NAME.to_string(),
iters.len(),
));

Ok(iters)
}

Expand Down
7 changes: 5 additions & 2 deletions analytic_engine/src/memtable/mod.rs
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@ use common_types::{
};
use common_util::{define_result, error::GenericError};
use snafu::{Backtrace, Snafu};
use trace_metric::MetricsCollector;

use crate::memtable::key::KeySequence;

Expand Down Expand Up @@ -129,14 +130,16 @@ pub struct ScanRequest {
pub projected_schema: ProjectedSchema,
pub need_dedup: bool,
pub reverse: bool,
/// Collector for scan metrics.
pub metrics_collector: Option<MetricsCollector>,
}

/// In memory storage for table's data.
///
/// # Concurrency
/// The memtable is designed for single-writer and mutltiple-reader usage, so
/// The memtable is designed for single-writer and multiple-reader usage, so
/// not all function supports concurrent writer, the caller should guarantee not
/// writing to the memtable concurrrently.
/// writing to the memtable concurrently.
// All operation is done in memory, no need to use async trait
pub trait MemTable {
/// Schema of this memtable
Expand Down
4 changes: 4 additions & 0 deletions analytic_engine/src/memtable/skiplist/mod.rs
Original file line number Diff line number Diff line change
Expand Up @@ -204,6 +204,7 @@ mod tests {
projected_schema: projected_schema.clone(),
need_dedup: true,
reverse: false,
metrics_collector: None,
},
vec![
build_row(b"a", 1, 10.0, "v1", 1000, 1_000_000),
Expand All @@ -223,6 +224,7 @@ mod tests {
projected_schema: projected_schema.clone(),
need_dedup: true,
reverse: false,
metrics_collector: None,
},
vec![
build_row(b"a", 1, 10.0, "v1", 1000, 1_000_000),
Expand All @@ -241,6 +243,7 @@ mod tests {
projected_schema,
need_dedup: true,
reverse: false,
metrics_collector: None,
},
vec![
build_row(b"a", 1, 10.0, "v1", 1000, 1_000_000),
Expand Down Expand Up @@ -272,6 +275,7 @@ mod tests {
projected_schema,
need_dedup: true,
reverse: false,
metrics_collector: None,
},
vec![
build_row_for_two_column(b"a", 1),
Expand Down
3 changes: 3 additions & 0 deletions analytic_engine/src/row_iter/chain.rs
Original file line number Diff line number Diff line change
Expand Up @@ -116,6 +116,7 @@ impl<'a> Builder<'a> {
false,
self.config.predicate.as_ref(),
self.config.deadline,
None,
)
.context(BuildStreamFromMemtable)?;
streams.push(stream);
Expand All @@ -131,6 +132,7 @@ impl<'a> Builder<'a> {
false,
self.config.predicate.as_ref(),
self.config.deadline,
None,
)
.context(BuildStreamFromMemtable)?;
streams.push(stream);
Expand All @@ -145,6 +147,7 @@ impl<'a> Builder<'a> {
self.config.sst_factory,
&self.config.sst_read_options,
self.config.store_picker,
None,
)
.await
.context(BuildStreamFromSst)?;
Expand Down
Loading