refactor: remove querier sync loop (#4146)
Namespaces are now created on demand and contain their full schema. Tombstones/chunks are created on demand during the query. Closes #4123. Co-authored-by: kodiakhq[bot] <49736102+kodiakhq[bot]@users.noreply.github.com>pull/24376/head
parent
e154d77bf5
commit
decd018a6a
|
@ -4262,6 +4262,7 @@ dependencies = [
|
||||||
"data_types",
|
"data_types",
|
||||||
"datafusion 0.1.0",
|
"datafusion 0.1.0",
|
||||||
"db",
|
"db",
|
||||||
|
"iox_catalog",
|
||||||
"iox_tests",
|
"iox_tests",
|
||||||
"itertools",
|
"itertools",
|
||||||
"metric",
|
"metric",
|
||||||
|
|
|
@ -1,29 +1,15 @@
|
||||||
//! Database for the querier that contains all namespaces.
|
//! Database for the querier that contains all namespaces.
|
||||||
|
|
||||||
use crate::{
|
use crate::{cache::CatalogCache, chunk::ParquetChunkAdapter, namespace::QuerierNamespace};
|
||||||
cache::CatalogCache,
|
|
||||||
chunk::ParquetChunkAdapter,
|
|
||||||
namespace::QuerierNamespace,
|
|
||||||
poison::{PoisonCabinet, PoisonPill},
|
|
||||||
};
|
|
||||||
use async_trait::async_trait;
|
use async_trait::async_trait;
|
||||||
use backoff::{Backoff, BackoffConfig};
|
use backoff::BackoffConfig;
|
||||||
use data_types2::NamespaceId;
|
|
||||||
use iox_catalog::interface::Catalog;
|
use iox_catalog::interface::Catalog;
|
||||||
use object_store::DynObjectStore;
|
use object_store::DynObjectStore;
|
||||||
use observability_deps::tracing::{error, info};
|
|
||||||
use parking_lot::RwLock;
|
use parking_lot::RwLock;
|
||||||
use query::exec::Executor;
|
use query::exec::Executor;
|
||||||
use service_common::QueryDatabaseProvider;
|
use service_common::QueryDatabaseProvider;
|
||||||
use std::{
|
use std::{collections::HashMap, sync::Arc};
|
||||||
collections::{BTreeMap, HashMap},
|
|
||||||
sync::Arc,
|
|
||||||
time::Duration,
|
|
||||||
};
|
|
||||||
use time::TimeProvider;
|
use time::TimeProvider;
|
||||||
use tokio_util::sync::CancellationToken;
|
|
||||||
|
|
||||||
const SYNC_INTERVAL: Duration = Duration::from_secs(1);
|
|
||||||
|
|
||||||
/// Database for the querier.
|
/// Database for the querier.
|
||||||
///
|
///
|
||||||
|
@ -63,14 +49,12 @@ impl QueryDatabaseProvider for QuerierDatabase {
|
||||||
type Db = QuerierNamespace;
|
type Db = QuerierNamespace;
|
||||||
|
|
||||||
async fn db(&self, name: &str) -> Option<Arc<Self::Db>> {
|
async fn db(&self, name: &str) -> Option<Arc<Self::Db>> {
|
||||||
self.namespace(name)
|
self.namespace(name).await
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
impl QuerierDatabase {
|
impl QuerierDatabase {
|
||||||
/// Create new, empty database.
|
/// Create new database.
|
||||||
///
|
|
||||||
/// You may call [`sync`](Self::sync) to fill this database with content.
|
|
||||||
pub fn new(
|
pub fn new(
|
||||||
catalog: Arc<dyn Catalog>,
|
catalog: Arc<dyn Catalog>,
|
||||||
metric_registry: Arc<metric::Registry>,
|
metric_registry: Arc<metric::Registry>,
|
||||||
|
@ -102,108 +86,21 @@ impl QuerierDatabase {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
/// List of namespaces.
|
|
||||||
pub fn namespaces(&self) -> Vec<Arc<QuerierNamespace>> {
|
|
||||||
self.namespaces.read().values().cloned().collect()
|
|
||||||
}
|
|
||||||
|
|
||||||
/// Get namespace if it exists
|
/// Get namespace if it exists
|
||||||
pub fn namespace(&self, name: &str) -> Option<Arc<QuerierNamespace>> {
|
pub async fn namespace(&self, name: &str) -> Option<Arc<QuerierNamespace>> {
|
||||||
self.namespaces.read().get(name).cloned()
|
let name = Arc::from(name.to_owned());
|
||||||
}
|
let schema = self
|
||||||
|
.catalog_cache
|
||||||
/// Sync set of namespaces and the data of the namespaces themselves.
|
.namespace()
|
||||||
///
|
.schema(Arc::clone(&name))
|
||||||
/// Should be called regularly.
|
.await?;
|
||||||
pub async fn sync(&self) {
|
Some(Arc::new(QuerierNamespace::new(
|
||||||
let namespaces = Backoff::new(&self.backoff_config)
|
self.backoff_config.clone(),
|
||||||
.retry_all_errors("get namespaces", || async {
|
Arc::clone(&self.chunk_adapter),
|
||||||
self.catalog.repositories().await.namespaces().list().await
|
schema,
|
||||||
})
|
name,
|
||||||
.await
|
Arc::clone(&self.exec),
|
||||||
.expect("unlimited retry");
|
)))
|
||||||
let namespaces: BTreeMap<Arc<str>, NamespaceId> = namespaces
|
|
||||||
.into_iter()
|
|
||||||
.map(|ns| (ns.name.into(), ns.id))
|
|
||||||
.collect();
|
|
||||||
|
|
||||||
// lock namespaces AFTER IO
|
|
||||||
let querier_namespaces: Vec<_> = {
|
|
||||||
let mut namespaces_guard = self.namespaces.write();
|
|
||||||
|
|
||||||
// calculate set differences
|
|
||||||
let to_add: Vec<(Arc<str>, NamespaceId)> = namespaces
|
|
||||||
.iter()
|
|
||||||
.filter_map(|(name, id)| {
|
|
||||||
(!namespaces_guard.contains_key(name)).then(|| (Arc::clone(name), *id))
|
|
||||||
})
|
|
||||||
.collect();
|
|
||||||
let to_delete: Vec<Arc<str>> = namespaces_guard
|
|
||||||
.keys()
|
|
||||||
.filter_map(|name| (!namespaces.contains_key(name)).then(|| Arc::clone(name)))
|
|
||||||
.collect();
|
|
||||||
info!(
|
|
||||||
add = to_add.len(),
|
|
||||||
delete = to_delete.len(),
|
|
||||||
actual = namespaces_guard.len(),
|
|
||||||
desired = namespaces.len(),
|
|
||||||
"Syncing namespaces",
|
|
||||||
);
|
|
||||||
|
|
||||||
// perform modification
|
|
||||||
for name in to_delete {
|
|
||||||
// TODO(marco): this is currently untested because `iox_catalog` doesn't implement namespace deletion
|
|
||||||
namespaces_guard.remove(&name);
|
|
||||||
}
|
|
||||||
for (name, id) in to_add {
|
|
||||||
namespaces_guard.insert(
|
|
||||||
Arc::clone(&name),
|
|
||||||
Arc::new(QuerierNamespace::new(
|
|
||||||
self.backoff_config.clone(),
|
|
||||||
Arc::clone(&self.chunk_adapter),
|
|
||||||
name,
|
|
||||||
id,
|
|
||||||
Arc::clone(&self.exec),
|
|
||||||
)),
|
|
||||||
);
|
|
||||||
}
|
|
||||||
|
|
||||||
// get a clone of namespace Arcs so that we can run an async operation
|
|
||||||
namespaces_guard.values().cloned().collect()
|
|
||||||
};
|
|
||||||
|
|
||||||
// downgrade guard so that other readers are allowed again and sync namespace states.
|
|
||||||
for namespace in querier_namespaces {
|
|
||||||
namespace.sync().await;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
/// Run regular [`sync`](QuerierDatabase::sync) until shutdown token is canceled.
|
|
||||||
pub(crate) async fn database_sync_loop(
|
|
||||||
database: Arc<QuerierDatabase>,
|
|
||||||
shutdown: CancellationToken,
|
|
||||||
poison_cabinet: Arc<PoisonCabinet>,
|
|
||||||
) {
|
|
||||||
loop {
|
|
||||||
if shutdown.is_cancelled() {
|
|
||||||
info!("Database sync shutdown");
|
|
||||||
return;
|
|
||||||
}
|
|
||||||
if poison_cabinet.contains(&PoisonPill::DatabaseSyncPanic) {
|
|
||||||
panic!("Database sync poisened, panic");
|
|
||||||
}
|
|
||||||
if poison_cabinet.contains(&PoisonPill::DatabaseSyncExit) {
|
|
||||||
error!("Database sync poisened, exit early");
|
|
||||||
return;
|
|
||||||
}
|
|
||||||
|
|
||||||
database.sync().await;
|
|
||||||
|
|
||||||
tokio::select!(
|
|
||||||
_ = tokio::time::sleep(SYNC_INTERVAL) => {},
|
|
||||||
_ = shutdown.cancelled() => {},
|
|
||||||
);
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -214,7 +111,7 @@ mod tests {
|
||||||
use super::*;
|
use super::*;
|
||||||
|
|
||||||
#[tokio::test]
|
#[tokio::test]
|
||||||
async fn test_sync() {
|
async fn test_namespace() {
|
||||||
let catalog = TestCatalog::new();
|
let catalog = TestCatalog::new();
|
||||||
|
|
||||||
let db = QuerierDatabase::new(
|
let db = QuerierDatabase::new(
|
||||||
|
@ -224,36 +121,10 @@ mod tests {
|
||||||
catalog.time_provider(),
|
catalog.time_provider(),
|
||||||
catalog.exec(),
|
catalog.exec(),
|
||||||
);
|
);
|
||||||
assert_eq!(ns_names(&db), vec![]);
|
|
||||||
|
|
||||||
db.sync().await;
|
|
||||||
assert_eq!(ns_names(&db), vec![]);
|
|
||||||
|
|
||||||
catalog.create_namespace("ns1").await;
|
catalog.create_namespace("ns1").await;
|
||||||
catalog.create_namespace("ns2").await;
|
|
||||||
db.sync().await;
|
|
||||||
assert_eq!(ns_names(&db), vec![Arc::from("ns1"), Arc::from("ns2")]);
|
|
||||||
let ns1_a = db.namespace("ns1").unwrap();
|
|
||||||
let ns2_a = db.namespace("ns2").unwrap();
|
|
||||||
assert_eq!(ns1_a.name().as_ref(), "ns1");
|
|
||||||
assert_eq!(ns2_a.name().as_ref(), "ns2");
|
|
||||||
assert!(db.namespace("ns3").is_none());
|
|
||||||
|
|
||||||
catalog.create_namespace("ns3").await;
|
assert!(db.namespace("ns1").await.is_some());
|
||||||
db.sync().await;
|
assert!(db.namespace("ns2").await.is_none());
|
||||||
assert_eq!(
|
|
||||||
ns_names(&db),
|
|
||||||
vec![Arc::from("ns1"), Arc::from("ns2"), Arc::from("ns3")]
|
|
||||||
);
|
|
||||||
let ns1_b = db.namespace("ns1").unwrap();
|
|
||||||
let ns2_b = db.namespace("ns2").unwrap();
|
|
||||||
assert!(Arc::ptr_eq(&ns1_a, &ns1_b));
|
|
||||||
assert!(Arc::ptr_eq(&ns2_a, &ns2_b));
|
|
||||||
}
|
|
||||||
|
|
||||||
fn ns_names(db: &QuerierDatabase) -> Vec<Arc<str>> {
|
|
||||||
let mut names: Vec<Arc<str>> = db.namespaces().iter().map(|ns| ns.name()).collect();
|
|
||||||
names.sort();
|
|
||||||
names
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -12,10 +12,7 @@ use thiserror::Error;
|
||||||
use tokio::task::{JoinError, JoinHandle};
|
use tokio::task::{JoinError, JoinHandle};
|
||||||
use tokio_util::sync::CancellationToken;
|
use tokio_util::sync::CancellationToken;
|
||||||
|
|
||||||
use crate::{
|
use crate::{database::QuerierDatabase, poison::PoisonCabinet};
|
||||||
database::{database_sync_loop, QuerierDatabase},
|
|
||||||
poison::PoisonCabinet,
|
|
||||||
};
|
|
||||||
|
|
||||||
#[derive(Debug, Error)]
|
#[derive(Debug, Error)]
|
||||||
#[allow(missing_copy_implementations, missing_docs)]
|
#[allow(missing_copy_implementations, missing_docs)]
|
||||||
|
@ -63,14 +60,7 @@ impl QuerierHandlerImpl {
|
||||||
let shutdown = CancellationToken::new();
|
let shutdown = CancellationToken::new();
|
||||||
let poison_cabinet = Arc::new(PoisonCabinet::new());
|
let poison_cabinet = Arc::new(PoisonCabinet::new());
|
||||||
|
|
||||||
let join_handles = vec![(
|
let join_handles = vec![];
|
||||||
String::from("database sync"),
|
|
||||||
shared_handle(tokio::spawn(database_sync_loop(
|
|
||||||
Arc::clone(&database),
|
|
||||||
shutdown.clone(),
|
|
||||||
Arc::clone(&poison_cabinet),
|
|
||||||
))),
|
|
||||||
)];
|
|
||||||
Self {
|
Self {
|
||||||
database,
|
database,
|
||||||
join_handles,
|
join_handles,
|
||||||
|
@ -98,6 +88,8 @@ impl QuerierHandler for QuerierHandlerImpl {
|
||||||
panic!("Background worker '{name}' exited early!");
|
panic!("Background worker '{name}' exited early!");
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
self.shutdown.cancelled().await;
|
||||||
}
|
}
|
||||||
|
|
||||||
fn shutdown(&self) {
|
fn shutdown(&self) {
|
||||||
|
@ -123,8 +115,6 @@ mod tests {
|
||||||
use query::exec::Executor;
|
use query::exec::Executor;
|
||||||
use time::{MockProvider, Time};
|
use time::{MockProvider, Time};
|
||||||
|
|
||||||
use crate::poison::PoisonPill;
|
|
||||||
|
|
||||||
use super::*;
|
use super::*;
|
||||||
|
|
||||||
#[tokio::test]
|
#[tokio::test]
|
||||||
|
@ -144,26 +134,6 @@ mod tests {
|
||||||
.unwrap();
|
.unwrap();
|
||||||
}
|
}
|
||||||
|
|
||||||
#[tokio::test]
|
|
||||||
#[should_panic(expected = "Background worker 'database sync' exited early!")]
|
|
||||||
async fn test_supervise_database_sync_early_exit() {
|
|
||||||
let querier = TestQuerier::new().querier;
|
|
||||||
querier.poison_cabinet.add(PoisonPill::DatabaseSyncExit);
|
|
||||||
tokio::time::timeout(Duration::from_millis(2000), querier.join())
|
|
||||||
.await
|
|
||||||
.unwrap();
|
|
||||||
}
|
|
||||||
|
|
||||||
#[tokio::test]
|
|
||||||
#[should_panic(expected = "JoinError::Panic")]
|
|
||||||
async fn test_supervise_database_sync_panic() {
|
|
||||||
let querier = TestQuerier::new().querier;
|
|
||||||
querier.poison_cabinet.add(PoisonPill::DatabaseSyncPanic);
|
|
||||||
tokio::time::timeout(Duration::from_millis(2000), querier.join())
|
|
||||||
.await
|
|
||||||
.unwrap();
|
|
||||||
}
|
|
||||||
|
|
||||||
struct TestQuerier {
|
struct TestQuerier {
|
||||||
querier: QuerierHandlerImpl,
|
querier: QuerierHandlerImpl,
|
||||||
}
|
}
|
||||||
|
|
|
@ -1,11 +1,9 @@
|
||||||
//! Namespace within the whole database.
|
//! Namespace within the whole database.
|
||||||
use crate::{cache::CatalogCache, chunk::ParquetChunkAdapter, table::QuerierTable};
|
use crate::{cache::CatalogCache, chunk::ParquetChunkAdapter, table::QuerierTable};
|
||||||
use backoff::BackoffConfig;
|
use backoff::BackoffConfig;
|
||||||
use data_types2::NamespaceId;
|
use data_types2::{NamespaceId, NamespaceSchema};
|
||||||
use iox_catalog::interface::Catalog;
|
use iox_catalog::interface::Catalog;
|
||||||
use object_store::DynObjectStore;
|
use object_store::DynObjectStore;
|
||||||
use observability_deps::tracing::warn;
|
|
||||||
use parking_lot::RwLock;
|
|
||||||
use query::exec::Executor;
|
use query::exec::Executor;
|
||||||
use schema::Schema;
|
use schema::Schema;
|
||||||
use std::{collections::HashMap, sync::Arc};
|
use std::{collections::HashMap, sync::Arc};
|
||||||
|
@ -19,71 +17,72 @@ mod test_util;
|
||||||
/// Maps a catalog namespace to all the in-memory resources and sync-state that the querier needs.
|
/// Maps a catalog namespace to all the in-memory resources and sync-state that the querier needs.
|
||||||
///
|
///
|
||||||
/// # Data Structures & Sync
|
/// # Data Structures & Sync
|
||||||
/// Tables and schemas are synced ahead of usage (via [`sync`](Self::sync)) because DataFusion does not implement async
|
/// Tables and schemas are created when [`QuerierNamespace`] is created because DataFusion does not implement async
|
||||||
/// schema inspection. The actual payload (chunks and tombstones) are only queried on demand.
|
/// schema inspection. The actual payload (chunks and tombstones) are only queried on demand.
|
||||||
///
|
///
|
||||||
/// Most access to the [IOx Catalog](Catalog) are cached.
|
/// Most access to the [IOx Catalog](Catalog) are cached.
|
||||||
#[derive(Debug)]
|
#[derive(Debug)]
|
||||||
pub struct QuerierNamespace {
|
pub struct QuerierNamespace {
|
||||||
/// Backoff config for IO operations.
|
|
||||||
backoff_config: BackoffConfig,
|
|
||||||
|
|
||||||
/// The catalog.
|
|
||||||
catalog: Arc<dyn Catalog>,
|
|
||||||
|
|
||||||
/// Catalog IO cache.
|
|
||||||
catalog_cache: Arc<CatalogCache>,
|
|
||||||
|
|
||||||
/// Tables in this namespace.
|
|
||||||
tables: RwLock<Arc<HashMap<Arc<str>, Arc<QuerierTable>>>>,
|
|
||||||
|
|
||||||
/// Adapter to create chunks.
|
|
||||||
chunk_adapter: Arc<ParquetChunkAdapter>,
|
|
||||||
|
|
||||||
/// ID of this namespace.
|
/// ID of this namespace.
|
||||||
id: NamespaceId,
|
id: NamespaceId,
|
||||||
|
|
||||||
/// Name of this namespace.
|
/// Name of this namespace.
|
||||||
name: Arc<str>,
|
name: Arc<str>,
|
||||||
|
|
||||||
|
/// Tables in this namespace.
|
||||||
|
tables: Arc<HashMap<Arc<str>, Arc<QuerierTable>>>,
|
||||||
|
|
||||||
/// Executor for queries.
|
/// Executor for queries.
|
||||||
exec: Arc<Executor>,
|
exec: Arc<Executor>,
|
||||||
}
|
}
|
||||||
|
|
||||||
impl QuerierNamespace {
|
impl QuerierNamespace {
|
||||||
/// Create new, empty namespace.
|
/// Create new namespace for given schema.
|
||||||
///
|
|
||||||
/// You may call [`sync`](Self::sync) to fill the namespace with chunks.
|
|
||||||
pub fn new(
|
pub fn new(
|
||||||
backoff_config: BackoffConfig,
|
backoff_config: BackoffConfig,
|
||||||
chunk_adapter: Arc<ParquetChunkAdapter>,
|
chunk_adapter: Arc<ParquetChunkAdapter>,
|
||||||
|
schema: Arc<NamespaceSchema>,
|
||||||
name: Arc<str>,
|
name: Arc<str>,
|
||||||
id: NamespaceId,
|
|
||||||
exec: Arc<Executor>,
|
exec: Arc<Executor>,
|
||||||
) -> Self {
|
) -> Self {
|
||||||
let catalog_cache = Arc::clone(chunk_adapter.catalog_cache());
|
let tables: HashMap<_, _> = schema
|
||||||
let catalog = catalog_cache.catalog();
|
.tables
|
||||||
|
.iter()
|
||||||
|
.map(|(name, table_schema)| {
|
||||||
|
let name = Arc::from(name.clone());
|
||||||
|
let id = table_schema.id;
|
||||||
|
let schema = Schema::try_from(table_schema.clone()).expect("cannot build schema");
|
||||||
|
|
||||||
|
let table = Arc::new(QuerierTable::new(
|
||||||
|
backoff_config.clone(),
|
||||||
|
id,
|
||||||
|
Arc::clone(&name),
|
||||||
|
Arc::new(schema),
|
||||||
|
Arc::clone(&chunk_adapter),
|
||||||
|
));
|
||||||
|
|
||||||
|
(name, table)
|
||||||
|
})
|
||||||
|
.collect();
|
||||||
|
|
||||||
|
let id = schema.id;
|
||||||
|
|
||||||
Self {
|
Self {
|
||||||
backoff_config,
|
|
||||||
catalog,
|
|
||||||
catalog_cache: Arc::clone(&catalog_cache),
|
|
||||||
tables: RwLock::new(Arc::new(HashMap::new())),
|
|
||||||
chunk_adapter,
|
|
||||||
id,
|
id,
|
||||||
name,
|
name,
|
||||||
|
tables: Arc::new(tables),
|
||||||
exec,
|
exec,
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
/// Create new empty namespace for testing.
|
/// Create new namespace for given schema, for testing.
|
||||||
pub fn new_testing(
|
pub fn new_testing(
|
||||||
catalog: Arc<dyn Catalog>,
|
catalog: Arc<dyn Catalog>,
|
||||||
object_store: Arc<DynObjectStore>,
|
object_store: Arc<DynObjectStore>,
|
||||||
metric_registry: Arc<metric::Registry>,
|
metric_registry: Arc<metric::Registry>,
|
||||||
time_provider: Arc<dyn TimeProvider>,
|
time_provider: Arc<dyn TimeProvider>,
|
||||||
name: Arc<str>,
|
name: Arc<str>,
|
||||||
id: NamespaceId,
|
schema: Arc<NamespaceSchema>,
|
||||||
exec: Arc<Executor>,
|
exec: Arc<Executor>,
|
||||||
) -> Self {
|
) -> Self {
|
||||||
let catalog_cache = Arc::new(CatalogCache::new(catalog, Arc::clone(&time_provider)));
|
let catalog_cache = Arc::new(CatalogCache::new(catalog, Arc::clone(&time_provider)));
|
||||||
|
@ -94,124 +93,44 @@ impl QuerierNamespace {
|
||||||
time_provider,
|
time_provider,
|
||||||
));
|
));
|
||||||
|
|
||||||
Self::new(BackoffConfig::default(), chunk_adapter, name, id, exec)
|
Self::new(BackoffConfig::default(), chunk_adapter, schema, name, exec)
|
||||||
}
|
}
|
||||||
|
|
||||||
/// Namespace name.
|
/// Namespace name.
|
||||||
pub fn name(&self) -> Arc<str> {
|
pub fn name(&self) -> Arc<str> {
|
||||||
Arc::clone(&self.name)
|
Arc::clone(&self.name)
|
||||||
}
|
}
|
||||||
|
|
||||||
/// Sync partial namespace state.
|
|
||||||
///
|
|
||||||
/// This includes:
|
|
||||||
/// - tables
|
|
||||||
/// - schemas
|
|
||||||
///
|
|
||||||
/// Chunks and tombstones are queried on-demand.
|
|
||||||
///
|
|
||||||
/// Should be called regularly.
|
|
||||||
pub async fn sync(&self) {
|
|
||||||
let catalog_schema_desired = match self
|
|
||||||
.catalog_cache
|
|
||||||
.namespace()
|
|
||||||
.schema(Arc::clone(&self.name))
|
|
||||||
.await
|
|
||||||
{
|
|
||||||
Some(schema) => schema,
|
|
||||||
None => {
|
|
||||||
warn!(
|
|
||||||
namespace = self.name.as_ref(),
|
|
||||||
"Cannot sync namespace because it is gone",
|
|
||||||
);
|
|
||||||
return;
|
|
||||||
}
|
|
||||||
};
|
|
||||||
|
|
||||||
let tables: HashMap<_, _> = catalog_schema_desired
|
|
||||||
.tables
|
|
||||||
.iter()
|
|
||||||
.map(|(name, table_schema)| {
|
|
||||||
let name = Arc::from(name.clone());
|
|
||||||
let id = table_schema.id;
|
|
||||||
let schema = Schema::try_from(table_schema.clone()).expect("cannot build schema");
|
|
||||||
|
|
||||||
let table = Arc::new(QuerierTable::new(
|
|
||||||
self.backoff_config.clone(),
|
|
||||||
id,
|
|
||||||
Arc::clone(&name),
|
|
||||||
Arc::new(schema),
|
|
||||||
Arc::clone(&self.chunk_adapter),
|
|
||||||
));
|
|
||||||
|
|
||||||
(name, table)
|
|
||||||
})
|
|
||||||
.collect();
|
|
||||||
|
|
||||||
*self.tables.write() = Arc::new(tables);
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
#[cfg(test)]
|
#[cfg(test)]
|
||||||
mod tests {
|
mod tests {
|
||||||
use super::*;
|
use super::*;
|
||||||
use crate::{cache::namespace::TTL_EXISTING, namespace::test_util::querier_namespace};
|
use crate::namespace::test_util::querier_namespace;
|
||||||
use data_types2::ColumnType;
|
use data_types2::ColumnType;
|
||||||
use iox_tests::util::TestCatalog;
|
use iox_tests::util::TestCatalog;
|
||||||
use schema::{builder::SchemaBuilder, InfluxColumnType, InfluxFieldType};
|
use schema::{builder::SchemaBuilder, InfluxColumnType, InfluxFieldType};
|
||||||
|
|
||||||
#[tokio::test]
|
|
||||||
async fn test_sync_namespace_gone() {
|
|
||||||
let catalog = TestCatalog::new();
|
|
||||||
|
|
||||||
let catalog_cache = Arc::new(CatalogCache::new(
|
|
||||||
catalog.catalog(),
|
|
||||||
catalog.time_provider(),
|
|
||||||
));
|
|
||||||
let chunk_adapter = Arc::new(ParquetChunkAdapter::new(
|
|
||||||
catalog_cache,
|
|
||||||
catalog.object_store(),
|
|
||||||
catalog.metric_registry(),
|
|
||||||
catalog.time_provider(),
|
|
||||||
));
|
|
||||||
let querier_namespace = QuerierNamespace::new(
|
|
||||||
BackoffConfig::default(),
|
|
||||||
chunk_adapter,
|
|
||||||
"ns".into(),
|
|
||||||
NamespaceId::new(1),
|
|
||||||
catalog.exec(),
|
|
||||||
);
|
|
||||||
|
|
||||||
// The container (`QuerierDatabase`) should prune the namespace if it's gone, however the `sync` might still be
|
|
||||||
// in-progress and must not block or panic.
|
|
||||||
querier_namespace.sync().await;
|
|
||||||
}
|
|
||||||
|
|
||||||
#[tokio::test]
|
#[tokio::test]
|
||||||
async fn test_sync_tables() {
|
async fn test_sync_tables() {
|
||||||
let catalog = TestCatalog::new();
|
let catalog = TestCatalog::new();
|
||||||
|
|
||||||
let ns = catalog.create_namespace("ns").await;
|
let ns = catalog.create_namespace("ns").await;
|
||||||
|
|
||||||
let querier_namespace = querier_namespace(&ns);
|
let qns = querier_namespace(&ns).await;
|
||||||
|
assert_eq!(tables(&qns), Vec::<String>::new());
|
||||||
querier_namespace.sync().await;
|
|
||||||
assert_eq!(tables(&querier_namespace), Vec::<String>::new());
|
|
||||||
|
|
||||||
ns.create_table("table1").await;
|
ns.create_table("table1").await;
|
||||||
ns.create_table("table2").await;
|
ns.create_table("table2").await;
|
||||||
catalog.mock_time_provider().inc(TTL_EXISTING);
|
let qns = querier_namespace(&ns).await;
|
||||||
querier_namespace.sync().await;
|
|
||||||
assert_eq!(
|
assert_eq!(
|
||||||
tables(&querier_namespace),
|
tables(&qns),
|
||||||
vec![String::from("table1"), String::from("table2")]
|
vec![String::from("table1"), String::from("table2")]
|
||||||
);
|
);
|
||||||
|
|
||||||
ns.create_table("table3").await;
|
ns.create_table("table3").await;
|
||||||
catalog.mock_time_provider().inc(TTL_EXISTING);
|
let qns = querier_namespace(&ns).await;
|
||||||
querier_namespace.sync().await;
|
|
||||||
assert_eq!(
|
assert_eq!(
|
||||||
tables(&querier_namespace),
|
tables(&qns),
|
||||||
vec![
|
vec![
|
||||||
String::from("table1"),
|
String::from("table1"),
|
||||||
String::from("table2"),
|
String::from("table2"),
|
||||||
|
@ -227,31 +146,27 @@ mod tests {
|
||||||
let ns = catalog.create_namespace("ns").await;
|
let ns = catalog.create_namespace("ns").await;
|
||||||
let table = ns.create_table("table").await;
|
let table = ns.create_table("table").await;
|
||||||
|
|
||||||
let querier_namespace = querier_namespace(&ns);
|
let qns = querier_namespace(&ns).await;
|
||||||
|
|
||||||
querier_namespace.sync().await;
|
|
||||||
let expected_schema = SchemaBuilder::new().build().unwrap();
|
let expected_schema = SchemaBuilder::new().build().unwrap();
|
||||||
let actual_schema = schema(&querier_namespace, "table");
|
let actual_schema = schema(&qns, "table");
|
||||||
assert_eq!(actual_schema.as_ref(), &expected_schema,);
|
assert_eq!(actual_schema.as_ref(), &expected_schema,);
|
||||||
|
|
||||||
table.create_column("col1", ColumnType::I64).await;
|
table.create_column("col1", ColumnType::I64).await;
|
||||||
table.create_column("col2", ColumnType::Bool).await;
|
table.create_column("col2", ColumnType::Bool).await;
|
||||||
table.create_column("col3", ColumnType::Tag).await;
|
table.create_column("col3", ColumnType::Tag).await;
|
||||||
catalog.mock_time_provider().inc(TTL_EXISTING);
|
let qns = querier_namespace(&ns).await;
|
||||||
querier_namespace.sync().await;
|
|
||||||
let expected_schema = SchemaBuilder::new()
|
let expected_schema = SchemaBuilder::new()
|
||||||
.influx_column("col1", InfluxColumnType::Field(InfluxFieldType::Integer))
|
.influx_column("col1", InfluxColumnType::Field(InfluxFieldType::Integer))
|
||||||
.influx_column("col2", InfluxColumnType::Field(InfluxFieldType::Boolean))
|
.influx_column("col2", InfluxColumnType::Field(InfluxFieldType::Boolean))
|
||||||
.influx_column("col3", InfluxColumnType::Tag)
|
.influx_column("col3", InfluxColumnType::Tag)
|
||||||
.build()
|
.build()
|
||||||
.unwrap();
|
.unwrap();
|
||||||
let actual_schema = schema(&querier_namespace, "table");
|
let actual_schema = schema(&qns, "table");
|
||||||
assert_eq!(actual_schema.as_ref(), &expected_schema,);
|
assert_eq!(actual_schema.as_ref(), &expected_schema,);
|
||||||
|
|
||||||
table.create_column("col4", ColumnType::Tag).await;
|
table.create_column("col4", ColumnType::Tag).await;
|
||||||
table.create_column("col5", ColumnType::Time).await;
|
table.create_column("col5", ColumnType::Time).await;
|
||||||
catalog.mock_time_provider().inc(TTL_EXISTING);
|
let qns = querier_namespace(&ns).await;
|
||||||
querier_namespace.sync().await;
|
|
||||||
let expected_schema = SchemaBuilder::new()
|
let expected_schema = SchemaBuilder::new()
|
||||||
.influx_column("col1", InfluxColumnType::Field(InfluxFieldType::Integer))
|
.influx_column("col1", InfluxColumnType::Field(InfluxFieldType::Integer))
|
||||||
.influx_column("col2", InfluxColumnType::Field(InfluxFieldType::Boolean))
|
.influx_column("col2", InfluxColumnType::Field(InfluxFieldType::Boolean))
|
||||||
|
@ -260,7 +175,7 @@ mod tests {
|
||||||
.influx_column("col5", InfluxColumnType::Timestamp)
|
.influx_column("col5", InfluxColumnType::Timestamp)
|
||||||
.build()
|
.build()
|
||||||
.unwrap();
|
.unwrap();
|
||||||
let actual_schema = schema(&querier_namespace, "table");
|
let actual_schema = schema(&qns, "table");
|
||||||
assert_eq!(actual_schema.as_ref(), &expected_schema,);
|
assert_eq!(actual_schema.as_ref(), &expected_schema,);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -276,7 +191,6 @@ mod tests {
|
||||||
sorted(
|
sorted(
|
||||||
querier_namespace
|
querier_namespace
|
||||||
.tables
|
.tables
|
||||||
.read()
|
|
||||||
.keys()
|
.keys()
|
||||||
.map(|s| s.to_string())
|
.map(|s| s.to_string())
|
||||||
.collect(),
|
.collect(),
|
||||||
|
@ -284,6 +198,6 @@ mod tests {
|
||||||
}
|
}
|
||||||
|
|
||||||
fn schema(querier_namespace: &QuerierNamespace, table: &str) -> Arc<Schema> {
|
fn schema(querier_namespace: &QuerierNamespace, table: &str) -> Arc<Schema> {
|
||||||
Arc::clone(querier_namespace.tables.read().get(table).unwrap().schema())
|
Arc::clone(querier_namespace.tables.get(table).unwrap().schema())
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -17,16 +17,13 @@ use crate::{namespace::QuerierNamespace, table::QuerierTable};
|
||||||
|
|
||||||
impl QueryDatabaseMeta for QuerierNamespace {
|
impl QueryDatabaseMeta for QuerierNamespace {
|
||||||
fn table_names(&self) -> Vec<String> {
|
fn table_names(&self) -> Vec<String> {
|
||||||
let mut names: Vec<_> = self.tables.read().keys().map(|s| s.to_string()).collect();
|
let mut names: Vec<_> = self.tables.keys().map(|s| s.to_string()).collect();
|
||||||
names.sort();
|
names.sort();
|
||||||
names
|
names
|
||||||
}
|
}
|
||||||
|
|
||||||
fn table_schema(&self, table_name: &str) -> Option<Arc<Schema>> {
|
fn table_schema(&self, table_name: &str) -> Option<Arc<Schema>> {
|
||||||
self.tables
|
self.tables.get(table_name).map(|t| Arc::clone(t.schema()))
|
||||||
.read()
|
|
||||||
.get(table_name)
|
|
||||||
.map(|t| Arc::clone(t.schema()))
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -34,7 +31,7 @@ impl QueryDatabaseMeta for QuerierNamespace {
|
||||||
impl QueryDatabase for QuerierNamespace {
|
impl QueryDatabase for QuerierNamespace {
|
||||||
async fn chunks(&self, table_name: &str, predicate: &Predicate) -> Vec<Arc<dyn QueryChunk>> {
|
async fn chunks(&self, table_name: &str, predicate: &Predicate) -> Vec<Arc<dyn QueryChunk>> {
|
||||||
// get table metadata
|
// get table metadata
|
||||||
let table = match self.tables.read().get(table_name).map(Arc::clone) {
|
let table = match self.tables.get(table_name).map(Arc::clone) {
|
||||||
Some(table) => table,
|
Some(table) => table,
|
||||||
None => {
|
None => {
|
||||||
// table gone
|
// table gone
|
||||||
|
@ -85,7 +82,7 @@ pub struct QuerierCatalogProvider {
|
||||||
impl QuerierCatalogProvider {
|
impl QuerierCatalogProvider {
|
||||||
fn from_namespace(namespace: &QuerierNamespace) -> Self {
|
fn from_namespace(namespace: &QuerierNamespace) -> Self {
|
||||||
Self {
|
Self {
|
||||||
tables: Arc::clone(&namespace.tables.read()),
|
tables: Arc::clone(&namespace.tables),
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -249,8 +246,7 @@ mod tests {
|
||||||
.create_tombstone(1, 1, 13, "host=d")
|
.create_tombstone(1, 1, 13, "host=d")
|
||||||
.await;
|
.await;
|
||||||
|
|
||||||
let querier_namespace = Arc::new(querier_namespace(&ns));
|
let querier_namespace = Arc::new(querier_namespace(&ns).await);
|
||||||
querier_namespace.sync().await;
|
|
||||||
|
|
||||||
assert_query(
|
assert_query(
|
||||||
&querier_namespace,
|
&querier_namespace,
|
||||||
|
|
|
@ -1,18 +1,26 @@
|
||||||
use std::sync::Arc;
|
use std::sync::Arc;
|
||||||
|
|
||||||
|
use iox_catalog::interface::get_schema_by_name;
|
||||||
use iox_tests::util::TestNamespace;
|
use iox_tests::util::TestNamespace;
|
||||||
|
|
||||||
use super::QuerierNamespace;
|
use super::QuerierNamespace;
|
||||||
|
|
||||||
/// Create [`QuerierNamespace`] for testing.
|
/// Create [`QuerierNamespace`] for testing.
|
||||||
pub fn querier_namespace(ns: &Arc<TestNamespace>) -> QuerierNamespace {
|
pub async fn querier_namespace(ns: &Arc<TestNamespace>) -> QuerierNamespace {
|
||||||
|
let mut repos = ns.catalog.catalog.repositories().await;
|
||||||
|
let schema = Arc::new(
|
||||||
|
get_schema_by_name(&ns.namespace.name, repos.as_mut())
|
||||||
|
.await
|
||||||
|
.unwrap(),
|
||||||
|
);
|
||||||
|
|
||||||
QuerierNamespace::new_testing(
|
QuerierNamespace::new_testing(
|
||||||
ns.catalog.catalog(),
|
ns.catalog.catalog(),
|
||||||
ns.catalog.object_store(),
|
ns.catalog.object_store(),
|
||||||
ns.catalog.metric_registry(),
|
ns.catalog.metric_registry(),
|
||||||
ns.catalog.time_provider(),
|
ns.catalog.time_provider(),
|
||||||
ns.namespace.name.clone().into(),
|
ns.namespace.name.clone().into(),
|
||||||
ns.namespace.id,
|
schema,
|
||||||
ns.catalog.exec(),
|
ns.catalog.exec(),
|
||||||
)
|
)
|
||||||
}
|
}
|
||||||
|
|
|
@ -8,10 +8,7 @@ use parking_lot::{RwLock, RwLockUpgradableReadGuard};
|
||||||
use pin_project::pin_project;
|
use pin_project::pin_project;
|
||||||
|
|
||||||
#[derive(Debug, Clone, PartialEq, Eq)]
|
#[derive(Debug, Clone, PartialEq, Eq)]
|
||||||
pub enum PoisonPill {
|
pub enum PoisonPill {}
|
||||||
DatabaseSyncPanic,
|
|
||||||
DatabaseSyncExit,
|
|
||||||
}
|
|
||||||
|
|
||||||
#[derive(Debug)]
|
#[derive(Debug)]
|
||||||
struct PoisonCabinetInner {
|
struct PoisonCabinetInner {
|
||||||
|
|
|
@ -12,6 +12,7 @@ async-trait = "0.1"
|
||||||
data_types = { path = "../data_types" }
|
data_types = { path = "../data_types" }
|
||||||
datafusion = { path = "../datafusion" }
|
datafusion = { path = "../datafusion" }
|
||||||
db = { path = "../db" }
|
db = { path = "../db" }
|
||||||
|
iox_catalog = { path = "../iox_catalog" }
|
||||||
iox_tests = { path = "../iox_tests" }
|
iox_tests = { path = "../iox_tests" }
|
||||||
itertools = "0.10"
|
itertools = "0.10"
|
||||||
mutable_batch_lp = { path = "../mutable_batch_lp" }
|
mutable_batch_lp = { path = "../mutable_batch_lp" }
|
||||||
|
|
|
@ -7,6 +7,7 @@ use db::{
|
||||||
utils::{count_mub_table_chunks, count_os_table_chunks, count_rub_table_chunks, make_db},
|
utils::{count_mub_table_chunks, count_os_table_chunks, count_rub_table_chunks, make_db},
|
||||||
Db,
|
Db,
|
||||||
};
|
};
|
||||||
|
use iox_catalog::interface::get_schema_by_name;
|
||||||
use iox_tests::util::{TestCatalog, TestNamespace};
|
use iox_tests::util::{TestCatalog, TestNamespace};
|
||||||
use itertools::Itertools;
|
use itertools::Itertools;
|
||||||
use querier::QuerierNamespace;
|
use querier::QuerierNamespace;
|
||||||
|
@ -1227,16 +1228,20 @@ async fn make_ng_chunk(ns: Arc<TestNamespace>, chunk: ChunkDataNew<'_, '_>) -> S
|
||||||
}
|
}
|
||||||
|
|
||||||
async fn make_querier_namespace(ns: Arc<TestNamespace>) -> Arc<QuerierNamespace> {
|
async fn make_querier_namespace(ns: Arc<TestNamespace>) -> Arc<QuerierNamespace> {
|
||||||
let db = Arc::new(QuerierNamespace::new_testing(
|
let mut repos = ns.catalog.catalog.repositories().await;
|
||||||
|
let schema = Arc::new(
|
||||||
|
get_schema_by_name(&ns.namespace.name, repos.as_mut())
|
||||||
|
.await
|
||||||
|
.unwrap(),
|
||||||
|
);
|
||||||
|
|
||||||
|
Arc::new(QuerierNamespace::new_testing(
|
||||||
ns.catalog.catalog(),
|
ns.catalog.catalog(),
|
||||||
ns.catalog.object_store(),
|
ns.catalog.object_store(),
|
||||||
ns.catalog.metric_registry(),
|
ns.catalog.metric_registry(),
|
||||||
ns.catalog.time_provider(),
|
ns.catalog.time_provider(),
|
||||||
ns.namespace.name.clone().into(),
|
ns.namespace.name.clone().into(),
|
||||||
ns.namespace.id,
|
schema,
|
||||||
ns.catalog.exec(),
|
ns.catalog.exec(),
|
||||||
));
|
))
|
||||||
db.sync().await;
|
|
||||||
|
|
||||||
db
|
|
||||||
}
|
}
|
||||||
|
|
Loading…
Reference in New Issue