risingwave_meta::hummock

Struct HummockManager

source
pub struct HummockManager {
Show 20 fields pub env: MetaSrvEnv, metadata_manager: MetadataManager, compaction: MonitoredRwLock<Compaction>, versioning: MonitoredRwLock<Versioning>, compaction_group_manager: MonitoredRwLock<CompactionGroupManager>, context_info: MonitoredRwLock<ContextInfo>, pub metrics: Arc<MetaMetrics>, pub compactor_manager: CompactorManagerRef, event_sender: UnboundedSender<HummockManagerEvent>, object_store: ObjectStoreRef, version_checkpoint_path: String, version_archive_dir: String, pause_version_checkpoint: AtomicBool, history_table_throughput: RwLock<HashMap<u32, VecDeque<u64>>>, compactor_streams_change_tx: UnboundedSender<(u32, Streaming<SubscribeCompactionEventRequest>)>, pub compaction_state: CompactionState, full_gc_state: Arc<FullGcState>, now: Mutex<u64>, inflight_time_travel_query: Semaphore, gc_manager: GcManager,
}

Fields§

§env: MetaSrvEnv§metadata_manager: MetadataManager§compaction: MonitoredRwLock<Compaction>

Lock order: compaction, versioning, compaction_group_manager, context_info

  • Lock compaction first, then versioning, then compaction_group_manager and finally context_info.
  • This order should be strictly followed to prevent deadlock.
§versioning: MonitoredRwLock<Versioning>§compaction_group_manager: MonitoredRwLock<CompactionGroupManager>

CompactionGroupManager manages compaction configs for compaction groups.

§context_info: MonitoredRwLock<ContextInfo>§metrics: Arc<MetaMetrics>§compactor_manager: CompactorManagerRef§event_sender: UnboundedSender<HummockManagerEvent>§object_store: ObjectStoreRef§version_checkpoint_path: String§version_archive_dir: String§pause_version_checkpoint: AtomicBool§history_table_throughput: RwLock<HashMap<u32, VecDeque<u64>>>§compactor_streams_change_tx: UnboundedSender<(u32, Streaming<SubscribeCompactionEventRequest>)>§compaction_state: CompactionState§full_gc_state: Arc<FullGcState>§now: Mutex<u64>§inflight_time_travel_query: Semaphore§gc_manager: GcManager

Implementations§

source§

impl HummockManager

source

pub async fn release_contexts( &self, context_ids: impl AsRef<[HummockContextId]>, ) -> Result<()>

source

pub async fn check_context(&self, context_id: HummockContextId) -> Result<bool>

Checks whether context_id is valid.

source

async fn check_context_with_meta_node( &self, context_id: HummockContextId, context_info: &ContextInfo, ) -> Result<()>

source

pub async fn get_min_pinned_version_id(&self) -> HummockVersionId

source§

impl HummockManager

source

pub(super) async fn release_invalid_contexts( &self, ) -> Result<Vec<HummockContextId>>

Release invalid contexts, aka worker node ids which are no longer valid in ClusterManager.

source

pub async fn commit_epoch_sanity_check( &self, tables_to_commit: &HashMap<TableId, u64>, sstables: &[LocalSstableInfo], sst_to_context: &HashMap<HummockSstableObjectId, HummockContextId>, current_version: &HummockVersion, ) -> Result<()>

source

pub async fn release_meta_context(&self) -> Result<()>

source

pub(crate) async fn report_compaction_sanity_check( &self, object_timestamps: &HashMap<HummockSstableObjectId, u64>, ) -> Result<()>

source§

impl HummockManager

source

pub async fn pin_version( &self, context_id: HummockContextId, ) -> Result<HummockVersion>

Pin the current greatest hummock version. The pin belongs to context_id and will be unpinned when context_id is invalidated.

source

pub async fn unpin_version_before( &self, context_id: HummockContextId, unpin_before: HummockVersionId, ) -> Result<()>

Unpin all pins which belongs to context_id and has an id which is older than unpin_before. All versions >= unpin_before will be treated as if they are all pinned by this context_id so they will not be vacuumed.

source§

impl HummockManager

source§

impl HummockManager

source

pub async fn delete_version_deltas( &self, batch_size: usize, ) -> Result<(usize, usize)>

Deletes at most batch_size deltas.

Returns (number of deleted deltas, number of remain deltas_to_delete).

source

pub async fn finalize_objects_to_delete( &self, object_ids: impl Iterator<Item = HummockSstableObjectId> + Clone, ) -> Result<Vec<HummockSstableObjectId>>

Filters by Hummock version and Writes GC history.

source

pub async fn start_full_gc( &self, sst_retention_time: Duration, prefix: Option<String>, backup_manager: Option<BackupManagerRef>, ) -> Result<()>

