Skip to content

Commit

Permalink
feat: extend region leases in Metasrv (GreptimeTeam#1784)
Browse files Browse the repository at this point in the history
* feat: extend region leases in Metasrv

* fix: resolve PR comments
  • Loading branch information
MichaelScofield authored Jun 19, 2023
1 parent 128c6ec commit 2dd86b6
Show file tree
Hide file tree
Showing 27 changed files with 608 additions and 89 deletions.
4 changes: 3 additions & 1 deletion Cargo.lock

Some generated files are not rendered by default. Learn more about how customized files appear on GitHub.

2 changes: 1 addition & 1 deletion Cargo.toml
Original file line number Diff line number Diff line change
Expand Up @@ -72,7 +72,7 @@ datafusion-sql = { git = "https://github.com/waynexia/arrow-datafusion.git", rev
datafusion-substrait = { git = "https://github.com/waynexia/arrow-datafusion.git", rev = "63e52dde9e44cac4b1f6c6e6b6bf6368ba3bd323" }
futures = "0.3"
futures-util = "0.3"
greptime-proto = { git = "https://github.com/GreptimeTeam/greptime-proto.git", rev = "aee86f4a68c59873961c9b99ee7ed6a4341bf773" }
greptime-proto = { git = "https://github.com/GreptimeTeam/greptime-proto.git", rev = "5d5eb65bb985ff47b3a417fb2505e315e2f5c319" }
itertools = "0.10"
parquet = "40.0"
paste = "1.0"
Expand Down
14 changes: 13 additions & 1 deletion src/catalog/src/helper.rs
Original file line number Diff line number Diff line change
Expand Up @@ -91,7 +91,7 @@ pub fn build_table_regional_prefix(
}

/// Table global info has only one key across all datanodes so it does not have `node_id` field.
#[derive(Clone)]
#[derive(Clone, Hash, Eq, PartialEq)]
pub struct TableGlobalKey {
pub catalog_name: String,
pub schema_name: String,
Expand Down Expand Up @@ -124,6 +124,14 @@ impl TableGlobalKey {
table_name: captures[3].to_string(),
})
}

pub fn to_raw_key(&self) -> Vec<u8> {
self.to_string().into_bytes()
}

pub fn try_from_raw_key(key: &[u8]) -> Result<Self, Error> {
Self::parse(String::from_utf8_lossy(key))
}
}

/// Table global info contains necessary info for a datanode to create table regions, including
Expand All @@ -141,6 +149,10 @@ impl TableGlobalValue {
pub fn table_id(&self) -> TableId {
self.table_info.ident.table_id
}

pub fn engine(&self) -> &str {
&self.table_info.meta.engine
}
}

/// Table regional info that varies between datanode, so it contains a `node_id` field.
Expand Down
3 changes: 0 additions & 3 deletions src/catalog/src/remote.rs
Original file line number Diff line number Diff line change
Expand Up @@ -29,9 +29,6 @@ mod manager;

#[cfg(feature = "testing")]
pub mod mock;

// FIXME(LFC): Used in next PR.
#[allow(dead_code)]
pub mod region_alive_keeper;

#[derive(Debug, Clone)]
Expand Down
141 changes: 135 additions & 6 deletions src/catalog/src/remote/region_alive_keeper.rs
Original file line number Diff line number Diff line change
Expand Up @@ -16,10 +16,15 @@ use std::collections::HashMap;
use std::future::Future;
use std::sync::Arc;

use async_trait::async_trait;
use common_meta::error::InvalidProtoMsgSnafu;
use common_meta::heartbeat::handler::{
HandleControl, HeartbeatResponseHandler, HeartbeatResponseHandlerContext,
};
use common_meta::ident::TableIdent;
use common_meta::RegionIdent;
use common_telemetry::{debug, error, info, warn};
use snafu::ResultExt;
use snafu::{OptionExt, ResultExt};
use store_api::storage::RegionNumber;
use table::engine::manager::TableEngineManagerRef;
use table::engine::{CloseTableResult, EngineContext, TableEngineRef};
Expand All @@ -35,13 +40,20 @@ use crate::error::{Result, TableEngineNotFoundSnafu};
pub struct RegionAliveKeepers {
table_engine_manager: TableEngineManagerRef,
keepers: Arc<Mutex<HashMap<TableIdent, Arc<RegionAliveKeeper>>>>,

/// The epoch when [RegionAliveKeepers] is created. It's used to get a monotonically non-decreasing
/// elapsed time when submitting heartbeats to Metasrv (because [Instant] is monotonically
/// non-decreasing). The heartbeat request will carry the duration since this epoch, and the
/// duration acts like an "invariant point" for region's keep alive lease.
epoch: Instant,
}

impl RegionAliveKeepers {
pub fn new(table_engine_manager: TableEngineManagerRef) -> Self {
Self {
table_engine_manager,
keepers: Arc::new(Mutex::new(HashMap::new())),
epoch: Instant::now(),
}
}

Expand Down Expand Up @@ -107,6 +119,50 @@ impl RegionAliveKeepers {
keeper.start(heartbeat_interval_millis).await;
}
}

