pub struct PlanRef(Rc<dyn PlanNode>);
Tuple Fields§
§0: Rc<dyn PlanNode>
Implementations§
source§impl PlanRef
impl PlanRef
pub fn expect_stream_key(&self) -> &[usize]
fn prune_col_inner( &self, required_cols: &[usize], ctx: &mut ColumnPruningContext, ) -> PlanRef
fn predicate_pushdown_inner( &self, predicate: Condition, ctx: &mut PredicatePushdownContext, ) -> PlanRef
source§impl PlanRef
impl PlanRef
fn optimize_by_rules_inner( self, heuristic_optimizer: &mut HeuristicOptimizer<'_>, stage_name: &str, ) -> Result<PlanRef, RwError>
pub(crate) fn optimize_by_rules( self, _: &OptimizationStage, ) -> Result<PlanRef, RwError>
pub(crate) fn optimize_by_rules_until_fix_point( self, _: &OptimizationStage, ) -> Result<PlanRef, RwError>
Methods from Deref<Target = dyn PlanNode>§
sourcepub fn is<__T: PlanNode>(&self) -> bool
pub fn is<__T: PlanNode>(&self) -> bool
Returns true if the trait object wraps an object of type __T
.
sourcepub fn downcast_rc<__T: PlanNode>(self: Rc<Self>) -> Result<Rc<__T>, Rc<Self>>
pub fn downcast_rc<__T: PlanNode>(self: Rc<Self>) -> Result<Rc<__T>, Rc<Self>>
Returns an Rc
-ed object from an Rc
-ed trait object if the underlying object is of
type __T
. Returns the original Rc
-ed trait if it isn’t.
sourcepub fn downcast_ref<__T: PlanNode>(&self) -> Option<&__T>
pub fn downcast_ref<__T: PlanNode>(&self) -> Option<&__T>
Returns a reference to the object within the trait object if it is of type __T
, or
None
if it isn’t.
pub fn id(&self) -> PlanNodeId
pub fn ctx(&self) -> OptimizerContextRef
pub fn schema(&self) -> &Schema
pub fn stream_key(&self) -> Option<&[usize]>
pub fn functional_dependency(&self) -> &FunctionalDependencySet
sourcepub fn to_stream_prost(
&self,
state: &mut BuildFragmentGraphState,
) -> Result<PbStreamPlan, SchedulerError>
pub fn to_stream_prost( &self, state: &mut BuildFragmentGraphState, ) -> Result<PbStreamPlan, SchedulerError>
Serialize the plan node and its children to a stream plan proto.
Note that some operators has their own implementation of to_stream_prost
. We have a
hook inside to do some ad-hoc things.
sourcepub fn to_batch_prost(&self) -> Result<PbBatchPlan, SchedulerError>
pub fn to_batch_prost(&self) -> Result<PbBatchPlan, SchedulerError>
Serialize the plan node and its children to a batch plan proto.
sourcepub fn to_batch_prost_identity(
&self,
identity: bool,
) -> Result<PbBatchPlan, SchedulerError>
pub fn to_batch_prost_identity( &self, identity: bool, ) -> Result<PbBatchPlan, SchedulerError>
Serialize the plan node and its children to a batch plan proto without the identity field (for testing).
pub fn explain_myself_to_string(&self) -> String
pub fn as_logical_agg(&self) -> Option<&LogicalAgg>
pub fn as_logical_apply(&self) -> Option<&LogicalApply>
pub fn as_logical_filter(&self) -> Option<&LogicalFilter>
pub fn as_logical_project(&self) -> Option<&LogicalProject>
pub fn as_logical_scan(&self) -> Option<&LogicalScan>
pub fn as_logical_cdc_scan(&self) -> Option<&LogicalCdcScan>
pub fn as_logical_sys_scan(&self) -> Option<&LogicalSysScan>
pub fn as_logical_source(&self) -> Option<&LogicalSource>
pub fn as_logical_insert(&self) -> Option<&LogicalInsert>
pub fn as_logical_delete(&self) -> Option<&LogicalDelete>
pub fn as_logical_update(&self) -> Option<&LogicalUpdate>
pub fn as_logical_join(&self) -> Option<&LogicalJoin>
pub fn as_logical_values(&self) -> Option<&LogicalValues>
pub fn as_logical_limit(&self) -> Option<&LogicalLimit>
pub fn as_logical_top_n(&self) -> Option<&LogicalTopN>
pub fn as_logical_hop_window(&self) -> Option<&LogicalHopWindow>
pub fn as_logical_table_function(&self) -> Option<&LogicalTableFunction>
pub fn as_logical_multi_join(&self) -> Option<&LogicalMultiJoin>
pub fn as_logical_expand(&self) -> Option<&LogicalExpand>
pub fn as_logical_project_set(&self) -> Option<&LogicalProjectSet>
pub fn as_logical_union(&self) -> Option<&LogicalUnion>
pub fn as_logical_over_window(&self) -> Option<&LogicalOverWindow>
pub fn as_logical_now(&self) -> Option<&LogicalNow>
pub fn as_logical_dedup(&self) -> Option<&LogicalDedup>
pub fn as_logical_intersect(&self) -> Option<&LogicalIntersect>
pub fn as_logical_except(&self) -> Option<&LogicalExcept>
pub fn as_logical_max_one_row(&self) -> Option<&LogicalMaxOneRow>
pub fn as_logical_kafka_scan(&self) -> Option<&LogicalKafkaScan>
pub fn as_logical_iceberg_scan(&self) -> Option<&LogicalIcebergScan>
pub fn as_logical_recursive_union(&self) -> Option<&LogicalRecursiveUnion>
pub fn as_logical_cte_ref(&self) -> Option<&LogicalCteRef>
pub fn as_logical_change_log(&self) -> Option<&LogicalChangeLog>
pub fn as_logical_file_scan(&self) -> Option<&LogicalFileScan>
pub fn as_logical_postgres_query(&self) -> Option<&LogicalPostgresQuery>
pub fn as_logical_my_sql_query(&self) -> Option<&LogicalMySqlQuery>
pub fn as_batch_simple_agg(&self) -> Option<&BatchSimpleAgg>
pub fn as_batch_hash_agg(&self) -> Option<&BatchHashAgg>
pub fn as_batch_sort_agg(&self) -> Option<&BatchSortAgg>
pub fn as_batch_project(&self) -> Option<&BatchProject>
pub fn as_batch_filter(&self) -> Option<&BatchFilter>
pub fn as_batch_insert(&self) -> Option<&BatchInsert>
pub fn as_batch_delete(&self) -> Option<&BatchDelete>
pub fn as_batch_update(&self) -> Option<&BatchUpdate>
pub fn as_batch_seq_scan(&self) -> Option<&BatchSeqScan>
pub fn as_batch_sys_seq_scan(&self) -> Option<&BatchSysSeqScan>
pub fn as_batch_log_seq_scan(&self) -> Option<&BatchLogSeqScan>
pub fn as_batch_hash_join(&self) -> Option<&BatchHashJoin>
pub fn as_batch_nested_loop_join(&self) -> Option<&BatchNestedLoopJoin>
pub fn as_batch_values(&self) -> Option<&BatchValues>
pub fn as_batch_sort(&self) -> Option<&BatchSort>
pub fn as_batch_exchange(&self) -> Option<&BatchExchange>
pub fn as_batch_limit(&self) -> Option<&BatchLimit>
pub fn as_batch_top_n(&self) -> Option<&BatchTopN>
pub fn as_batch_hop_window(&self) -> Option<&BatchHopWindow>
pub fn as_batch_table_function(&self) -> Option<&BatchTableFunction>
pub fn as_batch_expand(&self) -> Option<&BatchExpand>
pub fn as_batch_lookup_join(&self) -> Option<&BatchLookupJoin>
pub fn as_batch_project_set(&self) -> Option<&BatchProjectSet>
pub fn as_batch_union(&self) -> Option<&BatchUnion>
pub fn as_batch_group_top_n(&self) -> Option<&BatchGroupTopN>
pub fn as_batch_source(&self) -> Option<&BatchSource>
pub fn as_batch_over_window(&self) -> Option<&BatchOverWindow>
pub fn as_batch_max_one_row(&self) -> Option<&BatchMaxOneRow>
pub fn as_batch_kafka_scan(&self) -> Option<&BatchKafkaScan>
pub fn as_batch_iceberg_scan(&self) -> Option<&BatchIcebergScan>
pub fn as_batch_file_scan(&self) -> Option<&BatchFileScan>
pub fn as_batch_postgres_query(&self) -> Option<&BatchPostgresQuery>
pub fn as_batch_my_sql_query(&self) -> Option<&BatchMySqlQuery>
pub fn as_stream_project(&self) -> Option<&StreamProject>
pub fn as_stream_filter(&self) -> Option<&StreamFilter>
pub fn as_stream_table_scan(&self) -> Option<&StreamTableScan>
pub fn as_stream_cdc_table_scan(&self) -> Option<&StreamCdcTableScan>
pub fn as_stream_sink(&self) -> Option<&StreamSink>
pub fn as_stream_source(&self) -> Option<&StreamSource>
pub fn as_stream_source_scan(&self) -> Option<&StreamSourceScan>
pub fn as_stream_hash_join(&self) -> Option<&StreamHashJoin>
pub fn as_stream_exchange(&self) -> Option<&StreamExchange>
pub fn as_stream_hash_agg(&self) -> Option<&StreamHashAgg>
pub fn as_stream_simple_agg(&self) -> Option<&StreamSimpleAgg>
pub fn as_stream_stateless_simple_agg( &self, ) -> Option<&StreamStatelessSimpleAgg>
pub fn as_stream_materialize(&self) -> Option<&StreamMaterialize>
pub fn as_stream_top_n(&self) -> Option<&StreamTopN>
pub fn as_stream_hop_window(&self) -> Option<&StreamHopWindow>
pub fn as_stream_delta_join(&self) -> Option<&StreamDeltaJoin>
pub fn as_stream_expand(&self) -> Option<&StreamExpand>
pub fn as_stream_dynamic_filter(&self) -> Option<&StreamDynamicFilter>
pub fn as_stream_project_set(&self) -> Option<&StreamProjectSet>
pub fn as_stream_group_top_n(&self) -> Option<&StreamGroupTopN>
pub fn as_stream_union(&self) -> Option<&StreamUnion>
pub fn as_stream_row_id_gen(&self) -> Option<&StreamRowIdGen>
pub fn as_stream_dml(&self) -> Option<&StreamDml>
pub fn as_stream_now(&self) -> Option<&StreamNow>
pub fn as_stream_watermark_filter(&self) -> Option<&StreamWatermarkFilter>
pub fn as_stream_temporal_join(&self) -> Option<&StreamTemporalJoin>
pub fn as_stream_values(&self) -> Option<&StreamValues>
pub fn as_stream_dedup(&self) -> Option<&StreamDedup>
pub fn as_stream_eowc_over_window(&self) -> Option<&StreamEowcOverWindow>
pub fn as_stream_eowc_sort(&self) -> Option<&StreamEowcSort>
pub fn as_stream_over_window(&self) -> Option<&StreamOverWindow>
pub fn as_stream_fs_fetch(&self) -> Option<&StreamFsFetch>
pub fn as_stream_change_log(&self) -> Option<&StreamChangeLog>
pub fn as_stream_global_approx_percentile( &self, ) -> Option<&StreamGlobalApproxPercentile>
pub fn as_stream_local_approx_percentile( &self, ) -> Option<&StreamLocalApproxPercentile>
pub fn as_stream_row_merge(&self) -> Option<&StreamRowMerge>
pub fn as_stream_as_of_join(&self) -> Option<&StreamAsOfJoin>
Trait Implementations§
source§impl AnyPlanNodeMeta for PlanRef
impl AnyPlanNodeMeta for PlanRef
Implement again for the dyn
newtype wrapper.
fn node_type(&self) -> PlanNodeType
fn plan_base(&self) -> PlanBaseRef<'_>
fn convention(&self) -> Convention
source§impl BatchPlanRef for PlanRef
impl BatchPlanRef for PlanRef
Allow access to all fields defined in BatchPlanRef
for the type-erased plan node.
source§impl ColPrunable for PlanRef
impl ColPrunable for PlanRef
source§impl<V: Hash + Eq> Endo<PlanRef> for Merger<V>
impl<V: Hash + Eq> Endo<PlanRef> for Merger<V>
source§fn apply(&mut self, t: PlanRef) -> PlanRef
fn apply(&mut self, t: PlanRef) -> PlanRef
tree_apply
in the implementation.
But for more complicated requirements,
e.g. skipping over certain subtrees, custom logic can be added.fn pre(&mut self, t: T) -> T
fn post(&mut self, t: T) -> T
source§fn tree_apply(&mut self, t: T) -> T
fn tree_apply(&mut self, t: T) -> T
pre
and post
.source§impl Endo<PlanRef> for Pruner<'_>
impl Endo<PlanRef> for Pruner<'_>
fn pre(&mut self, t: PlanRef) -> PlanRef
source§fn apply(&mut self, t: PlanRef) -> PlanRef
fn apply(&mut self, t: PlanRef) -> PlanRef
tree_apply
in the implementation.
But for more complicated requirements,
e.g. skipping over certain subtrees, custom logic can be added.fn post(&mut self, t: T) -> T
source§fn tree_apply(&mut self, t: T) -> T
fn tree_apply(&mut self, t: T) -> T
pre
and post
.source§impl Explain for PlanRef
impl Explain for PlanRef
source§fn explain_with_id<'a>(&self) -> Pretty<'a>
fn explain_with_id<'a>(&self) -> Pretty<'a>
Write explain the whole plan tree with node id.
source§fn explain_to_string(&self) -> String
fn explain_to_string(&self) -> String
Explain the plan node and return a string.
source§fn explain_to_json(&self) -> String
fn explain_to_json(&self) -> String
Explain the plan node and return a json string.
source§fn explain_to_xml(&self) -> String
fn explain_to_xml(&self) -> String
Explain the plan node and return a xml string.
source§fn explain_to_yaml(&self) -> String
fn explain_to_yaml(&self) -> String
Explain the plan node and return a yaml string.
source§fn explain_to_dot(&self) -> String
fn explain_to_dot(&self) -> String
Explain the plan node and return a dot format string.
source§impl ExprRewritable for PlanRef
impl ExprRewritable for PlanRef
fn has_rewritable_expr(&self) -> bool
fn rewrite_exprs(&self, r: &mut dyn ExprRewriter) -> PlanRef
source§impl ExprVisitable for PlanRef
impl ExprVisitable for PlanRef
fn visit_exprs(&self, v: &mut dyn ExprVisitor)
source§impl GenericPlanRef for PlanRef
impl GenericPlanRef for PlanRef
Allow access to all fields defined in GenericPlanRef
for the type-erased plan node.
fn id(&self) -> PlanNodeId
fn schema(&self) -> &Schema
fn stream_key(&self) -> Option<&[usize]>
fn ctx(&self) -> OptimizerContextRef
fn functional_dependency(&self) -> &FunctionalDependencySet
source§impl Layer for PlanRef
impl Layer for PlanRef
source§impl LogicalCardinalityExt for PlanRef
impl LogicalCardinalityExt for PlanRef
source§impl PhysicalPlanRef for PlanRef
impl PhysicalPlanRef for PlanRef
Allow access to all fields defined in PhysicalPlanRef
for the type-erased plan node.
fn distribution(&self) -> &Distribution
source§impl PlanTreeNode for PlanRef
impl PlanTreeNode for PlanRef
source§impl PredicatePushdown for PlanRef
impl PredicatePushdown for PlanRef
source§fn predicate_pushdown(
&self,
predicate: Condition,
ctx: &mut PredicatePushdownContext,
) -> PlanRef
fn predicate_pushdown( &self, predicate: Condition, ctx: &mut PredicatePushdownContext, ) -> PlanRef
source§impl RewriteExprsRecursive for PlanRef
impl RewriteExprsRecursive for PlanRef
fn rewrite_exprs_recursive(&self, r: &mut impl ExprRewriter) -> PlanRef
source§impl StreamPlanRef for PlanRef
impl StreamPlanRef for PlanRef
Allow access to all fields defined in StreamPlanRef
for the type-erased plan node.
fn append_only(&self) -> bool
fn emit_on_window_close(&self) -> bool
fn watermark_columns(&self) -> &FixedBitSet
fn columns_monotonicity(&self) -> &MonotonicityMap
source§impl TryFrom<PlanRef> for ExecutionPlanNode
impl TryFrom<PlanRef> for ExecutionPlanNode
source§impl VisitExprsRecursive for PlanRef
impl VisitExprsRecursive for PlanRef
fn visit_exprs_recursive(&self, r: &mut impl ExprVisitor)
impl Eq for PlanRef
Auto Trait Implementations§
impl Freeze for PlanRef
impl !RefUnwindSafe for PlanRef
impl !Send for PlanRef
impl !Sync for PlanRef
impl Unpin for PlanRef
impl !UnwindSafe for PlanRef
Blanket Implementations§
source§impl<T> BorrowMut<T> for Twhere
T: ?Sized,
impl<T> BorrowMut<T> for Twhere
T: ?Sized,
source§fn borrow_mut(&mut self) -> &mut T
fn borrow_mut(&mut self) -> &mut T
source§impl<T> CloneToUninit for Twhere
T: Clone,
impl<T> CloneToUninit for Twhere
T: Clone,
source§unsafe fn clone_to_uninit(&self, dst: *mut T)
unsafe fn clone_to_uninit(&self, dst: *mut T)
clone_to_uninit
)§impl<T> Conv for T
impl<T> Conv for T
§impl<Choices> CoproductSubsetter<CNil, HNil> for Choices
impl<Choices> CoproductSubsetter<CNil, HNil> for Choices
§impl<T> Downcast for Twhere
T: Any,
impl<T> Downcast for Twhere
T: Any,
§fn into_any(self: Box<T>) -> Box<dyn Any>
fn into_any(self: Box<T>) -> Box<dyn Any>
Box<dyn Trait>
(where Trait: Downcast
) to Box<dyn Any>
. Box<dyn Any>
can
then be further downcast
into Box<ConcreteType>
where ConcreteType
implements Trait
.§fn into_any_rc(self: Rc<T>) -> Rc<dyn Any>
fn into_any_rc(self: Rc<T>) -> Rc<dyn Any>
Rc<Trait>
(where Trait: Downcast
) to Rc<Any>
. Rc<Any>
can then be
further downcast
into Rc<ConcreteType>
where ConcreteType
implements Trait
.§fn as_any(&self) -> &(dyn Any + 'static)
fn as_any(&self) -> &(dyn Any + 'static)
&Trait
(where Trait: Downcast
) to &Any
. This is needed since Rust cannot
generate &Any
’s vtable from &Trait
’s.§fn as_any_mut(&mut self) -> &mut (dyn Any + 'static)
fn as_any_mut(&mut self) -> &mut (dyn Any + 'static)
&mut Trait
(where Trait: Downcast
) to &Any
. This is needed since Rust cannot
generate &mut Any
’s vtable from &mut Trait
’s.§impl<Q, K> Equivalent<K> for Q
impl<Q, K> Equivalent<K> for Q
§fn equivalent(&self, key: &K) -> bool
fn equivalent(&self, key: &K) -> bool
§impl<Q, K> Equivalent<K> for Q
impl<Q, K> Equivalent<K> for Q
§fn equivalent(&self, key: &K) -> bool
fn equivalent(&self, key: &K) -> bool
key
and return true
if they are equal.source§impl<Q, K> Equivalent<K> for Q
impl<Q, K> Equivalent<K> for Q
source§fn equivalent(&self, key: &K) -> bool
fn equivalent(&self, key: &K) -> bool
key
and return true
if they are equal.§impl<Q, K> Equivalent<K> for Q
impl<Q, K> Equivalent<K> for Q
§fn equivalent(&self, key: &K) -> bool
fn equivalent(&self, key: &K) -> bool
§impl<Q, K> Equivalent<K> for Q
impl<Q, K> Equivalent<K> for Q
§fn equivalent(&self, key: &K) -> bool
fn equivalent(&self, key: &K) -> bool
§impl<T> FmtForward for T
impl<T> FmtForward for T
§fn fmt_binary(self) -> FmtBinary<Self>where
Self: Binary,
fn fmt_binary(self) -> FmtBinary<Self>where
Self: Binary,
self
to use its Binary
implementation when Debug
-formatted.§fn fmt_display(self) -> FmtDisplay<Self>where
Self: Display,
fn fmt_display(self) -> FmtDisplay<Self>where
Self: Display,
self
to use its Display
implementation when
Debug
-formatted.§fn fmt_lower_exp(self) -> FmtLowerExp<Self>where
Self: LowerExp,
fn fmt_lower_exp(self) -> FmtLowerExp<Self>where
Self: LowerExp,
self
to use its LowerExp
implementation when
Debug
-formatted.§fn fmt_lower_hex(self) -> FmtLowerHex<Self>where
Self: LowerHex,
fn fmt_lower_hex(self) -> FmtLowerHex<Self>where
Self: LowerHex,
self
to use its LowerHex
implementation when
Debug
-formatted.§fn fmt_octal(self) -> FmtOctal<Self>where
Self: Octal,
fn fmt_octal(self) -> FmtOctal<Self>where
Self: Octal,
self
to use its Octal
implementation when Debug
-formatted.§fn fmt_pointer(self) -> FmtPointer<Self>where
Self: Pointer,
fn fmt_pointer(self) -> FmtPointer<Self>where
Self: Pointer,
self
to use its Pointer
implementation when
Debug
-formatted.§fn fmt_upper_exp(self) -> FmtUpperExp<Self>where
Self: UpperExp,
fn fmt_upper_exp(self) -> FmtUpperExp<Self>where
Self: UpperExp,
self
to use its UpperExp
implementation when
Debug
-formatted.§fn fmt_upper_hex(self) -> FmtUpperHex<Self>where
Self: UpperHex,
fn fmt_upper_hex(self) -> FmtUpperHex<Self>where
Self: UpperHex,
self
to use its UpperHex
implementation when
Debug
-formatted.§fn fmt_list(self) -> FmtList<Self>where
&'a Self: for<'a> IntoIterator,
fn fmt_list(self) -> FmtList<Self>where
&'a Self: for<'a> IntoIterator,
§impl<T> FutureExt for T
impl<T> FutureExt for T
§fn with_context(self, otel_cx: Context) -> WithContext<Self>
fn with_context(self, otel_cx: Context) -> WithContext<Self>
§fn with_current_context(self) -> WithContext<Self>
fn with_current_context(self) -> WithContext<Self>
§impl<T> Instrument for T
impl<T> Instrument for T
§fn instrument(self, span: Span) -> Instrumented<Self>
fn instrument(self, span: Span) -> Instrumented<Self>
§fn in_current_span(self) -> Instrumented<Self>
fn in_current_span(self) -> Instrumented<Self>
source§impl<T> Instrument for T
impl<T> Instrument for T
source§fn instrument(self, span: Span) -> Instrumented<Self>
fn instrument(self, span: Span) -> Instrumented<Self>
source§fn in_current_span(self) -> Instrumented<Self>
fn in_current_span(self) -> Instrumented<Self>
source§impl<T> IntoEither for T
impl<T> IntoEither for T
source§fn into_either(self, into_left: bool) -> Either<Self, Self>
fn into_either(self, into_left: bool) -> Either<Self, Self>
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 moresource§fn into_either_with<F>(self, into_left: F) -> Either<Self, Self>
fn into_either_with<F>(self, into_left: F) -> Either<Self, Self>
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 moresource§impl<T> IntoRequest<T> for T
impl<T> IntoRequest<T> for T
source§fn into_request(self) -> Request<T>
fn into_request(self) -> Request<T>
T
in a tonic::Request
§impl<T> IntoResult<T> for T
impl<T> IntoResult<T> for T
type Err = Infallible
fn into_result(self) -> Result<T, <T as IntoResult<T>>::Err>
§impl<T, U, I> LiftInto<U, I> for Twhere
U: LiftFrom<T, I>,
impl<T, U, I> LiftInto<U, I> for Twhere
U: LiftFrom<T, I>,
source§impl<M> MetricVecRelabelExt for M
impl<M> MetricVecRelabelExt for M
source§fn relabel(
self,
metric_level: MetricLevel,
relabel_threshold: MetricLevel,
) -> RelabeledMetricVec<M>
fn relabel( self, metric_level: MetricLevel, relabel_threshold: MetricLevel, ) -> RelabeledMetricVec<M>
RelabeledMetricVec::with_metric_level
.source§fn relabel_n(
self,
metric_level: MetricLevel,
relabel_threshold: MetricLevel,
relabel_num: usize,
) -> RelabeledMetricVec<M>
fn relabel_n( self, metric_level: MetricLevel, relabel_threshold: MetricLevel, relabel_num: usize, ) -> RelabeledMetricVec<M>
RelabeledMetricVec::with_metric_level_relabel_n
.source§fn relabel_debug_1(
self,
relabel_threshold: MetricLevel,
) -> RelabeledMetricVec<M>
fn relabel_debug_1( self, relabel_threshold: MetricLevel, ) -> RelabeledMetricVec<M>
RelabeledMetricVec::with_metric_level_relabel_n
with metric_level
set to
MetricLevel::Debug
and relabel_num
set to 1.§impl<T> Pipe for Twhere
T: ?Sized,
impl<T> Pipe for Twhere
T: ?Sized,
§fn pipe<R>(self, func: impl FnOnce(Self) -> R) -> Rwhere
Self: Sized,
fn pipe<R>(self, func: impl FnOnce(Self) -> R) -> Rwhere
Self: Sized,
§fn pipe_ref<'a, R>(&'a self, func: impl FnOnce(&'a Self) -> R) -> Rwhere
R: 'a,
fn pipe_ref<'a, R>(&'a self, func: impl FnOnce(&'a Self) -> R) -> Rwhere
R: 'a,
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) -> Rwhere
R: 'a,
fn pipe_ref_mut<'a, R>(&'a mut self, func: impl FnOnce(&'a mut Self) -> R) -> Rwhere
R: 'a,
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
fn pipe_borrow<'a, B, R>(&'a self, func: impl FnOnce(&'a B) -> R) -> R
§fn pipe_borrow_mut<'a, B, R>(
&'a mut self,
func: impl FnOnce(&'a mut B) -> R,
) -> R
fn pipe_borrow_mut<'a, B, R>( &'a mut self, func: impl FnOnce(&'a mut B) -> R, ) -> R
§fn pipe_as_ref<'a, U, R>(&'a self, func: impl FnOnce(&'a U) -> R) -> R
fn pipe_as_ref<'a, U, R>(&'a self, func: impl FnOnce(&'a U) -> R) -> R
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
fn pipe_as_mut<'a, U, R>(&'a mut self, func: impl FnOnce(&'a mut U) -> R) -> R
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
fn pipe_deref<'a, T, R>(&'a self, func: impl FnOnce(&'a T) -> R) -> R
self
, then passes self.deref()
into the pipe function.§impl<T> Pointable for T
impl<T> Pointable for T
§impl<Source> Sculptor<HNil, HNil> for Source
impl<Source> Sculptor<HNil, HNil> for Source
§impl<T> Tap for T
impl<T> Tap for T
§fn tap_borrow<B>(self, func: impl FnOnce(&B)) -> Self
fn tap_borrow<B>(self, func: impl FnOnce(&B)) -> Self
Borrow<B>
of a value. Read more§fn tap_borrow_mut<B>(self, func: impl FnOnce(&mut B)) -> Self
fn tap_borrow_mut<B>(self, func: impl FnOnce(&mut B)) -> Self
BorrowMut<B>
of a value. Read more§fn tap_ref<R>(self, func: impl FnOnce(&R)) -> Self
fn tap_ref<R>(self, func: impl FnOnce(&R)) -> Self
AsRef<R>
view of a value. Read more§fn tap_ref_mut<R>(self, func: impl FnOnce(&mut R)) -> Self
fn tap_ref_mut<R>(self, func: impl FnOnce(&mut R)) -> Self
AsMut<R>
view of a value. Read more§fn tap_deref<T>(self, func: impl FnOnce(&T)) -> Self
fn tap_deref<T>(self, func: impl FnOnce(&T)) -> Self
Deref::Target
of a value. Read more§fn tap_deref_mut<T>(self, func: impl FnOnce(&mut T)) -> Self
fn tap_deref_mut<T>(self, func: impl FnOnce(&mut T)) -> Self
Deref::Target
of a value. Read more§fn tap_dbg(self, func: impl FnOnce(&Self)) -> Self
fn tap_dbg(self, func: impl FnOnce(&Self)) -> Self
.tap()
only in debug builds, and is erased in release builds.§fn tap_mut_dbg(self, func: impl FnOnce(&mut Self)) -> Self
fn tap_mut_dbg(self, func: impl FnOnce(&mut Self)) -> Self
.tap_mut()
only in debug builds, and is erased in release
builds.§fn tap_borrow_dbg<B>(self, func: impl FnOnce(&B)) -> Self
fn tap_borrow_dbg<B>(self, func: impl FnOnce(&B)) -> Self
.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
fn tap_borrow_mut_dbg<B>(self, func: impl FnOnce(&mut B)) -> Self
.tap_borrow_mut()
only in debug builds, and is erased in release
builds.§fn tap_ref_dbg<R>(self, func: impl FnOnce(&R)) -> Self
fn tap_ref_dbg<R>(self, func: impl FnOnce(&R)) -> Self
.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
fn tap_ref_mut_dbg<R>(self, func: impl FnOnce(&mut R)) -> Self
.tap_ref_mut()
only in debug builds, and is erased in release
builds.§fn tap_deref_dbg<T>(self, func: impl FnOnce(&T)) -> Self
fn tap_deref_dbg<T>(self, func: impl FnOnce(&T)) -> Self
.tap_deref()
only in debug builds, and is erased in release
builds.