LIST object store and DELETE stale objects, in batches. GC can be very slow. Spawn a dedicated tokio task for it.

source

pub(crate) async fn complete_gc_batch( &self, object_ids: HashSet<HummockSstableObjectId>, backup_manager: Option<BackupManagerRef>, ) -> Result<usize>

Given candidate SSTs to delete, filter out false positive. Returns number of SSTs to delete.

source

pub async fn now(&self) -> Result<u64>

source

pub(crate) async fn load_now(&self) -> Result<Option<u64>>

source

async fn write_gc_history( &self, object_ids: impl Iterator<Item = HummockSstableObjectId>, ) -> Result<()>

source

pub async fn delete_metadata(&self) -> MetaResult<usize>

Deletes stale Hummock metadata.

Returns number of deleted deltas

source

pub async fn delete_objects( &self, objects_to_delete: Vec<HummockSstableObjectId>, ) -> Result<usize>

Deletes stale SST objects from object store.

Returns the total count of deleted SST objects.

source

pub async fn try_start_minor_gc( &self, backup_manager: BackupManagerRef, ) -> Result<()>

Minor GC attempts to delete objects that were part of Hummock version but are no longer in use.

source§

impl HummockManager

source

pub async fn list_pinned_version(&self) -> Vec<HummockPinnedVersion>

source

pub async fn list_workers( &self, context_ids: &[HummockContextId], ) -> MetaResult<HashMap<HummockContextId, WorkerNode>>

source

pub async fn get_current_version(&self) -> HummockVersion

Gets current version without pinning it. Should not be called inside HummockManager, because it requests locks internally.

Note: this method can hurt performance because it will clone a large object.

source

pub async fn on_current_version<T>( &self, f: impl FnMut(&HummockVersion) -> T, ) -> T

source

pub async fn get_version_id(&self) -> HummockVersionId

source

pub async fn get_table_compaction_group_id_mapping( &self, ) -> HashMap<StateTableId, CompactionGroupId>

Gets the mapping from table id to compaction group id

source

pub async fn list_version_deltas( &self, start_id: HummockVersionId, num_limit: u32, ) -> Result<Vec<HummockVersionDelta>>

Get version deltas from meta store

source

pub async fn get_version_stats(&self) -> HummockVersionStats

source

pub(super) async fn try_update_write_limits( &self, target_group_ids: &[CompactionGroupId], ) -> bool

Updates write limits for target_groups and sends notification. Returns true if write_limit has been modified. The implementation acquires versioning lock and compaction_group_manager lock.

source

pub async fn write_limits(&self) -> HashMap<CompactionGroupId, WriteLimit>

Gets write limits. The implementation acquires versioning lock.

source

pub async fn list_branched_objects( &self, ) -> BTreeMap<HummockSstableObjectId, BranchedSstInfo>

source

pub async fn rebuild_table_stats(&self) -> Result<()>

source

pub async fn may_fill_backward_state_table_info(&self) -> Result<()>

source§

impl HummockManager

A hummock version checkpoint compacts previous hummock version delta logs, and stores stale objects from those delta logs.

source

pub async fn try_read_checkpoint( &self, ) -> Result<Option<HummockVersionCheckpoint>>

Returns Ok(None) if not found.

source

pub(super) async fn write_checkpoint( &self, checkpoint: &HummockVersionCheckpoint, ) -> Result<()>

source

pub(super) async fn write_version_archive( &self, archive: &PbHummockVersionArchive, ) -> Result<()>

source

pub async fn create_version_checkpoint( &self, min_delta_log_num: u64, ) -> Result<u64>

Creates a hummock version checkpoint. Returns the diff between new and old checkpoint id. Note that this method must not be called concurrently, because internally it doesn’t hold lock throughout the method.

source

pub fn pause_version_checkpoint(&self)

source

pub fn resume_version_checkpoint(&self)

source

pub fn is_version_checkpoint_paused(&self) -> bool

source

pub async fn get_checkpoint_version(&self) -> HummockVersion

source§

impl HummockManager

source

pub async fn commit_epoch(&self, commit_info: CommitEpochInfo) -> Result<()>

Caller should ensure epoch > committed_epoch of tables_to_commit if tables are not newly added via new_table_fragment_info

source

async fn collect_table_write_throughput(&self, table_stats: PbTableStatsMap)

source

async fn correct_commit_ssts( &self, sstables: Vec<LocalSstableInfo>, table_compaction_group_mapping: &HashMap<TableId, CompactionGroupId>, ) -> Result<BTreeMap<CompactionGroupId, Vec<SstableInfo>>>

source§

impl HummockManager

source

pub async fn compaction_group_ids(&self) -> Vec<CompactionGroupId>

Should not be called inside HummockManager, because it requests locks internally. The implementation acquires versioning lock.

source

