risingwave_batch::executor::hash_join

Struct HashJoinExecutor

source
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:

  1. Iterate over the build side (i.e. right table) and build a hash map.
  2. 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.
  3. Concatenate the matched pair of probe side row and build side row into a single row and push it into the data chunk builder.
  4. 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>

source

fn do_execute( self: Box<Self>, ) -> Pin<Box<dyn Stream<Item = Result<DataChunk, BatchError>> + Send>>

source

pub fn do_inner_join( _: EquiJoinParams<K>, ) -> Pin<Box<dyn Stream<Item = Result<DataChunk, BatchError>> + Send>>

source

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>>

source

pub fn do_left_outer_join( _: EquiJoinParams<K>, ) -> Pin<Box<dyn Stream<Item = Result<DataChunk, BatchError>> + Send>>

source

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>>

source

pub fn do_left_semi_anti_join<const ANTI_JOIN: bool>( _: EquiJoinParams<K>, ) -> Pin<Box<dyn Stream<Item = Result<DataChunk, BatchError>> + Send>>

source

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:

  1. For each probe_row, append candidate rows to buffer. Candidate rows: Those satisfying equi_predicate (==).
  2. 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.
  3. If we matched probe_row in spilled chunk, stop appending its candidate rows, to avoid matching it again in next spilled chunk.
source

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>>

source

pub fn do_right_outer_join( _: EquiJoinParams<K>, ) -> Pin<Box<dyn Stream<Item = Result<DataChunk, BatchError>> + Send>>

source

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>>

source

pub fn do_right_semi_anti_join<const ANTI_JOIN: bool>( _: EquiJoinParams<K>, ) -> Pin<Box<dyn Stream<Item = Result<DataChunk, BatchError>> + Send>>

source

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>>

source

pub fn do_full_outer_join( _: EquiJoinParams<K>, ) -> Pin<Box<dyn Stream<Item = Result<DataChunk, BatchError>> + Send>>

source

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>>

source

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 from do_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 in chunk 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).

offsetv1v2v3
0121
1111
223-
3333
4333

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.

  1. Evaluate the non-equi condition on the chunk. Here the condition is t1.v2 <> t2.v3.

We get the result array:

offsetvalue
0true
1false
2false
3false
4false
  1. Set the build side columns to NULL if the corresponding result value is false.

The chunk is changed to:

offsetv1v2v3
0121
111-
223-
333-
433-
  1. Remove duplicate rows with NULL build side. This is done by setting the visibility bitmap of the chunk.
offsetv1v2v3
0121
111-
223-
333-
433-

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:

offsetv1v2v3
0333
1333
255-
353-
453-

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.

offsetv1v2v3
0333
133-
255-
353-
453-

After we add these chunks to output chunk builder in do_execute, we get the final output:

Chunk 1

offsetv1v2v3
0121
111-
223-
333-
455-

Chunk 2

offsetv1v2v3
053-

For more information about how process_*_join_non_equi_condition work, see their unit tests.

source

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.

source

async fn process_right_outer_join_non_equi_condition( chunk: DataChunk, cond: &dyn Expression, __arg2: &mut RightNonEquiJoinState, ) -> Result<DataChunk>

source

async fn process_right_semi_anti_join_non_equi_condition( chunk: DataChunk, cond: &dyn Expression, __arg2: &mut RightNonEquiJoinState, ) -> Result<()>

source

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>

source

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

source

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

source

fn append_one_row( chunk_builder: &mut DataChunkBuilder, probe_chunk: &DataChunk, probe_row_id: usize, build_chunk: &DataChunk, build_row_id: usize, ) -> Option<DataChunk>

source

fn append_one_probe_row( chunk_builder: &mut DataChunkBuilder, probe_chunk: &DataChunk, probe_row_id: usize, ) -> Option<DataChunk>

source

fn append_one_build_row( chunk_builder: &mut DataChunkBuilder, build_chunk: &DataChunk, build_row_id: usize, ) -> Option<DataChunk>

source

fn append_one_row_with_null_build_side( chunk_builder: &mut DataChunkBuilder, probe_row_ref: RowRef<'_>, build_column_count: usize, ) -> Option<DataChunk>

source

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>

source

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

source

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<()>

source§

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>

source§

fn schema(&self) -> &Schema

Returns the schema of the executor’s return data. Read more
source§

fn identity(&self) -> &str

Identity string of the executor
source§

fn execute(self: Box<Self>) -> BoxedDataChunkStream

Executes to return the data chunk stream. Read more

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> 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> Downcast for T
where T: Any,

§

fn into_any(self: Box<T>) -> Box<dyn Any>

Convert 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>

Convert 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)

Convert &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)

Convert &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

§

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
§

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

§

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