mirror of
https://github.com/GreptimeTeam/greptimedb.git
synced 2025-12-22 22:20:02 +00:00
Compare commits
14 Commits
ec77a5d53a
...
v0.15.0
| Author | SHA1 | Date | |
|---|---|---|---|
|
|
8612bb066f | ||
|
|
467593d329 | ||
|
|
9e4ae070b2 | ||
|
|
d8261dda51 | ||
|
|
7ab9b335a1 | ||
|
|
60835afb47 | ||
|
|
aba5bf7431 | ||
|
|
7897fe8dbe | ||
|
|
cc8ec706a1 | ||
|
|
7c688718db | ||
|
|
8a0e554e5a | ||
|
|
80fae1c559 | ||
|
|
c37c4df20d | ||
|
|
f712c1b356 |
9
Cargo.lock
generated
9
Cargo.lock
generated
@@ -4699,6 +4699,7 @@ version = "0.15.0"
|
||||
dependencies = [
|
||||
"api",
|
||||
"arc-swap",
|
||||
"async-stream",
|
||||
"async-trait",
|
||||
"auth",
|
||||
"bytes",
|
||||
@@ -5144,7 +5145,7 @@ dependencies = [
|
||||
[[package]]
|
||||
name = "greptime-proto"
|
||||
version = "0.1.0"
|
||||
source = "git+https://github.com/GreptimeTeam/greptime-proto.git?rev=82fe5c6282f623c185b86f03e898ee8952e50cf9#82fe5c6282f623c185b86f03e898ee8952e50cf9"
|
||||
source = "git+https://github.com/GreptimeTeam/greptime-proto.git?rev=96c733f8472284d3c83a4c011dc6de9cf830c353#96c733f8472284d3c83a4c011dc6de9cf830c353"
|
||||
dependencies = [
|
||||
"prost 0.13.5",
|
||||
"serde",
|
||||
@@ -9568,7 +9569,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index"
|
||||
checksum = "be769465445e8c1474e9c5dac2018218498557af32d9ed057325ec9a41ae81bf"
|
||||
dependencies = [
|
||||
"heck 0.5.0",
|
||||
"itertools 0.14.0",
|
||||
"itertools 0.11.0",
|
||||
"log",
|
||||
"multimap",
|
||||
"once_cell",
|
||||
@@ -9614,7 +9615,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index"
|
||||
checksum = "8a56d757972c98b346a9b766e3f02746cde6dd1cd1d1d563472929fdd74bec4d"
|
||||
dependencies = [
|
||||
"anyhow",
|
||||
"itertools 0.14.0",
|
||||
"itertools 0.11.0",
|
||||
"proc-macro2",
|
||||
"quote",
|
||||
"syn 2.0.100",
|
||||
@@ -14183,7 +14184,7 @@ version = "0.1.9"
|
||||
source = "registry+https://github.com/rust-lang/crates.io-index"
|
||||
checksum = "cf221c93e13a30d793f7645a0e7762c55d169dbb0a49671918a2319d289b10bb"
|
||||
dependencies = [
|
||||
"windows-sys 0.59.0",
|
||||
"windows-sys 0.48.0",
|
||||
]
|
||||
|
||||
[[package]]
|
||||
|
||||
@@ -134,7 +134,7 @@ etcd-client = "0.14"
|
||||
fst = "0.4.7"
|
||||
futures = "0.3"
|
||||
futures-util = "0.3"
|
||||
greptime-proto = { git = "https://github.com/GreptimeTeam/greptime-proto.git", rev = "82fe5c6282f623c185b86f03e898ee8952e50cf9" }
|
||||
greptime-proto = { git = "https://github.com/GreptimeTeam/greptime-proto.git", rev = "96c733f8472284d3c83a4c011dc6de9cf830c353" }
|
||||
hex = "0.4"
|
||||
http = "1"
|
||||
humantime = "2.1"
|
||||
|
||||
@@ -226,18 +226,20 @@ mod tests {
|
||||
assert!(options.is_none());
|
||||
|
||||
let mut schema = ColumnSchema::new("test", ConcreteDataType::string_datatype(), true)
|
||||
.with_fulltext_options(FulltextOptions {
|
||||
enable: true,
|
||||
analyzer: FulltextAnalyzer::English,
|
||||
case_sensitive: false,
|
||||
backend: FulltextBackend::Bloom,
|
||||
})
|
||||
.with_fulltext_options(FulltextOptions::new_unchecked(
|
||||
true,
|
||||
FulltextAnalyzer::English,
|
||||
false,
|
||||
FulltextBackend::Bloom,
|
||||
10240,
|
||||
0.01,
|
||||
))
|
||||
.unwrap();
|
||||
schema.set_inverted_index(true);
|
||||
let options = options_from_column_schema(&schema).unwrap();
|
||||
assert_eq!(
|
||||
options.options.get(FULLTEXT_GRPC_KEY).unwrap(),
|
||||
"{\"enable\":true,\"analyzer\":\"English\",\"case-sensitive\":false,\"backend\":\"bloom\"}"
|
||||
"{\"enable\":true,\"analyzer\":\"English\",\"case-sensitive\":false,\"backend\":\"bloom\",\"granularity\":10240,\"false-positive-rate-in-10000\":100}"
|
||||
);
|
||||
assert_eq!(
|
||||
options.options.get(INVERTED_INDEX_GRPC_KEY).unwrap(),
|
||||
@@ -247,16 +249,18 @@ mod tests {
|
||||
|
||||
#[test]
|
||||
fn test_options_with_fulltext() {
|
||||
let fulltext = FulltextOptions {
|
||||
enable: true,
|
||||
analyzer: FulltextAnalyzer::English,
|
||||
case_sensitive: false,
|
||||
backend: FulltextBackend::Bloom,
|
||||
};
|
||||
let fulltext = FulltextOptions::new_unchecked(
|
||||
true,
|
||||
FulltextAnalyzer::English,
|
||||
false,
|
||||
FulltextBackend::Bloom,
|
||||
10240,
|
||||
0.01,
|
||||
);
|
||||
let options = options_from_fulltext(&fulltext).unwrap().unwrap();
|
||||
assert_eq!(
|
||||
options.options.get(FULLTEXT_GRPC_KEY).unwrap(),
|
||||
"{\"enable\":true,\"analyzer\":\"English\",\"case-sensitive\":false,\"backend\":\"bloom\"}"
|
||||
"{\"enable\":true,\"analyzer\":\"English\",\"case-sensitive\":false,\"backend\":\"bloom\",\"granularity\":10240,\"false-positive-rate-in-10000\":100}"
|
||||
);
|
||||
}
|
||||
|
||||
|
||||
@@ -28,7 +28,7 @@ use common_meta::cache::{
|
||||
use common_meta::key::catalog_name::CatalogNameKey;
|
||||
use common_meta::key::flow::FlowMetadataManager;
|
||||
use common_meta::key::schema_name::SchemaNameKey;
|
||||
use common_meta::key::table_info::TableInfoValue;
|
||||
use common_meta::key::table_info::{TableInfoManager, TableInfoValue};
|
||||
use common_meta::key::table_name::TableNameKey;
|
||||
use common_meta::key::{TableMetadataManager, TableMetadataManagerRef};
|
||||
use common_meta::kv_backend::KvBackendRef;
|
||||
@@ -39,6 +39,7 @@ use moka::sync::Cache;
|
||||
use partition::manager::{PartitionRuleManager, PartitionRuleManagerRef};
|
||||
use session::context::{Channel, QueryContext};
|
||||
use snafu::prelude::*;
|
||||
use store_api::metric_engine_consts::METRIC_ENGINE_NAME;
|
||||
use table::dist_table::DistTable;
|
||||
use table::metadata::TableId;
|
||||
use table::table::numbers::{NumbersTable, NUMBERS_TABLE_NAME};
|
||||
@@ -142,6 +143,61 @@ impl KvBackendCatalogManager {
|
||||
pub fn procedure_manager(&self) -> Option<ProcedureManagerRef> {
|
||||
self.procedure_manager.clone()
|
||||
}
|
||||
|
||||
// Override logical table's partition key indices with physical table's.
|
||||
async fn override_logical_table_partition_key_indices(
|
||||
table_route_cache: &TableRouteCacheRef,
|
||||
table_info_manager: &TableInfoManager,
|
||||
table: TableRef,
|
||||
) -> Result<TableRef> {
|
||||
// If the table is not a metric table, return the table directly.
|
||||
if table.table_info().meta.engine != METRIC_ENGINE_NAME {
|
||||
return Ok(table);
|
||||
}
|
||||
|
||||
if let Some(table_route_value) = table_route_cache
|
||||
.get(table.table_info().table_id())
|
||||
.await
|
||||
.context(TableMetadataManagerSnafu)?
|
||||
&& let TableRoute::Logical(logical_route) = &*table_route_value
|
||||
&& let Some(physical_table_info_value) = table_info_manager
|
||||
.get(logical_route.physical_table_id())
|
||||
.await
|
||||
.context(TableMetadataManagerSnafu)?
|
||||
{
|
||||
let mut new_table_info = (*table.table_info()).clone();
|
||||
|
||||
// Remap partition key indices from physical table to logical table
|
||||
new_table_info.meta.partition_key_indices = physical_table_info_value
|
||||
.table_info
|
||||
.meta
|
||||
.partition_key_indices
|
||||
.iter()
|
||||
.filter_map(|&physical_index| {
|
||||
// Get the column name from the physical table using the physical index
|
||||
physical_table_info_value
|
||||
.table_info
|
||||
.meta
|
||||
.schema
|
||||
.column_schemas
|
||||
.get(physical_index)
|
||||
.and_then(|physical_column| {
|
||||
// Find the corresponding index in the logical table schema
|
||||
new_table_info
|
||||
.meta
|
||||
.schema
|
||||
.column_index_by_name(physical_column.name.as_str())
|
||||
})
|
||||
})
|
||||
.collect();
|
||||
|
||||
let new_table = DistTable::table(Arc::new(new_table_info));
|
||||
|
||||
return Ok(new_table);
|
||||
}
|
||||
|
||||
Ok(table)
|
||||
}
|
||||
}
|
||||
|
||||
#[async_trait::async_trait]
|
||||
@@ -268,10 +324,7 @@ impl CatalogManager for KvBackendCatalogManager {
|
||||
let table_cache: TableCacheRef = self.cache_registry.get().context(CacheNotFoundSnafu {
|
||||
name: "table_cache",
|
||||
})?;
|
||||
let table_route_cache: TableRouteCacheRef =
|
||||
self.cache_registry.get().context(CacheNotFoundSnafu {
|
||||
name: "table_route_cache",
|
||||
})?;
|
||||
|
||||
let table = table_cache
|
||||
.get_by_ref(&TableName {
|
||||
catalog_name: catalog_name.to_string(),
|
||||
@@ -281,55 +334,18 @@ impl CatalogManager for KvBackendCatalogManager {
|
||||
.await
|
||||
.context(GetTableCacheSnafu)?;
|
||||
|
||||
// Override logical table's partition key indices with physical table's.
|
||||
if let Some(table) = &table
|
||||
&& let Some(table_route_value) = table_route_cache
|
||||
.get(table.table_info().table_id())
|
||||
.await
|
||||
.context(TableMetadataManagerSnafu)?
|
||||
&& let TableRoute::Logical(logical_route) = &*table_route_value
|
||||
&& let Some(physical_table_info_value) = self
|
||||
.table_metadata_manager
|
||||
.table_info_manager()
|
||||
.get(logical_route.physical_table_id())
|
||||
.await
|
||||
.context(TableMetadataManagerSnafu)?
|
||||
{
|
||||
let mut new_table_info = (*table.table_info()).clone();
|
||||
// Gather all column names from the logical table
|
||||
let logical_column_names: std::collections::HashSet<_> = new_table_info
|
||||
.meta
|
||||
.schema
|
||||
.column_schemas()
|
||||
.iter()
|
||||
.map(|col| &col.name)
|
||||
.collect();
|
||||
|
||||
// Only preserve partition key indices where the corresponding columns exist in logical table
|
||||
new_table_info.meta.partition_key_indices = physical_table_info_value
|
||||
.table_info
|
||||
.meta
|
||||
.partition_key_indices
|
||||
.iter()
|
||||
.filter(|&&index| {
|
||||
if let Some(physical_column) = physical_table_info_value
|
||||
.table_info
|
||||
.meta
|
||||
.schema
|
||||
.column_schemas
|
||||
.get(index)
|
||||
{
|
||||
logical_column_names.contains(&physical_column.name)
|
||||
} else {
|
||||
false
|
||||
}
|
||||
})
|
||||
.cloned()
|
||||
.collect();
|
||||
|
||||
let new_table = DistTable::table(Arc::new(new_table_info));
|
||||
|
||||
return Ok(Some(new_table));
|
||||
if let Some(table) = table {
|
||||
let table_route_cache: TableRouteCacheRef =
|
||||
self.cache_registry.get().context(CacheNotFoundSnafu {
|
||||
name: "table_route_cache",
|
||||
})?;
|
||||
return Self::override_logical_table_partition_key_indices(
|
||||
&table_route_cache,
|
||||
self.table_metadata_manager.table_info_manager(),
|
||||
table,
|
||||
)
|
||||
.await
|
||||
.map(Some);
|
||||
}
|
||||
|
||||
if channel == Channel::Postgres {
|
||||
@@ -342,7 +358,7 @@ impl CatalogManager for KvBackendCatalogManager {
|
||||
}
|
||||
}
|
||||
|
||||
Ok(table)
|
||||
Ok(None)
|
||||
}
|
||||
|
||||
async fn tables_by_ids(
|
||||
@@ -394,8 +410,20 @@ impl CatalogManager for KvBackendCatalogManager {
|
||||
let catalog = catalog.to_string();
|
||||
let schema = schema.to_string();
|
||||
let semaphore = Arc::new(Semaphore::new(CONCURRENCY));
|
||||
let table_route_cache: Result<TableRouteCacheRef> =
|
||||
self.cache_registry.get().context(CacheNotFoundSnafu {
|
||||
name: "table_route_cache",
|
||||
});
|
||||
|
||||
common_runtime::spawn_global(async move {
|
||||
let table_route_cache = match table_route_cache {
|
||||
Ok(table_route_cache) => table_route_cache,
|
||||
Err(e) => {
|
||||
let _ = tx.send(Err(e)).await;
|
||||
return;
|
||||
}
|
||||
};
|
||||
|
||||
let table_id_stream = metadata_manager
|
||||
.table_name_manager()
|
||||
.tables(&catalog, &schema)
|
||||
@@ -422,6 +450,7 @@ impl CatalogManager for KvBackendCatalogManager {
|
||||
let metadata_manager = metadata_manager.clone();
|
||||
let tx = tx.clone();
|
||||
let semaphore = semaphore.clone();
|
||||
let table_route_cache = table_route_cache.clone();
|
||||
common_runtime::spawn_global(async move {
|
||||
// we don't explicitly close the semaphore so just ignore the potential error.
|
||||
let _ = semaphore.acquire().await;
|
||||
@@ -439,6 +468,16 @@ impl CatalogManager for KvBackendCatalogManager {
|
||||
};
|
||||
|
||||
for table in table_info_values.into_values().map(build_table) {
|
||||
let table = if let Ok(table) = table {
|
||||
Self::override_logical_table_partition_key_indices(
|
||||
&table_route_cache,
|
||||
metadata_manager.table_info_manager(),
|
||||
table,
|
||||
)
|
||||
.await
|
||||
} else {
|
||||
table
|
||||
};
|
||||
if tx.send(table).await.is_err() {
|
||||
return;
|
||||
}
|
||||
|
||||
@@ -21,7 +21,7 @@ use std::sync::{Arc, RwLock};
|
||||
use api::v1::frontend::{KillProcessRequest, ListProcessRequest, ProcessInfo};
|
||||
use common_base::cancellation::CancellationHandle;
|
||||
use common_frontend::selector::{FrontendSelector, MetaClientSelector};
|
||||
use common_telemetry::{debug, info};
|
||||
use common_telemetry::{debug, info, warn};
|
||||
use common_time::util::current_time_millis;
|
||||
use meta_client::MetaClientRef;
|
||||
use snafu::{ensure, OptionExt, ResultExt};
|
||||
@@ -141,14 +141,20 @@ impl ProcessManager {
|
||||
.await
|
||||
.context(error::InvokeFrontendSnafu)?;
|
||||
for mut f in frontends {
|
||||
processes.extend(
|
||||
f.list_process(ListProcessRequest {
|
||||
let result = f
|
||||
.list_process(ListProcessRequest {
|
||||
catalog: catalog.unwrap_or_default().to_string(),
|
||||
})
|
||||
.await
|
||||
.context(error::InvokeFrontendSnafu)?
|
||||
.processes,
|
||||
);
|
||||
.context(error::InvokeFrontendSnafu);
|
||||
match result {
|
||||
Ok(resp) => {
|
||||
processes.extend(resp.processes);
|
||||
}
|
||||
Err(e) => {
|
||||
warn!(e; "Skipping failing node: {:?}", f)
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
processes.extend(self.local_processes(catalog)?);
|
||||
|
||||
@@ -30,20 +30,16 @@ use common_catalog::consts::{MIN_USER_FLOW_ID, MIN_USER_TABLE_ID};
|
||||
use common_config::{metadata_store_dir, Configurable, KvBackendConfig};
|
||||
use common_error::ext::BoxedError;
|
||||
use common_meta::cache::LayeredCacheRegistryBuilder;
|
||||
use common_meta::cache_invalidator::CacheInvalidatorRef;
|
||||
use common_meta::cluster::{NodeInfo, NodeStatus};
|
||||
use common_meta::datanode::RegionStat;
|
||||
use common_meta::ddl::flow_meta::{FlowMetadataAllocator, FlowMetadataAllocatorRef};
|
||||
use common_meta::ddl::table_meta::{TableMetadataAllocator, TableMetadataAllocatorRef};
|
||||
use common_meta::ddl::flow_meta::FlowMetadataAllocator;
|
||||
use common_meta::ddl::table_meta::TableMetadataAllocator;
|
||||
use common_meta::ddl::{DdlContext, NoopRegionFailureDetectorControl, ProcedureExecutorRef};
|
||||
use common_meta::ddl_manager::DdlManager;
|
||||
#[cfg(feature = "enterprise")]
|
||||
use common_meta::ddl_manager::TriggerDdlManagerRef;
|
||||
use common_meta::key::flow::flow_state::FlowStat;
|
||||
use common_meta::key::flow::{FlowMetadataManager, FlowMetadataManagerRef};
|
||||
use common_meta::key::flow::FlowMetadataManager;
|
||||
use common_meta::key::{TableMetadataManager, TableMetadataManagerRef};
|
||||
use common_meta::kv_backend::KvBackendRef;
|
||||
use common_meta::node_manager::NodeManagerRef;
|
||||
use common_meta::peer::Peer;
|
||||
use common_meta::region_keeper::MemoryRegionKeeper;
|
||||
use common_meta::region_registry::LeaderRegionRegistry;
|
||||
@@ -594,28 +590,36 @@ impl StartCommand {
|
||||
.await
|
||||
.context(error::BuildWalOptionsAllocatorSnafu)?;
|
||||
let wal_options_allocator = Arc::new(wal_options_allocator);
|
||||
let table_meta_allocator = Arc::new(TableMetadataAllocator::new(
|
||||
let table_metadata_allocator = Arc::new(TableMetadataAllocator::new(
|
||||
table_id_sequence,
|
||||
wal_options_allocator.clone(),
|
||||
));
|
||||
let flow_meta_allocator = Arc::new(FlowMetadataAllocator::with_noop_peer_allocator(
|
||||
let flow_metadata_allocator = Arc::new(FlowMetadataAllocator::with_noop_peer_allocator(
|
||||
flow_id_sequence,
|
||||
));
|
||||
|
||||
let ddl_context = DdlContext {
|
||||
node_manager: node_manager.clone(),
|
||||
cache_invalidator: layered_cache_registry.clone(),
|
||||
memory_region_keeper: Arc::new(MemoryRegionKeeper::default()),
|
||||
leader_region_registry: Arc::new(LeaderRegionRegistry::default()),
|
||||
table_metadata_manager: table_metadata_manager.clone(),
|
||||
table_metadata_allocator: table_metadata_allocator.clone(),
|
||||
flow_metadata_manager: flow_metadata_manager.clone(),
|
||||
flow_metadata_allocator: flow_metadata_allocator.clone(),
|
||||
region_failure_detector_controller: Arc::new(NoopRegionFailureDetectorControl),
|
||||
};
|
||||
let procedure_manager_c = procedure_manager.clone();
|
||||
|
||||
let ddl_manager = DdlManager::try_new(ddl_context, procedure_manager_c, true)
|
||||
.context(error::InitDdlManagerSnafu)?;
|
||||
#[cfg(feature = "enterprise")]
|
||||
let trigger_ddl_manager: Option<TriggerDdlManagerRef> = plugins.get();
|
||||
let ddl_task_executor = Self::create_ddl_task_executor(
|
||||
procedure_manager.clone(),
|
||||
node_manager.clone(),
|
||||
layered_cache_registry.clone(),
|
||||
table_metadata_manager,
|
||||
table_meta_allocator,
|
||||
flow_metadata_manager,
|
||||
flow_meta_allocator,
|
||||
#[cfg(feature = "enterprise")]
|
||||
trigger_ddl_manager,
|
||||
)
|
||||
.await?;
|
||||
let ddl_manager = {
|
||||
let trigger_ddl_manager: Option<common_meta::ddl_manager::TriggerDdlManagerRef> =
|
||||
plugins.get();
|
||||
ddl_manager.with_trigger_ddl_manager(trigger_ddl_manager)
|
||||
};
|
||||
let ddl_task_executor: ProcedureExecutorRef = Arc::new(ddl_manager);
|
||||
|
||||
let fe_instance = FrontendBuilder::new(
|
||||
fe_opts.clone(),
|
||||
@@ -679,41 +683,6 @@ impl StartCommand {
|
||||
})
|
||||
}
|
||||
|
||||
#[allow(clippy::too_many_arguments)]
|
||||
pub async fn create_ddl_task_executor(
|
||||
procedure_manager: ProcedureManagerRef,
|
||||
node_manager: NodeManagerRef,
|
||||
cache_invalidator: CacheInvalidatorRef,
|
||||
table_metadata_manager: TableMetadataManagerRef,
|
||||
table_metadata_allocator: TableMetadataAllocatorRef,
|
||||
flow_metadata_manager: FlowMetadataManagerRef,
|
||||
flow_metadata_allocator: FlowMetadataAllocatorRef,
|
||||
#[cfg(feature = "enterprise")] trigger_ddl_manager: Option<TriggerDdlManagerRef>,
|
||||
) -> Result<ProcedureExecutorRef> {
|
||||
let procedure_executor: ProcedureExecutorRef = Arc::new(
|
||||
DdlManager::try_new(
|
||||
DdlContext {
|
||||
node_manager,
|
||||
cache_invalidator,
|
||||
memory_region_keeper: Arc::new(MemoryRegionKeeper::default()),
|
||||
leader_region_registry: Arc::new(LeaderRegionRegistry::default()),
|
||||
table_metadata_manager,
|
||||
table_metadata_allocator,
|
||||
flow_metadata_manager,
|
||||
flow_metadata_allocator,
|
||||
region_failure_detector_controller: Arc::new(NoopRegionFailureDetectorControl),
|
||||
},
|
||||
procedure_manager,
|
||||
true,
|
||||
#[cfg(feature = "enterprise")]
|
||||
trigger_ddl_manager,
|
||||
)
|
||||
.context(error::InitDdlManagerSnafu)?,
|
||||
);
|
||||
|
||||
Ok(procedure_executor)
|
||||
}
|
||||
|
||||
pub async fn create_table_metadata_manager(
|
||||
kv_backend: KvBackendRef,
|
||||
) -> Result<TableMetadataManagerRef> {
|
||||
|
||||
@@ -12,6 +12,7 @@
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
use std::fmt::Debug;
|
||||
use std::time::Duration;
|
||||
|
||||
use common_grpc::channel_manager::{ChannelConfig, ChannelManager};
|
||||
@@ -30,7 +31,7 @@ use crate::error::{MetaSnafu, Result};
|
||||
pub type FrontendClientPtr = Box<dyn FrontendClient>;
|
||||
|
||||
#[async_trait::async_trait]
|
||||
pub trait FrontendClient: Send {
|
||||
pub trait FrontendClient: Send + Debug {
|
||||
async fn list_process(&mut self, req: ListProcessRequest) -> Result<ListProcessResponse>;
|
||||
|
||||
async fn kill_process(&mut self, req: KillProcessRequest) -> Result<KillProcessResponse>;
|
||||
|
||||
@@ -34,7 +34,7 @@ use table::requests::{
|
||||
};
|
||||
|
||||
use crate::error::{
|
||||
InvalidColumnDefSnafu, InvalidSetFulltextOptionRequestSnafu,
|
||||
InvalidColumnDefSnafu, InvalidIndexOptionSnafu, InvalidSetFulltextOptionRequestSnafu,
|
||||
InvalidSetSkippingIndexOptionRequestSnafu, InvalidSetTableOptionRequestSnafu,
|
||||
InvalidUnsetTableOptionRequestSnafu, MissingAlterIndexOptionSnafu, MissingFieldSnafu,
|
||||
MissingTimestampColumnSnafu, Result, UnknownLocationTypeSnafu,
|
||||
@@ -126,18 +126,21 @@ pub fn alter_expr_to_request(table_id: TableId, expr: AlterTableExpr) -> Result<
|
||||
api::v1::set_index::Options::Fulltext(f) => AlterKind::SetIndex {
|
||||
options: SetIndexOptions::Fulltext {
|
||||
column_name: f.column_name.clone(),
|
||||
options: FulltextOptions {
|
||||
enable: f.enable,
|
||||
analyzer: as_fulltext_option_analyzer(
|
||||
options: FulltextOptions::new(
|
||||
f.enable,
|
||||
as_fulltext_option_analyzer(
|
||||
Analyzer::try_from(f.analyzer)
|
||||
.context(InvalidSetFulltextOptionRequestSnafu)?,
|
||||
),
|
||||
case_sensitive: f.case_sensitive,
|
||||
backend: as_fulltext_option_backend(
|
||||
f.case_sensitive,
|
||||
as_fulltext_option_backend(
|
||||
PbFulltextBackend::try_from(f.backend)
|
||||
.context(InvalidSetFulltextOptionRequestSnafu)?,
|
||||
),
|
||||
},
|
||||
f.granularity as u32,
|
||||
f.false_positive_rate,
|
||||
)
|
||||
.context(InvalidIndexOptionSnafu)?,
|
||||
},
|
||||
},
|
||||
api::v1::set_index::Options::Inverted(i) => AlterKind::SetIndex {
|
||||
@@ -148,13 +151,15 @@ pub fn alter_expr_to_request(table_id: TableId, expr: AlterTableExpr) -> Result<
|
||||
api::v1::set_index::Options::Skipping(s) => AlterKind::SetIndex {
|
||||
options: SetIndexOptions::Skipping {
|
||||
column_name: s.column_name,
|
||||
options: SkippingIndexOptions {
|
||||
granularity: s.granularity as u32,
|
||||
index_type: as_skipping_index_type(
|
||||
options: SkippingIndexOptions::new(
|
||||
s.granularity as u32,
|
||||
s.false_positive_rate,
|
||||
as_skipping_index_type(
|
||||
PbSkippingIndexType::try_from(s.skipping_index_type)
|
||||
.context(InvalidSetSkippingIndexOptionRequestSnafu)?,
|
||||
),
|
||||
},
|
||||
)
|
||||
.context(InvalidIndexOptionSnafu)?,
|
||||
},
|
||||
},
|
||||
},
|
||||
|
||||
@@ -153,6 +153,14 @@ pub enum Error {
|
||||
#[snafu(implicit)]
|
||||
location: Location,
|
||||
},
|
||||
|
||||
#[snafu(display("Invalid index option"))]
|
||||
InvalidIndexOption {
|
||||
#[snafu(implicit)]
|
||||
location: Location,
|
||||
#[snafu(source)]
|
||||
error: datatypes::error::Error,
|
||||
},
|
||||
}
|
||||
|
||||
pub type Result<T> = std::result::Result<T, Error>;
|
||||
@@ -180,7 +188,8 @@ impl ErrorExt for Error {
|
||||
| Error::InvalidUnsetTableOptionRequest { .. }
|
||||
| Error::InvalidSetFulltextOptionRequest { .. }
|
||||
| Error::InvalidSetSkippingIndexOptionRequest { .. }
|
||||
| Error::MissingAlterIndexOption { .. } => StatusCode::InvalidArguments,
|
||||
| Error::MissingAlterIndexOption { .. }
|
||||
| Error::InvalidIndexOption { .. } => StatusCode::InvalidArguments,
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@@ -125,13 +125,12 @@ impl DdlManager {
|
||||
ddl_context: DdlContext,
|
||||
procedure_manager: ProcedureManagerRef,
|
||||
register_loaders: bool,
|
||||
#[cfg(feature = "enterprise")] trigger_ddl_manager: Option<TriggerDdlManagerRef>,
|
||||
) -> Result<Self> {
|
||||
let manager = Self {
|
||||
ddl_context,
|
||||
procedure_manager,
|
||||
#[cfg(feature = "enterprise")]
|
||||
trigger_ddl_manager,
|
||||
trigger_ddl_manager: None,
|
||||
};
|
||||
if register_loaders {
|
||||
manager.register_loaders()?;
|
||||
@@ -139,6 +138,15 @@ impl DdlManager {
|
||||
Ok(manager)
|
||||
}
|
||||
|
||||
#[cfg(feature = "enterprise")]
|
||||
pub fn with_trigger_ddl_manager(
|
||||
mut self,
|
||||
trigger_ddl_manager: Option<TriggerDdlManagerRef>,
|
||||
) -> Self {
|
||||
self.trigger_ddl_manager = trigger_ddl_manager;
|
||||
self
|
||||
}
|
||||
|
||||
/// Returns the [TableMetadataManagerRef].
|
||||
pub fn table_metadata_manager(&self) -> &TableMetadataManagerRef {
|
||||
&self.ddl_context.table_metadata_manager
|
||||
@@ -964,8 +972,6 @@ mod tests {
|
||||
},
|
||||
procedure_manager.clone(),
|
||||
true,
|
||||
#[cfg(feature = "enterprise")]
|
||||
None,
|
||||
);
|
||||
|
||||
let expected_loaders = vec![
|
||||
|
||||
@@ -14,13 +14,14 @@
|
||||
|
||||
use std::collections::HashMap;
|
||||
|
||||
use common_telemetry::debug;
|
||||
use snafu::ensure;
|
||||
|
||||
use crate::error::{self, Result};
|
||||
use crate::key::txn_helper::TxnOpGetResponseSet;
|
||||
use crate::kv_backend::txn::{Compare, CompareOp, Txn, TxnOp};
|
||||
use crate::kv_backend::KvBackendRef;
|
||||
use crate::rpc::store::BatchGetRequest;
|
||||
use crate::rpc::store::{BatchDeleteRequest, BatchGetRequest};
|
||||
|
||||
/// [TombstoneManager] provides the ability to:
|
||||
/// - logically delete values
|
||||
@@ -28,6 +29,9 @@ use crate::rpc::store::BatchGetRequest;
|
||||
pub struct TombstoneManager {
|
||||
kv_backend: KvBackendRef,
|
||||
tombstone_prefix: String,
|
||||
// Only used for testing.
|
||||
#[cfg(test)]
|
||||
max_txn_ops: Option<usize>,
|
||||
}
|
||||
|
||||
const TOMBSTONE_PREFIX: &str = "__tombstone/";
|
||||
@@ -35,10 +39,7 @@ const TOMBSTONE_PREFIX: &str = "__tombstone/";
|
||||
impl TombstoneManager {
|
||||
/// Returns [TombstoneManager].
|
||||
pub fn new(kv_backend: KvBackendRef) -> Self {
|
||||
Self {
|
||||
kv_backend,
|
||||
tombstone_prefix: TOMBSTONE_PREFIX.to_string(),
|
||||
}
|
||||
Self::new_with_prefix(kv_backend, TOMBSTONE_PREFIX)
|
||||
}
|
||||
|
||||
/// Returns [TombstoneManager] with a custom tombstone prefix.
|
||||
@@ -46,6 +47,8 @@ impl TombstoneManager {
|
||||
Self {
|
||||
kv_backend,
|
||||
tombstone_prefix: prefix.to_string(),
|
||||
#[cfg(test)]
|
||||
max_txn_ops: None,
|
||||
}
|
||||
}
|
||||
|
||||
@@ -53,6 +56,11 @@ impl TombstoneManager {
|
||||
[self.tombstone_prefix.as_bytes(), key].concat()
|
||||
}
|
||||
|
||||
#[cfg(test)]
|
||||
pub fn set_max_txn_ops(&mut self, max_txn_ops: usize) {
|
||||
self.max_txn_ops = Some(max_txn_ops);
|
||||
}
|
||||
|
||||
/// Moves value to `dest_key`.
|
||||
///
|
||||
/// Puts `value` to `dest_key` if the value of `src_key` equals `value`.
|
||||
@@ -83,7 +91,11 @@ impl TombstoneManager {
|
||||
ensure!(
|
||||
keys.len() == dest_keys.len(),
|
||||
error::UnexpectedSnafu {
|
||||
err_msg: "The length of keys does not match the length of dest_keys."
|
||||
err_msg: format!(
|
||||
"The length of keys({}) does not match the length of dest_keys({}).",
|
||||
keys.len(),
|
||||
dest_keys.len()
|
||||
),
|
||||
}
|
||||
);
|
||||
// The key -> dest key mapping.
|
||||
@@ -136,19 +148,45 @@ impl TombstoneManager {
|
||||
.fail()
|
||||
}
|
||||
|
||||
fn max_txn_ops(&self) -> usize {
|
||||
#[cfg(test)]
|
||||
if let Some(max_txn_ops) = self.max_txn_ops {
|
||||
return max_txn_ops;
|
||||
}
|
||||
self.kv_backend.max_txn_ops()
|
||||
}
|
||||
|
||||
/// Moves values to `dest_key`.
|
||||
///
|
||||
/// Returns the number of keys that were moved.
|
||||
async fn move_values(&self, keys: Vec<Vec<u8>>, dest_keys: Vec<Vec<u8>>) -> Result<usize> {
|
||||
let chunk_size = self.kv_backend.max_txn_ops() / 2;
|
||||
if keys.len() > chunk_size {
|
||||
let keys_chunks = keys.chunks(chunk_size).collect::<Vec<_>>();
|
||||
let dest_keys_chunks = keys.chunks(chunk_size).collect::<Vec<_>>();
|
||||
for (keys, dest_keys) in keys_chunks.into_iter().zip(dest_keys_chunks) {
|
||||
self.move_values_inner(keys, dest_keys).await?;
|
||||
ensure!(
|
||||
keys.len() == dest_keys.len(),
|
||||
error::UnexpectedSnafu {
|
||||
err_msg: format!(
|
||||
"The length of keys({}) does not match the length of dest_keys({}).",
|
||||
keys.len(),
|
||||
dest_keys.len()
|
||||
),
|
||||
}
|
||||
|
||||
Ok(keys.len())
|
||||
);
|
||||
if keys.is_empty() {
|
||||
return Ok(0);
|
||||
}
|
||||
let chunk_size = self.max_txn_ops() / 2;
|
||||
if keys.len() > chunk_size {
|
||||
debug!(
|
||||
"Moving values with multiple chunks, keys len: {}, chunk_size: {}",
|
||||
keys.len(),
|
||||
chunk_size
|
||||
);
|
||||
let mut moved_keys = 0;
|
||||
let keys_chunks = keys.chunks(chunk_size).collect::<Vec<_>>();
|
||||
let dest_keys_chunks = dest_keys.chunks(chunk_size).collect::<Vec<_>>();
|
||||
for (keys, dest_keys) in keys_chunks.into_iter().zip(dest_keys_chunks) {
|
||||
moved_keys += self.move_values_inner(keys, dest_keys).await?;
|
||||
}
|
||||
Ok(moved_keys)
|
||||
} else {
|
||||
self.move_values_inner(&keys, &dest_keys).await
|
||||
}
|
||||
@@ -196,15 +234,18 @@ impl TombstoneManager {
|
||||
///
|
||||
/// Returns the number of keys that were deleted.
|
||||
pub async fn delete(&self, keys: Vec<Vec<u8>>) -> Result<usize> {
|
||||
let operations = keys
|
||||
let keys = keys
|
||||
.iter()
|
||||
.map(|key| TxnOp::Delete(self.to_tombstone(key)))
|
||||
.map(|key| self.to_tombstone(key))
|
||||
.collect::<Vec<_>>();
|
||||
|
||||
let txn = Txn::new().and_then(operations);
|
||||
// Always success.
|
||||
let _ = self.kv_backend.txn(txn).await?;
|
||||
Ok(keys.len())
|
||||
let num_keys = keys.len();
|
||||
let _ = self
|
||||
.kv_backend
|
||||
.batch_delete(BatchDeleteRequest::new().with_keys(keys))
|
||||
.await?;
|
||||
|
||||
Ok(num_keys)
|
||||
}
|
||||
}
|
||||
|
||||
@@ -392,16 +433,73 @@ mod tests {
|
||||
.into_iter()
|
||||
.map(|kv| (kv.key, kv.dest_key))
|
||||
.unzip();
|
||||
tombstone_manager
|
||||
let moved_keys = tombstone_manager
|
||||
.move_values(keys.clone(), dest_keys.clone())
|
||||
.await
|
||||
.unwrap();
|
||||
assert_eq!(kvs.len(), moved_keys);
|
||||
check_moved_values(kv_backend.clone(), &move_values).await;
|
||||
// Moves again
|
||||
tombstone_manager
|
||||
let moved_keys = tombstone_manager
|
||||
.move_values(keys.clone(), dest_keys.clone())
|
||||
.await
|
||||
.unwrap();
|
||||
assert_eq!(0, moved_keys);
|
||||
check_moved_values(kv_backend.clone(), &move_values).await;
|
||||
}
|
||||
|
||||
#[tokio::test]
|
||||
async fn test_move_values_with_max_txn_ops() {
|
||||
common_telemetry::init_default_ut_logging();
|
||||
let kv_backend = Arc::new(MemoryKvBackend::default());
|
||||
let mut tombstone_manager = TombstoneManager::new(kv_backend.clone());
|
||||
tombstone_manager.set_max_txn_ops(4);
|
||||
let kvs = HashMap::from([
|
||||
(b"bar".to_vec(), b"baz".to_vec()),
|
||||
(b"foo".to_vec(), b"hi".to_vec()),
|
||||
(b"baz".to_vec(), b"hello".to_vec()),
|
||||
(b"qux".to_vec(), b"world".to_vec()),
|
||||
(b"quux".to_vec(), b"world".to_vec()),
|
||||
(b"quuux".to_vec(), b"world".to_vec()),
|
||||
(b"quuuux".to_vec(), b"world".to_vec()),
|
||||
(b"quuuuux".to_vec(), b"world".to_vec()),
|
||||
(b"quuuuuux".to_vec(), b"world".to_vec()),
|
||||
]);
|
||||
for (key, value) in &kvs {
|
||||
kv_backend
|
||||
.put(
|
||||
PutRequest::new()
|
||||
.with_key(key.clone())
|
||||
.with_value(value.clone()),
|
||||
)
|
||||
.await
|
||||
.unwrap();
|
||||
}
|
||||
let move_values = kvs
|
||||
.iter()
|
||||
.map(|(key, value)| MoveValue {
|
||||
key: key.clone(),
|
||||
dest_key: tombstone_manager.to_tombstone(key),
|
||||
value: value.clone(),
|
||||
})
|
||||
.collect::<Vec<_>>();
|
||||
let (keys, dest_keys): (Vec<_>, Vec<_>) = move_values
|
||||
.clone()
|
||||
.into_iter()
|
||||
.map(|kv| (kv.key, kv.dest_key))
|
||||
.unzip();
|
||||
let moved_keys = tombstone_manager
|
||||
.move_values(keys.clone(), dest_keys.clone())
|
||||
.await
|
||||
.unwrap();
|
||||
assert_eq!(kvs.len(), moved_keys);
|
||||
check_moved_values(kv_backend.clone(), &move_values).await;
|
||||
// Moves again
|
||||
let moved_keys = tombstone_manager
|
||||
.move_values(keys.clone(), dest_keys.clone())
|
||||
.await
|
||||
.unwrap();
|
||||
assert_eq!(0, moved_keys);
|
||||
check_moved_values(kv_backend.clone(), &move_values).await;
|
||||
}
|
||||
|
||||
@@ -439,17 +537,19 @@ mod tests {
|
||||
.unzip();
|
||||
keys.push(b"non-exists".to_vec());
|
||||
dest_keys.push(b"hi/non-exists".to_vec());
|
||||
tombstone_manager
|
||||
let moved_keys = tombstone_manager
|
||||
.move_values(keys.clone(), dest_keys.clone())
|
||||
.await
|
||||
.unwrap();
|
||||
check_moved_values(kv_backend.clone(), &move_values).await;
|
||||
assert_eq!(3, moved_keys);
|
||||
// Moves again
|
||||
tombstone_manager
|
||||
let moved_keys = tombstone_manager
|
||||
.move_values(keys.clone(), dest_keys.clone())
|
||||
.await
|
||||
.unwrap();
|
||||
check_moved_values(kv_backend.clone(), &move_values).await;
|
||||
assert_eq!(0, moved_keys);
|
||||
}
|
||||
|
||||
#[tokio::test]
|
||||
@@ -490,10 +590,11 @@ mod tests {
|
||||
.into_iter()
|
||||
.map(|kv| (kv.key, kv.dest_key))
|
||||
.unzip();
|
||||
tombstone_manager
|
||||
let moved_keys = tombstone_manager
|
||||
.move_values(keys, dest_keys)
|
||||
.await
|
||||
.unwrap();
|
||||
assert_eq!(kvs.len(), moved_keys);
|
||||
}
|
||||
|
||||
#[tokio::test]
|
||||
@@ -571,4 +672,24 @@ mod tests {
|
||||
.unwrap();
|
||||
check_moved_values(kv_backend.clone(), &move_values).await;
|
||||
}
|
||||
|
||||
#[tokio::test]
|
||||
async fn test_move_values_with_different_lengths() {
|
||||
let kv_backend = Arc::new(MemoryKvBackend::default());
|
||||
let tombstone_manager = TombstoneManager::new(kv_backend.clone());
|
||||
|
||||
let keys = vec![b"bar".to_vec(), b"foo".to_vec()];
|
||||
let dest_keys = vec![b"bar".to_vec(), b"foo".to_vec(), b"baz".to_vec()];
|
||||
|
||||
let err = tombstone_manager
|
||||
.move_values(keys, dest_keys)
|
||||
.await
|
||||
.unwrap_err();
|
||||
assert!(err
|
||||
.to_string()
|
||||
.contains("The length of keys(2) does not match the length of dest_keys(3)."),);
|
||||
|
||||
let moved_keys = tombstone_manager.move_values(vec![], vec![]).await.unwrap();
|
||||
assert_eq!(0, moved_keys);
|
||||
}
|
||||
}
|
||||
|
||||
@@ -178,8 +178,6 @@ pub enum Error {
|
||||
StreamTimeout {
|
||||
#[snafu(implicit)]
|
||||
location: Location,
|
||||
#[snafu(source)]
|
||||
error: tokio::time::error::Elapsed,
|
||||
},
|
||||
|
||||
#[snafu(display("RecordBatch slice index overflow: {visit_index} > {size}"))]
|
||||
|
||||
@@ -475,7 +475,7 @@ mod test {
|
||||
async fn region_alive_keeper() {
|
||||
common_telemetry::init_default_ut_logging();
|
||||
let mut region_server = mock_region_server();
|
||||
let mut engine_env = TestEnv::with_prefix("region-alive-keeper");
|
||||
let mut engine_env = TestEnv::with_prefix("region-alive-keeper").await;
|
||||
let engine = engine_env.create_engine(MitoConfig::default()).await;
|
||||
let engine = Arc::new(engine);
|
||||
region_server.register_engine(engine.clone());
|
||||
|
||||
@@ -278,7 +278,7 @@ mod tests {
|
||||
let mut region_server = mock_region_server();
|
||||
let heartbeat_handler = RegionHeartbeatResponseHandler::new(region_server.clone());
|
||||
|
||||
let mut engine_env = TestEnv::with_prefix("close-region");
|
||||
let mut engine_env = TestEnv::with_prefix("close-region").await;
|
||||
let engine = engine_env.create_engine(MitoConfig::default()).await;
|
||||
region_server.register_engine(Arc::new(engine));
|
||||
let region_id = RegionId::new(1024, 1);
|
||||
@@ -326,7 +326,7 @@ mod tests {
|
||||
let mut region_server = mock_region_server();
|
||||
let heartbeat_handler = RegionHeartbeatResponseHandler::new(region_server.clone());
|
||||
|
||||
let mut engine_env = TestEnv::with_prefix("open-region");
|
||||
let mut engine_env = TestEnv::with_prefix("open-region").await;
|
||||
let engine = engine_env.create_engine(MitoConfig::default()).await;
|
||||
region_server.register_engine(Arc::new(engine));
|
||||
let region_id = RegionId::new(1024, 1);
|
||||
@@ -374,7 +374,7 @@ mod tests {
|
||||
let mut region_server = mock_region_server();
|
||||
let heartbeat_handler = RegionHeartbeatResponseHandler::new(region_server.clone());
|
||||
|
||||
let mut engine_env = TestEnv::with_prefix("open-not-exists-region");
|
||||
let mut engine_env = TestEnv::with_prefix("open-not-exists-region").await;
|
||||
let engine = engine_env.create_engine(MitoConfig::default()).await;
|
||||
region_server.register_engine(Arc::new(engine));
|
||||
let region_id = RegionId::new(1024, 1);
|
||||
@@ -406,7 +406,7 @@ mod tests {
|
||||
let mut region_server = mock_region_server();
|
||||
let heartbeat_handler = RegionHeartbeatResponseHandler::new(region_server.clone());
|
||||
|
||||
let mut engine_env = TestEnv::with_prefix("downgrade-region");
|
||||
let mut engine_env = TestEnv::with_prefix("downgrade-region").await;
|
||||
let engine = engine_env.create_engine(MitoConfig::default()).await;
|
||||
region_server.register_engine(Arc::new(engine));
|
||||
let region_id = RegionId::new(1024, 1);
|
||||
|
||||
@@ -31,9 +31,10 @@ pub use crate::schema::column_schema::{
|
||||
ColumnSchema, FulltextAnalyzer, FulltextBackend, FulltextOptions, Metadata,
|
||||
SkippingIndexOptions, SkippingIndexType, COLUMN_FULLTEXT_CHANGE_OPT_KEY_ENABLE,
|
||||
COLUMN_FULLTEXT_OPT_KEY_ANALYZER, COLUMN_FULLTEXT_OPT_KEY_BACKEND,
|
||||
COLUMN_FULLTEXT_OPT_KEY_CASE_SENSITIVE, COLUMN_SKIPPING_INDEX_OPT_KEY_GRANULARITY,
|
||||
COLUMN_SKIPPING_INDEX_OPT_KEY_TYPE, COMMENT_KEY, FULLTEXT_KEY, INVERTED_INDEX_KEY,
|
||||
SKIPPING_INDEX_KEY, TIME_INDEX_KEY,
|
||||
COLUMN_FULLTEXT_OPT_KEY_CASE_SENSITIVE, COLUMN_FULLTEXT_OPT_KEY_FALSE_POSITIVE_RATE,
|
||||
COLUMN_FULLTEXT_OPT_KEY_GRANULARITY, COLUMN_SKIPPING_INDEX_OPT_KEY_FALSE_POSITIVE_RATE,
|
||||
COLUMN_SKIPPING_INDEX_OPT_KEY_GRANULARITY, COLUMN_SKIPPING_INDEX_OPT_KEY_TYPE, COMMENT_KEY,
|
||||
FULLTEXT_KEY, INVERTED_INDEX_KEY, SKIPPING_INDEX_KEY, TIME_INDEX_KEY,
|
||||
};
|
||||
pub use crate::schema::constraint::ColumnDefaultConstraint;
|
||||
pub use crate::schema::raw::RawSchema;
|
||||
|
||||
@@ -47,13 +47,18 @@ pub const COLUMN_FULLTEXT_CHANGE_OPT_KEY_ENABLE: &str = "enable";
|
||||
pub const COLUMN_FULLTEXT_OPT_KEY_ANALYZER: &str = "analyzer";
|
||||
pub const COLUMN_FULLTEXT_OPT_KEY_CASE_SENSITIVE: &str = "case_sensitive";
|
||||
pub const COLUMN_FULLTEXT_OPT_KEY_BACKEND: &str = "backend";
|
||||
pub const COLUMN_FULLTEXT_OPT_KEY_GRANULARITY: &str = "granularity";
|
||||
pub const COLUMN_FULLTEXT_OPT_KEY_FALSE_POSITIVE_RATE: &str = "false_positive_rate";
|
||||
|
||||
/// Keys used in SKIPPING index options
|
||||
pub const COLUMN_SKIPPING_INDEX_OPT_KEY_GRANULARITY: &str = "granularity";
|
||||
pub const COLUMN_SKIPPING_INDEX_OPT_KEY_FALSE_POSITIVE_RATE: &str = "false_positive_rate";
|
||||
pub const COLUMN_SKIPPING_INDEX_OPT_KEY_TYPE: &str = "type";
|
||||
|
||||
pub const DEFAULT_GRANULARITY: u32 = 10240;
|
||||
|
||||
pub const DEFAULT_FALSE_POSITIVE_RATE: f64 = 0.01;
|
||||
|
||||
/// Schema of a column, used as an immutable struct.
|
||||
#[derive(Clone, PartialEq, Eq, Serialize, Deserialize)]
|
||||
pub struct ColumnSchema {
|
||||
@@ -504,7 +509,7 @@ impl TryFrom<&ColumnSchema> for Field {
|
||||
}
|
||||
|
||||
/// Fulltext options for a column.
|
||||
#[derive(Debug, Clone, PartialEq, Eq, Serialize, Deserialize, Default, Visit, VisitMut)]
|
||||
#[derive(Debug, Clone, PartialEq, Eq, Serialize, Deserialize, Visit, VisitMut)]
|
||||
#[serde(rename_all = "kebab-case")]
|
||||
pub struct FulltextOptions {
|
||||
/// Whether the fulltext index is enabled.
|
||||
@@ -518,6 +523,92 @@ pub struct FulltextOptions {
|
||||
/// The fulltext backend to use.
|
||||
#[serde(default)]
|
||||
pub backend: FulltextBackend,
|
||||
/// The granularity of the fulltext index (for bloom backend only)
|
||||
#[serde(default = "fulltext_options_default_granularity")]
|
||||
pub granularity: u32,
|
||||
/// The false positive rate of the fulltext index (for bloom backend only)
|
||||
#[serde(default = "fulltext_options_default_false_positive_rate_in_10000")]
|
||||
pub false_positive_rate_in_10000: u32,
|
||||
}
|
||||
|
||||
fn fulltext_options_default_granularity() -> u32 {
|
||||
DEFAULT_GRANULARITY
|
||||
}
|
||||
|
||||
fn fulltext_options_default_false_positive_rate_in_10000() -> u32 {
|
||||
(DEFAULT_FALSE_POSITIVE_RATE * 10000.0) as u32
|
||||
}
|
||||
|
||||
impl FulltextOptions {
|
||||
/// Creates a new fulltext options.
|
||||
pub fn new(
|
||||
enable: bool,
|
||||
analyzer: FulltextAnalyzer,
|
||||
case_sensitive: bool,
|
||||
backend: FulltextBackend,
|
||||
granularity: u32,
|
||||
false_positive_rate: f64,
|
||||
) -> Result<Self> {
|
||||
ensure!(
|
||||
0.0 < false_positive_rate && false_positive_rate <= 1.0,
|
||||
error::InvalidFulltextOptionSnafu {
|
||||
msg: format!(
|
||||
"Invalid false positive rate: {false_positive_rate}, expected: 0.0 < rate <= 1.0"
|
||||
),
|
||||
}
|
||||
);
|
||||
ensure!(
|
||||
granularity > 0,
|
||||
error::InvalidFulltextOptionSnafu {
|
||||
msg: format!("Invalid granularity: {granularity}, expected: positive integer"),
|
||||
}
|
||||
);
|
||||
Ok(Self::new_unchecked(
|
||||
enable,
|
||||
analyzer,
|
||||
case_sensitive,
|
||||
backend,
|
||||
granularity,
|
||||
false_positive_rate,
|
||||
))
|
||||
}
|
||||
|
||||
/// Creates a new fulltext options without checking `false_positive_rate` and `granularity`.
|
||||
pub fn new_unchecked(
|
||||
enable: bool,
|
||||
analyzer: FulltextAnalyzer,
|
||||
case_sensitive: bool,
|
||||
backend: FulltextBackend,
|
||||
granularity: u32,
|
||||
false_positive_rate: f64,
|
||||
) -> Self {
|
||||
Self {
|
||||
enable,
|
||||
analyzer,
|
||||
case_sensitive,
|
||||
backend,
|
||||
granularity,
|
||||
false_positive_rate_in_10000: (false_positive_rate * 10000.0) as u32,
|
||||
}
|
||||
}
|
||||
|
||||
/// Gets the false positive rate.
|
||||
pub fn false_positive_rate(&self) -> f64 {
|
||||
self.false_positive_rate_in_10000 as f64 / 10000.0
|
||||
}
|
||||
}
|
||||
|
||||
impl Default for FulltextOptions {
|
||||
fn default() -> Self {
|
||||
Self::new_unchecked(
|
||||
false,
|
||||
FulltextAnalyzer::default(),
|
||||
false,
|
||||
FulltextBackend::default(),
|
||||
DEFAULT_GRANULARITY,
|
||||
DEFAULT_FALSE_POSITIVE_RATE,
|
||||
)
|
||||
}
|
||||
}
|
||||
|
||||
impl fmt::Display for FulltextOptions {
|
||||
@@ -527,6 +618,10 @@ impl fmt::Display for FulltextOptions {
|
||||
write!(f, ", analyzer={}", self.analyzer)?;
|
||||
write!(f, ", case_sensitive={}", self.case_sensitive)?;
|
||||
write!(f, ", backend={}", self.backend)?;
|
||||
if self.backend == FulltextBackend::Bloom {
|
||||
write!(f, ", granularity={}", self.granularity)?;
|
||||
write!(f, ", false_positive_rate={}", self.false_positive_rate())?;
|
||||
}
|
||||
}
|
||||
Ok(())
|
||||
}
|
||||
@@ -611,6 +706,45 @@ impl TryFrom<HashMap<String, String>> for FulltextOptions {
|
||||
}
|
||||
}
|
||||
|
||||
if fulltext_options.backend == FulltextBackend::Bloom {
|
||||
// Parse granularity with default value 10240
|
||||
let granularity = match options.get(COLUMN_FULLTEXT_OPT_KEY_GRANULARITY) {
|
||||
Some(value) => value
|
||||
.parse::<u32>()
|
||||
.ok()
|
||||
.filter(|&v| v > 0)
|
||||
.ok_or_else(|| {
|
||||
error::InvalidFulltextOptionSnafu {
|
||||
msg: format!(
|
||||
"Invalid granularity: {value}, expected: positive integer"
|
||||
),
|
||||
}
|
||||
.build()
|
||||
})?,
|
||||
None => DEFAULT_GRANULARITY,
|
||||
};
|
||||
fulltext_options.granularity = granularity;
|
||||
|
||||
// Parse false positive rate with default value 0.01
|
||||
let false_positive_rate = match options.get(COLUMN_FULLTEXT_OPT_KEY_FALSE_POSITIVE_RATE)
|
||||
{
|
||||
Some(value) => value
|
||||
.parse::<f64>()
|
||||
.ok()
|
||||
.filter(|&v| v > 0.0 && v <= 1.0)
|
||||
.ok_or_else(|| {
|
||||
error::InvalidFulltextOptionSnafu {
|
||||
msg: format!(
|
||||
"Invalid false positive rate: {value}, expected: 0.0 < rate <= 1.0"
|
||||
),
|
||||
}
|
||||
.build()
|
||||
})?,
|
||||
None => DEFAULT_FALSE_POSITIVE_RATE,
|
||||
};
|
||||
fulltext_options.false_positive_rate_in_10000 = (false_positive_rate * 10000.0) as u32;
|
||||
}
|
||||
|
||||
Ok(fulltext_options)
|
||||
}
|
||||
}
|
||||
@@ -638,23 +772,72 @@ impl fmt::Display for FulltextAnalyzer {
|
||||
pub struct SkippingIndexOptions {
|
||||
/// The granularity of the skip index.
|
||||
pub granularity: u32,
|
||||
/// The false positive rate of the skip index (in ten-thousandths, e.g., 100 = 1%).
|
||||
pub false_positive_rate_in_10000: u32,
|
||||
/// The type of the skip index.
|
||||
#[serde(default)]
|
||||
pub index_type: SkippingIndexType,
|
||||
}
|
||||
|
||||
impl SkippingIndexOptions {
|
||||
/// Creates a new skipping index options without checking `false_positive_rate` and `granularity`.
|
||||
pub fn new_unchecked(
|
||||
granularity: u32,
|
||||
false_positive_rate: f64,
|
||||
index_type: SkippingIndexType,
|
||||
) -> Self {
|
||||
Self {
|
||||
granularity,
|
||||
false_positive_rate_in_10000: (false_positive_rate * 10000.0) as u32,
|
||||
index_type,
|
||||
}
|
||||
}
|
||||
|
||||
/// Creates a new skipping index options.
|
||||
pub fn new(
|
||||
granularity: u32,
|
||||
false_positive_rate: f64,
|
||||
index_type: SkippingIndexType,
|
||||
) -> Result<Self> {
|
||||
ensure!(
|
||||
0.0 < false_positive_rate && false_positive_rate <= 1.0,
|
||||
error::InvalidSkippingIndexOptionSnafu {
|
||||
msg: format!("Invalid false positive rate: {false_positive_rate}, expected: 0.0 < rate <= 1.0"),
|
||||
}
|
||||
);
|
||||
ensure!(
|
||||
granularity > 0,
|
||||
error::InvalidSkippingIndexOptionSnafu {
|
||||
msg: format!("Invalid granularity: {granularity}, expected: positive integer"),
|
||||
}
|
||||
);
|
||||
Ok(Self::new_unchecked(
|
||||
granularity,
|
||||
false_positive_rate,
|
||||
index_type,
|
||||
))
|
||||
}
|
||||
|
||||
/// Gets the false positive rate.
|
||||
pub fn false_positive_rate(&self) -> f64 {
|
||||
self.false_positive_rate_in_10000 as f64 / 10000.0
|
||||
}
|
||||
}
|
||||
|
||||
impl Default for SkippingIndexOptions {
|
||||
fn default() -> Self {
|
||||
Self {
|
||||
granularity: DEFAULT_GRANULARITY,
|
||||
index_type: SkippingIndexType::default(),
|
||||
}
|
||||
Self::new_unchecked(
|
||||
DEFAULT_GRANULARITY,
|
||||
DEFAULT_FALSE_POSITIVE_RATE,
|
||||
SkippingIndexType::default(),
|
||||
)
|
||||
}
|
||||
}
|
||||
|
||||
impl fmt::Display for SkippingIndexOptions {
|
||||
fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
|
||||
write!(f, "granularity={}", self.granularity)?;
|
||||
write!(f, ", false_positive_rate={}", self.false_positive_rate())?;
|
||||
write!(f, ", index_type={}", self.index_type)?;
|
||||
Ok(())
|
||||
}
|
||||
@@ -681,15 +864,37 @@ impl TryFrom<HashMap<String, String>> for SkippingIndexOptions {
|
||||
fn try_from(options: HashMap<String, String>) -> Result<Self> {
|
||||
// Parse granularity with default value 1
|
||||
let granularity = match options.get(COLUMN_SKIPPING_INDEX_OPT_KEY_GRANULARITY) {
|
||||
Some(value) => value.parse::<u32>().map_err(|_| {
|
||||
error::InvalidSkippingIndexOptionSnafu {
|
||||
msg: format!("Invalid granularity: {value}, expected: positive integer"),
|
||||
}
|
||||
.build()
|
||||
})?,
|
||||
Some(value) => value
|
||||
.parse::<u32>()
|
||||
.ok()
|
||||
.filter(|&v| v > 0)
|
||||
.ok_or_else(|| {
|
||||
error::InvalidSkippingIndexOptionSnafu {
|
||||
msg: format!("Invalid granularity: {value}, expected: positive integer"),
|
||||
}
|
||||
.build()
|
||||
})?,
|
||||
None => DEFAULT_GRANULARITY,
|
||||
};
|
||||
|
||||
// Parse false positive rate with default value 100
|
||||
let false_positive_rate =
|
||||
match options.get(COLUMN_SKIPPING_INDEX_OPT_KEY_FALSE_POSITIVE_RATE) {
|
||||
Some(value) => value
|
||||
.parse::<f64>()
|
||||
.ok()
|
||||
.filter(|&v| v > 0.0 && v <= 1.0)
|
||||
.ok_or_else(|| {
|
||||
error::InvalidSkippingIndexOptionSnafu {
|
||||
msg: format!(
|
||||
"Invalid false positive rate: {value}, expected: 0.0 < rate <= 1.0"
|
||||
),
|
||||
}
|
||||
.build()
|
||||
})?,
|
||||
None => DEFAULT_FALSE_POSITIVE_RATE,
|
||||
};
|
||||
|
||||
// Parse index type with default value BloomFilter
|
||||
let index_type = match options.get(COLUMN_SKIPPING_INDEX_OPT_KEY_TYPE) {
|
||||
Some(typ) => match typ.to_ascii_uppercase().as_str() {
|
||||
@@ -704,10 +909,11 @@ impl TryFrom<HashMap<String, String>> for SkippingIndexOptions {
|
||||
None => SkippingIndexType::default(),
|
||||
};
|
||||
|
||||
Ok(SkippingIndexOptions {
|
||||
Ok(SkippingIndexOptions::new_unchecked(
|
||||
granularity,
|
||||
false_positive_rate,
|
||||
index_type,
|
||||
})
|
||||
))
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@@ -899,7 +899,7 @@ impl StreamingEngine {
|
||||
let rows_send = self.run_available(true).await?;
|
||||
let row = self.send_writeback_requests().await?;
|
||||
debug!(
|
||||
"Done to flush flow_id={:?} with {} input rows flushed, {} rows sended and {} output rows flushed",
|
||||
"Done to flush flow_id={:?} with {} input rows flushed, {} rows sent and {} output rows flushed",
|
||||
flow_id, flushed_input_rows, rows_send, row
|
||||
);
|
||||
Ok(row)
|
||||
|
||||
@@ -14,6 +14,7 @@ workspace = true
|
||||
[dependencies]
|
||||
api.workspace = true
|
||||
arc-swap = "1.0"
|
||||
async-stream.workspace = true
|
||||
async-trait.workspace = true
|
||||
auth.workspace = true
|
||||
bytes.workspace = true
|
||||
|
||||
@@ -363,6 +363,12 @@ pub enum Error {
|
||||
#[snafu(implicit)]
|
||||
location: Location,
|
||||
},
|
||||
|
||||
#[snafu(display("Canceling statement due to statement timeout"))]
|
||||
StatementTimeout {
|
||||
#[snafu(implicit)]
|
||||
location: Location,
|
||||
},
|
||||
}
|
||||
|
||||
pub type Result<T> = std::result::Result<T, Error>;
|
||||
@@ -443,6 +449,8 @@ impl ErrorExt for Error {
|
||||
Error::DataFusion { error, .. } => datafusion_status_code::<Self>(error, None),
|
||||
|
||||
Error::Cancelled { .. } => StatusCode::Cancelled,
|
||||
|
||||
Error::StatementTimeout { .. } => StatusCode::Cancelled,
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@@ -25,9 +25,11 @@ mod promql;
|
||||
mod region_query;
|
||||
pub mod standalone;
|
||||
|
||||
use std::pin::Pin;
|
||||
use std::sync::Arc;
|
||||
use std::time::SystemTime;
|
||||
use std::time::{Duration, SystemTime};
|
||||
|
||||
use async_stream::stream;
|
||||
use async_trait::async_trait;
|
||||
use auth::{PermissionChecker, PermissionCheckerRef, PermissionReq};
|
||||
use catalog::process_manager::ProcessManagerRef;
|
||||
@@ -44,8 +46,11 @@ use common_procedure::local::{LocalManager, ManagerConfig};
|
||||
use common_procedure::options::ProcedureConfig;
|
||||
use common_procedure::ProcedureManagerRef;
|
||||
use common_query::Output;
|
||||
use common_recordbatch::error::StreamTimeoutSnafu;
|
||||
use common_recordbatch::RecordBatchStreamWrapper;
|
||||
use common_telemetry::{debug, error, info, tracing};
|
||||
use datafusion_expr::LogicalPlan;
|
||||
use futures::{Stream, StreamExt};
|
||||
use log_store::raft_engine::RaftEngineBackend;
|
||||
use operator::delete::DeleterRef;
|
||||
use operator::insert::InserterRef;
|
||||
@@ -65,20 +70,21 @@ use servers::interceptor::{
|
||||
};
|
||||
use servers::prometheus_handler::PrometheusHandler;
|
||||
use servers::query_handler::sql::SqlQueryHandler;
|
||||
use session::context::QueryContextRef;
|
||||
use session::context::{Channel, QueryContextRef};
|
||||
use session::table_name::table_idents_to_full_name;
|
||||
use snafu::prelude::*;
|
||||
use sql::dialect::Dialect;
|
||||
use sql::parser::{ParseOptions, ParserContext};
|
||||
use sql::statements::copy::{CopyDatabase, CopyTable};
|
||||
use sql::statements::statement::Statement;
|
||||
use sql::statements::tql::Tql;
|
||||
use sqlparser::ast::ObjectName;
|
||||
pub use standalone::StandaloneDatanodeManager;
|
||||
|
||||
use crate::error::{
|
||||
self, Error, ExecLogicalPlanSnafu, ExecutePromqlSnafu, ExternalSnafu, InvalidSqlSnafu,
|
||||
ParseSqlSnafu, PermissionSnafu, PlanStatementSnafu, Result, SqlExecInterceptedSnafu,
|
||||
TableOperationSnafu,
|
||||
StatementTimeoutSnafu, TableOperationSnafu,
|
||||
};
|
||||
use crate::limiter::LimiterRef;
|
||||
use crate::slow_query_recorder::SlowQueryRecorder;
|
||||
@@ -188,56 +194,7 @@ impl Instance {
|
||||
Some(query_ctx.process_id()),
|
||||
);
|
||||
|
||||
let query_fut = async {
|
||||
match stmt {
|
||||
Statement::Query(_) | Statement::Explain(_) | Statement::Delete(_) => {
|
||||
// TODO: remove this when format is supported in datafusion
|
||||
if let Statement::Explain(explain) = &stmt {
|
||||
if let Some(format) = explain.format() {
|
||||
query_ctx.set_explain_format(format.to_string());
|
||||
}
|
||||
}
|
||||
|
||||
let stmt = QueryStatement::Sql(stmt);
|
||||
let plan = self
|
||||
.statement_executor
|
||||
.plan(&stmt, query_ctx.clone())
|
||||
.await?;
|
||||
|
||||
let QueryStatement::Sql(stmt) = stmt else {
|
||||
unreachable!()
|
||||
};
|
||||
query_interceptor.pre_execute(&stmt, Some(&plan), query_ctx.clone())?;
|
||||
self.statement_executor
|
||||
.exec_plan(plan, query_ctx)
|
||||
.await
|
||||
.context(TableOperationSnafu)
|
||||
}
|
||||
Statement::Tql(tql) => {
|
||||
let plan = self
|
||||
.statement_executor
|
||||
.plan_tql(tql.clone(), &query_ctx)
|
||||
.await?;
|
||||
|
||||
query_interceptor.pre_execute(
|
||||
&Statement::Tql(tql),
|
||||
Some(&plan),
|
||||
query_ctx.clone(),
|
||||
)?;
|
||||
self.statement_executor
|
||||
.exec_plan(plan, query_ctx)
|
||||
.await
|
||||
.context(TableOperationSnafu)
|
||||
}
|
||||
_ => {
|
||||
query_interceptor.pre_execute(&stmt, None, query_ctx.clone())?;
|
||||
self.statement_executor
|
||||
.execute_sql(stmt, query_ctx)
|
||||
.await
|
||||
.context(TableOperationSnafu)
|
||||
}
|
||||
}
|
||||
};
|
||||
let query_fut = self.exec_statement_with_timeout(stmt, query_ctx, query_interceptor);
|
||||
|
||||
CancellableFuture::new(query_fut, ticket.cancellation_handle.clone())
|
||||
.await
|
||||
@@ -254,6 +211,153 @@ impl Instance {
|
||||
Output { data, meta }
|
||||
})
|
||||
}
|
||||
|
||||
async fn exec_statement_with_timeout(
|
||||
&self,
|
||||
stmt: Statement,
|
||||
query_ctx: QueryContextRef,
|
||||
query_interceptor: Option<&SqlQueryInterceptorRef<Error>>,
|
||||
) -> Result<Output> {
|
||||
let timeout = derive_timeout(&stmt, &query_ctx);
|
||||
match timeout {
|
||||
Some(timeout) => {
|
||||
let start = tokio::time::Instant::now();
|
||||
let output = tokio::time::timeout(
|
||||
timeout,
|
||||
self.exec_statement(stmt, query_ctx, query_interceptor),
|
||||
)
|
||||
.await
|
||||
.map_err(|_| StatementTimeoutSnafu.build())??;
|
||||
// compute remaining timeout
|
||||
let remaining_timeout = timeout.checked_sub(start.elapsed()).unwrap_or_default();
|
||||
attach_timeout(output, remaining_timeout)
|
||||
}
|
||||
None => {
|
||||
self.exec_statement(stmt, query_ctx, query_interceptor)
|
||||
.await
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
async fn exec_statement(
|
||||
&self,
|
||||
stmt: Statement,
|
||||
query_ctx: QueryContextRef,
|
||||
query_interceptor: Option<&SqlQueryInterceptorRef<Error>>,
|
||||
) -> Result<Output> {
|
||||
match stmt {
|
||||
Statement::Query(_) | Statement::Explain(_) | Statement::Delete(_) => {
|
||||
// TODO: remove this when format is supported in datafusion
|
||||
if let Statement::Explain(explain) = &stmt {
|
||||
if let Some(format) = explain.format() {
|
||||
query_ctx.set_explain_format(format.to_string());
|
||||
}
|
||||
}
|
||||
|
||||
self.plan_and_exec_sql(stmt, &query_ctx, query_interceptor)
|
||||
.await
|
||||
}
|
||||
Statement::Tql(tql) => {
|
||||
self.plan_and_exec_tql(&query_ctx, query_interceptor, tql)
|
||||
.await
|
||||
}
|
||||
_ => {
|
||||
query_interceptor.pre_execute(&stmt, None, query_ctx.clone())?;
|
||||
self.statement_executor
|
||||
.execute_sql(stmt, query_ctx)
|
||||
.await
|
||||
.context(TableOperationSnafu)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
async fn plan_and_exec_sql(
|
||||
&self,
|
||||
stmt: Statement,
|
||||
query_ctx: &QueryContextRef,
|
||||
query_interceptor: Option<&SqlQueryInterceptorRef<Error>>,
|
||||
) -> Result<Output> {
|
||||
let stmt = QueryStatement::Sql(stmt);
|
||||
let plan = self
|
||||
.statement_executor
|
||||
.plan(&stmt, query_ctx.clone())
|
||||
.await?;
|
||||
let QueryStatement::Sql(stmt) = stmt else {
|
||||
unreachable!()
|
||||
};
|
||||
query_interceptor.pre_execute(&stmt, Some(&plan), query_ctx.clone())?;
|
||||
self.statement_executor
|
||||
.exec_plan(plan, query_ctx.clone())
|
||||
.await
|
||||
.context(TableOperationSnafu)
|
||||
}
|
||||
|
||||
async fn plan_and_exec_tql(
|
||||
&self,
|
||||
query_ctx: &QueryContextRef,
|
||||
query_interceptor: Option<&SqlQueryInterceptorRef<Error>>,
|
||||
tql: Tql,
|
||||
) -> Result<Output> {
|
||||
let plan = self
|
||||
.statement_executor
|
||||
.plan_tql(tql.clone(), query_ctx)
|
||||
.await?;
|
||||
query_interceptor.pre_execute(&Statement::Tql(tql), Some(&plan), query_ctx.clone())?;
|
||||
self.statement_executor
|
||||
.exec_plan(plan, query_ctx.clone())
|
||||
.await
|
||||
.context(TableOperationSnafu)
|
||||
}
|
||||
}
|
||||
|
||||
/// If the relevant variables are set, the timeout is enforced for all PostgreSQL statements.
|
||||
/// For MySQL, it applies only to read-only statements.
|
||||
fn derive_timeout(stmt: &Statement, query_ctx: &QueryContextRef) -> Option<Duration> {
|
||||
let query_timeout = query_ctx.query_timeout()?;
|
||||
if query_timeout.is_zero() {
|
||||
return None;
|
||||
}
|
||||
match query_ctx.channel() {
|
||||
Channel::Mysql if stmt.is_readonly() => Some(query_timeout),
|
||||
Channel::Postgres => Some(query_timeout),
|
||||
_ => None,
|
||||
}
|
||||
}
|
||||
|
||||
fn attach_timeout(output: Output, mut timeout: Duration) -> Result<Output> {
|
||||
if timeout.is_zero() {
|
||||
return StatementTimeoutSnafu.fail();
|
||||
}
|
||||
|
||||
let output = match output.data {
|
||||
OutputData::AffectedRows(_) | OutputData::RecordBatches(_) => output,
|
||||
OutputData::Stream(mut stream) => {
|
||||
let schema = stream.schema();
|
||||
let s = Box::pin(stream! {
|
||||
let mut start = tokio::time::Instant::now();
|
||||
while let Some(item) = tokio::time::timeout(timeout, stream.next()).await.map_err(|_| StreamTimeoutSnafu.build())? {
|
||||
yield item;
|
||||
|
||||
let now = tokio::time::Instant::now();
|
||||
timeout = timeout.checked_sub(now - start).unwrap_or(Duration::ZERO);
|
||||
start = now;
|
||||
// tokio::time::timeout may not return an error immediately when timeout is 0.
|
||||
if timeout.is_zero() {
|
||||
StreamTimeoutSnafu.fail()?;
|
||||
}
|
||||
}
|
||||
}) as Pin<Box<dyn Stream<Item = _> + Send>>;
|
||||
let stream = RecordBatchStreamWrapper {
|
||||
schema,
|
||||
stream: s,
|
||||
output_ordering: None,
|
||||
metrics: Default::default(),
|
||||
};
|
||||
Output::new(OutputData::Stream(Box::pin(stream)), output.meta)
|
||||
}
|
||||
};
|
||||
|
||||
Ok(output)
|
||||
}
|
||||
|
||||
#[async_trait]
|
||||
|
||||
@@ -218,6 +218,7 @@ mod tests {
|
||||
let mut writer = Cursor::new(Vec::new());
|
||||
let mut creator = BloomFilterCreator::new(
|
||||
4,
|
||||
0.01,
|
||||
Arc::new(MockExternalTempFileProvider::new()),
|
||||
Arc::new(AtomicUsize::new(0)),
|
||||
None,
|
||||
|
||||
@@ -30,9 +30,6 @@ use crate::bloom_filter::SEED;
|
||||
use crate::external_provider::ExternalTempFileProvider;
|
||||
use crate::Bytes;
|
||||
|
||||
/// The false positive rate of the Bloom filter.
|
||||
pub const FALSE_POSITIVE_RATE: f64 = 0.01;
|
||||
|
||||
/// `BloomFilterCreator` is responsible for creating and managing bloom filters
|
||||
/// for a set of elements. It divides the rows into segments and creates
|
||||
/// bloom filters for each segment.
|
||||
@@ -79,6 +76,7 @@ impl BloomFilterCreator {
|
||||
/// `rows_per_segment` <= 0
|
||||
pub fn new(
|
||||
rows_per_segment: usize,
|
||||
false_positive_rate: f64,
|
||||
intermediate_provider: Arc<dyn ExternalTempFileProvider>,
|
||||
global_memory_usage: Arc<AtomicUsize>,
|
||||
global_memory_usage_threshold: Option<usize>,
|
||||
@@ -95,6 +93,7 @@ impl BloomFilterCreator {
|
||||
cur_seg_distinct_elems_mem_usage: 0,
|
||||
global_memory_usage: global_memory_usage.clone(),
|
||||
finalized_bloom_filters: FinalizedBloomFilterStorage::new(
|
||||
false_positive_rate,
|
||||
intermediate_provider,
|
||||
global_memory_usage,
|
||||
global_memory_usage_threshold,
|
||||
@@ -263,6 +262,7 @@ mod tests {
|
||||
let mut writer = Cursor::new(Vec::new());
|
||||
let mut creator = BloomFilterCreator::new(
|
||||
2,
|
||||
0.01,
|
||||
Arc::new(MockExternalTempFileProvider::new()),
|
||||
Arc::new(AtomicUsize::new(0)),
|
||||
None,
|
||||
@@ -337,6 +337,7 @@ mod tests {
|
||||
let mut writer = Cursor::new(Vec::new());
|
||||
let mut creator: BloomFilterCreator = BloomFilterCreator::new(
|
||||
2,
|
||||
0.01,
|
||||
Arc::new(MockExternalTempFileProvider::new()),
|
||||
Arc::new(AtomicUsize::new(0)),
|
||||
None,
|
||||
@@ -418,6 +419,7 @@ mod tests {
|
||||
let mut writer = Cursor::new(Vec::new());
|
||||
let mut creator = BloomFilterCreator::new(
|
||||
2,
|
||||
0.01,
|
||||
Arc::new(MockExternalTempFileProvider::new()),
|
||||
Arc::new(AtomicUsize::new(0)),
|
||||
None,
|
||||
|
||||
@@ -23,7 +23,7 @@ use futures::{stream, AsyncWriteExt, Stream};
|
||||
use snafu::ResultExt;
|
||||
|
||||
use crate::bloom_filter::creator::intermediate_codec::IntermediateBloomFilterCodecV1;
|
||||
use crate::bloom_filter::creator::{FALSE_POSITIVE_RATE, SEED};
|
||||
use crate::bloom_filter::creator::SEED;
|
||||
use crate::bloom_filter::error::{IntermediateSnafu, IoSnafu, Result};
|
||||
use crate::external_provider::ExternalTempFileProvider;
|
||||
use crate::Bytes;
|
||||
@@ -33,6 +33,9 @@ const MIN_MEMORY_USAGE_THRESHOLD: usize = 1024 * 1024; // 1MB
|
||||
|
||||
/// Storage for finalized Bloom filters.
|
||||
pub struct FinalizedBloomFilterStorage {
|
||||
/// The false positive rate of the Bloom filter.
|
||||
false_positive_rate: f64,
|
||||
|
||||
/// Indices of the segments in the sequence of finalized Bloom filters.
|
||||
segment_indices: Vec<usize>,
|
||||
|
||||
@@ -65,12 +68,14 @@ pub struct FinalizedBloomFilterStorage {
|
||||
impl FinalizedBloomFilterStorage {
|
||||
/// Creates a new `FinalizedBloomFilterStorage`.
|
||||
pub fn new(
|
||||
false_positive_rate: f64,
|
||||
intermediate_provider: Arc<dyn ExternalTempFileProvider>,
|
||||
global_memory_usage: Arc<AtomicUsize>,
|
||||
global_memory_usage_threshold: Option<usize>,
|
||||
) -> Self {
|
||||
let external_prefix = format!("intm-bloom-filters-{}", uuid::Uuid::new_v4());
|
||||
Self {
|
||||
false_positive_rate,
|
||||
segment_indices: Vec::new(),
|
||||
in_memory: Vec::new(),
|
||||
intermediate_file_id_counter: 0,
|
||||
@@ -96,7 +101,7 @@ impl FinalizedBloomFilterStorage {
|
||||
elems: impl IntoIterator<Item = Bytes>,
|
||||
element_count: usize,
|
||||
) -> Result<()> {
|
||||
let mut bf = BloomFilter::with_false_pos(FALSE_POSITIVE_RATE)
|
||||
let mut bf = BloomFilter::with_false_pos(self.false_positive_rate)
|
||||
.seed(&SEED)
|
||||
.expected_items(element_count);
|
||||
for elem in elems.into_iter() {
|
||||
@@ -284,6 +289,7 @@ mod tests {
|
||||
let global_memory_usage_threshold = Some(1024 * 1024); // 1MB
|
||||
let provider = Arc::new(mock_provider);
|
||||
let mut storage = FinalizedBloomFilterStorage::new(
|
||||
0.01,
|
||||
provider,
|
||||
global_memory_usage.clone(),
|
||||
global_memory_usage_threshold,
|
||||
@@ -340,6 +346,7 @@ mod tests {
|
||||
let global_memory_usage_threshold = Some(1024 * 1024); // 1MB
|
||||
let provider = Arc::new(mock_provider);
|
||||
let mut storage = FinalizedBloomFilterStorage::new(
|
||||
0.01,
|
||||
provider,
|
||||
global_memory_usage.clone(),
|
||||
global_memory_usage_threshold,
|
||||
|
||||
@@ -222,6 +222,7 @@ mod tests {
|
||||
let mut writer = Cursor::new(vec![]);
|
||||
let mut creator = BloomFilterCreator::new(
|
||||
2,
|
||||
0.01,
|
||||
Arc::new(MockExternalTempFileProvider::new()),
|
||||
Arc::new(AtomicUsize::new(0)),
|
||||
None,
|
||||
|
||||
@@ -45,6 +45,7 @@ impl BloomFilterFulltextIndexCreator {
|
||||
pub fn new(
|
||||
config: Config,
|
||||
rows_per_segment: usize,
|
||||
false_positive_rate: f64,
|
||||
intermediate_provider: Arc<dyn ExternalTempFileProvider>,
|
||||
global_memory_usage: Arc<AtomicUsize>,
|
||||
global_memory_usage_threshold: Option<usize>,
|
||||
@@ -57,6 +58,7 @@ impl BloomFilterFulltextIndexCreator {
|
||||
|
||||
let inner = BloomFilterCreator::new(
|
||||
rows_per_segment,
|
||||
false_positive_rate,
|
||||
intermediate_provider,
|
||||
global_memory_usage,
|
||||
global_memory_usage_threshold,
|
||||
|
||||
@@ -353,30 +353,28 @@ impl MetasrvBuilder {
|
||||
|
||||
let leader_region_registry = Arc::new(LeaderRegionRegistry::default());
|
||||
|
||||
let ddl_context = DdlContext {
|
||||
node_manager,
|
||||
cache_invalidator: cache_invalidator.clone(),
|
||||
memory_region_keeper: memory_region_keeper.clone(),
|
||||
leader_region_registry: leader_region_registry.clone(),
|
||||
table_metadata_manager: table_metadata_manager.clone(),
|
||||
table_metadata_allocator: table_metadata_allocator.clone(),
|
||||
flow_metadata_manager: flow_metadata_manager.clone(),
|
||||
flow_metadata_allocator: flow_metadata_allocator.clone(),
|
||||
region_failure_detector_controller,
|
||||
};
|
||||
let procedure_manager_c = procedure_manager.clone();
|
||||
let ddl_manager = DdlManager::try_new(ddl_context, procedure_manager_c, true)
|
||||
.context(error::InitDdlManagerSnafu)?;
|
||||
#[cfg(feature = "enterprise")]
|
||||
let trigger_ddl_manager = plugins
|
||||
.as_ref()
|
||||
.and_then(|plugins| plugins.get::<common_meta::ddl_manager::TriggerDdlManagerRef>());
|
||||
let ddl_manager = Arc::new(
|
||||
DdlManager::try_new(
|
||||
DdlContext {
|
||||
node_manager,
|
||||
cache_invalidator: cache_invalidator.clone(),
|
||||
memory_region_keeper: memory_region_keeper.clone(),
|
||||
leader_region_registry: leader_region_registry.clone(),
|
||||
table_metadata_manager: table_metadata_manager.clone(),
|
||||
table_metadata_allocator: table_metadata_allocator.clone(),
|
||||
flow_metadata_manager: flow_metadata_manager.clone(),
|
||||
flow_metadata_allocator: flow_metadata_allocator.clone(),
|
||||
region_failure_detector_controller,
|
||||
},
|
||||
procedure_manager.clone(),
|
||||
true,
|
||||
#[cfg(feature = "enterprise")]
|
||||
trigger_ddl_manager,
|
||||
)
|
||||
.context(error::InitDdlManagerSnafu)?,
|
||||
);
|
||||
let ddl_manager = {
|
||||
let trigger_ddl_manager = plugins.as_ref().and_then(|plugins| {
|
||||
plugins.get::<common_meta::ddl_manager::TriggerDdlManagerRef>()
|
||||
});
|
||||
ddl_manager.with_trigger_ddl_manager(trigger_ddl_manager)
|
||||
};
|
||||
let ddl_manager = Arc::new(ddl_manager);
|
||||
|
||||
// remote WAL prune ticker and manager
|
||||
let wal_prune_ticker = if is_remote_wal && options.wal.enable_active_wal_pruning() {
|
||||
|
||||
@@ -145,12 +145,19 @@ impl DataRegion {
|
||||
IndexOptions::Inverted => {
|
||||
c.column_schema.set_inverted_index(true);
|
||||
}
|
||||
IndexOptions::Skipping { granularity } => {
|
||||
IndexOptions::Skipping {
|
||||
granularity,
|
||||
false_positive_rate,
|
||||
} => {
|
||||
c.column_schema
|
||||
.set_skipping_options(&SkippingIndexOptions {
|
||||
granularity,
|
||||
index_type: SkippingIndexType::BloomFilter,
|
||||
})
|
||||
.set_skipping_options(
|
||||
&SkippingIndexOptions::new(
|
||||
granularity,
|
||||
false_positive_rate,
|
||||
SkippingIndexType::BloomFilter,
|
||||
)
|
||||
.context(SetSkippingIndexOptionSnafu)?,
|
||||
)
|
||||
.context(SetSkippingIndexOptionSnafu)?;
|
||||
}
|
||||
}
|
||||
|
||||
@@ -55,6 +55,7 @@ use crate::utils::{
|
||||
};
|
||||
|
||||
const DEFAULT_TABLE_ID_SKIPPING_INDEX_GRANULARITY: u32 = 1024;
|
||||
const DEFAULT_TABLE_ID_SKIPPING_INDEX_FALSE_POSITIVE_RATE: f64 = 0.01;
|
||||
|
||||
impl MetricEngineInner {
|
||||
pub async fn create_regions(
|
||||
@@ -542,10 +543,11 @@ impl MetricEngineInner {
|
||||
ConcreteDataType::uint32_datatype(),
|
||||
false,
|
||||
)
|
||||
.with_skipping_options(SkippingIndexOptions {
|
||||
granularity: DEFAULT_TABLE_ID_SKIPPING_INDEX_GRANULARITY,
|
||||
index_type: datatypes::schema::SkippingIndexType::BloomFilter,
|
||||
})
|
||||
.with_skipping_options(SkippingIndexOptions::new_unchecked(
|
||||
DEFAULT_TABLE_ID_SKIPPING_INDEX_GRANULARITY,
|
||||
DEFAULT_TABLE_ID_SKIPPING_INDEX_FALSE_POSITIVE_RATE,
|
||||
datatypes::schema::SkippingIndexType::BloomFilter,
|
||||
))
|
||||
.unwrap(),
|
||||
};
|
||||
let tsid_col = ColumnMetadata {
|
||||
|
||||
@@ -17,6 +17,8 @@
|
||||
use std::collections::HashMap;
|
||||
|
||||
use store_api::metric_engine_consts::{
|
||||
METRIC_ENGINE_INDEX_SKIPPING_INDEX_FALSE_POSITIVE_RATE_OPTION,
|
||||
METRIC_ENGINE_INDEX_SKIPPING_INDEX_FALSE_POSITIVE_RATE_OPTION_DEFAULT,
|
||||
METRIC_ENGINE_INDEX_SKIPPING_INDEX_GRANULARITY_OPTION,
|
||||
METRIC_ENGINE_INDEX_SKIPPING_INDEX_GRANULARITY_OPTION_DEFAULT, METRIC_ENGINE_INDEX_TYPE_OPTION,
|
||||
};
|
||||
@@ -31,19 +33,20 @@ use crate::error::{Error, ParseRegionOptionsSnafu, Result};
|
||||
const SEG_ROW_COUNT_FOR_DATA_REGION: u32 = 256;
|
||||
|
||||
/// Physical region options.
|
||||
#[derive(Debug, Clone, Copy, PartialEq, Eq)]
|
||||
#[derive(Debug, Clone, Copy, PartialEq)]
|
||||
pub struct PhysicalRegionOptions {
|
||||
pub index: IndexOptions,
|
||||
}
|
||||
|
||||
/// Index options for auto created columns
|
||||
#[derive(Debug, Clone, Copy, Default, PartialEq, Eq)]
|
||||
#[derive(Debug, Clone, Copy, Default, PartialEq)]
|
||||
pub enum IndexOptions {
|
||||
#[default]
|
||||
None,
|
||||
Inverted,
|
||||
Skipping {
|
||||
granularity: u32,
|
||||
false_positive_rate: f64,
|
||||
},
|
||||
}
|
||||
|
||||
@@ -54,6 +57,7 @@ pub fn set_data_region_options(
|
||||
) {
|
||||
options.remove(METRIC_ENGINE_INDEX_TYPE_OPTION);
|
||||
options.remove(METRIC_ENGINE_INDEX_SKIPPING_INDEX_GRANULARITY_OPTION);
|
||||
options.remove(METRIC_ENGINE_INDEX_SKIPPING_INDEX_FALSE_POSITIVE_RATE_OPTION);
|
||||
options.insert(
|
||||
"index.inverted_index.segment_row_count".to_string(),
|
||||
SEG_ROW_COUNT_FOR_DATA_REGION.to_string(),
|
||||
@@ -93,7 +97,23 @@ impl TryFrom<&HashMap<String, String>> for PhysicalRegionOptions {
|
||||
})
|
||||
},
|
||||
)?;
|
||||
Ok(IndexOptions::Skipping { granularity })
|
||||
let false_positive_rate = value
|
||||
.get(METRIC_ENGINE_INDEX_SKIPPING_INDEX_FALSE_POSITIVE_RATE_OPTION)
|
||||
.map_or(
|
||||
Ok(METRIC_ENGINE_INDEX_SKIPPING_INDEX_FALSE_POSITIVE_RATE_OPTION_DEFAULT),
|
||||
|f| {
|
||||
f.parse().ok().filter(|f| *f > 0.0 && *f <= 1.0).ok_or(
|
||||
ParseRegionOptionsSnafu {
|
||||
reason: format!("Invalid false positive rate: {}", f),
|
||||
}
|
||||
.build(),
|
||||
)
|
||||
},
|
||||
)?;
|
||||
Ok(IndexOptions::Skipping {
|
||||
granularity,
|
||||
false_positive_rate,
|
||||
})
|
||||
}
|
||||
Some(index_type) => ParseRegionOptionsSnafu {
|
||||
reason: format!("Invalid index type: {}", index_type),
|
||||
@@ -121,11 +141,16 @@ mod tests {
|
||||
METRIC_ENGINE_INDEX_SKIPPING_INDEX_GRANULARITY_OPTION.to_string(),
|
||||
"102400".to_string(),
|
||||
);
|
||||
options.insert(
|
||||
METRIC_ENGINE_INDEX_SKIPPING_INDEX_FALSE_POSITIVE_RATE_OPTION.to_string(),
|
||||
"0.01".to_string(),
|
||||
);
|
||||
set_data_region_options(&mut options, false);
|
||||
|
||||
for key in [
|
||||
METRIC_ENGINE_INDEX_TYPE_OPTION,
|
||||
METRIC_ENGINE_INDEX_SKIPPING_INDEX_GRANULARITY_OPTION,
|
||||
METRIC_ENGINE_INDEX_SKIPPING_INDEX_FALSE_POSITIVE_RATE_OPTION,
|
||||
] {
|
||||
assert_eq!(options.get(key), None);
|
||||
}
|
||||
@@ -154,11 +179,16 @@ mod tests {
|
||||
METRIC_ENGINE_INDEX_SKIPPING_INDEX_GRANULARITY_OPTION.to_string(),
|
||||
"102400".to_string(),
|
||||
);
|
||||
options.insert(
|
||||
METRIC_ENGINE_INDEX_SKIPPING_INDEX_FALSE_POSITIVE_RATE_OPTION.to_string(),
|
||||
"0.01".to_string(),
|
||||
);
|
||||
let physical_region_options = PhysicalRegionOptions::try_from(&options).unwrap();
|
||||
assert_eq!(
|
||||
physical_region_options.index,
|
||||
IndexOptions::Skipping {
|
||||
granularity: 102400
|
||||
granularity: 102400,
|
||||
false_positive_rate: 0.01,
|
||||
}
|
||||
);
|
||||
}
|
||||
|
||||
@@ -59,7 +59,7 @@ impl TestEnv {
|
||||
|
||||
/// Returns a new env with specific `prefix` and `config` for test.
|
||||
pub async fn with_prefix_and_config(prefix: &str, config: EngineConfig) -> Self {
|
||||
let mut mito_env = MitoTestEnv::with_prefix(prefix);
|
||||
let mut mito_env = MitoTestEnv::with_prefix(prefix).await;
|
||||
let mito = mito_env.create_engine(MitoConfig::default()).await;
|
||||
let metric = MetricEngine::try_new(mito.clone(), config).unwrap();
|
||||
Self {
|
||||
|
||||
2
src/mito2/src/cache/index/inverted_index.rs
vendored
2
src/mito2/src/cache/index/inverted_index.rs
vendored
@@ -245,7 +245,7 @@ mod test {
|
||||
let blob = create_inverted_index_blob().await;
|
||||
|
||||
// Init a test range reader in local fs.
|
||||
let mut env = TestEnv::new();
|
||||
let mut env = TestEnv::new().await;
|
||||
let file_size = blob.len() as u64;
|
||||
let store = env.init_object_store_manager();
|
||||
let temp_path = "data";
|
||||
|
||||
10
src/mito2/src/cache/index/result_cache.rs
vendored
10
src/mito2/src/cache/index/result_cache.rs
vendored
@@ -31,6 +31,11 @@ use crate::sst::parquet::row_selection::RowGroupSelection;
|
||||
const INDEX_RESULT_TYPE: &str = "index_result";
|
||||
|
||||
/// Cache for storing index query results.
|
||||
///
|
||||
/// The `RowGroupSelection` is a collection of row groups that match the predicate.
|
||||
///
|
||||
/// Row groups can be partially searched. Row groups that not contained in `RowGroupSelection` are not searched.
|
||||
/// User can retrieve the partial results and handle uncontained row groups required by the predicate subsequently.
|
||||
pub struct IndexResultCache {
|
||||
cache: Cache<(PredicateKey, FileId), Arc<RowGroupSelection>>,
|
||||
}
|
||||
@@ -64,6 +69,8 @@ impl IndexResultCache {
|
||||
}
|
||||
|
||||
/// Puts a query result into the cache.
|
||||
///
|
||||
/// Allow user to put a partial result (not containing all row groups) into the cache.
|
||||
pub fn put(&self, key: PredicateKey, file_id: FileId, result: Arc<RowGroupSelection>) {
|
||||
let key = (key, file_id);
|
||||
let size = Self::index_result_cache_weight(&key, &result);
|
||||
@@ -74,6 +81,9 @@ impl IndexResultCache {
|
||||
}
|
||||
|
||||
/// Gets a query result from the cache.
|
||||
///
|
||||
/// Note: the returned `RowGroupSelection` only contains the row groups that are searched.
|
||||
/// Caller should handle the uncontained row groups required by the predicate subsequently.
|
||||
pub fn get(&self, key: &PredicateKey, file_id: FileId) -> Option<Arc<RowGroupSelection>> {
|
||||
let res = self.cache.get(&(key.clone(), file_id));
|
||||
if res.is_some() {
|
||||
|
||||
6
src/mito2/src/cache/write_cache.rs
vendored
6
src/mito2/src/cache/write_cache.rs
vendored
@@ -449,7 +449,7 @@ mod tests {
|
||||
async fn test_write_and_upload_sst() {
|
||||
// TODO(QuenKar): maybe find a way to create some object server for testing,
|
||||
// and now just use local file system to mock.
|
||||
let mut env = TestEnv::new();
|
||||
let mut env = TestEnv::new().await;
|
||||
let mock_store = env.init_object_store_manager();
|
||||
let path_provider = RegionFilePathFactory::new("test".to_string());
|
||||
|
||||
@@ -537,7 +537,7 @@ mod tests {
|
||||
#[tokio::test]
|
||||
async fn test_read_metadata_from_write_cache() {
|
||||
common_telemetry::init_default_ut_logging();
|
||||
let mut env = TestEnv::new();
|
||||
let mut env = TestEnv::new().await;
|
||||
let data_home = env.data_home().display().to_string();
|
||||
let mock_store = env.init_object_store_manager();
|
||||
|
||||
@@ -606,7 +606,7 @@ mod tests {
|
||||
#[tokio::test]
|
||||
async fn test_write_cache_clean_tmp_files() {
|
||||
common_telemetry::init_default_ut_logging();
|
||||
let mut env = TestEnv::new();
|
||||
let mut env = TestEnv::new().await;
|
||||
let data_home = env.data_home().display().to_string();
|
||||
let mock_store = env.init_object_store_manager();
|
||||
|
||||
|
||||
@@ -84,12 +84,14 @@ fn alter_column_fulltext_options() -> RegionAlterRequest {
|
||||
kind: AlterKind::SetIndex {
|
||||
options: ApiSetIndexOptions::Fulltext {
|
||||
column_name: "tag_0".to_string(),
|
||||
options: FulltextOptions {
|
||||
enable: true,
|
||||
analyzer: FulltextAnalyzer::English,
|
||||
case_sensitive: false,
|
||||
backend: FulltextBackend::Bloom,
|
||||
},
|
||||
options: FulltextOptions::new_unchecked(
|
||||
true,
|
||||
FulltextAnalyzer::English,
|
||||
false,
|
||||
FulltextBackend::Bloom,
|
||||
1000,
|
||||
0.01,
|
||||
),
|
||||
},
|
||||
},
|
||||
}
|
||||
@@ -115,7 +117,7 @@ fn check_region_version(
|
||||
async fn test_alter_region() {
|
||||
common_telemetry::init_default_ut_logging();
|
||||
|
||||
let mut env = TestEnv::new();
|
||||
let mut env = TestEnv::new().await;
|
||||
let engine = env.create_engine(MitoConfig::default()).await;
|
||||
|
||||
let region_id = RegionId::new(1, 1);
|
||||
@@ -211,7 +213,7 @@ fn build_rows_for_tags(
|
||||
|
||||
#[tokio::test]
|
||||
async fn test_put_after_alter() {
|
||||
let mut env = TestEnv::new();
|
||||
let mut env = TestEnv::new().await;
|
||||
let engine = env.create_engine(MitoConfig::default()).await;
|
||||
let region_id = RegionId::new(1, 1);
|
||||
let request = CreateRequestBuilder::new().build();
|
||||
@@ -316,7 +318,7 @@ async fn test_put_after_alter() {
|
||||
async fn test_alter_region_retry() {
|
||||
common_telemetry::init_default_ut_logging();
|
||||
|
||||
let mut env = TestEnv::new();
|
||||
let mut env = TestEnv::new().await;
|
||||
let engine = env.create_engine(MitoConfig::default()).await;
|
||||
|
||||
let region_id = RegionId::new(1, 1);
|
||||
@@ -374,7 +376,7 @@ async fn test_alter_region_retry() {
|
||||
async fn test_alter_on_flushing() {
|
||||
common_telemetry::init_default_ut_logging();
|
||||
|
||||
let mut env = TestEnv::new();
|
||||
let mut env = TestEnv::new().await;
|
||||
let listener = Arc::new(AlterFlushListener::default());
|
||||
let engine = env
|
||||
.create_engine_with(MitoConfig::default(), None, Some(listener.clone()))
|
||||
@@ -478,7 +480,7 @@ async fn test_alter_on_flushing() {
|
||||
async fn test_alter_column_fulltext_options() {
|
||||
common_telemetry::init_default_ut_logging();
|
||||
|
||||
let mut env = TestEnv::new();
|
||||
let mut env = TestEnv::new().await;
|
||||
let listener = Arc::new(AlterFlushListener::default());
|
||||
let engine = env
|
||||
.create_engine_with(MitoConfig::default(), None, Some(listener.clone()))
|
||||
@@ -553,12 +555,14 @@ async fn test_alter_column_fulltext_options() {
|
||||
// Wait for the write job.
|
||||
alter_job.await.unwrap();
|
||||
|
||||
let expect_fulltext_options = FulltextOptions {
|
||||
enable: true,
|
||||
analyzer: FulltextAnalyzer::English,
|
||||
case_sensitive: false,
|
||||
backend: FulltextBackend::Bloom,
|
||||
};
|
||||
let expect_fulltext_options = FulltextOptions::new_unchecked(
|
||||
true,
|
||||
FulltextAnalyzer::English,
|
||||
false,
|
||||
FulltextBackend::Bloom,
|
||||
1000,
|
||||
0.01,
|
||||
);
|
||||
let check_fulltext_options = |engine: &MitoEngine, expected: &FulltextOptions| {
|
||||
let current_fulltext_options = engine
|
||||
.get_region(region_id)
|
||||
@@ -597,7 +601,7 @@ async fn test_alter_column_fulltext_options() {
|
||||
async fn test_alter_column_set_inverted_index() {
|
||||
common_telemetry::init_default_ut_logging();
|
||||
|
||||
let mut env = TestEnv::new();
|
||||
let mut env = TestEnv::new().await;
|
||||
let listener = Arc::new(AlterFlushListener::default());
|
||||
let engine = env
|
||||
.create_engine_with(MitoConfig::default(), None, Some(listener.clone()))
|
||||
@@ -707,7 +711,7 @@ async fn test_alter_column_set_inverted_index() {
|
||||
async fn test_alter_region_ttl_options() {
|
||||
common_telemetry::init_default_ut_logging();
|
||||
|
||||
let mut env = TestEnv::new();
|
||||
let mut env = TestEnv::new().await;
|
||||
let listener = Arc::new(AlterFlushListener::default());
|
||||
let engine = env
|
||||
.create_engine_with(MitoConfig::default(), None, Some(listener.clone()))
|
||||
@@ -757,7 +761,7 @@ async fn test_alter_region_ttl_options() {
|
||||
async fn test_write_stall_on_altering() {
|
||||
common_telemetry::init_default_ut_logging();
|
||||
|
||||
let mut env = TestEnv::new();
|
||||
let mut env = TestEnv::new().await;
|
||||
let listener = Arc::new(NotifyRegionChangeResultListener::default());
|
||||
let engine = env
|
||||
.create_engine_with(MitoConfig::default(), None, Some(listener.clone()))
|
||||
|
||||
@@ -31,7 +31,7 @@ use crate::test_util::{
|
||||
async fn test_append_mode_write_query() {
|
||||
common_telemetry::init_default_ut_logging();
|
||||
|
||||
let mut env = TestEnv::new();
|
||||
let mut env = TestEnv::new().await;
|
||||
let engine = env.create_engine(MitoConfig::default()).await;
|
||||
|
||||
let region_id = RegionId::new(1, 1);
|
||||
@@ -89,7 +89,7 @@ async fn test_append_mode_write_query() {
|
||||
|
||||
#[tokio::test]
|
||||
async fn test_append_mode_compaction() {
|
||||
let mut env = TestEnv::new();
|
||||
let mut env = TestEnv::new().await;
|
||||
let engine = env
|
||||
.create_engine(MitoConfig {
|
||||
..Default::default()
|
||||
|
||||
@@ -42,7 +42,7 @@ use crate::test_util::{
|
||||
|
||||
#[tokio::test]
|
||||
async fn test_engine_new_stop() {
|
||||
let mut env = TestEnv::with_prefix("engine-stop");
|
||||
let mut env = TestEnv::with_prefix("engine-stop").await;
|
||||
let engine = env.create_engine(MitoConfig::default()).await;
|
||||
|
||||
let region_id = RegionId::new(1, 1);
|
||||
@@ -69,7 +69,7 @@ async fn test_engine_new_stop() {
|
||||
|
||||
#[tokio::test]
|
||||
async fn test_write_to_region() {
|
||||
let mut env = TestEnv::with_prefix("write-to-region");
|
||||
let mut env = TestEnv::with_prefix("write-to-region").await;
|
||||
let engine = env.create_engine(MitoConfig::default()).await;
|
||||
|
||||
let region_id = RegionId::new(1, 1);
|
||||
@@ -97,7 +97,9 @@ async fn test_region_replay(factory: Option<LogStoreFactory>) {
|
||||
let Some(factory) = factory else {
|
||||
return;
|
||||
};
|
||||
let mut env = TestEnv::with_prefix("region-replay").with_log_store_factory(factory.clone());
|
||||
let mut env = TestEnv::with_prefix("region-replay")
|
||||
.await
|
||||
.with_log_store_factory(factory.clone());
|
||||
let engine = env.create_engine(MitoConfig::default()).await;
|
||||
|
||||
let region_id = RegionId::new(1, 1);
|
||||
@@ -173,7 +175,7 @@ async fn test_region_replay(factory: Option<LogStoreFactory>) {
|
||||
|
||||
#[tokio::test]
|
||||
async fn test_write_query_region() {
|
||||
let mut env = TestEnv::new();
|
||||
let mut env = TestEnv::new().await;
|
||||
let engine = env.create_engine(MitoConfig::default()).await;
|
||||
|
||||
let region_id = RegionId::new(1, 1);
|
||||
@@ -207,7 +209,7 @@ async fn test_write_query_region() {
|
||||
|
||||
#[tokio::test]
|
||||
async fn test_different_order() {
|
||||
let mut env = TestEnv::new();
|
||||
let mut env = TestEnv::new().await;
|
||||
let engine = env.create_engine(MitoConfig::default()).await;
|
||||
|
||||
let region_id = RegionId::new(1, 1);
|
||||
@@ -268,7 +270,7 @@ async fn test_different_order() {
|
||||
|
||||
#[tokio::test]
|
||||
async fn test_different_order_and_type() {
|
||||
let mut env = TestEnv::new();
|
||||
let mut env = TestEnv::new().await;
|
||||
let engine = env.create_engine(MitoConfig::default()).await;
|
||||
|
||||
let region_id = RegionId::new(1, 1);
|
||||
@@ -332,7 +334,7 @@ async fn test_different_order_and_type() {
|
||||
async fn test_put_delete() {
|
||||
common_telemetry::init_default_ut_logging();
|
||||
|
||||
let mut env = TestEnv::new();
|
||||
let mut env = TestEnv::new().await;
|
||||
let engine = env.create_engine(MitoConfig::default()).await;
|
||||
|
||||
let region_id = RegionId::new(1, 1);
|
||||
@@ -384,7 +386,7 @@ async fn test_put_delete() {
|
||||
|
||||
#[tokio::test]
|
||||
async fn test_delete_not_null_fields() {
|
||||
let mut env = TestEnv::new();
|
||||
let mut env = TestEnv::new().await;
|
||||
let engine = env.create_engine(MitoConfig::default()).await;
|
||||
|
||||
let region_id = RegionId::new(1, 1);
|
||||
@@ -433,7 +435,7 @@ async fn test_delete_not_null_fields() {
|
||||
|
||||
#[tokio::test]
|
||||
async fn test_put_overwrite() {
|
||||
let mut env = TestEnv::new();
|
||||
let mut env = TestEnv::new().await;
|
||||
let engine = env.create_engine(MitoConfig::default()).await;
|
||||
|
||||
let region_id = RegionId::new(1, 1);
|
||||
@@ -493,7 +495,7 @@ async fn test_put_overwrite() {
|
||||
|
||||
#[tokio::test]
|
||||
async fn test_absent_and_invalid_columns() {
|
||||
let mut env = TestEnv::new();
|
||||
let mut env = TestEnv::new().await;
|
||||
let engine = env.create_engine(MitoConfig::default()).await;
|
||||
|
||||
let region_id = RegionId::new(1, 1);
|
||||
@@ -541,7 +543,7 @@ async fn test_absent_and_invalid_columns() {
|
||||
|
||||
#[tokio::test]
|
||||
async fn test_region_usage() {
|
||||
let mut env = TestEnv::with_prefix("region_usage");
|
||||
let mut env = TestEnv::with_prefix("region_usage").await;
|
||||
let engine = env.create_engine(MitoConfig::default()).await;
|
||||
|
||||
let region_id = RegionId::new(1, 1);
|
||||
@@ -595,7 +597,7 @@ async fn test_region_usage() {
|
||||
async fn test_engine_with_write_cache() {
|
||||
common_telemetry::init_default_ut_logging();
|
||||
|
||||
let mut env = TestEnv::new();
|
||||
let mut env = TestEnv::new().await;
|
||||
let path = env.data_home().to_str().unwrap().to_string();
|
||||
let mito_config = MitoConfig::default().enable_write_cache(path, ReadableSize::mb(512), None);
|
||||
let engine = env.create_engine(mito_config).await;
|
||||
@@ -635,7 +637,7 @@ async fn test_engine_with_write_cache() {
|
||||
|
||||
#[tokio::test]
|
||||
async fn test_cache_null_primary_key() {
|
||||
let mut env = TestEnv::new();
|
||||
let mut env = TestEnv::new().await;
|
||||
let engine = env
|
||||
.create_engine(MitoConfig {
|
||||
vector_cache_size: ReadableSize::mb(32),
|
||||
|
||||
@@ -39,8 +39,9 @@ async fn test_batch_open(factory: Option<LogStoreFactory>) {
|
||||
let Some(factory) = factory else {
|
||||
return;
|
||||
};
|
||||
let mut env =
|
||||
TestEnv::with_prefix("open-batch-regions").with_log_store_factory(factory.clone());
|
||||
let mut env = TestEnv::with_prefix("open-batch-regions")
|
||||
.await
|
||||
.with_log_store_factory(factory.clone());
|
||||
let engine = env.create_engine(MitoConfig::default()).await;
|
||||
let topic = prepare_test_for_kafka_log_store(&factory).await;
|
||||
|
||||
@@ -160,8 +161,9 @@ async fn test_batch_open_err(factory: Option<LogStoreFactory>) {
|
||||
let Some(factory) = factory else {
|
||||
return;
|
||||
};
|
||||
let mut env =
|
||||
TestEnv::with_prefix("open-batch-regions-err").with_log_store_factory(factory.clone());
|
||||
let mut env = TestEnv::with_prefix("open-batch-regions-err")
|
||||
.await
|
||||
.with_log_store_factory(factory.clone());
|
||||
let engine = env.create_engine(MitoConfig::default()).await;
|
||||
let topic = prepare_test_for_kafka_log_store(&factory).await;
|
||||
let mut options = HashMap::new();
|
||||
|
||||
@@ -57,7 +57,9 @@ async fn test_catchup_with_last_entry_id(factory: Option<LogStoreFactory>) {
|
||||
return;
|
||||
};
|
||||
|
||||
let mut env = TestEnv::with_prefix("last_entry_id").with_log_store_factory(factory.clone());
|
||||
let mut env = TestEnv::with_prefix("last_entry_id")
|
||||
.await
|
||||
.with_log_store_factory(factory.clone());
|
||||
let topic = prepare_test_for_kafka_log_store(&factory).await;
|
||||
let leader_engine = env.create_engine(MitoConfig::default()).await;
|
||||
let follower_engine = env.create_follower_engine(MitoConfig::default()).await;
|
||||
@@ -175,8 +177,9 @@ async fn test_catchup_with_incorrect_last_entry_id(factory: Option<LogStoreFacto
|
||||
return;
|
||||
};
|
||||
|
||||
let mut env =
|
||||
TestEnv::with_prefix("incorrect_last_entry_id").with_log_store_factory(factory.clone());
|
||||
let mut env = TestEnv::with_prefix("incorrect_last_entry_id")
|
||||
.await
|
||||
.with_log_store_factory(factory.clone());
|
||||
let topic = prepare_test_for_kafka_log_store(&factory).await;
|
||||
let leader_engine = env.create_engine(MitoConfig::default()).await;
|
||||
let follower_engine = env.create_follower_engine(MitoConfig::default()).await;
|
||||
@@ -277,8 +280,9 @@ async fn test_catchup_without_last_entry_id(factory: Option<LogStoreFactory>) {
|
||||
return;
|
||||
};
|
||||
|
||||
let mut env =
|
||||
TestEnv::with_prefix("without_last_entry_id").with_log_store_factory(factory.clone());
|
||||
let mut env = TestEnv::with_prefix("without_last_entry_id")
|
||||
.await
|
||||
.with_log_store_factory(factory.clone());
|
||||
let topic = prepare_test_for_kafka_log_store(&factory).await;
|
||||
let leader_engine = env.create_engine(MitoConfig::default()).await;
|
||||
let follower_engine = env.create_follower_engine(MitoConfig::default()).await;
|
||||
@@ -380,8 +384,9 @@ async fn test_catchup_with_manifest_update(factory: Option<LogStoreFactory>) {
|
||||
return;
|
||||
};
|
||||
|
||||
let mut env =
|
||||
TestEnv::with_prefix("without_manifest_update").with_log_store_factory(factory.clone());
|
||||
let mut env = TestEnv::with_prefix("without_manifest_update")
|
||||
.await
|
||||
.with_log_store_factory(factory.clone());
|
||||
let topic = prepare_test_for_kafka_log_store(&factory).await;
|
||||
let leader_engine = env.create_engine(MitoConfig::default()).await;
|
||||
let follower_engine = env.create_follower_engine(MitoConfig::default()).await;
|
||||
@@ -545,7 +550,9 @@ async fn test_local_catchup(factory: Option<LogStoreFactory>) {
|
||||
return;
|
||||
};
|
||||
|
||||
let mut env = TestEnv::with_prefix("local_catchup").with_log_store_factory(factory.clone());
|
||||
let mut env = TestEnv::with_prefix("local_catchup")
|
||||
.await
|
||||
.with_log_store_factory(factory.clone());
|
||||
let leader_engine = env.create_engine(MitoConfig::default()).await;
|
||||
let Some(LogStoreImpl::RaftEngine(log_store)) = env.get_log_store() else {
|
||||
unreachable!()
|
||||
@@ -686,7 +693,7 @@ async fn test_local_catchup(factory: Option<LogStoreFactory>) {
|
||||
|
||||
#[tokio::test]
|
||||
async fn test_catchup_not_exist() {
|
||||
let mut env = TestEnv::new();
|
||||
let mut env = TestEnv::new().await;
|
||||
let engine = env.create_engine(MitoConfig::default()).await;
|
||||
|
||||
let non_exist_region_id = RegionId::new(1, 1);
|
||||
|
||||
@@ -21,7 +21,7 @@ use crate::test_util::{CreateRequestBuilder, TestEnv};
|
||||
|
||||
#[tokio::test]
|
||||
async fn test_engine_close_region() {
|
||||
let mut env = TestEnv::with_prefix("close");
|
||||
let mut env = TestEnv::with_prefix("close").await;
|
||||
let engine = env.create_engine(MitoConfig::default()).await;
|
||||
|
||||
let region_id = RegionId::new(1, 1);
|
||||
|
||||
@@ -136,7 +136,7 @@ async fn collect_stream_ts(stream: SendableRecordBatchStream) -> Vec<i64> {
|
||||
#[tokio::test]
|
||||
async fn test_compaction_region() {
|
||||
common_telemetry::init_default_ut_logging();
|
||||
let mut env = TestEnv::new();
|
||||
let mut env = TestEnv::new().await;
|
||||
let engine = env.create_engine(MitoConfig::default()).await;
|
||||
|
||||
let region_id = RegionId::new(1, 1);
|
||||
@@ -202,7 +202,7 @@ async fn test_compaction_region() {
|
||||
#[tokio::test]
|
||||
async fn test_infer_compaction_time_window() {
|
||||
common_telemetry::init_default_ut_logging();
|
||||
let mut env = TestEnv::new();
|
||||
let mut env = TestEnv::new().await;
|
||||
let engine = env.create_engine(MitoConfig::default()).await;
|
||||
|
||||
let region_id = RegionId::new(1, 1);
|
||||
@@ -341,7 +341,7 @@ async fn test_infer_compaction_time_window() {
|
||||
#[tokio::test]
|
||||
async fn test_compaction_overlapping_files() {
|
||||
common_telemetry::init_default_ut_logging();
|
||||
let mut env = TestEnv::new();
|
||||
let mut env = TestEnv::new().await;
|
||||
let engine = env.create_engine(MitoConfig::default()).await;
|
||||
|
||||
let region_id = RegionId::new(1, 1);
|
||||
@@ -402,7 +402,7 @@ async fn test_compaction_overlapping_files() {
|
||||
#[tokio::test]
|
||||
async fn test_compaction_region_with_overlapping() {
|
||||
common_telemetry::init_default_ut_logging();
|
||||
let mut env = TestEnv::new();
|
||||
let mut env = TestEnv::new().await;
|
||||
let engine = env.create_engine(MitoConfig::default()).await;
|
||||
let region_id = RegionId::new(1, 1);
|
||||
|
||||
@@ -450,7 +450,7 @@ async fn test_compaction_region_with_overlapping() {
|
||||
#[tokio::test]
|
||||
async fn test_compaction_region_with_overlapping_delete_all() {
|
||||
common_telemetry::init_default_ut_logging();
|
||||
let mut env = TestEnv::new();
|
||||
let mut env = TestEnv::new().await;
|
||||
let engine = env.create_engine(MitoConfig::default()).await;
|
||||
|
||||
let region_id = RegionId::new(1, 1);
|
||||
@@ -506,7 +506,7 @@ async fn test_compaction_region_with_overlapping_delete_all() {
|
||||
#[tokio::test]
|
||||
async fn test_readonly_during_compaction() {
|
||||
common_telemetry::init_default_ut_logging();
|
||||
let mut env = TestEnv::new();
|
||||
let mut env = TestEnv::new().await;
|
||||
let listener = Arc::new(CompactionListener::default());
|
||||
let engine = env
|
||||
.create_engine_with(
|
||||
@@ -590,7 +590,7 @@ async fn test_readonly_during_compaction() {
|
||||
#[tokio::test]
|
||||
async fn test_compaction_update_time_window() {
|
||||
common_telemetry::init_default_ut_logging();
|
||||
let mut env = TestEnv::new();
|
||||
let mut env = TestEnv::new().await;
|
||||
let engine = env.create_engine(MitoConfig::default()).await;
|
||||
|
||||
let region_id = RegionId::new(1, 1);
|
||||
@@ -686,7 +686,7 @@ async fn test_compaction_update_time_window() {
|
||||
#[tokio::test]
|
||||
async fn test_change_region_compaction_window() {
|
||||
common_telemetry::init_default_ut_logging();
|
||||
let mut env = TestEnv::new();
|
||||
let mut env = TestEnv::new().await;
|
||||
let engine = env.create_engine(MitoConfig::default()).await;
|
||||
|
||||
let region_id = RegionId::new(1, 1);
|
||||
@@ -811,7 +811,7 @@ async fn test_change_region_compaction_window() {
|
||||
#[tokio::test]
|
||||
async fn test_open_overwrite_compaction_window() {
|
||||
common_telemetry::init_default_ut_logging();
|
||||
let mut env = TestEnv::new();
|
||||
let mut env = TestEnv::new().await;
|
||||
let engine = env.create_engine(MitoConfig::default()).await;
|
||||
|
||||
let region_id = RegionId::new(1, 1);
|
||||
|
||||
@@ -26,7 +26,7 @@ use crate::test_util::{build_rows, put_rows, rows_schema, CreateRequestBuilder,
|
||||
|
||||
#[tokio::test]
|
||||
async fn test_engine_create_new_region() {
|
||||
let mut env = TestEnv::with_prefix("new-region");
|
||||
let mut env = TestEnv::with_prefix("new-region").await;
|
||||
let engine = env.create_engine(MitoConfig::default()).await;
|
||||
|
||||
let region_id = RegionId::new(1, 1);
|
||||
@@ -41,7 +41,7 @@ async fn test_engine_create_new_region() {
|
||||
|
||||
#[tokio::test]
|
||||
async fn test_engine_create_existing_region() {
|
||||
let mut env = TestEnv::with_prefix("create-existing");
|
||||
let mut env = TestEnv::with_prefix("create-existing").await;
|
||||
let engine = env.create_engine(MitoConfig::default()).await;
|
||||
|
||||
let region_id = RegionId::new(1, 1);
|
||||
@@ -61,7 +61,7 @@ async fn test_engine_create_existing_region() {
|
||||
#[tokio::test]
|
||||
async fn test_engine_create_close_create_region() {
|
||||
// This test will trigger create_or_open function.
|
||||
let mut env = TestEnv::with_prefix("create-close-create");
|
||||
let mut env = TestEnv::with_prefix("create-close-create").await;
|
||||
let engine = env.create_engine(MitoConfig::default()).await;
|
||||
|
||||
let region_id = RegionId::new(1, 1);
|
||||
@@ -91,7 +91,7 @@ async fn test_engine_create_close_create_region() {
|
||||
|
||||
#[tokio::test]
|
||||
async fn test_engine_create_with_different_id() {
|
||||
let mut env = TestEnv::new();
|
||||
let mut env = TestEnv::new().await;
|
||||
let engine = env.create_engine(MitoConfig::default()).await;
|
||||
|
||||
let region_id = RegionId::new(1, 1);
|
||||
@@ -110,7 +110,7 @@ async fn test_engine_create_with_different_id() {
|
||||
|
||||
#[tokio::test]
|
||||
async fn test_engine_create_with_different_schema() {
|
||||
let mut env = TestEnv::new();
|
||||
let mut env = TestEnv::new().await;
|
||||
let engine = env.create_engine(MitoConfig::default()).await;
|
||||
|
||||
let region_id = RegionId::new(1, 1);
|
||||
@@ -130,7 +130,7 @@ async fn test_engine_create_with_different_schema() {
|
||||
|
||||
#[tokio::test]
|
||||
async fn test_engine_create_with_different_primary_key() {
|
||||
let mut env = TestEnv::new();
|
||||
let mut env = TestEnv::new().await;
|
||||
let engine = env.create_engine(MitoConfig::default()).await;
|
||||
|
||||
let region_id = RegionId::new(1, 1);
|
||||
@@ -150,7 +150,7 @@ async fn test_engine_create_with_different_primary_key() {
|
||||
|
||||
#[tokio::test]
|
||||
async fn test_engine_create_with_options() {
|
||||
let mut env = TestEnv::new();
|
||||
let mut env = TestEnv::new().await;
|
||||
let engine = env.create_engine(MitoConfig::default()).await;
|
||||
|
||||
let region_id = RegionId::new(1, 1);
|
||||
@@ -172,7 +172,7 @@ async fn test_engine_create_with_options() {
|
||||
|
||||
#[tokio::test]
|
||||
async fn test_engine_create_with_custom_store() {
|
||||
let mut env = TestEnv::new();
|
||||
let mut env = TestEnv::new().await;
|
||||
let engine = env
|
||||
.create_engine_with_multiple_object_stores(MitoConfig::default(), None, None, &["Gcs"])
|
||||
.await;
|
||||
@@ -204,7 +204,7 @@ async fn test_engine_create_with_custom_store() {
|
||||
|
||||
#[tokio::test]
|
||||
async fn test_engine_create_with_memtable_opts() {
|
||||
let mut env = TestEnv::new();
|
||||
let mut env = TestEnv::new().await;
|
||||
let engine = env.create_engine(MitoConfig::default()).await;
|
||||
|
||||
let region_id = RegionId::new(1, 1);
|
||||
|
||||
@@ -35,7 +35,7 @@ use crate::worker::DROPPING_MARKER_FILE;
|
||||
async fn test_engine_drop_region() {
|
||||
common_telemetry::init_default_ut_logging();
|
||||
|
||||
let mut env = TestEnv::with_prefix("drop");
|
||||
let mut env = TestEnv::with_prefix("drop").await;
|
||||
let listener = Arc::new(DropListener::new(Duration::from_millis(100)));
|
||||
let engine = env
|
||||
.create_engine_with(MitoConfig::default(), None, Some(listener.clone()))
|
||||
@@ -143,7 +143,7 @@ async fn test_engine_drop_region_for_custom_store() {
|
||||
put_rows(engine, region_id, rows).await;
|
||||
flush_region(engine, region_id, None).await;
|
||||
}
|
||||
let mut env = TestEnv::with_prefix("drop");
|
||||
let mut env = TestEnv::with_prefix("drop").await;
|
||||
let listener = Arc::new(DropListener::new(Duration::from_millis(100)));
|
||||
let engine = env
|
||||
.create_engine_with_multiple_object_stores(
|
||||
|
||||
@@ -33,7 +33,7 @@ use crate::test_util::{CreateRequestBuilder, TestEnv};
|
||||
|
||||
#[tokio::test]
|
||||
async fn test_edit_region_schedule_compaction() {
|
||||
let mut env = TestEnv::new();
|
||||
let mut env = TestEnv::new().await;
|
||||
|
||||
struct EditRegionListener {
|
||||
tx: Mutex<Option<oneshot::Sender<RegionId>>>,
|
||||
@@ -122,7 +122,7 @@ async fn test_edit_region_schedule_compaction() {
|
||||
|
||||
#[tokio::test]
|
||||
async fn test_edit_region_fill_cache() {
|
||||
let mut env = TestEnv::new();
|
||||
let mut env = TestEnv::new().await;
|
||||
|
||||
struct EditRegionListener {
|
||||
tx: Mutex<Option<oneshot::Sender<FileId>>>,
|
||||
@@ -241,7 +241,7 @@ async fn test_edit_region_concurrently() {
|
||||
}
|
||||
}
|
||||
|
||||
let mut env = TestEnv::new();
|
||||
let mut env = TestEnv::new().await;
|
||||
let engine = env
|
||||
.create_engine(MitoConfig {
|
||||
// Suppress the compaction to not impede the speed of this kinda stress testing.
|
||||
|
||||
@@ -28,7 +28,7 @@ use crate::test_util::{
|
||||
async fn test_scan_without_filtering_deleted() {
|
||||
common_telemetry::init_default_ut_logging();
|
||||
|
||||
let mut env = TestEnv::new();
|
||||
let mut env = TestEnv::new().await;
|
||||
let engine = env.create_engine(MitoConfig::default()).await;
|
||||
|
||||
let region_id = RegionId::new(1, 1);
|
||||
|
||||
@@ -41,7 +41,7 @@ use crate::worker::MAX_INITIAL_CHECK_DELAY_SECS;
|
||||
|
||||
#[tokio::test]
|
||||
async fn test_manual_flush() {
|
||||
let mut env = TestEnv::new();
|
||||
let mut env = TestEnv::new().await;
|
||||
let engine = env.create_engine(MitoConfig::default()).await;
|
||||
|
||||
let region_id = RegionId::new(1, 1);
|
||||
@@ -91,7 +91,7 @@ async fn test_manual_flush() {
|
||||
|
||||
#[tokio::test]
|
||||
async fn test_flush_engine() {
|
||||
let mut env = TestEnv::new();
|
||||
let mut env = TestEnv::new().await;
|
||||
let write_buffer_manager = Arc::new(MockWriteBufferManager::default());
|
||||
let listener = Arc::new(FlushListener::default());
|
||||
let engine = env
|
||||
@@ -161,7 +161,7 @@ async fn test_flush_engine() {
|
||||
|
||||
#[tokio::test]
|
||||
async fn test_write_stall() {
|
||||
let mut env = TestEnv::new();
|
||||
let mut env = TestEnv::new().await;
|
||||
let write_buffer_manager = Arc::new(MockWriteBufferManager::default());
|
||||
let listener = Arc::new(StallListener::default());
|
||||
let engine = env
|
||||
@@ -236,7 +236,7 @@ async fn test_write_stall() {
|
||||
|
||||
#[tokio::test]
|
||||
async fn test_flush_empty() {
|
||||
let mut env = TestEnv::new();
|
||||
let mut env = TestEnv::new().await;
|
||||
let write_buffer_manager = Arc::new(MockWriteBufferManager::default());
|
||||
let engine = env
|
||||
.create_engine_with(
|
||||
@@ -289,7 +289,7 @@ async fn test_flush_reopen_region(factory: Option<LogStoreFactory>) {
|
||||
return;
|
||||
};
|
||||
|
||||
let mut env = TestEnv::new().with_log_store_factory(factory.clone());
|
||||
let mut env = TestEnv::new().await.with_log_store_factory(factory.clone());
|
||||
let engine = env.create_engine(MitoConfig::default()).await;
|
||||
let region_id = RegionId::new(1, 1);
|
||||
env.get_schema_metadata_manager()
|
||||
@@ -396,7 +396,7 @@ impl MockTimeProvider {
|
||||
|
||||
#[tokio::test]
|
||||
async fn test_auto_flush_engine() {
|
||||
let mut env = TestEnv::new();
|
||||
let mut env = TestEnv::new().await;
|
||||
let write_buffer_manager = Arc::new(MockWriteBufferManager::default());
|
||||
let listener = Arc::new(FlushListener::default());
|
||||
let now = current_time_millis();
|
||||
@@ -467,7 +467,7 @@ async fn test_auto_flush_engine() {
|
||||
|
||||
#[tokio::test]
|
||||
async fn test_flush_workers() {
|
||||
let mut env = TestEnv::new();
|
||||
let mut env = TestEnv::new().await;
|
||||
let write_buffer_manager = Arc::new(MockWriteBufferManager::default());
|
||||
let listener = Arc::new(FlushListener::default());
|
||||
let engine = env
|
||||
@@ -554,7 +554,7 @@ async fn test_update_topic_latest_entry_id(factory: Option<LogStoreFactory>) {
|
||||
let write_buffer_manager = Arc::new(MockWriteBufferManager::default());
|
||||
let listener = Arc::new(FlushListener::default());
|
||||
|
||||
let mut env = TestEnv::new().with_log_store_factory(factory.clone());
|
||||
let mut env = TestEnv::new().await.with_log_store_factory(factory.clone());
|
||||
let engine = env
|
||||
.create_engine_with(
|
||||
MitoConfig::default(),
|
||||
|
||||
@@ -31,7 +31,7 @@ use crate::test_util::{
|
||||
async fn test_merge_mode_write_query() {
|
||||
common_telemetry::init_default_ut_logging();
|
||||
|
||||
let mut env = TestEnv::new();
|
||||
let mut env = TestEnv::new().await;
|
||||
let engine = env.create_engine(MitoConfig::default()).await;
|
||||
|
||||
let region_id = RegionId::new(1, 1);
|
||||
@@ -89,7 +89,7 @@ async fn test_merge_mode_write_query() {
|
||||
async fn test_merge_mode_compaction() {
|
||||
common_telemetry::init_default_ut_logging();
|
||||
|
||||
let mut env = TestEnv::new();
|
||||
let mut env = TestEnv::new().await;
|
||||
let engine = env
|
||||
.create_engine(MitoConfig {
|
||||
..Default::default()
|
||||
|
||||
@@ -36,7 +36,7 @@ use crate::test_util::{
|
||||
|
||||
#[tokio::test]
|
||||
async fn test_engine_open_empty() {
|
||||
let mut env = TestEnv::with_prefix("open-empty");
|
||||
let mut env = TestEnv::with_prefix("open-empty").await;
|
||||
let engine = env.create_engine(MitoConfig::default()).await;
|
||||
|
||||
let region_id = RegionId::new(1, 1);
|
||||
@@ -63,7 +63,7 @@ async fn test_engine_open_empty() {
|
||||
|
||||
#[tokio::test]
|
||||
async fn test_engine_open_existing() {
|
||||
let mut env = TestEnv::with_prefix("open-exiting");
|
||||
let mut env = TestEnv::with_prefix("open-exiting").await;
|
||||
let engine = env.create_engine(MitoConfig::default()).await;
|
||||
|
||||
let region_id = RegionId::new(1, 1);
|
||||
@@ -90,7 +90,7 @@ async fn test_engine_open_existing() {
|
||||
|
||||
#[tokio::test]
|
||||
async fn test_engine_reopen_region() {
|
||||
let mut env = TestEnv::with_prefix("reopen-region");
|
||||
let mut env = TestEnv::with_prefix("reopen-region").await;
|
||||
let engine = env.create_engine(MitoConfig::default()).await;
|
||||
|
||||
let region_id = RegionId::new(1, 1);
|
||||
@@ -107,7 +107,7 @@ async fn test_engine_reopen_region() {
|
||||
|
||||
#[tokio::test]
|
||||
async fn test_engine_open_readonly() {
|
||||
let mut env = TestEnv::new();
|
||||
let mut env = TestEnv::new().await;
|
||||
let engine = env.create_engine(MitoConfig::default()).await;
|
||||
|
||||
let region_id = RegionId::new(1, 1);
|
||||
@@ -150,7 +150,7 @@ async fn test_engine_open_readonly() {
|
||||
|
||||
#[tokio::test]
|
||||
async fn test_engine_region_open_with_options() {
|
||||
let mut env = TestEnv::new();
|
||||
let mut env = TestEnv::new().await;
|
||||
let engine = env.create_engine(MitoConfig::default()).await;
|
||||
|
||||
let region_id = RegionId::new(1, 1);
|
||||
@@ -190,7 +190,7 @@ async fn test_engine_region_open_with_options() {
|
||||
|
||||
#[tokio::test]
|
||||
async fn test_engine_region_open_with_custom_store() {
|
||||
let mut env = TestEnv::new();
|
||||
let mut env = TestEnv::new().await;
|
||||
let engine = env
|
||||
.create_engine_with_multiple_object_stores(MitoConfig::default(), None, None, &["Gcs"])
|
||||
.await;
|
||||
@@ -244,7 +244,7 @@ async fn test_engine_region_open_with_custom_store() {
|
||||
|
||||
#[tokio::test]
|
||||
async fn test_open_region_skip_wal_replay() {
|
||||
let mut env = TestEnv::new();
|
||||
let mut env = TestEnv::new().await;
|
||||
let engine = env.create_engine(MitoConfig::default()).await;
|
||||
|
||||
let region_id = RegionId::new(1, 1);
|
||||
@@ -344,7 +344,7 @@ async fn test_open_region_skip_wal_replay() {
|
||||
|
||||
#[tokio::test]
|
||||
async fn test_open_region_wait_for_opening_region_ok() {
|
||||
let mut env = TestEnv::with_prefix("wait-for-opening-region-ok");
|
||||
let mut env = TestEnv::with_prefix("wait-for-opening-region-ok").await;
|
||||
let engine = env.create_engine(MitoConfig::default()).await;
|
||||
let region_id = RegionId::new(1, 1);
|
||||
let worker = engine.inner.workers.worker(region_id);
|
||||
@@ -383,7 +383,7 @@ async fn test_open_region_wait_for_opening_region_ok() {
|
||||
|
||||
#[tokio::test]
|
||||
async fn test_open_region_wait_for_opening_region_err() {
|
||||
let mut env = TestEnv::with_prefix("wait-for-opening-region-err");
|
||||
let mut env = TestEnv::with_prefix("wait-for-opening-region-err").await;
|
||||
let engine = env.create_engine(MitoConfig::default()).await;
|
||||
let region_id = RegionId::new(1, 1);
|
||||
let worker = engine.inner.workers.worker(region_id);
|
||||
@@ -428,7 +428,7 @@ async fn test_open_region_wait_for_opening_region_err() {
|
||||
|
||||
#[tokio::test]
|
||||
async fn test_open_compaction_region() {
|
||||
let mut env = TestEnv::new();
|
||||
let mut env = TestEnv::new().await;
|
||||
let mut mito_config = MitoConfig::default();
|
||||
mito_config
|
||||
.sanitize(&env.data_home().display().to_string())
|
||||
|
||||
@@ -73,7 +73,7 @@ async fn scan_in_parallel(
|
||||
|
||||
#[tokio::test]
|
||||
async fn test_parallel_scan() {
|
||||
let mut env = TestEnv::new();
|
||||
let mut env = TestEnv::new().await;
|
||||
let engine = env.create_engine(MitoConfig::default()).await;
|
||||
|
||||
let region_id = RegionId::new(1, 1);
|
||||
|
||||
@@ -53,7 +53,7 @@ fn build_rows_multi_tags_fields(
|
||||
|
||||
#[tokio::test]
|
||||
async fn test_scan_projection() {
|
||||
let mut env = TestEnv::new();
|
||||
let mut env = TestEnv::new().await;
|
||||
let engine = env.create_engine(MitoConfig::default()).await;
|
||||
|
||||
let region_id = RegionId::new(1, 1);
|
||||
|
||||
@@ -27,7 +27,7 @@ use crate::test_util::{
|
||||
};
|
||||
|
||||
async fn check_prune_row_groups(exprs: Vec<Expr>, expected: &str) {
|
||||
let mut env = TestEnv::new();
|
||||
let mut env = TestEnv::new().await;
|
||||
let engine = env.create_engine(MitoConfig::default()).await;
|
||||
|
||||
let region_id = RegionId::new(1, 1);
|
||||
@@ -147,7 +147,7 @@ fn time_range_expr(start_sec: i64, end_sec: i64) -> Expr {
|
||||
|
||||
#[tokio::test]
|
||||
async fn test_prune_memtable() {
|
||||
let mut env = TestEnv::new();
|
||||
let mut env = TestEnv::new().await;
|
||||
let engine = env.create_engine(MitoConfig::default()).await;
|
||||
|
||||
let region_id = RegionId::new(1, 1);
|
||||
@@ -221,7 +221,7 @@ async fn test_prune_memtable() {
|
||||
|
||||
#[tokio::test]
|
||||
async fn test_prune_memtable_complex_expr() {
|
||||
let mut env = TestEnv::new();
|
||||
let mut env = TestEnv::new().await;
|
||||
let engine = env.create_engine(MitoConfig::default()).await;
|
||||
|
||||
let region_id = RegionId::new(1, 1);
|
||||
@@ -274,7 +274,7 @@ async fn test_prune_memtable_complex_expr() {
|
||||
|
||||
#[tokio::test]
|
||||
async fn test_mem_range_prune() {
|
||||
let mut env = TestEnv::new();
|
||||
let mut env = TestEnv::new().await;
|
||||
let engine = env.create_engine(MitoConfig::default()).await;
|
||||
|
||||
let region_id = RegionId::new(1, 1);
|
||||
|
||||
@@ -25,7 +25,7 @@ use crate::test_util::{
|
||||
};
|
||||
|
||||
async fn test_last_row(append_mode: bool) {
|
||||
let mut env = TestEnv::new();
|
||||
let mut env = TestEnv::new().await;
|
||||
let engine = env.create_engine(MitoConfig::default()).await;
|
||||
let region_id = RegionId::new(1, 1);
|
||||
|
||||
|
||||
@@ -27,7 +27,7 @@ use crate::test_util::{CreateRequestBuilder, TestEnv};
|
||||
|
||||
#[tokio::test]
|
||||
async fn test_scan_with_min_sst_sequence() {
|
||||
let mut env = TestEnv::with_prefix("test_scan_with_min_sst_sequence");
|
||||
let mut env = TestEnv::with_prefix("test_scan_with_min_sst_sequence").await;
|
||||
let engine = env.create_engine(MitoConfig::default()).await;
|
||||
|
||||
let region_id = RegionId::new(1, 1);
|
||||
@@ -153,7 +153,7 @@ async fn test_scan_with_min_sst_sequence() {
|
||||
|
||||
#[tokio::test]
|
||||
async fn test_series_scan() {
|
||||
let mut env = TestEnv::with_prefix("test_series_scan");
|
||||
let mut env = TestEnv::with_prefix("test_series_scan").await;
|
||||
let engine = env.create_engine(MitoConfig::default()).await;
|
||||
|
||||
let region_id = RegionId::new(1, 1);
|
||||
|
||||
@@ -32,7 +32,7 @@ async fn test_set_role_state_gracefully() {
|
||||
SettableRegionRoleState::DowngradingLeader,
|
||||
];
|
||||
for settable_role_state in settable_role_states {
|
||||
let mut env = TestEnv::new();
|
||||
let mut env = TestEnv::new().await;
|
||||
let engine = env.create_engine(MitoConfig::default()).await;
|
||||
|
||||
let region_id = RegionId::new(1, 1);
|
||||
@@ -101,7 +101,7 @@ async fn test_set_role_state_gracefully() {
|
||||
|
||||
#[tokio::test]
|
||||
async fn test_set_role_state_gracefully_not_exist() {
|
||||
let mut env = TestEnv::new();
|
||||
let mut env = TestEnv::new().await;
|
||||
let engine = env.create_engine(MitoConfig::default()).await;
|
||||
|
||||
let non_exist_region_id = RegionId::new(1, 1);
|
||||
@@ -116,7 +116,7 @@ async fn test_set_role_state_gracefully_not_exist() {
|
||||
|
||||
#[tokio::test]
|
||||
async fn test_write_downgrading_region() {
|
||||
let mut env = TestEnv::with_prefix("write-to-downgrading-region");
|
||||
let mut env = TestEnv::with_prefix("write-to-downgrading-region").await;
|
||||
let engine = env.create_engine(MitoConfig::default()).await;
|
||||
|
||||
let region_id = RegionId::new(1, 1);
|
||||
|
||||
@@ -71,7 +71,7 @@ async fn scan_check(
|
||||
#[tokio::test]
|
||||
async fn test_sync_after_flush_region() {
|
||||
common_telemetry::init_default_ut_logging();
|
||||
let mut env = TestEnv::new();
|
||||
let mut env = TestEnv::new().await;
|
||||
let engine = env.create_engine(MitoConfig::default()).await;
|
||||
let region_id = RegionId::new(1, 1);
|
||||
env.get_schema_metadata_manager()
|
||||
@@ -163,7 +163,7 @@ async fn test_sync_after_flush_region() {
|
||||
async fn test_sync_after_alter_region() {
|
||||
common_telemetry::init_default_ut_logging();
|
||||
|
||||
let mut env = TestEnv::new();
|
||||
let mut env = TestEnv::new().await;
|
||||
let engine = env.create_engine(MitoConfig::default()).await;
|
||||
|
||||
let region_id = RegionId::new(1, 1);
|
||||
|
||||
@@ -33,7 +33,7 @@ use crate::test_util::{
|
||||
|
||||
#[tokio::test]
|
||||
async fn test_engine_truncate_region_basic() {
|
||||
let mut env = TestEnv::with_prefix("truncate-basic");
|
||||
let mut env = TestEnv::with_prefix("truncate-basic").await;
|
||||
let engine = env.create_engine(MitoConfig::default()).await;
|
||||
|
||||
// Create the region.
|
||||
@@ -83,7 +83,7 @@ async fn test_engine_truncate_region_basic() {
|
||||
|
||||
#[tokio::test]
|
||||
async fn test_engine_put_data_after_truncate() {
|
||||
let mut env = TestEnv::with_prefix("truncate-put");
|
||||
let mut env = TestEnv::with_prefix("truncate-put").await;
|
||||
let engine = env.create_engine(MitoConfig::default()).await;
|
||||
|
||||
// Create the region.
|
||||
@@ -146,7 +146,7 @@ async fn test_engine_put_data_after_truncate() {
|
||||
|
||||
#[tokio::test]
|
||||
async fn test_engine_truncate_after_flush() {
|
||||
let mut env = TestEnv::with_prefix("truncate-flush");
|
||||
let mut env = TestEnv::with_prefix("truncate-flush").await;
|
||||
let engine = env.create_engine(MitoConfig::default()).await;
|
||||
|
||||
// Create the region.
|
||||
@@ -223,7 +223,7 @@ async fn test_engine_truncate_after_flush() {
|
||||
|
||||
#[tokio::test]
|
||||
async fn test_engine_truncate_reopen() {
|
||||
let mut env = TestEnv::with_prefix("truncate-reopen");
|
||||
let mut env = TestEnv::with_prefix("truncate-reopen").await;
|
||||
let engine = env.create_engine(MitoConfig::default()).await;
|
||||
|
||||
// Create the region.
|
||||
@@ -282,7 +282,7 @@ async fn test_engine_truncate_reopen() {
|
||||
#[tokio::test]
|
||||
async fn test_engine_truncate_during_flush() {
|
||||
init_default_ut_logging();
|
||||
let mut env = TestEnv::with_prefix("truncate-during-flush");
|
||||
let mut env = TestEnv::with_prefix("truncate-during-flush").await;
|
||||
let write_buffer_manager = Arc::new(MockWriteBufferManager::default());
|
||||
let listener = Arc::new(FlushTruncateListener::default());
|
||||
let engine = env
|
||||
|
||||
@@ -594,7 +594,7 @@ mod test {
|
||||
#[tokio::test]
|
||||
async fn create_manifest_manager() {
|
||||
let metadata = Arc::new(basic_region_metadata());
|
||||
let env = TestEnv::new();
|
||||
let env = TestEnv::new().await;
|
||||
let manager = env
|
||||
.create_manifest_manager(CompressionType::Uncompressed, 10, Some(metadata.clone()))
|
||||
.await
|
||||
@@ -606,7 +606,7 @@ mod test {
|
||||
|
||||
#[tokio::test]
|
||||
async fn open_manifest_manager() {
|
||||
let env = TestEnv::new();
|
||||
let env = TestEnv::new().await;
|
||||
// Try to opens an empty manifest.
|
||||
assert!(env
|
||||
.create_manifest_manager(CompressionType::Uncompressed, 10, None)
|
||||
@@ -637,7 +637,7 @@ mod test {
|
||||
#[tokio::test]
|
||||
async fn region_change_add_column() {
|
||||
let metadata = Arc::new(basic_region_metadata());
|
||||
let env = TestEnv::new();
|
||||
let env = TestEnv::new().await;
|
||||
let mut manager = env
|
||||
.create_manifest_manager(CompressionType::Uncompressed, 10, Some(metadata.clone()))
|
||||
.await
|
||||
@@ -696,7 +696,7 @@ mod test {
|
||||
let metadata = Arc::new(basic_region_metadata());
|
||||
let data_home = create_temp_dir("");
|
||||
let data_home_path = data_home.path().to_str().unwrap().to_string();
|
||||
let env = TestEnv::with_data_home(data_home);
|
||||
let env = TestEnv::with_data_home(data_home).await;
|
||||
|
||||
let manifest_dir = format!("{}/manifest", data_home_path);
|
||||
|
||||
|
||||
@@ -34,7 +34,7 @@ async fn build_manager(
|
||||
compress_type: CompressionType,
|
||||
) -> (TestEnv, RegionManifestManager) {
|
||||
let metadata = Arc::new(basic_region_metadata());
|
||||
let env = TestEnv::new();
|
||||
let env = TestEnv::new().await;
|
||||
let manager = env
|
||||
.create_manifest_manager(compress_type, checkpoint_distance, Some(metadata.clone()))
|
||||
.await
|
||||
|
||||
@@ -45,8 +45,6 @@ pub(crate) const TYPE_INVERTED_INDEX: &str = "inverted_index";
|
||||
pub(crate) const TYPE_FULLTEXT_INDEX: &str = "fulltext_index";
|
||||
pub(crate) const TYPE_BLOOM_FILTER_INDEX: &str = "bloom_filter_index";
|
||||
|
||||
const DEFAULT_FULLTEXT_BLOOM_ROW_GRANULARITY: usize = 8096;
|
||||
|
||||
/// Output of the index creation.
|
||||
#[derive(Debug, Clone, Default)]
|
||||
pub struct IndexOutput {
|
||||
@@ -293,7 +291,6 @@ impl IndexerBuilderImpl {
|
||||
&self.intermediate_manager,
|
||||
&self.metadata,
|
||||
self.fulltext_index_config.compress,
|
||||
DEFAULT_FULLTEXT_BLOOM_ROW_GRANULARITY,
|
||||
mem_limit,
|
||||
)
|
||||
.await;
|
||||
@@ -455,10 +452,11 @@ mod tests {
|
||||
if with_skipping_bloom {
|
||||
let column_schema =
|
||||
ColumnSchema::new("bloom", ConcreteDataType::string_datatype(), false)
|
||||
.with_skipping_options(SkippingIndexOptions {
|
||||
granularity: 42,
|
||||
index_type: SkippingIndexType::BloomFilter,
|
||||
})
|
||||
.with_skipping_options(SkippingIndexOptions::new_unchecked(
|
||||
42,
|
||||
0.01,
|
||||
SkippingIndexType::BloomFilter,
|
||||
))
|
||||
.unwrap();
|
||||
|
||||
let column = ColumnMetadata {
|
||||
|
||||
@@ -128,6 +128,9 @@ impl BloomFilterIndexApplier {
|
||||
/// list of row group ranges that match the predicates.
|
||||
///
|
||||
/// The `row_groups` iterator provides the row group lengths and whether to search in the row group.
|
||||
///
|
||||
/// Row group id existing in the returned result means that the row group is searched.
|
||||
/// Empty ranges means that the row group is searched but no rows are found.
|
||||
pub async fn apply(
|
||||
&self,
|
||||
file_id: FileId,
|
||||
@@ -195,7 +198,6 @@ impl BloomFilterIndexApplier {
|
||||
range.end -= start;
|
||||
}
|
||||
}
|
||||
output.retain(|(_, ranges)| !ranges.is_empty());
|
||||
|
||||
Ok(output)
|
||||
}
|
||||
@@ -386,6 +388,9 @@ mod tests {
|
||||
.apply(file_id, None, row_groups.into_iter())
|
||||
.await
|
||||
.unwrap()
|
||||
.into_iter()
|
||||
.filter(|(_, ranges)| !ranges.is_empty())
|
||||
.collect()
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
@@ -97,6 +97,7 @@ impl BloomFilterIndexer {
|
||||
|
||||
let creator = BloomFilterCreator::new(
|
||||
options.granularity as _,
|
||||
options.false_positive_rate(),
|
||||
temp_file_provider.clone(),
|
||||
global_memory_usage.clone(),
|
||||
memory_usage_threshold,
|
||||
@@ -408,10 +409,11 @@ pub(crate) mod tests {
|
||||
ConcreteDataType::string_datatype(),
|
||||
false,
|
||||
)
|
||||
.with_skipping_options(SkippingIndexOptions {
|
||||
index_type: SkippingIndexType::BloomFilter,
|
||||
granularity: 2,
|
||||
})
|
||||
.with_skipping_options(SkippingIndexOptions::new_unchecked(
|
||||
2,
|
||||
0.01,
|
||||
SkippingIndexType::BloomFilter,
|
||||
))
|
||||
.unwrap(),
|
||||
semantic_type: SemanticType::Tag,
|
||||
column_id: 1,
|
||||
@@ -431,10 +433,11 @@ pub(crate) mod tests {
|
||||
ConcreteDataType::uint64_datatype(),
|
||||
false,
|
||||
)
|
||||
.with_skipping_options(SkippingIndexOptions {
|
||||
index_type: SkippingIndexType::BloomFilter,
|
||||
granularity: 4,
|
||||
})
|
||||
.with_skipping_options(SkippingIndexOptions::new_unchecked(
|
||||
4,
|
||||
0.01,
|
||||
SkippingIndexType::BloomFilter,
|
||||
))
|
||||
.unwrap(),
|
||||
semantic_type: SemanticType::Field,
|
||||
column_id: 3,
|
||||
|
||||
@@ -231,6 +231,9 @@ impl FulltextIndexApplier {
|
||||
impl FulltextIndexApplier {
|
||||
/// Applies coarse-grained fulltext index to the specified SST file.
|
||||
/// Returns (row group id -> ranges) that match the queries.
|
||||
///
|
||||
/// Row group id existing in the returned result means that the row group is searched.
|
||||
/// Empty ranges means that the row group is searched but no rows are found.
|
||||
pub async fn apply_coarse(
|
||||
&self,
|
||||
file_id: FileId,
|
||||
@@ -367,7 +370,7 @@ impl FulltextIndexApplier {
|
||||
/// Adjusts the coarse output. Makes the output ranges based on row group start.
|
||||
fn adjust_coarse_output(
|
||||
input: Vec<(usize, Range<usize>)>,
|
||||
output: &mut Vec<(usize, Vec<Range<usize>>)>,
|
||||
output: &mut [(usize, Vec<Range<usize>>)],
|
||||
) {
|
||||
// adjust ranges to be based on row group
|
||||
for ((_, output), (_, input)) in output.iter_mut().zip(input) {
|
||||
@@ -377,7 +380,6 @@ impl FulltextIndexApplier {
|
||||
range.end -= start;
|
||||
}
|
||||
}
|
||||
output.retain(|(_, ranges)| !ranges.is_empty());
|
||||
}
|
||||
|
||||
/// Converts terms to predicates.
|
||||
@@ -463,7 +465,6 @@ impl IndexSource {
|
||||
/// Returns the blob with the specified key from local cache or remote store.
|
||||
///
|
||||
/// Returns `None` if the blob is not found.
|
||||
#[allow(unused)]
|
||||
async fn blob(
|
||||
&self,
|
||||
file_id: FileId,
|
||||
|
||||
@@ -60,7 +60,6 @@ impl FulltextIndexer {
|
||||
intermediate_manager: &IntermediateManager,
|
||||
metadata: &RegionMetadataRef,
|
||||
compress: bool,
|
||||
bloom_row_granularity: usize,
|
||||
mem_limit: usize,
|
||||
) -> Result<Option<Self>> {
|
||||
let mut creators = HashMap::new();
|
||||
@@ -106,7 +105,8 @@ impl FulltextIndexer {
|
||||
let global_memory_usage = Arc::new(AtomicUsize::new(0));
|
||||
let creator = BloomFilterFulltextIndexCreator::new(
|
||||
config,
|
||||
bloom_row_granularity,
|
||||
options.granularity as _,
|
||||
options.false_positive_rate(),
|
||||
temp_file_provider,
|
||||
global_memory_usage,
|
||||
Some(mem_limit),
|
||||
@@ -400,12 +400,14 @@ mod tests {
|
||||
ConcreteDataType::string_datatype(),
|
||||
true,
|
||||
)
|
||||
.with_fulltext_options(FulltextOptions {
|
||||
enable: true,
|
||||
analyzer: FulltextAnalyzer::English,
|
||||
case_sensitive: true,
|
||||
backend: backend.clone(),
|
||||
})
|
||||
.with_fulltext_options(FulltextOptions::new_unchecked(
|
||||
true,
|
||||
FulltextAnalyzer::English,
|
||||
true,
|
||||
backend.clone(),
|
||||
1,
|
||||
0.01,
|
||||
))
|
||||
.unwrap(),
|
||||
semantic_type: SemanticType::Field,
|
||||
column_id: 1,
|
||||
@@ -416,12 +418,14 @@ mod tests {
|
||||
ConcreteDataType::string_datatype(),
|
||||
true,
|
||||
)
|
||||
.with_fulltext_options(FulltextOptions {
|
||||
enable: true,
|
||||
analyzer: FulltextAnalyzer::English,
|
||||
case_sensitive: false,
|
||||
backend: backend.clone(),
|
||||
})
|
||||
.with_fulltext_options(FulltextOptions::new_unchecked(
|
||||
true,
|
||||
FulltextAnalyzer::English,
|
||||
false,
|
||||
backend.clone(),
|
||||
1,
|
||||
0.01,
|
||||
))
|
||||
.unwrap(),
|
||||
semantic_type: SemanticType::Field,
|
||||
column_id: 2,
|
||||
@@ -432,12 +436,14 @@ mod tests {
|
||||
ConcreteDataType::string_datatype(),
|
||||
true,
|
||||
)
|
||||
.with_fulltext_options(FulltextOptions {
|
||||
enable: true,
|
||||
analyzer: FulltextAnalyzer::Chinese,
|
||||
case_sensitive: false,
|
||||
backend: backend.clone(),
|
||||
})
|
||||
.with_fulltext_options(FulltextOptions::new_unchecked(
|
||||
true,
|
||||
FulltextAnalyzer::Chinese,
|
||||
false,
|
||||
backend.clone(),
|
||||
1,
|
||||
0.01,
|
||||
))
|
||||
.unwrap(),
|
||||
semantic_type: SemanticType::Field,
|
||||
column_id: 3,
|
||||
@@ -547,7 +553,6 @@ mod tests {
|
||||
&intm_mgr,
|
||||
®ion_metadata,
|
||||
true,
|
||||
1,
|
||||
1024,
|
||||
)
|
||||
.await
|
||||
@@ -625,6 +630,7 @@ mod tests {
|
||||
.unwrap();
|
||||
resp.map(|r| {
|
||||
r.into_iter()
|
||||
.filter(|(_, ranges)| !ranges.is_empty())
|
||||
.map(|(row_group_id, _)| row_group_id as RowId)
|
||||
.collect()
|
||||
})
|
||||
|
||||
@@ -14,10 +14,8 @@
|
||||
|
||||
use common_telemetry::{debug, warn};
|
||||
use puffin::puffin_manager::{PuffinManager, PuffinWriter};
|
||||
use store_api::storage::ColumnId;
|
||||
|
||||
use crate::sst::index::bloom_filter::creator::BloomFilterIndexer;
|
||||
use crate::sst::index::fulltext_index::creator::FulltextIndexer;
|
||||
use crate::sst::index::inverted_index::creator::InvertedIndexer;
|
||||
use crate::sst::index::puffin_manager::SstPuffinWriter;
|
||||
use crate::sst::index::statistics::{ByteCount, RowCount};
|
||||
use crate::sst::index::{
|
||||
@@ -113,13 +111,14 @@ impl Indexer {
|
||||
return true;
|
||||
};
|
||||
|
||||
let column_ids = indexer.column_ids().collect();
|
||||
let err = match indexer.finish(puffin_writer).await {
|
||||
Ok((row_count, byte_count)) => {
|
||||
self.fill_inverted_index_output(
|
||||
&mut index_output.inverted_index,
|
||||
row_count,
|
||||
byte_count,
|
||||
&indexer,
|
||||
column_ids,
|
||||
);
|
||||
return true;
|
||||
}
|
||||
@@ -150,13 +149,14 @@ impl Indexer {
|
||||
return true;
|
||||
};
|
||||
|
||||
let column_ids = indexer.column_ids().collect();
|
||||
let err = match indexer.finish(puffin_writer).await {
|
||||
Ok((row_count, byte_count)) => {
|
||||
self.fill_fulltext_index_output(
|
||||
&mut index_output.fulltext_index,
|
||||
row_count,
|
||||
byte_count,
|
||||
&indexer,
|
||||
column_ids,
|
||||
);
|
||||
return true;
|
||||
}
|
||||
@@ -187,13 +187,14 @@ impl Indexer {
|
||||
return true;
|
||||
};
|
||||
|
||||
let column_ids = indexer.column_ids().collect();
|
||||
let err = match indexer.finish(puffin_writer).await {
|
||||
Ok((row_count, byte_count)) => {
|
||||
self.fill_bloom_filter_output(
|
||||
&mut index_output.bloom_filter,
|
||||
row_count,
|
||||
byte_count,
|
||||
&indexer,
|
||||
column_ids,
|
||||
);
|
||||
return true;
|
||||
}
|
||||
@@ -220,16 +221,16 @@ impl Indexer {
|
||||
output: &mut InvertedIndexOutput,
|
||||
row_count: RowCount,
|
||||
byte_count: ByteCount,
|
||||
indexer: &InvertedIndexer,
|
||||
column_ids: Vec<ColumnId>,
|
||||
) {
|
||||
debug!(
|
||||
"Inverted index created, region_id: {}, file_id: {}, written_bytes: {}, written_rows: {}",
|
||||
self.region_id, self.file_id, byte_count, row_count
|
||||
"Inverted index created, region_id: {}, file_id: {}, written_bytes: {}, written_rows: {}, columns: {:?}",
|
||||
self.region_id, self.file_id, byte_count, row_count, column_ids
|
||||
);
|
||||
|
||||
output.index_size = byte_count;
|
||||
output.row_count = row_count;
|
||||
output.columns = indexer.column_ids().collect();
|
||||
output.columns = column_ids;
|
||||
}
|
||||
|
||||
fn fill_fulltext_index_output(
|
||||
@@ -237,16 +238,16 @@ impl Indexer {
|
||||
output: &mut FulltextIndexOutput,
|
||||
row_count: RowCount,
|
||||
byte_count: ByteCount,
|
||||
indexer: &FulltextIndexer,
|
||||
column_ids: Vec<ColumnId>,
|
||||
) {
|
||||
debug!(
|
||||
"Full-text index created, region_id: {}, file_id: {}, written_bytes: {}, written_rows: {}",
|
||||
self.region_id, self.file_id, byte_count, row_count
|
||||
"Full-text index created, region_id: {}, file_id: {}, written_bytes: {}, written_rows: {}, columns: {:?}",
|
||||
self.region_id, self.file_id, byte_count, row_count, column_ids
|
||||
);
|
||||
|
||||
output.index_size = byte_count;
|
||||
output.row_count = row_count;
|
||||
output.columns = indexer.column_ids().collect();
|
||||
output.columns = column_ids;
|
||||
}
|
||||
|
||||
fn fill_bloom_filter_output(
|
||||
@@ -254,15 +255,15 @@ impl Indexer {
|
||||
output: &mut BloomFilterOutput,
|
||||
row_count: RowCount,
|
||||
byte_count: ByteCount,
|
||||
indexer: &BloomFilterIndexer,
|
||||
column_ids: Vec<ColumnId>,
|
||||
) {
|
||||
debug!(
|
||||
"Bloom filter created, region_id: {}, file_id: {}, written_bytes: {}, written_rows: {}",
|
||||
self.region_id, self.file_id, byte_count, row_count
|
||||
"Bloom filter created, region_id: {}, file_id: {}, written_bytes: {}, written_rows: {}, columns: {:?}",
|
||||
self.region_id, self.file_id, byte_count, row_count, column_ids
|
||||
);
|
||||
|
||||
output.index_size = byte_count;
|
||||
output.row_count = row_count;
|
||||
output.columns = indexer.column_ids().collect();
|
||||
output.columns = column_ids;
|
||||
}
|
||||
}
|
||||
|
||||
@@ -88,11 +88,12 @@ pub struct SstInfo {
|
||||
|
||||
#[cfg(test)]
|
||||
mod tests {
|
||||
use std::collections::HashSet;
|
||||
use std::sync::Arc;
|
||||
|
||||
use common_time::Timestamp;
|
||||
use datafusion_common::{Column, ScalarValue};
|
||||
use datafusion_expr::{BinaryExpr, Expr, Operator};
|
||||
use datafusion_expr::{col, lit, BinaryExpr, Expr, Operator};
|
||||
use datatypes::arrow;
|
||||
use datatypes::arrow::array::RecordBatch;
|
||||
use datatypes::arrow::datatypes::{DataType, Field, Schema};
|
||||
@@ -104,12 +105,17 @@ mod tests {
|
||||
use tokio_util::compat::FuturesAsyncWriteCompatExt;
|
||||
|
||||
use super::*;
|
||||
use crate::access_layer::{FilePathProvider, RegionFilePathFactory};
|
||||
use crate::access_layer::{FilePathProvider, OperationType, RegionFilePathFactory};
|
||||
use crate::cache::{CacheManager, CacheStrategy, PageKey};
|
||||
use crate::read::BatchReader;
|
||||
use crate::sst::index::{Indexer, IndexerBuilder};
|
||||
use crate::region::options::{IndexOptions, InvertedIndexOptions};
|
||||
use crate::sst::file::{FileHandle, FileMeta};
|
||||
use crate::sst::file_purger::NoopFilePurger;
|
||||
use crate::sst::index::bloom_filter::applier::BloomFilterIndexApplierBuilder;
|
||||
use crate::sst::index::inverted_index::applier::builder::InvertedIndexApplierBuilder;
|
||||
use crate::sst::index::{Indexer, IndexerBuilder, IndexerBuilderImpl};
|
||||
use crate::sst::parquet::format::WriteFormat;
|
||||
use crate::sst::parquet::reader::ParquetReaderBuilder;
|
||||
use crate::sst::parquet::reader::{ParquetReader, ParquetReaderBuilder, ReaderMetrics};
|
||||
use crate::sst::parquet::writer::ParquetWriter;
|
||||
use crate::sst::{location, DEFAULT_WRITE_CONCURRENCY};
|
||||
use crate::test_util::sst_util::{
|
||||
@@ -147,7 +153,7 @@ mod tests {
|
||||
|
||||
#[tokio::test]
|
||||
async fn test_write_read() {
|
||||
let mut env = TestEnv::new();
|
||||
let mut env = TestEnv::new().await;
|
||||
let object_store = env.init_object_store_manager();
|
||||
let handle = sst_file_handle(0, 1000);
|
||||
let file_path = FixedPathProvider {
|
||||
@@ -205,7 +211,7 @@ mod tests {
|
||||
|
||||
#[tokio::test]
|
||||
async fn test_read_with_cache() {
|
||||
let mut env = TestEnv::new();
|
||||
let mut env = TestEnv::new().await;
|
||||
let object_store = env.init_object_store_manager();
|
||||
let handle = sst_file_handle(0, 1000);
|
||||
let metadata = Arc::new(sst_region_metadata());
|
||||
@@ -275,7 +281,7 @@ mod tests {
|
||||
#[tokio::test]
|
||||
async fn test_parquet_metadata_eq() {
|
||||
// create test env
|
||||
let mut env = crate::test_util::TestEnv::new();
|
||||
let mut env = crate::test_util::TestEnv::new().await;
|
||||
let object_store = env.init_object_store_manager();
|
||||
let handle = sst_file_handle(0, 1000);
|
||||
let metadata = Arc::new(sst_region_metadata());
|
||||
@@ -318,7 +324,7 @@ mod tests {
|
||||
|
||||
#[tokio::test]
|
||||
async fn test_read_with_tag_filter() {
|
||||
let mut env = TestEnv::new();
|
||||
let mut env = TestEnv::new().await;
|
||||
let object_store = env.init_object_store_manager();
|
||||
let handle = sst_file_handle(0, 1000);
|
||||
let metadata = Arc::new(sst_region_metadata());
|
||||
@@ -370,7 +376,7 @@ mod tests {
|
||||
|
||||
#[tokio::test]
|
||||
async fn test_read_empty_batch() {
|
||||
let mut env = TestEnv::new();
|
||||
let mut env = TestEnv::new().await;
|
||||
let object_store = env.init_object_store_manager();
|
||||
let handle = sst_file_handle(0, 1000);
|
||||
let metadata = Arc::new(sst_region_metadata());
|
||||
@@ -407,7 +413,7 @@ mod tests {
|
||||
|
||||
#[tokio::test]
|
||||
async fn test_read_with_field_filter() {
|
||||
let mut env = TestEnv::new();
|
||||
let mut env = TestEnv::new().await;
|
||||
let object_store = env.init_object_store_manager();
|
||||
let handle = sst_file_handle(0, 1000);
|
||||
let metadata = Arc::new(sst_region_metadata());
|
||||
@@ -453,7 +459,7 @@ mod tests {
|
||||
|
||||
#[tokio::test]
|
||||
async fn test_read_large_binary() {
|
||||
let mut env = TestEnv::new();
|
||||
let mut env = TestEnv::new().await;
|
||||
let object_store = env.init_object_store_manager();
|
||||
let handle = sst_file_handle(0, 1000);
|
||||
let file_path = handle.file_path(FILE_DIR);
|
||||
@@ -544,7 +550,7 @@ mod tests {
|
||||
async fn test_write_multiple_files() {
|
||||
common_telemetry::init_default_ut_logging();
|
||||
// create test env
|
||||
let mut env = TestEnv::new();
|
||||
let mut env = TestEnv::new().await;
|
||||
let object_store = env.init_object_store_manager();
|
||||
let metadata = Arc::new(sst_region_metadata());
|
||||
let batches = &[
|
||||
@@ -593,4 +599,300 @@ mod tests {
|
||||
}
|
||||
assert_eq!(total_rows, rows_read);
|
||||
}
|
||||
|
||||
#[tokio::test]
|
||||
async fn test_write_read_with_index() {
|
||||
let mut env = TestEnv::new().await;
|
||||
let object_store = env.init_object_store_manager();
|
||||
let file_path = RegionFilePathFactory::new(FILE_DIR.to_string());
|
||||
let metadata = Arc::new(sst_region_metadata());
|
||||
let row_group_size = 50;
|
||||
|
||||
let source = new_source(&[
|
||||
new_batch_by_range(&["a", "d"], 0, 20),
|
||||
new_batch_by_range(&["b", "d"], 0, 20),
|
||||
new_batch_by_range(&["c", "d"], 0, 20),
|
||||
new_batch_by_range(&["c", "f"], 0, 40),
|
||||
new_batch_by_range(&["c", "h"], 100, 200),
|
||||
]);
|
||||
// Use a small row group size for test.
|
||||
let write_opts = WriteOptions {
|
||||
row_group_size,
|
||||
..Default::default()
|
||||
};
|
||||
|
||||
let puffin_manager = env
|
||||
.get_puffin_manager()
|
||||
.build(object_store.clone(), file_path.clone());
|
||||
let intermediate_manager = env.get_intermediate_manager();
|
||||
|
||||
let indexer_builder = IndexerBuilderImpl {
|
||||
op_type: OperationType::Flush,
|
||||
metadata: metadata.clone(),
|
||||
row_group_size,
|
||||
puffin_manager,
|
||||
intermediate_manager,
|
||||
index_options: IndexOptions {
|
||||
inverted_index: InvertedIndexOptions {
|
||||
segment_row_count: 1,
|
||||
..Default::default()
|
||||
},
|
||||
},
|
||||
inverted_index_config: Default::default(),
|
||||
fulltext_index_config: Default::default(),
|
||||
bloom_filter_index_config: Default::default(),
|
||||
};
|
||||
|
||||
let mut writer = ParquetWriter::new_with_object_store(
|
||||
object_store.clone(),
|
||||
metadata.clone(),
|
||||
indexer_builder,
|
||||
file_path.clone(),
|
||||
)
|
||||
.await;
|
||||
|
||||
let info = writer
|
||||
.write_all(source, None, &write_opts)
|
||||
.await
|
||||
.unwrap()
|
||||
.remove(0);
|
||||
assert_eq!(200, info.num_rows);
|
||||
assert!(info.file_size > 0);
|
||||
assert!(info.index_metadata.file_size > 0);
|
||||
|
||||
assert!(info.index_metadata.inverted_index.index_size > 0);
|
||||
assert_eq!(info.index_metadata.inverted_index.row_count, 200);
|
||||
assert_eq!(info.index_metadata.inverted_index.columns, vec![0]);
|
||||
|
||||
assert!(info.index_metadata.bloom_filter.index_size > 0);
|
||||
assert_eq!(info.index_metadata.bloom_filter.row_count, 200);
|
||||
assert_eq!(info.index_metadata.bloom_filter.columns, vec![1]);
|
||||
|
||||
assert_eq!(
|
||||
(
|
||||
Timestamp::new_millisecond(0),
|
||||
Timestamp::new_millisecond(199)
|
||||
),
|
||||
info.time_range
|
||||
);
|
||||
|
||||
let handle = FileHandle::new(
|
||||
FileMeta {
|
||||
region_id: metadata.region_id,
|
||||
file_id: info.file_id,
|
||||
time_range: info.time_range,
|
||||
level: 0,
|
||||
file_size: info.file_size,
|
||||
available_indexes: info.index_metadata.build_available_indexes(),
|
||||
index_file_size: info.index_metadata.file_size,
|
||||
num_row_groups: info.num_row_groups,
|
||||
num_rows: info.num_rows as u64,
|
||||
sequence: None,
|
||||
},
|
||||
Arc::new(NoopFilePurger),
|
||||
);
|
||||
|
||||
let cache = Arc::new(
|
||||
CacheManager::builder()
|
||||
.index_result_cache_size(1024 * 1024)
|
||||
.index_metadata_size(1024 * 1024)
|
||||
.index_content_page_size(1024 * 1024)
|
||||
.index_content_size(1024 * 1024)
|
||||
.puffin_metadata_size(1024 * 1024)
|
||||
.build(),
|
||||
);
|
||||
let index_result_cache = cache.index_result_cache().unwrap();
|
||||
|
||||
let build_inverted_index_applier = |exprs: &[Expr]| {
|
||||
InvertedIndexApplierBuilder::new(
|
||||
FILE_DIR.to_string(),
|
||||
object_store.clone(),
|
||||
&metadata,
|
||||
HashSet::from_iter([0]),
|
||||
env.get_puffin_manager(),
|
||||
)
|
||||
.with_puffin_metadata_cache(cache.puffin_metadata_cache().cloned())
|
||||
.with_inverted_index_cache(cache.inverted_index_cache().cloned())
|
||||
.build(exprs)
|
||||
.unwrap()
|
||||
.map(Arc::new)
|
||||
};
|
||||
|
||||
let build_bloom_filter_applier = |exprs: &[Expr]| {
|
||||
BloomFilterIndexApplierBuilder::new(
|
||||
FILE_DIR.to_string(),
|
||||
object_store.clone(),
|
||||
&metadata,
|
||||
env.get_puffin_manager(),
|
||||
)
|
||||
.with_puffin_metadata_cache(cache.puffin_metadata_cache().cloned())
|
||||
.with_bloom_filter_index_cache(cache.bloom_filter_index_cache().cloned())
|
||||
.build(exprs)
|
||||
.unwrap()
|
||||
.map(Arc::new)
|
||||
};
|
||||
|
||||
// Data: ts tag_0 tag_1
|
||||
// Data: 0-20 [a, d]
|
||||
// 0-20 [b, d]
|
||||
// 0-20 [c, d]
|
||||
// 0-40 [c, f]
|
||||
// 100-200 [c, h]
|
||||
//
|
||||
// Pred: tag_0 = "b"
|
||||
//
|
||||
// Row groups & rows pruning:
|
||||
//
|
||||
// Row Groups:
|
||||
// - min-max: filter out row groups 1..=3
|
||||
//
|
||||
// Rows:
|
||||
// - inverted index: hit row group 0, hit 20 rows
|
||||
let preds = vec![col("tag_0").eq(lit("b"))];
|
||||
let inverted_index_applier = build_inverted_index_applier(&preds);
|
||||
let bloom_filter_applier = build_bloom_filter_applier(&preds);
|
||||
|
||||
let builder =
|
||||
ParquetReaderBuilder::new(FILE_DIR.to_string(), handle.clone(), object_store.clone())
|
||||
.predicate(Some(Predicate::new(preds)))
|
||||
.inverted_index_applier(inverted_index_applier.clone())
|
||||
.bloom_filter_index_applier(bloom_filter_applier.clone())
|
||||
.cache(CacheStrategy::EnableAll(cache.clone()));
|
||||
|
||||
let mut metrics = ReaderMetrics::default();
|
||||
let (context, selection) = builder.build_reader_input(&mut metrics).await.unwrap();
|
||||
let mut reader = ParquetReader::new(Arc::new(context), selection)
|
||||
.await
|
||||
.unwrap();
|
||||
check_reader_result(&mut reader, &[new_batch_by_range(&["b", "d"], 0, 20)]).await;
|
||||
|
||||
assert_eq!(metrics.filter_metrics.rg_total, 4);
|
||||
assert_eq!(metrics.filter_metrics.rg_minmax_filtered, 3);
|
||||
assert_eq!(metrics.filter_metrics.rg_inverted_filtered, 0);
|
||||
assert_eq!(metrics.filter_metrics.rows_inverted_filtered, 30);
|
||||
let cached = index_result_cache
|
||||
.get(
|
||||
inverted_index_applier.unwrap().predicate_key(),
|
||||
handle.file_id(),
|
||||
)
|
||||
.unwrap();
|
||||
// inverted index will search all row groups
|
||||
assert!(cached.contains_row_group(0));
|
||||
assert!(cached.contains_row_group(1));
|
||||
assert!(cached.contains_row_group(2));
|
||||
assert!(cached.contains_row_group(3));
|
||||
|
||||
// Data: ts tag_0 tag_1
|
||||
// Data: 0-20 [a, d]
|
||||
// 0-20 [b, d]
|
||||
// 0-20 [c, d]
|
||||
// 0-40 [c, f]
|
||||
// 100-200 [c, h]
|
||||
//
|
||||
// Pred: 50 <= ts && ts < 200 && tag_1 = "d"
|
||||
//
|
||||
// Row groups & rows pruning:
|
||||
//
|
||||
// Row Groups:
|
||||
// - min-max: filter out row groups 0..=1
|
||||
// - bloom filter: filter out row groups 2..=3
|
||||
let preds = vec![
|
||||
col("ts").gt_eq(lit(ScalarValue::TimestampMillisecond(Some(50), None))),
|
||||
col("ts").lt(lit(ScalarValue::TimestampMillisecond(Some(200), None))),
|
||||
col("tag_1").eq(lit("d")),
|
||||
];
|
||||
let inverted_index_applier = build_inverted_index_applier(&preds);
|
||||
let bloom_filter_applier = build_bloom_filter_applier(&preds);
|
||||
|
||||
let builder =
|
||||
ParquetReaderBuilder::new(FILE_DIR.to_string(), handle.clone(), object_store.clone())
|
||||
.predicate(Some(Predicate::new(preds)))
|
||||
.inverted_index_applier(inverted_index_applier.clone())
|
||||
.bloom_filter_index_applier(bloom_filter_applier.clone())
|
||||
.cache(CacheStrategy::EnableAll(cache.clone()));
|
||||
|
||||
let mut metrics = ReaderMetrics::default();
|
||||
let (context, selection) = builder.build_reader_input(&mut metrics).await.unwrap();
|
||||
let mut reader = ParquetReader::new(Arc::new(context), selection)
|
||||
.await
|
||||
.unwrap();
|
||||
check_reader_result(&mut reader, &[]).await;
|
||||
|
||||
assert_eq!(metrics.filter_metrics.rg_total, 4);
|
||||
assert_eq!(metrics.filter_metrics.rg_minmax_filtered, 2);
|
||||
assert_eq!(metrics.filter_metrics.rg_bloom_filtered, 2);
|
||||
assert_eq!(metrics.filter_metrics.rows_bloom_filtered, 100);
|
||||
let cached = index_result_cache
|
||||
.get(
|
||||
bloom_filter_applier.unwrap().predicate_key(),
|
||||
handle.file_id(),
|
||||
)
|
||||
.unwrap();
|
||||
assert!(cached.contains_row_group(2));
|
||||
assert!(cached.contains_row_group(3));
|
||||
assert!(!cached.contains_row_group(0));
|
||||
assert!(!cached.contains_row_group(1));
|
||||
|
||||
// Remove the pred of `ts`, continue to use the pred of `tag_1`
|
||||
// to test if cache works.
|
||||
|
||||
// Data: ts tag_0 tag_1
|
||||
// Data: 0-20 [a, d]
|
||||
// 0-20 [b, d]
|
||||
// 0-20 [c, d]
|
||||
// 0-40 [c, f]
|
||||
// 100-200 [c, h]
|
||||
//
|
||||
// Pred: tag_1 = "d"
|
||||
//
|
||||
// Row groups & rows pruning:
|
||||
//
|
||||
// Row Groups:
|
||||
// - bloom filter: filter out row groups 2..=3
|
||||
//
|
||||
// Rows:
|
||||
// - bloom filter: hit row group 0, hit 50 rows
|
||||
// hit row group 1, hit 10 rows
|
||||
let preds = vec![col("tag_1").eq(lit("d"))];
|
||||
let inverted_index_applier = build_inverted_index_applier(&preds);
|
||||
let bloom_filter_applier = build_bloom_filter_applier(&preds);
|
||||
|
||||
let builder =
|
||||
ParquetReaderBuilder::new(FILE_DIR.to_string(), handle.clone(), object_store.clone())
|
||||
.predicate(Some(Predicate::new(preds)))
|
||||
.inverted_index_applier(inverted_index_applier.clone())
|
||||
.bloom_filter_index_applier(bloom_filter_applier.clone())
|
||||
.cache(CacheStrategy::EnableAll(cache.clone()));
|
||||
|
||||
let mut metrics = ReaderMetrics::default();
|
||||
let (context, selection) = builder.build_reader_input(&mut metrics).await.unwrap();
|
||||
let mut reader = ParquetReader::new(Arc::new(context), selection)
|
||||
.await
|
||||
.unwrap();
|
||||
check_reader_result(
|
||||
&mut reader,
|
||||
&[
|
||||
new_batch_by_range(&["a", "d"], 0, 20),
|
||||
new_batch_by_range(&["b", "d"], 0, 20),
|
||||
new_batch_by_range(&["c", "d"], 0, 10),
|
||||
new_batch_by_range(&["c", "d"], 10, 20),
|
||||
],
|
||||
)
|
||||
.await;
|
||||
|
||||
assert_eq!(metrics.filter_metrics.rg_total, 4);
|
||||
assert_eq!(metrics.filter_metrics.rg_minmax_filtered, 0);
|
||||
assert_eq!(metrics.filter_metrics.rg_bloom_filtered, 2);
|
||||
assert_eq!(metrics.filter_metrics.rows_bloom_filtered, 140);
|
||||
let cached = index_result_cache
|
||||
.get(
|
||||
bloom_filter_applier.unwrap().predicate_key(),
|
||||
handle.file_id(),
|
||||
)
|
||||
.unwrap();
|
||||
assert!(cached.contains_row_group(0));
|
||||
assert!(cached.contains_row_group(1));
|
||||
assert!(cached.contains_row_group(2));
|
||||
assert!(cached.contains_row_group(3));
|
||||
}
|
||||
}
|
||||
|
||||
@@ -378,14 +378,24 @@ impl ParquetReaderBuilder {
|
||||
}
|
||||
|
||||
let fulltext_filtered = self
|
||||
.prune_row_groups_by_fulltext_index(row_group_size, parquet_meta, &mut output, metrics)
|
||||
.prune_row_groups_by_fulltext_index(
|
||||
row_group_size,
|
||||
num_row_groups,
|
||||
&mut output,
|
||||
metrics,
|
||||
)
|
||||
.await;
|
||||
if output.is_empty() {
|
||||
return output;
|
||||
}
|
||||
|
||||
self.prune_row_groups_by_inverted_index(row_group_size, &mut output, metrics)
|
||||
.await;
|
||||
self.prune_row_groups_by_inverted_index(
|
||||
row_group_size,
|
||||
num_row_groups,
|
||||
&mut output,
|
||||
metrics,
|
||||
)
|
||||
.await;
|
||||
if output.is_empty() {
|
||||
return output;
|
||||
}
|
||||
@@ -412,7 +422,7 @@ impl ParquetReaderBuilder {
|
||||
async fn prune_row_groups_by_fulltext_index(
|
||||
&self,
|
||||
row_group_size: usize,
|
||||
parquet_meta: &ParquetMetaData,
|
||||
num_row_groups: usize,
|
||||
output: &mut RowGroupSelection,
|
||||
metrics: &mut ReaderFilterMetrics,
|
||||
) -> bool {
|
||||
@@ -425,14 +435,15 @@ impl ParquetReaderBuilder {
|
||||
|
||||
let predicate_key = index_applier.predicate_key();
|
||||
// Fast path: return early if the result is in the cache.
|
||||
if self.index_result_cache_get(
|
||||
predicate_key,
|
||||
self.file_handle.file_id(),
|
||||
output,
|
||||
metrics,
|
||||
INDEX_TYPE_FULLTEXT,
|
||||
) {
|
||||
return true;
|
||||
let cached = self
|
||||
.cache_strategy
|
||||
.index_result_cache()
|
||||
.and_then(|cache| cache.get(predicate_key, self.file_handle.file_id()));
|
||||
if let Some(result) = cached.as_ref() {
|
||||
if all_required_row_groups_searched(output, result) {
|
||||
apply_selection_and_update_metrics(output, result, metrics, INDEX_TYPE_FULLTEXT);
|
||||
return true;
|
||||
}
|
||||
}
|
||||
|
||||
// Slow path: apply the index from the file.
|
||||
@@ -441,9 +452,7 @@ impl ParquetReaderBuilder {
|
||||
.apply_fine(self.file_handle.file_id(), Some(file_size_hint))
|
||||
.await;
|
||||
let selection = match apply_res {
|
||||
Ok(Some(res)) => {
|
||||
RowGroupSelection::from_row_ids(res, row_group_size, parquet_meta.num_row_groups())
|
||||
}
|
||||
Ok(Some(res)) => RowGroupSelection::from_row_ids(res, row_group_size, num_row_groups),
|
||||
Ok(None) => return false,
|
||||
Err(err) => {
|
||||
handle_index_error!(err, self.file_handle, INDEX_TYPE_FULLTEXT);
|
||||
@@ -470,6 +479,7 @@ impl ParquetReaderBuilder {
|
||||
async fn prune_row_groups_by_inverted_index(
|
||||
&self,
|
||||
row_group_size: usize,
|
||||
num_row_groups: usize,
|
||||
output: &mut RowGroupSelection,
|
||||
metrics: &mut ReaderFilterMetrics,
|
||||
) -> bool {
|
||||
@@ -482,14 +492,15 @@ impl ParquetReaderBuilder {
|
||||
|
||||
let predicate_key = index_applier.predicate_key();
|
||||
// Fast path: return early if the result is in the cache.
|
||||
if self.index_result_cache_get(
|
||||
predicate_key,
|
||||
self.file_handle.file_id(),
|
||||
output,
|
||||
metrics,
|
||||
INDEX_TYPE_INVERTED,
|
||||
) {
|
||||
return true;
|
||||
let cached = self
|
||||
.cache_strategy
|
||||
.index_result_cache()
|
||||
.and_then(|cache| cache.get(predicate_key, self.file_handle.file_id()));
|
||||
if let Some(result) = cached.as_ref() {
|
||||
if all_required_row_groups_searched(output, result) {
|
||||
apply_selection_and_update_metrics(output, result, metrics, INDEX_TYPE_INVERTED);
|
||||
return true;
|
||||
}
|
||||
}
|
||||
|
||||
// Slow path: apply the index from the file.
|
||||
@@ -498,9 +509,11 @@ impl ParquetReaderBuilder {
|
||||
.apply(self.file_handle.file_id(), Some(file_size_hint))
|
||||
.await;
|
||||
let selection = match apply_res {
|
||||
Ok(output) => {
|
||||
RowGroupSelection::from_inverted_index_apply_output(row_group_size, output)
|
||||
}
|
||||
Ok(output) => RowGroupSelection::from_inverted_index_apply_output(
|
||||
row_group_size,
|
||||
num_row_groups,
|
||||
output,
|
||||
),
|
||||
Err(err) => {
|
||||
handle_index_error!(err, self.file_handle, INDEX_TYPE_INVERTED);
|
||||
return false;
|
||||
@@ -534,27 +547,34 @@ impl ParquetReaderBuilder {
|
||||
|
||||
let predicate_key = index_applier.predicate_key();
|
||||
// Fast path: return early if the result is in the cache.
|
||||
if self.index_result_cache_get(
|
||||
predicate_key,
|
||||
self.file_handle.file_id(),
|
||||
output,
|
||||
metrics,
|
||||
INDEX_TYPE_BLOOM,
|
||||
) {
|
||||
return true;
|
||||
let cached = self
|
||||
.cache_strategy
|
||||
.index_result_cache()
|
||||
.and_then(|cache| cache.get(predicate_key, self.file_handle.file_id()));
|
||||
if let Some(result) = cached.as_ref() {
|
||||
if all_required_row_groups_searched(output, result) {
|
||||
apply_selection_and_update_metrics(output, result, metrics, INDEX_TYPE_BLOOM);
|
||||
return true;
|
||||
}
|
||||
}
|
||||
|
||||
// Slow path: apply the index from the file.
|
||||
let file_size_hint = self.file_handle.meta_ref().index_file_size();
|
||||
let rgs = parquet_meta
|
||||
.row_groups()
|
||||
.iter()
|
||||
.enumerate()
|
||||
.map(|(i, rg)| (rg.num_rows() as usize, output.contains_row_group(i)));
|
||||
let rgs = parquet_meta.row_groups().iter().enumerate().map(|(i, rg)| {
|
||||
(
|
||||
rg.num_rows() as usize,
|
||||
// Optimize: only search the row group that required by `output` and not stored in `cached`.
|
||||
output.contains_non_empty_row_group(i)
|
||||
&& cached
|
||||
.as_ref()
|
||||
.map(|c| !c.contains_row_group(i))
|
||||
.unwrap_or(true),
|
||||
)
|
||||
});
|
||||
let apply_res = index_applier
|
||||
.apply(self.file_handle.file_id(), Some(file_size_hint), rgs)
|
||||
.await;
|
||||
let selection = match apply_res {
|
||||
let mut selection = match apply_res {
|
||||
Ok(apply_output) => RowGroupSelection::from_row_ranges(apply_output, row_group_size),
|
||||
Err(err) => {
|
||||
handle_index_error!(err, self.file_handle, INDEX_TYPE_BLOOM);
|
||||
@@ -562,6 +582,11 @@ impl ParquetReaderBuilder {
|
||||
}
|
||||
};
|
||||
|
||||
// New searched row groups are added to `selection`, concat them with `cached`.
|
||||
if let Some(cached) = cached.as_ref() {
|
||||
selection.concat(cached);
|
||||
}
|
||||
|
||||
self.apply_index_result_and_update_cache(
|
||||
predicate_key,
|
||||
self.file_handle.file_id(),
|
||||
@@ -589,27 +614,34 @@ impl ParquetReaderBuilder {
|
||||
|
||||
let predicate_key = index_applier.predicate_key();
|
||||
// Fast path: return early if the result is in the cache.
|
||||
if self.index_result_cache_get(
|
||||
predicate_key,
|
||||
self.file_handle.file_id(),
|
||||
output,
|
||||
metrics,
|
||||
INDEX_TYPE_FULLTEXT,
|
||||
) {
|
||||
return true;
|
||||
let cached = self
|
||||
.cache_strategy
|
||||
.index_result_cache()
|
||||
.and_then(|cache| cache.get(predicate_key, self.file_handle.file_id()));
|
||||
if let Some(result) = cached.as_ref() {
|
||||
if all_required_row_groups_searched(output, result) {
|
||||
apply_selection_and_update_metrics(output, result, metrics, INDEX_TYPE_FULLTEXT);
|
||||
return true;
|
||||
}
|
||||
}
|
||||
|
||||
// Slow path: apply the index from the file.
|
||||
let file_size_hint = self.file_handle.meta_ref().index_file_size();
|
||||
let rgs = parquet_meta
|
||||
.row_groups()
|
||||
.iter()
|
||||
.enumerate()
|
||||
.map(|(i, rg)| (rg.num_rows() as usize, output.contains_row_group(i)));
|
||||
let rgs = parquet_meta.row_groups().iter().enumerate().map(|(i, rg)| {
|
||||
(
|
||||
rg.num_rows() as usize,
|
||||
// Optimize: only search the row group that required by `output` and not stored in `cached`.
|
||||
output.contains_non_empty_row_group(i)
|
||||
&& cached
|
||||
.as_ref()
|
||||
.map(|c| !c.contains_row_group(i))
|
||||
.unwrap_or(true),
|
||||
)
|
||||
});
|
||||
let apply_res = index_applier
|
||||
.apply_coarse(self.file_handle.file_id(), Some(file_size_hint), rgs)
|
||||
.await;
|
||||
let selection = match apply_res {
|
||||
let mut selection = match apply_res {
|
||||
Ok(Some(apply_output)) => {
|
||||
RowGroupSelection::from_row_ranges(apply_output, row_group_size)
|
||||
}
|
||||
@@ -620,6 +652,11 @@ impl ParquetReaderBuilder {
|
||||
}
|
||||
};
|
||||
|
||||
// New searched row groups are added to `selection`, concat them with `cached`.
|
||||
if let Some(cached) = cached.as_ref() {
|
||||
selection.concat(cached);
|
||||
}
|
||||
|
||||
self.apply_index_result_and_update_cache(
|
||||
predicate_key,
|
||||
self.file_handle.file_id(),
|
||||
@@ -674,24 +711,6 @@ impl ParquetReaderBuilder {
|
||||
true
|
||||
}
|
||||
|
||||
fn index_result_cache_get(
|
||||
&self,
|
||||
predicate_key: &PredicateKey,
|
||||
file_id: FileId,
|
||||
output: &mut RowGroupSelection,
|
||||
metrics: &mut ReaderFilterMetrics,
|
||||
index_type: &str,
|
||||
) -> bool {
|
||||
if let Some(index_result_cache) = &self.cache_strategy.index_result_cache() {
|
||||
let result = index_result_cache.get(predicate_key, file_id);
|
||||
if let Some(result) = result {
|
||||
apply_selection_and_update_metrics(output, &result, metrics, index_type);
|
||||
return true;
|
||||
}
|
||||
}
|
||||
false
|
||||
}
|
||||
|
||||
fn apply_index_result_and_update_cache(
|
||||
&self,
|
||||
predicate_key: &PredicateKey,
|
||||
@@ -725,6 +744,18 @@ fn apply_selection_and_update_metrics(
|
||||
*output = intersection;
|
||||
}
|
||||
|
||||
fn all_required_row_groups_searched(
|
||||
required_row_groups: &RowGroupSelection,
|
||||
cached_row_groups: &RowGroupSelection,
|
||||
) -> bool {
|
||||
required_row_groups.iter().all(|(rg_id, _)| {
|
||||
// Row group with no rows is not required to search.
|
||||
!required_row_groups.contains_non_empty_row_group(*rg_id)
|
||||
// The row group is already searched.
|
||||
|| cached_row_groups.contains_row_group(*rg_id)
|
||||
})
|
||||
}
|
||||
|
||||
/// Metrics of filtering rows groups and rows.
|
||||
#[derive(Debug, Default, Clone, Copy)]
|
||||
pub(crate) struct ReaderFilterMetrics {
|
||||
@@ -1131,7 +1162,10 @@ impl Drop for ParquetReader {
|
||||
|
||||
impl ParquetReader {
|
||||
/// Creates a new reader.
|
||||
async fn new(context: FileRangeContextRef, mut selection: RowGroupSelection) -> Result<Self> {
|
||||
pub(crate) async fn new(
|
||||
context: FileRangeContextRef,
|
||||
mut selection: RowGroupSelection,
|
||||
) -> Result<Self> {
|
||||
// No more items in current row group, reads next row group.
|
||||
let reader_state = if let Some((row_group_idx, row_selection)) = selection.pop_first() {
|
||||
let parquet_reader = context
|
||||
|
||||
@@ -31,7 +31,7 @@ pub struct RowGroupSelection {
|
||||
}
|
||||
|
||||
/// A row selection with its count.
|
||||
#[derive(Debug, Clone)]
|
||||
#[derive(Debug, Clone, Default)]
|
||||
struct RowSelectionWithCount {
|
||||
/// Row selection.
|
||||
selection: RowSelection,
|
||||
@@ -95,6 +95,7 @@ impl RowGroupSelection {
|
||||
/// * The last row group may have fewer rows than `row_group_size`
|
||||
pub fn from_inverted_index_apply_output(
|
||||
row_group_size: usize,
|
||||
num_row_groups: usize,
|
||||
apply_output: ApplyOutput,
|
||||
) -> Self {
|
||||
// Step 1: Convert segment IDs to row ranges within row groups
|
||||
@@ -116,7 +117,7 @@ impl RowGroupSelection {
|
||||
// Step 2: Group ranges by row group ID and create row selections
|
||||
let mut total_row_count = 0;
|
||||
let mut total_selector_len = 0;
|
||||
let selection_in_rg = row_group_ranges
|
||||
let mut selection_in_rg = row_group_ranges
|
||||
.chunk_by(|(row_group_id, _)| *row_group_id)
|
||||
.into_iter()
|
||||
.map(|(row_group_id, group)| {
|
||||
@@ -141,7 +142,9 @@ impl RowGroupSelection {
|
||||
},
|
||||
)
|
||||
})
|
||||
.collect();
|
||||
.collect::<BTreeMap<_, _>>();
|
||||
|
||||
Self::fill_missing_row_groups(&mut selection_in_rg, num_row_groups);
|
||||
|
||||
Self {
|
||||
selection_in_rg,
|
||||
@@ -173,7 +176,7 @@ impl RowGroupSelection {
|
||||
// Step 2: Create row selections for each row group
|
||||
let mut total_row_count = 0;
|
||||
let mut total_selector_len = 0;
|
||||
let selection_in_rg = row_group_to_row_ids
|
||||
let mut selection_in_rg = row_group_to_row_ids
|
||||
.into_iter()
|
||||
.map(|(row_group_id, row_ids)| {
|
||||
let selection =
|
||||
@@ -191,7 +194,9 @@ impl RowGroupSelection {
|
||||
},
|
||||
)
|
||||
})
|
||||
.collect();
|
||||
.collect::<BTreeMap<_, _>>();
|
||||
|
||||
Self::fill_missing_row_groups(&mut selection_in_rg, num_row_groups);
|
||||
|
||||
Self {
|
||||
selection_in_rg,
|
||||
@@ -324,19 +329,26 @@ impl RowGroupSelection {
|
||||
}
|
||||
|
||||
/// Returns the first row group in the selection.
|
||||
///
|
||||
/// Skip the row group if the row count is 0.
|
||||
pub fn pop_first(&mut self) -> Option<(usize, RowSelection)> {
|
||||
let (
|
||||
while let Some((
|
||||
row_group_id,
|
||||
RowSelectionWithCount {
|
||||
selection,
|
||||
row_count,
|
||||
selector_len,
|
||||
},
|
||||
) = self.selection_in_rg.pop_first()?;
|
||||
)) = self.selection_in_rg.pop_first()
|
||||
{
|
||||
if row_count > 0 {
|
||||
self.row_count -= row_count;
|
||||
self.selector_len -= selector_len;
|
||||
return Some((row_group_id, selection));
|
||||
}
|
||||
}
|
||||
|
||||
self.row_count -= row_count;
|
||||
self.selector_len -= selector_len;
|
||||
Some((row_group_id, selection))
|
||||
None
|
||||
}
|
||||
|
||||
/// Removes a row group from the selection.
|
||||
@@ -363,6 +375,14 @@ impl RowGroupSelection {
|
||||
self.selection_in_rg.contains_key(&row_group_id)
|
||||
}
|
||||
|
||||
/// Returns true if the selection contains a row group with the given ID and the row selection is not empty.
|
||||
pub fn contains_non_empty_row_group(&self, row_group_id: usize) -> bool {
|
||||
self.selection_in_rg
|
||||
.get(&row_group_id)
|
||||
.map(|r| r.row_count > 0)
|
||||
.unwrap_or(false)
|
||||
}
|
||||
|
||||
/// Returns an iterator over the row groups in the selection.
|
||||
pub fn iter(&self) -> impl Iterator<Item = (&usize, &RowSelection)> {
|
||||
self.selection_in_rg
|
||||
@@ -375,6 +395,32 @@ impl RowGroupSelection {
|
||||
self.selector_len * size_of::<RowSelector>()
|
||||
+ self.selection_in_rg.len() * size_of::<RowSelectionWithCount>()
|
||||
}
|
||||
|
||||
/// Concatenates `other` into `self`. `other` must not contain row groups that `self` contains.
|
||||
///
|
||||
/// Panics if `self` contains row groups that `other` contains.
|
||||
pub fn concat(&mut self, other: &Self) {
|
||||
for (rg_id, other_rs) in other.selection_in_rg.iter() {
|
||||
if self.selection_in_rg.contains_key(rg_id) {
|
||||
panic!("row group {} is already in `self`", rg_id);
|
||||
}
|
||||
|
||||
self.selection_in_rg.insert(*rg_id, other_rs.clone());
|
||||
self.row_count += other_rs.row_count;
|
||||
self.selector_len += other_rs.selector_len;
|
||||
}
|
||||
}
|
||||
|
||||
/// Fills the missing row groups with empty selections.
|
||||
/// This is to indicate that the row groups are searched even if no rows are found.
|
||||
fn fill_missing_row_groups(
|
||||
selection_in_rg: &mut BTreeMap<usize, RowSelectionWithCount>,
|
||||
num_row_groups: usize,
|
||||
) {
|
||||
for rg_id in 0..num_row_groups {
|
||||
selection_in_rg.entry(rg_id).or_default();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/// Converts an iterator of row ranges into a `RowSelection` by creating a sequence of `RowSelector`s.
|
||||
@@ -678,15 +724,14 @@ mod tests {
|
||||
let selection =
|
||||
RowGroupSelection::from_row_ids(empty_row_ids, row_group_size, num_row_groups);
|
||||
assert_eq!(selection.row_count(), 0);
|
||||
assert_eq!(selection.row_group_count(), 0);
|
||||
assert!(selection.get(0).is_none());
|
||||
assert_eq!(selection.row_group_count(), 3);
|
||||
|
||||
// Test with consecutive row IDs
|
||||
let consecutive_row_ids: BTreeSet<u32> = vec![5, 6, 7, 8, 9].into_iter().collect();
|
||||
let selection =
|
||||
RowGroupSelection::from_row_ids(consecutive_row_ids, row_group_size, num_row_groups);
|
||||
assert_eq!(selection.row_count(), 5);
|
||||
assert_eq!(selection.row_group_count(), 1);
|
||||
assert_eq!(selection.row_group_count(), 3);
|
||||
|
||||
let row_selection = selection.get(0).unwrap();
|
||||
assert_eq!(row_selection.row_count(), 5); // 5, 6, 7, 8, 9
|
||||
@@ -1047,4 +1092,25 @@ mod tests {
|
||||
let empty_selection = RowGroupSelection::from_row_ranges(vec![], row_group_size);
|
||||
assert!(!empty_selection.contains_row_group(0));
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_concat() {
|
||||
let row_group_size = 100;
|
||||
let ranges1 = vec![
|
||||
(0, vec![5..15]), // Within [0, 100)
|
||||
(1, vec![5..15]), // Within [0, 100)
|
||||
];
|
||||
|
||||
let ranges2 = vec![
|
||||
(2, vec![5..15]), // Within [0, 100)
|
||||
(3, vec![5..15]), // Within [0, 100)
|
||||
];
|
||||
|
||||
let mut selection1 = RowGroupSelection::from_row_ranges(ranges1, row_group_size);
|
||||
let selection2 = RowGroupSelection::from_row_ranges(ranges2, row_group_size);
|
||||
|
||||
selection1.concat(&selection2);
|
||||
assert_eq!(selection1.row_count(), 40);
|
||||
assert_eq!(selection1.row_group_count(), 4);
|
||||
}
|
||||
}
|
||||
|
||||
@@ -202,6 +202,8 @@ pub(crate) enum LogStoreImpl {
|
||||
pub struct TestEnv {
|
||||
/// Path to store data.
|
||||
data_home: TempDir,
|
||||
intermediate_manager: IntermediateManager,
|
||||
puffin_manager: PuffinManagerFactory,
|
||||
log_store: Option<LogStoreImpl>,
|
||||
log_store_factory: LogStoreFactory,
|
||||
object_store_manager: Option<ObjectStoreManagerRef>,
|
||||
@@ -209,44 +211,33 @@ pub struct TestEnv {
|
||||
kv_backend: KvBackendRef,
|
||||
}
|
||||
|
||||
impl Default for TestEnv {
|
||||
fn default() -> Self {
|
||||
TestEnv::new()
|
||||
}
|
||||
}
|
||||
|
||||
impl TestEnv {
|
||||
/// Returns a new env with empty prefix for test.
|
||||
pub fn new() -> TestEnv {
|
||||
let (schema_metadata_manager, kv_backend) = mock_schema_metadata_manager();
|
||||
TestEnv {
|
||||
data_home: create_temp_dir(""),
|
||||
log_store: None,
|
||||
log_store_factory: LogStoreFactory::RaftEngine(RaftEngineLogStoreFactory),
|
||||
object_store_manager: None,
|
||||
schema_metadata_manager,
|
||||
kv_backend,
|
||||
}
|
||||
pub async fn new() -> TestEnv {
|
||||
Self::with_prefix("").await
|
||||
}
|
||||
|
||||
/// Returns a new env with specific `prefix` for test.
|
||||
pub fn with_prefix(prefix: &str) -> TestEnv {
|
||||
let (schema_metadata_manager, kv_backend) = mock_schema_metadata_manager();
|
||||
TestEnv {
|
||||
data_home: create_temp_dir(prefix),
|
||||
log_store: None,
|
||||
log_store_factory: LogStoreFactory::RaftEngine(RaftEngineLogStoreFactory),
|
||||
object_store_manager: None,
|
||||
schema_metadata_manager,
|
||||
kv_backend,
|
||||
}
|
||||
pub async fn with_prefix(prefix: &str) -> TestEnv {
|
||||
Self::with_data_home(create_temp_dir(prefix)).await
|
||||
}
|
||||
|
||||
/// Returns a new env with specific `data_home` for test.
|
||||
pub fn with_data_home(data_home: TempDir) -> TestEnv {
|
||||
pub async fn with_data_home(data_home: TempDir) -> TestEnv {
|
||||
let (schema_metadata_manager, kv_backend) = mock_schema_metadata_manager();
|
||||
|
||||
let index_aux_path = data_home.path().join("index_aux");
|
||||
let puffin_manager = PuffinManagerFactory::new(&index_aux_path, 4096, None, None)
|
||||
.await
|
||||
.unwrap();
|
||||
let intermediate_manager = IntermediateManager::init_fs(index_aux_path.to_str().unwrap())
|
||||
.await
|
||||
.unwrap();
|
||||
|
||||
TestEnv {
|
||||
data_home,
|
||||
intermediate_manager,
|
||||
puffin_manager,
|
||||
log_store: None,
|
||||
log_store_factory: LogStoreFactory::RaftEngine(RaftEngineLogStoreFactory),
|
||||
object_store_manager: None,
|
||||
@@ -587,17 +578,15 @@ impl TestEnv {
|
||||
local_store: ObjectStore,
|
||||
capacity: ReadableSize,
|
||||
) -> WriteCacheRef {
|
||||
let index_aux_path = self.data_home.path().join("index_aux");
|
||||
let puffin_mgr = PuffinManagerFactory::new(&index_aux_path, 4096, None, None)
|
||||
.await
|
||||
.unwrap();
|
||||
let intm_mgr = IntermediateManager::init_fs(index_aux_path.to_str().unwrap())
|
||||
.await
|
||||
.unwrap();
|
||||
|
||||
let write_cache = WriteCache::new(local_store, capacity, None, puffin_mgr, intm_mgr)
|
||||
.await
|
||||
.unwrap();
|
||||
let write_cache = WriteCache::new(
|
||||
local_store,
|
||||
capacity,
|
||||
None,
|
||||
self.puffin_manager.clone(),
|
||||
self.intermediate_manager.clone(),
|
||||
)
|
||||
.await
|
||||
.unwrap();
|
||||
|
||||
Arc::new(write_cache)
|
||||
}
|
||||
@@ -608,17 +597,15 @@ impl TestEnv {
|
||||
path: &str,
|
||||
capacity: ReadableSize,
|
||||
) -> WriteCacheRef {
|
||||
let index_aux_path = self.data_home.path().join("index_aux");
|
||||
let puffin_mgr = PuffinManagerFactory::new(&index_aux_path, 4096, None, None)
|
||||
.await
|
||||
.unwrap();
|
||||
let intm_mgr = IntermediateManager::init_fs(index_aux_path.to_str().unwrap())
|
||||
.await
|
||||
.unwrap();
|
||||
|
||||
let write_cache = WriteCache::new_fs(path, capacity, None, puffin_mgr, intm_mgr)
|
||||
.await
|
||||
.unwrap();
|
||||
let write_cache = WriteCache::new_fs(
|
||||
path,
|
||||
capacity,
|
||||
None,
|
||||
self.puffin_manager.clone(),
|
||||
self.intermediate_manager.clone(),
|
||||
)
|
||||
.await
|
||||
.unwrap();
|
||||
|
||||
Arc::new(write_cache)
|
||||
}
|
||||
@@ -634,6 +621,14 @@ impl TestEnv {
|
||||
pub(crate) fn get_log_store(&self) -> Option<LogStoreImpl> {
|
||||
self.log_store.as_ref().cloned()
|
||||
}
|
||||
|
||||
pub fn get_puffin_manager(&self) -> PuffinManagerFactory {
|
||||
self.puffin_manager.clone()
|
||||
}
|
||||
|
||||
pub fn get_intermediate_manager(&self) -> IntermediateManager {
|
||||
self.intermediate_manager.clone()
|
||||
}
|
||||
}
|
||||
|
||||
/// Builder to mock a [RegionCreateRequest].
|
||||
|
||||
@@ -20,7 +20,7 @@ use api::v1::{OpType, SemanticType};
|
||||
use common_time::Timestamp;
|
||||
use datatypes::arrow::array::{BinaryArray, TimestampMillisecondArray, UInt64Array, UInt8Array};
|
||||
use datatypes::prelude::ConcreteDataType;
|
||||
use datatypes::schema::ColumnSchema;
|
||||
use datatypes::schema::{ColumnSchema, SkippingIndexOptions};
|
||||
use datatypes::value::ValueRef;
|
||||
use mito_codec::row_converter::{DensePrimaryKeyCodec, PrimaryKeyCodecExt, SortField};
|
||||
use parquet::file::metadata::ParquetMetaData;
|
||||
@@ -57,7 +57,12 @@ pub fn sst_region_metadata() -> RegionMetadata {
|
||||
"tag_1".to_string(),
|
||||
ConcreteDataType::string_datatype(),
|
||||
true,
|
||||
),
|
||||
)
|
||||
.with_skipping_options(SkippingIndexOptions {
|
||||
granularity: 1,
|
||||
..Default::default()
|
||||
})
|
||||
.unwrap(),
|
||||
semantic_type: SemanticType::Tag,
|
||||
column_id: 1,
|
||||
})
|
||||
|
||||
@@ -1142,7 +1142,7 @@ mod tests {
|
||||
|
||||
#[tokio::test]
|
||||
async fn test_worker_group_start_stop() {
|
||||
let env = TestEnv::with_prefix("group-stop");
|
||||
let env = TestEnv::with_prefix("group-stop").await;
|
||||
let group = env
|
||||
.create_worker_group(MitoConfig {
|
||||
num_workers: 4,
|
||||
|
||||
@@ -23,7 +23,6 @@ use datafusion::parquet;
|
||||
use datatypes::arrow::error::ArrowError;
|
||||
use snafu::{Location, Snafu};
|
||||
use table::metadata::TableType;
|
||||
use tokio::time::error::Elapsed;
|
||||
|
||||
#[derive(Snafu)]
|
||||
#[snafu(visibility(pub))]
|
||||
@@ -786,14 +785,6 @@ pub enum Error {
|
||||
json: String,
|
||||
},
|
||||
|
||||
#[snafu(display("Canceling statement due to statement timeout"))]
|
||||
StatementTimeout {
|
||||
#[snafu(implicit)]
|
||||
location: Location,
|
||||
#[snafu(source)]
|
||||
error: Elapsed,
|
||||
},
|
||||
|
||||
#[snafu(display("Cursor {name} is not found"))]
|
||||
CursorNotFound { name: String },
|
||||
|
||||
@@ -983,7 +974,6 @@ impl ErrorExt for Error {
|
||||
Error::ExecuteAdminFunction { source, .. } => source.status_code(),
|
||||
Error::BuildRecordBatch { source, .. } => source.status_code(),
|
||||
Error::UpgradeCatalogManagerRef { .. } => StatusCode::Internal,
|
||||
Error::StatementTimeout { .. } => StatusCode::Cancelled,
|
||||
Error::ColumnOptions { source, .. } => source.status_code(),
|
||||
Error::DecodeFlightData { source, .. } => source.status_code(),
|
||||
Error::ComputeArrow { .. } => StatusCode::Internal,
|
||||
|
||||
@@ -593,6 +593,8 @@ pub(crate) fn to_alter_table_expr(
|
||||
FulltextBackend::Bloom => PbFulltextBackend::Bloom.into(),
|
||||
FulltextBackend::Tantivy => PbFulltextBackend::Tantivy.into(),
|
||||
},
|
||||
granularity: options.granularity as u64,
|
||||
false_positive_rate: options.false_positive_rate(),
|
||||
})),
|
||||
},
|
||||
sql::statements::alter::SetIndexOperation::Inverted { column_name } => SetIndex {
|
||||
@@ -608,6 +610,7 @@ pub(crate) fn to_alter_table_expr(
|
||||
column_name: column_name.value,
|
||||
enable: true,
|
||||
granularity: options.granularity as u64,
|
||||
false_positive_rate: options.false_positive_rate(),
|
||||
skipping_index_type: match options.index_type {
|
||||
SkippingIndexType::BloomFilter => PbSkippingIndexType::BloomFilter.into(),
|
||||
},
|
||||
|
||||
@@ -27,15 +27,12 @@ mod show;
|
||||
mod tql;
|
||||
|
||||
use std::collections::HashMap;
|
||||
use std::pin::Pin;
|
||||
use std::sync::Arc;
|
||||
use std::time::Duration;
|
||||
|
||||
use async_stream::stream;
|
||||
use catalog::kvbackend::KvBackendCatalogManager;
|
||||
use catalog::process_manager::ProcessManagerRef;
|
||||
use catalog::CatalogManagerRef;
|
||||
use client::{OutputData, RecordBatches};
|
||||
use client::RecordBatches;
|
||||
use common_error::ext::BoxedError;
|
||||
use common_meta::cache::TableRouteCacheRef;
|
||||
use common_meta::cache_invalidator::CacheInvalidatorRef;
|
||||
@@ -46,13 +43,10 @@ use common_meta::key::view_info::{ViewInfoManager, ViewInfoManagerRef};
|
||||
use common_meta::key::{TableMetadataManager, TableMetadataManagerRef};
|
||||
use common_meta::kv_backend::KvBackendRef;
|
||||
use common_query::Output;
|
||||
use common_recordbatch::error::StreamTimeoutSnafu;
|
||||
use common_recordbatch::RecordBatchStreamWrapper;
|
||||
use common_telemetry::tracing;
|
||||
use common_time::range::TimestampRange;
|
||||
use common_time::Timestamp;
|
||||
use datafusion_expr::LogicalPlan;
|
||||
use futures::stream::{Stream, StreamExt};
|
||||
use partition::manager::{PartitionRuleManager, PartitionRuleManagerRef};
|
||||
use query::parser::QueryStatement;
|
||||
use query::QueryEngineRef;
|
||||
@@ -79,8 +73,8 @@ use self::set::{
|
||||
};
|
||||
use crate::error::{
|
||||
self, CatalogSnafu, ExecLogicalPlanSnafu, ExternalSnafu, InvalidSqlSnafu, NotSupportedSnafu,
|
||||
PlanStatementSnafu, Result, SchemaNotFoundSnafu, StatementTimeoutSnafu,
|
||||
TableMetadataManagerSnafu, TableNotFoundSnafu, UpgradeCatalogManagerRefSnafu,
|
||||
PlanStatementSnafu, Result, SchemaNotFoundSnafu, TableMetadataManagerSnafu, TableNotFoundSnafu,
|
||||
UpgradeCatalogManagerRefSnafu,
|
||||
};
|
||||
use crate::insert::InserterRef;
|
||||
use crate::statement::copy_database::{COPY_DATABASE_TIME_END_KEY, COPY_DATABASE_TIME_START_KEY};
|
||||
@@ -490,19 +484,8 @@ impl StatementExecutor {
|
||||
|
||||
#[tracing::instrument(skip_all)]
|
||||
async fn plan_exec(&self, stmt: QueryStatement, query_ctx: QueryContextRef) -> Result<Output> {
|
||||
let timeout = derive_timeout(&stmt, &query_ctx);
|
||||
match timeout {
|
||||
Some(timeout) => {
|
||||
let start = tokio::time::Instant::now();
|
||||
let output = tokio::time::timeout(timeout, self.plan_exec_inner(stmt, query_ctx))
|
||||
.await
|
||||
.context(StatementTimeoutSnafu)?;
|
||||
// compute remaining timeout
|
||||
let remaining_timeout = timeout.checked_sub(start.elapsed()).unwrap_or_default();
|
||||
Ok(attach_timeout(output?, remaining_timeout))
|
||||
}
|
||||
None => self.plan_exec_inner(stmt, query_ctx).await,
|
||||
}
|
||||
let plan = self.plan(&stmt, query_ctx.clone()).await?;
|
||||
self.exec_plan(plan, query_ctx).await
|
||||
}
|
||||
|
||||
async fn get_table(&self, table_ref: &TableReference<'_>) -> Result<TableRef> {
|
||||
@@ -519,49 +502,6 @@ impl StatementExecutor {
|
||||
table_name: table_ref.to_string(),
|
||||
})
|
||||
}
|
||||
|
||||
async fn plan_exec_inner(
|
||||
&self,
|
||||
stmt: QueryStatement,
|
||||
query_ctx: QueryContextRef,
|
||||
) -> Result<Output> {
|
||||
let plan = self.plan(&stmt, query_ctx.clone()).await?;
|
||||
self.exec_plan(plan, query_ctx).await
|
||||
}
|
||||
}
|
||||
|
||||
fn attach_timeout(output: Output, mut timeout: Duration) -> Output {
|
||||
match output.data {
|
||||
OutputData::AffectedRows(_) | OutputData::RecordBatches(_) => output,
|
||||
OutputData::Stream(mut stream) => {
|
||||
let schema = stream.schema();
|
||||
let s = Box::pin(stream! {
|
||||
let start = tokio::time::Instant::now();
|
||||
while let Some(item) = tokio::time::timeout(timeout, stream.next()).await.context(StreamTimeoutSnafu)? {
|
||||
yield item;
|
||||
timeout = timeout.checked_sub(tokio::time::Instant::now() - start).unwrap_or(Duration::ZERO);
|
||||
}
|
||||
}) as Pin<Box<dyn Stream<Item = _> + Send>>;
|
||||
let stream = RecordBatchStreamWrapper {
|
||||
schema,
|
||||
stream: s,
|
||||
output_ordering: None,
|
||||
metrics: Default::default(),
|
||||
};
|
||||
Output::new(OutputData::Stream(Box::pin(stream)), output.meta)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/// If the relevant variables are set, the timeout is enforced for all PostgreSQL statements.
|
||||
/// For MySQL, it applies only to read-only statements.
|
||||
fn derive_timeout(stmt: &QueryStatement, query_ctx: &QueryContextRef) -> Option<Duration> {
|
||||
let query_timeout = query_ctx.query_timeout()?;
|
||||
match (query_ctx.channel(), stmt) {
|
||||
(Channel::Mysql, QueryStatement::Sql(Statement::Query(_)))
|
||||
| (Channel::Postgres, QueryStatement::Sql(_)) => Some(query_timeout),
|
||||
(_, _) => None,
|
||||
}
|
||||
}
|
||||
|
||||
fn to_copy_query_request(stmt: CopyQueryToArgument) -> Result<CopyQueryToRequest> {
|
||||
|
||||
@@ -1569,7 +1569,7 @@ fn parse_partitions(
|
||||
}
|
||||
}
|
||||
}
|
||||
MultiDimPartitionRule::try_new(partition_columns.clone(), vec![], exprs)
|
||||
MultiDimPartitionRule::try_new(partition_columns.clone(), vec![], exprs, true)
|
||||
.context(InvalidPartitionSnafu)?;
|
||||
|
||||
Ok((
|
||||
|
||||
@@ -112,7 +112,7 @@ fn create_test_rule(num_columns: usize) -> MultiDimPartitionRule {
|
||||
};
|
||||
|
||||
let regions = (0..exprs.len()).map(|v| v as u32).collect();
|
||||
MultiDimPartitionRule::try_new(columns, regions, exprs).unwrap()
|
||||
MultiDimPartitionRule::try_new(columns, regions, exprs, true).unwrap()
|
||||
}
|
||||
|
||||
fn create_test_batch(size: usize) -> RecordBatch {
|
||||
|
||||
604
src/partition/src/checker.rs
Normal file
604
src/partition/src/checker.rs
Normal file
@@ -0,0 +1,604 @@
|
||||
// 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::collections::{BTreeMap, HashMap};
|
||||
use std::sync::Arc;
|
||||
|
||||
use datatypes::arrow::array::{BooleanArray, Float64Array, Float64Builder, RecordBatch};
|
||||
use datatypes::arrow::datatypes::{DataType, Field, Schema};
|
||||
use datatypes::value::OrderedF64;
|
||||
|
||||
use crate::collider::{Collider, CHECK_STEP, NORMALIZE_STEP};
|
||||
use crate::error::{
|
||||
CheckpointNotCoveredSnafu, CheckpointOverlappedSnafu, DuplicateExprSnafu, Result,
|
||||
};
|
||||
use crate::expr::{PartitionExpr, RestrictedOp};
|
||||
use crate::multi_dim::MultiDimPartitionRule;
|
||||
|
||||
pub struct PartitionChecker<'a> {
|
||||
rule: &'a MultiDimPartitionRule,
|
||||
collider: Collider<'a>,
|
||||
}
|
||||
|
||||
impl<'a> PartitionChecker<'a> {
|
||||
pub fn try_new(rule: &'a MultiDimPartitionRule) -> Result<Self> {
|
||||
let collider = Collider::new(rule.exprs())?;
|
||||
Ok(Self { rule, collider })
|
||||
}
|
||||
|
||||
pub fn check(&self) -> Result<()> {
|
||||
self.run()?;
|
||||
Ok(())
|
||||
}
|
||||
}
|
||||
|
||||
// Logic of checking rules
|
||||
impl<'a> PartitionChecker<'a> {
|
||||
fn run(&self) -> Result<()> {
|
||||
// Sort atomic exprs and check uniqueness
|
||||
let mut atomic_exprs = BTreeMap::new();
|
||||
for expr in self.collider.atomic_exprs.iter() {
|
||||
let key = &expr.nucleons;
|
||||
atomic_exprs.insert(key, expr);
|
||||
}
|
||||
if atomic_exprs.len() != self.collider.atomic_exprs.len() {
|
||||
// Find the duplication for error message
|
||||
for expr in self.collider.atomic_exprs.iter() {
|
||||
if atomic_exprs.get(&expr.nucleons).unwrap().source_expr_index
|
||||
!= expr.source_expr_index
|
||||
{
|
||||
let expr = self.rule.exprs()[expr.source_expr_index].clone();
|
||||
return DuplicateExprSnafu { expr }.fail();
|
||||
}
|
||||
}
|
||||
// Or return a placeholder. This should never happen.
|
||||
return DuplicateExprSnafu {
|
||||
expr: PartitionExpr::new(
|
||||
crate::expr::Operand::Column("unknown".to_string()),
|
||||
RestrictedOp::Eq,
|
||||
crate::expr::Operand::Column("expr".to_string()),
|
||||
),
|
||||
}
|
||||
.fail();
|
||||
}
|
||||
|
||||
// TODO(ruihang): merge atomic exprs to improve checker's performance
|
||||
|
||||
// matrix test
|
||||
let mut matrix_foundation = HashMap::new();
|
||||
for (col, values) in self.collider.normalized_values.iter() {
|
||||
if values.is_empty() {
|
||||
continue;
|
||||
}
|
||||
|
||||
let mut cornerstones = Vec::with_capacity(values.len() * 2 + 1);
|
||||
cornerstones.push(values[0].1 - CHECK_STEP);
|
||||
for value in values {
|
||||
cornerstones.push(value.1);
|
||||
cornerstones.push(value.1 + CHECK_STEP);
|
||||
}
|
||||
matrix_foundation.insert(col.as_str(), cornerstones);
|
||||
}
|
||||
|
||||
// If there are no values, the rule is empty and valid.
|
||||
if matrix_foundation.is_empty() {
|
||||
return Ok(());
|
||||
}
|
||||
|
||||
let matrix_generator = MatrixGenerator::new(matrix_foundation);
|
||||
|
||||
// Process data in batches using iterator
|
||||
let mut results = Vec::with_capacity(self.collider.atomic_exprs.len());
|
||||
let physical_exprs = self
|
||||
.collider
|
||||
.atomic_exprs
|
||||
.iter()
|
||||
.map(|expr| expr.to_physical_expr(matrix_generator.schema()))
|
||||
.collect::<Vec<_>>();
|
||||
for batch in matrix_generator {
|
||||
results.clear();
|
||||
for physical_expr in &physical_exprs {
|
||||
let columnar_result = physical_expr.evaluate(&batch).unwrap();
|
||||
let array_result = columnar_result.into_array(batch.num_rows()).unwrap();
|
||||
results.push(array_result);
|
||||
}
|
||||
let boolean_results = results
|
||||
.iter()
|
||||
.map(|result| result.as_any().downcast_ref::<BooleanArray>().unwrap())
|
||||
.collect::<Vec<_>>();
|
||||
|
||||
// sum and check results for this batch
|
||||
for i in 0..batch.num_rows() {
|
||||
let mut true_count = 0;
|
||||
for result in boolean_results.iter() {
|
||||
if result.value(i) {
|
||||
true_count += 1;
|
||||
}
|
||||
}
|
||||
|
||||
if true_count == 0 {
|
||||
return CheckpointNotCoveredSnafu {
|
||||
checkpoint: self.remap_checkpoint(i, &batch),
|
||||
}
|
||||
.fail();
|
||||
} else if true_count > 1 {
|
||||
return CheckpointOverlappedSnafu {
|
||||
checkpoint: self.remap_checkpoint(i, &batch),
|
||||
}
|
||||
.fail();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
Ok(())
|
||||
}
|
||||
|
||||
/// Remap the normalized checkpoint data to the original values.
|
||||
fn remap_checkpoint(&self, i: usize, batch: &RecordBatch) -> String {
|
||||
let normalized_row = batch
|
||||
.columns()
|
||||
.iter()
|
||||
.map(|col| {
|
||||
let array = col.as_any().downcast_ref::<Float64Array>().unwrap();
|
||||
array.value(i)
|
||||
})
|
||||
.collect::<Vec<_>>();
|
||||
|
||||
let mut check_point = String::new();
|
||||
let schema = batch.schema();
|
||||
for (col_index, normalized_value) in normalized_row.iter().enumerate() {
|
||||
let col_name = schema.field(col_index).name();
|
||||
|
||||
if col_index > 0 {
|
||||
check_point.push_str(", ");
|
||||
}
|
||||
|
||||
// Check if point is on NORMALIZE_STEP or between steps
|
||||
if let Some(values) = self.collider.normalized_values.get(col_name) {
|
||||
let normalize_step = NORMALIZE_STEP.0;
|
||||
|
||||
// Check if the normalized value is on a NORMALIZE_STEP boundary
|
||||
let remainder = normalized_value % normalize_step;
|
||||
let is_on_step = remainder.abs() < f64::EPSILON
|
||||
|| (normalize_step - remainder).abs() < f64::EPSILON * 2.0;
|
||||
|
||||
if is_on_step {
|
||||
let index = (normalized_value / normalize_step).round() as usize;
|
||||
if index < values.len() {
|
||||
let original_value = &values[index].0;
|
||||
check_point.push_str(&format!("{}={}", col_name, original_value));
|
||||
} else {
|
||||
check_point.push_str(&format!("{}=unknown", col_name));
|
||||
}
|
||||
} else {
|
||||
let lower_index = (normalized_value / normalize_step).floor() as usize;
|
||||
let upper_index = (normalized_value / normalize_step).ceil() as usize;
|
||||
|
||||
// Handle edge cases: value is outside the valid range
|
||||
if lower_index == upper_index && lower_index == 0 {
|
||||
// Value is less than the first value
|
||||
let first_original = &values[0].0;
|
||||
check_point.push_str(&format!("{}<{}", col_name, first_original));
|
||||
} else if upper_index == values.len() {
|
||||
// Value is greater than the last value
|
||||
let last_original = &values[values.len() - 1].0;
|
||||
check_point.push_str(&format!("{}>{}", col_name, last_original));
|
||||
} else {
|
||||
// Normal case: value is between two valid values
|
||||
let lower_original = if lower_index < values.len() {
|
||||
values[lower_index].0.to_string()
|
||||
} else {
|
||||
"unknown".to_string()
|
||||
};
|
||||
|
||||
let upper_original = if upper_index < values.len() {
|
||||
values[upper_index].0.to_string()
|
||||
} else {
|
||||
"unknown".to_string()
|
||||
};
|
||||
|
||||
check_point.push_str(&format!(
|
||||
"{}<{}<{}",
|
||||
lower_original, col_name, upper_original
|
||||
));
|
||||
}
|
||||
}
|
||||
} else {
|
||||
// Fallback if column not found in normalized values
|
||||
check_point.push_str(&format!("{}:unknown", col_name));
|
||||
}
|
||||
}
|
||||
|
||||
check_point
|
||||
}
|
||||
}
|
||||
|
||||
/// Generates a point matrix that contains permutations of `matrix_foundation`'s values
|
||||
struct MatrixGenerator {
|
||||
matrix_foundation: HashMap<String, Vec<OrderedF64>>,
|
||||
// Iterator state
|
||||
current_index: usize,
|
||||
schema: Schema,
|
||||
column_names: Vec<String>,
|
||||
// Preprocessed attributes
|
||||
/// Total number of combinations of `matrix_foundation`'s values
|
||||
total_combinations: usize,
|
||||
/// Biased suffix product of `matrix_foundation`'s values
|
||||
///
|
||||
/// The i-th element is the product of the sizes of all columns after the i-th column.
|
||||
/// For example, if `matrix_foundation` is `{"a": [1, 2, 3], "b": [4, 5, 6]}`,
|
||||
/// then `biased_suffix_product` is `[3, 1]`.
|
||||
biased_suffix_product: Vec<usize>,
|
||||
}
|
||||
|
||||
const MAX_BATCH_SIZE: usize = 8192;
|
||||
|
||||
impl MatrixGenerator {
|
||||
pub fn new(matrix_foundation: HashMap<&str, Vec<OrderedF64>>) -> Self {
|
||||
// Convert to owned HashMap to avoid lifetime issues
|
||||
let owned_matrix_foundation: HashMap<String, Vec<OrderedF64>> = matrix_foundation
|
||||
.into_iter()
|
||||
.map(|(k, v)| (k.to_string(), v))
|
||||
.collect();
|
||||
|
||||
let mut fields = owned_matrix_foundation
|
||||
.keys()
|
||||
.map(|k| Field::new(k.clone(), DataType::Float64, false))
|
||||
.collect::<Vec<_>>();
|
||||
fields.sort_unstable();
|
||||
let schema = Schema::new(fields.clone());
|
||||
|
||||
// Store column names in the same order as fields
|
||||
let column_names: Vec<String> = fields.iter().map(|field| field.name().clone()).collect();
|
||||
|
||||
// Calculate total number of combinations and suffix product
|
||||
let mut biased_suffix_product = Vec::with_capacity(column_names.len() + 1);
|
||||
let mut product = 1;
|
||||
biased_suffix_product.push(product);
|
||||
for col_name in column_names.iter().rev() {
|
||||
product *= owned_matrix_foundation[col_name].len();
|
||||
biased_suffix_product.push(product);
|
||||
}
|
||||
biased_suffix_product.pop();
|
||||
biased_suffix_product.reverse();
|
||||
|
||||
Self {
|
||||
matrix_foundation: owned_matrix_foundation,
|
||||
current_index: 0,
|
||||
schema,
|
||||
column_names,
|
||||
total_combinations: product,
|
||||
biased_suffix_product,
|
||||
}
|
||||
}
|
||||
|
||||
pub fn schema(&self) -> &Schema {
|
||||
&self.schema
|
||||
}
|
||||
|
||||
fn generate_batch(&self, start_index: usize, batch_size: usize) -> RecordBatch {
|
||||
let actual_batch_size = batch_size.min(self.total_combinations - start_index);
|
||||
|
||||
// Create array builders
|
||||
let mut array_builders: Vec<Float64Builder> = Vec::with_capacity(self.column_names.len());
|
||||
for _ in 0..self.column_names.len() {
|
||||
array_builders.push(Float64Builder::with_capacity(actual_batch_size));
|
||||
}
|
||||
|
||||
// Generate combinations for this batch
|
||||
for combination_offset in 0..actual_batch_size {
|
||||
let combination_index = start_index + combination_offset;
|
||||
|
||||
// For each column, determine which value to use for this combination
|
||||
for (col_idx, col_name) in self.column_names.iter().enumerate() {
|
||||
let values = &self.matrix_foundation[col_name];
|
||||
let stride = self.biased_suffix_product[col_idx];
|
||||
let value_index = (combination_index / stride) % values.len();
|
||||
let value = *values[value_index].as_ref();
|
||||
|
||||
array_builders[col_idx].append_value(value);
|
||||
}
|
||||
}
|
||||
|
||||
// Finish arrays and create RecordBatch
|
||||
let arrays: Vec<_> = array_builders
|
||||
.into_iter()
|
||||
.map(|mut builder| Arc::new(builder.finish()) as _)
|
||||
.collect();
|
||||
|
||||
RecordBatch::try_new(Arc::new(self.schema.clone()), arrays)
|
||||
.expect("Failed to create RecordBatch from generated arrays")
|
||||
}
|
||||
}
|
||||
|
||||
impl Iterator for MatrixGenerator {
|
||||
type Item = RecordBatch;
|
||||
|
||||
fn next(&mut self) -> Option<Self::Item> {
|
||||
if self.current_index >= self.total_combinations {
|
||||
return None;
|
||||
}
|
||||
|
||||
let remaining = self.total_combinations - self.current_index;
|
||||
let batch_size = remaining.min(MAX_BATCH_SIZE);
|
||||
|
||||
let batch = self.generate_batch(self.current_index, batch_size);
|
||||
self.current_index += batch_size;
|
||||
|
||||
Some(batch)
|
||||
}
|
||||
}
|
||||
|
||||
#[cfg(test)]
|
||||
mod tests {
|
||||
use std::collections::HashMap;
|
||||
|
||||
use datatypes::value::Value;
|
||||
|
||||
use super::*;
|
||||
use crate::expr::col;
|
||||
use crate::multi_dim::MultiDimPartitionRule;
|
||||
|
||||
#[test]
|
||||
fn test_matrix_generator_single_column() {
|
||||
let mut matrix_foundation = HashMap::new();
|
||||
matrix_foundation.insert(
|
||||
"col1",
|
||||
vec![
|
||||
OrderedF64::from(1.0),
|
||||
OrderedF64::from(2.0),
|
||||
OrderedF64::from(3.0),
|
||||
],
|
||||
);
|
||||
|
||||
let mut generator = MatrixGenerator::new(matrix_foundation);
|
||||
let batch = generator.next().unwrap();
|
||||
|
||||
assert_eq!(batch.num_rows(), 3);
|
||||
assert_eq!(batch.num_columns(), 1);
|
||||
assert_eq!(batch.schema().field(0).name(), "col1");
|
||||
|
||||
let col1_array = batch
|
||||
.column(0)
|
||||
.as_any()
|
||||
.downcast_ref::<datatypes::arrow::array::Float64Array>()
|
||||
.unwrap();
|
||||
assert_eq!(col1_array.value(0), 1.0);
|
||||
assert_eq!(col1_array.value(1), 2.0);
|
||||
assert_eq!(col1_array.value(2), 3.0);
|
||||
|
||||
// Should be no more batches for such a small dataset
|
||||
assert!(generator.next().is_none());
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_matrix_generator_three_columns_cartesian_product() {
|
||||
let mut matrix_foundation = HashMap::new();
|
||||
matrix_foundation.insert("a", vec![OrderedF64::from(1.0), OrderedF64::from(2.0)]);
|
||||
matrix_foundation.insert("b", vec![OrderedF64::from(10.0), OrderedF64::from(20.0)]);
|
||||
matrix_foundation.insert(
|
||||
"c",
|
||||
vec![
|
||||
OrderedF64::from(100.0),
|
||||
OrderedF64::from(200.0),
|
||||
OrderedF64::from(300.0),
|
||||
],
|
||||
);
|
||||
|
||||
let mut generator = MatrixGenerator::new(matrix_foundation);
|
||||
let batch = generator.next().unwrap();
|
||||
|
||||
// Should have 2 * 2 * 3 = 12 combinations
|
||||
assert_eq!(batch.num_rows(), 12);
|
||||
assert_eq!(batch.num_columns(), 3);
|
||||
|
||||
let a_array = batch
|
||||
.column(0)
|
||||
.as_any()
|
||||
.downcast_ref::<datatypes::arrow::array::Float64Array>()
|
||||
.unwrap();
|
||||
let b_array = batch
|
||||
.column(1)
|
||||
.as_any()
|
||||
.downcast_ref::<datatypes::arrow::array::Float64Array>()
|
||||
.unwrap();
|
||||
let c_array = batch
|
||||
.column(2)
|
||||
.as_any()
|
||||
.downcast_ref::<datatypes::arrow::array::Float64Array>()
|
||||
.unwrap();
|
||||
|
||||
// Verify first few combinations (a changes slowest, c changes fastest)
|
||||
let expected = vec![
|
||||
(1.0, 10.0, 100.0),
|
||||
(1.0, 10.0, 200.0),
|
||||
(1.0, 10.0, 300.0),
|
||||
(1.0, 20.0, 100.0),
|
||||
(1.0, 20.0, 200.0),
|
||||
(1.0, 20.0, 300.0),
|
||||
(2.0, 10.0, 100.0),
|
||||
(2.0, 10.0, 200.0),
|
||||
(2.0, 10.0, 300.0),
|
||||
(2.0, 20.0, 100.0),
|
||||
(2.0, 20.0, 200.0),
|
||||
(2.0, 20.0, 300.0),
|
||||
];
|
||||
#[allow(clippy::needless_range_loop)]
|
||||
for i in 0..batch.num_rows() {
|
||||
assert_eq!(
|
||||
(a_array.value(i), b_array.value(i), c_array.value(i)),
|
||||
expected[i]
|
||||
);
|
||||
}
|
||||
|
||||
// Should be no more batches for such a small dataset
|
||||
assert!(generator.next().is_none());
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_matrix_generator_iterator_small_batches() {
|
||||
let mut matrix_foundation = HashMap::new();
|
||||
matrix_foundation.insert("col1", vec![OrderedF64::from(1.0), OrderedF64::from(2.0)]);
|
||||
matrix_foundation.insert(
|
||||
"col2",
|
||||
vec![
|
||||
OrderedF64::from(10.0),
|
||||
OrderedF64::from(20.0),
|
||||
OrderedF64::from(30.0),
|
||||
],
|
||||
);
|
||||
|
||||
let generator = MatrixGenerator::new(matrix_foundation);
|
||||
|
||||
// Total combinations should be 2 * 3 = 6
|
||||
assert_eq!(generator.total_combinations, 6);
|
||||
|
||||
let mut total_rows = 0;
|
||||
|
||||
for batch in generator {
|
||||
total_rows += batch.num_rows();
|
||||
assert_eq!(batch.num_columns(), 2);
|
||||
|
||||
// Verify each batch is valid
|
||||
assert!(batch.num_rows() > 0);
|
||||
assert!(batch.num_rows() <= MAX_BATCH_SIZE);
|
||||
}
|
||||
|
||||
assert_eq!(total_rows, 6);
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_matrix_generator_empty_column_values() {
|
||||
let mut matrix_foundation = HashMap::new();
|
||||
matrix_foundation.insert("col1", vec![]);
|
||||
|
||||
let mut generator = MatrixGenerator::new(matrix_foundation);
|
||||
|
||||
// Should have 0 total combinations when any column is empty
|
||||
assert_eq!(generator.total_combinations, 0);
|
||||
|
||||
// Should have no batches when total combinations is 0
|
||||
assert!(generator.next().is_none());
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_matrix_generator_large_dataset_batching() {
|
||||
// Create a dataset that will exceed MAX_BATCH_SIZE (8192)
|
||||
// 20 * 20 * 21 = 8400 > 8192
|
||||
let mut matrix_foundation = HashMap::new();
|
||||
|
||||
let values1: Vec<OrderedF64> = (0..20).map(|i| OrderedF64::from(i as f64)).collect();
|
||||
let values2: Vec<OrderedF64> = (0..20)
|
||||
.map(|i| OrderedF64::from(i as f64 + 100.0))
|
||||
.collect();
|
||||
let values3: Vec<OrderedF64> = (0..21)
|
||||
.map(|i| OrderedF64::from(i as f64 + 1000.0))
|
||||
.collect();
|
||||
|
||||
matrix_foundation.insert("col1", values1);
|
||||
matrix_foundation.insert("col2", values2);
|
||||
matrix_foundation.insert("col3", values3);
|
||||
|
||||
let generator = MatrixGenerator::new(matrix_foundation);
|
||||
|
||||
assert_eq!(generator.total_combinations, 8400);
|
||||
|
||||
let mut total_rows = 0;
|
||||
let mut batch_count = 0;
|
||||
let mut first_batch_size = None;
|
||||
|
||||
for batch in generator {
|
||||
batch_count += 1;
|
||||
let batch_size = batch.num_rows();
|
||||
total_rows += batch_size;
|
||||
|
||||
if first_batch_size.is_none() {
|
||||
first_batch_size = Some(batch_size);
|
||||
}
|
||||
|
||||
// Each batch should not exceed MAX_BATCH_SIZE
|
||||
assert!(batch_size <= MAX_BATCH_SIZE);
|
||||
assert_eq!(batch.num_columns(), 3);
|
||||
}
|
||||
|
||||
assert_eq!(total_rows, 8400);
|
||||
assert!(batch_count > 1);
|
||||
assert_eq!(first_batch_size.unwrap(), MAX_BATCH_SIZE);
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_remap_checkpoint_values() {
|
||||
// Create rule with single column
|
||||
let rule = MultiDimPartitionRule::try_new(
|
||||
vec!["host".to_string(), "value".to_string()],
|
||||
vec![1, 2, 3],
|
||||
vec![
|
||||
col("host")
|
||||
.lt(Value::Int64(0))
|
||||
.and(col("value").lt(Value::Int64(0))),
|
||||
col("host")
|
||||
.lt(Value::Int64(0))
|
||||
.and(col("value").gt_eq(Value::Int64(0))),
|
||||
col("host")
|
||||
.gt_eq(Value::Int64(0))
|
||||
.and(col("host").lt(Value::Int64(1)))
|
||||
.and(col("value").lt(Value::Int64(1))),
|
||||
col("host")
|
||||
.gt_eq(Value::Int64(0))
|
||||
.and(col("host").lt(Value::Int64(1)))
|
||||
.and(col("value").gt_eq(Value::Int64(1))),
|
||||
col("host")
|
||||
.gt_eq(Value::Int64(1))
|
||||
.and(col("host").lt(Value::Int64(2)))
|
||||
.and(col("value").lt(Value::Int64(2))),
|
||||
col("host")
|
||||
.gt_eq(Value::Int64(1))
|
||||
.and(col("host").lt(Value::Int64(2)))
|
||||
.and(col("value").gt_eq(Value::Int64(2))),
|
||||
col("host")
|
||||
.gt_eq(Value::Int64(2))
|
||||
.and(col("host").lt(Value::Int64(3)))
|
||||
.and(col("value").lt(Value::Int64(3))),
|
||||
col("host")
|
||||
.gt_eq(Value::Int64(2))
|
||||
.and(col("host").lt(Value::Int64(3)))
|
||||
.and(col("value").gt_eq(Value::Int64(3))),
|
||||
col("host").gt_eq(Value::Int64(3)),
|
||||
],
|
||||
true,
|
||||
)
|
||||
.unwrap();
|
||||
let checker = PartitionChecker::try_new(&rule).unwrap();
|
||||
|
||||
let schema = Arc::new(Schema::new(vec![
|
||||
Field::new("host", DataType::Float64, false),
|
||||
Field::new("value", DataType::Float64, false),
|
||||
]));
|
||||
let host_array = Float64Array::from(vec![-0.5, 0.0, 0.5, 1.0, 1.5, 2.0, 2.5, 3.0, 3.5]);
|
||||
let value_array = Float64Array::from(vec![-0.5, 0.0, 0.5, 1.0, 1.5, 2.0, 2.5, 3.0, 3.5]);
|
||||
let batch = RecordBatch::try_new(schema, vec![Arc::new(host_array), Arc::new(value_array)])
|
||||
.unwrap();
|
||||
|
||||
let checkpoint = checker.remap_checkpoint(0, &batch);
|
||||
assert_eq!(checkpoint, "host<0, value<0");
|
||||
let checkpoint = checker.remap_checkpoint(1, &batch);
|
||||
assert_eq!(checkpoint, "host=0, value=0");
|
||||
let checkpoint = checker.remap_checkpoint(6, &batch);
|
||||
assert_eq!(checkpoint, "2<host<3, 2<value<3");
|
||||
let checkpoint = checker.remap_checkpoint(7, &batch);
|
||||
assert_eq!(checkpoint, "host=3, value=3");
|
||||
let checkpoint = checker.remap_checkpoint(8, &batch);
|
||||
assert_eq!(checkpoint, "host>3, value>3");
|
||||
}
|
||||
}
|
||||
692
src/partition/src/collider.rs
Normal file
692
src/partition/src/collider.rs
Normal file
@@ -0,0 +1,692 @@
|
||||
// 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.
|
||||
|
||||
//! Provides a Collider tool to convert [`PartitionExpr`] into a form that is easier to operate by program.
|
||||
//!
|
||||
//! This mod provides the following major structs:
|
||||
//!
|
||||
//! - [`Collider`]: The main struct that converts [`PartitionExpr`].
|
||||
//! - [`AtomicExpr`]: An "atomic" Expression, which isn't composed (OR-ed) of other expressions.
|
||||
//! - [`NucleonExpr`]: A simplified expression representation.
|
||||
//! - [`GluonOp`]: Further restricted operation set.
|
||||
//!
|
||||
//! On the naming aspect, "collider" is a high-energy machine that cracks particles, "atomic" is a typical
|
||||
//! non-divisible particle before ~100 years ago, "nucleon" is what composes an atom and "gluon" is the
|
||||
//! force inside nucleons.
|
||||
|
||||
use std::collections::HashMap;
|
||||
use std::fmt::Debug;
|
||||
use std::sync::Arc;
|
||||
|
||||
use datafusion_expr::Operator;
|
||||
use datafusion_physical_expr::expressions::{col, lit, BinaryExpr};
|
||||
use datafusion_physical_expr::PhysicalExpr;
|
||||
use datatypes::arrow::datatypes::Schema;
|
||||
use datatypes::value::{OrderedF64, OrderedFloat, Value};
|
||||
|
||||
use crate::error;
|
||||
use crate::error::Result;
|
||||
use crate::expr::{Operand, PartitionExpr, RestrictedOp};
|
||||
|
||||
const ZERO: OrderedF64 = OrderedFloat(0.0f64);
|
||||
pub(crate) const NORMALIZE_STEP: OrderedF64 = OrderedFloat(1.0f64);
|
||||
pub(crate) const CHECK_STEP: OrderedF64 = OrderedFloat(0.5f64);
|
||||
|
||||
/// Represents an "atomic" Expression, which isn't composed (OR-ed) of other expressions.
|
||||
#[allow(unused)]
|
||||
#[derive(Debug, Clone, PartialEq, Eq)]
|
||||
pub(crate) struct AtomicExpr {
|
||||
/// A (ordered) list of simplified expressions. They are [`RestrictedOp::And`]'ed together.
|
||||
pub(crate) nucleons: Vec<NucleonExpr>,
|
||||
/// Index to reference the [`PartitionExpr`] that this [`AtomicExpr`] is derived from.
|
||||
/// This index is used with `exprs` field in [`MultiDimPartitionRule`](crate::multi_dim::MultiDimPartitionRule).
|
||||
pub(crate) source_expr_index: usize,
|
||||
}
|
||||
|
||||
impl AtomicExpr {
|
||||
pub fn to_physical_expr(&self, schema: &Schema) -> Arc<dyn PhysicalExpr> {
|
||||
let mut exprs = Vec::with_capacity(self.nucleons.len());
|
||||
for nucleon in &self.nucleons {
|
||||
exprs.push(nucleon.to_physical_expr(schema));
|
||||
}
|
||||
let result: Arc<dyn PhysicalExpr> = exprs
|
||||
.into_iter()
|
||||
.reduce(|l, r| Arc::new(BinaryExpr::new(l, Operator::And, r)))
|
||||
.unwrap();
|
||||
result
|
||||
}
|
||||
}
|
||||
|
||||
impl PartialOrd for AtomicExpr {
|
||||
fn partial_cmp(&self, other: &Self) -> Option<std::cmp::Ordering> {
|
||||
Some(self.nucleons.cmp(&other.nucleons))
|
||||
}
|
||||
}
|
||||
|
||||
/// A simplified expression representation.
|
||||
///
|
||||
/// This struct is used to compose [`AtomicExpr`], hence "nucleon".
|
||||
#[derive(Debug, Clone, PartialEq, Eq, PartialOrd, Ord, Hash)]
|
||||
pub(crate) struct NucleonExpr {
|
||||
column: String,
|
||||
op: GluonOp,
|
||||
/// Normalized [`Value`].
|
||||
value: OrderedF64,
|
||||
}
|
||||
|
||||
impl NucleonExpr {
|
||||
pub fn to_physical_expr(&self, schema: &Schema) -> Arc<dyn PhysicalExpr> {
|
||||
Arc::new(BinaryExpr::new(
|
||||
col(&self.column, schema).unwrap(),
|
||||
self.op.to_operator(),
|
||||
lit(*self.value.as_ref()),
|
||||
))
|
||||
}
|
||||
}
|
||||
|
||||
/// Further restricted operation set.
|
||||
///
|
||||
/// Conjunction operations are removed from [`RestrictedOp`].
|
||||
/// This enumeration is used to bind elements in [`NucleonExpr`], hence "gluon".
|
||||
#[derive(Debug, Clone, PartialEq, Eq, PartialOrd, Ord, Hash)]
|
||||
enum GluonOp {
|
||||
Eq,
|
||||
NotEq,
|
||||
Lt,
|
||||
LtEq,
|
||||
Gt,
|
||||
GtEq,
|
||||
}
|
||||
|
||||
impl GluonOp {
|
||||
pub fn to_operator(&self) -> Operator {
|
||||
match self {
|
||||
GluonOp::Eq => Operator::Eq,
|
||||
GluonOp::NotEq => Operator::NotEq,
|
||||
GluonOp::Lt => Operator::Lt,
|
||||
GluonOp::LtEq => Operator::LtEq,
|
||||
GluonOp::Gt => Operator::Gt,
|
||||
GluonOp::GtEq => Operator::GtEq,
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/// Collider is used to collide a list of [`PartitionExpr`] into a list of [`AtomicExpr`]
|
||||
///
|
||||
/// It also normalizes the values of the columns in the expressions.
|
||||
#[allow(unused)]
|
||||
pub struct Collider<'a> {
|
||||
source_exprs: &'a [PartitionExpr],
|
||||
|
||||
pub(crate) atomic_exprs: Vec<AtomicExpr>,
|
||||
/// A map of column name to a list of `(value, normalized value)` pairs.
|
||||
///
|
||||
/// The normalized value is used for comparison. The normalization process keeps the order of the values.
|
||||
pub(crate) normalized_values: HashMap<String, Vec<(Value, OrderedF64)>>,
|
||||
}
|
||||
|
||||
impl<'a> Collider<'a> {
|
||||
pub fn new(source_exprs: &'a [PartitionExpr]) -> Result<Self> {
|
||||
// first walk to collect all values
|
||||
let mut values: HashMap<String, Vec<Value>> = HashMap::new();
|
||||
for expr in source_exprs {
|
||||
Self::collect_column_values_from_expr(expr, &mut values)?;
|
||||
}
|
||||
|
||||
// normalize values, assumes all values on a column are the same type
|
||||
let mut normalized_values: HashMap<String, HashMap<Value, OrderedF64>> =
|
||||
HashMap::with_capacity(values.len());
|
||||
for (column, mut column_values) in values {
|
||||
column_values.sort_unstable();
|
||||
column_values.dedup(); // Remove duplicates
|
||||
let mut value_map = HashMap::with_capacity(column_values.len());
|
||||
let mut start_value = ZERO;
|
||||
for value in column_values {
|
||||
value_map.insert(value, start_value);
|
||||
start_value += NORMALIZE_STEP;
|
||||
}
|
||||
normalized_values.insert(column, value_map);
|
||||
}
|
||||
|
||||
// second walk to get atomic exprs
|
||||
let mut atomic_exprs = Vec::with_capacity(source_exprs.len());
|
||||
for (index, expr) in source_exprs.iter().enumerate() {
|
||||
Self::collide_expr(expr, index, &normalized_values, &mut atomic_exprs)?;
|
||||
}
|
||||
// sort nucleon exprs
|
||||
for expr in &mut atomic_exprs {
|
||||
expr.nucleons.sort_unstable();
|
||||
}
|
||||
|
||||
// convert normalized values to a map
|
||||
let normalized_values = normalized_values
|
||||
.into_iter()
|
||||
.map(|(col, values)| {
|
||||
let mut values = values.into_iter().collect::<Vec<_>>();
|
||||
values.sort_unstable_by_key(|(_, v)| *v);
|
||||
(col, values)
|
||||
})
|
||||
.collect();
|
||||
|
||||
Ok(Self {
|
||||
source_exprs,
|
||||
atomic_exprs,
|
||||
normalized_values,
|
||||
})
|
||||
}
|
||||
|
||||
/// Helper to collect values with their associated columns from an expression
|
||||
fn collect_column_values_from_expr(
|
||||
expr: &PartitionExpr,
|
||||
values: &mut HashMap<String, Vec<Value>>,
|
||||
) -> Result<()> {
|
||||
// Handle binary operations between column and value
|
||||
match (&*expr.lhs, &*expr.rhs) {
|
||||
(Operand::Column(col), Operand::Value(val))
|
||||
| (Operand::Value(val), Operand::Column(col)) => {
|
||||
values.entry(col.clone()).or_default().push(val.clone());
|
||||
Ok(())
|
||||
}
|
||||
(Operand::Expr(left_expr), Operand::Expr(right_expr)) => {
|
||||
Self::collect_column_values_from_expr(left_expr, values)?;
|
||||
Self::collect_column_values_from_expr(right_expr, values)
|
||||
}
|
||||
// Other combinations don't directly contribute column-value pairs
|
||||
_ => error::InvalidExprSnafu { expr: expr.clone() }.fail(),
|
||||
}
|
||||
}
|
||||
|
||||
/// Collide a [`PartitionExpr`] into multiple [`AtomicExpr`]s.
|
||||
///
|
||||
/// Split the [`PartitionExpr`] on every [`RestrictedOp::Or`] (disjunction), each branch is an [`AtomicExpr`].
|
||||
/// Since [`PartitionExpr`] doesn't allow parentheses, Expression like `(a = 1 OR b = 2) AND c = 3` won't occur.
|
||||
/// We can safely split on every [`RestrictedOp::Or`].
|
||||
fn collide_expr(
|
||||
expr: &PartitionExpr,
|
||||
index: usize,
|
||||
normalized_values: &HashMap<String, HashMap<Value, OrderedF64>>,
|
||||
result: &mut Vec<AtomicExpr>,
|
||||
) -> Result<()> {
|
||||
match expr.op {
|
||||
RestrictedOp::Or => {
|
||||
// Split on OR operation - each side becomes a separate atomic expression
|
||||
|
||||
// Process left side
|
||||
match &*expr.lhs {
|
||||
Operand::Expr(left_expr) => {
|
||||
Self::collide_expr(left_expr, index, normalized_values, result)?;
|
||||
}
|
||||
_ => {
|
||||
// Single operand - this shouldn't happen with OR
|
||||
// OR should always connect two sub-expressions
|
||||
return error::InvalidExprSnafu { expr: expr.clone() }.fail();
|
||||
}
|
||||
}
|
||||
|
||||
// Process right side
|
||||
match &*expr.rhs {
|
||||
Operand::Expr(right_expr) => {
|
||||
Self::collide_expr(right_expr, index, normalized_values, result)?;
|
||||
}
|
||||
_ => {
|
||||
// Single operand - this shouldn't happen with OR
|
||||
// OR should always connect two sub-expressions
|
||||
return error::InvalidExprSnafu { expr: expr.clone() }.fail();
|
||||
}
|
||||
}
|
||||
}
|
||||
RestrictedOp::And => {
|
||||
// For AND operations, we need to combine nucleons
|
||||
let mut nucleons = Vec::new();
|
||||
Self::collect_nucleons_from_expr(expr, &mut nucleons, normalized_values)?;
|
||||
|
||||
result.push(AtomicExpr {
|
||||
nucleons,
|
||||
source_expr_index: index,
|
||||
});
|
||||
}
|
||||
_ => {
|
||||
// For other operations, create a single atomic expression
|
||||
let mut nucleons = Vec::new();
|
||||
Self::collect_nucleons_from_expr(expr, &mut nucleons, normalized_values)?;
|
||||
|
||||
result.push(AtomicExpr {
|
||||
nucleons,
|
||||
source_expr_index: index,
|
||||
});
|
||||
}
|
||||
}
|
||||
Ok(())
|
||||
}
|
||||
|
||||
/// Collect nucleons from an expression (handles AND operations recursively)
|
||||
fn collect_nucleons_from_expr(
|
||||
expr: &PartitionExpr,
|
||||
nucleons: &mut Vec<NucleonExpr>,
|
||||
normalized_values: &HashMap<String, HashMap<Value, OrderedF64>>,
|
||||
) -> Result<()> {
|
||||
match expr.op {
|
||||
RestrictedOp::And => {
|
||||
// For AND operations, collect nucleons from both sides
|
||||
Self::collect_nucleons_from_operand(&expr.lhs, nucleons, normalized_values)?;
|
||||
Self::collect_nucleons_from_operand(&expr.rhs, nucleons, normalized_values)?;
|
||||
}
|
||||
_ => {
|
||||
// For non-AND operations, try to create a nucleon directly
|
||||
nucleons.push(Self::try_create_nucleon(
|
||||
&expr.lhs,
|
||||
&expr.op,
|
||||
&expr.rhs,
|
||||
normalized_values,
|
||||
)?);
|
||||
}
|
||||
}
|
||||
Ok(())
|
||||
}
|
||||
|
||||
/// Collect nucleons from an operand
|
||||
fn collect_nucleons_from_operand(
|
||||
operand: &Operand,
|
||||
nucleons: &mut Vec<NucleonExpr>,
|
||||
normalized_values: &HashMap<String, HashMap<Value, OrderedF64>>,
|
||||
) -> Result<()> {
|
||||
match operand {
|
||||
Operand::Expr(expr) => {
|
||||
Self::collect_nucleons_from_expr(expr, nucleons, normalized_values)
|
||||
}
|
||||
_ => {
|
||||
// Only `Operand::Expr` can be conjuncted by AND.
|
||||
error::NoExprOperandSnafu {
|
||||
operand: operand.clone(),
|
||||
}
|
||||
.fail()
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/// Try to create a nucleon from operands
|
||||
fn try_create_nucleon(
|
||||
lhs: &Operand,
|
||||
op: &RestrictedOp,
|
||||
rhs: &Operand,
|
||||
normalized_values: &HashMap<String, HashMap<Value, OrderedF64>>,
|
||||
) -> Result<NucleonExpr> {
|
||||
let gluon_op = match op {
|
||||
RestrictedOp::Eq => GluonOp::Eq,
|
||||
RestrictedOp::NotEq => GluonOp::NotEq,
|
||||
RestrictedOp::Lt => GluonOp::Lt,
|
||||
RestrictedOp::LtEq => GluonOp::LtEq,
|
||||
RestrictedOp::Gt => GluonOp::Gt,
|
||||
RestrictedOp::GtEq => GluonOp::GtEq,
|
||||
RestrictedOp::And | RestrictedOp::Or => {
|
||||
// These should be handled elsewhere
|
||||
return error::UnexpectedSnafu {
|
||||
err_msg: format!("Conjunction operation {:?} should be handled elsewhere", op),
|
||||
}
|
||||
.fail();
|
||||
}
|
||||
};
|
||||
|
||||
match (lhs, rhs) {
|
||||
(Operand::Column(col), Operand::Value(val)) => {
|
||||
if let Some(column_values) = normalized_values.get(col) {
|
||||
if let Some(&normalized_val) = column_values.get(val) {
|
||||
return Ok(NucleonExpr {
|
||||
column: col.clone(),
|
||||
op: gluon_op,
|
||||
value: normalized_val,
|
||||
});
|
||||
}
|
||||
}
|
||||
}
|
||||
(Operand::Value(val), Operand::Column(col)) => {
|
||||
if let Some(column_values) = normalized_values.get(col) {
|
||||
if let Some(&normalized_val) = column_values.get(val) {
|
||||
// Flip the operation for value op column
|
||||
let flipped_op = match gluon_op {
|
||||
GluonOp::Lt => GluonOp::Gt,
|
||||
GluonOp::LtEq => GluonOp::GtEq,
|
||||
GluonOp::Gt => GluonOp::Lt,
|
||||
GluonOp::GtEq => GluonOp::LtEq,
|
||||
op => op, // Eq and NotEq remain the same
|
||||
};
|
||||
return Ok(NucleonExpr {
|
||||
column: col.clone(),
|
||||
op: flipped_op,
|
||||
value: normalized_val,
|
||||
});
|
||||
}
|
||||
}
|
||||
}
|
||||
_ => {}
|
||||
}
|
||||
|
||||
// Other combinations not supported for nucleons
|
||||
error::InvalidExprSnafu {
|
||||
expr: PartitionExpr::new(lhs.clone(), op.clone(), rhs.clone()),
|
||||
}
|
||||
.fail()
|
||||
}
|
||||
}
|
||||
|
||||
#[cfg(test)]
|
||||
mod test {
|
||||
use super::*;
|
||||
use crate::expr::col;
|
||||
|
||||
#[test]
|
||||
fn test_collider_basic_value_normalization() {
|
||||
// Test with different value types in different columns
|
||||
let exprs = vec![
|
||||
// Integer values
|
||||
col("age").eq(Value::UInt32(25)),
|
||||
col("age").eq(Value::UInt32(30)),
|
||||
col("age").eq(Value::UInt32(25)), // Duplicate should be handled
|
||||
// String values
|
||||
col("name").eq(Value::String("alice".into())),
|
||||
col("name").eq(Value::String("bob".into())),
|
||||
// Boolean values
|
||||
col("active").eq(Value::Boolean(true)),
|
||||
col("active").eq(Value::Boolean(false)),
|
||||
// Float values
|
||||
col("score").eq(Value::Float64(OrderedFloat(95.5))),
|
||||
col("score").eq(Value::Float64(OrderedFloat(87.2))),
|
||||
];
|
||||
|
||||
let collider = Collider::new(&exprs).expect("Failed to create collider");
|
||||
|
||||
// Check that we have the right number of columns
|
||||
assert_eq!(collider.normalized_values.len(), 4);
|
||||
|
||||
// Check age column - should have 2 unique values (25, 30)
|
||||
let age_values = &collider.normalized_values["age"];
|
||||
assert_eq!(age_values.len(), 2);
|
||||
assert_eq!(
|
||||
age_values,
|
||||
&[
|
||||
(Value::UInt32(25), OrderedFloat(0.0f64)),
|
||||
(Value::UInt32(30), OrderedFloat(1.0f64))
|
||||
]
|
||||
);
|
||||
|
||||
// Check name column - should have 2 values
|
||||
let name_values = &collider.normalized_values["name"];
|
||||
assert_eq!(name_values.len(), 2);
|
||||
assert_eq!(
|
||||
name_values,
|
||||
&[
|
||||
(Value::String("alice".into()), OrderedFloat(0.0f64)),
|
||||
(Value::String("bob".into()), OrderedFloat(1.0f64))
|
||||
]
|
||||
);
|
||||
|
||||
// Check active column - should have 2 values
|
||||
let active_values = &collider.normalized_values["active"];
|
||||
assert_eq!(active_values.len(), 2);
|
||||
assert_eq!(
|
||||
active_values,
|
||||
&[
|
||||
(Value::Boolean(false), OrderedFloat(0.0f64)),
|
||||
(Value::Boolean(true), OrderedFloat(1.0f64))
|
||||
]
|
||||
);
|
||||
|
||||
// Check score column - should have 2 values
|
||||
let score_values = &collider.normalized_values["score"];
|
||||
assert_eq!(score_values.len(), 2);
|
||||
assert_eq!(
|
||||
score_values,
|
||||
&[
|
||||
(Value::Float64(OrderedFloat(87.2)), OrderedFloat(0.0f64)),
|
||||
(Value::Float64(OrderedFloat(95.5)), OrderedFloat(1.0f64))
|
||||
]
|
||||
);
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_collider_simple_expressions() {
|
||||
// Test simple equality
|
||||
let exprs = vec![col("id").eq(Value::UInt32(1))];
|
||||
|
||||
let collider = Collider::new(&exprs).unwrap();
|
||||
assert_eq!(collider.atomic_exprs.len(), 1);
|
||||
assert_eq!(collider.atomic_exprs[0].nucleons.len(), 1);
|
||||
assert_eq!(collider.atomic_exprs[0].source_expr_index, 0);
|
||||
|
||||
// Test simple AND
|
||||
let exprs = vec![col("id")
|
||||
.eq(Value::UInt32(1))
|
||||
.and(col("status").eq(Value::String("active".into())))];
|
||||
|
||||
let collider = Collider::new(&exprs).unwrap();
|
||||
assert_eq!(collider.atomic_exprs.len(), 1);
|
||||
assert_eq!(collider.atomic_exprs[0].nucleons.len(), 2);
|
||||
|
||||
// Test simple OR - should create 2 atomic expressions
|
||||
let expr = PartitionExpr::new(
|
||||
Operand::Expr(col("id").eq(Value::UInt32(1))),
|
||||
RestrictedOp::Or,
|
||||
Operand::Expr(col("id").eq(Value::UInt32(2))),
|
||||
);
|
||||
let exprs = vec![expr];
|
||||
|
||||
let collider = Collider::new(&exprs).unwrap();
|
||||
assert_eq!(collider.atomic_exprs.len(), 2);
|
||||
assert_eq!(collider.atomic_exprs[0].nucleons.len(), 1);
|
||||
assert_eq!(collider.atomic_exprs[1].nucleons.len(), 1);
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_collider_complex_nested_expressions() {
|
||||
// Test: (id = 1 AND status = 'active') OR (id = 2 AND status = 'inactive') OR (id = 3)
|
||||
let branch1 = col("id")
|
||||
.eq(Value::UInt32(1))
|
||||
.and(col("status").eq(Value::String("active".into())));
|
||||
let branch2 = col("id")
|
||||
.eq(Value::UInt32(2))
|
||||
.and(col("status").eq(Value::String("inactive".into())));
|
||||
let branch3 = col("id").eq(Value::UInt32(3));
|
||||
|
||||
let expr = PartitionExpr::new(
|
||||
Operand::Expr(PartitionExpr::new(
|
||||
Operand::Expr(branch1),
|
||||
RestrictedOp::Or,
|
||||
Operand::Expr(branch2),
|
||||
)),
|
||||
RestrictedOp::Or,
|
||||
Operand::Expr(branch3),
|
||||
);
|
||||
|
||||
let exprs = vec![expr];
|
||||
let collider = Collider::new(&exprs).unwrap();
|
||||
|
||||
assert_eq!(collider.atomic_exprs.len(), 3);
|
||||
|
||||
let total_nucleons: usize = collider
|
||||
.atomic_exprs
|
||||
.iter()
|
||||
.map(|ae| ae.nucleons.len())
|
||||
.sum();
|
||||
assert_eq!(total_nucleons, 5);
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_collider_deep_nesting() {
|
||||
// Test deeply nested AND operations: a = 1 AND b = 2 AND c = 3 AND d = 4
|
||||
let expr = col("a")
|
||||
.eq(Value::UInt32(1))
|
||||
.and(col("b").eq(Value::UInt32(2)))
|
||||
.and(col("c").eq(Value::UInt32(3)))
|
||||
.and(col("d").eq(Value::UInt32(4)));
|
||||
|
||||
let exprs = vec![expr];
|
||||
let collider = Collider::new(&exprs).unwrap();
|
||||
|
||||
assert_eq!(collider.atomic_exprs.len(), 1);
|
||||
assert_eq!(collider.atomic_exprs[0].nucleons.len(), 4);
|
||||
|
||||
// All nucleons should have Eq operation
|
||||
for nucleon in &collider.atomic_exprs[0].nucleons {
|
||||
assert_eq!(nucleon.op, GluonOp::Eq);
|
||||
}
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_collider_multiple_expressions() {
|
||||
// Test multiple separate expressions
|
||||
let exprs = vec![
|
||||
col("id").eq(Value::UInt32(1)),
|
||||
col("name").eq(Value::String("alice".into())),
|
||||
col("score").gt_eq(Value::Float64(OrderedFloat(90.0))),
|
||||
];
|
||||
|
||||
let collider = Collider::new(&exprs).unwrap();
|
||||
|
||||
// Should create 3 atomic expressions (one for each input expression)
|
||||
assert_eq!(collider.atomic_exprs.len(), 3);
|
||||
|
||||
// Each should have exactly 1 nucleon
|
||||
for atomic_expr in &collider.atomic_exprs {
|
||||
assert_eq!(atomic_expr.nucleons.len(), 1);
|
||||
}
|
||||
|
||||
// Check that source indices are correct
|
||||
let indices: Vec<usize> = collider
|
||||
.atomic_exprs
|
||||
.iter()
|
||||
.map(|ae| ae.source_expr_index)
|
||||
.collect();
|
||||
assert!(indices.contains(&0));
|
||||
assert!(indices.contains(&1));
|
||||
assert!(indices.contains(&2));
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_collider_value_column_order() {
|
||||
// Test expressions where value comes before column (should flip operation)
|
||||
let expr1 = PartitionExpr::new(
|
||||
Operand::Value(Value::UInt32(10)),
|
||||
RestrictedOp::Lt,
|
||||
Operand::Column("age".to_string()),
|
||||
); // 10 < age should become age > 10
|
||||
|
||||
let expr2 = PartitionExpr::new(
|
||||
Operand::Value(Value::UInt32(20)),
|
||||
RestrictedOp::GtEq,
|
||||
Operand::Column("score".to_string()),
|
||||
); // 20 >= score should become score <= 20
|
||||
|
||||
let exprs = vec![expr1, expr2];
|
||||
let collider = Collider::new(&exprs).unwrap();
|
||||
|
||||
assert_eq!(collider.atomic_exprs.len(), 2);
|
||||
|
||||
// Check that operations were flipped correctly
|
||||
let operations: Vec<GluonOp> = collider
|
||||
.atomic_exprs
|
||||
.iter()
|
||||
.map(|ae| ae.nucleons[0].op.clone())
|
||||
.collect();
|
||||
|
||||
assert!(operations.contains(&GluonOp::Gt)); // 10 < age -> age > 10
|
||||
assert!(operations.contains(&GluonOp::LtEq)); // 20 >= score -> score <= 20
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_collider_complex_or_with_different_columns() {
|
||||
// Test: (name = 'alice' AND age = 25) OR (status = 'active' AND score > 90)
|
||||
let branch1 = col("name")
|
||||
.eq(Value::String("alice".into()))
|
||||
.and(col("age").eq(Value::UInt32(25)));
|
||||
|
||||
let branch2 = col("status")
|
||||
.eq(Value::String("active".into()))
|
||||
.and(PartitionExpr::new(
|
||||
Operand::Column("score".to_string()),
|
||||
RestrictedOp::Gt,
|
||||
Operand::Value(Value::Float64(OrderedFloat(90.0))),
|
||||
));
|
||||
|
||||
let expr = PartitionExpr::new(
|
||||
Operand::Expr(branch1),
|
||||
RestrictedOp::Or,
|
||||
Operand::Expr(branch2),
|
||||
);
|
||||
|
||||
let exprs = vec![expr];
|
||||
let collider = Collider::new(&exprs).expect("Failed to create collider");
|
||||
|
||||
// Should create 2 atomic expressions
|
||||
assert_eq!(collider.atomic_exprs.len(), 2);
|
||||
|
||||
// Each atomic expression should have 2 nucleons
|
||||
for atomic_expr in &collider.atomic_exprs {
|
||||
assert_eq!(atomic_expr.nucleons.len(), 2);
|
||||
}
|
||||
|
||||
// Should have normalized values for all 4 columns
|
||||
assert_eq!(collider.normalized_values.len(), 4);
|
||||
assert!(collider.normalized_values.contains_key("name"));
|
||||
assert!(collider.normalized_values.contains_key("age"));
|
||||
assert!(collider.normalized_values.contains_key("status"));
|
||||
assert!(collider.normalized_values.contains_key("score"));
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_try_create_nucleon_edge_cases() {
|
||||
let normalized_values = HashMap::new();
|
||||
|
||||
// Test with AND operation
|
||||
let result = Collider::try_create_nucleon(
|
||||
&col("a"),
|
||||
&RestrictedOp::And,
|
||||
&Operand::Value(Value::UInt32(1)),
|
||||
&normalized_values,
|
||||
);
|
||||
assert!(result.is_err());
|
||||
|
||||
// Test with OR operation
|
||||
let result = Collider::try_create_nucleon(
|
||||
&col("a"),
|
||||
&RestrictedOp::Or,
|
||||
&Operand::Value(Value::UInt32(1)),
|
||||
&normalized_values,
|
||||
);
|
||||
assert!(result.is_err());
|
||||
|
||||
// Test with Column-Column
|
||||
let result = Collider::try_create_nucleon(
|
||||
&col("a"),
|
||||
&RestrictedOp::Eq,
|
||||
&col("b"),
|
||||
&normalized_values,
|
||||
);
|
||||
assert!(result.is_err());
|
||||
|
||||
// Test with Value-Value
|
||||
let result = Collider::try_create_nucleon(
|
||||
&Operand::Value(Value::UInt32(1)),
|
||||
&RestrictedOp::Eq,
|
||||
&Operand::Value(Value::UInt32(2)),
|
||||
&normalized_values,
|
||||
);
|
||||
assert!(result.is_err());
|
||||
|
||||
// Test empty expression list
|
||||
let exprs = vec![];
|
||||
let collider = Collider::new(&exprs).unwrap();
|
||||
assert_eq!(collider.atomic_exprs.len(), 0);
|
||||
assert_eq!(collider.normalized_values.len(), 0);
|
||||
}
|
||||
}
|
||||
@@ -24,7 +24,7 @@ use snafu::{Location, Snafu};
|
||||
use store_api::storage::RegionId;
|
||||
use table::metadata::TableId;
|
||||
|
||||
use crate::expr::PartitionExpr;
|
||||
use crate::expr::{Operand, PartitionExpr};
|
||||
|
||||
#[derive(Snafu)]
|
||||
#[snafu(visibility(pub))]
|
||||
@@ -140,21 +140,6 @@ pub enum Error {
|
||||
source: common_meta::error::Error,
|
||||
},
|
||||
|
||||
#[snafu(display("Conjunct expr with non-expr is invalid"))]
|
||||
ConjunctExprWithNonExpr {
|
||||
expr: PartitionExpr,
|
||||
#[snafu(implicit)]
|
||||
location: Location,
|
||||
},
|
||||
|
||||
#[snafu(display("Unclosed value {} on column {}", value, column))]
|
||||
UnclosedValue {
|
||||
value: String,
|
||||
column: String,
|
||||
#[snafu(implicit)]
|
||||
location: Location,
|
||||
},
|
||||
|
||||
#[snafu(display("Invalid partition expr: {:?}", expr))]
|
||||
InvalidExpr {
|
||||
expr: PartitionExpr,
|
||||
@@ -162,6 +147,13 @@ pub enum Error {
|
||||
location: Location,
|
||||
},
|
||||
|
||||
#[snafu(display("Unexpected operand: {:?}, want Expr", operand))]
|
||||
NoExprOperand {
|
||||
operand: Operand,
|
||||
#[snafu(implicit)]
|
||||
location: Location,
|
||||
},
|
||||
|
||||
#[snafu(display("Undefined column: {}", column))]
|
||||
UndefinedColumn {
|
||||
column: String,
|
||||
@@ -228,6 +220,27 @@ pub enum Error {
|
||||
#[snafu(implicit)]
|
||||
location: Location,
|
||||
},
|
||||
|
||||
#[snafu(display("Duplicate expr: {:?}", expr))]
|
||||
DuplicateExpr {
|
||||
expr: PartitionExpr,
|
||||
#[snafu(implicit)]
|
||||
location: Location,
|
||||
},
|
||||
|
||||
#[snafu(display("Checkpoint `{}` is not covered", checkpoint))]
|
||||
CheckpointNotCovered {
|
||||
checkpoint: String,
|
||||
#[snafu(implicit)]
|
||||
location: Location,
|
||||
},
|
||||
|
||||
#[snafu(display("Checkpoint `{}` is overlapped", checkpoint))]
|
||||
CheckpointOverlapped {
|
||||
checkpoint: String,
|
||||
#[snafu(implicit)]
|
||||
location: Location,
|
||||
},
|
||||
}
|
||||
|
||||
impl ErrorExt for Error {
|
||||
@@ -236,10 +249,12 @@ impl ErrorExt for Error {
|
||||
Error::GetCache { .. } => StatusCode::StorageUnavailable,
|
||||
Error::FindLeader { .. } => StatusCode::TableUnavailable,
|
||||
|
||||
Error::ConjunctExprWithNonExpr { .. }
|
||||
| Error::UnclosedValue { .. }
|
||||
| Error::InvalidExpr { .. }
|
||||
| Error::UndefinedColumn { .. } => StatusCode::InvalidArguments,
|
||||
Error::InvalidExpr { .. }
|
||||
| Error::NoExprOperand { .. }
|
||||
| Error::UndefinedColumn { .. }
|
||||
| Error::DuplicateExpr { .. }
|
||||
| Error::CheckpointNotCovered { .. }
|
||||
| Error::CheckpointOverlapped { .. } => StatusCode::InvalidArguments,
|
||||
|
||||
Error::RegionKeysSize { .. }
|
||||
| Error::InvalidInsertRequest { .. }
|
||||
|
||||
@@ -115,6 +115,18 @@ impl Operand {
|
||||
pub fn eq(self, rhs: impl Into<Self>) -> PartitionExpr {
|
||||
PartitionExpr::new(self, RestrictedOp::Eq, rhs.into())
|
||||
}
|
||||
|
||||
pub fn not_eq(self, rhs: impl Into<Self>) -> PartitionExpr {
|
||||
PartitionExpr::new(self, RestrictedOp::NotEq, rhs.into())
|
||||
}
|
||||
|
||||
pub fn gt(self, rhs: impl Into<Self>) -> PartitionExpr {
|
||||
PartitionExpr::new(self, RestrictedOp::Gt, rhs.into())
|
||||
}
|
||||
|
||||
pub fn lt_eq(self, rhs: impl Into<Self>) -> PartitionExpr {
|
||||
PartitionExpr::new(self, RestrictedOp::LtEq, rhs.into())
|
||||
}
|
||||
}
|
||||
|
||||
impl Display for Operand {
|
||||
|
||||
@@ -16,6 +16,8 @@
|
||||
#![feature(let_chains)]
|
||||
//! Structs and traits for partitioning rule.
|
||||
|
||||
pub mod checker;
|
||||
pub mod collider;
|
||||
pub mod error;
|
||||
pub mod expr;
|
||||
pub mod manager;
|
||||
|
||||
@@ -167,7 +167,8 @@ impl PartitionRuleManager {
|
||||
})
|
||||
.collect::<Vec<_>>();
|
||||
|
||||
let rule = MultiDimPartitionRule::try_new(partition_columns.clone(), regions, exprs)?;
|
||||
let rule =
|
||||
MultiDimPartitionRule::try_new(partition_columns.clone(), regions, exprs, false)?;
|
||||
Ok(Arc::new(rule) as _)
|
||||
}
|
||||
|
||||
|
||||
@@ -30,10 +30,8 @@ use serde::{Deserialize, Serialize};
|
||||
use snafu::{ensure, OptionExt, ResultExt};
|
||||
use store_api::storage::RegionNumber;
|
||||
|
||||
use crate::error::{
|
||||
self, ConjunctExprWithNonExprSnafu, InvalidExprSnafu, Result, UnclosedValueSnafu,
|
||||
UndefinedColumnSnafu,
|
||||
};
|
||||
use crate::checker::PartitionChecker;
|
||||
use crate::error::{self, Result, UndefinedColumnSnafu};
|
||||
use crate::expr::{Operand, PartitionExpr, RestrictedOp};
|
||||
use crate::partition::RegionMask;
|
||||
use crate::PartitionRule;
|
||||
@@ -65,10 +63,15 @@ pub struct MultiDimPartitionRule {
|
||||
}
|
||||
|
||||
impl MultiDimPartitionRule {
|
||||
/// Create a new [`MultiDimPartitionRule`].
|
||||
///
|
||||
/// If `check_exprs` is true, the function will check if the expressions are valid. This is
|
||||
/// required when constructing a new partition rule like `CREATE TABLE` or `ALTER TABLE`.
|
||||
pub fn try_new(
|
||||
partition_columns: Vec<String>,
|
||||
regions: Vec<RegionNumber>,
|
||||
exprs: Vec<PartitionExpr>,
|
||||
check_exprs: bool,
|
||||
) -> Result<Self> {
|
||||
let name_to_index = partition_columns
|
||||
.iter()
|
||||
@@ -84,12 +87,18 @@ impl MultiDimPartitionRule {
|
||||
physical_expr_cache: RwLock::new(None),
|
||||
};
|
||||
|
||||
let mut checker = RuleChecker::new(&rule);
|
||||
checker.check()?;
|
||||
if check_exprs {
|
||||
let checker = PartitionChecker::try_new(&rule)?;
|
||||
checker.check()?;
|
||||
}
|
||||
|
||||
Ok(rule)
|
||||
}
|
||||
|
||||
pub fn exprs(&self) -> &[PartitionExpr] {
|
||||
&self.exprs
|
||||
}
|
||||
|
||||
fn find_region(&self, values: &[Value]) -> Result<RegionNumber> {
|
||||
ensure!(
|
||||
values.len() == self.partition_columns.len(),
|
||||
@@ -167,7 +176,7 @@ impl MultiDimPartitionRule {
|
||||
.map(|col_name| {
|
||||
record_batch
|
||||
.column_by_name(col_name)
|
||||
.context(error::UndefinedColumnSnafu { column: col_name })
|
||||
.context(UndefinedColumnSnafu { column: col_name })
|
||||
.and_then(|array| {
|
||||
Helper::try_into_vector(array).context(error::ConvertToVectorSnafu)
|
||||
})
|
||||
@@ -341,142 +350,13 @@ impl PartitionRule for MultiDimPartitionRule {
|
||||
}
|
||||
}
|
||||
|
||||
/// Helper for [RuleChecker]
|
||||
type Axis = HashMap<Value, SplitPoint>;
|
||||
|
||||
/// Helper for [RuleChecker]
|
||||
struct SplitPoint {
|
||||
is_equal: bool,
|
||||
less_than_counter: isize,
|
||||
}
|
||||
|
||||
/// Check if the rule set covers all the possible values.
|
||||
///
|
||||
/// Note this checker have false-negative on duplicated exprs. E.g.:
|
||||
/// `a != 20`, `a <= 20` and `a > 20`.
|
||||
///
|
||||
/// It works on the observation that each projected split point should be included (`is_equal`)
|
||||
/// and have a balanced `<` and `>` counter.
|
||||
struct RuleChecker<'a> {
|
||||
axis: Vec<Axis>,
|
||||
rule: &'a MultiDimPartitionRule,
|
||||
}
|
||||
|
||||
impl<'a> RuleChecker<'a> {
|
||||
pub fn new(rule: &'a MultiDimPartitionRule) -> Self {
|
||||
let mut projections = Vec::with_capacity(rule.partition_columns.len());
|
||||
projections.resize_with(rule.partition_columns.len(), Default::default);
|
||||
|
||||
Self {
|
||||
axis: projections,
|
||||
rule,
|
||||
}
|
||||
}
|
||||
|
||||
pub fn check(&mut self) -> Result<()> {
|
||||
for expr in &self.rule.exprs {
|
||||
self.walk_expr(expr)?
|
||||
}
|
||||
|
||||
self.check_axis()
|
||||
}
|
||||
|
||||
#[allow(clippy::mutable_key_type)]
|
||||
fn walk_expr(&mut self, expr: &PartitionExpr) -> Result<()> {
|
||||
// recursively check the expr
|
||||
match expr.op {
|
||||
RestrictedOp::And | RestrictedOp::Or => {
|
||||
match (expr.lhs.as_ref(), expr.rhs.as_ref()) {
|
||||
(Operand::Expr(lhs), Operand::Expr(rhs)) => {
|
||||
self.walk_expr(lhs)?;
|
||||
self.walk_expr(rhs)?
|
||||
}
|
||||
_ => ConjunctExprWithNonExprSnafu { expr: expr.clone() }.fail()?,
|
||||
}
|
||||
|
||||
return Ok(());
|
||||
}
|
||||
// Not conjunction
|
||||
_ => {}
|
||||
}
|
||||
|
||||
let (col, val) = match (expr.lhs.as_ref(), expr.rhs.as_ref()) {
|
||||
(Operand::Expr(_), _)
|
||||
| (_, Operand::Expr(_))
|
||||
| (Operand::Column(_), Operand::Column(_))
|
||||
| (Operand::Value(_), Operand::Value(_)) => {
|
||||
InvalidExprSnafu { expr: expr.clone() }.fail()?
|
||||
}
|
||||
|
||||
(Operand::Column(col), Operand::Value(val))
|
||||
| (Operand::Value(val), Operand::Column(col)) => (col, val),
|
||||
};
|
||||
|
||||
let col_index =
|
||||
*self
|
||||
.rule
|
||||
.name_to_index
|
||||
.get(col)
|
||||
.with_context(|| UndefinedColumnSnafu {
|
||||
column: col.clone(),
|
||||
})?;
|
||||
let axis = &mut self.axis[col_index];
|
||||
let split_point = axis.entry(val.clone()).or_insert(SplitPoint {
|
||||
is_equal: false,
|
||||
less_than_counter: 0,
|
||||
});
|
||||
match expr.op {
|
||||
RestrictedOp::Eq => {
|
||||
split_point.is_equal = true;
|
||||
}
|
||||
RestrictedOp::NotEq => {
|
||||
// less_than +1 -1
|
||||
}
|
||||
RestrictedOp::Lt => {
|
||||
split_point.less_than_counter += 1;
|
||||
}
|
||||
RestrictedOp::LtEq => {
|
||||
split_point.less_than_counter += 1;
|
||||
split_point.is_equal = true;
|
||||
}
|
||||
RestrictedOp::Gt => {
|
||||
split_point.less_than_counter -= 1;
|
||||
}
|
||||
RestrictedOp::GtEq => {
|
||||
split_point.less_than_counter -= 1;
|
||||
split_point.is_equal = true;
|
||||
}
|
||||
RestrictedOp::And | RestrictedOp::Or => {
|
||||
unreachable!("conjunct expr should be handled above")
|
||||
}
|
||||
}
|
||||
|
||||
Ok(())
|
||||
}
|
||||
|
||||
/// Return if the rule is legal.
|
||||
fn check_axis(&self) -> Result<()> {
|
||||
for (col_index, axis) in self.axis.iter().enumerate() {
|
||||
for (val, split_point) in axis {
|
||||
if split_point.less_than_counter != 0 || !split_point.is_equal {
|
||||
UnclosedValueSnafu {
|
||||
value: format!("{val:?}"),
|
||||
column: self.rule.partition_columns[col_index].clone(),
|
||||
}
|
||||
.fail()?;
|
||||
}
|
||||
}
|
||||
}
|
||||
Ok(())
|
||||
}
|
||||
}
|
||||
|
||||
#[cfg(test)]
|
||||
mod tests {
|
||||
use std::assert_matches::assert_matches;
|
||||
|
||||
use super::*;
|
||||
use crate::error::{self, Error};
|
||||
use crate::expr::col;
|
||||
|
||||
#[test]
|
||||
fn test_find_region() {
|
||||
@@ -513,6 +393,7 @@ mod tests {
|
||||
Operand::Value(datatypes::value::Value::String("sh".into())),
|
||||
),
|
||||
],
|
||||
true,
|
||||
)
|
||||
.unwrap();
|
||||
assert_matches!(
|
||||
@@ -556,6 +437,7 @@ mod tests {
|
||||
)),
|
||||
)),
|
||||
)],
|
||||
true,
|
||||
);
|
||||
|
||||
// check rule
|
||||
@@ -579,10 +461,11 @@ mod tests {
|
||||
RestrictedOp::And,
|
||||
Operand::Value(datatypes::value::Value::String("sh".into())),
|
||||
)],
|
||||
true,
|
||||
);
|
||||
|
||||
// check rule
|
||||
assert_matches!(rule.unwrap_err(), Error::ConjunctExprWithNonExpr { .. });
|
||||
assert_matches!(rule.unwrap_err(), Error::InvalidExpr { .. });
|
||||
}
|
||||
|
||||
/// ```ignore
|
||||
@@ -614,10 +497,11 @@ mod tests {
|
||||
Operand::Value(datatypes::value::Value::String("s".into())),
|
||||
),
|
||||
],
|
||||
true,
|
||||
);
|
||||
|
||||
// check rule
|
||||
assert_matches!(rule.unwrap_err(), Error::UnclosedValue { .. });
|
||||
assert_matches!(rule.unwrap_err(), Error::CheckpointNotCovered { .. });
|
||||
}
|
||||
|
||||
/// ```
|
||||
@@ -757,10 +641,11 @@ mod tests {
|
||||
)),
|
||||
),
|
||||
],
|
||||
true,
|
||||
);
|
||||
|
||||
// check rule
|
||||
assert_matches!(rule.unwrap_err(), Error::UnclosedValue { .. });
|
||||
assert_matches!(rule.unwrap_err(), Error::CheckpointNotCovered { .. });
|
||||
}
|
||||
|
||||
#[test]
|
||||
@@ -784,14 +669,14 @@ mod tests {
|
||||
Operand::Value(datatypes::value::Value::Int64(10)),
|
||||
),
|
||||
],
|
||||
true,
|
||||
);
|
||||
|
||||
// check rule
|
||||
assert_matches!(rule.unwrap_err(), Error::UnclosedValue { .. });
|
||||
assert_matches!(rule.unwrap_err(), Error::CheckpointOverlapped { .. });
|
||||
}
|
||||
|
||||
#[test]
|
||||
#[ignore = "checker cannot detect this kind of duplicate for now"]
|
||||
fn duplicate_expr_case_2() {
|
||||
// PARTITION ON COLUMNS (a) (
|
||||
// a != 20,
|
||||
@@ -818,10 +703,40 @@ mod tests {
|
||||
Operand::Value(datatypes::value::Value::Int64(20)),
|
||||
),
|
||||
],
|
||||
true,
|
||||
);
|
||||
|
||||
// check rule
|
||||
assert!(rule.is_err());
|
||||
assert_matches!(rule.unwrap_err(), Error::CheckpointOverlapped { .. });
|
||||
}
|
||||
|
||||
/// ```ignore
|
||||
/// value
|
||||
/// │
|
||||
/// │
|
||||
/// value=10 --------------------│
|
||||
/// │
|
||||
/// ────────────────────────────────┼──► host
|
||||
/// │
|
||||
/// host=server10
|
||||
/// ```
|
||||
#[test]
|
||||
fn test_partial_divided() {
|
||||
let _rule = MultiDimPartitionRule::try_new(
|
||||
vec!["host".to_string(), "value".to_string()],
|
||||
vec![0, 1, 2, 3],
|
||||
vec![
|
||||
col("host")
|
||||
.lt(Value::String("server10".into()))
|
||||
.and(col("value").lt(Value::Int64(10))),
|
||||
col("host")
|
||||
.lt(Value::String("server10".into()))
|
||||
.and(col("value").gt_eq(Value::Int64(10))),
|
||||
col("host").gt_eq(Value::String("server10".into())),
|
||||
],
|
||||
true,
|
||||
)
|
||||
.unwrap();
|
||||
}
|
||||
}
|
||||
|
||||
@@ -868,6 +783,7 @@ mod test_split_record_batch {
|
||||
col("host").lt(Value::String("server1".into())),
|
||||
col("host").gt_eq(Value::String("server1".into())),
|
||||
],
|
||||
true,
|
||||
)
|
||||
.unwrap();
|
||||
|
||||
@@ -892,11 +808,11 @@ mod test_split_record_batch {
|
||||
let rule = MultiDimPartitionRule::try_new(
|
||||
vec!["host".to_string()],
|
||||
vec![1],
|
||||
vec![PartitionExpr::new(
|
||||
Operand::Column("host".to_string()),
|
||||
RestrictedOp::Eq,
|
||||
Operand::Value(Value::String("server1".into())),
|
||||
)],
|
||||
vec![
|
||||
col("host").lt(Value::String("server1".into())),
|
||||
col("host").gt_eq(Value::String("server1".into())),
|
||||
],
|
||||
true,
|
||||
)
|
||||
.unwrap();
|
||||
|
||||
@@ -929,6 +845,7 @@ mod test_split_record_batch {
|
||||
.gt_eq(Value::String("server10".into()))
|
||||
.and(col("value").gt_eq(Value::Int64(10))),
|
||||
],
|
||||
true,
|
||||
)
|
||||
.unwrap();
|
||||
|
||||
@@ -941,118 +858,6 @@ mod test_split_record_batch {
|
||||
}
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_default_region() {
|
||||
let rule = MultiDimPartitionRule::try_new(
|
||||
vec!["host".to_string(), "value".to_string()],
|
||||
vec![0, 1, 2, 3],
|
||||
vec![
|
||||
col("host")
|
||||
.lt(Value::String("server10".into()))
|
||||
.and(col("value").eq(Value::Int64(10))),
|
||||
col("host")
|
||||
.lt(Value::String("server10".into()))
|
||||
.and(col("value").eq(Value::Int64(20))),
|
||||
col("host")
|
||||
.gt_eq(Value::String("server10".into()))
|
||||
.and(col("value").eq(Value::Int64(10))),
|
||||
col("host")
|
||||
.gt_eq(Value::String("server10".into()))
|
||||
.and(col("value").eq(Value::Int64(20))),
|
||||
],
|
||||
)
|
||||
.unwrap();
|
||||
|
||||
let schema = test_schema();
|
||||
let host_array = StringArray::from(vec!["server1", "server1", "server1", "server100"]);
|
||||
let value_array = Int64Array::from(vec![10, 20, 30, 10]);
|
||||
let batch = RecordBatch::try_new(schema, vec![Arc::new(host_array), Arc::new(value_array)])
|
||||
.unwrap();
|
||||
let result = rule.split_record_batch(&batch).unwrap();
|
||||
let expected = rule.split_record_batch_naive(&batch).unwrap();
|
||||
for (region, value) in &result {
|
||||
assert_eq!(value.array(), expected.get(region).unwrap());
|
||||
}
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_default_region_with_unselected_rows() {
|
||||
// Create a rule where some rows won't match any partition
|
||||
let rule = MultiDimPartitionRule::try_new(
|
||||
vec!["host".to_string(), "value".to_string()],
|
||||
vec![1, 2, 3],
|
||||
vec![
|
||||
col("value").eq(Value::Int64(10)),
|
||||
col("value").eq(Value::Int64(20)),
|
||||
col("value").eq(Value::Int64(30)),
|
||||
],
|
||||
)
|
||||
.unwrap();
|
||||
|
||||
let schema = test_schema();
|
||||
let host_array =
|
||||
StringArray::from(vec!["server1", "server2", "server3", "server4", "server5"]);
|
||||
let value_array = Int64Array::from(vec![10, 20, 30, 40, 50]);
|
||||
let batch = RecordBatch::try_new(schema, vec![Arc::new(host_array), Arc::new(value_array)])
|
||||
.unwrap();
|
||||
|
||||
let result = rule.split_record_batch(&batch).unwrap();
|
||||
|
||||
// Check that we have 4 regions (3 defined + default)
|
||||
assert_eq!(result.len(), 4);
|
||||
|
||||
// Check that default region (0) contains the unselected rows
|
||||
assert!(result.contains_key(&DEFAULT_REGION));
|
||||
let default_mask = result.get(&DEFAULT_REGION).unwrap();
|
||||
|
||||
// The default region should have 2 rows (with values 40 and 50)
|
||||
assert_eq!(default_mask.selected_rows(), 2);
|
||||
|
||||
// Verify each region has the correct number of rows
|
||||
assert_eq!(result.get(&1).unwrap().selected_rows(), 1); // value = 10
|
||||
assert_eq!(result.get(&2).unwrap().selected_rows(), 1); // value = 20
|
||||
assert_eq!(result.get(&3).unwrap().selected_rows(), 1); // value = 30
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_default_region_with_existing_default() {
|
||||
// Create a rule where some rows are explicitly assigned to default region
|
||||
// and some rows are implicitly assigned to default region
|
||||
let rule = MultiDimPartitionRule::try_new(
|
||||
vec!["host".to_string(), "value".to_string()],
|
||||
vec![0, 1, 2],
|
||||
vec![
|
||||
col("value").eq(Value::Int64(10)), // Explicitly assign value=10 to region 0 (default)
|
||||
col("value").eq(Value::Int64(20)),
|
||||
col("value").eq(Value::Int64(30)),
|
||||
],
|
||||
)
|
||||
.unwrap();
|
||||
|
||||
let schema = test_schema();
|
||||
let host_array =
|
||||
StringArray::from(vec!["server1", "server2", "server3", "server4", "server5"]);
|
||||
let value_array = Int64Array::from(vec![10, 20, 30, 40, 50]);
|
||||
let batch = RecordBatch::try_new(schema, vec![Arc::new(host_array), Arc::new(value_array)])
|
||||
.unwrap();
|
||||
|
||||
let result = rule.split_record_batch(&batch).unwrap();
|
||||
|
||||
// Check that we have 3 regions
|
||||
assert_eq!(result.len(), 3);
|
||||
|
||||
// Check that default region contains both explicitly assigned and unselected rows
|
||||
assert!(result.contains_key(&DEFAULT_REGION));
|
||||
let default_mask = result.get(&DEFAULT_REGION).unwrap();
|
||||
|
||||
// The default region should have 3 rows (value=10, 40, 50)
|
||||
assert_eq!(default_mask.selected_rows(), 3);
|
||||
|
||||
// Verify each region has the correct number of rows
|
||||
assert_eq!(result.get(&1).unwrap().selected_rows(), 1); // value = 20
|
||||
assert_eq!(result.get(&2).unwrap().selected_rows(), 1); // value = 30
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_all_rows_selected() {
|
||||
// Test the fast path where all rows are selected by some partition
|
||||
@@ -1063,6 +868,7 @@ mod test_split_record_batch {
|
||||
col("value").lt(Value::Int64(30)),
|
||||
col("value").gt_eq(Value::Int64(30)),
|
||||
],
|
||||
true,
|
||||
)
|
||||
.unwrap();
|
||||
|
||||
|
||||
@@ -18,7 +18,6 @@ use std::fmt::{Debug, Display, Formatter};
|
||||
use std::sync::Arc;
|
||||
|
||||
use common_meta::rpc::router::Partition as MetaPartition;
|
||||
use datafusion_expr::Operator;
|
||||
use datatypes::arrow::array::{BooleanArray, RecordBatch};
|
||||
use datatypes::prelude::Value;
|
||||
use itertools::Itertools;
|
||||
@@ -157,27 +156,6 @@ impl TryFrom<PartitionDef> for MetaPartition {
|
||||
}
|
||||
}
|
||||
|
||||
#[derive(Debug, PartialEq, Eq)]
|
||||
pub struct PartitionExpr {
|
||||
pub column: String,
|
||||
pub op: Operator,
|
||||
pub value: Value,
|
||||
}
|
||||
|
||||
impl PartitionExpr {
|
||||
pub fn new(column: impl Into<String>, op: Operator, value: Value) -> Self {
|
||||
Self {
|
||||
column: column.into(),
|
||||
op,
|
||||
value,
|
||||
}
|
||||
}
|
||||
|
||||
pub fn value(&self) -> &Value {
|
||||
&self.value
|
||||
}
|
||||
}
|
||||
|
||||
pub struct RegionMask {
|
||||
array: BooleanArray,
|
||||
selected_rows: usize,
|
||||
|
||||
@@ -220,6 +220,10 @@ impl ContextReq {
|
||||
pub fn ref_all_req(&self) -> impl Iterator<Item = &RowInsertRequest> {
|
||||
self.req.values().flatten()
|
||||
}
|
||||
|
||||
pub fn map_len(&self) -> usize {
|
||||
self.req.len()
|
||||
}
|
||||
}
|
||||
|
||||
// ContextReqIter is an iterator that iterates over the ContextReq.
|
||||
|
||||
@@ -24,7 +24,7 @@ pub use etl::processor::Processor;
|
||||
pub use etl::transform::transformer::greptime::{GreptimePipelineParams, SchemaInfo};
|
||||
pub use etl::transform::transformer::identity_pipeline;
|
||||
pub use etl::transform::GreptimeTransformer;
|
||||
pub use etl::value::{Array, Map, Value};
|
||||
pub use etl::value::{Array, Map, Timestamp, Value};
|
||||
pub use etl::{
|
||||
json_array_to_map, json_to_map, parse, simd_json_array_to_map, simd_json_to_map, Content,
|
||||
DispatchedTo, Pipeline, PipelineExecOutput, TransformedOutput, TransformerMode,
|
||||
@@ -50,24 +50,3 @@ macro_rules! unwrap_or_continue_if_err {
|
||||
}
|
||||
}};
|
||||
}
|
||||
|
||||
#[macro_export]
|
||||
macro_rules! unwrap_or_warn_continue {
|
||||
($expr:expr, $msg:expr) => {
|
||||
if let Some(value) = $expr {
|
||||
value
|
||||
} else {
|
||||
warn!($msg);
|
||||
continue;
|
||||
}
|
||||
};
|
||||
|
||||
($expr:expr, $fmt:expr, $($arg:tt)*) => {
|
||||
if let Some(value) = $expr {
|
||||
value
|
||||
} else {
|
||||
warn!($fmt, $($arg)*);
|
||||
continue;
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
@@ -201,6 +201,12 @@ pub enum Error {
|
||||
#[snafu(implicit)]
|
||||
location: Location,
|
||||
},
|
||||
|
||||
#[snafu(display("vector cannot contain metrics with the same labelset"))]
|
||||
SameLabelSet {
|
||||
#[snafu(implicit)]
|
||||
location: Location,
|
||||
},
|
||||
}
|
||||
|
||||
impl ErrorExt for Error {
|
||||
@@ -221,6 +227,7 @@ impl ErrorExt for Error {
|
||||
| CombineTableColumnMismatch { .. }
|
||||
| UnexpectedPlanExpr { .. }
|
||||
| UnsupportedMatcherOp { .. }
|
||||
| SameLabelSet { .. }
|
||||
| TimestampOutOfRange { .. } => StatusCode::InvalidArguments,
|
||||
|
||||
UnknownTable { .. } => StatusCode::Internal,
|
||||
|
||||
@@ -76,7 +76,7 @@ use crate::promql::error::{
|
||||
CatalogSnafu, ColumnNotFoundSnafu, CombineTableColumnMismatchSnafu, DataFusionPlanningSnafu,
|
||||
ExpectRangeSelectorSnafu, FunctionInvalidArgumentSnafu, InvalidTimeRangeSnafu,
|
||||
MultiFieldsNotSupportedSnafu, MultipleMetricMatchersSnafu, MultipleVectorSnafu,
|
||||
NoMetricMatcherSnafu, PromqlPlanNodeSnafu, Result, TableNameNotFoundSnafu,
|
||||
NoMetricMatcherSnafu, PromqlPlanNodeSnafu, Result, SameLabelSetSnafu, TableNameNotFoundSnafu,
|
||||
TimeIndexNotFoundSnafu, UnexpectedPlanExprSnafu, UnexpectedTokenSnafu, UnknownTableSnafu,
|
||||
UnsupportedExprSnafu, UnsupportedMatcherOpSnafu, UnsupportedVectorMatchSnafu,
|
||||
ValueNotFoundSnafu, ZeroRangeSelectorSnafu,
|
||||
@@ -786,7 +786,7 @@ impl PromPlanner {
|
||||
),
|
||||
})
|
||||
};
|
||||
let mut func_exprs =
|
||||
let (mut func_exprs, new_tags) =
|
||||
self.create_function_expr(func, args.literals.clone(), session_state)?;
|
||||
func_exprs.insert(0, self.create_time_index_column_expr()?);
|
||||
func_exprs.extend_from_slice(&self.create_tag_column_exprs()?);
|
||||
@@ -822,6 +822,12 @@ impl PromPlanner {
|
||||
_ => builder,
|
||||
};
|
||||
|
||||
// Update context tags after building plan
|
||||
// We can't push them before planning, because they won't exist until projection.
|
||||
for tag in new_tags {
|
||||
self.ctx.tag_columns.push(tag);
|
||||
}
|
||||
|
||||
builder.build().context(DataFusionPlanningSnafu)
|
||||
}
|
||||
|
||||
@@ -1458,21 +1464,25 @@ impl PromPlanner {
|
||||
Ok(result)
|
||||
}
|
||||
|
||||
/// Creates function expressions for projection and returns the expressions and new tags.
|
||||
///
|
||||
/// # Side Effects
|
||||
///
|
||||
/// This method will update [PromPlannerContext]'s value fields.
|
||||
/// This method will update [PromPlannerContext]'s fields and tags if needed.
|
||||
fn create_function_expr(
|
||||
&mut self,
|
||||
func: &Function,
|
||||
other_input_exprs: Vec<DfExpr>,
|
||||
session_state: &SessionState,
|
||||
) -> Result<Vec<DfExpr>> {
|
||||
) -> Result<(Vec<DfExpr>, Vec<String>)> {
|
||||
// TODO(ruihang): check function args list
|
||||
let mut other_input_exprs: VecDeque<DfExpr> = other_input_exprs.into();
|
||||
|
||||
// TODO(ruihang): set this according to in-param list
|
||||
let field_column_pos = 0;
|
||||
let mut exprs = Vec::with_capacity(self.ctx.field_columns.len());
|
||||
// New labels after executing the function, e.g. `label_replace` etc.
|
||||
let mut new_tags = vec![];
|
||||
let scalar_func = match func.name {
|
||||
"increase" => ScalarFunc::ExtrapolateUdf(
|
||||
Arc::new(Increase::scalar_udf()),
|
||||
@@ -1605,32 +1615,41 @@ impl PromPlanner {
|
||||
}
|
||||
}
|
||||
|
||||
// Remove it from tag columns
|
||||
// Remove it from tag columns if exists to avoid duplicated column names
|
||||
self.ctx.tag_columns.retain(|tag| *tag != dst_label);
|
||||
|
||||
// Add the new label expr
|
||||
new_tags.push(dst_label);
|
||||
// Add the new label expr to evaluate
|
||||
exprs.push(concat_expr);
|
||||
|
||||
ScalarFunc::GeneratedExpr
|
||||
}
|
||||
"label_replace" => {
|
||||
let (replace_expr, dst_label) =
|
||||
Self::build_regexp_replace_label_expr(&mut other_input_exprs, session_state)?;
|
||||
if let Some((replace_expr, dst_label)) =
|
||||
self.build_regexp_replace_label_expr(&mut other_input_exprs, session_state)?
|
||||
{
|
||||
// Reserve the current field columns except the `dst_label`.
|
||||
for value in &self.ctx.field_columns {
|
||||
if *value != dst_label {
|
||||
let expr = DfExpr::Column(Column::from_name(value));
|
||||
exprs.push(expr);
|
||||
}
|
||||
}
|
||||
|
||||
// Reserve the current field columns except the `dst_label`.
|
||||
for value in &self.ctx.field_columns {
|
||||
if *value != dst_label {
|
||||
ensure!(
|
||||
!self.ctx.tag_columns.contains(&dst_label),
|
||||
SameLabelSetSnafu
|
||||
);
|
||||
new_tags.push(dst_label);
|
||||
// Add the new label expr to evaluate
|
||||
exprs.push(replace_expr);
|
||||
} else {
|
||||
// Keep the current field columns
|
||||
for value in &self.ctx.field_columns {
|
||||
let expr = DfExpr::Column(Column::from_name(value));
|
||||
exprs.push(expr);
|
||||
}
|
||||
}
|
||||
|
||||
// Remove it from tag columns
|
||||
self.ctx.tag_columns.retain(|tag| *tag != dst_label);
|
||||
|
||||
// Add the new label expr
|
||||
exprs.push(replace_expr);
|
||||
|
||||
ScalarFunc::GeneratedExpr
|
||||
}
|
||||
"sort" | "sort_desc" | "sort_by_label" | "sort_by_label_desc" => {
|
||||
@@ -1730,29 +1749,34 @@ impl PromPlanner {
|
||||
}
|
||||
}
|
||||
|
||||
// update value columns' name, and alias them to remove qualifiers
|
||||
let mut new_field_columns = Vec::with_capacity(exprs.len());
|
||||
// Update value columns' name, and alias them to remove qualifiers
|
||||
// For label functions such as `label_join`, `label_replace`, etc.,
|
||||
// we keep the fields unchanged.
|
||||
if !matches!(func.name, "label_join" | "label_replace") {
|
||||
let mut new_field_columns = Vec::with_capacity(exprs.len());
|
||||
|
||||
exprs = exprs
|
||||
.into_iter()
|
||||
.map(|expr| {
|
||||
let display_name = expr.schema_name().to_string();
|
||||
new_field_columns.push(display_name.clone());
|
||||
Ok(expr.alias(display_name))
|
||||
})
|
||||
.collect::<std::result::Result<Vec<_>, _>>()
|
||||
.context(DataFusionPlanningSnafu)?;
|
||||
exprs = exprs
|
||||
.into_iter()
|
||||
.map(|expr| {
|
||||
let display_name = expr.schema_name().to_string();
|
||||
new_field_columns.push(display_name.clone());
|
||||
Ok(expr.alias(display_name))
|
||||
})
|
||||
.collect::<std::result::Result<Vec<_>, _>>()
|
||||
.context(DataFusionPlanningSnafu)?;
|
||||
|
||||
self.ctx.field_columns = new_field_columns;
|
||||
self.ctx.field_columns = new_field_columns;
|
||||
}
|
||||
|
||||
Ok(exprs)
|
||||
Ok((exprs, new_tags))
|
||||
}
|
||||
|
||||
/// Build expr for `label_replace` function
|
||||
fn build_regexp_replace_label_expr(
|
||||
&self,
|
||||
other_input_exprs: &mut VecDeque<DfExpr>,
|
||||
session_state: &SessionState,
|
||||
) -> Result<(DfExpr, String)> {
|
||||
) -> Result<Option<(DfExpr, String)>> {
|
||||
// label_replace(vector, dst_label, replacement, src_label, regex)
|
||||
let dst_label = match other_input_exprs.pop_front() {
|
||||
Some(DfExpr::Literal(ScalarValue::Utf8(Some(d)))) => d,
|
||||
@@ -1775,6 +1799,7 @@ impl PromPlanner {
|
||||
}
|
||||
.fail()?,
|
||||
};
|
||||
|
||||
let regex = match other_input_exprs.pop_front() {
|
||||
Some(DfExpr::Literal(ScalarValue::Utf8(Some(r)))) => r,
|
||||
other => UnexpectedPlanExprSnafu {
|
||||
@@ -1783,6 +1808,30 @@ impl PromPlanner {
|
||||
.fail()?,
|
||||
};
|
||||
|
||||
// If the src_label exists and regex is empty, keep everything unchanged.
|
||||
if self.ctx.tag_columns.contains(&src_label) && regex.is_empty() {
|
||||
return Ok(None);
|
||||
}
|
||||
|
||||
// If the src_label doesn't exists, and
|
||||
if !self.ctx.tag_columns.contains(&src_label) {
|
||||
if replacement.is_empty() {
|
||||
// the replacement is empty, keep everything unchanged.
|
||||
return Ok(None);
|
||||
} else {
|
||||
// the replacement is not empty, always adds dst_label with replacement value.
|
||||
return Ok(Some((
|
||||
// alias literal `replacement` as dst_label
|
||||
DfExpr::Literal(ScalarValue::Utf8(Some(replacement))).alias(&dst_label),
|
||||
dst_label,
|
||||
)));
|
||||
}
|
||||
}
|
||||
|
||||
// Preprocess the regex:
|
||||
// https://github.com/prometheus/prometheus/blob/d902abc50d6652ba8fe9a81ff8e5cce936114eba/promql/functions.go#L1575C32-L1575C37
|
||||
let regex = format!("^(?s:{regex})$");
|
||||
|
||||
let func = session_state
|
||||
.scalar_functions()
|
||||
.get("regexp_replace")
|
||||
@@ -1801,14 +1850,14 @@ impl PromPlanner {
|
||||
DfExpr::Literal(ScalarValue::Utf8(Some(replacement))),
|
||||
];
|
||||
|
||||
Ok((
|
||||
Ok(Some((
|
||||
DfExpr::ScalarFunction(ScalarFunction {
|
||||
func: func.clone(),
|
||||
args,
|
||||
})
|
||||
.alias(&dst_label),
|
||||
dst_label,
|
||||
))
|
||||
)))
|
||||
}
|
||||
|
||||
/// Build expr for `label_join` function
|
||||
@@ -3763,7 +3812,7 @@ mod test {
|
||||
lookback_delta: Duration::from_secs(1),
|
||||
};
|
||||
|
||||
let case = r#"label_replace(histogram_quantile(0.99, sum by(pod, le, path, code) (rate(greptime_servers_grpc_requests_elapsed_bucket{container="frontend"}[1m0s]))), "pod", "$1", "pod", "greptimedb-frontend-[0-9a-z]*-(.*)")"#;
|
||||
let case = r#"label_replace(histogram_quantile(0.99, sum by(pod, le, path, code) (rate(greptime_servers_grpc_requests_elapsed_bucket{container="frontend"}[1m0s]))), "pod_new", "$1", "pod", "greptimedb-frontend-[0-9a-z]*-(.*)")"#;
|
||||
|
||||
let prom_expr = parser::parse(case).unwrap();
|
||||
eval_stmt.expr = prom_expr;
|
||||
@@ -4339,15 +4388,17 @@ mod test {
|
||||
.await
|
||||
.unwrap();
|
||||
|
||||
let expected = "Filter: field_0 IS NOT NULL AND foo IS NOT NULL [timestamp:Timestamp(Millisecond, None), field_0:Float64;N, foo:Utf8;N, tag_0:Utf8, tag_1:Utf8, tag_2:Utf8, tag_3:Utf8]\
|
||||
\n Projection: up.timestamp, up.field_0 AS field_0, concat_ws(Utf8(\",\"), up.tag_1, up.tag_2, up.tag_3) AS foo AS foo, up.tag_0, up.tag_1, up.tag_2, up.tag_3 [timestamp:Timestamp(Millisecond, None), field_0:Float64;N, foo:Utf8;N, tag_0:Utf8, tag_1:Utf8, tag_2:Utf8, tag_3:Utf8]\
|
||||
\n PromInstantManipulate: range=[0..100000000], lookback=[1000], interval=[5000], time index=[timestamp] [tag_0:Utf8, tag_1:Utf8, tag_2:Utf8, tag_3:Utf8, timestamp:Timestamp(Millisecond, None), field_0:Float64;N]\
|
||||
\n PromSeriesDivide: tags=[\"tag_0\", \"tag_1\", \"tag_2\", \"tag_3\"] [tag_0:Utf8, tag_1:Utf8, tag_2:Utf8, tag_3:Utf8, timestamp:Timestamp(Millisecond, None), field_0:Float64;N]\
|
||||
\n Sort: up.tag_0 ASC NULLS FIRST, up.tag_1 ASC NULLS FIRST, up.tag_2 ASC NULLS FIRST, up.tag_3 ASC NULLS FIRST, up.timestamp ASC NULLS FIRST [tag_0:Utf8, tag_1:Utf8, tag_2:Utf8, tag_3:Utf8, timestamp:Timestamp(Millisecond, None), field_0:Float64;N]\
|
||||
\n Filter: up.tag_0 = Utf8(\"api-server\") AND up.timestamp >= TimestampMillisecond(-1000, None) AND up.timestamp <= TimestampMillisecond(100001000, None) [tag_0:Utf8, tag_1:Utf8, tag_2:Utf8, tag_3:Utf8, timestamp:Timestamp(Millisecond, None), field_0:Float64;N]\
|
||||
\n TableScan: up [tag_0:Utf8, tag_1:Utf8, tag_2:Utf8, tag_3:Utf8, timestamp:Timestamp(Millisecond, None), field_0:Float64;N]";
|
||||
let expected = r#"
|
||||
Filter: up.field_0 IS NOT NULL [timestamp:Timestamp(Millisecond, None), field_0:Float64;N, foo:Utf8;N, tag_0:Utf8, tag_1:Utf8, tag_2:Utf8, tag_3:Utf8]
|
||||
Projection: up.timestamp, up.field_0, concat_ws(Utf8(","), up.tag_1, up.tag_2, up.tag_3) AS foo, up.tag_0, up.tag_1, up.tag_2, up.tag_3 [timestamp:Timestamp(Millisecond, None), field_0:Float64;N, foo:Utf8;N, tag_0:Utf8, tag_1:Utf8, tag_2:Utf8, tag_3:Utf8]
|
||||
PromInstantManipulate: range=[0..100000000], lookback=[1000], interval=[5000], time index=[timestamp] [tag_0:Utf8, tag_1:Utf8, tag_2:Utf8, tag_3:Utf8, timestamp:Timestamp(Millisecond, None), field_0:Float64;N]
|
||||
PromSeriesDivide: tags=["tag_0", "tag_1", "tag_2", "tag_3"] [tag_0:Utf8, tag_1:Utf8, tag_2:Utf8, tag_3:Utf8, timestamp:Timestamp(Millisecond, None), field_0:Float64;N]
|
||||
Sort: up.tag_0 ASC NULLS FIRST, up.tag_1 ASC NULLS FIRST, up.tag_2 ASC NULLS FIRST, up.tag_3 ASC NULLS FIRST, up.timestamp ASC NULLS FIRST [tag_0:Utf8, tag_1:Utf8, tag_2:Utf8, tag_3:Utf8, timestamp:Timestamp(Millisecond, None), field_0:Float64;N]
|
||||
Filter: up.tag_0 = Utf8("api-server") AND up.timestamp >= TimestampMillisecond(-1000, None) AND up.timestamp <= TimestampMillisecond(100001000, None) [tag_0:Utf8, tag_1:Utf8, tag_2:Utf8, tag_3:Utf8, timestamp:Timestamp(Millisecond, None), field_0:Float64;N]
|
||||
TableScan: up [tag_0:Utf8, tag_1:Utf8, tag_2:Utf8, tag_3:Utf8, timestamp:Timestamp(Millisecond, None), field_0:Float64;N]"#;
|
||||
|
||||
assert_eq!(plan.display_indent_schema().to_string(), expected);
|
||||
let ret = plan.display_indent_schema().to_string();
|
||||
assert_eq!(format!("\n{ret}"), expected, "\n{}", ret);
|
||||
}
|
||||
|
||||
#[tokio::test]
|
||||
@@ -4373,15 +4424,17 @@ mod test {
|
||||
.await
|
||||
.unwrap();
|
||||
|
||||
let expected = "Filter: field_0 IS NOT NULL AND foo IS NOT NULL [timestamp:Timestamp(Millisecond, None), field_0:Float64;N, foo:Utf8;N, tag_0:Utf8]\
|
||||
\n Projection: up.timestamp, up.field_0 AS field_0, regexp_replace(up.tag_0, Utf8(\"(.*):.*\"), Utf8(\"$1\")) AS foo AS foo, up.tag_0 [timestamp:Timestamp(Millisecond, None), field_0:Float64;N, foo:Utf8;N, tag_0:Utf8]\
|
||||
\n PromInstantManipulate: range=[0..100000000], lookback=[1000], interval=[5000], time index=[timestamp] [tag_0:Utf8, timestamp:Timestamp(Millisecond, None), field_0:Float64;N]\
|
||||
\n PromSeriesDivide: tags=[\"tag_0\"] [tag_0:Utf8, timestamp:Timestamp(Millisecond, None), field_0:Float64;N]\
|
||||
\n Sort: up.tag_0 ASC NULLS FIRST, up.timestamp ASC NULLS FIRST [tag_0:Utf8, timestamp:Timestamp(Millisecond, None), field_0:Float64;N]\
|
||||
\n Filter: up.tag_0 = Utf8(\"a:c\") AND up.timestamp >= TimestampMillisecond(-1000, None) AND up.timestamp <= TimestampMillisecond(100001000, None) [tag_0:Utf8, timestamp:Timestamp(Millisecond, None), field_0:Float64;N]\
|
||||
\n TableScan: up [tag_0:Utf8, timestamp:Timestamp(Millisecond, None), field_0:Float64;N]";
|
||||
let expected = r#"
|
||||
Filter: up.field_0 IS NOT NULL [timestamp:Timestamp(Millisecond, None), field_0:Float64;N, foo:Utf8;N, tag_0:Utf8]
|
||||
Projection: up.timestamp, up.field_0, regexp_replace(up.tag_0, Utf8("^(?s:(.*):.*)$"), Utf8("$1")) AS foo, up.tag_0 [timestamp:Timestamp(Millisecond, None), field_0:Float64;N, foo:Utf8;N, tag_0:Utf8]
|
||||
PromInstantManipulate: range=[0..100000000], lookback=[1000], interval=[5000], time index=[timestamp] [tag_0:Utf8, timestamp:Timestamp(Millisecond, None), field_0:Float64;N]
|
||||
PromSeriesDivide: tags=["tag_0"] [tag_0:Utf8, timestamp:Timestamp(Millisecond, None), field_0:Float64;N]
|
||||
Sort: up.tag_0 ASC NULLS FIRST, up.timestamp ASC NULLS FIRST [tag_0:Utf8, timestamp:Timestamp(Millisecond, None), field_0:Float64;N]
|
||||
Filter: up.tag_0 = Utf8("a:c") AND up.timestamp >= TimestampMillisecond(-1000, None) AND up.timestamp <= TimestampMillisecond(100001000, None) [tag_0:Utf8, timestamp:Timestamp(Millisecond, None), field_0:Float64;N]
|
||||
TableScan: up [tag_0:Utf8, timestamp:Timestamp(Millisecond, None), field_0:Float64;N]"#;
|
||||
|
||||
assert_eq!(plan.display_indent_schema().to_string(), expected);
|
||||
let ret = plan.display_indent_schema().to_string();
|
||||
assert_eq!(format!("\n{ret}"), expected, "\n{}", ret);
|
||||
}
|
||||
|
||||
#[tokio::test]
|
||||
|
||||
@@ -18,8 +18,10 @@ use std::collections::HashMap;
|
||||
|
||||
use common_meta::SchemaOptions;
|
||||
use datatypes::schema::{
|
||||
ColumnDefaultConstraint, ColumnSchema, SchemaRef, COLUMN_FULLTEXT_OPT_KEY_ANALYZER,
|
||||
COLUMN_FULLTEXT_OPT_KEY_BACKEND, COLUMN_FULLTEXT_OPT_KEY_CASE_SENSITIVE,
|
||||
ColumnDefaultConstraint, ColumnSchema, FulltextBackend, SchemaRef,
|
||||
COLUMN_FULLTEXT_OPT_KEY_ANALYZER, COLUMN_FULLTEXT_OPT_KEY_BACKEND,
|
||||
COLUMN_FULLTEXT_OPT_KEY_CASE_SENSITIVE, COLUMN_FULLTEXT_OPT_KEY_FALSE_POSITIVE_RATE,
|
||||
COLUMN_FULLTEXT_OPT_KEY_GRANULARITY, COLUMN_SKIPPING_INDEX_OPT_KEY_FALSE_POSITIVE_RATE,
|
||||
COLUMN_SKIPPING_INDEX_OPT_KEY_GRANULARITY, COLUMN_SKIPPING_INDEX_OPT_KEY_TYPE, COMMENT_KEY,
|
||||
};
|
||||
use snafu::ResultExt;
|
||||
@@ -104,7 +106,7 @@ fn create_column(column_schema: &ColumnSchema, quote_style: char) -> Result<Colu
|
||||
.context(GetFulltextOptionsSnafu)?
|
||||
&& opt.enable
|
||||
{
|
||||
let map = HashMap::from([
|
||||
let mut map = HashMap::from([
|
||||
(
|
||||
COLUMN_FULLTEXT_OPT_KEY_ANALYZER.to_string(),
|
||||
opt.analyzer.to_string(),
|
||||
@@ -118,6 +120,16 @@ fn create_column(column_schema: &ColumnSchema, quote_style: char) -> Result<Colu
|
||||
opt.backend.to_string(),
|
||||
),
|
||||
]);
|
||||
if opt.backend == FulltextBackend::Bloom {
|
||||
map.insert(
|
||||
COLUMN_FULLTEXT_OPT_KEY_GRANULARITY.to_string(),
|
||||
opt.granularity.to_string(),
|
||||
);
|
||||
map.insert(
|
||||
COLUMN_FULLTEXT_OPT_KEY_FALSE_POSITIVE_RATE.to_string(),
|
||||
opt.false_positive_rate().to_string(),
|
||||
);
|
||||
}
|
||||
extensions.fulltext_index_options = Some(map.into());
|
||||
}
|
||||
|
||||
@@ -130,6 +142,10 @@ fn create_column(column_schema: &ColumnSchema, quote_style: char) -> Result<Colu
|
||||
COLUMN_SKIPPING_INDEX_OPT_KEY_GRANULARITY.to_string(),
|
||||
opt.granularity.to_string(),
|
||||
),
|
||||
(
|
||||
COLUMN_SKIPPING_INDEX_OPT_KEY_FALSE_POSITIVE_RATE.to_string(),
|
||||
opt.false_positive_rate().to_string(),
|
||||
),
|
||||
(
|
||||
COLUMN_SKIPPING_INDEX_OPT_KEY_TYPE.to_string(),
|
||||
opt.index_type.to_string(),
|
||||
@@ -327,11 +343,11 @@ mod tests {
|
||||
assert_eq!(
|
||||
r#"
|
||||
CREATE TABLE IF NOT EXISTS "system_metrics" (
|
||||
"id" INT UNSIGNED NULL SKIPPING INDEX WITH(granularity = '4096', type = 'BLOOM'),
|
||||
"id" INT UNSIGNED NULL SKIPPING INDEX WITH(false_positive_rate = '0.01', granularity = '4096', type = 'BLOOM'),
|
||||
"host" STRING NULL INVERTED INDEX,
|
||||
"cpu" DOUBLE NULL,
|
||||
"disk" FLOAT NULL,
|
||||
"msg" STRING NULL FULLTEXT INDEX WITH(analyzer = 'English', backend = 'bloom', case_sensitive = 'false'),
|
||||
"msg" STRING NULL FULLTEXT INDEX WITH(analyzer = 'English', backend = 'bloom', case_sensitive = 'false', false_positive_rate = '0.01', granularity = '10240'),
|
||||
"ts" TIMESTAMP(3) NOT NULL DEFAULT current_timestamp(),
|
||||
TIME INDEX ("ts"),
|
||||
PRIMARY KEY ("id", "host")
|
||||
|
||||
@@ -1 +1 @@
|
||||
v0.10.0
|
||||
v0.10.1
|
||||
|
||||
@@ -39,6 +39,7 @@ use crate::http::event::{
|
||||
extract_pipeline_params_map_from_headers, ingest_logs_inner, LogIngesterQueryParams, LogState,
|
||||
PipelineIngestRequest,
|
||||
};
|
||||
use crate::http::header::constants::GREPTIME_PIPELINE_NAME_HEADER_NAME;
|
||||
use crate::metrics::{
|
||||
METRIC_ELASTICSEARCH_LOGS_DOCS_COUNT, METRIC_ELASTICSEARCH_LOGS_INGESTION_ELAPSED,
|
||||
};
|
||||
@@ -132,7 +133,7 @@ async fn do_handle_bulk_api(
|
||||
// The `schema` is already set in the query_ctx in auth process.
|
||||
query_ctx.set_channel(Channel::Elasticsearch);
|
||||
|
||||
let db = params.db.unwrap_or_else(|| "public".to_string());
|
||||
let db = query_ctx.current_schema();
|
||||
|
||||
// Record the ingestion time histogram.
|
||||
let _timer = METRIC_ELASTICSEARCH_LOGS_INGESTION_ELAPSED
|
||||
@@ -140,11 +141,12 @@ async fn do_handle_bulk_api(
|
||||
.start_timer();
|
||||
|
||||
// If pipeline_name is not provided, use the internal pipeline.
|
||||
let pipeline_name = if let Some(pipeline) = params.pipeline_name {
|
||||
pipeline
|
||||
} else {
|
||||
GREPTIME_INTERNAL_IDENTITY_PIPELINE_NAME.to_string()
|
||||
};
|
||||
let pipeline_name = params.pipeline_name.as_deref().unwrap_or_else(|| {
|
||||
headers
|
||||
.get(GREPTIME_PIPELINE_NAME_HEADER_NAME)
|
||||
.and_then(|v| v.to_str().ok())
|
||||
.unwrap_or(GREPTIME_INTERNAL_IDENTITY_PIPELINE_NAME)
|
||||
});
|
||||
|
||||
// Read the ndjson payload and convert it to a vector of Value.
|
||||
let requests = match parse_bulk_request(&payload, &index, ¶ms.msg_field) {
|
||||
@@ -164,7 +166,7 @@ async fn do_handle_bulk_api(
|
||||
};
|
||||
let log_num = requests.len();
|
||||
|
||||
let pipeline = match PipelineDefinition::from_name(&pipeline_name, None, None) {
|
||||
let pipeline = match PipelineDefinition::from_name(pipeline_name, None, None) {
|
||||
Ok(pipeline) => pipeline,
|
||||
Err(e) => {
|
||||
// should be unreachable
|
||||
|
||||
@@ -117,7 +117,7 @@ const DEFAULT_BODY_LIMIT: ReadableSize = ReadableSize::mb(64);
|
||||
pub const AUTHORIZATION_HEADER: &str = "x-greptime-auth";
|
||||
|
||||
// TODO(fys): This is a temporary workaround, it will be improved later
|
||||
pub static PUBLIC_APIS: [&str; 2] = ["/v1/influxdb/ping", "/v1/influxdb/health"];
|
||||
pub static PUBLIC_APIS: [&str; 3] = ["/v1/influxdb/ping", "/v1/influxdb/health", "/v1/health"];
|
||||
|
||||
#[derive(Default)]
|
||||
pub struct HttpServer {
|
||||
@@ -306,7 +306,8 @@ pub enum GreptimeQueryOutput {
|
||||
#[derive(Default, Debug, Clone, Copy, PartialEq, Eq)]
|
||||
pub enum ResponseFormat {
|
||||
Arrow,
|
||||
Csv,
|
||||
// (with_names, with_types)
|
||||
Csv(bool, bool),
|
||||
Table,
|
||||
#[default]
|
||||
GreptimedbV1,
|
||||
@@ -318,7 +319,9 @@ impl ResponseFormat {
|
||||
pub fn parse(s: &str) -> Option<Self> {
|
||||
match s {
|
||||
"arrow" => Some(ResponseFormat::Arrow),
|
||||
"csv" => Some(ResponseFormat::Csv),
|
||||
"csv" => Some(ResponseFormat::Csv(false, false)),
|
||||
"csvwithnames" => Some(ResponseFormat::Csv(true, false)),
|
||||
"csvwithnamesandtypes" => Some(ResponseFormat::Csv(true, true)),
|
||||
"table" => Some(ResponseFormat::Table),
|
||||
"greptimedb_v1" => Some(ResponseFormat::GreptimedbV1),
|
||||
"influxdb_v1" => Some(ResponseFormat::InfluxdbV1),
|
||||
@@ -330,7 +333,7 @@ impl ResponseFormat {
|
||||
pub fn as_str(&self) -> &'static str {
|
||||
match self {
|
||||
ResponseFormat::Arrow => "arrow",
|
||||
ResponseFormat::Csv => "csv",
|
||||
ResponseFormat::Csv(_, _) => "csv",
|
||||
ResponseFormat::Table => "table",
|
||||
ResponseFormat::GreptimedbV1 => "greptimedb_v1",
|
||||
ResponseFormat::InfluxdbV1 => "influxdb_v1",
|
||||
@@ -720,6 +723,10 @@ impl HttpServer {
|
||||
"/health",
|
||||
routing::get(handler::health).post(handler::health),
|
||||
)
|
||||
.route(
|
||||
&format!("/{HTTP_API_VERSION}/health"),
|
||||
routing::get(handler::health).post(handler::health),
|
||||
)
|
||||
.route(
|
||||
"/ready",
|
||||
routing::get(handler::health).post(handler::health),
|
||||
@@ -1296,6 +1303,16 @@ mod test {
|
||||
"*"
|
||||
);
|
||||
|
||||
let res = client.get("/v1/health").send().await;
|
||||
|
||||
assert_eq!(res.status(), StatusCode::OK);
|
||||
assert_eq!(
|
||||
res.headers()
|
||||
.get(http::header::ACCESS_CONTROL_ALLOW_ORIGIN)
|
||||
.expect("expect cors header origin"),
|
||||
"*"
|
||||
);
|
||||
|
||||
let res = client
|
||||
.options("/health")
|
||||
.header("Access-Control-Request-Headers", "x-greptime-auth")
|
||||
@@ -1480,7 +1497,7 @@ mod test {
|
||||
for format in [
|
||||
ResponseFormat::GreptimedbV1,
|
||||
ResponseFormat::InfluxdbV1,
|
||||
ResponseFormat::Csv,
|
||||
ResponseFormat::Csv(true, true),
|
||||
ResponseFormat::Table,
|
||||
ResponseFormat::Arrow,
|
||||
ResponseFormat::Json,
|
||||
@@ -1490,7 +1507,9 @@ mod test {
|
||||
let outputs = vec![Ok(Output::new_with_record_batches(recordbatches))];
|
||||
let json_resp = match format {
|
||||
ResponseFormat::Arrow => ArrowResponse::from_output(outputs, None).await,
|
||||
ResponseFormat::Csv => CsvResponse::from_output(outputs).await,
|
||||
ResponseFormat::Csv(with_names, with_types) => {
|
||||
CsvResponse::from_output(outputs, with_names, with_types).await
|
||||
}
|
||||
ResponseFormat::Table => TableResponse::from_output(outputs).await,
|
||||
ResponseFormat::GreptimedbV1 => GreptimedbV1Response::from_output(outputs).await,
|
||||
ResponseFormat::InfluxdbV1 => InfluxdbV1Response::from_output(outputs, None).await,
|
||||
@@ -1583,4 +1602,46 @@ mod test {
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_response_format_misc() {
|
||||
assert_eq!(ResponseFormat::default(), ResponseFormat::GreptimedbV1);
|
||||
assert_eq!(ResponseFormat::parse("arrow"), Some(ResponseFormat::Arrow));
|
||||
assert_eq!(
|
||||
ResponseFormat::parse("csv"),
|
||||
Some(ResponseFormat::Csv(false, false))
|
||||
);
|
||||
assert_eq!(
|
||||
ResponseFormat::parse("csvwithnames"),
|
||||
Some(ResponseFormat::Csv(true, false))
|
||||
);
|
||||
assert_eq!(
|
||||
ResponseFormat::parse("csvwithnamesandtypes"),
|
||||
Some(ResponseFormat::Csv(true, true))
|
||||
);
|
||||
assert_eq!(ResponseFormat::parse("table"), Some(ResponseFormat::Table));
|
||||
assert_eq!(
|
||||
ResponseFormat::parse("greptimedb_v1"),
|
||||
Some(ResponseFormat::GreptimedbV1)
|
||||
);
|
||||
assert_eq!(
|
||||
ResponseFormat::parse("influxdb_v1"),
|
||||
Some(ResponseFormat::InfluxdbV1)
|
||||
);
|
||||
assert_eq!(ResponseFormat::parse("json"), Some(ResponseFormat::Json));
|
||||
|
||||
// invalid formats
|
||||
assert_eq!(ResponseFormat::parse("invalid"), None);
|
||||
assert_eq!(ResponseFormat::parse(""), None);
|
||||
assert_eq!(ResponseFormat::parse("CSV"), None); // Case sensitive
|
||||
|
||||
// as str
|
||||
assert_eq!(ResponseFormat::Arrow.as_str(), "arrow");
|
||||
assert_eq!(ResponseFormat::Csv(false, false).as_str(), "csv");
|
||||
assert_eq!(ResponseFormat::Csv(true, true).as_str(), "csv");
|
||||
assert_eq!(ResponseFormat::Table.as_str(), "table");
|
||||
assert_eq!(ResponseFormat::GreptimedbV1.as_str(), "greptimedb_v1");
|
||||
assert_eq!(ResponseFormat::InfluxdbV1.as_str(), "influxdb_v1");
|
||||
assert_eq!(ResponseFormat::Json.as_str(), "json");
|
||||
}
|
||||
}
|
||||
|
||||
@@ -138,7 +138,9 @@ pub async fn sql(
|
||||
ResponseFormat::Arrow => {
|
||||
ArrowResponse::from_output(outputs, query_params.compression).await
|
||||
}
|
||||
ResponseFormat::Csv => CsvResponse::from_output(outputs).await,
|
||||
ResponseFormat::Csv(with_names, with_types) => {
|
||||
CsvResponse::from_output(outputs, with_names, with_types).await
|
||||
}
|
||||
ResponseFormat::Table => TableResponse::from_output(outputs).await,
|
||||
ResponseFormat::GreptimedbV1 => GreptimedbV1Response::from_output(outputs).await,
|
||||
ResponseFormat::InfluxdbV1 => InfluxdbV1Response::from_output(outputs, epoch).await,
|
||||
@@ -327,7 +329,9 @@ pub async fn promql(
|
||||
|
||||
match format {
|
||||
ResponseFormat::Arrow => ArrowResponse::from_output(outputs, compression).await,
|
||||
ResponseFormat::Csv => CsvResponse::from_output(outputs).await,
|
||||
ResponseFormat::Csv(with_names, with_types) => {
|
||||
CsvResponse::from_output(outputs, with_names, with_types).await
|
||||
}
|
||||
ResponseFormat::Table => TableResponse::from_output(outputs).await,
|
||||
ResponseFormat::GreptimedbV1 => GreptimedbV1Response::from_output(outputs).await,
|
||||
ResponseFormat::InfluxdbV1 => InfluxdbV1Response::from_output(outputs, epoch).await,
|
||||
|
||||
@@ -12,15 +12,14 @@
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
use std::collections::BTreeMap;
|
||||
use std::collections::{BTreeMap, VecDeque};
|
||||
use std::sync::Arc;
|
||||
use std::time::Instant;
|
||||
|
||||
use ahash::{HashMap, HashMapExt};
|
||||
use api::v1::value::ValueData;
|
||||
use api::v1::{
|
||||
ColumnDataType, ColumnDataTypeExtension, ColumnSchema, JsonTypeExtension, Row,
|
||||
RowInsertRequest, RowInsertRequests, Rows, SemanticType, Value as GreptimeValue,
|
||||
RowInsertRequest, Rows, SemanticType, Value as GreptimeValue,
|
||||
};
|
||||
use axum::extract::State;
|
||||
use axum::Extension;
|
||||
@@ -32,8 +31,10 @@ use common_telemetry::{error, warn};
|
||||
use headers::ContentType;
|
||||
use jsonb::Value;
|
||||
use lazy_static::lazy_static;
|
||||
use loki_proto::prost_types::Timestamp;
|
||||
use pipeline::unwrap_or_warn_continue;
|
||||
use loki_proto::logproto::LabelPairAdapter;
|
||||
use loki_proto::prost_types::Timestamp as LokiTimestamp;
|
||||
use pipeline::util::to_pipeline_version;
|
||||
use pipeline::{ContextReq, PipelineContext, PipelineDefinition, SchemaInfo};
|
||||
use prost::Message;
|
||||
use quoted_string::test_utils::TestSpec;
|
||||
use session::context::{Channel, QueryContext};
|
||||
@@ -41,22 +42,28 @@ use snafu::{ensure, OptionExt, ResultExt};
|
||||
|
||||
use crate::error::{
|
||||
DecodeOtlpRequestSnafu, InvalidLokiLabelsSnafu, InvalidLokiPayloadSnafu, ParseJsonSnafu,
|
||||
Result, UnsupportedContentTypeSnafu,
|
||||
PipelineSnafu, Result, UnsupportedContentTypeSnafu,
|
||||
};
|
||||
use crate::http::event::{LogState, JSON_CONTENT_TYPE, PB_CONTENT_TYPE};
|
||||
use crate::http::extractor::LogTableName;
|
||||
use crate::http::event::{LogState, PipelineIngestRequest, JSON_CONTENT_TYPE, PB_CONTENT_TYPE};
|
||||
use crate::http::extractor::{LogTableName, PipelineInfo};
|
||||
use crate::http::result::greptime_result_v1::GreptimedbV1Response;
|
||||
use crate::http::HttpResponse;
|
||||
use crate::metrics::{
|
||||
METRIC_FAILURE_VALUE, METRIC_LOKI_LOGS_INGESTION_COUNTER, METRIC_LOKI_LOGS_INGESTION_ELAPSED,
|
||||
METRIC_SUCCESS_VALUE,
|
||||
};
|
||||
use crate::pipeline::run_pipeline;
|
||||
use crate::prom_store;
|
||||
|
||||
const LOKI_TABLE_NAME: &str = "loki_logs";
|
||||
const LOKI_LINE_COLUMN: &str = "line";
|
||||
const LOKI_STRUCTURED_METADATA_COLUMN: &str = "structured_metadata";
|
||||
|
||||
const LOKI_LINE_COLUMN_NAME: &str = "loki_line";
|
||||
|
||||
const LOKI_PIPELINE_METADATA_PREFIX: &str = "loki_metadata_";
|
||||
const LOKI_PIPELINE_LABEL_PREFIX: &str = "loki_label_";
|
||||
|
||||
const STREAMS_KEY: &str = "streams";
|
||||
const LABEL_KEY: &str = "stream";
|
||||
const LINES_KEY: &str = "values";
|
||||
@@ -97,6 +104,7 @@ pub async fn loki_ingest(
|
||||
Extension(mut ctx): Extension<QueryContext>,
|
||||
TypedHeader(content_type): TypedHeader<ContentType>,
|
||||
LogTableName(table_name): LogTableName,
|
||||
pipeline_info: PipelineInfo,
|
||||
bytes: Bytes,
|
||||
) -> Result<HttpResponse> {
|
||||
ctx.set_channel(Channel::Loki);
|
||||
@@ -106,213 +114,507 @@ pub async fn loki_ingest(
|
||||
let db_str = db.as_str();
|
||||
let exec_timer = Instant::now();
|
||||
|
||||
// init schemas
|
||||
let mut schemas = LOKI_INIT_SCHEMAS.clone();
|
||||
|
||||
let mut rows = match content_type {
|
||||
x if x == *JSON_CONTENT_TYPE => handle_json_req(bytes, &mut schemas).await,
|
||||
x if x == *PB_CONTENT_TYPE => handle_pb_req(bytes, &mut schemas).await,
|
||||
_ => UnsupportedContentTypeSnafu { content_type }.fail(),
|
||||
}?;
|
||||
|
||||
// fill Null for missing values
|
||||
for row in rows.iter_mut() {
|
||||
row.resize(schemas.len(), GreptimeValue::default());
|
||||
}
|
||||
|
||||
let rows = Rows {
|
||||
rows: rows.into_iter().map(|values| Row { values }).collect(),
|
||||
schema: schemas,
|
||||
};
|
||||
let ins_req = RowInsertRequest {
|
||||
table_name,
|
||||
rows: Some(rows),
|
||||
};
|
||||
let ins_reqs = RowInsertRequests {
|
||||
inserts: vec![ins_req],
|
||||
};
|
||||
|
||||
let handler = log_state.log_handler;
|
||||
let output = handler.insert(ins_reqs, ctx).await;
|
||||
|
||||
if let Ok(Output {
|
||||
data: OutputData::AffectedRows(rows),
|
||||
meta: _,
|
||||
}) = &output
|
||||
{
|
||||
METRIC_LOKI_LOGS_INGESTION_COUNTER
|
||||
.with_label_values(&[db_str])
|
||||
.inc_by(*rows as u64);
|
||||
METRIC_LOKI_LOGS_INGESTION_ELAPSED
|
||||
.with_label_values(&[db_str, METRIC_SUCCESS_VALUE])
|
||||
.observe(exec_timer.elapsed().as_secs_f64());
|
||||
let ctx_req = if let Some(pipeline_name) = pipeline_info.pipeline_name {
|
||||
// go pipeline
|
||||
let version = to_pipeline_version(pipeline_info.pipeline_version.as_deref())
|
||||
.context(PipelineSnafu)?;
|
||||
let def =
|
||||
PipelineDefinition::from_name(&pipeline_name, version, None).context(PipelineSnafu)?;
|
||||
let pipeline_ctx =
|
||||
PipelineContext::new(&def, &pipeline_info.pipeline_params, Channel::Loki);
|
||||
|
||||
let v = extract_item::<LokiPipeline>(content_type, bytes)?
|
||||
.map(|i| i.map)
|
||||
.collect::<Vec<_>>();
|
||||
|
||||
let req = PipelineIngestRequest {
|
||||
table: table_name,
|
||||
values: v,
|
||||
};
|
||||
|
||||
run_pipeline(&handler, &pipeline_ctx, req, &ctx, true).await?
|
||||
} else {
|
||||
METRIC_LOKI_LOGS_INGESTION_ELAPSED
|
||||
.with_label_values(&[db_str, METRIC_FAILURE_VALUE])
|
||||
.observe(exec_timer.elapsed().as_secs_f64());
|
||||
// init schemas
|
||||
let mut schema_info = SchemaInfo::from_schema_list(LOKI_INIT_SCHEMAS.clone());
|
||||
let mut rows = Vec::with_capacity(256);
|
||||
for loki_row in extract_item::<LokiRawItem>(content_type, bytes)? {
|
||||
let mut row = init_row(
|
||||
schema_info.schema.len(),
|
||||
loki_row.ts,
|
||||
loki_row.line,
|
||||
loki_row.structured_metadata,
|
||||
);
|
||||
process_labels(&mut schema_info, &mut row, loki_row.labels);
|
||||
rows.push(row);
|
||||
}
|
||||
|
||||
let schemas = schema_info.schema;
|
||||
// fill Null for missing values
|
||||
for row in rows.iter_mut() {
|
||||
row.resize(schemas.len(), GreptimeValue::default());
|
||||
}
|
||||
let rows = Rows {
|
||||
rows: rows.into_iter().map(|values| Row { values }).collect(),
|
||||
schema: schemas,
|
||||
};
|
||||
let ins_req = RowInsertRequest {
|
||||
table_name,
|
||||
rows: Some(rows),
|
||||
};
|
||||
|
||||
ContextReq::default_opt_with_reqs(vec![ins_req])
|
||||
};
|
||||
|
||||
let mut outputs = Vec::with_capacity(ctx_req.map_len());
|
||||
for (temp_ctx, req) in ctx_req.as_req_iter(ctx) {
|
||||
let output = handler.insert(req, temp_ctx).await;
|
||||
|
||||
if let Ok(Output {
|
||||
data: OutputData::AffectedRows(rows),
|
||||
meta: _,
|
||||
}) = &output
|
||||
{
|
||||
METRIC_LOKI_LOGS_INGESTION_COUNTER
|
||||
.with_label_values(&[db_str])
|
||||
.inc_by(*rows as u64);
|
||||
METRIC_LOKI_LOGS_INGESTION_ELAPSED
|
||||
.with_label_values(&[db_str, METRIC_SUCCESS_VALUE])
|
||||
.observe(exec_timer.elapsed().as_secs_f64());
|
||||
} else {
|
||||
METRIC_LOKI_LOGS_INGESTION_ELAPSED
|
||||
.with_label_values(&[db_str, METRIC_FAILURE_VALUE])
|
||||
.observe(exec_timer.elapsed().as_secs_f64());
|
||||
}
|
||||
outputs.push(output);
|
||||
}
|
||||
|
||||
let response = GreptimedbV1Response::from_output(vec![output])
|
||||
let response = GreptimedbV1Response::from_output(outputs)
|
||||
.await
|
||||
.with_execution_time(exec_timer.elapsed().as_millis() as u64);
|
||||
Ok(response)
|
||||
}
|
||||
|
||||
async fn handle_json_req(
|
||||
bytes: Bytes,
|
||||
schemas: &mut Vec<ColumnSchema>,
|
||||
) -> Result<Vec<Vec<GreptimeValue>>> {
|
||||
let mut column_indexer: HashMap<String, u16> = HashMap::new();
|
||||
column_indexer.insert(GREPTIME_TIMESTAMP.to_string(), 0);
|
||||
column_indexer.insert(LOKI_LINE_COLUMN.to_string(), 1);
|
||||
|
||||
let payload: serde_json::Value =
|
||||
serde_json::from_slice(bytes.as_ref()).context(ParseJsonSnafu)?;
|
||||
|
||||
let streams = payload
|
||||
.get(STREAMS_KEY)
|
||||
.context(InvalidLokiPayloadSnafu {
|
||||
msg: "missing streams",
|
||||
})?
|
||||
.as_array()
|
||||
.context(InvalidLokiPayloadSnafu {
|
||||
msg: "streams is not an array",
|
||||
})?;
|
||||
|
||||
let mut rows = Vec::with_capacity(1000);
|
||||
|
||||
for (stream_index, stream) in streams.iter().enumerate() {
|
||||
// parse lines first
|
||||
// do not use `?` in case there are multiple streams
|
||||
let lines = unwrap_or_warn_continue!(
|
||||
stream.get(LINES_KEY),
|
||||
"missing values on stream {}",
|
||||
stream_index
|
||||
);
|
||||
let lines = unwrap_or_warn_continue!(
|
||||
lines.as_array(),
|
||||
"values is not an array on stream {}",
|
||||
stream_index
|
||||
);
|
||||
|
||||
// get labels
|
||||
let labels = stream
|
||||
.get(LABEL_KEY)
|
||||
.and_then(|label| label.as_object())
|
||||
.map(|l| {
|
||||
l.iter()
|
||||
.filter_map(|(k, v)| v.as_str().map(|v| (k.clone(), v.to_string())))
|
||||
.collect::<BTreeMap<String, String>>()
|
||||
});
|
||||
|
||||
// process each line
|
||||
for (line_index, line) in lines.iter().enumerate() {
|
||||
let line = unwrap_or_warn_continue!(
|
||||
line.as_array(),
|
||||
"missing line on stream {} index {}",
|
||||
stream_index,
|
||||
line_index
|
||||
);
|
||||
if line.len() < 2 {
|
||||
warn!(
|
||||
"line on stream {} index {} is too short",
|
||||
stream_index, line_index
|
||||
);
|
||||
continue;
|
||||
}
|
||||
// get ts
|
||||
let ts = unwrap_or_warn_continue!(
|
||||
line.first()
|
||||
.and_then(|ts| ts.as_str())
|
||||
.and_then(|ts| ts.parse::<i64>().ok()),
|
||||
"missing or invalid timestamp on stream {} index {}",
|
||||
stream_index,
|
||||
line_index
|
||||
);
|
||||
// get line
|
||||
let line_text = unwrap_or_warn_continue!(
|
||||
line.get(1)
|
||||
.and_then(|line| line.as_str())
|
||||
.map(|line| line.to_string()),
|
||||
"missing or invalid line on stream {} index {}",
|
||||
stream_index,
|
||||
line_index
|
||||
);
|
||||
|
||||
let structured_metadata = match line.get(2) {
|
||||
Some(sdata) if sdata.is_object() => sdata
|
||||
.as_object()
|
||||
.unwrap()
|
||||
.iter()
|
||||
.filter_map(|(k, v)| v.as_str().map(|s| (k.clone(), Value::String(s.into()))))
|
||||
.collect(),
|
||||
_ => BTreeMap::new(),
|
||||
};
|
||||
let structured_metadata = Value::Object(structured_metadata);
|
||||
|
||||
let mut row = init_row(schemas.len(), ts, line_text, structured_metadata);
|
||||
|
||||
process_labels(&mut column_indexer, schemas, &mut row, labels.as_ref());
|
||||
|
||||
rows.push(row);
|
||||
}
|
||||
}
|
||||
|
||||
Ok(rows)
|
||||
/// This is the holder of the loki lines parsed from json or protobuf.
|
||||
/// The generic here is either [serde_json::Value] or [Vec<LabelPairAdapter>].
|
||||
/// Depending on the target destination, this can be converted to [LokiRawItem] or [LokiPipeline].
|
||||
pub struct LokiMiddleItem<T> {
|
||||
pub ts: i64,
|
||||
pub line: String,
|
||||
pub structured_metadata: Option<T>,
|
||||
pub labels: Option<BTreeMap<String, String>>,
|
||||
}
|
||||
|
||||
async fn handle_pb_req(
|
||||
bytes: Bytes,
|
||||
schemas: &mut Vec<ColumnSchema>,
|
||||
) -> Result<Vec<Vec<GreptimeValue>>> {
|
||||
let decompressed = prom_store::snappy_decompress(&bytes).unwrap();
|
||||
let req = loki_proto::logproto::PushRequest::decode(&decompressed[..])
|
||||
.context(DecodeOtlpRequestSnafu)?;
|
||||
/// This is the line item for the Loki raw ingestion.
|
||||
/// We'll persist the line in its whole, set labels into tags,
|
||||
/// and structured metadata into a big JSON.
|
||||
pub struct LokiRawItem {
|
||||
pub ts: i64,
|
||||
pub line: String,
|
||||
pub structured_metadata: Vec<u8>,
|
||||
pub labels: Option<BTreeMap<String, String>>,
|
||||
}
|
||||
|
||||
let mut column_indexer: HashMap<String, u16> = HashMap::new();
|
||||
column_indexer.insert(GREPTIME_TIMESTAMP.to_string(), 0);
|
||||
column_indexer.insert(LOKI_LINE_COLUMN.to_string(), 1);
|
||||
/// This is the line item prepared for the pipeline engine.
|
||||
pub struct LokiPipeline {
|
||||
pub map: pipeline::Value,
|
||||
}
|
||||
|
||||
let cnt = req.streams.iter().map(|s| s.entries.len()).sum::<usize>();
|
||||
let mut rows = Vec::with_capacity(cnt);
|
||||
/// This is the flow of the Loki ingestion.
|
||||
/// +--------+
|
||||
/// | bytes |
|
||||
/// +--------+
|
||||
/// |
|
||||
/// +----------------------+----------------------+
|
||||
/// | | |
|
||||
/// | JSON content type | PB content type |
|
||||
/// +----------------------+----------------------+
|
||||
/// | | |
|
||||
/// | JsonStreamItem | PbStreamItem |
|
||||
/// | stream: serde_json | stream: adapter |
|
||||
/// +----------------------+----------------------+
|
||||
/// | | |
|
||||
/// | MiddleItem<serde_json> | MiddleItem<entry> |
|
||||
/// +----------------------+----------------------+
|
||||
/// \ /
|
||||
/// \ /
|
||||
/// \ /
|
||||
/// +----------------------+
|
||||
/// | MiddleItem<T> |
|
||||
/// +----------------------+
|
||||
/// |
|
||||
/// +----------------+----------------+
|
||||
/// | |
|
||||
/// +------------------+ +---------------------+
|
||||
/// | LokiRawItem | | LokiPipelineItem |
|
||||
/// +------------------+ +---------------------+
|
||||
/// | |
|
||||
/// +------------------+ +---------------------+
|
||||
/// | Loki ingest | | run_pipeline |
|
||||
/// +------------------+ +---------------------+
|
||||
fn extract_item<T>(content_type: ContentType, bytes: Bytes) -> Result<Box<dyn Iterator<Item = T>>>
|
||||
where
|
||||
LokiMiddleItem<serde_json::Value>: Into<T>,
|
||||
LokiMiddleItem<Vec<LabelPairAdapter>>: Into<T>,
|
||||
{
|
||||
match content_type {
|
||||
x if x == *JSON_CONTENT_TYPE => Ok(Box::new(
|
||||
LokiJsonParser::from_bytes(bytes)?.flat_map(|item| item.into_iter().map(|i| i.into())),
|
||||
)),
|
||||
x if x == *PB_CONTENT_TYPE => Ok(Box::new(
|
||||
LokiPbParser::from_bytes(bytes)?.flat_map(|item| item.into_iter().map(|i| i.into())),
|
||||
)),
|
||||
_ => UnsupportedContentTypeSnafu { content_type }.fail(),
|
||||
}
|
||||
}
|
||||
|
||||
struct LokiJsonParser {
|
||||
pub streams: VecDeque<serde_json::Value>,
|
||||
}
|
||||
|
||||
impl LokiJsonParser {
|
||||
pub fn from_bytes(bytes: Bytes) -> Result<Self> {
|
||||
let payload: serde_json::Value =
|
||||
serde_json::from_slice(bytes.as_ref()).context(ParseJsonSnafu)?;
|
||||
|
||||
let serde_json::Value::Object(mut map) = payload else {
|
||||
return InvalidLokiPayloadSnafu {
|
||||
msg: "payload is not an object",
|
||||
}
|
||||
.fail();
|
||||
};
|
||||
|
||||
let streams = map.remove(STREAMS_KEY).context(InvalidLokiPayloadSnafu {
|
||||
msg: "missing streams",
|
||||
})?;
|
||||
|
||||
let serde_json::Value::Array(streams) = streams else {
|
||||
return InvalidLokiPayloadSnafu {
|
||||
msg: "streams is not an array",
|
||||
}
|
||||
.fail();
|
||||
};
|
||||
|
||||
Ok(Self {
|
||||
streams: streams.into(),
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
impl Iterator for LokiJsonParser {
|
||||
type Item = JsonStreamItem;
|
||||
|
||||
fn next(&mut self) -> Option<Self::Item> {
|
||||
while let Some(stream) = self.streams.pop_front() {
|
||||
// get lines from the map
|
||||
let serde_json::Value::Object(mut map) = stream else {
|
||||
warn!("stream is not an object, {:?}", stream);
|
||||
continue;
|
||||
};
|
||||
let Some(lines) = map.remove(LINES_KEY) else {
|
||||
warn!("missing lines on stream, {:?}", map);
|
||||
continue;
|
||||
};
|
||||
let serde_json::Value::Array(lines) = lines else {
|
||||
warn!("lines is not an array, {:?}", lines);
|
||||
continue;
|
||||
};
|
||||
|
||||
// get labels
|
||||
let labels = map
|
||||
.remove(LABEL_KEY)
|
||||
.and_then(|m| match m {
|
||||
serde_json::Value::Object(labels) => Some(labels),
|
||||
_ => None,
|
||||
})
|
||||
.map(|m| {
|
||||
m.into_iter()
|
||||
.filter_map(|(k, v)| match v {
|
||||
serde_json::Value::String(v) => Some((k, v)),
|
||||
_ => None,
|
||||
})
|
||||
.collect::<BTreeMap<String, String>>()
|
||||
});
|
||||
|
||||
return Some(JsonStreamItem {
|
||||
lines: lines.into(),
|
||||
labels,
|
||||
});
|
||||
}
|
||||
None
|
||||
}
|
||||
}
|
||||
|
||||
struct JsonStreamItem {
|
||||
pub lines: VecDeque<serde_json::Value>,
|
||||
pub labels: Option<BTreeMap<String, String>>,
|
||||
}
|
||||
|
||||
impl Iterator for JsonStreamItem {
|
||||
type Item = LokiMiddleItem<serde_json::Value>;
|
||||
|
||||
fn next(&mut self) -> Option<Self::Item> {
|
||||
while let Some(line) = self.lines.pop_front() {
|
||||
let serde_json::Value::Array(line) = line else {
|
||||
warn!("line is not an array, {:?}", line);
|
||||
continue;
|
||||
};
|
||||
if line.len() < 2 {
|
||||
warn!("line is too short, {:?}", line);
|
||||
continue;
|
||||
}
|
||||
let mut line: VecDeque<serde_json::Value> = line.into();
|
||||
|
||||
// get ts
|
||||
let ts = line.pop_front().and_then(|ts| match ts {
|
||||
serde_json::Value::String(ts) => ts.parse::<i64>().ok(),
|
||||
_ => {
|
||||
warn!("missing or invalid timestamp, {:?}", ts);
|
||||
None
|
||||
}
|
||||
});
|
||||
let Some(ts) = ts else {
|
||||
continue;
|
||||
};
|
||||
|
||||
let line_text = line.pop_front().and_then(|l| match l {
|
||||
serde_json::Value::String(l) => Some(l),
|
||||
_ => {
|
||||
warn!("missing or invalid line, {:?}", l);
|
||||
None
|
||||
}
|
||||
});
|
||||
let Some(line_text) = line_text else {
|
||||
continue;
|
||||
};
|
||||
|
||||
let structured_metadata = line.pop_front();
|
||||
|
||||
return Some(LokiMiddleItem {
|
||||
ts,
|
||||
line: line_text,
|
||||
structured_metadata,
|
||||
labels: self.labels.clone(),
|
||||
});
|
||||
}
|
||||
None
|
||||
}
|
||||
}
|
||||
|
||||
impl From<LokiMiddleItem<serde_json::Value>> for LokiRawItem {
|
||||
fn from(val: LokiMiddleItem<serde_json::Value>) -> Self {
|
||||
let LokiMiddleItem {
|
||||
ts,
|
||||
line,
|
||||
structured_metadata,
|
||||
labels,
|
||||
} = val;
|
||||
|
||||
let structured_metadata = structured_metadata
|
||||
.and_then(|m| match m {
|
||||
serde_json::Value::Object(m) => Some(m),
|
||||
_ => None,
|
||||
})
|
||||
.map(|m| {
|
||||
m.into_iter()
|
||||
.filter_map(|(k, v)| match v {
|
||||
serde_json::Value::String(v) => Some((k, Value::String(v.into()))),
|
||||
_ => None,
|
||||
})
|
||||
.collect::<BTreeMap<String, Value>>()
|
||||
})
|
||||
.unwrap_or_default();
|
||||
let structured_metadata = Value::Object(structured_metadata).to_vec();
|
||||
|
||||
LokiRawItem {
|
||||
ts,
|
||||
line,
|
||||
structured_metadata,
|
||||
labels,
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
impl From<LokiMiddleItem<serde_json::Value>> for LokiPipeline {
|
||||
fn from(value: LokiMiddleItem<serde_json::Value>) -> Self {
|
||||
let LokiMiddleItem {
|
||||
ts,
|
||||
line,
|
||||
structured_metadata,
|
||||
labels,
|
||||
} = value;
|
||||
|
||||
let mut map = BTreeMap::new();
|
||||
map.insert(
|
||||
GREPTIME_TIMESTAMP.to_string(),
|
||||
pipeline::Value::Timestamp(pipeline::Timestamp::Nanosecond(ts)),
|
||||
);
|
||||
map.insert(
|
||||
LOKI_LINE_COLUMN_NAME.to_string(),
|
||||
pipeline::Value::String(line),
|
||||
);
|
||||
|
||||
if let Some(serde_json::Value::Object(m)) = structured_metadata {
|
||||
for (k, v) in m {
|
||||
match pipeline::Value::try_from(v) {
|
||||
Ok(v) => {
|
||||
map.insert(format!("{}{}", LOKI_PIPELINE_METADATA_PREFIX, k), v);
|
||||
}
|
||||
Err(e) => {
|
||||
warn!("not a valid value, {:?}", e);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
if let Some(v) = labels {
|
||||
v.into_iter().for_each(|(k, v)| {
|
||||
map.insert(
|
||||
format!("{}{}", LOKI_PIPELINE_LABEL_PREFIX, k),
|
||||
pipeline::Value::String(v),
|
||||
);
|
||||
});
|
||||
}
|
||||
|
||||
LokiPipeline {
|
||||
map: pipeline::Value::Map(pipeline::Map::from(map)),
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
pub struct LokiPbParser {
|
||||
pub streams: VecDeque<loki_proto::logproto::StreamAdapter>,
|
||||
}
|
||||
|
||||
impl LokiPbParser {
|
||||
pub fn from_bytes(bytes: Bytes) -> Result<Self> {
|
||||
let decompressed = prom_store::snappy_decompress(&bytes).unwrap();
|
||||
let req = loki_proto::logproto::PushRequest::decode(&decompressed[..])
|
||||
.context(DecodeOtlpRequestSnafu)?;
|
||||
|
||||
Ok(Self {
|
||||
streams: req.streams.into(),
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
impl Iterator for LokiPbParser {
|
||||
type Item = PbStreamItem;
|
||||
|
||||
fn next(&mut self) -> Option<Self::Item> {
|
||||
let stream = self.streams.pop_front()?;
|
||||
|
||||
for stream in req.streams {
|
||||
let labels = parse_loki_labels(&stream.labels)
|
||||
.inspect_err(|e| {
|
||||
error!(e; "failed to parse loki labels");
|
||||
error!(e; "failed to parse loki labels, {:?}", stream.labels);
|
||||
})
|
||||
.ok();
|
||||
|
||||
// process entries
|
||||
for entry in stream.entries {
|
||||
Some(PbStreamItem {
|
||||
entries: stream.entries.into(),
|
||||
labels,
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
pub struct PbStreamItem {
|
||||
pub entries: VecDeque<loki_proto::logproto::EntryAdapter>,
|
||||
pub labels: Option<BTreeMap<String, String>>,
|
||||
}
|
||||
|
||||
impl Iterator for PbStreamItem {
|
||||
type Item = LokiMiddleItem<Vec<LabelPairAdapter>>;
|
||||
|
||||
fn next(&mut self) -> Option<Self::Item> {
|
||||
while let Some(entry) = self.entries.pop_front() {
|
||||
let ts = if let Some(ts) = entry.timestamp {
|
||||
ts
|
||||
} else {
|
||||
warn!("missing timestamp, {:?}", entry);
|
||||
continue;
|
||||
};
|
||||
let line = entry.line;
|
||||
|
||||
let structured_metadata = entry
|
||||
.structured_metadata
|
||||
.into_iter()
|
||||
.map(|d| (d.name, Value::String(d.value.into())))
|
||||
.collect::<BTreeMap<String, Value>>();
|
||||
let structured_metadata = Value::Object(structured_metadata);
|
||||
let structured_metadata = entry.structured_metadata;
|
||||
|
||||
let mut row = init_row(
|
||||
schemas.len(),
|
||||
prost_ts_to_nano(&ts),
|
||||
return Some(LokiMiddleItem {
|
||||
ts: prost_ts_to_nano(&ts),
|
||||
line,
|
||||
structured_metadata,
|
||||
);
|
||||
structured_metadata: Some(structured_metadata),
|
||||
labels: self.labels.clone(),
|
||||
});
|
||||
}
|
||||
None
|
||||
}
|
||||
}
|
||||
|
||||
process_labels(&mut column_indexer, schemas, &mut row, labels.as_ref());
|
||||
impl From<LokiMiddleItem<Vec<LabelPairAdapter>>> for LokiRawItem {
|
||||
fn from(val: LokiMiddleItem<Vec<LabelPairAdapter>>) -> Self {
|
||||
let LokiMiddleItem {
|
||||
ts,
|
||||
line,
|
||||
structured_metadata,
|
||||
labels,
|
||||
} = val;
|
||||
|
||||
rows.push(row);
|
||||
let structured_metadata = structured_metadata
|
||||
.unwrap_or_default()
|
||||
.into_iter()
|
||||
.map(|d| (d.name, Value::String(d.value.into())))
|
||||
.collect::<BTreeMap<String, Value>>();
|
||||
let structured_metadata = Value::Object(structured_metadata).to_vec();
|
||||
|
||||
LokiRawItem {
|
||||
ts,
|
||||
line,
|
||||
structured_metadata,
|
||||
labels,
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
Ok(rows)
|
||||
impl From<LokiMiddleItem<Vec<LabelPairAdapter>>> for LokiPipeline {
|
||||
fn from(value: LokiMiddleItem<Vec<LabelPairAdapter>>) -> Self {
|
||||
let LokiMiddleItem {
|
||||
ts,
|
||||
line,
|
||||
structured_metadata,
|
||||
labels,
|
||||
} = value;
|
||||
|
||||
let mut map = BTreeMap::new();
|
||||
map.insert(
|
||||
GREPTIME_TIMESTAMP.to_string(),
|
||||
pipeline::Value::Timestamp(pipeline::Timestamp::Nanosecond(ts)),
|
||||
);
|
||||
map.insert(
|
||||
LOKI_LINE_COLUMN_NAME.to_string(),
|
||||
pipeline::Value::String(line),
|
||||
);
|
||||
|
||||
structured_metadata
|
||||
.unwrap_or_default()
|
||||
.into_iter()
|
||||
.for_each(|d| {
|
||||
map.insert(
|
||||
format!("{}{}", LOKI_PIPELINE_METADATA_PREFIX, d.name),
|
||||
pipeline::Value::String(d.value),
|
||||
);
|
||||
});
|
||||
|
||||
if let Some(v) = labels {
|
||||
v.into_iter().for_each(|(k, v)| {
|
||||
map.insert(
|
||||
format!("{}{}", LOKI_PIPELINE_LABEL_PREFIX, k),
|
||||
pipeline::Value::String(v),
|
||||
);
|
||||
});
|
||||
}
|
||||
|
||||
LokiPipeline {
|
||||
map: pipeline::Value::Map(pipeline::Map::from(map)),
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/// since we're hand-parsing the labels, if any error is encountered, we'll just skip the label
|
||||
@@ -391,7 +693,7 @@ pub fn parse_loki_labels(labels: &str) -> Result<BTreeMap<String, String>> {
|
||||
}
|
||||
|
||||
#[inline]
|
||||
fn prost_ts_to_nano(ts: &Timestamp) -> i64 {
|
||||
fn prost_ts_to_nano(ts: &LokiTimestamp) -> i64 {
|
||||
ts.seconds * 1_000_000_000 + ts.nanos as i64
|
||||
}
|
||||
|
||||
@@ -399,7 +701,7 @@ fn init_row(
|
||||
schema_len: usize,
|
||||
ts: i64,
|
||||
line: String,
|
||||
structured_metadata: Value,
|
||||
structured_metadata: Vec<u8>,
|
||||
) -> Vec<GreptimeValue> {
|
||||
// create and init row
|
||||
let mut row = Vec::with_capacity(schema_len);
|
||||
@@ -411,7 +713,7 @@ fn init_row(
|
||||
value_data: Some(ValueData::StringValue(line)),
|
||||
});
|
||||
row.push(GreptimeValue {
|
||||
value_data: Some(ValueData::BinaryValue(structured_metadata.to_vec())),
|
||||
value_data: Some(ValueData::BinaryValue(structured_metadata)),
|
||||
});
|
||||
for _ in 0..(schema_len - 3) {
|
||||
row.push(GreptimeValue { value_data: None });
|
||||
@@ -420,22 +722,24 @@ fn init_row(
|
||||
}
|
||||
|
||||
fn process_labels(
|
||||
column_indexer: &mut HashMap<String, u16>,
|
||||
schemas: &mut Vec<ColumnSchema>,
|
||||
schema_info: &mut SchemaInfo,
|
||||
row: &mut Vec<GreptimeValue>,
|
||||
labels: Option<&BTreeMap<String, String>>,
|
||||
labels: Option<BTreeMap<String, String>>,
|
||||
) {
|
||||
let Some(labels) = labels else {
|
||||
return;
|
||||
};
|
||||
|
||||
let column_indexer = &mut schema_info.index;
|
||||
let schemas = &mut schema_info.schema;
|
||||
|
||||
// insert labels
|
||||
for (k, v) in labels {
|
||||
if let Some(index) = column_indexer.get(k) {
|
||||
if let Some(index) = column_indexer.get(&k) {
|
||||
// exist in schema
|
||||
// insert value using index
|
||||
row[*index as usize] = GreptimeValue {
|
||||
value_data: Some(ValueData::StringValue(v.clone())),
|
||||
row[*index] = GreptimeValue {
|
||||
value_data: Some(ValueData::StringValue(v)),
|
||||
};
|
||||
} else {
|
||||
// not exist
|
||||
@@ -447,10 +751,10 @@ fn process_labels(
|
||||
datatype_extension: None,
|
||||
options: None,
|
||||
});
|
||||
column_indexer.insert(k.clone(), (schemas.len() - 1) as u16);
|
||||
column_indexer.insert(k, schemas.len() - 1);
|
||||
|
||||
row.push(GreptimeValue {
|
||||
value_data: Some(ValueData::StringValue(v.clone())),
|
||||
value_data: Some(ValueData::StringValue(v)),
|
||||
});
|
||||
}
|
||||
}
|
||||
|
||||
@@ -18,9 +18,9 @@ use common_error::status_code::StatusCode;
|
||||
use common_query::Output;
|
||||
use mime_guess::mime;
|
||||
use serde::{Deserialize, Serialize};
|
||||
use serde_json::Value as JsonValue;
|
||||
|
||||
use crate::http::header::{GREPTIME_DB_HEADER_EXECUTION_TIME, GREPTIME_DB_HEADER_FORMAT};
|
||||
// use super::process_with_limit;
|
||||
use crate::http::result::error_result::ErrorResponse;
|
||||
use crate::http::{handler, process_with_limit, GreptimeQueryOutput, HttpResponse, ResponseFormat};
|
||||
|
||||
@@ -28,10 +28,16 @@ use crate::http::{handler, process_with_limit, GreptimeQueryOutput, HttpResponse
|
||||
pub struct CsvResponse {
|
||||
output: Vec<GreptimeQueryOutput>,
|
||||
execution_time_ms: u64,
|
||||
with_names: bool,
|
||||
with_types: bool,
|
||||
}
|
||||
|
||||
impl CsvResponse {
|
||||
pub async fn from_output(outputs: Vec<crate::error::Result<Output>>) -> HttpResponse {
|
||||
pub async fn from_output(
|
||||
outputs: Vec<crate::error::Result<Output>>,
|
||||
with_names: bool,
|
||||
with_types: bool,
|
||||
) -> HttpResponse {
|
||||
match handler::from_output(outputs).await {
|
||||
Err(err) => HttpResponse::Error(err),
|
||||
Ok((output, _)) => {
|
||||
@@ -41,10 +47,14 @@ impl CsvResponse {
|
||||
"cannot output multi-statements result in csv format".to_string(),
|
||||
))
|
||||
} else {
|
||||
HttpResponse::Csv(CsvResponse {
|
||||
let csv_resp = CsvResponse {
|
||||
output,
|
||||
execution_time_ms: 0,
|
||||
})
|
||||
with_names: false,
|
||||
with_types: false,
|
||||
};
|
||||
|
||||
HttpResponse::Csv(csv_resp.with_names(with_names).with_types(with_types))
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -67,6 +77,21 @@ impl CsvResponse {
|
||||
self.output = process_with_limit(self.output, limit);
|
||||
self
|
||||
}
|
||||
|
||||
pub fn with_names(mut self, with_names: bool) -> Self {
|
||||
self.with_names = with_names;
|
||||
self
|
||||
}
|
||||
|
||||
pub fn with_types(mut self, with_types: bool) -> Self {
|
||||
self.with_types = with_types;
|
||||
|
||||
// If `with_type` is true, than always set `with_names` to be true.
|
||||
if with_types {
|
||||
self.with_names = true;
|
||||
}
|
||||
self
|
||||
}
|
||||
}
|
||||
|
||||
macro_rules! http_try {
|
||||
@@ -100,11 +125,50 @@ impl IntoResponse for CsvResponse {
|
||||
format!("{n}\n")
|
||||
}
|
||||
Some(GreptimeQueryOutput::Records(records)) => {
|
||||
let mut wtr = csv::Writer::from_writer(Vec::new());
|
||||
let mut wtr = csv::WriterBuilder::new()
|
||||
.terminator(csv::Terminator::CRLF) // RFC 4180
|
||||
.from_writer(Vec::new());
|
||||
|
||||
if self.with_names {
|
||||
let names = records
|
||||
.schema
|
||||
.column_schemas
|
||||
.iter()
|
||||
.map(|c| &c.name)
|
||||
.collect::<Vec<_>>();
|
||||
http_try!(wtr.serialize(names));
|
||||
}
|
||||
|
||||
if self.with_types {
|
||||
let types = records
|
||||
.schema
|
||||
.column_schemas
|
||||
.iter()
|
||||
.map(|c| &c.data_type)
|
||||
.collect::<Vec<_>>();
|
||||
http_try!(wtr.serialize(types));
|
||||
}
|
||||
|
||||
for row in records.rows {
|
||||
let row = row
|
||||
.into_iter()
|
||||
.map(|value| {
|
||||
match value {
|
||||
// Cast array and object to string
|
||||
JsonValue::Array(a) => {
|
||||
JsonValue::String(serde_json::to_string(&a).unwrap_or_default())
|
||||
}
|
||||
JsonValue::Object(o) => {
|
||||
JsonValue::String(serde_json::to_string(&o).unwrap_or_default())
|
||||
}
|
||||
v => v,
|
||||
}
|
||||
})
|
||||
.collect::<Vec<_>>();
|
||||
|
||||
http_try!(wtr.serialize(row));
|
||||
}
|
||||
|
||||
http_try!(wtr.flush());
|
||||
|
||||
let bytes = http_try!(wtr.into_inner());
|
||||
@@ -122,7 +186,9 @@ impl IntoResponse for CsvResponse {
|
||||
.into_response();
|
||||
resp.headers_mut().insert(
|
||||
&GREPTIME_DB_HEADER_FORMAT,
|
||||
HeaderValue::from_static(ResponseFormat::Csv.as_str()),
|
||||
HeaderValue::from_static(
|
||||
ResponseFormat::Csv(self.with_names, self.with_types).as_str(),
|
||||
),
|
||||
);
|
||||
resp.headers_mut().insert(
|
||||
&GREPTIME_DB_HEADER_EXECUTION_TIME,
|
||||
@@ -131,3 +197,97 @@ impl IntoResponse for CsvResponse {
|
||||
resp
|
||||
}
|
||||
}
|
||||
|
||||
#[cfg(test)]
|
||||
mod tests {
|
||||
use std::sync::Arc;
|
||||
|
||||
use common_query::Output;
|
||||
use common_recordbatch::{RecordBatch, RecordBatches};
|
||||
use datatypes::prelude::{ConcreteDataType, ScalarVector};
|
||||
use datatypes::schema::{ColumnSchema, Schema};
|
||||
use datatypes::vectors::{BinaryVector, Float32Vector, StringVector, UInt32Vector, VectorRef};
|
||||
|
||||
use super::*;
|
||||
#[tokio::test]
|
||||
async fn test_csv_response_with_names_and_types() {
|
||||
let (schema, columns) = create_test_data();
|
||||
|
||||
let data = r#"1,,-1000.1400146484375,"{""a"":{""b"":2},""b"":2,""c"":3}"
|
||||
2,hello,1.9900000095367432,"{""a"":4,""b"":{""c"":6},""c"":6}""#
|
||||
.replace("\n", "\r\n");
|
||||
|
||||
// Test with_names=true, with_types=true
|
||||
{
|
||||
let body = get_csv_body(&schema, &columns, true, true).await;
|
||||
assert!(body.starts_with("col1,col2,col3,col4\r\nUInt32,String,Float32,Json\r\n"));
|
||||
assert!(body.contains(&data));
|
||||
}
|
||||
|
||||
// Test with_names=true, with_types=false
|
||||
{
|
||||
let body = get_csv_body(&schema, &columns, true, false).await;
|
||||
assert!(body.starts_with("col1,col2,col3,col4\r\n"));
|
||||
assert!(!body.contains("UInt32,String,Float32,Json"));
|
||||
assert!(body.contains(&data));
|
||||
}
|
||||
|
||||
// Test with_names=false, with_types=false
|
||||
{
|
||||
let body = get_csv_body(&schema, &columns, false, false).await;
|
||||
assert!(!body.starts_with("col1,col2,col3,col4"));
|
||||
assert!(!body.contains("UInt32,String,Float32,Json"));
|
||||
assert!(body.contains(&data));
|
||||
}
|
||||
}
|
||||
|
||||
fn create_test_data() -> (Arc<Schema>, Vec<VectorRef>) {
|
||||
let column_schemas = vec![
|
||||
ColumnSchema::new("col1", ConcreteDataType::uint32_datatype(), false),
|
||||
ColumnSchema::new("col2", ConcreteDataType::string_datatype(), true),
|
||||
ColumnSchema::new("col3", ConcreteDataType::float32_datatype(), true),
|
||||
ColumnSchema::new("col4", ConcreteDataType::json_datatype(), true),
|
||||
];
|
||||
let schema = Arc::new(Schema::new(column_schemas));
|
||||
|
||||
let json_strings = [
|
||||
r#"{"a": {"b": 2}, "b": 2, "c": 3}"#,
|
||||
r#"{"a": 4, "b": {"c": 6}, "c": 6}"#,
|
||||
];
|
||||
|
||||
let jsonbs = json_strings
|
||||
.iter()
|
||||
.map(|s| {
|
||||
let value = jsonb::parse_value(s.as_bytes()).unwrap();
|
||||
value.to_vec()
|
||||
})
|
||||
.collect::<Vec<_>>();
|
||||
|
||||
let columns: Vec<VectorRef> = vec![
|
||||
Arc::new(UInt32Vector::from_slice(vec![1, 2])),
|
||||
Arc::new(StringVector::from(vec![None, Some("hello")])),
|
||||
Arc::new(Float32Vector::from_slice(vec![-1000.14, 1.99])),
|
||||
Arc::new(BinaryVector::from_vec(jsonbs)),
|
||||
];
|
||||
|
||||
(schema, columns)
|
||||
}
|
||||
|
||||
async fn get_csv_body(
|
||||
schema: &Arc<Schema>,
|
||||
columns: &[VectorRef],
|
||||
with_names: bool,
|
||||
with_types: bool,
|
||||
) -> String {
|
||||
let recordbatch = RecordBatch::new(schema.clone(), columns.to_vec()).unwrap();
|
||||
let recordbatches = RecordBatches::try_new(schema.clone(), vec![recordbatch]).unwrap();
|
||||
let output = Output::new_with_record_batches(recordbatches);
|
||||
let outputs = vec![Ok(output)];
|
||||
|
||||
let resp = CsvResponse::from_output(outputs, with_names, with_types)
|
||||
.await
|
||||
.into_response();
|
||||
let bytes = axum::body::to_bytes(resp.into_body(), 1024).await.unwrap();
|
||||
String::from_utf8(bytes.to_vec()).unwrap()
|
||||
}
|
||||
}
|
||||
|
||||
@@ -142,7 +142,7 @@ async fn test_sql_output_rows() {
|
||||
axum::body::to_bytes(resp.into_body(), usize::MAX)
|
||||
.await
|
||||
.unwrap(),
|
||||
Bytes::from_static(b"4950\n"),
|
||||
Bytes::from_static(b"4950\r\n"),
|
||||
);
|
||||
}
|
||||
HttpResponse::Table(resp) => {
|
||||
@@ -289,7 +289,7 @@ async fn test_sql_form() {
|
||||
axum::body::to_bytes(resp.into_body(), usize::MAX)
|
||||
.await
|
||||
.unwrap(),
|
||||
Bytes::from_static(b"4950\n"),
|
||||
Bytes::from_static(b"4950\r\n"),
|
||||
);
|
||||
}
|
||||
HttpResponse::Table(resp) => {
|
||||
|
||||
@@ -1006,7 +1006,7 @@ mod tests {
|
||||
|
||||
#[test]
|
||||
fn test_parse_alter_column_fulltext() {
|
||||
let sql = "ALTER TABLE test_table MODIFY COLUMN a SET FULLTEXT INDEX WITH(analyzer='English',case_sensitive='false',backend='bloom')";
|
||||
let sql = "ALTER TABLE test_table MODIFY COLUMN a SET FULLTEXT INDEX WITH(analyzer='English',case_sensitive='false',backend='bloom',granularity=1000,false_positive_rate=0.01)";
|
||||
let mut result =
|
||||
ParserContext::create_with_dialect(sql, &GreptimeDbDialect {}, ParseOptions::default())
|
||||
.unwrap();
|
||||
@@ -1029,12 +1029,14 @@ mod tests {
|
||||
} => {
|
||||
assert_eq!("a", column_name.value);
|
||||
assert_eq!(
|
||||
FulltextOptions {
|
||||
enable: true,
|
||||
analyzer: FulltextAnalyzer::English,
|
||||
case_sensitive: false,
|
||||
backend: FulltextBackend::Bloom,
|
||||
},
|
||||
FulltextOptions::new_unchecked(
|
||||
true,
|
||||
FulltextAnalyzer::English,
|
||||
false,
|
||||
FulltextBackend::Bloom,
|
||||
1000,
|
||||
0.01,
|
||||
),
|
||||
*options
|
||||
);
|
||||
}
|
||||
|
||||
@@ -29,8 +29,9 @@ use datafusion_sql::TableReference;
|
||||
use datatypes::arrow::datatypes::DataType;
|
||||
use datatypes::schema::{
|
||||
COLUMN_FULLTEXT_OPT_KEY_ANALYZER, COLUMN_FULLTEXT_OPT_KEY_BACKEND,
|
||||
COLUMN_FULLTEXT_OPT_KEY_CASE_SENSITIVE, COLUMN_SKIPPING_INDEX_OPT_KEY_GRANULARITY,
|
||||
COLUMN_SKIPPING_INDEX_OPT_KEY_TYPE,
|
||||
COLUMN_FULLTEXT_OPT_KEY_CASE_SENSITIVE, COLUMN_FULLTEXT_OPT_KEY_FALSE_POSITIVE_RATE,
|
||||
COLUMN_FULLTEXT_OPT_KEY_GRANULARITY, COLUMN_SKIPPING_INDEX_OPT_KEY_FALSE_POSITIVE_RATE,
|
||||
COLUMN_SKIPPING_INDEX_OPT_KEY_GRANULARITY, COLUMN_SKIPPING_INDEX_OPT_KEY_TYPE,
|
||||
};
|
||||
use snafu::ResultExt;
|
||||
|
||||
@@ -126,6 +127,8 @@ pub fn validate_column_fulltext_create_option(key: &str) -> bool {
|
||||
COLUMN_FULLTEXT_OPT_KEY_ANALYZER,
|
||||
COLUMN_FULLTEXT_OPT_KEY_CASE_SENSITIVE,
|
||||
COLUMN_FULLTEXT_OPT_KEY_BACKEND,
|
||||
COLUMN_FULLTEXT_OPT_KEY_GRANULARITY,
|
||||
COLUMN_FULLTEXT_OPT_KEY_FALSE_POSITIVE_RATE,
|
||||
]
|
||||
.contains(&key)
|
||||
}
|
||||
@@ -134,6 +137,7 @@ pub fn validate_column_skipping_index_create_option(key: &str) -> bool {
|
||||
[
|
||||
COLUMN_SKIPPING_INDEX_OPT_KEY_GRANULARITY,
|
||||
COLUMN_SKIPPING_INDEX_OPT_KEY_TYPE,
|
||||
COLUMN_SKIPPING_INDEX_OPT_KEY_FALSE_POSITIVE_RATE,
|
||||
]
|
||||
.contains(&key)
|
||||
}
|
||||
|
||||
@@ -143,6 +143,66 @@ pub enum Statement {
|
||||
Kill(Kill),
|
||||
}
|
||||
|
||||
impl Statement {
|
||||
pub fn is_readonly(&self) -> bool {
|
||||
match self {
|
||||
// Read-only operations
|
||||
Statement::Query(_)
|
||||
| Statement::ShowDatabases(_)
|
||||
| Statement::ShowTables(_)
|
||||
| Statement::ShowTableStatus(_)
|
||||
| Statement::ShowColumns(_)
|
||||
| Statement::ShowCharset(_)
|
||||
| Statement::ShowCollation(_)
|
||||
| Statement::ShowIndex(_)
|
||||
| Statement::ShowRegion(_)
|
||||
| Statement::ShowCreateDatabase(_)
|
||||
| Statement::ShowCreateTable(_)
|
||||
| Statement::ShowCreateFlow(_)
|
||||
| Statement::ShowFlows(_)
|
||||
| Statement::ShowCreateView(_)
|
||||
| Statement::ShowStatus(_)
|
||||
| Statement::ShowSearchPath(_)
|
||||
| Statement::ShowViews(_)
|
||||
| Statement::DescribeTable(_)
|
||||
| Statement::Explain(_)
|
||||
| Statement::ShowVariables(_)
|
||||
| Statement::FetchCursor(_)
|
||||
| Statement::Tql(_) => true,
|
||||
|
||||
#[cfg(feature = "enterprise")]
|
||||
Statement::ShowTriggers(_) => true,
|
||||
|
||||
// Write operations
|
||||
Statement::Insert(_)
|
||||
| Statement::Delete(_)
|
||||
| Statement::CreateTable(_)
|
||||
| Statement::CreateExternalTable(_)
|
||||
| Statement::CreateTableLike(_)
|
||||
| Statement::CreateFlow(_)
|
||||
| Statement::CreateView(_)
|
||||
| Statement::DropTable(_)
|
||||
| Statement::DropDatabase(_)
|
||||
| Statement::DropFlow(_)
|
||||
| Statement::DropView(_)
|
||||
| Statement::CreateDatabase(_)
|
||||
| Statement::AlterTable(_)
|
||||
| Statement::AlterDatabase(_)
|
||||
| Statement::Copy(_)
|
||||
| Statement::TruncateTable(_)
|
||||
| Statement::SetVariables(_)
|
||||
| Statement::Use(_)
|
||||
| Statement::DeclareCursor(_)
|
||||
| Statement::CloseCursor(_)
|
||||
| Statement::Kill(_)
|
||||
| Statement::Admin(_) => false,
|
||||
|
||||
#[cfg(feature = "enterprise")]
|
||||
Statement::CreateTrigger(_) => false,
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
impl Display for Statement {
|
||||
fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
|
||||
match self {
|
||||
|
||||
@@ -1064,6 +1064,14 @@ pub enum MetadataError {
|
||||
#[snafu(implicit)]
|
||||
location: Location,
|
||||
},
|
||||
|
||||
#[snafu(display("Invalid index option"))]
|
||||
InvalidIndexOption {
|
||||
#[snafu(implicit)]
|
||||
location: Location,
|
||||
#[snafu(source)]
|
||||
error: datatypes::error::Error,
|
||||
},
|
||||
}
|
||||
|
||||
impl ErrorExt for MetadataError {
|
||||
@@ -1620,12 +1628,14 @@ mod test {
|
||||
.alter(AlterKind::SetIndex {
|
||||
options: ApiSetIndexOptions::Fulltext {
|
||||
column_name: "b".to_string(),
|
||||
options: FulltextOptions {
|
||||
enable: true,
|
||||
analyzer: FulltextAnalyzer::Chinese,
|
||||
case_sensitive: true,
|
||||
backend: FulltextBackend::Bloom,
|
||||
},
|
||||
options: FulltextOptions::new_unchecked(
|
||||
true,
|
||||
FulltextAnalyzer::Chinese,
|
||||
true,
|
||||
FulltextBackend::Bloom,
|
||||
1000,
|
||||
0.01,
|
||||
),
|
||||
},
|
||||
})
|
||||
.unwrap();
|
||||
|
||||
@@ -113,13 +113,19 @@ pub const METRIC_ENGINE_INDEX_TYPE_OPTION: &str = "index.type";
|
||||
/// physical_metric_table = "",
|
||||
/// index.type = "skipping",
|
||||
/// index.granularity = "102400",
|
||||
/// index.false_positive_rate = "0.01",
|
||||
/// );
|
||||
/// ```
|
||||
pub const METRIC_ENGINE_INDEX_SKIPPING_INDEX_GRANULARITY_OPTION: &str = "index.granularity";
|
||||
pub const METRIC_ENGINE_INDEX_SKIPPING_INDEX_FALSE_POSITIVE_RATE_OPTION: &str =
|
||||
"index.false_positive_rate";
|
||||
|
||||
/// Default granularity for the skipping index in the metric engine.
|
||||
pub const METRIC_ENGINE_INDEX_SKIPPING_INDEX_GRANULARITY_OPTION_DEFAULT: u32 = 102400;
|
||||
|
||||
/// Default false positive rate for the skipping index in the metric engine.
|
||||
pub const METRIC_ENGINE_INDEX_SKIPPING_INDEX_FALSE_POSITIVE_RATE_OPTION_DEFAULT: f64 = 0.01;
|
||||
|
||||
/// Returns true if the `key` is a valid option key for the metric engine.
|
||||
pub fn is_metric_engine_option_key(key: &str) -> bool {
|
||||
[
|
||||
@@ -127,6 +133,7 @@ pub fn is_metric_engine_option_key(key: &str) -> bool {
|
||||
LOGICAL_TABLE_METADATA_KEY,
|
||||
METRIC_ENGINE_INDEX_TYPE_OPTION,
|
||||
METRIC_ENGINE_INDEX_SKIPPING_INDEX_GRANULARITY_OPTION,
|
||||
METRIC_ENGINE_INDEX_SKIPPING_INDEX_FALSE_POSITIVE_RATE_OPTION,
|
||||
]
|
||||
.contains(&key)
|
||||
}
|
||||
|
||||
@@ -42,8 +42,8 @@ use strum::{AsRefStr, IntoStaticStr};
|
||||
|
||||
use crate::logstore::entry;
|
||||
use crate::metadata::{
|
||||
ColumnMetadata, DecodeProtoSnafu, FlightCodecSnafu, InvalidRawRegionRequestSnafu,
|
||||
InvalidRegionRequestSnafu, InvalidSetRegionOptionRequestSnafu,
|
||||
ColumnMetadata, DecodeProtoSnafu, FlightCodecSnafu, InvalidIndexOptionSnafu,
|
||||
InvalidRawRegionRequestSnafu, InvalidRegionRequestSnafu, InvalidSetRegionOptionRequestSnafu,
|
||||
InvalidUnsetRegionOptionRequestSnafu, MetadataError, RegionMetadata, Result, UnexpectedSnafu,
|
||||
};
|
||||
use crate::metric_engine_consts::PHYSICAL_TABLE_METADATA_KEY;
|
||||
@@ -760,16 +760,19 @@ impl TryFrom<alter_request::Kind> for AlterKind {
|
||||
set_index::Options::Fulltext(x) => AlterKind::SetIndex {
|
||||
options: ApiSetIndexOptions::Fulltext {
|
||||
column_name: x.column_name.clone(),
|
||||
options: FulltextOptions {
|
||||
enable: x.enable,
|
||||
analyzer: as_fulltext_option_analyzer(
|
||||
options: FulltextOptions::new(
|
||||
x.enable,
|
||||
as_fulltext_option_analyzer(
|
||||
Analyzer::try_from(x.analyzer).context(DecodeProtoSnafu)?,
|
||||
),
|
||||
case_sensitive: x.case_sensitive,
|
||||
backend: as_fulltext_option_backend(
|
||||
x.case_sensitive,
|
||||
as_fulltext_option_backend(
|
||||
PbFulltextBackend::try_from(x.backend).context(DecodeProtoSnafu)?,
|
||||
),
|
||||
},
|
||||
x.granularity as u32,
|
||||
x.false_positive_rate,
|
||||
)
|
||||
.context(InvalidIndexOptionSnafu)?,
|
||||
},
|
||||
},
|
||||
set_index::Options::Inverted(i) => AlterKind::SetIndex {
|
||||
@@ -780,13 +783,15 @@ impl TryFrom<alter_request::Kind> for AlterKind {
|
||||
set_index::Options::Skipping(s) => AlterKind::SetIndex {
|
||||
options: ApiSetIndexOptions::Skipping {
|
||||
column_name: s.column_name,
|
||||
options: SkippingIndexOptions {
|
||||
index_type: as_skipping_index_type(
|
||||
options: SkippingIndexOptions::new(
|
||||
s.granularity as u32,
|
||||
s.false_positive_rate,
|
||||
as_skipping_index_type(
|
||||
PbSkippingIndexType::try_from(s.skipping_index_type)
|
||||
.context(DecodeProtoSnafu)?,
|
||||
),
|
||||
granularity: s.granularity as u32,
|
||||
},
|
||||
)
|
||||
.context(InvalidIndexOptionSnafu)?,
|
||||
},
|
||||
},
|
||||
},
|
||||
@@ -1644,12 +1649,14 @@ mod tests {
|
||||
let kind = AlterKind::SetIndex {
|
||||
options: ApiSetIndexOptions::Fulltext {
|
||||
column_name: "tag_0".to_string(),
|
||||
options: FulltextOptions {
|
||||
enable: true,
|
||||
analyzer: FulltextAnalyzer::Chinese,
|
||||
case_sensitive: false,
|
||||
backend: FulltextBackend::Bloom,
|
||||
},
|
||||
options: FulltextOptions::new_unchecked(
|
||||
true,
|
||||
FulltextAnalyzer::Chinese,
|
||||
false,
|
||||
FulltextBackend::Bloom,
|
||||
1000,
|
||||
0.01,
|
||||
),
|
||||
},
|
||||
};
|
||||
let request = RegionAlterRequest { kind };
|
||||
|
||||
Some files were not shown because too many files have changed in this diff Show More
Reference in New Issue
Block a user