pub async fn get_compaction_group_map( &self, ) -> BTreeMap<CompactionGroupId, CompactionGroup>

The implementation acquires compaction_group_manager lock.

source

pub async fn purge(&self, valid_ids: &HashSet<TableId>) -> Result<()>

Unregisters stale members and groups The caller should ensure table_fragments_list remain unchanged during purge. Currently purge is only called during meta service start ups.

source

pub async fn register_table_ids_for_test( &self, pairs: &[(StateTableId, CompactionGroupId)], ) -> Result<()>

The implementation acquires versioning lock.

The method name is temporarily added with a _for_test prefix to mark that it’s currently only used in test.

source

pub async fn unregister_table_ids( &self, table_ids: impl IntoIterator<Item = TableId> + Send, ) -> Result<()>

source

pub async fn update_compaction_config( &self, compaction_group_ids: &[CompactionGroupId], config_to_update: &[MutableConfig], ) -> Result<()>

source

pub async fn list_compaction_group(&self) -> Vec<CompactionGroupInfo>

Gets complete compaction group info. It is the aggregate of HummockVersion and CompactionGroupConfig

source

pub async fn calculate_compaction_group_statistic( &self, ) -> Vec<CompactionGroupStatistic>

source

pub(crate) async fn initial_compaction_group_config_after_load( &self, versioning_guard: &Versioning, compaction_group_manager: &mut CompactionGroupManager, ) -> Result<()>

source§

impl HummockManager

source

pub async fn merge_compaction_group( &self, group_1: CompactionGroupId, group_2: CompactionGroupId, ) -> Result<()>

source

pub async fn merge_compaction_group_for_test( &self, group_1: CompactionGroupId, group_2: CompactionGroupId, created_tables: HashSet<u32>, ) -> Result<()>

source

pub async fn merge_compaction_group_impl( &self, group_1: CompactionGroupId, group_2: CompactionGroupId, created_tables: Option<HashSet<u32>>, ) -> Result<()>

source§

impl HummockManager

source

async fn split_compaction_group_impl( &self, parent_group_id: CompactionGroupId, split_table_ids: &[StateTableId], table_id_to_split: StateTableId, vnode_to_split: VirtualNode, partition_vnode_count: Option<u32>, ) -> Result<Vec<(CompactionGroupId, Vec<StateTableId>)>>

Split table_ids to a dedicated compaction group.(will be split by the table_id and vnode.) Returns the compaction group id containing the table_ids and the mapping of compaction group id to table ids. The split will follow the following rules

  1. ssts with key_range.left greater than split_key will be split to the right group
  2. the sst containing split_key will be split into two separate ssts and their key_range will be changed sst_1: [sst.key_range.left, split_key) sst_2: [split_key, sst.key_range.right]
  3. currently only vnode 0 and vnode max is supported. (Due to the above rule, vnode max will be rewritten as table_id + 1, vnode 0) parent_group_id: the group_id to split split_table_ids: the table_ids to split, now we still support to split multiple tables to one group at once, pass split_table_ids for per split operation for checking table_id_to_split: the table_id to split vnode_to_split: the vnode to split partition_vnode_count: the partition count for the single table group if need
source

pub async fn move_state_tables_to_dedicated_compaction_group( &self, parent_group_id: CompactionGroupId, table_ids: &[StateTableId], partition_vnode_count: Option<u32>, ) -> Result<(CompactionGroupId, BTreeMap<CompactionGroupId, Vec<StateTableId>>)>

Split table_ids to a dedicated compaction group. Returns the compaction group id containing the table_ids and the mapping of compaction group id to table ids.

source§

impl HummockManager

source

pub async fn try_split_compaction_group( &self, table_write_throughput: &HashMap<u32, VecDeque<u64>>, checkpoint_secs: u64, group: CompactionGroupStatistic, )

Split the compaction group if the group is too large or contains high throughput tables.

source

pub async fn try_move_high_throughput_table_to_dedicated_cg( &self, table_write_throughput: &HashMap<u32, VecDeque<u64>>, table_id: &u32, _table_size: &u64, checkpoint_secs: u64, parent_group_id: u64, )

Try to move the high throughput table to a dedicated compaction group.

source

pub async fn try_split_huge_compaction_group( &self, group: CompactionGroupStatistic, )

source

pub async fn try_merge_compaction_group( &self, table_write_throughput: &HashMap<u32, VecDeque<u64>>, group: &CompactionGroupStatistic, next_group: &CompactionGroupStatistic, checkpoint_secs: u64, created_tables: &HashSet<u32>, ) -> Result<()>

source§

impl HummockManager

source

pub async fn get_assigned_compact_task_num(&self) -> u64

source

pub async fn list_compaction_status( &self, ) -> (Vec<PbCompactStatus>, Vec<CompactTaskAssignment>)

source

