Skip to content

Commit

Permalink
refactor: drop table procedure (#3688)
Browse files Browse the repository at this point in the history
* refactor: refactor drop table procedure

* refactor: refactor test utils
  • Loading branch information
WenyXu authored Apr 10, 2024
1 parent c00c1d9 commit c1e005b
Show file tree
Hide file tree
Showing 14 changed files with 473 additions and 333 deletions.
74 changes: 37 additions & 37 deletions src/common/meta/src/ddl/drop_table.rs
Original file line number Diff line number Diff line change
Expand Up @@ -12,7 +12,8 @@
// See the License for the specific language governing permissions and
// limitations under the License.

pub mod executor;
pub(crate) mod executor;
mod metadata;

use async_trait::async_trait;
use common_procedure::error::{FromJsonSnafu, ToJsonSnafu};
Expand All @@ -23,7 +24,7 @@ use common_telemetry::info;
use serde::{Deserialize, Serialize};
use snafu::{OptionExt, ResultExt};
use strum::AsRefStr;
use table::metadata::{RawTableInfo, TableId};
use table::metadata::TableId;
use table::table_reference::TableReference;

use self::executor::DropTableExecutor;
Expand Down Expand Up @@ -51,16 +52,10 @@ pub struct DropTableProcedure {
impl DropTableProcedure {
pub const TYPE_NAME: &'static str = "metasrv-procedure::DropTable";

pub fn new(
cluster_id: u64,
task: DropTableTask,
table_route_value: DeserializedValueWithBytes<TableRouteValue>,
table_info_value: DeserializedValueWithBytes<TableInfoValue>,
context: DdlContext,
) -> Self {
pub fn new(cluster_id: u64, task: DropTableTask, context: DdlContext) -> Self {
Self {
context,
data: DropTableData::new(cluster_id, task, table_route_value, table_info_value),
data: DropTableData::new(cluster_id, task),
dropping_regions: vec![],
}
}
Expand All @@ -74,18 +69,20 @@ impl DropTableProcedure {
})
}

async fn on_prepare<'a>(&mut self, executor: &DropTableExecutor) -> Result<Status> {
pub(crate) async fn on_prepare<'a>(&mut self, executor: &DropTableExecutor) -> Result<Status> {
if executor.on_prepare(&self.context).await?.stop() {
return Ok(Status::done());
}
self.fill_table_metadata().await?;
self.data.state = DropTableState::RemoveMetadata;

Ok(Status::executing(true))
}

/// Register dropping regions if doesn't exist.
fn register_dropping_regions(&mut self) -> Result<()> {
let region_routes = self.data.region_routes()?;
// Safety: filled in `on_prepare`.
let region_routes = self.data.region_routes().unwrap()?;

let dropping_regions = operating_leader_regions(region_routes);

Expand Down Expand Up @@ -121,7 +118,11 @@ impl DropTableProcedure {
// TODO(weny): Considers introducing a RegionStatus to indicate the region is dropping.
let table_id = self.data.table_id();
executor
.on_remove_metadata(&self.context, self.data.region_routes()?)
.on_remove_metadata(
&self.context,
// Safety: filled in `on_prepare`.
self.data.region_routes().unwrap()?,
)
.await?;
info!("Deleted table metadata for table {table_id}");
self.data.state = DropTableState::InvalidateTableCache;
Expand All @@ -138,10 +139,22 @@ impl DropTableProcedure {

pub async fn on_datanode_drop_regions(&self, executor: &DropTableExecutor) -> Result<Status> {
executor
.on_drop_regions(&self.context, self.data.region_routes()?)
.on_drop_regions(
&self.context,
// Safety: filled in `on_prepare`.
self.data.region_routes().unwrap()?,
)
.await?;
Ok(Status::done())
}

pub(crate) fn executor(&self) -> DropTableExecutor {
DropTableExecutor::new(
self.data.task.table_name(),
self.data.table_id(),
self.data.task.drop_if_exists,
)
}
}

#[async_trait]
Expand All @@ -151,11 +164,7 @@ impl Procedure for DropTableProcedure {
}

async fn execute(&mut self, _ctx: &ProcedureContext) -> ProcedureResult<Status> {
let executor = DropTableExecutor::new(
self.data.task.table_name(),
self.data.table_id(),
self.data.task.drop_if_exists,
);
let executor = self.executor();
let state = &self.data.state;
let _timer = metrics::METRIC_META_PROCEDURE_DROP_TABLE
.with_label_values(&[state.as_ref()])
Expand Down Expand Up @@ -188,48 +197,39 @@ impl Procedure for DropTableProcedure {
}

#[derive(Debug, Serialize, Deserialize)]
/// TODO(weny): simplify the table data.
pub struct DropTableData {
pub state: DropTableState,
pub cluster_id: u64,
pub task: DropTableTask,
pub table_route_value: DeserializedValueWithBytes<TableRouteValue>,
pub table_info_value: DeserializedValueWithBytes<TableInfoValue>,
pub table_route_value: Option<DeserializedValueWithBytes<TableRouteValue>>,
pub table_info_value: Option<DeserializedValueWithBytes<TableInfoValue>>,
}

impl DropTableData {
pub fn new(
cluster_id: u64,
task: DropTableTask,
table_route_value: DeserializedValueWithBytes<TableRouteValue>,
table_info_value: DeserializedValueWithBytes<TableInfoValue>,
) -> Self {
pub fn new(cluster_id: u64, task: DropTableTask) -> Self {
Self {
state: DropTableState::Prepare,
cluster_id,
task,
table_info_value,
table_route_value,
table_route_value: None,
table_info_value: None,
}
}

fn table_ref(&self) -> TableReference {
self.task.table_ref()
}

fn region_routes(&self) -> Result<&Vec<RegionRoute>> {
self.table_route_value.region_routes()
}

fn table_info(&self) -> &RawTableInfo {
&self.table_info_value.table_info
fn region_routes(&self) -> Option<Result<&Vec<RegionRoute>>> {
self.table_route_value.as_ref().map(|v| v.region_routes())
}

fn table_id(&self) -> TableId {
self.table_info().ident.table_id
self.task.table_id
}
}

/// The state of drop table.
#[derive(Debug, Serialize, Deserialize, AsRefStr)]
pub enum DropTableState {
/// Prepares to drop the table
Expand Down
46 changes: 46 additions & 0 deletions src/common/meta/src/ddl/drop_table/metadata.rs
Original file line number Diff line number Diff line change
@@ -0,0 +1,46 @@
// 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_catalog::format_full_table_name;
use snafu::OptionExt;

use crate::ddl::drop_table::DropTableProcedure;
use crate::error::{self, Result};

impl DropTableProcedure {
/// Fetches the table info and table route.
pub(crate) async fn fill_table_metadata(&mut self) -> Result<()> {
let task = &self.data.task;
let table_info_value = self
.context
.table_metadata_manager
.table_info_manager()
.get(task.table_id)
.await?
.with_context(|| error::TableInfoNotFoundSnafu {
table: format_full_table_name(&task.catalog, &task.schema, &task.table),
})?;
let (_, table_route_value) = self
.context
.table_metadata_manager
.table_route_manager()
.table_route_storage()
.get_raw_physical_table_route(task.table_id)
.await?;

self.data.table_info_value = Some(table_info_value);
self.data.table_route_value = Some(table_route_value);
Ok(())
}
}
1 change: 1 addition & 0 deletions src/common/meta/src/ddl/test_util.rs
Original file line number Diff line number Diff line change
Expand Up @@ -15,6 +15,7 @@
pub mod alter_table;
pub mod columns;
pub mod create_table;
pub mod datanode_handler;

use std::collections::HashMap;

Expand Down
50 changes: 49 additions & 1 deletion src/common/meta/src/ddl/test_util/create_table.rs
Original file line number Diff line number Diff line change
Expand Up @@ -15,7 +15,8 @@
use std::collections::HashMap;

use api::v1::column_def::try_as_column_schema;
use api::v1::{ColumnDef, CreateTableExpr, SemanticType};
use api::v1::meta::Partition;
use api::v1::{ColumnDataType, ColumnDef, CreateTableExpr, SemanticType};
use chrono::DateTime;
use common_catalog::consts::{DEFAULT_CATALOG_NAME, DEFAULT_SCHEMA_NAME, MITO2_ENGINE};
use datatypes::schema::RawSchema;
Expand All @@ -24,6 +25,9 @@ use store_api::storage::TableId;
use table::metadata::{RawTableInfo, RawTableMeta, TableIdent, TableType};
use table::requests::TableOptions;

use crate::ddl::test_util::columns::TestColumnDefBuilder;
use crate::rpc::ddl::CreateTableTask;

#[derive(Default, Builder)]
#[builder(default)]
pub struct TestCreateTableExpr {
Expand Down Expand Up @@ -130,3 +134,47 @@ pub fn build_raw_table_info_from_expr(expr: &CreateTableExpr) -> RawTableInfo {
table_type: TableType::Base,
}
}

pub fn test_create_table_task(name: &str, table_id: TableId) -> CreateTableTask {
let create_table = TestCreateTableExprBuilder::default()
.column_defs([
TestColumnDefBuilder::default()
.name("ts")
.data_type(ColumnDataType::TimestampMillisecond)
.semantic_type(SemanticType::Timestamp)
.build()
.unwrap()
.into(),
TestColumnDefBuilder::default()
.name("host")
.data_type(ColumnDataType::String)
.semantic_type(SemanticType::Tag)
.build()
.unwrap()
.into(),
TestColumnDefBuilder::default()
.name("cpu")
.data_type(ColumnDataType::Float64)
.semantic_type(SemanticType::Field)
.build()
.unwrap()
.into(),
])
.table_id(table_id)
.time_index("ts")
.primary_keys(["host".into()])
.table_name(name)
.build()
.unwrap()
.into();
let table_info = build_raw_table_info_from_expr(&create_table);
CreateTableTask {
create_table,
// Single region
partitions: vec![Partition {
column_list: vec![],
value_list: vec![],
}],
table_info,
}
}
Loading

0 comments on commit c1e005b

Please sign in to comment.