Compare commits

..

17 Commits

Author SHA1 Message Date
discord9
05751084e7 chore: bump to v0.8.1 (#4055) 2024-05-30 07:59:52 +00:00
Jeremyhi
8b6596faa0 feat: avoid some cloning when mirror requests to flownode (#4068)
* feat: some refactor mirror requests to flownode

* feat: use spawn_bg to avoid impact front-ground write

* feat: add mirror row count metric
2024-05-30 07:29:13 +00:00
Weny Xu
eab309ff7e fix: avoid acquiring lock during reading stats (#4070)
* fix: avoid acquiring lock during reading stats

* chore: apply suggestions from CR

* chore: apply suggestions from CR
2024-05-30 07:08:04 +00:00
discord9
7de336f087 fix(flow): correctness bugs (#4018)
* fix: optional args of tumble

* fix(WIP): choose

* feat: rename default ts to GREPTIME_TIMESTAMP

* fix: default timestamp name

* fix: reorder write requests

* fix: expire state

* fix: test of tumble

* fix: send buf clear

* fix: ts<start time correct window

* fix: window_start when ts<start_time

* mend

* fix: range begin>range end

* refactor: per reviews

* feat!: ts placeholder rename to __ts_placeholder

* refactor: better condition

* tests(WIP): func sig choose

* tests(WIP): tumble func

* feat: make run_available optional blocking

* tests: tumble transform

* chore: clippy

* fix?: lagged missing data

* fix: flow source break on empty chnl
2024-05-30 03:49:11 +00:00
Weny Xu
6e9a9dc333 refactor(log_store): remove associated type Namespace and Entry in LogStore (#4038)
* refactor(log_store): remove associated type `Namespace` in `LogStore`

* fix(test): filter entries

* refactor: ignore incomplete parts

* refactor: simplify `RawEntryReaderFilter`

* chore: avoid cloning

* test: add tests for `maybe_emit_entry`

* refactor: remove `Namespace` trait and rename `LogStoreNamespace` to `Namespace`

* chore: apply suggestions from CR

* refactor: refine `entry` method signature

* feat: ingore  any potential incomplete parts

* refactor: rename `namespace` to `provider`

* chore: add debug assertion

* refactor: associated type `Entry` in `LogStore`

* refactor: renamse `namespace` to `provider`

* refactor: remove unwrap

* refactor: let `remaining_entries` return a optional vector

* test: add basic tests for kafka logstore

* refactor: move `append` method under `cfg(test)`

* refactor: rename `RawEntry` to `Entry`

* refactor: rename `CorruptedLogEntry` to `CorruptedEntry`

* test: add tests for handling corrupted raw entry stream

* refactor: rename `ns` to `provider`

* refactor: remove `entry_stream.rs` file

* chore: remove unused code

* chore: update comments

* chore: apply suggestions from CR

* chore: update comments

* chore: apply suggestions from CR

* chore: remove Deref

* chore: add comments

* fix: ignores tail corrupted data

* chore: add comments

* fix: add `MIN_BATCH_SIZE` limit
2024-05-29 13:44:01 +00:00
Yingwen
848bd7e553 feat: Implements row group level parallel unordered scanner (#3992)
* feat: unordered scanner

* feat: support compat

* chore: update debug print

fix: missing ranges in scan parts

* fix: ensure chunk size > 0

* fix: parallel is disabled if there is only one file and memtable

* chore: reader metrics

* chore: remove todo

* refactor: add ScanPartBuilder trait

* chore: pass file meta to the part builder

* chore: make part builder private

* docs: update comment

* chore: remove meta()

* refactor: only prune file ranges in ScanInput

replaces ScanPartBuilder with FileRangeCollector which only collect file
ranges

* chore: address typo

* fix: panic when no partition

* feat: Postpone part distribution

* chore: handle empty partition in mito

* style: fix clippy
2024-05-29 11:06:08 +00:00
LYZJU2019
f0effd2680 feat: Implement SHOW STATUS (#4050)
* show status returning empty contents

* return an empty set instead of affected rows

* chore: Update src/query/src/sql.rs

---------

Co-authored-by: Yingwen <realevenyag@gmail.com>
2024-05-29 04:49:05 +00:00
dennis zhuang
aafb468547 fix: set local or session time_zone not work (#4064)
* fix: set local or session time_zone not work

* chore: supports PostgreSQL-specific setting time zone
2024-05-29 00:06:13 +00:00
Weny Xu
4aa756c896 feat: open region in background (#4052)
* feat: open region in background

* feat: trace opening regions

* feat: wait for the opening region

* feat: let engine to handle the future open request

* fix: fix `test_region_registering`
2024-05-28 13:58:15 +00:00
Weny Xu
d3860671a8 chore: add LAST_SENT_HEARTBEAT_ELAPSED metric (#4062) 2024-05-28 08:05:37 +00:00
tison
9dd6e033a7 refactor: move Database to client crate behind testing feature (#4059)
* refactor: move Database to client crate behind testing feature

Signed-off-by: tison <wander4096@gmail.com>

* partial move

Signed-off-by: tison <wander4096@gmail.com>

* catch up more

Signed-off-by: tison <wander4096@gmail.com>

* fix imports

Signed-off-by: tison <wander4096@gmail.com>

* finish

Signed-off-by: tison <wander4096@gmail.com>

* tidy

Signed-off-by: tison <wander4096@gmail.com>

---------

Signed-off-by: tison <wander4096@gmail.com>
2024-05-28 03:21:43 +00:00
Weny Xu
097f62f459 refactor(fuzz-tests): generate ts value separately (#4056)
refactor: generate ts value separately
2024-05-27 14:07:32 +00:00
Weny Xu
048368fd87 feat: invoke flush_table and compact_table in fuzz tests (#4045)
* feat: invoke `flush_table` and `compact_table` in fuzz tests

* feat: support to flush and compact physical metric table

* fix: avoid to create tables with the same name

* feat: validate values after flushing or compacting table
2024-05-27 09:26:50 +00:00
tison
f9db5ff0d6 build(deps): upgrade opendal to 0.46 (#4037)
* build(deps): upgrade opendal to 0.46

Signed-off-by: tison <wander4096@gmail.com>

* migrate writes

Signed-off-by: tison <wander4096@gmail.com>

* migrate reads

Signed-off-by: tison <wander4096@gmail.com>

* fixup object safety

Signed-off-by: tison <wander4096@gmail.com>

* fixup names

Signed-off-by: tison <wander4096@gmail.com>

* fixup compilation

Signed-off-by: tison <wander4096@gmail.com>

* fixup compilation

Signed-off-by: tison <wander4096@gmail.com>

* a few Buffer to Vec

Signed-off-by: tison <wander4096@gmail.com>

* Make greptime buildable with opendal 0.46 (#5)

Signed-off-by: Xuanwo <github@xuanwo.io>

* fixup toml check

Signed-off-by: tison <wander4096@gmail.com>

* test_orc_opener

Signed-off-by: tison <wander4096@gmail.com>

* Fix lru cache (#6)

Signed-off-by: Xuanwo <github@xuanwo.io>

* clippy

Signed-off-by: tison <wander4096@gmail.com>

* improve comments

Signed-off-by: tison <wander4096@gmail.com>

* address comments

Signed-off-by: tison <wander4096@gmail.com>

* reduce buf copy

Signed-off-by: tison <wander4096@gmail.com>

* upgrade to reqwest 0.12

Signed-off-by: tison <wander4096@gmail.com>

---------

Signed-off-by: tison <wander4096@gmail.com>
Signed-off-by: Xuanwo <github@xuanwo.io>
Co-authored-by: Xuanwo <github@xuanwo.io>
2024-05-27 09:12:23 +00:00
Weny Xu
20ce7d428d fix(metric-engine): missing catchup implementation (#4048)
* fix(metric-engine): missing catchup implementation

* fix: should be `metadata_region_id`
2024-05-27 07:56:46 +00:00
Weny Xu
75bddc0bf5 fix(fuzz-tests): avoid to drop in-use database (#4049)
* fix(fuzz-tests): avoid to drop in-use database

* fix: correct datahome path

* fix: correct `schema_name`

* chore: apply suggestions from CR
2024-05-27 07:44:59 +00:00
tison
c78043d526 build(deps): merge tower deps to workspace (#4036)
Signed-off-by: tison <wander4096@gmail.com>
2024-05-27 07:15:30 +00:00
135 changed files with 4064 additions and 2785 deletions

373
Cargo.lock generated

File diff suppressed because it is too large Load Diff

View File

@@ -64,7 +64,7 @@ members = [
resolver = "2"
[workspace.package]
version = "0.8.0"
version = "0.8.1"
edition = "2021"
license = "Apache-2.0"
@@ -146,7 +146,7 @@ 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",
@@ -172,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"
@@ -232,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"

View File

@@ -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

View File

@@ -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);

View File

@@ -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(())
)
}
}

View File

@@ -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};

View File

@@ -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"

View File

@@ -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;

View File

@@ -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(())
}
}

View File

@@ -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,

View File

@@ -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(),

View File

@@ -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);

View File

@@ -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 })
});

View File

@@ -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);

View File

@@ -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);

View File

@@ -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);

View File

@@ -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 })
})
}