pub async fn get_compaction_scores( &self, compaction_group_id: CompactionGroupId, ) -> Vec<PickerInfo>

source§

impl HummockManager

source

async fn handle_pull_task_event( &self, context_id: u32, pull_task_count: usize, compaction_selectors: &mut HashMap<TaskType, Box<dyn CompactionSelector>>, max_get_task_probe_times: usize, )

source

pub fn compaction_event_loop( hummock_manager: Arc<Self>, compactor_streams_change_rx: UnboundedReceiver<(u32, Streaming<SubscribeCompactionEventRequest>)>, ) -> Vec<(JoinHandle<()>, Sender<()>)>

dedicated event runtime for CPU/IO bound event

source

pub fn add_compactor_stream( &self, context_id: u32, req_stream: Streaming<SubscribeCompactionEventRequest>, )

source

pub async fn auto_pick_compaction_group_and_type( &self, ) -> Option<(CompactionGroupId, TaskType)>

source

async fn auto_pick_compaction_groups_and_type( &self, ) -> (Vec<CompactionGroupId>, TaskType)

This method will return all compaction group id in a random order and task type. If there are any group block by write_limit, it will return a single array with TaskType::Emergency. If these groups get different task-type, it will return all group id with TaskType::Dynamic if the first group get TaskType::Dynamic, otherwise it will return the single group with other task type.

source

async fn compact_task_dedicated_event_handler( hummock_manager: Arc<HummockManager>, rx: UnboundedReceiver<(u32, Event)>, shutdown_rx_shared: Shared<OneShotReceiver<()>>, )

dedicated event runtime for CPU/IO bound event

source§

impl HummockManager

source

pub async fn get_compact_tasks_impl( &self, compaction_groups: Vec<CompactionGroupId>, max_select_count: usize, selector: &mut Box<dyn CompactionSelector>, ) -> Result<(Vec<CompactTask>, Vec<CompactionGroupId>)>

source

pub async fn cancel_compact_task( &self, task_id: u64, task_status: TaskStatus, ) -> Result<bool>

Cancels a compaction task no matter it’s assigned or unassigned.

source

pub async fn cancel_compact_tasks( &self, tasks: Vec<u64>, task_status: TaskStatus, ) -> Result<Vec<bool>>

source

async fn cancel_compact_task_impl( &self, task_ids: Vec<u64>, task_status: TaskStatus, ) -> Result<Vec<bool>>

source

async fn get_compact_tasks( &self, compaction_groups: Vec<CompactionGroupId>, max_select_count: usize, selector: &mut Box<dyn CompactionSelector>, ) -> Result<(Vec<CompactTask>, Vec<CompactionGroupId>)>

source

pub async fn get_compact_task( &self, compaction_group_id: CompactionGroupId, selector: &mut Box<dyn CompactionSelector>, ) -> Result<Option<CompactTask>>

source

pub async fn manual_get_compact_task( &self, compaction_group_id: CompactionGroupId, manual_compaction_option: ManualCompactionOption, ) -> Result<Option<CompactTask>>

source

pub async fn report_compact_task( &self, task_id: u64, task_status: TaskStatus, sorted_output_ssts: Vec<SstableInfo>, table_stats_change: Option<PbTableStatsMap>, object_timestamps: HashMap<HummockSstableObjectId, u64>, ) -> Result<bool>

source

pub async fn report_compact_tasks( &self, report_tasks: Vec<ReportTask>, ) -> Result<Vec<bool>>

Finishes or cancels a compaction task, according to task_status.

If context_id is not None, its validity will be checked when writing meta store. Its ownership of the task is checked as well.

Return Ok(false) indicates either the task is not found, or the task is not owned by context_id when context_id is not None.

source

pub async fn trigger_compaction_deterministic( &self, _base_version_id: HummockVersionId, compaction_groups: Vec<CompactionGroupId>, ) -> Result<()>

Triggers compacitons to specified compaction groups. Don’t wait for compaction finish

source

pub async fn trigger_manual_compaction( &self, compaction_group: CompactionGroupId, manual_compaction_option: ManualCompactionOption, ) -> Result<()>

source

pub fn try_send_compaction_request( &self, compaction_group: CompactionGroupId, task_type: TaskType, ) -> bool

Sends a compaction request.

source

pub(crate) async fn calculate_vnode_partition( &self, compact_task: &mut CompactTask, compaction_config: &CompactionConfig, )

source§

impl HummockManager

source

pub fn compactor_manager_ref_for_test(&self) -> CompactorManagerRef

source

pub async fn compaction_task_from_assignment_for_test( &self, task_id: u64, ) -> Option<CompactTaskAssignment>

source

pub async fn report_compact_task_for_test( &self, task_id: u64, compact_task: Option<CompactTask>, task_status: TaskStatus, sorted_output_ssts: Vec<SstableInfo>, table_stats_change: Option<PbTableStatsMap>, ) -> Result<()>

