feat: support querying metric engine from frontend (#2987)

* query one logical table

Signed-off-by: Ruihang Xia <waynestxia@gmail.com>

* map column id

Signed-off-by: Ruihang Xia <waynestxia@gmail.com>

* remove deadcode

Signed-off-by: Ruihang Xia <waynestxia@gmail.com>

* fix typo

Signed-off-by: Ruihang Xia <waynestxia@gmail.com>

* remove redundent column name

Signed-off-by: Ruihang Xia <waynestxia@gmail.com>

---------

Signed-off-by: Ruihang Xia <waynestxia@gmail.com>
This commit is contained in:
Ruihang Xia
2023-12-25 18:47:57 +08:00
committed by GitHub
parent 0d42651047
commit 48cd22d459
11 changed files with 247 additions and 123 deletions

View File

@@ -152,7 +152,10 @@ impl RegionEngine for MetricEngine {
/// Retrieves region's metadata. /// Retrieves region's metadata.
async fn get_metadata(&self, region_id: RegionId) -> Result<RegionMetadataRef, BoxedError> { async fn get_metadata(&self, region_id: RegionId) -> Result<RegionMetadataRef, BoxedError> {
todo!() self.inner
.load_region_metadata(region_id)
.await
.map_err(BoxedError::new)
} }
/// Retrieves region's disk usage. /// Retrieves region's disk usage.
@@ -261,7 +264,7 @@ mod test {
.await .await
.unwrap_err(); .unwrap_err();
// open nonexistent region // open nonexistent region won't report error
let invalid_open_request = RegionOpenRequest { let invalid_open_request = RegionOpenRequest {
engine: METRIC_ENGINE_NAME.to_string(), engine: METRIC_ENGINE_NAME.to_string(),
region_dir: env.default_region_dir(), region_dir: env.default_region_dir(),
@@ -274,6 +277,6 @@ mod test {
RegionRequest::Open(invalid_open_request), RegionRequest::Open(invalid_open_request),
) )
.await .await
.unwrap_err(); .unwrap();
} }
} }

View File

@@ -93,12 +93,7 @@ impl MetricEngineInner {
// register columns to logical region // register columns to logical region
for col in columns { for col in columns {
self.metadata_region self.metadata_region
.add_column( .add_column(metadata_region_id, region_id, &col.column_metadata)
metadata_region_id,
region_id,
&col.column_metadata.column_schema.name,
col.column_metadata.semantic_type,
)
.await?; .await?;
} }

View File

@@ -185,12 +185,7 @@ impl MetricEngineInner {
.await?; .await?;
for col in &request.column_metadatas { for col in &request.column_metadatas {
self.metadata_region self.metadata_region
.add_column( .add_column(metadata_region_id, logical_region_id, col)
metadata_region_id,
logical_region_id,
&col.column_schema.name,
col.semantic_type,
)
.await?; .await?;
} }
@@ -221,12 +216,7 @@ impl MetricEngineInner {
// register columns to metadata region // register columns to metadata region
for col in &new_columns { for col in &new_columns {
self.metadata_region self.metadata_region
.add_column( .add_column(metadata_region_id, logical_region_id, col)
metadata_region_id,
logical_region_id,
&col.column_schema.name,
col.semantic_type,
)
.await?; .await?;
} }

View File

@@ -52,19 +52,13 @@ impl MetricEngineInner {
self.open_physical_region(region_id, request).await?; self.open_physical_region(region_id, request).await?;
self.recover_states(region_id).await?; self.recover_states(region_id).await?;
Ok(0)
} else if self
.state
.read()
.await
.logical_regions()
.contains_key(&region_id)
{
// if the logical region is already open, do nothing
Ok(0) Ok(0)
} else { } else {
// throw RegionNotFound error // Don't check if the logical region exist. Because a logical region cannot be opened
Err(LogicalRegionNotFoundSnafu { region_id }.build()) // individually, it is always "open" if its physical region is open. But the engine
// can't tell if the logical region is not exist or the physical region is not opened
// yet. Thus simply return `Ok` here to ignore all those errors.
Ok(0)
} }
} }