View File

@@ -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.

View File

@@ -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"

View File

@@ -179,7 +179,7 @@ impl StateStore for ObjectStateStore {
))
})
.context(ListStateSnafu { path: key })?;
yield (key.into(), value);
yield (key.into(), value.to_vec());
}
}
});

View File

@@ -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(),

View File

@@ -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");

View File

@@ -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",

View File

@@ -189,7 +189,7 @@ impl RegionServer {
pub async fn region_disk_usage(&self, region_id: RegionId) -> Option<i64> {
match self.inner.region_map.get(&region_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(&region_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(&region_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 {

View File

@@ -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)?;

View File

@@ -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!()
}

View File

@@ -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
}

View File

@@ -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())
}

View File

@@ -26,7 +26,6 @@ use common_base::Plugins;
use common_error::ext::BoxedError;
use common_frontend::handler::FrontendInvoker;
use common_meta::key::TableMetadataManagerRef;
use common_query::prelude::GREPTIME_TIMESTAMP;
use common_runtime::JoinHandle;
use common_telemetry::{debug, info};
use datatypes::schema::ColumnSchema;
@@ -36,12 +35,12 @@ 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::table_source::TableSource;
use crate::adapter::util::column_schemas_to_proto;
@@ -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 == GREPTIME_TIMESTAMP)
.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
@@ -314,13 +324,13 @@ 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(
GREPTIME_TIMESTAMP,
AUTO_CREATED_PLACEHOLDER_TS_COL,
ConcreteDataType::timestamp_millisecond_datatype(),
true,
)
@@ -348,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!(
@@ -357,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) => {
@@ -379,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,
@@ -499,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;
}
@@ -510,17 +524,33 @@ 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> {
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
match self.node_context.lock().await.flush_all_sender() {
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);
@@ -528,7 +558,6 @@ impl FlownodeManager {
}
};
// if no thing in send buf then break
let buf_len = self.node_context.lock().await.get_send_buf_size();
if buf_len == 0 {
break;
} else {
@@ -553,7 +582,7 @@ 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().await?;
// self.run_available(false).await?;
Ok(())
}
}
@@ -666,7 +695,9 @@ impl FlownodeManager {
/// TSO coord mess
#[derive(Clone, Debug)]
pub struct FlowTickManager {
/// The starting instant of the flow, used with `start_timestamp` to calculate the current timestamp
start: Instant,
/// The timestamp when the flow started
start_timestamp: repr::Timestamp,
}

View File

@@ -71,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(),
}
}
@@ -107,7 +108,7 @@ impl SourceSender {
}
if row_cnt > 0 {
debug!("Send {} rows", row_cnt);
debug!("Send buf len = {}", self.send_buf.len());
debug!("Remaining Send buf.len() = {}", self.send_buf.len());
}
Ok(row_cnt)

View File

@@ -406,10 +406,15 @@ fn reduce_accum_subgraph(
err_collector.run(|| {
if let Some(expired) = expire_man.get_expire_duration(now, &key)? {
is_expired = true;
DataAlreadyExpiredSnafu {
expired_by: expired,
}
.fail()
// 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(())
}

View File

@@ -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)]
@@ -65,11 +67,33 @@ impl<'referred, 'df> Context<'referred, 'df> {
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();

View File

@@ -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,
@@ -579,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)
///
@@ -770,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,

View File

@@ -101,7 +101,7 @@ impl TypedExpr {
.unzip();
match arg_len {
1 if UnaryFunc::from_str_and_type(fn_name, None).is_ok() => {
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());
@@ -123,13 +123,7 @@ impl TypedExpr {
Ok(TypedExpr::new(arg.call_unary(func), ret_type))
}
2 if BinaryFunc::from_str_expr_and_type(
fn_name,
&arg_exprs,
arg_types.get(0..2).expect("arg have 2 elements"),
)
.is_ok() =>
{
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])?;
@@ -171,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,
@@ -179,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),
@@ -328,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};
@@ -514,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)),
)
}
}

View File

@@ -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

View File

@@ -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)?;
}

View File

@@ -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)]

View File

@@ -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);
}
}

View File

@@ -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)

View File

@@ -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(&region_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(&region_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);
}
}
}

View File

@@ -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(&region_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 { .. });
}
}

View File

@@ -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;

View File

@@ -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();
}
}

View File

@@ -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,
})
}
}

View File

@@ -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);
}
}

View File

@@ -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"] }

View File

@@ -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"

View File

@@ -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());
}
}

View 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)
}
}

View File

@@ -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(),

View File

@@ -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());
}
}

View File

@@ -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]

View File

@@ -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.

View File

@@ -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",

View File

@@ -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()

View File

@@ -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

View File

@@ -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
);
}

View File

@@ -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,

View File

@@ -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

View File

@@ -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 {

View File

@@ -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,
}
}
}

View File

@@ -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)
}
}

View File

@@ -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>,
);
}

View File

@@ -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
}
}

View File

@@ -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>;
@@ -232,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();
@@ -242,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,
@@ -471,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(&region_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(&region_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(&region_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;

View File

@@ -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, &region_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(&region_options.wal_options);
let object_store = self.object_store(&region_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: {}",

View File

@@ -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(())

View File

@@ -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
}
}

View File

@@ -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(&region_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(&region_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(),

View File

@@ -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,

View File

@@ -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.
///

View File

@@ -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)

View File

@@ -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 {

View File

@@ -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));

View File

@@ -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>()

View File

@@ -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
}
}

View File

@@ -109,6 +109,7 @@ impl SchedulerEnv {
compress_type: CompressionType::Uncompressed,
checkpoint_distance: 10,
},
Default::default(),
)
.await
.unwrap(),