source§

impl HummockManager

Time travel.

source

pub(crate) async fn init_time_travel_state(&self) -> Result<()>

source

pub(crate) async fn truncate_time_travel_metadata( &self, epoch_watermark: HummockEpoch, ) -> Result<()>

source

pub(crate) async fn filter_out_objects_by_time_travel( &self, objects: impl Iterator<Item = HummockSstableObjectId>, ) -> Result<HashSet<HummockSstableObjectId>>

source

pub(crate) async fn time_travel_pinned_object_count(&self) -> Result<u64>

source

pub async fn epoch_to_version( &self, query_epoch: HummockEpoch, table_id: u32, ) -> Result<HummockVersion>

Attempt to locate the version corresponding to query_epoch.

The version is retrieved from hummock_epoch_to_version, selecting the entry with the largest epoch that’s lte query_epoch.

The resulted version is complete, i.e. with correct SstableInfo.

source

pub(crate) async fn write_time_travel_metadata( &self, txn: &DatabaseTransaction, version: Option<&HummockVersion>, delta: HummockVersionDelta, group_parents: &HashMap<CompactionGroupId, CompactionGroupId>, skip_sst_ids: &HashSet<HummockSstableId>, tables_to_commit: impl Iterator<Item = (&TableId, &CompactionGroupId, u64)>, ) -> Result<Option<HashSet<HummockSstableId>>>

source§

impl HummockManager

source

pub fn hummock_timer_task( hummock_manager: Arc<Self>, backup_manager: Option<BackupManagerRef>, ) -> (JoinHandle<()>, Sender<()>)

source§

impl HummockManager

source

async fn check_dead_task(&self)

source

async fn on_handle_schedule_group_split(&self)

Try to schedule a compaction split for the given compaction groups. The split will be triggered if the following conditions are met:

  1. state table throughput: If the table is in a high throughput state and it belongs to a multi table group, then an attempt will be made to split the table into separate compaction groups to increase its throughput and reduce the impact on write amplification.
  2. group size: If the group size has exceeded the set upper limit, e.g. max_group_size * split_group_size_ratio
source

async fn on_handle_trigger_multi_group(&self, task_type: TaskType)

source

async fn on_handle_schedule_group_merge(&self)

Try to schedule a compaction merge for the given compaction groups. The merge will be triggered if the following conditions are met:

  1. The compaction group is not contains creating table.
  2. The compaction group is a small group.
  3. All tables in compaction group is in a low throughput state.
source§

impl HummockManager

source

pub async fn get_new_sst_ids(&self, number: u32) -> Result<SstObjectIdRange>

source§

impl HummockManager

source

pub async fn start_worker( self: &HummockManagerRef, receiver: UnboundedReceiver<HummockManagerEvent>, ) -> JoinHandle<()>

source

async fn handle_hummock_manager_event(&self, event: HummockManagerEvent) -> bool

Returns false indicates to shutdown worker

source

async fn handle_local_notification(&self, notification: LocalNotification)

source§

impl HummockManager

source

pub async fn new( env: MetaSrvEnv, metadata_manager: MetadataManager, metrics: Arc<MetaMetrics>, compactor_manager: CompactorManagerRef, compactor_streams_change_tx: UnboundedSender<(u32, Streaming<SubscribeCompactionEventRequest>)>, ) -> Result<HummockManagerRef>

source

pub(super) async fn with_config( env: MetaSrvEnv, cluster_controller: ClusterControllerRef, catalog_controller: CatalogControllerRef, metrics: Arc<MetaMetrics>, compactor_manager: CompactorManagerRef, config: CompactionConfig, compactor_streams_change_tx: UnboundedSender<(u32, Streaming<SubscribeCompactionEventRequest>)>, ) -> HummockManagerRef

source

async fn new_impl( env: MetaSrvEnv, metadata_manager: MetadataManager, metrics: Arc<MetaMetrics>, compactor_manager: CompactorManagerRef, compaction_group_manager: CompactionGroupManager, compactor_streams_change_tx: UnboundedSender<(u32, Streaming<SubscribeCompactionEventRequest>)>, ) -> Result<HummockManagerRef>

source

fn meta_store_ref(&self) -> &SqlMetaStore

source

async fn load_meta_store_state(&self) -> Result<()>

Load state from meta store.

source

async fn load_meta_store_state_impl( &self, compaction_guard: &mut Compaction, versioning_guard: &mut Versioning, context_info: &mut ContextInfo, ) -> Result<()>

Load state from meta store.

source

pub fn init_metadata_for_version_replay( &self, _table_catalogs: Vec<Table>, _compaction_groups: Vec<PbCompactionGroupInfo>, ) -> Result<()>

source

