pub struct HashJoinExecutor<K> {Show 18 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,
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: JoinType
Join type e.g. inner, left outer, …
original_schema: Schema
Output schema without applying output_indices
schema: Schema
Output schema after applying output_indices
output_indices: Vec<usize>
output_indices
are the indices of the columns that we needed.
probe_side_source: BoxedExecutor
Left child executor
build_side_source: BoxedExecutor
Right 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
§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 satisfying equi_predicate (==).
- If buffer becomes full, process it.
Apply non_equi_join predicates e.g.
>=
,<=
to filter rows. Track if probe_row is matched to avoid duplicates. - If we matched probe_row in 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
NULL
build 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 inchunk
for 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>
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, 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, 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<()>
fn new_boxed_executor<'life0, 'life1, 'async_trait, C>(
context: &'life0 ExecutorBuilder<'life1, C>,
inputs: Vec<BoxedExecutor>,
) -> Pin<Box<dyn Future<Output = Result<BoxedExecutor>> + Send + 'async_trait>>where
C: 'async_trait + BatchTaskContext,
'life0: 'async_trait,
'life1: 'async_trait,
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<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<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.