View File

@@ -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);
}

View File

@@ -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
);

View File

@@ -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 { .. });
}
}

View File

@@ -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)

View File

@@ -75,7 +75,7 @@ impl<S: LogStore> RegionWorkerLoop<S> {
let timer = Instant::now();
let last_entry_id = replay_memtable(
&self.wal,
&region.wal_options,
&region.provider,
region_id,
flushed_entry_id,
&region.version_control,

View File

@@ -16,6 +16,7 @@
use std::time::Duration;
use bytes::Bytes;
use common_telemetry::{error, info, warn};
use futures::TryStreamExt;
use object_store::util::join_path;
@@ -50,7 +51,7 @@ impl<S> RegionWorkerLoop<S> {
region
.access_layer
.object_store()
.write(&marker_path, vec![])
.write(&marker_path, Bytes::new())
.await
.context(OpenDalSnafu)
.inspect_err(|e| {

View File

@@ -212,7 +212,7 @@ impl<S: LogStore> RegionWorkerLoop<S> {
);
if let Err(e) = self
.wal
.obsolete(region_id, request.flushed_entry_id, &region.wal_options)
.obsolete(region_id, request.flushed_entry_id, &region.provider)
.await
{
error!(e; "Failed to write wal, region: {}", region_id);

View File

@@ -20,24 +20,24 @@ use common_telemetry::info;
use object_store::util::join_path;
use snafu::{OptionExt, ResultExt};
use store_api::logstore::LogStore;
use store_api::region_request::{AffectedRows, RegionOpenRequest};
use store_api::region_request::RegionOpenRequest;
use store_api::storage::RegionId;
use crate::error::{ObjectStoreNotFoundSnafu, OpenDalSnafu, RegionNotFoundSnafu, Result};
use crate::error::{
ObjectStoreNotFoundSnafu, OpenDalSnafu, OpenRegionSnafu, RegionNotFoundSnafu, Result,
};
use crate::metrics::REGION_COUNT;
use crate::region::opener::RegionOpener;
use crate::request::OptionOutputTx;
use crate::worker::handle_drop::remove_region_dir_once;
use crate::worker::{RegionWorkerLoop, DROPPING_MARKER_FILE};
impl<S: LogStore> RegionWorkerLoop<S> {
pub(crate) async fn handle_open_request(
&mut self,
async fn check_and_cleanup_region(
&self,
region_id: RegionId,
request: RegionOpenRequest,
) -> Result<AffectedRows> {
if self.regions.is_region_exists(region_id) {
return Ok(0);
}
request: &RegionOpenRequest,
) -> Result<()> {
let object_store = if let Some(storage_name) = request.options.get("storage") {
self.object_store_manager
.find(storage_name)
@@ -59,10 +59,33 @@ impl<S: LogStore> RegionWorkerLoop<S> {
return RegionNotFoundSnafu { region_id }.fail();
}
Ok(())
}
pub(crate) async fn handle_open_request(
&mut self,
region_id: RegionId,
request: RegionOpenRequest,
sender: OptionOutputTx,
) {
if self.regions.is_region_exists(region_id) {
sender.send(Ok(0));
return;
}
let Some(sender) = self
.opening_regions
.wait_for_opening_region(region_id, sender)
else {
return;
};
if let Err(err) = self.check_and_cleanup_region(region_id, &request).await {
sender.send(Err(err));
return;
}
info!("Try to open region {}", region_id);
// Open region from specific region dir.
let region = RegionOpener::new(
let opener = match RegionOpener::new(
region_id,
&request.region_dir,
self.memtable_builder_provider.clone(),
@@ -71,18 +94,43 @@ impl<S: LogStore> RegionWorkerLoop<S> {
self.intermediate_manager.clone(),
)
.skip_wal_replay(request.skip_wal_replay)
.parse_options(request.options)?
.cache(Some(self.cache_manager.clone()))
.open(&self.config, &self.wal)
.await?;
.parse_options(request.options)
{
Ok(opener) => opener,
Err(err) => {
sender.send(Err(err));
return;
}
};
info!("Region {} is opened", region_id);
let regions = self.regions.clone();
let wal = self.wal.clone();
let config = self.config.clone();
let opening_regions = self.opening_regions.clone();
opening_regions.insert_sender(region_id, sender);
common_runtime::spawn_bg(async move {
match opener.open(&config, &wal).await {
Ok(region) => {
info!("Region {} is opened", region_id);
REGION_COUNT.inc();
REGION_COUNT.inc();
// Insert the Region into the RegionMap.
regions.insert_region(Arc::new(region));
// Insert the MitoRegion into the RegionMap.
self.regions.insert_region(Arc::new(region));
Ok(0)
let senders = opening_regions.remove_sender(region_id);
for sender in senders {
sender.send(Ok(0));
}
}
Err(err) => {
let senders = opening_regions.remove_sender(region_id);
let err = Arc::new(err);
for sender in senders {
sender.send(Err(err.clone()).context(OpenRegionSnafu));
}
}
}
});
}
}

View File

@@ -82,7 +82,7 @@ impl<S: LogStore> RegionWorkerLoop<S> {
.obsolete(
region_id,
truncate_result.truncated_entry_id,
&region.wal_options,
&region.provider,
)
.await
{

View File

@@ -84,8 +84,7 @@ impl<S: LogStore> RegionWorkerLoop<S> {
for (region_id, region_ctx) in region_ctxs.iter_mut() {
// Safety: the log store implementation ensures that either the `write_to_wal` fails and no
// response is returned or the last entry ids for each region do exist.
let last_entry_id =
response.last_entry_ids.get(&region_id.as_u64()).unwrap();
let last_entry_id = response.last_entry_ids.get(region_id).unwrap();
region_ctx.set_next_entry_id(last_entry_id + 1);
}
}
@@ -162,7 +161,7 @@ impl<S> RegionWorkerLoop<S> {
let region_ctx = RegionWriteCtx::new(
region.region_id,
&region.version_control,
region.wal_options.clone(),
region.provider.clone(),
);
e.insert(region_ctx);

View File

@@ -11,23 +11,21 @@ workspace = true
services-memory = ["opendal/services-memory"]
[dependencies]
async-trait = "0.1"
bytes.workspace = true
common-telemetry.workspace = true
futures.workspace = true
lazy_static.workspace = true
md5 = "0.7"
moka = { workspace = true, features = ["future"] }
opendal = { version = "0.45", features = [
opendal = { version = "0.46", features = [
"layers-tracing",
"rustls",
"services-azblob",
"services-fs",
"services-gcs",
"services-http",
"services-oss",
"services-s3",
], default-features = false }
] }
prometheus.workspace = true
uuid.workspace = true
@@ -35,5 +33,4 @@ uuid.workspace = true
anyhow = "1.0"
common-telemetry.workspace = true
common-test-util.workspace = true
opendal = { version = "0.45", features = ["services-memory"] }
tokio.workspace = true

View File

@@ -14,27 +14,26 @@
use std::sync::Arc;
use async_trait::async_trait;
use opendal::raw::oio::Read;
use opendal::raw::oio::ReadDyn;
use opendal::raw::{
Accessor, Layer, LayeredAccessor, OpDelete, OpList, OpRead, OpWrite, RpDelete, RpList, RpRead,
Access, Layer, LayeredAccess, OpDelete, OpList, OpRead, OpWrite, RpDelete, RpList, RpRead,
RpWrite,
};
use opendal::Result;
use opendal::{Operator, Result};
mod read_cache;
use common_telemetry::info;
use read_cache::ReadCache;
/// An opendal layer with local LRU file cache supporting.
#[derive(Clone)]
pub struct LruCacheLayer<C: Clone> {
pub struct LruCacheLayer {
// The read cache
read_cache: ReadCache<C>,
read_cache: ReadCache,
}
impl<C: Accessor + Clone> LruCacheLayer<C> {
impl LruCacheLayer {
/// Create a `[LruCacheLayer]` with local file cache and capacity in bytes.
pub async fn new(file_cache: Arc<C>, capacity: usize) -> Result<Self> {
pub async fn new(file_cache: Operator, capacity: usize) -> Result<Self> {
let read_cache = ReadCache::new(file_cache, capacity);
let (entries, bytes) = read_cache.recover_cache().await?;
@@ -57,11 +56,11 @@ impl<C: Accessor + Clone> LruCacheLayer<C> {
}
}
impl<I: Accessor, C: Accessor + Clone> Layer<I> for LruCacheLayer<C> {
type LayeredAccessor = LruCacheAccessor<I, C>;
impl<I: Access> Layer<I> for LruCacheLayer {
type LayeredAccess = LruCacheAccess<I>;
fn layer(&self, inner: I) -> Self::LayeredAccessor {
LruCacheAccessor {
fn layer(&self, inner: I) -> Self::LayeredAccess {
LruCacheAccess {
inner,
read_cache: self.read_cache.clone(),
}
@@ -69,15 +68,14 @@ impl<I: Accessor, C: Accessor + Clone> Layer<I> for LruCacheLayer<C> {
}
#[derive(Debug)]
pub struct LruCacheAccessor<I, C: Clone> {
pub struct LruCacheAccess<I> {
inner: I,
read_cache: ReadCache<C>,
read_cache: ReadCache,
}
#[async_trait]
impl<I: Accessor, C: Accessor + Clone> LayeredAccessor for LruCacheAccessor<I, C> {
impl<I: Access> LayeredAccess for LruCacheAccess<I> {
type Inner = I;
type Reader = Box<dyn Read>;
type Reader = Arc<dyn ReadDyn>;
type BlockingReader = I::BlockingReader;
type Writer = I::Writer;
type BlockingWriter = I::BlockingWriter;

View File

@@ -15,12 +15,12 @@
use std::sync::Arc;
use common_telemetry::debug;
use futures::FutureExt;
use futures::{FutureExt, StreamExt};
use moka::future::Cache;
use moka::notification::ListenerFuture;
use opendal::raw::oio::{ListExt, Read, ReadExt, Reader, WriteExt};
use opendal::raw::{Accessor, OpDelete, OpList, OpRead, OpStat, OpWrite, RpRead};
use opendal::{Error as OpendalError, ErrorKind, Result};
use opendal::raw::oio::{Read, ReadDyn, Reader};
use opendal::raw::{Access, BytesRange, OpRead, OpStat, RpRead};
use opendal::{Buffer, Error as OpendalError, ErrorKind, Operator, Result};
use crate::metrics::{
OBJECT_STORE_LRU_CACHE_BYTES, OBJECT_STORE_LRU_CACHE_ENTRIES, OBJECT_STORE_LRU_CACHE_HIT,
@@ -52,26 +52,22 @@ fn can_cache(path: &str) -> bool {
}
/// Generate an unique cache key for the read path and range.
fn read_cache_key(path: &str, args: &OpRead) -> String {
format!(
"{:x}.cache-{}",
md5::compute(path),
args.range().to_header()
)
fn read_cache_key(path: &str, range: BytesRange) -> String {
format!("{:x}.cache-{}", md5::compute(path), range.to_header())
}
/// Local read cache for files in object storage
#[derive(Clone, Debug)]
pub(crate) struct ReadCache<C: Clone> {
pub(crate) struct ReadCache {
/// Local file cache backend
file_cache: Arc<C>,
file_cache: Operator,
/// Local memory cache to track local cache files
mem_cache: Cache<String, ReadResult>,
}
impl<C: Accessor + Clone> ReadCache<C> {
impl ReadCache {
/// Create a [`ReadCache`] with capacity in bytes.
pub(crate) fn new(file_cache: Arc<C>, capacity: usize) -> Self {
pub(crate) fn new(file_cache: Operator, capacity: usize) -> Self {
let file_cache_cloned = file_cache.clone();
let eviction_listener =
move |read_key: Arc<String>, read_result: ReadResult, cause| -> ListenerFuture {
@@ -83,7 +79,7 @@ impl<C: Accessor + Clone> ReadCache<C> {
if let ReadResult::Success(size) = read_result {
OBJECT_STORE_LRU_CACHE_BYTES.sub(size as i64);
let result = file_cache_cloned.delete(&read_key, OpDelete::new()).await;
let result = file_cache_cloned.delete(&read_key).await;
debug!(
"Deleted local cache file `{}`, result: {:?}, cause: {:?}.",
read_key, result, cause
@@ -133,17 +129,17 @@ impl<C: Accessor + Clone> ReadCache<C> {
/// Recover existing cache items from `file_cache` to `mem_cache`.
/// Return entry count and total approximate entry size in bytes.
pub(crate) async fn recover_cache(&self) -> Result<(u64, u64)> {
let (_, mut pager) = self.file_cache.list("/", OpList::default()).await?;
let mut pager = self.file_cache.lister("/").await?;
while let Some(entry) = pager.next().await? {
while let Some(entry) = pager.next().await.transpose()? {
let read_key = entry.path();
// We can't retrieve the metadata from `[opendal::raw::oio::Entry]` directly,
// because it's private field.
let size = {
let stat = self.file_cache.stat(read_key, OpStat::default()).await?;
let stat = self.file_cache.stat(read_key).await?;
stat.into_metadata().content_length()
stat.content_length()
};
OBJECT_STORE_LRU_CACHE_ENTRIES.inc();
@@ -159,8 +155,7 @@ impl<C: Accessor + Clone> ReadCache<C> {
/// Returns true when the read cache contains the specific file.
pub(crate) async fn contains_file(&self, path: &str) -> bool {
self.mem_cache.run_pending_tasks().await;
self.mem_cache.contains_key(path)
&& self.file_cache.stat(path, OpStat::default()).await.is_ok()
self.mem_cache.contains_key(path) && self.file_cache.stat(path).await.is_ok()
}
/// Read from a specific path using the OpRead operation.
@@ -173,86 +168,54 @@ impl<C: Accessor + Clone> ReadCache<C> {
inner: &I,
path: &str,
args: OpRead,
) -> Result<(RpRead, Box<dyn Read>)>
) -> Result<(RpRead, Arc<dyn ReadDyn>)>
where
I: Accessor,
I: Access,
{
if !can_cache(path) {
return inner.read(path, args).await.map(to_output_reader);
}
let read_key = read_cache_key(path, &args);
let read_result = self
.mem_cache
.try_get_with(
read_key.clone(),
self.read_remote(inner, &read_key, path, args.clone()),
)
.await
.map_err(|e| OpendalError::new(e.kind(), &e.to_string()))?;
match read_result {
ReadResult::Success(_) => {
// There is a concurrent issue here, the local cache may be purged
// while reading, we have to fallback to remote read
match self.file_cache.read(&read_key, OpRead::default()).await {
Ok(ret) => {
OBJECT_STORE_LRU_CACHE_HIT
.with_label_values(&["success"])
.inc();
Ok(to_output_reader(ret))
}
Err(_) => {
OBJECT_STORE_LRU_CACHE_MISS.inc();
inner.read(path, args).await.map(to_output_reader)
}
}
}
ReadResult::NotFound => {
OBJECT_STORE_LRU_CACHE_HIT
.with_label_values(&["not_found"])
.inc();
Err(OpendalError::new(
ErrorKind::NotFound,
&format!("File not found: {path}"),
))
}
}
// FIXME: remove this block after opendal v0.47 released.
let meta = inner.stat(path, OpStat::new()).await?;
let (rp, reader) = inner.read(path, args).await?;
let reader: ReadCacheReader<I> = ReadCacheReader {
path: Arc::new(path.to_string()),
inner_reader: reader,
size: meta.into_metadata().content_length(),
file_cache: self.file_cache.clone(),
mem_cache: self.mem_cache.clone(),
};
Ok((rp, Arc::new(reader)))
}
}
async fn try_write_cache<I>(&self, mut reader: I::Reader, read_key: &str) -> Result<usize>
where
I: Accessor,
{
let (_, mut writer) = self.file_cache.write(read_key, OpWrite::new()).await?;
let mut total = 0;
while let Some(bytes) = reader.next().await {
let bytes = &bytes?;
total += bytes.len();
writer.write(bytes).await?;
}
// Call `close` to ensure data is written.
writer.close().await?;
Ok(total)
}
pub struct ReadCacheReader<I: Access> {
/// Path of the file
path: Arc<String>,
/// Remote file reader.
inner_reader: I::Reader,
/// FIXME: remove this field after opendal v0.47 released.
///
/// OpenDAL's read_at takes `offset, limit` which means the underlying storage
/// services could return less data than limit. We store size here as a workaround.
///
/// This API has been refactor into `offset, size` instead. After opendal v0.47 released,
/// we don't need this anymore.
size: u64,
/// Local file cache backend
file_cache: Operator,
/// Local memory cache to track local cache files
mem_cache: Cache<String, ReadResult>,
}
/// Read the file from remote storage. If success, write the content into local cache.
async fn read_remote<I>(
&self,
inner: &I,
read_key: &str,
path: &str,
args: OpRead,
) -> Result<ReadResult>
where
I: Accessor,
{
impl<I: Access> ReadCacheReader<I> {
/// TODO: we can return the Buffer directly to avoid another read from cache.
async fn read_remote(&self, offset: u64, limit: usize) -> Result<ReadResult> {
OBJECT_STORE_LRU_CACHE_MISS.inc();
let (_, reader) = inner.read(path, args).await?;
let result = self.try_write_cache::<I>(reader, read_key).await;
let buf = self.inner_reader.read_at(offset, limit).await?;
let result = self.try_write_cache(buf, offset).await;
match result {
Ok(read_bytes) => {
@@ -279,10 +242,59 @@ impl<C: Accessor + Clone> ReadCache<C> {
}
}
}
async fn try_write_cache(&self, buf: Buffer, offset: u64) -> Result<usize> {
let size = buf.len();
let read_key = read_cache_key(&self.path, BytesRange::new(offset, Some(size as _)));
self.file_cache.write(&read_key, buf).await?;
Ok(size)
}
}
impl<I: Access> Read for ReadCacheReader<I> {
async fn read_at(&self, offset: u64, limit: usize) -> Result<Buffer> {
let size = self.size.min(offset + limit as u64) - offset;
let read_key = read_cache_key(&self.path, BytesRange::new(offset, Some(size as _)));
let read_result = self
.mem_cache
.try_get_with(read_key.clone(), self.read_remote(offset, limit))
.await
.map_err(|e| OpendalError::new(e.kind(), &e.to_string()))?;
match read_result {
ReadResult::Success(_) => {
// There is a concurrent issue here, the local cache may be purged
// while reading, we have to fallback to remote read
match self.file_cache.read(&read_key).await {
Ok(ret) => {
OBJECT_STORE_LRU_CACHE_HIT
.with_label_values(&["success"])
.inc();
Ok(ret)
}
Err(_) => {
OBJECT_STORE_LRU_CACHE_MISS.inc();
self.inner_reader.read_at(offset, limit).await
}
}
}
ReadResult::NotFound => {
OBJECT_STORE_LRU_CACHE_HIT
.with_label_values(&["not_found"])
.inc();
Err(OpendalError::new(
ErrorKind::NotFound,
&format!("File not found: {}", self.path),
))
}
}
}
}
fn to_output_reader<R: Read + 'static>(input: (RpRead, R)) -> (RpRead, Reader) {
(input.0, Box::new(input.1))
(input.0, Arc::new(input.1))
}
#[cfg(test)]

View File

@@ -15,16 +15,11 @@
//! code originally from <https://github.com/apache/incubator-opendal/blob/main/core/src/layers/prometheus.rs>, make a tiny change to avoid crash in multi thread env
use std::fmt::{Debug, Formatter};
use std::io;
use std::task::{Context, Poll};
use async_trait::async_trait;
use bytes::Bytes;
use common_telemetry::debug;
use futures::FutureExt;
use lazy_static::lazy_static;
use opendal::raw::*;
use opendal::ErrorKind;
use opendal::{Buffer, ErrorKind};
use prometheus::{
exponential_buckets, histogram_opts, register_histogram_vec, register_int_counter_vec,
Histogram, HistogramTimer, HistogramVec, IntCounterVec,
@@ -89,14 +84,14 @@ fn increment_errors_total(op: Operation, kind: ErrorKind) {
#[derive(Default, Debug, Clone)]
pub struct PrometheusMetricsLayer;
impl<A: Accessor> Layer<A> for PrometheusMetricsLayer {
type LayeredAccessor = PrometheusAccessor<A>;
impl<A: Access> Layer<A> for PrometheusMetricsLayer {
type LayeredAccess = PrometheusAccess<A>;
fn layer(&self, inner: A) -> Self::LayeredAccessor {
fn layer(&self, inner: A) -> Self::LayeredAccess {
let meta = inner.info();
let scheme = meta.scheme();
PrometheusAccessor {
PrometheusAccess {
inner,
scheme: scheme.to_string(),
}
@@ -104,12 +99,12 @@ impl<A: Accessor> Layer<A> for PrometheusMetricsLayer {
}
#[derive(Clone)]
pub struct PrometheusAccessor<A: Accessor> {
pub struct PrometheusAccess<A: Access> {
inner: A,
scheme: String,
}
impl<A: Accessor> Debug for PrometheusAccessor<A> {
impl<A: Access> Debug for PrometheusAccess<A> {
fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result {
f.debug_struct("PrometheusAccessor")
.field("inner", &self.inner)
@@ -117,8 +112,7 @@ impl<A: Accessor> Debug for PrometheusAccessor<A> {
}
}
#[async_trait]
impl<A: Accessor> LayeredAccessor for PrometheusAccessor<A> {
impl<A: Access> LayeredAccess for PrometheusAccess<A> {
type Inner = A;
type Reader = PrometheusMetricWrapper<A::Reader>;
type BlockingReader = PrometheusMetricWrapper<A::BlockingReader>;
@@ -157,27 +151,20 @@ impl<A: Accessor> LayeredAccessor for PrometheusAccessor<A> {
.with_label_values(&[&self.scheme, Operation::Read.into_static()])
.start_timer();
self.inner
.read(path, args)
.map(|v| {
v.map(|(rp, r)| {
(
rp,
PrometheusMetricWrapper::new(
r,
Operation::Read,
BYTES_TOTAL
.with_label_values(&[&self.scheme, Operation::Read.into_static()]),
timer,
),
)
})
})
.await
.map_err(|e| {
increment_errors_total(Operation::Read, e.kind());
e
})
let (rp, r) = self.inner.read(path, args).await.map_err(|e| {
increment_errors_total(Operation::Read, e.kind());
e
})?;
Ok((
rp,
PrometheusMetricWrapper::new(
r,
Operation::Read,
BYTES_TOTAL.with_label_values(&[&self.scheme, Operation::Read.into_static()]),
timer,
),
))
}
async fn write(&self, path: &str, args: OpWrite) -> Result<(RpWrite, Self::Writer)> {
@@ -189,27 +176,20 @@ impl<A: Accessor> LayeredAccessor for PrometheusAccessor<A> {
.with_label_values(&[&self.scheme, Operation::Write.into_static()])
.start_timer();
self.inner
.write(path, args)
.map(|v| {
v.map(|(rp, r)| {
(
rp,
PrometheusMetricWrapper::new(
r,
Operation::Write,
BYTES_TOTAL
.with_label_values(&[&self.scheme, Operation::Write.into_static()]),
timer,
),
)
})
})
.await
.map_err(|e| {
increment_errors_total(Operation::Write, e.kind());
e
})
let (rp, r) = self.inner.write(path, args).await.map_err(|e| {
increment_errors_total(Operation::Write, e.kind());
e
})?;
Ok((
rp,
PrometheusMetricWrapper::new(
r,
Operation::Write,
BYTES_TOTAL.with_label_values(&[&self.scheme, Operation::Write.into_static()]),
timer,
),
))
}
async fn stat(&self, path: &str, args: OpStat) -> Result<RpStat> {
@@ -461,103 +441,46 @@ impl<R> PrometheusMetricWrapper<R> {
}
impl<R: oio::Read> oio::Read for PrometheusMetricWrapper<R> {
fn poll_read(&mut self, cx: &mut Context<'_>, buf: &mut [u8]) -> Poll<Result<usize>> {
self.inner.poll_read(cx, buf).map(|res| match res {
Ok(bytes) => {
self.bytes += bytes as u64;
Ok(bytes)
}
Err(e) => {
increment_errors_total(self.op, e.kind());
Err(e)
}
})
}
fn poll_seek(&mut self, cx: &mut Context<'_>, pos: io::SeekFrom) -> Poll<Result<u64>> {
self.inner.poll_seek(cx, pos).map(|res| match res {
Ok(n) => Ok(n),
Err(e) => {
increment_errors_total(self.op, e.kind());
Err(e)
}
})
}
fn poll_next(&mut self, cx: &mut Context<'_>) -> Poll<Option<Result<Bytes>>> {
self.inner.poll_next(cx).map(|res| match res {
Some(Ok(bytes)) => {
self.bytes += bytes.len() as u64;
Some(Ok(bytes))
}
Some(Err(e)) => {
increment_errors_total(self.op, e.kind());
Some(Err(e))
}
None => None,
async fn read_at(&self, offset: u64, limit: usize) -> Result<Buffer> {
self.inner.read_at(offset, limit).await.map_err(|err| {
increment_errors_total(self.op, err.kind());
err
})
}
}
impl<R: oio::BlockingRead> oio::BlockingRead for PrometheusMetricWrapper<R> {
fn read(&mut self, buf: &mut [u8]) -> Result<usize> {
self.inner
.read(buf)
.map(|n| {
self.bytes += n as u64;
n
})
.map_err(|e| {
increment_errors_total(self.op, e.kind());
e
})
}
fn seek(&mut self, pos: io::SeekFrom) -> Result<u64> {
self.inner.seek(pos).map_err(|err| {
fn read_at(&self, offset: u64, limit: usize) -> opendal::Result<Buffer> {
self.inner.read_at(offset, limit).map_err(|err| {
increment_errors_total(self.op, err.kind());
err
})
}
fn next(&mut self) -> Option<Result<Bytes>> {
self.inner.next().map(|res| match res {
Ok(bytes) => {
self.bytes += bytes.len() as u64;
Ok(bytes)
}
Err(e) => {
increment_errors_total(self.op, e.kind());
Err(e)
}
})
}
}
#[async_trait]
impl<R: oio::Write> oio::Write for PrometheusMetricWrapper<R> {
fn poll_write(&mut self, cx: &mut Context<'_>, bs: &dyn oio::WriteBuf) -> Poll<Result<usize>> {
self.inner
.poll_write(cx, bs)
.map_ok(|n| {
async fn write(&mut self, bs: Buffer) -> Result<usize> {
match self.inner.write(bs).await {
Ok(n) => {
self.bytes += n as u64;
n
})
.map_err(|err| {
Ok(n)
}
Err(err) => {
increment_errors_total(self.op, err.kind());
err
})
Err(err)
}
}
}
fn poll_abort(&mut self, cx: &mut Context<'_>) -> Poll<Result<()>> {
self.inner.poll_abort(cx).map_err(|err| {
async fn close(&mut self) -> Result<()> {
self.inner.close().await.map_err(|err| {
increment_errors_total(self.op, err.kind());
err
})
}
fn poll_close(&mut self, cx: &mut Context<'_>) -> Poll<Result<()>> {
self.inner.poll_close(cx).map_err(|err| {
async fn abort(&mut self) -> Result<()> {
self.inner.close().await.map_err(|err| {
increment_errors_total(self.op, err.kind());
err
})
@@ -565,7 +488,7 @@ impl<R: oio::Write> oio::Write for PrometheusMetricWrapper<R> {
}
impl<R: oio::BlockingWrite> oio::BlockingWrite for PrometheusMetricWrapper<R> {
fn write(&mut self, bs: &dyn oio::WriteBuf) -> Result<usize> {
fn write(&mut self, bs: Buffer) -> Result<usize> {
self.inner
.write(bs)
.map(|n| {

View File

@@ -14,8 +14,9 @@
pub use opendal::raw::{normalize_path as raw_normalize_path, HttpClient};
pub use opendal::{
services, Builder as ObjectStoreBuilder, Entry, EntryMode, Error, ErrorKind, Lister, Metakey,
Operator as ObjectStore, Reader, Result, Writer,
services, Builder as ObjectStoreBuilder, Entry, EntryMode, Error, ErrorKind,
FuturesAsyncReader, FuturesAsyncWriter, Lister, Metakey, Operator as ObjectStore, Reader,
Result, Writer,
};
pub mod layers;

View File

@@ -22,7 +22,6 @@ use object_store::layers::LruCacheLayer;
use object_store::services::{Fs, S3};
use object_store::test_util::TempFolder;
use object_store::{ObjectStore, ObjectStoreBuilder};
use opendal::raw::Accessor;
use opendal::services::{Azblob, Gcs, Oss};
use opendal::{EntryMode, Operator, OperatorBuilder};
@@ -36,11 +35,11 @@ async fn test_object_crud(store: &ObjectStore) -> Result<()> {
// Read data from object;
let bs = store.read(file_name).await?;
assert_eq!("Hello, World!", String::from_utf8(bs)?);
assert_eq!("Hello, World!", String::from_utf8(bs.to_vec())?);
// Read range from object;
let bs = store.read_with(file_name).range(1..=11).await?;
assert_eq!("ello, World", String::from_utf8(bs)?);
assert_eq!("ello, World", String::from_utf8(bs.to_vec())?);
// Get object's Metadata
let meta = store.stat(file_name).await?;
@@ -77,7 +76,7 @@ async fn test_object_list(store: &ObjectStore) -> Result<()> {
assert_eq!(p2, entries.first().unwrap().path());
let content = store.read(p2).await?;
assert_eq!("Hello, object2!", String::from_utf8(content)?);
assert_eq!("Hello, object2!", String::from_utf8(content.to_vec())?);
store.delete(p2).await?;
let entries = store.list("/").await?;
@@ -236,11 +235,9 @@ async fn test_file_backend_with_lru_cache() -> Result<()> {
let _ = builder
.root(&cache_dir.path().to_string_lossy())
.atomic_write_dir(&cache_dir.path().to_string_lossy());
let file_cache = Arc::new(builder.build().unwrap());
let file_cache = Operator::new(builder).unwrap().finish();
LruCacheLayer::new(Arc::new(file_cache.clone()), 32)
.await
.unwrap()
LruCacheLayer::new(file_cache, 32).await.unwrap()
};
let store = store.layer(cache_layer.clone());
@@ -253,10 +250,7 @@ async fn test_file_backend_with_lru_cache() -> Result<()> {
Ok(())
}
async fn assert_lru_cache<C: Accessor + Clone>(
cache_layer: &LruCacheLayer<C>,
file_names: &[&str],
) {
async fn assert_lru_cache(cache_layer: &LruCacheLayer, file_names: &[&str]) {
for file_name in file_names {
assert!(cache_layer.contains_file(file_name).await);
}
@@ -278,7 +272,7 @@ async fn assert_cache_files(
let bs = store.read(o.path()).await.unwrap();
assert_eq!(
file_contents[position],
String::from_utf8(bs.clone())?,
String::from_utf8(bs.to_vec())?,
"file content not match: {}",
o.name()
);
@@ -312,9 +306,7 @@ async fn test_object_store_cache_policy() -> Result<()> {
let cache_store = OperatorBuilder::new(file_cache.clone()).finish();
// create operator for cache dir to verify cache file
let cache_layer = LruCacheLayer::new(Arc::new(file_cache.clone()), 38)
.await
.unwrap();
let cache_layer = LruCacheLayer::new(cache_store.clone(), 38).await.unwrap();
let store = store.layer(cache_layer.clone());
// create several object handler.
@@ -386,7 +378,7 @@ async fn test_object_store_cache_policy() -> Result<()> {
// instead of returning `NotFound` during the reader creation.
// The entry count is 4, because we have the p2 `NotFound` cache.
assert!(store.read_with(p2).range(0..4).await.is_err());
assert_eq!(cache_layer.read_cache_stat().await, (4, 35));
assert_eq!(cache_layer.read_cache_stat().await, (3, 35));
assert_cache_files(
&cache_store,
@@ -414,7 +406,7 @@ async fn test_object_store_cache_policy() -> Result<()> {
assert!(store.read(p2).await.is_err());
// Read p1 with range `1..` , the existing p1 with range `0..` must be evicted.
let _ = store.read_with(p1).range(1..15).await.unwrap();
assert_eq!(cache_layer.read_cache_stat().await, (4, 34));
assert_eq!(cache_layer.read_cache_stat().await, (3, 34));
assert_cache_files(
&cache_store,
&[
@@ -442,7 +434,7 @@ async fn test_object_store_cache_policy() -> Result<()> {
drop(cache_layer);
// Test recover
let cache_layer = LruCacheLayer::new(Arc::new(file_cache), 38).await.unwrap();
let cache_layer = LruCacheLayer::new(cache_store, 38).await.unwrap();
// The p2 `NotFound` cache will not be recovered
assert_eq!(cache_layer.read_cache_stat().await, (3, 34));

View File

@@ -56,6 +56,7 @@ store-api.workspace = true
substrait.workspace = true
table.workspace = true
tokio.workspace = true
tokio-util.workspace = true
tonic.workspace = true
[dev-dependencies]

View File

@@ -15,7 +15,6 @@
use std::collections::HashMap;
use std::sync::Arc;
use api::region::RegionResponse;
use api::v1::alter_expr::Kind;
use api::v1::region::{InsertRequests as RegionInsertRequests, RegionRequestHeader};
use api::v1::{
@@ -191,41 +190,6 @@ impl Inserter {
}
impl Inserter {
fn post_request(&self, requests: RegionInsertRequests) {
let node_manager = self.node_manager.clone();
let table_flownode_set_cache = self.table_flownode_set_cache.clone();
// Spawn all tasks that do job for mirror insert requests for flownode
common_runtime::spawn_bg(async move {
match Self::mirror_flow_node_requests(table_flownode_set_cache, requests).await {
Ok(flow_tasks) => {
let flow_tasks = flow_tasks.into_iter().map(|(peer, inserts)| {
let node_manager = node_manager.clone();
common_runtime::spawn_write(async move {
node_manager
.flownode(&peer)
.await
.handle_inserts(inserts)
.await
.map(|flow_response| RegionResponse {
affected_rows: flow_response.affected_rows as AffectedRows,
extension: flow_response.extension,
})
.context(RequestInsertsSnafu)
})
});
if let Err(err) = future::try_join_all(flow_tasks)
.await
.context(JoinTaskSnafu)
{
warn!(err; "Failed to insert data into flownode");
}
}
Err(err) => warn!(err; "Failed to mirror request to flownode"),
}
});
}
async fn do_request(
&self,
requests: RegionInsertRequests,
@@ -238,8 +202,44 @@ impl Inserter {
..Default::default()
});
let tasks = self
.group_requests_by_peer(requests.clone())
// Mirror requests for source table to flownode
match self.mirror_flow_node_requests(&requests).await {
Ok(flow_requests) => {
let node_manager = self.node_manager.clone();
let flow_tasks = flow_requests.into_iter().map(|(peer, inserts)| {
let node_manager = node_manager.clone();
common_runtime::spawn_bg(async move {
node_manager
.flownode(&peer)
.await
.handle_inserts(inserts)
.await
.context(RequestInsertsSnafu)
})
});
match future::try_join_all(flow_tasks)
.await
.context(JoinTaskSnafu)
{
Ok(ret) => {
let affected_rows = ret
.into_iter()
.map(|resp| resp.map(|r| r.affected_rows))
.sum::<Result<u64>>()
.unwrap_or(0);
crate::metrics::DIST_MIRROR_ROW_COUNT.inc_by(affected_rows);
}
Err(err) => {
warn!(err; "Failed to insert data into flownode");
}
}
}
Err(err) => warn!(err; "Failed to mirror request to flownode"),
}
let write_tasks = self
.group_requests_by_peer(requests)
.await?
.into_iter()
.map(|(peer, inserts)| {
@@ -254,8 +254,9 @@ impl Inserter {
.context(RequestInsertsSnafu)
})
});
let results = future::try_join_all(tasks).await.context(JoinTaskSnafu)?;
self.post_request(requests);
let results = future::try_join_all(write_tasks)
.await
.context(JoinTaskSnafu)?;
let affected_rows = results
.into_iter()
.map(|resp| resp.map(|r| r.affected_rows))
@@ -269,21 +270,22 @@ impl Inserter {
/// Mirror requests for source table to flownode
async fn mirror_flow_node_requests(
table_flownode_set_cache: TableFlownodeSetCacheRef,
requests: RegionInsertRequests,
&self,
requests: &RegionInsertRequests,
) -> Result<HashMap<Peer, RegionInsertRequests>> {
// store partial source table requests used by flow node(only store what's used)
let mut src_table_reqs: HashMap<TableId, Option<(Vec<Peer>, RegionInsertRequests)>> =
HashMap::new();
for req in requests.requests {
match src_table_reqs.get_mut(&RegionId::from_u64(req.region_id).table_id()) {
Some(Some((_peers, reqs))) => reqs.requests.push(req),
for req in &requests.requests {
let table_id = RegionId::from_u64(req.region_id).table_id();
match src_table_reqs.get_mut(&table_id) {
Some(Some((_peers, reqs))) => reqs.requests.push(req.clone()),
// already know this is not source table
Some(None) => continue,
_ => {
let table_id = RegionId::from_u64(req.region_id).table_id();
// TODO(discord9): determine where to store the flow node address in distributed mode
let peers = table_flownode_set_cache
let peers = self
.table_flownode_set_cache
.get(table_id)
.await
.context(RequestInsertsSnafu)?
@@ -294,7 +296,7 @@ impl Inserter {
if !peers.is_empty() {
let mut reqs = RegionInsertRequests::default();
reqs.requests.push(req);
reqs.requests.push(req.clone());
src_table_reqs.insert(table_id, Some((peers, reqs)));
} else {
// insert a empty entry to avoid repeat query
@@ -310,14 +312,26 @@ impl Inserter {
.into_iter()
.filter_map(|(k, v)| v.map(|v| (k, v)))
{
for flownode in peers {
if peers.len() == 1 {
// fast path, zero copy
inserts
.entry(flownode.clone())
.entry(peers[0].clone())
.or_default()
.requests
.extend(reqs.requests.clone());
.extend(reqs.requests);
continue;
} else {
// TODO(discord9): need to split requests to multiple flownodes
for flownode in peers {
inserts
.entry(flownode.clone())
.or_default()
.requests
.extend(reqs.requests.clone());
}
}
}
Ok(inserts)
}

View File

@@ -36,6 +36,11 @@ lazy_static! {
"table operator ingest rows"
)
.unwrap();
pub static ref DIST_MIRROR_ROW_COUNT: IntCounter = register_int_counter!(
"greptime_table_operator_mirror_rows",
"table operator mirror rows"
)
.unwrap();
pub static ref DIST_DELETE_ROW_COUNT: IntCounter = register_int_counter!(
"greptime_table_operator_delete_rows",
"table operator delete rows"

View File

@@ -263,6 +263,7 @@ impl StatementExecutor {
self.show_columns(show_columns, query_ctx).await
}
Statement::ShowIndex(show_index) => self.show_index(show_index, query_ctx).await,
Statement::ShowStatus(_) => self.show_status(query_ctx).await,
}
}

View File

@@ -20,7 +20,7 @@ use client::{Output, OutputData, OutputMeta};
use common_base::readable_size::ReadableSize;
use common_datasource::file_format::csv::{CsvConfigBuilder, CsvFormat, CsvOpener};
use common_datasource::file_format::json::{JsonFormat, JsonOpener};
use common_datasource::file_format::orc::{infer_orc_schema, new_orc_stream_reader};
use common_datasource::file_format::orc::{infer_orc_schema, new_orc_stream_reader, ReaderAdapter};
use common_datasource::file_format::{FileFormat, Format};
use common_datasource::lister::{Lister, Source};
use common_datasource::object_store::{build_backend, parse_url};
@@ -46,6 +46,7 @@ use session::context::QueryContextRef;
use snafu::ResultExt;
use table::requests::{CopyTableRequest, InsertRequest};
use table::table_reference::TableReference;
use tokio_util::compat::FuturesAsyncReadCompatExt;
use crate::error::{self, IntoVectorsSnafu, Result};
use crate::statement::StatementExecutor;
@@ -146,10 +147,16 @@ impl StatementExecutor {
path,
}),
Format::Parquet(_) => {
let meta = object_store
.stat(&path)
.await
.context(error::ReadObjectSnafu { path: &path })?;
let mut reader = object_store
.reader(&path)
.await
.context(error::ReadObjectSnafu { path: &path })?;
.context(error::ReadObjectSnafu { path: &path })?
.into_futures_async_read(0..meta.content_length())
.compat();
let metadata = ArrowReaderMetadata::load_async(&mut reader, Default::default())
.await
.context(error::ReadParquetMetadataSnafu)?;
@@ -161,12 +168,17 @@ impl StatementExecutor {
})
}
Format::Orc(_) => {
let meta = object_store
.stat(&path)
.await
.context(error::ReadObjectSnafu { path: &path })?;
let reader = object_store
.reader(&path)
.await
.context(error::ReadObjectSnafu { path: &path })?;
let schema = infer_orc_schema(reader)
let schema = infer_orc_schema(ReaderAdapter::new(reader, meta.content_length()))
.await
.context(error::ReadOrcSnafu)?;
@@ -279,11 +291,17 @@ impl StatementExecutor {
)))
}
FileMetadata::Parquet { metadata, path, .. } => {
let reader = object_store
.reader_with(path)
.buffer(DEFAULT_READ_BUFFER)
let meta = object_store
.stat(path)
.await
.context(error::ReadObjectSnafu { path })?;
let reader = object_store
.reader_with(path)
.chunk(DEFAULT_READ_BUFFER)
.await
.context(error::ReadObjectSnafu { path })?
.into_futures_async_read(0..meta.content_length())
.compat();
let builder =
ParquetRecordBatchStreamBuilder::new_with_metadata(reader, metadata.clone());
let stream = builder
@@ -302,14 +320,20 @@ impl StatementExecutor {
)))
}
FileMetadata::Orc { path, .. } => {
let reader = object_store
.reader_with(path)
.buffer(DEFAULT_READ_BUFFER)
let meta = object_store
.stat(path)
.await
.context(error::ReadObjectSnafu { path })?;
let stream = new_orc_stream_reader(reader)
let reader = object_store
.reader_with(path)
.chunk(DEFAULT_READ_BUFFER)
.await
.context(error::ReadOrcSnafu)?;
.context(error::ReadObjectSnafu { path })?;
let stream =
new_orc_stream_reader(ReaderAdapter::new(reader, meta.content_length()))
.await
.context(error::ReadOrcSnafu)?;
let projected_schema = Arc::new(
compat_schema

View File

@@ -117,6 +117,13 @@ impl StatementExecutor {
.await
.context(error::ExecuteStatementSnafu)
}
#[tracing::instrument(skip_all)]
pub async fn show_status(&self, query_ctx: QueryContextRef) -> Result<Output> {
query::sql::show_status(query_ctx)
.await
.context(error::ExecuteStatementSnafu)
}
}
pub(crate) fn create_partitions_stmt(partitions: Vec<PartitionInfo>) -> Result<Option<Partitions>> {

View File

@@ -79,7 +79,7 @@ impl RegionEngine for MetaRegionEngine {
})
}
async fn region_disk_usage(&self, _region_id: RegionId) -> Option<i64> {
fn region_disk_usage(&self, _region_id: RegionId) -> Option<i64> {
None
}

Some files were not shown because too many files have changed in this diff Show More