pub fn epoch(&self) -> Instant {
self.epoch
}
}

#[async_trait]
impl HeartbeatResponseHandler for RegionAliveKeepers {
fn is_acceptable(&self, ctx: &HeartbeatResponseHandlerContext) -> bool {
!ctx.response.region_leases.is_empty()
}

async fn handle(
&self,
ctx: &mut HeartbeatResponseHandlerContext,
) -> common_meta::error::Result<HandleControl> {
let leases = ctx.response.region_leases.drain(..).collect::<Vec<_>>();
for lease in leases {
let table_ident: TableIdent = match lease
.table_ident
.context(InvalidProtoMsgSnafu {
err_msg: "'table_ident' is missing in RegionLease",
})
.and_then(|x| x.try_into())
{
Ok(x) => x,
Err(e) => {
error!(e; "");
continue;
}
};

let Some(keeper) = self.keepers.lock().await.get(&table_ident).cloned() else {
// Alive keeper could be affected by lagging msg, just warn and ignore.
warn!("Alive keeper for table {table_ident} is not found!");
continue;
};

let start_instant = self.epoch + Duration::from_millis(lease.duration_since_epoch);
let deadline = start_instant + Duration::from_secs(lease.lease_seconds);
keeper.keep_lived(lease.regions, deadline).await;
}
Ok(HandleControl::Continue)
}
}

