mirror of
https://github.com/neondatabase/neon.git
synced 2026-01-07 21:42:56 +00:00
Add tracing for shared locks in id_lock_map (#7618)
## Problem Storage controller shared locks do not print a warning when held for long time spans. ## Summary of changes Extension of issue https://github.com/neondatabase/neon/issues/7108 in tracing to exclusive lock in `id_lock_map` was added, to add the same for shared locks. It was mentioned in the comment https://github.com/neondatabase/neon/pull/7397#discussion_r1587961160
This commit is contained in:
@@ -8,14 +8,15 @@ use crate::service::RECONCILE_TIMEOUT;
|
||||
|
||||
const LOCK_TIMEOUT_ALERT_THRESHOLD: Duration = RECONCILE_TIMEOUT;
|
||||
|
||||
/// A wrapper around `OwnedRwLockWriteGuard` that when dropped changes the
|
||||
/// current holding operation in lock.
|
||||
pub struct WrappedWriteGuard<T: Display> {
|
||||
/// A wrapper around `OwnedRwLockWriteGuard` used for tracking the
|
||||
/// operation that holds the lock, and print a warning if it exceeds
|
||||
/// the LOCK_TIMEOUT_ALERT_THRESHOLD time
|
||||
pub struct TracingExclusiveGuard<T: Display> {
|
||||
guard: tokio::sync::OwnedRwLockWriteGuard<Option<T>>,
|
||||
start: Instant,
|
||||
}
|
||||
|
||||
impl<T: Display> WrappedWriteGuard<T> {
|
||||
impl<T: Display> TracingExclusiveGuard<T> {
|
||||
pub fn new(guard: tokio::sync::OwnedRwLockWriteGuard<Option<T>>) -> Self {
|
||||
Self {
|
||||
guard,
|
||||
@@ -24,12 +25,12 @@ impl<T: Display> WrappedWriteGuard<T> {
|
||||
}
|
||||
}
|
||||
|
||||
impl<T: Display> Drop for WrappedWriteGuard<T> {
|
||||
impl<T: Display> Drop for TracingExclusiveGuard<T> {
|
||||
fn drop(&mut self) {
|
||||
let duration = self.start.elapsed();
|
||||
if duration > LOCK_TIMEOUT_ALERT_THRESHOLD {
|
||||
tracing::warn!(
|
||||
"Lock on {} was held for {:?}",
|
||||
"Exclusive lock by {} was held for {:?}",
|
||||
self.guard.as_ref().unwrap(),
|
||||
duration
|
||||
);
|
||||
@@ -38,6 +39,38 @@ impl<T: Display> Drop for WrappedWriteGuard<T> {
|
||||
}
|
||||
}
|
||||
|
||||
// A wrapper around `OwnedRwLockReadGuard` used for tracking the
|
||||
/// operation that holds the lock, and print a warning if it exceeds
|
||||
/// the LOCK_TIMEOUT_ALERT_THRESHOLD time
|
||||
pub struct TracingSharedGuard<T: Display> {
|
||||
_guard: tokio::sync::OwnedRwLockReadGuard<Option<T>>,
|
||||
operation: T,
|
||||
start: Instant,
|
||||
}
|
||||
|
||||
impl<T: Display> TracingSharedGuard<T> {
|
||||
pub fn new(guard: tokio::sync::OwnedRwLockReadGuard<Option<T>>, operation: T) -> Self {
|
||||
Self {
|
||||
_guard: guard,
|
||||
operation,
|
||||
start: Instant::now(),
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
impl<T: Display> Drop for TracingSharedGuard<T> {
|
||||
fn drop(&mut self) {
|
||||
let duration = self.start.elapsed();
|
||||
if duration > LOCK_TIMEOUT_ALERT_THRESHOLD {
|
||||
tracing::warn!(
|
||||
"Shared lock by {} was held for {:?}",
|
||||
self.operation,
|
||||
duration
|
||||
);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/// A map of locks covering some arbitrary identifiers. Useful if you have a collection of objects but don't
|
||||
/// want to embed a lock in each one, or if your locking granularity is different to your object granularity.
|
||||
/// For example, used in the storage controller where the objects are tenant shards, but sometimes locking
|
||||
@@ -58,21 +91,22 @@ where
|
||||
pub(crate) fn shared(
|
||||
&self,
|
||||
key: T,
|
||||
) -> impl std::future::Future<Output = tokio::sync::OwnedRwLockReadGuard<Option<I>>> {
|
||||
operation: I,
|
||||
) -> impl std::future::Future<Output = TracingSharedGuard<I>> {
|
||||
let mut locked = self.entities.lock().unwrap();
|
||||
let entry = locked.entry(key).or_default();
|
||||
entry.clone().read_owned()
|
||||
let entry = locked.entry(key).or_default().clone();
|
||||
async move { TracingSharedGuard::new(entry.read_owned().await, operation) }
|
||||
}
|
||||
|
||||
pub(crate) fn exclusive(
|
||||
&self,
|
||||
key: T,
|
||||
operation: I,
|
||||
) -> impl std::future::Future<Output = WrappedWriteGuard<I>> {
|
||||
) -> impl std::future::Future<Output = TracingExclusiveGuard<I>> {
|
||||
let mut locked = self.entities.lock().unwrap();
|
||||
let entry = locked.entry(key).or_default().clone();
|
||||
async move {
|
||||
let mut guard = WrappedWriteGuard::new(entry.clone().write_owned().await);
|
||||
let mut guard = TracingExclusiveGuard::new(entry.write_owned().await);
|
||||
*guard.guard = Some(operation);
|
||||
guard
|
||||
}
|
||||
@@ -99,12 +133,12 @@ where
|
||||
|
||||
pub async fn trace_exclusive_lock<
|
||||
T: Clone + Display + Eq + PartialEq + std::hash::Hash,
|
||||
I: Display + Clone,
|
||||
I: Clone + Display,
|
||||
>(
|
||||
op_locks: &IdLockMap<T, I>,
|
||||
key: T,
|
||||
operation: I,
|
||||
) -> WrappedWriteGuard<I> {
|
||||
) -> TracingExclusiveGuard<I> {
|
||||
let start = Instant::now();
|
||||
let guard = op_locks.exclusive(key.clone(), operation.clone()).await;
|
||||
|
||||
@@ -123,14 +157,14 @@ pub async fn trace_exclusive_lock<
|
||||
|
||||
pub async fn trace_shared_lock<
|
||||
T: Clone + Display + Eq + PartialEq + std::hash::Hash,
|
||||
I: Display,
|
||||
I: Clone + Display,
|
||||
>(
|
||||
op_locks: &IdLockMap<T, I>,
|
||||
key: T,
|
||||
operation: I,
|
||||
) -> tokio::sync::OwnedRwLockReadGuard<Option<I>> {
|
||||
) -> TracingSharedGuard<I> {
|
||||
let start = Instant::now();
|
||||
let guard = op_locks.shared(key.clone()).await;
|
||||
let guard = op_locks.shared(key.clone(), operation.clone()).await;
|
||||
|
||||
let duration = start.elapsed();
|
||||
if duration > LOCK_TIMEOUT_ALERT_THRESHOLD {
|
||||
@@ -159,11 +193,11 @@ mod tests {
|
||||
async fn multiple_shared_locks() {
|
||||
let id_lock_map: IdLockMap<i32, Operations> = IdLockMap::default();
|
||||
|
||||
let shared_lock_1 = id_lock_map.shared(1).await;
|
||||
let shared_lock_2 = id_lock_map.shared(1).await;
|
||||
let shared_lock_1 = id_lock_map.shared(1, Operations::Op1).await;
|
||||
let shared_lock_2 = id_lock_map.shared(1, Operations::Op2).await;
|
||||
|
||||
assert!(shared_lock_1.is_none());
|
||||
assert!(shared_lock_2.is_none());
|
||||
assert_eq!(shared_lock_1.operation, Operations::Op1);
|
||||
assert_eq!(shared_lock_2.operation, Operations::Op2);
|
||||
}
|
||||
|
||||
#[tokio::test]
|
||||
@@ -183,7 +217,7 @@ mod tests {
|
||||
assert!(_ex_lock_2.is_err());
|
||||
}
|
||||
|
||||
let shared_lock_1 = id_lock_map.shared(resource_id).await;
|
||||
assert!(shared_lock_1.is_none());
|
||||
let shared_lock_1 = id_lock_map.shared(resource_id, Operations::Op1).await;
|
||||
assert_eq!(shared_lock_1.operation, Operations::Op1);
|
||||
}
|
||||
}
|
||||
|
||||
@@ -13,7 +13,7 @@ use crate::{
|
||||
Drain, Fill, Operation, OperationError, OperationHandler, MAX_RECONCILES_PER_OPERATION,
|
||||
},
|
||||
compute_hook::NotifyError,
|
||||
id_lock_map::{trace_exclusive_lock, trace_shared_lock, IdLockMap, WrappedWriteGuard},
|
||||
id_lock_map::{trace_exclusive_lock, trace_shared_lock, IdLockMap, TracingExclusiveGuard},
|
||||
persistence::{AbortShardSplitStatus, TenantFilter},
|
||||
reconciler::{ReconcileError, ReconcileUnits},
|
||||
scheduler::{MaySchedule, ScheduleContext, ScheduleMode},
|
||||
@@ -359,7 +359,7 @@ struct TenantShardSplitAbort {
|
||||
new_shard_count: ShardCount,
|
||||
new_stripe_size: Option<ShardStripeSize>,
|
||||
/// Until this abort op is complete, no other operations may be done on the tenant
|
||||
_tenant_lock: WrappedWriteGuard<TenantOperations>,
|
||||
_tenant_lock: TracingExclusiveGuard<TenantOperations>,
|
||||
}
|
||||
|
||||
#[derive(thiserror::Error, Debug)]
|
||||
@@ -1429,7 +1429,7 @@ impl Service {
|
||||
async fn node_activate_reconcile(
|
||||
&self,
|
||||
mut node: Node,
|
||||
_lock: &WrappedWriteGuard<NodeOperations>,
|
||||
_lock: &TracingExclusiveGuard<NodeOperations>,
|
||||
) -> Result<(), ApiError> {
|
||||
// This Node is a mutable local copy: we will set it active so that we can use its
|
||||
// API client to reconcile with the node. The Node in [`Self::nodes`] will get updated
|
||||
@@ -2658,6 +2658,7 @@ impl Service {
|
||||
TenantOperations::TimelineCreate,
|
||||
)
|
||||
.await;
|
||||
failpoint_support::sleep_millis_async!("tenant-create-timeline-shared-lock");
|
||||
|
||||
self.ensure_attached_wait(tenant_id).await?;
|
||||
|
||||
|
||||
Reference in New Issue
Block a user