mirror of
https://github.com/GreptimeTeam/greptimedb.git
synced 2025-12-22 22:20:02 +00:00
Compare commits
23 Commits
v0.9.0-nig
...
v0.8.1
| Author | SHA1 | Date | |
|---|---|---|---|
|
|
05751084e7 | ||
|
|
8b6596faa0 | ||
|
|
eab309ff7e | ||
|
|
7de336f087 | ||
|
|
6e9a9dc333 | ||
|
|
848bd7e553 | ||
|
|
f0effd2680 | ||
|
|
aafb468547 | ||
|
|
4aa756c896 | ||
|
|
d3860671a8 | ||
|
|
9dd6e033a7 | ||
|
|
097f62f459 | ||
|
|
048368fd87 | ||
|
|
f9db5ff0d6 | ||
|
|
20ce7d428d | ||
|
|
75bddc0bf5 | ||
|
|
c78043d526 | ||
|
|
297105266b | ||
|
|
1de17aec74 | ||
|
|
389ded93d1 | ||
|
|
af486ec0d0 | ||
|
|
25d64255a3 | ||
|
|
3790020d78 |
570
Cargo.lock
generated
570
Cargo.lock
generated
File diff suppressed because it is too large
Load Diff
10
Cargo.toml
10
Cargo.toml
@@ -64,7 +64,7 @@ members = [
|
||||
resolver = "2"
|
||||
|
||||
[workspace.package]
|
||||
version = "0.8.0"
|
||||
version = "0.8.1"
|
||||
edition = "2021"
|
||||
license = "Apache-2.0"
|
||||
|
||||
@@ -120,7 +120,7 @@ etcd-client = { git = "https://github.com/MichaelScofield/etcd-client.git", rev
|
||||
fst = "0.4.7"
|
||||
futures = "0.3"
|
||||
futures-util = "0.3"
|
||||
greptime-proto = { git = "https://github.com/GreptimeTeam/greptime-proto.git", rev = "3cd71167ee067c5679a7fb17cf58bdfbb5487a0d" }
|
||||
greptime-proto = { git = "https://github.com/GreptimeTeam/greptime-proto.git", rev = "902f75fdd170c572e90b1f640161d90995f20218" }
|
||||
humantime = "2.1"
|
||||
humantime-serde = "1.1"
|
||||
itertools = "0.10"
|
||||
@@ -140,12 +140,13 @@ parquet = { version = "51.0.0", default-features = false, features = ["arrow", "
|
||||
paste = "1.0"
|
||||
pin-project = "1.0"
|
||||
prometheus = { version = "0.13.3", features = ["process"] }
|
||||
promql-parser = { version = "0.4" }
|
||||
prost = "0.12"
|
||||
raft-engine = { version = "0.4.1", default-features = false }
|
||||
rand = "0.8"
|
||||
regex = "1.8"
|
||||
regex-automata = { version = "0.4" }
|
||||
reqwest = { version = "0.11", default-features = false, features = [
|
||||
reqwest = { version = "0.12", default-features = false, features = [
|
||||
"json",
|
||||
"rustls-tls-native-roots",
|
||||
"stream",
|
||||
@@ -171,6 +172,7 @@ tokio-stream = { version = "0.1" }
|
||||
tokio-util = { version = "0.7", features = ["io-util", "compat"] }
|
||||
toml = "0.8.8"
|
||||
tonic = { version = "0.11", features = ["tls", "gzip", "zstd"] }
|
||||
tower = { version = "0.4" }
|
||||
uuid = { version = "1.7", features = ["serde", "v4", "fast-rng"] }
|
||||
zstd = "0.13"
|
||||
|
||||
@@ -231,8 +233,6 @@ sql = { path = "src/sql" }
|
||||
store-api = { path = "src/store-api" }
|
||||
substrait = { path = "src/common/substrait" }
|
||||
table = { path = "src/table" }
|
||||
# TODO some code depends on this
|
||||
tests-integration = { path = "tests-integration" }
|
||||
|
||||
[workspace.dependencies.meter-macros]
|
||||
git = "https://github.com/GreptimeTeam/greptime-meter.git"
|
||||
|
||||
@@ -12,7 +12,7 @@ api.workspace = true
|
||||
arrow.workspace = true
|
||||
chrono.workspace = true
|
||||
clap.workspace = true
|
||||
client.workspace = true
|
||||
client = { workspace = true, features = ["testing"] }
|
||||
common-base.workspace = true
|
||||
common-telemetry.workspace = true
|
||||
common-wal.workspace = true
|
||||
@@ -33,8 +33,6 @@ rand.workspace = true
|
||||
rskafka.workspace = true
|
||||
serde.workspace = true
|
||||
store-api.workspace = true
|
||||
# TODO depend `Database` client
|
||||
tests-integration.workspace = true
|
||||
tokio.workspace = true
|
||||
toml.workspace = true
|
||||
uuid.workspace = true
|
||||
|
||||
@@ -28,6 +28,7 @@ use rand::distributions::{Alphanumeric, DistString, Uniform};
|
||||
use rand::rngs::SmallRng;
|
||||
use rand::{Rng, SeedableRng};
|
||||
use serde::{Deserialize, Serialize};
|
||||
use store_api::logstore::provider::Provider;
|
||||
use store_api::logstore::LogStore;
|
||||
use store_api::storage::RegionId;
|
||||
|
||||
@@ -210,7 +211,7 @@ impl From<Args> for Config {
|
||||
pub struct Region {
|
||||
id: RegionId,
|
||||
schema: Vec<ColumnSchema>,
|
||||
wal_options: WalOptions,
|
||||
provider: Provider,
|
||||
next_sequence: AtomicU64,
|
||||
next_entry_id: AtomicU64,
|
||||
next_timestamp: AtomicI64,
|
||||
@@ -227,10 +228,14 @@ impl Region {
|
||||
num_rows: u32,
|
||||
rng_seed: u64,
|
||||
) -> Self {
|
||||
let provider = match wal_options {
|
||||
WalOptions::RaftEngine => Provider::raft_engine_provider(id.as_u64()),
|
||||
WalOptions::Kafka(opts) => Provider::kafka_provider(opts.topic),
|
||||
};
|
||||
Self {
|
||||
id,
|
||||
schema,
|
||||
wal_options,
|
||||
provider,
|
||||
next_sequence: AtomicU64::new(1),
|
||||
next_entry_id: AtomicU64::new(1),
|
||||
next_timestamp: AtomicI64::new(1655276557000),
|
||||
@@ -258,14 +263,14 @@ impl Region {
|
||||
self.id,
|
||||
self.next_entry_id.fetch_add(1, Ordering::Relaxed),
|
||||
&entry,
|
||||
&self.wal_options,
|
||||
&self.provider,
|
||||
)
|
||||
.unwrap();
|
||||
}
|
||||
|
||||
/// Replays the region.
|
||||
pub async fn replay<S: LogStore>(&self, wal: &Arc<Wal<S>>) {
|
||||
let mut wal_stream = wal.scan(self.id, 0, &self.wal_options).unwrap();
|
||||
let mut wal_stream = wal.scan(self.id, 0, &self.provider).unwrap();
|
||||
while let Some(res) = wal_stream.next().await {
|
||||
let (_, entry) = res.unwrap();
|
||||
metrics::METRIC_WAL_READ_BYTES_TOTAL.inc_by(Self::entry_estimated_size(&entry) as u64);
|
||||
|
||||
@@ -23,8 +23,6 @@ use api::v1::{
|
||||
};
|
||||
use arrow_flight::Ticket;
|
||||
use async_stream::stream;
|
||||
use client::error::{ConvertFlightDataSnafu, Error, IllegalFlightMessagesSnafu, ServerSnafu};
|
||||
use client::{from_grpc_response, Client, Result};
|
||||
use common_error::ext::{BoxedError, ErrorExt};
|
||||
use common_grpc::flight::{FlightDecoder, FlightMessage};
|
||||
use common_query::Output;
|
||||
@@ -37,7 +35,8 @@ use prost::Message;
|
||||
use snafu::{ensure, ResultExt};
|
||||
use tonic::transport::Channel;
|
||||
|
||||
pub const DEFAULT_LOOKBACK_STRING: &str = "5m";
|
||||
use crate::error::{ConvertFlightDataSnafu, Error, IllegalFlightMessagesSnafu, ServerSnafu};
|
||||
use crate::{from_grpc_response, Client, Result};
|
||||
|
||||
#[derive(Clone, Debug, Default)]
|
||||
pub struct Database {
|
||||
@@ -105,10 +104,18 @@ impl Database {
|
||||
self.catalog = catalog.into();
|
||||
}
|
||||
|
||||
pub fn catalog(&self) -> &String {
|
||||
&self.catalog
|
||||
}
|
||||
|
||||
pub fn set_schema(&mut self, schema: impl Into<String>) {
|
||||
self.schema = schema.into();
|
||||
}
|
||||
|
||||
pub fn schema(&self) -> &String {
|
||||
&self.schema
|
||||
}
|
||||
|
||||
pub fn set_timezone(&mut self, timezone: impl Into<String>) {
|
||||
self.timezone = timezone.into();
|
||||
}
|
||||
@@ -156,6 +163,13 @@ impl Database {
|
||||
.await
|
||||
}
|
||||
|
||||
pub async fn logical_plan(&self, logical_plan: Vec<u8>) -> Result<Output> {
|
||||
self.do_get(Request::Query(QueryRequest {
|
||||
query: Some(Query::LogicalPlan(logical_plan)),
|
||||
}))
|
||||
.await
|
||||
}
|
||||
|
||||
pub async fn create(&self, expr: CreateTableExpr) -> Result<Output> {
|
||||
self.do_get(Request::Ddl(DdlRequest {
|
||||
expr: Some(DdlExpr::CreateTable(expr)),
|
||||
@@ -269,17 +283,12 @@ struct FlightContext {
|
||||
|
||||
#[cfg(test)]
|
||||
mod tests {
|
||||
use std::assert_matches::assert_matches;
|
||||
|
||||
use api::v1::auth_header::AuthScheme;
|
||||
use api::v1::{AuthHeader, Basic};
|
||||
use clap::Parser;
|
||||
use client::Client;
|
||||
use cmd::error::Result as CmdResult;
|
||||
use cmd::options::GlobalOptions;
|
||||
use cmd::{cli, standalone, App};
|
||||
use common_catalog::consts::{DEFAULT_CATALOG_NAME, DEFAULT_SCHEMA_NAME};
|
||||
use common_telemetry::logging::LoggingOptions;
|
||||
|
||||
use super::{Database, FlightContext};
|
||||
use super::*;
|
||||
|
||||
#[test]
|
||||
fn test_flight_ctx() {
|
||||
@@ -295,76 +304,11 @@ mod tests {
|
||||
auth_scheme: Some(basic),
|
||||
});
|
||||
|
||||
assert!(matches!(
|
||||
assert_matches!(
|
||||
ctx.auth_header,
|
||||
Some(AuthHeader {
|
||||
auth_scheme: Some(AuthScheme::Basic(_)),
|
||||
})
|
||||
))
|
||||
}
|
||||
|
||||
#[tokio::test(flavor = "multi_thread")]
|
||||
async fn test_export_create_table_with_quoted_names() -> CmdResult<()> {
|
||||
let output_dir = tempfile::tempdir().unwrap();
|
||||
|
||||
let standalone = standalone::Command::parse_from([
|
||||
"standalone",
|
||||
"start",
|
||||
"--data-home",
|
||||
&*output_dir.path().to_string_lossy(),
|
||||
]);
|
||||
|
||||
let standalone_opts = standalone.load_options(&GlobalOptions::default()).unwrap();
|
||||
let mut instance = standalone.build(standalone_opts).await?;
|
||||
instance.start().await?;
|
||||
|
||||
let client = Client::with_urls(["127.0.0.1:4001"]);
|
||||
let database = Database::new(DEFAULT_CATALOG_NAME, DEFAULT_SCHEMA_NAME, client);
|
||||
database
|
||||
.sql(r#"CREATE DATABASE "cli.export.create_table";"#)
|
||||
.await
|
||||
.unwrap();
|
||||
database
|
||||
.sql(
|
||||
r#"CREATE TABLE "cli.export.create_table"."a.b.c"(
|
||||
ts TIMESTAMP,
|
||||
TIME INDEX (ts)
|
||||
) engine=mito;
|
||||
"#,
|
||||
)
|
||||
.await
|
||||
.unwrap();
|
||||
|
||||
let output_dir = tempfile::tempdir().unwrap();
|
||||
let cli = cli::Command::parse_from([
|
||||
"cli",
|
||||
"export",
|
||||
"--addr",
|
||||
"127.0.0.1:4000",
|
||||
"--output-dir",
|
||||
&*output_dir.path().to_string_lossy(),
|
||||
"--target",
|
||||
"create-table",
|
||||
]);
|
||||
let mut cli_app = cli.build(LoggingOptions::default()).await?;
|
||||
cli_app.start().await?;
|
||||
|
||||
instance.stop().await?;
|
||||
|
||||
let output_file = output_dir
|
||||
.path()
|
||||
.join("greptime-cli.export.create_table.sql");
|
||||
let res = std::fs::read_to_string(output_file).unwrap();
|
||||
let expect = r#"CREATE TABLE IF NOT EXISTS "a.b.c" (
|
||||
"ts" TIMESTAMP(3) NOT NULL,
|
||||
TIME INDEX ("ts")
|
||||
)
|
||||
|
||||
ENGINE=mito
|
||||
;
|
||||
"#;
|
||||
assert_eq!(res.trim(), expect.trim());
|
||||
|
||||
Ok(())
|
||||
)
|
||||
}
|
||||
}
|
||||
@@ -12,8 +12,12 @@
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
#![feature(assert_matches)]
|
||||
|
||||
mod client;
|
||||
pub mod client_manager;
|
||||
#[cfg(feature = "testing")]
|
||||
mod database;
|
||||
pub mod error;
|
||||
pub mod load_balance;
|
||||
mod metrics;
|
||||
@@ -29,6 +33,8 @@ pub use common_recordbatch::{RecordBatches, SendableRecordBatchStream};
|
||||
use snafu::OptionExt;
|
||||
|
||||
pub use self::client::Client;
|
||||
#[cfg(feature = "testing")]
|
||||
pub use self::database::Database;
|
||||
pub use self::error::{Error, Result};
|
||||
use crate::error::{IllegalDatabaseResponseSnafu, ServerSnafu};
|
||||
|
||||
|
||||
@@ -80,6 +80,7 @@ tracing-appender = "0.2"
|
||||
tikv-jemallocator = "0.5"
|
||||
|
||||
[dev-dependencies]
|
||||
client = { workspace = true, features = ["testing"] }
|
||||
common-test-util.workspace = true
|
||||
serde.workspace = true
|
||||
temp-env = "0.3"
|
||||
|
||||
@@ -22,8 +22,8 @@ mod helper;
|
||||
|
||||
// Wait for https://github.com/GreptimeTeam/greptimedb/issues/2373
|
||||
#[allow(unused)]
|
||||
// mod repl;
|
||||
// TODO(weny): Removes it
|
||||
mod repl;
|
||||
// TODO(tisonkun): migrate deprecated methods
|
||||
#[allow(deprecated)]
|
||||
mod upgrade;
|
||||
|
||||
@@ -31,8 +31,8 @@ use async_trait::async_trait;
|
||||
use bench::BenchTableMetadataCommand;
|
||||
use clap::Parser;
|
||||
use common_telemetry::logging::{LoggingOptions, TracingOptions};
|
||||
pub use repl::Repl;
|
||||
use tracing_appender::non_blocking::WorkerGuard;
|
||||
// pub use repl::Repl;
|
||||
use upgrade::UpgradeCommand;
|
||||
|
||||
use self::export::ExportCommand;
|
||||
|
||||
@@ -434,3 +434,80 @@ fn split_database(database: &str) -> Result<(String, Option<String>)> {
|
||||
Ok((catalog.to_string(), Some(schema.to_string())))
|
||||
}
|
||||
}
|
||||
|
||||
#[cfg(test)]
|
||||
mod tests {
|
||||
use clap::Parser;
|
||||
use client::{Client, Database};
|
||||
use common_catalog::consts::{DEFAULT_CATALOG_NAME, DEFAULT_SCHEMA_NAME};
|
||||
use common_telemetry::logging::LoggingOptions;
|
||||
|
||||
use crate::error::Result as CmdResult;
|
||||
use crate::options::GlobalOptions;
|
||||
use crate::{cli, standalone, App};
|
||||
|
||||
#[tokio::test(flavor = "multi_thread")]
|
||||
async fn test_export_create_table_with_quoted_names() -> CmdResult<()> {
|
||||
let output_dir = tempfile::tempdir().unwrap();
|
||||
|
||||
let standalone = standalone::Command::parse_from([
|
||||
"standalone",
|
||||
"start",
|
||||
"--data-home",
|
||||
&*output_dir.path().to_string_lossy(),
|
||||
]);
|
||||
|
||||
let standalone_opts = standalone.load_options(&GlobalOptions::default()).unwrap();
|
||||
let mut instance = standalone.build(standalone_opts).await?;
|
||||
instance.start().await?;
|
||||
|
||||
let client = Client::with_urls(["127.0.0.1:4001"]);
|
||||
let database = Database::new(DEFAULT_CATALOG_NAME, DEFAULT_SCHEMA_NAME, client);
|
||||
database
|
||||
.sql(r#"CREATE DATABASE "cli.export.create_table";"#)
|
||||
.await
|
||||
.unwrap();
|
||||
database
|
||||
.sql(
|
||||
r#"CREATE TABLE "cli.export.create_table"."a.b.c"(
|
||||
ts TIMESTAMP,
|
||||
TIME INDEX (ts)
|
||||
) engine=mito;
|
||||
"#,
|
||||
)
|
||||
.await
|
||||
.unwrap();
|
||||
|
||||
let output_dir = tempfile::tempdir().unwrap();
|
||||
let cli = cli::Command::parse_from([
|
||||
"cli",
|
||||
"export",
|
||||
"--addr",
|
||||
"127.0.0.1:4000",
|
||||
"--output-dir",
|
||||
&*output_dir.path().to_string_lossy(),
|
||||
"--target",
|
||||
"create-table",
|
||||
]);
|
||||
let mut cli_app = cli.build(LoggingOptions::default()).await?;
|
||||
cli_app.start().await?;
|
||||
|
||||
instance.stop().await?;
|
||||
|
||||
let output_file = output_dir
|
||||
.path()
|
||||
.join("greptime-cli.export.create_table.sql");
|
||||
let res = std::fs::read_to_string(output_file).unwrap();
|
||||
let expect = r#"CREATE TABLE IF NOT EXISTS "a.b.c" (
|
||||
"ts" TIMESTAMP(3) NOT NULL,
|
||||
TIME INDEX ("ts")
|
||||
)
|
||||
|
||||
ENGINE=mito
|
||||
;
|
||||
"#;
|
||||
assert_eq!(res.trim(), expect.trim());
|
||||
|
||||
Ok(())
|
||||
}
|
||||
}
|
||||
|
||||
@@ -16,14 +16,18 @@ use std::path::PathBuf;
|
||||
use std::sync::Arc;
|
||||
use std::time::Instant;
|
||||
|
||||
use catalog::kvbackend::{
|
||||
CachedMetaKvBackend, CachedMetaKvBackendBuilder, KvBackendCatalogManager,
|
||||
use cache::{
|
||||
build_fundamental_cache_registry, with_default_composite_cache_registry, TABLE_CACHE_NAME,
|
||||
TABLE_ROUTE_CACHE_NAME,
|
||||
};
|
||||
use client::{Client, OutputData, DEFAULT_CATALOG_NAME, DEFAULT_SCHEMA_NAME};
|
||||
use catalog::kvbackend::{
|
||||
CachedMetaKvBackend, CachedMetaKvBackendBuilder, KvBackendCatalogManager, MetaKvBackend,
|
||||
};
|
||||
use client::{Client, Database, OutputData, DEFAULT_CATALOG_NAME, DEFAULT_SCHEMA_NAME};
|
||||
use common_base::Plugins;
|
||||
use common_config::Mode;
|
||||
use common_error::ext::ErrorExt;
|
||||
use common_meta::cache_invalidator::MultiCacheInvalidator;
|
||||
use common_meta::cache::{CacheRegistryBuilder, LayeredCacheRegistryBuilder};
|
||||
use common_query::Output;
|
||||
use common_recordbatch::RecordBatches;
|
||||
use common_telemetry::debug;
|
||||
@@ -38,12 +42,13 @@ use query::QueryEngine;
|
||||
use rustyline::error::ReadlineError;
|
||||
use rustyline::Editor;
|
||||
use session::context::QueryContext;
|
||||
use snafu::ResultExt;
|
||||
use snafu::{OptionExt, ResultExt};
|
||||
use substrait::{DFLogicalSubstraitConvertor, SubstraitPlan};
|
||||
|
||||
use crate::cli::cmd::ReplCommand;
|
||||
use crate::cli::helper::RustylineHelper;
|
||||
use crate::cli::AttachCommand;
|
||||
use crate::error;
|
||||
use crate::error::{
|
||||
CollectRecordBatchesSnafu, ParseSqlSnafu, PlanStatementSnafu, PrettyPrintRecordBatchesSnafu,
|
||||
ReadlineSnafu, ReplCreationSnafu, RequestDatabaseSnafu, Result, StartMetaClientSnafu,
|
||||
@@ -257,19 +262,42 @@ async fn create_query_engine(meta_addr: &str) -> Result<DatafusionQueryEngine> {
|
||||
|
||||
let cached_meta_backend =
|
||||
Arc::new(CachedMetaKvBackendBuilder::new(meta_client.clone()).build());
|
||||
let multi_cache_invalidator = Arc::new(MultiCacheInvalidator::with_invalidators(vec![
|
||||
cached_meta_backend.clone(),
|
||||
]));
|
||||
let catalog_list = KvBackendCatalogManager::new(
|
||||
let layered_cache_builder = LayeredCacheRegistryBuilder::default().add_cache_registry(
|
||||
CacheRegistryBuilder::default()
|
||||
.add_cache(cached_meta_backend.clone())
|
||||
.build(),
|
||||
);
|
||||
let fundamental_cache_registry =
|
||||
build_fundamental_cache_registry(Arc::new(MetaKvBackend::new(meta_client.clone())));
|
||||
let layered_cache_registry = Arc::new(
|
||||
with_default_composite_cache_registry(
|
||||
layered_cache_builder.add_cache_registry(fundamental_cache_registry),
|
||||
)
|
||||
.context(error::BuildCacheRegistrySnafu)?
|
||||
.build(),
|
||||
);
|
||||
|
||||
let table_cache = layered_cache_registry
|
||||
.get()
|
||||
.context(error::CacheRequiredSnafu {
|
||||
name: TABLE_CACHE_NAME,
|
||||
})?;
|
||||
let table_route_cache = layered_cache_registry
|
||||
.get()
|
||||
.context(error::CacheRequiredSnafu {
|
||||
name: TABLE_ROUTE_CACHE_NAME,
|
||||
})?;
|
||||
let catalog_manager = KvBackendCatalogManager::new(
|
||||
Mode::Distributed,
|
||||
Some(meta_client.clone()),
|
||||
cached_meta_backend.clone(),
|
||||
multi_cache_invalidator,
|
||||
table_cache,
|
||||
table_route_cache,
|
||||
)
|
||||
.await;
|
||||
let plugins: Plugins = Default::default();
|
||||
let state = Arc::new(QueryEngineState::new(
|
||||
catalog_list,
|
||||
catalog_manager,
|
||||
None,
|
||||
None,
|
||||
None,
|
||||
|
||||
@@ -163,6 +163,15 @@ pub enum Error {
|
||||
location: Location,
|
||||
},
|
||||
|
||||
#[snafu(display("Failed to request database, sql: {sql}"))]
|
||||
RequestDatabase {
|
||||
sql: String,
|
||||
#[snafu(source)]
|
||||
source: client::Error,
|
||||
#[snafu(implicit)]
|
||||
location: Location,
|
||||
},
|
||||
|
||||
#[snafu(display("Failed to collect RecordBatches"))]
|
||||
CollectRecordBatches {
|
||||
#[snafu(implicit)]
|
||||
@@ -354,6 +363,7 @@ impl ErrorExt for Error {
|
||||
Error::ReplCreation { .. } | Error::Readline { .. } | Error::HttpQuerySql { .. } => {
|
||||
StatusCode::Internal
|
||||
}
|
||||
Error::RequestDatabase { source, .. } => source.status_code(),
|
||||
Error::CollectRecordBatches { source, .. }
|
||||
| Error::PrettyPrintRecordBatches { source, .. } => source.status_code(),
|
||||
Error::StartMetaClient { source, .. } => source.status_code(),
|
||||
|
||||
@@ -306,7 +306,7 @@ impl StartCommand {
|
||||
}
|
||||
|
||||
// The precedence order is: cli > config file > environment variables > default values.
|
||||
fn merge_with_cli_options(
|
||||
pub fn merge_with_cli_options(
|
||||
&self,
|
||||
global_options: &GlobalOptions,
|
||||
mut opts: StandaloneOptions,
|
||||
|
||||
@@ -92,34 +92,44 @@ impl CompressionType {
|
||||
macro_rules! impl_compression_type {
|
||||
($(($enum_item:ident, $prefix:ident)),*) => {
|
||||
paste::item! {
|
||||
use bytes::{Buf, BufMut, BytesMut};
|
||||
|
||||
impl CompressionType {
|
||||
pub async fn encode(&self, content: impl AsRef<[u8]>) -> io::Result<Vec<u8>> {
|
||||
pub async fn encode<B: Buf>(&self, mut content: B) -> io::Result<Vec<u8>> {
|
||||
match self {
|
||||
$(
|
||||
CompressionType::$enum_item => {
|
||||
let mut buffer = Vec::with_capacity(content.as_ref().len());
|
||||
let mut buffer = Vec::with_capacity(content.remaining());
|
||||
let mut encoder = write::[<$prefix Encoder>]::new(&mut buffer);
|
||||
encoder.write_all(content.as_ref()).await?;
|
||||
encoder.write_all_buf(&mut content).await?;
|
||||
encoder.shutdown().await?;
|
||||
Ok(buffer)
|
||||
}
|
||||
)*
|
||||
CompressionType::Uncompressed => Ok(content.as_ref().to_vec()),
|
||||
CompressionType::Uncompressed => {
|
||||
let mut bs = BytesMut::with_capacity(content.remaining());
|
||||
bs.put(content);
|
||||
Ok(bs.to_vec())
|
||||
},
|
||||
}
|
||||
}
|
||||
|
||||
pub async fn decode(&self, content: impl AsRef<[u8]>) -> io::Result<Vec<u8>> {
|
||||
pub async fn decode<B: Buf>(&self, mut content: B) -> io::Result<Vec<u8>> {
|
||||
match self {
|
||||
$(
|
||||
CompressionType::$enum_item => {
|
||||
let mut buffer = Vec::with_capacity(content.as_ref().len() * 2);
|
||||
let mut buffer = Vec::with_capacity(content.remaining() * 2);
|
||||
let mut encoder = write::[<$prefix Decoder>]::new(&mut buffer);
|
||||
encoder.write_all(content.as_ref()).await?;
|
||||
encoder.write_all_buf(&mut content).await?;
|
||||
encoder.shutdown().await?;
|
||||
Ok(buffer)
|
||||
}
|
||||
)*
|
||||
CompressionType::Uncompressed => Ok(content.as_ref().to_vec()),
|
||||
CompressionType::Uncompressed => {
|
||||
let mut bs = BytesMut::with_capacity(content.remaining());
|
||||
bs.put(content);
|
||||
Ok(bs.to_vec())
|
||||
},
|
||||
}
|
||||
}
|
||||
|
||||
@@ -151,13 +161,13 @@ macro_rules! impl_compression_type {
|
||||
$(
|
||||
#[tokio::test]
|
||||
async fn [<test_ $enum_item:lower _compression>]() {
|
||||
let string = "foo_bar".as_bytes().to_vec();
|
||||
let string = "foo_bar".as_bytes();
|
||||
let compress = CompressionType::$enum_item
|
||||
.encode(&string)
|
||||
.encode(string)
|
||||
.await
|
||||
.unwrap();
|
||||
let decompress = CompressionType::$enum_item
|
||||
.decode(&compress)
|
||||
.decode(compress.as_slice())
|
||||
.await
|
||||
.unwrap();
|
||||
assert_eq!(decompress, string);
|
||||
@@ -165,13 +175,13 @@ macro_rules! impl_compression_type {
|
||||
|
||||
#[tokio::test]
|
||||
async fn test_uncompression() {
|
||||
let string = "foo_bar".as_bytes().to_vec();
|
||||
let string = "foo_bar".as_bytes();
|
||||
let compress = CompressionType::Uncompressed
|
||||
.encode(&string)
|
||||
.encode(string)
|
||||
.await
|
||||
.unwrap();
|
||||
let decompress = CompressionType::Uncompressed
|
||||
.decode(&compress)
|
||||
.decode(compress.as_slice())
|
||||
.await
|
||||
.unwrap();
|
||||
assert_eq!(decompress, string);
|
||||
|
||||
@@ -36,6 +36,7 @@ use datafusion::physical_plan::SendableRecordBatchStream;
|
||||
use futures::StreamExt;
|
||||
use object_store::ObjectStore;
|
||||
use snafu::ResultExt;
|
||||
use tokio_util::compat::FuturesAsyncWriteCompatExt;
|
||||
|
||||
use self::csv::CsvFormat;
|
||||
use self::json::JsonFormat;
|
||||
@@ -146,7 +147,8 @@ pub fn open_with_decoder<T: ArrowDecoder, F: Fn() -> DataFusionResult<T>>(
|
||||
let reader = object_store
|
||||
.reader(&path)
|
||||
.await
|
||||
.map_err(|e| DataFusionError::External(Box::new(e)))?;
|
||||
.map_err(|e| DataFusionError::External(Box::new(e)))?
|
||||
.into_bytes_stream(..);
|
||||
|
||||
let mut upstream = compression_type.convert_stream(reader).fuse();
|
||||
|
||||
@@ -203,6 +205,7 @@ pub async fn stream_to_file<T: DfRecordBatchEncoder, U: Fn(SharedBuffer) -> T>(
|
||||
.writer_with(&path)
|
||||
.concurrent(concurrency)
|
||||
.await
|
||||
.map(|v| v.into_futures_async_write().compat_write())
|
||||
.context(error::WriteObjectSnafu { path })
|
||||
});
|
||||
|
||||
|
||||
@@ -29,6 +29,7 @@ use datafusion::physical_plan::SendableRecordBatchStream;
|
||||
use derive_builder::Builder;
|
||||
use object_store::ObjectStore;
|
||||
use snafu::ResultExt;
|
||||
use tokio_util::compat::FuturesAsyncReadCompatExt;
|
||||
use tokio_util::io::SyncIoBridge;
|
||||
|
||||
use super::stream_to_file;
|
||||
@@ -164,10 +165,16 @@ impl FileOpener for CsvOpener {
|
||||
#[async_trait]
|
||||
impl FileFormat for CsvFormat {
|
||||
async fn infer_schema(&self, store: &ObjectStore, path: &str) -> Result<Schema> {
|
||||
let meta = store
|
||||
.stat(path)
|
||||
.await
|
||||
.context(error::ReadObjectSnafu { path })?;
|
||||
let reader = store
|
||||
.reader(path)
|
||||
.await
|
||||
.context(error::ReadObjectSnafu { path })?;
|
||||
.context(error::ReadObjectSnafu { path })?
|
||||
.into_futures_async_read(0..meta.content_length())
|
||||
.compat();
|
||||
|
||||
let decoded = self.compression_type.convert_async_read(reader);
|
||||
|
||||
|
||||
@@ -31,6 +31,7 @@ use datafusion::error::{DataFusionError, Result as DataFusionResult};
|
||||
use datafusion::physical_plan::SendableRecordBatchStream;
|
||||
use object_store::ObjectStore;
|
||||
use snafu::ResultExt;
|
||||
use tokio_util::compat::FuturesAsyncReadCompatExt;
|
||||
use tokio_util::io::SyncIoBridge;
|
||||
|
||||
use super::stream_to_file;
|
||||
@@ -82,10 +83,16 @@ impl Default for JsonFormat {
|
||||
#[async_trait]
|
||||
impl FileFormat for JsonFormat {
|
||||
async fn infer_schema(&self, store: &ObjectStore, path: &str) -> Result<Schema> {
|
||||
let meta = store
|
||||
.stat(path)
|
||||
.await
|
||||
.context(error::ReadObjectSnafu { path })?;
|
||||
let reader = store
|
||||
.reader(path)
|
||||
.await
|
||||
.context(error::ReadObjectSnafu { path })?;
|
||||
.context(error::ReadObjectSnafu { path })?
|
||||
.into_futures_async_read(0..meta.content_length())
|
||||
.compat();
|
||||
|
||||
let decoded = self.compression_type.convert_async_read(reader);
|
||||
|
||||
|
||||
@@ -16,15 +16,17 @@ use std::sync::Arc;
|
||||
|
||||
use arrow_schema::{ArrowError, Schema, SchemaRef};
|
||||
use async_trait::async_trait;
|
||||
use bytes::Bytes;
|
||||
use common_recordbatch::adapter::RecordBatchStreamTypeAdapter;
|
||||
use datafusion::datasource::physical_plan::{FileMeta, FileOpenFuture, FileOpener};
|
||||
use datafusion::error::{DataFusionError, Result as DfResult};
|
||||
use futures::{StreamExt, TryStreamExt};
|
||||
use futures::future::BoxFuture;
|
||||
use futures::{FutureExt, StreamExt, TryStreamExt};
|
||||
use object_store::ObjectStore;
|
||||
use orc_rust::arrow_reader::ArrowReaderBuilder;
|
||||
use orc_rust::async_arrow_reader::ArrowStreamReader;
|
||||
use orc_rust::reader::AsyncChunkReader;
|
||||
use snafu::ResultExt;
|
||||
use tokio::io::{AsyncRead, AsyncSeek};
|
||||
|
||||
use crate::error::{self, Result};
|
||||
use crate::file_format::FileFormat;
|
||||
@@ -32,18 +34,49 @@ use crate::file_format::FileFormat;
|
||||
#[derive(Debug, Clone, Copy, PartialEq, Eq, Default)]
|
||||
pub struct OrcFormat;
|
||||
|
||||
pub async fn new_orc_stream_reader<R: AsyncRead + AsyncSeek + Unpin + Send + 'static>(
|
||||
reader: R,
|
||||
) -> Result<ArrowStreamReader<R>> {
|
||||
#[derive(Clone)]
|
||||
pub struct ReaderAdapter {
|
||||
reader: object_store::Reader,
|
||||
len: u64,
|
||||
}
|
||||
|
||||
impl ReaderAdapter {
|
||||
pub fn new(reader: object_store::Reader, len: u64) -> Self {
|
||||
Self { reader, len }
|
||||
}
|
||||
}
|
||||
|
||||
impl AsyncChunkReader for ReaderAdapter {
|
||||
fn len(&mut self) -> BoxFuture<'_, std::io::Result<u64>> {
|
||||
async move { Ok(self.len) }.boxed()
|
||||
}
|
||||
|
||||
fn get_bytes(
|
||||
&mut self,
|
||||
offset_from_start: u64,
|
||||
length: u64,
|
||||
) -> BoxFuture<'_, std::io::Result<Bytes>> {
|
||||
async move {
|
||||
let bytes = self
|
||||
.reader
|
||||
.read(offset_from_start..offset_from_start + length)
|
||||
.await?;
|
||||
Ok(bytes.to_bytes())
|
||||
}
|
||||
.boxed()
|
||||
}
|
||||
}
|
||||
|
||||
pub async fn new_orc_stream_reader(
|
||||
reader: ReaderAdapter,
|
||||
) -> Result<ArrowStreamReader<ReaderAdapter>> {
|
||||
let reader_build = ArrowReaderBuilder::try_new_async(reader)
|
||||
.await
|
||||
.context(error::OrcReaderSnafu)?;
|
||||
Ok(reader_build.build_async())
|
||||
}
|
||||
|
||||
pub async fn infer_orc_schema<R: AsyncRead + AsyncSeek + Unpin + Send + 'static>(
|
||||
reader: R,
|
||||
) -> Result<Schema> {
|
||||
pub async fn infer_orc_schema(reader: ReaderAdapter) -> Result<Schema> {
|
||||
let reader = new_orc_stream_reader(reader).await?;
|
||||
Ok(reader.schema().as_ref().clone())
|
||||
}
|
||||
@@ -51,13 +84,15 @@ pub async fn infer_orc_schema<R: AsyncRead + AsyncSeek + Unpin + Send + 'static>
|
||||
#[async_trait]
|
||||
impl FileFormat for OrcFormat {
|
||||
async fn infer_schema(&self, store: &ObjectStore, path: &str) -> Result<Schema> {
|
||||
let meta = store
|
||||
.stat(path)
|
||||
.await
|
||||
.context(error::ReadObjectSnafu { path })?;
|
||||
let reader = store
|
||||
.reader(path)
|
||||
.await
|
||||
.context(error::ReadObjectSnafu { path })?;
|
||||
|
||||
let schema = infer_orc_schema(reader).await?;
|
||||
|
||||
let schema = infer_orc_schema(ReaderAdapter::new(reader, meta.content_length())).await?;
|
||||
Ok(schema)
|
||||
}
|
||||
}
|
||||
@@ -97,15 +132,23 @@ impl FileOpener for OrcOpener {
|
||||
};
|
||||
let projection = self.projection.clone();
|
||||
Ok(Box::pin(async move {
|
||||
let reader = object_store
|
||||
.reader(meta.location().to_string().as_str())
|
||||
let path = meta.location().to_string();
|
||||
|
||||
let meta = object_store
|
||||
.stat(&path)
|
||||
.await
|
||||
.map_err(|e| DataFusionError::External(Box::new(e)))?;
|
||||
|
||||
let stream_reader = new_orc_stream_reader(reader)
|
||||
let reader = object_store
|
||||
.reader(&path)
|
||||
.await
|
||||
.map_err(|e| DataFusionError::External(Box::new(e)))?;
|
||||
|
||||
let stream_reader =
|
||||
new_orc_stream_reader(ReaderAdapter::new(reader, meta.content_length()))
|
||||
.await
|
||||
.map_err(|e| DataFusionError::External(Box::new(e)))?;
|
||||
|
||||
let stream =
|
||||
RecordBatchStreamTypeAdapter::new(projected_schema, stream_reader, projection);
|
||||
|
||||
|
||||
@@ -29,10 +29,11 @@ use datafusion::physical_plan::metrics::ExecutionPlanMetricsSet;
|
||||
use datafusion::physical_plan::SendableRecordBatchStream;
|
||||
use futures::future::BoxFuture;
|
||||
use futures::StreamExt;
|
||||
use object_store::{ObjectStore, Reader, Writer};
|
||||
use object_store::{FuturesAsyncReader, ObjectStore};
|
||||
use parquet::basic::{Compression, ZstdLevel};
|
||||
use parquet::file::properties::WriterProperties;
|
||||
use snafu::ResultExt;
|
||||
use tokio_util::compat::{Compat, FuturesAsyncReadCompatExt, FuturesAsyncWriteCompatExt};
|
||||
|
||||
use crate::buffered_writer::{ArrowWriterCloser, DfRecordBatchEncoder, LazyBufferedWriter};
|
||||
use crate::error::{self, Result};
|
||||
@@ -45,10 +46,16 @@ pub struct ParquetFormat {}
|
||||
#[async_trait]
|
||||
impl FileFormat for ParquetFormat {
|
||||
async fn infer_schema(&self, store: &ObjectStore, path: &str) -> Result<Schema> {
|
||||
let meta = store
|
||||
.stat(path)
|
||||
.await
|
||||
.context(error::ReadObjectSnafu { path })?;
|
||||
let mut reader = store
|
||||
.reader(path)
|
||||
.await
|
||||
.context(error::ReadObjectSnafu { path })?;
|
||||
.context(error::ReadObjectSnafu { path })?
|
||||
.into_futures_async_read(0..meta.content_length())
|
||||
.compat();
|
||||
|
||||
let metadata = reader
|
||||
.get_metadata()
|
||||
@@ -98,7 +105,7 @@ impl ParquetFileReaderFactory for DefaultParquetFileReaderFactory {
|
||||
|
||||
pub struct LazyParquetFileReader {
|
||||
object_store: ObjectStore,
|
||||
reader: Option<Reader>,
|
||||
reader: Option<Compat<FuturesAsyncReader>>,
|
||||
path: String,
|
||||
}
|
||||
|
||||
@@ -114,7 +121,13 @@ impl LazyParquetFileReader {
|
||||
/// Must initialize the reader, or throw an error from the future.
|
||||
async fn maybe_initialize(&mut self) -> result::Result<(), object_store::Error> {
|
||||
if self.reader.is_none() {
|
||||
let reader = self.object_store.reader(&self.path).await?;
|
||||
let meta = self.object_store.stat(&self.path).await?;
|
||||
let reader = self
|
||||
.object_store
|
||||
.reader(&self.path)
|
||||
.await?
|
||||
.into_futures_async_read(0..meta.content_length())
|
||||
.compat();
|
||||
self.reader = Some(reader);
|
||||
}
|
||||
|
||||
@@ -167,16 +180,17 @@ pub struct BufferedWriter {
|
||||
}
|
||||
|
||||
type InnerBufferedWriter = LazyBufferedWriter<
|
||||
object_store::Writer,
|
||||
Compat<object_store::FuturesAsyncWriter>,
|
||||
ArrowWriter<SharedBuffer>,
|
||||
impl Fn(String) -> BoxFuture<'static, Result<Writer>>,
|
||||
impl Fn(String) -> BoxFuture<'static, Result<Compat<object_store::FuturesAsyncWriter>>>,
|
||||
>;
|
||||
|
||||
impl BufferedWriter {
|
||||
fn make_write_factory(
|
||||
store: ObjectStore,
|
||||
concurrency: usize,
|
||||
) -> impl Fn(String) -> BoxFuture<'static, Result<Writer>> {
|
||||
) -> impl Fn(String) -> BoxFuture<'static, Result<Compat<object_store::FuturesAsyncWriter>>>
|
||||
{
|
||||
move |path| {
|
||||
let store = store.clone();
|
||||
Box::pin(async move {
|
||||
@@ -184,6 +198,7 @@ impl BufferedWriter {
|
||||
.writer_with(&path)
|
||||
.concurrent(concurrency)
|
||||
.await
|
||||
.map(|v| v.into_futures_async_write().compat_write())
|
||||
.context(error::WriteObjectSnafu { path })
|
||||
})
|
||||
}
|
||||
|
||||
@@ -120,7 +120,7 @@ pub async fn setup_stream_to_json_test(origin_path: &str, threshold: impl Fn(usi
|
||||
|
||||
let written = tmp_store.read(&output_path).await.unwrap();
|
||||
let origin = store.read(origin_path).await.unwrap();
|
||||
assert_eq_lines(written, origin);
|
||||
assert_eq_lines(written.to_vec(), origin.to_vec());
|
||||
}
|
||||
|
||||
pub async fn setup_stream_to_csv_test(origin_path: &str, threshold: impl Fn(usize) -> usize) {
|
||||
@@ -158,7 +158,7 @@ pub async fn setup_stream_to_csv_test(origin_path: &str, threshold: impl Fn(usiz
|
||||
|
||||
let written = tmp_store.read(&output_path).await.unwrap();
|
||||
let origin = store.read(origin_path).await.unwrap();
|
||||
assert_eq_lines(written, origin);
|
||||
assert_eq_lines(written.to_vec(), origin.to_vec());
|
||||
}
|
||||
|
||||
// Ignore the CRLF difference across operating systems.
|
||||
|
||||
@@ -25,7 +25,7 @@ prost.workspace = true
|
||||
snafu.workspace = true
|
||||
tokio.workspace = true
|
||||
tonic.workspace = true
|
||||
tower = "0.4"
|
||||
tower.workspace = true
|
||||
|
||||
[dev-dependencies]
|
||||
criterion = "0.4"
|
||||
|
||||
@@ -180,7 +180,7 @@ mod tests {
|
||||
catalog_name: DEFAULT_CATALOG_NAME.to_string(),
|
||||
flow_name: "my_flow".to_string(),
|
||||
raw_sql: "sql".to_string(),
|
||||
expire_when: "expire".to_string(),
|
||||
expire_after: Some(300),
|
||||
comment: "comment".to_string(),
|
||||
options: Default::default(),
|
||||
},
|
||||
|
||||
@@ -18,6 +18,7 @@ use std::collections::BTreeMap;
|
||||
|
||||
use api::v1::flow::flow_request::Body as PbFlowRequest;
|
||||
use api::v1::flow::{CreateRequest, FlowRequest, FlowRequestHeader};
|
||||
use api::v1::ExpireAfter;
|
||||
use async_trait::async_trait;
|
||||
use common_catalog::format_full_flow_name;
|
||||
use common_procedure::error::{FromJsonSnafu, ToJsonSnafu};
|
||||
@@ -283,7 +284,7 @@ impl From<&CreateFlowData> for CreateRequest {
|
||||
sink_table_name: Some(value.task.sink_table_name.clone().into()),
|
||||
// Always be true
|
||||
create_if_not_exists: true,
|
||||
expire_when: value.task.expire_when.clone(),
|
||||
expire_after: value.task.expire_after.map(|value| ExpireAfter { value }),
|
||||
comment: value.task.comment.clone(),
|
||||
sql: value.task.sql.clone(),
|
||||
flow_options: value.task.flow_options.clone(),
|
||||
@@ -297,7 +298,7 @@ impl From<&CreateFlowData> for FlowInfoValue {
|
||||
catalog_name,
|
||||
flow_name,
|
||||
sink_table_name,
|
||||
expire_when,
|
||||
expire_after,
|
||||
comment,
|
||||
sql,
|
||||
flow_options: options,
|
||||
@@ -318,7 +319,7 @@ impl From<&CreateFlowData> for FlowInfoValue {
|
||||
catalog_name,
|
||||
flow_name,
|
||||
raw_sql: sql,
|
||||
expire_when,
|
||||
expire_after,
|
||||
comment,
|
||||
options,
|
||||
}
|
||||
|
||||
@@ -44,7 +44,7 @@ pub(crate) fn test_create_flow_task(
|
||||
sink_table_name,
|
||||
or_replace: false,
|
||||
create_if_not_exists,
|
||||
expire_when: "".to_string(),
|
||||
expire_after: Some(300),
|
||||
comment: "".to_string(),
|
||||
sql: "raw_sql".to_string(),
|
||||
flow_options: Default::default(),
|
||||
|
||||
@@ -328,7 +328,7 @@ mod tests {
|
||||
sink_table_name,
|
||||
flownode_ids,
|
||||
raw_sql: "raw".to_string(),
|
||||
expire_when: "expr".to_string(),
|
||||
expire_after: Some(300),
|
||||
comment: "hi".to_string(),
|
||||
options: Default::default(),
|
||||
}
|
||||
@@ -420,7 +420,7 @@ mod tests {
|
||||
sink_table_name: another_sink_table_name,
|
||||
flownode_ids: [(0, 1u64)].into(),
|
||||
raw_sql: "raw".to_string(),
|
||||
expire_when: "expr".to_string(),
|
||||
expire_after: Some(300),
|
||||
comment: "hi".to_string(),
|
||||
options: Default::default(),
|
||||
};
|
||||
|
||||
@@ -123,7 +123,8 @@ pub struct FlowInfoValue {
|
||||
/// The raw sql.
|
||||
pub(crate) raw_sql: String,
|
||||
/// The expr of expire.
|
||||
pub(crate) expire_when: String,
|
||||
/// Duration in seconds as `i64`.
|
||||
pub(crate) expire_after: Option<i64>,
|
||||
/// The comment.
|
||||
pub(crate) comment: String,
|
||||
/// The options.
|
||||
|
||||
@@ -28,8 +28,8 @@ use api::v1::meta::{
|
||||
};
|
||||
use api::v1::{
|
||||
AlterExpr, CreateDatabaseExpr, CreateFlowExpr, CreateTableExpr, CreateViewExpr,
|
||||
DropDatabaseExpr, DropFlowExpr, DropTableExpr, DropViewExpr, QueryContext as PbQueryContext,
|
||||
TruncateTableExpr,
|
||||
DropDatabaseExpr, DropFlowExpr, DropTableExpr, DropViewExpr, ExpireAfter,
|
||||
QueryContext as PbQueryContext, TruncateTableExpr,
|
||||
};
|
||||
use base64::engine::general_purpose;
|
||||
use base64::Engine as _;
|
||||
@@ -898,7 +898,8 @@ pub struct CreateFlowTask {
|
||||
pub sink_table_name: TableName,
|
||||
pub or_replace: bool,
|
||||
pub create_if_not_exists: bool,
|
||||
pub expire_when: String,
|
||||
/// Duration in seconds. Data older than this duration will not be used.
|
||||
pub expire_after: Option<i64>,
|
||||
pub comment: String,
|
||||
pub sql: String,
|
||||
pub flow_options: HashMap<String, String>,
|
||||
@@ -915,7 +916,7 @@ impl TryFrom<PbCreateFlowTask> for CreateFlowTask {
|
||||
sink_table_name,
|
||||
or_replace,
|
||||
create_if_not_exists,
|
||||
expire_when,
|
||||
expire_after,
|
||||
comment,
|
||||
sql,
|
||||
flow_options,
|
||||
@@ -934,7 +935,7 @@ impl TryFrom<PbCreateFlowTask> for CreateFlowTask {
|
||||
.into(),
|
||||
or_replace,
|
||||
create_if_not_exists,
|
||||
expire_when,
|
||||
expire_after: expire_after.map(|e| e.value),
|
||||
comment,
|
||||
sql,
|
||||
flow_options,
|
||||
@@ -951,7 +952,7 @@ impl From<CreateFlowTask> for PbCreateFlowTask {
|
||||
sink_table_name,
|
||||
or_replace,
|
||||
create_if_not_exists,
|
||||
expire_when,
|
||||
expire_after,
|
||||
comment,
|
||||
sql,
|
||||
flow_options,
|
||||
@@ -965,7 +966,7 @@ impl From<CreateFlowTask> for PbCreateFlowTask {
|
||||
sink_table_name: Some(sink_table_name.into()),
|
||||
or_replace,
|
||||
create_if_not_exists,
|
||||
expire_when,
|
||||
expire_after: expire_after.map(|value| ExpireAfter { value }),
|
||||
comment,
|
||||
sql,
|
||||
flow_options,
|
||||
|
||||
@@ -179,7 +179,7 @@ impl StateStore for ObjectStateStore {
|
||||
))
|
||||
})
|
||||
.context(ListStateSnafu { path: key })?;
|
||||
yield (key.into(), value);
|
||||
yield (key.into(), value.to_vec());
|
||||
}
|
||||
}
|
||||
});
|
||||
|
||||
@@ -94,7 +94,7 @@ pub fn init_default_ut_logging() {
|
||||
env::var("UNITTEST_LOG_DIR").unwrap_or_else(|_| "/tmp/__unittest_logs".to_string());
|
||||
|
||||
let level = env::var("UNITTEST_LOG_LEVEL").unwrap_or_else(|_|
|
||||
"debug,hyper=warn,tower=warn,datafusion=warn,reqwest=warn,sqlparser=warn,h2=info,opendal=info".to_string()
|
||||
"debug,hyper=warn,tower=warn,datafusion=warn,reqwest=warn,sqlparser=warn,h2=info,opendal=info,rskafka=info".to_string()
|
||||
);
|
||||
let opts = LoggingOptions {
|
||||
dir: dir.clone(),
|
||||
|
||||
@@ -367,6 +367,19 @@ pub enum Error {
|
||||
#[snafu(source(from(common_config::error::Error, Box::new)))]
|
||||
source: Box<common_config::error::Error>,
|
||||
},
|
||||
|
||||
#[snafu(display(
|
||||
"Failed to get region metadata from engine {} for region_id {}",
|
||||
engine,
|
||||
region_id,
|
||||
))]
|
||||
GetRegionMetadata {
|
||||
engine: String,
|
||||
region_id: RegionId,
|
||||
#[snafu(implicit)]
|
||||
location: Location,
|
||||
source: BoxedError,
|
||||
},
|
||||
}
|
||||
|
||||
pub type Result<T> = std::result::Result<T, Error>;
|
||||
@@ -433,7 +446,9 @@ impl ErrorExt for Error {
|
||||
TableIdProviderNotFound { .. } | UnsupportedGrpcRequest { .. } => {
|
||||
StatusCode::Unsupported
|
||||
}
|
||||
HandleRegionRequest { source, .. } => source.status_code(),
|
||||
HandleRegionRequest { source, .. } | GetRegionMetadata { source, .. } => {
|
||||
source.status_code()
|
||||
}
|
||||
StopRegionEngine { source, .. } => source.status_code(),
|
||||
|
||||
FindLogicalRegions { source, .. } => source.status_code(),
|
||||
|
||||
@@ -213,6 +213,7 @@ impl HeartbeatTask {
|
||||
let epoch = self.region_alive_keeper.epoch();
|
||||
|
||||
self.region_alive_keeper.start(Some(event_receiver)).await?;
|
||||
let mut last_sent = Instant::now();
|
||||
|
||||
common_runtime::spawn_bg(async move {
|
||||
let sleep = tokio::time::sleep(Duration::from_millis(0));
|
||||
@@ -271,6 +272,10 @@ impl HeartbeatTask {
|
||||
}
|
||||
};
|
||||
if let Some(req) = req {
|
||||
metrics::LAST_SENT_HEARTBEAT_ELAPSED
|
||||
.set(last_sent.elapsed().as_millis() as i64);
|
||||
// Resets the timer.
|
||||
last_sent = Instant::now();
|
||||
debug!("Sending heartbeat request: {:?}", req);
|
||||
if let Err(e) = tx.send(req).await {
|
||||
error!(e; "Failed to send heartbeat to metasrv");
|
||||
|
||||
@@ -35,6 +35,12 @@ lazy_static! {
|
||||
"last received heartbeat lease elapsed",
|
||||
)
|
||||
.unwrap();
|
||||
/// The elapsed time since the last sent heartbeat.
|
||||
pub static ref LAST_SENT_HEARTBEAT_ELAPSED: IntGauge = register_int_gauge!(
|
||||
"greptime_last_sent_heartbeat_lease_elapsed",
|
||||
"last sent heartbeat lease elapsed",
|
||||
)
|
||||
.unwrap();
|
||||
pub static ref LEASE_EXPIRED_REGION: IntGaugeVec = register_int_gauge_vec!(
|
||||
"greptime_lease_expired_region",
|
||||
"lease expired region",
|
||||
|
||||
@@ -189,7 +189,7 @@ impl RegionServer {
|
||||
|
||||
pub async fn region_disk_usage(&self, region_id: RegionId) -> Option<i64> {
|
||||
match self.inner.region_map.get(®ion_id) {
|
||||
Some(e) => e.region_disk_usage(region_id).await,
|
||||
Some(e) => e.region_disk_usage(region_id),
|
||||
None => None,
|
||||
}
|
||||
}
|
||||
@@ -409,9 +409,7 @@ impl RegionServerInner {
|
||||
let engine = match region_change {
|
||||
RegionChange::Register(attribute) => match current_region_status {
|
||||
Some(status) => match status.clone() {
|
||||
RegionEngineWithStatus::Registering(_) => {
|
||||
return Ok(CurrentEngine::EarlyReturn(0))
|
||||
}
|
||||
RegionEngineWithStatus::Registering(engine) => engine,
|
||||
RegionEngineWithStatus::Deregistering(_) => {
|
||||
return error::RegionBusySnafu { region_id }.fail()
|
||||
}
|
||||
@@ -781,34 +779,32 @@ mod tests {
|
||||
let mut mock_region_server = mock_region_server();
|
||||
let (engine, _receiver) = MockRegionEngine::new(MITO_ENGINE_NAME);
|
||||
let engine_name = engine.name();
|
||||
|
||||
mock_region_server.register_engine(engine.clone());
|
||||
|
||||
let region_id = RegionId::new(1, 1);
|
||||
let builder = CreateRequestBuilder::new();
|
||||
let create_req = builder.build();
|
||||
|
||||
// Tries to create/open a registering region.
|
||||
mock_region_server.inner.region_map.insert(
|
||||
region_id,
|
||||
RegionEngineWithStatus::Registering(engine.clone()),
|
||||
);
|
||||
|
||||
let response = mock_region_server
|
||||
.handle_request(region_id, RegionRequest::Create(create_req))
|
||||
.await
|
||||
.unwrap();
|
||||
assert_eq!(response.affected_rows, 0);
|
||||
|
||||
let status = mock_region_server
|
||||
.inner
|
||||
.region_map
|
||||
.get(®ion_id)
|
||||
.unwrap()
|
||||
.clone();
|
||||
assert!(matches!(status, RegionEngineWithStatus::Ready(_)));
|
||||
|
||||
assert!(matches!(status, RegionEngineWithStatus::Registering(_)));
|
||||
|
||||
mock_region_server.inner.region_map.insert(
|
||||
region_id,
|
||||
RegionEngineWithStatus::Registering(engine.clone()),
|
||||
);
|
||||
let response = mock_region_server
|
||||
.handle_request(
|
||||
region_id,
|
||||
@@ -822,14 +818,13 @@ mod tests {
|
||||
.await
|
||||
.unwrap();
|
||||
assert_eq!(response.affected_rows, 0);
|
||||
|
||||
let status = mock_region_server
|
||||
.inner
|
||||
.region_map
|
||||
.get(®ion_id)
|
||||
.unwrap()
|
||||
.clone();
|
||||
assert!(matches!(status, RegionEngineWithStatus::Registering(_)));
|
||||
assert!(matches!(status, RegionEngineWithStatus::Ready(_)));
|
||||
}
|
||||
|
||||
#[tokio::test]
|
||||
@@ -1020,7 +1015,7 @@ mod tests {
|
||||
region_change: RegionChange::Register(RegionAttribute::Mito),
|
||||
assert: Box::new(|result| {
|
||||
let current_engine = result.unwrap();
|
||||
assert_matches!(current_engine, CurrentEngine::EarlyReturn(_));
|
||||
assert_matches!(current_engine, CurrentEngine::Engine(_));
|
||||
}),
|
||||
},
|
||||
CurrentEngineTest {
|
||||
|
||||
@@ -20,7 +20,6 @@ mod gcs;
|
||||
mod oss;
|
||||
mod s3;
|
||||
|
||||
use std::sync::Arc;
|
||||
use std::time::Duration;
|
||||
use std::{env, path};
|
||||
|
||||
@@ -29,7 +28,7 @@ use common_telemetry::info;
|
||||
use object_store::layers::{LruCacheLayer, RetryLayer};
|
||||
use object_store::services::Fs;
|
||||
use object_store::util::{join_dir, normalize_dir, with_instrument_layers};
|
||||
use object_store::{HttpClient, ObjectStore, ObjectStoreBuilder};
|
||||
use object_store::{HttpClient, ObjectStore};
|
||||
use snafu::prelude::*;
|
||||
|
||||
use crate::config::{ObjectStoreConfig, DEFAULT_OBJECT_STORE_CACHE_SIZE};
|
||||
@@ -107,13 +106,13 @@ async fn create_object_store_with_cache(
|
||||
if let Some(path) = cache_path {
|
||||
let atomic_temp_dir = join_dir(path, ".tmp/");
|
||||
clean_temp_dir(&atomic_temp_dir)?;
|
||||
let cache_store = Fs::default()
|
||||
.root(path)
|
||||
.atomic_write_dir(&atomic_temp_dir)
|
||||
.build()
|
||||
.context(error::InitBackendSnafu)?;
|
||||
let mut builder = Fs::default();
|
||||
builder.root(path).atomic_write_dir(&atomic_temp_dir);
|
||||
let cache_store = ObjectStore::new(builder)
|
||||
.context(error::InitBackendSnafu)?
|
||||
.finish();
|
||||
|
||||
let cache_layer = LruCacheLayer::new(Arc::new(cache_store), cache_capacity.0 as usize)
|
||||
let cache_layer = LruCacheLayer::new(cache_store, cache_capacity.0 as usize)
|
||||
.await
|
||||
.context(error::InitBackendSnafu)?;
|
||||
|
||||
|
||||
@@ -200,7 +200,7 @@ impl RegionEngine for MockRegionEngine {
|
||||
unimplemented!()
|
||||
}
|
||||
|
||||
async fn region_disk_usage(&self, _region_id: RegionId) -> Option<i64> {
|
||||
fn region_disk_usage(&self, _region_id: RegionId) -> Option<i64> {
|
||||
unimplemented!()
|
||||
}
|
||||
|
||||
|
||||
@@ -107,7 +107,7 @@ impl RegionEngine for FileRegionEngine {
|
||||
self.inner.stop().await.map_err(BoxedError::new)
|
||||
}
|
||||
|
||||
async fn region_disk_usage(&self, _: RegionId) -> Option<i64> {
|
||||
fn region_disk_usage(&self, _: RegionId) -> Option<i64> {
|
||||
None
|
||||
}
|
||||
|
||||
|
||||
@@ -71,7 +71,8 @@ impl FileRegionManifest {
|
||||
let bs = object_store
|
||||
.read(path)
|
||||
.await
|
||||
.context(LoadRegionManifestSnafu { region_id })?;
|
||||
.context(LoadRegionManifestSnafu { region_id })?
|
||||
.to_vec();
|
||||
Self::decode(bs.as_slice())
|
||||
}
|
||||
|
||||
|
||||
@@ -35,14 +35,13 @@ use itertools::Itertools;
|
||||
use query::{QueryEngine, QueryEngineFactory};
|
||||
use serde::{Deserialize, Serialize};
|
||||
use session::context::QueryContext;
|
||||
use snafu::{OptionExt, ResultExt};
|
||||
use snafu::{ensure, OptionExt, ResultExt};
|
||||
use store_api::storage::{ConcreteDataType, RegionId};
|
||||
use table::metadata::TableId;
|
||||
use tokio::sync::{oneshot, watch, Mutex, RwLock};
|
||||
|
||||
use crate::adapter::error::{ExternalSnafu, TableNotFoundSnafu, UnexpectedSnafu};
|
||||
use crate::adapter::error::{ExternalSnafu, InternalSnafu, TableNotFoundSnafu, UnexpectedSnafu};
|
||||
pub(crate) use crate::adapter::node_context::FlownodeContext;
|
||||
use crate::adapter::parse_expr::parse_fixed;
|
||||
use crate::adapter::table_source::TableSource;
|
||||
use crate::adapter::util::column_schemas_to_proto;
|
||||
use crate::adapter::worker::{create_worker, Worker, WorkerHandle};
|
||||
@@ -67,6 +66,11 @@ use error::Error;
|
||||
|
||||
pub const PER_REQ_MAX_ROW_CNT: usize = 8192;
|
||||
|
||||
// TODO: replace this with `GREPTIME_TIMESTAMP` before v0.9
|
||||
pub const AUTO_CREATED_PLACEHOLDER_TS_COL: &str = "__ts_placeholder";
|
||||
|
||||
pub const UPDATE_AT_TS_COL: &str = "update_at";
|
||||
|
||||
// TODO: refactor common types for flow to a separate module
|
||||
/// FlowId is a unique identifier for a flow task
|
||||
pub type FlowId = u64;
|
||||
@@ -280,10 +284,16 @@ impl FlownodeManager {
|
||||
.map(|i| meta.schema.column_schemas[i].name.clone())
|
||||
.collect_vec();
|
||||
let schema = meta.schema.column_schemas;
|
||||
let is_auto_create = schema
|
||||
.last()
|
||||
.map(|s| s.name == "__ts_placeholder")
|
||||
.unwrap_or(false);
|
||||
// check if the last column is the auto created timestamp column, hence the table is auto created from
|
||||
// flow's plan type
|
||||
let is_auto_create = {
|
||||
let correct_name = schema
|
||||
.last()
|
||||
.map(|s| s.name == AUTO_CREATED_PLACEHOLDER_TS_COL)
|
||||
.unwrap_or(false);
|
||||
let correct_time_index = meta.schema.timestamp_index == Some(schema.len() - 1);
|
||||
correct_name && correct_time_index
|
||||
};
|
||||
(primary_keys, schema, is_auto_create)
|
||||
} else {
|
||||
// TODO(discord9): condiser remove buggy auto create by schema
|
||||
@@ -303,6 +313,7 @@ impl FlownodeManager {
|
||||
.clone();
|
||||
// TODO(discord9): use default key from schema
|
||||
let primary_keys = schema
|
||||
.typ()
|
||||
.keys
|
||||
.first()
|
||||
.map(|v| {
|
||||
@@ -313,24 +324,31 @@ impl FlownodeManager {
|
||||
})
|
||||
.unwrap_or_default();
|
||||
let update_at = ColumnSchema::new(
|
||||
"update_at",
|
||||
UPDATE_AT_TS_COL,
|
||||
ConcreteDataType::timestamp_millisecond_datatype(),
|
||||
true,
|
||||
);
|
||||
// TODO(discord9): bugged so we can't infer time index from flow plan, so we have to manually set one
|
||||
let ts_col = ColumnSchema::new(
|
||||
"__ts_placeholder",
|
||||
AUTO_CREATED_PLACEHOLDER_TS_COL,
|
||||
ConcreteDataType::timestamp_millisecond_datatype(),
|
||||
true,
|
||||
)
|
||||
.with_time_index(true);
|
||||
|
||||
let wout_ts = schema
|
||||
.typ()
|
||||
.column_types
|
||||
.clone()
|
||||
.into_iter()
|
||||
.enumerate()
|
||||
.map(|(idx, typ)| {
|
||||
ColumnSchema::new(format!("Col_{idx}"), typ.scalar_type, typ.nullable)
|
||||
let name = schema
|
||||
.names
|
||||
.get(idx)
|
||||
.cloned()
|
||||
.unwrap_or(format!("Col_{}", idx));
|
||||
ColumnSchema::new(name, typ.scalar_type, typ.nullable)
|
||||
})
|
||||
.collect_vec();
|
||||
|
||||
@@ -340,7 +358,7 @@ impl FlownodeManager {
|
||||
|
||||
(primary_keys, with_ts, true)
|
||||
};
|
||||
|
||||
let schema_len = schema.len();
|
||||
let proto_schema = column_schemas_to_proto(schema, &primary_keys)?;
|
||||
|
||||
debug!(
|
||||
@@ -349,16 +367,7 @@ impl FlownodeManager {
|
||||
table_name.join("."),
|
||||
reqs
|
||||
);
|
||||
let now = SystemTime::now();
|
||||
let now = now
|
||||
.duration_since(SystemTime::UNIX_EPOCH)
|
||||
.map(|s| s.as_millis() as repr::Timestamp)
|
||||
.unwrap_or_else(|_| {
|
||||
-(SystemTime::UNIX_EPOCH
|
||||
.duration_since(now)
|
||||
.unwrap()
|
||||
.as_millis() as repr::Timestamp)
|
||||
});
|
||||
let now = self.tick_manager.tick();
|
||||
for req in reqs {
|
||||
match req {
|
||||
DiffRequest::Insert(insert) => {
|
||||
@@ -371,13 +380,23 @@ impl FlownodeManager {
|
||||
))]);
|
||||
// ts col, if auto create
|
||||
if is_auto_create {
|
||||
ensure!(
|
||||
row.len() == schema_len - 1,
|
||||
InternalSnafu {
|
||||
reason: format!(
|
||||
"Row len mismatch, expect {} got {}",
|
||||
schema_len - 1,
|
||||
row.len()
|
||||
)
|
||||
}
|
||||
);
|
||||
row.extend([Value::from(
|
||||
common_time::Timestamp::new_millisecond(0),
|
||||
)]);
|
||||
}
|
||||
row.into()
|
||||
Ok(row.into())
|
||||
})
|
||||
.collect::<Vec<_>>();
|
||||
.collect::<Result<Vec<_>, Error>>()?;
|
||||
let table_name = table_name.last().unwrap().clone();
|
||||
let req = RowInsertRequest {
|
||||
table_name,
|
||||
@@ -491,9 +510,12 @@ impl FlownodeManager {
|
||||
debug!("Starting to run");
|
||||
loop {
|
||||
// TODO(discord9): only run when new inputs arrive or scheduled to
|
||||
self.run_available().await.unwrap();
|
||||
debug!("call run_available in run every second");
|
||||
self.run_available(true).await.unwrap();
|
||||
debug!("call send_writeback_requests in run every second");
|
||||
// TODO(discord9): error handling
|
||||
self.send_writeback_requests().await.unwrap();
|
||||
debug!("call log_all_errors in run every second");
|
||||
self.log_all_errors().await;
|
||||
tokio::time::sleep(std::time::Duration::from_secs(1)).await;
|
||||
}
|
||||
@@ -502,29 +524,44 @@ impl FlownodeManager {
|
||||
/// Run all available subgraph in the flow node
|
||||
/// This will try to run all dataflow in this node
|
||||
///
|
||||
/// However this is not blocking and can sometimes return while actual computation is still running in worker thread
|
||||
/// set `blocking` to true to wait until lock is acquired
|
||||
/// and false to return immediately if lock is not acquired
|
||||
/// TODO(discord9): add flag for subgraph that have input since last run
|
||||
pub async fn run_available(&self) -> Result<(), Error> {
|
||||
let now = self.tick_manager.tick();
|
||||
|
||||
pub async fn run_available(&self, blocking: bool) -> Result<(), Error> {
|
||||
loop {
|
||||
let now = self.tick_manager.tick();
|
||||
for worker in self.worker_handles.iter() {
|
||||
// TODO(discord9): consider how to handle error in individual worker
|
||||
worker.lock().await.run_available(now).await.unwrap();
|
||||
if blocking {
|
||||
worker.lock().await.run_available(now).await?;
|
||||
} else if let Ok(worker) = worker.try_lock() {
|
||||
worker.run_available(now).await?;
|
||||
} else {
|
||||
return Ok(());
|
||||
}
|
||||
}
|
||||
// first check how many inputs were sent
|
||||
let send_cnt = match self.node_context.lock().await.flush_all_sender() {
|
||||
Ok(cnt) => cnt,
|
||||
let (flush_res, buf_len) = if blocking {
|
||||
let mut ctx = self.node_context.lock().await;
|
||||
(ctx.flush_all_sender(), ctx.get_send_buf_size())
|
||||
} else {
|
||||
match self.node_context.try_lock() {
|
||||
Ok(mut ctx) => (ctx.flush_all_sender(), ctx.get_send_buf_size()),
|
||||
Err(_) => return Ok(()),
|
||||
}
|
||||
};
|
||||
match flush_res {
|
||||
Ok(_) => (),
|
||||
Err(err) => {
|
||||
common_telemetry::error!("Flush send buf errors: {:?}", err);
|
||||
break;
|
||||
}
|
||||
};
|
||||
// if no inputs
|
||||
if send_cnt == 0 {
|
||||
// if no thing in send buf then break
|
||||
if buf_len == 0 {
|
||||
break;
|
||||
} else {
|
||||
debug!("FlownodeManager::run_available: send_cnt={}", send_cnt);
|
||||
debug!("Send buf len = {}", buf_len);
|
||||
}
|
||||
}
|
||||
|
||||
@@ -544,6 +581,8 @@ impl FlownodeManager {
|
||||
);
|
||||
let table_id = region_id.table_id();
|
||||
self.node_context.lock().await.send(table_id, rows)?;
|
||||
// TODO(discord9): put it in a background task?
|
||||
// self.run_available(false).await?;
|
||||
Ok(())
|
||||
}
|
||||
}
|
||||
@@ -565,7 +604,7 @@ impl FlownodeManager {
|
||||
/// Return task id if a new task is created, otherwise return None
|
||||
///
|
||||
/// steps to create task:
|
||||
/// 1. parse query into typed plan(and optional parse expire_when expr)
|
||||
/// 1. parse query into typed plan(and optional parse expire_after expr)
|
||||
/// 2. render source/sink with output table id and used input table id
|
||||
#[allow(clippy::too_many_arguments)]
|
||||
pub async fn create_flow(
|
||||
@@ -573,14 +612,14 @@ impl FlownodeManager {
|
||||
flow_id: FlowId,
|
||||
sink_table_name: TableName,
|
||||
source_table_ids: &[TableId],
|
||||
create_if_not_exist: bool,
|
||||
expire_when: Option<String>,
|
||||
create_if_not_exists: bool,
|
||||
expire_after: Option<i64>,
|
||||
comment: Option<String>,
|
||||
sql: String,
|
||||
flow_options: HashMap<String, String>,
|
||||
query_ctx: Option<QueryContext>,
|
||||
) -> Result<Option<FlowId>, Error> {
|
||||
if create_if_not_exist {
|
||||
if create_if_not_exists {
|
||||
// check if the task already exists
|
||||
for handle in self.worker_handles.iter() {
|
||||
if handle.lock().await.contains_flow(flow_id).await? {
|
||||
@@ -608,22 +647,6 @@ impl FlownodeManager {
|
||||
debug!("Flow {:?}'s Plan is {:?}", flow_id, flow_plan);
|
||||
node_ctx.assign_table_schema(&sink_table_name, flow_plan.typ.clone())?;
|
||||
|
||||
let expire_when = expire_when
|
||||
.and_then(|s| {
|
||||
if s.is_empty() || s.split_whitespace().join("").is_empty() {
|
||||
None
|
||||
} else {
|
||||
Some(s)
|
||||
}
|
||||
})
|
||||
.map(|d| {
|
||||
let d = d.as_ref();
|
||||
parse_fixed(d)
|
||||
.map(|(_, n)| n)
|
||||
.map_err(|err| err.to_string())
|
||||
})
|
||||
.transpose()
|
||||
.map_err(|err| UnexpectedSnafu { reason: err }.build())?;
|
||||
let _ = comment;
|
||||
let _ = flow_options;
|
||||
|
||||
@@ -656,8 +679,8 @@ impl FlownodeManager {
|
||||
sink_sender,
|
||||
source_ids,
|
||||
src_recvs: source_receivers,
|
||||
expire_when,
|
||||
create_if_not_exist,
|
||||
expire_after,
|
||||
create_if_not_exists,
|
||||
err_collector,
|
||||
};
|
||||
handle.create_flow(create_request).await?;
|
||||
@@ -670,21 +693,22 @@ impl FlownodeManager {
|
||||
///
|
||||
/// TODO(discord9): better way to do it, and not expose flow tick even to other flow to avoid
|
||||
/// TSO coord mess
|
||||
#[derive(Clone)]
|
||||
#[derive(Clone, Debug)]
|
||||
pub struct FlowTickManager {
|
||||
/// The starting instant of the flow, used with `start_timestamp` to calculate the current timestamp
|
||||
start: Instant,
|
||||
}
|
||||
|
||||
impl std::fmt::Debug for FlowTickManager {
|
||||
fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
|
||||
f.debug_struct("FlowTickManager").finish()
|
||||
}
|
||||
/// The timestamp when the flow started
|
||||
start_timestamp: repr::Timestamp,
|
||||
}
|
||||
|
||||
impl FlowTickManager {
|
||||
pub fn new() -> Self {
|
||||
FlowTickManager {
|
||||
start: Instant::now(),
|
||||
start_timestamp: SystemTime::now()
|
||||
.duration_since(SystemTime::UNIX_EPOCH)
|
||||
.unwrap()
|
||||
.as_millis() as repr::Timestamp,
|
||||
}
|
||||
}
|
||||
|
||||
@@ -694,6 +718,6 @@ impl FlowTickManager {
|
||||
pub fn tick(&self) -> repr::Timestamp {
|
||||
let current = Instant::now();
|
||||
let since_the_epoch = current - self.start;
|
||||
since_the_epoch.as_millis() as repr::Timestamp
|
||||
since_the_epoch.as_millis() as repr::Timestamp + self.start_timestamp
|
||||
}
|
||||
}
|
||||
|
||||
@@ -14,13 +14,17 @@
|
||||
|
||||
//! impl `FlowNode` trait for FlowNodeManager so standalone can call them
|
||||
|
||||
use std::collections::HashMap;
|
||||
|
||||
use api::v1::flow::{flow_request, CreateRequest, DropRequest, FlowRequest, FlowResponse};
|
||||
use api::v1::region::InsertRequests;
|
||||
use common_error::ext::BoxedError;
|
||||
use common_meta::error::{ExternalSnafu, Result, UnexpectedSnafu};
|
||||
use common_meta::node_manager::Flownode;
|
||||
use common_telemetry::debug;
|
||||
use itertools::Itertools;
|
||||
use snafu::ResultExt;
|
||||
use snafu::{OptionExt, ResultExt};
|
||||
use store_api::storage::RegionId;
|
||||
|
||||
use crate::adapter::FlownodeManager;
|
||||
use crate::repr::{self, DiffRow};
|
||||
@@ -45,7 +49,7 @@ impl Flownode for FlownodeManager {
|
||||
source_table_ids,
|
||||
sink_table_name: Some(sink_table_name),
|
||||
create_if_not_exists,
|
||||
expire_when,
|
||||
expire_after,
|
||||
comment,
|
||||
sql,
|
||||
flow_options,
|
||||
@@ -56,13 +60,14 @@ impl Flownode for FlownodeManager {
|
||||
sink_table_name.schema_name,
|
||||
sink_table_name.table_name,
|
||||
];
|
||||
let expire_after = expire_after.map(|e| e.value);
|
||||
let ret = self
|
||||
.create_flow(
|
||||
task_id.id as u64,
|
||||
sink_table_name,
|
||||
&source_table_ids,
|
||||
create_if_not_exists,
|
||||
Some(expire_when),
|
||||
expire_after,
|
||||
Some(comment),
|
||||
sql,
|
||||
flow_options,
|
||||
@@ -100,12 +105,57 @@ impl Flownode for FlownodeManager {
|
||||
async fn handle_inserts(&self, request: InsertRequests) -> Result<FlowResponse> {
|
||||
for write_request in request.requests {
|
||||
let region_id = write_request.region_id;
|
||||
let rows_proto = write_request.rows.map(|r| r.rows).unwrap_or(vec![]);
|
||||
let table_id = RegionId::from(region_id).table_id();
|
||||
|
||||
let (insert_schema, rows_proto) = write_request
|
||||
.rows
|
||||
.map(|r| (r.schema, r.rows))
|
||||
.unwrap_or_default();
|
||||
|
||||
// TODO(discord9): reconsider time assignment mechanism
|
||||
let now = self.tick_manager.tick();
|
||||
|
||||
let fetch_order = {
|
||||
let ctx = self.node_context.lock().await;
|
||||
let table_col_names = ctx
|
||||
.table_repr
|
||||
.get_by_table_id(&table_id)
|
||||
.map(|r| r.1)
|
||||
.and_then(|id| ctx.schema.get(&id))
|
||||
.map(|desc| &desc.names)
|
||||
.context(UnexpectedSnafu {
|
||||
err_msg: format!("Table not found: {}", table_id),
|
||||
})?;
|
||||
let name_to_col = HashMap::<_, _>::from_iter(
|
||||
insert_schema
|
||||
.iter()
|
||||
.enumerate()
|
||||
.map(|(i, name)| (&name.column_name, i)),
|
||||
);
|
||||
let fetch_order: Vec<usize> = table_col_names
|
||||
.iter()
|
||||
.map(|names| {
|
||||
name_to_col.get(names).copied().context(UnexpectedSnafu {
|
||||
err_msg: format!("Column not found: {}", names),
|
||||
})
|
||||
})
|
||||
.try_collect()?;
|
||||
if !fetch_order.iter().enumerate().all(|(i, &v)| i == v) {
|
||||
debug!("Reordering columns: {:?}", fetch_order)
|
||||
}
|
||||
fetch_order
|
||||
};
|
||||
|
||||
let rows: Vec<DiffRow> = rows_proto
|
||||
.into_iter()
|
||||
.map(repr::Row::from)
|
||||
.map(|r| {
|
||||
let r = repr::Row::from(r);
|
||||
let reordered = fetch_order
|
||||
.iter()
|
||||
.map(|&i| r.inner[i].clone())
|
||||
.collect_vec();
|
||||
repr::Row::new(reordered)
|
||||
})
|
||||
.map(|r| (r, now, 1))
|
||||
.collect_vec();
|
||||
self.handle_write_request(region_id.into(), rows)
|
||||
|
||||
@@ -27,7 +27,7 @@ use crate::adapter::error::{Error, EvalSnafu, TableNotFoundSnafu};
|
||||
use crate::adapter::{FlowId, TableName, TableSource};
|
||||
use crate::expr::error::InternalSnafu;
|
||||
use crate::expr::GlobalId;
|
||||
use crate::repr::{DiffRow, RelationType, BROADCAST_CAP};
|
||||
use crate::repr::{DiffRow, RelationDesc, RelationType, BROADCAST_CAP};
|
||||
|
||||
/// A context that holds the information of the dataflow
|
||||
#[derive(Default, Debug)]
|
||||
@@ -51,10 +51,8 @@ pub struct FlownodeContext {
|
||||
mpsc::UnboundedReceiver<DiffRow>,
|
||||
),
|
||||
>,
|
||||
/// store source in buffer for each source table, in case broadcast channel is full
|
||||
pub send_buffer: BTreeMap<TableId, VecDeque<DiffRow>>,
|
||||
/// the schema of the table, query from metasrv or inferred from TypedPlan
|
||||
pub schema: HashMap<GlobalId, RelationType>,
|
||||
pub schema: HashMap<GlobalId, RelationDesc>,
|
||||
/// All the tables that have been registered in the worker
|
||||
pub table_repr: IdToNameMap,
|
||||
pub query_context: Option<Arc<QueryContext>>,
|
||||
@@ -73,7 +71,8 @@ pub struct SourceSender {
|
||||
impl Default for SourceSender {
|
||||
fn default() -> Self {
|
||||
Self {
|
||||
sender: broadcast::Sender::new(BROADCAST_CAP),
|
||||
// TODO(discord9): found a better way then increase this to prevent lagging and hence missing input data
|
||||
sender: broadcast::Sender::new(BROADCAST_CAP * 2),
|
||||
send_buf: Default::default(),
|
||||
}
|
||||
}
|
||||
@@ -109,6 +108,7 @@ impl SourceSender {
|
||||
}
|
||||
if row_cnt > 0 {
|
||||
debug!("Send {} rows", row_cnt);
|
||||
debug!("Remaining Send buf.len() = {}", self.send_buf.len());
|
||||
}
|
||||
|
||||
Ok(row_cnt)
|
||||
@@ -140,12 +140,19 @@ impl FlownodeContext {
|
||||
}
|
||||
|
||||
/// flush all sender's buf
|
||||
///
|
||||
/// return numbers being sent
|
||||
pub fn flush_all_sender(&mut self) -> Result<usize, Error> {
|
||||
self.source_sender
|
||||
.iter_mut()
|
||||
.map(|(_table_id, src_sender)| src_sender.try_send_all())
|
||||
.try_fold(0, |acc, x| x.map(|x| x + acc))
|
||||
}
|
||||
|
||||
/// Return the sum number of rows in all send buf
|
||||
pub fn get_send_buf_size(&self) -> usize {
|
||||
self.source_sender.values().map(|v| v.send_buf.len()).sum()
|
||||
}
|
||||
}
|
||||
|
||||
impl FlownodeContext {
|
||||
@@ -226,7 +233,7 @@ impl FlownodeContext {
|
||||
/// Retrieves a GlobalId and table schema representing a table previously registered by calling the [register_table] function.
|
||||
///
|
||||
/// Returns an error if no table has been registered with the provided names
|
||||
pub fn table(&self, name: &TableName) -> Result<(GlobalId, RelationType), Error> {
|
||||
pub fn table(&self, name: &TableName) -> Result<(GlobalId, RelationDesc), Error> {
|
||||
let id = self
|
||||
.table_repr
|
||||
.get_by_name(name)
|
||||
@@ -297,7 +304,7 @@ impl FlownodeContext {
|
||||
.get_by_name(table_name)
|
||||
.map(|(_, gid)| gid)
|
||||
.unwrap();
|
||||
self.schema.insert(gid, schema);
|
||||
self.schema.insert(gid, schema.into_unnamed());
|
||||
Ok(())
|
||||
}
|
||||
|
||||
|
||||
@@ -17,7 +17,6 @@
|
||||
use common_error::ext::BoxedError;
|
||||
use common_meta::key::table_info::{TableInfoManager, TableInfoValue};
|
||||
use common_meta::key::table_name::{TableNameKey, TableNameManager};
|
||||
use itertools::Itertools;
|
||||
use snafu::{OptionExt, ResultExt};
|
||||
use table::metadata::TableId;
|
||||
|
||||
@@ -25,7 +24,7 @@ use crate::adapter::error::{
|
||||
Error, ExternalSnafu, TableNotFoundMetaSnafu, TableNotFoundSnafu, UnexpectedSnafu,
|
||||
};
|
||||
use crate::adapter::TableName;
|
||||
use crate::repr::{self, ColumnType, RelationType};
|
||||
use crate::repr::{self, ColumnType, RelationDesc, RelationType};
|
||||
|
||||
/// mapping of table name <-> table id should be query from tableinfo manager
|
||||
pub struct TableSource {
|
||||
@@ -107,7 +106,7 @@ impl TableSource {
|
||||
pub async fn get_table_name_schema(
|
||||
&self,
|
||||
table_id: &TableId,
|
||||
) -> Result<(TableName, RelationType), Error> {
|
||||
) -> Result<(TableName, RelationDesc), Error> {
|
||||
let table_info_value = self
|
||||
.get_table_info_value(table_id)
|
||||
.await?
|
||||
@@ -123,14 +122,20 @@ impl TableSource {
|
||||
];
|
||||
|
||||
let raw_schema = table_info_value.table_info.meta.schema;
|
||||
let column_types = raw_schema
|
||||
let (column_types, col_names): (Vec<_>, Vec<_>) = raw_schema
|
||||
.column_schemas
|
||||
.clone()
|
||||
.into_iter()
|
||||
.map(|col| ColumnType {
|
||||
nullable: col.is_nullable(),
|
||||
scalar_type: col.data_type,
|
||||
.map(|col| {
|
||||
(
|
||||
ColumnType {
|
||||
nullable: col.is_nullable(),
|
||||
scalar_type: col.data_type,
|
||||
},
|
||||
col.name,
|
||||
)
|
||||
})
|
||||
.collect_vec();
|
||||
.unzip();
|
||||
|
||||
let key = table_info_value.table_info.meta.primary_key_indices;
|
||||
let keys = vec![repr::Key::from(key)];
|
||||
@@ -138,10 +143,13 @@ impl TableSource {
|
||||
let time_index = raw_schema.timestamp_index;
|
||||
Ok((
|
||||
table_name,
|
||||
RelationType {
|
||||
column_types,
|
||||
keys,
|
||||
time_index,
|
||||
RelationDesc {
|
||||
typ: RelationType {
|
||||
column_types,
|
||||
keys,
|
||||
time_index,
|
||||
},
|
||||
names: col_names,
|
||||
},
|
||||
))
|
||||
}
|
||||
|
||||
@@ -232,12 +232,12 @@ impl<'s> Worker<'s> {
|
||||
source_ids: &[GlobalId],
|
||||
src_recvs: Vec<broadcast::Receiver<DiffRow>>,
|
||||
// TODO(discord9): set expire duration for all arrangement and compare to sys timestamp instead
|
||||
expire_when: Option<repr::Duration>,
|
||||
create_if_not_exist: bool,
|
||||
expire_after: Option<repr::Duration>,
|
||||
create_if_not_exists: bool,
|
||||
err_collector: ErrCollector,
|
||||
) -> Result<Option<FlowId>, Error> {
|
||||
let already_exist = self.task_states.contains_key(&flow_id);
|
||||
match (already_exist, create_if_not_exist) {
|
||||
let already_exists = self.task_states.contains_key(&flow_id);
|
||||
match (already_exists, create_if_not_exists) {
|
||||
(true, true) => return Ok(None),
|
||||
(true, false) => FlowAlreadyExistSnafu { id: flow_id }.fail()?,
|
||||
(false, _) => (),
|
||||
@@ -247,7 +247,7 @@ impl<'s> Worker<'s> {
|
||||
err_collector,
|
||||
..Default::default()
|
||||
};
|
||||
cur_task_state.state.set_expire_after(expire_when);
|
||||
cur_task_state.state.set_expire_after(expire_after);
|
||||
|
||||
{
|
||||
let mut ctx = cur_task_state.new_ctx(sink_id);
|
||||
@@ -319,8 +319,8 @@ impl<'s> Worker<'s> {
|
||||
sink_sender,
|
||||
source_ids,
|
||||
src_recvs,
|
||||
expire_when,
|
||||
create_if_not_exist,
|
||||
expire_after,
|
||||
create_if_not_exists,
|
||||
err_collector,
|
||||
} => {
|
||||
let task_create_result = self.create_flow(
|
||||
@@ -330,8 +330,8 @@ impl<'s> Worker<'s> {
|
||||
sink_sender,
|
||||
&source_ids,
|
||||
src_recvs,
|
||||
expire_when,
|
||||
create_if_not_exist,
|
||||
expire_after,
|
||||
create_if_not_exists,
|
||||
err_collector,
|
||||
);
|
||||
Some((
|
||||
@@ -368,8 +368,8 @@ pub enum Request {
|
||||
sink_sender: mpsc::UnboundedSender<DiffRow>,
|
||||
source_ids: Vec<GlobalId>,
|
||||
src_recvs: Vec<broadcast::Receiver<DiffRow>>,
|
||||
expire_when: Option<repr::Duration>,
|
||||
create_if_not_exist: bool,
|
||||
expire_after: Option<repr::Duration>,
|
||||
create_if_not_exists: bool,
|
||||
err_collector: ErrCollector,
|
||||
},
|
||||
Remove {
|
||||
@@ -524,8 +524,8 @@ mod test {
|
||||
sink_sender: sink_tx,
|
||||
source_ids: src_ids,
|
||||
src_recvs: vec![rx],
|
||||
expire_when: None,
|
||||
create_if_not_exist: true,
|
||||
expire_after: None,
|
||||
create_if_not_exists: true,
|
||||
err_collector: ErrCollector::default(),
|
||||
};
|
||||
handle.create_flow(create_reqs).await.unwrap();
|
||||
|
||||
@@ -124,9 +124,13 @@ fn mfp_subgraph(
|
||||
// 1. Read all updates that were emitted between the last time this arrangement had updates and the current time.
|
||||
// 2. Output the updates.
|
||||
// 3. Truncate all updates within that range.
|
||||
let from = arrange.read().last_compaction_time().map(|n| n + 1);
|
||||
let from = arrange.read().last_compaction_time();
|
||||
let from = from.unwrap_or(repr::Timestamp::MIN);
|
||||
let output_kv = arrange.read().get_updates_in_range(from..=now);
|
||||
let range = (
|
||||
std::ops::Bound::Excluded(from),
|
||||
std::ops::Bound::Included(now),
|
||||
);
|
||||
let output_kv = arrange.read().get_updates_in_range(range);
|
||||
// the output is expected to be key -> empty val
|
||||
let output = output_kv
|
||||
.into_iter()
|
||||
|
||||
@@ -26,7 +26,7 @@ use crate::adapter::error::{Error, PlanSnafu};
|
||||
use crate::compute::render::{Context, SubgraphArg};
|
||||
use crate::compute::state::Scheduler;
|
||||
use crate::compute::types::{Arranged, Collection, CollectionBundle, ErrCollector, Toff};
|
||||
use crate::expr::error::{DataTypeSnafu, InternalSnafu};
|
||||
use crate::expr::error::{DataAlreadyExpiredSnafu, DataTypeSnafu, InternalSnafu};
|
||||
use crate::expr::{AggregateExpr, EvalError, ScalarExpr};
|
||||
use crate::plan::{AccumulablePlan, AggrWithIndex, KeyValPlan, Plan, ReducePlan, TypedPlan};
|
||||
use crate::repr::{self, DiffRow, KeyValDiffRow, RelationType, Row};
|
||||
@@ -301,9 +301,13 @@ fn update_reduce_distinct_arrange(
|
||||
// Deal with output:
|
||||
|
||||
// 1. Read all updates that were emitted between the last time this arrangement had updates and the current time.
|
||||
let from = arrange.read().last_compaction_time().map(|n| n + 1);
|
||||
let from = arrange.read().last_compaction_time();
|
||||
let from = from.unwrap_or(repr::Timestamp::MIN);
|
||||
let output_kv = arrange.read().get_updates_in_range(from..=now);
|
||||
let range = (
|
||||
std::ops::Bound::Excluded(from),
|
||||
std::ops::Bound::Included(now),
|
||||
);
|
||||
let output_kv = arrange.read().get_updates_in_range(range);
|
||||
|
||||
// 2. Truncate all updates stored in arrangement within that range.
|
||||
let run_compaction = || {
|
||||
@@ -397,6 +401,29 @@ fn reduce_accum_subgraph(
|
||||
// TODO(discord9): consider key-based lock
|
||||
let mut arrange = arrange.write();
|
||||
for (key, value_diffs) in key_to_vals {
|
||||
if let Some(expire_man) = &arrange.get_expire_state() {
|
||||
let mut is_expired = false;
|
||||
err_collector.run(|| {
|
||||
if let Some(expired) = expire_man.get_expire_duration(now, &key)? {
|
||||
is_expired = true;
|
||||
// expired data is ignored in computation, and a simple warning is logged
|
||||
common_telemetry::warn!(
|
||||
"Data already expired: {}",
|
||||
DataAlreadyExpiredSnafu {
|
||||
expired_by: expired,
|
||||
}
|
||||
.build()
|
||||
);
|
||||
Ok(())
|
||||
} else {
|
||||
Ok(())
|
||||
}
|
||||
});
|
||||
if is_expired {
|
||||
// errors already collected, we can just continue to next key
|
||||
continue;
|
||||
}
|
||||
}
|
||||
let col_diffs = {
|
||||
let row_len = value_diffs[0].0.len();
|
||||
let res = err_collector.run(|| get_col_diffs(value_diffs, row_len));
|
||||
|
||||
@@ -20,12 +20,14 @@ use common_telemetry::{debug, info};
|
||||
use hydroflow::scheduled::graph_ext::GraphExt;
|
||||
use itertools::Itertools;
|
||||
use snafu::OptionExt;
|
||||
use tokio::sync::broadcast::error::TryRecvError;
|
||||
use tokio::sync::{broadcast, mpsc};
|
||||
|
||||
use crate::adapter::error::{Error, PlanSnafu};
|
||||
use crate::compute::render::Context;
|
||||
use crate::compute::types::{Arranged, Collection, CollectionBundle, Toff};
|
||||
use crate::expr::GlobalId;
|
||||
use crate::expr::error::InternalSnafu;
|
||||
use crate::expr::{EvalError, GlobalId};
|
||||
use crate::repr::{DiffRow, Row, BROADCAST_CAP};
|
||||
|
||||
#[allow(clippy::mutable_key_type)]
|
||||
@@ -55,18 +57,43 @@ impl<'referred, 'df> Context<'referred, 'df> {
|
||||
.df
|
||||
.add_subgraph_source("source", send_port, move |_ctx, send| {
|
||||
let now = *now.borrow();
|
||||
let arr = arrange_handler_inner.write().get_updates_in_range(..=now);
|
||||
err_collector.run(|| arrange_handler_inner.write().compact_to(now));
|
||||
// write lock to prevent unexpected mutation
|
||||
let mut arranged = arrange_handler_inner.write();
|
||||
let arr = arranged.get_updates_in_range(..=now);
|
||||
err_collector.run(|| arranged.compact_to(now));
|
||||
|
||||
debug!("Call source");
|
||||
let prev_avail = arr.into_iter().map(|((k, _), t, d)| (k, t, d));
|
||||
let mut to_send = Vec::new();
|
||||
let mut to_arrange = Vec::new();
|
||||
// TODO(discord9): handling tokio broadcast error
|
||||
while let Ok((r, t, d)) = src_recv.try_recv() {
|
||||
if t <= now {
|
||||
to_send.push((r, t, d));
|
||||
} else {
|
||||
to_arrange.push(((r, Row::empty()), t, d));
|
||||
loop {
|
||||
match src_recv.try_recv() {
|
||||
Ok((r, t, d)) => {
|
||||
if t <= now {
|
||||
to_send.push((r, t, d));
|
||||
} else {
|
||||
to_arrange.push(((r, Row::empty()), t, d));
|
||||
}
|
||||
}
|
||||
Err(TryRecvError::Empty) => {
|
||||
break;
|
||||
}
|
||||
Err(TryRecvError::Lagged(lag_offset)) => {
|
||||
common_telemetry::error!("Flow missing {} rows behind", lag_offset);
|
||||
break;
|
||||
}
|
||||
Err(err) => {
|
||||
err_collector.run(|| -> Result<(), EvalError> {
|
||||
InternalSnafu {
|
||||
reason: format!(
|
||||
"Error receiving from broadcast channel: {}",
|
||||
err
|
||||
),
|
||||
}
|
||||
.fail()
|
||||
});
|
||||
}
|
||||
}
|
||||
}
|
||||
let all = prev_avail.chain(to_send).collect_vec();
|
||||
@@ -77,10 +104,10 @@ impl<'referred, 'df> Context<'referred, 'df> {
|
||||
to_arrange.len()
|
||||
);
|
||||
}
|
||||
err_collector.run(|| arrange_handler_inner.write().apply_updates(now, to_arrange));
|
||||
err_collector.run(|| arranged.apply_updates(now, to_arrange));
|
||||
send.give(all);
|
||||
// always schedule source to run at next tick
|
||||
inner_schd.schedule_at(now + 1);
|
||||
// always schedule source to run at now so we can repeatedly run source if needed
|
||||
inner_schd.schedule_at(now);
|
||||
});
|
||||
schd.set_cur_subgraph(sub);
|
||||
let arranged = Arranged::new(arrange_handler);
|
||||
|
||||
@@ -100,4 +100,11 @@ pub enum EvalError {
|
||||
#[snafu(implicit)]
|
||||
location: Location,
|
||||
},
|
||||
|
||||
#[snafu(display("Incoming data already expired by {} ms", expired_by))]
|
||||
DataAlreadyExpired {
|
||||
expired_by: i64,
|
||||
#[snafu(implicit)]
|
||||
location: Location,
|
||||
},
|
||||
}
|
||||
|
||||
@@ -76,6 +76,13 @@ impl UnmaterializableFunc {
|
||||
}
|
||||
}
|
||||
|
||||
pub fn is_valid_func_name(name: &str) -> bool {
|
||||
matches!(
|
||||
name.to_lowercase().as_str(),
|
||||
"now" | "current_schema" | "tumble"
|
||||
)
|
||||
}
|
||||
|
||||
/// Create a UnmaterializableFunc from a string of the function name
|
||||
pub fn from_str_args(name: &str, args: Vec<TypedExpr>) -> Result<Self, Error> {
|
||||
match name.to_lowercase().as_str() {
|
||||
@@ -183,6 +190,13 @@ impl UnaryFunc {
|
||||
}
|
||||
}
|
||||
|
||||
pub fn is_valid_func_name(name: &str) -> bool {
|
||||
matches!(
|
||||
name.to_lowercase().as_str(),
|
||||
"not" | "is_null" | "is_true" | "is_false" | "step_timestamp" | "cast"
|
||||
)
|
||||
}
|
||||
|
||||
/// Create a UnaryFunc from a string of the function name and given argument type(optional)
|
||||
pub fn from_str_and_type(
|
||||
name: &str,
|
||||
@@ -278,9 +292,9 @@ impl UnaryFunc {
|
||||
start_time,
|
||||
} => {
|
||||
let ts = get_ts_as_millisecond(arg)?;
|
||||
let start_time = start_time.map(|t| t.val()).unwrap_or(0);
|
||||
let start_time = start_time.map(|t| t.val());
|
||||
let window_size = (window_size.to_nanosecond() / 1_000_000) as repr::Duration; // nanosecond to millisecond
|
||||
let window_start = start_time + (ts - start_time) / window_size * window_size;
|
||||
let window_start = get_window_start(ts, window_size, start_time);
|
||||
|
||||
let ret = Timestamp::new_millisecond(window_start);
|
||||
Ok(Value::from(ret))
|
||||
@@ -290,9 +304,9 @@ impl UnaryFunc {
|
||||
start_time,
|
||||
} => {
|
||||
let ts = get_ts_as_millisecond(arg)?;
|
||||
let start_time = start_time.map(|t| t.val()).unwrap_or(0);
|
||||
let start_time = start_time.map(|t| t.val());
|
||||
let window_size = (window_size.to_nanosecond() / 1_000_000) as repr::Duration; // nanosecond to millisecond
|
||||
let window_start = start_time + (ts - start_time) / window_size * window_size;
|
||||
let window_start = get_window_start(ts, window_size, start_time);
|
||||
|
||||
let window_end = window_start + window_size;
|
||||
let ret = Timestamp::new_millisecond(window_end);
|
||||
@@ -302,6 +316,35 @@ impl UnaryFunc {
|
||||
}
|
||||
}
|
||||
|
||||
fn get_window_start(
|
||||
ts: repr::Timestamp,
|
||||
window_size: repr::Duration,
|
||||
start_time: Option<repr::Timestamp>,
|
||||
) -> repr::Timestamp {
|
||||
let start_time = start_time.unwrap_or(0);
|
||||
// left close right open
|
||||
if ts >= start_time {
|
||||
start_time + (ts - start_time) / window_size * window_size
|
||||
} else {
|
||||
start_time + (ts - start_time) / window_size * window_size
|
||||
- if ((start_time - ts) % window_size) != 0 {
|
||||
window_size
|
||||
} else {
|
||||
0
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_get_window_start() {
|
||||
assert_eq!(get_window_start(1, 3, None), 0);
|
||||
assert_eq!(get_window_start(3, 3, None), 3);
|
||||
assert_eq!(get_window_start(0, 3, None), 0);
|
||||
|
||||
assert_eq!(get_window_start(-1, 3, None), -3);
|
||||
assert_eq!(get_window_start(-3, 3, None), -3);
|
||||
}
|
||||
|
||||
fn get_ts_as_millisecond(arg: Value) -> Result<repr::Timestamp, EvalError> {
|
||||
let ts = if let Some(ts) = arg.as_timestamp() {
|
||||
ts.convert_to(TimeUnit::Millisecond)
|
||||
@@ -550,6 +593,27 @@ impl BinaryFunc {
|
||||
Ok(ret)
|
||||
}
|
||||
|
||||
pub fn is_valid_func_name(name: &str) -> bool {
|
||||
matches!(
|
||||
name.to_lowercase().as_str(),
|
||||
"eq" | "equal"
|
||||
| "not_eq"
|
||||
| "not_equal"
|
||||
| "lt"
|
||||
| "lte"
|
||||
| "gt"
|
||||
| "gte"
|
||||
| "add"
|
||||
| "sub"
|
||||
| "subtract"
|
||||
| "mul"
|
||||
| "multiply"
|
||||
| "div"
|
||||
| "divide"
|
||||
| "mod"
|
||||
)
|
||||
}
|
||||
|
||||
/// choose the appropriate specialization based on the input types
|
||||
/// return a specialization of the binary function and it's actual input and output type(so no null type present)
|
||||
///
|
||||
@@ -741,6 +805,10 @@ impl VariadicFunc {
|
||||
}
|
||||
}
|
||||
|
||||
pub fn is_valid_func_name(name: &str) -> bool {
|
||||
matches!(name.to_lowercase().as_str(), "and" | "or")
|
||||
}
|
||||
|
||||
/// Create a VariadicFunc from a string of the function name and given argument types(optional)
|
||||
pub fn from_str_and_types(
|
||||
name: &str,
|
||||
|
||||
@@ -45,6 +45,8 @@ impl TypedExpr {
|
||||
|
||||
impl TypedExpr {
|
||||
/// expand multi-value expression to multiple expressions with new indices
|
||||
///
|
||||
/// Currently it just mean expand `TumbleWindow` to `TumbleWindowFloor` and `TumbleWindowCeiling`
|
||||
pub fn expand_multi_value(
|
||||
input_typ: &RelationType,
|
||||
exprs: &[TypedExpr],
|
||||
|
||||
@@ -262,6 +262,19 @@ impl RelationType {
|
||||
|
||||
true
|
||||
}
|
||||
|
||||
/// Return relation describe with column names
|
||||
pub fn into_named(self, names: Vec<ColumnName>) -> RelationDesc {
|
||||
RelationDesc { typ: self, names }
|
||||
}
|
||||
|
||||
/// Return relation describe without column names
|
||||
pub fn into_unnamed(self) -> RelationDesc {
|
||||
RelationDesc {
|
||||
typ: self,
|
||||
names: vec![],
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/// The type of a `Value`
|
||||
@@ -325,8 +338,8 @@ fn return_true() -> bool {
|
||||
/// Individual column names are optional.
|
||||
#[derive(Clone, Debug, Eq, PartialEq, Serialize, Deserialize, Hash)]
|
||||
pub struct RelationDesc {
|
||||
typ: RelationType,
|
||||
names: Vec<ColumnName>,
|
||||
pub typ: RelationType,
|
||||
pub names: Vec<ColumnName>,
|
||||
}
|
||||
|
||||
impl RelationDesc {
|
||||
|
||||
@@ -211,7 +211,7 @@ mod test {
|
||||
let schema = RelationType::new(vec![ColumnType::new(CDT::uint32_datatype(), false)]);
|
||||
|
||||
tri_map.insert(Some(name.clone()), Some(1024), gid);
|
||||
schemas.insert(gid, schema);
|
||||
schemas.insert(gid, schema.into_unnamed());
|
||||
}
|
||||
|
||||
{
|
||||
@@ -225,7 +225,7 @@ mod test {
|
||||
ColumnType::new(CDT::uint32_datatype(), false),
|
||||
ColumnType::new(CDT::datetime_datatype(), false),
|
||||
]);
|
||||
schemas.insert(gid, schema);
|
||||
schemas.insert(gid, schema.into_unnamed());
|
||||
tri_map.insert(Some(name.clone()), Some(1025), gid);
|
||||
}
|
||||
|
||||
|
||||
@@ -435,6 +435,236 @@ mod test {
|
||||
use crate::repr::{self, ColumnType, RelationType};
|
||||
use crate::transform::test::{create_test_ctx, create_test_query_engine, sql_to_substrait};
|
||||
|
||||
/// TODO(discord9): add more illegal sql tests
|
||||
#[tokio::test]
|
||||
async fn test_tumble_composite() {
|
||||
let engine = create_test_query_engine();
|
||||
let sql =
|
||||
"SELECT number, avg(number) FROM numbers_with_ts GROUP BY tumble(ts, '1 hour'), number";
|
||||
let plan = sql_to_substrait(engine.clone(), sql).await;
|
||||
|
||||
let mut ctx = create_test_ctx();
|
||||
let flow_plan = TypedPlan::from_substrait_plan(&mut ctx, &plan).unwrap();
|
||||
|
||||
let aggr_exprs = vec![
|
||||
AggregateExpr {
|
||||
func: AggregateFunc::SumUInt32,
|
||||
expr: ScalarExpr::Column(0),
|
||||
distinct: false,
|
||||
},
|
||||
AggregateExpr {
|
||||
func: AggregateFunc::Count,
|
||||
expr: ScalarExpr::Column(0),
|
||||
distinct: false,
|
||||
},
|
||||
];
|
||||
let avg_expr = ScalarExpr::If {
|
||||
cond: Box::new(ScalarExpr::Column(4).call_binary(
|
||||
ScalarExpr::Literal(Value::from(0i64), CDT::int64_datatype()),
|
||||
BinaryFunc::NotEq,
|
||||
)),
|
||||
then: Box::new(ScalarExpr::Column(3).call_binary(
|
||||
ScalarExpr::Column(4).call_unary(UnaryFunc::Cast(CDT::uint64_datatype())),
|
||||
BinaryFunc::DivUInt64,
|
||||
)),
|
||||
els: Box::new(ScalarExpr::Literal(Value::Null, CDT::uint64_datatype())),
|
||||
};
|
||||
let expected = TypedPlan {
|
||||
// TODO(discord9): mfp indirectly ref to key columns
|
||||
/*
|
||||
.with_key(vec![1])
|
||||
.with_time_index(Some(0)),*/
|
||||
plan: Plan::Mfp {
|
||||
input: Box::new(
|
||||
Plan::Reduce {
|
||||
input: Box::new(
|
||||
Plan::Get {
|
||||
id: crate::expr::Id::Global(GlobalId::User(1)),
|
||||
}
|
||||
.with_types(RelationType::new(vec![
|
||||
ColumnType::new(ConcreteDataType::uint32_datatype(), false),
|
||||
ColumnType::new(ConcreteDataType::datetime_datatype(), false),
|
||||
])),
|
||||
),
|
||||
key_val_plan: KeyValPlan {
|
||||
key_plan: MapFilterProject::new(2)
|
||||
.map(vec![
|
||||
ScalarExpr::Column(1).call_unary(
|
||||
UnaryFunc::TumbleWindowFloor {
|
||||
window_size: Interval::from_month_day_nano(
|
||||
0,
|
||||
0,
|
||||
3_600_000_000_000,
|
||||
),
|
||||
start_time: None,
|
||||
},
|
||||
),
|
||||
ScalarExpr::Column(1).call_unary(
|
||||
UnaryFunc::TumbleWindowCeiling {
|
||||
window_size: Interval::from_month_day_nano(
|
||||
0,
|
||||
0,
|
||||
3_600_000_000_000,
|
||||
),
|
||||
start_time: None,
|
||||
},
|
||||
),
|
||||
ScalarExpr::Column(0),
|
||||
])
|
||||
.unwrap()
|
||||
.project(vec![2, 3, 4])
|
||||
.unwrap()
|
||||
.into_safe(),
|
||||
val_plan: MapFilterProject::new(2)
|
||||
.project(vec![0, 1])
|
||||
.unwrap()
|
||||
.into_safe(),
|
||||
},
|
||||
reduce_plan: ReducePlan::Accumulable(AccumulablePlan {
|
||||
full_aggrs: aggr_exprs.clone(),
|
||||
simple_aggrs: vec![
|
||||
AggrWithIndex::new(aggr_exprs[0].clone(), 0, 0),
|
||||
AggrWithIndex::new(aggr_exprs[1].clone(), 0, 1),
|
||||
],
|
||||
distinct_aggrs: vec![],
|
||||
}),
|
||||
}
|
||||
.with_types(
|
||||
RelationType::new(vec![
|
||||
// keys
|
||||
ColumnType::new(CDT::datetime_datatype(), false), // window start(time index)
|
||||
ColumnType::new(CDT::datetime_datatype(), false), // window end(pk)
|
||||
ColumnType::new(CDT::uint32_datatype(), false), // number(pk)
|
||||
// values
|
||||
ColumnType::new(CDT::uint64_datatype(), true), // avg.sum(number)
|
||||
ColumnType::new(CDT::int64_datatype(), true), // avg.count(number)
|
||||
])
|
||||
.with_key(vec![1, 2])
|
||||
.with_time_index(Some(0)),
|
||||
),
|
||||
),
|
||||
mfp: MapFilterProject::new(5)
|
||||
.map(vec![
|
||||
avg_expr,
|
||||
ScalarExpr::Column(2), // number(pk)
|
||||
ScalarExpr::Column(5), // avg.sum(number)
|
||||
ScalarExpr::Column(0), // window start
|
||||
ScalarExpr::Column(1), // window end
|
||||
])
|
||||
.unwrap()
|
||||
.project(vec![6, 7, 8, 9])
|
||||
.unwrap(),
|
||||
},
|
||||
typ: RelationType::new(vec![
|
||||
ColumnType::new(CDT::uint32_datatype(), false), // number
|
||||
ColumnType::new(CDT::uint64_datatype(), true), // avg(number)
|
||||
ColumnType::new(CDT::datetime_datatype(), false), // window start
|
||||
ColumnType::new(CDT::datetime_datatype(), false), // window end
|
||||
]),
|
||||
};
|
||||
assert_eq!(flow_plan, expected);
|
||||
}
|
||||
|
||||
#[tokio::test]
|
||||
async fn test_tumble_parse_optional() {
|
||||
let engine = create_test_query_engine();
|
||||
let sql = "SELECT sum(number) FROM numbers_with_ts GROUP BY tumble(ts, '1 hour')";
|
||||
let plan = sql_to_substrait(engine.clone(), sql).await;
|
||||
|
||||
let mut ctx = create_test_ctx();
|
||||
let flow_plan = TypedPlan::from_substrait_plan(&mut ctx, &plan).unwrap();
|
||||
|
||||
let aggr_expr = AggregateExpr {
|
||||
func: AggregateFunc::SumUInt32,
|
||||
expr: ScalarExpr::Column(0),
|
||||
distinct: false,
|
||||
};
|
||||
let expected = TypedPlan {
|
||||
typ: RelationType::new(vec![
|
||||
ColumnType::new(CDT::uint64_datatype(), true), // sum(number)
|
||||
ColumnType::new(CDT::datetime_datatype(), false), // window start
|
||||
ColumnType::new(CDT::datetime_datatype(), false), // window end
|
||||
]),
|
||||
// TODO(discord9): mfp indirectly ref to key columns
|
||||
/*
|
||||
.with_key(vec![1])
|
||||
.with_time_index(Some(0)),*/
|
||||
plan: Plan::Mfp {
|
||||
input: Box::new(
|
||||
Plan::Reduce {
|
||||
input: Box::new(
|
||||
Plan::Get {
|
||||
id: crate::expr::Id::Global(GlobalId::User(1)),
|
||||
}
|
||||
.with_types(RelationType::new(vec![
|
||||
ColumnType::new(ConcreteDataType::uint32_datatype(), false),
|
||||
ColumnType::new(ConcreteDataType::datetime_datatype(), false),
|
||||
])),
|
||||
),
|
||||
key_val_plan: KeyValPlan {
|
||||
key_plan: MapFilterProject::new(2)
|
||||
.map(vec![
|
||||
ScalarExpr::Column(1).call_unary(
|
||||
UnaryFunc::TumbleWindowFloor {
|
||||
window_size: Interval::from_month_day_nano(
|
||||
0,
|
||||
0,
|
||||
3_600_000_000_000,
|
||||
),
|
||||
start_time: None,
|
||||
},
|
||||
),
|
||||
ScalarExpr::Column(1).call_unary(
|
||||
UnaryFunc::TumbleWindowCeiling {
|
||||
window_size: Interval::from_month_day_nano(
|
||||
0,
|
||||
0,
|
||||
3_600_000_000_000,
|
||||
),
|
||||
start_time: None,
|
||||
},
|
||||
),
|
||||
])
|
||||
.unwrap()
|
||||
.project(vec![2, 3])
|
||||
.unwrap()
|
||||
.into_safe(),
|
||||
val_plan: MapFilterProject::new(2)
|
||||
.project(vec![0, 1])
|
||||
.unwrap()
|
||||
.into_safe(),
|
||||
},
|
||||
reduce_plan: ReducePlan::Accumulable(AccumulablePlan {
|
||||
full_aggrs: vec![aggr_expr.clone()],
|
||||
simple_aggrs: vec![AggrWithIndex::new(aggr_expr.clone(), 0, 0)],
|
||||
distinct_aggrs: vec![],
|
||||
}),
|
||||
}
|
||||
.with_types(
|
||||
RelationType::new(vec![
|
||||
ColumnType::new(CDT::datetime_datatype(), false), // window start
|
||||
ColumnType::new(CDT::datetime_datatype(), false), // window end
|
||||
ColumnType::new(CDT::uint64_datatype(), true), //sum(number)
|
||||
])
|
||||
.with_key(vec![1])
|
||||
.with_time_index(Some(0)),
|
||||
),
|
||||
),
|
||||
mfp: MapFilterProject::new(3)
|
||||
.map(vec![
|
||||
ScalarExpr::Column(2),
|
||||
ScalarExpr::Column(3),
|
||||
ScalarExpr::Column(0),
|
||||
ScalarExpr::Column(1),
|
||||
])
|
||||
.unwrap()
|
||||
.project(vec![4, 5, 6])
|
||||
.unwrap(),
|
||||
},
|
||||
};
|
||||
assert_eq!(flow_plan, expected);
|
||||
}
|
||||
|
||||
#[tokio::test]
|
||||
async fn test_tumble_parse() {
|
||||
let engine = create_test_query_engine();
|
||||
|
||||
@@ -101,8 +101,7 @@ impl TypedExpr {
|
||||
.unzip();
|
||||
|
||||
match arg_len {
|
||||
// because variadic function can also have 1 arguments, we need to check if it's a variadic function first
|
||||
1 if VariadicFunc::from_str_and_types(fn_name, &arg_types).is_err() => {
|
||||
1 if UnaryFunc::is_valid_func_name(fn_name) => {
|
||||
let func = UnaryFunc::from_str_and_type(fn_name, None)?;
|
||||
let arg = arg_exprs[0].clone();
|
||||
let ret_type = ColumnType::new_nullable(func.signature().output.clone());
|
||||
@@ -124,8 +123,7 @@ impl TypedExpr {
|
||||
|
||||
Ok(TypedExpr::new(arg.call_unary(func), ret_type))
|
||||
}
|
||||
// because variadic function can also have 2 arguments, we need to check if it's a variadic function first
|
||||
2 if VariadicFunc::from_str_and_types(fn_name, &arg_types).is_err() => {
|
||||
2 if BinaryFunc::is_valid_func_name(fn_name) => {
|
||||
let (func, signature) =
|
||||
BinaryFunc::from_str_expr_and_type(fn_name, &arg_exprs, &arg_types[0..2])?;
|
||||
|
||||
@@ -167,7 +165,8 @@ impl TypedExpr {
|
||||
Ok(TypedExpr::new(ret_expr, ret_type))
|
||||
}
|
||||
_var => {
|
||||
if let Ok(func) = VariadicFunc::from_str_and_types(fn_name, &arg_types) {
|
||||
if VariadicFunc::is_valid_func_name(fn_name) {
|
||||
let func = VariadicFunc::from_str_and_types(fn_name, &arg_types)?;
|
||||
let ret_type = ColumnType::new_nullable(func.signature().output.clone());
|
||||
let mut expr = ScalarExpr::CallVariadic {
|
||||
func,
|
||||
@@ -175,9 +174,8 @@ impl TypedExpr {
|
||||
};
|
||||
expr.optimize();
|
||||
Ok(TypedExpr::new(expr, ret_type))
|
||||
} else if let Ok(func) =
|
||||
UnmaterializableFunc::from_str_args(fn_name, arg_typed_exprs)
|
||||
{
|
||||
} else if UnmaterializableFunc::is_valid_func_name(fn_name) {
|
||||
let func = UnmaterializableFunc::from_str_args(fn_name, arg_typed_exprs)?;
|
||||
let ret_type = ColumnType::new_nullable(func.signature().output.clone());
|
||||
Ok(TypedExpr::new(
|
||||
ScalarExpr::CallUnmaterializable(func),
|
||||
@@ -324,8 +322,12 @@ impl TypedExpr {
|
||||
|
||||
#[cfg(test)]
|
||||
mod test {
|
||||
use std::collections::HashMap;
|
||||
|
||||
use common_time::{DateTime, Interval};
|
||||
use datatypes::prelude::ConcreteDataType;
|
||||
use datatypes::value::Value;
|
||||
use pretty_assertions::assert_eq;
|
||||
|
||||
use super::*;
|
||||
use crate::expr::{GlobalId, MapFilterProject};
|
||||
@@ -510,4 +512,162 @@ mod test {
|
||||
|
||||
assert_eq!(flow_plan.unwrap(), expected);
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_func_sig() {
|
||||
fn lit(v: impl ToString) -> substrait_proto::proto::FunctionArgument {
|
||||
use substrait_proto::proto::expression;
|
||||
let expr = Expression {
|
||||
rex_type: Some(expression::RexType::Literal(expression::Literal {
|
||||
nullable: false,
|
||||
type_variation_reference: 0,
|
||||
literal_type: Some(expression::literal::LiteralType::String(v.to_string())),
|
||||
})),
|
||||
};
|
||||
substrait_proto::proto::FunctionArgument {
|
||||
arg_type: Some(substrait_proto::proto::function_argument::ArgType::Value(
|
||||
expr,
|
||||
)),
|
||||
}
|
||||
}
|
||||
fn col(i: usize) -> substrait_proto::proto::FunctionArgument {
|
||||
use substrait_proto::proto::expression;
|
||||
let expr = Expression {
|
||||
rex_type: Some(expression::RexType::Selection(Box::new(
|
||||
expression::FieldReference {
|
||||
reference_type: Some(
|
||||
expression::field_reference::ReferenceType::DirectReference(
|
||||
expression::ReferenceSegment {
|
||||
reference_type: Some(
|
||||
expression::reference_segment::ReferenceType::StructField(
|
||||
Box::new(expression::reference_segment::StructField {
|
||||
field: i as i32,
|
||||
child: None,
|
||||
}),
|
||||
),
|
||||
),
|
||||
},
|
||||
),
|
||||
),
|
||||
root_type: None,
|
||||
},
|
||||
))),
|
||||
};
|
||||
substrait_proto::proto::FunctionArgument {
|
||||
arg_type: Some(substrait_proto::proto::function_argument::ArgType::Value(
|
||||
expr,
|
||||
)),
|
||||
}
|
||||
}
|
||||
|
||||
let f = substrait_proto::proto::expression::ScalarFunction {
|
||||
function_reference: 0,
|
||||
arguments: vec![col(0)],
|
||||
options: vec![],
|
||||
output_type: None,
|
||||
..Default::default()
|
||||
};
|
||||
let input_schema = RelationType::new(vec![ColumnType::new(CDT::uint32_datatype(), false)]);
|
||||
let extensions = FunctionExtensions {
|
||||
anchor_to_name: HashMap::from([(0, "is_null".to_string())]),
|
||||
};
|
||||
let res = TypedExpr::from_substrait_scalar_func(&f, &input_schema, &extensions).unwrap();
|
||||
|
||||
assert_eq!(
|
||||
res,
|
||||
TypedExpr {
|
||||
expr: ScalarExpr::Column(0).call_unary(UnaryFunc::IsNull),
|
||||
typ: ColumnType {
|
||||
scalar_type: CDT::boolean_datatype(),
|
||||
nullable: true,
|
||||
},
|
||||
}
|
||||
);
|
||||
|
||||
let f = substrait_proto::proto::expression::ScalarFunction {
|
||||
function_reference: 0,
|
||||
arguments: vec![col(0), col(1)],
|
||||
options: vec![],
|
||||
output_type: None,
|
||||
..Default::default()
|
||||
};
|
||||
let input_schema = RelationType::new(vec![
|
||||
ColumnType::new(CDT::uint32_datatype(), false),
|
||||
ColumnType::new(CDT::uint32_datatype(), false),
|
||||
]);
|
||||
let extensions = FunctionExtensions {
|
||||
anchor_to_name: HashMap::from([(0, "add".to_string())]),
|
||||
};
|
||||
let res = TypedExpr::from_substrait_scalar_func(&f, &input_schema, &extensions).unwrap();
|
||||
|
||||
assert_eq!(
|
||||
res,
|
||||
TypedExpr {
|
||||
expr: ScalarExpr::Column(0)
|
||||
.call_binary(ScalarExpr::Column(1), BinaryFunc::AddUInt32,),
|
||||
typ: ColumnType {
|
||||
scalar_type: CDT::uint32_datatype(),
|
||||
nullable: true,
|
||||
},
|
||||
}
|
||||
);
|
||||
|
||||
let f = substrait_proto::proto::expression::ScalarFunction {
|
||||
function_reference: 0,
|
||||
arguments: vec![col(0), lit("1 second"), lit("2021-07-01 00:00:00")],
|
||||
options: vec![],
|
||||
output_type: None,
|
||||
..Default::default()
|
||||
};
|
||||
let input_schema = RelationType::new(vec![
|
||||
ColumnType::new(CDT::timestamp_nanosecond_datatype(), false),
|
||||
ColumnType::new(CDT::string_datatype(), false),
|
||||
]);
|
||||
let extensions = FunctionExtensions {
|
||||
anchor_to_name: HashMap::from([(0, "tumble".to_string())]),
|
||||
};
|
||||
let res = TypedExpr::from_substrait_scalar_func(&f, &input_schema, &extensions).unwrap();
|
||||
|
||||
assert_eq!(
|
||||
res,
|
||||
ScalarExpr::CallUnmaterializable(UnmaterializableFunc::TumbleWindow {
|
||||
ts: Box::new(
|
||||
ScalarExpr::Column(0)
|
||||
.with_type(ColumnType::new(CDT::timestamp_nanosecond_datatype(), false))
|
||||
),
|
||||
window_size: Interval::from_month_day_nano(0, 0, 1_000_000_000),
|
||||
start_time: Some(DateTime::new(1625097600000))
|
||||
})
|
||||
.with_type(ColumnType::new(CDT::timestamp_millisecond_datatype(), true)),
|
||||
);
|
||||
|
||||
let f = substrait_proto::proto::expression::ScalarFunction {
|
||||
function_reference: 0,
|
||||
arguments: vec![col(0), lit("1 second")],
|
||||
options: vec![],
|
||||
output_type: None,
|
||||
..Default::default()
|
||||
};
|
||||
let input_schema = RelationType::new(vec![
|
||||
ColumnType::new(CDT::timestamp_nanosecond_datatype(), false),
|
||||
ColumnType::new(CDT::string_datatype(), false),
|
||||
]);
|
||||
let extensions = FunctionExtensions {
|
||||
anchor_to_name: HashMap::from([(0, "tumble".to_string())]),
|
||||
};
|
||||
let res = TypedExpr::from_substrait_scalar_func(&f, &input_schema, &extensions).unwrap();
|
||||
|
||||
assert_eq!(
|
||||
res,
|
||||
ScalarExpr::CallUnmaterializable(UnmaterializableFunc::TumbleWindow {
|
||||
ts: Box::new(
|
||||
ScalarExpr::Column(0)
|
||||
.with_type(ColumnType::new(CDT::timestamp_nanosecond_datatype(), false))
|
||||
),
|
||||
window_size: Interval::from_month_day_nano(0, 0, 1_000_000_000),
|
||||
start_time: None
|
||||
})
|
||||
.with_type(ColumnType::new(CDT::timestamp_millisecond_datatype(), true)),
|
||||
)
|
||||
}
|
||||
}
|
||||
|
||||
@@ -269,7 +269,7 @@ impl TypedPlan {
|
||||
id: crate::expr::Id::Global(table.0),
|
||||
};
|
||||
let get_table = TypedPlan {
|
||||
typ: table.1,
|
||||
typ: table.1.typ().clone(),
|
||||
plan: get_table,
|
||||
};
|
||||
|
||||
|
||||
@@ -18,6 +18,7 @@ use std::collections::{BTreeMap, BTreeSet};
|
||||
use std::ops::Bound;
|
||||
use std::sync::Arc;
|
||||
|
||||
use common_telemetry::debug;
|
||||
use itertools::Itertools;
|
||||
use serde::{Deserialize, Serialize};
|
||||
use smallvec::{smallvec, SmallVec};
|
||||
@@ -86,7 +87,7 @@ impl KeyExpiryManager {
|
||||
///
|
||||
/// - If given key is expired by now (that is less than `now - expiry_duration`), return the amount of time it's expired.
|
||||
/// - If it's not expired, return None
|
||||
pub fn update_event_ts(
|
||||
pub fn get_expire_duration_and_update_event_ts(
|
||||
&mut self,
|
||||
now: Timestamp,
|
||||
row: &Row,
|
||||
@@ -95,6 +96,33 @@ impl KeyExpiryManager {
|
||||
return Ok(None);
|
||||
};
|
||||
|
||||
self.event_ts_to_key
|
||||
.entry(event_ts)
|
||||
.or_default()
|
||||
.insert(row.clone());
|
||||
|
||||
if let Some(expire_time) = self.compute_expiration_timestamp(now) {
|
||||
if expire_time > event_ts {
|
||||
// return how much time it's expired
|
||||
return Ok(Some(expire_time - event_ts));
|
||||
}
|
||||
}
|
||||
|
||||
Ok(None)
|
||||
}
|
||||
|
||||
/// Get the expire duration of a key, if it's expired by now.
|
||||
///
|
||||
/// Return None if the key is not expired
|
||||
pub fn get_expire_duration(
|
||||
&self,
|
||||
now: Timestamp,
|
||||
row: &Row,
|
||||
) -> Result<Option<Duration>, EvalError> {
|
||||
let Some(event_ts) = self.extract_event_ts(row)? else {
|
||||
return Ok(None);
|
||||
};
|
||||
|
||||
if let Some(expire_time) = self.compute_expiration_timestamp(now) {
|
||||
if expire_time > event_ts {
|
||||
// return how much time it's expired
|
||||
@@ -102,10 +130,6 @@ impl KeyExpiryManager {
|
||||
}
|
||||
}
|
||||
|
||||
self.event_ts_to_key
|
||||
.entry(event_ts)
|
||||
.or_default()
|
||||
.insert(row.clone());
|
||||
Ok(None)
|
||||
}
|
||||
|
||||
@@ -189,6 +213,10 @@ impl Arrangement {
|
||||
}
|
||||
}
|
||||
|
||||
pub fn get_expire_state(&self) -> Option<&KeyExpiryManager> {
|
||||
self.expire_state.as_ref()
|
||||
}
|
||||
|
||||
pub fn set_expire_state(&mut self, expire_state: KeyExpiryManager) {
|
||||
self.expire_state = Some(expire_state);
|
||||
}
|
||||
@@ -208,8 +236,12 @@ impl Arrangement {
|
||||
for ((key, val), update_ts, diff) in updates {
|
||||
// check if the key is expired
|
||||
if let Some(s) = &mut self.expire_state {
|
||||
if let Some(expired_by) = s.update_event_ts(now, &key)? {
|
||||
if let Some(expired_by) = s.get_expire_duration_and_update_event_ts(now, &key)? {
|
||||
max_expired_by = max_expired_by.max(Some(expired_by));
|
||||
debug!(
|
||||
"Expired key: {:?}, expired by: {:?} with time being now={}",
|
||||
key, expired_by, now
|
||||
);
|
||||
continue;
|
||||
}
|
||||
}
|
||||
@@ -335,7 +367,9 @@ impl Arrangement {
|
||||
for (key, updates) in batch {
|
||||
// check if the key is expired
|
||||
if let Some(s) = &mut self.expire_state {
|
||||
if let Some(expired_by) = s.update_event_ts(now, &key)? {
|
||||
if let Some(expired_by) =
|
||||
s.get_expire_duration_and_update_event_ts(now, &key)?
|
||||
{
|
||||
max_expired_by = max_expired_by.max(Some(expired_by));
|
||||
continue;
|
||||
}
|
||||
@@ -540,6 +574,10 @@ impl ArrangeHandler {
|
||||
pub fn set_full_arrangement(&self, full: bool) {
|
||||
self.write().full_arrangement = full;
|
||||
}
|
||||
|
||||
pub fn is_full_arrangement(&self) -> bool {
|
||||
self.read().full_arrangement
|
||||
}
|
||||
}
|
||||
|
||||
#[cfg(test)]
|
||||
|
||||
@@ -68,5 +68,5 @@ datanode.workspace = true
|
||||
futures = "0.3"
|
||||
meta-srv = { workspace = true, features = ["mock"] }
|
||||
strfmt = "0.2"
|
||||
tower = "0.4"
|
||||
tower.workspace = true
|
||||
uuid.workspace = true
|
||||
|
||||
@@ -554,6 +554,7 @@ pub fn check_permission(
|
||||
Statement::ShowIndex(stmt) => {
|
||||
validate_db_permission!(stmt, query_ctx);
|
||||
}
|
||||
Statement::ShowStatus(_stmt) => {}
|
||||
Statement::DescribeTable(stmt) => {
|
||||
validate_param(stmt.name(), query_ctx)?;
|
||||
}
|
||||
|
||||
@@ -21,12 +21,18 @@ use serde_json::error::Error as JsonError;
|
||||
use snafu::{Location, Snafu};
|
||||
use store_api::storage::RegionId;
|
||||
|
||||
use crate::kafka::NamespaceImpl as KafkaNamespace;
|
||||
|
||||
#[derive(Snafu)]
|
||||
#[snafu(visibility(pub))]
|
||||
#[stack_trace_debug]
|
||||
pub enum Error {
|
||||
#[snafu(display("Invalid provider type, expected: {}, actual: {}", expected, actual))]
|
||||
InvalidProvider {
|
||||
#[snafu(implicit)]
|
||||
location: Location,
|
||||
expected: String,
|
||||
actual: String,
|
||||
},
|
||||
|
||||
#[snafu(display("Failed to start log store gc task"))]
|
||||
StartGcTask {
|
||||
#[snafu(implicit)]
|
||||
@@ -170,34 +176,28 @@ pub enum Error {
|
||||
location: Location,
|
||||
},
|
||||
|
||||
#[snafu(display(
|
||||
"Failed to produce records to Kafka, topic: {}, size: {}, limit: {}",
|
||||
topic,
|
||||
size,
|
||||
limit,
|
||||
))]
|
||||
#[snafu(display("Failed to produce records to Kafka, topic: {}, size: {}", topic, size))]
|
||||
ProduceRecord {
|
||||
topic: String,
|
||||
size: usize,
|
||||
limit: usize,
|
||||
#[snafu(implicit)]
|
||||
location: Location,
|
||||
#[snafu(source)]
|
||||
error: rskafka::client::producer::Error,
|
||||
},
|
||||
|
||||
#[snafu(display("Failed to read a record from Kafka, ns: {}", ns))]
|
||||
#[snafu(display("Failed to read a record from Kafka, topic: {}", topic))]
|
||||
ConsumeRecord {
|
||||
ns: KafkaNamespace,
|
||||
topic: String,
|
||||
#[snafu(implicit)]
|
||||
location: Location,
|
||||
#[snafu(source)]
|
||||
error: rskafka::client::error::Error,
|
||||
},
|
||||
|
||||
#[snafu(display("Failed to get the latest offset, ns: {}", ns))]
|
||||
#[snafu(display("Failed to get the latest offset, topic: {}", topic))]
|
||||
GetOffset {
|
||||
ns: KafkaNamespace,
|
||||
topic: String,
|
||||
#[snafu(implicit)]
|
||||
location: Location,
|
||||
#[snafu(source)]
|
||||
|
||||
@@ -12,17 +12,12 @@
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
use std::mem::size_of;
|
||||
pub(crate) mod client_manager;
|
||||
pub mod log_store;
|
||||
pub(crate) mod util;
|
||||
|
||||
use std::fmt::Display;
|
||||
|
||||
use serde::{Deserialize, Serialize};
|
||||
use store_api::logstore::entry::{Entry, Id as EntryId, RawEntry};
|
||||
use store_api::logstore::namespace::Namespace;
|
||||
use store_api::storage::RegionId;
|
||||
use store_api::logstore::entry::Id as EntryId;
|
||||
|
||||
/// Kafka Namespace implementation.
|
||||
#[derive(Debug, PartialEq, Eq, Hash, Clone, Serialize, Deserialize)]
|
||||
@@ -31,18 +26,6 @@ pub struct NamespaceImpl {
|
||||
pub topic: String,
|
||||
}
|
||||
|
||||
impl Namespace for NamespaceImpl {
|
||||
fn id(&self) -> u64 {
|
||||
self.region_id
|
||||
}
|
||||
}
|
||||
|
||||
impl Display for NamespaceImpl {
|
||||
fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
|
||||
write!(f, "[topic: {}, region: {}]", self.topic, self.region_id)
|
||||
}
|
||||
}
|
||||
|
||||
/// Kafka Entry implementation.
|
||||
#[derive(Debug, PartialEq, Clone)]
|
||||
pub struct EntryImpl {
|
||||
@@ -53,65 +36,3 @@ pub struct EntryImpl {
|
||||
/// The namespace used to identify and isolate log entries from different regions.
|
||||
pub ns: NamespaceImpl,
|
||||
}
|
||||
|
||||
impl Entry for EntryImpl {
|
||||
fn into_raw_entry(self) -> RawEntry {
|
||||
RawEntry {
|
||||
region_id: self.region_id(),
|
||||
entry_id: self.id(),
|
||||
data: self.data,
|
||||
}
|
||||
}
|
||||
|
||||
fn data(&self) -> &[u8] {
|
||||
&self.data
|
||||
}
|
||||
|
||||
fn id(&self) -> EntryId {
|
||||
self.id
|
||||
}
|
||||
|
||||
fn region_id(&self) -> RegionId {
|
||||
RegionId::from_u64(self.ns.region_id)
|
||||
}
|
||||
|
||||
fn estimated_size(&self) -> usize {
|
||||
size_of::<Self>() + self.data.capacity() * size_of::<u8>() + self.ns.topic.capacity()
|
||||
}
|
||||
}
|
||||
|
||||
impl Display for EntryImpl {
|
||||
fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
|
||||
write!(
|
||||
f,
|
||||
"Entry [ns: {}, id: {}, data_len: {}]",
|
||||
self.ns,
|
||||
self.id,
|
||||
self.data.len()
|
||||
)
|
||||
}
|
||||
}
|
||||
|
||||
#[cfg(test)]
|
||||
mod tests {
|
||||
use std::mem::size_of;
|
||||
|
||||
use store_api::logstore::entry::Entry;
|
||||
|
||||
use crate::kafka::{EntryImpl, NamespaceImpl};
|
||||
|
||||
#[test]
|
||||
fn test_estimated_size() {
|
||||
let entry = EntryImpl {
|
||||
data: Vec::with_capacity(100),
|
||||
id: 0,
|
||||
ns: NamespaceImpl {
|
||||
region_id: 0,
|
||||
topic: String::with_capacity(10),
|
||||
},
|
||||
};
|
||||
let expected = size_of::<EntryImpl>() + 100 * size_of::<u8>() + 10;
|
||||
let got = entry.estimated_size();
|
||||
assert_eq!(expected, got);
|
||||
}
|
||||
}
|
||||
|
||||
@@ -27,6 +27,7 @@ use tokio::sync::RwLock;
|
||||
use crate::error::{
|
||||
BuildClientSnafu, BuildPartitionClientSnafu, ResolveKafkaEndpointSnafu, Result,
|
||||
};
|
||||
use crate::kafka::util::record::MIN_BATCH_SIZE;
|
||||
|
||||
// Each topic only has one partition for now.
|
||||
// The `DEFAULT_PARTITION` refers to the index of the partition.
|
||||
@@ -48,7 +49,8 @@ pub(crate) struct Client {
|
||||
impl Client {
|
||||
/// Creates a Client from the raw client.
|
||||
pub(crate) fn new(raw_client: Arc<PartitionClient>, config: &DatanodeKafkaConfig) -> Self {
|
||||
let record_aggregator = RecordAggregator::new(config.max_batch_size.as_bytes() as usize);
|
||||
let record_aggregator =
|
||||
RecordAggregator::new((config.max_batch_size.as_bytes() as usize).max(MIN_BATCH_SIZE));
|
||||
let batch_producer = BatchProducerBuilder::new(raw_client.clone())
|
||||
.with_compression(config.compression)
|
||||
.with_linger(config.linger)
|
||||
|
||||
@@ -17,21 +17,23 @@ use std::sync::Arc;
|
||||
|
||||
use common_telemetry::{debug, warn};
|
||||
use common_wal::config::kafka::DatanodeKafkaConfig;
|
||||
use common_wal::options::WalOptions;
|
||||
use futures_util::StreamExt;
|
||||
use rskafka::client::consumer::{StartOffset, StreamConsumerBuilder};
|
||||
use rskafka::client::partition::OffsetAt;
|
||||
use snafu::ResultExt;
|
||||
use store_api::logstore::entry::{Entry as EntryTrait, Id as EntryId};
|
||||
use store_api::logstore::entry_stream::SendableEntryStream;
|
||||
use store_api::logstore::namespace::Id as NamespaceId;
|
||||
use store_api::logstore::{AppendBatchResponse, AppendResponse, LogStore};
|
||||
use snafu::{OptionExt, ResultExt};
|
||||
use store_api::logstore::entry::{
|
||||
Entry, Id as EntryId, MultiplePartEntry, MultiplePartHeader, NaiveEntry,
|
||||
};
|
||||
use store_api::logstore::provider::{KafkaProvider, Provider};
|
||||
use store_api::logstore::{AppendBatchResponse, LogStore, SendableEntryStream};
|
||||
use store_api::storage::RegionId;
|
||||
|
||||
use crate::error::{ConsumeRecordSnafu, Error, GetOffsetSnafu, IllegalSequenceSnafu, Result};
|
||||
use crate::error::{self, ConsumeRecordSnafu, Error, GetOffsetSnafu, InvalidProviderSnafu, Result};
|
||||
use crate::kafka::client_manager::{ClientManager, ClientManagerRef};
|
||||
use crate::kafka::util::offset::Offset;
|
||||
use crate::kafka::util::record::{maybe_emit_entry, Record, RecordProducer};
|
||||
use crate::kafka::{EntryImpl, NamespaceImpl};
|
||||
use crate::kafka::util::record::{
|
||||
maybe_emit_entry, remaining_entries, Record, RecordProducer, ESTIMATED_META_SIZE,
|
||||
};
|
||||
use crate::metrics;
|
||||
|
||||
/// A log store backed by Kafka.
|
||||
@@ -52,41 +54,81 @@ impl KafkaLogStore {
|
||||
}
|
||||
}
|
||||
|
||||
fn build_entry(
|
||||
data: &mut Vec<u8>,
|
||||
entry_id: EntryId,
|
||||
region_id: RegionId,
|
||||
provider: &Provider,
|
||||
max_data_size: usize,
|
||||
) -> Entry {
|
||||
if data.len() <= max_data_size {
|
||||
Entry::Naive(NaiveEntry {
|
||||
provider: provider.clone(),
|
||||
region_id,
|
||||
entry_id,
|
||||
data: std::mem::take(data),
|
||||
})
|
||||
} else {
|
||||
let parts = std::mem::take(data)
|
||||
.chunks(max_data_size)
|
||||
.map(|s| s.into())
|
||||
.collect::<Vec<_>>();
|
||||
let num_parts = parts.len();
|
||||
|
||||
let mut headers = Vec::with_capacity(num_parts);
|
||||
headers.push(MultiplePartHeader::First);
|
||||
headers.extend((1..num_parts - 1).map(MultiplePartHeader::Middle));
|
||||
headers.push(MultiplePartHeader::Last);
|
||||
|
||||
Entry::MultiplePart(MultiplePartEntry {
|
||||
provider: provider.clone(),
|
||||
region_id,
|
||||
entry_id,
|
||||
headers,
|
||||
parts,
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
#[async_trait::async_trait]
|
||||
impl LogStore for KafkaLogStore {
|
||||
type Error = Error;
|
||||
type Entry = EntryImpl;
|
||||
type Namespace = NamespaceImpl;
|
||||
|
||||
/// Creates an entry of the associated Entry type.
|
||||
fn entry(&self, data: &mut Vec<u8>, entry_id: EntryId, ns: Self::Namespace) -> Self::Entry {
|
||||
EntryImpl {
|
||||
data: std::mem::take(data),
|
||||
id: entry_id,
|
||||
ns,
|
||||
}
|
||||
}
|
||||
|
||||
/// Appends an entry to the log store and returns a response containing the entry id of the appended entry.
|
||||
async fn append(&self, entry: Self::Entry) -> Result<AppendResponse> {
|
||||
let entry_id = RecordProducer::new(entry.ns.clone())
|
||||
.with_entries(vec![entry])
|
||||
.produce(&self.client_manager)
|
||||
.await
|
||||
.map(TryInto::try_into)??;
|
||||
Ok(AppendResponse {
|
||||
last_entry_id: entry_id,
|
||||
})
|
||||
/// Creates an [Entry].
|
||||
fn entry(
|
||||
&self,
|
||||
data: &mut Vec<u8>,
|
||||
entry_id: EntryId,
|
||||
region_id: RegionId,
|
||||
provider: &Provider,
|
||||
) -> Result<Entry> {
|
||||
provider
|
||||
.as_kafka_provider()
|
||||
.with_context(|| InvalidProviderSnafu {
|
||||
expected: KafkaProvider::type_name(),
|
||||
actual: provider.type_name(),
|
||||
})?;
|
||||
|
||||
let max_data_size =
|
||||
self.client_manager.config.max_batch_size.as_bytes() as usize - ESTIMATED_META_SIZE;
|
||||
Ok(build_entry(
|
||||
data,
|
||||
entry_id,
|
||||
region_id,
|
||||
provider,
|
||||
max_data_size,
|
||||
))
|
||||
}
|
||||
|
||||
// TODO(weny): refactor the writing.
|
||||
/// Appends a batch of entries and returns a response containing a map where the key is a region id
|
||||
/// while the value is the id of the last successfully written entry of the region.
|
||||
async fn append_batch(&self, entries: Vec<Self::Entry>) -> Result<AppendBatchResponse> {
|
||||
async fn append_batch(&self, entries: Vec<Entry>) -> Result<AppendBatchResponse> {
|
||||
metrics::METRIC_KAFKA_APPEND_BATCH_CALLS_TOTAL.inc();
|
||||
metrics::METRIC_KAFKA_APPEND_BATCH_BYTES_TOTAL.inc_by(
|
||||
entries
|
||||
.iter()
|
||||
.map(EntryTrait::estimated_size)
|
||||
.map(|entry| entry.estimated_size())
|
||||
.sum::<usize>() as u64,
|
||||
);
|
||||
let _timer = metrics::METRIC_KAFKA_APPEND_BATCH_ELAPSED.start_timer();
|
||||
@@ -98,9 +140,17 @@ impl LogStore for KafkaLogStore {
|
||||
// Groups entries by region id and pushes them to an associated record producer.
|
||||
let mut producers = HashMap::with_capacity(entries.len());
|
||||
for entry in entries {
|
||||
let provider = entry
|
||||
.provider()
|
||||
.as_kafka_provider()
|
||||
.context(error::InvalidProviderSnafu {
|
||||
expected: KafkaProvider::type_name(),
|
||||
actual: entry.provider().type_name(),
|
||||
})?
|
||||
.clone();
|
||||
producers
|
||||
.entry(entry.ns.region_id)
|
||||
.or_insert_with(|| RecordProducer::new(entry.ns.clone()))
|
||||
.entry(entry.region_id())
|
||||
.or_insert_with(|| RecordProducer::new(provider))
|
||||
.push(entry);
|
||||
}
|
||||
|
||||
@@ -122,20 +172,27 @@ impl LogStore for KafkaLogStore {
|
||||
Ok(AppendBatchResponse { last_entry_ids })
|
||||
}
|
||||
|
||||
/// Creates a new `EntryStream` to asynchronously generates `Entry` with entry ids
|
||||
/// starting from `entry_id`. The generated entries will be filtered by the namespace.
|
||||
/// Creates a new `EntryStream` to asynchronously generates `Entry` with entry ids.
|
||||
/// Returns entries belonging to `provider`, starting from `entry_id`.
|
||||
async fn read(
|
||||
&self,
|
||||
ns: &Self::Namespace,
|
||||
provider: &Provider,
|
||||
entry_id: EntryId,
|
||||
) -> Result<SendableEntryStream<Self::Entry, Self::Error>> {
|
||||
) -> Result<SendableEntryStream<'static, Entry, Self::Error>> {
|
||||
let provider = provider
|
||||
.as_kafka_provider()
|
||||
.with_context(|| InvalidProviderSnafu {
|
||||
expected: KafkaProvider::type_name(),
|
||||
actual: provider.type_name(),
|
||||
})?;
|
||||
|
||||
metrics::METRIC_KAFKA_READ_CALLS_TOTAL.inc();
|
||||
let _timer = metrics::METRIC_KAFKA_READ_ELAPSED.start_timer();
|
||||
|
||||
// Gets the client associated with the topic.
|
||||
let client = self
|
||||
.client_manager
|
||||
.get_or_insert(&ns.topic)
|
||||
.get_or_insert(&provider.topic)
|
||||
.await?
|
||||
.raw_client
|
||||
.clone();
|
||||
@@ -147,14 +204,16 @@ impl LogStore for KafkaLogStore {
|
||||
let end_offset = client
|
||||
.get_offset(OffsetAt::Latest)
|
||||
.await
|
||||
.context(GetOffsetSnafu { ns: ns.clone() })?
|
||||
.context(GetOffsetSnafu {
|
||||
topic: &provider.topic,
|
||||
})?
|
||||
- 1;
|
||||
// Reads entries with offsets in the range [start_offset, end_offset].
|
||||
let start_offset = Offset::try_from(entry_id)?.0;
|
||||
|
||||
debug!(
|
||||
"Start reading entries in range [{}, {}] for ns {}",
|
||||
start_offset, end_offset, ns
|
||||
start_offset, end_offset, provider
|
||||
);
|
||||
|
||||
// Abort if there're no new entries.
|
||||
@@ -162,7 +221,7 @@ impl LogStore for KafkaLogStore {
|
||||
if start_offset > end_offset {
|
||||
warn!(
|
||||
"No new entries for ns {} in range [{}, {}]",
|
||||
ns, start_offset, end_offset
|
||||
provider, start_offset, end_offset
|
||||
);
|
||||
return Ok(futures_util::stream::empty().boxed());
|
||||
}
|
||||
@@ -174,20 +233,20 @@ impl LogStore for KafkaLogStore {
|
||||
|
||||
debug!(
|
||||
"Built a stream consumer for ns {} to consume entries in range [{}, {}]",
|
||||
ns, start_offset, end_offset
|
||||
provider, start_offset, end_offset
|
||||
);
|
||||
|
||||
// Key: entry id, Value: the records associated with the entry.
|
||||
let mut entry_records: HashMap<_, Vec<_>> = HashMap::new();
|
||||
let ns_clone = ns.clone();
|
||||
// A buffer is used to collect records to construct a complete entry.
|
||||
let mut entry_records: HashMap<RegionId, Vec<Record>> = HashMap::new();
|
||||
let provider = provider.clone();
|
||||
let stream = async_stream::stream!({
|
||||
while let Some(consume_result) = stream_consumer.next().await {
|
||||
// Each next on the stream consumer produces a `RecordAndOffset` and a high watermark offset.
|
||||
// The `RecordAndOffset` contains the record data and its start offset.
|
||||
// The high watermark offset is the offset of the last record plus one.
|
||||
let (record_and_offset, high_watermark) =
|
||||
consume_result.with_context(|_| ConsumeRecordSnafu {
|
||||
ns: ns_clone.clone(),
|
||||
consume_result.context(ConsumeRecordSnafu {
|
||||
topic: &provider.topic,
|
||||
})?;
|
||||
let (kafka_record, offset) = (record_and_offset.record, record_and_offset.offset);
|
||||
|
||||
@@ -195,37 +254,35 @@ impl LogStore for KafkaLogStore {
|
||||
.inc_by(kafka_record.approximate_size() as u64);
|
||||
|
||||
debug!(
|
||||
"Read a record at offset {} for ns {}, high watermark: {}",
|
||||
offset, ns_clone, high_watermark
|
||||
"Read a record at offset {} for topic {}, high watermark: {}",
|
||||
offset, provider.topic, high_watermark
|
||||
);
|
||||
|
||||
// Ignores no-op records.
|
||||
if kafka_record.value.is_none() {
|
||||
if check_termination(offset, end_offset, &entry_records)? {
|
||||
if check_termination(offset, end_offset) {
|
||||
if let Some(entries) = remaining_entries(&provider, &mut entry_records) {
|
||||
yield Ok(entries);
|
||||
}
|
||||
break;
|
||||
}
|
||||
continue;
|
||||
}
|
||||
|
||||
// Filters records by namespace.
|
||||
let record = Record::try_from(kafka_record)?;
|
||||
if record.meta.ns != ns_clone {
|
||||
if check_termination(offset, end_offset, &entry_records)? {
|
||||
break;
|
||||
}
|
||||
continue;
|
||||
}
|
||||
|
||||
// Tries to construct an entry from records consumed so far.
|
||||
if let Some(mut entry) = maybe_emit_entry(record, &mut entry_records)? {
|
||||
if let Some(mut entry) = maybe_emit_entry(&provider, record, &mut entry_records)? {
|
||||
// We don't rely on the EntryId generated by mito2.
|
||||
// Instead, we use the offset return from Kafka as EntryId.
|
||||
// Therefore, we MUST overwrite the EntryId with RecordOffset.
|
||||
entry.id = offset as u64;
|
||||
entry.set_entry_id(offset as u64);
|
||||
yield Ok(vec![entry]);
|
||||
}
|
||||
|
||||
if check_termination(offset, end_offset, &entry_records)? {
|
||||
if check_termination(offset, end_offset) {
|
||||
if let Some(entries) = remaining_entries(&provider, &mut entry_records) {
|
||||
yield Ok(entries);
|
||||
}
|
||||
break;
|
||||
}
|
||||
}
|
||||
@@ -233,39 +290,25 @@ impl LogStore for KafkaLogStore {
|
||||
Ok(Box::pin(stream))
|
||||
}
|
||||
|
||||
/// Creates a namespace of the associated Namespace type.
|
||||
fn namespace(&self, ns_id: NamespaceId, wal_options: &WalOptions) -> Self::Namespace {
|
||||
// Safety: upon start, the datanode checks the consistency of the wal providers in the wal config of the
|
||||
// datanode and that of the metasrv. Therefore, the wal options passed into the kafka log store
|
||||
// must be of type WalOptions::Kafka.
|
||||
let WalOptions::Kafka(kafka_options) = wal_options else {
|
||||
unreachable!()
|
||||
};
|
||||
NamespaceImpl {
|
||||
region_id: ns_id,
|
||||
topic: kafka_options.topic.clone(),
|
||||
}
|
||||
}
|
||||
|
||||
/// Creates a new `Namespace` from the given ref.
|
||||
async fn create_namespace(&self, _ns: &Self::Namespace) -> Result<()> {
|
||||
async fn create_namespace(&self, _provider: &Provider) -> Result<()> {
|
||||
Ok(())
|
||||
}
|
||||
|
||||
/// Deletes an existing `Namespace` specified by the given ref.
|
||||
async fn delete_namespace(&self, _ns: &Self::Namespace) -> Result<()> {
|
||||
async fn delete_namespace(&self, _provider: &Provider) -> Result<()> {
|
||||
Ok(())
|
||||
}
|
||||
|
||||
/// Lists all existing namespaces.
|
||||
async fn list_namespaces(&self) -> Result<Vec<Self::Namespace>> {
|
||||
async fn list_namespaces(&self) -> Result<Vec<Provider>> {
|
||||
Ok(vec![])
|
||||
}
|
||||
|
||||
/// Marks all entries with ids `<=entry_id` of the given `namespace` as obsolete,
|
||||
/// so that the log store can safely delete those entries. This method does not guarantee
|
||||
/// that the obsolete entries are deleted immediately.
|
||||
async fn obsolete(&self, _ns: Self::Namespace, _entry_id: EntryId) -> Result<()> {
|
||||
async fn obsolete(&self, _provider: &Provider, _entry_id: EntryId) -> Result<()> {
|
||||
Ok(())
|
||||
}
|
||||
|
||||
@@ -275,227 +318,249 @@ impl LogStore for KafkaLogStore {
|
||||
}
|
||||
}
|
||||
|
||||
fn check_termination(
|
||||
offset: i64,
|
||||
end_offset: i64,
|
||||
entry_records: &HashMap<EntryId, Vec<Record>>,
|
||||
) -> Result<bool> {
|
||||
fn check_termination(offset: i64, end_offset: i64) -> bool {
|
||||
// Terminates the stream if the entry with the end offset was read.
|
||||
if offset >= end_offset {
|
||||
debug!("Stream consumer terminates at offset {}", offset);
|
||||
// There must have no records when the stream terminates.
|
||||
if !entry_records.is_empty() {
|
||||
return IllegalSequenceSnafu {
|
||||
error: "Found records leftover",
|
||||
}
|
||||
.fail();
|
||||
}
|
||||
Ok(true)
|
||||
true
|
||||
} else {
|
||||
Ok(false)
|
||||
false
|
||||
}
|
||||
}
|
||||
|
||||
#[cfg(test)]
|
||||
mod tests {
|
||||
|
||||
use std::assert_matches::assert_matches;
|
||||
use std::collections::HashMap;
|
||||
|
||||
use common_base::readable_size::ReadableSize;
|
||||
use rand::seq::IteratorRandom;
|
||||
use common_telemetry::info;
|
||||
use common_telemetry::tracing::warn;
|
||||
use common_wal::config::kafka::DatanodeKafkaConfig;
|
||||
use futures::TryStreamExt;
|
||||
use rand::prelude::SliceRandom;
|
||||
use rand::Rng;
|
||||
use store_api::logstore::entry::{Entry, MultiplePartEntry, MultiplePartHeader, NaiveEntry};
|
||||
use store_api::logstore::provider::Provider;
|
||||
use store_api::logstore::LogStore;
|
||||
use store_api::storage::RegionId;
|
||||
|
||||
use super::*;
|
||||
use crate::test_util::kafka::{
|
||||
create_topics, entries_with_random_data, new_namespace, EntryBuilder,
|
||||
};
|
||||
use super::build_entry;
|
||||
use crate::kafka::log_store::KafkaLogStore;
|
||||
|
||||
// Stores test context for a region.
|
||||
struct RegionContext {
|
||||
ns: NamespaceImpl,
|
||||
entry_builder: EntryBuilder,
|
||||
expected: Vec<EntryImpl>,
|
||||
flushed_entry_id: EntryId,
|
||||
}
|
||||
#[test]
|
||||
fn test_build_naive_entry() {
|
||||
let provider = Provider::kafka_provider("my_topic".to_string());
|
||||
let region_id = RegionId::new(1, 1);
|
||||
let entry = build_entry(&mut vec![1; 100], 1, region_id, &provider, 120);
|
||||
|
||||
/// Prepares for a test in that a log store is constructed and a collection of topics is created.
|
||||
async fn prepare(
|
||||
test_name: &str,
|
||||
num_topics: usize,
|
||||
broker_endpoints: Vec<String>,
|
||||
) -> (KafkaLogStore, Vec<String>) {
|
||||
let topics = create_topics(
|
||||
num_topics,
|
||||
|i| format!("{test_name}_{}_{}", i, uuid::Uuid::new_v4()),
|
||||
&broker_endpoints,
|
||||
assert_eq!(
|
||||
entry.into_naive_entry().unwrap(),
|
||||
NaiveEntry {
|
||||
provider,
|
||||
region_id,
|
||||
entry_id: 1,
|
||||
data: vec![1; 100]
|
||||
}
|
||||
)
|
||||
.await;
|
||||
|
||||
let config = DatanodeKafkaConfig {
|
||||
broker_endpoints,
|
||||
max_batch_size: ReadableSize::kb(32),
|
||||
..Default::default()
|
||||
};
|
||||
let logstore = KafkaLogStore::try_new(&config).await.unwrap();
|
||||
|
||||
// Appends a no-op record to each topic.
|
||||
for topic in topics.iter() {
|
||||
let last_entry_id = logstore
|
||||
.append(EntryImpl {
|
||||
data: vec![],
|
||||
id: 0,
|
||||
ns: new_namespace(topic, 0),
|
||||
})
|
||||
.await
|
||||
.unwrap()
|
||||
.last_entry_id;
|
||||
assert_eq!(last_entry_id, 0);
|
||||
}
|
||||
|
||||
(logstore, topics)
|
||||
}
|
||||
|
||||
/// Creates a vector containing indexes of all regions if the `all` is true.
|
||||
/// Otherwise, creates a subset of the indexes. The cardinality of the subset
|
||||
/// is nearly a quarter of that of the universe set.
|
||||
fn all_or_subset(all: bool, num_regions: usize) -> Vec<u64> {
|
||||
assert!(num_regions > 0);
|
||||
let amount = if all {
|
||||
num_regions
|
||||
} else {
|
||||
(num_regions / 4).max(1)
|
||||
};
|
||||
(0..num_regions as u64).choose_multiple(&mut rand::thread_rng(), amount)
|
||||
#[test]
|
||||
fn test_build_into_multiple_part_entry() {
|
||||
let provider = Provider::kafka_provider("my_topic".to_string());
|
||||
let region_id = RegionId::new(1, 1);
|
||||
let entry = build_entry(&mut vec![1; 100], 1, region_id, &provider, 50);
|
||||
|
||||
assert_eq!(
|
||||
entry.into_multiple_part_entry().unwrap(),
|
||||
MultiplePartEntry {
|
||||
provider: provider.clone(),
|
||||
region_id,
|
||||
entry_id: 1,
|
||||
headers: vec![MultiplePartHeader::First, MultiplePartHeader::Last],
|
||||
parts: vec![vec![1; 50], vec![1; 50]],
|
||||
}
|
||||
);
|
||||
|
||||
let region_id = RegionId::new(1, 1);
|
||||
let entry = build_entry(&mut vec![1; 100], 1, region_id, &provider, 21);
|
||||
|
||||
assert_eq!(
|
||||
entry.into_multiple_part_entry().unwrap(),
|
||||
MultiplePartEntry {
|
||||
provider,
|
||||
region_id,
|
||||
entry_id: 1,
|
||||
headers: vec![
|
||||
MultiplePartHeader::First,
|
||||
MultiplePartHeader::Middle(1),
|
||||
MultiplePartHeader::Middle(2),
|
||||
MultiplePartHeader::Middle(3),
|
||||
MultiplePartHeader::Last
|
||||
],
|
||||
parts: vec![
|
||||
vec![1; 21],
|
||||
vec![1; 21],
|
||||
vec![1; 21],
|
||||
vec![1; 21],
|
||||
vec![1; 16]
|
||||
],
|
||||
}
|
||||
)
|
||||
}
|
||||
|
||||
/// Builds entries for regions specified by `which`. Builds large entries if `large` is true.
|
||||
/// Returns the aggregated entries.
|
||||
fn build_entries(
|
||||
region_contexts: &mut HashMap<u64, RegionContext>,
|
||||
which: &[u64],
|
||||
large: bool,
|
||||
) -> Vec<EntryImpl> {
|
||||
let mut aggregated = Vec::with_capacity(which.len());
|
||||
for region_id in which {
|
||||
let ctx = region_contexts.get_mut(region_id).unwrap();
|
||||
// Builds entries for the region.
|
||||
ctx.expected = if !large {
|
||||
entries_with_random_data(3, &ctx.entry_builder)
|
||||
} else {
|
||||
// Builds a large entry of size 256KB which is way greater than the configured `max_batch_size` which is 32KB.
|
||||
let large_entry = ctx.entry_builder.with_data([b'1'; 256 * 1024]);
|
||||
vec![large_entry]
|
||||
};
|
||||
// Aggregates entries of all regions.
|
||||
aggregated.push(ctx.expected.clone());
|
||||
}
|
||||
aggregated.into_iter().flatten().collect()
|
||||
fn generate_entries(
|
||||
logstore: &KafkaLogStore,
|
||||
provider: &Provider,
|
||||
num_entries: usize,
|
||||
region_id: RegionId,
|
||||
data_len: usize,
|
||||
) -> Vec<Entry> {
|
||||
(0..num_entries)
|
||||
.map(|_| {
|
||||
let mut data: Vec<u8> = (0..data_len).map(|_| rand::random::<u8>()).collect();
|
||||
// Always set `entry_id` to 0, the real entry_id will be set during the read.
|
||||
logstore.entry(&mut data, 0, region_id, provider).unwrap()
|
||||
})
|
||||
.collect()
|
||||
}
|
||||
|
||||
/// Starts a test with:
|
||||
/// * `test_name` - The name of the test.
|
||||
/// * `num_topics` - Number of topics to be created in the preparation phase.
|
||||
/// * `num_regions` - Number of regions involved in the test.
|
||||
/// * `num_appends` - Number of append operations to be performed.
|
||||
/// * `all` - All regions will be involved in an append operation if `all` is true. Otherwise,
|
||||
/// an append operation will only randomly choose a subset of regions.
|
||||
/// * `large` - Builds large entries for each region is `large` is true.
|
||||
async fn test_with(
|
||||
test_name: &str,
|
||||
num_topics: usize,
|
||||
num_regions: usize,
|
||||
num_appends: usize,
|
||||
all: bool,
|
||||
large: bool,
|
||||
) {
|
||||
#[tokio::test]
|
||||
async fn test_append_batch_basic() {
|
||||
common_telemetry::init_default_ut_logging();
|
||||
let Ok(broker_endpoints) = std::env::var("GT_KAFKA_ENDPOINTS") else {
|
||||
warn!("The endpoints is empty, skipping the test {test_name}");
|
||||
warn!("The endpoints is empty, skipping the test 'test_append_batch_basic'");
|
||||
return;
|
||||
};
|
||||
let broker_endpoints = broker_endpoints
|
||||
.split(',')
|
||||
.map(|s| s.trim().to_string())
|
||||
.collect::<Vec<_>>();
|
||||
|
||||
let (logstore, topics) = prepare(test_name, num_topics, broker_endpoints).await;
|
||||
let mut region_contexts = (0..num_regions)
|
||||
let config = DatanodeKafkaConfig {
|
||||
broker_endpoints,
|
||||
max_batch_size: ReadableSize::kb(32),
|
||||
..Default::default()
|
||||
};
|
||||
let logstore = KafkaLogStore::try_new(&config).await.unwrap();
|
||||
let topic_name = uuid::Uuid::new_v4().to_string();
|
||||
let provider = Provider::kafka_provider(topic_name);
|
||||
let region_entries = (0..5)
|
||||
.map(|i| {
|
||||
let topic = &topics[i % topics.len()];
|
||||
let ns = new_namespace(topic, i as u64);
|
||||
let entry_builder = EntryBuilder::new(ns.clone());
|
||||
let region_id = RegionId::new(1, i);
|
||||
(
|
||||
i as u64,
|
||||
RegionContext {
|
||||
ns,
|
||||
entry_builder,
|
||||
expected: Vec::new(),
|
||||
flushed_entry_id: 0,
|
||||
},
|
||||
region_id,
|
||||
generate_entries(&logstore, &provider, 20, region_id, 1024),
|
||||
)
|
||||
})
|
||||
.collect();
|
||||
.collect::<HashMap<RegionId, Vec<_>>>();
|
||||
|
||||
for _ in 0..num_appends {
|
||||
// Appends entries for a subset of regions.
|
||||
let which = all_or_subset(all, num_regions);
|
||||
let entries = build_entries(&mut region_contexts, &which, large);
|
||||
let last_entry_ids = logstore.append_batch(entries).await.unwrap().last_entry_ids;
|
||||
let mut all_entries = region_entries
|
||||
.values()
|
||||
.flatten()
|
||||
.cloned()
|
||||
.collect::<Vec<_>>();
|
||||
all_entries.shuffle(&mut rand::thread_rng());
|
||||
|
||||
// Reads entries for regions and checks for each region that the gotten entries are identical with the expected ones.
|
||||
for region_id in which {
|
||||
let ctx = region_contexts.get_mut(®ion_id).unwrap();
|
||||
let stream = logstore
|
||||
.read(&ctx.ns, ctx.flushed_entry_id + 1)
|
||||
.await
|
||||
.unwrap();
|
||||
let mut got = stream
|
||||
.collect::<Vec<_>>()
|
||||
.await
|
||||
.into_iter()
|
||||
.flat_map(|x| x.unwrap())
|
||||
.collect::<Vec<_>>();
|
||||
//FIXME(weny): https://github.com/GreptimeTeam/greptimedb/issues/3152
|
||||
ctx.expected.iter_mut().for_each(|entry| entry.id = 0);
|
||||
got.iter_mut().for_each(|entry| entry.id = 0);
|
||||
assert_eq!(ctx.expected, got);
|
||||
}
|
||||
|
||||
// Simulates a flush for regions.
|
||||
for (region_id, last_entry_id) in last_entry_ids {
|
||||
let ctx = region_contexts.get_mut(®ion_id).unwrap();
|
||||
ctx.flushed_entry_id = last_entry_id;
|
||||
}
|
||||
let response = logstore.append_batch(all_entries.clone()).await.unwrap();
|
||||
// 5 region
|
||||
assert_eq!(response.last_entry_ids.len(), 5);
|
||||
let got_entries = logstore
|
||||
.read(&provider, 0)
|
||||
.await
|
||||
.unwrap()
|
||||
.try_collect::<Vec<_>>()
|
||||
.await
|
||||
.unwrap()
|
||||
.into_iter()
|
||||
.flatten()
|
||||
.collect::<Vec<_>>();
|
||||
for (region_id, _) in region_entries {
|
||||
let expected_entries = all_entries
|
||||
.iter()
|
||||
.filter(|entry| entry.region_id() == region_id)
|
||||
.cloned()
|
||||
.collect::<Vec<_>>();
|
||||
let mut actual_entries = got_entries
|
||||
.iter()
|
||||
.filter(|entry| entry.region_id() == region_id)
|
||||
.cloned()
|
||||
.collect::<Vec<_>>();
|
||||
actual_entries
|
||||
.iter_mut()
|
||||
.for_each(|entry| entry.set_entry_id(0));
|
||||
assert_eq!(expected_entries, actual_entries);
|
||||
}
|
||||
}
|
||||
|
||||
/// Appends entries for one region and checks all entries can be read successfully.
|
||||
#[tokio::test]
|
||||
async fn test_one_region() {
|
||||
test_with("test_one_region", 1, 1, 1, true, false).await;
|
||||
}
|
||||
async fn test_append_batch_basic_large() {
|
||||
common_telemetry::init_default_ut_logging();
|
||||
let Ok(broker_endpoints) = std::env::var("GT_KAFKA_ENDPOINTS") else {
|
||||
warn!("The endpoints is empty, skipping the test 'test_append_batch_basic_large'");
|
||||
return;
|
||||
};
|
||||
let data_size_kb = rand::thread_rng().gen_range(9..31usize);
|
||||
info!("Entry size: {}Ki", data_size_kb);
|
||||
let broker_endpoints = broker_endpoints
|
||||
.split(',')
|
||||
.map(|s| s.trim().to_string())
|
||||
.collect::<Vec<_>>();
|
||||
let config = DatanodeKafkaConfig {
|
||||
broker_endpoints,
|
||||
max_batch_size: ReadableSize::kb(8),
|
||||
..Default::default()
|
||||
};
|
||||
let logstore = KafkaLogStore::try_new(&config).await.unwrap();
|
||||
let topic_name = uuid::Uuid::new_v4().to_string();
|
||||
let provider = Provider::kafka_provider(topic_name);
|
||||
let region_entries = (0..5)
|
||||
.map(|i| {
|
||||
let region_id = RegionId::new(1, i);
|
||||
(
|
||||
region_id,
|
||||
generate_entries(&logstore, &provider, 20, region_id, data_size_kb * 1024),
|
||||
)
|
||||
})
|
||||
.collect::<HashMap<RegionId, Vec<_>>>();
|
||||
|
||||
/// Appends entries for multiple regions and checks entries for each region can be read successfully.
|
||||
/// A topic is assigned only a single region.
|
||||
#[tokio::test]
|
||||
async fn test_multi_regions_disjoint() {
|
||||
test_with("test_multi_regions_disjoint", 5, 5, 1, true, false).await;
|
||||
}
|
||||
let mut all_entries = region_entries
|
||||
.values()
|
||||
.flatten()
|
||||
.cloned()
|
||||
.collect::<Vec<_>>();
|
||||
assert_matches!(all_entries[0], Entry::MultiplePart(_));
|
||||
all_entries.shuffle(&mut rand::thread_rng());
|
||||
|
||||
/// Appends entries for multiple regions and checks entries for each region can be read successfully.
|
||||
/// A topic is assigned multiple regions.
|
||||
#[tokio::test]
|
||||
async fn test_multi_regions_overlapped() {
|
||||
test_with("test_multi_regions_overlapped", 5, 20, 1, true, false).await;
|
||||
}
|
||||
|
||||
/// Appends entries for multiple regions and checks entries for each region can be read successfully.
|
||||
/// A topic may be assigned multiple regions. The append operation repeats for a several iterations.
|
||||
/// Each append operation will only append entries for a subset of randomly chosen regions.
|
||||
#[tokio::test]
|
||||
async fn test_multi_appends() {
|
||||
test_with("test_multi_appends", 5, 20, 3, false, false).await;
|
||||
}
|
||||
|
||||
/// Appends large entries for multiple regions and checks entries for each region can be read successfully.
|
||||
/// A topic may be assigned multiple regions.
|
||||
#[tokio::test]
|
||||
async fn test_append_large_entries() {
|
||||
test_with("test_append_large_entries", 5, 20, 3, true, true).await;
|
||||
let response = logstore.append_batch(all_entries.clone()).await.unwrap();
|
||||
// 5 region
|
||||
assert_eq!(response.last_entry_ids.len(), 5);
|
||||
let got_entries = logstore
|
||||
.read(&provider, 0)
|
||||
.await
|
||||
.unwrap()
|
||||
.try_collect::<Vec<_>>()
|
||||
.await
|
||||
.unwrap()
|
||||
.into_iter()
|
||||
.flatten()
|
||||
.collect::<Vec<_>>();
|
||||
for (region_id, _) in region_entries {
|
||||
let expected_entries = all_entries
|
||||
.iter()
|
||||
.filter(|entry| entry.region_id() == region_id)
|
||||
.cloned()
|
||||
.collect::<Vec<_>>();
|
||||
let mut actual_entries = got_entries
|
||||
.iter()
|
||||
.filter(|entry| entry.region_id() == region_id)
|
||||
.cloned()
|
||||
.collect::<Vec<_>>();
|
||||
actual_entries
|
||||
.iter_mut()
|
||||
.for_each(|entry| entry.set_entry_id(0));
|
||||
assert_eq!(expected_entries, actual_entries);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@@ -13,10 +13,14 @@
|
||||
// limitations under the License.
|
||||
|
||||
use std::collections::HashMap;
|
||||
use std::sync::Arc;
|
||||
|
||||
use rskafka::record::Record as KafkaRecord;
|
||||
use serde::{Deserialize, Serialize};
|
||||
use snafu::{ensure, OptionExt, ResultExt};
|
||||
use store_api::logstore::entry::{Entry, MultiplePartEntry, MultiplePartHeader, NaiveEntry};
|
||||
use store_api::logstore::provider::{KafkaProvider, Provider};
|
||||
use store_api::storage::RegionId;
|
||||
|
||||
use crate::error::{
|
||||
DecodeJsonSnafu, EmptyEntriesSnafu, EncodeJsonSnafu, GetClientSnafu, IllegalSequenceSnafu,
|
||||
@@ -24,7 +28,7 @@ use crate::error::{
|
||||
};
|
||||
use crate::kafka::client_manager::ClientManagerRef;
|
||||
use crate::kafka::util::offset::Offset;
|
||||
use crate::kafka::{EntryId, EntryImpl, NamespaceImpl};
|
||||
use crate::kafka::{EntryId, NamespaceImpl};
|
||||
use crate::metrics;
|
||||
|
||||
/// The current version of Record.
|
||||
@@ -32,7 +36,10 @@ pub(crate) const VERSION: u32 = 0;
|
||||
|
||||
/// The estimated size in bytes of a serialized RecordMeta.
|
||||
/// A record is guaranteed to have sizeof(meta) + sizeof(data) <= max_batch_size - ESTIMATED_META_SIZE.
|
||||
const ESTIMATED_META_SIZE: usize = 256;
|
||||
pub(crate) const ESTIMATED_META_SIZE: usize = 256;
|
||||
|
||||
/// The minimum batch size
|
||||
pub(crate) const MIN_BATCH_SIZE: usize = 4 * 1024;
|
||||
|
||||
/// The type of a record.
|
||||
///
|
||||
@@ -110,43 +117,25 @@ impl TryFrom<KafkaRecord> for Record {
|
||||
}
|
||||
}
|
||||
|
||||
impl From<Vec<Record>> for EntryImpl {
|
||||
fn from(records: Vec<Record>) -> Self {
|
||||
let entry_id = records[0].meta.entry_id;
|
||||
let ns = records[0].meta.ns.clone();
|
||||
let data = records.into_iter().flat_map(|record| record.data).collect();
|
||||
EntryImpl {
|
||||
data,
|
||||
id: entry_id,
|
||||
ns,
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/// Produces a record to a kafka topic.
|
||||
pub(crate) struct RecordProducer {
|
||||
/// The namespace of the entries.
|
||||
ns: NamespaceImpl,
|
||||
/// The provide of the entries.
|
||||
provider: Arc<KafkaProvider>,
|
||||
/// Entries are buffered before being built into a record.
|
||||
entries: Vec<EntryImpl>,
|
||||
entries: Vec<Entry>,
|
||||
}
|
||||
|
||||
impl RecordProducer {
|
||||
/// Creates a new producer for producing entries with the given namespace.
|
||||
pub(crate) fn new(ns: NamespaceImpl) -> Self {
|
||||
pub(crate) fn new(provider: Arc<KafkaProvider>) -> Self {
|
||||
Self {
|
||||
ns,
|
||||
provider,
|
||||
entries: Vec::new(),
|
||||
}
|
||||
}
|
||||
|
||||
/// Populates the entry buffer with the given entries.
|
||||
pub(crate) fn with_entries(self, entries: Vec<EntryImpl>) -> Self {
|
||||
Self { entries, ..self }
|
||||
}
|
||||
|
||||
/// Pushes an entry into the entry buffer.
|
||||
pub(crate) fn push(&mut self, entry: EntryImpl) {
|
||||
pub(crate) fn push(&mut self, entry: Entry) {
|
||||
self.entries.push(entry);
|
||||
}
|
||||
|
||||
@@ -158,11 +147,11 @@ impl RecordProducer {
|
||||
|
||||
// Gets the producer in which a record buffer is maintained.
|
||||
let producer = client_manager
|
||||
.get_or_insert(&self.ns.topic)
|
||||
.get_or_insert(&self.provider.topic)
|
||||
.await
|
||||
.map_err(|e| {
|
||||
GetClientSnafu {
|
||||
topic: &self.ns.topic,
|
||||
topic: &self.provider.topic,
|
||||
error: e.to_string(),
|
||||
}
|
||||
.build()
|
||||
@@ -171,10 +160,8 @@ impl RecordProducer {
|
||||
|
||||
// Stores the offset of the last successfully produced record.
|
||||
let mut last_offset = None;
|
||||
let max_record_size =
|
||||
client_manager.config.max_batch_size.as_bytes() as usize - ESTIMATED_META_SIZE;
|
||||
for entry in self.entries {
|
||||
for record in build_records(entry, max_record_size) {
|
||||
for record in convert_to_records(entry) {
|
||||
let kafka_record = KafkaRecord::try_from(record)?;
|
||||
|
||||
metrics::METRIC_KAFKA_PRODUCE_RECORD_COUNTS.inc();
|
||||
@@ -187,9 +174,8 @@ impl RecordProducer {
|
||||
.await
|
||||
.map(Offset)
|
||||
.with_context(|_| ProduceRecordSnafu {
|
||||
topic: &self.ns.topic,
|
||||
topic: &self.provider.topic,
|
||||
size: kafka_record.approximate_size(),
|
||||
limit: max_record_size,
|
||||
})?;
|
||||
last_offset = Some(offset);
|
||||
}
|
||||
@@ -199,100 +185,188 @@ impl RecordProducer {
|
||||
}
|
||||
}
|
||||
|
||||
fn record_type(seq: usize, num_records: usize) -> RecordType {
|
||||
if seq == 0 {
|
||||
RecordType::First
|
||||
} else if seq == num_records - 1 {
|
||||
RecordType::Last
|
||||
} else {
|
||||
RecordType::Middle(seq)
|
||||
}
|
||||
}
|
||||
|
||||
fn build_records(entry: EntryImpl, max_record_size: usize) -> Vec<Record> {
|
||||
if entry.data.len() <= max_record_size {
|
||||
let record = Record {
|
||||
fn convert_to_records(entry: Entry) -> Vec<Record> {
|
||||
match entry {
|
||||
Entry::Naive(entry) => vec![Record {
|
||||
meta: RecordMeta {
|
||||
version: VERSION,
|
||||
tp: RecordType::Full,
|
||||
entry_id: entry.id,
|
||||
ns: entry.ns,
|
||||
// TODO(weny): refactor the record meta.
|
||||
entry_id: 0,
|
||||
ns: NamespaceImpl {
|
||||
region_id: entry.region_id.as_u64(),
|
||||
// TODO(weny): refactor the record meta.
|
||||
topic: String::new(),
|
||||
},
|
||||
},
|
||||
data: entry.data,
|
||||
};
|
||||
return vec![record];
|
||||
}
|
||||
}],
|
||||
Entry::MultiplePart(entry) => {
|
||||
let mut entries = Vec::with_capacity(entry.parts.len());
|
||||
|
||||
let chunks = entry.data.chunks(max_record_size);
|
||||
let num_chunks = chunks.len();
|
||||
chunks
|
||||
.enumerate()
|
||||
.map(|(i, chunk)| Record {
|
||||
meta: RecordMeta {
|
||||
version: VERSION,
|
||||
tp: record_type(i, num_chunks),
|
||||
entry_id: entry.id,
|
||||
ns: entry.ns.clone(),
|
||||
},
|
||||
data: chunk.to_vec(),
|
||||
})
|
||||
.collect()
|
||||
for (idx, part) in entry.parts.into_iter().enumerate() {
|
||||
let tp = match entry.headers[idx] {
|
||||
MultiplePartHeader::First => RecordType::First,
|
||||
MultiplePartHeader::Middle(i) => RecordType::Middle(i),
|
||||
MultiplePartHeader::Last => RecordType::Last,
|
||||
};
|
||||
entries.push(Record {
|
||||
meta: RecordMeta {
|
||||
version: VERSION,
|
||||
tp,
|
||||
// TODO(weny): refactor the record meta.
|
||||
entry_id: 0,
|
||||
ns: NamespaceImpl {
|
||||
region_id: entry.region_id.as_u64(),
|
||||
topic: String::new(),
|
||||
},
|
||||
},
|
||||
data: part,
|
||||
})
|
||||
}
|
||||
entries
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
pub fn maybe_emit_entry(
|
||||
fn convert_to_naive_entry(provider: Arc<KafkaProvider>, record: Record) -> Entry {
|
||||
let region_id = RegionId::from_u64(record.meta.ns.region_id);
|
||||
|
||||
Entry::Naive(NaiveEntry {
|
||||
provider: Provider::Kafka(provider),
|
||||
region_id,
|
||||
// TODO(weny): should be the offset in the topic
|
||||
entry_id: record.meta.entry_id,
|
||||
data: record.data,
|
||||
})
|
||||
}
|
||||
|
||||
fn convert_to_multiple_entry(
|
||||
provider: Arc<KafkaProvider>,
|
||||
region_id: RegionId,
|
||||
records: Vec<Record>,
|
||||
) -> Entry {
|
||||
let mut headers = Vec::with_capacity(records.len());
|
||||
let mut parts = Vec::with_capacity(records.len());
|
||||
|
||||
for record in records {
|
||||
let header = match record.meta.tp {
|
||||
RecordType::Full => unreachable!(),
|
||||
RecordType::First => MultiplePartHeader::First,
|
||||
RecordType::Middle(i) => MultiplePartHeader::Middle(i),
|
||||
RecordType::Last => MultiplePartHeader::Last,
|
||||
};
|
||||
headers.push(header);
|
||||
parts.push(record.data);
|
||||
}
|
||||
|
||||
Entry::MultiplePart(MultiplePartEntry {
|
||||
provider: Provider::Kafka(provider),
|
||||
region_id,
|
||||
// TODO(weny): should be the offset in the topic
|
||||
entry_id: 0,
|
||||
headers,
|
||||
parts,
|
||||
})
|
||||
}
|
||||
|
||||
/// Constructs entries from `buffered_records`
|
||||
pub fn remaining_entries(
|
||||
provider: &Arc<KafkaProvider>,
|
||||
buffered_records: &mut HashMap<RegionId, Vec<Record>>,
|
||||
) -> Option<Vec<Entry>> {
|
||||
if buffered_records.is_empty() {
|
||||
None
|
||||
} else {
|
||||
let mut entries = Vec::with_capacity(buffered_records.len());
|
||||
for (region_id, records) in buffered_records.drain() {
|
||||
entries.push(convert_to_multiple_entry(
|
||||
provider.clone(),
|
||||
region_id,
|
||||
records,
|
||||
));
|
||||
}
|
||||
Some(entries)
|
||||
}
|
||||
}
|
||||
|
||||
/// For type of [Entry::Naive] Entry:
|
||||
/// - Emits a [RecordType::Full] type record immediately.
|
||||
///
|
||||
/// For type of [Entry::MultiplePart] Entry:
|
||||
/// - Emits a complete or incomplete [Entry] while the next same [RegionId] record arrives.
|
||||
///
|
||||
/// **Incomplete Entry:**
|
||||
/// If the records arrive in the following order, it emits **the incomplete [Entry]** when the next record arrives.
|
||||
/// - **[RecordType::First], [RecordType::Middle]**, [RecordType::First]
|
||||
/// - **[RecordType::Middle]**, [RecordType::First]
|
||||
/// - **[RecordType::Last]**
|
||||
pub(crate) fn maybe_emit_entry(
|
||||
provider: &Arc<KafkaProvider>,
|
||||
record: Record,
|
||||
entry_records: &mut HashMap<EntryId, Vec<Record>>,
|
||||
) -> Result<Option<EntryImpl>> {
|
||||
buffered_records: &mut HashMap<RegionId, Vec<Record>>,
|
||||
) -> Result<Option<Entry>> {
|
||||
let mut entry = None;
|
||||
match record.meta.tp {
|
||||
RecordType::Full => {
|
||||
entry = Some(EntryImpl::from(vec![record]));
|
||||
}
|
||||
RecordType::Full => entry = Some(convert_to_naive_entry(provider.clone(), record)),
|
||||
RecordType::First => {
|
||||
ensure!(
|
||||
!entry_records.contains_key(&record.meta.entry_id),
|
||||
IllegalSequenceSnafu {
|
||||
error: "First record must be the first"
|
||||
}
|
||||
);
|
||||
entry_records.insert(record.meta.entry_id, vec![record]);
|
||||
let region_id = record.meta.ns.region_id.into();
|
||||
if let Some(records) = buffered_records.insert(region_id, vec![record]) {
|
||||
// Incomplete entry
|
||||
entry = Some(convert_to_multiple_entry(
|
||||
provider.clone(),
|
||||
region_id,
|
||||
records,
|
||||
))
|
||||
}
|
||||
}
|
||||
RecordType::Middle(seq) => {
|
||||
let prefix =
|
||||
entry_records
|
||||
.get_mut(&record.meta.entry_id)
|
||||
.context(IllegalSequenceSnafu {
|
||||
error: "Middle record must not be the first",
|
||||
})?;
|
||||
// Safety: the records are guaranteed not empty if the key exists.
|
||||
let last_record = prefix.last().unwrap();
|
||||
let legal = match last_record.meta.tp {
|
||||
// Legal if this record follows a First record.
|
||||
RecordType::First => seq == 1,
|
||||
// Legal if this record follows a Middle record just prior to this record.
|
||||
RecordType::Middle(last_seq) => last_seq + 1 == seq,
|
||||
// Illegal sequence.
|
||||
_ => false,
|
||||
};
|
||||
ensure!(
|
||||
legal,
|
||||
IllegalSequenceSnafu {
|
||||
error: "Illegal prefix for a Middle record"
|
||||
}
|
||||
);
|
||||
let region_id = record.meta.ns.region_id.into();
|
||||
let records = buffered_records.entry(region_id).or_default();
|
||||
|
||||
prefix.push(record);
|
||||
// Only validate complete entries.
|
||||
if !records.is_empty() {
|
||||
// Safety: the records are guaranteed not empty if the key exists.
|
||||
let last_record = records.last().unwrap();
|
||||
let legal = match last_record.meta.tp {
|
||||
// Legal if this record follows a First record.
|
||||
RecordType::First => seq == 1,
|
||||
// Legal if this record follows a Middle record just prior to this record.
|
||||
RecordType::Middle(last_seq) => last_seq + 1 == seq,
|
||||
// Illegal sequence.
|
||||
_ => false,
|
||||
};
|
||||
ensure!(
|
||||
legal,
|
||||
IllegalSequenceSnafu {
|
||||
error: format!(
|
||||
"Illegal sequence of a middle record, last record: {:?}, incoming record: {:?}",
|
||||
last_record.meta.tp,
|
||||
record.meta.tp
|
||||
)
|
||||
}
|
||||
);
|
||||
}
|
||||
|
||||
records.push(record);
|
||||
}
|
||||
RecordType::Last => {
|
||||
// There must have a sequence prefix before a Last record is read.
|
||||
let mut records =
|
||||
entry_records
|
||||
.remove(&record.meta.entry_id)
|
||||
.context(IllegalSequenceSnafu {
|
||||
error: "Missing prefix for a Last record",
|
||||
})?;
|
||||
records.push(record);
|
||||
entry = Some(EntryImpl::from(records));
|
||||
let region_id = record.meta.ns.region_id.into();
|
||||
if let Some(mut records) = buffered_records.remove(®ion_id) {
|
||||
records.push(record);
|
||||
entry = Some(convert_to_multiple_entry(
|
||||
provider.clone(),
|
||||
region_id,
|
||||
records,
|
||||
))
|
||||
} else {
|
||||
// Incomplete entry
|
||||
entry = Some(convert_to_multiple_entry(
|
||||
provider.clone(),
|
||||
region_id,
|
||||
vec![record],
|
||||
))
|
||||
}
|
||||
}
|
||||
}
|
||||
Ok(entry)
|
||||
@@ -300,278 +374,141 @@ pub fn maybe_emit_entry(
|
||||
|
||||
#[cfg(test)]
|
||||
mod tests {
|
||||
use std::assert_matches::assert_matches;
|
||||
use std::sync::Arc;
|
||||
|
||||
use common_base::readable_size::ReadableSize;
|
||||
use common_wal::config::kafka::DatanodeKafkaConfig;
|
||||
use common_wal::test_util::run_test_with_kafka_wal;
|
||||
use uuid::Uuid;
|
||||
|
||||
use super::*;
|
||||
use crate::kafka::client_manager::ClientManager;
|
||||
use crate::error;
|
||||
|
||||
// Implements some utility methods for testing.
|
||||
impl Default for Record {
|
||||
fn default() -> Self {
|
||||
Self {
|
||||
meta: RecordMeta {
|
||||
version: VERSION,
|
||||
tp: RecordType::Full,
|
||||
ns: NamespaceImpl {
|
||||
region_id: 0,
|
||||
topic: "greptimedb_wal_topic".to_string(),
|
||||
},
|
||||
entry_id: 0,
|
||||
},
|
||||
data: Vec::new(),
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
impl Record {
|
||||
/// Overrides tp.
|
||||
fn with_tp(&self, tp: RecordType) -> Self {
|
||||
Self {
|
||||
meta: RecordMeta {
|
||||
tp,
|
||||
..self.meta.clone()
|
||||
},
|
||||
..self.clone()
|
||||
}
|
||||
}
|
||||
|
||||
/// Overrides data with the given data.
|
||||
fn with_data(&self, data: &[u8]) -> Self {
|
||||
Self {
|
||||
data: data.to_vec(),
|
||||
..self.clone()
|
||||
}
|
||||
}
|
||||
|
||||
/// Overrides entry id.
|
||||
fn with_entry_id(&self, entry_id: EntryId) -> Self {
|
||||
Self {
|
||||
meta: RecordMeta {
|
||||
entry_id,
|
||||
..self.meta.clone()
|
||||
},
|
||||
..self.clone()
|
||||
}
|
||||
}
|
||||
|
||||
/// Overrides namespace.
|
||||
fn with_ns(&self, ns: NamespaceImpl) -> Self {
|
||||
Self {
|
||||
meta: RecordMeta { ns, ..self.meta },
|
||||
..self.clone()
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
fn new_test_entry<D: AsRef<[u8]>>(data: D, entry_id: EntryId, ns: NamespaceImpl) -> EntryImpl {
|
||||
EntryImpl {
|
||||
data: data.as_ref().to_vec(),
|
||||
id: entry_id,
|
||||
ns,
|
||||
}
|
||||
}
|
||||
|
||||
/// Tests that the `build_records` works as expected.
|
||||
#[test]
|
||||
fn test_build_records() {
|
||||
let max_record_size = 128;
|
||||
|
||||
// On a small entry.
|
||||
let ns = NamespaceImpl {
|
||||
region_id: 1,
|
||||
topic: "greptimedb_wal_topic".to_string(),
|
||||
};
|
||||
let entry = new_test_entry([b'1'; 100], 0, ns.clone());
|
||||
let records = build_records(entry.clone(), max_record_size);
|
||||
assert!(records.len() == 1);
|
||||
assert_eq!(entry.data, records[0].data);
|
||||
|
||||
// On a large entry.
|
||||
let entry = new_test_entry([b'1'; 150], 0, ns.clone());
|
||||
let records = build_records(entry.clone(), max_record_size);
|
||||
assert!(records.len() == 2);
|
||||
assert_eq!(&records[0].data, &[b'1'; 128]);
|
||||
assert_eq!(&records[1].data, &[b'1'; 22]);
|
||||
|
||||
// On a way-too large entry.
|
||||
let entry = new_test_entry([b'1'; 5000], 0, ns.clone());
|
||||
let records = build_records(entry.clone(), max_record_size);
|
||||
let matched = entry
|
||||
.data
|
||||
.chunks(max_record_size)
|
||||
.enumerate()
|
||||
.all(|(i, chunk)| records[i].data == chunk);
|
||||
assert!(matched);
|
||||
}
|
||||
|
||||
/// Tests that Record and KafkaRecord are able to be converted back and forth.
|
||||
#[test]
|
||||
fn test_record_conversion() {
|
||||
let record = Record {
|
||||
fn new_test_record(tp: RecordType, entry_id: EntryId, region_id: u64, data: Vec<u8>) -> Record {
|
||||
Record {
|
||||
meta: RecordMeta {
|
||||
version: VERSION,
|
||||
tp: RecordType::Full,
|
||||
entry_id: 1,
|
||||
tp,
|
||||
ns: NamespaceImpl {
|
||||
region_id: 1,
|
||||
region_id,
|
||||
topic: "greptimedb_wal_topic".to_string(),
|
||||
},
|
||||
entry_id,
|
||||
},
|
||||
data: b"12345".to_vec(),
|
||||
};
|
||||
let kafka_record: KafkaRecord = record.clone().try_into().unwrap();
|
||||
let got = Record::try_from(kafka_record).unwrap();
|
||||
assert_eq!(record, got);
|
||||
data,
|
||||
}
|
||||
}
|
||||
|
||||
/// Tests that the reconstruction of an entry works as expected.
|
||||
#[test]
|
||||
fn test_reconstruct_entry() {
|
||||
let template = Record::default();
|
||||
let records = vec![
|
||||
template.with_data(b"111").with_tp(RecordType::First),
|
||||
template.with_data(b"222").with_tp(RecordType::Middle(1)),
|
||||
template.with_data(b"333").with_tp(RecordType::Last),
|
||||
];
|
||||
let entry = EntryImpl::from(records.clone());
|
||||
assert_eq!(records[0].meta.entry_id, entry.id);
|
||||
assert_eq!(records[0].meta.ns, entry.ns);
|
||||
fn test_maybe_emit_entry_emit_naive_entry() {
|
||||
let provider = Arc::new(KafkaProvider::new("my_topic".to_string()));
|
||||
let region_id = RegionId::new(1, 1);
|
||||
let mut buffer = HashMap::new();
|
||||
let record = new_test_record(RecordType::Full, 1, region_id.as_u64(), vec![1; 100]);
|
||||
let entry = maybe_emit_entry(&provider, record, &mut buffer)
|
||||
.unwrap()
|
||||
.unwrap();
|
||||
assert_eq!(
|
||||
entry.data,
|
||||
records
|
||||
.into_iter()
|
||||
.flat_map(|record| record.data)
|
||||
.collect::<Vec<_>>()
|
||||
);
|
||||
}
|
||||
|
||||
/// Tests that `maybe_emit_entry` works as expected.
|
||||
/// This test does not check for illegal record sequences since they're already tested in the `test_check_records` test.
|
||||
#[test]
|
||||
fn test_maybe_emit_entry() {
|
||||
let ns = NamespaceImpl {
|
||||
region_id: 1,
|
||||
topic: "greptimedb_wal_topic".to_string(),
|
||||
};
|
||||
let template = Record::default().with_ns(ns);
|
||||
let mut entry_records = HashMap::from([
|
||||
(
|
||||
1,
|
||||
vec![template.with_entry_id(1).with_tp(RecordType::First)],
|
||||
),
|
||||
(
|
||||
2,
|
||||
vec![template.with_entry_id(2).with_tp(RecordType::First)],
|
||||
),
|
||||
(
|
||||
3,
|
||||
vec![
|
||||
template.with_entry_id(3).with_tp(RecordType::First),
|
||||
template.with_entry_id(3).with_tp(RecordType::Middle(1)),
|
||||
],
|
||||
),
|
||||
]);
|
||||
|
||||
// A Full record arrives.
|
||||
let got = maybe_emit_entry(
|
||||
template.with_entry_id(0).with_tp(RecordType::Full),
|
||||
&mut entry_records,
|
||||
)
|
||||
.unwrap();
|
||||
assert!(got.is_some());
|
||||
|
||||
// A First record arrives with no prefix.
|
||||
let got = maybe_emit_entry(
|
||||
template.with_entry_id(0).with_tp(RecordType::First),
|
||||
&mut entry_records,
|
||||
)
|
||||
.unwrap();
|
||||
assert!(got.is_none());
|
||||
|
||||
// A First record arrives with some prefix.
|
||||
let got = maybe_emit_entry(
|
||||
template.with_entry_id(1).with_tp(RecordType::First),
|
||||
&mut entry_records,
|
||||
);
|
||||
assert!(got.is_err());
|
||||
|
||||
// A Middle record arrives with legal prefix (First).
|
||||
let got = maybe_emit_entry(
|
||||
template.with_entry_id(2).with_tp(RecordType::Middle(1)),
|
||||
&mut entry_records,
|
||||
)
|
||||
.unwrap();
|
||||
assert!(got.is_none());
|
||||
|
||||
// A Middle record arrives with legal prefix (Middle).
|
||||
let got = maybe_emit_entry(
|
||||
template.with_entry_id(2).with_tp(RecordType::Middle(2)),
|
||||
&mut entry_records,
|
||||
)
|
||||
.unwrap();
|
||||
assert!(got.is_none());
|
||||
|
||||
// A Middle record arrives with illegal prefix.
|
||||
let got = maybe_emit_entry(
|
||||
template.with_entry_id(2).with_tp(RecordType::Middle(1)),
|
||||
&mut entry_records,
|
||||
);
|
||||
assert!(got.is_err());
|
||||
|
||||
// A Middle record arrives with no prefix.
|
||||
let got = maybe_emit_entry(
|
||||
template.with_entry_id(22).with_tp(RecordType::Middle(1)),
|
||||
&mut entry_records,
|
||||
);
|
||||
assert!(got.is_err());
|
||||
|
||||
// A Last record arrives with no prefix.
|
||||
let got = maybe_emit_entry(
|
||||
template.with_entry_id(33).with_tp(RecordType::Last),
|
||||
&mut entry_records,
|
||||
);
|
||||
assert!(got.is_err());
|
||||
|
||||
// A Last record arrives with legal prefix.
|
||||
let got = maybe_emit_entry(
|
||||
template.with_entry_id(3).with_tp(RecordType::Last),
|
||||
&mut entry_records,
|
||||
)
|
||||
.unwrap();
|
||||
assert!(got.is_some());
|
||||
|
||||
// Check state.
|
||||
assert_eq!(entry_records.len(), 3);
|
||||
assert_eq!(entry_records[&0].len(), 1);
|
||||
assert_eq!(entry_records[&1].len(), 1);
|
||||
assert_eq!(entry_records[&2].len(), 3);
|
||||
}
|
||||
|
||||
#[tokio::test]
|
||||
async fn test_produce_large_entry() {
|
||||
run_test_with_kafka_wal(|broker_endpoints| {
|
||||
Box::pin(async {
|
||||
let topic = format!("greptimedb_wal_topic_{}", Uuid::new_v4());
|
||||
let ns = NamespaceImpl {
|
||||
region_id: 1,
|
||||
topic,
|
||||
};
|
||||
let entry = new_test_entry([b'1'; 2000000], 0, ns.clone());
|
||||
let producer = RecordProducer::new(ns.clone()).with_entries(vec![entry]);
|
||||
let config = DatanodeKafkaConfig {
|
||||
broker_endpoints,
|
||||
max_batch_size: ReadableSize::mb(1),
|
||||
..Default::default()
|
||||
};
|
||||
let manager = Arc::new(ClientManager::try_new(&config).await.unwrap());
|
||||
producer.produce(&manager).await.unwrap();
|
||||
entry,
|
||||
Entry::Naive(NaiveEntry {
|
||||
provider: Provider::Kafka(provider),
|
||||
region_id,
|
||||
entry_id: 1,
|
||||
data: vec![1; 100]
|
||||
})
|
||||
})
|
||||
.await
|
||||
);
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_maybe_emit_entry_emit_incomplete_entry() {
|
||||
let provider = Arc::new(KafkaProvider::new("my_topic".to_string()));
|
||||
let region_id = RegionId::new(1, 1);
|
||||
// `First` overwrite `First`
|
||||
let mut buffer = HashMap::new();
|
||||
let record = new_test_record(RecordType::First, 1, region_id.as_u64(), vec![1; 100]);
|
||||
assert!(maybe_emit_entry(&provider, record, &mut buffer)
|
||||
.unwrap()
|
||||
.is_none());
|
||||
let record = new_test_record(RecordType::First, 2, region_id.as_u64(), vec![2; 100]);
|
||||
let incomplete_entry = maybe_emit_entry(&provider, record, &mut buffer)
|
||||
.unwrap()
|
||||
.unwrap();
|
||||
|
||||
assert_eq!(
|
||||
incomplete_entry,
|
||||
Entry::MultiplePart(MultiplePartEntry {
|
||||
provider: Provider::Kafka(provider.clone()),
|
||||
region_id,
|
||||
// TODO(weny): always be 0.
|
||||
entry_id: 0,
|
||||
headers: vec![MultiplePartHeader::First],
|
||||
parts: vec![vec![1; 100]],
|
||||
})
|
||||
);
|
||||
|
||||
// `Last` overwrite `None`
|
||||
let mut buffer = HashMap::new();
|
||||
let record = new_test_record(RecordType::Last, 1, region_id.as_u64(), vec![1; 100]);
|
||||
let incomplete_entry = maybe_emit_entry(&provider, record, &mut buffer)
|
||||
.unwrap()
|
||||
.unwrap();
|
||||
|
||||
assert_eq!(
|
||||
incomplete_entry,
|
||||
Entry::MultiplePart(MultiplePartEntry {
|
||||
provider: Provider::Kafka(provider.clone()),
|
||||
region_id,
|
||||
// TODO(weny): always be 0.
|
||||
entry_id: 0,
|
||||
headers: vec![MultiplePartHeader::Last],
|
||||
parts: vec![vec![1; 100]],
|
||||
})
|
||||
);
|
||||
|
||||
// `First` overwrite `Middle(0)`
|
||||
let mut buffer = HashMap::new();
|
||||
let record = new_test_record(RecordType::Middle(0), 1, region_id.as_u64(), vec![1; 100]);
|
||||
assert!(maybe_emit_entry(&provider, record, &mut buffer)
|
||||
.unwrap()
|
||||
.is_none());
|
||||
let record = new_test_record(RecordType::First, 2, region_id.as_u64(), vec![2; 100]);
|
||||
let incomplete_entry = maybe_emit_entry(&provider, record, &mut buffer)
|
||||
.unwrap()
|
||||
.unwrap();
|
||||
|
||||
assert_eq!(
|
||||
incomplete_entry,
|
||||
Entry::MultiplePart(MultiplePartEntry {
|
||||
provider: Provider::Kafka(provider),
|
||||
region_id,
|
||||
// TODO(weny): always be 0.
|
||||
entry_id: 0,
|
||||
headers: vec![MultiplePartHeader::Middle(0)],
|
||||
parts: vec![vec![1; 100]],
|
||||
})
|
||||
);
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_maybe_emit_entry_illegal_seq() {
|
||||
let provider = Arc::new(KafkaProvider::new("my_topic".to_string()));
|
||||
let region_id = RegionId::new(1, 1);
|
||||
let mut buffer = HashMap::new();
|
||||
let record = new_test_record(RecordType::First, 1, region_id.as_u64(), vec![1; 100]);
|
||||
assert!(maybe_emit_entry(&provider, record, &mut buffer)
|
||||
.unwrap()
|
||||
.is_none());
|
||||
let record = new_test_record(RecordType::Middle(2), 1, region_id.as_u64(), vec![2; 100]);
|
||||
let err = maybe_emit_entry(&provider, record, &mut buffer).unwrap_err();
|
||||
assert_matches!(err, error::Error::IllegalSequence { .. });
|
||||
|
||||
let mut buffer = HashMap::new();
|
||||
let record = new_test_record(RecordType::First, 1, region_id.as_u64(), vec![1; 100]);
|
||||
assert!(maybe_emit_entry(&provider, record, &mut buffer)
|
||||
.unwrap()
|
||||
.is_none());
|
||||
let record = new_test_record(RecordType::Middle(1), 1, region_id.as_u64(), vec![2; 100]);
|
||||
assert!(maybe_emit_entry(&provider, record, &mut buffer)
|
||||
.unwrap()
|
||||
.is_none());
|
||||
let record = new_test_record(RecordType::Middle(3), 1, region_id.as_u64(), vec![2; 100]);
|
||||
let err = maybe_emit_entry(&provider, record, &mut buffer).unwrap_err();
|
||||
assert_matches!(err, error::Error::IllegalSequence { .. });
|
||||
}
|
||||
}
|
||||
|
||||
@@ -14,12 +14,10 @@
|
||||
|
||||
#![feature(let_chains)]
|
||||
#![feature(io_error_more)]
|
||||
#![feature(assert_matches)]
|
||||
|
||||
pub mod error;
|
||||
pub mod kafka;
|
||||
pub mod metrics;
|
||||
mod noop;
|
||||
pub mod raft_engine;
|
||||
pub mod test_util;
|
||||
|
||||
pub use noop::NoopLogStore;
|
||||
|
||||
@@ -1,153 +0,0 @@
|
||||
// Copyright 2023 Greptime Team
|
||||
//
|
||||
// Licensed under the Apache License, Version 2.0 (the "License");
|
||||
// you may not use this file except in compliance with the License.
|
||||
// You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
use common_wal::options::WalOptions;
|
||||
use store_api::logstore::entry::{Entry, Id as EntryId, RawEntry};
|
||||
use store_api::logstore::namespace::{Id as NamespaceId, Namespace};
|
||||
use store_api::logstore::{AppendBatchResponse, AppendResponse, LogStore};
|
||||
use store_api::storage::RegionId;
|
||||
|
||||
use crate::error::{Error, Result};
|
||||
|
||||
/// A noop log store which only for test
|
||||
#[derive(Debug, Default)]
|
||||
pub struct NoopLogStore;
|
||||
|
||||
#[derive(Debug, Default, Clone, PartialEq)]
|
||||
pub struct EntryImpl;
|
||||
|
||||
#[derive(Debug, Clone, Default, Eq, PartialEq, Hash)]
|
||||
pub struct NamespaceImpl;
|
||||
|
||||
impl Namespace for NamespaceImpl {
|
||||
fn id(&self) -> NamespaceId {
|
||||
0
|
||||
}
|
||||
}
|
||||
|
||||
impl Entry for EntryImpl {
|
||||
fn into_raw_entry(self) -> RawEntry {
|
||||
RawEntry {
|
||||
region_id: self.region_id(),
|
||||
entry_id: self.id(),
|
||||
data: vec![],
|
||||
}
|
||||
}
|
||||
|
||||
fn data(&self) -> &[u8] {
|
||||
&[]
|
||||
}
|
||||
|
||||
fn id(&self) -> EntryId {
|
||||
0
|
||||
}
|
||||
|
||||
fn region_id(&self) -> RegionId {
|
||||
RegionId::from_u64(0)
|
||||
}
|
||||
|
||||
fn estimated_size(&self) -> usize {
|
||||
0
|
||||
}
|
||||
}
|
||||
|
||||
#[async_trait::async_trait]
|
||||
impl LogStore for NoopLogStore {
|
||||
type Error = Error;
|
||||
type Namespace = NamespaceImpl;
|
||||
type Entry = EntryImpl;
|
||||
|
||||
async fn stop(&self) -> Result<()> {
|
||||
Ok(())
|
||||
}
|
||||
|
||||
async fn append(&self, mut _e: Self::Entry) -> Result<AppendResponse> {
|
||||
Ok(AppendResponse::default())
|
||||
}
|
||||
|
||||
async fn append_batch(&self, _e: Vec<Self::Entry>) -> Result<AppendBatchResponse> {
|
||||
Ok(AppendBatchResponse::default())
|
||||
}
|
||||
|
||||
async fn read(
|
||||
&self,
|
||||
_ns: &Self::Namespace,
|
||||
_entry_id: EntryId,
|
||||
) -> Result<store_api::logstore::entry_stream::SendableEntryStream<'_, Self::Entry, Self::Error>>
|
||||
{
|
||||
Ok(Box::pin(futures::stream::once(futures::future::ready(Ok(
|
||||
vec![],
|
||||
)))))
|
||||
}
|
||||
|
||||
async fn create_namespace(&self, _ns: &Self::Namespace) -> Result<()> {
|
||||
Ok(())
|
||||
}
|
||||
|
||||
async fn delete_namespace(&self, _ns: &Self::Namespace) -> Result<()> {
|
||||
Ok(())
|
||||
}
|
||||
|
||||
async fn list_namespaces(&self) -> Result<Vec<Self::Namespace>> {
|
||||
Ok(vec![])
|
||||
}
|
||||
|
||||
fn entry(&self, data: &mut Vec<u8>, entry_id: EntryId, ns: Self::Namespace) -> Self::Entry {
|
||||
let _ = data;
|
||||
let _ = entry_id;
|
||||
let _ = ns;
|
||||
EntryImpl
|
||||
}
|
||||
|
||||
fn namespace(&self, ns_id: NamespaceId, wal_options: &WalOptions) -> Self::Namespace {
|
||||
let _ = ns_id;
|
||||
let _ = wal_options;
|
||||
NamespaceImpl
|
||||
}
|
||||
|
||||
async fn obsolete(
|
||||
&self,
|
||||
ns: Self::Namespace,
|
||||
entry_id: EntryId,
|
||||
) -> std::result::Result<(), Self::Error> {
|
||||
let _ = ns;
|
||||
let _ = entry_id;
|
||||
Ok(())
|
||||
}
|
||||
}
|
||||
|
||||
#[cfg(test)]
|
||||
mod tests {
|
||||
use super::*;
|
||||
|
||||
#[test]
|
||||
fn test_mock_entry() {
|
||||
let e = EntryImpl;
|
||||
assert_eq!(0, e.data().len());
|
||||
assert_eq!(0, e.id());
|
||||
}
|
||||
|
||||
#[tokio::test]
|
||||
async fn test_noop_logstore() {
|
||||
let store = NoopLogStore;
|
||||
let e = store.entry(&mut vec![], 1, NamespaceImpl);
|
||||
let _ = store.append(e.clone()).await.unwrap();
|
||||
assert!(store.append_batch(vec![e]).await.is_ok());
|
||||
store.create_namespace(&NamespaceImpl).await.unwrap();
|
||||
assert_eq!(0, store.list_namespaces().await.unwrap().len());
|
||||
store.delete_namespace(&NamespaceImpl).await.unwrap();
|
||||
assert_eq!(NamespaceImpl, store.namespace(0, &WalOptions::default()));
|
||||
store.obsolete(NamespaceImpl, 1).await.unwrap();
|
||||
}
|
||||
}
|
||||
@@ -12,20 +12,16 @@
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
use std::hash::{Hash, Hasher};
|
||||
use std::mem::size_of;
|
||||
|
||||
use store_api::logstore::entry::{Entry, Id as EntryId, RawEntry};
|
||||
use store_api::logstore::namespace::{Id as NamespaceId, Namespace};
|
||||
use store_api::storage::RegionId;
|
||||
|
||||
use crate::raft_engine::protos::logstore::{EntryImpl, NamespaceImpl};
|
||||
use crate::raft_engine::protos::logstore::EntryImpl;
|
||||
|
||||
mod backend;
|
||||
pub mod log_store;
|
||||
|
||||
pub use backend::RaftEngineBackend;
|
||||
pub use raft_engine::Config;
|
||||
use store_api::logstore::entry::{Entry, NaiveEntry};
|
||||
use store_api::logstore::provider::Provider;
|
||||
use store_api::storage::RegionId;
|
||||
|
||||
pub mod protos {
|
||||
include!(concat!(env!("OUT_DIR"), concat!("/", "protos/", "mod.rs")));
|
||||
@@ -42,65 +38,20 @@ impl EntryImpl {
|
||||
}
|
||||
}
|
||||
|
||||
impl NamespaceImpl {
|
||||
pub fn with_id(id: NamespaceId) -> Self {
|
||||
Self {
|
||||
impl From<EntryImpl> for Entry {
|
||||
fn from(
|
||||
EntryImpl {
|
||||
id,
|
||||
..Default::default()
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
#[allow(clippy::derived_hash_with_manual_eq)]
|
||||
impl Hash for NamespaceImpl {
|
||||
fn hash<H: Hasher>(&self, state: &mut H) {
|
||||
self.id.hash(state);
|
||||
}
|
||||
}
|
||||
|
||||
impl Eq for NamespaceImpl {}
|
||||
|
||||
impl Namespace for NamespaceImpl {
|
||||
fn id(&self) -> NamespaceId {
|
||||
self.id
|
||||
}
|
||||
}
|
||||
|
||||
impl Entry for EntryImpl {
|
||||
fn into_raw_entry(self) -> RawEntry {
|
||||
RawEntry {
|
||||
region_id: self.region_id(),
|
||||
entry_id: self.id(),
|
||||
data: self.data,
|
||||
}
|
||||
}
|
||||
|
||||
fn data(&self) -> &[u8] {
|
||||
self.data.as_slice()
|
||||
}
|
||||
|
||||
fn id(&self) -> EntryId {
|
||||
self.id
|
||||
}
|
||||
|
||||
fn region_id(&self) -> RegionId {
|
||||
RegionId::from_u64(self.id)
|
||||
}
|
||||
|
||||
fn estimated_size(&self) -> usize {
|
||||
self.data.len() + size_of::<u64>() + size_of::<u64>()
|
||||
}
|
||||
}
|
||||
|
||||
#[cfg(test)]
|
||||
mod tests {
|
||||
use store_api::logstore::entry::Entry;
|
||||
|
||||
use crate::raft_engine::protos::logstore::EntryImpl;
|
||||
|
||||
#[test]
|
||||
fn test_estimated_size() {
|
||||
let entry = EntryImpl::create(1, 1, b"hello, world".to_vec());
|
||||
assert_eq!(28, entry.estimated_size());
|
||||
namespace_id,
|
||||
data,
|
||||
..
|
||||
}: EntryImpl,
|
||||
) -> Self {
|
||||
Entry::Naive(NaiveEntry {
|
||||
provider: Provider::raft_engine_provider(namespace_id),
|
||||
region_id: RegionId::from_u64(namespace_id),
|
||||
entry_id: id,
|
||||
data,
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
@@ -12,8 +12,7 @@
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
use std::collections::hash_map::Entry;
|
||||
use std::collections::HashMap;
|
||||
use std::collections::{hash_map, HashMap};
|
||||
use std::fmt::{Debug, Formatter};
|
||||
use std::sync::atomic::{AtomicI64, Ordering};
|
||||
use std::sync::Arc;
|
||||
@@ -22,22 +21,21 @@ use async_stream::stream;
|
||||
use common_runtime::{RepeatedTask, TaskFunction};
|
||||
use common_telemetry::{error, info};
|
||||
use common_wal::config::raft_engine::RaftEngineConfig;
|
||||
use common_wal::options::WalOptions;
|
||||
use raft_engine::{Config, Engine, LogBatch, MessageExt, ReadableSize, RecoveryMode};
|
||||
use snafu::{ensure, ResultExt};
|
||||
use store_api::logstore::entry::{Entry as EntryTrait, Id as EntryId};
|
||||
use store_api::logstore::entry_stream::SendableEntryStream;
|
||||
use store_api::logstore::namespace::{Id as NamespaceId, Namespace as NamespaceTrait};
|
||||
use store_api::logstore::{AppendBatchResponse, AppendResponse, LogStore};
|
||||
use snafu::{ensure, OptionExt, ResultExt};
|
||||
use store_api::logstore::entry::{Entry, Id as EntryId, NaiveEntry};
|
||||
use store_api::logstore::provider::{Provider, RaftEngineProvider};
|
||||
use store_api::logstore::{AppendBatchResponse, LogStore, SendableEntryStream};
|
||||
use store_api::storage::RegionId;
|
||||
|
||||
use crate::error::{
|
||||
AddEntryLogBatchSnafu, DiscontinuousLogIndexSnafu, Error, FetchEntrySnafu,
|
||||
IllegalNamespaceSnafu, IllegalStateSnafu, OverrideCompactedEntrySnafu, RaftEngineSnafu, Result,
|
||||
StartGcTaskSnafu, StopGcTaskSnafu,
|
||||
IllegalNamespaceSnafu, IllegalStateSnafu, InvalidProviderSnafu, OverrideCompactedEntrySnafu,
|
||||
RaftEngineSnafu, Result, StartGcTaskSnafu, StopGcTaskSnafu,
|
||||
};
|
||||
use crate::metrics;
|
||||
use crate::raft_engine::backend::SYSTEM_NAMESPACE;
|
||||
use crate::raft_engine::protos::logstore::{EntryImpl, NamespaceImpl as Namespace};
|
||||
use crate::raft_engine::protos::logstore::{EntryImpl, NamespaceImpl};
|
||||
|
||||
const NAMESPACE_PREFIX: &str = "$sys/";
|
||||
|
||||
@@ -117,10 +115,10 @@ impl RaftEngineLogStore {
|
||||
.context(StartGcTaskSnafu)
|
||||
}
|
||||
|
||||
fn span(&self, namespace: &<Self as LogStore>::Namespace) -> (Option<u64>, Option<u64>) {
|
||||
fn span(&self, provider: &RaftEngineProvider) -> (Option<u64>, Option<u64>) {
|
||||
(
|
||||
self.engine.first_index(namespace.id()),
|
||||
self.engine.last_index(namespace.id()),
|
||||
self.engine.first_index(provider.id),
|
||||
self.engine.last_index(provider.id),
|
||||
)
|
||||
}
|
||||
|
||||
@@ -129,56 +127,65 @@ impl RaftEngineLogStore {
|
||||
/// to append in each namespace(region).
|
||||
fn entries_to_batch(
|
||||
&self,
|
||||
entries: Vec<EntryImpl>,
|
||||
) -> Result<(LogBatch, HashMap<NamespaceId, EntryId>)> {
|
||||
entries: Vec<Entry>,
|
||||
) -> Result<(LogBatch, HashMap<RegionId, EntryId>)> {
|
||||
// Records the last entry id for each region's entries.
|
||||
let mut entry_ids: HashMap<NamespaceId, EntryId> = HashMap::with_capacity(entries.len());
|
||||
let mut entry_ids: HashMap<RegionId, EntryId> = HashMap::with_capacity(entries.len());
|
||||
let mut batch = LogBatch::with_capacity(entries.len());
|
||||
|
||||
for e in entries {
|
||||
let ns_id = e.namespace_id;
|
||||
match entry_ids.entry(ns_id) {
|
||||
Entry::Occupied(mut o) => {
|
||||
let region_id = e.region_id();
|
||||
let entry_id = e.entry_id();
|
||||
match entry_ids.entry(region_id) {
|
||||
hash_map::Entry::Occupied(mut o) => {
|
||||
let prev = *o.get();
|
||||
ensure!(
|
||||
e.id == prev + 1,
|
||||
entry_id == prev + 1,
|
||||
DiscontinuousLogIndexSnafu {
|
||||
region_id: ns_id,
|
||||
region_id,
|
||||
last_index: prev,
|
||||
attempt_index: e.id
|
||||
attempt_index: entry_id
|
||||
}
|
||||
);
|
||||
o.insert(e.id);
|
||||
o.insert(entry_id);
|
||||
}
|
||||
Entry::Vacant(v) => {
|
||||
hash_map::Entry::Vacant(v) => {
|
||||
// this entry is the first in batch of given region.
|
||||
if let Some(first_index) = self.engine.first_index(ns_id) {
|
||||
if let Some(first_index) = self.engine.first_index(region_id.as_u64()) {
|
||||
// ensure the first in batch does not override compacted entry.
|
||||
ensure!(
|
||||
e.id > first_index,
|
||||
entry_id > first_index,
|
||||
OverrideCompactedEntrySnafu {
|
||||
namespace: ns_id,
|
||||
namespace: region_id,
|
||||
first_index,
|
||||
attempt_index: e.id,
|
||||
attempt_index: entry_id,
|
||||
}
|
||||
);
|
||||
}
|
||||
// ensure the first in batch does not form a hole in raft-engine.
|
||||
if let Some(last_index) = self.engine.last_index(ns_id) {
|
||||
if let Some(last_index) = self.engine.last_index(region_id.as_u64()) {
|
||||
ensure!(
|
||||
e.id == last_index + 1,
|
||||
entry_id == last_index + 1,
|
||||
DiscontinuousLogIndexSnafu {
|
||||
region_id: ns_id,
|
||||
region_id,
|
||||
last_index,
|
||||
attempt_index: e.id
|
||||
attempt_index: entry_id
|
||||
}
|
||||
);
|
||||
}
|
||||
v.insert(e.id);
|
||||
v.insert(entry_id);
|
||||
}
|
||||
}
|
||||
batch
|
||||
.add_entries::<MessageType>(ns_id, &[e])
|
||||
.add_entries::<MessageType>(
|
||||
region_id.as_u64(),
|
||||
&[EntryImpl {
|
||||
id: entry_id,
|
||||
namespace_id: region_id.as_u64(),
|
||||
data: e.into_bytes(),
|
||||
..Default::default()
|
||||
}],
|
||||
)
|
||||
.context(AddEntryLogBatchSnafu)?;
|
||||
}
|
||||
|
||||
@@ -198,62 +205,19 @@ impl Debug for RaftEngineLogStore {
|
||||
#[async_trait::async_trait]
|
||||
impl LogStore for RaftEngineLogStore {
|
||||
type Error = Error;
|
||||
type Namespace = Namespace;
|
||||
type Entry = EntryImpl;
|
||||
|
||||
async fn stop(&self) -> Result<()> {
|
||||
self.gc_task.stop().await.context(StopGcTaskSnafu)
|
||||
}
|
||||
|
||||
/// Appends an entry to logstore. Currently the existence of the entry's namespace is not checked.
|
||||
async fn append(&self, e: Self::Entry) -> Result<AppendResponse> {
|
||||
ensure!(self.started(), IllegalStateSnafu);
|
||||
let entry_id = e.id;
|
||||
let namespace_id = e.namespace_id;
|
||||
let mut batch = LogBatch::with_capacity(1);
|
||||
batch
|
||||
.add_entries::<MessageType>(namespace_id, &[e])
|
||||
.context(AddEntryLogBatchSnafu)?;
|
||||
|
||||
if let Some(first_index) = self.engine.first_index(namespace_id) {
|
||||
ensure!(
|
||||
entry_id > first_index,
|
||||
OverrideCompactedEntrySnafu {
|
||||
namespace: namespace_id,
|
||||
first_index,
|
||||
attempt_index: entry_id,
|
||||
}
|
||||
);
|
||||
}
|
||||
|
||||
if let Some(last_index) = self.engine.last_index(namespace_id) {
|
||||
ensure!(
|
||||
entry_id == last_index + 1,
|
||||
DiscontinuousLogIndexSnafu {
|
||||
region_id: namespace_id,
|
||||
last_index,
|
||||
attempt_index: entry_id
|
||||
}
|
||||
);
|
||||
}
|
||||
|
||||
let _ = self
|
||||
.engine
|
||||
.write(&mut batch, self.config.sync_write)
|
||||
.context(RaftEngineSnafu)?;
|
||||
Ok(AppendResponse {
|
||||
last_entry_id: entry_id,
|
||||
})
|
||||
}
|
||||
|
||||
/// Appends a batch of entries to logstore. `RaftEngineLogStore` assures the atomicity of
|
||||
/// batch append.
|
||||
async fn append_batch(&self, entries: Vec<Self::Entry>) -> Result<AppendBatchResponse> {
|
||||
async fn append_batch(&self, entries: Vec<Entry>) -> Result<AppendBatchResponse> {
|
||||
metrics::METRIC_RAFT_ENGINE_APPEND_BATCH_CALLS_TOTAL.inc();
|
||||
metrics::METRIC_RAFT_ENGINE_APPEND_BATCH_BYTES_TOTAL.inc_by(
|
||||
entries
|
||||
.iter()
|
||||
.map(EntryTrait::estimated_size)
|
||||
.map(|entry| entry.estimated_size())
|
||||
.sum::<usize>() as u64,
|
||||
);
|
||||
let _timer = metrics::METRIC_RAFT_ENGINE_APPEND_BATCH_ELAPSED.start_timer();
|
||||
@@ -287,40 +251,47 @@ impl LogStore for RaftEngineLogStore {
|
||||
/// determined by the current "last index" of the namespace.
|
||||
async fn read(
|
||||
&self,
|
||||
ns: &Self::Namespace,
|
||||
provider: &Provider,
|
||||
entry_id: EntryId,
|
||||
) -> Result<SendableEntryStream<'_, Self::Entry, Self::Error>> {
|
||||
) -> Result<SendableEntryStream<'static, Entry, Self::Error>> {
|
||||
let ns = provider
|
||||
.as_raft_engine_provider()
|
||||
.with_context(|| InvalidProviderSnafu {
|
||||
expected: RaftEngineProvider::type_name(),
|
||||
actual: provider.type_name(),
|
||||
})?;
|
||||
let namespace_id = ns.id;
|
||||
metrics::METRIC_RAFT_ENGINE_READ_CALLS_TOTAL.inc();
|
||||
let _timer = metrics::METRIC_RAFT_ENGINE_READ_ELAPSED.start_timer();
|
||||
|
||||
ensure!(self.started(), IllegalStateSnafu);
|
||||
let engine = self.engine.clone();
|
||||
|
||||
let last_index = engine.last_index(ns.id()).unwrap_or(0);
|
||||
let mut start_index = entry_id.max(engine.first_index(ns.id()).unwrap_or(last_index + 1));
|
||||
let last_index = engine.last_index(namespace_id).unwrap_or(0);
|
||||
let mut start_index =
|
||||
entry_id.max(engine.first_index(namespace_id).unwrap_or(last_index + 1));
|
||||
|
||||
info!(
|
||||
"Read logstore, namespace: {}, start: {}, span: {:?}",
|
||||
ns.id(),
|
||||
namespace_id,
|
||||
entry_id,
|
||||
self.span(ns)
|
||||
);
|
||||
let max_batch_size = self.config.read_batch_size;
|
||||
let (tx, mut rx) = tokio::sync::mpsc::channel(max_batch_size);
|
||||
let ns = ns.clone();
|
||||
let _handle = common_runtime::spawn_read(async move {
|
||||
while start_index <= last_index {
|
||||
let mut vec = Vec::with_capacity(max_batch_size);
|
||||
match engine
|
||||
.fetch_entries_to::<MessageType>(
|
||||
ns.id,
|
||||
namespace_id,
|
||||
start_index,
|
||||
last_index + 1,
|
||||
Some(max_batch_size),
|
||||
&mut vec,
|
||||
)
|
||||
.context(FetchEntrySnafu {
|
||||
ns: ns.id,
|
||||
ns: namespace_id,
|
||||
start: start_index,
|
||||
end: last_index,
|
||||
max_size: max_batch_size,
|
||||
@@ -344,22 +315,40 @@ impl LogStore for RaftEngineLogStore {
|
||||
|
||||
let s = stream!({
|
||||
while let Some(res) = rx.recv().await {
|
||||
yield res;
|
||||
let res = res?;
|
||||
|
||||
yield Ok(res.into_iter().map(Entry::from).collect::<Vec<_>>());
|
||||
}
|
||||
});
|
||||
Ok(Box::pin(s))
|
||||
}
|
||||
|
||||
async fn create_namespace(&self, ns: &Self::Namespace) -> Result<()> {
|
||||
async fn create_namespace(&self, ns: &Provider) -> Result<()> {
|
||||
let ns = ns
|
||||
.as_raft_engine_provider()
|
||||
.with_context(|| InvalidProviderSnafu {
|
||||
expected: RaftEngineProvider::type_name(),
|
||||
actual: ns.type_name(),
|
||||
})?;
|
||||
let namespace_id = ns.id;
|
||||
ensure!(
|
||||
ns.id != SYSTEM_NAMESPACE,
|
||||
IllegalNamespaceSnafu { ns: ns.id }
|
||||
namespace_id != SYSTEM_NAMESPACE,
|
||||
IllegalNamespaceSnafu { ns: namespace_id }
|
||||
);
|
||||
ensure!(self.started(), IllegalStateSnafu);
|
||||
let key = format!("{}{}", NAMESPACE_PREFIX, ns.id).as_bytes().to_vec();
|
||||
let key = format!("{}{}", NAMESPACE_PREFIX, namespace_id)
|
||||
.as_bytes()
|
||||
.to_vec();
|
||||
let mut batch = LogBatch::with_capacity(1);
|
||||
batch
|
||||
.put_message::<Namespace>(SYSTEM_NAMESPACE, key, ns)
|
||||
.put_message::<NamespaceImpl>(
|
||||
SYSTEM_NAMESPACE,
|
||||
key,
|
||||
&NamespaceImpl {
|
||||
id: namespace_id,
|
||||
..Default::default()
|
||||
},
|
||||
)
|
||||
.context(RaftEngineSnafu)?;
|
||||
let _ = self
|
||||
.engine
|
||||
@@ -368,13 +357,22 @@ impl LogStore for RaftEngineLogStore {
|
||||
Ok(())
|
||||
}
|
||||
|
||||
async fn delete_namespace(&self, ns: &Self::Namespace) -> Result<()> {
|
||||
async fn delete_namespace(&self, ns: &Provider) -> Result<()> {
|
||||
let ns = ns
|
||||
.as_raft_engine_provider()
|
||||
.with_context(|| InvalidProviderSnafu {
|
||||
expected: RaftEngineProvider::type_name(),
|
||||
actual: ns.type_name(),
|
||||
})?;
|
||||
let namespace_id = ns.id;
|
||||
ensure!(
|
||||
ns.id != SYSTEM_NAMESPACE,
|
||||
IllegalNamespaceSnafu { ns: ns.id }
|
||||
namespace_id != SYSTEM_NAMESPACE,
|
||||
IllegalNamespaceSnafu { ns: namespace_id }
|
||||
);
|
||||
ensure!(self.started(), IllegalStateSnafu);
|
||||
let key = format!("{}{}", NAMESPACE_PREFIX, ns.id).as_bytes().to_vec();
|
||||
let key = format!("{}{}", NAMESPACE_PREFIX, namespace_id)
|
||||
.as_bytes()
|
||||
.to_vec();
|
||||
let mut batch = LogBatch::with_capacity(1);
|
||||
batch.delete(SYSTEM_NAMESPACE, key);
|
||||
let _ = self
|
||||
@@ -384,17 +382,17 @@ impl LogStore for RaftEngineLogStore {
|
||||
Ok(())
|
||||
}
|
||||
|
||||
async fn list_namespaces(&self) -> Result<Vec<Self::Namespace>> {
|
||||
async fn list_namespaces(&self) -> Result<Vec<Provider>> {
|
||||
ensure!(self.started(), IllegalStateSnafu);
|
||||
let mut namespaces: Vec<Namespace> = vec![];
|
||||
let mut namespaces: Vec<Provider> = vec![];
|
||||
self.engine
|
||||
.scan_messages::<Namespace, _>(
|
||||
.scan_messages::<NamespaceImpl, _>(
|
||||
SYSTEM_NAMESPACE,
|
||||
Some(NAMESPACE_PREFIX.as_bytes()),
|
||||
None,
|
||||
false,
|
||||
|_, v| {
|
||||
namespaces.push(v);
|
||||
namespaces.push(Provider::RaftEngine(RaftEngineProvider { id: v.id }));
|
||||
true
|
||||
},
|
||||
)
|
||||
@@ -402,32 +400,41 @@ impl LogStore for RaftEngineLogStore {
|
||||
Ok(namespaces)
|
||||
}
|
||||
|
||||
fn entry(&self, data: &mut Vec<u8>, entry_id: EntryId, ns: Self::Namespace) -> Self::Entry {
|
||||
EntryImpl {
|
||||
id: entry_id,
|
||||
fn entry(
|
||||
&self,
|
||||
data: &mut Vec<u8>,
|
||||
entry_id: EntryId,
|
||||
region_id: RegionId,
|
||||
provider: &Provider,
|
||||
) -> Result<Entry> {
|
||||
debug_assert_eq!(
|
||||
provider.as_raft_engine_provider().unwrap().id,
|
||||
region_id.as_u64()
|
||||
);
|
||||
Ok(Entry::Naive(NaiveEntry {
|
||||
provider: provider.clone(),
|
||||
region_id,
|
||||
entry_id,
|
||||
data: std::mem::take(data),
|
||||
namespace_id: ns.id(),
|
||||
..Default::default()
|
||||
}
|
||||
}))
|
||||
}
|
||||
|
||||
fn namespace(&self, ns_id: NamespaceId, wal_options: &WalOptions) -> Self::Namespace {
|
||||
let _ = wal_options;
|
||||
Namespace {
|
||||
id: ns_id,
|
||||
..Default::default()
|
||||
}
|
||||
}
|
||||
|
||||
async fn obsolete(&self, ns: Self::Namespace, entry_id: EntryId) -> Result<()> {
|
||||
async fn obsolete(&self, provider: &Provider, entry_id: EntryId) -> Result<()> {
|
||||
let ns = provider
|
||||
.as_raft_engine_provider()
|
||||
.with_context(|| InvalidProviderSnafu {
|
||||
expected: RaftEngineProvider::type_name(),
|
||||
actual: provider.type_name(),
|
||||
})?;
|
||||
let namespace_id = ns.id;
|
||||
ensure!(self.started(), IllegalStateSnafu);
|
||||
let obsoleted = self.engine.compact_to(ns.id(), entry_id + 1);
|
||||
let obsoleted = self.engine.compact_to(namespace_id, entry_id + 1);
|
||||
info!(
|
||||
"Namespace {} obsoleted {} entries, compacted index: {}, span: {:?}",
|
||||
ns.id(),
|
||||
namespace_id,
|
||||
obsoleted,
|
||||
entry_id,
|
||||
self.span(&ns)
|
||||
self.span(ns)
|
||||
);
|
||||
Ok(())
|
||||
}
|
||||
@@ -444,6 +451,19 @@ impl MessageExt for MessageType {
|
||||
}
|
||||
}
|
||||
|
||||
#[cfg(test)]
|
||||
impl RaftEngineLogStore {
|
||||
/// Appends a batch of entries and returns a response containing a map where the key is a region id
|
||||
/// while the value is the id of the last successfully written entry of the region.
|
||||
async fn append(&self, entry: Entry) -> Result<store_api::logstore::AppendResponse> {
|
||||
let response = self.append_batch(vec![entry]).await?;
|
||||
if let Some((_, last_entry_id)) = response.last_entry_ids.into_iter().next() {
|
||||
return Ok(store_api::logstore::AppendResponse { last_entry_id });
|
||||
}
|
||||
unreachable!()
|
||||
}
|
||||
}
|
||||
|
||||
#[cfg(test)]
|
||||
mod tests {
|
||||
use std::collections::HashSet;
|
||||
@@ -453,14 +473,12 @@ mod tests {
|
||||
use common_telemetry::debug;
|
||||
use common_test_util::temp_dir::{create_temp_dir, TempDir};
|
||||
use futures_util::StreamExt;
|
||||
use store_api::logstore::entry_stream::SendableEntryStream;
|
||||
use store_api::logstore::namespace::Namespace as NamespaceTrait;
|
||||
use store_api::logstore::LogStore;
|
||||
use store_api::logstore::{LogStore, SendableEntryStream};
|
||||
|
||||
use super::*;
|
||||
use crate::error::Error;
|
||||
use crate::raft_engine::log_store::RaftEngineLogStore;
|
||||
use crate::raft_engine::protos::logstore::{EntryImpl as Entry, NamespaceImpl as Namespace};
|
||||
use crate::raft_engine::protos::logstore::EntryImpl;
|
||||
|
||||
#[tokio::test]
|
||||
async fn test_open_logstore() {
|
||||
@@ -487,15 +505,15 @@ mod tests {
|
||||
assert!(logstore.list_namespaces().await.unwrap().is_empty());
|
||||
|
||||
logstore
|
||||
.create_namespace(&Namespace::with_id(42))
|
||||
.create_namespace(&Provider::raft_engine_provider(42))
|
||||
.await
|
||||
.unwrap();
|
||||
let namespaces = logstore.list_namespaces().await.unwrap();
|
||||
assert_eq!(1, namespaces.len());
|
||||
assert_eq!(Namespace::with_id(42), namespaces[0]);
|
||||
assert_eq!(Provider::raft_engine_provider(42), namespaces[0]);
|
||||
|
||||
logstore
|
||||
.delete_namespace(&Namespace::with_id(42))
|
||||
.delete_namespace(&Provider::raft_engine_provider(42))
|
||||
.await
|
||||
.unwrap();
|
||||
assert!(logstore.list_namespaces().await.unwrap().is_empty());
|
||||
@@ -511,24 +529,25 @@ mod tests {
|
||||
.await
|
||||
.unwrap();
|
||||
|
||||
let namespace = Namespace::with_id(1);
|
||||
let namespace_id = 1;
|
||||
let cnt = 1024;
|
||||
for i in 0..cnt {
|
||||
let response = logstore
|
||||
.append(Entry::create(
|
||||
i,
|
||||
namespace.id,
|
||||
i.to_string().as_bytes().to_vec(),
|
||||
))
|
||||
.append(
|
||||
EntryImpl::create(i, namespace_id, i.to_string().as_bytes().to_vec()).into(),
|
||||
)
|
||||
.await
|
||||
.unwrap();
|
||||
assert_eq!(i, response.last_entry_id);
|
||||
}
|
||||
let mut entries = HashSet::with_capacity(1024);
|
||||
let mut s = logstore.read(&Namespace::with_id(1), 0).await.unwrap();
|
||||
let mut s = logstore
|
||||
.read(&Provider::raft_engine_provider(1), 0)
|
||||
.await
|
||||
.unwrap();
|
||||
while let Some(r) = s.next().await {
|
||||
let vec = r.unwrap();
|
||||
entries.extend(vec.into_iter().map(|e| e.id));
|
||||
entries.extend(vec.into_iter().map(|e| e.entry_id()));
|
||||
}
|
||||
assert_eq!((0..cnt).collect::<HashSet<_>>(), entries);
|
||||
}
|
||||
@@ -552,11 +571,11 @@ mod tests {
|
||||
.await
|
||||
.unwrap();
|
||||
assert!(logstore
|
||||
.append(Entry::create(1, 1, "1".as_bytes().to_vec()))
|
||||
.append(EntryImpl::create(1, 1, "1".as_bytes().to_vec()).into())
|
||||
.await
|
||||
.is_ok());
|
||||
let entries = logstore
|
||||
.read(&Namespace::with_id(1), 1)
|
||||
.read(&Provider::raft_engine_provider(1), 1)
|
||||
.await
|
||||
.unwrap()
|
||||
.collect::<Vec<_>>()
|
||||
@@ -572,11 +591,16 @@ mod tests {
|
||||
.await
|
||||
.unwrap();
|
||||
|
||||
let entries =
|
||||
collect_entries(logstore.read(&Namespace::with_id(1), 1).await.unwrap()).await;
|
||||
let entries = collect_entries(
|
||||
logstore
|
||||
.read(&Provider::raft_engine_provider(1), 1)
|
||||
.await
|
||||
.unwrap(),
|
||||
)
|
||||
.await;
|
||||
assert_eq!(1, entries.len());
|
||||
assert_eq!(1, entries[0].id);
|
||||
assert_eq!(1, entries[0].namespace_id);
|
||||
assert_eq!(1, entries[0].entry_id());
|
||||
assert_eq!(1, entries[0].region_id().as_u64());
|
||||
}
|
||||
|
||||
async fn wal_dir_usage(path: impl AsRef<str>) -> usize {
|
||||
@@ -615,14 +639,15 @@ mod tests {
|
||||
let dir = create_temp_dir("raft-engine-logstore-test");
|
||||
let logstore = new_test_log_store(&dir).await;
|
||||
|
||||
let namespace = Namespace::with_id(42);
|
||||
let namespace_id = 42;
|
||||
let namespace = Provider::raft_engine_provider(namespace_id);
|
||||
for id in 0..4096 {
|
||||
let entry = Entry::create(id, namespace.id(), [b'x'; 4096].to_vec());
|
||||
let entry = EntryImpl::create(id, namespace_id, [b'x'; 4096].to_vec()).into();
|
||||
let _ = logstore.append(entry).await.unwrap();
|
||||
}
|
||||
|
||||
let before_purge = wal_dir_usage(dir.path().to_str().unwrap()).await;
|
||||
logstore.obsolete(namespace, 4000).await.unwrap();
|
||||
logstore.obsolete(&namespace, 4000).await.unwrap();
|
||||
|
||||
tokio::time::sleep(Duration::from_secs(6)).await;
|
||||
let after_purge = wal_dir_usage(dir.path().to_str().unwrap()).await;
|
||||
@@ -639,19 +664,20 @@ mod tests {
|
||||
let dir = create_temp_dir("raft-engine-logstore-test");
|
||||
let logstore = new_test_log_store(&dir).await;
|
||||
|
||||
let namespace = Namespace::with_id(42);
|
||||
let namespace_id = 42;
|
||||
let namespace = Provider::raft_engine_provider(namespace_id);
|
||||
for id in 0..1024 {
|
||||
let entry = Entry::create(id, namespace.id(), [b'x'; 4096].to_vec());
|
||||
let entry = EntryImpl::create(id, namespace_id, [b'x'; 4096].to_vec()).into();
|
||||
let _ = logstore.append(entry).await.unwrap();
|
||||
}
|
||||
|
||||
logstore.obsolete(namespace.clone(), 100).await.unwrap();
|
||||
assert_eq!(101, logstore.engine.first_index(namespace.id).unwrap());
|
||||
logstore.obsolete(&namespace, 100).await.unwrap();
|
||||
assert_eq!(101, logstore.engine.first_index(namespace_id).unwrap());
|
||||
|
||||
let res = logstore.read(&namespace, 100).await.unwrap();
|
||||
let mut vec = collect_entries(res).await;
|
||||
vec.sort_by(|a, b| a.id.partial_cmp(&b.id).unwrap());
|
||||
assert_eq!(101, vec.first().unwrap().id);
|
||||
vec.sort_by(|a, b| a.entry_id().partial_cmp(&b.entry_id()).unwrap());
|
||||
assert_eq!(101, vec.first().unwrap().entry_id());
|
||||
}
|
||||
|
||||
#[tokio::test]
|
||||
@@ -663,14 +689,14 @@ mod tests {
|
||||
let entries = (0..8)
|
||||
.flat_map(|ns_id| {
|
||||
let data = [ns_id as u8].repeat(4096);
|
||||
(0..16).map(move |idx| Entry::create(idx, ns_id, data.clone()))
|
||||
(0..16).map(move |idx| EntryImpl::create(idx, ns_id, data.clone()).into())
|
||||
})
|
||||
.collect();
|
||||
|
||||
logstore.append_batch(entries).await.unwrap();
|
||||
for ns_id in 0..8 {
|
||||
let namespace = Namespace::with_id(ns_id);
|
||||
let (first, last) = logstore.span(&namespace);
|
||||
let namespace = &RaftEngineProvider::new(ns_id);
|
||||
let (first, last) = logstore.span(namespace);
|
||||
assert_eq!(0, first.unwrap());
|
||||
assert_eq!(15, last.unwrap());
|
||||
}
|
||||
@@ -681,19 +707,24 @@ mod tests {
|
||||
common_telemetry::init_default_ut_logging();
|
||||
let dir = create_temp_dir("logstore-append-batch-test");
|
||||
let logstore = new_test_log_store(&dir).await;
|
||||
|
||||
let entries = vec![
|
||||
Entry::create(0, 0, [b'0'; 4096].to_vec()),
|
||||
Entry::create(1, 0, [b'0'; 4096].to_vec()),
|
||||
Entry::create(0, 1, [b'1'; 4096].to_vec()),
|
||||
Entry::create(2, 0, [b'0'; 4096].to_vec()),
|
||||
Entry::create(1, 1, [b'1'; 4096].to_vec()),
|
||||
EntryImpl::create(0, 0, [b'0'; 4096].to_vec()).into(),
|
||||
EntryImpl::create(1, 0, [b'0'; 4096].to_vec()).into(),
|
||||
EntryImpl::create(0, 1, [b'1'; 4096].to_vec()).into(),
|
||||
EntryImpl::create(2, 0, [b'0'; 4096].to_vec()).into(),
|
||||
EntryImpl::create(1, 1, [b'1'; 4096].to_vec()).into(),
|
||||
];
|
||||
|
||||
logstore.append_batch(entries).await.unwrap();
|
||||
|
||||
assert_eq!((Some(0), Some(2)), logstore.span(&Namespace::with_id(0)));
|
||||
assert_eq!((Some(0), Some(1)), logstore.span(&Namespace::with_id(1)));
|
||||
assert_eq!(
|
||||
(Some(0), Some(2)),
|
||||
logstore.span(&RaftEngineProvider::new(0))
|
||||
);
|
||||
assert_eq!(
|
||||
(Some(0), Some(1)),
|
||||
logstore.span(&RaftEngineProvider::new(1))
|
||||
);
|
||||
}
|
||||
|
||||
#[tokio::test]
|
||||
@@ -704,21 +735,21 @@ mod tests {
|
||||
|
||||
let entries = vec![
|
||||
// Entry[0] from region 0.
|
||||
Entry::create(0, 0, [b'0'; 4096].to_vec()),
|
||||
EntryImpl::create(0, 0, [b'0'; 4096].to_vec()).into(),
|
||||
// Entry[0] from region 1.
|
||||
Entry::create(0, 1, [b'1'; 4096].to_vec()),
|
||||
EntryImpl::create(0, 1, [b'1'; 4096].to_vec()).into(),
|
||||
// Entry[1] from region 1.
|
||||
Entry::create(1, 0, [b'1'; 4096].to_vec()),
|
||||
EntryImpl::create(1, 0, [b'1'; 4096].to_vec()).into(),
|
||||
// Entry[1] from region 0.
|
||||
Entry::create(1, 1, [b'0'; 4096].to_vec()),
|
||||
EntryImpl::create(1, 1, [b'0'; 4096].to_vec()).into(),
|
||||
// Entry[2] from region 2.
|
||||
Entry::create(2, 2, [b'2'; 4096].to_vec()),
|
||||
EntryImpl::create(2, 2, [b'2'; 4096].to_vec()).into(),
|
||||
];
|
||||
|
||||
// Ensure the last entry id returned for each region is the expected one.
|
||||
let last_entry_ids = logstore.append_batch(entries).await.unwrap().last_entry_ids;
|
||||
assert_eq!(last_entry_ids[&0], 1);
|
||||
assert_eq!(last_entry_ids[&1], 1);
|
||||
assert_eq!(last_entry_ids[&2], 2);
|
||||
assert_eq!(last_entry_ids[&(0.into())], 1);
|
||||
assert_eq!(last_entry_ids[&(1.into())], 1);
|
||||
assert_eq!(last_entry_ids[&(2.into())], 2);
|
||||
}
|
||||
}
|
||||
|
||||
@@ -27,6 +27,6 @@ tonic.workspace = true
|
||||
datatypes.workspace = true
|
||||
futures = "0.3"
|
||||
meta-srv = { workspace = true, features = ["mock"] }
|
||||
tower = "0.4"
|
||||
tower.workspace = true
|
||||
tracing = "0.1"
|
||||
tracing-subscriber = { version = "0.3", features = ["env-filter"] }
|
||||
|
||||
@@ -55,7 +55,7 @@ tokio.workspace = true
|
||||
tokio-stream = { workspace = true, features = ["net"] }
|
||||
toml.workspace = true
|
||||
tonic.workspace = true
|
||||
tower = "0.4"
|
||||
tower.workspace = true
|
||||
typetag = "0.2"
|
||||
url = "2.3"
|
||||
|
||||
|
||||
@@ -13,6 +13,7 @@
|
||||
// limitations under the License.
|
||||
|
||||
mod alter;
|
||||
mod catchup;
|
||||
mod close;
|
||||
mod create;
|
||||
mod drop;
|
||||
@@ -33,6 +34,7 @@ use common_error::ext::{BoxedError, ErrorExt};
|
||||
use common_error::status_code::StatusCode;
|
||||
use common_recordbatch::SendableRecordBatchStream;
|
||||
use mito2::engine::MitoEngine;
|
||||
use snafu::ResultExt;
|
||||
use store_api::metadata::RegionMetadataRef;
|
||||
use store_api::metric_engine_consts::METRIC_ENGINE_NAME;
|
||||
use store_api::region_engine::{
|
||||
@@ -43,7 +45,7 @@ use store_api::storage::{RegionId, ScanRequest};
|
||||
|
||||
use self::state::MetricEngineState;
|
||||
use crate::data_region::DataRegion;
|
||||
use crate::error::{Result, UnsupportedRegionRequestSnafu};
|
||||
use crate::error::{self, Result, UnsupportedRegionRequestSnafu};
|
||||
use crate::metadata_region::MetadataRegion;
|
||||
use crate::utils;
|
||||
|
||||
@@ -143,12 +145,34 @@ impl RegionEngine for MetricEngine {
|
||||
.alter_region(region_id, alter, &mut extension_return_value)
|
||||
.await
|
||||
}
|
||||
RegionRequest::Delete(_)
|
||||
| RegionRequest::Flush(_)
|
||||
| RegionRequest::Compact(_)
|
||||
| RegionRequest::Truncate(_) => UnsupportedRegionRequestSnafu { request }.fail(),
|
||||
// It always Ok(0), all data is the latest.
|
||||
RegionRequest::Catchup(_) => Ok(0),
|
||||
RegionRequest::Flush(_) => {
|
||||
if self.inner.is_physical_region(region_id) {
|
||||
self.inner
|
||||
.mito
|
||||
.handle_request(region_id, request)
|
||||
.await
|
||||
.context(error::MitoFlushOperationSnafu)
|
||||
.map(|response| response.affected_rows)
|
||||
} else {
|
||||
UnsupportedRegionRequestSnafu { request }.fail()
|
||||
}
|
||||
}
|
||||
RegionRequest::Compact(_) => {
|
||||
if self.inner.is_physical_region(region_id) {
|
||||
self.inner
|
||||
.mito
|
||||
.handle_request(region_id, request)
|
||||
.await
|
||||
.context(error::MitoFlushOperationSnafu)
|
||||
.map(|response| response.affected_rows)
|
||||
} else {
|
||||
UnsupportedRegionRequestSnafu { request }.fail()
|
||||
}
|
||||
}
|
||||
RegionRequest::Delete(_) | RegionRequest::Truncate(_) => {
|
||||
UnsupportedRegionRequestSnafu { request }.fail()
|
||||
}
|
||||
RegionRequest::Catchup(ref req) => self.inner.catchup_region(region_id, *req).await,
|
||||
};
|
||||
|
||||
result.map_err(BoxedError::new).map(|rows| RegionResponse {
|
||||
@@ -178,9 +202,9 @@ impl RegionEngine for MetricEngine {
|
||||
/// Retrieves region's disk usage.
|
||||
///
|
||||
/// Note: Returns `None` if it's a logical region.
|
||||
async fn region_disk_usage(&self, region_id: RegionId) -> Option<i64> {
|
||||
fn region_disk_usage(&self, region_id: RegionId) -> Option<i64> {
|
||||
if self.inner.is_physical_region(region_id) {
|
||||
self.inner.mito.region_disk_usage(region_id).await
|
||||
self.inner.mito.region_disk_usage(region_id)
|
||||
} else {
|
||||
None
|
||||
}
|
||||
@@ -359,15 +383,7 @@ mod test {
|
||||
let logical_region_id = env.default_logical_region_id();
|
||||
let physical_region_id = env.default_physical_region_id();
|
||||
|
||||
assert!(env
|
||||
.metric()
|
||||
.region_disk_usage(logical_region_id)
|
||||
.await
|
||||
.is_none());
|
||||
assert!(env
|
||||
.metric()
|
||||
.region_disk_usage(physical_region_id)
|
||||
.await
|
||||
.is_some());
|
||||
assert!(env.metric().region_disk_usage(logical_region_id).is_none());
|
||||
assert!(env.metric().region_disk_usage(physical_region_id).is_some());
|
||||
}
|
||||
}
|
||||
|
||||
61
src/metric-engine/src/engine/catchup.rs
Normal file
61
src/metric-engine/src/engine/catchup.rs
Normal file
@@ -0,0 +1,61 @@
|
||||
// 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 snafu::ResultExt;
|
||||
use store_api::region_engine::RegionEngine;
|
||||
use store_api::region_request::{AffectedRows, RegionCatchupRequest, RegionRequest};
|
||||
use store_api::storage::RegionId;
|
||||
|
||||
use crate::engine::MetricEngineInner;
|
||||
use crate::error::{MitoCatchupOperationSnafu, Result, UnsupportedRegionRequestSnafu};
|
||||
use crate::utils;
|
||||
|
||||
impl MetricEngineInner {
|
||||
pub async fn catchup_region(
|
||||
&self,
|
||||
region_id: RegionId,
|
||||
req: RegionCatchupRequest,
|
||||
) -> Result<AffectedRows> {
|
||||
if !self.is_physical_region(region_id) {
|
||||
return UnsupportedRegionRequestSnafu {
|
||||
request: RegionRequest::Catchup(req),
|
||||
}
|
||||
.fail();
|
||||
}
|
||||
let metadata_region_id = utils::to_metadata_region_id(region_id);
|
||||
// TODO(weny): improve the catchup, we can read the wal entries only once.
|
||||
self.mito
|
||||
.handle_request(
|
||||
metadata_region_id,
|
||||
RegionRequest::Catchup(RegionCatchupRequest {
|
||||
set_writable: req.set_writable,
|
||||
entry_id: None,
|
||||
}),
|
||||
)
|
||||
.await
|
||||
.context(MitoCatchupOperationSnafu)?;
|
||||
|
||||
self.mito
|
||||
.handle_request(
|
||||
region_id,
|
||||
RegionRequest::Catchup(RegionCatchupRequest {
|
||||
set_writable: req.set_writable,
|
||||
entry_id: req.entry_id,
|
||||
}),
|
||||
)
|
||||
.await
|
||||
.context(MitoCatchupOperationSnafu)
|
||||
.map(|response| response.affected_rows)
|
||||
}
|
||||
}
|
||||
@@ -121,6 +121,27 @@ pub enum Error {
|
||||
location: Location,
|
||||
},
|
||||
|
||||
#[snafu(display("Mito flush operation fails"))]
|
||||
MitoFlushOperation {
|
||||
source: BoxedError,
|
||||
#[snafu(implicit)]
|
||||
location: Location,
|
||||
},
|
||||
|
||||
#[snafu(display("Mito catchup operation fails"))]
|
||||
MitoCatchupOperation {
|
||||
source: BoxedError,
|
||||
#[snafu(implicit)]
|
||||
location: Location,
|
||||
},
|
||||
|
||||
#[snafu(display("Mito compact operation fails"))]
|
||||
MitoCompactOperation {
|
||||
source: BoxedError,
|
||||
#[snafu(implicit)]
|
||||
location: Location,
|
||||
},
|
||||
|
||||
#[snafu(display("Failed to collect record batch stream"))]
|
||||
CollectRecordBatchStream {
|
||||
source: common_recordbatch::error::Error,
|
||||
@@ -267,7 +288,10 @@ impl ErrorExt for Error {
|
||||
| OpenMitoRegion { source, .. }
|
||||
| CloseMitoRegion { source, .. }
|
||||
| MitoReadOperation { source, .. }
|
||||
| MitoWriteOperation { source, .. } => source.status_code(),
|
||||
| MitoWriteOperation { source, .. }
|
||||
| MitoCatchupOperation { source, .. }
|
||||
| MitoFlushOperation { source, .. }
|
||||
| MitoCompactOperation { source, .. } => source.status_code(),
|
||||
|
||||
CollectRecordBatchStream { source, .. } => source.status_code(),
|
||||
|
||||
|
||||
19
src/mito2/src/cache/file_cache.rs
vendored
19
src/mito2/src/cache/file_cache.rs
vendored
@@ -112,6 +112,10 @@ impl FileCache {
|
||||
self.memory_index.insert(key, value).await;
|
||||
}
|
||||
|
||||
pub(crate) async fn get(&self, key: IndexKey) -> Option<IndexValue> {
|
||||
self.memory_index.get(&key).await
|
||||
}
|
||||
|
||||
/// Reads a file from the cache.
|
||||
pub(crate) async fn reader(&self, key: IndexKey) -> Option<Reader> {
|
||||
// We must use `get()` to update the estimator of the cache.
|
||||
@@ -372,7 +376,6 @@ fn parse_index_key(name: &str) -> Option<IndexKey> {
|
||||
#[cfg(test)]
|
||||
mod tests {
|
||||
use common_test_util::temp_dir::create_temp_dir;
|
||||
use futures::AsyncReadExt;
|
||||
use object_store::services::Fs;
|
||||
|
||||
use super::*;
|
||||
@@ -451,10 +454,9 @@ mod tests {
|
||||
.await;
|
||||
|
||||
// Read file content.
|
||||
let mut reader = cache.reader(key).await.unwrap();
|
||||
let mut buf = String::new();
|
||||
reader.read_to_string(&mut buf).await.unwrap();
|
||||
assert_eq!("hello", buf);
|
||||
let reader = cache.reader(key).await.unwrap();
|
||||
let buf = reader.read(..).await.unwrap().to_vec();
|
||||
assert_eq!("hello", String::from_utf8(buf).unwrap());
|
||||
|
||||
// Get weighted size.
|
||||
cache.memory_index.run_pending_tasks().await;
|
||||
@@ -549,10 +551,9 @@ mod tests {
|
||||
|
||||
for (i, file_id) in file_ids.iter().enumerate() {
|
||||
let key = IndexKey::new(region_id, *file_id, file_type);
|
||||
let mut reader = cache.reader(key).await.unwrap();
|
||||
let mut buf = String::new();
|
||||
reader.read_to_string(&mut buf).await.unwrap();
|
||||
assert_eq!(i.to_string(), buf);
|
||||
let reader = cache.reader(key).await.unwrap();
|
||||
let buf = reader.read(..).await.unwrap().to_vec();
|
||||
assert_eq!(i.to_string(), String::from_utf8(buf).unwrap());
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
25
src/mito2/src/cache/write_cache.rs
vendored
25
src/mito2/src/cache/write_cache.rs
vendored
@@ -19,6 +19,7 @@ use std::time::Duration;
|
||||
|
||||
use common_base::readable_size::ReadableSize;
|
||||
use common_telemetry::{debug, info};
|
||||
use futures::AsyncWriteExt;
|
||||
use object_store::manager::ObjectStoreManagerRef;
|
||||
use object_store::ObjectStore;
|
||||
use snafu::ResultExt;
|
||||
@@ -175,19 +176,27 @@ impl WriteCache {
|
||||
}])
|
||||
.start_timer();
|
||||
|
||||
let cached_value = self
|
||||
.file_cache
|
||||
.local_store()
|
||||
.stat(&cache_path)
|
||||
.await
|
||||
.context(error::OpenDalSnafu)?;
|
||||
let reader = self
|
||||
.file_cache
|
||||
.local_store()
|
||||
.reader(&cache_path)
|
||||
.await
|
||||
.context(error::OpenDalSnafu)?;
|
||||
.context(error::OpenDalSnafu)?
|
||||
.into_futures_async_read(0..cached_value.content_length());
|
||||
|
||||
let mut writer = remote_store
|
||||
.writer_with(upload_path)
|
||||
.buffer(DEFAULT_WRITE_BUFFER_SIZE.as_bytes() as usize)
|
||||
.chunk(DEFAULT_WRITE_BUFFER_SIZE.as_bytes() as usize)
|
||||
.concurrent(DEFAULT_WRITE_CONCURRENCY)
|
||||
.await
|
||||
.context(error::OpenDalSnafu)?;
|
||||
.context(error::OpenDalSnafu)?
|
||||
.into_futures_async_write();
|
||||
|
||||
let bytes_written =
|
||||
futures::io::copy(reader, &mut writer)
|
||||
@@ -199,7 +208,11 @@ impl WriteCache {
|
||||
})?;
|
||||
|
||||
// Must close to upload all data.
|
||||
writer.close().await.context(error::OpenDalSnafu)?;
|
||||
writer.close().await.context(error::UploadSnafu {
|
||||
region_id,
|
||||
file_id,
|
||||
file_type,
|
||||
})?;
|
||||
|
||||
UPLOAD_BYTES_TOTAL.inc_by(bytes_written);
|
||||
|
||||
@@ -315,7 +328,7 @@ mod tests {
|
||||
.read(&write_cache.file_cache.cache_file_path(key))
|
||||
.await
|
||||
.unwrap();
|
||||
assert_eq!(remote_data, cache_data);
|
||||
assert_eq!(remote_data.to_vec(), cache_data.to_vec());
|
||||
|
||||
// Check write cache contains the index key
|
||||
let index_key = IndexKey::new(region_id, file_id, FileType::Puffin);
|
||||
@@ -326,7 +339,7 @@ mod tests {
|
||||
.read(&write_cache.file_cache.cache_file_path(index_key))
|
||||
.await
|
||||
.unwrap();
|
||||
assert_eq!(remote_index_data, cache_index_data);
|
||||
assert_eq!(remote_index_data.to_vec(), cache_index_data.to_vec());
|
||||
}
|
||||
|
||||
#[tokio::test]
|
||||
|
||||
@@ -656,7 +656,7 @@ mod tests {
|
||||
let file_metas: Vec<_> = data.version.ssts.levels()[0]
|
||||
.files
|
||||
.values()
|
||||
.map(|file| file.meta())
|
||||
.map(|file| file.meta_ref().clone())
|
||||
.collect();
|
||||
|
||||
// 5 files for next compaction and removes old files.
|
||||
|
||||
@@ -110,7 +110,7 @@ impl CompactionTaskImpl {
|
||||
Vec::with_capacity(self.outputs.iter().map(|o| o.inputs.len()).sum());
|
||||
|
||||
for output in self.outputs.drain(..) {
|
||||
compacted_inputs.extend(output.inputs.iter().map(FileHandle::meta));
|
||||
compacted_inputs.extend(output.inputs.iter().map(|f| f.meta_ref().clone()));
|
||||
|
||||
info!(
|
||||
"Compaction region {} output [{}]-> {}",
|
||||
@@ -229,7 +229,7 @@ impl CompactionTaskImpl {
|
||||
return Err(e);
|
||||
}
|
||||
};
|
||||
deleted.extend(self.expired_ssts.iter().map(FileHandle::meta));
|
||||
deleted.extend(self.expired_ssts.iter().map(|f| f.meta_ref().clone()));
|
||||
let merge_time = merge_timer.stop_and_record();
|
||||
info!(
|
||||
"Compacted SST files, region_id: {}, input: {:?}, output: {:?}, window: {:?}, waiter_num: {}, merge_time: {}s",
|
||||
|
||||
@@ -104,15 +104,20 @@ impl MitoEngine {
|
||||
self.inner.workers.is_region_exists(region_id)
|
||||
}
|
||||
|
||||
/// Returns true if the specific region exists.
|
||||
pub fn is_region_opening(&self, region_id: RegionId) -> bool {
|
||||
self.inner.workers.is_region_opening(region_id)
|
||||
}
|
||||
|
||||
/// Returns the region disk/memory usage information.
|
||||
pub async fn get_region_usage(&self, region_id: RegionId) -> Result<RegionUsage> {
|
||||
pub fn get_region_usage(&self, region_id: RegionId) -> Result<RegionUsage> {
|
||||
let region = self
|
||||
.inner
|
||||
.workers
|
||||
.get_region(region_id)
|
||||
.context(RegionNotFoundSnafu { region_id })?;
|
||||
|
||||
Ok(region.region_usage().await)
|
||||
Ok(region.region_usage())
|
||||
}
|
||||
|
||||
/// Handle substrait query and return a stream of record batches
|
||||
@@ -121,12 +126,11 @@ impl MitoEngine {
|
||||
&self,
|
||||
region_id: RegionId,
|
||||
request: ScanRequest,
|
||||
) -> std::result::Result<SendableRecordBatchStream, BoxedError> {
|
||||
) -> Result<SendableRecordBatchStream, BoxedError> {
|
||||
self.scanner(region_id, request)
|
||||
.map_err(BoxedError::new)?
|
||||
.scan()
|
||||
.await
|
||||
.map_err(BoxedError::new)
|
||||
}
|
||||
|
||||
/// Returns a scanner to scan for `request`.
|
||||
@@ -364,10 +368,9 @@ impl RegionEngine for MitoEngine {
|
||||
self.inner.stop().await.map_err(BoxedError::new)
|
||||
}
|
||||
|
||||
async fn region_disk_usage(&self, region_id: RegionId) -> Option<i64> {
|
||||
fn region_disk_usage(&self, region_id: RegionId) -> Option<i64> {
|
||||
let size = self
|
||||
.get_region_usage(region_id)
|
||||
.await
|
||||
.map(|usage| usage.disk_usage())
|
||||
.ok()?;
|
||||
size.try_into().ok()
|
||||
|
||||
@@ -524,7 +524,7 @@ async fn test_region_usage() {
|
||||
.unwrap();
|
||||
// region is empty now, check manifest size
|
||||
let region = engine.get_region(region_id).unwrap();
|
||||
let region_stat = region.region_usage().await;
|
||||
let region_stat = region.region_usage();
|
||||
assert_eq!(region_stat.manifest_usage, 686);
|
||||
|
||||
// put some rows
|
||||
@@ -535,7 +535,7 @@ async fn test_region_usage() {
|
||||
|
||||
put_rows(&engine, region_id, rows).await;
|
||||
|
||||
let region_stat = region.region_usage().await;
|
||||
let region_stat = region.region_usage();
|
||||
assert!(region_stat.wal_usage > 0);
|
||||
|
||||
// delete some rows
|
||||
@@ -545,13 +545,13 @@ async fn test_region_usage() {
|
||||
};
|
||||
delete_rows(&engine, region_id, rows).await;
|
||||
|
||||
let region_stat = region.region_usage().await;
|
||||
let region_stat = region.region_usage();
|
||||
assert!(region_stat.wal_usage > 0);
|
||||
|
||||
// flush region
|
||||
flush_region(&engine, region_id, None).await;
|
||||
|
||||
let region_stat = region.region_usage().await;
|
||||
let region_stat = region.region_usage();
|
||||
assert_eq!(region_stat.sst_usage, 3010);
|
||||
|
||||
// region total usage
|
||||
|
||||
@@ -24,8 +24,10 @@ use store_api::region_request::{
|
||||
RegionCloseRequest, RegionOpenRequest, RegionPutRequest, RegionRequest,
|
||||
};
|
||||
use store_api::storage::{RegionId, ScanRequest};
|
||||
use tokio::sync::oneshot;
|
||||
|
||||
use crate::config::MitoConfig;
|
||||
use crate::error;
|
||||
use crate::test_util::{
|
||||
build_rows, flush_region, put_rows, reopen_region, rows_schema, CreateRequestBuilder, TestEnv,
|
||||
};
|
||||
@@ -319,3 +321,87 @@ async fn test_open_region_skip_wal_replay() {
|
||||
+-------+---------+---------------------+";
|
||||
assert_eq!(expected, batches.pretty_print().unwrap());
|
||||
}
|
||||
|
||||
#[tokio::test]
|
||||
async fn test_open_region_wait_for_opening_region_ok() {
|
||||
let mut env = TestEnv::with_prefix("wait-for-opening-region-ok");
|
||||
let engine = env.create_engine(MitoConfig::default()).await;
|
||||
let region_id = RegionId::new(1, 1);
|
||||
let worker = engine.inner.workers.worker(region_id);
|
||||
let (tx, rx) = oneshot::channel();
|
||||
let opening_regions = worker.opening_regions().clone();
|
||||
opening_regions.insert_sender(region_id, tx.into());
|
||||
assert!(engine.is_region_opening(region_id));
|
||||
|
||||
let handle_open = tokio::spawn(async move {
|
||||
engine
|
||||
.handle_request(
|
||||
region_id,
|
||||
RegionRequest::Open(RegionOpenRequest {
|
||||
engine: String::new(),
|
||||
region_dir: "empty".to_string(),
|
||||
options: HashMap::default(),
|
||||
skip_wal_replay: false,
|
||||
}),
|
||||
)
|
||||
.await
|
||||
});
|
||||
|
||||
// Wait for conditions
|
||||
while opening_regions.sender_len(region_id) != 2 {
|
||||
tokio::time::sleep(Duration::from_millis(100)).await;
|
||||
}
|
||||
|
||||
let senders = opening_regions.remove_sender(region_id);
|
||||
for sender in senders {
|
||||
sender.send(Ok(0));
|
||||
}
|
||||
|
||||
assert_eq!(handle_open.await.unwrap().unwrap().affected_rows, 0);
|
||||
assert_eq!(rx.await.unwrap().unwrap(), 0);
|
||||
}
|
||||
|
||||
#[tokio::test]
|
||||
async fn test_open_region_wait_for_opening_region_err() {
|
||||
let mut env = TestEnv::with_prefix("wait-for-opening-region-err");
|
||||
let engine = env.create_engine(MitoConfig::default()).await;
|
||||
let region_id = RegionId::new(1, 1);
|
||||
let worker = engine.inner.workers.worker(region_id);
|
||||
let (tx, rx) = oneshot::channel();
|
||||
let opening_regions = worker.opening_regions().clone();
|
||||
opening_regions.insert_sender(region_id, tx.into());
|
||||
assert!(engine.is_region_opening(region_id));
|
||||
|
||||
let handle_open = tokio::spawn(async move {
|
||||
engine
|
||||
.handle_request(
|
||||
region_id,
|
||||
RegionRequest::Open(RegionOpenRequest {
|
||||
engine: String::new(),
|
||||
region_dir: "empty".to_string(),
|
||||
options: HashMap::default(),
|
||||
skip_wal_replay: false,
|
||||
}),
|
||||
)
|
||||
.await
|
||||
});
|
||||
|
||||
// Wait for conditions
|
||||
while opening_regions.sender_len(region_id) != 2 {
|
||||
tokio::time::sleep(Duration::from_millis(100)).await;
|
||||
}
|
||||
|
||||
let senders = opening_regions.remove_sender(region_id);
|
||||
for sender in senders {
|
||||
sender.send(Err(error::RegionNotFoundSnafu { region_id }.build()));
|
||||
}
|
||||
|
||||
assert_eq!(
|
||||
handle_open.await.unwrap().unwrap_err().status_code(),
|
||||
StatusCode::RegionNotFound
|
||||
);
|
||||
assert_eq!(
|
||||
rx.await.unwrap().unwrap_err().status_code(),
|
||||
StatusCode::RegionNotFound
|
||||
);
|
||||
}
|
||||
|
||||
@@ -27,6 +27,7 @@ use datatypes::prelude::ConcreteDataType;
|
||||
use object_store::ErrorKind;
|
||||
use prost::{DecodeError, EncodeError};
|
||||
use snafu::{Location, Snafu};
|
||||
use store_api::logstore::provider::Provider;
|
||||
use store_api::manifest::ManifestVersion;
|
||||
use store_api::storage::RegionId;
|
||||
|
||||
@@ -226,6 +227,14 @@ pub enum Error {
|
||||
source: datatypes::Error,
|
||||
},
|
||||
|
||||
#[snafu(display("Failed to build entry, region_id: {}", region_id))]
|
||||
BuildEntry {
|
||||
region_id: RegionId,
|
||||
#[snafu(implicit)]
|
||||
location: Location,
|
||||
source: BoxedError,
|
||||
},
|
||||
|
||||
#[snafu(display("Failed to encode WAL entry, region_id: {}", region_id))]
|
||||
EncodeWal {
|
||||
region_id: RegionId,
|
||||
@@ -242,17 +251,9 @@ pub enum Error {
|
||||
source: BoxedError,
|
||||
},
|
||||
|
||||
#[snafu(display("Failed to read WAL, region_id: {}", region_id))]
|
||||
#[snafu(display("Failed to read WAL, provider: {}", provider))]
|
||||
ReadWal {
|
||||
region_id: RegionId,
|
||||
#[snafu(implicit)]
|
||||
location: Location,
|
||||
source: BoxedError,
|
||||
},
|
||||
|
||||
#[snafu(display("Failed to read WAL, topic: {}", topic))]
|
||||
ReadKafkaWal {
|
||||
topic: String,
|
||||
provider: Provider,
|
||||
#[snafu(implicit)]
|
||||
location: Location,
|
||||
source: BoxedError,
|
||||
@@ -636,6 +637,13 @@ pub enum Error {
|
||||
unexpected_entry_id: u64,
|
||||
},
|
||||
|
||||
#[snafu(display("Read the corrupted log entry, region_id: {}", region_id))]
|
||||
CorruptedEntry {
|
||||
region_id: RegionId,
|
||||
#[snafu(implicit)]
|
||||
location: Location,
|
||||
},
|
||||
|
||||
#[snafu(display(
|
||||
"Failed to upload file, region_id: {}, file_id: {}, file_type: {:?}",
|
||||
region_id,
|
||||
@@ -722,6 +730,13 @@ pub enum Error {
|
||||
#[snafu(implicit)]
|
||||
location: Location,
|
||||
},
|
||||
|
||||
#[snafu(display("Failed to open region"))]
|
||||
OpenRegion {
|
||||
#[snafu(implicit)]
|
||||
location: Location,
|
||||
source: Arc<Error>,
|
||||
},
|
||||
}
|
||||
|
||||
pub type Result<T, E = Error> = std::result::Result<T, E>;
|
||||
@@ -750,7 +765,6 @@ impl ErrorExt for Error {
|
||||
| ReadParquet { .. }
|
||||
| WriteWal { .. }
|
||||
| ReadWal { .. }
|
||||
| ReadKafkaWal { .. }
|
||||
| DeleteWal { .. } => StatusCode::StorageUnavailable,
|
||||
CompressObject { .. }
|
||||
| DecompressObject { .. }
|
||||
@@ -782,7 +796,10 @@ impl ErrorExt for Error {
|
||||
| WorkerStopped { .. }
|
||||
| Recv { .. }
|
||||
| EncodeWal { .. }
|
||||
| DecodeWal { .. } => StatusCode::Internal,
|
||||
| DecodeWal { .. }
|
||||
| BuildEntry { .. } => StatusCode::Internal,
|
||||
OpenRegion { source, .. } => source.status_code(),
|
||||
|
||||
WriteBuffer { source, .. } => source.status_code(),
|
||||
WriteGroup { source, .. } => source.status_code(),
|
||||
FieldTypeMismatch { source, .. } => source.status_code(),
|
||||
@@ -829,7 +846,9 @@ impl ErrorExt for Error {
|
||||
|
||||
Upload { .. } => StatusCode::StorageUnavailable,
|
||||
BiError { .. } => StatusCode::Internal,
|
||||
EncodeMemtable { .. } | ReadDataPart { .. } => StatusCode::Internal,
|
||||
EncodeMemtable { .. } | ReadDataPart { .. } | CorruptedEntry { .. } => {
|
||||
StatusCode::Internal
|
||||
}
|
||||
ChecksumMismatch { .. } => StatusCode::Unexpected,
|
||||
RegionStopped { .. } => StatusCode::RegionNotReady,
|
||||
TimeRangePredicateOverflow { .. } => StatusCode::InvalidArguments,
|
||||
|
||||
@@ -12,6 +12,7 @@
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
use std::sync::atomic::AtomicU64;
|
||||
use std::sync::Arc;
|
||||
|
||||
use common_datasource::compression::CompressionType;
|
||||
@@ -121,12 +122,17 @@ pub struct RegionManifestManager {
|
||||
|
||||
impl RegionManifestManager {
|
||||
/// Constructs a region's manifest and persist it.
|
||||
pub async fn new(metadata: RegionMetadataRef, options: RegionManifestOptions) -> Result<Self> {
|
||||
pub async fn new(
|
||||
metadata: RegionMetadataRef,
|
||||
options: RegionManifestOptions,
|
||||
total_manifest_size: Arc<AtomicU64>,
|
||||
) -> Result<Self> {
|
||||
// construct storage
|
||||
let mut store = ManifestObjectStore::new(
|
||||
&options.manifest_dir,
|
||||
options.object_store.clone(),
|
||||
options.compress_type,
|
||||
total_manifest_size,
|
||||
);
|
||||
|
||||
info!(
|
||||
@@ -168,7 +174,10 @@ impl RegionManifestManager {
|
||||
/// Opens an existing manifest.
|
||||
///
|
||||
/// Returns `Ok(None)` if no such manifest.
|
||||
pub async fn open(options: RegionManifestOptions) -> Result<Option<Self>> {
|
||||
pub async fn open(
|
||||
options: RegionManifestOptions,
|
||||
total_manifest_size: Arc<AtomicU64>,
|
||||
) -> Result<Option<Self>> {
|
||||
let _t = MANIFEST_OP_ELAPSED
|
||||
.with_label_values(&["open"])
|
||||
.start_timer();
|
||||
@@ -178,6 +187,7 @@ impl RegionManifestManager {
|
||||
&options.manifest_dir,
|
||||
options.object_store.clone(),
|
||||
options.compress_type,
|
||||
total_manifest_size,
|
||||
);
|
||||
|
||||
// recover from storage
|
||||
|
||||
@@ -15,6 +15,8 @@
|
||||
use std::collections::HashMap;
|
||||
use std::iter::Iterator;
|
||||
use std::str::FromStr;
|
||||
use std::sync::atomic::{AtomicU64, Ordering};
|
||||
use std::sync::Arc;
|
||||
|
||||
use common_datasource::compression::CompressionType;
|
||||
use common_telemetry::debug;
|
||||
@@ -133,15 +135,22 @@ pub struct ManifestObjectStore {
|
||||
path: String,
|
||||
/// Stores the size of each manifest file.
|
||||
manifest_size_map: HashMap<FileKey, u64>,
|
||||
total_manifest_size: Arc<AtomicU64>,
|
||||
}
|
||||
|
||||
impl ManifestObjectStore {
|
||||
pub fn new(path: &str, object_store: ObjectStore, compress_type: CompressionType) -> Self {
|
||||
pub fn new(
|
||||
path: &str,
|
||||
object_store: ObjectStore,
|
||||
compress_type: CompressionType,
|
||||
total_manifest_size: Arc<AtomicU64>,
|
||||
) -> Self {
|
||||
Self {
|
||||
object_store,
|
||||
compress_type,
|
||||
path: util::normalize_dir(path),
|
||||
manifest_size_map: HashMap::new(),
|
||||
total_manifest_size,
|
||||
}
|
||||
}
|
||||
|
||||
@@ -338,10 +347,9 @@ impl ManifestObjectStore {
|
||||
// delete manifest sizes
|
||||
for (_, is_checkpoint, version) in &del_entries {
|
||||
if *is_checkpoint {
|
||||
self.manifest_size_map
|
||||
.remove(&FileKey::Checkpoint(*version));
|
||||
self.unset_file_size(&FileKey::Checkpoint(*version));
|
||||
} else {
|
||||
self.manifest_size_map.remove(&FileKey::Delta(*version));
|
||||
self.unset_file_size(&FileKey::Delta(*version));
|
||||
}
|
||||
}
|
||||
|
||||
@@ -489,7 +497,7 @@ impl ManifestObjectStore {
|
||||
}
|
||||
};
|
||||
|
||||
let checkpoint_metadata = CheckpointMetadata::decode(&last_checkpoint_data)?;
|
||||
let checkpoint_metadata = CheckpointMetadata::decode(&last_checkpoint_data.to_vec())?;
|
||||
|
||||
debug!(
|
||||
"Load checkpoint in path: {}, metadata: {:?}",
|
||||
@@ -501,7 +509,11 @@ impl ManifestObjectStore {
|
||||
|
||||
#[cfg(test)]
|
||||
pub async fn read_file(&self, path: &str) -> Result<Vec<u8>> {
|
||||
self.object_store.read(path).await.context(OpenDalSnafu)
|
||||
self.object_store
|
||||
.read(path)
|
||||
.await
|
||||
.context(OpenDalSnafu)
|
||||
.map(|v| v.to_vec())
|
||||
}
|
||||
|
||||
#[cfg(test)]
|
||||
@@ -560,12 +572,28 @@ impl ManifestObjectStore {
|
||||
/// Set the size of the delta file by delta version.
|
||||
pub(crate) fn set_delta_file_size(&mut self, version: ManifestVersion, size: u64) {
|
||||
self.manifest_size_map.insert(FileKey::Delta(version), size);
|
||||
self.inc_total_manifest_size(size);
|
||||
}
|
||||
|
||||
/// Set the size of the checkpoint file by checkpoint version.
|
||||
pub(crate) fn set_checkpoint_file_size(&mut self, version: ManifestVersion, size: u64) {
|
||||
self.manifest_size_map
|
||||
.insert(FileKey::Checkpoint(version), size);
|
||||
self.inc_total_manifest_size(size);
|
||||
}
|
||||
|
||||
fn unset_file_size(&mut self, key: &FileKey) {
|
||||
if let Some(val) = self.manifest_size_map.remove(key) {
|
||||
self.dec_total_manifest_size(val);
|
||||
}
|
||||
}
|
||||
|
||||
fn inc_total_manifest_size(&self, val: u64) {
|
||||
self.total_manifest_size.fetch_add(val, Ordering::Relaxed);
|
||||
}
|
||||
|
||||
fn dec_total_manifest_size(&self, val: u64) {
|
||||
self.total_manifest_size.fetch_sub(val, Ordering::Relaxed);
|
||||
}
|
||||
}
|
||||
|
||||
@@ -606,7 +634,12 @@ mod tests {
|
||||
let mut builder = Fs::default();
|
||||
let _ = builder.root(&tmp_dir.path().to_string_lossy());
|
||||
let object_store = ObjectStore::new(builder).unwrap().finish();
|
||||
ManifestObjectStore::new("/", object_store, CompressionType::Uncompressed)
|
||||
ManifestObjectStore::new(
|
||||
"/",
|
||||
object_store,
|
||||
CompressionType::Uncompressed,
|
||||
Default::default(),
|
||||
)
|
||||
}
|
||||
|
||||
fn new_checkpoint_metadata_with_version(version: ManifestVersion) -> CheckpointMetadata {
|
||||
|
||||
@@ -50,6 +50,7 @@ use crate::error::{
|
||||
ComputeArrowSnafu, ComputeVectorSnafu, ConvertVectorSnafu, InvalidBatchSnafu, Result,
|
||||
};
|
||||
use crate::memtable::BoxedBatchIterator;
|
||||
use crate::sst::parquet::reader::RowGroupReader;
|
||||
|
||||
/// Storage internal representation of a batch of rows for a primary key (time series).
|
||||
///
|
||||
@@ -699,6 +700,8 @@ pub enum Source {
|
||||
Iter(BoxedBatchIterator),
|
||||
/// Source from a [BoxedBatchStream].
|
||||
Stream(BoxedBatchStream),
|
||||
/// Source from a [RowGroupReader].
|
||||
RowGroupReader(RowGroupReader),
|
||||
}
|
||||
|
||||
impl Source {
|
||||
@@ -708,6 +711,7 @@ impl Source {
|
||||
Source::Reader(reader) => reader.next_batch().await,
|
||||
Source::Iter(iter) => iter.next().transpose(),
|
||||
Source::Stream(stream) => stream.try_next().await,
|
||||
Source::RowGroupReader(reader) => reader.next_batch().await,
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@@ -32,10 +32,8 @@ use crate::row_converter::{McmpRowCodec, RowCodec, SortField};
|
||||
pub struct CompatReader<R> {
|
||||
/// Underlying reader.
|
||||
reader: R,
|
||||
/// Optional primary key adapter.
|
||||
compat_pk: Option<CompatPrimaryKey>,
|
||||
/// Optional fields adapter.
|
||||
compat_fields: Option<CompatFields>,
|
||||
/// Helper to compat batches.
|
||||
compat: CompatBatch,
|
||||
}
|
||||
|
||||
impl<R> CompatReader<R> {
|
||||
@@ -48,13 +46,9 @@ impl<R> CompatReader<R> {
|
||||
reader_meta: RegionMetadataRef,
|
||||
reader: R,
|
||||
) -> Result<CompatReader<R>> {
|
||||
let compat_pk = may_compat_primary_key(mapper.metadata(), &reader_meta)?;
|
||||
let compat_fields = may_compat_fields(mapper, &reader_meta)?;
|
||||
|
||||
Ok(CompatReader {
|
||||
reader,
|
||||
compat_pk,
|
||||
compat_fields,
|
||||
compat: CompatBatch::new(mapper, reader_meta)?,
|
||||
})
|
||||
}
|
||||
}
|
||||
@@ -66,6 +60,36 @@ impl<R: BatchReader> BatchReader for CompatReader<R> {
|
||||
return Ok(None);
|
||||
};
|
||||
|
||||
batch = self.compat.compat_batch(batch)?;
|
||||
|
||||
Ok(Some(batch))
|
||||
}
|
||||
}
|
||||
|
||||
/// A helper struct to adapt schema of the batch to an expected schema.
|
||||
pub(crate) struct CompatBatch {
|
||||
/// Optional primary key adapter.
|
||||
compat_pk: Option<CompatPrimaryKey>,
|
||||
/// Optional fields adapter.
|
||||
compat_fields: Option<CompatFields>,
|
||||
}
|
||||
|
||||
impl CompatBatch {
|
||||
/// Creates a new [CompatBatch].
|
||||
/// - `mapper` is built from the metadata users expect to see.
|
||||
/// - `reader_meta` is the metadata of the input reader.
|
||||
pub(crate) fn new(mapper: &ProjectionMapper, reader_meta: RegionMetadataRef) -> Result<Self> {
|
||||
let compat_pk = may_compat_primary_key(mapper.metadata(), &reader_meta)?;
|
||||
let compat_fields = may_compat_fields(mapper, &reader_meta)?;
|
||||
|
||||
Ok(Self {
|
||||
compat_pk,
|
||||
compat_fields,
|
||||
})
|
||||
}
|
||||
|
||||
/// Adapts the `batch` to the expected schema.
|
||||
pub(crate) fn compat_batch(&self, mut batch: Batch) -> Result<Batch> {
|
||||
if let Some(compat_pk) = &self.compat_pk {
|
||||
batch = compat_pk.compat(batch)?;
|
||||
}
|
||||
@@ -73,7 +97,7 @@ impl<R: BatchReader> BatchReader for CompatReader<R> {
|
||||
batch = compat_fields.compat(batch);
|
||||
}
|
||||
|
||||
Ok(Some(batch))
|
||||
Ok(batch)
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@@ -14,9 +14,11 @@
|
||||
|
||||
//! Scans a region according to the scan request.
|
||||
|
||||
use std::fmt;
|
||||
use std::sync::Arc;
|
||||
use std::time::Instant;
|
||||
|
||||
use common_error::ext::BoxedError;
|
||||
use common_recordbatch::SendableRecordBatchStream;
|
||||
use common_telemetry::{debug, error, warn};
|
||||
use common_time::range::TimestampRange;
|
||||
@@ -32,15 +34,16 @@ use crate::cache::CacheManagerRef;
|
||||
use crate::error::Result;
|
||||
use crate::memtable::MemtableRef;
|
||||
use crate::metrics::READ_SST_COUNT;
|
||||
use crate::read::compat::CompatReader;
|
||||
use crate::read::compat::{CompatBatch, CompatReader};
|
||||
use crate::read::projection::ProjectionMapper;
|
||||
use crate::read::seq_scan::SeqScan;
|
||||
use crate::read::unordered_scan::UnorderedScan;
|
||||
use crate::read::{compat, Batch, Source};
|
||||
use crate::region::version::VersionRef;
|
||||
use crate::sst::file::FileHandle;
|
||||
use crate::sst::file::{FileHandle, FileMeta};
|
||||
use crate::sst::index::applier::builder::SstIndexApplierBuilder;
|
||||
use crate::sst::index::applier::SstIndexApplierRef;
|
||||
use crate::sst::parquet::file_range::FileRange;
|
||||
|
||||
/// A scanner scans a region and returns a [SendableRecordBatchStream].
|
||||
pub(crate) enum Scanner {
|
||||
@@ -51,20 +54,24 @@ pub(crate) enum Scanner {
|
||||
}
|
||||
|
||||
impl Scanner {
|
||||
/// Returns a [SendableRecordBatchStream] to retrieve scan results.
|
||||
pub(crate) async fn scan(&self) -> Result<SendableRecordBatchStream> {
|
||||
/// Returns a [SendableRecordBatchStream] to retrieve scan results from all partitions.
|
||||
pub(crate) async fn scan(&self) -> Result<SendableRecordBatchStream, BoxedError> {
|
||||
match self {
|
||||
Scanner::Seq(seq_scan) => seq_scan.build_stream().await,
|
||||
Scanner::Seq(seq_scan) => seq_scan.build_stream().await.map_err(BoxedError::new),
|
||||
Scanner::Unordered(unordered_scan) => unordered_scan.build_stream().await,
|
||||
}
|
||||
}
|
||||
|
||||
/// Returns a [RegionScanner] to scan the region.
|
||||
pub(crate) async fn region_scanner(&self) -> Result<RegionScannerRef> {
|
||||
let stream = self.scan().await?;
|
||||
let scanner = SinglePartitionScanner::new(stream);
|
||||
|
||||
Ok(Arc::new(scanner))
|
||||
pub(crate) async fn region_scanner(self) -> Result<RegionScannerRef> {
|
||||
match self {
|
||||
Scanner::Seq(seq_scan) => {
|
||||
let stream = seq_scan.build_stream().await?;
|
||||
let scanner = Arc::new(SinglePartitionScanner::new(stream));
|
||||
Ok(scanner)
|
||||
}
|
||||
Scanner::Unordered(unordered_scan) => Ok(Arc::new(unordered_scan)),
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@@ -222,9 +229,7 @@ impl ScanRegion {
|
||||
/// Unordered scan.
|
||||
pub(crate) fn unordered_scan(self) -> Result<UnorderedScan> {
|
||||
let input = self.scan_input(true)?;
|
||||
let scan = UnorderedScan::new(input);
|
||||
|
||||
Ok(scan)
|
||||
Ok(UnorderedScan::new(input))
|
||||
}
|
||||
|
||||
#[cfg(test)]
|
||||
@@ -386,7 +391,7 @@ pub(crate) struct ScanInput {
|
||||
/// Time range filter for time index.
|
||||
time_range: Option<TimestampRange>,
|
||||
/// Predicate to push down.
|
||||
predicate: Option<Predicate>,
|
||||
pub(crate) predicate: Option<Predicate>,
|
||||
/// Memtables to scan.
|
||||
pub(crate) memtables: Vec<MemtableRef>,
|
||||
/// Handles to SST files to scan.
|
||||
@@ -498,7 +503,6 @@ impl ScanInput {
|
||||
}
|
||||
|
||||
/// Sets whether to remove deletion markers during scan.
|
||||
#[allow(unused)]
|
||||
#[must_use]
|
||||
pub(crate) fn with_filter_deleted(mut self, filter_deleted: bool) -> Self {
|
||||
self.filter_deleted = filter_deleted;
|
||||
@@ -572,6 +576,61 @@ impl ScanInput {
|
||||
Ok(sources)
|
||||
}
|
||||
|
||||
/// Prunes file ranges to scan and adds them tothe `collector`.
|
||||
pub(crate) async fn prune_file_ranges(
|
||||
&self,
|
||||
collector: &mut impl FileRangeCollector,
|
||||
) -> Result<()> {
|
||||
for file in &self.files {
|
||||
let res = self
|
||||
.access_layer
|
||||
.read_sst(file.clone())
|
||||
.predicate(self.predicate.clone())
|
||||
.time_range(self.time_range)
|
||||
.projection(Some(self.mapper.column_ids().to_vec()))
|
||||
.cache(self.cache_manager.clone())
|
||||
.index_applier(self.index_applier.clone())
|
||||
.expected_metadata(Some(self.mapper.metadata().clone()))
|
||||
.build_reader_input()
|
||||
.await;
|
||||
let (mut file_range_ctx, row_groups) = match res {
|
||||
Ok(x) => x,
|
||||
Err(e) => {
|
||||
if e.is_object_not_found() && self.ignore_file_not_found {
|
||||
error!(e; "File to scan does not exist, region_id: {}, file: {}", file.region_id(), file.file_id());
|
||||
continue;
|
||||
} else {
|
||||
return Err(e);
|
||||
}
|
||||
}
|
||||
};
|
||||
if !compat::has_same_columns(
|
||||
self.mapper.metadata(),
|
||||
file_range_ctx.read_format().metadata(),
|
||||
) {
|
||||
// They have different schema. We need to adapt the batch first so the
|
||||
// mapper can convert it.
|
||||
let compat = CompatBatch::new(
|
||||
&self.mapper,
|
||||
file_range_ctx.read_format().metadata().clone(),
|
||||
)?;
|
||||
file_range_ctx.set_compat_batch(Some(compat));
|
||||
}
|
||||
// Build ranges from row groups.
|
||||
let file_range_ctx = Arc::new(file_range_ctx);
|
||||
let file_ranges = row_groups
|
||||
.into_iter()
|
||||
.map(|(row_group_idx, row_selection)| {
|
||||
FileRange::new(file_range_ctx.clone(), row_group_idx, row_selection)
|
||||
});
|
||||
collector.append_file_ranges(file.meta_ref(), file_ranges);
|
||||
}
|
||||
|
||||
READ_SST_COUNT.observe(self.files.len() as f64);
|
||||
|
||||
Ok(())
|
||||
}
|
||||
|
||||
/// Scans the input source in another task and sends batches to the sender.
|
||||
pub(crate) fn spawn_scan_task(
|
||||
&self,
|
||||
@@ -620,3 +679,35 @@ impl ScanInput {
|
||||
self.files.iter().map(|file| file.file_id()).collect()
|
||||
}
|
||||
}
|
||||
|
||||
/// A partition of a scanner to read.
|
||||
/// It contains memtables and file ranges to scan.
|
||||
#[derive(Default)]
|
||||
pub(crate) struct ScanPart {
|
||||
/// Memtables to scan.
|
||||
/// We scan the whole memtable now. We might scan a range of the memtable in the future.
|
||||
pub(crate) memtables: Vec<MemtableRef>,
|
||||
/// File ranges to scan.
|
||||
pub(crate) file_ranges: Vec<FileRange>,
|
||||
}
|
||||
|
||||
impl fmt::Debug for ScanPart {
|
||||
fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
|
||||
write!(
|
||||
f,
|
||||
"ScanPart({} memtables, {} file ranges)",
|
||||
self.memtables.len(),
|
||||
self.file_ranges.len()
|
||||
)
|
||||
}
|
||||
}
|
||||
|
||||
/// A trait to collect file ranges to scan.
|
||||
pub(crate) trait FileRangeCollector {
|
||||
/// Appends file ranges from the **same file** to the collector.
|
||||
fn append_file_ranges(
|
||||
&mut self,
|
||||
file_meta: &FileMeta,
|
||||
file_ranges: impl Iterator<Item = FileRange>,
|
||||
);
|
||||
}
|
||||
|
||||
@@ -14,148 +14,101 @@
|
||||
|
||||
//! Unordered scanner.
|
||||
|
||||
use std::fmt;
|
||||
use std::sync::Arc;
|
||||
use std::time::{Duration, Instant};
|
||||
|
||||
use async_stream::try_stream;
|
||||
use async_stream::{stream, try_stream};
|
||||
use common_error::ext::BoxedError;
|
||||
use common_recordbatch::error::ExternalSnafu;
|
||||
use common_recordbatch::{RecordBatch, RecordBatchStreamWrapper, SendableRecordBatchStream};
|
||||
use common_telemetry::debug;
|
||||
use datafusion::physical_plan::{DisplayAs, DisplayFormatType};
|
||||
use datatypes::schema::SchemaRef;
|
||||
use futures::StreamExt;
|
||||
use snafu::ResultExt;
|
||||
use tokio::sync::{mpsc, Semaphore};
|
||||
use tokio_stream::wrappers::ReceiverStream;
|
||||
use store_api::region_engine::{RegionScanner, ScannerPartitioning, ScannerProperties};
|
||||
use tokio::sync::Mutex;
|
||||
|
||||
use crate::cache::CacheManager;
|
||||
use crate::error::Result;
|
||||
use crate::memtable::MemtableRef;
|
||||
use crate::metrics::{READ_BATCHES_RETURN, READ_ROWS_RETURN, READ_STAGE_ELAPSED};
|
||||
use crate::read::compat::CompatBatch;
|
||||
use crate::read::projection::ProjectionMapper;
|
||||
use crate::read::scan_region::ScanInput;
|
||||
use crate::read::scan_region::{FileRangeCollector, ScanInput, ScanPart};
|
||||
use crate::read::Source;
|
||||
use crate::sst::file::FileMeta;
|
||||
use crate::sst::parquet::file_range::FileRange;
|
||||
use crate::sst::parquet::reader::ReaderMetrics;
|
||||
|
||||
/// Scans a region without providing any output ordering guarantee.
|
||||
///
|
||||
/// Only an append only table should use this scanner.
|
||||
pub struct UnorderedScan {
|
||||
input: ScanInput,
|
||||
/// Properties of the scanner.
|
||||
properties: ScannerProperties,
|
||||
/// Context of streams.
|
||||
stream_ctx: Arc<StreamContext>,
|
||||
}
|
||||
|
||||
impl UnorderedScan {
|
||||
/// Creates a new [UnorderedScan].
|
||||
pub(crate) fn new(input: ScanInput) -> Self {
|
||||
Self { input }
|
||||
let query_start = input.query_start.unwrap_or_else(Instant::now);
|
||||
let prepare_scan_cost = query_start.elapsed();
|
||||
let properties =
|
||||
ScannerProperties::new(ScannerPartitioning::Unknown(input.parallelism.parallelism));
|
||||
|
||||
// Observes metrics.
|
||||
READ_STAGE_ELAPSED
|
||||
.with_label_values(&["prepare_scan"])
|
||||
.observe(prepare_scan_cost.as_secs_f64());
|
||||
|
||||
let stream_ctx = Arc::new(StreamContext {
|
||||
input,
|
||||
parts: Mutex::new(ScanPartList::default()),
|
||||
query_start,
|
||||
prepare_scan_cost,
|
||||
});
|
||||
|
||||
Self {
|
||||
properties,
|
||||
stream_ctx,
|
||||
}
|
||||
}
|
||||
|
||||
/// Scans the region and returns a stream.
|
||||
pub async fn build_stream(&self) -> Result<SendableRecordBatchStream> {
|
||||
let enable_parallel = self.enable_parallel_scan();
|
||||
if enable_parallel {
|
||||
self.scan_in_parallel().await
|
||||
} else {
|
||||
self.scan_sources().await
|
||||
}
|
||||
}
|
||||
|
||||
/// Scans all sources one by one.
|
||||
async fn scan_sources(&self) -> Result<SendableRecordBatchStream> {
|
||||
let mut metrics = Metrics::default();
|
||||
let build_start = Instant::now();
|
||||
let query_start = self.input.query_start.unwrap_or(build_start);
|
||||
metrics.prepare_scan_cost = query_start.elapsed();
|
||||
|
||||
// Scans all memtables and SSTs.
|
||||
let sources = self.input.build_sources().await?;
|
||||
metrics.build_source_cost = build_start.elapsed();
|
||||
Self::observe_metrics_on_start(&metrics);
|
||||
|
||||
let mapper = self.input.mapper.clone();
|
||||
let cache_manager = self.input.cache_manager.clone();
|
||||
let stream = try_stream! {
|
||||
for mut source in sources {
|
||||
let cache = cache_manager.as_deref();
|
||||
while let Some(batch) = Self::fetch_from_source(&mut source, &mapper, cache, &mut metrics).await? {
|
||||
metrics.num_batches += 1;
|
||||
metrics.num_rows += batch.num_rows();
|
||||
yield batch;
|
||||
pub(crate) async fn build_stream(&self) -> Result<SendableRecordBatchStream, BoxedError> {
|
||||
let part_num = self.properties.partitioning().num_partitions();
|
||||
let streams = (0..part_num)
|
||||
.map(|i| self.scan_partition(i))
|
||||
.collect::<Result<Vec<_>, BoxedError>>()?;
|
||||
let stream = stream! {
|
||||
for mut stream in streams {
|
||||
while let Some(rb) = stream.next().await {
|
||||
yield rb;
|
||||
}
|
||||
}
|
||||
|
||||
metrics.total_cost = query_start.elapsed();
|
||||
Self::observe_metrics_on_finish(&metrics);
|
||||
debug!("Unordered scan finished, region_id: {}, metrics: {:?}", mapper.metadata().region_id, metrics);
|
||||
};
|
||||
let stream = Box::pin(RecordBatchStreamWrapper::new(
|
||||
self.input.mapper.output_schema(),
|
||||
self.schema(),
|
||||
Box::pin(stream),
|
||||
));
|
||||
|
||||
Ok(stream)
|
||||
}
|
||||
|
||||
/// Scans all sources in parallel.
|
||||
async fn scan_in_parallel(&self) -> Result<SendableRecordBatchStream> {
|
||||
debug_assert!(self.input.parallelism.allow_parallel_scan());
|
||||
|
||||
let mut metrics = Metrics::default();
|
||||
let build_start = Instant::now();
|
||||
let query_start = self.input.query_start.unwrap_or(build_start);
|
||||
metrics.prepare_scan_cost = query_start.elapsed();
|
||||
|
||||
// Scans all memtables and SSTs.
|
||||
let sources = self.input.build_sources().await?;
|
||||
metrics.build_source_cost = build_start.elapsed();
|
||||
Self::observe_metrics_on_start(&metrics);
|
||||
|
||||
let (sender, receiver) = mpsc::channel(self.input.parallelism.channel_size);
|
||||
let semaphore = Arc::new(Semaphore::new(self.input.parallelism.parallelism));
|
||||
// Spawn a task for each source.
|
||||
for source in sources {
|
||||
self.input
|
||||
.spawn_scan_task(source, semaphore.clone(), sender.clone());
|
||||
}
|
||||
let stream = Box::pin(ReceiverStream::new(receiver));
|
||||
|
||||
let mapper = self.input.mapper.clone();
|
||||
let cache_manager = self.input.cache_manager.clone();
|
||||
// For simplicity, we wrap the receiver into a stream to reuse code. We can use the channel directly if it
|
||||
// becomes a bottleneck.
|
||||
let mut source = Source::Stream(stream);
|
||||
let stream = try_stream! {
|
||||
let cache = cache_manager.as_deref();
|
||||
while let Some(batch) = Self::fetch_from_source(&mut source, &mapper, cache, &mut metrics).await? {
|
||||
metrics.num_batches += 1;
|
||||
metrics.num_rows += batch.num_rows();
|
||||
yield batch;
|
||||
}
|
||||
|
||||
metrics.total_cost = query_start.elapsed();
|
||||
Self::observe_metrics_on_finish(&metrics);
|
||||
debug!("Unordered scan in parallel finished, region_id: {}, metrics: {:?}", mapper.metadata().region_id, metrics);
|
||||
};
|
||||
let stream = Box::pin(RecordBatchStreamWrapper::new(
|
||||
self.input.mapper.output_schema(),
|
||||
Box::pin(stream),
|
||||
));
|
||||
|
||||
Ok(stream)
|
||||
}
|
||||
|
||||
/// Returns whether to scan in parallel.
|
||||
fn enable_parallel_scan(&self) -> bool {
|
||||
self.input.parallelism.allow_parallel_scan()
|
||||
&& (self.input.files.len() + self.input.memtables.len()) > 1
|
||||
}
|
||||
|
||||
/// Fetch a batch from the source and convert it into a record batch.
|
||||
async fn fetch_from_source(
|
||||
source: &mut Source,
|
||||
mapper: &ProjectionMapper,
|
||||
cache: Option<&CacheManager>,
|
||||
compat_batch: Option<&CompatBatch>,
|
||||
metrics: &mut Metrics,
|
||||
) -> common_recordbatch::error::Result<Option<RecordBatch>> {
|
||||
let start = Instant::now();
|
||||
|
||||
let Some(batch) = source
|
||||
let Some(mut batch) = source
|
||||
.next_batch()
|
||||
.await
|
||||
.map_err(BoxedError::new)
|
||||
@@ -166,6 +119,13 @@ impl UnorderedScan {
|
||||
return Ok(None);
|
||||
};
|
||||
|
||||
if let Some(compat) = compat_batch {
|
||||
batch = compat
|
||||
.compat_batch(batch)
|
||||
.map_err(BoxedError::new)
|
||||
.context(ExternalSnafu)?;
|
||||
}
|
||||
|
||||
let convert_start = Instant::now();
|
||||
let record_batch = mapper.convert(&batch, cache)?;
|
||||
metrics.convert_cost += convert_start.elapsed();
|
||||
@@ -174,15 +134,6 @@ impl UnorderedScan {
|
||||
Ok(Some(record_batch))
|
||||
}
|
||||
|
||||
fn observe_metrics_on_start(metrics: &Metrics) {
|
||||
READ_STAGE_ELAPSED
|
||||
.with_label_values(&["prepare_scan"])
|
||||
.observe(metrics.prepare_scan_cost.as_secs_f64());
|
||||
READ_STAGE_ELAPSED
|
||||
.with_label_values(&["build_source"])
|
||||
.observe(metrics.build_source_cost.as_secs_f64());
|
||||
}
|
||||
|
||||
fn observe_metrics_on_finish(metrics: &Metrics) {
|
||||
READ_STAGE_ELAPSED
|
||||
.with_label_values(&["convert_rb"])
|
||||
@@ -198,21 +149,168 @@ impl UnorderedScan {
|
||||
}
|
||||
}
|
||||
|
||||
impl RegionScanner for UnorderedScan {
|
||||
fn properties(&self) -> &ScannerProperties {
|
||||
&self.properties
|
||||
}
|
||||
|
||||
fn schema(&self) -> SchemaRef {
|
||||
self.stream_ctx.input.mapper.output_schema()
|
||||
}
|
||||
|
||||
fn scan_partition(&self, partition: usize) -> Result<SendableRecordBatchStream, BoxedError> {
|
||||
let mut metrics = Metrics {
|
||||
prepare_scan_cost: self.stream_ctx.prepare_scan_cost,
|
||||
..Default::default()
|
||||
};
|
||||
let stream_ctx = self.stream_ctx.clone();
|
||||
let stream = try_stream! {
|
||||
let mut parts = stream_ctx.parts.lock().await;
|
||||
parts
|
||||
.maybe_init_parts(&stream_ctx.input, &mut metrics)
|
||||
.await
|
||||
.map_err(BoxedError::new)
|
||||
.context(ExternalSnafu)?;
|
||||
let Some(part) = parts.get_part(partition) else {
|
||||
return;
|
||||
};
|
||||
|
||||
let mapper = &stream_ctx.input.mapper;
|
||||
let memtable_sources = part
|
||||
.memtables
|
||||
.iter()
|
||||
.map(|mem| {
|
||||
let iter = mem.iter(
|
||||
Some(mapper.column_ids()),
|
||||
stream_ctx.input.predicate.clone(),
|
||||
)?;
|
||||
Ok(Source::Iter(iter))
|
||||
})
|
||||
.collect::<Result<Vec<_>>>()
|
||||
.map_err(BoxedError::new)
|
||||
.context(ExternalSnafu)?;
|
||||
let query_start = stream_ctx.query_start;
|
||||
let cache = stream_ctx.input.cache_manager.as_deref();
|
||||
// Scans memtables first.
|
||||
for mut source in memtable_sources {
|
||||
while let Some(batch) = Self::fetch_from_source(&mut source, mapper, cache, None, &mut metrics).await? {
|
||||
metrics.num_batches += 1;
|
||||
metrics.num_rows += batch.num_rows();
|
||||
yield batch;
|
||||
}
|
||||
}
|
||||
// Then scans file ranges.
|
||||
let mut reader_metrics = ReaderMetrics::default();
|
||||
for file_range in &part.file_ranges {
|
||||
let reader = file_range.reader().await.map_err(BoxedError::new).context(ExternalSnafu)?;
|
||||
let compat_batch = file_range.compat_batch();
|
||||
let mut source = Source::RowGroupReader(reader);
|
||||
while let Some(batch) = Self::fetch_from_source(&mut source, mapper, cache, compat_batch, &mut metrics).await? {
|
||||
metrics.num_batches += 1;
|
||||
metrics.num_rows += batch.num_rows();
|
||||
yield batch;
|
||||
}
|
||||
if let Source::RowGroupReader(reader) = source {
|
||||
reader_metrics.merge_from(reader.metrics());
|
||||
}
|
||||
}
|
||||
|
||||
metrics.total_cost = query_start.elapsed();
|
||||
Self::observe_metrics_on_finish(&metrics);
|
||||
debug!(
|
||||
"Unordered scan partition {} finished, region_id: {}, metrics: {:?}, reader_metrics: {:?}",
|
||||
partition, mapper.metadata().region_id, metrics, reader_metrics
|
||||
);
|
||||
};
|
||||
let stream = Box::pin(RecordBatchStreamWrapper::new(
|
||||
self.stream_ctx.input.mapper.output_schema(),
|
||||
Box::pin(stream),
|
||||
));
|
||||
|
||||
Ok(stream)
|
||||
}
|
||||
}
|
||||
|
||||
impl DisplayAs for UnorderedScan {
|
||||
fn fmt_as(&self, _t: DisplayFormatType, f: &mut fmt::Formatter) -> fmt::Result {
|
||||
write!(f, "UnorderedScan: [{:?}]", self.stream_ctx.parts)
|
||||
}
|
||||
}
|
||||
|
||||
impl fmt::Debug for UnorderedScan {
|
||||
fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
|
||||
f.debug_struct("UnorderedScan")
|
||||
.field("parts", &self.stream_ctx.parts)
|
||||
.field("prepare_scan_cost", &self.stream_ctx.prepare_scan_cost)
|
||||
.finish()
|
||||
}
|
||||
}
|
||||
|
||||
#[cfg(test)]
|
||||
impl UnorderedScan {
|
||||
/// Returns the input.
|
||||
pub(crate) fn input(&self) -> &ScanInput {
|
||||
&self.input
|
||||
&self.stream_ctx.input
|
||||
}
|
||||
}
|
||||
|
||||
/// List of [ScanPart]s.
|
||||
#[derive(Debug, Default)]
|
||||
struct ScanPartList(Option<Vec<ScanPart>>);
|
||||
|
||||
impl ScanPartList {
|
||||
/// Initializes parts if they are not built yet.
|
||||
async fn maybe_init_parts(&mut self, input: &ScanInput, metrics: &mut Metrics) -> Result<()> {
|
||||
if self.0.is_none() {
|
||||
let now = Instant::now();
|
||||
let mut distributor = UnorderedDistributor::default();
|
||||
input.prune_file_ranges(&mut distributor).await?;
|
||||
self.0 = Some(distributor.build_parts(&input.memtables, input.parallelism.parallelism));
|
||||
|
||||
metrics.build_parts_cost = now.elapsed();
|
||||
READ_STAGE_ELAPSED
|
||||
.with_label_values(&["build_parts"])
|
||||
.observe(metrics.build_parts_cost.as_secs_f64());
|
||||
}
|
||||
Ok(())
|
||||
}
|
||||
|
||||
/// Gets the part by index, returns None if the index is out of bound.
|
||||
/// # Panics
|
||||
/// Panics if parts are not initialized.
|
||||
fn get_part(&mut self, index: usize) -> Option<&ScanPart> {
|
||||
let parts = self.0.as_ref().unwrap();
|
||||
parts.get(index)
|
||||
}
|
||||
}
|
||||
|
||||
/// Context shared by different streams.
|
||||
/// It contains the input and distributes input to multiple parts
|
||||
/// to scan.
|
||||
struct StreamContext {
|
||||
/// Input memtables and files.
|
||||
input: ScanInput,
|
||||
/// Parts to scan.
|
||||
/// The scanner builds parts to scan from the input lazily.
|
||||
/// The mutex is used to ensure the parts are only built once.
|
||||
parts: Mutex<ScanPartList>,
|
||||
|
||||
// Metrics:
|
||||
/// The start time of the query.
|
||||
query_start: Instant,
|
||||
/// Time elapsed before creating the scanner.
|
||||
prepare_scan_cost: Duration,
|
||||
}
|
||||
|
||||
/// Metrics for [UnorderedScan].
|
||||
// We print all fields in logs so we disable the dead_code lint.
|
||||
#[allow(dead_code)]
|
||||
#[derive(Debug, Default)]
|
||||
struct Metrics {
|
||||
/// Duration to prepare the scan task.
|
||||
prepare_scan_cost: Duration,
|
||||
/// Duration to build sources.
|
||||
build_source_cost: Duration,
|
||||
/// Duration to build parts.
|
||||
build_parts_cost: Duration,
|
||||
/// Duration to scan data.
|
||||
scan_cost: Duration,
|
||||
/// Duration to convert batches.
|
||||
@@ -224,3 +322,66 @@ struct Metrics {
|
||||
/// Number of rows returned.
|
||||
num_rows: usize,
|
||||
}
|
||||
|
||||
/// Builds [ScanPart]s without preserving order. It distributes file ranges and memtables
|
||||
/// across partitions. Each partition scans a subset of memtables and file ranges. There
|
||||
/// is no output ordering guarantee of each partition.
|
||||
#[derive(Default)]
|
||||
struct UnorderedDistributor {
|
||||
file_ranges: Vec<FileRange>,
|
||||
}
|
||||
|
||||
impl FileRangeCollector for UnorderedDistributor {
|
||||
fn append_file_ranges(
|
||||
&mut self,
|
||||
_file_meta: &FileMeta,
|
||||
file_ranges: impl Iterator<Item = FileRange>,
|
||||
) {
|
||||
self.file_ranges.extend(file_ranges);
|
||||
}
|
||||
}
|
||||
|
||||
impl UnorderedDistributor {
|
||||
/// Distributes file ranges and memtables across partitions according to the `parallelism`.
|
||||
/// The output number of parts may be `<= parallelism`.
|
||||
fn build_parts(self, memtables: &[MemtableRef], parallelism: usize) -> Vec<ScanPart> {
|
||||
if parallelism <= 1 {
|
||||
// Returns a single part.
|
||||
let part = ScanPart {
|
||||
memtables: memtables.to_vec(),
|
||||
file_ranges: self.file_ranges,
|
||||
};
|
||||
return vec![part];
|
||||
}
|
||||
|
||||
let mems_per_part = ((memtables.len() + parallelism - 1) / parallelism).max(1);
|
||||
let ranges_per_part = ((self.file_ranges.len() + parallelism - 1) / parallelism).max(1);
|
||||
common_telemetry::debug!(
|
||||
"Parallel scan is enabled, parallelism: {}, {} memtables, {} file_ranges, mems_per_part: {}, ranges_per_part: {}",
|
||||
parallelism,
|
||||
memtables.len(),
|
||||
self.file_ranges.len(),
|
||||
mems_per_part,
|
||||
ranges_per_part
|
||||
);
|
||||
let mut scan_parts = memtables
|
||||
.chunks(mems_per_part)
|
||||
.map(|mems| ScanPart {
|
||||
memtables: mems.to_vec(),
|
||||
file_ranges: Vec::new(),
|
||||
})
|
||||
.collect::<Vec<_>>();
|
||||
for (i, ranges) in self.file_ranges.chunks(ranges_per_part).enumerate() {
|
||||
if i == scan_parts.len() {
|
||||
scan_parts.push(ScanPart {
|
||||
memtables: Vec::new(),
|
||||
file_ranges: ranges.to_vec(),
|
||||
});
|
||||
} else {
|
||||
scan_parts[i].file_ranges = ranges.to_vec();
|
||||
}
|
||||
}
|
||||
|
||||
scan_parts
|
||||
}
|
||||
}
|
||||
|
||||
@@ -18,14 +18,15 @@ pub(crate) mod opener;
|
||||
pub mod options;
|
||||
pub(crate) mod version;
|
||||
|
||||
use std::collections::hash_map::Entry;
|
||||
use std::collections::HashMap;
|
||||
use std::sync::atomic::{AtomicI64, Ordering};
|
||||
use std::sync::atomic::{AtomicI64, AtomicU64, Ordering};
|
||||
use std::sync::{Arc, RwLock};
|
||||
|
||||
use common_telemetry::{error, info, warn};
|
||||
use common_wal::options::WalOptions;
|
||||
use crossbeam_utils::atomic::AtomicCell;
|
||||
use snafu::{ensure, OptionExt};
|
||||
use store_api::logstore::provider::Provider;
|
||||
use store_api::metadata::RegionMetadataRef;
|
||||
use store_api::storage::RegionId;
|
||||
|
||||
@@ -35,7 +36,7 @@ use crate::manifest::action::{RegionMetaAction, RegionMetaActionList};
|
||||
use crate::manifest::manager::RegionManifestManager;
|
||||
use crate::memtable::MemtableBuilderRef;
|
||||
use crate::region::version::{VersionControlRef, VersionRef};
|
||||
use crate::request::OnFailure;
|
||||
use crate::request::{OnFailure, OptionOutputTx};
|
||||
use crate::sst::file_purger::FilePurgerRef;
|
||||
use crate::time_provider::TimeProviderRef;
|
||||
|
||||
@@ -97,14 +98,16 @@ pub(crate) struct MitoRegion {
|
||||
pub(crate) manifest_ctx: ManifestContextRef,
|
||||
/// SST file purger.
|
||||
pub(crate) file_purger: FilePurgerRef,
|
||||
/// Wal options of this region.
|
||||
pub(crate) wal_options: WalOptions,
|
||||
/// The provider of log store.
|
||||
pub(crate) provider: Provider,
|
||||
/// Last flush time in millis.
|
||||
last_flush_millis: AtomicI64,
|
||||
/// Provider to get current time.
|
||||
time_provider: TimeProviderRef,
|
||||
/// Memtable builder for the region.
|
||||
pub(crate) memtable_builder: MemtableBuilderRef,
|
||||
/// manifest stats
|
||||
stats: ManifestStats,
|
||||
}
|
||||
|
||||
pub(crate) type MitoRegionRef = Arc<MitoRegion>;
|
||||
@@ -168,10 +171,24 @@ impl MitoRegion {
|
||||
if writable {
|
||||
// Only sets the region to writable if it is read only.
|
||||
// This prevents others updating the manifest.
|
||||
let _ = self
|
||||
match self
|
||||
.manifest_ctx
|
||||
.state
|
||||
.compare_exchange(RegionState::ReadOnly, RegionState::Writable);
|
||||
.compare_exchange(RegionState::ReadOnly, RegionState::Writable)
|
||||
{
|
||||
Ok(state) => info!(
|
||||
"Set region {} to writable, previous state: {:?}",
|
||||
self.region_id, state
|
||||
),
|
||||
Err(state) => {
|
||||
if state != RegionState::Writable {
|
||||
warn!(
|
||||
"Failed to set region {} to writable, current state: {:?}",
|
||||
self.region_id, state
|
||||
)
|
||||
}
|
||||
}
|
||||
}
|
||||
} else {
|
||||
self.manifest_ctx.state.store(RegionState::ReadOnly);
|
||||
}
|
||||
@@ -218,7 +235,7 @@ impl MitoRegion {
|
||||
}
|
||||
|
||||
/// Returns the region usage in bytes.
|
||||
pub(crate) async fn region_usage(&self) -> RegionUsage {
|
||||
pub(crate) fn region_usage(&self) -> RegionUsage {
|
||||
let region_id = self.region_id;
|
||||
|
||||
let version = self.version();
|
||||
@@ -228,13 +245,7 @@ impl MitoRegion {
|
||||
let sst_usage = version.ssts.sst_usage();
|
||||
|
||||
let wal_usage = self.estimated_wal_usage(memtable_usage);
|
||||
|
||||
let manifest_usage = self
|
||||
.manifest_ctx
|
||||
.manifest_manager
|
||||
.read()
|
||||
.await
|
||||
.manifest_usage();
|
||||
let manifest_usage = self.stats.total_manifest_size();
|
||||
|
||||
RegionUsage {
|
||||
region_id,
|
||||
@@ -457,6 +468,72 @@ impl RegionMap {
|
||||
|
||||
pub(crate) type RegionMapRef = Arc<RegionMap>;
|
||||
|
||||
/// Opening regions
|
||||
#[derive(Debug, Default)]
|
||||
pub(crate) struct OpeningRegions {
|
||||
regions: RwLock<HashMap<RegionId, Vec<OptionOutputTx>>>,
|
||||
}
|
||||
|
||||
impl OpeningRegions {
|
||||
/// Registers `sender` for an opening region; Otherwise, it returns `None`.
|
||||
pub(crate) fn wait_for_opening_region(
|
||||
&self,
|
||||
region_id: RegionId,
|
||||
sender: OptionOutputTx,
|
||||
) -> Option<OptionOutputTx> {
|
||||
let mut regions = self.regions.write().unwrap();
|
||||
match regions.entry(region_id) {
|
||||
Entry::Occupied(mut senders) => {
|
||||
senders.get_mut().push(sender);
|
||||
None
|
||||
}
|
||||
Entry::Vacant(_) => Some(sender),
|
||||
}
|
||||
}
|
||||
|
||||
/// Returns true if the region exists.
|
||||
pub(crate) fn is_region_exists(&self, region_id: RegionId) -> bool {
|
||||
let regions = self.regions.read().unwrap();
|
||||
regions.contains_key(®ion_id)
|
||||
}
|
||||
|
||||
/// Inserts a new region into the map.
|
||||
pub(crate) fn insert_sender(&self, region: RegionId, sender: OptionOutputTx) {
|
||||
let mut regions = self.regions.write().unwrap();
|
||||
regions.insert(region, vec![sender]);
|
||||
}
|
||||
|
||||
/// Remove region by id.
|
||||
pub(crate) fn remove_sender(&self, region_id: RegionId) -> Vec<OptionOutputTx> {
|
||||
let mut regions = self.regions.write().unwrap();
|
||||
regions.remove(®ion_id).unwrap_or_default()
|
||||
}
|
||||
|
||||
#[cfg(test)]
|
||||
pub(crate) fn sender_len(&self, region_id: RegionId) -> usize {
|
||||
let regions = self.regions.read().unwrap();
|
||||
if let Some(senders) = regions.get(®ion_id) {
|
||||
senders.len()
|
||||
} else {
|
||||
0
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
pub(crate) type OpeningRegionsRef = Arc<OpeningRegions>;
|
||||
|
||||
/// Manifest stats.
|
||||
#[derive(Default, Debug, Clone)]
|
||||
pub(crate) struct ManifestStats {
|
||||
total_manifest_size: Arc<AtomicU64>,
|
||||
}
|
||||
|
||||
impl ManifestStats {
|
||||
fn total_manifest_size(&self) -> u64 {
|
||||
self.total_manifest_size.load(Ordering::Relaxed)
|
||||
}
|
||||
}
|
||||
|
||||
#[cfg(test)]
|
||||
mod tests {
|
||||
use crossbeam_utils::atomic::AtomicCell;
|
||||
|
||||
@@ -24,6 +24,7 @@ use futures::StreamExt;
|
||||
use object_store::manager::ObjectStoreManagerRef;
|
||||
use object_store::util::{join_dir, normalize_dir};
|
||||
use snafu::{ensure, OptionExt};
|
||||
use store_api::logstore::provider::Provider;
|
||||
use store_api::logstore::LogStore;
|
||||
use store_api::metadata::{ColumnMetadata, RegionMetadata};
|
||||
use store_api::storage::{ColumnId, RegionId};
|
||||
@@ -40,7 +41,7 @@ use crate::memtable::time_partition::TimePartitions;
|
||||
use crate::memtable::MemtableBuilderProvider;
|
||||
use crate::region::options::RegionOptions;
|
||||
use crate::region::version::{VersionBuilder, VersionControl, VersionControlRef};
|
||||
use crate::region::{ManifestContext, MitoRegion, RegionState};
|
||||
use crate::region::{ManifestContext, ManifestStats, MitoRegion, RegionState};
|
||||
use crate::region_write_ctx::RegionWriteCtx;
|
||||
use crate::request::OptionOutputTx;
|
||||
use crate::schedule::scheduler::SchedulerRef;
|
||||
@@ -62,6 +63,7 @@ pub(crate) struct RegionOpener {
|
||||
skip_wal_replay: bool,
|
||||
intermediate_manager: IntermediateManager,
|
||||
time_provider: Option<TimeProviderRef>,
|
||||
stats: ManifestStats,
|
||||
}
|
||||
|
||||
impl RegionOpener {
|
||||
@@ -86,6 +88,7 @@ impl RegionOpener {
|
||||
skip_wal_replay: false,
|
||||
intermediate_manager,
|
||||
time_provider: None,
|
||||
stats: Default::default(),
|
||||
}
|
||||
}
|
||||
|
||||
@@ -162,14 +165,18 @@ impl RegionOpener {
|
||||
}
|
||||
}
|
||||
let options = self.options.take().unwrap();
|
||||
let wal_options = options.wal_options.clone();
|
||||
let provider = self.provider(&options.wal_options);
|
||||
let object_store = self.object_store(&options.storage)?.clone();
|
||||
|
||||
// Create a manifest manager for this region and writes regions to the manifest file.
|
||||
let region_manifest_options = self.manifest_options(config, &options)?;
|
||||
let metadata = Arc::new(self.metadata.unwrap());
|
||||
let manifest_manager =
|
||||
RegionManifestManager::new(metadata.clone(), region_manifest_options).await?;
|
||||
let manifest_manager = RegionManifestManager::new(
|
||||
metadata.clone(),
|
||||
region_manifest_options,
|
||||
self.stats.total_manifest_size.clone(),
|
||||
)
|
||||
.await?;
|
||||
|
||||
let memtable_builder = self
|
||||
.memtable_builder_provider
|
||||
@@ -212,10 +219,11 @@ impl RegionOpener {
|
||||
access_layer,
|
||||
self.cache_manager,
|
||||
)),
|
||||
wal_options,
|
||||
provider,
|
||||
last_flush_millis: AtomicI64::new(time_provider.current_time_millis()),
|
||||
time_provider,
|
||||
memtable_builder,
|
||||
stats: self.stats,
|
||||
})
|
||||
}
|
||||
|
||||
@@ -250,6 +258,13 @@ impl RegionOpener {
|
||||
Ok(region)
|
||||
}
|
||||
|
||||
fn provider(&self, wal_options: &WalOptions) -> Provider {
|
||||
match wal_options {
|
||||
WalOptions::RaftEngine => Provider::raft_engine_provider(self.region_id.as_u64()),
|
||||
WalOptions::Kafka(options) => Provider::kafka_provider(options.topic.to_string()),
|
||||
}
|
||||
}
|
||||
|
||||
/// Tries to open the region and returns `None` if the region directory is empty.
|
||||
async fn maybe_open<S: LogStore>(
|
||||
&self,
|
||||
@@ -257,10 +272,13 @@ impl RegionOpener {
|
||||
wal: &Wal<S>,
|
||||
) -> Result<Option<MitoRegion>> {
|
||||
let region_options = self.options.as_ref().unwrap().clone();
|
||||
let wal_options = region_options.wal_options.clone();
|
||||
|
||||
let region_manifest_options = self.manifest_options(config, ®ion_options)?;
|
||||
let Some(manifest_manager) = RegionManifestManager::open(region_manifest_options).await?
|
||||
let Some(manifest_manager) = RegionManifestManager::open(
|
||||
region_manifest_options,
|
||||
self.stats.total_manifest_size.clone(),
|
||||
)
|
||||
.await?
|
||||
else {
|
||||
return Ok(None);
|
||||
};
|
||||
@@ -269,6 +287,7 @@ impl RegionOpener {
|
||||
let metadata = manifest.metadata.clone();
|
||||
|
||||
let region_id = self.region_id;
|
||||
let provider = self.provider(®ion_options.wal_options);
|
||||
let object_store = self.object_store(®ion_options.storage)?.clone();
|
||||
|
||||
debug!("Open region {} with options: {:?}", region_id, self.options);
|
||||
@@ -313,7 +332,7 @@ impl RegionOpener {
|
||||
);
|
||||
replay_memtable(
|
||||
wal,
|
||||
&wal_options,
|
||||
&provider,
|
||||
region_id,
|
||||
flushed_entry_id,
|
||||
&version_control,
|
||||
@@ -338,10 +357,11 @@ impl RegionOpener {
|
||||
RegionState::ReadOnly,
|
||||
)),
|
||||
file_purger,
|
||||
wal_options,
|
||||
provider,
|
||||
last_flush_millis: AtomicI64::new(time_provider.current_time_millis()),
|
||||
time_provider,
|
||||
memtable_builder,
|
||||
stats: self.stats.clone(),
|
||||
};
|
||||
Ok(Some(region))
|
||||
}
|
||||
@@ -430,7 +450,7 @@ pub(crate) fn check_recovered_region(
|
||||
/// Replays the mutations from WAL and inserts mutations to memtable of given region.
|
||||
pub(crate) async fn replay_memtable<S: LogStore>(
|
||||
wal: &Wal<S>,
|
||||
wal_options: &WalOptions,
|
||||
provider: &Provider,
|
||||
region_id: RegionId,
|
||||
flushed_entry_id: EntryId,
|
||||
version_control: &VersionControlRef,
|
||||
@@ -442,7 +462,7 @@ pub(crate) async fn replay_memtable<S: LogStore>(
|
||||
let mut last_entry_id = flushed_entry_id;
|
||||
let replay_from_entry_id = flushed_entry_id + 1;
|
||||
|
||||
let mut wal_stream = wal.scan(region_id, replay_from_entry_id, wal_options)?;
|
||||
let mut wal_stream = wal.scan(region_id, replay_from_entry_id, provider)?;
|
||||
while let Some(res) = wal_stream.next().await {
|
||||
let (entry_id, entry) = res?;
|
||||
if entry_id <= flushed_entry_id {
|
||||
@@ -459,7 +479,7 @@ pub(crate) async fn replay_memtable<S: LogStore>(
|
||||
last_entry_id = last_entry_id.max(entry_id);
|
||||
|
||||
let mut region_write_ctx =
|
||||
RegionWriteCtx::new(region_id, version_control, wal_options.clone());
|
||||
RegionWriteCtx::new(region_id, version_control, provider.clone());
|
||||
for mutation in entry.mutations {
|
||||
rows_replayed += mutation
|
||||
.rows
|
||||
@@ -474,8 +494,9 @@ pub(crate) async fn replay_memtable<S: LogStore>(
|
||||
region_write_ctx.write_memtable();
|
||||
}
|
||||
|
||||
wal.obsolete(region_id, flushed_entry_id, wal_options)
|
||||
.await?;
|
||||
// TODO(weny): We need to update `flushed_entry_id` in the region manifest
|
||||
// to avoid reading potentially incomplete entries in the future.
|
||||
wal.obsolete(region_id, flushed_entry_id, provider).await?;
|
||||
|
||||
info!(
|
||||
"Replay WAL for region: {}, rows recovered: {}, last entry id: {}",
|
||||
|
||||
@@ -16,8 +16,8 @@ use std::mem;
|
||||
use std::sync::Arc;
|
||||
|
||||
use api::v1::{Mutation, OpType, Rows, WalEntry};
|
||||
use common_wal::options::WalOptions;
|
||||
use snafu::ResultExt;
|
||||
use store_api::logstore::provider::Provider;
|
||||
use store_api::logstore::LogStore;
|
||||
use store_api::storage::{RegionId, SequenceNumber};
|
||||
|
||||
@@ -86,7 +86,7 @@ pub(crate) struct RegionWriteCtx {
|
||||
/// out of the context to construct the wal entry when we write to the wal.
|
||||
wal_entry: WalEntry,
|
||||
/// Wal options of the region being written to.
|
||||
wal_options: WalOptions,
|
||||
provider: Provider,
|
||||
/// Notifiers to send write results to waiters.
|
||||
///
|
||||
/// The i-th notify is for i-th mutation.
|
||||
@@ -106,7 +106,7 @@ impl RegionWriteCtx {
|
||||
pub(crate) fn new(
|
||||
region_id: RegionId,
|
||||
version_control: &VersionControlRef,
|
||||
wal_options: WalOptions,
|
||||
provider: Provider,
|
||||
) -> RegionWriteCtx {
|
||||
let VersionControlData {
|
||||
version,
|
||||
@@ -122,7 +122,7 @@ impl RegionWriteCtx {
|
||||
next_sequence: committed_sequence + 1,
|
||||
next_entry_id: last_entry_id + 1,
|
||||
wal_entry: WalEntry::default(),
|
||||
wal_options,
|
||||
provider,
|
||||
notifiers: Vec::new(),
|
||||
failed: false,
|
||||
put_num: 0,
|
||||
@@ -163,7 +163,7 @@ impl RegionWriteCtx {
|
||||
self.region_id,
|
||||
self.next_entry_id,
|
||||
&self.wal_entry,
|
||||
&self.wal_options,
|
||||
&self.provider,
|
||||
)?;
|
||||
self.next_entry_id += 1;
|
||||
Ok(())
|
||||
|
||||
@@ -175,8 +175,9 @@ impl FileHandle {
|
||||
self.inner.compacting.store(compacting, Ordering::Relaxed);
|
||||
}
|
||||
|
||||
pub fn meta(&self) -> FileMeta {
|
||||
self.inner.meta.clone()
|
||||
/// Returns a reference to the [FileMeta].
|
||||
pub fn meta_ref(&self) -> &FileMeta {
|
||||
&self.inner.meta
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@@ -121,9 +121,17 @@ impl SstIndexApplier {
|
||||
return Ok(None);
|
||||
};
|
||||
|
||||
let Some(indexed_value) = file_cache
|
||||
.get(IndexKey::new(self.region_id, file_id, FileType::Puffin))
|
||||
.await
|
||||
else {
|
||||
return Ok(None);
|
||||
};
|
||||
|
||||
Ok(file_cache
|
||||
.reader(IndexKey::new(self.region_id, file_id, FileType::Puffin))
|
||||
.await
|
||||
.map(|v| v.into_futures_async_read(0..indexed_value.file_size as u64))
|
||||
.map(PuffinFileReader::new))
|
||||
}
|
||||
|
||||
@@ -190,7 +198,13 @@ mod tests {
|
||||
let region_dir = "region_dir".to_string();
|
||||
let path = location::index_file_path(®ion_dir, file_id);
|
||||
|
||||
let mut puffin_writer = PuffinFileWriter::new(object_store.writer(&path).await.unwrap());
|
||||
let mut puffin_writer = PuffinFileWriter::new(
|
||||
object_store
|
||||
.writer(&path)
|
||||
.await
|
||||
.unwrap()
|
||||
.into_futures_async_write(),
|
||||
);
|
||||
puffin_writer
|
||||
.add_blob(Blob {
|
||||
blob_type: INDEX_BLOB_TYPE.to_string(),
|
||||
@@ -236,7 +250,13 @@ mod tests {
|
||||
let region_dir = "region_dir".to_string();
|
||||
let path = location::index_file_path(®ion_dir, file_id);
|
||||
|
||||
let mut puffin_writer = PuffinFileWriter::new(object_store.writer(&path).await.unwrap());
|
||||
let mut puffin_writer = PuffinFileWriter::new(
|
||||
object_store
|
||||
.writer(&path)
|
||||
.await
|
||||
.unwrap()
|
||||
.into_futures_async_write(),
|
||||
);
|
||||
puffin_writer
|
||||
.add_blob(Blob {
|
||||
blob_type: "invalid_blob_type".to_string(),
|
||||
|
||||
@@ -26,6 +26,8 @@ use crate::error::{OpenDalSnafu, Result};
|
||||
|
||||
/// A wrapper around [`ObjectStore`] that adds instrumentation for monitoring
|
||||
/// metrics such as bytes read, bytes written, and the number of seek operations.
|
||||
///
|
||||
/// TODO: Consider refactor InstrumentedStore to use async in trait instead of AsyncRead.
|
||||
#[derive(Clone)]
|
||||
pub(crate) struct InstrumentedStore {
|
||||
/// The underlying object store.
|
||||
@@ -58,8 +60,14 @@ impl InstrumentedStore {
|
||||
read_byte_count: &'a IntCounter,
|
||||
read_count: &'a IntCounter,
|
||||
seek_count: &'a IntCounter,
|
||||
) -> Result<InstrumentedAsyncRead<'a, object_store::Reader>> {
|
||||
let reader = self.object_store.reader(path).await.context(OpenDalSnafu)?;
|
||||
) -> Result<InstrumentedAsyncRead<'a, object_store::FuturesAsyncReader>> {
|
||||
let meta = self.object_store.stat(path).await.context(OpenDalSnafu)?;
|
||||
let reader = self
|
||||
.object_store
|
||||
.reader(path)
|
||||
.await
|
||||
.context(OpenDalSnafu)?
|
||||
.into_futures_async_read(0..meta.content_length());
|
||||
Ok(InstrumentedAsyncRead::new(
|
||||
reader,
|
||||
read_byte_count,
|
||||
@@ -77,15 +85,21 @@ impl InstrumentedStore {
|
||||
write_byte_count: &'a IntCounter,
|
||||
write_count: &'a IntCounter,
|
||||
flush_count: &'a IntCounter,
|
||||
) -> Result<InstrumentedAsyncWrite<'a, object_store::Writer>> {
|
||||
) -> Result<InstrumentedAsyncWrite<'a, object_store::FuturesAsyncWriter>> {
|
||||
let writer = match self.write_buffer_size {
|
||||
Some(size) => self
|
||||
.object_store
|
||||
.writer_with(path)
|
||||
.buffer(size)
|
||||
.chunk(size)
|
||||
.await
|
||||
.context(OpenDalSnafu)?,
|
||||
None => self.object_store.writer(path).await.context(OpenDalSnafu)?,
|
||||
.context(OpenDalSnafu)?
|
||||
.into_futures_async_write(),
|
||||
None => self
|
||||
.object_store
|
||||
.writer(path)
|
||||
.await
|
||||
.context(OpenDalSnafu)?
|
||||
.into_futures_async_write(),
|
||||
};
|
||||
Ok(InstrumentedAsyncWrite::new(
|
||||
writer,
|
||||
|
||||
@@ -25,6 +25,7 @@ use parquet::arrow::arrow_reader::RowSelection;
|
||||
use snafu::ResultExt;
|
||||
|
||||
use crate::error::{FieldTypeMismatchSnafu, FilterRecordBatchSnafu, Result};
|
||||
use crate::read::compat::CompatBatch;
|
||||
use crate::read::Batch;
|
||||
use crate::row_converter::{McmpRowCodec, RowCodec};
|
||||
use crate::sst::parquet::format::ReadFormat;
|
||||
@@ -32,6 +33,7 @@ use crate::sst::parquet::reader::{RowGroupReader, RowGroupReaderBuilder, SimpleF
|
||||
|
||||
/// A range of a parquet SST. Now it is a row group.
|
||||
/// We can read different file ranges in parallel.
|
||||
#[derive(Clone)]
|
||||
pub struct FileRange {
|
||||
/// Shared context.
|
||||
context: FileRangeContextRef,
|
||||
@@ -56,7 +58,6 @@ impl FileRange {
|
||||
}
|
||||
|
||||
/// Returns a reader to read the [FileRange].
|
||||
#[allow(dead_code)]
|
||||
pub(crate) async fn reader(&self) -> Result<RowGroupReader> {
|
||||
let parquet_reader = self
|
||||
.context
|
||||
@@ -66,6 +67,11 @@ impl FileRange {
|
||||
|
||||
Ok(RowGroupReader::new(self.context.clone(), parquet_reader))
|
||||
}
|
||||
|
||||
/// Returns the helper to compat batches.
|
||||
pub(crate) fn compat_batch(&self) -> Option<&CompatBatch> {
|
||||
self.context.compat_batch()
|
||||
}
|
||||
}
|
||||
|
||||
/// Context shared by ranges of the same parquet SST.
|
||||
@@ -78,6 +84,8 @@ pub(crate) struct FileRangeContext {
|
||||
read_format: ReadFormat,
|
||||
/// Decoder for primary keys
|
||||
codec: McmpRowCodec,
|
||||
/// Optional helper to compat batches.
|
||||
compat_batch: Option<CompatBatch>,
|
||||
}
|
||||
|
||||
pub(crate) type FileRangeContextRef = Arc<FileRangeContext>;
|
||||
@@ -95,6 +103,7 @@ impl FileRangeContext {
|
||||
filters,
|
||||
read_format,
|
||||
codec,
|
||||
compat_batch: None,
|
||||
}
|
||||
}
|
||||
|
||||
@@ -118,6 +127,16 @@ impl FileRangeContext {
|
||||
&self.reader_builder
|
||||
}
|
||||
|
||||
/// Returns the helper to compat batches.
|
||||
pub(crate) fn compat_batch(&self) -> Option<&CompatBatch> {
|
||||
self.compat_batch.as_ref()
|
||||
}
|
||||
|
||||
/// Sets the `CompatBatch` to the context.
|
||||
pub(crate) fn set_compat_batch(&mut self, compat: Option<CompatBatch>) {
|
||||
self.compat_batch = compat;
|
||||
}
|
||||
|
||||
/// 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.
|
||||
///
|
||||
|
||||
@@ -121,7 +121,7 @@ async fn fetch_ranges_seq(
|
||||
.read_with(&file_path)
|
||||
.range(range.start..range.end)
|
||||
.call()?;
|
||||
Ok::<_, object_store::Error>(Bytes::from(data))
|
||||
Ok::<_, object_store::Error>(data.to_bytes())
|
||||
})
|
||||
.collect::<object_store::Result<Vec<_>>>()
|
||||
};
|
||||
@@ -141,7 +141,7 @@ async fn fetch_ranges_concurrent(
|
||||
let future_read = object_store.read_with(file_path);
|
||||
handles.push(async move {
|
||||
let data = future_read.range(range.start..range.end).await?;
|
||||
Ok::<_, object_store::Error>(Bytes::from(data))
|
||||
Ok::<_, object_store::Error>(data.to_bytes())
|
||||
});
|
||||
}
|
||||
let results = futures::future::try_join_all(handles).await?;
|
||||
@@ -164,7 +164,7 @@ where
|
||||
}
|
||||
}
|
||||
|
||||
// https://github.com/apache/incubator-opendal/blob/7144ab1ca2409dff0c324bfed062ce985997f8ce/core/src/raw/tokio_util.rs#L21-L23
|
||||
// https://github.com/apache/opendal/blob/v0.46.0/core/src/raw/tokio_util.rs#L21-L24
|
||||
/// Parse tokio error into opendal::Error.
|
||||
fn new_task_join_error(e: tokio::task::JoinError) -> object_store::Error {
|
||||
object_store::Error::new(ErrorKind::Unexpected, "tokio task join failed").set_source(e)
|
||||
|
||||
@@ -85,7 +85,8 @@ impl<'a> MetadataLoader<'a> {
|
||||
.read_with(path)
|
||||
.range(buffer_start..file_size)
|
||||
.await
|
||||
.context(error::OpenDalSnafu)?;
|
||||
.context(error::OpenDalSnafu)?
|
||||
.to_vec();
|
||||
let buffer_len = buffer.len();
|
||||
|
||||
let mut footer = [0; 8];
|
||||
@@ -129,7 +130,8 @@ impl<'a> MetadataLoader<'a> {
|
||||
.read_with(path)
|
||||
.range(metadata_start..(file_size - FOOTER_SIZE as u64))
|
||||
.await
|
||||
.context(error::OpenDalSnafu)?;
|
||||
.context(error::OpenDalSnafu)?
|
||||
.to_vec();
|
||||
|
||||
let metadata = decode_metadata(&data).map_err(|e| {
|
||||
error::InvalidParquetSnafu {
|
||||
|
||||
@@ -53,7 +53,7 @@ use crate::read::{Batch, BatchReader};
|
||||
use crate::row_converter::{McmpRowCodec, SortField};
|
||||
use crate::sst::file::FileHandle;
|
||||
use crate::sst::index::applier::SstIndexApplierRef;
|
||||
use crate::sst::parquet::file_range::{FileRange, FileRangeContext, FileRangeContextRef};
|
||||
use crate::sst::parquet::file_range::{FileRangeContext, FileRangeContextRef};
|
||||
use crate::sst::parquet::format::ReadFormat;
|
||||
use crate::sst::parquet::metadata::MetadataLoader;
|
||||
use crate::sst::parquet::row_group::InMemoryRowGroup;
|
||||
@@ -155,29 +155,17 @@ impl ParquetReaderBuilder {
|
||||
/// This needs to perform IO operation.
|
||||
pub async fn build(&self) -> Result<ParquetReader> {
|
||||
let (context, row_groups) = self.build_reader_input().await?;
|
||||
ParquetReader::new(context, row_groups).await
|
||||
}
|
||||
|
||||
/// Builds [FileRange]s to read and pushes them to `file_ranges`.
|
||||
#[allow(dead_code)]
|
||||
pub async fn build_file_ranges(&self, file_ranges: &mut Vec<FileRange>) -> Result<()> {
|
||||
let (context, row_groups) = self.build_reader_input().await?;
|
||||
file_ranges.reserve_exact(row_groups.len());
|
||||
for (row_group_idx, row_selection) in row_groups {
|
||||
let file_range = FileRange::new(context.clone(), row_group_idx, row_selection);
|
||||
file_ranges.push(file_range);
|
||||
}
|
||||
Ok(())
|
||||
ParquetReader::new(Arc::new(context), row_groups).await
|
||||
}
|
||||
|
||||
/// Builds a [FileRangeContext] and collects row groups to read.
|
||||
///
|
||||
/// This needs to perform IO operation.
|
||||
async fn build_reader_input(&self) -> Result<(FileRangeContextRef, RowGroupMap)> {
|
||||
pub(crate) async fn build_reader_input(&self) -> Result<(FileRangeContext, RowGroupMap)> {
|
||||
let start = Instant::now();
|
||||
|
||||
let file_path = self.file_handle.file_path(&self.file_dir);
|
||||
let file_size = self.file_handle.meta().file_size;
|
||||
let file_size = self.file_handle.meta_ref().file_size;
|
||||
// Loads parquet metadata of the file.
|
||||
let parquet_meta = self.read_parquet_metadata(&file_path, file_size).await?;
|
||||
// Decodes region metadata.
|
||||
@@ -211,7 +199,7 @@ impl ParquetReaderBuilder {
|
||||
parquet_to_arrow_field_levels(parquet_schema_desc, projection_mask.clone(), hint)
|
||||
.context(ReadParquetSnafu { path: &file_path })?;
|
||||
|
||||
let mut metrics = Metrics::default();
|
||||
let mut metrics = ReaderMetrics::default();
|
||||
|
||||
let row_groups = self
|
||||
.row_groups_to_read(&read_format, &parquet_meta, &mut metrics)
|
||||
@@ -258,7 +246,7 @@ impl ParquetReaderBuilder {
|
||||
);
|
||||
|
||||
let context = FileRangeContext::new(reader_builder, filters, read_format, codec);
|
||||
Ok((Arc::new(context), row_groups))
|
||||
Ok((context, row_groups))
|
||||
}
|
||||
|
||||
/// Decodes region metadata from key value.
|
||||
@@ -324,7 +312,7 @@ impl ParquetReaderBuilder {
|
||||
&self,
|
||||
read_format: &ReadFormat,
|
||||
parquet_meta: &ParquetMetaData,
|
||||
metrics: &mut Metrics,
|
||||
metrics: &mut ReaderMetrics,
|
||||
) -> BTreeMap<usize, Option<RowSelection>> {
|
||||
let num_row_groups = parquet_meta.num_row_groups();
|
||||
if num_row_groups == 0 {
|
||||
@@ -346,13 +334,13 @@ impl ParquetReaderBuilder {
|
||||
async fn prune_row_groups_by_inverted_index(
|
||||
&self,
|
||||
parquet_meta: &ParquetMetaData,
|
||||
metrics: &mut Metrics,
|
||||
metrics: &mut ReaderMetrics,
|
||||
) -> Option<BTreeMap<usize, Option<RowSelection>>> {
|
||||
let Some(index_applier) = &self.index_applier else {
|
||||
return None;
|
||||
};
|
||||
|
||||
if !self.file_handle.meta().inverted_index_available() {
|
||||
if !self.file_handle.meta_ref().inverted_index_available() {
|
||||
return None;
|
||||
}
|
||||
|
||||
@@ -428,7 +416,7 @@ impl ParquetReaderBuilder {
|
||||
&self,
|
||||
read_format: &ReadFormat,
|
||||
parquet_meta: &ParquetMetaData,
|
||||
metrics: &mut Metrics,
|
||||
metrics: &mut ReaderMetrics,
|
||||
) -> Option<BTreeMap<usize, Option<RowSelection>>> {
|
||||
let Some(predicate) = &self.predicate else {
|
||||
return None;
|
||||
@@ -513,7 +501,7 @@ fn time_range_to_predicate(
|
||||
|
||||
/// Parquet reader metrics.
|
||||
#[derive(Debug, Default)]
|
||||
struct Metrics {
|
||||
pub(crate) struct ReaderMetrics {
|
||||
/// Number of row groups before filtering.
|
||||
num_row_groups_before_filtering: usize,
|
||||
/// Number of row groups filtered by inverted index.
|
||||
@@ -538,6 +526,24 @@ struct Metrics {
|
||||
num_rows: usize,
|
||||
}
|
||||
|
||||
impl ReaderMetrics {
|
||||
/// Adds `other` metrics to this metrics.
|
||||
pub(crate) fn merge_from(&mut self, other: &ReaderMetrics) {
|
||||
self.num_row_groups_before_filtering += other.num_row_groups_before_filtering;
|
||||
self.num_row_groups_inverted_index_filtered += other.num_row_groups_inverted_index_filtered;
|
||||
self.num_row_groups_min_max_filtered += other.num_row_groups_min_max_filtered;
|
||||
self.num_rows_precise_filtered += other.num_rows_precise_filtered;
|
||||
self.num_rows_in_row_group_before_filtering += other.num_rows_in_row_group_before_filtering;
|
||||
self.num_rows_in_row_group_inverted_index_filtered +=
|
||||
other.num_rows_in_row_group_inverted_index_filtered;
|
||||
self.build_cost += other.build_cost;
|
||||
self.scan_cost += other.scan_cost;
|
||||
self.num_record_batches += other.num_record_batches;
|
||||
self.num_batches += other.num_batches;
|
||||
self.num_rows += other.num_rows;
|
||||
}
|
||||
}
|
||||
|
||||
/// Builder to build a [ParquetRecordBatchReader] for a row group.
|
||||
pub(crate) struct RowGroupReaderBuilder {
|
||||
/// SST file to read.
|
||||
@@ -606,12 +612,12 @@ enum ReaderState {
|
||||
/// The reader is reading a row group.
|
||||
Readable(RowGroupReader),
|
||||
/// The reader is exhausted.
|
||||
Exhausted(Metrics),
|
||||
Exhausted(ReaderMetrics),
|
||||
}
|
||||
|
||||
impl ReaderState {
|
||||
/// Returns the metrics of the reader.
|
||||
fn metrics(&self) -> &Metrics {
|
||||
fn metrics(&self) -> &ReaderMetrics {
|
||||
match self {
|
||||
ReaderState::Readable(reader) => &reader.metrics,
|
||||
ReaderState::Exhausted(m) => m,
|
||||
@@ -807,7 +813,7 @@ impl ParquetReader {
|
||||
.await?;
|
||||
ReaderState::Readable(RowGroupReader::new(context.clone(), parquet_reader))
|
||||
} else {
|
||||
ReaderState::Exhausted(Metrics::default())
|
||||
ReaderState::Exhausted(ReaderMetrics::default())
|
||||
};
|
||||
|
||||
Ok(ParquetReader {
|
||||
@@ -829,7 +835,7 @@ impl ParquetReader {
|
||||
}
|
||||
|
||||
/// Reader to read a row group of a parquet file.
|
||||
pub(crate) struct RowGroupReader {
|
||||
pub struct RowGroupReader {
|
||||
/// Context for file ranges.
|
||||
context: FileRangeContextRef,
|
||||
/// Inner parquet reader.
|
||||
@@ -837,7 +843,7 @@ pub(crate) struct RowGroupReader {
|
||||
/// Buffered batches to return.
|
||||
batches: VecDeque<Batch>,
|
||||
/// Local scan metrics.
|
||||
metrics: Metrics,
|
||||
metrics: ReaderMetrics,
|
||||
}
|
||||
|
||||
impl RowGroupReader {
|
||||
@@ -847,17 +853,22 @@ impl RowGroupReader {
|
||||
context,
|
||||
reader,
|
||||
batches: VecDeque::new(),
|
||||
metrics: Metrics::default(),
|
||||
metrics: ReaderMetrics::default(),
|
||||
}
|
||||
}
|
||||
|
||||
/// Gets the metrics.
|
||||
pub(crate) fn metrics(&self) -> &ReaderMetrics {
|
||||
&self.metrics
|
||||
}
|
||||
|
||||
/// Resets the parquet reader.
|
||||
fn reset_reader(&mut self, reader: ParquetRecordBatchReader) {
|
||||
self.reader = reader;
|
||||
}
|
||||
|
||||
/// Tries to fetch next [Batch] from the reader.
|
||||
async fn next_batch(&mut self) -> Result<Option<Batch>> {
|
||||
pub(crate) async fn next_batch(&mut self) -> Result<Option<Batch>> {
|
||||
if let Some(batch) = self.batches.pop_front() {
|
||||
self.metrics.num_rows += batch.num_rows();
|
||||
return Ok(Some(batch));
|
||||
|
||||
@@ -93,7 +93,7 @@ impl SstVersion {
|
||||
.files
|
||||
.values()
|
||||
.map(|file_handle| {
|
||||
let meta = file_handle.meta();
|
||||
let meta = file_handle.meta_ref();
|
||||
meta.file_size + meta.index_file_size
|
||||
})
|
||||
.sum::<u64>()
|
||||
|
||||
@@ -356,11 +356,11 @@ impl TestEnv {
|
||||
};
|
||||
|
||||
if let Some(metadata) = initial_metadata {
|
||||
RegionManifestManager::new(metadata, manifest_opts)
|
||||
RegionManifestManager::new(metadata, manifest_opts, Default::default())
|
||||
.await
|
||||
.map(Some)
|
||||
} else {
|
||||
RegionManifestManager::open(manifest_opts).await
|
||||
RegionManifestManager::open(manifest_opts, Default::default()).await
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@@ -109,6 +109,7 @@ impl SchedulerEnv {
|
||||
compress_type: CompressionType::Uncompressed,
|
||||
checkpoint_distance: 10,
|
||||
},
|
||||
Default::default(),
|
||||
)
|
||||
.await
|
||||
.unwrap(),
|
||||
|
||||
@@ -26,20 +26,18 @@ use std::mem;
|
||||
use std::sync::Arc;
|
||||
|
||||
use api::v1::WalEntry;
|
||||
use async_stream::try_stream;
|
||||
use common_error::ext::BoxedError;
|
||||
use common_wal::options::WalOptions;
|
||||
use futures::stream::BoxStream;
|
||||
use futures::StreamExt;
|
||||
use prost::Message;
|
||||
use snafu::ResultExt;
|
||||
use store_api::logstore::entry::Entry;
|
||||
use store_api::logstore::provider::Provider;
|
||||
use store_api::logstore::{AppendBatchResponse, LogStore};
|
||||
use store_api::storage::RegionId;
|
||||
|
||||
use crate::error::{
|
||||
DecodeWalSnafu, DeleteWalSnafu, EncodeWalSnafu, ReadWalSnafu, Result, WriteWalSnafu,
|
||||
};
|
||||
use crate::error::{BuildEntrySnafu, DeleteWalSnafu, EncodeWalSnafu, Result, WriteWalSnafu};
|
||||
use crate::wal::raw_entry_reader::{LogStoreRawEntryReader, RegionRawEntryReader};
|
||||
use crate::wal::wal_entry_reader::{LogStoreEntryReader, WalEntryReader};
|
||||
|
||||
/// WAL entry id.
|
||||
pub type EntryId = store_api::logstore::entry::Id;
|
||||
@@ -49,7 +47,7 @@ pub type WalEntryStream<'a> = BoxStream<'a, Result<(EntryId, WalEntry)>>;
|
||||
/// Write ahead log.
|
||||
///
|
||||
/// All regions in the engine shares the same WAL instance.
|
||||
#[derive(Debug, Clone)]
|
||||
#[derive(Debug)]
|
||||
pub struct Wal<S> {
|
||||
/// The underlying log store.
|
||||
store: Arc<S>,
|
||||
@@ -60,6 +58,18 @@ impl<S> Wal<S> {
|
||||
pub fn new(store: Arc<S>) -> Self {
|
||||
Self { store }
|
||||
}
|
||||
|
||||
pub fn store(&self) -> &Arc<S> {
|
||||
&self.store
|
||||
}
|
||||
}
|
||||
|
||||
impl<S> Clone for Wal<S> {
|
||||
fn clone(&self) -> Self {
|
||||
Self {
|
||||
store: self.store.clone(),
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
impl<S: LogStore> Wal<S> {
|
||||
@@ -69,7 +79,7 @@ impl<S: LogStore> Wal<S> {
|
||||
store: self.store.clone(),
|
||||
entries: Vec::new(),
|
||||
entry_encode_buf: Vec::new(),
|
||||
namespaces: HashMap::new(),
|
||||
providers: HashMap::new(),
|
||||
}
|
||||
}
|
||||
|
||||
@@ -78,29 +88,19 @@ impl<S: LogStore> Wal<S> {
|
||||
&'a self,
|
||||
region_id: RegionId,
|
||||
start_id: EntryId,
|
||||
wal_options: &'a WalOptions,
|
||||
) -> Result<WalEntryStream> {
|
||||
let stream = try_stream!({
|
||||
let namespace = self.store.namespace(region_id.into(), wal_options);
|
||||
let mut stream = self
|
||||
.store
|
||||
.read(&namespace, start_id)
|
||||
.await
|
||||
.map_err(BoxedError::new)
|
||||
.context(ReadWalSnafu { region_id })?;
|
||||
|
||||
while let Some(entries) = stream.next().await {
|
||||
let entries = entries
|
||||
.map_err(BoxedError::new)
|
||||
.context(ReadWalSnafu { region_id })?;
|
||||
|
||||
for entry in entries {
|
||||
yield decode_entry(region_id, entry)?;
|
||||
}
|
||||
namespace: &'a Provider,
|
||||
) -> Result<WalEntryStream<'a>> {
|
||||
match namespace {
|
||||
Provider::RaftEngine(_) => {
|
||||
LogStoreEntryReader::new(LogStoreRawEntryReader::new(self.store.clone()))
|
||||
.read(namespace, start_id)
|
||||
}
|
||||
});
|
||||
|
||||
Ok(Box::pin(stream))
|
||||
Provider::Kafka(_) => LogStoreEntryReader::new(RegionRawEntryReader::new(
|
||||
LogStoreRawEntryReader::new(self.store.clone()),
|
||||
region_id,
|
||||
))
|
||||
.read(namespace, start_id),
|
||||
}
|
||||
}
|
||||
|
||||
/// Mark entries whose ids `<= last_id` as deleted.
|
||||
@@ -108,37 +108,26 @@ impl<S: LogStore> Wal<S> {
|
||||
&self,
|
||||
region_id: RegionId,
|
||||
last_id: EntryId,
|
||||
wal_options: &WalOptions,
|
||||
provider: &Provider,
|
||||
) -> Result<()> {
|
||||
let namespace = self.store.namespace(region_id.into(), wal_options);
|
||||
self.store
|
||||
.obsolete(namespace, last_id)
|
||||
.obsolete(provider, last_id)
|
||||
.await
|
||||
.map_err(BoxedError::new)
|
||||
.context(DeleteWalSnafu { region_id })
|
||||
}
|
||||
}
|
||||
|
||||
/// Decode Wal entry from log store.
|
||||
fn decode_entry<E: Entry>(region_id: RegionId, entry: E) -> Result<(EntryId, WalEntry)> {
|
||||
let entry_id = entry.id();
|
||||
let data = entry.data();
|
||||
|
||||
let wal_entry = WalEntry::decode(data).context(DecodeWalSnafu { region_id })?;
|
||||
|
||||
Ok((entry_id, wal_entry))
|
||||
}
|
||||
|
||||
/// WAL batch writer.
|
||||
pub struct WalWriter<S: LogStore> {
|
||||
/// Log store of the WAL.
|
||||
store: Arc<S>,
|
||||
/// Entries to write.
|
||||
entries: Vec<S::Entry>,
|
||||
entries: Vec<Entry>,
|
||||
/// Buffer to encode WAL entry.
|
||||
entry_encode_buf: Vec<u8>,
|
||||
/// Namespaces of regions being written into.
|
||||
namespaces: HashMap<RegionId, S::Namespace>,
|
||||
/// Providers of regions being written into.
|
||||
providers: HashMap<RegionId, Provider>,
|
||||
}
|
||||
|
||||
impl<S: LogStore> WalWriter<S> {
|
||||
@@ -148,14 +137,13 @@ impl<S: LogStore> WalWriter<S> {
|
||||
region_id: RegionId,
|
||||
entry_id: EntryId,
|
||||
wal_entry: &WalEntry,
|
||||
wal_options: &WalOptions,
|
||||
provider: &Provider,
|
||||
) -> Result<()> {
|
||||
// Gets or inserts with a newly built namespace.
|
||||
let namespace = self
|
||||
.namespaces
|
||||
// Gets or inserts with a newly built provider.
|
||||
let provider = self
|
||||
.providers
|
||||
.entry(region_id)
|
||||
.or_insert_with(|| self.store.namespace(region_id.into(), wal_options))
|
||||
.clone();
|
||||
.or_insert_with(|| provider.clone());
|
||||
|
||||
// Encode wal entry to log store entry.
|
||||
self.entry_encode_buf.clear();
|
||||
@@ -164,7 +152,9 @@ impl<S: LogStore> WalWriter<S> {
|
||||
.context(EncodeWalSnafu { region_id })?;
|
||||
let entry = self
|
||||
.store
|
||||
.entry(&mut self.entry_encode_buf, entry_id, namespace);
|
||||
.entry(&mut self.entry_encode_buf, entry_id, region_id, provider)
|
||||
.map_err(BoxedError::new)
|
||||
.context(BuildEntrySnafu { region_id })?;
|
||||
|
||||
self.entries.push(entry);
|
||||
|
||||
@@ -264,7 +254,6 @@ mod tests {
|
||||
async fn test_write_wal() {
|
||||
let env = WalEnv::new().await;
|
||||
let wal = env.new_wal();
|
||||
let wal_options = WalOptions::default();
|
||||
|
||||
let entry = WalEntry {
|
||||
mutations: vec![
|
||||
@@ -274,16 +263,34 @@ mod tests {
|
||||
};
|
||||
let mut writer = wal.writer();
|
||||
// Region 1 entry 1.
|
||||
let region_id = RegionId::new(1, 1);
|
||||
writer
|
||||
.add_entry(RegionId::new(1, 1), 1, &entry, &wal_options)
|
||||
.add_entry(
|
||||
region_id,
|
||||
1,
|
||||
&entry,
|
||||
&Provider::raft_engine_provider(region_id.as_u64()),
|
||||
)
|
||||
.unwrap();
|
||||
// Region 2 entry 1.
|
||||
let region_id = RegionId::new(1, 2);
|
||||
writer
|
||||
.add_entry(RegionId::new(1, 2), 1, &entry, &wal_options)
|
||||
.add_entry(
|
||||
region_id,
|
||||
1,
|
||||
&entry,
|
||||
&Provider::raft_engine_provider(region_id.as_u64()),
|
||||
)
|
||||
.unwrap();
|
||||
// Region 1 entry 2.
|
||||
let region_id = RegionId::new(1, 2);
|
||||
writer
|
||||
.add_entry(RegionId::new(1, 1), 2, &entry, &wal_options)
|
||||
.add_entry(
|
||||
region_id,
|
||||
2,
|
||||
&entry,
|
||||
&Provider::raft_engine_provider(region_id.as_u64()),
|
||||
)
|
||||
.unwrap();
|
||||
|
||||
// Test writing multiple region to wal.
|
||||
@@ -331,32 +338,33 @@ mod tests {
|
||||
async fn test_scan_wal() {
|
||||
let env = WalEnv::new().await;
|
||||
let wal = env.new_wal();
|
||||
let wal_options = WalOptions::default();
|
||||
|
||||
let entries = sample_entries();
|
||||
let (id1, id2) = (RegionId::new(1, 1), RegionId::new(1, 2));
|
||||
let ns1 = Provider::raft_engine_provider(id1.as_u64());
|
||||
let ns2 = Provider::raft_engine_provider(id2.as_u64());
|
||||
let mut writer = wal.writer();
|
||||
writer.add_entry(id1, 1, &entries[0], &wal_options).unwrap();
|
||||
writer.add_entry(id1, 1, &entries[0], &ns1).unwrap();
|
||||
// Insert one entry into region2. Scan should not return this entry.
|
||||
writer.add_entry(id2, 1, &entries[0], &wal_options).unwrap();
|
||||
writer.add_entry(id1, 2, &entries[1], &wal_options).unwrap();
|
||||
writer.add_entry(id1, 3, &entries[2], &wal_options).unwrap();
|
||||
writer.add_entry(id1, 4, &entries[3], &wal_options).unwrap();
|
||||
writer.add_entry(id2, 1, &entries[0], &ns2).unwrap();
|
||||
writer.add_entry(id1, 2, &entries[1], &ns1).unwrap();
|
||||
writer.add_entry(id1, 3, &entries[2], &ns1).unwrap();
|
||||
writer.add_entry(id1, 4, &entries[3], &ns1).unwrap();
|
||||
|
||||
writer.write_to_wal().await.unwrap();
|
||||
|
||||
// Scan all contents region1
|
||||
let stream = wal.scan(id1, 1, &wal_options).unwrap();
|
||||
let stream = wal.scan(id1, 1, &ns1).unwrap();
|
||||
let actual: Vec<_> = stream.try_collect().await.unwrap();
|
||||
check_entries(&entries, 1, &actual);
|
||||
|
||||
// Scan parts of contents
|
||||
let stream = wal.scan(id1, 2, &wal_options).unwrap();
|
||||
let stream = wal.scan(id1, 2, &ns1).unwrap();
|
||||
let actual: Vec<_> = stream.try_collect().await.unwrap();
|
||||
check_entries(&entries[1..], 2, &actual);
|
||||
|
||||
// Scan out of range
|
||||
let stream = wal.scan(id1, 5, &wal_options).unwrap();
|
||||
let stream = wal.scan(id1, 5, &ns1).unwrap();
|
||||
let actual: Vec<_> = stream.try_collect().await.unwrap();
|
||||
assert!(actual.is_empty());
|
||||
}
|
||||
@@ -365,35 +373,27 @@ mod tests {
|
||||
async fn test_obsolete_wal() {
|
||||
let env = WalEnv::new().await;
|
||||
let wal = env.new_wal();
|
||||
let wal_options = WalOptions::default();
|
||||
|
||||
let entries = sample_entries();
|
||||
let mut writer = wal.writer();
|
||||
let region_id = RegionId::new(1, 1);
|
||||
writer
|
||||
.add_entry(region_id, 1, &entries[0], &wal_options)
|
||||
.unwrap();
|
||||
writer
|
||||
.add_entry(region_id, 2, &entries[1], &wal_options)
|
||||
.unwrap();
|
||||
writer
|
||||
.add_entry(region_id, 3, &entries[2], &wal_options)
|
||||
.unwrap();
|
||||
let ns = Provider::raft_engine_provider(region_id.as_u64());
|
||||
writer.add_entry(region_id, 1, &entries[0], &ns).unwrap();
|
||||
writer.add_entry(region_id, 2, &entries[1], &ns).unwrap();
|
||||
writer.add_entry(region_id, 3, &entries[2], &ns).unwrap();
|
||||
|
||||
writer.write_to_wal().await.unwrap();
|
||||
|
||||
// Delete 1, 2.
|
||||
wal.obsolete(region_id, 2, &wal_options).await.unwrap();
|
||||
wal.obsolete(region_id, 2, &ns).await.unwrap();
|
||||
|
||||
// Put 4.
|
||||
let mut writer = wal.writer();
|
||||
writer
|
||||
.add_entry(region_id, 4, &entries[3], &wal_options)
|
||||
.unwrap();
|
||||
writer.add_entry(region_id, 4, &entries[3], &ns).unwrap();
|
||||
writer.write_to_wal().await.unwrap();
|
||||
|
||||
// Scan all
|
||||
let stream = wal.scan(region_id, 1, &wal_options).unwrap();
|
||||
let stream = wal.scan(region_id, 1, &ns).unwrap();
|
||||
let actual: Vec<_> = stream.try_collect().await.unwrap();
|
||||
check_entries(&entries[2..], 3, &actual);
|
||||
}
|
||||
|
||||
@@ -20,7 +20,8 @@ use common_wal::options::{KafkaWalOptions, WalOptions};
|
||||
use futures::stream::BoxStream;
|
||||
use futures::TryStreamExt;
|
||||
use snafu::ResultExt;
|
||||
use store_api::logstore::entry::{Entry, RawEntry};
|
||||
use store_api::logstore::entry::Entry;
|
||||
use store_api::logstore::provider::{KafkaProvider, Provider, RaftEngineProvider};
|
||||
use store_api::logstore::LogStore;
|
||||
use store_api::storage::RegionId;
|
||||
use tokio_stream::StreamExt;
|
||||
@@ -28,38 +29,12 @@ use tokio_stream::StreamExt;
|
||||
use crate::error::{self, Result};
|
||||
use crate::wal::EntryId;
|
||||
|
||||
/// A stream that yields [RawEntry].
|
||||
pub type RawEntryStream<'a> = BoxStream<'a, Result<RawEntry>>;
|
||||
/// A stream that yields [Entry].
|
||||
pub type EntryStream<'a> = BoxStream<'a, Result<Entry>>;
|
||||
|
||||
// The namespace of kafka log store
|
||||
pub struct KafkaNamespace<'a> {
|
||||
topic: &'a str,
|
||||
}
|
||||
|
||||
// The namespace of raft engine log store
|
||||
pub struct RaftEngineNamespace {
|
||||
region_id: RegionId,
|
||||
}
|
||||
|
||||
impl RaftEngineNamespace {
|
||||
pub fn new(region_id: RegionId) -> Self {
|
||||
Self { region_id }
|
||||
}
|
||||
}
|
||||
|
||||
/// The namespace of [RawEntryReader].
|
||||
pub(crate) enum LogStoreNamespace<'a> {
|
||||
RaftEngine(RaftEngineNamespace),
|
||||
Kafka(KafkaNamespace<'a>),
|
||||
}
|
||||
|
||||
/// [RawEntryReader] provides the ability to read [RawEntry] from the underlying [LogStore].
|
||||
/// [RawEntryReader] provides the ability to read [Entry] from the underlying [LogStore].
|
||||
pub(crate) trait RawEntryReader: Send + Sync {
|
||||
fn read<'a>(
|
||||
&'a self,
|
||||
ctx: LogStoreNamespace<'a>,
|
||||
start_id: EntryId,
|
||||
) -> Result<RawEntryStream<'a>>;
|
||||
fn read(&self, provider: &Provider, start_id: EntryId) -> Result<EntryStream<'static>>;
|
||||
}
|
||||
|
||||
/// Implement the [RawEntryReader] for the [LogStore].
|
||||
@@ -67,121 +42,70 @@ pub struct LogStoreRawEntryReader<S> {
|
||||
store: Arc<S>,
|
||||
}
|
||||
|
||||
impl<S: LogStore> LogStoreRawEntryReader<S> {
|
||||
impl<S> LogStoreRawEntryReader<S> {
|
||||
pub fn new(store: Arc<S>) -> Self {
|
||||
Self { store }
|
||||
}
|
||||
|
||||
fn read_region(&self, ns: RaftEngineNamespace, start_id: EntryId) -> Result<RawEntryStream> {
|
||||
let region_id = ns.region_id;
|
||||
let stream = try_stream!({
|
||||
// TODO(weny): refactor the `namespace` method.
|
||||
let namespace = self.store.namespace(region_id.into(), &Default::default());
|
||||
let mut stream = self
|
||||
.store
|
||||
.read(&namespace, start_id)
|
||||
.await
|
||||
.map_err(BoxedError::new)
|
||||
.context(error::ReadWalSnafu { region_id })?;
|
||||
|
||||
while let Some(entries) = stream.next().await {
|
||||
let entries = entries
|
||||
.map_err(BoxedError::new)
|
||||
.context(error::ReadWalSnafu { region_id })?;
|
||||
|
||||
for entry in entries {
|
||||
yield entry.into_raw_entry()
|
||||
}
|
||||
}
|
||||
});
|
||||
|
||||
Ok(Box::pin(stream))
|
||||
}
|
||||
|
||||
fn read_topic<'a>(
|
||||
&'a self,
|
||||
ns: KafkaNamespace<'a>,
|
||||
start_id: EntryId,
|
||||
) -> Result<RawEntryStream> {
|
||||
let topic = ns.topic;
|
||||
let stream = try_stream!({
|
||||
// TODO(weny): refactor the `namespace` method.
|
||||
let namespace = self.store.namespace(
|
||||
RegionId::from_u64(0).into(),
|
||||
&WalOptions::Kafka(KafkaWalOptions {
|
||||
topic: topic.to_string(),
|
||||
}),
|
||||
);
|
||||
|
||||
let mut stream = self
|
||||
.store
|
||||
.read(&namespace, start_id)
|
||||
.await
|
||||
.map_err(BoxedError::new)
|
||||
.context(error::ReadKafkaWalSnafu { topic })?;
|
||||
|
||||
while let Some(entries) = stream.next().await {
|
||||
let entries = entries
|
||||
.map_err(BoxedError::new)
|
||||
.context(error::ReadKafkaWalSnafu { topic })?;
|
||||
|
||||
for entry in entries {
|
||||
yield entry.into_raw_entry()
|
||||
}
|
||||
}
|
||||
});
|
||||
|
||||
Ok(Box::pin(stream))
|
||||
}
|
||||
}
|
||||
|
||||
impl<S: LogStore> RawEntryReader for LogStoreRawEntryReader<S> {
|
||||
fn read<'a>(
|
||||
&'a self,
|
||||
ctx: LogStoreNamespace<'a>,
|
||||
start_id: EntryId,
|
||||
) -> Result<RawEntryStream<'a>> {
|
||||
let stream = match ctx {
|
||||
LogStoreNamespace::RaftEngine(ns) => self.read_region(ns, start_id)?,
|
||||
LogStoreNamespace::Kafka(ns) => self.read_topic(ns, start_id)?,
|
||||
};
|
||||
fn read(&self, provider: &Provider, start_id: EntryId) -> Result<EntryStream<'static>> {
|
||||
let store = self.store.clone();
|
||||
let provider = provider.clone();
|
||||
let stream = try_stream!({
|
||||
let mut stream = store
|
||||
.read(&provider, start_id)
|
||||
.await
|
||||
.map_err(BoxedError::new)
|
||||
.with_context(|_| error::ReadWalSnafu {
|
||||
provider: provider.clone(),
|
||||
})?;
|
||||
|
||||
while let Some(entries) = stream.next().await {
|
||||
let entries =
|
||||
entries
|
||||
.map_err(BoxedError::new)
|
||||
.with_context(|_| error::ReadWalSnafu {
|
||||
provider: provider.clone(),
|
||||
})?;
|
||||
|
||||
for entry in entries {
|
||||
yield entry
|
||||
}
|
||||
}
|
||||
});
|
||||
|
||||
Ok(Box::pin(stream))
|
||||
}
|
||||
}
|
||||
|
||||
/// A filter implement the [RawEntryReader]
|
||||
pub struct RawEntryReaderFilter<R, F> {
|
||||
/// A [RawEntryReader] reads [RawEntry] belongs to a specific region.
|
||||
pub struct RegionRawEntryReader<R> {
|
||||
reader: R,
|
||||
filter: F,
|
||||
region_id: RegionId,
|
||||
}
|
||||
|
||||
impl<R, F> RawEntryReaderFilter<R, F>
|
||||
impl<R> RegionRawEntryReader<R>
|
||||
where
|
||||
R: RawEntryReader,
|
||||
F: Fn(&RawEntry) -> bool + Sync + Send,
|
||||
{
|
||||
pub fn new(reader: R, filter: F) -> Self {
|
||||
Self { reader, filter }
|
||||
pub fn new(reader: R, region_id: RegionId) -> Self {
|
||||
Self { reader, region_id }
|
||||
}
|
||||
}
|
||||
|
||||
impl<R, F> RawEntryReader for RawEntryReaderFilter<R, F>
|
||||
impl<R> RawEntryReader for RegionRawEntryReader<R>
|
||||
where
|
||||
R: RawEntryReader,
|
||||
F: Fn(&RawEntry) -> bool + Sync + Send,
|
||||
{
|
||||
fn read<'a>(
|
||||
&'a self,
|
||||
ctx: LogStoreNamespace<'a>,
|
||||
start_id: EntryId,
|
||||
) -> Result<RawEntryStream<'a>> {
|
||||
fn read(&self, ctx: &Provider, start_id: EntryId) -> Result<EntryStream<'static>> {
|
||||
let mut stream = self.reader.read(ctx, start_id)?;
|
||||
let filter = &(self.filter);
|
||||
let region_id = self.region_id;
|
||||
|
||||
let stream = try_stream!({
|
||||
while let Some(entry) = stream.next().await {
|
||||
let entry = entry?;
|
||||
if filter(&entry) {
|
||||
if entry.region_id() == region_id {
|
||||
yield entry
|
||||
}
|
||||
}
|
||||
@@ -197,11 +121,9 @@ mod tests {
|
||||
|
||||
use common_wal::options::WalOptions;
|
||||
use futures::stream;
|
||||
use store_api::logstore::entry::{Entry, RawEntry};
|
||||
use store_api::logstore::entry_stream::SendableEntryStream;
|
||||
use store_api::logstore::namespace::Namespace;
|
||||
use store_api::logstore::entry::{Entry, NaiveEntry};
|
||||
use store_api::logstore::{
|
||||
AppendBatchResponse, AppendResponse, EntryId, LogStore, NamespaceId,
|
||||
AppendBatchResponse, AppendResponse, EntryId, LogStore, SendableEntryStream,
|
||||
};
|
||||
use store_api::storage::RegionId;
|
||||
|
||||
@@ -210,93 +132,79 @@ mod tests {
|
||||
|
||||
#[derive(Debug)]
|
||||
struct MockLogStore {
|
||||
entries: Vec<RawEntry>,
|
||||
}
|
||||
|
||||
#[derive(Debug, Eq, PartialEq, Clone, Copy, Default, Hash)]
|
||||
struct MockNamespace;
|
||||
|
||||
impl Namespace for MockNamespace {
|
||||
fn id(&self) -> NamespaceId {
|
||||
0
|
||||
}
|
||||
entries: Vec<Entry>,
|
||||
}
|
||||
|
||||
#[async_trait::async_trait]
|
||||
impl LogStore for MockLogStore {
|
||||
type Entry = RawEntry;
|
||||
type Error = error::Error;
|
||||
type Namespace = MockNamespace;
|
||||
|
||||
async fn stop(&self) -> Result<(), Self::Error> {
|
||||
unreachable!()
|
||||
}
|
||||
|
||||
async fn append(&self, entry: Self::Entry) -> Result<AppendResponse, Self::Error> {
|
||||
unreachable!()
|
||||
}
|
||||
|
||||
async fn append_batch(
|
||||
&self,
|
||||
entries: Vec<Self::Entry>,
|
||||
entries: Vec<Entry>,
|
||||
) -> Result<AppendBatchResponse, Self::Error> {
|
||||
unreachable!()
|
||||
}
|
||||
|
||||
async fn read(
|
||||
&self,
|
||||
ns: &Self::Namespace,
|
||||
provider: &Provider,
|
||||
id: EntryId,
|
||||
) -> Result<SendableEntryStream<Self::Entry, Self::Error>, Self::Error> {
|
||||
) -> Result<SendableEntryStream<'static, Entry, Self::Error>, Self::Error> {
|
||||
Ok(Box::pin(stream::iter(vec![Ok(self.entries.clone())])))
|
||||
}
|
||||
|
||||
async fn create_namespace(&self, ns: &Self::Namespace) -> Result<(), Self::Error> {
|
||||
async fn create_namespace(&self, ns: &Provider) -> Result<(), Self::Error> {
|
||||
unreachable!()
|
||||
}
|
||||
|
||||
async fn delete_namespace(&self, ns: &Self::Namespace) -> Result<(), Self::Error> {
|
||||
async fn delete_namespace(&self, ns: &Provider) -> Result<(), Self::Error> {
|
||||
unreachable!()
|
||||
}
|
||||
|
||||
async fn list_namespaces(&self) -> Result<Vec<Self::Namespace>, Self::Error> {
|
||||
async fn list_namespaces(&self) -> Result<Vec<Provider>, Self::Error> {
|
||||
unreachable!()
|
||||
}
|
||||
|
||||
async fn obsolete(
|
||||
&self,
|
||||
ns: Self::Namespace,
|
||||
provider: &Provider,
|
||||
entry_id: EntryId,
|
||||
) -> Result<(), Self::Error> {
|
||||
unreachable!()
|
||||
}
|
||||
|
||||
fn entry(&self, data: &mut Vec<u8>, entry_id: EntryId, ns: Self::Namespace) -> Self::Entry {
|
||||
fn entry(
|
||||
&self,
|
||||
data: &mut Vec<u8>,
|
||||
entry_id: EntryId,
|
||||
region_id: RegionId,
|
||||
provider: &Provider,
|
||||
) -> Result<Entry, Self::Error> {
|
||||
unreachable!()
|
||||
}
|
||||
|
||||
fn namespace(&self, _ns_id: NamespaceId, _wal_options: &WalOptions) -> Self::Namespace {
|
||||
MockNamespace
|
||||
}
|
||||
}
|
||||
|
||||
#[tokio::test]
|
||||
async fn test_raw_entry_reader() {
|
||||
let expected_entries = vec![RawEntry {
|
||||
let provider = Provider::raft_engine_provider(RegionId::new(1024, 1).as_u64());
|
||||
let expected_entries = vec![Entry::Naive(NaiveEntry {
|
||||
provider: provider.clone(),
|
||||
region_id: RegionId::new(1024, 1),
|
||||
entry_id: 1,
|
||||
data: vec![],
|
||||
}];
|
||||
data: vec![1],
|
||||
})];
|
||||
let store = MockLogStore {
|
||||
entries: expected_entries.clone(),
|
||||
};
|
||||
|
||||
let reader = LogStoreRawEntryReader::new(Arc::new(store));
|
||||
let entries = reader
|
||||
.read(
|
||||
LogStoreNamespace::RaftEngine(RaftEngineNamespace::new(RegionId::new(1024, 1))),
|
||||
0,
|
||||
)
|
||||
.read(&provider, 0)
|
||||
.unwrap()
|
||||
.try_collect::<Vec<_>>()
|
||||
.await
|
||||
@@ -306,37 +214,38 @@ mod tests {
|
||||
|
||||
#[tokio::test]
|
||||
async fn test_raw_entry_reader_filter() {
|
||||
let provider = Provider::raft_engine_provider(RegionId::new(1024, 1).as_u64());
|
||||
let all_entries = vec![
|
||||
RawEntry {
|
||||
Entry::Naive(NaiveEntry {
|
||||
provider: provider.clone(),
|
||||
region_id: RegionId::new(1024, 1),
|
||||
entry_id: 1,
|
||||
data: vec![1],
|
||||
},
|
||||
RawEntry {
|
||||
}),
|
||||
Entry::Naive(NaiveEntry {
|
||||
provider: provider.clone(),
|
||||
region_id: RegionId::new(1024, 2),
|
||||
entry_id: 2,
|
||||
data: vec![2],
|
||||
},
|
||||
RawEntry {
|
||||
}),
|
||||
Entry::Naive(NaiveEntry {
|
||||
provider: provider.clone(),
|
||||
region_id: RegionId::new(1024, 3),
|
||||
entry_id: 3,
|
||||
data: vec![3],
|
||||
},
|
||||
}),
|
||||
];
|
||||
let store = MockLogStore {
|
||||
entries: all_entries.clone(),
|
||||
};
|
||||
|
||||
let expected_region_id = RegionId::new(1024, 3);
|
||||
let reader =
|
||||
RawEntryReaderFilter::new(LogStoreRawEntryReader::new(Arc::new(store)), |entry| {
|
||||
entry.region_id == expected_region_id
|
||||
});
|
||||
let reader = RegionRawEntryReader::new(
|
||||
LogStoreRawEntryReader::new(Arc::new(store)),
|
||||
expected_region_id,
|
||||
);
|
||||
let entries = reader
|
||||
.read(
|
||||
LogStoreNamespace::RaftEngine(RaftEngineNamespace::new(RegionId::new(1024, 1))),
|
||||
0,
|
||||
)
|
||||
.read(&provider, 0)
|
||||
.unwrap()
|
||||
.try_collect::<Vec<_>>()
|
||||
.await
|
||||
@@ -344,7 +253,7 @@ mod tests {
|
||||
assert_eq!(
|
||||
all_entries
|
||||
.into_iter()
|
||||
.filter(|entry| entry.region_id == expected_region_id)
|
||||
.filter(|entry| entry.region_id() == expected_region_id)
|
||||
.collect::<Vec<_>>(),
|
||||
entries
|
||||
);
|
||||
|
||||
@@ -12,13 +12,183 @@
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
use api::v1::WalEntry;
|
||||
use common_telemetry::info;
|
||||
use futures::StreamExt;
|
||||
use prost::Message;
|
||||
use snafu::{ensure, ResultExt};
|
||||
use store_api::logstore::entry::Entry;
|
||||
use store_api::logstore::provider::Provider;
|
||||
use store_api::storage::RegionId;
|
||||
|
||||
use crate::error::Result;
|
||||
use crate::wal::raw_entry_reader::LogStoreNamespace;
|
||||
use crate::error::{CorruptedEntrySnafu, DecodeWalSnafu, Result};
|
||||
use crate::wal::raw_entry_reader::RawEntryReader;
|
||||
use crate::wal::{EntryId, WalEntryStream};
|
||||
|
||||
/// [OneshotWalEntryReader] provides the ability to read and decode entries from the underlying store.
|
||||
pub(crate) trait OneshotWalEntryReader: Send + Sync {
|
||||
fn read(self, ctx: LogStoreNamespace, start_id: EntryId) -> Result<WalEntryStream>;
|
||||
pub(crate) fn decode_raw_entry(raw_entry: Entry) -> Result<(EntryId, WalEntry)> {
|
||||
let entry_id = raw_entry.entry_id();
|
||||
let region_id = raw_entry.region_id();
|
||||
ensure!(raw_entry.is_complete(), CorruptedEntrySnafu { region_id });
|
||||
// TODO(weny): implement the [Buf] for return value, avoid extra memory allocation.
|
||||
let bytes = raw_entry.into_bytes();
|
||||
let wal_entry = WalEntry::decode(bytes.as_slice()).context(DecodeWalSnafu { region_id })?;
|
||||
|
||||
Ok((entry_id, wal_entry))
|
||||
}
|
||||
|
||||
/// [WalEntryReader] provides the ability to read and decode entries from the underlying store.
|
||||
pub(crate) trait WalEntryReader: Send + Sync {
|
||||
fn read(self, ns: &'_ Provider, start_id: EntryId) -> Result<WalEntryStream<'static>>;
|
||||
}
|
||||
|
||||
/// A Reader reads the [RawEntry] from [RawEntryReader] and decodes [RawEntry] into [WalEntry].
|
||||
pub struct LogStoreEntryReader<R> {
|
||||
reader: R,
|
||||
}
|
||||
|
||||
impl<R> LogStoreEntryReader<R> {
|
||||
pub fn new(reader: R) -> Self {
|
||||
Self { reader }
|
||||
}
|
||||
}
|
||||
|
||||
impl<R: RawEntryReader> WalEntryReader for LogStoreEntryReader<R> {
|
||||
fn read(self, ns: &'_ Provider, start_id: EntryId) -> Result<WalEntryStream<'static>> {
|
||||
let LogStoreEntryReader { reader } = self;
|
||||
let mut stream = reader.read(ns, start_id)?;
|
||||
|
||||
let stream = async_stream::stream! {
|
||||
let mut buffered_entry = None;
|
||||
while let Some(next_entry) = stream.next().await {
|
||||
match buffered_entry.take() {
|
||||
Some(entry) => {
|
||||
yield decode_raw_entry(entry);
|
||||
buffered_entry = Some(next_entry?);
|
||||
},
|
||||
None => {
|
||||
buffered_entry = Some(next_entry?);
|
||||
}
|
||||
};
|
||||
}
|
||||
if let Some(entry) = buffered_entry {
|
||||
// Ignores tail corrupted data.
|
||||
if entry.is_complete() {
|
||||
yield decode_raw_entry(entry);
|
||||
}
|
||||
}
|
||||
};
|
||||
|
||||
Ok(Box::pin(stream))
|
||||
}
|
||||
}
|
||||
|
||||
#[cfg(test)]
|
||||
mod tests {
|
||||
use std::assert_matches::assert_matches;
|
||||
|
||||
use api::v1::{Mutation, OpType, WalEntry};
|
||||
use futures::{stream, TryStreamExt};
|
||||
use prost::Message;
|
||||
use store_api::logstore::entry::{Entry, MultiplePartEntry, MultiplePartHeader};
|
||||
use store_api::logstore::provider::Provider;
|
||||
use store_api::storage::RegionId;
|
||||
|
||||
use crate::error::{self, Result};
|
||||
use crate::wal::raw_entry_reader::{EntryStream, RawEntryReader};
|
||||
use crate::wal::wal_entry_reader::{LogStoreEntryReader, WalEntryReader};
|
||||
use crate::wal::EntryId;
|
||||
|
||||
struct MockRawEntryStream {
|
||||
entries: Vec<Entry>,
|
||||
}
|
||||
|
||||
impl RawEntryReader for MockRawEntryStream {
|
||||
fn read(&self, ns: &Provider, start_id: EntryId) -> Result<EntryStream<'static>> {
|
||||
let entries = self.entries.clone().into_iter().map(Ok);
|
||||
|
||||
Ok(Box::pin(stream::iter(entries)))
|
||||
}
|
||||
}
|
||||
|
||||
#[tokio::test]
|
||||
async fn test_tail_corrupted_stream() {
|
||||
common_telemetry::init_default_ut_logging();
|
||||
let provider = Provider::kafka_provider("my_topic".to_string());
|
||||
let wal_entry = WalEntry {
|
||||
mutations: vec![Mutation {
|
||||
op_type: OpType::Put as i32,
|
||||
sequence: 1u64,
|
||||
rows: None,
|
||||
}],
|
||||
};
|
||||
let encoded_entry = wal_entry.encode_to_vec();
|
||||
let parts = encoded_entry
|
||||
.chunks(encoded_entry.len() / 2)
|
||||
.map(Into::into)
|
||||
.collect::<Vec<_>>();
|
||||
let raw_entry_stream = MockRawEntryStream {
|
||||
entries: vec![
|
||||
Entry::MultiplePart(MultiplePartEntry {
|
||||
provider: provider.clone(),
|
||||
region_id: RegionId::new(1, 1),
|
||||
entry_id: 2,
|
||||
headers: vec![MultiplePartHeader::First, MultiplePartHeader::Last],
|
||||
parts,
|
||||
}),
|
||||
// The tail corrupted data.
|
||||
Entry::MultiplePart(MultiplePartEntry {
|
||||
provider: provider.clone(),
|
||||
region_id: RegionId::new(1, 1),
|
||||
entry_id: 1,
|
||||
headers: vec![MultiplePartHeader::Last],
|
||||
parts: vec![vec![1; 100]],
|
||||
}),
|
||||
],
|
||||
};
|
||||
|
||||
let reader = LogStoreEntryReader::new(raw_entry_stream);
|
||||
let entries = reader
|
||||
.read(&provider, 0)
|
||||
.unwrap()
|
||||
.try_collect::<Vec<_>>()
|
||||
.await
|
||||
.unwrap()
|
||||
.into_iter()
|
||||
.map(|(_, entry)| entry)
|
||||
.collect::<Vec<_>>();
|
||||
|
||||
assert_eq!(entries, vec![wal_entry]);
|
||||
}
|
||||
|
||||
#[tokio::test]
|
||||
async fn test_corrupted_stream() {
|
||||
let provider = Provider::kafka_provider("my_topic".to_string());
|
||||
let raw_entry_stream = MockRawEntryStream {
|
||||
entries: vec![
|
||||
Entry::MultiplePart(MultiplePartEntry {
|
||||
provider: provider.clone(),
|
||||
region_id: RegionId::new(1, 1),
|
||||
entry_id: 1,
|
||||
headers: vec![MultiplePartHeader::Last],
|
||||
parts: vec![vec![1; 100]],
|
||||
}),
|
||||
Entry::MultiplePart(MultiplePartEntry {
|
||||
provider: provider.clone(),
|
||||
region_id: RegionId::new(1, 1),
|
||||
entry_id: 2,
|
||||
headers: vec![MultiplePartHeader::First],
|
||||
parts: vec![vec![1; 100]],
|
||||
}),
|
||||
],
|
||||
};
|
||||
|
||||
let reader = LogStoreEntryReader::new(raw_entry_stream);
|
||||
let err = reader
|
||||
.read(&provider, 0)
|
||||
.unwrap()
|
||||
.try_collect::<Vec<_>>()
|
||||
.await
|
||||
.unwrap_err();
|
||||
assert_matches!(err, error::Error::CorruptedEntry { .. });
|
||||
}
|
||||
}
|
||||
|
||||
@@ -51,7 +51,7 @@ use crate::error::{JoinSnafu, Result, WorkerStoppedSnafu};
|
||||
use crate::flush::{FlushScheduler, WriteBufferManagerImpl, WriteBufferManagerRef};
|
||||
use crate::memtable::MemtableBuilderProvider;
|
||||
use crate::metrics::WRITE_STALL_TOTAL;
|
||||
use crate::region::{MitoRegionRef, RegionMap, RegionMapRef};
|
||||
use crate::region::{MitoRegionRef, OpeningRegions, OpeningRegionsRef, RegionMap, RegionMapRef};
|
||||
use crate::request::{
|
||||
BackgroundNotify, DdlRequest, SenderDdlRequest, SenderWriteRequest, WorkerRequest,
|
||||
};
|
||||
@@ -212,6 +212,11 @@ impl WorkerGroup {
|
||||
self.worker(region_id).is_region_exists(region_id)
|
||||
}
|
||||
|
||||
/// Returns true if the specific region is opening.
|
||||
pub(crate) fn is_region_opening(&self, region_id: RegionId) -> bool {
|
||||
self.worker(region_id).is_region_opening(region_id)
|
||||
}
|
||||
|
||||
/// Returns region of specific `region_id`.
|
||||
///
|
||||
/// This method should not be public.
|
||||
@@ -225,7 +230,7 @@ impl WorkerGroup {
|
||||
}
|
||||
|
||||
/// Get worker for specific `region_id`.
|
||||
fn worker(&self, region_id: RegionId) -> &RegionWorker {
|
||||
pub(crate) fn worker(&self, region_id: RegionId) -> &RegionWorker {
|
||||
let index = region_id_to_index(region_id, self.workers.len());
|
||||
|
||||
&self.workers[index]
|
||||
@@ -364,6 +369,7 @@ impl<S: LogStore> WorkerStarter<S> {
|
||||
/// Starts a region worker and its background thread.
|
||||
fn start(self) -> RegionWorker {
|
||||
let regions = Arc::new(RegionMap::default());
|
||||
let opening_regions = Arc::new(OpeningRegions::default());
|
||||
let (sender, receiver) = mpsc::channel(self.config.worker_channel_size);
|
||||
|
||||
let running = Arc::new(AtomicBool::new(true));
|
||||
@@ -373,6 +379,7 @@ impl<S: LogStore> WorkerStarter<S> {
|
||||
config: self.config.clone(),
|
||||
regions: regions.clone(),
|
||||
dropping_regions: Arc::new(RegionMap::default()),
|
||||
opening_regions: opening_regions.clone(),
|
||||
sender: sender.clone(),
|
||||
receiver,
|
||||
wal: Wal::new(self.log_store),
|
||||
@@ -409,6 +416,7 @@ impl<S: LogStore> WorkerStarter<S> {
|
||||
RegionWorker {
|
||||
id: self.id,
|
||||
regions,
|
||||
opening_regions,
|
||||
sender,
|
||||
handle: Mutex::new(Some(handle)),
|
||||
running,
|
||||
@@ -422,6 +430,8 @@ pub(crate) struct RegionWorker {
|
||||
id: WorkerId,
|
||||
/// Regions bound to the worker.
|
||||
regions: RegionMapRef,
|
||||
/// The opening regions.
|
||||
opening_regions: OpeningRegionsRef,
|
||||
/// Request sender.
|
||||
sender: Sender<WorkerRequest>,
|
||||
/// Handle to the worker thread.
|
||||
@@ -481,10 +491,21 @@ impl RegionWorker {
|
||||
self.regions.is_region_exists(region_id)
|
||||
}
|
||||
|
||||
/// Returns true if the region is opening.
|
||||
fn is_region_opening(&self, region_id: RegionId) -> bool {
|
||||
self.opening_regions.is_region_exists(region_id)
|
||||
}
|
||||
|
||||
/// Returns region of specific `region_id`.
|
||||
fn get_region(&self, region_id: RegionId) -> Option<MitoRegionRef> {
|
||||
self.regions.get_region(region_id)
|
||||
}
|
||||
|
||||
#[cfg(test)]
|
||||
/// Returns the [OpeningRegionsRef].
|
||||
pub(crate) fn opening_regions(&self) -> &OpeningRegionsRef {
|
||||
&self.opening_regions
|
||||
}
|
||||
}
|
||||
|
||||
impl Drop for RegionWorker {
|
||||
@@ -531,6 +552,8 @@ struct RegionWorkerLoop<S> {
|
||||
regions: RegionMapRef,
|
||||
/// Regions that are not yet fully dropped.
|
||||
dropping_regions: RegionMapRef,
|
||||
/// Regions that are opening.
|
||||
opening_regions: OpeningRegionsRef,
|
||||
/// Request sender.
|
||||
sender: Sender<WorkerRequest>,
|
||||
/// Request receiver.
|
||||
@@ -698,7 +721,11 @@ impl<S: LogStore> RegionWorkerLoop<S> {
|
||||
let res = match ddl.request {
|
||||
DdlRequest::Create(req) => self.handle_create_request(ddl.region_id, req).await,
|
||||
DdlRequest::Drop(_) => self.handle_drop_request(ddl.region_id).await,
|
||||
DdlRequest::Open(req) => self.handle_open_request(ddl.region_id, req).await,
|
||||
DdlRequest::Open(req) => {
|
||||
self.handle_open_request(ddl.region_id, req, ddl.sender)
|
||||
.await;
|
||||
continue;
|
||||
}
|
||||
DdlRequest::Close(_) => self.handle_close_request(ddl.region_id).await,
|
||||
DdlRequest::Alter(req) => {
|
||||
self.handle_alter_request(ddl.region_id, req, ddl.sender)
|
||||
|
||||
Some files were not shown because too many files have changed in this diff Show More
Reference in New Issue
Block a user