mirror of
https://github.com/GreptimeTeam/greptimedb.git
synced 2026-01-04 20:32:56 +00:00
feat: benchmark table metadata managers (#2008)
* feat: benchmark table metadata managers * feat: benchmark table metadata managers
This commit is contained in:
4
Cargo.lock
generated
4
Cargo.lock
generated
@@ -1593,6 +1593,7 @@ dependencies = [
|
||||
"async-trait",
|
||||
"build-data",
|
||||
"catalog",
|
||||
"chrono",
|
||||
"clap 3.2.25",
|
||||
"client",
|
||||
"common-base",
|
||||
@@ -1604,6 +1605,7 @@ dependencies = [
|
||||
"common-test-util",
|
||||
"config",
|
||||
"datanode",
|
||||
"datatypes",
|
||||
"either",
|
||||
"etcd-client",
|
||||
"frontend",
|
||||
@@ -1614,6 +1616,7 @@ dependencies = [
|
||||
"nu-ansi-term",
|
||||
"partition",
|
||||
"query",
|
||||
"rand",
|
||||
"rexpect",
|
||||
"rustyline 10.1.1",
|
||||
"serde",
|
||||
@@ -1621,6 +1624,7 @@ dependencies = [
|
||||
"session",
|
||||
"snafu",
|
||||
"substrait 0.3.2",
|
||||
"table",
|
||||
"temp-env",
|
||||
"tikv-jemallocator",
|
||||
"tokio",
|
||||
|
||||
@@ -18,6 +18,7 @@ metrics-process = ["servers/metrics-process"]
|
||||
anymap = "1.0.0-beta.2"
|
||||
async-trait.workspace = true
|
||||
catalog = { path = "../catalog" }
|
||||
chrono.workspace = true
|
||||
clap = { version = "3.1", features = ["derive"] }
|
||||
client = { path = "../client" }
|
||||
common-base = { path = "../common/base" }
|
||||
@@ -30,6 +31,7 @@ common-telemetry = { path = "../common/telemetry", features = [
|
||||
] }
|
||||
config = "0.13"
|
||||
datanode = { path = "../datanode" }
|
||||
datatypes = { path = "../datatypes" }
|
||||
either = "1.8"
|
||||
etcd-client.workspace = true
|
||||
frontend = { path = "../frontend" }
|
||||
@@ -40,12 +42,14 @@ metrics.workspace = true
|
||||
nu-ansi-term = "0.46"
|
||||
partition = { path = "../partition" }
|
||||
query = { path = "../query" }
|
||||
rand.workspace = true
|
||||
rustyline = "10.1"
|
||||
serde.workspace = true
|
||||
servers = { path = "../servers" }
|
||||
session = { path = "../session" }
|
||||
snafu.workspace = true
|
||||
substrait = { path = "../common/substrait" }
|
||||
table = { path = "../table" }
|
||||
tikv-jemallocator = "0.5"
|
||||
tokio.workspace = true
|
||||
|
||||
|
||||
@@ -12,12 +12,14 @@
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
mod bench;
|
||||
mod cmd;
|
||||
mod helper;
|
||||
mod repl;
|
||||
mod upgrade;
|
||||
|
||||
use async_trait::async_trait;
|
||||
use bench::BenchTableMetadataCommand;
|
||||
use clap::Parser;
|
||||
use common_telemetry::logging::LoggingOptions;
|
||||
pub use repl::Repl;
|
||||
@@ -76,6 +78,7 @@ impl Command {
|
||||
enum SubCommand {
|
||||
Attach(AttachCommand),
|
||||
Upgrade(UpgradeCommand),
|
||||
Bench(BenchTableMetadataCommand),
|
||||
}
|
||||
|
||||
impl SubCommand {
|
||||
@@ -83,6 +86,7 @@ impl SubCommand {
|
||||
match self {
|
||||
SubCommand::Attach(cmd) => cmd.build().await,
|
||||
SubCommand::Upgrade(cmd) => cmd.build().await,
|
||||
SubCommand::Bench(cmd) => cmd.build().await,
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
198
src/cmd/src/cli/bench.rs
Normal file
198
src/cmd/src/cli/bench.rs
Normal file
@@ -0,0 +1,198 @@
|
||||
// Copyright 2023 Greptime Team
|
||||
//
|
||||
// Licensed under the Apache License, Version 2.0 (the "License");
|
||||
// you may not use this file except in compliance with the License.
|
||||
// You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
mod datanode_table;
|
||||
mod table_info;
|
||||
mod table_name;
|
||||
mod table_region;
|
||||
|
||||
use std::future::Future;
|
||||
use std::sync::Arc;
|
||||
use std::time::{Duration, Instant};
|
||||
|
||||
use async_trait::async_trait;
|
||||
use clap::Parser;
|
||||
use common_meta::key::table_region::RegionDistribution;
|
||||
use common_meta::key::{TableMetadataManager, TableMetadataManagerRef};
|
||||
use common_meta::table_name::TableName;
|
||||
use common_telemetry::info;
|
||||
use datatypes::data_type::ConcreteDataType;
|
||||
use datatypes::schema::{ColumnSchema, RawSchema};
|
||||
use meta_srv::service::store::etcd::EtcdStore;
|
||||
use meta_srv::service::store::kv::KvBackendAdapter;
|
||||
use rand::prelude::SliceRandom;
|
||||
use table::metadata::{RawTableInfo, RawTableMeta, TableId, TableIdent, TableType};
|
||||
|
||||
use crate::cli::bench::datanode_table::DatanodeTableBencher;
|
||||
use crate::cli::bench::table_info::TableInfoBencher;
|
||||
use crate::cli::bench::table_name::TableNameBencher;
|
||||
use crate::cli::bench::table_region::TableRegionBencher;
|
||||
use crate::cli::{Instance, Tool};
|
||||
use crate::error::Result;
|
||||
|
||||
async fn bench<F, Fut>(desc: &str, f: F, count: u32)
|
||||
where
|
||||
F: Fn(u32) -> Fut,
|
||||
Fut: Future<Output = ()>,
|
||||
{
|
||||
let mut total = Duration::default();
|
||||
|
||||
for i in 1..=count {
|
||||
let start = Instant::now();
|
||||
|
||||
f(i).await;
|
||||
|
||||
total += start.elapsed();
|
||||
}
|
||||
|
||||
let cost = total.as_millis() as f64 / count as f64;
|
||||
info!("{desc}, average operation cost: {cost:.2} ms");
|
||||
}
|
||||
|
||||
async fn bench_self_recorded<F, Fut>(desc: &str, f: F, count: u32)
|
||||
where
|
||||
F: Fn(u32) -> Fut,
|
||||
Fut: Future<Output = Duration>,
|
||||
{
|
||||
let mut total = Duration::default();
|
||||
|
||||
for i in 1..=count {
|
||||
total += f(i).await;
|
||||
}
|
||||
|
||||
let cost = total.as_millis() as f64 / count as f64;
|
||||
info!("{desc}, average operation cost: {cost:.2} ms");
|
||||
}
|
||||
|
||||
#[derive(Debug, Default, Parser)]
|
||||
pub struct BenchTableMetadataCommand {
|
||||
#[clap(long)]
|
||||
etcd_addr: String,
|
||||
#[clap(long)]
|
||||
count: u32,
|
||||
}
|
||||
|
||||
impl BenchTableMetadataCommand {
|
||||
pub async fn build(&self) -> Result<Instance> {
|
||||
let etcd_store = EtcdStore::with_endpoints([&self.etcd_addr]).await.unwrap();
|
||||
|
||||
let table_metadata_manager = Arc::new(TableMetadataManager::new(KvBackendAdapter::wrap(
|
||||
etcd_store,
|
||||
)));
|
||||
|
||||
let tool = BenchTableMetadata {
|
||||
table_metadata_manager,
|
||||
count: self.count,
|
||||
};
|
||||
Ok(Instance::Tool(Box::new(tool)))
|
||||
}
|
||||
}
|
||||
|
||||
struct BenchTableMetadata {
|
||||
table_metadata_manager: TableMetadataManagerRef,
|
||||
count: u32,
|
||||
}
|
||||
|
||||
#[async_trait]
|
||||
impl Tool for BenchTableMetadata {
|
||||
async fn do_work(&self) -> Result<()> {
|
||||
info!("Start benching table name manager ...");
|
||||
TableNameBencher::new(self.table_metadata_manager.table_name_manager(), self.count)
|
||||
.start()
|
||||
.await;
|
||||
|
||||
info!("Start benching table info manager ...");
|
||||
TableInfoBencher::new(self.table_metadata_manager.table_info_manager(), self.count)
|
||||
.start()
|
||||
.await;
|
||||
|
||||
info!("Start benching table region manager ...");
|
||||
TableRegionBencher::new(
|
||||
self.table_metadata_manager.table_region_manager(),
|
||||
self.count,
|
||||
)
|
||||
.start()
|
||||
.await;
|
||||
|
||||
info!("Start benching datanode table manager ...");
|
||||
DatanodeTableBencher::new(
|
||||
self.table_metadata_manager.datanode_table_manager(),
|
||||
self.count,
|
||||
)
|
||||
.start()
|
||||
.await;
|
||||
Ok(())
|
||||
}
|
||||
}
|
||||
|
||||
fn create_table_info(table_id: TableId, table_name: TableName) -> RawTableInfo {
|
||||
let columns = 100;
|
||||
let mut column_schemas = Vec::with_capacity(columns);
|
||||
column_schemas.push(
|
||||
ColumnSchema::new(
|
||||
"ts",
|
||||
ConcreteDataType::timestamp_millisecond_datatype(),
|
||||
true,
|
||||
)
|
||||
.with_time_index(true),
|
||||
);
|
||||
|
||||
for i in 1..columns {
|
||||
let column_name = format!("my_column_{i}");
|
||||
column_schemas.push(ColumnSchema::new(
|
||||
column_name,
|
||||
ConcreteDataType::string_datatype(),
|
||||
true,
|
||||
));
|
||||
}
|
||||
|
||||
let meta = RawTableMeta {
|
||||
schema: RawSchema::new(column_schemas),
|
||||
engine: "mito".to_string(),
|
||||
created_on: chrono::DateTime::default(),
|
||||
primary_key_indices: vec![],
|
||||
next_column_id: columns as u32 + 1,
|
||||
engine_options: Default::default(),
|
||||
value_indices: vec![],
|
||||
options: Default::default(),
|
||||
region_numbers: (1..=100).collect(),
|
||||
};
|
||||
|
||||
RawTableInfo {
|
||||
ident: TableIdent {
|
||||
table_id,
|
||||
version: 1,
|
||||
},
|
||||
name: table_name.table_name,
|
||||
desc: Some("blah".to_string()),
|
||||
catalog_name: table_name.catalog_name,
|
||||
schema_name: table_name.schema_name,
|
||||
meta,
|
||||
table_type: TableType::Base,
|
||||
}
|
||||
}
|
||||
|
||||
fn create_region_distribution() -> RegionDistribution {
|
||||
let mut regions = (1..=100).collect::<Vec<u32>>();
|
||||
regions.shuffle(&mut rand::thread_rng());
|
||||
|
||||
let mut region_distribution = RegionDistribution::new();
|
||||
for datanode_id in 0..10 {
|
||||
region_distribution.insert(
|
||||
datanode_id as u64,
|
||||
regions[datanode_id * 10..(datanode_id + 1) * 10].to_vec(),
|
||||
);
|
||||
}
|
||||
region_distribution
|
||||
}
|
||||
131
src/cmd/src/cli/bench/datanode_table.rs
Normal file
131
src/cmd/src/cli/bench/datanode_table.rs
Normal file
@@ -0,0 +1,131 @@
|
||||
// Copyright 2023 Greptime Team
|
||||
//
|
||||
// Licensed under the Apache License, Version 2.0 (the "License");
|
||||
// you may not use this file except in compliance with the License.
|
||||
// You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
use common_meta::key::datanode_table::{DatanodeTableKey, DatanodeTableManager};
|
||||
|
||||
use super::bench;
|
||||
|
||||
pub struct DatanodeTableBencher<'a> {
|
||||
datanode_table_manager: &'a DatanodeTableManager,
|
||||
count: u32,
|
||||
}
|
||||
|
||||
impl<'a> DatanodeTableBencher<'a> {
|
||||
pub fn new(datanode_table_manager: &'a DatanodeTableManager, count: u32) -> Self {
|
||||
Self {
|
||||
datanode_table_manager,
|
||||
count,
|
||||
}
|
||||
}
|
||||
|
||||
pub async fn start(&self) {
|
||||
self.bench_create().await;
|
||||
self.bench_get().await;
|
||||
self.bench_move_region().await;
|
||||
self.bench_tables().await;
|
||||
self.bench_remove().await;
|
||||
}
|
||||
|
||||
async fn bench_create(&self) {
|
||||
let desc = format!(
|
||||
"DatanodeTableBencher: create {} datanode table keys",
|
||||
self.count
|
||||
);
|
||||
bench(
|
||||
&desc,
|
||||
|i| async move {
|
||||
self.datanode_table_manager
|
||||
.create(1, i, vec![1, 2, 3, 4])
|
||||
.await
|
||||
.unwrap();
|
||||
},
|
||||
self.count,
|
||||
)
|
||||
.await;
|
||||
}
|
||||
|
||||
async fn bench_get(&self) {
|
||||
let desc = format!(
|
||||
"DatanodeTableBencher: get {} datanode table keys",
|
||||
self.count
|
||||
);
|
||||
bench(
|
||||
&desc,
|
||||
|i| async move {
|
||||
let key = DatanodeTableKey::new(1, i);
|
||||
assert!(self
|
||||
.datanode_table_manager
|
||||
.get(&key)
|
||||
.await
|
||||
.unwrap()
|
||||
.is_some());
|
||||
},
|
||||
self.count,
|
||||
)
|
||||
.await;
|
||||
}
|
||||
|
||||
async fn bench_move_region(&self) {
|
||||
let desc = format!(
|
||||
"DatanodeTableBencher: move {} datanode table regions",
|
||||
self.count
|
||||
);
|
||||
bench(
|
||||
&desc,
|
||||
|i| async move {
|
||||
self.datanode_table_manager
|
||||
.move_region(1, 2, i, 1)
|
||||
.await
|
||||
.unwrap();
|
||||
},
|
||||
self.count,
|
||||
)
|
||||
.await;
|
||||
}
|
||||
|
||||
async fn bench_tables(&self) {
|
||||
let desc = format!(
|
||||
"DatanodeTableBencher: list {} datanode table keys",
|
||||
self.count
|
||||
);
|
||||
bench(
|
||||
&desc,
|
||||
|_| async move {
|
||||
assert!(!self
|
||||
.datanode_table_manager
|
||||
.tables(1)
|
||||
.await
|
||||
.unwrap()
|
||||
.is_empty());
|
||||
},
|
||||
self.count,
|
||||
)
|
||||
.await;
|
||||
}
|
||||
|
||||
async fn bench_remove(&self) {
|
||||
let desc = format!(
|
||||
"DatanodeTableBencher: remove {} datanode table keys",
|
||||
self.count
|
||||
);
|
||||
bench(
|
||||
&desc,
|
||||
|i| async move {
|
||||
self.datanode_table_manager.remove(1, i).await.unwrap();
|
||||
},
|
||||
self.count,
|
||||
)
|
||||
.await;
|
||||
}
|
||||
}
|
||||
111
src/cmd/src/cli/bench/table_info.rs
Normal file
111
src/cmd/src/cli/bench/table_info.rs
Normal file
@@ -0,0 +1,111 @@
|
||||
// Copyright 2023 Greptime Team
|
||||
//
|
||||
// Licensed under the Apache License, Version 2.0 (the "License");
|
||||
// you may not use this file except in compliance with the License.
|
||||
// You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
use std::time::Instant;
|
||||
|
||||
use common_meta::key::table_info::TableInfoManager;
|
||||
use common_meta::table_name::TableName;
|
||||
|
||||
use super::{bench, bench_self_recorded, create_table_info};
|
||||
|
||||
pub struct TableInfoBencher<'a> {
|
||||
table_info_manager: &'a TableInfoManager,
|
||||
count: u32,
|
||||
}
|
||||
|
||||
impl<'a> TableInfoBencher<'a> {
|
||||
pub fn new(table_info_manager: &'a TableInfoManager, count: u32) -> Self {
|
||||
Self {
|
||||
table_info_manager,
|
||||
count,
|
||||
}
|
||||
}
|
||||
|
||||
pub async fn start(&self) {
|
||||
self.bench_create().await;
|
||||
self.bench_get().await;
|
||||
self.bench_compare_and_put().await;
|
||||
self.bench_remove().await;
|
||||
}
|
||||
|
||||
async fn bench_create(&self) {
|
||||
let desc = format!("TableInfoBencher: create {} table infos", self.count);
|
||||
bench(
|
||||
&desc,
|
||||
|i| async move {
|
||||
let table_name = format!("bench_table_name_{}", i);
|
||||
let table_name = TableName::new("bench_catalog", "bench_schema", table_name);
|
||||
let table_info = create_table_info(i, table_name);
|
||||
self.table_info_manager
|
||||
.create(i, &table_info)
|
||||
.await
|
||||
.unwrap();
|
||||
},
|
||||
self.count,
|
||||
)
|
||||
.await;
|
||||
}
|
||||
|
||||
async fn bench_get(&self) {
|
||||
let desc = format!("TableInfoBencher: get {} table infos", self.count);
|
||||
bench(
|
||||
&desc,
|
||||
|i| async move {
|
||||
assert!(self.table_info_manager.get(i).await.unwrap().is_some());
|
||||
},
|
||||
self.count,
|
||||
)
|
||||
.await;
|
||||
}
|
||||
|
||||
async fn bench_compare_and_put(&self) {
|
||||
let desc = format!(
|
||||
"TableInfoBencher: compare_and_put {} table infos",
|
||||
self.count
|
||||
);
|
||||
bench_self_recorded(
|
||||
&desc,
|
||||
|i| async move {
|
||||
let table_info_value = self.table_info_manager.get(i).await.unwrap().unwrap();
|
||||
|
||||
let mut new_table_info = table_info_value.table_info.clone();
|
||||
new_table_info.ident.version += 1;
|
||||
|
||||
let start = Instant::now();
|
||||
|
||||
self.table_info_manager
|
||||
.compare_and_put(i, Some(table_info_value), new_table_info)
|
||||
.await
|
||||
.unwrap()
|
||||
.unwrap();
|
||||
|
||||
start.elapsed()
|
||||
},
|
||||
self.count,
|
||||
)
|
||||
.await;
|
||||
}
|
||||
|
||||
async fn bench_remove(&self) {
|
||||
let desc = format!("TableInfoBencher: remove {} table infos", self.count);
|
||||
bench(
|
||||
&desc,
|
||||
|i| async move {
|
||||
self.table_info_manager.remove(i).await.unwrap();
|
||||
},
|
||||
self.count,
|
||||
)
|
||||
.await;
|
||||
}
|
||||
}
|
||||
131
src/cmd/src/cli/bench/table_name.rs
Normal file
131
src/cmd/src/cli/bench/table_name.rs
Normal file
@@ -0,0 +1,131 @@
|
||||
// Copyright 2023 Greptime Team
|
||||
//
|
||||
// Licensed under the Apache License, Version 2.0 (the "License");
|
||||
// you may not use this file except in compliance with the License.
|
||||
// You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
use common_meta::key::table_name::{TableNameKey, TableNameManager};
|
||||
|
||||
use super::bench;
|
||||
|
||||
pub struct TableNameBencher<'a> {
|
||||
table_name_manager: &'a TableNameManager,
|
||||
count: u32,
|
||||
}
|
||||
|
||||
impl<'a> TableNameBencher<'a> {
|
||||
pub fn new(table_name_manager: &'a TableNameManager, count: u32) -> Self {
|
||||
Self {
|
||||
table_name_manager,
|
||||
count,
|
||||
}
|
||||
}
|
||||
|
||||
pub async fn start(&self) {
|
||||
self.bench_create().await;
|
||||
self.bench_rename().await;
|
||||
self.bench_get().await;
|
||||
self.bench_tables().await;
|
||||
self.bench_remove().await;
|
||||
}
|
||||
|
||||
async fn bench_create(&self) {
|
||||
let desc = format!("TableNameBencher: create {} table names", self.count);
|
||||
bench(
|
||||
&desc,
|
||||
|i| async move {
|
||||
let table_name = format!("bench_table_name_{}", i);
|
||||
let table_name_key = create_table_name_key(&table_name);
|
||||
self.table_name_manager
|
||||
.create(&table_name_key, i)
|
||||
.await
|
||||
.unwrap();
|
||||
},
|
||||
self.count,
|
||||
)
|
||||
.await;
|
||||
}
|
||||
|
||||
async fn bench_rename(&self) {
|
||||
let desc = format!("TableNameBencher: rename {} table names", self.count);
|
||||
bench(
|
||||
&desc,
|
||||
|i| async move {
|
||||
let table_name = format!("bench_table_name_{}", i);
|
||||
let new_table_name = format!("bench_table_name_new_{}", i);
|
||||
let table_name_key = create_table_name_key(&table_name);
|
||||
self.table_name_manager
|
||||
.rename(table_name_key, i, &new_table_name)
|
||||
.await
|
||||
.unwrap();
|
||||
},
|
||||
self.count,
|
||||
)
|
||||
.await;
|
||||
}
|
||||
|
||||
async fn bench_get(&self) {
|
||||
let desc = format!("TableNameBencher: get {} table names", self.count);
|
||||
bench(
|
||||
&desc,
|
||||
|i| async move {
|
||||
let table_name = format!("bench_table_name_new_{}", i);
|
||||
let table_name_key = create_table_name_key(&table_name);
|
||||
assert!(self
|
||||
.table_name_manager
|
||||
.get(table_name_key)
|
||||
.await
|
||||
.unwrap()
|
||||
.is_some());
|
||||
},
|
||||
self.count,
|
||||
)
|
||||
.await;
|
||||
}
|
||||
|
||||
async fn bench_tables(&self) {
|
||||
let desc = format!("TableNameBencher: list all {} table names", self.count);
|
||||
bench(
|
||||
&desc,
|
||||
|_| async move {
|
||||
assert!(!self
|
||||
.table_name_manager
|
||||
.tables("bench_catalog", "bench_schema")
|
||||
.await
|
||||
.unwrap()
|
||||
.is_empty());
|
||||
},
|
||||
self.count,
|
||||
)
|
||||
.await;
|
||||
}
|
||||
|
||||
async fn bench_remove(&self) {
|
||||
let desc = format!("TableNameBencher: remove {} table names", self.count);
|
||||
bench(
|
||||
&desc,
|
||||
|i| async move {
|
||||
let table_name = format!("bench_table_name_new_{}", i);
|
||||
let table_name_key = create_table_name_key(&table_name);
|
||||
self.table_name_manager
|
||||
.remove(table_name_key)
|
||||
.await
|
||||
.unwrap();
|
||||
},
|
||||
self.count,
|
||||
)
|
||||
.await;
|
||||
}
|
||||
}
|
||||
|
||||
fn create_table_name_key(table_name: &str) -> TableNameKey {
|
||||
TableNameKey::new("bench_catalog", "bench_schema", table_name)
|
||||
}
|
||||
112
src/cmd/src/cli/bench/table_region.rs
Normal file
112
src/cmd/src/cli/bench/table_region.rs
Normal file
@@ -0,0 +1,112 @@
|
||||
// Copyright 2023 Greptime Team
|
||||
//
|
||||
// Licensed under the Apache License, Version 2.0 (the "License");
|
||||
// you may not use this file except in compliance with the License.
|
||||
// You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
use std::time::Instant;
|
||||
|
||||
use common_meta::key::table_region::TableRegionManager;
|
||||
|
||||
use super::{bench, bench_self_recorded, create_region_distribution};
|
||||
|
||||
pub struct TableRegionBencher<'a> {
|
||||
table_region_manager: &'a TableRegionManager,
|
||||
count: u32,
|
||||
}
|
||||
|
||||
impl<'a> TableRegionBencher<'a> {
|
||||
pub fn new(table_region_manager: &'a TableRegionManager, count: u32) -> Self {
|
||||
Self {
|
||||
table_region_manager,
|
||||
count,
|
||||
}
|
||||
}
|
||||
|
||||
pub async fn start(&self) {
|
||||
self.bench_create().await;
|
||||
self.bench_get().await;
|
||||
self.bench_compare_and_put().await;
|
||||
self.bench_remove().await;
|
||||
}
|
||||
|
||||
async fn bench_create(&self) {
|
||||
let desc = format!("TableRegionBencher: create {} table regions", self.count);
|
||||
bench_self_recorded(
|
||||
&desc,
|
||||
|i| async move {
|
||||
let region_distribution = create_region_distribution();
|
||||
|
||||
let start = Instant::now();
|
||||
|
||||
self.table_region_manager
|
||||
.create(i, ®ion_distribution)
|
||||
.await
|
||||
.unwrap();
|
||||
|
||||
start.elapsed()
|
||||
},
|
||||
self.count,
|
||||
)
|
||||
.await;
|
||||
}
|
||||
|
||||
async fn bench_get(&self) {
|
||||
let desc = format!("TableRegionBencher: get {} table regions", self.count);
|
||||
bench(
|
||||
&desc,
|
||||
|i| async move {
|
||||
assert!(self.table_region_manager.get(i).await.unwrap().is_some());
|
||||
},
|
||||
self.count,
|
||||
)
|
||||
.await;
|
||||
}
|
||||
|
||||
async fn bench_compare_and_put(&self) {
|
||||
let desc = format!(
|
||||
"TableRegionBencher: compare_and_put {} table regions",
|
||||
self.count
|
||||
);
|
||||
bench_self_recorded(
|
||||
&desc,
|
||||
|i| async move {
|
||||
let table_region_value = self.table_region_manager.get(i).await.unwrap().unwrap();
|
||||
|
||||
let new_region_distribution = create_region_distribution();
|
||||
|
||||
let start = Instant::now();
|
||||
|
||||
self.table_region_manager
|
||||
.compare_and_put(i, Some(table_region_value), new_region_distribution)
|
||||
.await
|
||||
.unwrap()
|
||||
.unwrap();
|
||||
|
||||
start.elapsed()
|
||||
},
|
||||
self.count,
|
||||
)
|
||||
.await;
|
||||
}
|
||||
|
||||
async fn bench_remove(&self) {
|
||||
let desc = format!("TableRegionBencher: remove {} table regions", self.count);
|
||||
bench(
|
||||
&desc,
|
||||
|i| async move {
|
||||
assert!(self.table_region_manager.remove(i).await.unwrap().is_some());
|
||||
},
|
||||
self.count,
|
||||
)
|
||||
.await;
|
||||
}
|
||||
}
|
||||
@@ -127,7 +127,7 @@ impl TableMetadataManager {
|
||||
}
|
||||
|
||||
macro_rules! impl_table_meta_key {
|
||||
( $($val_ty: ty), *) => {
|
||||
($($val_ty: ty), *) => {
|
||||
$(
|
||||
impl std::fmt::Display for $val_ty {
|
||||
fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
|
||||
@@ -146,7 +146,7 @@ impl_table_meta_key!(
|
||||
);
|
||||
|
||||
macro_rules! impl_table_meta_value {
|
||||
( $($val_ty: ty), *) => {
|
||||
($($val_ty: ty), *) => {
|
||||
$(
|
||||
impl $val_ty {
|
||||
pub fn try_from_raw_value(raw_value: Vec<u8>) -> Result<Self> {
|
||||
|
||||
Reference in New Issue
Block a user