/// [RegionAliveKeeper] starts a countdown for each region in a table. When deadline is reached,
Expand Down Expand Up @@ -309,8 +365,11 @@ impl CountdownTask {
debug!("Reset deadline to region {region} of table {table_ident} to {deadline:?}");
countdown.set(tokio::time::sleep_until(deadline));
}
// Else the countdown could be not started yet, or during startup protection.
// Can be safely ignored.
// Else the countdown could be either:
// - not started yet;
// - during startup protection;
// - received a lagging heartbeat message.
// All can be safely ignored.
},
None => {
info!(
Expand Down Expand Up @@ -367,6 +426,8 @@ mod test {
use std::sync::atomic::{AtomicBool, Ordering};
use std::sync::Arc;

use api::v1::meta::{HeartbeatResponse, RegionLease};
use common_meta::heartbeat::mailbox::HeartbeatMailbox;
use datatypes::schema::RawSchema;
use table::engine::manager::MemoryTableEngineManager;
use table::engine::{TableEngine, TableReference};
Expand All @@ -377,8 +438,7 @@ mod test {
use super::*;
use crate::remote::mock::MockTableEngine;

#[tokio::test(flavor = "multi_thread")]
async fn test_region_alive_keepers() {
async fn prepare_keepers() -> (TableIdent, RegionAliveKeepers) {
let table_engine = Arc::new(MockTableEngine::default());
let table_engine_manager = Arc::new(MemoryTableEngineManager::new(table_engine));
let keepers = RegionAliveKeepers::new(table_engine_manager);
Expand Down Expand Up @@ -410,13 +470,82 @@ mod test {
table_options: TableOptions::default(),
engine: "MockTableEngine".to_string(),
}));

keepers
.register_table(table_ident.clone(), table)
.await
.unwrap();
assert!(keepers.keepers.lock().await.contains_key(&table_ident));

(table_ident, keepers)
}

#[tokio::test(flavor = "multi_thread")]
async fn test_handle_heartbeat_response() {
let (table_ident, keepers) = prepare_keepers().await;

keepers.start(5000).await;
let startup_protection_until = Instant::now() + Duration::from_secs(21);

let duration_since_epoch = (Instant::now() - keepers.epoch).as_millis() as _;
let lease_seconds = 100;
let response = HeartbeatResponse {
region_leases: vec![RegionLease {
table_ident: Some(table_ident.clone().into()),
regions: vec![1, 3], // Not extending region 2's lease time.
duration_since_epoch,
lease_seconds,
}],
..Default::default()
};
let keep_alive_until = keepers.epoch
+ Duration::from_millis(duration_since_epoch)
+ Duration::from_secs(lease_seconds);

let (tx, _) = mpsc::channel(8);
let mailbox = Arc::new(HeartbeatMailbox::new(tx));
let mut ctx = HeartbeatResponseHandlerContext::new(mailbox, response);

assert!(keepers.handle(&mut ctx).await.unwrap() == HandleControl::Continue);

// sleep to wait for background task spawned in `handle`
tokio::time::sleep(Duration::from_secs(1)).await;

async fn test(
keeper: &Arc<RegionAliveKeeper>,
region_number: RegionNumber,
startup_protection_until: Instant,
keep_alive_until: Instant,
is_kept_live: bool,
) {
let handles = keeper.countdown_task_handles.lock().await;
let deadline = deadline(&handles.get(&region_number).unwrap().tx).await;
if is_kept_live {
assert!(deadline > startup_protection_until && deadline == keep_alive_until);
} else {
assert!(deadline <= startup_protection_until);
}
}

let keeper = &keepers
.keepers
.lock()
.await
.get(&table_ident)
.cloned()
.unwrap();

// Test region 1 and 3 is kept lived. Their deadlines are updated to desired instant.
test(keeper, 1, startup_protection_until, keep_alive_until, true).await;
test(keeper, 3, startup_protection_until, keep_alive_until, true).await;

// Test region 2 is not kept lived. It's deadline is not updated: still during startup protection period.
test(keeper, 2, startup_protection_until, keep_alive_until, false).await;
}

#[tokio::test(flavor = "multi_thread")]
async fn test_region_alive_keepers() {
let (table_ident, keepers) = prepare_keepers().await;

keepers
.register_region(&RegionIdent {
cluster_id: 1,
Expand Down
1 change: 1 addition & 0 deletions src/common/meta/Cargo.toml
Original file line number Diff line number Diff line change
Expand Up @@ -6,6 +6,7 @@ license.workspace = true

[dependencies]
api = { path = "../../api" }
async-trait.workspace = true
common-catalog = { path = "../catalog" }
common-error = { path = "../error" }
common-runtime = { path = "../runtime" }
Expand Down
12 changes: 8 additions & 4 deletions src/common/meta/src/heartbeat/handler.rs
Original file line number Diff line number Diff line change
Expand Up @@ -15,6 +15,7 @@
use std::sync::Arc;

use api::v1::meta::HeartbeatResponse;
use async_trait::async_trait;
use common_telemetry::error;

use crate::error::Result;
Expand Down Expand Up @@ -57,14 +58,16 @@ impl HeartbeatResponseHandlerContext {
/// [`HeartbeatResponseHandler::is_acceptable`] returns true if handler can handle incoming [`HeartbeatResponseHandlerContext`].
///
/// [`HeartbeatResponseHandler::handle`] handles all or part of incoming [`HeartbeatResponseHandlerContext`].
#[async_trait]
pub trait HeartbeatResponseHandler: Send + Sync {
fn is_acceptable(&self, ctx: &HeartbeatResponseHandlerContext) -> bool;

fn handle(&self, ctx: &mut HeartbeatResponseHandlerContext) -> Result<HandleControl>;
async fn handle(&self, ctx: &mut HeartbeatResponseHandlerContext) -> Result<HandleControl>;
}

#[async_trait]
pub trait HeartbeatResponseHandlerExecutor: Send + Sync {
fn handle(&self, ctx: HeartbeatResponseHandlerContext) -> Result<()>;
async fn handle(&self, ctx: HeartbeatResponseHandlerContext) -> Result<()>;
}

pub struct HandlerGroupExecutor {
Expand All @@ -77,14 +80,15 @@ impl HandlerGroupExecutor {
}
}

#[async_trait]
impl HeartbeatResponseHandlerExecutor for HandlerGroupExecutor {
fn handle(&self, mut ctx: HeartbeatResponseHandlerContext) -> Result<()> {
async fn handle(&self, mut ctx: HeartbeatResponseHandlerContext) -> Result<()> {
for handler in &self.handlers {
if !handler.is_acceptable(&ctx) {
continue;
}

match handler.handle(&mut ctx) {
match handler.handle(&mut ctx).await {
Ok(HandleControl::Done) => break,
Ok(HandleControl::Continue) => {}
Err(e) => {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -12,6 +12,8 @@
// See the License for the specific language governing permissions and
// limitations under the License.

use async_trait::async_trait;

use crate::error::Result;
use crate::heartbeat::handler::{
HandleControl, HeartbeatResponseHandler, HeartbeatResponseHandlerContext,
Expand All @@ -21,12 +23,13 @@ use crate::heartbeat::utils::mailbox_message_to_incoming_message;
#[derive(Default)]
pub struct ParseMailboxMessageHandler;

#[async_trait]
impl HeartbeatResponseHandler for ParseMailboxMessageHandler {
fn is_acceptable(&self, _ctx: &HeartbeatResponseHandlerContext) -> bool {
true
}

fn handle(&self, ctx: &mut HeartbeatResponseHandlerContext) -> Result<HandleControl> {
async fn handle(&self, ctx: &mut HeartbeatResponseHandlerContext) -> Result<HandleControl> {
if let Some(message) = &ctx.response.mailbox_message {
if message.payload.is_some() {
// mailbox_message_to_incoming_message will raise an error if payload is none
Expand Down
16 changes: 15 additions & 1 deletion src/common/meta/src/ident.rs
Original file line number Diff line number Diff line change
Expand Up @@ -14,7 +14,7 @@

use std::fmt::{Display, Formatter};

use api::v1::meta::TableIdent as RawTableIdent;
use api::v1::meta::{TableIdent as RawTableIdent, TableName};
use serde::{Deserialize, Serialize};
use snafu::OptionExt;

Expand Down Expand Up @@ -55,3 +55,17 @@ impl TryFrom<RawTableIdent> for TableIdent {
})
}
}

impl From<TableIdent> for RawTableIdent {
fn from(table_ident: TableIdent) -> Self {
Self {
table_id: table_ident.table_id,
engine: table_ident.engine,
table_name: Some(TableName {
catalog_name: table_ident.catalog,
schema_name: table_ident.schema,
table_name: table_ident.table,
}),
}
}
}
Loading

0 comments on commit 2dd86b6

Please sign in to comment.