refactor: update table metadata in single txn (#2172)

* refactor: table-metadata-manager

* feat: remove comparing when deleting metadata

* fix: fix comment typos

* chore: apply suggestions from CR

* test: add tests for updating DatanodeTable

* fix: fix clippy

* chore: apply suggestions from CR

* refactor: improve update table route tests

* refactor: return Txn instead of TxnRequest

* chore: apply suggestions from CR

* chore: apply suggestions from CR

* refactor: update table metadata in single txn

* feat: check table exists before drop table executing

* test: add tests for table metadata manager

* refactor: remove table region manager

* chore: apply suggestions from CR

* feat: add bench program

* chore: apply suggestions from CR
This commit is contained in:
Weny Xu
2023-08-17 14:29:19 +08:00
committed by GitHub
parent 4ba12155fe
commit 6e6ff5a606
38 changed files with 1081 additions and 2548 deletions

View File

@@ -23,6 +23,7 @@ use common_meta::key::datanode_table::DatanodeTableValue;
use common_meta::key::schema_name::SchemaNameKey; use common_meta::key::schema_name::SchemaNameKey;
use common_meta::key::TableMetadataManagerRef; use common_meta::key::TableMetadataManagerRef;
use common_telemetry::{error, info, warn}; use common_telemetry::{error, info, warn};
use futures_util::TryStreamExt;
use metrics::increment_gauge; use metrics::increment_gauge;
use snafu::{ensure, OptionExt, ResultExt}; use snafu::{ensure, OptionExt, ResultExt};
use table::engine::manager::TableEngineManagerRef; use table::engine::manager::TableEngineManagerRef;
@@ -74,6 +75,7 @@ impl RemoteCatalogManager {
.table_metadata_manager .table_metadata_manager
.datanode_table_manager() .datanode_table_manager()
.tables(self.node_id) .tables(self.node_id)
.try_collect::<Vec<_>>()
.await .await
.context(TableMetadataManagerSnafu)?; .context(TableMetadataManagerSnafu)?;

View File

@@ -12,53 +12,30 @@
// See the License for the specific language governing permissions and // See the License for the specific language governing permissions and
// limitations under the License. // limitations under the License.
mod datanode_table; use std::collections::BTreeMap;
mod table_info;
mod table_name;
mod table_region;
use std::future::Future; use std::future::Future;
use std::sync::Arc; use std::sync::Arc;
use std::time::{Duration, Instant}; use std::time::Duration;
use async_trait::async_trait; use async_trait::async_trait;
use clap::Parser; use clap::Parser;
use common_meta::key::table_region::RegionDistribution;
use common_meta::key::{TableMetadataManager, TableMetadataManagerRef}; use common_meta::key::{TableMetadataManager, TableMetadataManagerRef};
use common_meta::peer::Peer;
use common_meta::rpc::router::{Region, RegionRoute};
use common_meta::table_name::TableName; use common_meta::table_name::TableName;
use common_telemetry::info; use common_telemetry::info;
use datatypes::data_type::ConcreteDataType; use datatypes::data_type::ConcreteDataType;
use datatypes::schema::{ColumnSchema, RawSchema}; use datatypes::schema::{ColumnSchema, RawSchema};
use meta_srv::service::store::etcd::EtcdStore; use meta_srv::service::store::etcd::EtcdStore;
use meta_srv::service::store::kv::KvBackendAdapter; use meta_srv::service::store::kv::KvBackendAdapter;
use rand::prelude::SliceRandom; use rand::Rng;
use table::metadata::{RawTableInfo, RawTableMeta, TableId, TableIdent, TableType}; use table::metadata::{RawTableInfo, RawTableMeta, TableId, TableIdent, TableType};
use crate::cli::bench::datanode_table::DatanodeTableBencher; use self::metadata::TableMetadataBencher;
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::cli::{Instance, Tool};
use crate::error::Result; use crate::error::Result;
async fn bench<F, Fut>(desc: &str, f: F, count: u32) mod metadata;
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) async fn bench_self_recorded<F, Fut>(desc: &str, f: F, count: u32)
where where
@@ -107,31 +84,11 @@ struct BenchTableMetadata {
#[async_trait] #[async_trait]
impl Tool for BenchTableMetadata { impl Tool for BenchTableMetadata {
async fn do_work(&self) -> Result<()> { async fn do_work(&self) -> Result<()> {
info!("Start benching table name manager ..."); let bencher = TableMetadataBencher::new(self.table_metadata_manager.clone(), self.count);
TableNameBencher::new(self.table_metadata_manager.table_name_manager(), self.count) bencher.bench_create().await;
.start() bencher.bench_get().await;
.await; bencher.bench_rename().await;
bencher.bench_delete().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(()) Ok(())
} }
} }
@@ -184,16 +141,25 @@ fn create_table_info(table_id: TableId, table_name: TableName) -> RawTableInfo {
} }
} }
fn create_region_distribution() -> RegionDistribution { fn create_region_routes() -> Vec<RegionRoute> {
let mut regions = (1..=100).collect::<Vec<u32>>(); let mut regions = Vec::with_capacity(100);
regions.shuffle(&mut rand::thread_rng()); let mut rng = rand::thread_rng();
let mut region_distribution = RegionDistribution::new(); for region_id in 0..64u64 {
for datanode_id in 0..10 { regions.push(RegionRoute {
region_distribution.insert( region: Region {
datanode_id as u64, id: region_id.into(),
regions[datanode_id * 10..(datanode_id + 1) * 10].to_vec(), name: String::new(),
); partition: None,
attrs: BTreeMap::new(),
},
leader_peer: Some(Peer {
id: rng.gen_range(0..10),
addr: String::new(),
}),
follower_peers: vec![],
});
} }
region_distribution
regions
} }

View File

@@ -1,131 +0,0 @@
// 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;
}
}

View File

@@ -0,0 +1,136 @@
// 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::TableMetadataManagerRef;
use common_meta::table_name::TableName;
use super::{bench_self_recorded, create_region_routes, create_table_info};
pub struct TableMetadataBencher {
table_metadata_manager: TableMetadataManagerRef,
count: u32,
}
impl TableMetadataBencher {
pub fn new(table_metadata_manager: TableMetadataManagerRef, count: u32) -> Self {
Self {
table_metadata_manager,
count,
}
}
pub async fn bench_create(&self) {
let desc = format!(
"TableMetadataBencher: creating {} table metadata",
self.count
);
bench_self_recorded(
&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);
let region_routes = create_region_routes();
let start = Instant::now();
self.table_metadata_manager
.create_table_metadata(table_info, region_routes)
.await
.unwrap();
start.elapsed()
},
self.count,
)
.await;
}
pub async fn bench_get(&self) {
let desc = format!(
"TableMetadataBencher: getting {} table info and region routes",
self.count
);
bench_self_recorded(
&desc,
|i| async move {
let start = Instant::now();
self.table_metadata_manager
.get_full_table_info(i)
.await
.unwrap();
start.elapsed()
},
self.count,
)
.await;
}
pub async fn bench_delete(&self) {
let desc = format!(
"TableMetadataBencher: deleting {} table metadata",
self.count
);
bench_self_recorded(
&desc,
|i| async move {
let (table_info, table_route) = self
.table_metadata_manager
.get_full_table_info(i)
.await
.unwrap();
let start = Instant::now();
let _ = self
.table_metadata_manager
.delete_table_metadata(&table_info.unwrap(), &table_route.unwrap())
.await;
start.elapsed()
},
self.count,
)
.await;
}
pub async fn bench_rename(&self) {
let desc = format!("TableMetadataBencher: renaming {} table", self.count);
bench_self_recorded(
&desc,
|i| async move {
let (table_info, _) = self
.table_metadata_manager
.get_full_table_info(i)
.await
.unwrap();
let new_table_name = format!("renamed_{}", i);
let start = Instant::now();
let _ = self
.table_metadata_manager
.rename_table(table_info.unwrap(), new_table_name)
.await;
start.elapsed()
},
self.count,
)
.await;
}
}

View File

@@ -1,111 +0,0 @@
// 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;
}
}

View File

@@ -1,131 +0,0 @@
// 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)
}

View File

@@ -1,112 +0,0 @@
// 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, &region_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;
}
}

View File

@@ -24,9 +24,9 @@ use common_meta::key::datanode_table::{DatanodeTableKey, DatanodeTableValue};
use common_meta::key::schema_name::{SchemaNameKey, SchemaNameValue}; use common_meta::key::schema_name::{SchemaNameKey, SchemaNameValue};
use common_meta::key::table_info::{TableInfoKey, TableInfoValue}; use common_meta::key::table_info::{TableInfoKey, TableInfoValue};
use common_meta::key::table_name::{TableNameKey, TableNameValue}; use common_meta::key::table_name::{TableNameKey, TableNameValue};
use common_meta::key::table_region::{RegionDistribution, TableRegionKey, TableRegionValue}; use common_meta::key::table_region::{TableRegionKey, TableRegionValue};
use common_meta::key::table_route::{NextTableRouteKey, TableRouteValue as NextTableRouteValue}; use common_meta::key::table_route::{NextTableRouteKey, TableRouteValue as NextTableRouteValue};
use common_meta::key::TableMetaKey; use common_meta::key::{RegionDistribution, TableMetaKey};
use common_meta::range_stream::PaginationStream; use common_meta::range_stream::PaginationStream;
use common_meta::rpc::router::TableRoute; use common_meta::rpc::router::TableRoute;
use common_meta::rpc::store::{BatchDeleteRequest, BatchPutRequest, PutRequest, RangeRequest}; use common_meta::rpc::store::{BatchDeleteRequest, BatchPutRequest, PutRequest, RangeRequest};

View File