pub async fn replay_version_delta( &self, version_delta: HummockVersionDelta, ) -> Result<(HummockVersion, Vec<CompactionGroupId>)>

Replay a version delta to current hummock version. Returns the version_id, max_committed_epoch of the new version and the modified compaction groups

source

pub async fn disable_commit_epoch(&self) -> HummockVersion

source

pub fn metadata_manager(&self) -> &MetadataManager

source

pub fn object_store_media_type(&self) -> &'static str

Auto Trait Implementations§

Blanket Implementations§

source§

impl<T> Any for T
where T: 'static + ?Sized,

source§

fn type_id(&self) -> TypeId

Gets the TypeId of self. Read more
source§

impl<T> Borrow<T> for T
where T: ?Sized,

source§

fn borrow(&self) -> &T

Immutably borrows from an owned value. Read more
source§

impl<T> BorrowMut<T> for T
where T: ?Sized,

source§

fn borrow_mut(&mut self) -> &mut T

Mutably borrows from an owned value. Read more
§

impl<T> Conv for T

§

fn conv<T>(self) -> T
where Self: Into<T>,

Converts self into T using Into<T>. Read more
§

impl<Choices> CoproductSubsetter<CNil, HNil> for Choices

§

type Remainder = Choices

§

fn subset( self, ) -> Result<CNil, <Choices as CoproductSubsetter<CNil, HNil>>::Remainder>

Extract a subset of the possible types in a coproduct (or get the remaining possibilities) Read more
§

impl<T> FmtForward for T

§

fn fmt_binary(self) -> FmtBinary<Self>
where Self: Binary,

Causes self to use its Binary implementation when Debug-formatted.
§

fn fmt_display(self) -> FmtDisplay<Self>
where Self: Display,

Causes self to use its Display implementation when Debug-formatted.
§

fn fmt_lower_exp(self) -> FmtLowerExp<Self>
where Self: LowerExp,

Causes self to use its LowerExp implementation when Debug-formatted.
§

fn fmt_lower_hex(self) -> FmtLowerHex<Self>
where Self: LowerHex,

Causes self to use its LowerHex implementation when Debug-formatted.
§

fn fmt_octal(self) -> FmtOctal<Self>
where Self: Octal,

Causes self to use its Octal implementation when Debug-formatted.
§

fn fmt_pointer(self) -> FmtPointer<Self>
where Self: Pointer,

Causes self to use its Pointer implementation when Debug-formatted.
§

fn fmt_upper_exp(self) -> FmtUpperExp<Self>
where Self: UpperExp,

Causes self to use its UpperExp implementation when Debug-formatted.
§

fn fmt_upper_hex(self) -> FmtUpperHex<Self>
where Self: UpperHex,

Causes self to use its UpperHex implementation when Debug-formatted.
§

fn fmt_list(self) -> FmtList<Self>
where &'a Self: for<'a> IntoIterator,

Formats each item in a sequence. Read more
source§

impl<T> From<T> for T

source§

fn from(t: T) -> T

Returns the argument unchanged.

§

impl<T> FutureExt for T

§

fn with_context(self, otel_cx: Context) -> WithContext<Self>

Attaches the provided Context to this type, returning a WithContext wrapper. Read more
§

fn with_current_context(self) -> WithContext<Self>

Attaches the current Context to this type, returning a WithContext wrapper. Read more
§

impl<T> Instrument for T

§

fn instrument(self, span: Span) -> Instrumented<Self>

Instruments this type with the provided [Span], returning an Instrumented wrapper. Read more
§

fn in_current_span(self) -> Instrumented<Self>

Instruments this type with the current Span, returning an Instrumented wrapper. Read more
source§

impl<T> Instrument for T

source§

fn instrument(self, span: Span) -> Instrumented<Self>

Instruments this type with the provided Span, returning an Instrumented wrapper. Read more
source§

fn in_current_span(self) -> Instrumented<Self>

Instruments this type with the current Span, returning an Instrumented wrapper. Read more
source§

impl<T, U> Into<U> for T
where U: From<T>,

source§

fn into(self) -> U

Calls U::from(self).

That is, this conversion is whatever the implementation of From<T> for U chooses to do.

source§

impl<T> IntoEither for T

source§

fn into_either(self, into_left: bool) -> Either<Self, Self>

Converts self into a Left variant of Either<Self, Self> if into_left is true. Converts self into a Right variant of Either<Self, Self> otherwise. Read more
source§

fn into_either_with<F>(self, into_left: F) -> Either<Self, Self>
where F: FnOnce(&Self) -> bool,

Converts self into a Left variant of Either<Self, Self> if into_left(&self) returns true. Converts self into a Right variant of Either<Self, Self> otherwise. Read more
source§

impl<T> IntoRequest<T> for T

source§

fn into_request(self) -> Request<T>

Wrap the input message T in a tonic::Request
§

