pub struct HashJoinExecutor<K> {Show 19 fields
join_type: JoinType,
original_schema: Schema,
schema: Schema,
output_indices: Vec<usize>,
probe_side_source: BoxedExecutor,
build_side_source: BoxedExecutor,
probe_key_idxs: Vec<usize>,
build_key_idxs: Vec<usize>,
cond: Option<Arc<BoxedExpression>>,
null_matched: Vec<bool>,
identity: String,
chunk_size: usize,
asof_desc: Option<AsOfDesc>,
spill_backend: Option<SpillBackend>,
spill_metrics: Arc<BatchSpillMetrics>,
memory_upper_bound: Option<u64>,
shutdown_rx: ShutdownToken,
mem_ctx: MemoryContext,
_phantom: PhantomData<K>,
}Expand description
Hash Join Executor
High-level idea:
- Iterate over the build side (i.e. right table) and build a hash map.
- Iterate over the probe side (i.e. left table) and compute the hash value of each row. Then find the matched build side row for each probe side row in the hash map.
- Concatenate the matched pair of probe side row and build side row into a single row and push it into the data chunk builder.
- Yield chunks from the builder.
Fields§
§join_type: JoinTypeJoin type e.g. inner, left outer, …
original_schema: SchemaOutput schema without applying output_indices
schema: SchemaOutput schema after applying output_indices
output_indices: Vec<usize>output_indices are the indices of the columns that we needed.
probe_side_source: BoxedExecutorLeft child executor
build_side_source: BoxedExecutorRight child executor
probe_key_idxs: Vec<usize>Column indices of left keys in equi join
build_key_idxs: Vec<usize>Column indices of right keys in equi join
cond: Option<Arc<BoxedExpression>>Non-equi join condition (optional)
null_matched: Vec<bool>Whether or not to enable ‘IS NOT DISTINCT FROM’ semantics for a specific probe/build key column
identity: String§chunk_size: usize§asof_desc: Option<AsOfDesc>Whether the join is an as-of join
spill_backend: Option<SpillBackend>§spill_metrics: Arc<BatchSpillMetrics>§memory_upper_bound: Option<u64>The upper bound of memory usage for this executor.
shutdown_rx: ShutdownToken§mem_ctx: MemoryContext§_phantom: PhantomData<K>Implementations§
Source§impl<K: HashKey> HashJoinExecutor<K>
impl<K: HashKey> HashJoinExecutor<K>
fn do_execute( self: Box<Self>, ) -> Pin<Box<dyn Stream<Item = Result<DataChunk, BatchError>> + Send>>
pub fn do_inner_join( _: EquiJoinParams<K>, ) -> Pin<Box<dyn Stream<Item = Result<DataChunk, BatchError>> + Send>>
pub fn do_inner_join_with_non_equi_condition<'_async0>( params: EquiJoinParams<K>, cond: &'_async0 BoxedExpression, ) -> Pin<Box<dyn Stream<Item = Result<DataChunk, BatchError>> + Send + '_async0>>
pub fn do_left_outer_join( _: EquiJoinParams<K>, ) -> Pin<Box<dyn Stream<Item = Result<DataChunk, BatchError>> + Send>>
pub fn do_left_outer_join_with_non_equi_condition<'_async0>( _: EquiJoinParams<K>, cond: &'_async0 BoxedExpression, ) -> Pin<Box<dyn Stream<Item = Result<DataChunk, BatchError>> + Send + '_async0>>
pub fn do_left_semi_anti_join<const ANTI_JOIN: bool>( _: EquiJoinParams<K>, ) -> Pin<Box<dyn Stream<Item = Result<DataChunk, BatchError>> + Send>>
Sourcepub fn do_left_semi_join_with_non_equi_condition<'a>(
_: EquiJoinParams<K>,
cond: &'a BoxedExpression,
) -> Pin<Box<dyn Stream<Item = Result<DataChunk, BatchError>> + Send + 'a>>
pub fn do_left_semi_join_with_non_equi_condition<'a>( _: EquiJoinParams<K>, cond: &'a BoxedExpression, ) -> Pin<Box<dyn Stream<Item = Result<DataChunk, BatchError>> + Send + 'a>>
High-level idea:
- For each
probe_row, append candidate rows to buffer. Candidate rows: Those satisfyingequi_predicate(==). - If buffer becomes full, process it.
Apply
non_equi_joinpredicates e.g.>=,<=to filter rows. Track ifprobe_rowis matched to avoid duplicates. - If we matched
probe_rowin spilled chunk, stop appending its candidate rows, to avoid matching it again in next spilled chunk.
pub fn do_left_anti_join_with_non_equi_condition<'_async0>( _: EquiJoinParams<K>, cond: &'_async0 BoxedExpression, ) -> Pin<Box<dyn Stream<Item = Result<DataChunk, BatchError>> + Send + '_async0>>
pub fn do_right_outer_join( _: EquiJoinParams<K>, ) -> Pin<Box<dyn Stream<Item = Result<DataChunk, BatchError>> + Send>>
pub fn do_right_outer_join_with_non_equi_condition<'_async0>( _: EquiJoinParams<K>, cond: &'_async0 BoxedExpression, ) -> Pin<Box<dyn Stream<Item = Result<DataChunk, BatchError>> + Send + '_async0>>
pub fn do_right_semi_anti_join<const ANTI_JOIN: bool>( _: EquiJoinParams<K>, ) -> Pin<Box<dyn Stream<Item = Result<DataChunk, BatchError>> + Send>>
pub fn do_right_semi_anti_join_with_non_equi_condition<'_async0, const ANTI_JOIN: bool>( _: EquiJoinParams<K>, cond: &'_async0 BoxedExpression, ) -> Pin<Box<dyn Stream<Item = Result<DataChunk, BatchError>> + Send + '_async0>>
pub fn do_full_outer_join( _: EquiJoinParams<K>, ) -> Pin<Box<dyn Stream<Item = Result<DataChunk, BatchError>> + Send>>
pub fn do_full_outer_join_with_non_equi_condition<'_async0>( _: EquiJoinParams<K>, cond: &'_async0 BoxedExpression, ) -> Pin<Box<dyn Stream<Item = Result<DataChunk, BatchError>> + Send + '_async0>>
Sourceasync fn process_left_outer_join_non_equi_condition(
chunk: DataChunk,
cond: &dyn Expression,
__arg2: &mut LeftNonEquiJoinState,
) -> Result<DataChunk>
async fn process_left_outer_join_non_equi_condition( chunk: DataChunk, cond: &dyn Expression, __arg2: &mut LeftNonEquiJoinState, ) -> Result<DataChunk>
Process output chunk for left outer join when non-equi condition is presented.
§Arguments
chunk- Output chunk fromdo_left_outer_join_with_non_equi_condition, containing:- Concatenation of probe row and its corresponding build row according to the hash map.
- Concatenation of probe row and
NULLbuild row, if there is no matched build row found for the probe row.
cond- Non-equi join condition.probe_column_count- The number of columns in the probe side.first_output_row_id- The offset of the first output row inchunkfor each probe side row that has been processed.has_more_output_rows- Whether the probe row being processed currently has output rows in next output chunk.found_matched- Whether the probe row being processed currently has matched non-NULL build rows in last output chunk.
§Examples
Assume we have two tables t1 and t2 as probe side and build side, respectively.
CREATE TABLE t1 (v1 int, v2 int);
CREATE TABLE t2 (v3 int);Now we de left outer join on t1 and t2, as the following query shows:
SELECT * FROM t1 LEFT JOIN t2 ON t1.v1 = t2.v3 AND t1.v2 <> t2.v3;Assume the chunk builder in do_left_outer_join_with_non_equi_condition has buffer size 5,
and we have the following chunk as the first output (‘-’ represents NULL).
| offset | v1 | v2 | v3 |
|---|---|---|---|
| 0 | 1 | 2 | 1 |
| 1 | 1 | 1 | 1 |
| 2 | 2 | 3 | - |
| 3 | 3 | 3 | 3 |
| 4 | 3 | 3 | 3 |
We have the following precondition:
assert_eq!(probe_column_count, 2);
assert_eq!(first_out_row_id, vec![0, 1, 2, 3]);
assert_eq!(has_more_output_rows);
assert_eq!(!found_matched);In process_left_outer_join_non_equi_condition, we transform the chunk in following steps.
- Evaluate the non-equi condition on the chunk. Here the condition is
t1.v2 <> t2.v3.
We get the result array:
| offset | value |
|---|---|
| 0 | true |
| 1 | false |
| 2 | false |
| 3 | false |
| 4 | false |
- Set the build side columns to NULL if the corresponding result value is false.
The chunk is changed to:
| offset | v1 | v2 | v3 |
|---|---|---|---|
| 0 | 1 | 2 | 1 |
| 1 | 1 | 1 | - |
| 2 | 2 | 3 | - |
| 3 | 3 | 3 | - |
| 4 | 3 | 3 | - |
- Remove duplicate rows with NULL build side. This is done by setting the visibility bitmap of the chunk.
| offset | v1 | v2 | v3 |
|---|---|---|---|
| 0 | 1 | 2 | 1 |
| 1 | 1 | 1 | - |
| 2 | 2 | 3 | - |
| 3 | |||
| 4 |
For the probe row being processed currently ((3, 3) here), we don’t have output rows with
non-NULL build side, so we set found_matched to false.
In do_left_outer_join_with_non_equi_condition, we have next output chunk as follows:
| offset | v1 | v2 | v3 |
|---|---|---|---|
| 0 | 3 | 3 | 3 |
| 1 | 3 | 3 | 3 |
| 2 | 5 | 5 | - |
| 3 | 5 | 3 | - |
| 4 | 5 | 3 | - |
This time We have the following precondition:
assert_eq!(probe_column_count, 2);
assert_eq!(first_out_row_id, vec![2, 3]);
assert_eq!(!has_more_output_rows);
assert_eq!(!found_matched);The transformed chunk is as follows after the same steps.
| offset | v1 | v2 | v3 |
|---|---|---|---|
| 0 | |||
| 1 | 3 | 3 | - |
| 2 | 5 | 5 | - |
| 3 | 5 | 3 | - |
| 4 |
After we add these chunks to output chunk builder in do_execute, we get the final output:
Chunk 1
| offset | v1 | v2 | v3 |
|---|---|---|---|
| 0 | 1 | 2 | 1 |
| 1 | 1 | 1 | - |
| 2 | 2 | 3 | - |
| 3 | 3 | 3 | - |
| 4 | 5 | 5 | - |
Chunk 2
| offset | v1 | v2 | v3 |
|---|---|---|---|
| 0 | 5 | 3 | - |
For more information about how process_*_join_non_equi_condition work, see their unit
tests.
Sourceasync fn process_left_semi_anti_join_non_equi_condition<const ANTI_JOIN: bool>(
chunk: DataChunk,
cond: &dyn Expression,
__arg2: &mut LeftNonEquiJoinState,
) -> Result<DataChunk>
async fn process_left_semi_anti_join_non_equi_condition<const ANTI_JOIN: bool>( chunk: DataChunk, cond: &dyn Expression, __arg2: &mut LeftNonEquiJoinState, ) -> Result<DataChunk>
Filters for candidate rows which satisfy non_equi predicate.
Removes duplicate rows.
async fn process_right_outer_join_non_equi_condition( chunk: DataChunk, cond: &dyn Expression, __arg2: &mut RightNonEquiJoinState, ) -> Result<DataChunk>
async fn process_right_semi_anti_join_non_equi_condition( chunk: DataChunk, cond: &dyn Expression, __arg2: &mut RightNonEquiJoinState, ) -> Result<()>
async fn process_full_outer_join_non_equi_condition( chunk: DataChunk, cond: &dyn Expression, left_non_equi_state: &mut LeftNonEquiJoinState, right_non_equi_state: &mut RightNonEquiJoinState, ) -> Result<DataChunk>
fn handle_remaining_build_rows_for_right_outer_join<'a>( chunk_builder: &'a mut DataChunkBuilder, build_side: &'a [DataChunk], build_row_matched: &'a ChunkedData<bool>, probe_column_count: usize, ) -> impl Stream<Item = Result<DataChunk, BatchError>> + 'a
fn handle_remaining_build_rows_for_right_semi_anti_join<'a, const ANTI_JOIN: bool>( chunk_builder: &'a mut DataChunkBuilder, build_side: &'a [DataChunk], build_row_matched: &'a ChunkedData<bool>, ) -> impl Stream<Item = Result<DataChunk, BatchError>> + 'a
fn append_one_row( chunk_builder: &mut DataChunkBuilder, probe_chunk: &DataChunk, probe_row_id: usize, build_chunk: &DataChunk, build_row_id: usize, ) -> Option<DataChunk>
fn append_one_probe_row( chunk_builder: &mut DataChunkBuilder, probe_chunk: &DataChunk, probe_row_id: usize, ) -> Option<DataChunk>
fn append_one_build_row( chunk_builder: &mut DataChunkBuilder, build_chunk: &DataChunk, build_row_id: usize, ) -> Option<DataChunk>
fn append_one_row_with_null_build_side( chunk_builder: &mut DataChunkBuilder, probe_row_ref: RowRef<'_>, build_column_count: usize, ) -> Option<DataChunk>
fn append_one_row_with_null_probe_side( chunk_builder: &mut DataChunkBuilder, build_row_ref: RowRef<'_>, probe_column_count: usize, ) -> Option<DataChunk>
fn find_asof_matched_rows( probe_row_ref: RowRef<'_>, build_side: &[DataChunk], build_side_row_iter: RowIdIter<'_>, asof_join_condition: &AsOfDesc, ) -> Option<RowId>
Source§impl<K> HashJoinExecutor<K>
impl<K> HashJoinExecutor<K>
pub fn new( join_type: JoinType, output_indices: Vec<usize>, probe_side_source: BoxedExecutor, build_side_source: BoxedExecutor, probe_key_idxs: Vec<usize>, build_key_idxs: Vec<usize>, null_matched: Vec<bool>, cond: Option<Arc<BoxedExpression>>, identity: String, chunk_size: usize, asof_desc: Option<AsOfDesc>, spill_backend: Option<SpillBackend>, spill_metrics: Arc<BatchSpillMetrics>, shutdown_rx: ShutdownToken, mem_ctx: MemoryContext, ) -> Self
fn new_inner( join_type: JoinType, output_indices: Vec<usize>, probe_side_source: BoxedExecutor, build_side_source: BoxedExecutor, probe_key_idxs: Vec<usize>, build_key_idxs: Vec<usize>, null_matched: Vec<bool>, cond: Option<Arc<BoxedExpression>>, identity: String, chunk_size: usize, asof_desc: Option<AsOfDesc>, spill_backend: Option<SpillBackend>, spill_metrics: Arc<BatchSpillMetrics>, memory_upper_bound: Option<u64>, shutdown_rx: ShutdownToken, mem_ctx: MemoryContext, ) -> Self
Trait Implementations§
Source§impl BoxedExecutorBuilder for HashJoinExecutor<()>
impl BoxedExecutorBuilder for HashJoinExecutor<()>
async fn new_boxed_executor( context: &ExecutorBuilder<'_>, inputs: Vec<BoxedExecutor>, ) -> Result<BoxedExecutor>
Source§impl<K: HashKey> Executor for HashJoinExecutor<K>
impl<K: HashKey> Executor for HashJoinExecutor<K>
Auto Trait Implementations§
impl<K> Freeze for HashJoinExecutor<K>
impl<K> !RefUnwindSafe for HashJoinExecutor<K>
impl<K> Send for HashJoinExecutor<K>where
K: Send,
impl<K> !Sync for HashJoinExecutor<K>
impl<K> Unpin for HashJoinExecutor<K>where
K: Unpin,
impl<K> !UnwindSafe for HashJoinExecutor<K>
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
§impl<T> Conv for T
impl<T> Conv for T
§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>, which can then be
downcast into Box<dyn 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>, which 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<T> DowncastSend for T
impl<T> DowncastSend for T
§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<M> MetricVecRelabelExt for M
impl<M> MetricVecRelabelExt for M
§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].§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].§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<T> Scope for T
impl<T> Scope for T
§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.