View File

@@ -288,7 +288,7 @@ mod tests {
// add columns // add columns
let logical_region_id = env.default_logical_region_id(); let logical_region_id = env.default_logical_region_id();
let columns = &["odd", "even", "Ev_En"]; let columns = &["odd", "even", "Ev_En"];
let alter_request = test_util::alter_logical_region_add_tag_columns(columns); let alter_request = test_util::alter_logical_region_add_tag_columns(123456, columns);
engine engine
.handle_request(logical_region_id, RegionRequest::Alter(alter_request)) .handle_request(logical_region_id, RegionRequest::Alter(alter_request))
.await .await

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 api::v1::SemanticType;
use common_query::logical_plan::Expr; use common_query::logical_plan::Expr;
use common_recordbatch::SendableRecordBatchStream; use common_recordbatch::SendableRecordBatchStream;
use common_telemetry::{error, info, tracing}; use common_telemetry::{error, info, tracing};
use datafusion::logical_expr; use datafusion::logical_expr;
use snafu::{OptionExt, ResultExt}; use snafu::{OptionExt, ResultExt};
use store_api::metadata::{RegionMetadata, RegionMetadataBuilder, RegionMetadataRef};
use store_api::metric_engine_consts::DATA_SCHEMA_TABLE_ID_COLUMN_NAME; use store_api::metric_engine_consts::DATA_SCHEMA_TABLE_ID_COLUMN_NAME;
use store_api::region_engine::RegionEngine; use store_api::region_engine::RegionEngine;
use store_api::storage::consts::ReservedColumnId; use store_api::storage::consts::ReservedColumnId;
use store_api::storage::{RegionId, ScanRequest}; use store_api::storage::{RegionId, ScanRequest};
use crate::engine::MetricEngineInner; use crate::engine::MetricEngineInner;
use crate::error::{LogicalRegionNotFoundSnafu, MitoReadOperationSnafu, Result}; use crate::error::{
InvalidMetadataSnafu, LogicalRegionNotFoundSnafu, MitoReadOperationSnafu, Result,
};
use crate::utils; use crate::utils;
impl MetricEngineInner { impl MetricEngineInner {
@@ -67,17 +73,7 @@ impl MetricEngineInner {
logical_region_id: RegionId, logical_region_id: RegionId,
request: ScanRequest, request: ScanRequest,
) -> Result<SendableRecordBatchStream> { ) -> Result<SendableRecordBatchStream> {
let physical_region_id = { let physical_region_id = self.get_physical_region_id(logical_region_id).await?;
let state = &self.state.read().await;
state
.get_physical_region_id(logical_region_id)
.with_context(|| {
error!("Trying to read an nonexistent region {logical_region_id}");
LogicalRegionNotFoundSnafu {
region_id: logical_region_id,
}
})?
};
let data_region_id = utils::to_data_region_id(physical_region_id); let data_region_id = utils::to_data_region_id(physical_region_id);
let request = self let request = self
.transform_request(physical_region_id, logical_region_id, request) .transform_request(physical_region_id, logical_region_id, request)
@@ -88,6 +84,38 @@ impl MetricEngineInner {
.context(MitoReadOperationSnafu) .context(MitoReadOperationSnafu)
} }
pub async fn load_region_metadata(&self, region_id: RegionId) -> Result<RegionMetadataRef> {
let is_reading_physical_region = self
.state
.read()
.await
.physical_regions()
.contains_key(&region_id);
if is_reading_physical_region {
self.mito
.get_metadata(region_id)
.await
.context(MitoReadOperationSnafu)
} else {
let physical_region_id = self.get_physical_region_id(region_id).await?;
self.logical_region_metadata(physical_region_id, region_id)
.await
}
}
async fn get_physical_region_id(&self, logical_region_id: RegionId) -> Result<RegionId> {
let state = &self.state.read().await;
state
.get_physical_region_id(logical_region_id)
.with_context(|| {
error!("Trying to read an nonexistent region {logical_region_id}");
LogicalRegionNotFoundSnafu {
region_id: logical_region_id,
}
})
}
/// Transform the [ScanRequest] from logical region to physical data region. /// Transform the [ScanRequest] from logical region to physical data region.
async fn transform_request( async fn transform_request(
&self, &self,
@@ -130,9 +158,13 @@ impl MetricEngineInner {
origin_projection: &[usize], origin_projection: &[usize],
) -> Result<Vec<usize>> { ) -> Result<Vec<usize>> {
// project on logical columns // project on logical columns
let logical_columns = self let all_logical_columns = self
.load_logical_columns(physical_region_id, logical_region_id) .load_logical_columns(physical_region_id, logical_region_id)
.await?; .await?;
let projected_logical_names = origin_projection
.iter()
.map(|i| all_logical_columns[*i].column_schema.name.clone())
.collect::<Vec<_>>();
// generate physical projection // generate physical projection
let mut physical_projection = Vec::with_capacity(origin_projection.len()); let mut physical_projection = Vec::with_capacity(origin_projection.len());
@@ -142,10 +174,9 @@ impl MetricEngineInner {
.get_metadata(data_region_id) .get_metadata(data_region_id)
.await .await
.context(MitoReadOperationSnafu)?; .context(MitoReadOperationSnafu)?;
for logical_proj in origin_projection { for name in projected_logical_names {
let column_id = logical_columns[*logical_proj].column_id;
// Safety: logical columns is a strict subset of physical columns // Safety: logical columns is a strict subset of physical columns
physical_projection.push(physical_metadata.column_index_by_id(column_id).unwrap()); physical_projection.push(physical_metadata.column_index_by_name(&name).unwrap());
} }
Ok(physical_projection) Ok(physical_projection)
@@ -159,7 +190,9 @@ impl MetricEngineInner {
) -> Result<Vec<usize>> { ) -> Result<Vec<usize>> {
let logical_columns = self let logical_columns = self
.load_logical_columns(physical_region_id, logical_region_id) .load_logical_columns(physical_region_id, logical_region_id)
.await?; .await?
.into_iter()
.map(|col| col.column_schema.name);
let mut projection = Vec::with_capacity(logical_columns.len()); let mut projection = Vec::with_capacity(logical_columns.len());
let data_region_id = utils::to_data_region_id(physical_region_id); let data_region_id = utils::to_data_region_id(physical_region_id);
let physical_metadata = self let physical_metadata = self
@@ -167,13 +200,45 @@ impl MetricEngineInner {
.get_metadata(data_region_id) .get_metadata(data_region_id)
.await .await
.context(MitoReadOperationSnafu)?; .context(MitoReadOperationSnafu)?;
for logical_col in logical_columns { for name in logical_columns {
let column_id = logical_col.column_id;
// Safety: logical columns is a strict subset of physical columns // Safety: logical columns is a strict subset of physical columns
projection.push(physical_metadata.column_index_by_id(column_id).unwrap()); projection.push(physical_metadata.column_index_by_name(&name).unwrap());
} }
Ok(projection) Ok(projection)
} }
pub async fn logical_region_metadata(
&self,
physical_region_id: RegionId,
logical_region_id: RegionId,
) -> Result<RegionMetadataRef> {
let logical_columns = self
.load_logical_columns(physical_region_id, logical_region_id)
.await?;
let primary_keys = logical_columns
.iter()
.filter_map(|col| {
if col.semantic_type == SemanticType::Tag {
Some(col.column_id)
} else {
None
}
})
.collect::<Vec<_>>();
let mut logical_metadata_builder = RegionMetadataBuilder::new(logical_region_id);
for col in logical_columns {
logical_metadata_builder.push_column_metadata(col);
}
logical_metadata_builder.primary_key(primary_keys);
let logical_metadata = logical_metadata_builder
.build()
.context(InvalidMetadataSnafu)?;
Ok(Arc::new(logical_metadata))
}
} }
#[cfg(test)] #[cfg(test)]
@@ -205,7 +270,8 @@ mod test {
.unwrap(); .unwrap();
// add columns to the first logical region // add columns to the first logical region
let alter_request = alter_logical_region_add_tag_columns(&["987", "789", "654", "321"]); let alter_request =
alter_logical_region_add_tag_columns(123456, &["987", "798", "654", "321"]);
env.metric() env.metric()
.handle_request(logical_region_id, RegionRequest::Alter(alter_request)) .handle_request(logical_region_id, RegionRequest::Alter(alter_request))
.await .await
@@ -225,7 +291,7 @@ mod test {
.await .await
.unwrap(); .unwrap();
assert_eq!(scan_req.projection.unwrap(), vec![0, 1, 4, 7, 8, 9, 10]); assert_eq!(scan_req.projection.unwrap(), vec![0, 1, 4, 8, 9, 10, 11]);
assert_eq!(scan_req.filters.len(), 1); assert_eq!(scan_req.filters.len(), 1);
assert_eq!( assert_eq!(
scan_req.filters[0], scan_req.filters[0],
@@ -242,6 +308,6 @@ mod test {
.transform_request(physical_region_id, logical_region_id, scan_req) .transform_request(physical_region_id, logical_region_id, scan_req)
.await .await
.unwrap(); .unwrap();
assert_eq!(scan_req.projection.unwrap(), vec![0, 1, 4, 7, 8, 9, 10]); assert_eq!(scan_req.projection.unwrap(), vec![0, 1, 4, 8, 9, 10, 11]);
} }
} }

View File

@@ -27,34 +27,19 @@ use crate::error::Result;
impl MetricEngineInner { impl MetricEngineInner {
/// Load column metadata of a logical region. /// Load column metadata of a logical region.
/// ///
/// The return value is ordered. /// The return value is ordered on [ColumnId].
pub async fn load_logical_columns( pub async fn load_logical_columns(
&self, &self,
physical_region_id: RegionId, physical_region_id: RegionId,
logical_region_id: RegionId, logical_region_id: RegionId,
) -> Result<Vec<ColumnMetadata>> { ) -> Result<Vec<ColumnMetadata>> {
// load logical and physical columns, and intersect them to get logical column metadata // load logical and physical columns, and intersect them to get logical column metadata
let logical_columns = self let mut logical_column_metadata = self
.metadata_region .metadata_region
.logical_columns(physical_region_id, logical_region_id) .logical_columns(physical_region_id, logical_region_id)
.await? .await?
.into_iter() .into_iter()
.collect::<HashMap<String, SemanticType>>(); .map(|(_, column_metadata)| column_metadata)
let physical_columns = self
.data_region
.physical_columns(physical_region_id)
.await?;
let mut logical_column_metadata = physical_columns
.into_iter()
.filter_map(|mut col| {
// recover the semantic type of logical columns
logical_columns
.get(&col.column_schema.name)
.map(|semantic_type| {
col.semantic_type = *semantic_type;
col
})
})
.collect::<Vec<_>>(); .collect::<Vec<_>>();
// sort columns on column id to ensure the order // sort columns on column id to ensure the order

View File

@@ -63,6 +63,14 @@ pub enum Error {
location: Location, location: Location,
}, },
#[snafu(display("Failed to deserialize column metadata from {}", raw))]
DeserializeColumnMetadata {
raw: String,
#[snafu(source)]
error: serde_json::Error,
location: Location,
},
#[snafu(display("Failed to decode base64 column value"))] #[snafu(display("Failed to decode base64 column value"))]
DecodeColumnValue { DecodeColumnValue {
#[snafu(source)] #[snafu(source)]
@@ -132,6 +140,12 @@ pub enum Error {
#[snafu(display("Alter request to physical region is forbidden"))] #[snafu(display("Alter request to physical region is forbidden"))]
ForbiddenPhysicalAlter { location: Location }, ForbiddenPhysicalAlter { location: Location },
#[snafu(display("Invalid region metadata"))]
InvalidMetadata {
source: store_api::metadata::MetadataError,
location: Location,
},
} }
pub type Result<T, E = Error> = std::result::Result<T, E>; pub type Result<T, E = Error> = std::result::Result<T, E>;
@@ -150,8 +164,10 @@ impl ErrorExt for Error {
MissingInternalColumn { .. } MissingInternalColumn { .. }
| DeserializeSemanticType { .. } | DeserializeSemanticType { .. }
| DeserializeColumnMetadata { .. }
| DecodeColumnValue { .. } | DecodeColumnValue { .. }
| ParseRegionId { .. } => StatusCode::Unexpected, | ParseRegionId { .. }
| InvalidMetadata { .. } => StatusCode::Unexpected,
PhysicalRegionNotFound { .. } | LogicalRegionNotFound { .. } => { PhysicalRegionNotFound { .. } | LogicalRegionNotFound { .. } => {
StatusCode::RegionNotFound StatusCode::RegionNotFound

View File

@@ -22,6 +22,7 @@ use common_recordbatch::util::collect;
use datafusion::prelude::{col, lit}; use datafusion::prelude::{col, lit};
use mito2::engine::MitoEngine; use mito2::engine::MitoEngine;
use snafu::ResultExt; use snafu::ResultExt;
use store_api::metadata::ColumnMetadata;
use store_api::metric_engine_consts::{ use store_api::metric_engine_consts::{
METADATA_SCHEMA_KEY_COLUMN_INDEX, METADATA_SCHEMA_KEY_COLUMN_NAME, METADATA_SCHEMA_KEY_COLUMN_INDEX, METADATA_SCHEMA_KEY_COLUMN_NAME,
METADATA_SCHEMA_TIMESTAMP_COLUMN_NAME, METADATA_SCHEMA_VALUE_COLUMN_INDEX, METADATA_SCHEMA_TIMESTAMP_COLUMN_NAME, METADATA_SCHEMA_VALUE_COLUMN_INDEX,
@@ -32,7 +33,7 @@ use store_api::region_request::RegionPutRequest;
use store_api::storage::{RegionId, ScanRequest}; use store_api::storage::{RegionId, ScanRequest};
use crate::error::{ use crate::error::{
CollectRecordBatchStreamSnafu, DecodeColumnValueSnafu, DeserializeSemanticTypeSnafu, CollectRecordBatchStreamSnafu, DecodeColumnValueSnafu, DeserializeColumnMetadataSnafu,
MitoReadOperationSnafu, MitoWriteOperationSnafu, ParseRegionIdSnafu, RegionAlreadyExistsSnafu, MitoReadOperationSnafu, MitoWriteOperationSnafu, ParseRegionIdSnafu, RegionAlreadyExistsSnafu,
Result, Result,
}; };
@@ -96,16 +97,16 @@ impl MetadataRegion {
&self, &self,
physical_region_id: RegionId, physical_region_id: RegionId,
logical_region_id: RegionId, logical_region_id: RegionId,
column_name: &str, column_metadata: &ColumnMetadata,
semantic_type: SemanticType,
) -> Result<bool> { ) -> Result<bool> {
let region_id = utils::to_metadata_region_id(physical_region_id); let region_id = utils::to_metadata_region_id(physical_region_id);
let column_key = Self::concat_column_key(logical_region_id, column_name); let column_key =
Self::concat_column_key(logical_region_id, &column_metadata.column_schema.name);
self.put_if_absent( self.put_if_absent(
region_id, region_id,
column_key, column_key,
Self::serialize_semantic_type(semantic_type), Self::serialize_column_metadata(column_metadata),
) )
.await .await
} }
@@ -132,7 +133,7 @@ impl MetadataRegion {
let column_key = Self::concat_column_key(logical_region_id, column_name); let column_key = Self::concat_column_key(logical_region_id, column_name);
let semantic_type = self.get(region_id, &column_key).await?; let semantic_type = self.get(region_id, &column_key).await?;
semantic_type semantic_type
.map(|s| Self::deserialize_semantic_type(&s)) .map(|s| Self::deserialize_column_metadata(&s).map(|c| c.semantic_type))
.transpose() .transpose()
} }
@@ -143,7 +144,7 @@ impl MetadataRegion {
&self, &self,
physical_region_id: RegionId, physical_region_id: RegionId,
logical_region_id: RegionId, logical_region_id: RegionId,
) -> Result<Vec<(String, SemanticType)>> { ) -> Result<Vec<(String, ColumnMetadata)>> {
let metadata_region_id = utils::to_metadata_region_id(physical_region_id); let metadata_region_id = utils::to_metadata_region_id(physical_region_id);
let region_column_prefix = Self::concat_column_key_prefix(logical_region_id); let region_column_prefix = Self::concat_column_key_prefix(logical_region_id);
@@ -154,8 +155,8 @@ impl MetadataRegion {
} }
// Safety: we have checked the prefix // Safety: we have checked the prefix
let (_, column_name) = Self::parse_column_key(&k)?.unwrap(); let (_, column_name) = Self::parse_column_key(&k)?.unwrap();
let semantic_type = Self::deserialize_semantic_type(&v)?; let column_metadata = Self::deserialize_column_metadata(&v)?;
columns.push((column_name, semantic_type)); columns.push((column_name, column_metadata));
} }
Ok(columns) Ok(columns)
@@ -228,13 +229,14 @@ impl MetadataRegion {
} }
} }
pub fn serialize_semantic_type(semantic_type: SemanticType) -> String { pub fn serialize_column_metadata(column_metadata: &ColumnMetadata) -> String {
serde_json::to_string(&semantic_type).unwrap() serde_json::to_string(column_metadata).unwrap()
} }
pub fn deserialize_semantic_type(semantic_type: &str) -> Result<SemanticType> { pub fn deserialize_column_metadata(column_metadata: &str) -> Result<ColumnMetadata> {
serde_json::from_str(semantic_type) serde_json::from_str(column_metadata).with_context(|_| DeserializeColumnMetadataSnafu {
.with_context(|_| DeserializeSemanticTypeSnafu { raw: semantic_type }) raw: column_metadata,
})
} }
} }
@@ -411,6 +413,8 @@ impl MetadataRegion {
#[cfg(test)] #[cfg(test)]
mod test { mod test {
use datatypes::data_type::ConcreteDataType;
use datatypes::schema::ColumnSchema;
use store_api::region_request::RegionRequest; use store_api::region_request::RegionRequest;
use super::*; use super::*;
@@ -463,26 +467,21 @@ mod test {
} }
#[test] #[test]
fn test_serialize_semantic_type() { fn test_serialize_column_metadata() {
let semantic_type = SemanticType::Tag; let semantic_type = SemanticType::Tag;
let expected = "\"Tag\"".to_string(); let column_metadata = ColumnMetadata {
column_schema: ColumnSchema::new("blabla", ConcreteDataType::string_datatype(), false),
semantic_type,
column_id: 5,
};
let expected = "{\"column_schema\":{\"name\":\"blabla\",\"data_type\":{\"String\":null},\"is_nullable\":false,\"is_time_index\":false,\"default_constraint\":null,\"metadata\":{}},\"semantic_type\":\"Tag\",\"column_id\":5}".to_string();
assert_eq!( assert_eq!(
MetadataRegion::serialize_semantic_type(semantic_type), MetadataRegion::serialize_column_metadata(&column_metadata),
expected
);
}
#[test]
fn test_deserialize_semantic_type() {
let semantic_type = "\"Tag\"";
let expected = SemanticType::Tag;
assert_eq!(
MetadataRegion::deserialize_semantic_type(semantic_type).unwrap(),
expected expected
); );
let semantic_type = "\"InvalidType\""; let semantic_type = "\"Invalid Column Metadata\"";
assert!(MetadataRegion::deserialize_semantic_type(semantic_type).is_err()); assert!(MetadataRegion::deserialize_column_metadata(semantic_type).is_err());
} }
#[test] #[test]
@@ -620,13 +619,17 @@ mod test {
let logical_region_id = RegionId::new(868, 8390); let logical_region_id = RegionId::new(868, 8390);
let column_name = "column1"; let column_name = "column1";
let semantic_type = SemanticType::Tag; let semantic_type = SemanticType::Tag;
metadata_region let column_metadata = ColumnMetadata {
.add_column( column_schema: ColumnSchema::new(
physical_region_id,
logical_region_id,
column_name, column_name,
semantic_type, ConcreteDataType::string_datatype(),
) false,
),
semantic_type,
column_id: 5,
};
metadata_region
.add_column(physical_region_id, logical_region_id, &column_metadata)
.await .await
.unwrap(); .unwrap();
let actual_semantic_type = metadata_region let actual_semantic_type = metadata_region
@@ -637,12 +640,7 @@ mod test {
// duplicate column won't be updated // duplicate column won't be updated
let is_updated = metadata_region let is_updated = metadata_region
.add_column( .add_column(physical_region_id, logical_region_id, &column_metadata)
physical_region_id,
logical_region_id,
column_name,
SemanticType::Field,
)
.await .await
.unwrap(); .unwrap();
assert!(!is_updated); assert!(!is_updated);

View File

@@ -30,7 +30,7 @@ use store_api::region_engine::RegionEngine;
use store_api::region_request::{ use store_api::region_request::{
AddColumn, AlterKind, RegionAlterRequest, RegionCreateRequest, RegionRequest, AddColumn, AlterKind, RegionAlterRequest, RegionCreateRequest, RegionRequest,
}; };
use store_api::storage::RegionId; use store_api::storage::{ColumnId, RegionId};
use crate::data_region::DataRegion; use crate::data_region::DataRegion;
use crate::engine::MetricEngine; use crate::engine::MetricEngine;
@@ -156,12 +156,15 @@ impl TestEnv {
} }
/// Generate a [RegionAlterRequest] for adding tag columns. /// Generate a [RegionAlterRequest] for adding tag columns.
pub fn alter_logical_region_add_tag_columns(new_tags: &[&str]) -> RegionAlterRequest { pub fn alter_logical_region_add_tag_columns(
col_id_start: ColumnId,
new_tags: &[&str],
) -> RegionAlterRequest {
let mut new_columns = vec![]; let mut new_columns = vec![];
for (i, tag) in new_tags.iter().enumerate() { for (i, tag) in new_tags.iter().enumerate() {
new_columns.push(AddColumn { new_columns.push(AddColumn {
column_metadata: ColumnMetadata { column_metadata: ColumnMetadata {
column_id: i as u32, column_id: i as u32 + col_id_start,
semantic_type: SemanticType::Tag, semantic_type: SemanticType::Tag,
column_schema: ColumnSchema::new( column_schema: ColumnSchema::new(
tag.to_string(), tag.to_string(),
@@ -198,7 +201,7 @@ pub fn create_logical_region_request(
), ),
}, },
ColumnMetadata { ColumnMetadata {
column_id: 0, column_id: 1,
semantic_type: SemanticType::Field, semantic_type: SemanticType::Field,
column_schema: ColumnSchema::new( column_schema: ColumnSchema::new(
"greptime_value", "greptime_value",
@@ -209,7 +212,7 @@ pub fn create_logical_region_request(
]; ];
for tag in tags { for tag in tags {
column_metadatas.push(ColumnMetadata { column_metadatas.push(ColumnMetadata {
column_id: 0, column_id: 2,
semantic_type: SemanticType::Tag, semantic_type: SemanticType::Tag,
column_schema: ColumnSchema::new( column_schema: ColumnSchema::new(
tag.to_string(), tag.to_string(),
@@ -261,9 +264,10 @@ pub fn row_schema_with_tags(tags: &[&str]) -> Vec<PbColumnSchema> {
schema schema
} }
/// Build [Rows] for assembling [RegionPutRequest](store_api::region_request::RegionPutRequest). /// Build [Row]s for assembling [RegionPutRequest](store_api::region_request::RegionPutRequest).
/// ///
/// The schema is generated by [row_schema_with_tags]. /// The schema is generated by [row_schema_with_tags]. `num_tags` doesn't need to be precise,
/// it's used to determine the column id for new columns.
pub fn build_rows(num_tags: usize, num_rows: usize) -> Vec<Row> { pub fn build_rows(num_tags: usize, num_rows: usize) -> Vec<Row> {
let mut rows = vec![]; let mut rows = vec![];
for i in 0..num_rows { for i in 0..num_rows {

View File

@@ -217,6 +217,13 @@ impl RegionMetadata {
self.id_to_index.get(&column_id).copied() self.id_to_index.get(&column_id).copied()
} }
/// Find column index by name.
pub fn column_index_by_name(&self, column_name: &str) -> Option<usize> {
self.column_metadatas
.iter()
.position(|col| col.column_schema.name == column_name)
}
/// Returns the time index column /// Returns the time index column
/// ///
/// # Panics /// # Panics
@@ -261,6 +268,61 @@ impl RegionMetadata {
self.primary_key.iter().position(|id| *id == column_id) self.primary_key.iter().position(|id| *id == column_id)
} }
/// Project the metadata to a new one using specified column ids.
///
/// [RegionId] and schema version are preserved.
pub fn project(&self, projection: &[ColumnId]) -> Result<RegionMetadata> {
// check time index
ensure!(
projection.iter().any(|id| *id == self.time_index),
TimeIndexNotFoundSnafu
);
// prepare new indices
let indices_to_preserve = projection
.iter()
.map(|id| {
self.column_index_by_id(*id)
.with_context(|| InvalidRegionRequestSnafu {
region_id: self.region_id,
err: format!("column id {} not found", id),
})
})
.collect::<Result<Vec<_>>>()?;
// project schema
let projected_schema =
self.schema
.try_project(&indices_to_preserve)
.with_context(|_| SchemaProjectSnafu {
origin_schema: self.schema.clone(),
projection: projection.to_vec(),
})?;
// project columns, generate projected primary key and new id_to_index
let mut projected_column_metadatas = Vec::with_capacity(indices_to_preserve.len());
let mut projected_primary_key = vec![];
let mut projected_id_to_index = HashMap::with_capacity(indices_to_preserve.len());
for index in indices_to_preserve {
let col = self.column_metadatas[index].clone();
if col.semantic_type == SemanticType::Tag {
projected_primary_key.push(col.column_id);
}
projected_id_to_index.insert(col.column_id, projected_column_metadatas.len());
projected_column_metadatas.push(col);
}
Ok(RegionMetadata {
schema: Arc::new(projected_schema),
time_index: self.time_index,
id_to_index: projected_id_to_index,
column_metadatas: projected_column_metadatas,
primary_key: projected_primary_key,
region_id: self.region_id,
schema_version: self.schema_version,
})
}
/// Checks whether the metadata is valid. /// Checks whether the metadata is valid.
fn validate(&self) -> Result<()> { fn validate(&self) -> Result<()> {
// Id to name. // Id to name.
@@ -621,6 +683,17 @@ pub enum MetadataError {
err: String, err: String,
location: Location, location: Location,
}, },
#[snafu(display("Unexpected schema error during project"))]
SchemaProject {
origin_schema: SchemaRef,
projection: Vec<ColumnId>,
location: Location,
source: datatypes::Error,
},
#[snafu(display("Time index column not found"))]
TimeIndexNotFound { location: Location },
} }
impl ErrorExt for MetadataError { impl ErrorExt for MetadataError {