impl<T> IntoResult<T> for T

§

type Err = Infallible

§

fn into_result(self) -> Result<T, <T as IntoResult<T>>::Err>

§

impl<Unshared, Shared> IntoShared<Shared> for Unshared
where Shared: FromUnshared<Unshared>,

§

fn into_shared(self) -> Shared

Creates a shared type from an unshared type.
§

impl<T, U, I> LiftInto<U, I> for T
where U: LiftFrom<T, I>,

§

fn lift_into(self) -> U

Performs the indexed conversion.
source§

impl<M> MetricVecRelabelExt for M

source§

fn relabel( self, metric_level: MetricLevel, relabel_threshold: MetricLevel, ) -> RelabeledMetricVec<M>

source§

fn relabel_n( self, metric_level: MetricLevel, relabel_threshold: MetricLevel, relabel_num: usize, ) -> RelabeledMetricVec<M>

source§

fn relabel_debug_1( self, relabel_threshold: MetricLevel, ) -> RelabeledMetricVec<M>

Equivalent to RelabeledMetricVec::with_metric_level_relabel_n with metric_level set to MetricLevel::Debug and relabel_num set to 1.
§

impl<T> Pipe for T
where T: ?Sized,

§

fn pipe<R>(self, func: impl FnOnce(Self) -> R) -> R
where Self: Sized,

Pipes by value. This is generally the method you want to use. Read more
§

