Compare commits

..

6 Commits

Author SHA1 Message Date
dennis zhuang
61724386ef fix: potential failure in tests (#7167)
* fix: potential failure in the test_index_build_type_compact test

Signed-off-by: Dennis Zhuang <killme2008@gmail.com>

* fix: relax timestamp checking in test_timestamp_default_now

Signed-off-by: Dennis Zhuang <killme2008@gmail.com>

---------

Signed-off-by: Dennis Zhuang <killme2008@gmail.com>
2025-10-31 22:08:59 +00:00
Weny Xu
6960a0183a refactor: add support for batch region upgrade operations part1 (#7155)
* refactor: convert UpgradeRegion instruction to batch operation

Signed-off-by: WenyXu <wenymedia@gmail.com>

* feat: introduce `handle_batch_catchup_requests` fn for mito engine

Signed-off-by: WenyXu <wenymedia@gmail.com>

* test: add tests

Signed-off-by: WenyXu <wenymedia@gmail.com>

* feat: introduce `handle_batch_catchup_requests` fn for metric engine

Signed-off-by: WenyXu <wenymedia@gmail.com>

* chore: suggestion and add ser/de tests

Signed-off-by: WenyXu <wenymedia@gmail.com>

* chore: add comments

Signed-off-by: WenyXu <wenymedia@gmail.com>

* fix: fix unit tests

Signed-off-by: WenyXu <wenymedia@gmail.com>

---------

Signed-off-by: WenyXu <wenymedia@gmail.com>
2025-10-31 03:08:38 +00:00
Sicong Hu
30894d7599 feat(mito): Optimize async index building with priority-based batching (#7034)
* feat: add priority-based batching to IndexBuildScheduler

Signed-off-by: SNC123 <sinhco@outlook.com>

* fix: clean old puffin-related cache

Signed-off-by: SNC123 <sinhco@outlook.com>

* test: add test for IndexBuildScheduler

Signed-off-by: SNC123 <sinhco@outlook.com>

* feat: different index file id for read and async write

Signed-off-by: SNC123 <sinhco@outlook.com>

* feat: different index file id for delete

Signed-off-by: SNC123 <sinhco@outlook.com>

* chore: clippy

Signed-off-by: SNC123 <sinhco@outlook.com>

* fix: apply suggestions

Signed-off-by: SNC123 <sinhco@outlook.com>

* fix: apply comments

Signed-off-by: SNC123 <sinhco@outlook.com>

* combine files and index files

Signed-off-by: SNC123 <sinhco@outlook.com>

* feat: add index_file_id into ManifestSstEntry

Signed-off-by: SNC123 <sinhco@outlook.com>

* Update src/mito2/src/gc.rs

Signed-off-by: SNC123 <sinhco@outlook.com>

* resolve conflicts

Signed-off-by: SNC123 <sinhco@outlook.com>

* fix: sqlness

Signed-off-by: SNC123 <sinhco@outlook.com>

* chore: fmt

Signed-off-by: SNC123 <sinhco@outlook.com>

---------

Signed-off-by: SNC123 <sinhco@outlook.com>
2025-10-31 02:13:17 +00:00
Yingwen
acf38a7091 fix: avoid filtering rows with delete op by fields under merge mode (#7154)
* chore: clear allow dead_code for flat format

Signed-off-by: evenyag <realevenyag@gmail.com>

* chore: pass exprs to build appliers

Signed-off-by: evenyag <realevenyag@gmail.com>

* feat: split field filters and index appliers

Signed-off-by: evenyag <realevenyag@gmail.com>

* feat: support skip filtering fields in RowGroupPruningStats

Signed-off-by: evenyag <realevenyag@gmail.com>

* feat: add PreFilterMode to config whether to skip filtering fields

Adds the PreFilterMode to the RangeBase and sets it in
ParquetReaderBuilder

Signed-off-by: evenyag <realevenyag@gmail.com>

* feat: support skipping fields in prune reader

Signed-off-by: evenyag <realevenyag@gmail.com>

* feat: support pre filter mode in bulk memtable

Signed-off-by: evenyag <realevenyag@gmail.com>

* feat: pass PreFilterMode to memtable

Signed-off-by: evenyag <realevenyag@gmail.com>

* test: test mito filter delete

Signed-off-by: evenyag <realevenyag@gmail.com>

* chore: fix compiler errors

Signed-off-by: evenyag <realevenyag@gmail.com>

* chore: remove commented code

Signed-off-by: evenyag <realevenyag@gmail.com>

* refactor: move predicate and sequence to RangesOptions

Signed-off-by: evenyag <realevenyag@gmail.com>

* chore: fmt code

Signed-off-by: evenyag <realevenyag@gmail.com>

* ci: skip cargo gc

Signed-off-by: evenyag <realevenyag@gmail.com>

* chore: fix cargo build warning

Signed-off-by: evenyag <realevenyag@gmail.com>

* Revert "ci: skip cargo gc"

This reverts commit 1ec9594a6d.

Signed-off-by: evenyag <realevenyag@gmail.com>

---------

Signed-off-by: evenyag <realevenyag@gmail.com>
2025-10-30 12:14:45 +00:00
LFC
109b70750a refactor: convert to prometheus values directly from arrow (#7153)
* refactor: convert to prometheus values directly from arrow

Signed-off-by: luofucong <luofc@foxmail.com>

* resolve PR comments

Signed-off-by: luofucong <luofc@foxmail.com>

---------

Signed-off-by: luofucong <luofc@foxmail.com>
2025-10-30 10:24:12 +00:00
shuiyisong
ee5b7ff3c8 chore: unify initialization of channel manager (#7159)
* chore: unify initialization of channel manager and extract loading tls

Signed-off-by: shuiyisong <xixing.sys@gmail.com>

* chore: fix cr issue

Signed-off-by: shuiyisong <xixing.sys@gmail.com>

---------

Signed-off-by: shuiyisong <xixing.sys@gmail.com>
2025-10-30 04:26:02 +00:00
80 changed files with 2939 additions and 842 deletions

52
Cargo.lock generated
View File

@@ -1336,13 +1336,9 @@ checksum = "1fd0f2584146f6f2ef48085050886acf353beff7305ebd1ae69500e27c67f64b"
[[package]]
name = "bytes"
version = "1.10.1"
source = "git+https://github.com/discord9/bytes?rev=1572ab22c3cbad0e9b6681d1f68eca4139322a2a#1572ab22c3cbad0e9b6681d1f68eca4139322a2a"
source = "registry+https://github.com/rust-lang/crates.io-index"
checksum = "d71b6127be86fdcfddb610f7182ac57211d4b18a3e9c82eb2d17662f2227ad6a"
dependencies = [
"backtrace",
"crossbeam-channel",
"inferno 0.12.2",
"papaya",
"quanta",
"serde",
]
@@ -8866,16 +8862,6 @@ dependencies = [
"unicode-width 0.1.14",
]
[[package]]
name = "papaya"
version = "0.2.3"
source = "registry+https://github.com/rust-lang/crates.io-index"
checksum = "f92dd0b07c53a0a0c764db2ace8c541dc47320dad97c2200c2a637ab9dd2328f"
dependencies = [
"equivalent",
"seize",
]
[[package]]
name = "parking"
version = "2.2.1"
@@ -10109,21 +10095,6 @@ dependencies = [
"variadics",
]
[[package]]
name = "quanta"
version = "0.12.6"
source = "registry+https://github.com/rust-lang/crates.io-index"
checksum = "f3ab5a9d756f0d97bdc89019bd2e4ea098cf9cde50ee7564dde6b81ccc8f06c7"
dependencies = [
"crossbeam-utils",
"libc",
"once_cell",
"raw-cpuid",
"wasi 0.11.1+wasi-snapshot-preview1",
"web-sys",
"winapi",
]
[[package]]
name = "query"
version = "0.18.0"
@@ -10423,15 +10394,6 @@ dependencies = [
"thiserror 1.0.69",
]
[[package]]
name = "raw-cpuid"
version = "11.6.0"
source = "registry+https://github.com/rust-lang/crates.io-index"
checksum = "498cd0dc59d73224351ee52a95fee0f1a617a2eae0e7d9d720cc622c73a54186"
dependencies = [
"bitflags 2.9.1",
]
[[package]]
name = "rawpointer"
version = "0.2.1"
@@ -11372,16 +11334,6 @@ dependencies = [
"libc",
]
[[package]]
name = "seize"
version = "0.5.1"
source = "registry+https://github.com/rust-lang/crates.io-index"
checksum = "5b55fb86dfd3a2f5f76ea78310a88f96c4ea21a3031f8d212443d56123fd0521"
dependencies = [
"libc",
"windows-sys 0.52.0",
]
[[package]]
name = "semver"
version = "1.0.26"

View File

@@ -333,7 +333,6 @@ datafusion-datasource = { git = "https://github.com/GreptimeTeam/datafusion.git"
datafusion-sql = { git = "https://github.com/GreptimeTeam/datafusion.git", rev = "fd4b2abcf3c3e43e94951bda452c9fd35243aab0" }
datafusion-substrait = { git = "https://github.com/GreptimeTeam/datafusion.git", rev = "fd4b2abcf3c3e43e94951bda452c9fd35243aab0" }
sqlparser = { git = "https://github.com/GreptimeTeam/sqlparser-rs.git", rev = "4b519a5caa95472cc3988f5556813a583dd35af1" } # branch = "v0.58.x"
bytes = { git = "https://github.com/discord9/bytes", rev = "1572ab22c3cbad0e9b6681d1f68eca4139322a2a" }
[profile.release]
debug = 1

View File

@@ -92,9 +92,6 @@ curl -X POST localhost:4000/debug/prof/mem > greptime.hprof
curl -X POST "localhost:4000/debug/prof/mem?output=flamegraph" > greptime.svg
# or output pprof format
curl -X POST "localhost:4000/debug/prof/mem?output=proto" > greptime.pprof
curl -X POST "localhost:4000/debug/prof/bytes" > greptime.svg
```
You can periodically dump profiling data and compare them to find the delta memory usage.

View File

@@ -20,7 +20,9 @@ use api::v1::health_check_client::HealthCheckClient;
use api::v1::prometheus_gateway_client::PrometheusGatewayClient;
use api::v1::region::region_client::RegionClient as PbRegionClient;
use arrow_flight::flight_service_client::FlightServiceClient;
use common_grpc::channel_manager::{ChannelConfig, ChannelManager, ClientTlsOption};
use common_grpc::channel_manager::{
ChannelConfig, ChannelManager, ClientTlsOption, load_tls_config,
};
use parking_lot::RwLock;
use snafu::{OptionExt, ResultExt};
use tonic::codec::CompressionEncoding;
@@ -94,8 +96,9 @@ impl Client {
A: AsRef<[U]>,
{
let channel_config = ChannelConfig::default().client_tls_config(client_tls);
let channel_manager = ChannelManager::with_tls_config(channel_config)
let tls_config = load_tls_config(channel_config.client_tls.as_ref())
.context(error::CreateTlsChannelSnafu)?;
let channel_manager = ChannelManager::with_config(channel_config, tls_config);
Ok(Self::with_manager_and_urls(channel_manager, urls))
}

View File

@@ -74,7 +74,7 @@ impl FlownodeManager for NodeClients {
impl NodeClients {
pub fn new(config: ChannelConfig) -> Self {
Self {
channel_manager: ChannelManager::with_config(config),
channel_manager: ChannelManager::with_config(config, None),
clients: CacheBuilder::new(1024)
.time_to_live(Duration::from_secs(30 * 60))
.time_to_idle(Duration::from_secs(5 * 60))

View File

@@ -162,6 +162,7 @@ impl ObjbenchCommand {
file_size,
available_indexes: Default::default(),
index_file_size: 0,
index_file_id: None,
num_rows,
num_row_groups,
sequence: None,

View File

@@ -104,7 +104,7 @@ impl MetaClientSelector {
let cfg = ChannelConfig::new()
.connect_timeout(Duration::from_secs(30))
.timeout(Duration::from_secs(30));
let channel_manager = ChannelManager::with_config(cfg);
let channel_manager = ChannelManager::with_config(cfg, None);
Self {
meta_client,
channel_manager,

View File

@@ -22,14 +22,14 @@ use dashmap::DashMap;
use dashmap::mapref::entry::Entry;
use lazy_static::lazy_static;
use serde::{Deserialize, Serialize};
use snafu::{OptionExt, ResultExt};
use snafu::ResultExt;
use tokio_util::sync::CancellationToken;
use tonic::transport::{
Certificate, Channel as InnerChannel, ClientTlsConfig, Endpoint, Identity, Uri,
};
use tower::Service;
use crate::error::{CreateChannelSnafu, InvalidConfigFilePathSnafu, InvalidTlsConfigSnafu, Result};
use crate::error::{CreateChannelSnafu, InvalidConfigFilePathSnafu, Result};
const RECYCLE_CHANNEL_INTERVAL_SECS: u64 = 60;
pub const DEFAULT_GRPC_REQUEST_TIMEOUT_SECS: u64 = 10;
@@ -91,57 +91,18 @@ impl ChannelManager {
Default::default()
}
pub fn with_config(config: ChannelConfig) -> Self {
let inner = Inner::with_config(config);
/// unified with config function that support tls config
/// use [`load_tls_config`] to load tls config from file system
pub fn with_config(config: ChannelConfig, tls_config: Option<ClientTlsConfig>) -> Self {
let mut inner = Inner::with_config(config.clone());
if let Some(tls_config) = tls_config {
inner.client_tls_config = Some(tls_config);
}
Self {
inner: Arc::new(inner),
}
}
/// Read tls cert and key files and create a ChannelManager with TLS config.
pub fn with_tls_config(config: ChannelConfig) -> Result<Self> {
let mut inner = Inner::with_config(config.clone());
// setup tls
let path_config = config.client_tls.context(InvalidTlsConfigSnafu {
msg: "no config input",
})?;
if !path_config.enabled {
// if TLS not enabled, just ignore other tls config
// and not set `client_tls_config` hence not use TLS
return Ok(Self {
inner: Arc::new(inner),
});
}
let mut tls_config = ClientTlsConfig::new();
if let Some(server_ca) = path_config.server_ca_cert_path {
let server_root_ca_cert =
std::fs::read_to_string(server_ca).context(InvalidConfigFilePathSnafu)?;
let server_root_ca_cert = Certificate::from_pem(server_root_ca_cert);
tls_config = tls_config.ca_certificate(server_root_ca_cert);
}
if let (Some(client_cert_path), Some(client_key_path)) =
(&path_config.client_cert_path, &path_config.client_key_path)
{
let client_cert =
std::fs::read_to_string(client_cert_path).context(InvalidConfigFilePathSnafu)?;
let client_key =
std::fs::read_to_string(client_key_path).context(InvalidConfigFilePathSnafu)?;
let client_identity = Identity::from_pem(client_cert, client_key);
tls_config = tls_config.identity(client_identity);
}
inner.client_tls_config = Some(tls_config);
Ok(Self {
inner: Arc::new(inner),
})
}
pub fn config(&self) -> &ChannelConfig {
&self.inner.config
}
@@ -287,6 +248,34 @@ impl ChannelManager {
}
}
pub fn load_tls_config(tls_option: Option<&ClientTlsOption>) -> Result<Option<ClientTlsConfig>> {
let path_config = match tls_option {
Some(path_config) if path_config.enabled => path_config,
_ => return Ok(None),
};
let mut tls_config = ClientTlsConfig::new();
if let Some(server_ca) = &path_config.server_ca_cert_path {
let server_root_ca_cert =
std::fs::read_to_string(server_ca).context(InvalidConfigFilePathSnafu)?;
let server_root_ca_cert = Certificate::from_pem(server_root_ca_cert);
tls_config = tls_config.ca_certificate(server_root_ca_cert);
}
if let (Some(client_cert_path), Some(client_key_path)) =
(&path_config.client_cert_path, &path_config.client_key_path)
{
let client_cert =
std::fs::read_to_string(client_cert_path).context(InvalidConfigFilePathSnafu)?;
let client_key =
std::fs::read_to_string(client_key_path).context(InvalidConfigFilePathSnafu)?;
let client_identity = Identity::from_pem(client_cert, client_key);
tls_config = tls_config.identity(client_identity);
}
Ok(Some(tls_config))
}
#[derive(Clone, Debug, PartialEq, Eq, Serialize, Deserialize)]
pub struct ClientTlsOption {
/// Whether to enable TLS for client.
@@ -659,7 +648,7 @@ mod tests {
.http2_adaptive_window(true)
.tcp_keepalive(Duration::from_secs(2))
.tcp_nodelay(true);
let mgr = ChannelManager::with_config(config);
let mgr = ChannelManager::with_config(config, None);
let res = mgr.build_endpoint("test_addr");

View File

@@ -12,14 +12,17 @@
// See the License for the specific language governing permissions and
// limitations under the License.
use common_grpc::channel_manager::{ChannelConfig, ChannelManager, ClientTlsOption};
use common_grpc::channel_manager::{
ChannelConfig, ChannelManager, ClientTlsOption, load_tls_config,
};
#[tokio::test]
async fn test_mtls_config() {
// test no config
let config = ChannelConfig::new();
let re = ChannelManager::with_tls_config(config);
assert!(re.is_err());
let re = load_tls_config(config.client_tls.as_ref());
assert!(re.is_ok());
assert!(re.unwrap().is_none());
// test wrong file
let config = ChannelConfig::new().client_tls_config(ClientTlsOption {
@@ -29,7 +32,7 @@ async fn test_mtls_config() {
client_key_path: Some("tests/tls/wrong_client.key".to_string()),
});
let re = ChannelManager::with_tls_config(config);
let re = load_tls_config(config.client_tls.as_ref());
assert!(re.is_err());
// test corrupted file content
@@ -40,7 +43,9 @@ async fn test_mtls_config() {
client_key_path: Some("tests/tls/corrupted".to_string()),
});
let re = ChannelManager::with_tls_config(config).unwrap();
let tls_config = load_tls_config(config.client_tls.as_ref()).unwrap();
let re = ChannelManager::with_config(config, tls_config);
let re = re.get("127.0.0.1:0");
assert!(re.is_err());
@@ -52,7 +57,8 @@ async fn test_mtls_config() {
client_key_path: Some("tests/tls/client.key".to_string()),
});
let re = ChannelManager::with_tls_config(config).unwrap();
let tls_config = load_tls_config(config.client_tls.as_ref()).unwrap();
let re = ChannelManager::with_config(config, tls_config);
let re = re.get("127.0.0.1:0");
let _ = re.unwrap();
}

View File

@@ -507,13 +507,14 @@ pub enum Instruction {
/// Closes regions.
#[serde(deserialize_with = "single_or_multiple_from", alias = "CloseRegion")]
CloseRegions(Vec<RegionIdent>),
/// Upgrades a region.
UpgradeRegion(UpgradeRegion),
/// Upgrades regions.
#[serde(deserialize_with = "single_or_multiple_from", alias = "UpgradeRegion")]
UpgradeRegions(Vec<UpgradeRegion>),
#[serde(
deserialize_with = "single_or_multiple_from",
alias = "DowngradeRegion"
)]
/// Downgrades a region.
/// Downgrades regions.
DowngradeRegions(Vec<DowngradeRegion>),
/// Invalidates batch cache.
InvalidateCaches(Vec<CacheIdent>),
@@ -559,9 +560,9 @@ impl Instruction {
}
/// Converts the instruction into a [UpgradeRegion].
pub fn into_upgrade_regions(self) -> Option<UpgradeRegion> {
pub fn into_upgrade_regions(self) -> Option<Vec<UpgradeRegion>> {
match self {
Self::UpgradeRegion(upgrade_region) => Some(upgrade_region),
Self::UpgradeRegions(upgrade_region) => Some(upgrade_region),
_ => None,
}
}
@@ -584,6 +585,10 @@ impl Instruction {
/// The reply of [UpgradeRegion].
#[derive(Debug, Serialize, Deserialize, PartialEq, Eq, Clone)]
pub struct UpgradeRegionReply {
/// The [RegionId].
/// For compatibility, it is defaulted to [RegionId::new(0, 0)].
#[serde(default)]
pub region_id: RegionId,
/// Returns true if `last_entry_id` has been replayed to the latest.
pub ready: bool,
/// Indicates whether the region exists.
@@ -635,6 +640,39 @@ where
})
}
#[derive(Debug, Serialize, Deserialize, PartialEq, Eq, Clone)]
pub struct UpgradeRegionsReply {
pub replies: Vec<UpgradeRegionReply>,
}
impl UpgradeRegionsReply {
pub fn new(replies: Vec<UpgradeRegionReply>) -> Self {
Self { replies }
}
pub fn single(reply: UpgradeRegionReply) -> Self {
Self::new(vec![reply])
}
}
#[derive(Deserialize)]
#[serde(untagged)]
enum UpgradeRegionsCompat {
Single(UpgradeRegionReply),
Multiple(UpgradeRegionsReply),
}
fn upgrade_regions_compat_from<'de, D>(deserializer: D) -> Result<UpgradeRegionsReply, D::Error>
where
D: Deserializer<'de>,
{
let helper = UpgradeRegionsCompat::deserialize(deserializer)?;
Ok(match helper {
UpgradeRegionsCompat::Single(x) => UpgradeRegionsReply::new(vec![x]),
UpgradeRegionsCompat::Multiple(reply) => reply,
})
}
#[derive(Debug, Serialize, Deserialize, PartialEq, Eq, Clone)]
#[serde(tag = "type", rename_all = "snake_case")]
pub enum InstructionReply {
@@ -642,7 +680,11 @@ pub enum InstructionReply {
OpenRegions(SimpleReply),
#[serde(alias = "close_region")]
CloseRegions(SimpleReply),
UpgradeRegion(UpgradeRegionReply),
#[serde(
deserialize_with = "upgrade_regions_compat_from",
alias = "upgrade_region"
)]
UpgradeRegions(UpgradeRegionsReply),
#[serde(
alias = "downgrade_region",
deserialize_with = "downgrade_regions_compat_from"
@@ -658,9 +700,11 @@ impl Display for InstructionReply {
match self {
Self::OpenRegions(reply) => write!(f, "InstructionReply::OpenRegions({})", reply),
Self::CloseRegions(reply) => write!(f, "InstructionReply::CloseRegions({})", reply),
Self::UpgradeRegion(reply) => write!(f, "InstructionReply::UpgradeRegion({})", reply),
Self::UpgradeRegions(reply) => {
write!(f, "InstructionReply::UpgradeRegions({:?})", reply.replies)
}
Self::DowngradeRegions(reply) => {
write!(f, "InstructionReply::DowngradeRegions({:?})", reply)
write!(f, "InstructionReply::DowngradeRegions({:?})", reply.replies)
}
Self::FlushRegions(reply) => write!(f, "InstructionReply::FlushRegions({})", reply),
Self::GetFileRefs(reply) => write!(f, "InstructionReply::GetFileRefs({})", reply),
@@ -685,9 +729,9 @@ impl InstructionReply {
}
}
pub fn expect_upgrade_region_reply(self) -> UpgradeRegionReply {
pub fn expect_upgrade_regions_reply(self) -> Vec<UpgradeRegionReply> {
match self {
Self::UpgradeRegion(reply) => reply,
Self::UpgradeRegions(reply) => reply.replies,
_ => panic!("Expected UpgradeRegion reply"),
}
}
@@ -749,25 +793,58 @@ mod tests {
serialized
);
let downgrade_region = InstructionReply::DowngradeRegions(DowngradeRegionsReply::single(
DowngradeRegionReply {
let upgrade_region = Instruction::UpgradeRegions(vec![UpgradeRegion {
region_id: RegionId::new(1024, 1),
last_entry_id: None,
metadata_last_entry_id: None,
replay_timeout: Some(Duration::from_millis(1000)),
location_id: None,
replay_entry_id: None,
metadata_replay_entry_id: None,
}]);
let serialized = serde_json::to_string(&upgrade_region).unwrap();
assert_eq!(
r#"{"UpgradeRegions":[{"region_id":4398046511105,"last_entry_id":null,"metadata_last_entry_id":null,"replay_timeout":"1s","location_id":null}]}"#,
serialized
);
}
#[test]
fn test_serialize_instruction_reply() {
let downgrade_region_reply = InstructionReply::DowngradeRegions(
DowngradeRegionsReply::single(DowngradeRegionReply {
region_id: RegionId::new(1024, 1),
last_entry_id: None,
metadata_last_entry_id: None,
exists: true,
error: None,
},
));
}),
);
let serialized = serde_json::to_string(&downgrade_region).unwrap();
let serialized = serde_json::to_string(&downgrade_region_reply).unwrap();
assert_eq!(
r#"{"type":"downgrade_regions","replies":[{"region_id":4398046511105,"last_entry_id":null,"metadata_last_entry_id":null,"exists":true,"error":null}]}"#,
serialized
)
);
let upgrade_region_reply =
InstructionReply::UpgradeRegions(UpgradeRegionsReply::single(UpgradeRegionReply {
region_id: RegionId::new(1024, 1),
ready: true,
exists: true,
error: None,
}));
let serialized = serde_json::to_string(&upgrade_region_reply).unwrap();
assert_eq!(
r#"{"type":"upgrade_regions","replies":[{"region_id":4398046511105,"ready":true,"exists":true,"error":null}]}"#,
serialized
);
}
#[test]
fn test_deserialize_instruction() {
// legacy open region instruction
let open_region_instruction = r#"{"OpenRegion":{"region_ident":{"datanode_id":2,"table_id":1024,"region_number":1,"engine":"mito2"},"region_storage_path":"test/foo","region_options":{},"region_wal_options":{},"skip_wal_replay":false}}"#;
let open_region_instruction: Instruction =
serde_json::from_str(open_region_instruction).unwrap();
@@ -785,6 +862,7 @@ mod tests {
)]);
assert_eq!(open_region_instruction, open_region);
// legacy close region instruction
let close_region_instruction = r#"{"CloseRegion":{"datanode_id":2,"table_id":1024,"region_number":1,"engine":"mito2"}}"#;
let close_region_instruction: Instruction =
serde_json::from_str(close_region_instruction).unwrap();
@@ -796,6 +874,7 @@ mod tests {
}]);
assert_eq!(close_region_instruction, close_region);
// legacy downgrade region instruction
let downgrade_region_instruction = r#"{"DowngradeRegions":{"region_id":4398046511105,"flush_timeout":{"secs":1,"nanos":0}}}"#;
let downgrade_region_instruction: Instruction =
serde_json::from_str(downgrade_region_instruction).unwrap();
@@ -805,6 +884,25 @@ mod tests {
}]);
assert_eq!(downgrade_region_instruction, downgrade_region);
// legacy upgrade region instruction
let upgrade_region_instruction = r#"{"UpgradeRegion":{"region_id":4398046511105,"last_entry_id":null,"metadata_last_entry_id":null,"replay_timeout":"1s","location_id":null,"replay_entry_id":null,"metadata_replay_entry_id":null}}"#;
let upgrade_region_instruction: Instruction =
serde_json::from_str(upgrade_region_instruction).unwrap();
let upgrade_region = Instruction::UpgradeRegions(vec![UpgradeRegion {
region_id: RegionId::new(1024, 1),
last_entry_id: None,
metadata_last_entry_id: None,
replay_timeout: Some(Duration::from_millis(1000)),
location_id: None,
replay_entry_id: None,
metadata_replay_entry_id: None,
}]);
assert_eq!(upgrade_region_instruction, upgrade_region);
}
#[test]
fn test_deserialize_instruction_reply() {
// legacy close region reply
let close_region_instruction_reply =
r#"{"result":true,"error":null,"type":"close_region"}"#;
let close_region_instruction_reply: InstructionReply =
@@ -815,6 +913,7 @@ mod tests {
});
assert_eq!(close_region_instruction_reply, close_region_reply);
// legacy open region reply
let open_region_instruction_reply = r#"{"result":true,"error":null,"type":"open_region"}"#;
let open_region_instruction_reply: InstructionReply =
serde_json::from_str(open_region_instruction_reply).unwrap();
@@ -824,6 +923,7 @@ mod tests {
});
assert_eq!(open_region_instruction_reply, open_region_reply);
// legacy downgrade region reply
let downgrade_region_instruction_reply = r#"{"region_id":4398046511105,"last_entry_id":null,"metadata_last_entry_id":null,"exists":true,"error":null,"type":"downgrade_region"}"#;
let downgrade_region_instruction_reply: InstructionReply =
serde_json::from_str(downgrade_region_instruction_reply).unwrap();
@@ -837,6 +937,19 @@ mod tests {
}),
);
assert_eq!(downgrade_region_instruction_reply, downgrade_region_reply);
// legacy upgrade region reply
let upgrade_region_instruction_reply = r#"{"region_id":4398046511105,"ready":true,"exists":true,"error":null,"type":"upgrade_region"}"#;
let upgrade_region_instruction_reply: InstructionReply =
serde_json::from_str(upgrade_region_instruction_reply).unwrap();
let upgrade_region_reply =
InstructionReply::UpgradeRegions(UpgradeRegionsReply::single(UpgradeRegionReply {
region_id: RegionId::new(1024, 1),
ready: true,
exists: true,
error: None,
}));
assert_eq!(upgrade_region_instruction_reply, upgrade_region_reply);
}
#[derive(Debug, Clone, Serialize, Deserialize)]

View File

@@ -114,7 +114,7 @@ impl RegionHeartbeatResponseHandler {
)),
Instruction::FlushRegions(_) => Ok(Box::new(FlushRegionsHandler.into())),
Instruction::DowngradeRegions(_) => Ok(Box::new(DowngradeRegionsHandler.into())),
Instruction::UpgradeRegion(_) => Ok(Box::new(UpgradeRegionsHandler.into())),
Instruction::UpgradeRegions(_) => Ok(Box::new(UpgradeRegionsHandler.into())),
Instruction::GetFileRefs(_) => Ok(Box::new(GetFileRefsHandler.into())),
Instruction::GcRegions(_) => Ok(Box::new(GcRegionsHandler.into())),
Instruction::InvalidateCaches(_) => InvalidHeartbeatResponseSnafu.fail(),
@@ -194,7 +194,7 @@ dispatch_instr!(
OpenRegions => OpenRegions,
FlushRegions => FlushRegions,
DowngradeRegions => DowngradeRegions,
UpgradeRegion => UpgradeRegions,
UpgradeRegions => UpgradeRegions,
GetFileRefs => GetFileRefs,
GcRegions => GcRegions,
);
@@ -334,10 +334,10 @@ mod tests {
);
// Upgrade region
let instruction = Instruction::UpgradeRegion(UpgradeRegion {
let instruction = Instruction::UpgradeRegions(vec![UpgradeRegion {
region_id,
..Default::default()
});
}]);
assert!(
heartbeat_handler.is_acceptable(&heartbeat_env.create_handler_ctx((meta, instruction)))
);

View File

@@ -12,8 +12,11 @@
// See the License for the specific language governing permissions and
// limitations under the License.
use common_meta::instruction::{InstructionReply, UpgradeRegion, UpgradeRegionReply};
use common_meta::instruction::{
InstructionReply, UpgradeRegion, UpgradeRegionReply, UpgradeRegionsReply,
};
use common_telemetry::{info, warn};
use futures::future::join_all;
use store_api::region_request::{RegionCatchupRequest, RegionRequest, ReplayCheckpoint};
use crate::heartbeat::handler::{HandlerContext, InstructionHandler};
@@ -22,115 +25,151 @@ use crate::heartbeat::task_tracker::WaitResult;
#[derive(Debug, Clone, Copy, Default)]
pub struct UpgradeRegionsHandler;
impl UpgradeRegionsHandler {
// Handles upgrade regions instruction.
//
// Returns batch of upgrade region replies, the order of the replies is not guaranteed.
async fn handle_upgrade_regions(
&self,
ctx: &HandlerContext,
upgrade_regions: Vec<UpgradeRegion>,
) -> Vec<UpgradeRegionReply> {
let mut replies = Vec::with_capacity(upgrade_regions.len());
let mut catchup_request = Vec::with_capacity(upgrade_regions.len());
for upgrade_region in upgrade_regions {
let Some(writable) = ctx.region_server.is_region_leader(upgrade_region.region_id)
else {
replies.push(UpgradeRegionReply {
region_id: upgrade_region.region_id,
ready: false,
exists: false,
error: None,
});
continue;
};
if writable {
replies.push(UpgradeRegionReply {
region_id: upgrade_region.region_id,
ready: true,
exists: true,
error: None,
});
} else {
let UpgradeRegion {
last_entry_id,
metadata_last_entry_id,
location_id,
replay_entry_id,
metadata_replay_entry_id,
replay_timeout,
..
} = upgrade_region;
let checkpoint = match (replay_entry_id, metadata_replay_entry_id) {
(Some(entry_id), metadata_entry_id) => Some(ReplayCheckpoint {
entry_id,
metadata_entry_id,
}),
_ => None,
};
catchup_request.push((
upgrade_region.region_id,
replay_timeout.unwrap_or_default(),
RegionCatchupRequest {
set_writable: true,
entry_id: last_entry_id,
metadata_entry_id: metadata_last_entry_id,
location_id,
checkpoint,
},
));
}
}
let mut wait_results = Vec::with_capacity(catchup_request.len());
for (region_id, replay_timeout, catchup_request) in catchup_request {
let region_server_moved = ctx.region_server.clone();
// TODO(weny): parallelize the catchup tasks.
let result = ctx
.catchup_tasks
.try_register(
region_id,
Box::pin(async move {
info!(
"Executing region: {region_id} catchup to: last entry id {:?}",
catchup_request.entry_id
);
region_server_moved
.handle_request(region_id, RegionRequest::Catchup(catchup_request))
.await?;
Ok(())
}),
)
.await;
if result.is_busy() {
warn!("Another catchup task is running for the region: {region_id}");
}
// We don't care that it returns a newly registered or running task.
let mut watcher = result.into_watcher();
wait_results.push((
region_id,
ctx.catchup_tasks.wait(&mut watcher, replay_timeout).await,
));
}
let results = join_all(
wait_results
.into_iter()
.map(|(region_id, result)| async move {
match result {
WaitResult::Timeout => UpgradeRegionReply {
region_id,
ready: false,
exists: true,
error: None,
},
WaitResult::Finish(Ok(_)) => UpgradeRegionReply {
region_id,
ready: true,
exists: true,
error: None,
},
WaitResult::Finish(Err(err)) => UpgradeRegionReply {
region_id,
ready: false,
exists: true,
error: Some(format!("{err:?}")),
},
}
}),
)
.await;
replies.extend(results.into_iter());
replies
}
}
#[async_trait::async_trait]
impl InstructionHandler for UpgradeRegionsHandler {
type Instruction = UpgradeRegion;
type Instruction = Vec<UpgradeRegion>;
async fn handle(
&self,
ctx: &HandlerContext,
UpgradeRegion {
region_id,
last_entry_id,
metadata_last_entry_id,
replay_timeout,
location_id,
replay_entry_id,
metadata_replay_entry_id,
}: UpgradeRegion,
upgrade_regions: Self::Instruction,
) -> Option<InstructionReply> {
let Some(writable) = ctx.region_server.is_region_leader(region_id) else {
return Some(InstructionReply::UpgradeRegion(UpgradeRegionReply {
ready: false,
exists: false,
error: None,
}));
};
let replies = self.handle_upgrade_regions(ctx, upgrade_regions).await;
if writable {
return Some(InstructionReply::UpgradeRegion(UpgradeRegionReply {
ready: true,
exists: true,
error: None,
}));
}
let region_server_moved = ctx.region_server.clone();
let checkpoint = match (replay_entry_id, metadata_replay_entry_id) {
(Some(entry_id), metadata_entry_id) => Some(ReplayCheckpoint {
entry_id,
metadata_entry_id,
}),
_ => None,
};
// The catchup task is almost zero cost if the inside region is writable.
// Therefore, it always registers a new catchup task.
let register_result = ctx
.catchup_tasks
.try_register(
region_id,
Box::pin(async move {
info!(
"Executing region: {region_id} catchup to: last entry id {last_entry_id:?}"
);
region_server_moved
.handle_request(
region_id,
RegionRequest::Catchup(RegionCatchupRequest {
set_writable: true,
entry_id: last_entry_id,
metadata_entry_id: metadata_last_entry_id,
location_id,
checkpoint,
}),
)
.await?;
Ok(())
}),
)
.await;
if register_result.is_busy() {
warn!("Another catchup task is running for the region: {region_id}");
}
// Returns immediately
let Some(replay_timeout) = replay_timeout else {
return Some(InstructionReply::UpgradeRegion(UpgradeRegionReply {
ready: false,
exists: true,
error: None,
}));
};
// We don't care that it returns a newly registered or running task.
let mut watcher = register_result.into_watcher();
let result = ctx.catchup_tasks.wait(&mut watcher, replay_timeout).await;
match result {
WaitResult::Timeout => Some(InstructionReply::UpgradeRegion(UpgradeRegionReply {
ready: false,
exists: true,
error: None,
})),
WaitResult::Finish(Ok(_)) => {
Some(InstructionReply::UpgradeRegion(UpgradeRegionReply {
ready: true,
exists: true,
error: None,
}))
}
WaitResult::Finish(Err(err)) => {
Some(InstructionReply::UpgradeRegion(UpgradeRegionReply {
ready: false,
exists: true,
error: Some(format!("{err:?}")),
}))
}
}
Some(InstructionReply::UpgradeRegions(UpgradeRegionsReply::new(
replies,
)))
}
}
@@ -164,15 +203,15 @@ mod tests {
let reply = UpgradeRegionsHandler
.handle(
&handler_context,
UpgradeRegion {
vec![UpgradeRegion {
region_id,
replay_timeout,
..Default::default()
},
}],
)
.await;
let reply = reply.unwrap().expect_upgrade_region_reply();
let reply = &reply.unwrap().expect_upgrade_regions_reply()[0];
assert!(!reply.exists);
assert!(reply.error.is_none());
}
@@ -201,15 +240,15 @@ mod tests {
let reply = UpgradeRegionsHandler
.handle(
&handler_context,
UpgradeRegion {
vec![UpgradeRegion {
region_id,
replay_timeout,
..Default::default()
},
}],
)
.await;
let reply = reply.unwrap().expect_upgrade_region_reply();
let reply = &reply.unwrap().expect_upgrade_regions_reply()[0];
assert!(reply.ready);
assert!(reply.exists);
assert!(reply.error.is_none());
@@ -239,15 +278,15 @@ mod tests {
let reply = UpgradeRegionsHandler
.handle(
&handler_context,
UpgradeRegion {
vec![UpgradeRegion {
region_id,
replay_timeout,
..Default::default()
},
}],
)
.await;
let reply = reply.unwrap().expect_upgrade_region_reply();
let reply = &reply.unwrap().expect_upgrade_regions_reply()[0];
assert!(!reply.ready);
assert!(reply.exists);
assert!(reply.error.is_none());
@@ -280,15 +319,15 @@ mod tests {
let reply = UpgradeRegionsHandler
.handle(
&handler_context,
UpgradeRegion {
vec![UpgradeRegion {
region_id,
replay_timeout,
..Default::default()
},
}],
)
.await;
let reply = reply.unwrap().expect_upgrade_region_reply();
let reply = &reply.unwrap().expect_upgrade_regions_reply()[0];
assert!(!reply.ready);
assert!(reply.exists);
assert!(reply.error.is_none());
@@ -298,17 +337,17 @@ mod tests {
let reply = UpgradeRegionsHandler
.handle(
&handler_context,
UpgradeRegion {
vec![UpgradeRegion {
region_id,
replay_timeout: Some(Duration::from_millis(500)),
..Default::default()
},
}],
)
.await;
// Must less than 300 ms.
assert!(timer.elapsed().as_millis() < 300);
let reply = reply.unwrap().expect_upgrade_region_reply();
let reply = &reply.unwrap().expect_upgrade_regions_reply()[0];
assert!(reply.ready);
assert!(reply.exists);
assert!(reply.error.is_none());
@@ -339,15 +378,15 @@ mod tests {
let reply = UpgradeRegionsHandler
.handle(
&handler_context,
UpgradeRegion {
vec![UpgradeRegion {
region_id,
..Default::default()
},
}],
)
.await;
// It didn't wait for handle returns; it had no idea about the error.
let reply = reply.unwrap().expect_upgrade_region_reply();
let reply = &reply.unwrap().expect_upgrade_regions_reply()[0];
assert!(!reply.ready);
assert!(reply.exists);
assert!(reply.error.is_none());
@@ -355,18 +394,18 @@ mod tests {
let reply = UpgradeRegionsHandler
.handle(
&handler_context,
UpgradeRegion {
vec![UpgradeRegion {
region_id,
replay_timeout: Some(Duration::from_millis(200)),
..Default::default()
},
}],
)
.await;
let reply = reply.unwrap().expect_upgrade_region_reply();
let reply = &reply.unwrap().expect_upgrade_regions_reply()[0];
assert!(!reply.ready);
assert!(reply.exists);
assert!(reply.error.is_some());
assert!(reply.error.unwrap().contains("mock_error"));
assert!(reply.error.as_ref().unwrap().contains("mock_error"));
}
}

View File

@@ -23,7 +23,7 @@ use api::v1::query_request::Query;
use api::v1::{CreateTableExpr, QueryRequest};
use client::{Client, Database};
use common_error::ext::{BoxedError, ErrorExt};
use common_grpc::channel_manager::{ChannelConfig, ChannelManager};
use common_grpc::channel_manager::{ChannelConfig, ChannelManager, load_tls_config};
use common_meta::cluster::{NodeInfo, NodeInfoKey, Role};
use common_meta::peer::Peer;
use common_meta::rpc::store::RangeRequest;
@@ -123,12 +123,10 @@ impl FrontendClient {
let cfg = ChannelConfig::new()
.connect_timeout(batch_opts.grpc_conn_timeout)
.timeout(batch_opts.query_timeout);
if let Some(tls) = &batch_opts.frontend_tls {
let cfg = cfg.client_tls_config(tls.clone());
ChannelManager::with_tls_config(cfg).context(InvalidClientConfigSnafu)?
} else {
ChannelManager::with_config(cfg)
}
let tls_config = load_tls_config(batch_opts.frontend_tls.as_ref())
.context(InvalidClientConfigSnafu)?;
ChannelManager::with_config(cfg, tls_config)
},
auth,
query,

View File

@@ -36,7 +36,7 @@ async fn run() {
.timeout(Duration::from_secs(3))
.connect_timeout(Duration::from_secs(5))
.tcp_nodelay(true);
let channel_manager = ChannelManager::with_config(config);
let channel_manager = ChannelManager::with_config(config, None);
let mut meta_client = MetaClientBuilder::datanode_default_options(id)
.channel_manager(channel_manager)
.build();

View File

@@ -101,7 +101,7 @@ pub async fn create_meta_client(
if let MetaClientType::Frontend = client_type {
let ddl_config = base_config.clone().timeout(meta_client_options.ddl_timeout);
builder = builder.ddl_channel_manager(ChannelManager::with_config(ddl_config));
builder = builder.ddl_channel_manager(ChannelManager::with_config(ddl_config, None));
if let Some(plugins) = plugins {
let region_follower = plugins.get::<RegionFollowerClientRef>();
if let Some(region_follower) = region_follower {
@@ -112,8 +112,8 @@ pub async fn create_meta_client(
}
builder = builder
.channel_manager(ChannelManager::with_config(base_config))
.heartbeat_channel_manager(ChannelManager::with_config(heartbeat_config));
.channel_manager(ChannelManager::with_config(base_config, None))
.heartbeat_channel_manager(ChannelManager::with_config(heartbeat_config, None));
let mut meta_client = builder.build();

View File

@@ -134,7 +134,7 @@ pub async fn mock(
.timeout(Duration::from_secs(10))
.connect_timeout(Duration::from_secs(10))
.tcp_nodelay(true);
let channel_manager = ChannelManager::with_config(config);
let channel_manager = ChannelManager::with_config(config, None);
// Move client to an option so we can _move_ the inner value
// on the first attempt to connect. All other attempts will fail.

View File

@@ -17,7 +17,9 @@ use std::time::Duration;
use api::v1::meta::MailboxMessage;
use common_meta::ddl::utils::parse_region_wal_options;
use common_meta::instruction::{Instruction, InstructionReply, UpgradeRegion, UpgradeRegionReply};
use common_meta::instruction::{
Instruction, InstructionReply, UpgradeRegion, UpgradeRegionReply, UpgradeRegionsReply,
};
use common_meta::lock_key::RemoteWalLock;
use common_meta::wal_options_allocator::extract_topic_from_wal_options;
use common_procedure::{Context as ProcedureContext, Status};
@@ -131,7 +133,7 @@ impl UpgradeCandidateRegion {
None
};
let upgrade_instruction = Instruction::UpgradeRegion(
let upgrade_instruction = Instruction::UpgradeRegions(vec![
UpgradeRegion {
region_id,
last_entry_id,
@@ -143,7 +145,7 @@ impl UpgradeCandidateRegion {
}
.with_replay_entry_id(checkpoint.map(|c| c.entry_id))
.with_metadata_replay_entry_id(checkpoint.and_then(|c| c.metadata_entry_id)),
);
]);
Ok(upgrade_instruction)
}
@@ -193,11 +195,7 @@ impl UpgradeCandidateRegion {
match receiver.await {
Ok(msg) => {
let reply = HeartbeatMailbox::json_reply(&msg)?;
let InstructionReply::UpgradeRegion(UpgradeRegionReply {
ready,
exists,
error,
}) = reply
let InstructionReply::UpgradeRegions(UpgradeRegionsReply { replies }) = reply
else {
return error::UnexpectedInstructionReplySnafu {
mailbox_message: msg.to_string(),
@@ -205,6 +203,13 @@ impl UpgradeCandidateRegion {
}
.fail();
};
// TODO(weny): handle multiple replies.
let UpgradeRegionReply {
ready,
exists,
error,
..
} = &replies[0];
// Notes: The order of handling is important.
if error.is_some() {

View File

@@ -18,7 +18,7 @@ use api::v1::meta::mailbox_message::Payload;
use api::v1::meta::{HeartbeatResponse, MailboxMessage};
use common_meta::instruction::{
DowngradeRegionReply, DowngradeRegionsReply, FlushRegionReply, InstructionReply, SimpleReply,
UpgradeRegionReply,
UpgradeRegionReply, UpgradeRegionsReply,
};
use common_meta::key::TableMetadataManagerRef;
use common_meta::key::table_route::TableRouteValue;
@@ -212,11 +212,14 @@ pub fn new_upgrade_region_reply(
to: "meta".to_string(),
timestamp_millis: current_time_millis(),
payload: Some(Payload::Json(
serde_json::to_string(&InstructionReply::UpgradeRegion(UpgradeRegionReply {
ready,
exists,
error,
}))
serde_json::to_string(&InstructionReply::UpgradeRegions(
UpgradeRegionsReply::single(UpgradeRegionReply {
region_id: RegionId::new(0, 0),
ready,
exists,
error,
}),
))
.unwrap(),
)),
}

View File

@@ -46,7 +46,9 @@ use store_api::region_engine::{
RegionStatistic, SetRegionRoleStateResponse, SetRegionRoleStateSuccess,
SettableRegionRoleState, SyncManifestResponse,
};
use store_api::region_request::{BatchRegionDdlRequest, RegionOpenRequest, RegionRequest};
use store_api::region_request::{
BatchRegionDdlRequest, RegionCatchupRequest, RegionOpenRequest, RegionRequest,
};
use store_api::storage::{RegionId, ScanRequest, SequenceNumber};
use crate::config::EngineConfig;
@@ -142,6 +144,17 @@ impl RegionEngine for MetricEngine {
.map_err(BoxedError::new)
}
async fn handle_batch_catchup_requests(
&self,
parallelism: usize,
requests: Vec<(RegionId, RegionCatchupRequest)>,
) -> Result<BatchResponses, BoxedError> {
self.inner
.handle_batch_catchup_requests(parallelism, requests)
.await
.map_err(BoxedError::new)
}
async fn handle_batch_ddl_requests(
&self,
batch_request: BatchRegionDdlRequest,

View File

@@ -12,9 +12,12 @@
// See the License for the specific language governing permissions and
// limitations under the License.
use std::collections::HashMap;
use common_error::ext::BoxedError;
use common_telemetry::debug;
use snafu::{OptionExt, ResultExt};
use store_api::region_engine::RegionEngine;
use store_api::region_engine::{BatchResponses, RegionEngine};
use store_api::region_request::{
AffectedRows, RegionCatchupRequest, RegionRequest, ReplayCheckpoint,
};
@@ -22,11 +25,101 @@ use store_api::storage::RegionId;
use crate::engine::MetricEngineInner;
use crate::error::{
MitoCatchupOperationSnafu, PhysicalRegionNotFoundSnafu, Result, UnsupportedRegionRequestSnafu,
BatchCatchupMitoRegionSnafu, MitoCatchupOperationSnafu, PhysicalRegionNotFoundSnafu, Result,
UnsupportedRegionRequestSnafu,
};
use crate::utils;
impl MetricEngineInner {
pub async fn handle_batch_catchup_requests(
&self,
parallelism: usize,
requests: Vec<(RegionId, RegionCatchupRequest)>,
) -> Result<BatchResponses> {
let mut all_requests = Vec::with_capacity(requests.len() * 2);
let mut physical_region_options_list = Vec::with_capacity(requests.len());
for (region_id, req) in requests {
let metadata_region_id = utils::to_metadata_region_id(region_id);
let data_region_id = utils::to_data_region_id(region_id);
let physical_region_options = *self
.state
.read()
.unwrap()
.physical_region_states()
.get(&data_region_id)
.context(PhysicalRegionNotFoundSnafu {
region_id: data_region_id,
})?
.options();
physical_region_options_list.push((data_region_id, physical_region_options));
all_requests.push((
metadata_region_id,
RegionCatchupRequest {
set_writable: req.set_writable,
entry_id: req.metadata_entry_id,
metadata_entry_id: None,
location_id: req.location_id,
checkpoint: req.checkpoint.map(|c| ReplayCheckpoint {
entry_id: c.metadata_entry_id.unwrap_or_default(),
metadata_entry_id: None,
}),
},
));
all_requests.push((
data_region_id,
RegionCatchupRequest {
set_writable: req.set_writable,
entry_id: req.entry_id,
metadata_entry_id: None,
location_id: req.location_id,
checkpoint: req.checkpoint.map(|c| ReplayCheckpoint {
entry_id: c.entry_id,
metadata_entry_id: None,
}),
},
));
}
let mut results = self
.mito
.handle_batch_catchup_requests(parallelism, all_requests)
.await
.context(BatchCatchupMitoRegionSnafu {})?
.into_iter()
.collect::<HashMap<_, _>>();
let mut responses = Vec::with_capacity(physical_region_options_list.len());
for (physical_region_id, physical_region_options) in physical_region_options_list {
let metadata_region_id = utils::to_metadata_region_id(physical_region_id);
let data_region_id = utils::to_data_region_id(physical_region_id);
let metadata_region_result = results.remove(&metadata_region_id);
let data_region_result = results.remove(&data_region_id);
// Pass the optional `metadata_region_result` and `data_region_result` to
// `recover_physical_region_with_results`. This function handles errors for each
// catchup physical region request, allowing the process to continue with the
// remaining regions even if some requests fail.
let response = self
.recover_physical_region_with_results(
metadata_region_result,
data_region_result,
physical_region_id,
physical_region_options,
// Note: We intentionally dont close the region if recovery fails.
// Closing it here might confuse the region server since it links RegionIds to Engines.
// If recovery didnt succeed, the region should stay open.
false,
)
.await
.map_err(BoxedError::new);
responses.push((physical_region_id, response));
}
Ok(responses)
}
pub async fn catchup_region(
&self,
region_id: RegionId,

View File

@@ -119,6 +119,7 @@ mod tests {
.index_file_path
.map(|path| path.replace(&e.file_id, "<file_id>"));
e.file_id = "<file_id>".to_string();
e.index_file_id = e.index_file_id.map(|_| "<index_file_id>".to_string());
format!("\n{:?}", e)
})
.sorted()
@@ -127,12 +128,12 @@ mod tests {
assert_eq!(
debug_format,
r#"
ManifestSstEntry { table_dir: "test_metric_region/", region_id: 47244640257(11, 1), table_id: 11, region_number: 1, region_group: 0, region_sequence: 1, file_id: "<file_id>", level: 0, file_path: "test_metric_region/11_0000000001/data/<file_id>.parquet", file_size: 3173, index_file_path: Some("test_metric_region/11_0000000001/data/index/<file_id>.puffin"), index_file_size: Some(235), num_rows: 10, num_row_groups: 1, num_series: Some(1), min_ts: 0::Millisecond, max_ts: 9::Millisecond, sequence: Some(20), origin_region_id: 47244640257(11, 1), node_id: None, visible: true }
ManifestSstEntry { table_dir: "test_metric_region/", region_id: 47244640258(11, 2), table_id: 11, region_number: 2, region_group: 0, region_sequence: 2, file_id: "<file_id>", level: 0, file_path: "test_metric_region/11_0000000002/data/<file_id>.parquet", file_size: 3173, index_file_path: Some("test_metric_region/11_0000000002/data/index/<file_id>.puffin"), index_file_size: Some(235), num_rows: 10, num_row_groups: 1, num_series: Some(1), min_ts: 0::Millisecond, max_ts: 9::Millisecond, sequence: Some(10), origin_region_id: 47244640258(11, 2), node_id: None, visible: true }
ManifestSstEntry { table_dir: "test_metric_region/", region_id: 47261417473(11, 16777217), table_id: 11, region_number: 16777217, region_group: 1, region_sequence: 1, file_id: "<file_id>", level: 0, file_path: "test_metric_region/11_0000000001/metadata/<file_id>.parquet", file_size: 3505, index_file_path: None, index_file_size: None, num_rows: 8, num_row_groups: 1, num_series: Some(8), min_ts: 0::Millisecond, max_ts: 0::Millisecond, sequence: Some(8), origin_region_id: 47261417473(11, 16777217), node_id: None, visible: true }
ManifestSstEntry { table_dir: "test_metric_region/", region_id: 47261417474(11, 16777218), table_id: 11, region_number: 16777218, region_group: 1, region_sequence: 2, file_id: "<file_id>", level: 0, file_path: "test_metric_region/11_0000000002/metadata/<file_id>.parquet", file_size: 3489, index_file_path: None, index_file_size: None, num_rows: 4, num_row_groups: 1, num_series: Some(4), min_ts: 0::Millisecond, max_ts: 0::Millisecond, sequence: Some(4), origin_region_id: 47261417474(11, 16777218), node_id: None, visible: true }
ManifestSstEntry { table_dir: "test_metric_region/", region_id: 94489280554(22, 42), table_id: 22, region_number: 42, region_group: 0, region_sequence: 42, file_id: "<file_id>", level: 0, file_path: "test_metric_region/22_0000000042/data/<file_id>.parquet", file_size: 3173, index_file_path: Some("test_metric_region/22_0000000042/data/index/<file_id>.puffin"), index_file_size: Some(235), num_rows: 10, num_row_groups: 1, num_series: Some(1), min_ts: 0::Millisecond, max_ts: 9::Millisecond, sequence: Some(10), origin_region_id: 94489280554(22, 42), node_id: None, visible: true }
ManifestSstEntry { table_dir: "test_metric_region/", region_id: 94506057770(22, 16777258), table_id: 22, region_number: 16777258, region_group: 1, region_sequence: 42, file_id: "<file_id>", level: 0, file_path: "test_metric_region/22_0000000042/metadata/<file_id>.parquet", file_size: 3489, index_file_path: None, index_file_size: None, num_rows: 4, num_row_groups: 1, num_series: Some(4), min_ts: 0::Millisecond, max_ts: 0::Millisecond, sequence: Some(4), origin_region_id: 94506057770(22, 16777258), node_id: None, visible: true }"#
ManifestSstEntry { table_dir: "test_metric_region/", region_id: 47244640257(11, 1), table_id: 11, region_number: 1, region_group: 0, region_sequence: 1, file_id: "<file_id>", index_file_id: Some("<index_file_id>"), level: 0, file_path: "test_metric_region/11_0000000001/data/<file_id>.parquet", file_size: 3173, index_file_path: Some("test_metric_region/11_0000000001/data/index/<file_id>.puffin"), index_file_size: Some(235), num_rows: 10, num_row_groups: 1, num_series: Some(1), min_ts: 0::Millisecond, max_ts: 9::Millisecond, sequence: Some(20), origin_region_id: 47244640257(11, 1), node_id: None, visible: true }
ManifestSstEntry { table_dir: "test_metric_region/", region_id: 47244640258(11, 2), table_id: 11, region_number: 2, region_group: 0, region_sequence: 2, file_id: "<file_id>", index_file_id: Some("<index_file_id>"), level: 0, file_path: "test_metric_region/11_0000000002/data/<file_id>.parquet", file_size: 3173, index_file_path: Some("test_metric_region/11_0000000002/data/index/<file_id>.puffin"), index_file_size: Some(235), num_rows: 10, num_row_groups: 1, num_series: Some(1), min_ts: 0::Millisecond, max_ts: 9::Millisecond, sequence: Some(10), origin_region_id: 47244640258(11, 2), node_id: None, visible: true }
ManifestSstEntry { table_dir: "test_metric_region/", region_id: 47261417473(11, 16777217), table_id: 11, region_number: 16777217, region_group: 1, region_sequence: 1, file_id: "<file_id>", index_file_id: None, level: 0, file_path: "test_metric_region/11_0000000001/metadata/<file_id>.parquet", file_size: 3505, index_file_path: None, index_file_size: None, num_rows: 8, num_row_groups: 1, num_series: Some(8), min_ts: 0::Millisecond, max_ts: 0::Millisecond, sequence: Some(8), origin_region_id: 47261417473(11, 16777217), node_id: None, visible: true }
ManifestSstEntry { table_dir: "test_metric_region/", region_id: 47261417474(11, 16777218), table_id: 11, region_number: 16777218, region_group: 1, region_sequence: 2, file_id: "<file_id>", index_file_id: None, level: 0, file_path: "test_metric_region/11_0000000002/metadata/<file_id>.parquet", file_size: 3489, index_file_path: None, index_file_size: None, num_rows: 4, num_row_groups: 1, num_series: Some(4), min_ts: 0::Millisecond, max_ts: 0::Millisecond, sequence: Some(4), origin_region_id: 47261417474(11, 16777218), node_id: None, visible: true }
ManifestSstEntry { table_dir: "test_metric_region/", region_id: 94489280554(22, 42), table_id: 22, region_number: 42, region_group: 0, region_sequence: 42, file_id: "<file_id>", index_file_id: Some("<index_file_id>"), level: 0, file_path: "test_metric_region/22_0000000042/data/<file_id>.parquet", file_size: 3173, index_file_path: Some("test_metric_region/22_0000000042/data/index/<file_id>.puffin"), index_file_size: Some(235), num_rows: 10, num_row_groups: 1, num_series: Some(1), min_ts: 0::Millisecond, max_ts: 9::Millisecond, sequence: Some(10), origin_region_id: 94489280554(22, 42), node_id: None, visible: true }
ManifestSstEntry { table_dir: "test_metric_region/", region_id: 94506057770(22, 16777258), table_id: 22, region_number: 16777258, region_group: 1, region_sequence: 42, file_id: "<file_id>", index_file_id: None, level: 0, file_path: "test_metric_region/22_0000000042/metadata/<file_id>.parquet", file_size: 3489, index_file_path: None, index_file_size: None, num_rows: 4, num_row_groups: 1, num_series: Some(4), min_ts: 0::Millisecond, max_ts: 0::Millisecond, sequence: Some(4), origin_region_id: 94506057770(22, 16777258), node_id: None, visible: true }"#
);
// list from storage
let storage_entries = mito

View File

@@ -72,17 +72,19 @@ impl MetricEngineInner {
let metadata_region_id = utils::to_metadata_region_id(physical_region_id);
let data_region_id = utils::to_data_region_id(physical_region_id);
let metadata_region_result = results.remove(&metadata_region_id);
let data_region_result = results.remove(&data_region_id);
let data_region_result: Option<std::result::Result<RegionResponse, BoxedError>> =
results.remove(&data_region_id);
// Pass the optional `metadata_region_result` and `data_region_result` to
// `open_physical_region_with_results`. This function handles errors for each
// `recover_physical_region_with_results`. This function handles errors for each
// open physical region request, allowing the process to continue with the
// remaining regions even if some requests fail.
let response = self
.open_physical_region_with_results(
.recover_physical_region_with_results(
metadata_region_result,
data_region_result,
physical_region_id,
physical_region_options,
true,
)
.await
.map_err(BoxedError::new);
@@ -107,12 +109,13 @@ impl MetricEngineInner {
}
}
async fn open_physical_region_with_results(
pub(crate) async fn recover_physical_region_with_results(
&self,
metadata_region_result: Option<std::result::Result<RegionResponse, BoxedError>>,
data_region_result: Option<std::result::Result<RegionResponse, BoxedError>>,
physical_region_id: RegionId,
physical_region_options: PhysicalRegionOptions,
close_region_on_failure: bool,
) -> Result<RegionResponse> {
let metadata_region_id = utils::to_metadata_region_id(physical_region_id);
let data_region_id = utils::to_data_region_id(physical_region_id);
@@ -136,8 +139,10 @@ impl MetricEngineInner {
.recover_states(physical_region_id, physical_region_options)
.await
{
self.close_physical_region_on_recovery_failure(physical_region_id)
.await;
if close_region_on_failure {
self.close_physical_region_on_recovery_failure(physical_region_id)
.await;
}
return Err(err);
}
Ok(data_region_response)

View File

@@ -50,6 +50,13 @@ pub enum Error {
location: Location,
},
#[snafu(display("Failed to batch catchup mito region"))]
BatchCatchupMitoRegion {
source: BoxedError,
#[snafu(implicit)]
location: Location,
},
#[snafu(display("No open region result for region {}", region_id))]
NoOpenRegionResult {
region_id: RegionId,
@@ -361,7 +368,8 @@ impl ErrorExt for Error {
| MitoFlushOperation { source, .. }
| MitoDeleteOperation { source, .. }
| MitoSyncOperation { source, .. }
| BatchOpenMitoRegion { source, .. } => source.status_code(),
| BatchOpenMitoRegion { source, .. }
| BatchCatchupMitoRegion { source, .. } => source.status_code(),
EncodePrimaryKey { source, .. } => source.status_code(),

View File

@@ -20,12 +20,11 @@ use criterion::{Criterion, black_box, criterion_group, criterion_main};
use datatypes::data_type::ConcreteDataType;
use datatypes::schema::ColumnSchema;
use mito2::memtable::simple_bulk_memtable::SimpleBulkMemtable;
use mito2::memtable::{KeyValues, Memtable, MemtableRanges};
use mito2::memtable::{KeyValues, Memtable, MemtableRanges, RangesOptions};
use mito2::read;
use mito2::read::Source;
use mito2::read::dedup::DedupReader;
use mito2::read::merge::MergeReaderBuilder;
use mito2::read::scan_region::PredicateGroup;
use mito2::region::options::MergeMode;
use mito2::test_util::column_metadata_to_column_schema;
use store_api::metadata::{ColumnMetadata, RegionMetadataBuilder};
@@ -126,9 +125,7 @@ fn create_memtable_with_rows(num_batches: usize) -> SimpleBulkMemtable {
}
async fn flush(mem: &SimpleBulkMemtable) {
let MemtableRanges { ranges, .. } = mem
.ranges(None, PredicateGroup::default(), None, true)
.unwrap();
let MemtableRanges { ranges, .. } = mem.ranges(None, RangesOptions::for_flush()).unwrap();
let mut source = if ranges.len() == 1 {
let only_range = ranges.into_values().next().unwrap();

View File

@@ -213,7 +213,11 @@ impl AccessLayer {
}
/// Deletes a SST file (and its index file if it has one) with given file id.
pub(crate) async fn delete_sst(&self, region_file_id: &RegionFileId) -> Result<()> {
pub(crate) async fn delete_sst(
&self,
region_file_id: &RegionFileId,
index_file_id: &RegionFileId,
) -> Result<()> {
let path = location::sst_file_path(&self.table_dir, *region_file_id, self.path_type);
self.object_store
.delete(&path)
@@ -222,7 +226,7 @@ impl AccessLayer {
file_id: region_file_id.file_id(),
})?;
let path = location::index_file_path(&self.table_dir, *region_file_id, self.path_type);
let path = location::index_file_path(&self.table_dir, *index_file_id, self.path_type);
self.object_store
.delete(&path)
.await

View File

@@ -432,6 +432,7 @@ impl Compactor for DefaultCompactor {
file_size: sst_info.file_size,
available_indexes: sst_info.index_metadata.build_available_indexes(),
index_file_size: sst_info.index_metadata.file_size,
index_file_id: None,
num_rows: sst_info.num_rows as u64,
num_row_groups: sst_info.num_row_groups,
sequence: max_sequence,

View File

@@ -76,6 +76,7 @@ pub fn new_file_handle_with_size_and_sequence(
file_size,
available_indexes: Default::default(),
index_file_size: 0,
index_file_id: None,
num_rows: 0,
num_row_groups: 0,
num_series: 0,

View File

@@ -21,6 +21,8 @@ mod append_mode_test;
#[cfg(test)]
mod basic_test;
#[cfg(test)]
mod batch_catchup_test;
#[cfg(test)]
mod batch_open_test;
#[cfg(test)]
mod bump_committed_sequence_test;
@@ -91,7 +93,7 @@ use snafu::{OptionExt, ResultExt, ensure};
use store_api::ManifestVersion;
use store_api::codec::PrimaryKeyEncoding;
use store_api::logstore::LogStore;
use store_api::logstore::provider::Provider;
use store_api::logstore::provider::{KafkaProvider, Provider};
use store_api::metadata::{ColumnMetadata, RegionMetadataRef};
use store_api::metric_engine_consts::{
MANIFEST_INFO_EXTENSION_KEY, TABLE_COLUMN_METADATA_EXTENSION_KEY,
@@ -100,7 +102,9 @@ use store_api::region_engine::{
BatchResponses, RegionEngine, RegionManifestInfo, RegionRole, RegionScannerRef,
RegionStatistic, SetRegionRoleStateResponse, SettableRegionRoleState, SyncManifestResponse,
};
use store_api::region_request::{AffectedRows, RegionOpenRequest, RegionRequest};
use store_api::region_request::{
AffectedRows, RegionCatchupRequest, RegionOpenRequest, RegionRequest,
};
use store_api::sst_entry::{ManifestSstEntry, PuffinIndexMetaEntry, StorageSstEntry};
use store_api::storage::{FileId, FileRefsManifest, RegionId, ScanRequest, SequenceNumber};
use tokio::sync::{Semaphore, oneshot};
@@ -500,19 +504,21 @@ impl MitoEngine {
return Vec::new();
};
let file_id = match FileId::parse_str(&entry.file_id) {
let Some(index_file_id) = entry.index_file_id.as_ref() else {
return Vec::new();
};
let file_id = match FileId::parse_str(index_file_id) {
Ok(file_id) => file_id,
Err(err) => {
warn!(
err;
"Failed to parse puffin index file id, table_dir: {}, file_id: {}",
entry.table_dir,
entry.file_id
index_file_id
);
return Vec::new();
}
};
let region_file_id = RegionFileId::new(entry.region_id, file_id);
let context = IndexEntryContext {
table_dir: &entry.table_dir,
@@ -522,7 +528,7 @@ impl MitoEngine {
region_number: entry.region_number,
region_group: entry.region_group,
region_sequence: entry.region_sequence,
file_id: &entry.file_id,
file_id: index_file_id,
index_file_size: entry.index_file_size,
node_id,
};
@@ -770,6 +776,122 @@ impl EngineInner {
Ok(responses)
}
async fn catchup_topic_regions(
&self,
provider: Provider,
region_requests: Vec<(RegionId, RegionCatchupRequest)>,
) -> Result<Vec<(RegionId, Result<AffectedRows>)>> {
let now = Instant::now();
let region_ids = region_requests
.iter()
.map(|(region_id, _)| *region_id)
.collect::<Vec<_>>();
let (distributor, entry_receivers) = build_wal_entry_distributor_and_receivers(
provider.clone(),
self.wal_raw_entry_reader.clone(),
&region_ids,
DEFAULT_ENTRY_RECEIVER_BUFFER_SIZE,
);
let mut responses = Vec::with_capacity(region_requests.len());
for ((region_id, request), entry_receiver) in
region_requests.into_iter().zip(entry_receivers)
{
let (request, receiver) =
WorkerRequest::new_catchup_region_request(region_id, request, Some(entry_receiver));
self.workers.submit_to_worker(region_id, request).await?;
responses.push(async move { receiver.await.context(RecvSnafu)? });
}
// Wait for entries distribution.
let distribution =
common_runtime::spawn_global(async move { distributor.distribute().await });
// Wait for worker returns.
let responses = join_all(responses).await;
distribution.await.context(JoinSnafu)??;
let num_failure = responses.iter().filter(|r| r.is_err()).count();
info!(
"Caught up {} regions for topic '{}', failures: {}, elapsed: {:?}",
region_ids.len() - num_failure,
// Safety: provider is kafka provider.
provider.as_kafka_provider().unwrap(),
num_failure,
now.elapsed(),
);
Ok(region_ids.into_iter().zip(responses).collect())
}
async fn handle_batch_catchup_requests(
&self,
parallelism: usize,
requests: Vec<(RegionId, RegionCatchupRequest)>,
) -> Result<Vec<(RegionId, Result<AffectedRows>)>> {
let mut responses = Vec::with_capacity(requests.len());
let mut topic_regions: HashMap<Arc<KafkaProvider>, Vec<_>> = HashMap::new();
let mut remaining_region_requests = vec![];
for (region_id, request) in requests {
match self.workers.get_region(region_id) {
Some(region) => match region.provider.as_kafka_provider() {
Some(provider) => {
topic_regions
.entry(provider.clone())
.or_default()
.push((region_id, request));
}
None => {
remaining_region_requests.push((region_id, request));
}
},
None => responses.push((region_id, RegionNotFoundSnafu { region_id }.fail())),
}
}
let semaphore = Arc::new(Semaphore::new(parallelism));
if !topic_regions.is_empty() {
let mut tasks = Vec::with_capacity(topic_regions.len());
for (provider, region_requests) in topic_regions {
let semaphore_moved = semaphore.clone();
tasks.push(async move {
// Safety: semaphore must exist
let _permit = semaphore_moved.acquire().await.unwrap();
self.catchup_topic_regions(Provider::Kafka(provider), region_requests)
.await
})
}
let r = try_join_all(tasks).await?;
responses.extend(r.into_iter().flatten());
}
if !remaining_region_requests.is_empty() {
let mut tasks = Vec::with_capacity(remaining_region_requests.len());
let mut region_ids = Vec::with_capacity(remaining_region_requests.len());
for (region_id, request) in remaining_region_requests {
let semaphore_moved = semaphore.clone();
region_ids.push(region_id);
tasks.push(async move {
// Safety: semaphore must exist
let _permit = semaphore_moved.acquire().await.unwrap();
let (request, receiver) =
WorkerRequest::new_catchup_region_request(region_id, request, None);
self.workers.submit_to_worker(region_id, request).await?;
receiver.await.context(RecvSnafu)?
})
}
let results = join_all(tasks).await;
responses.extend(region_ids.into_iter().zip(results));
}
Ok(responses)
}
/// Handles [RegionRequest] and return its executed result.
async fn handle_request(
&self,
@@ -912,6 +1034,29 @@ impl RegionEngine for MitoEngine {
.map_err(BoxedError::new)
}
#[tracing::instrument(skip_all)]
async fn handle_batch_catchup_requests(
&self,
parallelism: usize,
requests: Vec<(RegionId, RegionCatchupRequest)>,
) -> Result<BatchResponses, BoxedError> {
self.inner
.handle_batch_catchup_requests(parallelism, requests)
.await
.map(|responses| {
responses
.into_iter()
.map(|(region_id, response)| {
(
region_id,
response.map(RegionResponse::new).map_err(BoxedError::new),
)
})
.collect::<Vec<_>>()
})
.map_err(BoxedError::new)
}
#[tracing::instrument(skip_all)]
async fn handle_request(
&self,

View File

@@ -859,9 +859,9 @@ async fn test_cache_null_primary_key_with_format(flat_format: bool) {
#[tokio::test]
async fn test_list_ssts() {
test_list_ssts_with_format(false, r#"
ManifestSstEntry { table_dir: "test/", region_id: 47244640257(11, 1), table_id: 11, region_number: 1, region_group: 0, region_sequence: 1, file_id: "<file_id>", level: 0, file_path: "test/11_0000000001/<file_id>.parquet", file_size: 2531, index_file_path: Some("test/11_0000000001/index/<file_id>.puffin"), index_file_size: Some(250), num_rows: 10, num_row_groups: 1, num_series: Some(1), min_ts: 0::Millisecond, max_ts: 9000::Millisecond, sequence: Some(10), origin_region_id: 47244640257(11, 1), node_id: None, visible: true }
ManifestSstEntry { table_dir: "test/", region_id: 47244640258(11, 2), table_id: 11, region_number: 2, region_group: 0, region_sequence: 2, file_id: "<file_id>", level: 0, file_path: "test/11_0000000002/<file_id>.parquet", file_size: 2531, index_file_path: Some("test/11_0000000002/index/<file_id>.puffin"), index_file_size: Some(250), num_rows: 10, num_row_groups: 1, num_series: Some(1), min_ts: 0::Millisecond, max_ts: 9000::Millisecond, sequence: Some(10), origin_region_id: 47244640258(11, 2), node_id: None, visible: true }
ManifestSstEntry { table_dir: "test/", region_id: 94489280554(22, 42), table_id: 22, region_number: 42, region_group: 0, region_sequence: 42, file_id: "<file_id>", level: 0, file_path: "test/22_0000000042/<file_id>.parquet", file_size: 2531, index_file_path: Some("test/22_0000000042/index/<file_id>.puffin"), index_file_size: Some(250), num_rows: 10, num_row_groups: 1, num_series: Some(1), min_ts: 0::Millisecond, max_ts: 9000::Millisecond, sequence: Some(10), origin_region_id: 94489280554(22, 42), node_id: None, visible: true }"# ,r#"
ManifestSstEntry { table_dir: "test/", region_id: 47244640257(11, 1), table_id: 11, region_number: 1, region_group: 0, region_sequence: 1, file_id: "<file_id>", index_file_id: Some("<index_file_id>"), level: 0, file_path: "test/11_0000000001/<file_id>.parquet", file_size: 2531, index_file_path: Some("test/11_0000000001/index/<file_id>.puffin"), index_file_size: Some(250), num_rows: 10, num_row_groups: 1, num_series: Some(1), min_ts: 0::Millisecond, max_ts: 9000::Millisecond, sequence: Some(10), origin_region_id: 47244640257(11, 1), node_id: None, visible: true }
ManifestSstEntry { table_dir: "test/", region_id: 47244640258(11, 2), table_id: 11, region_number: 2, region_group: 0, region_sequence: 2, file_id: "<file_id>", index_file_id: Some("<index_file_id>"), level: 0, file_path: "test/11_0000000002/<file_id>.parquet", file_size: 2531, index_file_path: Some("test/11_0000000002/index/<file_id>.puffin"), index_file_size: Some(250), num_rows: 10, num_row_groups: 1, num_series: Some(1), min_ts: 0::Millisecond, max_ts: 9000::Millisecond, sequence: Some(10), origin_region_id: 47244640258(11, 2), node_id: None, visible: true }
ManifestSstEntry { table_dir: "test/", region_id: 94489280554(22, 42), table_id: 22, region_number: 42, region_group: 0, region_sequence: 42, file_id: "<file_id>", index_file_id: Some("<index_file_id>"), level: 0, file_path: "test/22_0000000042/<file_id>.parquet", file_size: 2531, index_file_path: Some("test/22_0000000042/index/<file_id>.puffin"), index_file_size: Some(250), num_rows: 10, num_row_groups: 1, num_series: Some(1), min_ts: 0::Millisecond, max_ts: 9000::Millisecond, sequence: Some(10), origin_region_id: 94489280554(22, 42), node_id: None, visible: true }"# ,r#"
StorageSstEntry { file_path: "test/11_0000000001/<file_id>.parquet", file_size: None, last_modified_ms: None, node_id: None }
StorageSstEntry { file_path: "test/11_0000000001/index/<file_id>.puffin", file_size: None, last_modified_ms: None, node_id: None }
StorageSstEntry { file_path: "test/11_0000000002/<file_id>.parquet", file_size: None, last_modified_ms: None, node_id: None }
@@ -869,9 +869,9 @@ StorageSstEntry { file_path: "test/11_0000000002/index/<file_id>.puffin", file_s
StorageSstEntry { file_path: "test/22_0000000042/<file_id>.parquet", file_size: None, last_modified_ms: None, node_id: None }
StorageSstEntry { file_path: "test/22_0000000042/index/<file_id>.puffin", file_size: None, last_modified_ms: None, node_id: None }"#).await;
test_list_ssts_with_format(true, r#"
ManifestSstEntry { table_dir: "test/", region_id: 47244640257(11, 1), table_id: 11, region_number: 1, region_group: 0, region_sequence: 1, file_id: "<file_id>", level: 0, file_path: "test/11_0000000001/<file_id>.parquet", file_size: 2855, index_file_path: Some("test/11_0000000001/index/<file_id>.puffin"), index_file_size: Some(292), num_rows: 10, num_row_groups: 1, num_series: Some(1), min_ts: 0::Millisecond, max_ts: 9000::Millisecond, sequence: Some(10), origin_region_id: 47244640257(11, 1), node_id: None, visible: true }
ManifestSstEntry { table_dir: "test/", region_id: 47244640258(11, 2), table_id: 11, region_number: 2, region_group: 0, region_sequence: 2, file_id: "<file_id>", level: 0, file_path: "test/11_0000000002/<file_id>.parquet", file_size: 2855, index_file_path: Some("test/11_0000000002/index/<file_id>.puffin"), index_file_size: Some(292), num_rows: 10, num_row_groups: 1, num_series: Some(1), min_ts: 0::Millisecond, max_ts: 9000::Millisecond, sequence: Some(10), origin_region_id: 47244640258(11, 2), node_id: None, visible: true }
ManifestSstEntry { table_dir: "test/", region_id: 94489280554(22, 42), table_id: 22, region_number: 42, region_group: 0, region_sequence: 42, file_id: "<file_id>", level: 0, file_path: "test/22_0000000042/<file_id>.parquet", file_size: 2855, index_file_path: Some("test/22_0000000042/index/<file_id>.puffin"), index_file_size: Some(292), num_rows: 10, num_row_groups: 1, num_series: Some(1), min_ts: 0::Millisecond, max_ts: 9000::Millisecond, sequence: Some(10), origin_region_id: 94489280554(22, 42), node_id: None, visible: true }"#, r#"
ManifestSstEntry { table_dir: "test/", region_id: 47244640257(11, 1), table_id: 11, region_number: 1, region_group: 0, region_sequence: 1, file_id: "<file_id>", index_file_id: Some("<index_file_id>"), level: 0, file_path: "test/11_0000000001/<file_id>.parquet", file_size: 2855, index_file_path: Some("test/11_0000000001/index/<file_id>.puffin"), index_file_size: Some(292), num_rows: 10, num_row_groups: 1, num_series: Some(1), min_ts: 0::Millisecond, max_ts: 9000::Millisecond, sequence: Some(10), origin_region_id: 47244640257(11, 1), node_id: None, visible: true }
ManifestSstEntry { table_dir: "test/", region_id: 47244640258(11, 2), table_id: 11, region_number: 2, region_group: 0, region_sequence: 2, file_id: "<file_id>", index_file_id: Some("<index_file_id>"), level: 0, file_path: "test/11_0000000002/<file_id>.parquet", file_size: 2855, index_file_path: Some("test/11_0000000002/index/<file_id>.puffin"), index_file_size: Some(292), num_rows: 10, num_row_groups: 1, num_series: Some(1), min_ts: 0::Millisecond, max_ts: 9000::Millisecond, sequence: Some(10), origin_region_id: 47244640258(11, 2), node_id: None, visible: true }
ManifestSstEntry { table_dir: "test/", region_id: 94489280554(22, 42), table_id: 22, region_number: 42, region_group: 0, region_sequence: 42, file_id: "<file_id>", index_file_id: Some("<index_file_id>"), level: 0, file_path: "test/22_0000000042/<file_id>.parquet", file_size: 2855, index_file_path: Some("test/22_0000000042/index/<file_id>.puffin"), index_file_size: Some(292), num_rows: 10, num_row_groups: 1, num_series: Some(1), min_ts: 0::Millisecond, max_ts: 9000::Millisecond, sequence: Some(10), origin_region_id: 94489280554(22, 42), node_id: None, visible: true }"#, r#"
StorageSstEntry { file_path: "test/11_0000000001/<file_id>.parquet", file_size: None, last_modified_ms: None, node_id: None }
StorageSstEntry { file_path: "test/11_0000000001/index/<file_id>.puffin", file_size: None, last_modified_ms: None, node_id: None }
StorageSstEntry { file_path: "test/11_0000000002/<file_id>.parquet", file_size: None, last_modified_ms: None, node_id: None }
@@ -943,6 +943,7 @@ async fn test_list_ssts_with_format(
.index_file_path
.map(|p| p.replace(&e.file_id, "<file_id>"));
e.file_id = "<file_id>".to_string();
e.index_file_id = e.index_file_id.map(|_| "<index_file_id>".to_string());
format!("\n{:?}", e)
})
.sorted()

View File

@@ -0,0 +1,239 @@
// 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::HashMap;
use api::v1::Rows;
use common_error::ext::ErrorExt;
use common_error::status_code::StatusCode;
use common_recordbatch::RecordBatches;
use common_wal::options::{KafkaWalOptions, WAL_OPTIONS_KEY, WalOptions};
use rstest::rstest;
use rstest_reuse::apply;
use store_api::region_engine::RegionEngine;
use store_api::region_request::{PathType, RegionCatchupRequest, RegionOpenRequest, RegionRequest};
use store_api::storage::{RegionId, ScanRequest};
use crate::config::MitoConfig;
use crate::engine::MitoEngine;
use crate::test_util::{
CreateRequestBuilder, LogStoreFactory, TestEnv, build_rows, flush_region,
kafka_log_store_factory, prepare_test_for_kafka_log_store, put_rows, rows_schema,
single_kafka_log_store_factory,
};
#[apply(single_kafka_log_store_factory)]
async fn test_batch_catchup(factory: Option<LogStoreFactory>) {
test_batch_catchup_with_format(factory.clone(), false).await;
test_batch_catchup_with_format(factory, true).await;
}
async fn test_batch_catchup_with_format(factory: Option<LogStoreFactory>, flat_format: bool) {
common_telemetry::init_default_ut_logging();
let Some(factory) = factory else {
return;
};
let mut env = TestEnv::with_prefix("catchup-batch-regions")
.await
.with_log_store_factory(factory.clone());
let engine = env
.create_engine(MitoConfig {
default_experimental_flat_format: flat_format,
..Default::default()
})
.await;
let topic = prepare_test_for_kafka_log_store(&factory).await;
// FIXME(weny): change region number to 3.
let num_regions = 2u32;
let table_dir_fn = |region_id| format!("test/{region_id}");
let mut region_schema = HashMap::new();
for id in 1..=num_regions {
let engine = engine.clone();
let topic = topic.clone();
let region_id = RegionId::new(1, id);
let request = CreateRequestBuilder::new()
.table_dir(&table_dir_fn(region_id))
.kafka_topic(topic.clone())
.build();
let column_schemas = rows_schema(&request);
region_schema.insert(region_id, column_schemas);
engine
.handle_request(region_id, RegionRequest::Create(request))
.await
.unwrap();
}
for i in 0..10 {
for region_number in 1..=num_regions {
let region_id = RegionId::new(1, region_number);
let rows = Rows {
schema: region_schema[&region_id].clone(),
rows: build_rows(
(region_number as usize) * 120 + i as usize,
(region_number as usize) * 120 + i as usize + 1,
),
};
put_rows(&engine, region_id, rows).await;
if i % region_number == 0 {
flush_region(&engine, region_id, None).await;
}
}
}
let assert_result = |engine: MitoEngine| async move {
for i in 1..=num_regions {
let region_id = RegionId::new(1, i);
let request = ScanRequest::default();
let stream = engine.scan_to_stream(region_id, request).await.unwrap();
let batches = RecordBatches::try_collect(stream).await.unwrap();
let mut expected = String::new();
expected.push_str(
"+-------+---------+---------------------+\n| tag_0 | field_0 | ts |\n+-------+---------+---------------------+\n",
);
for row in 0..10 {
expected.push_str(&format!(
"| {} | {}.0 | 1970-01-01T00:{:02}:{:02} |\n",
i * 120 + row,
i * 120 + row,
2 * i,
row
));
}
expected.push_str("+-------+---------+---------------------+");
assert_eq!(expected, batches.pretty_print().unwrap());
}
};
assert_result(engine.clone()).await;
// Reopen engine.
let engine = env
.reopen_engine(
engine,
MitoConfig {
default_experimental_flat_format: flat_format,
..Default::default()
},
)
.await;
let mut options = HashMap::new();
if let Some(topic) = &topic {
options.insert(
WAL_OPTIONS_KEY.to_string(),
serde_json::to_string(&WalOptions::Kafka(KafkaWalOptions {
topic: topic.clone(),
}))
.unwrap(),
);
};
let requests = (1..=num_regions)
.map(|id| {
let region_id = RegionId::new(1, id);
(
region_id,
RegionOpenRequest {
engine: String::new(),
table_dir: table_dir_fn(region_id),
options: options.clone(),
skip_wal_replay: true,
path_type: PathType::Bare,
checkpoint: None,
},
)
})
.collect::<Vec<_>>();
let results = engine
.handle_batch_open_requests(4, requests)
.await
.unwrap();
for (_, result) in results {
assert!(result.is_ok());
}
let requests = (1..=num_regions)
.map(|id| {
let region_id = RegionId::new(1, id);
(
region_id,
RegionCatchupRequest {
set_writable: true,
entry_id: None,
metadata_entry_id: None,
location_id: None,
checkpoint: None,
},
)
})
.collect::<Vec<_>>();
let results = engine
.handle_batch_catchup_requests(4, requests)
.await
.unwrap();
for (_, result) in results {
assert!(result.is_ok());
}
assert_result(engine.clone()).await;
}
#[apply(single_kafka_log_store_factory)]
async fn test_batch_catchup_err(factory: Option<LogStoreFactory>) {
test_batch_catchup_err_with_format(factory.clone(), false).await;
test_batch_catchup_err_with_format(factory, true).await;
}
async fn test_batch_catchup_err_with_format(factory: Option<LogStoreFactory>, flat_format: bool) {
common_telemetry::init_default_ut_logging();
let Some(factory) = factory else {
return;
};
let mut env = TestEnv::with_prefix("catchup-regions-err")
.await
.with_log_store_factory(factory.clone());
let engine = env
.create_engine(MitoConfig {
default_experimental_flat_format: flat_format,
..Default::default()
})
.await;
let num_regions = 3u32;
let requests = (1..num_regions)
.map(|id| {
let region_id = RegionId::new(1, id);
(
region_id,
RegionCatchupRequest {
set_writable: true,
entry_id: None,
metadata_entry_id: None,
location_id: None,
checkpoint: None,
},
)
})
.collect::<Vec<_>>();
let results = engine
.handle_batch_catchup_requests(4, requests)
.await
.unwrap();
for (_, result) in results {
assert_eq!(
result.unwrap_err().status_code(),
StatusCode::RegionNotFound
);
}
}

View File

@@ -220,7 +220,8 @@ async fn test_index_build_type_compact() {
.await
.unwrap();
assert!(scanner.num_files() == 2);
assert!(num_of_index_files(&engine, &scanner, region_id).await < 2);
// Compaction is an async task, so it may be finished at this moment.
assert!(num_of_index_files(&engine, &scanner, region_id).await <= 2);
// Wait a while to make sure index build tasks are finished.
listener.wait_stop(5).await; // 4 flush + 1 compaction = some abort + some finish
@@ -229,6 +230,7 @@ async fn test_index_build_type_compact() {
.await
.unwrap();
assert!(scanner.num_files() == 2);
// Index files should be built.
assert!(num_of_index_files(&engine, &scanner, region_id).await == 2);
}

View File

@@ -22,8 +22,10 @@ use store_api::region_request::RegionRequest;
use store_api::storage::{RegionId, ScanRequest};
use crate::config::MitoConfig;
use crate::test_util::batch_util::sort_batches_and_print;
use crate::test_util::{
CreateRequestBuilder, TestEnv, build_rows, flush_region, put_rows, rows_schema,
CreateRequestBuilder, TestEnv, build_delete_rows, build_rows, delete_rows, delete_rows_schema,
flush_region, put_rows, rows_schema,
};
async fn check_prune_row_groups(exprs: Vec<Expr>, expected: &str, flat_format: bool) {
@@ -377,3 +379,99 @@ async fn test_mem_range_prune_with_format(flat_format: bool) {
+-------+---------+---------------------+";
assert_eq!(expected, batches.pretty_print().unwrap());
}
#[tokio::test]
async fn test_scan_filter_field_after_delete() {
test_scan_filter_field_after_delete_with_format(false).await;
test_scan_filter_field_after_delete_with_format(true).await;
}
async fn test_scan_filter_field_after_delete_with_format(flat_format: bool) {
common_telemetry::init_default_ut_logging();
let mut env = TestEnv::new().await;
let engine = env
.create_engine(MitoConfig {
default_experimental_flat_format: flat_format,
..Default::default()
})
.await;
let region_id = RegionId::new(1, 1);
env.get_schema_metadata_manager()
.register_region_table_info(
region_id.table_id(),
"test_table",
"test_catalog",
"test_schema",
None,
env.get_kv_backend(),
)
.await;
let request = CreateRequestBuilder::new()
.insert_option("compaction.type", "twcs")
.build();
let column_schemas = rows_schema(&request);
engine
.handle_request(region_id, RegionRequest::Create(request.clone()))
.await
.unwrap();
// put 1, 2, 3, 4 and flush
put_rows(
&engine,
region_id,
Rows {
schema: column_schemas,
rows: build_rows(1, 5),
},
)
.await;
flush_region(&engine, region_id, None).await;
// delete 2, 3
let delete_schemas = delete_rows_schema(&request);
delete_rows(
&engine,
region_id,
Rows {
schema: delete_schemas,
rows: build_delete_rows(2, 4),
},
)
.await;
// Scans and filter fields, the field should be deleted.
let request = ScanRequest {
filters: vec![col("field_0").eq(lit(3.0f64))],
..Default::default()
};
let stream = engine
.scan_to_stream(region_id, request.clone())
.await
.unwrap();
let batches = RecordBatches::try_collect(stream).await.unwrap();
let expected = "\
+-------+---------+----+
| tag_0 | field_0 | ts |
+-------+---------+----+
+-------+---------+----+";
assert_eq!(
expected,
sort_batches_and_print(&batches, &["tag_0", "field_0", "ts"])
);
// flush delete op
flush_region(&engine, region_id, None).await;
let stream = engine
.scan_to_stream(region_id, request.clone())
.await
.unwrap();
let batches = RecordBatches::try_collect(stream).await.unwrap();
assert_eq!(
expected,
sort_batches_and_print(&batches, &["tag_0", "field_0", "ts"])
);
}

View File

@@ -40,7 +40,9 @@ use crate::error::{
RegionDroppedSnafu, RegionTruncatedSnafu, Result,
};
use crate::manifest::action::{RegionEdit, RegionMetaAction, RegionMetaActionList};
use crate::memtable::{BoxedRecordBatchIterator, EncodedRange, IterBuilder, MemtableRanges};
use crate::memtable::{
BoxedRecordBatchIterator, EncodedRange, IterBuilder, MemtableRanges, RangesOptions,
};
use crate::metrics::{
FLUSH_BYTES_TOTAL, FLUSH_ELAPSED, FLUSH_FAILURE_TOTAL, FLUSH_REQUESTS_TOTAL,
INFLIGHT_FLUSH_COUNT,
@@ -49,7 +51,6 @@ use crate::read::dedup::{DedupReader, LastNonNull, LastRow};
use crate::read::flat_dedup::{FlatDedupIterator, FlatLastNonNull, FlatLastRow};
use crate::read::flat_merge::FlatMergeIterator;
use crate::read::merge::MergeReaderBuilder;
use crate::read::scan_region::PredicateGroup;
use crate::read::{FlatSource, Source};
use crate::region::options::{IndexOptions, MergeMode, RegionOptions};
use crate::region::version::{VersionControlData, VersionControlRef, VersionRef};
@@ -459,7 +460,7 @@ impl RegionFlushTask {
flush_metrics.compact_memtable += compact_cost;
// Sets `for_flush` flag to true.
let mem_ranges = mem.ranges(None, PredicateGroup::default(), None, true)?;
let mem_ranges = mem.ranges(None, RangesOptions::for_flush())?;
let num_mem_ranges = mem_ranges.ranges.len();
let num_mem_rows = mem_ranges.stats.num_rows();
let memtable_id = mem.id();
@@ -637,6 +638,7 @@ impl RegionFlushTask {
file_size: sst_info.file_size,
available_indexes: sst_info.index_metadata.build_available_indexes(),
index_file_size: sst_info.index_metadata.file_size,
index_file_id: None,
num_rows: sst_info.num_rows as u64,
num_row_groups: sst_info.num_row_groups,
sequence: NonZeroU64::new(max_sequence),

View File

@@ -365,7 +365,22 @@ impl LocalGcWorker {
unused_len, region_id
);
self.delete_files(region_id, &unused_files).await?;
let file_pairs: Vec<(FileId, FileId)> = unused_files
.iter()
.filter_map(|file_id| {
current_files
.get(file_id)
.map(|meta| (meta.file_id().file_id(), meta.index_file_id().file_id()))
})
.collect();
info!(
"Found {} unused index files to delete for region {}",
file_pairs.len(),
region_id
);
self.delete_files(region_id, &file_pairs).await?;
debug!(
"Successfully deleted {} unused files for region {}",
@@ -375,7 +390,7 @@ impl LocalGcWorker {
Ok(unused_files)
}
async fn delete_files(&self, region_id: RegionId, file_ids: &[FileId]) -> Result<()> {
async fn delete_files(&self, region_id: RegionId, file_ids: &[(FileId, FileId)]) -> Result<()> {
delete_files(
region_id,
file_ids,

View File

@@ -265,6 +265,7 @@ async fn checkpoint_with_different_compression_types() {
file_size: 1024000,
available_indexes: Default::default(),
index_file_size: 0,
index_file_id: None,
num_rows: 0,
num_row_groups: 0,
sequence: None,
@@ -331,6 +332,7 @@ fn generate_action_lists(num: usize) -> (Vec<FileId>, Vec<RegionMetaActionList>)
file_size: 1024000,
available_indexes: Default::default(),
index_file_size: 0,
index_file_id: None,
num_rows: 0,
num_row_groups: 0,
sequence: None,

View File

@@ -44,6 +44,7 @@ use crate::region::options::{MemtableOptions, MergeMode, RegionOptions};
use crate::sst::FormatType;
use crate::sst::file::FileTimeRange;
use crate::sst::parquet::SstInfo;
use crate::sst::parquet::file_range::PreFilterMode;
mod builder;
pub mod bulk;
@@ -73,6 +74,63 @@ pub enum MemtableConfig {
TimeSeries,
}
/// Options for querying ranges from a memtable.
#[derive(Clone)]
pub struct RangesOptions {
/// Whether the ranges are being queried for flush.
pub for_flush: bool,
/// Mode to pre-filter columns in ranges.
pub pre_filter_mode: PreFilterMode,
/// Predicate to filter the data.
pub predicate: PredicateGroup,
/// Sequence range to filter the data.
pub sequence: Option<SequenceRange>,
}
impl Default for RangesOptions {
fn default() -> Self {
Self {
for_flush: false,
pre_filter_mode: PreFilterMode::All,
predicate: PredicateGroup::default(),
sequence: None,
}
}
}
impl RangesOptions {
/// Creates a new [RangesOptions] for flushing.
pub fn for_flush() -> Self {
Self {
for_flush: true,
pre_filter_mode: PreFilterMode::All,
predicate: PredicateGroup::default(),
sequence: None,
}
}
/// Sets the pre-filter mode.
#[must_use]
pub fn with_pre_filter_mode(mut self, pre_filter_mode: PreFilterMode) -> Self {
self.pre_filter_mode = pre_filter_mode;
self
}
/// Sets the predicate.
#[must_use]
pub fn with_predicate(mut self, predicate: PredicateGroup) -> Self {
self.predicate = predicate;
self
}
/// Sets the sequence range.
#[must_use]
pub fn with_sequence(mut self, sequence: Option<SequenceRange>) -> Self {
self.sequence = sequence;
self
}
}
#[derive(Debug, Default, Clone)]
pub struct MemtableStats {
/// The estimated bytes allocated by this memtable from heap.
@@ -191,14 +249,11 @@ pub trait Memtable: Send + Sync + fmt::Debug {
/// Returns the ranges in the memtable.
///
/// The `for_flush` flag is true if the flush job calls this method for flush.
/// The returned map contains the range id and the range after applying the predicate.
fn ranges(
&self,
projection: Option<&[ColumnId]>,
predicate: PredicateGroup,
sequence: Option<SequenceRange>,
for_flush: bool,
options: RangesOptions,
) -> Result<MemtableRanges>;
/// Returns true if the memtable is empty.

View File

@@ -42,7 +42,7 @@ use crate::memtable::stats::WriteMetrics;
use crate::memtable::{
AllocTracker, BoxedBatchIterator, BoxedRecordBatchIterator, EncodedBulkPart, EncodedRange,
IterBuilder, KeyValues, MemScanMetrics, Memtable, MemtableBuilder, MemtableId, MemtableRange,
MemtableRangeContext, MemtableRanges, MemtableRef, MemtableStats, PredicateGroup,
MemtableRangeContext, MemtableRanges, MemtableRef, MemtableStats, RangesOptions,
};
use crate::read::flat_dedup::{FlatDedupIterator, FlatLastNonNull, FlatLastRow};
use crate::read::flat_merge::FlatMergeIterator;
@@ -243,7 +243,6 @@ pub struct BulkMemtable {
max_sequence: AtomicU64,
num_rows: AtomicUsize,
/// Cached flat SST arrow schema for memtable compaction.
#[allow(dead_code)]
flat_arrow_schema: SchemaRef,
/// Compactor for merging bulk parts
compactor: Arc<Mutex<MemtableCompactor>>,
@@ -331,19 +330,20 @@ impl Memtable for BulkMemtable {
fn ranges(
&self,
projection: Option<&[ColumnId]>,
predicate: PredicateGroup,
sequence: Option<SequenceRange>,
for_flush: bool,
options: RangesOptions,
) -> Result<MemtableRanges> {
let predicate = options.predicate;
let sequence = options.sequence;
let mut ranges = BTreeMap::new();
let mut range_id = 0;
// TODO(yingwen): Filter ranges by sequence.
let context = Arc::new(BulkIterContext::new(
let context = Arc::new(BulkIterContext::new_with_pre_filter_mode(
self.metadata.clone(),
projection,
predicate.predicate().cloned(),
for_flush,
options.for_flush,
options.pre_filter_mode,
)?);
// Adds ranges for regular parts and encoded parts
@@ -637,7 +637,6 @@ impl IterBuilder for BulkRangeIterBuilder {
/// Iterator builder for encoded bulk range
struct EncodedBulkRangeIterBuilder {
#[allow(dead_code)]
file_id: FileId,
part: EncodedBulkPart,
context: Arc<BulkIterContext>,
@@ -679,7 +678,6 @@ impl IterBuilder for EncodedBulkRangeIterBuilder {
struct BulkPartWrapper {
part: BulkPart,
/// The unique file id for this part in memtable.
#[allow(dead_code)]
file_id: FileId,
/// Whether this part is currently being merged.
merging: bool,
@@ -688,7 +686,6 @@ struct BulkPartWrapper {
struct EncodedPartWrapper {
part: EncodedBulkPart,
/// The unique file id for this part in memtable.
#[allow(dead_code)]
file_id: FileId,
/// Whether this part is currently being merged.
merging: bool,
@@ -1192,7 +1189,12 @@ mod tests {
assert_eq!(3000, max_ts.value());
let predicate_group = PredicateGroup::new(&metadata, &[]).unwrap();
let ranges = memtable.ranges(None, predicate_group, None, false).unwrap();
let ranges = memtable
.ranges(
None,
RangesOptions::default().with_predicate(predicate_group),
)
.unwrap();
assert_eq!(3, ranges.ranges.len());
assert_eq!(5, ranges.stats.num_rows);
@@ -1234,7 +1236,10 @@ mod tests {
let projection = vec![4u32];
let predicate_group = PredicateGroup::new(&metadata, &[]).unwrap();
let ranges = memtable
.ranges(Some(&projection), predicate_group, None, false)
.ranges(
Some(&projection),
RangesOptions::default().with_predicate(predicate_group),
)
.unwrap();
assert_eq!(1, ranges.ranges.len());
@@ -1350,7 +1355,12 @@ mod tests {
}
let predicate_group = PredicateGroup::new(&metadata, &[]).unwrap();
let ranges = memtable.ranges(None, predicate_group, None, false).unwrap();
let ranges = memtable
.ranges(
None,
RangesOptions::default().with_predicate(predicate_group),
)
.unwrap();
assert_eq!(3, ranges.ranges.len());
assert_eq!(5, ranges.stats.num_rows);
@@ -1383,7 +1393,12 @@ mod tests {
let predicate_group = PredicateGroup::new(&metadata, &[]).unwrap();
let sequence_filter = Some(SequenceRange::LtEq { max: 400 }); // Filters out rows with sequence > 400
let ranges = memtable
.ranges(None, predicate_group, sequence_filter, false)
.ranges(
None,
RangesOptions::default()
.with_predicate(predicate_group)
.with_sequence(sequence_filter),
)
.unwrap();
assert_eq!(1, ranges.ranges.len());
@@ -1415,7 +1430,12 @@ mod tests {
memtable.compact(false).unwrap();
let predicate_group = PredicateGroup::new(&metadata, &[]).unwrap();
let ranges = memtable.ranges(None, predicate_group, None, false).unwrap();
let ranges = memtable
.ranges(
None,
RangesOptions::default().with_predicate(predicate_group),
)
.unwrap();
// Should have ranges for both bulk parts and encoded parts
assert_eq!(3, ranges.ranges.len());

View File

@@ -24,7 +24,7 @@ use store_api::storage::ColumnId;
use table::predicate::Predicate;
use crate::error::Result;
use crate::sst::parquet::file_range::RangeBase;
use crate::sst::parquet::file_range::{PreFilterMode, RangeBase};
use crate::sst::parquet::flat_format::FlatReadFormat;
use crate::sst::parquet::format::ReadFormat;
use crate::sst::parquet::reader::SimpleFilterContext;
@@ -43,6 +43,22 @@ impl BulkIterContext {
projection: Option<&[ColumnId]>,
predicate: Option<Predicate>,
skip_auto_convert: bool,
) -> Result<Self> {
Self::new_with_pre_filter_mode(
region_metadata,
projection,
predicate,
skip_auto_convert,
PreFilterMode::All,
)
}
pub fn new_with_pre_filter_mode(
region_metadata: RegionMetadataRef,
projection: Option<&[ColumnId]>,
predicate: Option<Predicate>,
skip_auto_convert: bool,
pre_filter_mode: PreFilterMode,
) -> Result<Self> {
let codec = build_primary_key_codec(&region_metadata);
@@ -73,17 +89,23 @@ impl BulkIterContext {
codec,
// we don't need to compat batch since all batch in memtable have the same schema.
compat_batch: None,
pre_filter_mode,
},
predicate,
})
}
/// Prunes row groups by stats.
pub(crate) fn row_groups_to_read(&self, file_meta: &Arc<ParquetMetaData>) -> VecDeque<usize> {
pub(crate) fn row_groups_to_read(
&self,
file_meta: &Arc<ParquetMetaData>,
skip_fields: bool,
) -> VecDeque<usize> {
let region_meta = self.base.read_format.metadata();
let row_groups = file_meta.row_groups();
// expected_metadata is set to None since we always expect region metadata of memtable is up-to-date.
let stats = RowGroupPruningStats::new(row_groups, &self.base.read_format, None);
let stats =
RowGroupPruningStats::new(row_groups, &self.base.read_format, None, skip_fields);
if let Some(predicate) = self.predicate.as_ref() {
predicate
.prune_with_stats(&stats, region_meta.schema.arrow_schema())
@@ -104,4 +126,9 @@ impl BulkIterContext {
pub(crate) fn read_format(&self) -> &ReadFormat {
&self.base.read_format
}
/// Returns the pre-filter mode.
pub(crate) fn pre_filter_mode(&self) -> PreFilterMode {
self.base.pre_filter_mode
}
}

View File

@@ -65,6 +65,7 @@ use crate::memtable::bulk::context::BulkIterContextRef;
use crate::memtable::bulk::part_reader::EncodedBulkPartIter;
use crate::memtable::time_series::{ValueBuilder, Values};
use crate::sst::index::IndexOutput;
use crate::sst::parquet::file_range::{PreFilterMode, row_group_contains_delete};
use crate::sst::parquet::flat_format::primary_key_column_index;
use crate::sst::parquet::format::{PrimaryKeyArray, PrimaryKeyArrayBuilder, ReadFormat};
use crate::sst::parquet::helper::parse_parquet_metadata;
@@ -572,8 +573,13 @@ impl EncodedBulkPart {
context: BulkIterContextRef,
sequence: Option<SequenceRange>,
) -> Result<Option<BoxedRecordBatchIterator>> {
// Compute skip_fields for row group pruning using the same approach as compute_skip_fields in reader.rs.
let skip_fields_for_pruning =
Self::compute_skip_fields(context.pre_filter_mode(), &self.metadata.parquet_metadata);
// use predicate to find row groups to read.
let row_groups_to_read = context.row_groups_to_read(&self.metadata.parquet_metadata);
let row_groups_to_read =
context.row_groups_to_read(&self.metadata.parquet_metadata, skip_fields_for_pruning);
if row_groups_to_read.is_empty() {
// All row groups are filtered.
@@ -589,6 +595,20 @@ impl EncodedBulkPart {
)?;
Ok(Some(Box::new(iter) as BoxedRecordBatchIterator))
}
/// Computes whether to skip field columns based on PreFilterMode.
fn compute_skip_fields(pre_filter_mode: PreFilterMode, parquet_meta: &ParquetMetaData) -> bool {
match pre_filter_mode {
PreFilterMode::All => false,
PreFilterMode::SkipFields => true,
PreFilterMode::SkipFieldsOnDelete => {
// Check if any row group contains delete op
(0..parquet_meta.num_row_groups()).any(|rg_idx| {
row_group_contains_delete(parquet_meta, rg_idx, "memtable").unwrap_or(true)
})
}
}
}
}
#[derive(Debug, Clone)]

View File

@@ -27,6 +27,7 @@ use store_api::storage::SequenceRange;
use crate::error::{self, ComputeArrowSnafu, DecodeArrowRowGroupSnafu};
use crate::memtable::bulk::context::{BulkIterContext, BulkIterContextRef};
use crate::memtable::bulk::row_group_reader::MemtableRowGroupReaderBuilder;
use crate::sst::parquet::file_range::PreFilterMode;
use crate::sst::parquet::flat_format::sequence_column_index;
use crate::sst::parquet::reader::RowGroupReaderContext;
@@ -38,6 +39,8 @@ pub struct EncodedBulkPartIter {
builder: MemtableRowGroupReaderBuilder,
/// Sequence number filter.
sequence: Option<SequenceRange>,
/// Cached skip_fields for current row group.
current_skip_fields: bool,
}
impl EncodedBulkPartIter {
@@ -58,16 +61,22 @@ impl EncodedBulkPartIter {
let builder =
MemtableRowGroupReaderBuilder::try_new(&context, projection_mask, parquet_meta, data)?;
let init_reader = row_groups_to_read
.pop_front()
.map(|first_row_group| builder.build_row_group_reader(first_row_group, None))
.transpose()?;
let (init_reader, current_skip_fields) = match row_groups_to_read.pop_front() {
Some(first_row_group) => {
let skip_fields = builder.compute_skip_fields(&context, first_row_group);
let reader = builder.build_row_group_reader(first_row_group, None)?;
(Some(reader), skip_fields)
}
None => (None, false),
};
Ok(Self {
context,
row_groups_to_read,
current_reader: init_reader,
builder,
sequence,
current_skip_fields,
})
}
@@ -80,19 +89,34 @@ impl EncodedBulkPartIter {
for batch in current {
let batch = batch.context(DecodeArrowRowGroupSnafu)?;
if let Some(batch) = apply_combined_filters(&self.context, &self.sequence, batch)? {
if let Some(batch) = apply_combined_filters(
&self.context,
&self.sequence,
batch,
self.current_skip_fields,
)? {
return Ok(Some(batch));
}
}
// Previous row group exhausted, read next row group
while let Some(next_row_group) = self.row_groups_to_read.pop_front() {
// Compute skip_fields for this row group
self.current_skip_fields = self
.builder
.compute_skip_fields(&self.context, next_row_group);
let next_reader = self.builder.build_row_group_reader(next_row_group, None)?;
let current = self.current_reader.insert(next_reader);
for batch in current {
let batch = batch.context(DecodeArrowRowGroupSnafu)?;
if let Some(batch) = apply_combined_filters(&self.context, &self.sequence, batch)? {
if let Some(batch) = apply_combined_filters(
&self.context,
&self.sequence,
batch,
self.current_skip_fields,
)? {
return Ok(Some(batch));
}
}
@@ -152,8 +176,14 @@ impl BulkPartRecordBatchIter {
// Apply projection first.
let projected_batch = self.apply_projection(record_batch)?;
// Apply combined filtering (both predicate and sequence filters)
// For BulkPartRecordBatchIter, we don't have row group information.
let skip_fields = match self.context.pre_filter_mode() {
PreFilterMode::All => false,
PreFilterMode::SkipFields => true,
PreFilterMode::SkipFieldsOnDelete => true,
};
let Some(filtered_batch) =
apply_combined_filters(&self.context, &self.sequence, projected_batch)?
apply_combined_filters(&self.context, &self.sequence, projected_batch, skip_fields)?
else {
return Ok(None);
};
@@ -181,6 +211,7 @@ fn apply_combined_filters(
context: &BulkIterContext,
sequence: &Option<SequenceRange>,
record_batch: RecordBatch,
skip_fields: bool,
) -> error::Result<Option<RecordBatch>> {
// Converts the format to the flat format first.
let format = context.read_format().as_flat().unwrap();
@@ -191,7 +222,9 @@ fn apply_combined_filters(
// First, apply predicate filters using the shared method.
if !context.base.filters.is_empty() {
let predicate_mask = context.base.compute_filter_mask_flat(&record_batch)?;
let predicate_mask = context
.base
.compute_filter_mask_flat(&record_batch, skip_fields)?;
// If predicate filters out the entire batch, return None early
let Some(mask) = predicate_mask else {
return Ok(None);

View File

@@ -169,4 +169,23 @@ impl MemtableRowGroupReaderBuilder {
)
.context(ReadDataPartSnafu)
}
/// Computes whether to skip field filters for a specific row group based on PreFilterMode.
pub(crate) fn compute_skip_fields(
&self,
context: &BulkIterContextRef,
row_group_idx: usize,
) -> bool {
use crate::sst::parquet::file_range::{PreFilterMode, row_group_contains_delete};
match context.pre_filter_mode() {
PreFilterMode::All => false,
PreFilterMode::SkipFields => true,
PreFilterMode::SkipFieldsOnDelete => {
// Check if this specific row group contains delete op
row_group_contains_delete(&self.parquet_metadata, row_group_idx, "memtable")
.unwrap_or(true)
}
}
}
}

View File

@@ -44,7 +44,7 @@ use crate::memtable::stats::WriteMetrics;
use crate::memtable::{
AllocTracker, BoxedBatchIterator, IterBuilder, KeyValues, MemScanMetrics, Memtable,
MemtableBuilder, MemtableId, MemtableRange, MemtableRangeContext, MemtableRanges, MemtableRef,
MemtableStats, PredicateGroup,
MemtableStats, RangesOptions,
};
use crate::region::options::MergeMode;
@@ -190,10 +190,10 @@ impl Memtable for PartitionTreeMemtable {
fn ranges(
&self,
projection: Option<&[ColumnId]>,
predicate: PredicateGroup,
sequence: Option<SequenceRange>,
_for_flush: bool,
options: RangesOptions,
) -> Result<MemtableRanges> {
let predicate = options.predicate;
let sequence = options.sequence;
let projection = projection.map(|ids| ids.to_vec());
let builder = Box::new(PartitionTreeIterBuilder {
tree: self.tree.clone(),

View File

@@ -27,7 +27,7 @@ use mito_codec::key_values::KeyValue;
use rayon::prelude::*;
use snafu::{OptionExt, ResultExt};
use store_api::metadata::RegionMetadataRef;
use store_api::storage::{ColumnId, SequenceRange};
use store_api::storage::ColumnId;
use crate::flush::WriteBufferManagerRef;
use crate::memtable::bulk::part::BulkPart;
@@ -35,12 +35,11 @@ use crate::memtable::stats::WriteMetrics;
use crate::memtable::time_series::Series;
use crate::memtable::{
AllocTracker, BoxedBatchIterator, IterBuilder, KeyValues, MemScanMetrics, Memtable, MemtableId,
MemtableRange, MemtableRangeContext, MemtableRanges, MemtableRef, MemtableStats,
MemtableRange, MemtableRangeContext, MemtableRanges, MemtableRef, MemtableStats, RangesOptions,
};
use crate::metrics::MEMTABLE_ACTIVE_SERIES_COUNT;
use crate::read::Batch;
use crate::read::dedup::LastNonNullIter;
use crate::read::scan_region::PredicateGroup;
use crate::region::options::MergeMode;
use crate::{error, metrics};
@@ -223,7 +222,7 @@ impl Memtable for SimpleBulkMemtable {
&self,
projection: Option<&[ColumnId]>,
_predicate: Option<table::predicate::Predicate>,
sequence: Option<SequenceRange>,
sequence: Option<store_api::storage::SequenceRange>,
) -> error::Result<BoxedBatchIterator> {
let iter = self.create_iter(projection, sequence)?.build(None)?;
@@ -238,10 +237,10 @@ impl Memtable for SimpleBulkMemtable {
fn ranges(
&self,
projection: Option<&[ColumnId]>,
predicate: PredicateGroup,
sequence: Option<SequenceRange>,
_for_flush: bool,
options: RangesOptions,
) -> error::Result<MemtableRanges> {
let predicate = options.predicate;
let sequence = options.sequence;
let start_time = Instant::now();
let projection = Arc::new(self.build_projection(projection));
let values = self.series.read().unwrap().read_to_values();
@@ -412,7 +411,7 @@ mod tests {
use datatypes::value::Value;
use datatypes::vectors::TimestampMillisecondVector;
use store_api::metadata::{ColumnMetadata, RegionMetadataBuilder};
use store_api::storage::{RegionId, SequenceNumber};
use store_api::storage::{RegionId, SequenceNumber, SequenceRange};
use super::*;
use crate::read;
@@ -617,9 +616,7 @@ mod tests {
let kv = kvs.iter().next().unwrap();
memtable.write_one(kv).unwrap();
let ranges = memtable
.ranges(None, PredicateGroup::default(), None, false)
.unwrap();
let ranges = memtable.ranges(None, RangesOptions::default()).unwrap();
let mut source = vec![];
for r in ranges.ranges.values() {
source.push(Source::Iter(r.build_iter().unwrap()));
@@ -651,9 +648,7 @@ mod tests {
memtable.write_one(kv).unwrap();
memtable.freeze().unwrap();
let ranges = memtable
.ranges(None, PredicateGroup::default(), None, false)
.unwrap();
let ranges = memtable.ranges(None, RangesOptions::default()).unwrap();
let mut source = vec![];
for r in ranges.ranges.values() {
source.push(Source::Iter(r.build_iter().unwrap()));
@@ -694,9 +689,7 @@ mod tests {
memtable.write_one(kvs.iter().next().unwrap()).unwrap();
memtable.freeze().unwrap();
let ranges = memtable
.ranges(None, PredicateGroup::default(), None, false)
.unwrap();
let ranges = memtable.ranges(None, RangesOptions::default()).unwrap();
assert_eq!(ranges.ranges.len(), 1);
let range = ranges.ranges.into_values().next().unwrap();
let mut reader = range.context.builder.build(None).unwrap();
@@ -910,9 +903,8 @@ mod tests {
raw_data: None,
})
.unwrap();
let MemtableRanges { ranges, .. } = memtable
.ranges(None, PredicateGroup::default(), None, false)
.unwrap();
let MemtableRanges { ranges, .. } =
memtable.ranges(None, RangesOptions::default()).unwrap();
let mut source = if ranges.len() == 1 {
let only_range = ranges.into_values().next().unwrap();
Source::Iter(only_range.build_iter().unwrap())

View File

@@ -53,7 +53,7 @@ use crate::memtable::stats::WriteMetrics;
use crate::memtable::{
AllocTracker, BoxedBatchIterator, IterBuilder, KeyValues, MemScanMetrics, Memtable,
MemtableBuilder, MemtableId, MemtableRange, MemtableRangeContext, MemtableRanges, MemtableRef,
MemtableStats, PredicateGroup,
MemtableStats, RangesOptions,
};
use crate::metrics::{
MEMTABLE_ACTIVE_FIELD_BUILDER_COUNT, MEMTABLE_ACTIVE_SERIES_COUNT, READ_ROWS_TOTAL,
@@ -303,10 +303,10 @@ impl Memtable for TimeSeriesMemtable {
fn ranges(
&self,
projection: Option<&[ColumnId]>,
predicate: PredicateGroup,
sequence: Option<SequenceRange>,
_for_flush: bool,
options: RangesOptions,
) -> Result<MemtableRanges> {
let predicate = options.predicate;
let sequence = options.sequence;
let projection = if let Some(projection) = projection {
projection.iter().copied().collect()
} else {

View File

@@ -39,7 +39,6 @@ use crate::sst::{
///
/// This mapper support duplicate and unsorted projection indices.
/// The output schema is determined by the projection indices.
#[allow(dead_code)]
pub struct FlatProjectionMapper {
/// Metadata of the region.
metadata: RegionMetadataRef,

View File

@@ -49,33 +49,40 @@ pub struct PruneReader {
context: FileRangeContextRef,
source: Source,
metrics: ReaderMetrics,
/// Whether to skip field filters for this row group.
skip_fields: bool,
}
impl PruneReader {
pub(crate) fn new_with_row_group_reader(
ctx: FileRangeContextRef,
reader: RowGroupReader,
skip_fields: bool,
) -> Self {
Self {
context: ctx,
source: Source::RowGroup(reader),
metrics: Default::default(),
skip_fields,
}
}
pub(crate) fn new_with_last_row_reader(
ctx: FileRangeContextRef,
reader: RowGroupLastRowCachedReader,
skip_fields: bool,
) -> Self {
Self {
context: ctx,
source: Source::LastRow(reader),
metrics: Default::default(),
skip_fields,
}
}
pub(crate) fn reset_source(&mut self, source: Source) {
pub(crate) fn reset_source(&mut self, source: Source, skip_fields: bool) {
self.source = source;
self.skip_fields = skip_fields;
}
/// Merge metrics with the inner reader and return the merged metrics.
@@ -117,7 +124,7 @@ impl PruneReader {
}
let num_rows_before_filter = batch.num_rows();
let Some(batch_filtered) = self.context.precise_filter(batch)? else {
let Some(batch_filtered) = self.context.precise_filter(batch, self.skip_fields)? else {
// the entire batch is filtered out
self.metrics.filter_metrics.rows_precise_filtered += num_rows_before_filter;
return Ok(None);
@@ -257,17 +264,21 @@ pub struct FlatPruneReader {
context: FileRangeContextRef,
source: FlatSource,
metrics: ReaderMetrics,
/// Whether to skip field filters for this row group.
skip_fields: bool,
}
impl FlatPruneReader {
pub(crate) fn new_with_row_group_reader(
ctx: FileRangeContextRef,
reader: FlatRowGroupReader,
skip_fields: bool,
) -> Self {
Self {
context: ctx,
source: FlatSource::RowGroup(reader),
metrics: Default::default(),
skip_fields,
}
}
@@ -309,7 +320,10 @@ impl FlatPruneReader {
}
let num_rows_before_filter = record_batch.num_rows();
let Some(filtered_batch) = self.context.precise_filter_flat(record_batch)? else {
let Some(filtered_batch) = self
.context
.precise_filter_flat(record_batch, self.skip_fields)?
else {
// the entire batch is filtered out
self.metrics.filter_metrics.rows_precise_filtered += num_rows_before_filter;
return Ok(None);

View File

@@ -48,7 +48,7 @@ use crate::config::{DEFAULT_MAX_CONCURRENT_SCAN_FILES, DEFAULT_SCAN_CHANNEL_SIZE
use crate::error::{InvalidPartitionExprSnafu, Result};
#[cfg(feature = "enterprise")]
use crate::extension::{BoxedExtensionRange, BoxedExtensionRangeProvider};
use crate::memtable::MemtableRange;
use crate::memtable::{MemtableRange, RangesOptions};
use crate::metrics::READ_SST_COUNT;
use crate::read::compat::{self, CompatBatch, FlatCompatBatch, PrimaryKeyCompatBatch};
use crate::read::projection::ProjectionMapper;
@@ -68,6 +68,7 @@ use crate::sst::index::fulltext_index::applier::FulltextIndexApplierRef;
use crate::sst::index::fulltext_index::applier::builder::FulltextIndexApplierBuilder;
use crate::sst::index::inverted_index::applier::InvertedIndexApplierRef;
use crate::sst::index::inverted_index::applier::builder::InvertedIndexApplierBuilder;
use crate::sst::parquet::file_range::PreFilterMode;
use crate::sst::parquet::reader::ReaderMetrics;
/// Parallel scan channel size for flat format.
@@ -426,6 +427,10 @@ impl ScanRegion {
let memtables = self.version.memtables.list_memtables();
// Skip empty memtables and memtables out of time range.
let mut mem_range_builders = Vec::new();
let filter_mode = pre_filter_mode(
self.version.options.append_mode,
self.version.options.merge_mode(),
);
for m in memtables {
// check if memtable is empty by reading stats.
@@ -439,12 +444,13 @@ impl ScanRegion {
}
let ranges_in_memtable = m.ranges(
Some(mapper.column_ids()),
predicate.clone(),
SequenceRange::new(
self.request.memtable_min_sequence,
self.request.memtable_max_sequence,
),
false,
RangesOptions::default()
.with_predicate(predicate.clone())
.with_sequence(SequenceRange::new(
self.request.memtable_min_sequence,
self.request.memtable_max_sequence,
))
.with_pre_filter_mode(filter_mode),
)?;
mem_range_builders.extend(ranges_in_memtable.ranges.into_values().map(|v| {
// todo: we should add stats to MemtableRange
@@ -466,12 +472,19 @@ impl ScanRegion {
self.version.options.append_mode,
);
// Remove field filters for LastNonNull mode after logging the request.
self.maybe_remove_field_filters();
let inverted_index_applier = self.build_invereted_index_applier();
let bloom_filter_applier = self.build_bloom_filter_applier();
let fulltext_index_applier = self.build_fulltext_index_applier();
let (non_field_filters, field_filters) = self.partition_by_field_filters();
let inverted_index_appliers = [
self.build_invereted_index_applier(&non_field_filters),
self.build_invereted_index_applier(&field_filters),
];
let bloom_filter_appliers = [
self.build_bloom_filter_applier(&non_field_filters),
self.build_bloom_filter_applier(&field_filters),
];
let fulltext_index_appliers = [
self.build_fulltext_index_applier(&non_field_filters),
self.build_fulltext_index_applier(&field_filters),
];
let predicate = PredicateGroup::new(&self.version.metadata, &self.request.filters)?;
if self.flat_format {
@@ -485,9 +498,9 @@ impl ScanRegion {
.with_memtables(mem_range_builders)
.with_files(files)
.with_cache(self.cache_strategy)
.with_inverted_index_applier(inverted_index_applier)
.with_bloom_filter_index_applier(bloom_filter_applier)
.with_fulltext_index_applier(fulltext_index_applier)
.with_inverted_index_appliers(inverted_index_appliers)
.with_bloom_filter_index_appliers(bloom_filter_appliers)
.with_fulltext_index_appliers(fulltext_index_appliers)
.with_parallel_scan_channel_size(self.parallel_scan_channel_size)
.with_max_concurrent_scan_files(self.max_concurrent_scan_files)
.with_start_time(self.start_time)
@@ -527,40 +540,34 @@ impl ScanRegion {
build_time_range_predicate(&time_index.column_schema.name, unit, &self.request.filters)
}
/// Remove field filters if the merge mode is [MergeMode::LastNonNull].
fn maybe_remove_field_filters(&mut self) {
if self.version.options.merge_mode() != MergeMode::LastNonNull {
return;
}
// TODO(yingwen): We can ignore field filters only when there are multiple sources in the same time window.
/// Partitions filters into two groups: non-field filters and field filters.
/// Returns `(non_field_filters, field_filters)`.
fn partition_by_field_filters(&self) -> (Vec<Expr>, Vec<Expr>) {
let field_columns = self
.version
.metadata
.field_columns()
.map(|col| &col.column_schema.name)
.collect::<HashSet<_>>();
// Columns in the expr.
let mut columns = HashSet::new();
self.request.filters.retain(|expr| {
self.request.filters.iter().cloned().partition(|expr| {
columns.clear();
// `expr_to_columns` won't return error.
if expr_to_columns(expr, &mut columns).is_err() {
return false;
// If we can't extract columns, treat it as non-field filter
return true;
}
for column in &columns {
if field_columns.contains(&column.name) {
// This expr uses the field column.
return false;
}
}
true
});
// Return true for non-field filters (partition puts true cases in first vec)
!columns
.iter()
.any(|column| field_columns.contains(&column.name))
})
}
/// Use the latest schema to build the inverted index applier.
fn build_invereted_index_applier(&self) -> Option<InvertedIndexApplierRef> {
fn build_invereted_index_applier(&self, filters: &[Expr]) -> Option<InvertedIndexApplierRef> {
if self.ignore_inverted_index {
return None;
}
@@ -588,7 +595,7 @@ impl ScanRegion {
.with_file_cache(file_cache)
.with_inverted_index_cache(inverted_index_cache)
.with_puffin_metadata_cache(puffin_metadata_cache)
.build(&self.request.filters)
.build(filters)
.inspect_err(|err| warn!(err; "Failed to build invereted index applier"))
.ok()
.flatten()
@@ -596,7 +603,7 @@ impl ScanRegion {
}
/// Use the latest schema to build the bloom filter index applier.
fn build_bloom_filter_applier(&self) -> Option<BloomFilterIndexApplierRef> {
fn build_bloom_filter_applier(&self, filters: &[Expr]) -> Option<BloomFilterIndexApplierRef> {
if self.ignore_bloom_filter {
return None;
}
@@ -615,7 +622,7 @@ impl ScanRegion {
.with_file_cache(file_cache)
.with_bloom_filter_index_cache(bloom_filter_index_cache)
.with_puffin_metadata_cache(puffin_metadata_cache)
.build(&self.request.filters)
.build(filters)
.inspect_err(|err| warn!(err; "Failed to build bloom filter index applier"))
.ok()
.flatten()
@@ -623,7 +630,7 @@ impl ScanRegion {
}
/// Use the latest schema to build the fulltext index applier.
fn build_fulltext_index_applier(&self) -> Option<FulltextIndexApplierRef> {
fn build_fulltext_index_applier(&self, filters: &[Expr]) -> Option<FulltextIndexApplierRef> {
if self.ignore_fulltext_index {
return None;
}
@@ -641,7 +648,7 @@ impl ScanRegion {
.with_file_cache(file_cache)
.with_puffin_metadata_cache(puffin_metadata_cache)
.with_bloom_filter_cache(bloom_filter_index_cache)
.build(&self.request.filters)
.build(filters)
.inspect_err(|err| warn!(err; "Failed to build fulltext index applier"))
.ok()
.flatten()
@@ -685,9 +692,9 @@ pub struct ScanInput {
/// Maximum number of SST files to scan concurrently.
pub(crate) max_concurrent_scan_files: usize,
/// Index appliers.
inverted_index_applier: Option<InvertedIndexApplierRef>,
bloom_filter_index_applier: Option<BloomFilterIndexApplierRef>,
fulltext_index_applier: Option<FulltextIndexApplierRef>,
inverted_index_appliers: [Option<InvertedIndexApplierRef>; 2],
bloom_filter_index_appliers: [Option<BloomFilterIndexApplierRef>; 2],
fulltext_index_appliers: [Option<FulltextIndexApplierRef>; 2],
/// Start time of the query.
pub(crate) query_start: Option<Instant>,
/// The region is using append mode.
@@ -724,9 +731,9 @@ impl ScanInput {
ignore_file_not_found: false,
parallel_scan_channel_size: DEFAULT_SCAN_CHANNEL_SIZE,
max_concurrent_scan_files: DEFAULT_MAX_CONCURRENT_SCAN_FILES,
inverted_index_applier: None,
bloom_filter_index_applier: None,
fulltext_index_applier: None,
inverted_index_appliers: [None, None],
bloom_filter_index_appliers: [None, None],
fulltext_index_appliers: [None, None],
query_start: None,
append_mode: false,
filter_deleted: true,
@@ -803,33 +810,33 @@ impl ScanInput {
self
}
/// Sets invereted index applier.
/// Sets inverted index appliers.
#[must_use]
pub(crate) fn with_inverted_index_applier(
pub(crate) fn with_inverted_index_appliers(
mut self,
applier: Option<InvertedIndexApplierRef>,
appliers: [Option<InvertedIndexApplierRef>; 2],
) -> Self {
self.inverted_index_applier = applier;
self.inverted_index_appliers = appliers;
self
}
/// Sets bloom filter applier.
/// Sets bloom filter appliers.
#[must_use]
pub(crate) fn with_bloom_filter_index_applier(
pub(crate) fn with_bloom_filter_index_appliers(
mut self,
applier: Option<BloomFilterIndexApplierRef>,
appliers: [Option<BloomFilterIndexApplierRef>; 2],
) -> Self {
self.bloom_filter_index_applier = applier;
self.bloom_filter_index_appliers = appliers;
self
}
/// Sets fulltext index applier.
/// Sets fulltext index appliers.
#[must_use]
pub(crate) fn with_fulltext_index_applier(
pub(crate) fn with_fulltext_index_appliers(
mut self,
applier: Option<FulltextIndexApplierRef>,
appliers: [Option<FulltextIndexApplierRef>; 2],
) -> Self {
self.fulltext_index_applier = applier;
self.fulltext_index_appliers = appliers;
self
}
@@ -952,18 +959,20 @@ impl ScanInput {
reader_metrics: &mut ReaderMetrics,
) -> Result<FileRangeBuilder> {
let predicate = self.predicate_for_file(file);
let filter_mode = pre_filter_mode(self.append_mode, self.merge_mode);
let res = self
.access_layer
.read_sst(file.clone())
.predicate(predicate)
.projection(Some(self.mapper.column_ids().to_vec()))
.cache(self.cache_strategy.clone())
.inverted_index_applier(self.inverted_index_applier.clone())
.bloom_filter_index_applier(self.bloom_filter_index_applier.clone())
.fulltext_index_applier(self.fulltext_index_applier.clone())
.inverted_index_appliers(self.inverted_index_appliers.clone())
.bloom_filter_index_appliers(self.bloom_filter_index_appliers.clone())
.fulltext_index_appliers(self.fulltext_index_appliers.clone())
.expected_metadata(Some(self.mapper.metadata().clone()))
.flat_format(self.flat_format)
.compaction(self.compaction)
.pre_filter_mode(filter_mode)
.build_reader_input(reader_metrics)
.await;
let (mut file_range_ctx, selection) = match res {
@@ -1155,6 +1164,17 @@ impl ScanInput {
}
}
fn pre_filter_mode(append_mode: bool, merge_mode: MergeMode) -> PreFilterMode {
if append_mode {
return PreFilterMode::All;
}
match merge_mode {
MergeMode::LastRow => PreFilterMode::SkipFieldsOnDelete,
MergeMode::LastNonNull => PreFilterMode::SkipFields,
}
}
/// Context shared by different streams from a scanner.
/// It contains the input and ranges to scan.
pub struct StreamContext {

View File

@@ -657,7 +657,6 @@ pub(crate) fn scan_mem_ranges(
}
/// Scans memtable ranges at `index` using flat format that returns RecordBatch.
#[allow(dead_code)]
pub(crate) fn scan_flat_mem_ranges(
stream_ctx: Arc<StreamContext>,
part_metrics: PartitionMetrics,
@@ -709,7 +708,6 @@ pub(crate) async fn scan_file_ranges(
}
/// Scans file ranges at `index` using flat reader that returns RecordBatch.
#[allow(dead_code)]
pub(crate) async fn scan_flat_file_ranges(
stream_ctx: Arc<StreamContext>,
part_metrics: PartitionMetrics,
@@ -851,7 +849,6 @@ pub(crate) async fn maybe_scan_other_ranges(
}
}
#[allow(dead_code)]
pub(crate) async fn maybe_scan_flat_other_ranges(
context: &Arc<StreamContext>,
index: RowGroupIndex,

View File

@@ -590,11 +590,17 @@ impl MitoRegion {
.map(|meta| {
let region_id = self.region_id;
let origin_region_id = meta.region_id;
let (index_file_path, index_file_size) = if meta.index_file_size > 0 {
let index_file_path = index_file_path(table_dir, meta.file_id(), path_type);
(Some(index_file_path), Some(meta.index_file_size))
let (index_file_id, index_file_path, index_file_size) = if meta.index_file_size > 0
{
let index_file_path =
index_file_path(table_dir, meta.index_file_id(), path_type);
(
Some(meta.index_file_id().file_id().to_string()),
Some(index_file_path),
Some(meta.index_file_size),
)
} else {
(None, None)
(None, None, None)
};
let visible = visible_ssts.contains(&meta.file_id);
ManifestSstEntry {
@@ -605,6 +611,7 @@ impl MitoRegion {
region_group: region_id.region_group(),
region_sequence: region_id.region_sequence(),
file_id: meta.file_id.to_string(),
index_file_id,
level: meta.level,
file_path: sst_file_path(table_dir, meta.file_id(), path_type),
file_size: meta.file_size,

View File

@@ -427,6 +427,7 @@ mod tests {
file_size: 1024,
available_indexes: SmallVec::new(),
index_file_size: 0,
index_file_id: None,
num_rows: 100,
num_row_groups: 1,
sequence: NonZeroU64::new(1),

View File

@@ -39,7 +39,7 @@ use store_api::region_request::{
RegionCatchupRequest, RegionCloseRequest, RegionCompactRequest, RegionCreateRequest,
RegionFlushRequest, RegionOpenRequest, RegionRequest, RegionTruncateRequest,
};
use store_api::storage::RegionId;
use store_api::storage::{FileId, RegionId};
use tokio::sync::oneshot::{self, Receiver, Sender};
use crate::error::{
@@ -602,6 +602,7 @@ pub(crate) enum WorkerRequest {
}
impl WorkerRequest {
/// Creates a new open region request.
pub(crate) fn new_open_region_request(
region_id: RegionId,
request: RegionOpenRequest,
@@ -618,6 +619,21 @@ impl WorkerRequest {
(worker_request, receiver)
}
/// Creates a new catchup region request.
pub(crate) fn new_catchup_region_request(
region_id: RegionId,
request: RegionCatchupRequest,
entry_receiver: Option<WalEntryReceiver>,
) -> (WorkerRequest, Receiver<Result<AffectedRows>>) {
let (sender, receiver) = oneshot::channel();
let worker_request = WorkerRequest::Ddl(SenderDdlRequest {
region_id,
sender: sender.into(),
request: DdlRequest::Catchup((request, entry_receiver)),
});
(worker_request, receiver)
}
/// Converts request from a [RegionRequest].
pub(crate) fn try_from_region_request(
region_id: RegionId,
@@ -701,7 +717,7 @@ impl WorkerRequest {
RegionRequest::Catchup(v) => WorkerRequest::Ddl(SenderDdlRequest {
region_id,
sender: sender.into(),
request: DdlRequest::Catchup(v),
request: DdlRequest::Catchup((v, None)),
}),
RegionRequest::BulkInserts(region_bulk_inserts_request) => WorkerRequest::BulkInserts {
metadata: region_metadata,
@@ -757,7 +773,7 @@ pub(crate) enum DdlRequest {
Compact(RegionCompactRequest),
BuildIndex(RegionBuildIndexRequest),
Truncate(RegionTruncateRequest),
Catchup(RegionCatchupRequest),
Catchup((RegionCatchupRequest, Option<WalEntryReceiver>)),
}
/// Sender and Ddl request.
@@ -780,8 +796,9 @@ pub(crate) enum BackgroundNotify {
FlushFailed(FlushFailed),
/// Index build has finished.
IndexBuildFinished(IndexBuildFinished),
/// Index build has been stopped (aborted or succeeded).
IndexBuildStopped(IndexBuildStopped),
/// Index build has failed.
#[allow(dead_code)]
IndexBuildFailed(IndexBuildFailed),
/// Compaction has finished.
CompactionFinished(CompactionFinished),
@@ -846,10 +863,17 @@ pub(crate) struct IndexBuildFinished {
pub(crate) edit: RegionEdit,
}
/// Notifies an index build job has been stopped.
#[derive(Debug)]
pub(crate) struct IndexBuildStopped {
#[allow(dead_code)]
pub(crate) region_id: RegionId,
pub(crate) file_id: FileId,
}
/// Notifies an index build job has failed.
#[derive(Debug)]
pub(crate) struct IndexBuildFailed {
#[allow(dead_code)]
pub(crate) err: Arc<Error>,
}

View File

@@ -146,6 +146,12 @@ pub struct FileMeta {
pub available_indexes: SmallVec<[IndexType; 4]>,
/// Size of the index file.
pub index_file_size: u64,
/// File ID of the index file.
///
/// When this field is None, it means the index file id is the same as the file id.
/// Only meaningful when index_file_size > 0.
/// Used for rebuilding index files.
pub index_file_id: Option<FileId>,
/// Number of rows in the file.
///
/// For historical reasons, this field might be missing in old files. Thus
@@ -259,6 +265,16 @@ impl FileMeta {
pub fn file_id(&self) -> RegionFileId {
RegionFileId::new(self.region_id, self.file_id)
}
/// Returns the cross-region index file id.
/// If the index file id is not set, returns the file id.
pub fn index_file_id(&self) -> RegionFileId {
if let Some(index_file_id) = self.index_file_id {
RegionFileId::new(self.region_id, index_file_id)
} else {
self.file_id()
}
}
}
/// Handle to a SST file.
@@ -294,6 +310,16 @@ impl FileHandle {
RegionFileId::new(self.inner.meta.region_id, self.inner.meta.file_id)
}
/// Returns the cross-region index file id.
/// If the index file id is not set, returns the file id.
pub fn index_file_id(&self) -> RegionFileId {
if let Some(index_file_id) = self.inner.meta.index_file_id {
RegionFileId::new(self.inner.meta.region_id, index_file_id)
} else {
self.file_id()
}
}
/// Returns the complete file path of the file.
pub fn file_path(&self, table_dir: &str, path_type: PathType) -> String {
location::sst_file_path(table_dir, self.file_id(), path_type)
@@ -379,22 +405,28 @@ impl FileHandleInner {
/// Delete
pub async fn delete_files(
region_id: RegionId,
file_ids: &[FileId],
file_ids: &[(FileId, FileId)],
delete_index: bool,
access_layer: &AccessLayerRef,
cache_manager: &Option<CacheManagerRef>,
) -> crate::error::Result<()> {
// Remove meta of the file from cache.
if let Some(cache) = &cache_manager {
for file_id in file_ids {
for (file_id, _) in file_ids {
cache.remove_parquet_meta_data(RegionFileId::new(region_id, *file_id));
}
}
let mut deleted_files = Vec::with_capacity(file_ids.len());
for file_id in file_ids {
for (file_id, index_file_id) in file_ids {
let region_file_id = RegionFileId::new(region_id, *file_id);
match access_layer.delete_sst(&region_file_id).await {
match access_layer
.delete_sst(
&RegionFileId::new(region_id, *file_id),
&RegionFileId::new(region_id, *index_file_id),
)
.await
{
Ok(_) => {
deleted_files.push(*file_id);
}
@@ -411,14 +443,12 @@ pub async fn delete_files(
deleted_files
);
for file_id in file_ids {
let region_file_id = RegionFileId::new(region_id, *file_id);
for (file_id, index_file_id) in file_ids {
if let Some(write_cache) = cache_manager.as_ref().and_then(|cache| cache.write_cache()) {
// Removes index file from the cache.
if delete_index {
write_cache
.remove(IndexKey::new(region_id, *file_id, FileType::Puffin))
.remove(IndexKey::new(region_id, *index_file_id, FileType::Puffin))
.await;
}
@@ -431,11 +461,11 @@ pub async fn delete_files(
// Purges index content in the stager.
if let Err(e) = access_layer
.puffin_manager_factory()
.purge_stager(region_file_id)
.purge_stager(RegionFileId::new(region_id, *index_file_id))
.await
{
error!(e; "Failed to purge stager with index file, file_id: {}, region: {}",
file_id, region_id);
index_file_id, region_id);
}
}
Ok(())
@@ -459,6 +489,7 @@ mod tests {
file_size: 0,
available_indexes: SmallVec::from_iter([IndexType::InvertedIndex]),
index_file_size: 0,
index_file_id: None,
num_rows: 0,
num_row_groups: 0,
sequence: None,
@@ -505,6 +536,7 @@ mod tests {
file_size: 0,
available_indexes: SmallVec::from_iter([IndexType::InvertedIndex]),
index_file_size: 0,
index_file_id: None,
num_rows: 0,
num_row_groups: 0,
sequence: None,

View File

@@ -128,7 +128,7 @@ impl LocalFilePurger {
if let Err(e) = self.scheduler.schedule(Box::pin(async move {
if let Err(e) = delete_files(
file_meta.region_id,
&[file_meta.file_id],
&[(file_meta.file_id, file_meta.index_file_id().file_id())],
file_meta.exists_index(),
&sst_layer,
&cache_manager,
@@ -233,6 +233,7 @@ mod tests {
file_size: 4096,
available_indexes: Default::default(),
index_file_size: 0,
index_file_id: None,
num_rows: 0,
num_row_groups: 0,
sequence: None,
@@ -300,6 +301,7 @@ mod tests {
file_size: 4096,
available_indexes: SmallVec::from_iter([IndexType::InvertedIndex]),
index_file_size: 4096,
index_file_id: None,
num_rows: 1024,
num_row_groups: 1,
sequence: NonZeroU64::new(4096),

View File

@@ -235,6 +235,7 @@ mod tests {
file_size: 4096,
available_indexes: SmallVec::from_iter([IndexType::InvertedIndex]),
index_file_size: 4096,
index_file_id: None,
num_rows: 1024,
num_row_groups: 1,
sequence: NonZeroU64::new(4096),

View File

@@ -21,11 +21,13 @@ pub mod puffin_manager;
mod statistics;
pub(crate) mod store;
use std::cmp::Ordering;
use std::collections::{BinaryHeap, HashMap, HashSet};
use std::num::NonZeroUsize;
use std::sync::Arc;
use bloom_filter::creator::BloomFilterIndexer;
use common_telemetry::{debug, info, warn};
use common_telemetry::{debug, error, info, warn};
use datatypes::arrow::array::BinaryArray;
use datatypes::arrow::record_batch::RecordBatch;
use mito_codec::index::IndexValuesCodec;
@@ -43,7 +45,10 @@ use crate::access_layer::{AccessLayerRef, FilePathProvider, OperationType, Regio
use crate::cache::file_cache::{FileType, IndexKey};
use crate::cache::write_cache::{UploadTracker, WriteCacheRef};
use crate::config::{BloomFilterConfig, FulltextIndexConfig, InvertedIndexConfig};
use crate::error::{BuildIndexAsyncSnafu, DecodeSnafu, Error, InvalidRecordBatchSnafu, Result};
use crate::error::{
BuildIndexAsyncSnafu, DecodeSnafu, Error, InvalidRecordBatchSnafu, RegionClosedSnafu,
RegionDroppedSnafu, RegionTruncatedSnafu, Result,
};
use crate::manifest::action::{RegionEdit, RegionMetaAction, RegionMetaActionList};
use crate::metrics::INDEX_CREATE_MEMORY_USAGE;
use crate::read::{Batch, BatchReader};
@@ -51,7 +56,8 @@ use crate::region::options::IndexOptions;
use crate::region::version::VersionControlRef;
use crate::region::{ManifestContextRef, RegionLeaderState};
use crate::request::{
BackgroundNotify, IndexBuildFailed, IndexBuildFinished, WorkerRequest, WorkerRequestWithTime,
BackgroundNotify, IndexBuildFailed, IndexBuildFinished, IndexBuildStopped, WorkerRequest,
WorkerRequestWithTime,
};
use crate::schedule::scheduler::{Job, SchedulerRef};
use crate::sst::file::{FileHandle, FileMeta, IndexType, RegionFileId};
@@ -410,7 +416,7 @@ impl IndexerBuilderImpl {
}
/// Type of an index build task.
#[derive(Debug, Clone, PartialEq, IntoStaticStr)]
#[derive(Debug, Clone, IntoStaticStr)]
pub enum IndexBuildType {
/// Build index when schema change.
SchemaChange,
@@ -426,6 +432,16 @@ impl IndexBuildType {
fn as_str(&self) -> &'static str {
self.into()
}
// Higher value means higher priority.
fn priority(&self) -> u8 {
match self {
IndexBuildType::Manual => 3,
IndexBuildType::SchemaChange => 2,
IndexBuildType::Flush => 1,
IndexBuildType::Compact => 0,
}
}
}
impl From<OperationType> for IndexBuildType {
@@ -447,6 +463,7 @@ pub enum IndexBuildOutcome {
/// Mpsc output result sender.
pub type ResultMpscSender = Sender<Result<IndexBuildOutcome>>;
#[derive(Clone)]
pub struct IndexBuildTask {
/// The file meta to build index for.
pub file_meta: FileMeta,
@@ -465,14 +482,24 @@ pub struct IndexBuildTask {
pub(crate) result_sender: ResultMpscSender,
}
impl std::fmt::Debug for IndexBuildTask {
fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
f.debug_struct("IndexBuildTask")
.field("region_id", &self.file_meta.region_id)
.field("file_id", &self.file_meta.file_id)
.field("reason", &self.reason)
.finish()
}
}
impl IndexBuildTask {
/// Notify the caller the job is success.
pub async fn on_success(&mut self, outcome: IndexBuildOutcome) {
pub async fn on_success(&self, outcome: IndexBuildOutcome) {
let _ = self.result_sender.send(Ok(outcome)).await;
}
/// Send index build error to waiter.
pub async fn on_failure(&mut self, err: Arc<Error>) {
pub async fn on_failure(&self, err: Arc<Error>) {
let _ = self
.result_sender
.send(Err(err.clone()).context(BuildIndexAsyncSnafu {
@@ -503,7 +530,18 @@ impl IndexBuildTask {
);
self.on_failure(e.into()).await
}
}
let worker_request = WorkerRequest::Background {
region_id: self.file_meta.region_id,
notify: BackgroundNotify::IndexBuildStopped(IndexBuildStopped {
region_id: self.file_meta.region_id,
file_id: self.file_meta.file_id,
}),
};
let _ = self
.request_sender
.send(WorkerRequestWithTime::new(worker_request))
.await;
}
// Checks if the SST file still exists in object store and version to avoid conflict with compaction.
@@ -542,7 +580,13 @@ impl IndexBuildTask {
&mut self,
version_control: VersionControlRef,
) -> Result<IndexBuildOutcome> {
let mut indexer = self.indexer_builder.build(self.file_meta.file_id).await;
let index_file_id = if self.file_meta.index_file_size > 0 {
// Generate new file ID if index file exists to avoid overwrite.
FileId::random()
} else {
self.file_meta.file_id
};
let mut indexer = self.indexer_builder.build(index_file_id).await;
// Check SST file existence before building index to avoid failure of parquet reader.
if !self.check_sst_file_exists(&version_control).await {
@@ -602,9 +646,10 @@ impl IndexBuildTask {
}
// Upload index file if write cache is enabled.
self.maybe_upload_index_file(index_output.clone()).await?;
self.maybe_upload_index_file(index_output.clone(), index_file_id)
.await?;
let worker_request = match self.update_manifest(index_output).await {
let worker_request = match self.update_manifest(index_output, index_file_id).await {
Ok(edit) => {
let index_build_finished = IndexBuildFinished {
region_id: self.file_meta.region_id,
@@ -632,14 +677,18 @@ impl IndexBuildTask {
Ok(IndexBuildOutcome::Finished)
}
async fn maybe_upload_index_file(&self, output: IndexOutput) -> Result<()> {
async fn maybe_upload_index_file(
&self,
output: IndexOutput,
index_file_id: FileId,
) -> Result<()> {
if let Some(write_cache) = &self.write_cache {
let file_id = self.file_meta.file_id;
let region_id = self.file_meta.region_id;
let remote_store = self.access_layer.object_store();
let mut upload_tracker = UploadTracker::new(region_id);
let mut err = None;
let puffin_key = IndexKey::new(region_id, file_id, FileType::Puffin);
let puffin_key = IndexKey::new(region_id, index_file_id, FileType::Puffin);
let puffin_path = RegionFilePathFactory::new(
self.access_layer.table_dir().to_string(),
self.access_layer.path_type(),
@@ -673,9 +722,14 @@ impl IndexBuildTask {
Ok(())
}
async fn update_manifest(&mut self, output: IndexOutput) -> Result<RegionEdit> {
async fn update_manifest(
&mut self,
output: IndexOutput,
index_file_id: FileId,
) -> Result<RegionEdit> {
self.file_meta.available_indexes = output.build_available_indexes();
self.file_meta.index_file_size = output.file_size;
self.file_meta.index_file_id = Some(index_file_id);
let edit = RegionEdit {
files_to_add: vec![self.file_meta.clone()],
files_to_remove: vec![],
@@ -701,26 +755,205 @@ impl IndexBuildTask {
}
}
#[derive(Clone)]
pub struct IndexBuildScheduler {
scheduler: SchedulerRef,
impl PartialEq for IndexBuildTask {
fn eq(&self, other: &Self) -> bool {
self.reason.priority() == other.reason.priority()
}
}
impl IndexBuildScheduler {
pub fn new(scheduler: SchedulerRef) -> Self {
IndexBuildScheduler { scheduler }
impl Eq for IndexBuildTask {}
impl PartialOrd for IndexBuildTask {
fn partial_cmp(&self, other: &Self) -> Option<Ordering> {
Some(self.cmp(other))
}
}
impl Ord for IndexBuildTask {
fn cmp(&self, other: &Self) -> Ordering {
self.reason.priority().cmp(&other.reason.priority())
}
}
/// Tracks the index build status of a region scheduled by the [IndexBuildScheduler].
pub struct IndexBuildStatus {
pub region_id: RegionId,
pub building_files: HashSet<FileId>,
pub pending_tasks: BinaryHeap<IndexBuildTask>,
}
impl IndexBuildStatus {
pub fn new(region_id: RegionId) -> Self {
IndexBuildStatus {
region_id,
building_files: HashSet::new(),
pending_tasks: BinaryHeap::new(),
}
}
pub(crate) fn schedule_build(
async fn on_failure(self, err: Arc<Error>) {
for task in self.pending_tasks {
task.on_failure(err.clone()).await;
}
}
}
pub struct IndexBuildScheduler {
/// Background job scheduler.
scheduler: SchedulerRef,
/// Tracks regions need to build index.
region_status: HashMap<RegionId, IndexBuildStatus>,
/// Limit of files allowed to build index concurrently for a region.
files_limit: usize,
}
/// Manager background index build tasks of a worker.
impl IndexBuildScheduler {
pub fn new(scheduler: SchedulerRef, files_limit: usize) -> Self {
IndexBuildScheduler {
scheduler,
region_status: HashMap::new(),
files_limit,
}
}
pub(crate) async fn schedule_build(
&mut self,
version_control: &VersionControlRef,
task: IndexBuildTask,
) -> Result<()> {
// We should clone version control to expand the lifetime.
let job = task.into_index_build_job(version_control.clone());
self.scheduler.schedule(job)?;
let status = self
.region_status
.entry(task.file_meta.region_id)
.or_insert_with(|| IndexBuildStatus::new(task.file_meta.region_id));
if status.building_files.contains(&task.file_meta.file_id) {
let region_file_id =
RegionFileId::new(task.file_meta.region_id, task.file_meta.file_id);
debug!(
"Aborting index build task since index is already being built for region file {:?}",
region_file_id
);
task.on_success(IndexBuildOutcome::Aborted(format!(
"Index is already being built for region file {:?}",
region_file_id
)))
.await;
task.listener.on_index_build_abort(region_file_id).await;
return Ok(());
}
status.pending_tasks.push(task);
self.schedule_next_build_batch(version_control);
Ok(())
}
/// Schedule tasks until reaching the files limit or no more tasks.
fn schedule_next_build_batch(&mut self, version_control: &VersionControlRef) {
let mut building_count = 0;
for status in self.region_status.values() {
building_count += status.building_files.len();
}
while building_count < self.files_limit {
if let Some(task) = self.find_next_task() {
let region_id = task.file_meta.region_id;
let file_id = task.file_meta.file_id;
let job = task.into_index_build_job(version_control.clone());
if self.scheduler.schedule(job).is_ok() {
if let Some(status) = self.region_status.get_mut(&region_id) {
status.building_files.insert(file_id);
building_count += 1;
status
.pending_tasks
.retain(|t| t.file_meta.file_id != file_id);
} else {
error!(
"Region status not found when scheduling index build task, region: {}",
region_id
);
}
} else {
error!(
"Failed to schedule index build job, region: {}, file_id: {}",
region_id, file_id
);
}
} else {
// No more tasks to schedule.
break;
}
}
}
/// Find the next task which has the highest priority to run.
fn find_next_task(&self) -> Option<IndexBuildTask> {
self.region_status
.iter()
.filter_map(|(_, status)| status.pending_tasks.peek())
.max()
.cloned()
}
pub(crate) fn on_task_stopped(
&mut self,
region_id: RegionId,
file_id: FileId,
version_control: &VersionControlRef,
) {
if let Some(status) = self.region_status.get_mut(&region_id) {
status.building_files.remove(&file_id);
if status.building_files.is_empty() && status.pending_tasks.is_empty() {
// No more tasks for this region, remove it.
self.region_status.remove(&region_id);
}
}
self.schedule_next_build_batch(version_control);
}
pub(crate) async fn on_failure(&mut self, region_id: RegionId, err: Arc<Error>) {
error!(
err; "Index build scheduler encountered failure for region {}, removing all pending tasks.",
region_id
);
let Some(status) = self.region_status.remove(&region_id) else {
return;
};
status.on_failure(err).await;
}
/// Notifies the scheduler that the region is dropped.
pub(crate) async fn on_region_dropped(&mut self, region_id: RegionId) {
self.remove_region_on_failure(
region_id,
Arc::new(RegionDroppedSnafu { region_id }.build()),
)
.await;
}
/// Notifies the scheduler that the region is closed.
pub(crate) async fn on_region_closed(&mut self, region_id: RegionId) {
self.remove_region_on_failure(region_id, Arc::new(RegionClosedSnafu { region_id }.build()))
.await;
}
/// Notifies the scheduler that the region is truncated.
pub(crate) async fn on_region_truncated(&mut self, region_id: RegionId) {
self.remove_region_on_failure(
region_id,
Arc::new(RegionTruncatedSnafu { region_id }.build()),
)
.await;
}
async fn remove_region_on_failure(&mut self, region_id: RegionId, err: Arc<Error>) {
let Some(status) = self.region_status.remove(&region_id) else {
return;
};
status.on_failure(err).await;
}
}
/// Decodes primary keys from a flat format RecordBatch.
@@ -1192,7 +1425,7 @@ mod tests {
let env = SchedulerEnv::new().await;
let (tx, _rx) = mpsc::channel(4);
let (result_tx, mut result_rx) = mpsc::channel::<Result<IndexBuildOutcome>>(4);
let mut scheduler = env.mock_index_build_scheduler();
let mut scheduler = env.mock_index_build_scheduler(4);
let metadata = Arc::new(sst_region_metadata());
let manifest_ctx = env.mock_manifest_context(metadata.clone()).await;
let file_purger = Arc::new(NoopFilePurger {});
@@ -1222,7 +1455,10 @@ mod tests {
};
// Schedule the build task and check result.
scheduler.schedule_build(&version_control, task).unwrap();
scheduler
.schedule_build(&version_control, task)
.await
.unwrap();
match result_rx.recv().await.unwrap() {
Ok(outcome) => {
if outcome == IndexBuildOutcome::Finished {
@@ -1236,7 +1472,7 @@ mod tests {
#[tokio::test]
async fn test_index_build_task_sst_exist() {
let env = SchedulerEnv::new().await;
let mut scheduler = env.mock_index_build_scheduler();
let mut scheduler = env.mock_index_build_scheduler(4);
let metadata = Arc::new(sst_region_metadata());
let manifest_ctx = env.mock_manifest_context(metadata.clone()).await;
let region_id = metadata.region_id;
@@ -1272,7 +1508,10 @@ mod tests {
result_sender: result_tx,
};
scheduler.schedule_build(&version_control, task).unwrap();
scheduler
.schedule_build(&version_control, task)
.await
.unwrap();
// The task should finish successfully.
match result_rx.recv().await.unwrap() {
@@ -1304,7 +1543,7 @@ mod tests {
async fn schedule_index_build_task_with_mode(build_mode: IndexBuildMode) {
let env = SchedulerEnv::new().await;
let mut scheduler = env.mock_index_build_scheduler();
let mut scheduler = env.mock_index_build_scheduler(4);
let metadata = Arc::new(sst_region_metadata());
let manifest_ctx = env.mock_manifest_context(metadata.clone()).await;
let file_purger = Arc::new(NoopFilePurger {});
@@ -1340,7 +1579,10 @@ mod tests {
result_sender: result_tx,
};
scheduler.schedule_build(&version_control, task).unwrap();
scheduler
.schedule_build(&version_control, task)
.await
.unwrap();
let puffin_path = location::index_file_path(
env.access_layer.table_dir(),
@@ -1395,7 +1637,7 @@ mod tests {
#[tokio::test]
async fn test_index_build_task_no_index() {
let env = SchedulerEnv::new().await;
let mut scheduler = env.mock_index_build_scheduler();
let mut scheduler = env.mock_index_build_scheduler(4);
let mut metadata = sst_region_metadata();
// Unset indexes in metadata to simulate no index scenario.
metadata.column_metadatas.iter_mut().for_each(|col| {
@@ -1437,7 +1679,10 @@ mod tests {
result_sender: result_tx,
};
scheduler.schedule_build(&version_control, task).unwrap();
scheduler
.schedule_build(&version_control, task)
.await
.unwrap();
// The task should finish successfully.
match result_rx.recv().await.unwrap() {
@@ -1454,7 +1699,7 @@ mod tests {
#[tokio::test]
async fn test_index_build_task_with_write_cache() {
let env = SchedulerEnv::new().await;
let mut scheduler = env.mock_index_build_scheduler();
let mut scheduler = env.mock_index_build_scheduler(4);
let metadata = Arc::new(sst_region_metadata());
let manifest_ctx = env.mock_manifest_context(metadata.clone()).await;
let file_purger = Arc::new(NoopFilePurger {});
@@ -1518,7 +1763,10 @@ mod tests {
result_sender: result_tx,
};
scheduler.schedule_build(&version_control, task).unwrap();
scheduler
.schedule_build(&version_control, task)
.await
.unwrap();
// The task should finish successfully.
match result_rx.recv().await.unwrap() {
@@ -1532,4 +1780,188 @@ mod tests {
let index_key = IndexKey::new(region_id, file_meta.file_id, FileType::Puffin);
assert!(write_cache.file_cache().contains_key(&index_key));
}
async fn create_mock_task_for_schedule(
env: &SchedulerEnv,
file_id: FileId,
region_id: RegionId,
reason: IndexBuildType,
) -> IndexBuildTask {
let metadata = Arc::new(sst_region_metadata());
let manifest_ctx = env.mock_manifest_context(metadata.clone()).await;
let file_purger = Arc::new(NoopFilePurger {});
let indexer_builder = mock_indexer_builder(metadata, env).await;
let (tx, _rx) = mpsc::channel(4);
let (result_tx, _result_rx) = mpsc::channel::<Result<IndexBuildOutcome>>(4);
IndexBuildTask {
file_meta: FileMeta {
region_id,
file_id,
file_size: 100,
..Default::default()
},
reason,
access_layer: env.access_layer.clone(),
listener: WorkerListener::default(),
manifest_ctx,
write_cache: None,
file_purger,
indexer_builder,
request_sender: tx,
result_sender: result_tx,
}
}
#[tokio::test]
async fn test_scheduler_comprehensive() {
let env = SchedulerEnv::new().await;
let mut scheduler = env.mock_index_build_scheduler(2);
let metadata = Arc::new(sst_region_metadata());
let region_id = metadata.region_id;
let file_purger = Arc::new(NoopFilePurger {});
// Prepare multiple files for testing
let file_id1 = FileId::random();
let file_id2 = FileId::random();
let file_id3 = FileId::random();
let file_id4 = FileId::random();
let file_id5 = FileId::random();
let mut files = HashMap::new();
for file_id in [file_id1, file_id2, file_id3, file_id4, file_id5] {
files.insert(
file_id,
FileMeta {
region_id,
file_id,
file_size: 100,
..Default::default()
},
);
}
let version_control = mock_version_control(metadata, file_purger, files).await;
// Test 1: Basic scheduling
let task1 =
create_mock_task_for_schedule(&env, file_id1, region_id, IndexBuildType::Flush).await;
assert!(
scheduler
.schedule_build(&version_control, task1)
.await
.is_ok()
);
assert!(scheduler.region_status.contains_key(&region_id));
let status = scheduler.region_status.get(&region_id).unwrap();
assert_eq!(status.building_files.len(), 1);
assert!(status.building_files.contains(&file_id1));
// Test 2: Duplicate file scheduling (should be skipped)
let task1_dup =
create_mock_task_for_schedule(&env, file_id1, region_id, IndexBuildType::Flush).await;
scheduler
.schedule_build(&version_control, task1_dup)
.await
.unwrap();
let status = scheduler.region_status.get(&region_id).unwrap();
assert_eq!(status.building_files.len(), 1); // Still only one
// Test 3: Fill up to limit (2 building tasks)
let task2 =
create_mock_task_for_schedule(&env, file_id2, region_id, IndexBuildType::Flush).await;
scheduler
.schedule_build(&version_control, task2)
.await
.unwrap();
let status = scheduler.region_status.get(&region_id).unwrap();
assert_eq!(status.building_files.len(), 2); // Reached limit
assert_eq!(status.pending_tasks.len(), 0);
// Test 4: Add tasks with different priorities to pending queue
// Now all new tasks will be pending since we reached the limit
let task3 =
create_mock_task_for_schedule(&env, file_id3, region_id, IndexBuildType::Compact).await;
let task4 =
create_mock_task_for_schedule(&env, file_id4, region_id, IndexBuildType::SchemaChange)
.await;
let task5 =
create_mock_task_for_schedule(&env, file_id5, region_id, IndexBuildType::Manual).await;
scheduler
.schedule_build(&version_control, task3)
.await
.unwrap();
scheduler
.schedule_build(&version_control, task4)
.await
.unwrap();
scheduler
.schedule_build(&version_control, task5)
.await
.unwrap();
let status = scheduler.region_status.get(&region_id).unwrap();
assert_eq!(status.building_files.len(), 2); // Still at limit
assert_eq!(status.pending_tasks.len(), 3); // Three pending
// Test 5: Task completion triggers scheduling next highest priority task (Manual)
scheduler.on_task_stopped(region_id, file_id1, &version_control);
let status = scheduler.region_status.get(&region_id).unwrap();
assert!(!status.building_files.contains(&file_id1));
assert_eq!(status.building_files.len(), 2); // Should schedule next task
assert_eq!(status.pending_tasks.len(), 2); // One less pending
// The highest priority task (Manual) should now be building
assert!(status.building_files.contains(&file_id5));
// Test 6: Complete another task, should schedule SchemaChange (second highest priority)
scheduler.on_task_stopped(region_id, file_id2, &version_control);
let status = scheduler.region_status.get(&region_id).unwrap();
assert_eq!(status.building_files.len(), 2);
assert_eq!(status.pending_tasks.len(), 1); // One less pending
assert!(status.building_files.contains(&file_id4)); // SchemaChange should be building
// Test 7: Complete remaining tasks and cleanup
scheduler.on_task_stopped(region_id, file_id5, &version_control);
scheduler.on_task_stopped(region_id, file_id4, &version_control);
let status = scheduler.region_status.get(&region_id).unwrap();
assert_eq!(status.building_files.len(), 1); // Last task (Compact) should be building
assert_eq!(status.pending_tasks.len(), 0);
assert!(status.building_files.contains(&file_id3));
scheduler.on_task_stopped(region_id, file_id3, &version_control);
// Region should be removed when all tasks complete
assert!(!scheduler.region_status.contains_key(&region_id));
// Test 8: Region dropped with pending tasks
let task6 =
create_mock_task_for_schedule(&env, file_id1, region_id, IndexBuildType::Flush).await;
let task7 =
create_mock_task_for_schedule(&env, file_id2, region_id, IndexBuildType::Flush).await;
let task8 =
create_mock_task_for_schedule(&env, file_id3, region_id, IndexBuildType::Manual).await;
scheduler
.schedule_build(&version_control, task6)
.await
.unwrap();
scheduler
.schedule_build(&version_control, task7)
.await
.unwrap();
scheduler
.schedule_build(&version_control, task8)
.await
.unwrap();
assert!(scheduler.region_status.contains_key(&region_id));
let status = scheduler.region_status.get(&region_id).unwrap();
assert_eq!(status.building_files.len(), 2);
assert_eq!(status.pending_tasks.len(), 1);
scheduler.on_region_dropped(region_id).await;
assert!(!scheduler.region_status.contains_key(&region_id));
}
}

View File

@@ -768,6 +768,7 @@ mod tests {
file_size: info.file_size,
available_indexes: info.index_metadata.build_available_indexes(),
index_file_size: info.index_metadata.file_size,
index_file_id: None,
num_row_groups: info.num_row_groups,
num_rows: info.num_rows as u64,
sequence: None,
@@ -850,8 +851,8 @@ mod tests {
object_store.clone(),
)
.predicate(Some(Predicate::new(preds)))
.inverted_index_applier(inverted_index_applier.clone())
.bloom_filter_index_applier(bloom_filter_applier.clone())
.inverted_index_appliers([inverted_index_applier.clone(), None])
.bloom_filter_index_appliers([bloom_filter_applier.clone(), None])
.cache(CacheStrategy::EnableAll(cache.clone()));
let mut metrics = ReaderMetrics::default();
@@ -906,8 +907,8 @@ mod tests {
object_store.clone(),
)
.predicate(Some(Predicate::new(preds)))
.inverted_index_applier(inverted_index_applier.clone())
.bloom_filter_index_applier(bloom_filter_applier.clone())
.inverted_index_appliers([inverted_index_applier.clone(), None])
.bloom_filter_index_appliers([bloom_filter_applier.clone(), None])
.cache(CacheStrategy::EnableAll(cache.clone()));
let mut metrics = ReaderMetrics::default();
@@ -963,8 +964,8 @@ mod tests {
object_store.clone(),
)
.predicate(Some(Predicate::new(preds)))
.inverted_index_applier(inverted_index_applier.clone())
.bloom_filter_index_applier(bloom_filter_applier.clone())
.inverted_index_appliers([inverted_index_applier.clone(), None])
.bloom_filter_index_appliers([bloom_filter_applier.clone(), None])
.cache(CacheStrategy::EnableAll(cache.clone()));
let mut metrics = ReaderMetrics::default();

View File

@@ -26,6 +26,7 @@ use datatypes::arrow::buffer::BooleanBuffer;
use datatypes::arrow::record_batch::RecordBatch;
use mito_codec::row_converter::{CompositeValues, PrimaryKeyCodec};
use parquet::arrow::arrow_reader::RowSelection;
use parquet::file::metadata::ParquetMetaData;
use snafu::{OptionExt, ResultExt};
use store_api::codec::PrimaryKeyEncoding;
use store_api::storage::{ColumnId, TimeSeriesRowSelector};
@@ -44,6 +45,33 @@ use crate::sst::parquet::format::ReadFormat;
use crate::sst::parquet::reader::{
FlatRowGroupReader, MaybeFilter, RowGroupReader, RowGroupReaderBuilder, SimpleFilterContext,
};
/// Checks if a row group contains delete operations by examining the min value of op_type column.
///
/// Returns `Ok(true)` if the row group contains delete operations, `Ok(false)` if it doesn't,
/// or an error if the statistics are not present or cannot be decoded.
pub(crate) fn row_group_contains_delete(
parquet_meta: &ParquetMetaData,
row_group_index: usize,
file_path: &str,
) -> Result<bool> {
let row_group_metadata = &parquet_meta.row_groups()[row_group_index];
// safety: The last column of SST must be op_type
let column_metadata = &row_group_metadata.columns().last().unwrap();
let stats = column_metadata
.statistics()
.context(StatsNotPresentSnafu { file_path })?;
stats
.min_bytes_opt()
.context(StatsNotPresentSnafu { file_path })?
.try_into()
.map(i32::from_le_bytes)
.map(|min_op_type| min_op_type == OpType::Delete as i32)
.ok()
.context(DecodeStatsSnafu { file_path })
}
/// A range of a parquet SST. Now it is a row group.
/// We can read different file ranges in parallel.
#[derive(Clone)]
@@ -115,6 +143,9 @@ impl FileRange {
false
};
// Compute skip_fields once for this row group
let skip_fields = self.context.should_skip_fields(self.row_group_idx);
let prune_reader = if use_last_row_reader {
// Row group is PUT only, use LastRowReader to skip unnecessary rows.
let reader = RowGroupLastRowCachedReader::new(
@@ -123,12 +154,13 @@ impl FileRange {
self.context.reader_builder.cache_strategy().clone(),
RowGroupReader::new(self.context.clone(), parquet_reader),
);
PruneReader::new_with_last_row_reader(self.context.clone(), reader)
PruneReader::new_with_last_row_reader(self.context.clone(), reader, skip_fields)
} else {
// Row group contains DELETE, fallback to default reader.
PruneReader::new_with_row_group_reader(
self.context.clone(),
RowGroupReader::new(self.context.clone(), parquet_reader),
skip_fields,
)
};
@@ -143,9 +175,15 @@ impl FileRange {
.build(self.row_group_idx, self.row_selection.clone())
.await?;
// Compute skip_fields once for this row group
let skip_fields = self.context.should_skip_fields(self.row_group_idx);
let flat_row_group_reader = FlatRowGroupReader::new(self.context.clone(), parquet_reader);
let flat_prune_reader =
FlatPruneReader::new_with_row_group_reader(self.context.clone(), flat_row_group_reader);
let flat_prune_reader = FlatPruneReader::new_with_row_group_reader(
self.context.clone(),
flat_row_group_reader,
skip_fields,
);
Ok(flat_prune_reader)
}
@@ -178,6 +216,7 @@ impl FileRangeContext {
filters: Vec<SimpleFilterContext>,
read_format: ReadFormat,
codec: Arc<dyn PrimaryKeyCodec>,
pre_filter_mode: PreFilterMode,
) -> Self {
Self {
reader_builder,
@@ -186,6 +225,7 @@ impl FileRangeContext {
read_format,
codec,
compat_batch: None,
pre_filter_mode,
},
}
}
@@ -222,40 +262,50 @@ impl FileRangeContext {
/// TRY THE BEST to perform pushed down predicate precisely on the input batch.
/// Return the filtered batch. If the entire batch is filtered out, return None.
pub(crate) fn precise_filter(&self, input: Batch) -> Result<Option<Batch>> {
self.base.precise_filter(input)
pub(crate) fn precise_filter(&self, input: Batch, skip_fields: bool) -> Result<Option<Batch>> {
self.base.precise_filter(input, skip_fields)
}
/// Filters the input RecordBatch by the pushed down predicate and returns RecordBatch.
pub(crate) fn precise_filter_flat(&self, input: RecordBatch) -> Result<Option<RecordBatch>> {
self.base.precise_filter_flat(input)
pub(crate) fn precise_filter_flat(
&self,
input: RecordBatch,
skip_fields: bool,
) -> Result<Option<RecordBatch>> {
self.base.precise_filter_flat(input, skip_fields)
}
/// Determines whether to skip field filters based on PreFilterMode and row group delete status.
pub(crate) fn should_skip_fields(&self, row_group_idx: usize) -> bool {
match self.base.pre_filter_mode {
PreFilterMode::All => false,
PreFilterMode::SkipFields => true,
PreFilterMode::SkipFieldsOnDelete => {
// Check if this specific row group contains delete op
self.contains_delete(row_group_idx).unwrap_or(true)
}
}
}
//// Decodes parquet metadata and finds if row group contains delete op.
pub(crate) fn contains_delete(&self, row_group_index: usize) -> Result<bool> {
let metadata = self.reader_builder.parquet_metadata();
let row_group_metadata = &metadata.row_groups()[row_group_index];
// safety: The last column of SST must be op_type
let column_metadata = &row_group_metadata.columns().last().unwrap();
let stats = column_metadata.statistics().context(StatsNotPresentSnafu {
file_path: self.reader_builder.file_path(),
})?;
stats
.min_bytes_opt()
.context(StatsNotPresentSnafu {
file_path: self.reader_builder.file_path(),
})?
.try_into()
.map(i32::from_le_bytes)
.map(|min_op_type| min_op_type == OpType::Delete as i32)
.ok()
.context(DecodeStatsSnafu {
file_path: self.reader_builder.file_path(),
})
row_group_contains_delete(metadata, row_group_index, self.reader_builder.file_path())
}
}
/// Mode to pre-filter columns in a range.
#[derive(Debug, Clone, Copy)]
pub enum PreFilterMode {
/// Filters all columns.
All,
/// If the range doesn't contain delete op or doesn't have statistics, filters all columns.
/// Otherwise, skips filtering fields.
SkipFieldsOnDelete,
/// Always skip fields.
SkipFields,
}
/// Common fields for a range to read and filter batches.
pub(crate) struct RangeBase {
/// Filters pushed down.
@@ -266,6 +316,8 @@ pub(crate) struct RangeBase {
pub(crate) codec: Arc<dyn PrimaryKeyCodec>,
/// Optional helper to compat batches.
pub(crate) compat_batch: Option<CompatBatch>,
/// Mode to pre-filter columns.
pub(crate) pre_filter_mode: PreFilterMode,
}
impl RangeBase {
@@ -276,7 +328,15 @@ impl RangeBase {
///
/// When a filter is referencing primary key column, this method will decode
/// the primary key and put it into the batch.
pub(crate) fn precise_filter(&self, mut input: Batch) -> Result<Option<Batch>> {
///
/// # Arguments
/// * `input` - The batch to filter
/// * `skip_fields` - Whether to skip field filters based on PreFilterMode and row group delete status
pub(crate) fn precise_filter(
&self,
mut input: Batch,
skip_fields: bool,
) -> Result<Option<Batch>> {
let mut mask = BooleanBuffer::new_set(input.num_rows());
// Run filter one by one and combine them result
@@ -331,6 +391,10 @@ impl RangeBase {
}
}
SemanticType::Field => {
// Skip field filters if skip_fields is true
if skip_fields {
continue;
}
// Safety: Input is Batch so we are using primary key format.
let Some(field_index) = self
.read_format
@@ -361,8 +425,16 @@ impl RangeBase {
/// Filters the input RecordBatch by the pushed down predicate and returns RecordBatch.
///
/// It assumes all necessary tags are already decoded from the primary key.
pub(crate) fn precise_filter_flat(&self, input: RecordBatch) -> Result<Option<RecordBatch>> {
let mask = self.compute_filter_mask_flat(&input)?;
///
/// # Arguments
/// * `input` - The RecordBatch to filter
/// * `skip_fields` - Whether to skip field filters based on PreFilterMode and row group delete status
pub(crate) fn precise_filter_flat(
&self,
input: RecordBatch,
skip_fields: bool,
) -> Result<Option<RecordBatch>> {
let mask = self.compute_filter_mask_flat(&input, skip_fields)?;
// If mask is None, the entire batch is filtered out
let Some(mask) = mask else {
@@ -383,9 +455,14 @@ impl RangeBase {
/// Computes the filter mask for the input RecordBatch based on pushed down predicates.
///
/// Returns `None` if the entire batch is filtered out, otherwise returns the boolean mask.
///
/// # Arguments
/// * `input` - The RecordBatch to compute mask for
/// * `skip_fields` - Whether to skip field filters based on PreFilterMode and row group delete status
pub(crate) fn compute_filter_mask_flat(
&self,
input: &RecordBatch,
skip_fields: bool,
) -> Result<Option<BooleanBuffer>> {
let mut mask = BooleanBuffer::new_set(input.num_rows());
@@ -411,6 +488,11 @@ impl RangeBase {
MaybeFilter::Pruned => return Ok(None),
};
// Skip field filters if skip_fields is true
if skip_fields && filter_ctx.semantic_type() == SemanticType::Field {
continue;
}
// Get the column directly by its projected index
let column_idx = flat_format.projected_index_by_id(filter_ctx.column_id());
if let Some(idx) = column_idx {

View File

@@ -55,7 +55,9 @@ use crate::sst::file::FileHandle;
use crate::sst::index::bloom_filter::applier::BloomFilterIndexApplierRef;
use crate::sst::index::fulltext_index::applier::FulltextIndexApplierRef;
use crate::sst::index::inverted_index::applier::InvertedIndexApplierRef;
use crate::sst::parquet::file_range::{FileRangeContext, FileRangeContextRef};
use crate::sst::parquet::file_range::{
FileRangeContext, FileRangeContextRef, PreFilterMode, row_group_contains_delete,
};
use crate::sst::parquet::format::{ReadFormat, need_override_sequence};
use crate::sst::parquet::metadata::MetadataLoader;
use crate::sst::parquet::row_group::InMemoryRowGroup;
@@ -106,9 +108,9 @@ pub struct ParquetReaderBuilder {
/// Strategy to cache SST data.
cache_strategy: CacheStrategy,
/// Index appliers.
inverted_index_applier: Option<InvertedIndexApplierRef>,
bloom_filter_index_applier: Option<BloomFilterIndexApplierRef>,
fulltext_index_applier: Option<FulltextIndexApplierRef>,
inverted_index_appliers: [Option<InvertedIndexApplierRef>; 2],
bloom_filter_index_appliers: [Option<BloomFilterIndexApplierRef>; 2],
fulltext_index_appliers: [Option<FulltextIndexApplierRef>; 2],
/// Expected metadata of the region while reading the SST.
/// This is usually the latest metadata of the region. The reader use
/// it get the correct column id of a column by name.
@@ -117,6 +119,8 @@ pub struct ParquetReaderBuilder {
flat_format: bool,
/// Whether this reader is for compaction.
compaction: bool,
/// Mode to pre-filter columns.
pre_filter_mode: PreFilterMode,
}
impl ParquetReaderBuilder {
@@ -135,12 +139,13 @@ impl ParquetReaderBuilder {
predicate: None,
projection: None,
cache_strategy: CacheStrategy::Disabled,
inverted_index_applier: None,
bloom_filter_index_applier: None,
fulltext_index_applier: None,
inverted_index_appliers: [None, None],
bloom_filter_index_appliers: [None, None],
fulltext_index_appliers: [None, None],
expected_metadata: None,
flat_format: false,
compaction: false,
pre_filter_mode: PreFilterMode::All,
}
}
@@ -167,33 +172,33 @@ impl ParquetReaderBuilder {
self
}
/// Attaches the inverted index applier to the builder.
/// Attaches the inverted index appliers to the builder.
#[must_use]
pub(crate) fn inverted_index_applier(
pub(crate) fn inverted_index_appliers(
mut self,
index_applier: Option<InvertedIndexApplierRef>,
index_appliers: [Option<InvertedIndexApplierRef>; 2],
) -> Self {
self.inverted_index_applier = index_applier;
self.inverted_index_appliers = index_appliers;
self
}
/// Attaches the bloom filter index applier to the builder.
/// Attaches the bloom filter index appliers to the builder.
#[must_use]
pub(crate) fn bloom_filter_index_applier(
pub(crate) fn bloom_filter_index_appliers(
mut self,
index_applier: Option<BloomFilterIndexApplierRef>,
index_appliers: [Option<BloomFilterIndexApplierRef>; 2],
) -> Self {
self.bloom_filter_index_applier = index_applier;
self.bloom_filter_index_appliers = index_appliers;
self
}
/// Attaches the fulltext index applier to the builder.
/// Attaches the fulltext index appliers to the builder.
#[must_use]
pub(crate) fn fulltext_index_applier(
pub(crate) fn fulltext_index_appliers(
mut self,
index_applier: Option<FulltextIndexApplierRef>,
index_appliers: [Option<FulltextIndexApplierRef>; 2],
) -> Self {
self.fulltext_index_applier = index_applier;
self.fulltext_index_appliers = index_appliers;
self
}
@@ -218,6 +223,13 @@ impl ParquetReaderBuilder {
self
}
/// Sets the pre-filter mode.
#[must_use]
pub(crate) fn pre_filter_mode(mut self, pre_filter_mode: PreFilterMode) -> Self {
self.pre_filter_mode = pre_filter_mode;
self
}
/// Builds a [ParquetReader].
///
/// This needs to perform IO operation.
@@ -321,7 +333,13 @@ impl ParquetReaderBuilder {
let codec = build_primary_key_codec(read_format.metadata());
let context = FileRangeContext::new(reader_builder, filters, read_format, codec);
let context = FileRangeContext::new(
reader_builder,
filters,
read_format,
codec,
self.pre_filter_mode,
);
metrics.build_cost += start.elapsed();
@@ -407,7 +425,16 @@ impl ParquetReaderBuilder {
let mut output = RowGroupSelection::new(row_group_size, num_rows as _);
self.prune_row_groups_by_minmax(read_format, parquet_meta, &mut output, metrics);
// Compute skip_fields once for all pruning operations
let skip_fields = self.compute_skip_fields(parquet_meta);
self.prune_row_groups_by_minmax(
read_format,
parquet_meta,
&mut output,
metrics,
skip_fields,
);
if output.is_empty() {
return output;
}
@@ -418,6 +445,7 @@ impl ParquetReaderBuilder {
num_row_groups,
&mut output,
metrics,
skip_fields,
)
.await;
if output.is_empty() {
@@ -429,14 +457,21 @@ impl ParquetReaderBuilder {
num_row_groups,
&mut output,
metrics,
skip_fields,
)
.await;
if output.is_empty() {
return output;
}
self.prune_row_groups_by_bloom_filter(row_group_size, parquet_meta, &mut output, metrics)
.await;
self.prune_row_groups_by_bloom_filter(
row_group_size,
parquet_meta,
&mut output,
metrics,
skip_fields,
)
.await;
if output.is_empty() {
return output;
}
@@ -447,6 +482,7 @@ impl ParquetReaderBuilder {
parquet_meta,
&mut output,
metrics,
skip_fields,
)
.await;
}
@@ -460,50 +496,61 @@ impl ParquetReaderBuilder {
num_row_groups: usize,
output: &mut RowGroupSelection,
metrics: &mut ReaderFilterMetrics,
skip_fields: bool,
) -> bool {
let Some(index_applier) = &self.fulltext_index_applier else {
return false;
};
if !self.file_handle.meta_ref().fulltext_index_available() {
return false;
}
let predicate_key = index_applier.predicate_key();
// Fast path: return early if the result is in the cache.
let cached = self
.cache_strategy
.index_result_cache()
.and_then(|cache| cache.get(predicate_key, self.file_handle.file_id().file_id()));
if let Some(result) = cached.as_ref()
&& 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 apply_res = index_applier
.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, num_row_groups),
Ok(None) => return false,
Err(err) => {
handle_index_error!(err, self.file_handle, INDEX_TYPE_FULLTEXT);
return false;
}
let mut pruned = false;
// If skip_fields is true, only apply the first applier (for tags).
let appliers = if skip_fields {
&self.fulltext_index_appliers[..1]
} else {
&self.fulltext_index_appliers[..]
};
for index_applier in appliers.iter().flatten() {
let predicate_key = index_applier.predicate_key();
// Fast path: return early if the result is in the cache.
let cached = self
.cache_strategy
.index_result_cache()
.and_then(|cache| cache.get(predicate_key, self.file_handle.file_id().file_id()));
if let Some(result) = cached.as_ref()
&& all_required_row_groups_searched(output, result)
{
apply_selection_and_update_metrics(output, result, metrics, INDEX_TYPE_FULLTEXT);
pruned = true;
continue;
}
self.apply_index_result_and_update_cache(
predicate_key,
self.file_handle.file_id().file_id(),
selection,
output,
metrics,
INDEX_TYPE_FULLTEXT,
);
true
// Slow path: apply the index from the file.
let file_size_hint = self.file_handle.meta_ref().index_file_size();
let apply_res = index_applier
.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, num_row_groups)
}
Ok(None) => continue,
Err(err) => {
handle_index_error!(err, self.file_handle, INDEX_TYPE_FULLTEXT);
continue;
}
};
self.apply_index_result_and_update_cache(
predicate_key,
self.file_handle.file_id().file_id(),
selection,
output,
metrics,
INDEX_TYPE_FULLTEXT,
);
pruned = true;
}
pruned
}
/// Applies index to prune row groups.
@@ -517,53 +564,62 @@ impl ParquetReaderBuilder {
num_row_groups: usize,
output: &mut RowGroupSelection,
metrics: &mut ReaderFilterMetrics,
skip_fields: bool,
) -> bool {
let Some(index_applier) = &self.inverted_index_applier else {
return false;
};
if !self.file_handle.meta_ref().inverted_index_available() {
return false;
}
let predicate_key = index_applier.predicate_key();
// Fast path: return early if the result is in the cache.
let cached = self
.cache_strategy
.index_result_cache()
.and_then(|cache| cache.get(predicate_key, self.file_handle.file_id().file_id()));
if let Some(result) = cached.as_ref()
&& 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.
let file_size_hint = self.file_handle.meta_ref().index_file_size();
let apply_res = index_applier
.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,
num_row_groups,
output,
),
Err(err) => {
handle_index_error!(err, self.file_handle, INDEX_TYPE_INVERTED);
return false;
}
let mut pruned = false;
// If skip_fields is true, only apply the first applier (for tags).
let appliers = if skip_fields {
&self.inverted_index_appliers[..1]
} else {
&self.inverted_index_appliers[..]
};
for index_applier in appliers.iter().flatten() {
let predicate_key = index_applier.predicate_key();
// Fast path: return early if the result is in the cache.
let cached = self
.cache_strategy
.index_result_cache()
.and_then(|cache| cache.get(predicate_key, self.file_handle.file_id().file_id()));
if let Some(result) = cached.as_ref()
&& all_required_row_groups_searched(output, result)
{
apply_selection_and_update_metrics(output, result, metrics, INDEX_TYPE_INVERTED);
pruned = true;
continue;
}
self.apply_index_result_and_update_cache(
predicate_key,
self.file_handle.file_id().file_id(),
selection,
output,
metrics,
INDEX_TYPE_INVERTED,
);
true
// Slow path: apply the index from the file.
let file_size_hint = self.file_handle.meta_ref().index_file_size();
let apply_res = index_applier
.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,
num_row_groups,
output,
),
Err(err) => {
handle_index_error!(err, self.file_handle, INDEX_TYPE_INVERTED);
continue;
}
};
self.apply_index_result_and_update_cache(
predicate_key,
self.file_handle.file_id().file_id(),
selection,
output,
metrics,
INDEX_TYPE_INVERTED,
);
pruned = true;
}
pruned
}
async fn prune_row_groups_by_bloom_filter(
@@ -572,65 +628,76 @@ impl ParquetReaderBuilder {
parquet_meta: &ParquetMetaData,
output: &mut RowGroupSelection,
metrics: &mut ReaderFilterMetrics,
skip_fields: bool,
) -> bool {
let Some(index_applier) = &self.bloom_filter_index_applier else {
return false;
};
if !self.file_handle.meta_ref().bloom_filter_index_available() {
return false;
}
let predicate_key = index_applier.predicate_key();
// Fast path: return early if the result is in the cache.
let cached = self
.cache_strategy
.index_result_cache()
.and_then(|cache| cache.get(predicate_key, self.file_handle.file_id().file_id()));
if let Some(result) = cached.as_ref()
&& 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,
// 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 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);
return false;
}
let mut pruned = false;
// If skip_fields is true, only apply the first applier (for tags).
let appliers = if skip_fields {
&self.bloom_filter_index_appliers[..1]
} else {
&self.bloom_filter_index_appliers[..]
};
for index_applier in appliers.iter().flatten() {
let predicate_key = index_applier.predicate_key();
// Fast path: return early if the result is in the cache.
let cached = self
.cache_strategy
.index_result_cache()
.and_then(|cache| cache.get(predicate_key, self.file_handle.file_id().file_id()));
if let Some(result) = cached.as_ref()
&& all_required_row_groups_searched(output, result)
{
apply_selection_and_update_metrics(output, result, metrics, INDEX_TYPE_BLOOM);
pruned = true;
continue;
}
// New searched row groups are added to `selection`, concat them with `cached`.
if let Some(cached) = cached.as_ref() {
selection.concat(cached);
// 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,
// 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 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);
continue;
}
};
// 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().file_id(),
selection,
output,
metrics,
INDEX_TYPE_BLOOM,
);
pruned = true;
}
self.apply_index_result_and_update_cache(
predicate_key,
self.file_handle.file_id().file_id(),
selection,
output,
metrics,
INDEX_TYPE_BLOOM,
);
true
pruned
}
async fn prune_row_groups_by_fulltext_bloom(
@@ -639,68 +706,96 @@ impl ParquetReaderBuilder {
parquet_meta: &ParquetMetaData,
output: &mut RowGroupSelection,
metrics: &mut ReaderFilterMetrics,
skip_fields: bool,
) -> bool {
let Some(index_applier) = &self.fulltext_index_applier else {
return false;
};
if !self.file_handle.meta_ref().fulltext_index_available() {
return false;
}
let predicate_key = index_applier.predicate_key();
// Fast path: return early if the result is in the cache.
let cached = self
.cache_strategy
.index_result_cache()
.and_then(|cache| cache.get(predicate_key, self.file_handle.file_id().file_id()));
if let Some(result) = cached.as_ref()
&& 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,
// 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 mut selection = match apply_res {
Ok(Some(apply_output)) => {
RowGroupSelection::from_row_ranges(apply_output, row_group_size)
}
Ok(None) => return false,
Err(err) => {
handle_index_error!(err, self.file_handle, INDEX_TYPE_FULLTEXT);
return false;
}
let mut pruned = false;
// If skip_fields is true, only apply the first applier (for tags).
let appliers = if skip_fields {
&self.fulltext_index_appliers[..1]
} else {
&self.fulltext_index_appliers[..]
};
for index_applier in appliers.iter().flatten() {
let predicate_key = index_applier.predicate_key();
// Fast path: return early if the result is in the cache.
let cached = self
.cache_strategy
.index_result_cache()
.and_then(|cache| cache.get(predicate_key, self.file_handle.file_id().file_id()));
if let Some(result) = cached.as_ref()
&& all_required_row_groups_searched(output, result)
{
apply_selection_and_update_metrics(output, result, metrics, INDEX_TYPE_FULLTEXT);
pruned = true;
continue;
}
// New searched row groups are added to `selection`, concat them with `cached`.
if let Some(cached) = cached.as_ref() {
selection.concat(cached);
// 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,
// 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 mut selection = match apply_res {
Ok(Some(apply_output)) => {
RowGroupSelection::from_row_ranges(apply_output, row_group_size)
}
Ok(None) => continue,
Err(err) => {
handle_index_error!(err, self.file_handle, INDEX_TYPE_FULLTEXT);
continue;
}
};
// 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().file_id(),
selection,
output,
metrics,
INDEX_TYPE_FULLTEXT,
);
pruned = true;
}
pruned
}
self.apply_index_result_and_update_cache(
predicate_key,
self.file_handle.file_id().file_id(),
selection,
output,
metrics,
INDEX_TYPE_FULLTEXT,
);
true
/// Computes whether to skip field columns when building statistics based on PreFilterMode.
fn compute_skip_fields(&self, parquet_meta: &ParquetMetaData) -> bool {
match self.pre_filter_mode {
PreFilterMode::All => false,
PreFilterMode::SkipFields => true,
PreFilterMode::SkipFieldsOnDelete => {
// Check if any row group contains delete op
let file_path = self.file_handle.file_path(&self.table_dir, self.path_type);
(0..parquet_meta.num_row_groups()).any(|rg_idx| {
row_group_contains_delete(parquet_meta, rg_idx, &file_path)
.inspect_err(|e| {
warn!(e; "Failed to decode min value of op_type, fallback to not skipping fields");
})
.unwrap_or(false)
})
}
}
}
/// Prunes row groups by min-max index.
@@ -710,6 +805,7 @@ impl ParquetReaderBuilder {
parquet_meta: &ParquetMetaData,
output: &mut RowGroupSelection,
metrics: &mut ReaderFilterMetrics,
skip_fields: bool,
) -> bool {
let Some(predicate) = &self.predicate else {
return false;
@@ -719,8 +815,12 @@ impl ParquetReaderBuilder {
let region_meta = read_format.metadata();
let row_groups = parquet_meta.row_groups();
let stats =
RowGroupPruningStats::new(row_groups, read_format, self.expected_metadata.clone());
let stats = RowGroupPruningStats::new(
row_groups,
read_format,
self.expected_metadata.clone(),
skip_fields,
);
let prune_schema = self
.expected_metadata
.as_ref()
@@ -1151,10 +1251,12 @@ impl BatchReader for ParquetReader {
.await?;
// Resets the parquet reader.
reader.reset_source(Source::RowGroup(RowGroupReader::new(
self.context.clone(),
parquet_reader,
)));
// Compute skip_fields for this row group
let skip_fields = self.context.should_skip_fields(row_group_idx);
reader.reset_source(
Source::RowGroup(RowGroupReader::new(self.context.clone(), parquet_reader)),
skip_fields,
);
if let Some(batch) = reader.next_batch().await? {
return Ok(Some(batch));
}
@@ -1207,9 +1309,12 @@ impl ParquetReader {
.reader_builder()
.build(row_group_idx, Some(row_selection))
.await?;
// Compute skip_fields once for this row group
let skip_fields = context.should_skip_fields(row_group_idx);
ReaderState::Readable(PruneReader::new_with_row_group_reader(
context.clone(),
RowGroupReader::new(context.clone(), parquet_reader),
skip_fields,
))
} else {
ReaderState::Exhausted(ReaderMetrics::default())

View File

@@ -18,6 +18,7 @@ use std::borrow::Borrow;
use std::collections::HashSet;
use std::sync::Arc;
use api::v1::SemanticType;
use datafusion_common::pruning::PruningStatistics;
use datafusion_common::{Column, ScalarValue};
use datatypes::arrow::array::{ArrayRef, BooleanArray, UInt64Array};
@@ -38,6 +39,8 @@ pub(crate) struct RowGroupPruningStats<'a, T> {
/// of the metadata in the SST to get the column id of a column as the SST may have
/// different columns.
expected_metadata: Option<RegionMetadataRef>,
/// If true, skip columns with Field semantic type during pruning.
skip_fields: bool,
}
impl<'a, T> RowGroupPruningStats<'a, T> {
@@ -46,22 +49,32 @@ impl<'a, T> RowGroupPruningStats<'a, T> {
row_groups: &'a [T],
read_format: &'a ReadFormat,
expected_metadata: Option<RegionMetadataRef>,
skip_fields: bool,
) -> Self {
Self {
row_groups,
read_format,
expected_metadata,
skip_fields,
}
}
/// Returns the column id of specific column name if we need to read it.
/// Prefers the column id in the expected metadata if it exists.
/// Returns None if skip_fields is true and the column is a Field.
fn column_id_to_prune(&self, name: &str) -> Option<ColumnId> {
let metadata = self
.expected_metadata
.as_ref()
.unwrap_or_else(|| self.read_format.metadata());
metadata.column_by_name(name).map(|col| col.column_id)
let col = metadata.column_by_name(name)?;
// Skip field columns when skip_fields is enabled
if self.skip_fields && col.semantic_type == SemanticType::Field {
return None;
}
Some(col.column_id)
}
/// Returns the default value of all row groups for `column` according to the metadata.

View File

@@ -1068,6 +1068,19 @@ pub fn build_rows(start: usize, end: usize) -> Vec<Row> {
.collect()
}
/// Build rows with schema (string, ts_millis) in range `[start, end)`.
/// `start`, `end` are in second resolution.
pub fn build_delete_rows(start: usize, end: usize) -> Vec<Row> {
(start..end)
.map(|i| {
row(vec![
ValueData::StringValue(i.to_string()),
ValueData::TimestampMillisecondValue(i as i64 * 1000),
])
})
.collect()
}
/// Build rows with schema (string, f64, f64, ts_millis).
/// - `key`: A string key that is common across all rows.
/// - `timestamps`: Array of timestamp values.

View File

@@ -38,7 +38,7 @@ use crate::memtable::bulk::part::BulkPart;
use crate::memtable::partition_tree::data::{DataBatch, DataBuffer, timestamp_array_to_i64_slice};
use crate::memtable::{
BoxedBatchIterator, KeyValues, Memtable, MemtableBuilder, MemtableId, MemtableRanges,
MemtableRef, MemtableStats,
MemtableRef, MemtableStats, RangesOptions,
};
use crate::read::scan_region::PredicateGroup;
@@ -97,9 +97,7 @@ impl Memtable for EmptyMemtable {
fn ranges(
&self,
_projection: Option<&[ColumnId]>,
_predicate: PredicateGroup,
_sequence: Option<SequenceRange>,
_for_flush: bool,
_options: RangesOptions,
) -> Result<MemtableRanges> {
Ok(MemtableRanges::default())
}

View File

@@ -111,10 +111,10 @@ impl SchedulerEnv {
}
/// Creates a new index build scheduler.
pub(crate) fn mock_index_build_scheduler(&self) -> IndexBuildScheduler {
pub(crate) fn mock_index_build_scheduler(&self, files_limit: usize) -> IndexBuildScheduler {
let scheduler = self.get_scheduler();
IndexBuildScheduler::new(scheduler)
IndexBuildScheduler::new(scheduler, files_limit)
}
/// Creates a new manifest context.

View File

@@ -125,6 +125,7 @@ pub fn sst_file_handle_with_file_id(file_id: FileId, start_ms: i64, end_ms: i64)
file_size: 0,
available_indexes: Default::default(),
index_file_size: 0,
index_file_id: None,
num_rows: 0,
num_row_groups: 0,
num_series: 0,

View File

@@ -103,6 +103,7 @@ impl VersionControlBuilder {
file_size: 0, // We don't care file size.
available_indexes: Default::default(),
index_file_size: 0,
index_file_id: None,
num_rows: 0,
num_row_groups: 0,
num_series: 0,
@@ -192,6 +193,7 @@ pub(crate) fn apply_edit(
file_size: 0, // We don't care file size.
available_indexes: Default::default(),
index_file_size: 0,
index_file_id: None,
num_rows: 0,
num_row_groups: 0,
num_series: 0,

View File

@@ -509,7 +509,10 @@ impl<S: LogStore> WorkerStarter<S> {
),
purge_scheduler: self.purge_scheduler.clone(),
write_buffer_manager: self.write_buffer_manager,
index_build_scheduler: IndexBuildScheduler::new(self.index_build_job_pool),
index_build_scheduler: IndexBuildScheduler::new(
self.index_build_job_pool,
self.config.max_background_index_builds,
),
flush_scheduler: FlushScheduler::new(self.flush_job_pool),
compaction_scheduler: CompactionScheduler::new(
self.compact_job_pool,
@@ -1024,7 +1027,10 @@ impl<S: LogStore> RegionWorkerLoop<S> {
.await;
continue;
}
DdlRequest::Catchup(req) => self.handle_catchup_request(ddl.region_id, req).await,
DdlRequest::Catchup((req, wal_entry_receiver)) => {
self.handle_catchup_request(ddl.region_id, req, wal_entry_receiver)
.await
}
};
ddl.sender.send(res);
@@ -1059,6 +1065,9 @@ impl<S: LogStore> RegionWorkerLoop<S> {
BackgroundNotify::IndexBuildFinished(req) => {
self.handle_index_build_finished(region_id, req).await
}
BackgroundNotify::IndexBuildStopped(req) => {
self.handle_index_build_stopped(region_id, req).await
}
BackgroundNotify::IndexBuildFailed(req) => {
self.handle_index_build_failed(region_id, req).await
}

View File

@@ -28,6 +28,7 @@ use tokio::time::Instant;
use crate::error::{self, Result};
use crate::region::MitoRegion;
use crate::region::opener::{RegionOpener, replay_memtable};
use crate::wal::entry_distributor::WalEntryReceiver;
use crate::worker::RegionWorkerLoop;
impl<S: LogStore> RegionWorkerLoop<S> {
@@ -35,6 +36,7 @@ impl<S: LogStore> RegionWorkerLoop<S> {
&mut self,
region_id: RegionId,
request: RegionCatchupRequest,
entry_receiver: Option<WalEntryReceiver>,
) -> Result<AffectedRows> {
let Some(region) = self.regions.get_region(region_id) else {
return error::RegionNotFoundSnafu { region_id }.fail();
@@ -76,9 +78,10 @@ impl<S: LogStore> RegionWorkerLoop<S> {
region.provider
);
let timer = Instant::now();
let wal_entry_reader =
let wal_entry_reader = entry_receiver.map(|r| Box::new(r) as _).unwrap_or_else(|| {
self.wal
.wal_entry_reader(&region.provider, region_id, request.location_id);
.wal_entry_reader(&region.provider, region_id, request.location_id)
});
let on_region_opened = self.wal.on_region_opened();
let last_entry_id = replay_memtable(
&region.provider,

View File

@@ -38,6 +38,8 @@ impl<S> RegionWorkerLoop<S> {
self.flush_scheduler.on_region_closed(region_id);
// Clean compaction status.
self.compaction_scheduler.on_region_closed(region_id);
// clean index build status.
self.index_build_scheduler.on_region_closed(region_id).await;
info!("Region {} closed, worker: {}", region_id, self.id);

View File

@@ -83,6 +83,10 @@ where
self.flush_scheduler.on_region_dropped(region_id);
// Notifies compaction scheduler.
self.compaction_scheduler.on_region_dropped(region_id);
// notifies index build scheduler.
self.index_build_scheduler
.on_region_dropped(region_id)
.await;
// Marks region version as dropped
region

View File

@@ -22,9 +22,12 @@ use store_api::region_request::RegionBuildIndexRequest;
use store_api::storage::{FileId, RegionId};
use tokio::sync::mpsc;
use crate::cache::CacheStrategy;
use crate::error::Result;
use crate::region::MitoRegionRef;
use crate::request::{BuildIndexRequest, IndexBuildFailed, IndexBuildFinished, OptionOutputTx};
use crate::request::{
BuildIndexRequest, IndexBuildFailed, IndexBuildFinished, IndexBuildStopped, OptionOutputTx,
};
use crate::sst::file::{FileHandle, RegionFileId};
use crate::sst::index::{
IndexBuildOutcome, IndexBuildTask, IndexBuildType, IndexerBuilderImpl, ResultMpscSender,
@@ -172,7 +175,8 @@ impl<S> RegionWorkerLoop<S> {
);
let _ = self
.index_build_scheduler
.schedule_build(&region.version_control, task);
.schedule_build(&region.version_control, task)
.await;
}
// Wait for all index build tasks to finish and notify the caller.
common_runtime::spawn_global(async move {
@@ -203,11 +207,19 @@ impl<S> RegionWorkerLoop<S> {
}
};
// Clean old puffin-related cache for all rebuilt files.
let cache_strategy = CacheStrategy::EnableAll(self.cache_manager.clone());
for file_meta in &request.edit.files_to_add {
let region_file_id = RegionFileId::new(region_id, file_meta.file_id);
cache_strategy.evict_puffin_cache(region_file_id).await;
}
region.version_control.apply_edit(
Some(request.edit.clone()),
&[],
region.file_purger.clone(),
);
for file_meta in &request.edit.files_to_add {
self.listener
.on_index_build_finish(RegionFileId::new(region_id, file_meta.file_id))
@@ -221,6 +233,27 @@ impl<S> RegionWorkerLoop<S> {
request: IndexBuildFailed,
) {
error!(request.err; "Index build failed for region: {}", region_id);
// TODO(SNC123): Implement error handling logic after IndexBuildScheduler optimization.
self.index_build_scheduler
.on_failure(region_id, request.err.clone())
.await;
}
pub(crate) async fn handle_index_build_stopped(
&mut self,
region_id: RegionId,
request: IndexBuildStopped,
) {
let Some(region) = self.regions.get_region(region_id) else {
warn!(
"Region not found for index build stopped, region_id: {}",
region_id
);
return;
};
self.index_build_scheduler.on_task_stopped(
region_id,
request.file_id,
&region.version_control,
);
}
}

View File

@@ -142,6 +142,10 @@ impl<S: LogStore> RegionWorkerLoop<S> {
self.flush_scheduler.on_region_truncated(region_id);
// Notifies compaction scheduler.
self.compaction_scheduler.on_region_truncated(region_id);
// Notifies index build scheduler.
self.index_build_scheduler
.on_region_truncated(region_id)
.await;
if let TruncateKind::All {
truncated_entry_id,

View File

@@ -855,7 +855,7 @@ transform:
row.0.values.into_iter().for_each(|v| {
if let ValueData::TimestampNanosecondValue(v) = v.value_data.unwrap() {
let now = chrono::Utc::now().timestamp_nanos_opt().unwrap();
assert!(now - v < 1_000_000);
assert!(now - v < 5_000_000);
}
});
}

View File

@@ -913,7 +913,6 @@ impl HttpServer {
Router::new()
.route("/cpu", routing::post(pprof::pprof_handler))
.route("/mem", routing::post(mem_prof::mem_prof_handler))
.route("/bytes", routing::post(mem_prof::bytes_prof_handler))
.route(
"/mem/activate",
routing::post(mem_prof::activate_heap_prof_handler),

View File

@@ -39,14 +39,6 @@ pub struct MemPprofQuery {
output: Output,
}
pub async fn bytes_prof_handler() -> crate::error::Result<impl IntoResponse> {
let dump = bytes::GLOBAL_TRACER
.get_or_init(|| bytes::BytesTracer::new().0)
.collector
.render_flamegraph()?;
Ok((StatusCode::OK, dump))
}
#[cfg(feature = "mem-prof")]
#[axum_macros::debug_handler]
pub async fn mem_prof_handler(

View File

@@ -13,22 +13,43 @@
// limitations under the License.
//! prom supply the prometheus HTTP API Server compliance
use std::borrow::Borrow;
use std::collections::{BTreeMap, HashMap, HashSet};
use std::hash::{Hash, Hasher};
use std::sync::Arc;
use arrow::array::AsArray;
use arrow::datatypes::{
Date32Type, Date64Type, Decimal128Type, DurationMicrosecondType, DurationMillisecondType,
DurationNanosecondType, DurationSecondType, Float32Type, Float64Type, Int8Type, Int16Type,
Int32Type, Int64Type, IntervalDayTimeType, IntervalMonthDayNanoType, IntervalYearMonthType,
Time32MillisecondType, Time32SecondType, Time64MicrosecondType, Time64NanosecondType,
TimestampMicrosecondType, TimestampMillisecondType, TimestampNanosecondType,
TimestampSecondType, UInt8Type, UInt16Type, UInt32Type, UInt64Type,
};
use arrow_schema::{DataType, IntervalUnit, TimeUnit};
use axum::extract::{Path, Query, State};
use axum::{Extension, Form};
use catalog::CatalogManagerRef;
use common_catalog::parse_catalog_and_schema_from_db_string;
use common_decimal::Decimal128;
use common_error::ext::ErrorExt;
use common_error::status_code::StatusCode;
use common_query::{Output, OutputData};
use common_recordbatch::RecordBatches;
use common_recordbatch::{RecordBatch, RecordBatches};
use common_telemetry::{debug, tracing};
use common_time::time::Time;
use common_time::util::{current_time_rfc3339, yesterday_rfc3339};
use common_time::{
Date, Duration, IntervalDayTime, IntervalMonthDayNano, IntervalYearMonth, Timestamp,
};
use common_version::OwnedBuildInfo;
use datafusion_common::ScalarValue;
use datatypes::prelude::ConcreteDataType;
use datatypes::scalars::ScalarVector;
use datatypes::schema::{ColumnSchema, SchemaRef};
use datatypes::types::jsonb_to_string;
use datatypes::vectors::Float64Vector;
use futures::StreamExt;
use futures::future::join_all;
@@ -53,8 +74,9 @@ use store_api::metric_engine_consts::{
pub use super::result::prometheus_resp::PrometheusJsonResponse;
use crate::error::{
CatalogSnafu, CollectRecordbatchSnafu, Error, InvalidQuerySnafu, ParseTimestampSnafu, Result,
TableNotFoundSnafu, UnexpectedResultSnafu,
CatalogSnafu, CollectRecordbatchSnafu, ConvertScalarValueSnafu, DataFusionSnafu, Error,
InvalidQuerySnafu, NotSupportedSnafu, ParseTimestampSnafu, Result, TableNotFoundSnafu,
UnexpectedResultSnafu,
};
use crate::http::header::collect_plan_metrics;
use crate::prom_store::{DATABASE_LABEL, FIELD_NAME_LABEL, METRIC_NAME_LABEL, SCHEMA_LABEL};
@@ -98,12 +120,23 @@ pub struct PromData {
pub result: PromQueryResult,
}
/// A "holder" for the reference([Arc]) to a column name,
/// to help avoiding cloning [String]s when used as a [HashMap] key.
#[derive(Debug, Clone, PartialEq, Eq, PartialOrd, Ord, Hash, Serialize, Deserialize)]
pub struct Column(Arc<String>);
impl From<&str> for Column {
fn from(s: &str) -> Self {
Self(Arc::new(s.to_string()))
}
}
#[derive(Debug, Default, Serialize, Deserialize, PartialEq)]
#[serde(untagged)]
pub enum PrometheusResponse {
PromData(PromData),
Labels(Vec<String>),
Series(Vec<HashMap<String, String>>),
Series(Vec<HashMap<Column, String>>),
LabelValues(Vec<String>),
FormatQuery(String),
BuildInfo(OwnedBuildInfo),
@@ -622,7 +655,7 @@ async fn get_all_column_names(
async fn retrieve_series_from_query_result(
result: Result<Output>,
series: &mut Vec<HashMap<String, String>>,
series: &mut Vec<HashMap<Column, String>>,
query_ctx: &QueryContext,
table_name: &str,
manager: &CatalogManagerRef,
@@ -700,7 +733,7 @@ async fn retrieve_labels_name_from_query_result(
fn record_batches_to_series(
batches: RecordBatches,
series: &mut Vec<HashMap<String, String>>,
series: &mut Vec<HashMap<Column, String>>,
table_name: &str,
tag_columns: &HashSet<String>,
) -> Result<()> {
@@ -723,22 +756,355 @@ fn record_batches_to_series(
.try_project(&projection)
.context(CollectRecordbatchSnafu)?;
for row in batch.rows() {
let mut element: HashMap<String, String> = row
.iter()
.enumerate()
.map(|(idx, column)| {
let column_name = batch.schema.column_name_by_index(idx);
(column_name.to_string(), column.to_string())
})
.collect();
let _ = element.insert("__name__".to_string(), table_name.to_string());
series.push(element);
}
let mut writer = RowWriter::new(&batch.schema, table_name);
writer.write(batch, series)?;
}
Ok(())
}
/// Writer from a row in the record batch to a Prometheus time series:
///
/// `{__name__="<metric name>", <label name>="<label value>", ...}`
///
/// The metrics name is the table name; label names are the column names and
/// the label values are the corresponding row values (all are converted to strings).
struct RowWriter {
/// The template that is to produce a Prometheus time series. It is pre-filled with metrics name
/// and label names, waiting to be filled by row values afterward.
template: HashMap<Column, Option<String>>,
/// The current filling row.
current: Option<HashMap<Column, Option<String>>>,
}
impl RowWriter {
fn new(schema: &SchemaRef, table: &str) -> Self {
let mut template = schema
.column_schemas()
.iter()
.map(|x| (x.name.as_str().into(), None))
.collect::<HashMap<Column, Option<String>>>();
template.insert("__name__".into(), Some(table.to_string()));
Self {
template,
current: None,
}
}
fn insert(&mut self, column: ColumnRef, value: impl ToString) {
let current = self.current.get_or_insert_with(|| self.template.clone());
match current.get_mut(&column as &dyn AsColumnRef) {
Some(x) => {
let _ = x.insert(value.to_string());
}
None => {
let _ = current.insert(column.0.into(), Some(value.to_string()));
}
}
}
fn insert_bytes(&mut self, column_schema: &ColumnSchema, bytes: &[u8]) -> Result<()> {
let column_name = column_schema.name.as_str().into();
if column_schema.data_type.is_json() {
let s = jsonb_to_string(bytes).context(ConvertScalarValueSnafu)?;
self.insert(column_name, s);
} else {
let hex = bytes
.iter()
.map(|b| format!("{b:02x}"))
.collect::<Vec<String>>()
.join("");
self.insert(column_name, hex);
}
Ok(())
}
fn finish(&mut self) -> HashMap<Column, String> {
let Some(current) = self.current.take() else {
return HashMap::new();
};
current
.into_iter()
.filter_map(|(k, v)| v.map(|v| (k, v)))
.collect()
}
fn write(
&mut self,
record_batch: RecordBatch,
series: &mut Vec<HashMap<Column, String>>,
) -> Result<()> {
let schema = record_batch.schema.clone();
let record_batch = record_batch.into_df_record_batch();
for i in 0..record_batch.num_rows() {
for (j, array) in record_batch.columns().iter().enumerate() {
let column = schema.column_name_by_index(j).into();
if array.is_null(i) {
self.insert(column, "Null");
continue;
}
match array.data_type() {
DataType::Null => {
self.insert(column, "Null");
}
DataType::Boolean => {
let array = array.as_boolean();
let v = array.value(i);
self.insert(column, v);
}
DataType::UInt8 => {
let array = array.as_primitive::<UInt8Type>();
let v = array.value(i);
self.insert(column, v);
}
DataType::UInt16 => {
let array = array.as_primitive::<UInt16Type>();
let v = array.value(i);
self.insert(column, v);
}
DataType::UInt32 => {
let array = array.as_primitive::<UInt32Type>();
let v = array.value(i);
self.insert(column, v);
}
DataType::UInt64 => {
let array = array.as_primitive::<UInt64Type>();
let v = array.value(i);
self.insert(column, v);
}
DataType::Int8 => {
let array = array.as_primitive::<Int8Type>();
let v = array.value(i);
self.insert(column, v);
}
DataType::Int16 => {
let array = array.as_primitive::<Int16Type>();
let v = array.value(i);
self.insert(column, v);
}
DataType::Int32 => {
let array = array.as_primitive::<Int32Type>();
let v = array.value(i);
self.insert(column, v);
}
DataType::Int64 => {
let array = array.as_primitive::<Int64Type>();
let v = array.value(i);
self.insert(column, v);
}
DataType::Float32 => {
let array = array.as_primitive::<Float32Type>();
let v = array.value(i);
self.insert(column, v);
}
DataType::Float64 => {
let array = array.as_primitive::<Float64Type>();
let v = array.value(i);
self.insert(column, v);
}
DataType::Utf8 => {
let array = array.as_string::<i32>();
let v = array.value(i);
self.insert(column, v);
}
DataType::LargeUtf8 => {
let array = array.as_string::<i64>();
let v = array.value(i);
self.insert(column, v);
}
DataType::Utf8View => {
let array = array.as_string_view();
let v = array.value(i);
self.insert(column, v);
}
DataType::Binary => {
let array = array.as_binary::<i32>();
let v = array.value(i);
let column_schema = &schema.column_schemas()[j];
self.insert_bytes(column_schema, v)?;
}
DataType::LargeBinary => {
let array = array.as_binary::<i64>();
let v = array.value(i);
let column_schema = &schema.column_schemas()[j];
self.insert_bytes(column_schema, v)?;
}
DataType::BinaryView => {
let array = array.as_binary_view();
let v = array.value(i);
let column_schema = &schema.column_schemas()[j];
self.insert_bytes(column_schema, v)?;
}
DataType::Date32 => {
let array = array.as_primitive::<Date32Type>();
let v = Date::new(array.value(i));
self.insert(column, v);
}
DataType::Date64 => {
let array = array.as_primitive::<Date64Type>();
// `Date64` values are milliseconds representation of `Date32` values,
// according to its specification. So we convert the `Date64` value here to
// the `Date32` value to process them unified.
let v = Date::new((array.value(i) / 86_400_000) as i32);
self.insert(column, v);
}
DataType::Timestamp(time_unit, _) => {
let v = match time_unit {
TimeUnit::Second => {
let array = array.as_primitive::<TimestampSecondType>();
array.value(i)
}
TimeUnit::Millisecond => {
let array = array.as_primitive::<TimestampMillisecondType>();
array.value(i)
}
TimeUnit::Microsecond => {
let array = array.as_primitive::<TimestampMicrosecondType>();
array.value(i)
}
TimeUnit::Nanosecond => {
let array = array.as_primitive::<TimestampNanosecondType>();
array.value(i)
}
};
let v = Timestamp::new(v, time_unit.into());
self.insert(column, v.to_iso8601_string());
}
DataType::Time32(time_unit) | DataType::Time64(time_unit) => {
let v = match time_unit {
TimeUnit::Second => {
let array = array.as_primitive::<Time32SecondType>();
Time::new_second(array.value(i) as i64)
}
TimeUnit::Millisecond => {
let array = array.as_primitive::<Time32MillisecondType>();
Time::new_millisecond(array.value(i) as i64)
}
TimeUnit::Microsecond => {
let array = array.as_primitive::<Time64MicrosecondType>();
Time::new_microsecond(array.value(i))
}
TimeUnit::Nanosecond => {
let array = array.as_primitive::<Time64NanosecondType>();
Time::new_nanosecond(array.value(i))
}
};
self.insert(column, v.to_iso8601_string());
}
DataType::Interval(interval_unit) => match interval_unit {
IntervalUnit::YearMonth => {
let array = array.as_primitive::<IntervalYearMonthType>();
let v: IntervalYearMonth = array.value(i).into();
self.insert(column, v.to_iso8601_string());
}
IntervalUnit::DayTime => {
let array = array.as_primitive::<IntervalDayTimeType>();
let v: IntervalDayTime = array.value(i).into();
self.insert(column, v.to_iso8601_string());
}
IntervalUnit::MonthDayNano => {
let array = array.as_primitive::<IntervalMonthDayNanoType>();
let v: IntervalMonthDayNano = array.value(i).into();
self.insert(column, v.to_iso8601_string());
}
},
DataType::Duration(time_unit) => {
let v = match time_unit {
TimeUnit::Second => {
let array = array.as_primitive::<DurationSecondType>();
array.value(i)
}
TimeUnit::Millisecond => {
let array = array.as_primitive::<DurationMillisecondType>();
array.value(i)
}
TimeUnit::Microsecond => {
let array = array.as_primitive::<DurationMicrosecondType>();
array.value(i)
}
TimeUnit::Nanosecond => {
let array = array.as_primitive::<DurationNanosecondType>();
array.value(i)
}
};
let d = Duration::new(v, time_unit.into());
self.insert(column, d);
}
DataType::List(_) => {
let v = ScalarValue::try_from_array(array, i).context(DataFusionSnafu)?;
self.insert(column, v);
}
DataType::Struct(_) => {
let v = ScalarValue::try_from_array(array, i).context(DataFusionSnafu)?;
self.insert(column, v);
}
DataType::Decimal128(precision, scale) => {
let array = array.as_primitive::<Decimal128Type>();
let v = Decimal128::new(array.value(i), *precision, *scale);
self.insert(column, v);
}
_ => {
return NotSupportedSnafu {
feat: format!("convert {} to http value", array.data_type()),
}
.fail();
}
}
}
series.push(self.finish())
}
Ok(())
}
}
#[derive(Debug, Clone, Copy, PartialEq)]
struct ColumnRef<'a>(&'a str);
impl<'a> From<&'a str> for ColumnRef<'a> {
fn from(s: &'a str) -> Self {
Self(s)
}
}
trait AsColumnRef {
fn as_ref(&self) -> ColumnRef<'_>;
}
impl AsColumnRef for Column {
fn as_ref(&self) -> ColumnRef<'_> {
self.0.as_str().into()
}
}
impl AsColumnRef for ColumnRef<'_> {
fn as_ref(&self) -> ColumnRef<'_> {
*self
}
}
impl<'a> PartialEq for dyn AsColumnRef + 'a {
fn eq(&self, other: &Self) -> bool {
self.as_ref() == other.as_ref()
}
}
impl<'a> Eq for dyn AsColumnRef + 'a {}
impl<'a> Hash for dyn AsColumnRef + 'a {
fn hash<H: Hasher>(&self, state: &mut H) {
self.as_ref().0.hash(state);
}
}
impl<'a> Borrow<dyn AsColumnRef + 'a> for Column {
fn borrow(&self) -> &(dyn AsColumnRef + 'a) {
self
}
}
/// Retrieve labels name from record batches
fn record_batches_to_labels_name(
batches: RecordBatches,

View File

@@ -42,7 +42,7 @@ impl Display for KafkaProvider {
}
// The Provider of raft engine log store
#[derive(Debug, Clone, Copy, PartialEq, Eq)]
#[derive(Debug, Clone, Copy, PartialEq, Eq, Hash)]
pub struct RaftEngineProvider {
pub id: u64,
}
@@ -59,7 +59,7 @@ impl RaftEngineProvider {
}
/// The Provider of LogStore
#[derive(Debug, Clone, PartialEq, Eq)]
#[derive(Debug, Clone, PartialEq, Eq, Hash)]
pub enum Provider {
RaftEngine(RaftEngineProvider),
Kafka(Arc<KafkaProvider>),

View File

@@ -34,7 +34,9 @@ use tokio::sync::Semaphore;
use crate::logstore::entry;
use crate::metadata::RegionMetadataRef;
use crate::region_request::{BatchRegionDdlRequest, RegionOpenRequest, RegionRequest};
use crate::region_request::{
BatchRegionDdlRequest, RegionCatchupRequest, RegionOpenRequest, RegionRequest,
};
use crate::storage::{RegionId, ScanRequest, SequenceNumber};
/// The settable region role state.
@@ -715,6 +717,30 @@ pub trait RegionEngine: Send + Sync {
Ok(join_all(tasks).await)
}
async fn handle_batch_catchup_requests(
&self,
parallelism: usize,
requests: Vec<(RegionId, RegionCatchupRequest)>,
) -> Result<BatchResponses, BoxedError> {
let semaphore = Arc::new(Semaphore::new(parallelism));
let mut tasks = Vec::with_capacity(requests.len());
for (region_id, request) in requests {
let semaphore_moved = semaphore.clone();
tasks.push(async move {
// Safety: semaphore must exist
let _permit = semaphore_moved.acquire().await.unwrap();
let result = self
.handle_request(region_id, RegionRequest::Catchup(request))
.await;
(region_id, result)
});
}
Ok(join_all(tasks).await)
}
async fn handle_batch_ddl_requests(
&self,
request: BatchRegionDdlRequest,

View File

@@ -47,6 +47,8 @@ pub struct ManifestSstEntry {
pub region_sequence: RegionSeq,
/// Engine-specific file identifier (string form).
pub file_id: String,
/// Engine-specific index file identifier (string form).
pub index_file_id: Option<String>,
/// SST level.
pub level: u8,
/// Full path of the SST file in object store.
@@ -89,6 +91,7 @@ impl ManifestSstEntry {
ColumnSchema::new("region_group", Ty::uint8_datatype(), false),
ColumnSchema::new("region_sequence", Ty::uint32_datatype(), false),
ColumnSchema::new("file_id", Ty::string_datatype(), false),
ColumnSchema::new("index_file_id", Ty::string_datatype(), true),
ColumnSchema::new("level", Ty::uint8_datatype(), false),
ColumnSchema::new("file_path", Ty::string_datatype(), false),
ColumnSchema::new("file_size", Ty::uint64_datatype(), false),
@@ -116,6 +119,7 @@ impl ManifestSstEntry {
let region_groups = entries.iter().map(|e| e.region_group);
let region_sequences = entries.iter().map(|e| e.region_sequence);
let file_ids = entries.iter().map(|e| e.file_id.as_str());
let index_file_ids = entries.iter().map(|e| e.index_file_id.as_ref());
let levels = entries.iter().map(|e| e.level);
let file_paths = entries.iter().map(|e| e.file_path.as_str());
let file_sizes = entries.iter().map(|e| e.file_size);
@@ -147,6 +151,7 @@ impl ManifestSstEntry {
Arc::new(UInt8Array::from_iter_values(region_groups)),
Arc::new(UInt32Array::from_iter_values(region_sequences)),
Arc::new(StringArray::from_iter_values(file_ids)),
Arc::new(StringArray::from_iter(index_file_ids)),
Arc::new(UInt8Array::from_iter_values(levels)),
Arc::new(StringArray::from_iter_values(file_paths)),
Arc::new(UInt64Array::from_iter_values(file_sizes)),
@@ -432,6 +437,7 @@ mod tests {
region_group: region_group1,
region_sequence: region_seq1,
file_id: "f1".to_string(),
index_file_id: None,
level: 1,
file_path: "/p1".to_string(),
file_size: 100,
@@ -455,6 +461,7 @@ mod tests {
region_group: region_group2,
region_sequence: region_seq2,
file_id: "f2".to_string(),
index_file_id: Some("idx".to_string()),
level: 3,
file_path: "/p2".to_string(),
file_size: 200,
@@ -541,16 +548,24 @@ mod tests {
assert_eq!("f1", file_ids.value(0));
assert_eq!("f2", file_ids.value(1));
let levels = batch
let index_file_ids = batch
.column(7)
.as_any()
.downcast_ref::<StringArray>()
.unwrap();
assert!(index_file_ids.is_null(0));
assert_eq!("idx", index_file_ids.value(1));
let levels = batch
.column(8)
.as_any()
.downcast_ref::<UInt8Array>()
.unwrap();
assert_eq!(1, levels.value(0));
assert_eq!(3, levels.value(1));
let file_paths = batch
.column(8)
.column(9)
.as_any()
.downcast_ref::<StringArray>()
.unwrap();
@@ -558,7 +573,7 @@ mod tests {
assert_eq!("/p2", file_paths.value(1));
let file_sizes = batch
.column(9)
.column(10)
.as_any()
.downcast_ref::<UInt64Array>()
.unwrap();
@@ -566,7 +581,7 @@ mod tests {
assert_eq!(200, file_sizes.value(1));
let index_file_paths = batch
.column(10)
.column(11)
.as_any()
.downcast_ref::<StringArray>()
.unwrap();
@@ -574,7 +589,7 @@ mod tests {
assert_eq!("idx", index_file_paths.value(1));
let index_file_sizes = batch
.column(11)
.column(12)
.as_any()
.downcast_ref::<UInt64Array>()
.unwrap();
@@ -582,7 +597,7 @@ mod tests {
assert_eq!(11, index_file_sizes.value(1));
let num_rows = batch
.column(12)
.column(13)
.as_any()
.downcast_ref::<UInt64Array>()
.unwrap();
@@ -590,7 +605,7 @@ mod tests {
assert_eq!(20, num_rows.value(1));
let num_row_groups = batch
.column(13)
.column(14)
.as_any()
.downcast_ref::<UInt64Array>()
.unwrap();
@@ -598,7 +613,7 @@ mod tests {
assert_eq!(4, num_row_groups.value(1));
let num_series = batch
.column(14)
.column(15)
.as_any()
.downcast_ref::<UInt64Array>()
.unwrap();
@@ -606,7 +621,7 @@ mod tests {
assert!(num_series.is_null(1));
let min_ts = batch
.column(15)
.column(16)
.as_any()
.downcast_ref::<TimestampNanosecondArray>()
.unwrap();
@@ -614,7 +629,7 @@ mod tests {
assert_eq!(5, min_ts.value(1));
let max_ts = batch
.column(16)
.column(17)
.as_any()
.downcast_ref::<TimestampNanosecondArray>()
.unwrap();
@@ -622,7 +637,7 @@ mod tests {
assert_eq!(2_000_000, max_ts.value(1));
let sequences = batch
.column(17)
.column(18)
.as_any()
.downcast_ref::<UInt64Array>()
.unwrap();
@@ -630,7 +645,7 @@ mod tests {
assert_eq!(9, sequences.value(1));
let origin_region_ids = batch
.column(18)
.column(19)
.as_any()
.downcast_ref::<UInt64Array>()
.unwrap();
@@ -638,7 +653,7 @@ mod tests {
assert_eq!(region_id2.as_u64(), origin_region_ids.value(1));
let node_ids = batch
.column(19)
.column(20)
.as_any()
.downcast_ref::<UInt64Array>()
.unwrap();
@@ -646,7 +661,7 @@ mod tests {
assert!(node_ids.is_null(1));
let visible = batch
.column(20)
.column(21)
.as_any()
.downcast_ref::<BooleanArray>()
.unwrap();

View File

@@ -49,7 +49,7 @@ use servers::http::header::constants::{
GREPTIME_LOG_TABLE_NAME_HEADER_NAME, GREPTIME_PIPELINE_NAME_HEADER_NAME,
};
use servers::http::header::{GREPTIME_DB_HEADER_NAME, GREPTIME_TIMEZONE_HEADER_NAME};
use servers::http::prometheus::{PrometheusJsonResponse, PrometheusResponse};
use servers::http::prometheus::{Column, PrometheusJsonResponse, PrometheusResponse};
use servers::http::result::error_result::ErrorResponse;
use servers::http::result::greptime_result_v1::GreptimedbV1Response;
use servers::http::result::influxdb_result_v1::{InfluxdbOutput, InfluxdbV1Response};
@@ -849,10 +849,10 @@ pub async fn test_prom_http_api(store_type: StorageType) {
let actual = series
.remove(0)
.into_iter()
.collect::<BTreeMap<String, String>>();
.collect::<BTreeMap<Column, String>>();
let expected = BTreeMap::from([
("__name__".to_string(), "demo".to_string()),
("host".to_string(), "host1".to_string()),
("__name__".into(), "demo".to_string()),
("host".into(), "host1".to_string()),
]);
assert_eq!(actual, expected);

View File

@@ -10,6 +10,7 @@ DESC TABLE information_schema.ssts_manifest;
| region_group | UInt8 | | NO | | FIELD |
| region_sequence | UInt32 | | NO | | FIELD |
| file_id | String | | NO | | FIELD |
| index_file_id | String | | YES | | FIELD |
| level | UInt8 | | NO | | FIELD |
| file_path | String | | NO | | FIELD |
| file_size | UInt64 | | NO | | FIELD |
@@ -96,13 +97,13 @@ ADMIN FLUSH_TABLE('sst_case');
-- SQLNESS REPLACE (/public/\d+) /public/<TABLE_ID>
SELECT * FROM information_schema.ssts_manifest order by file_path;
+----------------------------+---------------+----------+---------------+--------------+-----------------+--------------------------------------+-------+----------------------------------------------------------------------------------------+-----------+---------------------------------------------------------------------------------------------+-----------------+----------+----------------+------------+-------------------------+-------------------------+----------+------------------+---------+---------+
| table_dir | region_id | table_id | region_number | region_group | region_sequence | file_id | level | file_path | file_size | index_file_path | index_file_size | num_rows | num_row_groups | num_series | min_ts | max_ts | sequence | origin_region_id | node_id | visible |
+----------------------------+---------------+----------+---------------+--------------+-----------------+--------------------------------------+-------+----------------------------------------------------------------------------------------+-----------+---------------------------------------------------------------------------------------------+-----------------+----------+----------------+------------+-------------------------+-------------------------+----------+------------------+---------+---------+
| data/greptime/public/<TABLE_ID>/ |<NUM>|<NUM>|<NUM>|<NUM>|<NUM>| <UUID> |<NUM>| data/greptime/public/<TABLE_ID>/<REGION_ID>_<REGION_NUMBER>/<UUID>.parquet |<NUM>| data/greptime/public/<TABLE_ID>/<REGION_ID>_<REGION_NUMBER>/index/<UUID>.puffin |<NUM>|<NUM>|<NUM>|<NUM>| <DATETIME> | <DATETIME> |<NUM>|<NUM>|<NUM>| true |
| data/greptime/public/<TABLE_ID>/ |<NUM>|<NUM>|<NUM>|<NUM>|<NUM>| <UUID> |<NUM>| data/greptime/public/<TABLE_ID>/<REGION_ID>_<REGION_NUMBER>/<UUID>.parquet |<NUM>| data/greptime/public/<TABLE_ID>/<REGION_ID>_<REGION_NUMBER>/index/<UUID>.puffin |<NUM>|<NUM>|<NUM>|<NUM>| <DATETIME> | <DATETIME> |<NUM>|<NUM>|<NUM>| true |
| data/greptime/public/<TABLE_ID>/ |<NUM>|<NUM>|<NUM>|<NUM>|<NUM>| <UUID> |<NUM>| data/greptime/public/<TABLE_ID>/<REGION_ID>_<REGION_NUMBER>/<UUID>.parquet |<NUM>| data/greptime/public/<TABLE_ID>/<REGION_ID>_<REGION_NUMBER>/index/<UUID>.puffin |<NUM>|<NUM>|<NUM>|<NUM>| <DATETIME> | <DATETIME> |<NUM>|<NUM>|<NUM>| true |
+----------------------------+---------------+----------+---------------+--------------+-----------------+--------------------------------------+-------+----------------------------------------------------------------------------------------+-----------+---------------------------------------------------------------------------------------------+-----------------+----------+----------------+------------+-------------------------+-------------------------+----------+------------------+---------+---------+
+----------------------------+---------------+----------+---------------+--------------+-----------------+--------------------------------------+--------------------------------------+-------+----------------------------------------------------------------------------------------+-----------+---------------------------------------------------------------------------------------------+-----------------+----------+----------------+------------+-------------------------+-------------------------+----------+------------------+---------+---------+
| table_dir | region_id | table_id | region_number | region_group | region_sequence | file_id | index_file_id | level | file_path | file_size | index_file_path | index_file_size | num_rows | num_row_groups | num_series | min_ts | max_ts | sequence | origin_region_id | node_id | visible |
+----------------------------+---------------+----------+---------------+--------------+-----------------+--------------------------------------+--------------------------------------+-------+----------------------------------------------------------------------------------------+-----------+---------------------------------------------------------------------------------------------+-----------------+----------+----------------+------------+-------------------------+-------------------------+----------+------------------+---------+---------+
| data/greptime/public/<TABLE_ID>/ |<NUM>|<NUM>|<NUM>|<NUM>|<NUM>| <UUID> | <UUID> |<NUM>| data/greptime/public/<TABLE_ID>/<REGION_ID>_<REGION_NUMBER>/<UUID>.parquet |<NUM>| data/greptime/public/<TABLE_ID>/<REGION_ID>_<REGION_NUMBER>/index/<UUID>.puffin |<NUM>|<NUM>|<NUM>|<NUM>| <DATETIME> | <DATETIME> |<NUM>|<NUM>|<NUM>| true |
| data/greptime/public/<TABLE_ID>/ |<NUM>|<NUM>|<NUM>|<NUM>|<NUM>| <UUID> | <UUID> |<NUM>| data/greptime/public/<TABLE_ID>/<REGION_ID>_<REGION_NUMBER>/<UUID>.parquet |<NUM>| data/greptime/public/<TABLE_ID>/<REGION_ID>_<REGION_NUMBER>/index/<UUID>.puffin |<NUM>|<NUM>|<NUM>|<NUM>| <DATETIME> | <DATETIME> |<NUM>|<NUM>|<NUM>| true |
| data/greptime/public/<TABLE_ID>/ |<NUM>|<NUM>|<NUM>|<NUM>|<NUM>| <UUID> | <UUID> |<NUM>| data/greptime/public/<TABLE_ID>/<REGION_ID>_<REGION_NUMBER>/<UUID>.parquet |<NUM>| data/greptime/public/<TABLE_ID>/<REGION_ID>_<REGION_NUMBER>/index/<UUID>.puffin |<NUM>|<NUM>|<NUM>|<NUM>| <DATETIME> | <DATETIME> |<NUM>|<NUM>|<NUM>| true |
+----------------------------+---------------+----------+---------------+--------------+-----------------+--------------------------------------+--------------------------------------+-------+----------------------------------------------------------------------------------------+-----------+---------------------------------------------------------------------------------------------+-----------------+----------+----------------+------------+-------------------------+-------------------------+----------+------------------+---------+---------+
-- SQLNESS REPLACE (\s+\d+\s+) <NUM>
-- SQLNESS REPLACE ([0-9a-fA-F]{8}-[0-9a-fA-F]{4}-[0-9a-fA-F]{4}-[0-9a-fA-F]{4}-[0-9a-fA-F]{12}) <UUID>
@@ -164,15 +165,15 @@ ADMIN FLUSH_TABLE('sst_case');
-- SQLNESS REPLACE (/public/\d+) /public/<TABLE_ID>
SELECT * FROM information_schema.ssts_manifest order by file_path;
+----------------------------+---------------+----------+---------------+--------------+-----------------+--------------------------------------+-------+----------------------------------------------------------------------------------------+-----------+---------------------------------------------------------------------------------------------+-----------------+----------+----------------+------------+-------------------------+-------------------------+----------+------------------+---------+---------+
| table_dir | region_id | table_id | region_number | region_group | region_sequence | file_id | level | file_path | file_size | index_file_path | index_file_size | num_rows | num_row_groups | num_series | min_ts | max_ts | sequence | origin_region_id | node_id | visible |
+----------------------------+---------------+----------+---------------+--------------+-----------------+--------------------------------------+-------+----------------------------------------------------------------------------------------+-----------+---------------------------------------------------------------------------------------------+-----------------+----------+----------------+------------+-------------------------+-------------------------+----------+------------------+---------+---------+
| data/greptime/public/<TABLE_ID>/ |<NUM>|<NUM>|<NUM>|<NUM>|<NUM>| <UUID> |<NUM>| data/greptime/public/<TABLE_ID>/<REGION_ID>_<REGION_NUMBER>/<UUID>.parquet |<NUM>| data/greptime/public/<TABLE_ID>/<REGION_ID>_<REGION_NUMBER>/index/<UUID>.puffin |<NUM>|<NUM>|<NUM>|<NUM>| <DATETIME> | <DATETIME> |<NUM>|<NUM>|<NUM>| true |
| data/greptime/public/<TABLE_ID>/ |<NUM>|<NUM>|<NUM>|<NUM>|<NUM>| <UUID> |<NUM>| data/greptime/public/<TABLE_ID>/<REGION_ID>_<REGION_NUMBER>/<UUID>.parquet |<NUM>| data/greptime/public/<TABLE_ID>/<REGION_ID>_<REGION_NUMBER>/index/<UUID>.puffin |<NUM>|<NUM>|<NUM>|<NUM>| <DATETIME> | <DATETIME> |<NUM>|<NUM>|<NUM>| true |
| data/greptime/public/<TABLE_ID>/ |<NUM>|<NUM>|<NUM>|<NUM>|<NUM>| <UUID> |<NUM>| data/greptime/public/<TABLE_ID>/<REGION_ID>_<REGION_NUMBER>/<UUID>.parquet |<NUM>| data/greptime/public/<TABLE_ID>/<REGION_ID>_<REGION_NUMBER>/index/<UUID>.puffin |<NUM>|<NUM>|<NUM>|<NUM>| <DATETIME> | <DATETIME> |<NUM>|<NUM>|<NUM>| true |
| data/greptime/public/<TABLE_ID>/ |<NUM>|<NUM>|<NUM>|<NUM>|<NUM>| <UUID> |<NUM>| data/greptime/public/<TABLE_ID>/<REGION_ID>_<REGION_NUMBER>/<UUID>.parquet |<NUM>| data/greptime/public/<TABLE_ID>/<REGION_ID>_<REGION_NUMBER>/index/<UUID>.puffin |<NUM>|<NUM>|<NUM>|<NUM>| <DATETIME> | <DATETIME> |<NUM>|<NUM>|<NUM>| true |
| data/greptime/public/<TABLE_ID>/ |<NUM>|<NUM>|<NUM>|<NUM>|<NUM>| <UUID> |<NUM>| data/greptime/public/<TABLE_ID>/<REGION_ID>_<REGION_NUMBER>/<UUID>.parquet |<NUM>| data/greptime/public/<TABLE_ID>/<REGION_ID>_<REGION_NUMBER>/index/<UUID>.puffin |<NUM>|<NUM>|<NUM>|<NUM>| <DATETIME> | <DATETIME> |<NUM>|<NUM>|<NUM>| true |
+----------------------------+---------------+----------+---------------+--------------+-----------------+--------------------------------------+-------+----------------------------------------------------------------------------------------+-----------+---------------------------------------------------------------------------------------------+-----------------+----------+----------------+------------+-------------------------+-------------------------+----------+------------------+---------+---------+
+----------------------------+---------------+----------+---------------+--------------+-----------------+--------------------------------------+--------------------------------------+-------+----------------------------------------------------------------------------------------+-----------+---------------------------------------------------------------------------------------------+-----------------+----------+----------------+------------+-------------------------+-------------------------+----------+------------------+---------+---------+
| table_dir | region_id | table_id | region_number | region_group | region_sequence | file_id | index_file_id | level | file_path | file_size | index_file_path | index_file_size | num_rows | num_row_groups | num_series | min_ts | max_ts | sequence | origin_region_id | node_id | visible |
+----------------------------+---------------+----------+---------------+--------------+-----------------+--------------------------------------+--------------------------------------+-------+----------------------------------------------------------------------------------------+-----------+---------------------------------------------------------------------------------------------+-----------------+----------+----------------+------------+-------------------------+-------------------------+----------+------------------+---------+---------+
| data/greptime/public/<TABLE_ID>/ |<NUM>|<NUM>|<NUM>|<NUM>|<NUM>| <UUID> | <UUID> |<NUM>| data/greptime/public/<TABLE_ID>/<REGION_ID>_<REGION_NUMBER>/<UUID>.parquet |<NUM>| data/greptime/public/<TABLE_ID>/<REGION_ID>_<REGION_NUMBER>/index/<UUID>.puffin |<NUM>|<NUM>|<NUM>|<NUM>| <DATETIME> | <DATETIME> |<NUM>|<NUM>|<NUM>| true |
| data/greptime/public/<TABLE_ID>/ |<NUM>|<NUM>|<NUM>|<NUM>|<NUM>| <UUID> | <UUID> |<NUM>| data/greptime/public/<TABLE_ID>/<REGION_ID>_<REGION_NUMBER>/<UUID>.parquet |<NUM>| data/greptime/public/<TABLE_ID>/<REGION_ID>_<REGION_NUMBER>/index/<UUID>.puffin |<NUM>|<NUM>|<NUM>|<NUM>| <DATETIME> | <DATETIME> |<NUM>|<NUM>|<NUM>| true |
| data/greptime/public/<TABLE_ID>/ |<NUM>|<NUM>|<NUM>|<NUM>|<NUM>| <UUID> | <UUID> |<NUM>| data/greptime/public/<TABLE_ID>/<REGION_ID>_<REGION_NUMBER>/<UUID>.parquet |<NUM>| data/greptime/public/<TABLE_ID>/<REGION_ID>_<REGION_NUMBER>/index/<UUID>.puffin |<NUM>|<NUM>|<NUM>|<NUM>| <DATETIME> | <DATETIME> |<NUM>|<NUM>|<NUM>| true |
| data/greptime/public/<TABLE_ID>/ |<NUM>|<NUM>|<NUM>|<NUM>|<NUM>| <UUID> | <UUID> |<NUM>| data/greptime/public/<TABLE_ID>/<REGION_ID>_<REGION_NUMBER>/<UUID>.parquet |<NUM>| data/greptime/public/<TABLE_ID>/<REGION_ID>_<REGION_NUMBER>/index/<UUID>.puffin |<NUM>|<NUM>|<NUM>|<NUM>| <DATETIME> | <DATETIME> |<NUM>|<NUM>|<NUM>| true |
| data/greptime/public/<TABLE_ID>/ |<NUM>|<NUM>|<NUM>|<NUM>|<NUM>| <UUID> | <UUID> |<NUM>| data/greptime/public/<TABLE_ID>/<REGION_ID>_<REGION_NUMBER>/<UUID>.parquet |<NUM>| data/greptime/public/<TABLE_ID>/<REGION_ID>_<REGION_NUMBER>/index/<UUID>.puffin |<NUM>|<NUM>|<NUM>|<NUM>| <DATETIME> | <DATETIME> |<NUM>|<NUM>|<NUM>| true |
+----------------------------+---------------+----------+---------------+--------------+-----------------+--------------------------------------+--------------------------------------+-------+----------------------------------------------------------------------------------------+-----------+---------------------------------------------------------------------------------------------+-----------------+----------+----------------+------------+-------------------------+-------------------------+----------+------------------+---------+---------+
-- SQLNESS REPLACE (\s+\d+\s+) <NUM>
-- SQLNESS REPLACE ([0-9a-fA-F]{8}-[0-9a-fA-F]{4}-[0-9a-fA-F]{4}-[0-9a-fA-F]{4}-[0-9a-fA-F]{12}) <UUID>

View File

@@ -406,26 +406,27 @@ select * from information_schema.columns order by table_schema, table_name, colu
| greptime | information_schema | ssts_index_meta | target_key | 12 | 2147483647 | 2147483647 | | | | utf8 | utf8_bin | | | select,insert | | String | string | FIELD | | No | string | | |
| greptime | information_schema | ssts_index_meta | target_type | 11 | 2147483647 | 2147483647 | | | | utf8 | utf8_bin | | | select,insert | | String | string | FIELD | | No | string | | |
| greptime | information_schema | ssts_manifest | file_id | 7 | 2147483647 | 2147483647 | | | | utf8 | utf8_bin | | | select,insert | | String | string | FIELD | | No | string | | |
| greptime | information_schema | ssts_manifest | file_path | 9 | 2147483647 | 2147483647 | | | | utf8 | utf8_bin | | | select,insert | | String | string | FIELD | | No | string | | |
| greptime | information_schema | ssts_manifest | file_size | 10 | | | 20 | 0 | | | | | | select,insert | | UInt64 | bigint unsigned | FIELD | | No | bigint unsigned | | |
| greptime | information_schema | ssts_manifest | index_file_path | 11 | 2147483647 | 2147483647 | | | | utf8 | utf8_bin | | | select,insert | | String | string | FIELD | | Yes | string | | |
| greptime | information_schema | ssts_manifest | index_file_size | 12 | | | 20 | 0 | | | | | | select,insert | | UInt64 | bigint unsigned | FIELD | | Yes | bigint unsigned | | |
| greptime | information_schema | ssts_manifest | level | 8 | | | 3 | 0 | | | | | | select,insert | | UInt8 | tinyint unsigned | FIELD | | No | tinyint unsigned | | |
| greptime | information_schema | ssts_manifest | max_ts | 17 | | | | | 9 | | | | | select,insert | | TimestampNanosecond | timestamp(9) | FIELD | | Yes | timestamp(9) | | |
| greptime | information_schema | ssts_manifest | min_ts | 16 | | | | | 9 | | | | | select,insert | | TimestampNanosecond | timestamp(9) | FIELD | | Yes | timestamp(9) | | |
| greptime | information_schema | ssts_manifest | node_id | 20 | | | 20 | 0 | | | | | | select,insert | | UInt64 | bigint unsigned | FIELD | | Yes | bigint unsigned | | |
| greptime | information_schema | ssts_manifest | num_row_groups | 14 | | | 20 | 0 | | | | | | select,insert | | UInt64 | bigint unsigned | FIELD | | No | bigint unsigned | | |
| greptime | information_schema | ssts_manifest | num_rows | 13 | | | 20 | 0 | | | | | | select,insert | | UInt64 | bigint unsigned | FIELD | | No | bigint unsigned | | |
| greptime | information_schema | ssts_manifest | num_series | 15 | | | 20 | 0 | | | | | | select,insert | | UInt64 | bigint unsigned | FIELD | | Yes | bigint unsigned | | |
| greptime | information_schema | ssts_manifest | origin_region_id | 19 | | | 20 | 0 | | | | | | select,insert | | UInt64 | bigint unsigned | FIELD | | No | bigint unsigned | | |
| greptime | information_schema | ssts_manifest | file_path | 10 | 2147483647 | 2147483647 | | | | utf8 | utf8_bin | | | select,insert | | String | string | FIELD | | No | string | | |
| greptime | information_schema | ssts_manifest | file_size | 11 | | | 20 | 0 | | | | | | select,insert | | UInt64 | bigint unsigned | FIELD | | No | bigint unsigned | | |
| greptime | information_schema | ssts_manifest | index_file_id | 8 | 2147483647 | 2147483647 | | | | utf8 | utf8_bin | | | select,insert | | String | string | FIELD | | Yes | string | | |
| greptime | information_schema | ssts_manifest | index_file_path | 12 | 2147483647 | 2147483647 | | | | utf8 | utf8_bin | | | select,insert | | String | string | FIELD | | Yes | string | | |
| greptime | information_schema | ssts_manifest | index_file_size | 13 | | | 20 | 0 | | | | | | select,insert | | UInt64 | bigint unsigned | FIELD | | Yes | bigint unsigned | | |
| greptime | information_schema | ssts_manifest | level | 9 | | | 3 | 0 | | | | | | select,insert | | UInt8 | tinyint unsigned | FIELD | | No | tinyint unsigned | | |
| greptime | information_schema | ssts_manifest | max_ts | 18 | | | | | 9 | | | | | select,insert | | TimestampNanosecond | timestamp(9) | FIELD | | Yes | timestamp(9) | | |
| greptime | information_schema | ssts_manifest | min_ts | 17 | | | | | 9 | | | | | select,insert | | TimestampNanosecond | timestamp(9) | FIELD | | Yes | timestamp(9) | | |
| greptime | information_schema | ssts_manifest | node_id | 21 | | | 20 | 0 | | | | | | select,insert | | UInt64 | bigint unsigned | FIELD | | Yes | bigint unsigned | | |
| greptime | information_schema | ssts_manifest | num_row_groups | 15 | | | 20 | 0 | | | | | | select,insert | | UInt64 | bigint unsigned | FIELD | | No | bigint unsigned | | |
| greptime | information_schema | ssts_manifest | num_rows | 14 | | | 20 | 0 | | | | | | select,insert | | UInt64 | bigint unsigned | FIELD | | No | bigint unsigned | | |
| greptime | information_schema | ssts_manifest | num_series | 16 | | | 20 | 0 | | | | | | select,insert | | UInt64 | bigint unsigned | FIELD | | Yes | bigint unsigned | | |
| greptime | information_schema | ssts_manifest | origin_region_id | 20 | | | 20 | 0 | | | | | | select,insert | | UInt64 | bigint unsigned | FIELD | | No | bigint unsigned | | |
| greptime | information_schema | ssts_manifest | region_group | 5 | | | 3 | 0 | | | | | | select,insert | | UInt8 | tinyint unsigned | FIELD | | No | tinyint unsigned | | |
| greptime | information_schema | ssts_manifest | region_id | 2 | | | 20 | 0 | | | | | | select,insert | | UInt64 | bigint unsigned | FIELD | | No | bigint unsigned | | |
| greptime | information_schema | ssts_manifest | region_number | 4 | | | 10 | 0 | | | | | | select,insert | | UInt32 | int unsigned | FIELD | | No | int unsigned | | |
| greptime | information_schema | ssts_manifest | region_sequence | 6 | | | 10 | 0 | | | | | | select,insert | | UInt32 | int unsigned | FIELD | | No | int unsigned | | |
| greptime | information_schema | ssts_manifest | sequence | 18 | | | 20 | 0 | | | | | | select,insert | | UInt64 | bigint unsigned | FIELD | | Yes | bigint unsigned | | |
| greptime | information_schema | ssts_manifest | sequence | 19 | | | 20 | 0 | | | | | | select,insert | | UInt64 | bigint unsigned | FIELD | | Yes | bigint unsigned | | |
| greptime | information_schema | ssts_manifest | table_dir | 1 | 2147483647 | 2147483647 | | | | utf8 | utf8_bin | | | select,insert | | String | string | FIELD | | No | string | | |
| greptime | information_schema | ssts_manifest | table_id | 3 | | | 10 | 0 | | | | | | select,insert | | UInt32 | int unsigned | FIELD | | No | int unsigned | | |
| greptime | information_schema | ssts_manifest | visible | 21 | | | | | | | | | | select,insert | | Boolean | boolean | FIELD | | No | boolean | | |
| greptime | information_schema | ssts_manifest | visible | 22 | | | | | | | | | | select,insert | | Boolean | boolean | FIELD | | No | boolean | | |
| greptime | information_schema | ssts_storage | file_path | 1 | 2147483647 | 2147483647 | | | | utf8 | utf8_bin | | | select,insert | | String | string | FIELD | | No | string | | |
| greptime | information_schema | ssts_storage | file_size | 2 | | | 20 | 0 | | | | | | select,insert | | UInt64 | bigint unsigned | FIELD | | Yes | bigint unsigned | | |
| greptime | information_schema | ssts_storage | last_modified_ms | 3 | | | | | 3 | | | | | select,insert | | TimestampMillisecond | timestamp(3) | FIELD | | Yes | timestamp(3) | | |