@@ -17,6 +17,7 @@ use std::fmt::{Display, Formatter};
use api::v1::meta::{TableIdent as RawTableIdent, TableName}; use api::v1::meta::{TableIdent as RawTableIdent, TableName};
use serde::{Deserialize, Serialize}; use serde::{Deserialize, Serialize};
use snafu::OptionExt; use snafu::OptionExt;
use table::engine::TableReference;
use crate::error::{Error, InvalidProtoMsgSnafu}; use crate::error::{Error, InvalidProtoMsgSnafu};
@@ -29,6 +30,12 @@ pub struct TableIdent {
pub engine: String, pub engine: String,
} }
impl TableIdent {
pub fn table_ref(&self) -> TableReference {
TableReference::full(&self.catalog, &self.schema, &self.table)
}
}
impl Display for TableIdent { impl Display for TableIdent {
fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result {
write!( write!(

View File

@@ -36,10 +36,6 @@
//! - The value is a [TableNameValue] struct; it contains the table id. //! - The value is a [TableNameValue] struct; it contains the table id.
//! - Used in the table name to table id lookup. //! - Used in the table name to table id lookup.
//! //!
//! 6. Table region key: `__table_region/{table_id}`
//! - The value is a [TableRegionValue] struct; it contains the region distribution of the
//! table in the Datanodes.
//!
//! All keys have related managers. The managers take care of the serialization and deserialization //! All keys have related managers. The managers take care of the serialization and deserialization
//! of keys and values, and the interaction with the underlying KV store backend. //! of keys and values, and the interaction with the underlying KV store backend.
//! //!
@@ -52,28 +48,35 @@ pub mod datanode_table;
pub mod schema_name; pub mod schema_name;
pub mod table_info; pub mod table_info;
pub mod table_name; pub mod table_name;
// TODO(weny): removes it.
#[allow(deprecated)]
pub mod table_region; pub mod table_region;
// TODO(weny): removes it.
#[allow(deprecated)]
pub mod table_route; pub mod table_route;
use std::collections::BTreeMap;
use std::sync::Arc; use std::sync::Arc;
use datanode_table::{DatanodeTableKey, DatanodeTableManager, DatanodeTableValue}; use datanode_table::{DatanodeTableKey, DatanodeTableManager, DatanodeTableValue};
use lazy_static::lazy_static; use lazy_static::lazy_static;
use regex::Regex; use regex::Regex;
use snafu::{ensure, OptionExt, ResultExt}; use snafu::{ensure, OptionExt, ResultExt};
use store_api::storage::RegionNumber;
use table::metadata::{RawTableInfo, TableId}; use table::metadata::{RawTableInfo, TableId};
use table_info::{TableInfoKey, TableInfoManager, TableInfoValue}; use table_info::{TableInfoKey, TableInfoManager, TableInfoValue};
use table_name::{TableNameKey, TableNameManager, TableNameValue}; use table_name::{TableNameKey, TableNameManager, TableNameValue};
use table_region::{TableRegionKey, TableRegionManager, TableRegionValue};
use self::catalog_name::{CatalogManager, CatalogNameValue}; use self::catalog_name::{CatalogManager, CatalogNameValue};
use self::schema_name::{SchemaManager, SchemaNameValue}; use self::schema_name::{SchemaManager, SchemaNameValue};
use self::table_route::{TableRouteManager, TableRouteValue}; use self::table_route::{TableRouteManager, TableRouteValue};
use crate::error::{self, Error, InvalidTableMetadataSnafu, Result, SerdeJsonSnafu}; use crate::error::{self, Error, InvalidTableMetadataSnafu, Result, SerdeJsonSnafu};
#[allow(deprecated)]
pub use crate::key::table_route::{TableRouteKey, TABLE_ROUTE_PREFIX}; pub use crate::key::table_route::{TableRouteKey, TABLE_ROUTE_PREFIX};
use crate::kv_backend::txn::Txn; use crate::kv_backend::txn::Txn;
use crate::kv_backend::KvBackendRef; use crate::kv_backend::KvBackendRef;
use crate::rpc::router::{region_distribution, RegionRoute}; use crate::rpc::router::{region_distribution, RegionRoute};
use crate::DatanodeId;
pub const REMOVED_PREFIX: &str = "__removed"; pub const REMOVED_PREFIX: &str = "__removed";
@@ -86,6 +89,8 @@ const TABLE_REGION_KEY_PREFIX: &str = "__table_region";
const CATALOG_NAME_KEY_PREFIX: &str = "__catalog_name"; const CATALOG_NAME_KEY_PREFIX: &str = "__catalog_name";
const SCHEMA_NAME_KEY_PREFIX: &str = "__schema_name"; const SCHEMA_NAME_KEY_PREFIX: &str = "__schema_name";
pub type RegionDistribution = BTreeMap<DatanodeId, Vec<RegionNumber>>;
lazy_static! { lazy_static! {
static ref DATANODE_TABLE_KEY_PATTERN: Regex = static ref DATANODE_TABLE_KEY_PATTERN: Regex =
Regex::new(&format!("^{DATANODE_TABLE_KEY_PREFIX}/([0-9])/([0-9])$")).unwrap(); Regex::new(&format!("^{DATANODE_TABLE_KEY_PREFIX}/([0-9])/([0-9])$")).unwrap();
@@ -127,7 +132,6 @@ pub type TableMetadataManagerRef = Arc<TableMetadataManager>;
pub struct TableMetadataManager { pub struct TableMetadataManager {
table_name_manager: TableNameManager, table_name_manager: TableNameManager,
table_info_manager: TableInfoManager, table_info_manager: TableInfoManager,
table_region_manager: TableRegionManager,
datanode_table_manager: DatanodeTableManager, datanode_table_manager: DatanodeTableManager,
catalog_manager: CatalogManager, catalog_manager: CatalogManager,
schema_manager: SchemaManager, schema_manager: SchemaManager,
@@ -154,7 +158,6 @@ impl TableMetadataManager {
TableMetadataManager { TableMetadataManager {
table_name_manager: TableNameManager::new(kv_backend.clone()), table_name_manager: TableNameManager::new(kv_backend.clone()),
table_info_manager: TableInfoManager::new(kv_backend.clone()), table_info_manager: TableInfoManager::new(kv_backend.clone()),
table_region_manager: TableRegionManager::new(kv_backend.clone()),
datanode_table_manager: DatanodeTableManager::new(kv_backend.clone()), datanode_table_manager: DatanodeTableManager::new(kv_backend.clone()),
catalog_manager: CatalogManager::new(kv_backend.clone()), catalog_manager: CatalogManager::new(kv_backend.clone()),
schema_manager: SchemaManager::new(kv_backend.clone()), schema_manager: SchemaManager::new(kv_backend.clone()),
@@ -171,10 +174,6 @@ impl TableMetadataManager {
&self.table_info_manager &self.table_info_manager
} }
pub fn table_region_manager(&self) -> &TableRegionManager {
&self.table_region_manager
}
pub fn datanode_table_manager(&self) -> &DatanodeTableManager { pub fn datanode_table_manager(&self) -> &DatanodeTableManager {
&self.datanode_table_manager &self.datanode_table_manager
} }
@@ -191,6 +190,27 @@ impl TableMetadataManager {
&self.table_route_manager &self.table_route_manager
} }
pub async fn get_full_table_info(
&self,
table_id: TableId,
) -> Result<(Option<TableInfoValue>, Option<TableRouteValue>)> {
let (get_table_route_txn, table_route_decoder) =
self.table_route_manager.build_get_txn(table_id);
let (get_table_info_txn, table_info_decoder) =
self.table_info_manager.build_get_txn(table_id);
let txn = Txn::merge_all(vec![get_table_route_txn, get_table_info_txn]);
let r = self.kv_backend.txn(txn).await?;
let table_info_value = table_info_decoder(&r.responses)?;
let table_route_value = table_route_decoder(&r.responses)?;
Ok((table_info_value, table_route_value))
}
/// Creates metadata for table and returns an error if different metadata exists. /// Creates metadata for table and returns an error if different metadata exists.
/// The caller MUST ensure it has the exclusive access to `TableNameKey`. /// The caller MUST ensure it has the exclusive access to `TableNameKey`.
pub async fn create_table_metadata( pub async fn create_table_metadata(
@@ -261,8 +281,8 @@ impl TableMetadataManager {
/// The caller MUST ensure it has the exclusive access to `TableNameKey`. /// The caller MUST ensure it has the exclusive access to `TableNameKey`.
pub async fn delete_table_metadata( pub async fn delete_table_metadata(
&self, &self,
table_info_value: TableInfoValue, table_info_value: &TableInfoValue,
region_routes: Vec<RegionRoute>, table_route_value: &TableRouteValue,
) -> Result<()> { ) -> Result<()> {
let table_info = &table_info_value.table_info; let table_info = &table_info_value.table_info;
let table_id = table_info.ident.table_id; let table_id = table_info.ident.table_id;
@@ -281,19 +301,18 @@ impl TableMetadataManager {
// Deletes table info. // Deletes table info.
let delete_table_info_txn = self let delete_table_info_txn = self
.table_info_manager() .table_info_manager()
.build_delete_txn(table_id, &table_info_value)?; .build_delete_txn(table_id, table_info_value)?;
// Deletes datanode table key value pairs. // Deletes datanode table key value pairs.
let distribution = region_distribution(&region_routes)?; let distribution = region_distribution(&table_route_value.region_routes)?;
let delete_datanode_txn = self let delete_datanode_txn = self
.datanode_table_manager() .datanode_table_manager()
.build_delete_txn(table_id, distribution)?; .build_delete_txn(table_id, distribution)?;
// Deletes table route. // Deletes table route.
let table_route_value = TableRouteValue::new(region_routes);
let delete_table_route_txn = self let delete_table_route_txn = self
.table_route_manager() .table_route_manager()
.build_delete_txn(table_id, &table_route_value)?; .build_delete_txn(table_id, table_route_value)?;
let txn = Txn::merge_all(vec![ let txn = Txn::merge_all(vec![
delete_table_name_txn, delete_table_name_txn,
@@ -338,8 +357,9 @@ impl TableMetadataManager {
table_id, table_id,
)?; )?;
let mut new_table_info_value = current_table_info_value.clone(); let new_table_info_value = current_table_info_value.with_update(move |table_info| {
new_table_info_value.table_info.name = new_table_name; table_info.name = new_table_name;
});
// Updates table info. // Updates table info.
let (update_table_info_txn, on_update_table_info_failure) = self let (update_table_info_txn, on_update_table_info_failure) = self
@@ -437,6 +457,7 @@ impl TableMetadataManager {
} }
} }
#[macro_export]
macro_rules! impl_table_meta_key { macro_rules! impl_table_meta_key {
($($val_ty: ty), *) => { ($($val_ty: ty), *) => {
$( $(
@@ -449,13 +470,9 @@ macro_rules! impl_table_meta_key {
} }
} }
impl_table_meta_key!( impl_table_meta_key!(TableNameKey<'_>, TableInfoKey, DatanodeTableKey);
TableNameKey<'_>,
TableInfoKey,
TableRegionKey,
DatanodeTableKey
);
#[macro_export]
macro_rules! impl_table_meta_value { macro_rules! impl_table_meta_value {
($($val_ty: ty), *) => { ($($val_ty: ty), *) => {
$( $(
@@ -492,14 +509,13 @@ macro_rules! impl_try_from {
}; };
} }
impl_try_from! {TableInfoValue, TableRouteValue} impl_try_from! {TableInfoValue, TableRouteValue, DatanodeTableValue}
impl_table_meta_value! { impl_table_meta_value! {
CatalogNameValue, CatalogNameValue,
SchemaNameValue, SchemaNameValue,
TableNameValue, TableNameValue,
TableInfoValue, TableInfoValue,
TableRegionValue,
DatanodeTableValue, DatanodeTableValue,
TableRouteValue TableRouteValue
} }
@@ -511,10 +527,12 @@ mod tests {
use datatypes::prelude::ConcreteDataType; use datatypes::prelude::ConcreteDataType;
use datatypes::schema::{ColumnSchema, SchemaBuilder}; use datatypes::schema::{ColumnSchema, SchemaBuilder};
use futures::TryStreamExt;
use table::metadata::{RawTableInfo, TableInfo, TableInfoBuilder, TableMetaBuilder}; use table::metadata::{RawTableInfo, TableInfo, TableInfoBuilder, TableMetaBuilder};
use super::datanode_table::DatanodeTableKey; use super::datanode_table::DatanodeTableKey;
use crate::key::table_info::TableInfoValue; use crate::key::table_info::TableInfoValue;
use crate::key::table_name::TableNameKey;
use crate::key::table_route::TableRouteValue; use crate::key::table_route::TableRouteValue;
use crate::key::{to_removed_key, TableMetadataManager}; use crate::key::{to_removed_key, TableMetadataManager};
use crate::kv_backend::memory::MemoryKvBackend; use crate::kv_backend::memory::MemoryKvBackend;
@@ -596,12 +614,20 @@ mod tests {
.await .await
.unwrap(); .unwrap();
let mut modified_region_routes = region_routes.clone(); let mut modified_region_routes = region_routes.clone();
modified_region_routes.push(region_route); modified_region_routes.push(region_route.clone());
// if remote metadata was exists, it should return an error. // if remote metadata was exists, it should return an error.
assert!(table_metadata_manager assert!(table_metadata_manager
.create_table_metadata(table_info, modified_region_routes) .create_table_metadata(table_info.clone(), modified_region_routes)
.await .await
.is_err()); .is_err());
let (remote_table_info, remote_table_route) = table_metadata_manager
.get_full_table_info(10)
.await
.unwrap();
assert_eq!(remote_table_info.unwrap().table_info, table_info);
assert_eq!(remote_table_route.unwrap().region_routes, region_routes);
} }
#[tokio::test] #[tokio::test]
@@ -609,24 +635,69 @@ mod tests {
let mem_kv = Arc::new(MemoryKvBackend::default()); let mem_kv = Arc::new(MemoryKvBackend::default());
let table_metadata_manager = TableMetadataManager::new(mem_kv); let table_metadata_manager = TableMetadataManager::new(mem_kv);
let table_info: RawTableInfo = new_test_table_info().into(); let table_info: RawTableInfo = new_test_table_info().into();
let table_id = table_info.ident.table_id;
let region_route = new_test_region_route(); let region_route = new_test_region_route();
let datanode_id = 2;
let region_routes = vec![region_route.clone()]; let region_routes = vec![region_route.clone()];
let table_route_value = TableRouteValue::new(region_routes.clone());
// creates metadata. // creates metadata.
table_metadata_manager table_metadata_manager
.create_table_metadata(table_info.clone(), region_routes.clone()) .create_table_metadata(table_info.clone(), region_routes.clone())
.await .await
.unwrap(); .unwrap();
let table_info_value = TableInfoValue::new(table_info);
let table_info_value = TableInfoValue::new(table_info.clone());
// deletes metadata. // deletes metadata.
table_metadata_manager table_metadata_manager
.delete_table_metadata(table_info_value.clone(), region_routes.clone()) .delete_table_metadata(&table_info_value, &table_route_value)
.await .await
.unwrap(); .unwrap();
// if metadata was already deleted, it should be ok. // if metadata was already deleted, it should be ok.
table_metadata_manager table_metadata_manager
.delete_table_metadata(table_info_value.clone(), region_routes.clone()) .delete_table_metadata(&table_info_value, &table_route_value)
.await .await
.unwrap(); .unwrap();
assert!(table_metadata_manager
.table_info_manager()
.get(table_id)
.await
.unwrap()
.is_none());
assert!(table_metadata_manager
.table_route_manager()
.get(table_id)
.await
.unwrap()
.is_none());
assert!(table_metadata_manager
.datanode_table_manager()
.tables(datanode_id)
.try_collect::<Vec<_>>()
.await
.unwrap()
.is_empty());
// Checks removed values
let removed_table_info = table_metadata_manager
.table_info_manager()
.get_removed(table_id)
.await
.unwrap()
.unwrap();
assert_eq!(removed_table_info.table_info, table_info);
let removed_table_route = table_metadata_manager
.table_route_manager()
.get_removed(table_id)
.await
.unwrap()
.unwrap();
assert_eq!(removed_table_route.region_routes, region_routes);
} }
#[tokio::test] #[tokio::test]
@@ -634,6 +705,7 @@ mod tests {
let mem_kv = Arc::new(MemoryKvBackend::default()); let mem_kv = Arc::new(MemoryKvBackend::default());
let table_metadata_manager = TableMetadataManager::new(mem_kv); let table_metadata_manager = TableMetadataManager::new(mem_kv);
let table_info: RawTableInfo = new_test_table_info().into(); let table_info: RawTableInfo = new_test_table_info().into();
let table_id = table_info.ident.table_id;
let region_route = new_test_region_route(); let region_route = new_test_region_route();
let region_routes = vec![region_route.clone()]; let region_routes = vec![region_route.clone()];
// creates metadata. // creates metadata.
@@ -652,7 +724,7 @@ mod tests {
.rename_table(table_info_value.clone(), new_table_name.clone()) .rename_table(table_info_value.clone(), new_table_name.clone())
.await .await
.unwrap(); .unwrap();
let mut modified_table_info = table_info; let mut modified_table_info = table_info.clone();
modified_table_info.name = "hi".to_string(); modified_table_info.name = "hi".to_string();
let modified_table_info_value = table_info_value.update(modified_table_info); let modified_table_info_value = table_info_value.update(modified_table_info);
// if the table_info_value is wrong, it should return an error. // if the table_info_value is wrong, it should return an error.
@@ -660,7 +732,36 @@ mod tests {
assert!(table_metadata_manager assert!(table_metadata_manager
.rename_table(modified_table_info_value.clone(), new_table_name.clone()) .rename_table(modified_table_info_value.clone(), new_table_name.clone())
.await .await
.is_err()) .is_err());
let old_table_name = TableNameKey::new(
&table_info.catalog_name,
&table_info.schema_name,
&table_info.name,
);
let new_table_name = TableNameKey::new(
&table_info.catalog_name,
&table_info.schema_name,
&new_table_name,
);
assert!(table_metadata_manager
.table_name_manager()
.get(old_table_name)
.await
.unwrap()
.is_none());
assert_eq!(
table_metadata_manager
.table_name_manager()
.get(new_table_name)
.await
.unwrap()
.unwrap()
.table_id(),
table_id
);
} }
#[tokio::test] #[tokio::test]
@@ -668,6 +769,7 @@ mod tests {
let mem_kv = Arc::new(MemoryKvBackend::default()); let mem_kv = Arc::new(MemoryKvBackend::default());
let table_metadata_manager = TableMetadataManager::new(mem_kv); let table_metadata_manager = TableMetadataManager::new(mem_kv);
let table_info: RawTableInfo = new_test_table_info().into(); let table_info: RawTableInfo = new_test_table_info().into();
let table_id = table_info.ident.table_id;
let region_route = new_test_region_route(); let region_route = new_test_region_route();
let region_routes = vec![region_route.clone()]; let region_routes = vec![region_route.clone()];
// creates metadata. // creates metadata.
@@ -688,6 +790,16 @@ mod tests {
.update_table_info(current_table_info_value.clone(), new_table_info.clone()) .update_table_info(current_table_info_value.clone(), new_table_info.clone())
.await .await
.unwrap(); .unwrap();
// updated table_info should equal the `new_table_info`
let updated_table_info = table_metadata_manager
.table_info_manager()
.get(table_id)
.await
.unwrap()
.unwrap();
assert_eq!(updated_table_info.table_info, new_table_info);
let mut wrong_table_info = table_info.clone(); let mut wrong_table_info = table_info.clone();
wrong_table_info.name = "wrong".to_string(); wrong_table_info.name = "wrong".to_string();
let wrong_table_info_value = current_table_info_value.update(wrong_table_info); let wrong_table_info_value = current_table_info_value.update(wrong_table_info);
@@ -759,7 +871,7 @@ mod tests {
.unwrap(); .unwrap();
let current_table_route_value = current_table_route_value.update(new_region_routes.clone()); let current_table_route_value = current_table_route_value.update(new_region_routes.clone());
let new_region_routes = vec![new_region_route(4, 4), new_region_route(5, 5)]; let new_region_routes = vec![new_region_route(2, 4), new_region_route(5, 5)];
// it should be ok. // it should be ok.
table_metadata_manager table_metadata_manager
.update_table_route( .update_table_route(

View File

@@ -12,18 +12,24 @@
// See the License for the specific language governing permissions and // See the License for the specific language governing permissions and
// limitations under the License. // limitations under the License.
use std::sync::Arc;
use futures::stream::BoxStream;
use futures::StreamExt;
use serde::{Deserialize, Serialize}; use serde::{Deserialize, Serialize};
use snafu::{ensure, OptionExt}; use snafu::OptionExt;
use store_api::storage::RegionNumber; use store_api::storage::RegionNumber;
use table::metadata::TableId; use table::metadata::TableId;
use super::table_region::RegionDistribution; use crate::error::{InvalidTableMetadataSnafu, Result};
use super::{DATANODE_TABLE_KEY_PATTERN, DATANODE_TABLE_KEY_PREFIX}; use crate::key::{
use crate::error::{InvalidTableMetadataSnafu, MoveRegionSnafu, Result, UnexpectedSnafu}; RegionDistribution, TableMetaKey, DATANODE_TABLE_KEY_PATTERN, DATANODE_TABLE_KEY_PREFIX,
use crate::key::{to_removed_key, TableMetaKey}; };
use crate::kv_backend::txn::{Compare, CompareOp, Txn, TxnOp}; use crate::kv_backend::txn::{Txn, TxnOp};
use crate::kv_backend::KvBackendRef; use crate::kv_backend::KvBackendRef;
use crate::rpc::store::{BatchGetRequest, CompareAndPutRequest, MoveValueRequest, RangeRequest}; use crate::range_stream::{PaginationStream, DEFAULT_PAGE_SIZE};
use crate::rpc::store::RangeRequest;
use crate::rpc::KeyValue;
use crate::DatanodeId; use crate::DatanodeId;
pub struct DatanodeTableKey { pub struct DatanodeTableKey {
@@ -43,6 +49,10 @@ impl DatanodeTableKey {
format!("{}/{datanode_id}", DATANODE_TABLE_KEY_PREFIX) format!("{}/{datanode_id}", DATANODE_TABLE_KEY_PREFIX)
} }
pub fn range_start_key(datanode_id: DatanodeId) -> String {
format!("{}/", Self::prefix(datanode_id))
}
pub fn strip_table_id(raw_key: &[u8]) -> Result<TableId> { pub fn strip_table_id(raw_key: &[u8]) -> Result<TableId> {
let key = String::from_utf8(raw_key.to_vec()).map_err(|e| { let key = String::from_utf8(raw_key.to_vec()).map_err(|e| {
InvalidTableMetadataSnafu { InvalidTableMetadataSnafu {
@@ -88,6 +98,13 @@ impl DatanodeTableValue {
} }
} }
/// Decodes `KeyValue` to ((),`DatanodeTableValue`)
pub fn datanode_table_value_decoder(kv: KeyValue) -> Result<((), DatanodeTableValue)> {
let value = DatanodeTableValue::try_from(&kv.value)?;
Ok(((), value))
}
pub struct DatanodeTableManager { pub struct DatanodeTableManager {
kv_backend: KvBackendRef, kv_backend: KvBackendRef,
} }
@@ -105,6 +122,23 @@ impl DatanodeTableManager {
.transpose() .transpose()
} }
pub fn tables(
&self,
datanode_id: DatanodeId,
) -> BoxStream<'static, Result<DatanodeTableValue>> {
let start_key = DatanodeTableKey::range_start_key(datanode_id);
let req = RangeRequest::new().with_prefix(start_key.as_bytes());
let stream = PaginationStream::new(
self.kv_backend.clone(),
req,
DEFAULT_PAGE_SIZE,
Arc::new(datanode_table_value_decoder),
);
Box::pin(stream.map(|kv| kv.map(|kv| kv.1)))
}
/// Builds the create datanode table transactions. It only executes while the primary keys comparing successes. /// Builds the create datanode table transactions. It only executes while the primary keys comparing successes.
pub fn build_create_txn( pub fn build_create_txn(
&self, &self,
@@ -121,7 +155,7 @@ impl DatanodeTableManager {
}) })
.collect::<Result<Vec<_>>>()?; .collect::<Result<Vec<_>>>()?;
let txn = Txn::default().and_then(txns); let txn = Txn::new().and_then(txns);
Ok(txn) Ok(txn)
} }
@@ -162,7 +196,7 @@ impl DatanodeTableManager {
} }
} }
let txn = Txn::default().and_then(opts); let txn = Txn::new().and_then(opts);
Ok(txn) Ok(txn)
} }
@@ -182,331 +216,16 @@ impl DatanodeTableManager {
}) })
.collect::<Result<Vec<_>>>()?; .collect::<Result<Vec<_>>>()?;
let txn = Txn::default().and_then(txns); let txn = Txn::new().and_then(txns);
Ok(txn) Ok(txn)
} }
/// Create DatanodeTable key and value. If the key already exists, check if the value is the same.
pub async fn create(
&self,
datanode_id: DatanodeId,
table_id: TableId,
regions: Vec<RegionNumber>,
) -> Result<()> {
let key = DatanodeTableKey::new(datanode_id, table_id);
let val = DatanodeTableValue::new(table_id, regions.clone());
let req = CompareAndPutRequest::new()
.with_key(key.as_raw_key())
.with_value(val.try_as_raw_value()?);
let resp = self.kv_backend.compare_and_put(req).await?;
if !resp.success {
let Some(curr) = resp
.prev_kv
.map(|kv| DatanodeTableValue::try_from_raw_value(kv.value))
.transpose()?
else {
return UnexpectedSnafu {
err_msg: format!("compare_and_put expect None but failed with current value None, key: {key}, val: {val:?}"),
}.fail();
};
ensure!(
curr.table_id == table_id && curr.regions == regions,
UnexpectedSnafu {
err_msg: format!("current value '{curr:?}' already existed for key '{key}', {val:?} is not set"),
}
);
}
Ok(())
}
pub async fn remove(&self, datanode_id: DatanodeId, table_id: TableId) -> Result<()> {
let key = DatanodeTableKey::new(datanode_id, table_id);
let removed_key = to_removed_key(&String::from_utf8_lossy(&key.as_raw_key()));
let req = MoveValueRequest::new(key.as_raw_key(), removed_key.as_bytes());
let _ = self.kv_backend.move_value(req).await?;
Ok(())
}
pub async fn move_region(
&self,
from_datanode: DatanodeId,
to_datanode: DatanodeId,
table_id: TableId,
region: RegionNumber,
) -> Result<()> {
let from_key = DatanodeTableKey::new(from_datanode, table_id);
let to_key = DatanodeTableKey::new(to_datanode, table_id);
let mut kvs = self
.kv_backend
.batch_get(BatchGetRequest {
keys: vec![from_key.as_raw_key(), to_key.as_raw_key()],
})
.await?
.kvs;
ensure!(
!kvs.is_empty(),
MoveRegionSnafu {
table_id,
region,
err_msg: format!("DatanodeTableKey not found for Datanode {from_datanode}"),
}
);
let mut from_value = DatanodeTableValue::try_from_raw_value(kvs.remove(0).value)?;
ensure!(
from_value.regions.contains(&region),
MoveRegionSnafu {
table_id,
region,
err_msg: format!("target region not found in Datanode {from_datanode}"),
}
);
let to_value = if !kvs.is_empty() {
Some(DatanodeTableValue::try_from_raw_value(kvs.remove(0).value)?)
} else {
None
};
if let Some(v) = to_value.as_ref() {
ensure!(
!v.regions.contains(&region),
MoveRegionSnafu {
table_id,
region,
err_msg: format!("target region already existed in Datanode {to_datanode}"),
}
);
}
let compares = vec![
Compare::with_value(
from_key.as_raw_key(),
CompareOp::Equal,
from_value.try_as_raw_value()?,
),
Compare::new(
to_key.as_raw_key(),
CompareOp::Equal,
to_value
.as_ref()
.map(|x| x.try_as_raw_value())
.transpose()?,
),
];
let mut operations = Vec::with_capacity(2);
from_value.regions.retain(|x| *x != region);
if from_value.regions.is_empty() {
operations.push(TxnOp::Delete(from_key.as_raw_key()));
} else {
from_value.version += 1;
operations.push(TxnOp::Put(
from_key.as_raw_key(),
from_value.try_as_raw_value()?,
));
}
if let Some(mut v) = to_value {
v.regions.push(region);
v.version += 1;
operations.push(TxnOp::Put(to_key.as_raw_key(), v.try_as_raw_value()?));
} else {
let v = DatanodeTableValue::new(table_id, vec![region]);
operations.push(TxnOp::Put(to_key.as_raw_key(), v.try_as_raw_value()?));
}
let txn = Txn::new().when(compares).and_then(operations);
let resp = self.kv_backend.txn(txn).await?;
ensure!(
resp.succeeded,
MoveRegionSnafu {
table_id,
region,
err_msg: format!("txn failed with responses: {:?}", resp.responses),
}
);
Ok(())
}
pub async fn tables(&self, datanode_id: DatanodeId) -> Result<Vec<DatanodeTableValue>> {
let prefix = DatanodeTableKey::prefix(datanode_id);
let req = RangeRequest::new().with_prefix(prefix.as_bytes());
let resp = self.kv_backend.range(req).await?;
let table_ids = resp
.kvs
.into_iter()
.map(|kv| DatanodeTableValue::try_from_raw_value(kv.value))
.collect::<Result<Vec<_>>>()?;
Ok(table_ids)
}
} }
#[cfg(test)] #[cfg(test)]
mod tests { mod tests {
use std::sync::Arc;
use super::*; use super::*;
use crate::kv_backend::memory::MemoryKvBackend;
use crate::kv_backend::KvBackend;
#[tokio::test]
async fn test_move_region() {
let manager = DatanodeTableManager::new(Arc::new(MemoryKvBackend::default()));
let result = manager.move_region(1, 2, 1, 1).await;
assert!(result.unwrap_err().to_string().contains(
"Failed to move region 1 in table 1, err: DatanodeTableKey not found for Datanode 1"
));
assert!(manager.create(1, 1, vec![1, 2, 3]).await.is_ok());
let result = manager.move_region(1, 2, 1, 100).await;
assert!(result.unwrap_err().to_string().contains(
"Failed to move region 100 in table 1, err: target region not found in Datanode 1"
));
// Move region 1 from datanode 1 to datanode 2.
// Note that the DatanodeTableValue is not existed for datanode 2 now.
assert!(manager.move_region(1, 2, 1, 1).await.is_ok());
let value = manager
.get(&DatanodeTableKey::new(1, 1))
.await
.unwrap()
.unwrap();
assert_eq!(
value,
DatanodeTableValue {
table_id: 1,
regions: vec![2, 3],
version: 1,
}
);
let value = manager
.get(&DatanodeTableKey::new(2, 1))
.await
.unwrap()
.unwrap();
assert_eq!(
value,
DatanodeTableValue {
table_id: 1,
regions: vec![1],
version: 0,
}
);
// Move region 2 from datanode 1 to datanode 2.
assert!(manager.move_region(1, 2, 1, 2).await.is_ok());
let value = manager
.get(&DatanodeTableKey::new(1, 1))
.await
.unwrap()
.unwrap();
assert_eq!(
value,
DatanodeTableValue {
table_id: 1,
regions: vec![3],
version: 2,
}
);
let value = manager
.get(&DatanodeTableKey::new(2, 1))
.await
.unwrap()
.unwrap();
assert_eq!(
value,
DatanodeTableValue {
table_id: 1,
regions: vec![1, 2],
version: 1,
}
);
// Move region 3 (the last region) from datanode 1 to datanode 2.
assert!(manager.move_region(1, 2, 1, 3).await.is_ok());
let value = manager.get(&DatanodeTableKey::new(1, 1)).await.unwrap();
assert!(value.is_none());
let value = manager
.get(&DatanodeTableKey::new(2, 1))
.await
.unwrap()
.unwrap();
assert_eq!(
value,
DatanodeTableValue {
table_id: 1,
regions: vec![1, 2, 3],
version: 2,
}
);
}
#[tokio::test]
async fn test_datanode_table_value_manager() {
let backend = Arc::new(MemoryKvBackend::default());
let manager = DatanodeTableManager::new(backend.clone());
assert!(manager.create(1, 1, vec![1, 2, 3]).await.is_ok());
assert!(manager.create(1, 2, vec![4, 5, 6]).await.is_ok());
assert!(manager.create(2, 1, vec![4, 5, 6]).await.is_ok());
assert!(manager.create(2, 2, vec![1, 2, 3]).await.is_ok());
// If the value is the same, "create" can be called again.
assert!(manager.create(2, 2, vec![1, 2, 3]).await.is_ok());
let err_msg = manager
.create(1, 1, vec![4, 5, 6])
.await
.unwrap_err()
.to_string();
assert!(err_msg.contains("Unexpected: current value 'DatanodeTableValue { table_id: 1, regions: [1, 2, 3], version: 0 }' already existed for key '__dn_table/1/1', DatanodeTableValue { table_id: 1, regions: [4, 5, 6], version: 0 } is not set"));
let to_be_removed_key = DatanodeTableKey::new(2, 1);
let expected_value = DatanodeTableValue {
table_id: 1,
regions: vec![4, 5, 6],
version: 0,
};
let value = manager.get(&to_be_removed_key).await.unwrap().unwrap();
assert_eq!(value, expected_value);
assert!(manager.remove(2, 1).await.is_ok());
assert!(manager.get(&to_be_removed_key).await.unwrap().is_none());
let kv = backend
.get(b"__removed-__dn_table/2/1")
.await
.unwrap()
.unwrap();
assert_eq!(b"__removed-__dn_table/2/1", kv.key());
let value = DatanodeTableValue::try_from_raw_value(kv.value).unwrap();
assert_eq!(value, expected_value);
let values = manager.tables(1).await.unwrap();
assert_eq!(values.len(), 2);
assert_eq!(
values[0],
DatanodeTableValue {
table_id: 1,
regions: vec![1, 2, 3],
version: 0,
}
);
assert_eq!(
values[1],
DatanodeTableValue {
table_id: 2,
regions: vec![4, 5, 6],
version: 0,
}
);
}
#[test] #[test]
fn test_serde() { fn test_serde() {

View File

@@ -72,7 +72,7 @@ impl TableMetaKey for SchemaNameKey<'_> {
} }
} }
/// Decoder `KeyValue` to ({schema},()) /// Decodes `KeyValue` to ({schema},())
pub fn schema_decoder(kv: KeyValue) -> Result<(String, ())> { pub fn schema_decoder(kv: KeyValue) -> Result<(String, ())> {
let str = std::str::from_utf8(&kv.key).context(error::ConvertRawKeySnafu)?; let str = std::str::from_utf8(&kv.key).context(error::ConvertRawKeySnafu)?;
let schema_name = SchemaNameKey::try_from(str)?; let schema_name = SchemaNameKey::try_from(str)?;

View File

@@ -13,15 +13,15 @@
// limitations under the License. // limitations under the License.
use serde::{Deserialize, Serialize}; use serde::{Deserialize, Serialize};
use snafu::ensure; use table::engine::TableReference;
use table::metadata::{RawTableInfo, TableId}; use table::metadata::{RawTableInfo, TableId};
use super::TABLE_INFO_KEY_PREFIX; use super::TABLE_INFO_KEY_PREFIX;
use crate::error::{Result, UnexpectedSnafu}; use crate::error::Result;
use crate::key::{to_removed_key, TableMetaKey}; use crate::key::{to_removed_key, TableMetaKey};
use crate::kv_backend::txn::{Compare, CompareOp, Txn, TxnOp, TxnOpResponse}; use crate::kv_backend::txn::{Compare, CompareOp, Txn, TxnOp, TxnOpResponse};
use crate::kv_backend::KvBackendRef; use crate::kv_backend::KvBackendRef;
use crate::rpc::store::{CompareAndPutRequest, MoveValueRequest}; use crate::table_name::TableName;
pub struct TableInfoKey { pub struct TableInfoKey {
table_id: TableId, table_id: TableId,
@@ -59,6 +59,34 @@ impl TableInfoValue {
version: self.version + 1, version: self.version + 1,
} }
} }
pub(crate) fn with_update<F>(&self, update: F) -> Self
where
F: FnOnce(&mut RawTableInfo),
{
let mut new_table_info = self.table_info.clone();
update(&mut new_table_info);
Self {
table_info: new_table_info,
version: self.version + 1,
}
}
pub fn table_ref(&self) -> TableReference {
TableReference::full(
&self.table_info.catalog_name,
&self.table_info.schema_name,
&self.table_info.name,
)
}
pub fn table_name(&self) -> TableName {
TableName {
catalog_name: self.table_info.catalog_name.to_string(),
schema_name: self.table_info.schema_name.to_string(),
table_name: self.table_info.name.to_string(),
}
}
} }
pub struct TableInfoManager { pub struct TableInfoManager {
@@ -70,6 +98,20 @@ impl TableInfoManager {
Self { kv_backend } Self { kv_backend }
} }
pub(crate) fn build_get_txn(
&self,
table_id: TableId,
) -> (
Txn,
impl FnOnce(&Vec<TxnOpResponse>) -> Result<Option<TableInfoValue>>,
) {
let key = TableInfoKey::new(table_id);
let raw_key = key.as_raw_key();
let txn = Txn::new().and_then(vec![TxnOp::Get(raw_key.clone())]);
(txn, Self::build_decode_fn(raw_key))
}
/// Builds a create table info transaction, it expected the `__table_info/{table_id}` wasn't occupied. /// Builds a create table info transaction, it expected the `__table_info/{table_id}` wasn't occupied.
pub(crate) fn build_create_txn( pub(crate) fn build_create_txn(
&self, &self,
@@ -82,7 +124,7 @@ impl TableInfoManager {
let key = TableInfoKey::new(table_id); let key = TableInfoKey::new(table_id);
let raw_key = key.as_raw_key(); let raw_key = key.as_raw_key();
let txn = Txn::default() let txn = Txn::new()
.when(vec![Compare::with_not_exist_value( .when(vec![Compare::with_not_exist_value(
raw_key.clone(), raw_key.clone(),
CompareOp::Equal, CompareOp::Equal,
@@ -111,7 +153,7 @@ impl TableInfoManager {
let raw_key = key.as_raw_key(); let raw_key = key.as_raw_key();
let raw_value = current_table_info_value.try_as_raw_value()?; let raw_value = current_table_info_value.try_as_raw_value()?;
let txn = Txn::default() let txn = Txn::new()
.when(vec![Compare::with_value( .when(vec![Compare::with_value(
raw_key.clone(), raw_key.clone(),
CompareOp::Equal, CompareOp::Equal,
@@ -137,7 +179,7 @@ impl TableInfoManager {
let raw_value = table_info_value.try_as_raw_value()?; let raw_value = table_info_value.try_as_raw_value()?;
let removed_key = to_removed_key(&String::from_utf8_lossy(&raw_key)); let removed_key = to_removed_key(&String::from_utf8_lossy(&raw_key));
let txn = Txn::default().and_then(vec![ let txn = Txn::new().and_then(vec![
TxnOp::Delete(raw_key.clone()), TxnOp::Delete(raw_key.clone()),
TxnOp::Put(removed_key.into_bytes(), raw_value), TxnOp::Put(removed_key.into_bytes(), raw_value),
]); ]);
@@ -164,6 +206,17 @@ impl TableInfoManager {
} }
} }
#[cfg(test)]
pub async fn get_removed(&self, table_id: TableId) -> Result<Option<TableInfoValue>> {
let key = TableInfoKey::new(table_id).to_string();
let removed_key = to_removed_key(&key).into_bytes();
self.kv_backend
.get(&removed_key)
.await?
.map(|x| TableInfoValue::try_from_raw_value(x.value))
.transpose()
}
pub async fn get(&self, table_id: TableId) -> Result<Option<TableInfoValue>> { pub async fn get(&self, table_id: TableId) -> Result<Option<TableInfoValue>> {
let key = TableInfoKey::new(table_id); let key = TableInfoKey::new(table_id);
let raw_key = key.as_raw_key(); let raw_key = key.as_raw_key();
@@ -173,95 +226,16 @@ impl TableInfoManager {
.map(|x| TableInfoValue::try_from_raw_value(x.value)) .map(|x| TableInfoValue::try_from_raw_value(x.value))
.transpose() .transpose()
} }
/// Create TableInfo key and value. If the key already exists, check if the value is the same.
pub async fn create(&self, table_id: TableId, table_info: &RawTableInfo) -> Result<()> {
let result = self
.compare_and_put(table_id, None, table_info.clone())
.await?;
if let Err(curr) = result {
let Some(curr) = curr else {
return UnexpectedSnafu {
err_msg: format!("compare_and_put expect None but failed with current value None, table_id: {table_id}, table_info: {table_info:?}"),
}.fail();
};
ensure!(
&curr.table_info == table_info,
UnexpectedSnafu {
err_msg: format!(
"TableInfoValue for table {table_id} is updated before it is created!"
)
}
)
}
Ok(())
}
/// Compare and put value of key. `expect` is the expected value, if backend's current value associated
/// with key is the same as `expect`, the value will be updated to `val`.
///
/// - If the compare-and-set operation successfully updated value, this method will return an `Ok(Ok())`
/// - If associated value is not the same as `expect`, no value will be updated and an
/// `Ok(Err(Option<TableInfoValue>))` will be returned. The `Option<TableInfoValue>` indicates
/// the current associated value of key.
/// - If any error happens during operation, an `Err(Error)` will be returned.
pub async fn compare_and_put(
&self,
table_id: TableId,
expect: Option<TableInfoValue>,
table_info: RawTableInfo,
) -> Result<std::result::Result<(), Option<TableInfoValue>>> {
let key = TableInfoKey::new(table_id);
let raw_key = key.as_raw_key();
let (expect, version) = if let Some(x) = expect {
(x.try_as_raw_value()?, x.version + 1)
} else {
(vec![], 0)
};
let value = TableInfoValue {
table_info,
version,
};
let raw_value = value.try_as_raw_value()?;
let req = CompareAndPutRequest::new()
.with_key(raw_key)
.with_expect(expect)
.with_value(raw_value);
let resp = self.kv_backend.compare_and_put(req).await?;
Ok(if resp.success {
Ok(())
} else {
Err(resp
.prev_kv
.map(|x| TableInfoValue::try_from_raw_value(x.value))
.transpose()?)
})
}
pub async fn remove(&self, table_id: TableId) -> Result<()> {
let key = TableInfoKey::new(table_id).as_raw_key();
let removed_key = to_removed_key(&String::from_utf8_lossy(&key));
let req = MoveValueRequest::new(key, removed_key.as_bytes());
self.kv_backend.move_value(req).await?;
Ok(())
}
} }
#[cfg(test)] #[cfg(test)]
mod tests { mod tests {
use std::sync::Arc;
use datatypes::prelude::ConcreteDataType; use datatypes::prelude::ConcreteDataType;
use datatypes::schema::{ColumnSchema, RawSchema, Schema}; use datatypes::schema::{ColumnSchema, RawSchema, Schema};
use table::metadata::{RawTableMeta, TableIdent, TableType}; use table::metadata::{RawTableMeta, TableIdent, TableType};
use super::*; use super::*;
use crate::kv_backend::memory::MemoryKvBackend;
use crate::kv_backend::KvBackend;
use crate::rpc::store::PutRequest;
#[test] #[test]
fn test_deserialization_compatibility() { fn test_deserialization_compatibility() {
@@ -270,97 +244,6 @@ mod tests {
assert!(v.table_info.meta.partition_key_indices.is_empty()); assert!(v.table_info.meta.partition_key_indices.is_empty());
} }
#[tokio::test]
async fn test_table_info_manager() {
let backend = Arc::new(MemoryKvBackend::default());
for i in 1..=3 {
let key = TableInfoKey::new(i).as_raw_key();
let val = TableInfoValue {
table_info: new_table_info(i),
version: 1,
}
.try_as_raw_value()
.unwrap();
let req = PutRequest::new().with_key(key).with_value(val);
backend.put(req).await.unwrap();
}
let manager = TableInfoManager::new(backend.clone());
assert!(manager.create(99, &new_table_info(99)).await.is_ok());
assert!(manager.create(99, &new_table_info(99)).await.is_ok());
let result = manager.create(99, &new_table_info(88)).await;
let err_msg = result.unwrap_err().to_string();
assert!(err_msg
.contains("Unexpected: TableInfoValue for table 99 is updated before it is created!"));
let val = manager.get(1).await.unwrap().unwrap();
assert_eq!(
val,
TableInfoValue {
table_info: new_table_info(1),
version: 1,
}
);
assert!(manager.get(4).await.unwrap().is_none());
// test cas failed, current value is not set
let table_info = new_table_info(4);
let result = manager
.compare_and_put(
4,
Some(TableInfoValue {
table_info: table_info.clone(),
version: 0,
}),
table_info.clone(),
)
.await
.unwrap();
assert!(result.unwrap_err().is_none());
let result = manager
.compare_and_put(4, None, table_info.clone())
.await
.unwrap();
assert!(result.is_ok());
// test cas failed, the new table info is not set
let new_table_info = new_table_info(4);
let result = manager
.compare_and_put(4, None, new_table_info.clone())
.await
.unwrap();
let actual = result.unwrap_err().unwrap();
assert_eq!(
actual,
TableInfoValue {
table_info: table_info.clone(),
version: 0,
}
);
// test cas success
let result = manager
.compare_and_put(4, Some(actual), new_table_info.clone())
.await
.unwrap();
assert!(result.is_ok());
assert!(manager.remove(4).await.is_ok());
let kv = backend
.get(b"__removed-__table_info/4")
.await
.unwrap()
.unwrap();
assert_eq!(b"__removed-__table_info/4", kv.key.as_slice());
let value = TableInfoValue::try_from_raw_value(kv.value).unwrap();
assert_eq!(value.table_info, new_table_info);
assert_eq!(value.version, 1);
}
#[test] #[test]
fn test_key_serde() { fn test_key_serde() {
let key = TableInfoKey::new(42); let key = TableInfoKey::new(42);

View File

@@ -15,19 +15,16 @@
use std::sync::Arc; use std::sync::Arc;
use serde::{Deserialize, Serialize}; use serde::{Deserialize, Serialize};
use snafu::{ensure, OptionExt}; use snafu::OptionExt;
use table::metadata::TableId; use table::metadata::TableId;
use super::{TABLE_NAME_KEY_PATTERN, TABLE_NAME_KEY_PREFIX}; use super::{TABLE_NAME_KEY_PATTERN, TABLE_NAME_KEY_PREFIX};
use crate::error::{ use crate::error::{Error, InvalidTableMetadataSnafu, Result};
Error, InvalidTableMetadataSnafu, RenameTableSnafu, Result, TableAlreadyExistsSnafu,
TableNotExistSnafu, UnexpectedSnafu,
};
use crate::key::{to_removed_key, TableMetaKey}; use crate::key::{to_removed_key, TableMetaKey};
use crate::kv_backend::memory::MemoryKvBackend; use crate::kv_backend::memory::MemoryKvBackend;
use crate::kv_backend::txn::{Compare, CompareOp, Txn, TxnOp}; use crate::kv_backend::txn::{Txn, TxnOp};
use crate::kv_backend::KvBackendRef; use crate::kv_backend::KvBackendRef;
use crate::rpc::store::{CompareAndPutRequest, MoveValueRequest, RangeRequest}; use crate::rpc::store::RangeRequest;
use crate::table_name::TableName; use crate::table_name::TableName;
#[derive(Debug, Clone, Copy)] #[derive(Debug, Clone, Copy)]
@@ -160,7 +157,7 @@ impl TableNameManager {
let value = TableNameValue::new(table_id); let value = TableNameValue::new(table_id);
let raw_value = value.try_as_raw_value()?; let raw_value = value.try_as_raw_value()?;
let txn = Txn::default().and_then(vec![TxnOp::Put(raw_key, raw_value)]); let txn = Txn::new().and_then(vec![TxnOp::Put(raw_key, raw_value)]);
Ok(txn) Ok(txn)
} }
@@ -177,7 +174,7 @@ impl TableNameManager {
let value = TableNameValue::new(table_id); let value = TableNameValue::new(table_id);
let raw_value = value.try_as_raw_value()?; let raw_value = value.try_as_raw_value()?;
let txn = Txn::default().and_then(vec![ let txn = Txn::new().and_then(vec![
TxnOp::Delete(raw_key), TxnOp::Delete(raw_key),
TxnOp::Put(new_raw_key, raw_value), TxnOp::Put(new_raw_key, raw_value),
]); ]);
@@ -195,7 +192,7 @@ impl TableNameManager {
let raw_value = value.try_as_raw_value()?; let raw_value = value.try_as_raw_value()?;
let removed_key = to_removed_key(&String::from_utf8_lossy(&raw_key)); let removed_key = to_removed_key(&String::from_utf8_lossy(&raw_key));
let txn = Txn::default().and_then(vec![ let txn = Txn::new().and_then(vec![
TxnOp::Delete(raw_key), TxnOp::Delete(raw_key),
TxnOp::Put(removed_key.into_bytes(), raw_value), TxnOp::Put(removed_key.into_bytes(), raw_value),
]); ]);
@@ -203,97 +200,6 @@ impl TableNameManager {
Ok(txn) Ok(txn)
} }
/// Create TableName key and value. If the key already exists, check if the value is the same.
pub async fn create(&self, key: &TableNameKey<'_>, table_id: TableId) -> Result<()> {
let raw_key = key.as_raw_key();
let value = TableNameValue::new(table_id);
let raw_value = value.try_as_raw_value()?;
let req = CompareAndPutRequest::new()
.with_key(raw_key)
.with_value(raw_value);
let result = self.kv_backend.compare_and_put(req).await?;
if !result.success {
let Some(curr) = result
.prev_kv
.map(|x| TableNameValue::try_from_raw_value(x.value))
.transpose()?
else {
return UnexpectedSnafu {
err_msg: format!("compare_and_put expect None but failed with current value None, key: {key}, value: {value:?}"),
}.fail();
};
ensure!(
curr.table_id == table_id,
TableAlreadyExistsSnafu {
table_id: curr.table_id
}
);
}
Ok(())
}
/// Rename a TableNameKey to a new table name. Will check whether the TableNameValue matches the
/// `expected_table_id` first. Can be executed again if the first invocation is successful.
pub async fn rename(
&self,
key: TableNameKey<'_>,
expected_table_id: TableId,
new_table_name: &str,
) -> Result<()> {
let new_key = TableNameKey::new(key.catalog, key.schema, new_table_name);
if let Some(value) = self.get(key).await? {
ensure!(
value.table_id == expected_table_id,
RenameTableSnafu {
reason: format!(
"the input table name '{}' and id '{expected_table_id}' not match",
Into::<TableName>::into(key)
),
}
);
let txn = Txn::new()
.when(vec![
Compare::with_value(
key.as_raw_key(),
CompareOp::Equal,
value.try_as_raw_value()?,
),
Compare::with_not_exist_value(new_key.as_raw_key(), CompareOp::Equal),
])
.and_then(vec![
TxnOp::Delete(key.as_raw_key()),
TxnOp::Put(new_key.as_raw_key(), value.try_as_raw_value()?),
]);
let resp = self.kv_backend.txn(txn).await?;
ensure!(
resp.succeeded,
RenameTableSnafu {
reason: format!("txn failed with response: {:?}", resp.responses)
}
);
} else {
let Some(value) = self.get(new_key).await? else {
// If we can't get the table by its original name, nor can we get by its altered
// name, then the table must not exist at the first place.
return TableNotExistSnafu {
table_name: TableName::from(key).to_string(),
}
.fail();
};
ensure!(
value.table_id == expected_table_id,
TableAlreadyExistsSnafu {
table_id: value.table_id
}
);
}
Ok(())
}
pub async fn get(&self, key: TableNameKey<'_>) -> Result<Option<TableNameValue>> { pub async fn get(&self, key: TableNameKey<'_>) -> Result<Option<TableNameValue>> {
let raw_key = key.as_raw_key(); let raw_key = key.as_raw_key();
self.kv_backend self.kv_backend
@@ -303,6 +209,11 @@ impl TableNameManager {
.transpose() .transpose()
} }
pub async fn exists(&self, key: TableNameKey<'_>) -> Result<bool> {
let raw_key = key.as_raw_key();
self.kv_backend.exists(&raw_key).await
}
pub async fn tables( pub async fn tables(
&self, &self,
catalog: &str, catalog: &str,
@@ -321,90 +232,12 @@ impl TableNameManager {
} }
Ok(res) Ok(res)
} }
pub async fn remove(&self, key: TableNameKey<'_>) -> Result<()> {
let raw_key = key.as_raw_key();
let removed_key = to_removed_key(&String::from_utf8_lossy(&raw_key));
let req = MoveValueRequest::new(raw_key, removed_key.as_bytes());
let _ = self.kv_backend.move_value(req).await?;
Ok(())
}
} }
#[cfg(test)] #[cfg(test)]
mod tests { mod tests {
use std::sync::Arc;
use super::*; use super::*;
use crate::kv_backend::memory::MemoryKvBackend;
use crate::kv_backend::KvBackend;
#[tokio::test]
async fn test_table_name_manager() {
let backend = Arc::new(MemoryKvBackend::default());
let manager = TableNameManager::new(backend.clone());
for i in 1..=3 {
let table_name = format!("table_{}", i);
let key = TableNameKey::new("my_catalog", "my_schema", &table_name);
assert!(manager.create(&key, i).await.is_ok());
}
let key = TableNameKey::new("my_catalog", "my_schema", "my_table");
assert!(manager.create(&key, 99).await.is_ok());
assert!(manager.create(&key, 99).await.is_ok());
let result = manager.create(&key, 9).await;
let err_msg = result.unwrap_err().to_string();
assert!(err_msg.contains("Table already exists, table_id: 99"));
let value = manager.get(key).await.unwrap().unwrap();
assert_eq!(value.table_id(), 99);
let not_existed = TableNameKey::new("x", "y", "z");
assert!(manager.get(not_existed).await.unwrap().is_none());
assert!(manager.remove(key).await.is_ok());
let kv = backend
.get(b"__removed-__table_name/my_catalog/my_schema/my_table")
.await
.unwrap()
.unwrap();
let value = TableNameValue::try_from_raw_value(kv.value).unwrap();
assert_eq!(value.table_id(), 99);
let key = TableNameKey::new("my_catalog", "my_schema", "table_1");
assert!(manager.rename(key, 1, "table_1_new").await.is_ok());
assert!(manager.rename(key, 1, "table_1_new").await.is_ok());
let result = manager.rename(key, 2, "table_1_new").await;
let err_msg = result.unwrap_err().to_string();
assert!(err_msg.contains("Table already exists, table_id: 1"));
let result = manager
.rename(
TableNameKey::new("my_catalog", "my_schema", "table_2"),
22,
"table_2_new",
)
.await;
let err_msg = result.unwrap_err().to_string();
assert!(err_msg.contains("Failed to rename table, reason: the input table name 'my_catalog.my_schema.table_2' and id '22' not match"));
let result = manager.rename(not_existed, 1, "zz").await;
let err_msg = result.unwrap_err().to_string();
assert!(err_msg.contains("Table does not exist, table_name: x.y.z"));
let tables = manager.tables("my_catalog", "my_schema").await.unwrap();
assert_eq!(tables.len(), 3);
assert_eq!(
tables,
vec![
("table_1_new".to_string(), TableNameValue::new(1)),
("table_2".to_string(), TableNameValue::new(2)),
("table_3".to_string(), TableNameValue::new(3))
]
)
}
#[test] #[test]
fn test_strip_table_name() { fn test_strip_table_name() {

View File

@@ -15,19 +15,21 @@
use std::collections::BTreeMap; use std::collections::BTreeMap;
use serde::{Deserialize, Serialize}; use serde::{Deserialize, Serialize};
use snafu::ensure; use snafu::ResultExt;
use store_api::storage::RegionNumber; use store_api::storage::RegionNumber;
use table::metadata::TableId; use table::metadata::TableId;
use super::TABLE_REGION_KEY_PREFIX; use super::TABLE_REGION_KEY_PREFIX;
use crate::error::{Result, UnexpectedSnafu}; use crate::error::{InvalidTableMetadataSnafu, Result, SerdeJsonSnafu};
use crate::key::{to_removed_key, TableMetaKey}; use crate::key::TableMetaKey;
use crate::kv_backend::KvBackendRef; use crate::{impl_table_meta_key, impl_table_meta_value, DatanodeId};
use crate::rpc::store::{CompareAndPutRequest, MoveValueRequest};
use crate::DatanodeId;
pub type RegionDistribution = BTreeMap<DatanodeId, Vec<RegionNumber>>; pub type RegionDistribution = BTreeMap<DatanodeId, Vec<RegionNumber>>;
#[deprecated(
since = "0.4.0",
note = "Please use the TableRouteManager's get_region_distribution method instead"
)]
pub struct TableRegionKey { pub struct TableRegionKey {
table_id: TableId, table_id: TableId,
} }
@@ -44,6 +46,12 @@ impl TableMetaKey for TableRegionKey {
} }
} }
impl_table_meta_key! {TableRegionKey}
#[deprecated(
since = "0.4.0",
note = "Please use the TableRouteManager's get_region_distribution method instead"
)]
#[derive(Debug, Clone, Serialize, Deserialize, PartialEq)] #[derive(Debug, Clone, Serialize, Deserialize, PartialEq)]
pub struct TableRegionValue { pub struct TableRegionValue {
pub region_distribution: RegionDistribution, pub region_distribution: RegionDistribution,
@@ -59,196 +67,12 @@ impl TableRegionValue {
} }
} }
pub struct TableRegionManager { impl_table_meta_value! {TableRegionValue}
kv_backend: KvBackendRef,
}
impl TableRegionManager {
pub fn new(kv_backend: KvBackendRef) -> Self {
Self { kv_backend }
}
pub async fn get(&self, table_id: TableId) -> Result<Option<TableRegionValue>> {
let key = TableRegionKey::new(table_id);
let raw_key = key.as_raw_key();
self.kv_backend
.get(&raw_key)
.await?
.map(|x| TableRegionValue::try_from_raw_value(x.value))
.transpose()
}
/// Create TableRegion key and value. If the key already exists, check if the value is the same.
pub async fn create(
&self,
table_id: TableId,
region_distribution: &RegionDistribution,
) -> Result<()> {
let result = self
.compare_and_put(table_id, None, region_distribution.clone())
.await?;
if let Err(curr) = result {
let Some(curr) = curr else {
return UnexpectedSnafu {
err_msg: format!("compare_and_put expect None but failed with current value None, table_id: {table_id}, region_distribution: {region_distribution:?}"),
}.fail();
};
ensure!(
&curr.region_distribution == region_distribution,
UnexpectedSnafu {
err_msg: format!(
"TableRegionValue for table {table_id} is updated before it is created!"
)
}
)
}
Ok(())
}
/// Compare and put value of key. `expect` is the expected value, if backend's current value associated
/// with key is the same as `expect`, the value will be updated to `val`.
///
/// - If the compare-and-set operation successfully updated value, this method will return an `Ok(Ok())`
/// - If associated value is not the same as `expect`, no value will be updated and an `Ok(Err(Vec<u8>))`
/// will be returned, the `Err(Vec<u8>)` indicates the current associated value of key.
/// - If any error happens during operation, an `Err(Error)` will be returned.
pub async fn compare_and_put(
&self,
table_id: TableId,
expect: Option<TableRegionValue>,
region_distribution: RegionDistribution,
) -> Result<std::result::Result<(), Option<TableRegionValue>>> {
let key = TableRegionKey::new(table_id);
let raw_key = key.as_raw_key();
let (expect, version) = if let Some(x) = expect {
(x.try_as_raw_value()?, x.version + 1)
} else {
(vec![], 0)
};
let value = TableRegionValue {
region_distribution,
version,
};
let raw_value = value.try_as_raw_value()?;
let req = CompareAndPutRequest::new()
.with_key(raw_key)
.with_expect(expect)
.with_value(raw_value);
let resp = self.kv_backend.compare_and_put(req).await?;
Ok(if resp.success {
Ok(())
} else {
Err(resp
.prev_kv
.map(|x| TableRegionValue::try_from_raw_value(x.value))
.transpose()?)
})
}
pub async fn remove(&self, table_id: TableId) -> Result<Option<TableRegionValue>> {
let key = TableRegionKey::new(table_id).as_raw_key();
let remove_key = to_removed_key(&String::from_utf8_lossy(&key));
let req = MoveValueRequest::new(key, remove_key.as_bytes());
let resp = self.kv_backend.move_value(req).await?;
resp.0
.map(|x| TableRegionValue::try_from_raw_value(x.value))
.transpose()
}
}
#[cfg(test)] #[cfg(test)]
mod tests { mod tests {
use std::sync::Arc;
use super::*; use super::*;
use crate::kv_backend::memory::MemoryKvBackend;
use crate::kv_backend::KvBackend;
#[tokio::test]
async fn test_table_region_manager() {
let backend = Arc::new(MemoryKvBackend::default());
let manager = TableRegionManager::new(backend.clone());
let region_distribution =
RegionDistribution::from([(1, vec![1, 2, 3]), (2, vec![4, 5, 6])]);
let new_region_distribution =
RegionDistribution::from([(1, vec![4, 5, 6]), (2, vec![1, 2, 3])]);
let result = manager
.compare_and_put(1, None, region_distribution.clone())
.await
.unwrap();
assert!(result.is_ok());
let curr = manager
.compare_and_put(1, None, new_region_distribution.clone())
.await
.unwrap()
.unwrap_err()
.unwrap();
assert_eq!(
curr,
TableRegionValue {
region_distribution: region_distribution.clone(),
version: 0
}
);
assert!(manager
.compare_and_put(1, Some(curr), new_region_distribution.clone())
.await
.unwrap()
.is_ok());
assert!(manager.create(99, &region_distribution).await.is_ok());
assert!(manager.create(99, &region_distribution).await.is_ok());
let result = manager.create(99, &new_region_distribution).await;
let err_msg = result.unwrap_err().to_string();
assert!(err_msg.contains("TableRegionValue for table 99 is updated before it is created!"));
let value = manager.get(1).await.unwrap().unwrap();
assert_eq!(
value,
TableRegionValue {
region_distribution: new_region_distribution.clone(),
version: 1
}
);
let value = manager.get(99).await.unwrap().unwrap();
assert_eq!(
value,
TableRegionValue {
region_distribution,
version: 0
}
);
assert!(manager.get(2).await.unwrap().is_none());
let value = manager.remove(1).await.unwrap().unwrap();
assert_eq!(
value,
TableRegionValue {
region_distribution: new_region_distribution.clone(),
version: 1
}
);
assert!(manager.remove(123).await.unwrap().is_none());
let kv = backend
.get(b"__removed-__table_region/1")
.await
.unwrap()
.unwrap();
assert_eq!(b"__removed-__table_region/1", kv.key.as_slice());
let value = TableRegionValue::try_from_raw_value(kv.value).unwrap();
assert_eq!(value.region_distribution, new_region_distribution);
assert_eq!(value.version, 1);
}
#[test] #[test]
fn test_serde() { fn test_serde() {

View File

@@ -16,15 +16,13 @@ use std::fmt::Display;
use api::v1::meta::TableName; use api::v1::meta::TableName;
use serde::{Deserialize, Serialize}; use serde::{Deserialize, Serialize};
use snafu::ensure;
use table::metadata::TableId; use table::metadata::TableId;
use crate::error::{Result, UnexpectedSnafu}; use crate::error::Result;
use crate::key::{to_removed_key, TableMetaKey}; use crate::key::{to_removed_key, RegionDistribution, TableMetaKey};
use crate::kv_backend::txn::{Compare, CompareOp, Txn, TxnOp, TxnOpResponse}; use crate::kv_backend::txn::{Compare, CompareOp, Txn, TxnOp, TxnOpResponse};
use crate::kv_backend::KvBackendRef; use crate::kv_backend::KvBackendRef;
use crate::rpc::router::RegionRoute; use crate::rpc::router::{region_distribution, RegionRoute};
use crate::rpc::store::{CompareAndPutRequest, MoveValueRequest};
pub const TABLE_ROUTE_PREFIX: &str = "__meta_table_route"; pub const TABLE_ROUTE_PREFIX: &str = "__meta_table_route";
@@ -84,6 +82,20 @@ impl TableRouteManager {
Self { kv_backend } Self { kv_backend }
} }
pub(crate) fn build_get_txn(
&self,
table_id: TableId,
) -> (
Txn,
impl FnOnce(&Vec<TxnOpResponse>) -> Result<Option<TableRouteValue>>,
) {
let key = NextTableRouteKey::new(table_id);
let raw_key = key.as_raw_key();
let txn = Txn::new().and_then(vec![TxnOp::Get(raw_key.clone())]);
(txn, Self::build_decode_fn(raw_key))
}
/// Builds a create table route transaction. it expected the `__table_route/{table_id}` wasn't occupied. /// Builds a create table route transaction. it expected the `__table_route/{table_id}` wasn't occupied.
pub(crate) fn build_create_txn( pub(crate) fn build_create_txn(
&self, &self,
@@ -96,7 +108,7 @@ impl TableRouteManager {
let key = NextTableRouteKey::new(table_id); let key = NextTableRouteKey::new(table_id);
let raw_key = key.as_raw_key(); let raw_key = key.as_raw_key();
let txn = Txn::default() let txn = Txn::new()
.when(vec![Compare::with_not_exist_value( .when(vec![Compare::with_not_exist_value(
raw_key.clone(), raw_key.clone(),
CompareOp::Equal, CompareOp::Equal,
@@ -126,7 +138,7 @@ impl TableRouteManager {
let raw_value = current_table_route_value.try_as_raw_value()?; let raw_value = current_table_route_value.try_as_raw_value()?;
let new_raw_value: Vec<u8> = new_table_route_value.try_as_raw_value()?; let new_raw_value: Vec<u8> = new_table_route_value.try_as_raw_value()?;
let txn = Txn::default() let txn = Txn::new()
.when(vec![Compare::with_value( .when(vec![Compare::with_value(
raw_key.clone(), raw_key.clone(),
CompareOp::Equal, CompareOp::Equal,
@@ -149,7 +161,7 @@ impl TableRouteManager {
let raw_value = table_route_value.try_as_raw_value()?; let raw_value = table_route_value.try_as_raw_value()?;
let removed_key = to_removed_key(&String::from_utf8_lossy(&raw_key)); let removed_key = to_removed_key(&String::from_utf8_lossy(&raw_key));
let txn = Txn::default().and_then(vec![ let txn = Txn::new().and_then(vec![
TxnOp::Delete(raw_key), TxnOp::Delete(raw_key),
TxnOp::Put(removed_key.into_bytes(), raw_value), TxnOp::Put(removed_key.into_bytes(), raw_value),
]); ]);
@@ -186,91 +198,29 @@ impl TableRouteManager {
.transpose() .transpose()
} }
// Creates TableRoute key and value. If the key already exists, check whether the value is the same. #[cfg(test)]
pub async fn create(&self, table_id: TableId, region_routes: Vec<RegionRoute>) -> Result<()> { pub async fn get_removed(&self, table_id: TableId) -> Result<Option<TableRouteValue>> {
let key = NextTableRouteKey::new(table_id); let key = NextTableRouteKey::new(table_id).to_string();
let val = TableRouteValue::new(region_routes); let removed_key = to_removed_key(&key).into_bytes();
let req = CompareAndPutRequest::new() self.kv_backend
.with_key(key.as_raw_key()) .get(&removed_key)
.with_value(val.try_as_raw_value()?); .await?
.map(|x| TableRouteValue::try_from_raw_value(x.value))
self.kv_backend.compare_and_put(req).await?.handle(|resp| { .transpose()
if !resp.success {
let Some(cur) = resp
.prev_kv
.map(|kv|TableRouteValue::try_from_raw_value(kv.value))
.transpose()?
else {
return UnexpectedSnafu {
err_msg: format!("compare_and_put expect None but failed with current value None, key: {key}, val: {val:?}"),
}.fail();
};
ensure!(
cur==val,
UnexpectedSnafu {
err_msg: format!("current value '{cur:?}' already existed for key '{key}', {val:?} is not set"),
}
);
}
Ok(())
})
} }
/// Compares and puts value of key. `expect` is the expected value, if backend's current value associated pub async fn get_region_distribution(
/// with key is the same as `expect`, the value will be updated to `val`.
///
/// - If the compare-and-set operation successfully updated value, this method will return an `Ok(Ok())`
/// - If associated value is not the same as `expect`, no value will be updated and an
/// `Ok(Err(Option<TableRoute>))` will be returned. The `Option<TableRoute>` indicates
/// the current associated value of key.
/// - If any error happens during operation, an `Err(Error)` will be returned.
pub async fn compare_and_put(
&self, &self,
table_id: TableId, table_id: TableId,
expect: Option<TableRouteValue>, ) -> Result<Option<RegionDistribution>> {
region_routes: Vec<RegionRoute>, self.get(table_id)
) -> Result<std::result::Result<(), Option<TableRouteValue>>> { .await?
let key = NextTableRouteKey::new(table_id); .map(|table_route| region_distribution(&table_route.region_routes))
let raw_key = key.as_raw_key(); .transpose()
let (expect, version) = if let Some(x) = expect {
(x.try_as_raw_value()?, x.version + 1)
} else {
(vec![], 0)
};
let value = TableRouteValue {
region_routes,
version,
};
let raw_value = value.try_as_raw_value()?;
let req = CompareAndPutRequest::new()
.with_key(raw_key)
.with_expect(expect)
.with_value(raw_value);
self.kv_backend.compare_and_put(req).await?.handle(|resp| {
Ok(if resp.success {
Ok(())
} else {
Err(resp
.prev_kv
.map(|x| TableRouteValue::try_from_raw_value(x.value))
.transpose()?)
})
})
}
pub async fn remove(&self, table_id: TableId) -> Result<()> {
let key = NextTableRouteKey::new(table_id).as_raw_key();
let removed_key = to_removed_key(&String::from_utf8_lossy(&key));
let req = MoveValueRequest::new(key, removed_key.as_bytes());
self.kv_backend.move_value(req).await?;
Ok(())
} }
} }
#[deprecated(since = "0.4.0", note = "Please use the NextTableRouteKey instead")]
#[derive(Copy, Clone)] #[derive(Copy, Clone)]
pub struct TableRouteKey<'a> { pub struct TableRouteKey<'a> {
pub table_id: TableId, pub table_id: TableId,
@@ -309,47 +259,10 @@ impl<'a> Display for TableRouteKey<'a> {
#[cfg(test)] #[cfg(test)]
mod tests { mod tests {
use std::sync::Arc;
use api::v1::meta::TableName as PbTableName; use api::v1::meta::TableName as PbTableName;
use super::TableRouteKey; use super::TableRouteKey;
use crate::key::table_route::{TableRouteManager, TableRouteValue};
use crate::kv_backend::memory::MemoryKvBackend;
use crate::rpc::router::RegionRoute;
#[tokio::test]
async fn test_table_route_manager() {
let mgr = TableRouteManager::new(Arc::new(MemoryKvBackend::default()));
let table_id = 1024u32;
let region_route = RegionRoute::default();
let region_routes = vec![region_route];
mgr.create(table_id, region_routes.clone()).await.unwrap();
let got = mgr.get(1024).await.unwrap().unwrap();
assert_eq!(got.region_routes, region_routes);
let empty = mgr.get(1023).await.unwrap();
assert!(empty.is_none());
let expect = TableRouteValue::new(region_routes);
let mut updated = expect.clone();
updated.region_routes.push(RegionRoute::default());
let _ = mgr
.compare_and_put(1024, Some(expect.clone()), updated.region_routes.clone())
.await
.unwrap();
let _ = mgr
.compare_and_put(1024, Some(expect.clone()), updated.region_routes)
.await
.unwrap();
}
#[test] #[test]
fn test_table_route_key() { fn test_table_route_key() {

View File

@@ -26,7 +26,7 @@ use store_api::storage::{RegionId, RegionNumber};
use table::metadata::TableId; use table::metadata::TableId;
use crate::error::{self, Result}; use crate::error::{self, Result};
use crate::key::table_region::RegionDistribution; use crate::key::RegionDistribution;
use crate::peer::Peer; use crate::peer::Peer;
use crate::rpc::util; use crate::rpc::util;
use crate::table_name::TableName; use crate::table_name::TableName;
@@ -74,7 +74,7 @@ impl TryFrom<PbRouteResponse> for RouteResponse {
} }
} }
pub(crate) fn region_distribution(region_routes: &[RegionRoute]) -> Result<RegionDistribution> { pub fn region_distribution(region_routes: &[RegionRoute]) -> Result<RegionDistribution> {
let mut regions_id_map = RegionDistribution::new(); let mut regions_id_map = RegionDistribution::new();
for route in region_routes.iter() { for route in region_routes.iter() {
let node_id = route let node_id = route
@@ -88,6 +88,10 @@ pub(crate) fn region_distribution(region_routes: &[RegionRoute]) -> Result<Regio
let region_id = route.region.id.region_number(); let region_id = route.region.id.region_number();
regions_id_map.entry(node_id).or_default().push(region_id); regions_id_map.entry(node_id).or_default().push(region_id);
} }
for (_, regions) in regions_id_map.iter_mut() {
// id asc
regions.sort()
}
Ok(regions_id_map) Ok(regions_id_map)
} }
@@ -98,6 +102,41 @@ pub struct TableRoute {
region_leaders: HashMap<RegionNumber, Option<Peer>>, region_leaders: HashMap<RegionNumber, Option<Peer>>,
} }
pub fn find_leaders(region_routes: &[RegionRoute]) -> HashSet<Peer> {
region_routes
.iter()
.flat_map(|x| &x.leader_peer)
.cloned()
.collect()
}
pub fn find_leader_regions(region_routes: &[RegionRoute], datanode: &Peer) -> Vec<RegionNumber> {
region_routes
.iter()
.filter_map(|x| {
if let Some(peer) = &x.leader_peer {
if peer == datanode {
return Some(x.region.id.region_number());
}
}
None
})
.collect()
}
pub fn extract_all_peers(region_routes: &[RegionRoute]) -> Vec<Peer> {
let mut peers = region_routes
.iter()
.flat_map(|x| x.leader_peer.iter().chain(x.follower_peers.iter()))
.collect::<HashSet<_>>()
.into_iter()
.cloned()
.collect::<Vec<_>>();
peers.sort_by_key(|x| x.id);
peers
}
impl TableRoute { impl TableRoute {
pub fn new(table: Table, region_routes: Vec<RegionRoute>) -> Self { pub fn new(table: Table, region_routes: Vec<RegionRoute>) -> Self {
let region_leaders = region_routes let region_leaders = region_routes
@@ -211,25 +250,11 @@ impl TableRoute {
} }
pub fn find_leaders(&self) -> HashSet<Peer> { pub fn find_leaders(&self) -> HashSet<Peer> {
self.region_routes find_leaders(&self.region_routes)
.iter()
.flat_map(|x| &x.leader_peer)
.cloned()
.collect()
} }
pub fn find_leader_regions(&self, datanode: &Peer) -> Vec<RegionNumber> { pub fn find_leader_regions(&self, datanode: &Peer) -> Vec<RegionNumber> {
self.region_routes find_leader_regions(&self.region_routes, datanode)
.iter()
.filter_map(|x| {
if let Some(peer) = &x.leader_peer {
if peer == datanode {
return Some(x.region.id.region_number());
}
}
None
})
.collect()
} }
pub fn find_region_leader(&self, region_number: RegionNumber) -> Option<&Peer> { pub fn find_region_leader(&self, region_number: RegionNumber) -> Option<&Peer> {
@@ -241,6 +266,7 @@ impl TableRoute {
impl TryFrom<PbTableRouteValue> for TableRoute { impl TryFrom<PbTableRouteValue> for TableRoute {
type Error = error::Error; type Error = error::Error;
fn try_from(pb: PbTableRouteValue) -> Result<Self> { fn try_from(pb: PbTableRouteValue) -> Result<Self> {
TableRoute::try_from_raw( TableRoute::try_from_raw(
&pb.peers, &pb.peers,
@@ -251,6 +277,19 @@ impl TryFrom<PbTableRouteValue> for TableRoute {
} }
} }
impl TryFrom<TableRoute> for PbTableRouteValue {
type Error = error::Error;
fn try_from(table_route: TableRoute) -> Result<Self> {
let (peers, table_route) = table_route.try_into_raw()?;
Ok(PbTableRouteValue {
peers,
table_route: Some(table_route),
})
}
}
#[derive(Debug, Clone, Deserialize, Serialize, PartialEq)] #[derive(Debug, Clone, Deserialize, Serialize, PartialEq)]
pub struct Table { pub struct Table {
pub id: u64, pub id: u64,

View File

@@ -34,7 +34,6 @@ use common_meta::key::catalog_name::CatalogNameKey;
use common_meta::key::schema_name::SchemaNameKey; use common_meta::key::schema_name::SchemaNameKey;
use common_meta::key::table_info::TableInfoKey; use common_meta::key::table_info::TableInfoKey;
use common_meta::key::table_name::TableNameKey; use common_meta::key::table_name::TableNameKey;
use common_meta::key::table_region::TableRegionKey;
use common_meta::key::{TableMetaKey, TableMetadataManagerRef}; use common_meta::key::{TableMetaKey, TableMetadataManagerRef};
use common_meta::kv_backend::KvBackendRef; use common_meta::kv_backend::KvBackendRef;
use common_meta::table_name::TableName; use common_meta::table_name::TableName;
@@ -134,15 +133,6 @@ impl FrontendCatalogManager {
String::from_utf8_lossy(&key.as_raw_key()) String::from_utf8_lossy(&key.as_raw_key())
); );
let key = TableRegionKey::new(table_id);
self.backend_cache_invalidator
.invalidate_key(&key.as_raw_key())
.await;
debug!(
"invalidated cache key: {}",
String::from_utf8_lossy(&key.as_raw_key())
);
self.partition_manager self.partition_manager
.table_routes() .table_routes()
.invalidate_table_route(table_id) .invalidate_table_route(table_id)

View File

@@ -22,7 +22,6 @@ use common_meta::ident::TableIdent;
use common_meta::instruction::{Instruction, InstructionReply, SimpleReply}; use common_meta::instruction::{Instruction, InstructionReply, SimpleReply};
use common_meta::key::table_info::TableInfoKey; use common_meta::key::table_info::TableInfoKey;
use common_meta::key::table_name::TableNameKey; use common_meta::key::table_name::TableNameKey;
use common_meta::key::table_region::TableRegionKey;
use common_meta::key::TableMetaKey; use common_meta::key::TableMetaKey;
use common_telemetry::error; use common_telemetry::error;
use partition::manager::TableRouteCacheInvalidatorRef; use partition::manager::TableRouteCacheInvalidatorRef;
@@ -90,10 +89,6 @@ impl InvalidateTableCacheHandler {
.invalidate_key(&TableInfoKey::new(table_id).as_raw_key()) .invalidate_key(&TableInfoKey::new(table_id).as_raw_key())
.await; .await;
self.backend_cache_invalidator
.invalidate_key(&TableRegionKey::new(table_id).as_raw_key())
.await;
self.backend_cache_invalidator self.backend_cache_invalidator
.invalidate_key( .invalidate_key(
&TableNameKey::new( &TableNameKey::new(

View File

@@ -24,7 +24,7 @@ use common_meta::heartbeat::handler::{
use common_meta::heartbeat::mailbox::{HeartbeatMailbox, MessageMeta}; use common_meta::heartbeat::mailbox::{HeartbeatMailbox, MessageMeta};
use common_meta::ident::TableIdent; use common_meta::ident::TableIdent;
use common_meta::instruction::{Instruction, InstructionReply, SimpleReply}; use common_meta::instruction::{Instruction, InstructionReply, SimpleReply};
use common_meta::key::table_region::TableRegionKey; use common_meta::key::table_info::TableInfoKey;
use common_meta::key::TableMetaKey; use common_meta::key::TableMetaKey;
use partition::manager::TableRouteCacheInvalidator; use partition::manager::TableRouteCacheInvalidator;
use table::metadata::TableId; use table::metadata::TableId;
@@ -58,8 +58,8 @@ impl TableRouteCacheInvalidator for MockTableRouteCacheInvalidator {
#[tokio::test] #[tokio::test]
async fn test_invalidate_table_cache_handler() { async fn test_invalidate_table_cache_handler() {
let table_id = 1; let table_id = 1;
let table_region_key = TableRegionKey::new(table_id); let table_info_key = TableInfoKey::new(table_id);
let inner = HashMap::from([(table_region_key.as_raw_key(), 1)]); let inner = HashMap::from([(table_info_key.as_raw_key(), 1)]);
let backend = Arc::new(MockKvCacheInvalidator { let backend = Arc::new(MockKvCacheInvalidator {
inner: Mutex::new(inner), inner: Mutex::new(inner),
}); });
@@ -99,7 +99,7 @@ async fn test_invalidate_table_cache_handler() {
.inner .inner
.lock() .lock()
.unwrap() .unwrap()
.contains_key(&table_region_key.as_raw_key())); .contains_key(&table_info_key.as_raw_key()));
assert!(!table_route.inner.lock().unwrap().contains_key(&table_id)); assert!(!table_route.inner.lock().unwrap().contains_key(&table_id));

View File

@@ -178,11 +178,10 @@ mod tests {
use common_meta::helper::{CatalogValue, SchemaValue}; use common_meta::helper::{CatalogValue, SchemaValue};
use common_meta::key::catalog_name::CatalogNameKey; use common_meta::key::catalog_name::CatalogNameKey;
use common_meta::key::schema_name::SchemaNameKey; use common_meta::key::schema_name::SchemaNameKey;
use common_meta::key::table_name::TableNameKey;
use common_meta::key::table_region::RegionDistribution;
use common_meta::key::{TableMetadataManager, TableMetadataManagerRef}; use common_meta::key::{TableMetadataManager, TableMetadataManagerRef};
use common_meta::kv_backend::memory::MemoryKvBackend; use common_meta::kv_backend::memory::MemoryKvBackend;
use common_meta::kv_backend::{KvBackend, KvBackendRef}; use common_meta::kv_backend::{KvBackend, KvBackendRef};
use common_meta::rpc::router::{Region, RegionRoute};
use common_meta::rpc::store::PutRequest; use common_meta::rpc::store::PutRequest;
use datatypes::prelude::{ConcreteDataType, VectorRef}; use datatypes::prelude::{ConcreteDataType, VectorRef};
use datatypes::schema::{ColumnDefaultConstraint, ColumnSchema, Schema}; use datatypes::schema::{ColumnDefaultConstraint, ColumnSchema, Schema};
@@ -247,26 +246,25 @@ mod tests {
.unwrap() .unwrap()
.into(); .into();
let key = TableNameKey::new(DEFAULT_CATALOG_NAME, DEFAULT_SCHEMA_NAME, table_name); let region_route_factory = |region_id: u64, peer: u64| RegionRoute {
assert!(table_metadata_manager region: Region {
.table_name_manager() id: region_id.into(),
.create(&key, table_id) ..Default::default()
.await },
.is_ok()); leader_peer: Some(Peer {
id: peer,
addr: String::new(),
}),
follower_peers: vec![],
};
assert!(table_metadata_manager let region_routes = vec![
.table_info_manager() region_route_factory(1, 1),
.compare_and_put(table_id, None, table_info) region_route_factory(2, 2),
.await region_route_factory(3, 3),
.is_ok()); ];
table_metadata_manager
let _ = table_metadata_manager .create_table_metadata(table_info, region_routes)
.table_region_manager()
.compare_and_put(
1,
None,
RegionDistribution::from([(1, vec![1]), (2, vec![2]), (3, vec![3])]),
)
.await .await
.unwrap(); .unwrap();
} }

View File

@@ -181,11 +181,10 @@ mod tests {
use common_catalog::consts::{DEFAULT_CATALOG_NAME, DEFAULT_SCHEMA_NAME}; use common_catalog::consts::{DEFAULT_CATALOG_NAME, DEFAULT_SCHEMA_NAME};
use common_meta::key::catalog_name::{CatalogManager, CatalogNameKey}; use common_meta::key::catalog_name::{CatalogManager, CatalogNameKey};
use common_meta::key::schema_name::{SchemaManager, SchemaNameKey}; use common_meta::key::schema_name::{SchemaManager, SchemaNameKey};
use common_meta::key::table_name::TableNameKey;
use common_meta::key::table_region::RegionDistribution;
use common_meta::key::{TableMetadataManager, TableMetadataManagerRef}; use common_meta::key::{TableMetadataManager, TableMetadataManagerRef};
use common_meta::kv_backend::memory::MemoryKvBackend; use common_meta::kv_backend::memory::MemoryKvBackend;
use common_meta::kv_backend::KvBackendRef; use common_meta::kv_backend::KvBackendRef;
use common_meta::rpc::router::{Region, RegionRoute};
use datatypes::prelude::{ConcreteDataType, VectorRef}; use datatypes::prelude::{ConcreteDataType, VectorRef};
use datatypes::schema::{ColumnDefaultConstraint, ColumnSchema, Schema}; use datatypes::schema::{ColumnDefaultConstraint, ColumnSchema, Schema};
use datatypes::vectors::Int32Vector; use datatypes::vectors::Int32Vector;
@@ -241,26 +240,25 @@ mod tests {
.unwrap() .unwrap()
.into(); .into();
let key = TableNameKey::new(DEFAULT_CATALOG_NAME, DEFAULT_SCHEMA_NAME, table_name); let region_route_factory = |region_id: u64, peer: u64| RegionRoute {
assert!(table_metadata_manager region: Region {
.table_name_manager() id: region_id.into(),
.create(&key, table_id) ..Default::default()
.await },
.is_ok()); leader_peer: Some(Peer {
id: peer,
addr: String::new(),
}),
follower_peers: vec![],
};
assert!(table_metadata_manager let region_routes = vec![
.table_info_manager() region_route_factory(1, 1),
.compare_and_put(table_id, None, table_info) region_route_factory(2, 2),
.await region_route_factory(3, 3),
.is_ok()); ];
table_metadata_manager
let _ = table_metadata_manager .create_table_metadata(table_info, region_routes)
.table_region_manager()
.compare_and_put(
1,
None,
RegionDistribution::from([(1, vec![1]), (2, vec![2]), (3, vec![3])]),
)
.await .await
.unwrap(); .unwrap();
} }

View File

@@ -15,13 +15,14 @@
use std::sync::Arc; use std::sync::Arc;
use client::client_manager::DatanodeClients; use client::client_manager::DatanodeClients;
use common_meta::key::table_info::TableInfoValue;
use common_meta::key::table_route::TableRouteValue;
use common_meta::key::TableMetadataManagerRef; use common_meta::key::TableMetadataManagerRef;
use common_meta::rpc::ddl::{AlterTableTask, CreateTableTask, DropTableTask, TruncateTableTask}; use common_meta::rpc::ddl::{AlterTableTask, CreateTableTask, DropTableTask, TruncateTableTask};
use common_meta::rpc::router::TableRoute; use common_meta::rpc::router::RegionRoute;
use common_procedure::{watcher, ProcedureId, ProcedureManagerRef, ProcedureWithId}; use common_procedure::{watcher, ProcedureId, ProcedureManagerRef, ProcedureWithId};
use common_telemetry::error; use common_telemetry::error;
use snafu::ResultExt; use snafu::ResultExt;
use table::metadata::RawTableInfo;
use table::requests::AlterTableRequest; use table::requests::AlterTableRequest;
use crate::error::{ use crate::error::{
@@ -32,13 +33,11 @@ use crate::procedure::alter_table::AlterTableProcedure;
use crate::procedure::create_table::CreateTableProcedure; use crate::procedure::create_table::CreateTableProcedure;
use crate::procedure::drop_table::DropTableProcedure; use crate::procedure::drop_table::DropTableProcedure;
use crate::service::mailbox::MailboxRef; use crate::service::mailbox::MailboxRef;
use crate::service::store::kv::KvStoreRef;
pub type DdlManagerRef = Arc<DdlManager>; pub type DdlManagerRef = Arc<DdlManager>;
pub struct DdlManager { pub struct DdlManager {
procedure_manager: ProcedureManagerRef, procedure_manager: ProcedureManagerRef,
kv_store: KvStoreRef,
datanode_clients: Arc<DatanodeClients>, datanode_clients: Arc<DatanodeClients>,
pub(crate) mailbox: MailboxRef, pub(crate) mailbox: MailboxRef,
pub(crate) server_addr: String, pub(crate) server_addr: String,
@@ -47,7 +46,6 @@ pub struct DdlManager {
#[derive(Clone)] #[derive(Clone)]
pub(crate) struct DdlContext { pub(crate) struct DdlContext {
pub(crate) kv_store: KvStoreRef,
pub(crate) datanode_clients: Arc<DatanodeClients>, pub(crate) datanode_clients: Arc<DatanodeClients>,
pub(crate) mailbox: MailboxRef, pub(crate) mailbox: MailboxRef,
pub(crate) server_addr: String, pub(crate) server_addr: String,
@@ -57,7 +55,6 @@ pub(crate) struct DdlContext {
impl DdlManager { impl DdlManager {
pub(crate) fn new( pub(crate) fn new(
procedure_manager: ProcedureManagerRef, procedure_manager: ProcedureManagerRef,
kv_store: KvStoreRef,
datanode_clients: Arc<DatanodeClients>, datanode_clients: Arc<DatanodeClients>,
mailbox: MailboxRef, mailbox: MailboxRef,
server_addr: String, server_addr: String,
@@ -65,7 +62,6 @@ impl DdlManager {
) -> Self { ) -> Self {
Self { Self {
procedure_manager, procedure_manager,
kv_store,
datanode_clients, datanode_clients,
mailbox, mailbox,
server_addr, server_addr,
@@ -75,7 +71,6 @@ impl DdlManager {
pub(crate) fn create_context(&self) -> DdlContext { pub(crate) fn create_context(&self) -> DdlContext {
DdlContext { DdlContext {
kv_store: self.kv_store.clone(),
datanode_clients: self.datanode_clients.clone(), datanode_clients: self.datanode_clients.clone(),
mailbox: self.mailbox.clone(), mailbox: self.mailbox.clone(),
server_addr: self.server_addr.clone(), server_addr: self.server_addr.clone(),
@@ -132,7 +127,7 @@ impl DdlManager {
cluster_id: u64, cluster_id: u64,
alter_table_task: AlterTableTask, alter_table_task: AlterTableTask,
alter_table_request: AlterTableRequest, alter_table_request: AlterTableRequest,
table_info: RawTableInfo, table_info_value: TableInfoValue,
) -> Result<ProcedureId> { ) -> Result<ProcedureId> {
let context = self.create_context(); let context = self.create_context();
@@ -140,7 +135,7 @@ impl DdlManager {
cluster_id, cluster_id,
alter_table_task, alter_table_task,
alter_table_request, alter_table_request,
table_info, table_info_value,
context, context,
); );
@@ -153,12 +148,12 @@ impl DdlManager {
&self, &self,
cluster_id: u64, cluster_id: u64,
create_table_task: CreateTableTask, create_table_task: CreateTableTask,
table_route: TableRoute, region_routes: Vec<RegionRoute>,
) -> Result<ProcedureId> { ) -> Result<ProcedureId> {
let context = self.create_context(); let context = self.create_context();
let procedure = let procedure =
CreateTableProcedure::new(cluster_id, create_table_task, table_route, context); CreateTableProcedure::new(cluster_id, create_table_task, region_routes, context);
let procedure_with_id = ProcedureWithId::with_random_id(Box::new(procedure)); let procedure_with_id = ProcedureWithId::with_random_id(Box::new(procedure));
@@ -169,11 +164,18 @@ impl DdlManager {
&self, &self,
cluster_id: u64, cluster_id: u64,
drop_table_task: DropTableTask, drop_table_task: DropTableTask,
table_route: TableRoute, table_info_value: TableInfoValue,
table_route_value: TableRouteValue,
) -> Result<ProcedureId> { ) -> Result<ProcedureId> {
let context = self.create_context(); let context = self.create_context();
let procedure = DropTableProcedure::new(cluster_id, drop_table_task, table_route, context); let procedure = DropTableProcedure::new(
cluster_id,
drop_table_task,
table_route_value,
table_info_value,
context,
);
let procedure_with_id = ProcedureWithId::with_random_id(Box::new(procedure)); let procedure_with_id = ProcedureWithId::with_random_id(Box::new(procedure));
@@ -184,10 +186,10 @@ impl DdlManager {
&self, &self,
cluster_id: u64, cluster_id: u64,
truncate_table_task: TruncateTableTask, truncate_table_task: TruncateTableTask,
table_route: TableRoute, region_routes: Vec<RegionRoute>,
) -> Result<ProcedureId> { ) -> Result<ProcedureId> {
error!("truncate table procedure is not supported, cluster_id = {}, truncate_table_task = {:?}, table_route = {:?}", error!("truncate table procedure is not supported, cluster_id = {}, truncate_table_task = {:?}, region_routes = {:?}",
cluster_id, truncate_table_task, table_route); cluster_id, truncate_table_task, region_routes);
UnsupportedSnafu { UnsupportedSnafu {
operation: "TRUNCATE TABLE", operation: "TRUNCATE TABLE",

View File

@@ -232,8 +232,17 @@ pub enum Error {
location: Location, location: Location,
}, },
#[snafu(display("Table route not found: {}", key))] #[snafu(display("Table route not found: {}", table_name))]
TableRouteNotFound { key: String, location: Location }, TableRouteNotFound {
table_name: String,
location: Location,
},
#[snafu(display("Table info not found: {}", table_name))]
TableInfoNotFound {
table_name: String,
location: Location,
},
#[snafu(display("Table route corrupted, key: {}, reason: {}", key, reason))] #[snafu(display("Table route corrupted, key: {}, reason: {}", key, reason))]
CorruptedTableRoute { CorruptedTableRoute {
@@ -478,6 +487,18 @@ pub enum Error {
location: Location, location: Location,
}, },
#[snafu(display("Failed to update table route: {}", source))]
UpdateTableRoute {
source: common_meta::error::Error,
location: Location,
},
#[snafu(display("Failed to get table info error: {}", source))]
GetFullTableInfo {
source: common_meta::error::Error,
location: Location,
},
#[snafu(display("Invalid heartbeat request: {}", err_msg))] #[snafu(display("Invalid heartbeat request: {}", err_msg))]
InvalidHeartbeatRequest { err_msg: String, location: Location }, InvalidHeartbeatRequest { err_msg: String, location: Location },
@@ -560,6 +581,7 @@ impl ErrorExt for Error {
| Error::StatValueFromUtf8 { .. } | Error::StatValueFromUtf8 { .. }
| Error::UnexpectedSequenceValue { .. } | Error::UnexpectedSequenceValue { .. }
| Error::TableRouteNotFound { .. } | Error::TableRouteNotFound { .. }
| Error::TableInfoNotFound { .. }
| Error::CorruptedTableRoute { .. } | Error::CorruptedTableRoute { .. }
| Error::NextSequence { .. } | Error::NextSequence { .. }
| Error::SequenceOutOfRange { .. } | Error::SequenceOutOfRange { .. }
@@ -594,7 +616,9 @@ impl ErrorExt for Error {
Error::TableRouteConversion { source, .. } Error::TableRouteConversion { source, .. }
| Error::ConvertProtoData { source, .. } | Error::ConvertProtoData { source, .. }
| Error::TableMetadataManager { source, .. } | Error::TableMetadataManager { source, .. }
| Error::ConvertEtcdTxnObject { source, .. } => source.status_code(), | Error::UpdateTableRoute { source, .. }
| Error::ConvertEtcdTxnObject { source, .. }
| Error::GetFullTableInfo { source, .. } => source.status_code(),
Error::Other { source, .. } => source.status_code(), Error::Other { source, .. } => source.status_code(),
} }

View File

@@ -177,7 +177,6 @@ impl MetaSrvBuilder {
&options, &options,
datanode_clients, datanode_clients,
&procedure_manager, &procedure_manager,
&kv_store,
&mailbox, &mailbox,
&table_metadata_manager, &table_metadata_manager,
); );
@@ -316,7 +315,6 @@ fn build_ddl_manager(
options: &MetaSrvOptions, options: &MetaSrvOptions,
datanode_clients: Option<Arc<DatanodeClients>>, datanode_clients: Option<Arc<DatanodeClients>>,
procedure_manager: &ProcedureManagerRef, procedure_manager: &ProcedureManagerRef,
kv_store: &KvStoreRef,
mailbox: &MailboxRef, mailbox: &MailboxRef,
table_metadata_manager: &TableMetadataManagerRef, table_metadata_manager: &TableMetadataManagerRef,
) -> DdlManagerRef { ) -> DdlManagerRef {
@@ -334,7 +332,6 @@ fn build_ddl_manager(
// TODO(weny): considers to modify the default config of procedure manager // TODO(weny): considers to modify the default config of procedure manager
Arc::new(DdlManager::new( Arc::new(DdlManager::new(
procedure_manager.clone(), procedure_manager.clone(),
kv_store.clone(),
datanode_clients, datanode_clients,
mailbox.clone(), mailbox.clone(),
options.server_addr.clone(), options.server_addr.clone(),

View File

@@ -21,10 +21,9 @@ use common_meta::ident::TableIdent;
use common_meta::instruction::Instruction; use common_meta::instruction::Instruction;
use common_meta::key::table_info::TableInfoValue; use common_meta::key::table_info::TableInfoValue;
use common_meta::key::table_name::TableNameKey; use common_meta::key::table_name::TableNameKey;
use common_meta::key::TableRouteKey; use common_meta::key::table_route::TableRouteValue;
use common_meta::kv_backend::txn::{Compare, CompareOp, Txn, TxnOp};
use common_meta::rpc::ddl::AlterTableTask; use common_meta::rpc::ddl::AlterTableTask;
use common_meta::rpc::router::TableRoute; use common_meta::rpc::router::{find_leaders, RegionRoute};
use common_meta::table_name::TableName; use common_meta::table_name::TableName;
use common_procedure::error::{FromJsonSnafu, Result as ProcedureResult, ToJsonSnafu}; use common_procedure::error::{FromJsonSnafu, Result as ProcedureResult, ToJsonSnafu};
use common_procedure::{ use common_procedure::{
@@ -39,10 +38,9 @@ use table::metadata::{RawTableInfo, TableId, TableInfo};
use table::requests::{AlterKind, AlterTableRequest}; use table::requests::{AlterKind, AlterTableRequest};
use crate::ddl::DdlContext; use crate::ddl::DdlContext;
use crate::error::{self, Result, TableMetadataManagerSnafu, UnexpectedSnafu}; use crate::error::{self, Result, TableMetadataManagerSnafu};
use crate::procedure::utils::handle_request_datanode_error; use crate::procedure::utils::handle_request_datanode_error;
use crate::service::mailbox::BroadcastChannel; use crate::service::mailbox::BroadcastChannel;
use crate::table_routes::fetch_table;
// TODO(weny): removes in following PRs. // TODO(weny): removes in following PRs.
#[allow(dead_code)] #[allow(dead_code)]
@@ -60,12 +58,12 @@ impl AlterTableProcedure {
cluster_id: u64, cluster_id: u64,
task: AlterTableTask, task: AlterTableTask,
alter_table_request: AlterTableRequest, alter_table_request: AlterTableRequest,
table_info: RawTableInfo, table_info_value: TableInfoValue,
context: DdlContext, context: DdlContext,
) -> Self { ) -> Self {
Self { Self {
context, context,
data: AlterTableData::new(task, alter_table_request, table_info, cluster_id), data: AlterTableData::new(task, alter_table_request, table_info_value, cluster_id),
} }
} }
@@ -75,11 +73,58 @@ impl AlterTableProcedure {
Ok(AlterTableProcedure { context, data }) Ok(AlterTableProcedure { context, data })
} }
// Checks whether the table exists.
async fn on_prepare(&mut self) -> Result<Status> {
let request = &self.data.alter_table_request;
let manager = &self.context.table_metadata_manager;
if let AlterKind::RenameTable { new_table_name } = &request.alter_kind {
let exist = manager
.table_name_manager()
.exists(TableNameKey::new(
&request.catalog_name,
&request.schema_name,
new_table_name,
))
.await
.context(TableMetadataManagerSnafu)?;
ensure!(
!exist,
error::TableAlreadyExistsSnafu {
table_name: request.table_ref().to_string()
}
)
}
let exist = manager
.table_name_manager()
.exists(TableNameKey::new(
&request.catalog_name,
&request.schema_name,
&request.table_name,
))
.await
.context(TableMetadataManagerSnafu)?;
ensure!(
exist,
error::TableNotFoundSnafu {
name: request.table_ref().to_string()
}
);
self.data.state = AlterTableState::UpdateMetadata;
Ok(Status::executing(true))
}
/// Alters table on datanode. /// Alters table on datanode.
async fn on_datanode_alter_table(&mut self) -> Result<Status> { async fn on_datanode_alter_table(&mut self) -> Result<Status> {
let table_route = self let region_routes = self
.data .data
.table_route .region_routes
.as_ref() .as_ref()
.context(error::UnexpectedSnafu { .context(error::UnexpectedSnafu {
violated: "expected table_route", violated: "expected table_route",
@@ -88,7 +133,7 @@ impl AlterTableProcedure {
let table_ref = self.data.table_ref(); let table_ref = self.data.table_ref();
let clients = self.context.datanode_clients.clone(); let clients = self.context.datanode_clients.clone();
let leaders = table_route.find_leaders(); let leaders = find_leaders(region_routes);
let mut joins = Vec::with_capacity(leaders.len()); let mut joins = Vec::with_capacity(leaders.len());
for datanode in leaders { for datanode in leaders {
@@ -113,104 +158,35 @@ impl AlterTableProcedure {
Ok(Status::Done) Ok(Status::Done)
} }
async fn update_table_info_value(
&self,
table_id: TableId,
table_info_value: &TableInfoValue,
new_table_info: RawTableInfo,
) -> Result<()> {
self.context.table_metadata_manager
.table_info_manager()
.compare_and_put(table_id, Some(table_info_value.clone()), new_table_info)
.await
.context(TableMetadataManagerSnafu)?
.map_err(|curr| {
// The table info metadata should be guarded by procedure locks.
UnexpectedSnafu {
violated: format!(
"TableInfoValue for table {table_id} is changed during table alternation, expected: '{table_info_value:?}', actual: '{curr:?}'",
)
}.build()
})
}
/// Update table metadata for rename table operation. /// Update table metadata for rename table operation.
async fn on_update_metadata_for_rename(&self, new_table_info: TableInfo) -> Result<TableRoute> { async fn on_update_metadata_for_rename(&self, new_table_name: String) -> Result<()> {
let table_metadata_manager = &self.context.table_metadata_manager; let table_metadata_manager = &self.context.table_metadata_manager;
let table_ref = self.data.table_ref(); let current_table_info_value = self.data.table_info_value.clone();
let new_table_name = new_table_info.name.clone();
let table_id = self.data.table_info.ident.table_id;
if let Some((table_info_value, table_route_value)) = table_metadata_manager
fetch_table(&self.context.kv_store, table_metadata_manager, table_id).await? .rename_table(current_table_info_value, new_table_name)
{ .await
self.update_table_info_value(table_id, &table_info_value, new_table_info.into()) .context(error::TableMetadataManagerSnafu)?;
.await?;
info!("Updated TableInfoValue for table {table_id} with new table name '{new_table_name}'");
table_metadata_manager Ok(())
.table_name_manager() }
.rename(
TableNameKey::new(table_ref.catalog, table_ref.schema, table_ref.table),
table_id,
&new_table_name,
)
.await
.context(TableMetadataManagerSnafu)?;
info!("Renamed TableNameKey to new table name '{new_table_name}' for table {table_id}");
let table_route = table_route_value async fn on_update_metadata_for_alter(&self, new_table_info: RawTableInfo) -> Result<()> {
.clone() let table_metadata_manager = &self.context.table_metadata_manager;
.try_into() let current_table_info_value = self.data.table_info_value.clone();
.context(error::TableRouteConversionSnafu)?;
let table_route_key = TableRouteKey { table_metadata_manager
table_id, .update_table_info(current_table_info_value, new_table_info)
catalog_name: table_ref.catalog, .await
schema_name: table_ref.schema, .context(error::TableMetadataManagerSnafu)?;
table_name: table_ref.table,
};
let new_table_route_key = TableRouteKey {
table_name: &new_table_name,
..table_route_key
};
let txn = Txn::new() Ok(())
.when(vec![Compare::with_value(
table_route_key.to_string().into_bytes(),
CompareOp::Equal,
table_route_value.clone().into(),
)])
.and_then(vec![
TxnOp::Delete(table_route_key.to_string().into_bytes()),
TxnOp::Put(
new_table_route_key.to_string().into_bytes(),
table_route_value.into(),
),
]);
let resp = self.context.kv_store.txn(txn).await?;
ensure!(
resp.succeeded,
error::TxnSnafu {
msg: "table metadata changed"
}
);
info!("Updated TableRouteValue for table {table_id} with new table name '{new_table_name}'");
return Ok(table_route);
}
error::TableNotFoundSnafu {
name: table_ref.to_string(),
}
.fail()
} }
fn build_new_table_info(&self) -> Result<TableInfo> { fn build_new_table_info(&self) -> Result<TableInfo> {
// Builds new_meta // Builds new_meta
let table_info = TableInfo::try_from(self.data.table_info.clone()) let table_info = TableInfo::try_from(self.data.table_info().clone())
.context(error::ConvertRawTableInfoSnafu)?; .context(error::ConvertRawTableInfoSnafu)?;
let table_ref = self.data.table_ref(); let table_ref = self.data.table_ref();
@@ -240,50 +216,38 @@ impl AlterTableProcedure {
/// Update table metadata. /// Update table metadata.
async fn on_update_metadata(&mut self) -> Result<Status> { async fn on_update_metadata(&mut self) -> Result<Status> {
let request = &self.data.alter_table_request; let request = &self.data.alter_table_request;
let table_id = self.data.table_info.ident.table_id; let table_id = self.data.table_id();
let table_ref = self.data.table_ref(); let table_ref = self.data.table_ref();
let new_info = self.build_new_table_info()?; let new_info = self.build_new_table_info()?;
let table_metadata_manager = &self.context.table_metadata_manager;
debug!( debug!(
"starting update table: {} metadata, new table info {:?}", "starting update table: {} metadata, new table info {:?}",
table_ref.to_string(), table_ref.to_string(),
new_info new_info
); );
if matches!(request.alter_kind, AlterKind::RenameTable { .. }) { if let AlterKind::RenameTable { new_table_name } = &request.alter_kind {
let table_route = self.on_update_metadata_for_rename(new_info).await?; self.on_update_metadata_for_rename(new_table_name.to_string())
self.data.state = AlterTableState::InvalidateTableCache;
self.data.table_route = Some(table_route);
return Ok(Status::executing(true));
}
if let Some((table_info_value, table_route_value)) = fetch_table(
&self.context.kv_store,
&self.context.table_metadata_manager,
table_id,
)
.await?
{
let table_route = table_route_value
.clone()
.try_into()
.context(error::TableRouteConversionSnafu)?;
let new_raw_info: RawTableInfo = new_info.into();
self.update_table_info_value(table_id, &table_info_value, new_raw_info)
.await?; .await?;
info!("Updated TableInfoValue for table {table_id} when altering table");
self.data.state = AlterTableState::InvalidateTableCache;
self.data.table_route = Some(table_route);
Ok(Status::executing(true))
} else { } else {
error::TableNotFoundSnafu { self.on_update_metadata_for_alter(new_info.into()).await?;
name: table_ref.to_string(),
}
.fail()
} }
info!("Updated table metadata for table {table_id}");
let TableRouteValue { region_routes, .. } = table_metadata_manager
.table_route_manager()
.get(table_id)
.await
.context(error::TableMetadataManagerSnafu)?
.with_context(|| error::TableRouteNotFoundSnafu {
table_name: table_ref.to_string(),
})?;
self.data.region_routes = Some(region_routes);
self.data.state = AlterTableState::InvalidateTableCache;
Ok(Status::executing(true))
} }
/// Broadcasts the invalidating table cache instructions. /// Broadcasts the invalidating table cache instructions.
@@ -294,8 +258,8 @@ impl AlterTableProcedure {
catalog: table_name.catalog_name, catalog: table_name.catalog_name,
schema: table_name.schema_name, schema: table_name.schema_name,
table: table_name.table_name, table: table_name.table_name,
table_id: self.data.table_info.ident.table_id, table_id: self.data.table_id(),
engine: self.data.table_info.meta.engine.to_string(), engine: self.data.table_info().meta.engine.to_string(),
}; };
let instruction = Instruction::InvalidateTableCache(table_ident); let instruction = Instruction::InvalidateTableCache(table_ident);
@@ -356,6 +320,7 @@ impl Procedure for AlterTableProcedure {
}; };
match self.data.state { match self.data.state {
AlterTableState::Prepare => self.on_prepare().await,
AlterTableState::UpdateMetadata => self.on_update_metadata().await, AlterTableState::UpdateMetadata => self.on_update_metadata().await,
AlterTableState::InvalidateTableCache => self.on_broadcast().await, AlterTableState::InvalidateTableCache => self.on_broadcast().await,
AlterTableState::DatanodeAlterTable => self.on_datanode_alter_table().await, AlterTableState::DatanodeAlterTable => self.on_datanode_alter_table().await,
@@ -376,6 +341,8 @@ impl Procedure for AlterTableProcedure {
#[derive(Debug, Serialize, Deserialize)] #[derive(Debug, Serialize, Deserialize)]
enum AlterTableState { enum AlterTableState {
/// Prepares to alter the table
Prepare,
/// Updates table metadata. /// Updates table metadata.
UpdateMetadata, UpdateMetadata,
/// Broadcasts the invalidating table cache instruction. /// Broadcasts the invalidating table cache instruction.
@@ -389,8 +356,8 @@ pub struct AlterTableData {
state: AlterTableState, state: AlterTableState,
task: AlterTableTask, task: AlterTableTask,
alter_table_request: AlterTableRequest, alter_table_request: AlterTableRequest,
table_route: Option<TableRoute>, region_routes: Option<Vec<RegionRoute>>,
table_info: RawTableInfo, table_info_value: TableInfoValue,
cluster_id: u64, cluster_id: u64,
} }
@@ -398,15 +365,15 @@ impl AlterTableData {
pub fn new( pub fn new(
task: AlterTableTask, task: AlterTableTask,
alter_table_request: AlterTableRequest, alter_table_request: AlterTableRequest,
table_info: RawTableInfo, table_info_value: TableInfoValue,
cluster_id: u64, cluster_id: u64,
) -> Self { ) -> Self {
Self { Self {
state: AlterTableState::UpdateMetadata, state: AlterTableState::Prepare,
task, task,
alter_table_request, alter_table_request,
table_route: None, region_routes: None,
table_info, table_info_value,
cluster_id, cluster_id,
} }
} }
@@ -418,4 +385,12 @@ impl AlterTableData {
fn table_name(&self) -> TableName { fn table_name(&self) -> TableName {
self.task.table_name() self.task.table_name()
} }
fn table_id(&self) -> TableId {
self.table_info().ident.table_id
}
fn table_info(&self) -> &RawTableInfo {
&self.table_info_value.table_info
}
} }

View File

@@ -12,16 +12,13 @@
// See the License for the specific language governing permissions and // See the License for the specific language governing permissions and
// limitations under the License. // limitations under the License.
use api::v1::meta::TableRouteValue;
use async_trait::async_trait; use async_trait::async_trait;
use client::Database; use client::Database;
use common_error::ext::ErrorExt; use common_error::ext::ErrorExt;
use common_error::status_code::StatusCode; use common_error::status_code::StatusCode;
use common_meta::key::table_name::TableNameKey; use common_meta::key::table_name::TableNameKey;
use common_meta::key::TableRouteKey;
use common_meta::kv_backend::txn::{Compare, CompareOp, Txn, TxnOp};
use common_meta::rpc::ddl::CreateTableTask; use common_meta::rpc::ddl::CreateTableTask;
use common_meta::rpc::router::TableRoute; use common_meta::rpc::router::{find_leader_regions, find_leaders, RegionRoute};
use common_meta::table_name::TableName; use common_meta::table_name::TableName;
use common_procedure::error::{FromJsonSnafu, Result as ProcedureResult, ToJsonSnafu}; use common_procedure::error::{FromJsonSnafu, Result as ProcedureResult, ToJsonSnafu};
use common_procedure::{Context as ProcedureContext, LockKey, Procedure, Status}; use common_procedure::{Context as ProcedureContext, LockKey, Procedure, Status};
@@ -30,12 +27,11 @@ use futures::future::join_all;
use serde::{Deserialize, Serialize}; use serde::{Deserialize, Serialize};
use snafu::{ensure, ResultExt}; use snafu::{ensure, ResultExt};
use table::engine::TableReference; use table::engine::TableReference;
use table::metadata::TableId; use table::metadata::{RawTableInfo, TableId};
use super::utils::{handle_request_datanode_error, handle_retry_error}; use super::utils::{handle_request_datanode_error, handle_retry_error};
use crate::ddl::DdlContext; use crate::ddl::DdlContext;
use crate::error::{self, Result, TableMetadataManagerSnafu}; use crate::error::{self, Result, TableMetadataManagerSnafu};
use crate::service::router::create_region_distribution;
pub struct CreateTableProcedure { pub struct CreateTableProcedure {
context: DdlContext, context: DdlContext,
@@ -48,12 +44,12 @@ impl CreateTableProcedure {
pub(crate) fn new( pub(crate) fn new(
cluster_id: u64, cluster_id: u64,
task: CreateTableTask, task: CreateTableTask,
table_route: TableRoute, region_routes: Vec<RegionRoute>,
context: DdlContext, context: DdlContext,
) -> Self { ) -> Self {
Self { Self {
context, context,
creator: TableCreator::new(cluster_id, task, table_route), creator: TableCreator::new(cluster_id, task, region_routes),
} }
} }
@@ -69,21 +65,30 @@ impl CreateTableProcedure {
self.creator.data.task.table_name() self.creator.data.task.table_name()
} }
pub fn table_info(&self) -> &RawTableInfo {
&self.creator.data.task.table_info
}
pub fn region_routes(&self) -> &Vec<RegionRoute> {
&self.creator.data.region_routes
}
/// Checks whether the table exists. /// Checks whether the table exists.
async fn on_prepare(&mut self) -> Result<Status> { async fn on_prepare(&mut self) -> Result<Status> {
let expr = &self.creator.data.task.create_table; let expr = &self.creator.data.task.create_table;
let value = self let exist = self
.context .context
.table_metadata_manager .table_metadata_manager
.table_name_manager() .table_name_manager()
.get(TableNameKey::new( .exists(TableNameKey::new(
&expr.catalog_name, &expr.catalog_name,
&expr.schema_name, &expr.schema_name,
&expr.table_name, &expr.table_name,
)) ))
.await .await
.context(TableMetadataManagerSnafu)?; .context(TableMetadataManagerSnafu)?;
if value.is_some() {
if exist {
ensure!( ensure!(
self.creator.data.task.create_table.create_if_not_exists, self.creator.data.task.create_table.create_if_not_exists,
error::TableAlreadyExistsSnafu { error::TableAlreadyExistsSnafu {
@@ -103,85 +108,20 @@ impl CreateTableProcedure {
let _timer = common_telemetry::timer!( let _timer = common_telemetry::timer!(
crate::metrics::METRIC_META_CREATE_TABLE_PROCEDURE_CREATE_META crate::metrics::METRIC_META_CREATE_TABLE_PROCEDURE_CREATE_META
); );
let table_name = self.table_name();
let table_id = self.creator.data.table_route.table.id as TableId; let table_id = self.table_info().ident.table_id as TableId;
let table_route_key = TableRouteKey::with_table_name(table_id, &table_name.clone().into())
.to_string()
.into_bytes();
let (peers, table_route) = self
.creator
.data
.table_route
.clone()
.try_into_raw()
.context(error::ConvertProtoDataSnafu)?;
let table_route_value = TableRouteValue {
peers,
table_route: Some(table_route),
};
let manager = &self.context.table_metadata_manager; let manager = &self.context.table_metadata_manager;
let region_distribution = create_region_distribution(&table_route_value)?; let raw_table_info = self.table_info().clone();
manager let region_routes = self.region_routes().clone();
.table_region_manager()
.create(table_id, &region_distribution)
.await
.context(TableMetadataManagerSnafu)?;
info!("Created TableRegionValue for table {table_id}");
manager manager
.table_info_manager() .create_table_metadata(raw_table_info, region_routes)
.create(table_id, &self.creator.data.task.table_info)
.await .await
.context(TableMetadataManagerSnafu)?; .context(TableMetadataManagerSnafu)?;
info!("Created TableInfoValue for table {table_id}"); info!("Created table metadata for table {table_id}");
for (datanode_id, regions) in region_distribution {
manager
.datanode_table_manager()
.create(datanode_id, table_id, regions)
.await
.context(TableMetadataManagerSnafu)?;
info!("Create DatanodeTableValue for table {table_id}");
}
let txn = Txn::new()
.when(vec![Compare::with_not_exist_value(
table_route_key.clone(),
CompareOp::Equal,
)])
.and_then(vec![TxnOp::Put(table_route_key, table_route_value.into())]);
let resp = self.context.kv_store.txn(txn).await?;
ensure!(
resp.succeeded,
error::TxnSnafu {
msg: "table_route_key or table_global_key exists"
}
);
info!("Created TableRouteValue for table {table_id}");
// Create TableNameValue at last, because we use it to check whether the table exists at
// the beginning of the procedure.
manager
.table_name_manager()
.create(
&TableNameKey::new(
&table_name.catalog_name,
&table_name.schema_name,
&table_name.table_name,
),
table_id,
)
.await
.context(TableMetadataManagerSnafu)?;
info!("Created TableNameValue for table {table_id}");
Ok(Status::Done) Ok(Status::Done)
} }
@@ -189,22 +129,21 @@ impl CreateTableProcedure {
let _timer = common_telemetry::timer!( let _timer = common_telemetry::timer!(
crate::metrics::METRIC_META_CREATE_TABLE_PROCEDURE_CREATE_TABLE crate::metrics::METRIC_META_CREATE_TABLE_PROCEDURE_CREATE_TABLE
); );
let table_route = &self.creator.data.table_route; let region_routes = &self.creator.data.region_routes;
let table_name = self.table_name(); let table_name = self.table_name();
let clients = self.context.datanode_clients.clone(); let clients = self.context.datanode_clients.clone();
let leaders = table_route.find_leaders(); let leaders = find_leaders(region_routes);
let mut joins = Vec::with_capacity(leaders.len()); let mut joins = Vec::with_capacity(leaders.len());
let table_id = self.table_info().ident.table_id;
for datanode in leaders { for datanode in leaders {
let client = clients.get_client(&datanode).await; let client = clients.get_client(&datanode).await;
let client = Database::new(&table_name.catalog_name, &table_name.schema_name, client); let client = Database::new(&table_name.catalog_name, &table_name.schema_name, client);
let regions = table_route.find_leader_regions(&datanode); let regions = find_leader_regions(region_routes, &datanode);
let mut create_expr_for_region = self.creator.data.task.create_table.clone(); let mut create_expr_for_region = self.creator.data.task.create_table.clone();
create_expr_for_region.region_numbers = regions; create_expr_for_region.region_numbers = regions;
create_expr_for_region.table_id = Some(api::v1::TableId { create_expr_for_region.table_id = Some(api::v1::TableId { id: table_id });
id: table_route.table.id as u32,
});
joins.push(common_runtime::spawn_bg(async move { joins.push(common_runtime::spawn_bg(async move {
if let Err(err) = client.create(create_expr_for_region).await { if let Err(err) = client.create(create_expr_for_region).await {
@@ -264,13 +203,13 @@ pub struct TableCreator {
} }
impl TableCreator { impl TableCreator {
pub fn new(cluster_id: u64, task: CreateTableTask, table_route: TableRoute) -> Self { pub fn new(cluster_id: u64, task: CreateTableTask, region_routes: Vec<RegionRoute>) -> Self {
Self { Self {
data: CreateTableData { data: CreateTableData {
state: CreateTableState::Prepare, state: CreateTableState::Prepare,
cluster_id, cluster_id,
task, task,
table_route, region_routes,
}, },
} }
} }
@@ -290,7 +229,7 @@ enum CreateTableState {
pub struct CreateTableData { pub struct CreateTableData {
state: CreateTableState, state: CreateTableState,
task: CreateTableTask, task: CreateTableTask,
table_route: TableRoute, region_routes: Vec<RegionRoute>,
cluster_id: u64, cluster_id: u64,
} }

View File

@@ -13,7 +13,7 @@
// limitations under the License. // limitations under the License.
use api::v1::meta::MailboxMessage; use api::v1::meta::MailboxMessage;
use api::v1::{DropTableExpr, TableId}; use api::v1::DropTableExpr;
use async_trait::async_trait; use async_trait::async_trait;
use client::Database; use client::Database;
use common_catalog::consts::MITO_ENGINE; use common_catalog::consts::MITO_ENGINE;
@@ -21,11 +21,11 @@ use common_error::ext::ErrorExt;
use common_error::status_code::StatusCode; use common_error::status_code::StatusCode;
use common_meta::ident::TableIdent; use common_meta::ident::TableIdent;
use common_meta::instruction::Instruction; use common_meta::instruction::Instruction;
use common_meta::key::table_info::TableInfoValue;
use common_meta::key::table_name::TableNameKey; use common_meta::key::table_name::TableNameKey;
use common_meta::key::TableRouteKey; use common_meta::key::table_route::TableRouteValue;
use common_meta::kv_backend::txn::{Compare, CompareOp, Txn, TxnOp};
use common_meta::rpc::ddl::DropTableTask; use common_meta::rpc::ddl::DropTableTask;
use common_meta::rpc::router::TableRoute; use common_meta::rpc::router::{find_leaders, RegionRoute};
use common_meta::table_name::TableName; use common_meta::table_name::TableName;
use common_procedure::error::{FromJsonSnafu, ToJsonSnafu}; use common_procedure::error::{FromJsonSnafu, ToJsonSnafu};
use common_procedure::{ use common_procedure::{
@@ -36,12 +36,12 @@ use futures::future::join_all;
use serde::{Deserialize, Serialize}; use serde::{Deserialize, Serialize};
use snafu::{ensure, ResultExt}; use snafu::{ensure, ResultExt};
use table::engine::TableReference; use table::engine::TableReference;
use table::metadata::{RawTableInfo, TableId};
use super::utils::handle_retry_error; use super::utils::handle_retry_error;
use crate::ddl::DdlContext; use crate::ddl::DdlContext;
use crate::error; use crate::error::{self, Result, TableMetadataManagerSnafu};
use crate::error::{Result, TableMetadataManagerSnafu}; use crate::procedure::utils::handle_request_datanode_error;
use crate::procedure::utils::{build_table_route_value, handle_request_datanode_error};
use crate::service::mailbox::BroadcastChannel; use crate::service::mailbox::BroadcastChannel;
pub struct DropTableProcedure { pub struct DropTableProcedure {
context: DdlContext, context: DdlContext,
@@ -54,12 +54,13 @@ impl DropTableProcedure {
pub(crate) fn new( pub(crate) fn new(
cluster_id: u64, cluster_id: u64,
task: DropTableTask, task: DropTableTask,
table_route: TableRoute, table_route_value: TableRouteValue,
table_info_value: TableInfoValue,
context: DdlContext, context: DdlContext,
) -> Self { ) -> Self {
Self { Self {
context, context,
data: DropTableData::new(cluster_id, task, table_route), data: DropTableData::new(cluster_id, task, table_route_value, table_info_value),
} }
} }
@@ -68,76 +69,46 @@ impl DropTableProcedure {
Ok(Self { context, data }) Ok(Self { context, data })
} }
/// Removes the table metadata. async fn on_prepare(&mut self) -> Result<Status> {
async fn on_remove_metadata(&mut self) -> Result<Status> { let table_ref = &self.data.table_ref();
let table_ref = self.data.table_ref();
let table_id = self.data.task.table_id;
let manager = &self.context.table_metadata_manager; let exist = self
manager .context
.table_info_manager() .table_metadata_manager
.remove(table_id)
.await
.context(TableMetadataManagerSnafu)?;
info!("Removed TableInfoValue for table: {table_id}");
let table_region_value = manager
.table_region_manager()
.remove(table_id)
.await
.context(TableMetadataManagerSnafu)?;
info!("Removed TableRegionValue for table: {table_id}");
if let Some(table_region_value) = table_region_value {
for datanode_id in table_region_value.region_distribution.keys() {
manager
.datanode_table_manager()
.remove(*datanode_id, table_id)
.await
.context(TableMetadataManagerSnafu)?;
info!("Removed DatanodeTableValue for table: {table_id} on Datanode {datanode_id}");
}
}
manager
.table_name_manager() .table_name_manager()
.remove(TableNameKey::new( .exists(TableNameKey::new(
table_ref.catalog, table_ref.catalog,
table_ref.schema, table_ref.schema,
table_ref.table, table_ref.table,
)) ))
.await .await
.context(TableMetadataManagerSnafu)?; .context(TableMetadataManagerSnafu)?;
info!("Removed TableNameValue for table: {table_id}");
let table_route_key = TableRouteKey {
table_id,
catalog_name: table_ref.catalog,
schema_name: table_ref.schema,
table_name: table_ref.table,
};
let table_route_value = build_table_route_value(self.data.table_route.clone())?;
// To protect the potential resource leak issues.
// We must compare the table route value, before deleting.
let txn = Txn::new()
.when(vec![Compare::with_value(
table_route_key.to_string().into_bytes(),
CompareOp::Equal,
table_route_value.into(),
)])
.and_then(vec![TxnOp::Delete(
table_route_key.to_string().into_bytes(),
)]);
let resp = self.context.kv_store.txn(txn).await?;
ensure!( ensure!(
resp.succeeded, exist,
error::TxnSnafu { error::TableNotFoundSnafu {
msg: "table_route_value changed" name: table_ref.to_string()
} }
); );
info!("Removed TableRouteValue for table: {table_id}");
self.data.state = DropTableState::RemoveMetadata;
Ok(Status::executing(true))
}
/// Removes the table metadata.
async fn on_remove_metadata(&mut self) -> Result<Status> {
let table_metadata_manager = &self.context.table_metadata_manager;
let table_info_value = &self.data.table_info_value;
let table_route_value = &self.data.table_route_value;
let table_id = self.data.table_id();
table_metadata_manager
.delete_table_metadata(table_info_value, table_route_value)
.await
.context(error::TableMetadataManagerSnafu)?;
info!("Deleted table metadata for table {table_id}");
self.data.state = DropTableState::InvalidateTableCache; self.data.state = DropTableState::InvalidateTableCache;
@@ -181,20 +152,20 @@ impl DropTableProcedure {
/// Executes drop table instruction on datanode. /// Executes drop table instruction on datanode.
async fn on_datanode_drop_table(&mut self) -> Result<Status> { async fn on_datanode_drop_table(&mut self) -> Result<Status> {
let table_route = &self.data.table_route; let region_routes = &self.data.region_routes();
let table_ref = self.data.table_ref(); let table_ref = self.data.table_ref();
let table_id = self.data.task.table_id; let table_id = self.data.task.table_id;
let clients = self.context.datanode_clients.clone(); let clients = self.context.datanode_clients.clone();
let leaders = table_route.find_leaders(); let leaders = find_leaders(region_routes);
let mut joins = Vec::with_capacity(leaders.len()); let mut joins = Vec::with_capacity(leaders.len());
let expr = DropTableExpr { let expr = DropTableExpr {
catalog_name: table_ref.catalog.to_string(), catalog_name: table_ref.catalog.to_string(),
schema_name: table_ref.schema.to_string(), schema_name: table_ref.schema.to_string(),
table_name: table_ref.table.to_string(), table_name: table_ref.table.to_string(),
table_id: Some(TableId { id: table_id }), table_id: Some(api::v1::TableId { id: table_id }),
}; };
for datanode in leaders { for datanode in leaders {
@@ -232,6 +203,7 @@ impl Procedure for DropTableProcedure {
async fn execute(&mut self, _ctx: &ProcedureContext) -> ProcedureResult<Status> { async fn execute(&mut self, _ctx: &ProcedureContext) -> ProcedureResult<Status> {
match self.data.state { match self.data.state {
DropTableState::Prepare => self.on_prepare().await,
DropTableState::RemoveMetadata => self.on_remove_metadata().await, DropTableState::RemoveMetadata => self.on_remove_metadata().await,
DropTableState::InvalidateTableCache => self.on_broadcast().await, DropTableState::InvalidateTableCache => self.on_broadcast().await,
DropTableState::DatanodeDropTable => self.on_datanode_drop_table().await, DropTableState::DatanodeDropTable => self.on_datanode_drop_table().await,
@@ -260,16 +232,23 @@ pub struct DropTableData {
state: DropTableState, state: DropTableState,
cluster_id: u64, cluster_id: u64,
task: DropTableTask, task: DropTableTask,
table_route: TableRoute, table_route_value: TableRouteValue,
table_info_value: TableInfoValue,
} }
impl DropTableData { impl DropTableData {
pub fn new(cluster_id: u64, task: DropTableTask, table_route: TableRoute) -> Self { pub fn new(
cluster_id: u64,
task: DropTableTask,
table_route_value: TableRouteValue,
table_info_value: TableInfoValue,
) -> Self {
Self { Self {
state: DropTableState::RemoveMetadata, state: DropTableState::Prepare,
cluster_id, cluster_id,
task, task,
table_route, table_info_value,
table_route_value,
} }
} }
@@ -280,10 +259,24 @@ impl DropTableData {
fn table_name(&self) -> TableName { fn table_name(&self) -> TableName {
self.task.table_name() self.task.table_name()
} }
fn region_routes(&self) -> &Vec<RegionRoute> {
&self.table_route_value.region_routes
}
fn table_info(&self) -> &RawTableInfo {
&self.table_info_value.table_info
}
fn table_id(&self) -> TableId {
self.table_info().ident.table_id
}
} }
#[derive(Debug, Serialize, Deserialize)] #[derive(Debug, Serialize, Deserialize)]
enum DropTableState { enum DropTableState {
/// Prepares to drop the table
Prepare,
/// Removes metadata /// Removes metadata
RemoveMetadata, RemoveMetadata,
/// Invalidates Table Cache /// Invalidates Table Cache

View File

@@ -201,8 +201,8 @@ impl RegionFailoverManager {
let table_ident = &failed_region.table_ident; let table_ident = &failed_region.table_ident;
Ok(self Ok(self
.table_metadata_manager .table_metadata_manager
.table_region_manager() .table_route_manager()
.get(table_ident.table_id) .get_region_distribution(table_ident.table_id)
.await .await
.context(TableMetadataManagerSnafu)? .context(TableMetadataManagerSnafu)?
.is_some()) .is_some())
@@ -410,16 +410,15 @@ mod tests {
impl TestingEnv { impl TestingEnv {
pub async fn failed_region(&self, region_number: u32) -> RegionIdent { pub async fn failed_region(&self, region_number: u32) -> RegionIdent {
let value = self let region_distribution = self
.context .context
.table_metadata_manager .table_metadata_manager
.table_region_manager() .table_route_manager()
.get(1) .get_region_distribution(1)
.await .await
.unwrap() .unwrap()
.unwrap(); .unwrap();
let failed_datanode = value let failed_datanode = region_distribution
.region_distribution
.iter() .iter()
.find_map(|(&datanode_id, regions)| { .find_map(|(&datanode_id, regions)| {
if regions.contains(&region_number) { if regions.contains(&region_number) {
@@ -473,15 +472,13 @@ mod tests {
table, table,
) )
.await; .await;
let table_region_value = table_metadata_manager let region_distribution = table_metadata_manager
.table_region_manager() .table_route_manager()
.get(1) .get_region_distribution(1)
.await .await
.unwrap() .unwrap()
.unwrap(); .unwrap();
let _ = table_routes::tests::prepare_table_route_value(&kv_store, table).await;
let pushers = Pushers::default(); let pushers = Pushers::default();
let mut heartbeat_receivers = HashMap::with_capacity(3); let mut heartbeat_receivers = HashMap::with_capacity(3);
for datanode_id in 1..=3 { for datanode_id in 1..=3 {
@@ -499,7 +496,7 @@ mod tests {
let mailbox = HeartbeatMailbox::create(pushers.clone(), mailbox_sequence); let mailbox = HeartbeatMailbox::create(pushers.clone(), mailbox_sequence);
let selector = self.selector.unwrap_or_else(|| { let selector = self.selector.unwrap_or_else(|| {
let nodes = (1..=table_region_value.region_distribution.len()) let nodes = (1..=region_distribution.len())
.map(|id| Peer { .map(|id| Peer {
id: id as u64, id: id as u64,
addr: "".to_string(), addr: "".to_string(),
@@ -639,23 +636,19 @@ mod tests {
); );
// Verifies that the failed region (region 1) is moved from failed datanode (datanode 1) to the candidate datanode. // Verifies that the failed region (region 1) is moved from failed datanode (datanode 1) to the candidate datanode.
let value = env let region_distribution = env
.context .context
.table_metadata_manager .table_metadata_manager
.table_region_manager() .table_route_manager()
.get(1) .get_region_distribution(1)
.await .await
.unwrap() .unwrap()
.unwrap(); .unwrap();
assert_eq!( assert_eq!(
value region_distribution.get(&failed_region.datanode_id).unwrap(),
.region_distribution
.get(&failed_region.datanode_id)
.unwrap(),
&vec![2] &vec![2]
); );
assert!(value assert!(region_distribution
.region_distribution
.get(&candidate_rx.recv().await.unwrap()) .get(&candidate_rx.recv().await.unwrap())
.unwrap() .unwrap()
.contains(&1)); .contains(&1));

View File

@@ -12,11 +12,10 @@
// See the License for the specific language governing permissions and // See the License for the specific language governing permissions and
// limitations under the License. // limitations under the License.
use api::v1::meta::{TableName as PbTableName, TableRouteValue};
use async_trait::async_trait; use async_trait::async_trait;
use common_meta::key::TableRouteKey; use common_meta::key::table_route::NextTableRouteKey;
use common_meta::peer::Peer; use common_meta::peer::Peer;
use common_meta::rpc::router::TableRoute; use common_meta::rpc::router::RegionRoute;
use common_meta::RegionIdent; use common_meta::RegionIdent;
use common_telemetry::info; use common_telemetry::info;
use serde::{Deserialize, Serialize}; use serde::{Deserialize, Serialize};
@@ -24,13 +23,9 @@ use snafu::{OptionExt, ResultExt};
use super::invalidate_cache::InvalidateCache; use super::invalidate_cache::InvalidateCache;
use super::{RegionFailoverContext, State}; use super::{RegionFailoverContext, State};
use crate::error::{ use crate::error::{self, Result, RetryLaterSnafu};
CorruptedTableRouteSnafu, Result, RetryLaterSnafu, TableMetadataManagerSnafu,
TableNotFoundSnafu, TableRouteConversionSnafu, UpdateTableMetadataSnafu,
};
use crate::lock::keys::table_metadata_lock_key; use crate::lock::keys::table_metadata_lock_key;
use crate::lock::Opts; use crate::lock::Opts;
use crate::table_routes;
#[derive(Serialize, Deserialize, Debug)] #[derive(Serialize, Deserialize, Debug)]
pub(super) struct UpdateRegionMetadata { pub(super) struct UpdateRegionMetadata {
@@ -51,133 +46,59 @@ impl UpdateRegionMetadata {
let key = table_metadata_lock_key(failed_region); let key = table_metadata_lock_key(failed_region);
let key = ctx.dist_lock.lock(key, Opts::default()).await?; let key = ctx.dist_lock.lock(key, Opts::default()).await?;
self.update_table_region_value(ctx, failed_region).await?;
self.update_region_placement(ctx, failed_region).await?;
self.update_table_route(ctx, failed_region).await?; self.update_table_route(ctx, failed_region).await?;
ctx.dist_lock.unlock(key).await?; ctx.dist_lock.unlock(key).await?;
Ok(()) Ok(())
} }
async fn update_table_region_value(
&self,
ctx: &RegionFailoverContext,
failed_region: &RegionIdent,
) -> Result<()> {
let table_region_manager = ctx.table_metadata_manager.table_region_manager();
let table_ident = &failed_region.table_ident;
let table_id = table_ident.table_id;
let value = table_region_manager
.get(table_id)
.await
.context(TableRouteConversionSnafu)?
.with_context(|| TableNotFoundSnafu {
name: table_ident.to_string(),
})?;
let mut region_distribution = value.region_distribution.clone();
if let Some(mut region_numbers) = region_distribution.remove(&failed_region.datanode_id) {
region_numbers.retain(|x| *x != failed_region.region_number);
if !region_numbers.is_empty() {
region_distribution.insert(failed_region.datanode_id, region_numbers);
}
}
let region_numbers = region_distribution
.entry(self.candidate.id)
.or_insert_with(Vec::new);
region_numbers.push(failed_region.region_number);
table_region_manager
.compare_and_put(table_id, Some(value.clone()), region_distribution.clone())
.await
.context(TableMetadataManagerSnafu)?
.map_err(|curr| UpdateTableMetadataSnafu {
err_msg: format!("region distribution is concurrently updating, expected '{value:?}' but actual: '{curr:?}'")
}.build())?;
info!(
"Region distribution of table (id = {table_id}) is updated to {:?}. \
Failed region {} was on Datanode {}.",
region_distribution, failed_region.region_number, failed_region.datanode_id,
);
Ok(())
}
async fn update_region_placement(
&self,
ctx: &RegionFailoverContext,
failed_region: &RegionIdent,
) -> Result<()> {
ctx.table_metadata_manager
.datanode_table_manager()
.move_region(
failed_region.datanode_id,
self.candidate.id,
failed_region.table_ident.table_id,
failed_region.region_number,
)
.await
.context(TableMetadataManagerSnafu)?;
Ok(())
}
async fn update_table_route( async fn update_table_route(
&self, &self,
ctx: &RegionFailoverContext, ctx: &RegionFailoverContext,
failed_region: &RegionIdent, failed_region: &RegionIdent,
) -> Result<()> { ) -> Result<()> {
let table_name = PbTableName { let table_id = failed_region.table_ident.table_id;
catalog_name: failed_region.table_ident.catalog.clone(),
schema_name: failed_region.table_ident.schema.clone(),
table_name: failed_region.table_ident.table.clone(),
};
let key =
TableRouteKey::with_table_name(failed_region.table_ident.table_id as _, &table_name);
let value = table_routes::get_table_route_value(&ctx.selector_ctx.kv_store, &key).await?;
let table_route = value let table_route_value = ctx
.table_route .table_metadata_manager
.with_context(|| CorruptedTableRouteSnafu { .table_route_manager()
key: key.to_string(), .get(table_id)
reason: "'table_route' is empty", .await
.context(error::TableMetadataManagerSnafu)?
.with_context(|| error::TableRouteNotFoundSnafu {
table_name: failed_region.table_ident.table_ref().to_string(),
})?; })?;
let mut table_route = TableRoute::try_from_raw(&value.peers, table_route)
.context(TableRouteConversionSnafu)?;
for region_route in table_route.region_routes.iter_mut() { let mut new_region_routes = table_route_value.region_routes.clone();
for region_route in new_region_routes.iter_mut() {
if region_route.region.id == failed_region.region_number as u64 { if region_route.region.id == failed_region.region_number as u64 {
region_route.leader_peer = Some(self.candidate.clone()); region_route.leader_peer = Some(self.candidate.clone());
break; break;
} }
} }
pretty_log_table_route_change(&key, &table_route, failed_region); pretty_log_table_route_change(
NextTableRouteKey::new(table_id).to_string(),
&new_region_routes,
failed_region,
);
let (peers, table_route) = table_route ctx.table_metadata_manager
.try_into_raw() .update_table_route(table_id, table_route_value, new_region_routes)
.context(TableRouteConversionSnafu)?; .await
.context(error::UpdateTableRouteSnafu)?;
let value = TableRouteValue {
peers,
table_route: Some(table_route),
};
table_routes::put_table_route_value(&ctx.selector_ctx.kv_store, &key, value).await?;
Ok(()) Ok(())
} }
} }
fn pretty_log_table_route_change( fn pretty_log_table_route_change(
key: &TableRouteKey, key: String,
table_route: &TableRoute, region_routes: &[RegionRoute],
failed_region: &RegionIdent, failed_region: &RegionIdent,
) { ) {
let region_routes = table_route let region_routes = region_routes
.region_routes
.iter() .iter()
.map(|x| { .map(|x| {
format!( format!(
@@ -199,7 +120,7 @@ fn pretty_log_table_route_change(
info!( info!(
"Updating region routes in table route value (key = '{}') to [{}]. \ "Updating region routes in table route value (key = '{}') to [{}]. \
Failed region {} was on Datanode {}.", Failed region {} was on Datanode {}.",
key.to_string(), key,
region_routes.join(", "), region_routes.join(", "),
failed_region.region_number, failed_region.region_number,
failed_region.datanode_id, failed_region.datanode_id,
@@ -231,11 +152,9 @@ impl State for UpdateRegionMetadata {
#[cfg(test)] #[cfg(test)]
mod tests { mod tests {
use api::v1::meta::TableRouteValue;
use common_meta::key::table_region::TableRegionValue; use common_meta::rpc::router::{extract_all_peers, region_distribution};
use common_meta::key::TableRouteKey; use futures::TryStreamExt;
use common_meta::DatanodeId;
use store_api::storage::RegionNumber;
use super::super::tests::{TestingEnv, TestingEnvBuilder}; use super::super::tests::{TestingEnv, TestingEnvBuilder};
use super::{State, *}; use super::{State, *};
@@ -255,84 +174,11 @@ mod tests {
assert_eq!(format!("{next_state:?}"), "InvalidateCache"); assert_eq!(format!("{next_state:?}"), "InvalidateCache");
} }
#[tokio::test]
async fn test_update_table_region_value() {
common_telemetry::init_default_ut_logging();
async fn test(
env: TestingEnv,
failed_region: RegionNumber,
candidate: DatanodeId,
) -> TableRegionValue {
let failed_region = env.failed_region(failed_region).await;
let state = UpdateRegionMetadata::new(Peer::new(candidate, ""));
state
.update_table_region_value(&env.context, &failed_region)
.await
.unwrap();
env.context
.table_metadata_manager
.table_region_manager()
.get(failed_region.table_ident.table_id)
.await
.unwrap()
.unwrap()
}
// Region distribution:
// Datanode => Regions
// 1 => 1, 2
// 2 => 3
// 3 => 4
// Testing failed region 1 moves to Datanode 2.
let env = TestingEnvBuilder::new().build().await;
let updated = test(env, 1, 2).await;
let new_region_id_map = updated.region_distribution;
assert_eq!(new_region_id_map.len(), 3);
assert_eq!(new_region_id_map.get(&1), Some(&vec![2]));
assert_eq!(new_region_id_map.get(&2), Some(&vec![3, 1]));
assert_eq!(new_region_id_map.get(&3), Some(&vec![4]));
// Testing failed region 3 moves to Datanode 3.
let env = TestingEnvBuilder::new().build().await;
let updated = test(env, 3, 3).await;
let new_region_id_map = updated.region_distribution;
assert_eq!(new_region_id_map.len(), 2);
assert_eq!(new_region_id_map.get(&1), Some(&vec![1, 2]));
assert_eq!(new_region_id_map.get(&3), Some(&vec![4, 3]));
// Testing failed region 1 moves to a new Datanode, 4.
let env = TestingEnvBuilder::new().build().await;
let updated = test(env, 1, 4).await;
let new_region_id_map = updated.region_distribution;
assert_eq!(new_region_id_map.len(), 4);
assert_eq!(new_region_id_map.get(&1), Some(&vec![2]));
assert_eq!(new_region_id_map.get(&2), Some(&vec![3]));
assert_eq!(new_region_id_map.get(&3), Some(&vec![4]));
assert_eq!(new_region_id_map.get(&4), Some(&vec![1]));
// Testing failed region 3 moves to a new Datanode, 4.
let env = TestingEnvBuilder::new().build().await;
let updated = test(env, 3, 4).await;
let new_region_id_map = updated.region_distribution;
assert_eq!(new_region_id_map.len(), 3);
assert_eq!(new_region_id_map.get(&1), Some(&vec![1, 2]));
assert_eq!(new_region_id_map.get(&3), Some(&vec![4]));
assert_eq!(new_region_id_map.get(&4), Some(&vec![3]));
}
#[tokio::test] #[tokio::test]
async fn test_update_table_route() { async fn test_update_table_route() {
common_telemetry::init_default_ut_logging(); common_telemetry::init_default_ut_logging();
async fn test(env: TestingEnv, failed_region: u32, candidate: u64) -> TableRouteValue { async fn test(env: TestingEnv, failed_region: u32, candidate: u64) -> Vec<RegionRoute> {
let failed_region = env.failed_region(failed_region).await; let failed_region = env.failed_region(failed_region).await;
let state = UpdateRegionMetadata::new(Peer::new(candidate, "")); let state = UpdateRegionMetadata::new(Peer::new(candidate, ""));
@@ -341,15 +187,16 @@ mod tests {
.await .await
.unwrap(); .unwrap();
let key = TableRouteKey { let table_id = failed_region.table_ident.table_id;
table_id: failed_region.table_ident.table_id,
catalog_name: &failed_region.table_ident.catalog, env.context
schema_name: &failed_region.table_ident.schema, .table_metadata_manager
table_name: &failed_region.table_ident.table, .table_route_manager()
}; .get(table_id)
table_routes::get_table_route_value(&env.context.selector_ctx.kv_store, &key)
.await .await
.unwrap() .unwrap()
.unwrap()
.region_routes
} }
// Original region routes: // Original region routes:
@@ -361,8 +208,7 @@ mod tests {
// Testing failed region 1 moves to Datanode 2. // Testing failed region 1 moves to Datanode 2.
let env = TestingEnvBuilder::new().build().await; let env = TestingEnvBuilder::new().build().await;
let updated = test(env, 1, 2).await; let actual = test(env, 1, 2).await;
let actual = &updated.table_route.as_ref().unwrap().region_routes;
// Expected region routes: // Expected region routes:
// region number => leader node // region number => leader node
@@ -370,9 +216,9 @@ mod tests {
// 2 => 1 // 2 => 1
// 3 => 2 // 3 => 2
// 4 => 3 // 4 => 3
let peers = &updated.peers; let peers = &extract_all_peers(&actual);
assert_eq!(peers.len(), 3); assert_eq!(peers.len(), 3);
let expected = &vec![ let expected = vec![
new_region_route(1, peers, 2), new_region_route(1, peers, 2),
new_region_route(2, peers, 1), new_region_route(2, peers, 1),
new_region_route(3, peers, 2), new_region_route(3, peers, 2),
@@ -382,8 +228,7 @@ mod tests {
// Testing failed region 3 moves to Datanode 3. // Testing failed region 3 moves to Datanode 3.
let env = TestingEnvBuilder::new().build().await; let env = TestingEnvBuilder::new().build().await;
let updated = test(env, 3, 3).await; let actual = test(env, 3, 3).await;
let actual = &updated.table_route.as_ref().unwrap().region_routes;
// Expected region routes: // Expected region routes:
// region number => leader node // region number => leader node
@@ -391,9 +236,9 @@ mod tests {
// 2 => 1 // 2 => 1
// 3 => 3 // 3 => 3
// 4 => 3 // 4 => 3
let peers = &updated.peers; let peers = &extract_all_peers(&actual);
assert_eq!(peers.len(), 2); assert_eq!(peers.len(), 2);
let expected = &vec![ let expected = vec![
new_region_route(1, peers, 1), new_region_route(1, peers, 1),
new_region_route(2, peers, 1), new_region_route(2, peers, 1),
new_region_route(3, peers, 3), new_region_route(3, peers, 3),
@@ -403,8 +248,7 @@ mod tests {
// Testing failed region 1 moves to a new Datanode, 4. // Testing failed region 1 moves to a new Datanode, 4.
let env = TestingEnvBuilder::new().build().await; let env = TestingEnvBuilder::new().build().await;
let updated = test(env, 1, 4).await; let actual = test(env, 1, 4).await;
let actual = &updated.table_route.as_ref().unwrap().region_routes;
// Expected region routes: // Expected region routes:
// region number => leader node // region number => leader node
@@ -412,9 +256,9 @@ mod tests {
// 2 => 1 // 2 => 1
// 3 => 2 // 3 => 2
// 4 => 3 // 4 => 3
let peers = &updated.peers; let peers = &extract_all_peers(&actual);
assert_eq!(peers.len(), 4); assert_eq!(peers.len(), 4);
let expected = &vec![ let expected = vec![
new_region_route(1, peers, 4), new_region_route(1, peers, 4),
new_region_route(2, peers, 1), new_region_route(2, peers, 1),
new_region_route(3, peers, 2), new_region_route(3, peers, 2),
@@ -424,8 +268,7 @@ mod tests {
// Testing failed region 3 moves to a new Datanode, 4. // Testing failed region 3 moves to a new Datanode, 4.
let env = TestingEnvBuilder::new().build().await; let env = TestingEnvBuilder::new().build().await;
let updated = test(env, 3, 4).await; let actual = test(env, 3, 4).await;
let actual = &updated.table_route.as_ref().unwrap().region_routes;
// Expected region routes: // Expected region routes:
// region number => leader node // region number => leader node
@@ -433,9 +276,9 @@ mod tests {
// 2 => 1 // 2 => 1
// 3 => 4 // 3 => 4
// 4 => 3 // 4 => 3
let peers = &updated.peers; let peers = &extract_all_peers(&actual);
assert_eq!(peers.len(), 3); assert_eq!(peers.len(), 3);
let expected = &vec![ let expected = vec![
new_region_route(1, peers, 1), new_region_route(1, peers, 1),
new_region_route(2, peers, 1), new_region_route(2, peers, 1),
new_region_route(3, peers, 4), new_region_route(3, peers, 4),
@@ -475,9 +318,6 @@ mod tests {
let failed_region_1 = env.failed_region(1).await; let failed_region_1 = env.failed_region(1).await;
let failed_region_2 = env.failed_region(2).await; let failed_region_2 = env.failed_region(2).await;
let catalog_name = failed_region_1.table_ident.catalog.clone();
let schema_name = failed_region_1.table_ident.schema.clone();
let table_name = failed_region_1.table_ident.table.clone();
let table_id = failed_region_1.table_ident.table_id; let table_id = failed_region_1.table_ident.table_id;
let _ = futures::future::join_all(vec![ let _ = futures::future::join_all(vec![
@@ -498,24 +338,17 @@ mod tests {
]) ])
.await; .await;
let table_route_key = TableRouteKey { let table_route_value = env
table_id, .context
catalog_name: &catalog_name, .table_metadata_manager
schema_name: &schema_name, .table_route_manager()
table_name: &table_name, .get(table_id)
}; .await
let table_route_value = table_routes::get_table_route_value(
&env.context.selector_ctx.kv_store,
&table_route_key,
)
.await
.unwrap();
let peers = &table_route_value.peers;
let actual = &table_route_value
.table_route
.as_ref()
.unwrap() .unwrap()
.region_routes; .unwrap();
let peers = &extract_all_peers(&table_route_value.region_routes);
let actual = &table_route_value.region_routes;
let expected = &vec![ let expected = &vec![
new_region_route(1, peers, 2), new_region_route(1, peers, 2),
new_region_route(2, peers, 3), new_region_route(2, peers, 3),
@@ -526,31 +359,44 @@ mod tests {
assert_eq!(actual, expected); assert_eq!(actual, expected);
let manager = &env.context.table_metadata_manager; let manager = &env.context.table_metadata_manager;
let map = manager let table_route_value = manager
.table_region_manager() .table_route_manager()
.get(table_id) .get(table_id)
.await .await
.unwrap() .unwrap()
.unwrap() .unwrap();
.region_distribution;
let map = region_distribution(&table_route_value.region_routes).unwrap();
assert_eq!(map.len(), 2); assert_eq!(map.len(), 2);
assert_eq!(map.get(&2), Some(&vec![3, 1])); assert_eq!(map.get(&2), Some(&vec![1, 3]));
assert_eq!(map.get(&3), Some(&vec![4, 2])); assert_eq!(map.get(&3), Some(&vec![2, 4]));
// test DatanodeTableValues matches the table region distribution // test DatanodeTableValues matches the table region distribution
let datanode_table_manager = manager.datanode_table_manager(); let datanode_table_manager = manager.datanode_table_manager();
let tables = datanode_table_manager.tables(1).await.unwrap(); let tables = datanode_table_manager
.tables(1)
.try_collect::<Vec<_>>()
.await
.unwrap();
assert!(tables.is_empty()); assert!(tables.is_empty());
let tables = datanode_table_manager.tables(2).await.unwrap(); let tables = datanode_table_manager
.tables(2)
.try_collect::<Vec<_>>()
.await
.unwrap();
assert_eq!(tables.len(), 1); assert_eq!(tables.len(), 1);
assert_eq!(tables[0].table_id, 1); assert_eq!(tables[0].table_id, 1);
assert_eq!(tables[0].regions, vec![3, 1]); assert_eq!(tables[0].regions, vec![1, 3]);
let tables = datanode_table_manager.tables(3).await.unwrap(); let tables = datanode_table_manager
.tables(3)
.try_collect::<Vec<_>>()
.await
.unwrap();
assert_eq!(tables.len(), 1); assert_eq!(tables.len(), 1);
assert_eq!(tables[0].table_id, 1); assert_eq!(tables[0].table_id, 1);
assert_eq!(tables[0].regions, vec![4, 2]); assert_eq!(tables[0].regions, vec![2, 4]);
} }
} }
} }

View File

@@ -12,24 +12,11 @@
// See the License for the specific language governing permissions and // See the License for the specific language governing permissions and
// limitations under the License. // limitations under the License.
use api::v1::meta::TableRouteValue;
use common_meta::peer::Peer; use common_meta::peer::Peer;
use common_meta::rpc::router::TableRoute;
use common_procedure::error::Error as ProcedureError; use common_procedure::error::Error as ProcedureError;
use snafu::{location, Location, ResultExt}; use snafu::{location, Location};
use crate::error::{self, Error, Result}; use crate::error::{self, Error};
pub fn build_table_route_value(table_route: TableRoute) -> Result<TableRouteValue> {
let (peers, table_route) = table_route
.try_into_raw()
.context(error::ConvertProtoDataSnafu)?;
Ok(TableRouteValue {
peers,
table_route: Some(table_route),
})
}
pub fn handle_request_datanode_error(datanode: Peer) -> impl FnOnce(client::error::Error) -> Error { pub fn handle_request_datanode_error(datanode: Peer) -> impl FnOnce(client::error::Error) -> Error {
move |err| { move |err| {

View File

@@ -165,13 +165,13 @@ impl HttpHandler for TableHandler {
if let Some(table_id) = table_id { if let Some(table_id) = table_id {
let table_region_value = self let table_region_value = self
.table_metadata_manager .table_metadata_manager
.table_region_manager() .table_route_manager()
.get(table_id) .get(table_id)
.await .await
.context(TableMetadataManagerSnafu)? .context(TableMetadataManagerSnafu)?
.map(|x| format!("{x:?}")) .map(|x| format!("{x:?}"))
.unwrap_or_else(|| "Not Found".to_string()); .unwrap_or_else(|| "Not Found".to_string());
result.insert("table_region_value", table_region_value); result.insert("table_route_value", table_region_value);
} }
http::Response::builder() http::Response::builder()

View File

@@ -13,29 +13,25 @@
// limitations under the License. // limitations under the License.
use api::v1::meta::{ use api::v1::meta::{
ddl_task_server, Partition, Region, RegionRoute, SubmitDdlTaskRequest, SubmitDdlTaskResponse, ddl_task_server, Partition, SubmitDdlTaskRequest, SubmitDdlTaskResponse, TableId,
Table, TableId, TableRoute,
}; };
use common_grpc_expr::alter_expr_to_request; use common_grpc_expr::alter_expr_to_request;
use common_meta::key::TableRouteKey;
use common_meta::rpc::ddl::{ use common_meta::rpc::ddl::{
AlterTableTask, CreateTableTask, DdlTask, DropTableTask, TruncateTableTask, AlterTableTask, CreateTableTask, DdlTask, DropTableTask, TruncateTableTask,
}; };
use common_meta::rpc::router; use common_meta::rpc::router::{Region, RegionRoute};
use common_meta::table_name::TableName; use common_meta::table_name::TableName;
use common_telemetry::{info, warn}; use common_telemetry::{info, warn};
use snafu::{ensure, OptionExt, ResultExt}; use snafu::{ensure, OptionExt, ResultExt};
use store_api::storage::MAX_REGION_SEQ; use store_api::storage::{RegionId, MAX_REGION_SEQ};
use table::metadata::RawTableInfo; use table::metadata::RawTableInfo;
use tonic::{Request, Response}; use tonic::{Request, Response};
use super::store::kv::KvStoreRef;
use super::GrpcResult; use super::GrpcResult;
use crate::ddl::DdlManagerRef; use crate::ddl::DdlManagerRef;
use crate::error::{self, Result, TableMetadataManagerSnafu, TooManyPartitionsSnafu}; use crate::error::{self, Result, TableMetadataManagerSnafu, TooManyPartitionsSnafu};
use crate::metasrv::{MetaSrv, SelectorContext, SelectorRef}; use crate::metasrv::{MetaSrv, SelectorContext, SelectorRef};
use crate::sequence::SequenceRef; use crate::sequence::SequenceRef;
use crate::table_routes::get_table_route_value;
#[async_trait::async_trait] #[async_trait::async_trait]
impl ddl_task_server::DdlTask for MetaSrv { impl ddl_task_server::DdlTask for MetaSrv {
@@ -77,7 +73,6 @@ impl ddl_task_server::DdlTask for MetaSrv {
handle_drop_table_task( handle_drop_table_task(
header.cluster_id, header.cluster_id,
drop_table_task, drop_table_task,
self.kv_store().clone(),
self.ddl_manager().clone(), self.ddl_manager().clone(),
) )
.await? .await?
@@ -94,7 +89,6 @@ impl ddl_task_server::DdlTask for MetaSrv {
handle_truncate_table_task( handle_truncate_table_task(
header.cluster_id, header.cluster_id,
truncate_table_task, truncate_table_task,
self.kv_store().clone(),
self.ddl_manager().clone(), self.ddl_manager().clone(),
) )
.await? .await?
@@ -132,7 +126,7 @@ async fn handle_create_table_task(
.map(Into::into) .map(Into::into)
.collect(); .collect();
let table_route = handle_create_table_route( let region_routes = handle_create_region_routes(
cluster_id, cluster_id,
table_name, table_name,
partitions, partitions,
@@ -142,25 +136,25 @@ async fn handle_create_table_task(
table_id_sequence, table_id_sequence,
) )
.await?; .await?;
let table_id = table_route.table.id;
let table_id = create_table_task.table_info.ident.table_id;
// TODO(weny): refactor the table route.
let id = ddl_manager let id = ddl_manager
.submit_create_table_task(cluster_id, create_table_task, table_route) .submit_create_table_task(cluster_id, create_table_task, region_routes)
.await?; .await?;
info!("Table: {table_id} is dropped via procedure_id {id:?}"); info!("Table: {table_id} is dropped via procedure_id {id:?}");
Ok(SubmitDdlTaskResponse { Ok(SubmitDdlTaskResponse {
key: id.to_string().into(), key: id.to_string().into(),
table_id: Some(TableId { table_id: Some(TableId { id: table_id }),
id: table_id as u32,
}),
..Default::default() ..Default::default()
}) })
} }
/// pre-calculates create table task's metadata. /// pre-calculates create table task's metadata.
async fn handle_create_table_route( async fn handle_create_region_routes(
cluster_id: u64, cluster_id: u64,
table_name: TableName, table_name: TableName,
partitions: Vec<Partition>, partitions: Vec<Partition>,
@@ -168,7 +162,7 @@ async fn handle_create_table_route(
ctx: SelectorContext, ctx: SelectorContext,
selector: SelectorRef, selector: SelectorRef,
table_id_sequence: SequenceRef, table_id_sequence: SequenceRef,
) -> Result<router::TableRoute> { ) -> Result<Vec<RegionRoute>> {
let mut peers = selector.select(cluster_id, &ctx).await?; let mut peers = selector.select(cluster_id, &ctx).await?;
if peers.len() < partitions.len() { if peers.len() < partitions.len() {
@@ -187,70 +181,61 @@ async fn handle_create_table_route(
let id = table_id_sequence.next().await?; let id = table_id_sequence.next().await?;
table_info.ident.table_id = id as u32; table_info.ident.table_id = id as u32;
let table = Table {
id,
table_name: Some(table_name.into()),
..Default::default()
};
ensure!( ensure!(
partitions.len() <= MAX_REGION_SEQ as usize, partitions.len() <= MAX_REGION_SEQ as usize,
TooManyPartitionsSnafu TooManyPartitionsSnafu
); );
let region_routes = partitions let region_routes = partitions
.into_iter() .into_iter()
.enumerate() .enumerate()
.map(|(i, partition)| { .map(|(i, partition)| {
let region = Region { let region = Region {
id: i as u64, id: RegionId::from_u64(i as u64),
partition: Some(partition), partition: Some(partition.into()),
..Default::default() ..Default::default()
}; };
let peer = peers[i % peers.len()].clone();
RegionRoute { RegionRoute {
region: Some(region), region,
leader_peer_index: (i % peers.len()) as u64, leader_peer: Some(peer.into()),
follower_peer_indexes: vec![], // follower_peers is not supported at the moment follower_peers: vec![], // follower_peers is not supported at the moment
} }
}) })
.collect::<Vec<_>>(); .collect::<Vec<_>>();
let table_route = TableRoute { Ok(region_routes)
table: Some(table),
region_routes,
};
router::TableRoute::try_from_raw(&peers, table_route).context(error::TableRouteConversionSnafu)
} }
async fn handle_drop_table_task( async fn handle_drop_table_task(
cluster_id: u64, cluster_id: u64,
drop_table_task: DropTableTask, drop_table_task: DropTableTask,
kv_store: KvStoreRef,
ddl_manager: DdlManagerRef, ddl_manager: DdlManagerRef,
) -> Result<SubmitDdlTaskResponse> { ) -> Result<SubmitDdlTaskResponse> {
let table_id = drop_table_task.table_id; let table_id = drop_table_task.table_id;
let table_metadata_manager = &ddl_manager.table_metadata_manager;
let table_ref = drop_table_task.table_ref();
let table_route_key = TableRouteKey { let (table_info_value, table_route_value) = table_metadata_manager
table_id, .get_full_table_info(table_id)
catalog_name: &drop_table_task.catalog, .await
schema_name: &drop_table_task.schema, .context(error::TableMetadataManagerSnafu)?;
table_name: &drop_table_task.table,
};
let table_route_value = get_table_route_value(&kv_store, &table_route_key).await?; let table_info_value = table_info_value.with_context(|| error::TableInfoNotFoundSnafu {
table_name: table_ref.to_string(),
})?;
let table_route = router::TableRoute::try_from_raw( let table_route_value = table_route_value.with_context(|| error::TableRouteNotFoundSnafu {
&table_route_value.peers, table_name: table_ref.to_string(),
table_route_value })?;
.table_route
.context(error::UnexpectedSnafu {
violated: "expected table_route",
})?,
)
.context(error::TableRouteConversionSnafu)?;
let id = ddl_manager let id = ddl_manager
.submit_drop_table_task(cluster_id, drop_table_task, table_route) .submit_drop_table_task(
cluster_id,
drop_table_task,
table_info_value,
table_route_value,
)
.await?; .await?;
Ok(SubmitDdlTaskResponse { Ok(SubmitDdlTaskResponse {
@@ -285,10 +270,11 @@ async fn handle_alter_table_task(
.get(table_id) .get(table_id)
.await .await
.context(TableMetadataManagerSnafu)? .context(TableMetadataManagerSnafu)?
.with_context(|| error::TableNotFoundSnafu { .with_context(|| error::TableInfoNotFoundSnafu {
name: table_ref.to_string(), table_name: table_ref.to_string(),
})?; })?;
let table_info = table_info_value.table_info;
let table_info = &table_info_value.table_info;
// Sets alter_table's table_version // Sets alter_table's table_version
alter_table_task.alter_table.table_version = table_info.ident.version; alter_table_task.alter_table.table_version = table_info.ident.version;
@@ -299,7 +285,7 @@ async fn handle_alter_table_task(
cluster_id, cluster_id,
alter_table_task, alter_table_task,
alter_table_request, alter_table_request,
table_info, table_info_value,
) )
.await?; .await?;
@@ -314,7 +300,6 @@ async fn handle_alter_table_task(
async fn handle_truncate_table_task( async fn handle_truncate_table_task(
cluster_id: u64, cluster_id: u64,
truncate_table_task: TruncateTableTask, truncate_table_task: TruncateTableTask,
kv_store: KvStoreRef,
ddl_manager: DdlManagerRef, ddl_manager: DdlManagerRef,
) -> Result<SubmitDdlTaskResponse> { ) -> Result<SubmitDdlTaskResponse> {
let truncate_table = &truncate_table_task.truncate_table; let truncate_table = &truncate_table_task.truncate_table;
@@ -326,24 +311,19 @@ async fn handle_truncate_table_task(
})? })?
.id; .id;
let table_route_key = TableRouteKey { let table_ref = truncate_table_task.table_ref();
table_id,
catalog_name: &truncate_table.catalog_name,
schema_name: &truncate_table.schema_name,
table_name: &truncate_table.table_name,
};
let table_route_value = get_table_route_value(&kv_store, &table_route_key).await?; let table_route_value = ddl_manager
.table_metadata_manager
.table_route_manager()
.get(table_id)
.await
.context(TableMetadataManagerSnafu)?
.with_context(|| error::TableRouteNotFoundSnafu {
table_name: table_ref.to_string(),
})?;
let table_route = router::TableRoute::try_from_raw( let table_route = table_route_value.region_routes;
&table_route_value.peers,
table_route_value
.table_route
.context(error::UnexpectedSnafu {
violated: "expected table_route",
})?,
)
.context(error::TableRouteConversionSnafu)?;
let id = ddl_manager let id = ddl_manager
.submit_truncate_table_task(cluster_id, truncate_table_task, table_route) .submit_truncate_table_task(cluster_id, truncate_table_task, table_route)

View File

@@ -17,12 +17,10 @@ use api::v1::meta::{
TableRouteValue, TableRouteValue,
}; };
use common_meta::key::table_info::TableInfoValue; use common_meta::key::table_info::TableInfoValue;
use common_meta::key::table_region::RegionDistribution;
use common_telemetry::timer; use common_telemetry::timer;
use snafu::{OptionExt, ResultExt}; use snafu::ResultExt;
use tonic::{Request, Response}; use tonic::{Request, Response};
use crate::error;
use crate::error::{Result, TableMetadataManagerSnafu}; use crate::error::{Result, TableMetadataManagerSnafu};
use crate::metasrv::{Context, MetaSrv}; use crate::metasrv::{Context, MetaSrv};
use crate::metrics::METRIC_META_ROUTE_REQUEST; use crate::metrics::METRIC_META_ROUTE_REQUEST;
@@ -50,33 +48,6 @@ impl router_server::Router for MetaSrv {
} }
} }
pub(crate) fn create_region_distribution(
table_route_value: &TableRouteValue,
) -> Result<RegionDistribution> {
let peers = &table_route_value.peers;
let region_routes = &table_route_value
.table_route
.as_ref()
.context(error::UnexpectedSnafu {
violated: "table route should have been set",
})?
.region_routes;
let mut regions_id_map = RegionDistribution::new();
for route in region_routes.iter() {
let node_id = peers[route.leader_peer_index as usize].id;
let region_id = route
.region
.as_ref()
.context(error::UnexpectedSnafu {
violated: "region should have been set",
})?
.id as u32;
regions_id_map.entry(node_id).or_default().push(region_id);
}
Ok(regions_id_map)
}
async fn handle_route(req: RouteRequest, ctx: Context) -> Result<RouteResponse> { async fn handle_route(req: RouteRequest, ctx: Context) -> Result<RouteResponse> {
let RouteRequest { header, table_ids } = req; let RouteRequest { header, table_ids } = req;
let cluster_id = header.as_ref().map_or(0, |h| h.cluster_id); let cluster_id = header.as_ref().map_or(0, |h| h.cluster_id);

View File

@@ -14,82 +14,55 @@
use api::v1::meta::TableRouteValue; use api::v1::meta::TableRouteValue;
use common_meta::key::table_info::TableInfoValue; use common_meta::key::table_info::TableInfoValue;
use common_meta::key::{TableMetadataManagerRef, TableRouteKey}; use common_meta::key::TableMetadataManagerRef;
use common_meta::rpc::store::PutRequest; use common_meta::rpc::router::{Table, TableRoute};
use snafu::{OptionExt, ResultExt}; use snafu::{OptionExt, ResultExt};
use table::metadata::TableId; use table::metadata::TableId;
use crate::error::{ use crate::error::{self, Result, TableMetadataManagerSnafu};
DecodeTableRouteSnafu, Result, TableMetadataManagerSnafu, TableRouteNotFoundSnafu,
};
use crate::metasrv::Context; use crate::metasrv::Context;
use crate::service::store::kv::KvStoreRef;
pub(crate) async fn get_table_route_value(
kv_store: &KvStoreRef,
key: &TableRouteKey<'_>,
) -> Result<TableRouteValue> {
let kv = kv_store
.get(key.to_string().as_bytes())
.await?
.with_context(|| TableRouteNotFoundSnafu {
key: key.to_string(),
})?;
kv.value().try_into().context(DecodeTableRouteSnafu)
}
pub(crate) async fn put_table_route_value(
kv_store: &KvStoreRef,
key: &TableRouteKey<'_>,
value: TableRouteValue,
) -> Result<()> {
let req = PutRequest {
key: key.to_string().into_bytes(),
value: value.into(),
prev_kv: false,
};
let _ = kv_store.put(req).await?;
Ok(())
}
pub(crate) async fn fetch_table( pub(crate) async fn fetch_table(
kv_store: &KvStoreRef,
table_metadata_manager: &TableMetadataManagerRef, table_metadata_manager: &TableMetadataManagerRef,
table_id: TableId, table_id: TableId,
) -> Result<Option<(TableInfoValue, TableRouteValue)>> { ) -> Result<Option<(TableInfoValue, TableRouteValue)>> {
let Some(table_info_value) = table_metadata_manager let (table_info, table_route) = table_metadata_manager
.table_info_manager() .get_full_table_info(table_id)
.get(table_id)
.await .await
.context(TableMetadataManagerSnafu)? .context(TableMetadataManagerSnafu)?;
else {
return Ok(None);
};
let table_info = &table_info_value.table_info; if let Some(table_info) = table_info {
let trk = TableRouteKey { let table_route = table_route.with_context(|| error::TableRouteNotFoundSnafu {
table_id, table_name: table_info.table_ref().to_string(),
catalog_name: &table_info.catalog_name, })?;
schema_name: &table_info.schema_name,
table_name: &table_info.name,
};
let table_route_value = get_table_route_value(kv_store, &trk).await?;
Ok(Some((table_info_value, table_route_value))) let table = Table {
id: table_id as u64,
table_name: table_info.table_name(),
table_schema: vec![],
};
let table_route = TableRoute::new(table, table_route.region_routes);
let table_route_value = table_route
.try_into()
.context(error::TableRouteConversionSnafu)?;
Ok(Some((table_info, table_route_value)))
} else {
Ok(None)
}
} }
pub(crate) async fn fetch_tables( pub(crate) async fn fetch_tables(
ctx: &Context, ctx: &Context,
table_ids: Vec<TableId>, table_ids: Vec<TableId>,
) -> Result<Vec<(TableInfoValue, TableRouteValue)>> { ) -> Result<Vec<(TableInfoValue, TableRouteValue)>> {
let kv_store = &ctx.kv_store;
let table_metadata_manager = &ctx.table_metadata_manager; let table_metadata_manager = &ctx.table_metadata_manager;
let mut tables = vec![]; let mut tables = vec![];
// Maybe we can optimize the for loop in the future, but in general, // Maybe we can optimize the for loop in the future, but in general,
// there won't be many keys, in fact, there is usually just one. // there won't be many keys, in fact, there is usually just one.
for table_id in table_ids { for table_id in table_ids {
if let Some(x) = fetch_table(kv_store, table_metadata_manager, table_id).await? { if let Some(x) = fetch_table(table_metadata_manager, table_id).await? {
tables.push(x); tables.push(x);
} }
} }
@@ -99,24 +72,18 @@ pub(crate) async fn fetch_tables(
#[cfg(test)] #[cfg(test)]
pub(crate) mod tests { pub(crate) mod tests {
use std::collections::HashMap; use std::collections::{BTreeMap, HashMap};
use std::sync::Arc;
use api::v1::meta::{Peer, Region, RegionRoute, Table, TableRoute};
use chrono::DateTime; use chrono::DateTime;
use common_catalog::consts::{DEFAULT_CATALOG_NAME, DEFAULT_SCHEMA_NAME, MITO_ENGINE}; use common_catalog::consts::{DEFAULT_CATALOG_NAME, DEFAULT_SCHEMA_NAME, MITO_ENGINE};
use common_meta::key::table_region::RegionDistribution;
use common_meta::key::TableMetadataManagerRef; use common_meta::key::TableMetadataManagerRef;
use common_meta::table_name::TableName; use common_meta::peer::Peer;
use common_meta::rpc::router::{Region, RegionRoute};
use datatypes::data_type::ConcreteDataType; use datatypes::data_type::ConcreteDataType;
use datatypes::schema::{ColumnSchema, RawSchema}; use datatypes::schema::{ColumnSchema, RawSchema};
use table::metadata::{RawTableInfo, RawTableMeta, TableIdent, TableType}; use table::metadata::{RawTableInfo, RawTableMeta, TableIdent, TableType};
use table::requests::TableOptions; use table::requests::TableOptions;
use super::*;
use crate::error;
use crate::service::store::memory::MemStore;
pub(crate) async fn prepare_table_region_and_info_value( pub(crate) async fn prepare_table_region_and_info_value(
table_metadata_manager: &TableMetadataManagerRef, table_metadata_manager: &TableMetadataManagerRef,
table: &str, table: &str,
@@ -145,82 +112,34 @@ pub(crate) mod tests {
}, },
table_type: TableType::Base, table_type: TableType::Base,
}; };
table_metadata_manager
.table_info_manager() let region_route_factory = |region_id: u64, peer: u64| RegionRoute {
.create(1, &table_info) region: Region {
.await id: region_id.into(),
.unwrap(); ..Default::default()
},
leader_peer: Some(Peer {
id: peer,
addr: String::new(),
}),
follower_peers: vec![],
};
// Region distribution: // Region distribution:
// Datanode => Regions // Datanode => Regions
// 1 => 1, 2 // 1 => 1, 2
// 2 => 3 // 2 => 3
// 3 => 4 // 3 => 4
let region_distribution =
RegionDistribution::from([(1, vec![1, 2]), (2, vec![3]), (3, vec![4])]);
table_metadata_manager
.table_region_manager()
.create(1, &region_distribution)
.await
.unwrap();
for (datanode_id, regions) in region_distribution {
table_metadata_manager
.datanode_table_manager()
.create(datanode_id, 1, regions)
.await
.unwrap();
}
}
pub(crate) async fn prepare_table_route_value<'a>(
kv_store: &'a KvStoreRef,
table: &'a str,
) -> (TableRouteKey<'a>, TableRouteValue) {
let key = TableRouteKey {
table_id: 1,
catalog_name: DEFAULT_CATALOG_NAME,
schema_name: DEFAULT_SCHEMA_NAME,
table_name: table,
};
let peers = (1..=3)
.map(|id| Peer {
id,
addr: "".to_string(),
})
.collect::<Vec<_>>();
// region routes:
// region number => leader node
// 1 => 1
// 2 => 1
// 3 => 2
// 4 => 3
let region_routes = vec![ let region_routes = vec![
new_region_route(1, &peers, 1), region_route_factory(1, 1),
new_region_route(2, &peers, 1), region_route_factory(2, 1),
new_region_route(3, &peers, 2), region_route_factory(3, 2),
new_region_route(4, &peers, 3), region_route_factory(4, 3),
]; ];
let table_route = TableRoute { table_metadata_manager
table: Some(Table { .create_table_metadata(table_info, region_routes)
id: 1,
table_name: Some(
TableName::new(DEFAULT_CATALOG_NAME, DEFAULT_SCHEMA_NAME, table).into(),
),
table_schema: vec![],
}),
region_routes,
};
let value = TableRouteValue {
peers,
table_route: Some(table_route),
};
put_table_route_value(kv_store, &key, value.clone())
.await .await
.unwrap(); .unwrap();
(key, value)
} }
pub(crate) fn new_region_route( pub(crate) fn new_region_route(
@@ -229,46 +148,18 @@ pub(crate) mod tests {
leader_node: u64, leader_node: u64,
) -> RegionRoute { ) -> RegionRoute {
let region = Region { let region = Region {
id: region_number, id: region_number.into(),
name: "".to_string(), name: "".to_string(),
partition: None, partition: None,
attrs: HashMap::new(), attrs: BTreeMap::new(),
}; };
let leader_peer_index = peers
.iter() let leader_peer = peers.iter().find(|peer| peer.id == leader_node).cloned();
.enumerate()
.find_map(|(i, peer)| {
if peer.id == leader_node {
Some(i as u64)
} else {
None
}
})
.unwrap();
RegionRoute { RegionRoute {
region: Some(region), region,
leader_peer_index, leader_peer,
follower_peer_indexes: vec![], follower_peers: vec![],
} }
} }
#[tokio::test]
async fn test_put_and_get_table_route_value() {
let kv_store = Arc::new(MemStore::new()) as _;
let key = TableRouteKey {
table_id: 1,
catalog_name: "not_exist_catalog",
schema_name: "not_exist_schema",
table_name: "not_exist_table",
};
assert!(matches!(
get_table_route_value(&kv_store, &key).await.unwrap_err(),
error::Error::TableRouteNotFound { .. }
));
let (key, value) = prepare_table_route_value(&kv_store, "my_table").await;
let actual = get_table_route_value(&kv_store, &key).await.unwrap();
assert_eq!(actual, value);
}
} }

View File

@@ -26,6 +26,7 @@ mod test {
DropTableExpr, FlushTableExpr, InsertRequest, InsertRequests, QueryRequest, SemanticType, DropTableExpr, FlushTableExpr, InsertRequest, InsertRequests, QueryRequest, SemanticType,
}; };
use common_catalog::consts::MITO_ENGINE; use common_catalog::consts::MITO_ENGINE;
use common_meta::rpc::router::region_distribution;
use common_query::Output; use common_query::Output;
use common_recordbatch::RecordBatches; use common_recordbatch::RecordBatches;
use frontend::instance::Instance; use frontend::instance::Instance;
@@ -334,16 +335,16 @@ CREATE TABLE {table_name} (
let table = table.as_any().downcast_ref::<DistTable>().unwrap(); let table = table.as_any().downcast_ref::<DistTable>().unwrap();
let table_id = table.table_info().table_id(); let table_id = table.table_info().table_id();
let table_region_value = instance let table_route_value = instance
.table_metadata_manager() .table_metadata_manager()
.table_region_manager() .table_route_manager()
.get(table_id) .get(table_id)
.await .await
.unwrap() .unwrap()
.unwrap(); .unwrap();
let region_to_dn_map = table_region_value let region_to_dn_map = region_distribution(&table_route_value.region_routes)
.region_distribution .unwrap()
.iter() .iter()
.map(|(k, v)| (v[0], *k)) .map(|(k, v)| (v[0], *k))
.collect::<HashMap<u32, u64>>(); .collect::<HashMap<u32, u64>>();
@@ -628,16 +629,16 @@ CREATE TABLE {table_name} (
.unwrap(); .unwrap();
let table = table.as_any().downcast_ref::<DistTable>().unwrap(); let table = table.as_any().downcast_ref::<DistTable>().unwrap();
let table_id = table.table_info().ident.table_id; let table_id = table.table_info().ident.table_id;
let table_region_value = instance let table_route_value = instance
.table_metadata_manager() .table_metadata_manager()
.table_region_manager() .table_route_manager()
.get(table_id) .get(table_id)
.await .await
.unwrap() .unwrap()
.unwrap(); .unwrap();
let region_to_dn_map = table_region_value let region_to_dn_map = region_distribution(&table_route_value.region_routes)
.region_distribution .unwrap()
.iter() .iter()
.map(|(k, v)| (v[0], *k)) .map(|(k, v)| (v[0], *k))
.collect::<HashMap<u32, u64>>(); .collect::<HashMap<u32, u64>>();

View File

@@ -22,8 +22,10 @@ mod tests {
use common_base::Plugins; use common_base::Plugins;
use common_catalog::consts::{DEFAULT_CATALOG_NAME, DEFAULT_SCHEMA_NAME}; use common_catalog::consts::{DEFAULT_CATALOG_NAME, DEFAULT_SCHEMA_NAME};
use common_meta::key::table_name::TableNameKey; use common_meta::key::table_name::TableNameKey;
use common_meta::rpc::router::region_distribution;
use common_query::Output; use common_query::Output;
use common_recordbatch::RecordBatches; use common_recordbatch::RecordBatches;
use common_telemetry::debug;
use frontend::error::{self, Error, Result}; use frontend::error::{self, Error, Result};
use frontend::instance::Instance; use frontend::instance::Instance;
use query::parser::QueryLanguageParser; use query::parser::QueryLanguageParser;
@@ -195,15 +197,17 @@ mod tests {
.unwrap() .unwrap()
.unwrap() .unwrap()
.table_id(); .table_id();
debug!("Reading table {table_id}");
let table_region_value = manager let table_route_value = manager
.table_region_manager() .table_route_manager()
.get(table_id) .get(table_id)
.await .await
.unwrap() .unwrap()
.unwrap(); .unwrap();
let region_to_dn_map = table_region_value
.region_distribution let region_to_dn_map = region_distribution(&table_route_value.region_routes)
.unwrap()
.iter() .iter()
.map(|(k, v)| (v[0], *k)) .map(|(k, v)| (v[0], *k))
.collect::<HashMap<u32, u64>>(); .collect::<HashMap<u32, u64>>();

View File

@@ -20,8 +20,7 @@ use catalog::remote::CachedMetaKvBackend;
use common_catalog::consts::{DEFAULT_CATALOG_NAME, DEFAULT_SCHEMA_NAME, MITO_ENGINE}; use common_catalog::consts::{DEFAULT_CATALOG_NAME, DEFAULT_SCHEMA_NAME, MITO_ENGINE};
use common_meta::ident::TableIdent; use common_meta::ident::TableIdent;
use common_meta::key::table_name::{TableNameKey, TableNameValue}; use common_meta::key::table_name::{TableNameKey, TableNameValue};
use common_meta::key::table_region::RegionDistribution; use common_meta::key::{RegionDistribution, TableMetaKey};
use common_meta::key::TableMetaKey;
use common_meta::rpc::router::TableRoute; use common_meta::rpc::router::TableRoute;
use common_meta::rpc::KeyValue; use common_meta::rpc::KeyValue;
use common_meta::RegionIdent; use common_meta::RegionIdent;
@@ -31,6 +30,7 @@ use common_telemetry::info;
use frontend::catalog::FrontendCatalogManager; use frontend::catalog::FrontendCatalogManager;
use frontend::error::Result as FrontendResult; use frontend::error::Result as FrontendResult;
use frontend::instance::Instance; use frontend::instance::Instance;
use futures::TryStreamExt;
use meta_srv::error::Result as MetaResult; use meta_srv::error::Result as MetaResult;
use meta_srv::metasrv::{SelectorContext, SelectorRef}; use meta_srv::metasrv::{SelectorContext, SelectorRef};
use meta_srv::procedure::region_failover::{RegionFailoverContext, RegionFailoverProcedure}; use meta_srv::procedure::region_failover::{RegionFailoverContext, RegionFailoverProcedure};
@@ -266,18 +266,18 @@ async fn find_region_distribution(
) -> RegionDistribution { ) -> RegionDistribution {
let manager = cluster.meta_srv.table_metadata_manager(); let manager = cluster.meta_srv.table_metadata_manager();
let region_distribution = manager let region_distribution = manager
.table_region_manager() .table_route_manager()
.get(table_id) .get_region_distribution(table_id)
.await .await
.unwrap() .unwrap()
.unwrap() .unwrap();
.region_distribution;
// test DatanodeTableValues match the table region distribution // test DatanodeTableValues match the table region distribution
for datanode_id in cluster.datanode_instances.keys() { for datanode_id in cluster.datanode_instances.keys() {
let mut actual = manager let mut actual = manager
.datanode_table_manager() .datanode_table_manager()
.tables(*datanode_id) .tables(*datanode_id)
.try_collect::<Vec<_>>()
.await .await
.unwrap() .unwrap()
.into_iter() .into_iter()