fn pipe_ref<'a, R>(&'a self, func: impl FnOnce(&'a Self) -> R) -> R
where R: 'a,

Borrows self and passes that borrow into the pipe function. Read more
§

fn pipe_ref_mut<'a, R>(&'a mut self, func: impl FnOnce(&'a mut Self) -> R) -> R
where R: 'a,

Mutably borrows self and passes that borrow into the pipe function. Read more
§

fn pipe_borrow<'a, B, R>(&'a self, func: impl FnOnce(&'a B) -> R) -> R
where Self: Borrow<B>, B: 'a + ?Sized, R: 'a,

Borrows self, then passes self.borrow() into the pipe function. Read more
§

fn pipe_borrow_mut<'a, B, R>( &'a mut self, func: impl FnOnce(&'a mut B) -> R, ) -> R
where Self: BorrowMut<B>, B: 'a + ?Sized, R: 'a,

Mutably borrows self, then passes self.borrow_mut() into the pipe function. Read more
§

fn pipe_as_ref<'a, U, R>(&'a self, func: impl FnOnce(&'a U) -> R) -> R
where Self: AsRef<U>, U: 'a + ?Sized, R: 'a,

Borrows self, then passes self.as_ref() into the pipe function.
§

fn pipe_as_mut<'a, U, R>(&'a mut self, func: impl FnOnce(&'a mut U) -> R) -> R
where Self: AsMut<U>, U: 'a + ?Sized, R: 'a,

Mutably borrows self, then passes self.as_mut() into the pipe function.
§

fn pipe_deref<'a, T, R>(&'a self, func: impl FnOnce(&'a T) -> R) -> R
where Self: Deref<Target = T>, T: 'a + ?Sized, R: 'a,

Borrows self, then passes self.deref() into the pipe function.
§

fn pipe_deref_mut<'a, T, R>( &'a mut self, func: impl FnOnce(&'a mut T) -> R, ) -> R
where Self: DerefMut<Target = T> + Deref, T: 'a + ?Sized, R: 'a,

Mutably borrows self, then passes self.deref_mut() into the pipe function.
§

impl<T> Pointable for T

§

const ALIGN: usize = _

The alignment of pointer.
§

type Init = T

The type for initializers.
§

unsafe fn init(init: <T as Pointable>::Init) -> usize

Initializes a with the given initializer. Read more
§

unsafe fn deref<'a>(ptr: usize) -> &'a T

Dereferences the given pointer. Read more
§

unsafe fn deref_mut<'a>(ptr: usize) -> &'a mut T

Mutably dereferences the given pointer. Read more
§

unsafe fn drop(ptr: usize)

Drops the object pointed to by the given pointer. Read more
source§

impl<T> Same for T

source§

type Output = T

Should always be Self
§

impl<Source> Sculptor<HNil, HNil> for Source

§

type Remainder = Source

§

fn sculpt(self) -> (HNil, <Source as Sculptor<HNil, HNil>>::Remainder)

Consumes the current HList and returns an HList with the requested shape. Read more
source§

impl<T> SerTo<T> for T

§

impl<T> Tap for T

§

fn tap(self, func: impl FnOnce(&Self)) -> Self

Immutable access to a value. Read more
§

fn tap_mut(self, func: impl FnOnce(&mut Self)) -> Self

Mutable access to a value. Read more
§

fn tap_borrow<B>(self, func: impl FnOnce(&B)) -> Self
where Self: Borrow<B>, B: ?Sized,

Immutable access to the Borrow<B> of a value. Read more
§

fn tap_borrow_mut<B>(self, func: impl FnOnce(&mut B)) -> Self
where Self: BorrowMut<B>, B: ?Sized,

Mutable access to the BorrowMut<B> of a value. Read more
§

fn tap_ref<R>(self, func: impl FnOnce(&R)) -> Self
where Self: AsRef<R>, R: ?Sized,

Immutable access to the AsRef<R> view of a value. Read more
§

fn tap_ref_mut<R>(self, func: impl FnOnce(&mut R)) -> Self
where Self: AsMut<R>, R: ?Sized,

Mutable access to the AsMut<R> view of a value. Read more
§

fn tap_deref<T>(self, func: impl FnOnce(&T)) -> Self
where Self: Deref<Target = T>, T: ?Sized,

Immutable access to the Deref::Target of a value. Read more
§

fn tap_deref_mut<T>(self, func: impl FnOnce(&mut T)) -> Self
where Self: DerefMut<Target = T> + Deref, T: ?Sized,

Mutable access to the Deref::Target of a value. Read more
§

fn tap_dbg(self, func: impl FnOnce(&Self)) -> Self

Calls .tap() only in debug builds, and is erased in release builds.
§

fn tap_mut_dbg(self, func: impl FnOnce(&mut Self)) -> Self

Calls .tap_mut() only in debug builds, and is erased in release builds.
§

fn tap_borrow_dbg<B>(self, func: impl FnOnce(&B)) -> Self
where Self: Borrow<B>, B: ?Sized,

Calls .tap_borrow() only in debug builds, and is erased in release builds.
§

fn tap_borrow_mut_dbg<B>(self, func: impl FnOnce(&mut B)) -> Self
where Self: BorrowMut<B>, B: ?Sized,

Calls .tap_borrow_mut() only in debug builds, and is erased in release builds.
§

fn tap_ref_dbg<R>(self, func: impl FnOnce(&R)) -> Self
where Self: AsRef<R>, R: ?Sized,

Calls .tap_ref() only in debug builds, and is erased in release builds.
§

fn tap_ref_mut_dbg<R>(self, func: impl FnOnce(&mut R)) -> Self
where Self: AsMut<R>, R: ?Sized,

Calls .tap_ref_mut() only in debug builds, and is erased in release builds.
§

fn tap_deref_dbg<T>(self, func: impl FnOnce(&T)) -> Self
where Self: Deref<Target = T>, T: ?Sized,

Calls .tap_deref() only in debug builds, and is erased in release builds.
§

fn tap_deref_mut_dbg<T>(self, func: impl FnOnce(&mut T)) -> Self
where Self: DerefMut<Target = T> + Deref, T: ?Sized,

Calls .tap_deref_mut() only in debug builds, and is erased in release builds.
§

impl<T> TryConv for T

§

fn try_conv<T>(self) -> Result<T, Self::Error>
where Self: TryInto<T>,

Attempts to convert self into T using TryInto<T>. Read more
source§

impl<T, U> TryFrom<U> for T
where U: Into<T>,

source§

type Error = Infallible

The type returned in the event of a conversion error.
source§

fn try_from(value: U) -> Result<T, <T as TryFrom<U>>::Error>

Performs the conversion.
source§

impl<T, U> TryInto<U> for T
where U: TryFrom<T>,

source§

type Error = <U as TryFrom<T>>::Error

The type returned in the event of a conversion error.
source§

fn try_into(self) -> Result<U, <U as TryFrom<T>>::Error>

Performs the conversion.
§

impl<V, T> VZip<V> for T
where V: MultiLane<T>,

§

fn vzip(self) -> V

§

impl<T> WithSubscriber for T

§

fn with_subscriber<S>(self, subscriber: S) -> WithDispatch<Self>
where S: Into<Dispatch>,

Attaches the provided Subscriber to this type, returning a [WithDispatch] wrapper. Read more
§

fn with_current_subscriber(self) -> WithDispatch<Self>

Attaches the current default Subscriber to this type, returning a [WithDispatch] wrapper. Read more
source§

impl<T> WithSubscriber for T

source§

fn with_subscriber<S>(self, subscriber: S) -> WithDispatch<Self>
where S: Into<Dispatch>,

Attaches the provided Subscriber to this type, returning a WithDispatch wrapper. Read more
source§

fn with_current_subscriber(self) -> WithDispatch<Self>

Attaches the current default Subscriber to this type, returning a WithDispatch wrapper. Read more
source§

impl<T> LruValue for T
where T: Send + Sync,

§

impl<T> MaybeSend for T
where T: Send,

§

impl<T> MaybeSend for T
where T: Send,

§

impl<T> Value for T
where T: Send + Sync + 'static,