pub(crate) struct NestedLoopJoinStream {Show 19 fields
pub(crate) output_schema: Arc<Schema>,
pub(crate) join_filter: Option<JoinFilter>,
pub(crate) join_type: JoinType,
pub(crate) right_data: SendableRecordBatchStream,
pub(crate) left_data: OnceFut<JoinLeftData>,
pub(crate) column_indices: Vec<ColumnIndex>,
pub(crate) metrics: NestedLoopJoinMetrics,
batch_size: usize,
should_track_unmatched_right: bool,
state: NLJState,
output_buffer: Box<BatchCoalescer>,
handled_empty_output: bool,
buffered_left_data: Option<Arc<JoinLeftData>>,
left_probe_idx: usize,
left_emit_idx: usize,
left_exhausted: bool,
left_buffered_in_one_pass: bool,
current_right_batch: Option<RecordBatch>,
current_right_batch_matched: Option<BooleanArray>,
}Fields§
§output_schema: Arc<Schema>Output schema
join_filter: Option<JoinFilter>join filter
join_type: JoinTypetype of the join
right_data: SendableRecordBatchStreamthe probe-side(right) table data of the nested loop join
left_data: OnceFut<JoinLeftData>the build-side table data of the nested loop join
column_indices: Vec<ColumnIndex>Projection to construct the output schema from the left and right tables. Example:
- output_schema: [‘a’, ‘c’]
- left_schema: [‘a’, ‘b’]
- right_schema: [‘c’]
The column indices would be [(left, 0), (right, 0)] – taking the left 0th column and right 0th column can construct the output schema.
Note there are other columns (‘b’ in the example) still kept after
projection pushdown; this is because they might be used to evaluate
the join filter (e.g., JOIN ON (b+c)>0).
metrics: NestedLoopJoinMetricsJoin execution metrics
batch_size: usizebatch_size from configuration
should_track_unmatched_right: boolSee comments in need_produce_right_in_final for more detail
state: NLJStateState Tracking
output_buffer: Box<BatchCoalescer>Output buffer holds the join result to output. It will emit eagerly when the threshold is reached.
handled_empty_output: boolSee comments in NLJState::Done for its purpose
buffered_left_data: Option<Arc<JoinLeftData>>The current buffered left data to join
left_probe_idx: usizeIndex into the left buffered batch. Used in ProbeRight state
left_emit_idx: usizeIndex into the left buffered batch. Used in EmitLeftUnmatched state
left_exhausted: boolShould we go back to BufferingLeft state again after EmitLeftUnmatched
state is over.
left_buffered_in_one_pass: boolIf we can buffer all left data in one pass TODO(now): this is for the (unimplemented) memory-limited execution
current_right_batch: Option<RecordBatch>The current probe batch to process
current_right_batch_matched: Option<BooleanArray>Implementations§
Source§impl NestedLoopJoinStream
impl NestedLoopJoinStream
pub(crate) fn new( schema: Arc<Schema>, filter: Option<JoinFilter>, join_type: JoinType, right_data: SendableRecordBatchStream, left_data: OnceFut<JoinLeftData>, column_indices: Vec<ColumnIndex>, metrics: NestedLoopJoinMetrics, batch_size: usize, ) -> Self
Sourcefn handle_buffering_left(
&mut self,
cx: &mut Context<'_>,
) -> ControlFlow<Poll<Option<Result<RecordBatch>>>>
fn handle_buffering_left( &mut self, cx: &mut Context<'_>, ) -> ControlFlow<Poll<Option<Result<RecordBatch>>>>
Handle BufferingLeft state - prepare left side batches
Sourcefn handle_fetching_right(
&mut self,
cx: &mut Context<'_>,
) -> ControlFlow<Poll<Option<Result<RecordBatch>>>>
fn handle_fetching_right( &mut self, cx: &mut Context<'_>, ) -> ControlFlow<Poll<Option<Result<RecordBatch>>>>
Handle FetchingRight state - fetch next right batch and prepare for processing
Sourcefn handle_probe_right(
&mut self,
) -> ControlFlow<Poll<Option<Result<RecordBatch>>>>
fn handle_probe_right( &mut self, ) -> ControlFlow<Poll<Option<Result<RecordBatch>>>>
Handle ProbeRight state - process current probe batch
Sourcefn handle_emit_right_unmatched(
&mut self,
) -> ControlFlow<Poll<Option<Result<RecordBatch>>>>
fn handle_emit_right_unmatched( &mut self, ) -> ControlFlow<Poll<Option<Result<RecordBatch>>>>
Handle EmitRightUnmatched state - emit unmatched right rows
Sourcefn handle_emit_left_unmatched(
&mut self,
) -> ControlFlow<Poll<Option<Result<RecordBatch>>>>
fn handle_emit_left_unmatched( &mut self, ) -> ControlFlow<Poll<Option<Result<RecordBatch>>>>
Handle EmitLeftUnmatched state - emit unmatched left rows
Sourcefn handle_done(&mut self) -> Poll<Option<Result<RecordBatch>>>
fn handle_done(&mut self) -> Poll<Option<Result<RecordBatch>>>
Handle Done state - final state processing
Sourcefn process_probe_batch(&mut self) -> Result<bool>
fn process_probe_batch(&mut self) -> Result<bool>
Returns bool to indicate should it continue probing true -> continue in the same ProbeRight state false -> It has done with the (buffered_left x cur_right_batch), go to next state (ProbeRight)
Sourcefn process_single_left_row_join(
&mut self,
left_data: &JoinLeftData,
right_batch: &RecordBatch,
l_index: usize,
) -> Result<Option<RecordBatch>>
fn process_single_left_row_join( &mut self, left_data: &JoinLeftData, right_batch: &RecordBatch, l_index: usize, ) -> Result<Option<RecordBatch>>
Process a single left row join with the current right batch. Returns a RecordBatch containing the join results (None if empty)
Sourcefn process_left_unmatched(&mut self) -> Result<bool>
fn process_left_unmatched(&mut self) -> Result<bool>
Returns bool to indicate should it continue processing unmatched rows true -> continue in the same EmitLeftUnmatched state false -> next state (Done)
Sourcefn process_left_unmatched_range(
&self,
left_data: &JoinLeftData,
start_idx: usize,
end_idx: usize,
) -> Result<Option<RecordBatch>>
fn process_left_unmatched_range( &self, left_data: &JoinLeftData, start_idx: usize, end_idx: usize, ) -> Result<Option<RecordBatch>>
Process unmatched rows from the left data within the specified range. Returns a RecordBatch containing the unmatched rows (None if empty).
§Arguments
left_data- The left side data containing the batch and bitmapstart_idx- Start index (inclusive) of the range to processend_idx- End index (exclusive) of the range to process
§Safety
The caller is responsible for ensuring that start_idx and end_idx are
within valid bounds of the left batch. This function does not perform
bounds checking.
Sourcefn process_right_unmatched(&mut self) -> Result<Option<RecordBatch>>
fn process_right_unmatched(&mut self) -> Result<Option<RecordBatch>>
Process unmatched rows from the current right batch and reset the bitmap. Returns a RecordBatch containing the unmatched right rows (None if empty).
Sourcefn get_left_data(&self) -> Result<&Arc<JoinLeftData>>
fn get_left_data(&self) -> Result<&Arc<JoinLeftData>>
Get the build-side data of the left input, errors if it’s None
Sourcefn maybe_flush_ready_batch(
&mut self,
) -> Option<Poll<Option<Result<RecordBatch>>>>
fn maybe_flush_ready_batch( &mut self, ) -> Option<Poll<Option<Result<RecordBatch>>>>
Flush the output_buffer if there are batches ready to output
None if no result batch ready.
Sourcefn update_matched_bitmap(
&mut self,
l_index: usize,
r_matched_bitmap: &BooleanArray,
) -> Result<()>
fn update_matched_bitmap( &mut self, l_index: usize, r_matched_bitmap: &BooleanArray, ) -> Result<()>
After joining (l_index@left_buffer x current_right_batch), it will result in a bitmap (the same length as current_right_batch) as the join match result. Use this bitmap to update the global bitmap, for special join types like full joins.
Example: After joining l_index=1 (1-indexed row in the left buffer), and the current right batch with 3 elements, this function will be called with arguments: l_index = 1, r_matched = [false, false, true]
- If the join type is FullJoin, the 1-index in the left bitmap will be set to true, and also the right bitmap will be bitwise-ORed with the input r_matched bitmap.
- For join types that don’t require output unmatched rows, this function can be a no-op. For inner joins, this function is a no-op; for left joins, only the left bitmap may be updated.
Trait Implementations§
Source§impl Stream for NestedLoopJoinStream
impl Stream for NestedLoopJoinStream
Source§fn poll_next(
self: Pin<&mut Self>,
cx: &mut Context<'_>,
) -> Poll<Option<Self::Item>>
fn poll_next( self: Pin<&mut Self>, cx: &mut Context<'_>, ) -> Poll<Option<Self::Item>>
See the comments NestedLoopJoinExec for high-level design ideas.
§Implementation
This function is the entry point of NLJ operator’s state machine transitions. The rough state transition graph is as follow, for more details see the comment in each state’s matching arm.
§============================ State transition graph:
§(start) –> BufferingLeft
BufferingLeft → FetchingRight
FetchingRight → ProbeRight (if right batch available) FetchingRight → EmitLeftUnmatched (if right exhausted)
ProbeRight → ProbeRight (next left row or after yielding output) ProbeRight → EmitRightUnmatched (for special join types like right join) ProbeRight → FetchingRight (done with the current right batch)
EmitRightUnmatched → FetchingRight
§EmitLeftUnmatched → EmitLeftUnmatched (only process 1 chunk for each iteration) EmitLeftUnmatched → Done (if finished)
Done → (end)
Source§type Item = Result<RecordBatch, DataFusionError>
type Item = Result<RecordBatch, DataFusionError>
Auto Trait Implementations§
impl Freeze for NestedLoopJoinStream
impl !RefUnwindSafe for NestedLoopJoinStream
impl Send for NestedLoopJoinStream
impl !Sync for NestedLoopJoinStream
impl Unpin for NestedLoopJoinStream
impl !UnwindSafe for NestedLoopJoinStream
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> 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> 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 more§impl<T> PolicyExt for Twhere
T: ?Sized,
impl<T> PolicyExt for Twhere
T: ?Sized,
§impl<T> StreamExt for Twhere
T: Stream + ?Sized,
impl<T> StreamExt for Twhere
T: Stream + ?Sized,
§fn next(&mut self) -> Next<'_, Self>where
Self: Unpin,
fn next(&mut self) -> Next<'_, Self>where
Self: Unpin,
§fn into_future(self) -> StreamFuture<Self>
fn into_future(self) -> StreamFuture<Self>
§fn map<T, F>(self, f: F) -> Map<Self, F>
fn map<T, F>(self, f: F) -> Map<Self, F>
§fn enumerate(self) -> Enumerate<Self>where
Self: Sized,
fn enumerate(self) -> Enumerate<Self>where
Self: Sized,
§fn filter<Fut, F>(self, f: F) -> Filter<Self, Fut, F>
fn filter<Fut, F>(self, f: F) -> Filter<Self, Fut, F>
§fn filter_map<Fut, T, F>(self, f: F) -> FilterMap<Self, Fut, F>
fn filter_map<Fut, T, F>(self, f: F) -> FilterMap<Self, Fut, F>
§fn then<Fut, F>(self, f: F) -> Then<Self, Fut, F>
fn then<Fut, F>(self, f: F) -> Then<Self, Fut, F>
§fn collect<C>(self) -> Collect<Self, C>
fn collect<C>(self) -> Collect<Self, C>
§fn unzip<A, B, FromA, FromB>(self) -> Unzip<Self, FromA, FromB>
fn unzip<A, B, FromA, FromB>(self) -> Unzip<Self, FromA, FromB>
§fn concat(self) -> Concat<Self>
fn concat(self) -> Concat<Self>
§fn count(self) -> Count<Self>where
Self: Sized,
fn count(self) -> Count<Self>where
Self: Sized,
§fn fold<T, Fut, F>(self, init: T, f: F) -> Fold<Self, Fut, T, F>
fn fold<T, Fut, F>(self, init: T, f: F) -> Fold<Self, Fut, T, F>
§fn any<Fut, F>(self, f: F) -> Any<Self, Fut, F>
fn any<Fut, F>(self, f: F) -> Any<Self, Fut, F>
true if any element in stream satisfied a predicate. Read more§fn all<Fut, F>(self, f: F) -> All<Self, Fut, F>
fn all<Fut, F>(self, f: F) -> All<Self, Fut, F>
true if all element in stream satisfied a predicate. Read more§fn flatten(self) -> Flatten<Self>where
Self::Item: Stream,
Self: Sized,
fn flatten(self) -> Flatten<Self>where
Self::Item: Stream,
Self: Sized,
§fn flatten_unordered(
self,
limit: impl Into<Option<usize>>,
) -> FlattenUnorderedWithFlowController<Self, ()>
fn flatten_unordered( self, limit: impl Into<Option<usize>>, ) -> FlattenUnorderedWithFlowController<Self, ()>
§fn flat_map_unordered<U, F>(
self,
limit: impl Into<Option<usize>>,
f: F,
) -> FlatMapUnordered<Self, U, F>
fn flat_map_unordered<U, F>( self, limit: impl Into<Option<usize>>, f: F, ) -> FlatMapUnordered<Self, U, F>
StreamExt::map] but flattens nested Streams
and polls them concurrently, yielding items in any order, as they made
available. Read more§fn scan<S, B, Fut, F>(self, initial_state: S, f: F) -> Scan<Self, S, Fut, F>
fn scan<S, B, Fut, F>(self, initial_state: S, f: F) -> Scan<Self, S, Fut, F>
StreamExt::fold] that holds internal state
and produces a new stream. Read more§fn skip_while<Fut, F>(self, f: F) -> SkipWhile<Self, Fut, F>
fn skip_while<Fut, F>(self, f: F) -> SkipWhile<Self, Fut, F>
true. Read more§fn take_while<Fut, F>(self, f: F) -> TakeWhile<Self, Fut, F>
fn take_while<Fut, F>(self, f: F) -> TakeWhile<Self, Fut, F>
true. Read more§fn take_until<Fut>(self, fut: Fut) -> TakeUntil<Self, Fut>
fn take_until<Fut>(self, fut: Fut) -> TakeUntil<Self, Fut>
§fn for_each<Fut, F>(self, f: F) -> ForEach<Self, Fut, F>
fn for_each<Fut, F>(self, f: F) -> ForEach<Self, Fut, F>
§fn for_each_concurrent<Fut, F>(
self,
limit: impl Into<Option<usize>>,
f: F,
) -> ForEachConcurrent<Self, Fut, F>
fn for_each_concurrent<Fut, F>( self, limit: impl Into<Option<usize>>, f: F, ) -> ForEachConcurrent<Self, Fut, F>
§fn take(self, n: usize) -> Take<Self>where
Self: Sized,
fn take(self, n: usize) -> Take<Self>where
Self: Sized,
n items of the underlying stream. Read more§fn skip(self, n: usize) -> Skip<Self>where
Self: Sized,
fn skip(self, n: usize) -> Skip<Self>where
Self: Sized,
n items of the underlying stream. Read more§fn catch_unwind(self) -> CatchUnwind<Self>where
Self: Sized + UnwindSafe,
fn catch_unwind(self) -> CatchUnwind<Self>where
Self: Sized + UnwindSafe,
§fn boxed<'a>(self) -> Pin<Box<dyn Stream<Item = Self::Item> + Send + 'a>>
fn boxed<'a>(self) -> Pin<Box<dyn Stream<Item = Self::Item> + Send + 'a>>
§fn boxed_local<'a>(self) -> Pin<Box<dyn Stream<Item = Self::Item> + 'a>>where
Self: Sized + 'a,
fn boxed_local<'a>(self) -> Pin<Box<dyn Stream<Item = Self::Item> + 'a>>where
Self: Sized + 'a,
§fn buffered(self, n: usize) -> Buffered<Self>
fn buffered(self, n: usize) -> Buffered<Self>
§fn buffer_unordered(self, n: usize) -> BufferUnordered<Self>
fn buffer_unordered(self, n: usize) -> BufferUnordered<Self>
§fn zip<St>(self, other: St) -> Zip<Self, St>where
St: Stream,
Self: Sized,
fn zip<St>(self, other: St) -> Zip<Self, St>where
St: Stream,
Self: Sized,
§fn chain<St>(self, other: St) -> Chain<Self, St>where
St: Stream<Item = Self::Item>,
Self: Sized,
fn chain<St>(self, other: St) -> Chain<Self, St>where
St: Stream<Item = Self::Item>,
Self: Sized,
§fn peekable(self) -> Peekable<Self>where
Self: Sized,
fn peekable(self) -> Peekable<Self>where
Self: Sized,
peek method. Read more§fn chunks(self, capacity: usize) -> Chunks<Self>where
Self: Sized,
fn chunks(self, capacity: usize) -> Chunks<Self>where
Self: Sized,
§fn ready_chunks(self, capacity: usize) -> ReadyChunks<Self>where
Self: Sized,
fn ready_chunks(self, capacity: usize) -> ReadyChunks<Self>where
Self: Sized,
§fn forward<S>(self, sink: S) -> Forward<Self, S>where
S: Sink<Self::Ok, Error = Self::Error>,
Self: Sized + TryStream,
fn forward<S>(self, sink: S) -> Forward<Self, S>where
S: Sink<Self::Ok, Error = Self::Error>,
Self: Sized + TryStream,
§fn inspect<F>(self, f: F) -> Inspect<Self, F>
fn inspect<F>(self, f: F) -> Inspect<Self, F>
§fn left_stream<B>(self) -> Either<Self, B>where
B: Stream<Item = Self::Item>,
Self: Sized,
fn left_stream<B>(self) -> Either<Self, B>where
B: Stream<Item = Self::Item>,
Self: Sized,
§fn right_stream<B>(self) -> Either<B, Self>where
B: Stream<Item = Self::Item>,
Self: Sized,
fn right_stream<B>(self) -> Either<B, Self>where
B: Stream<Item = Self::Item>,
Self: Sized,
§fn poll_next_unpin(&mut self, cx: &mut Context<'_>) -> Poll<Option<Self::Item>>where
Self: Unpin,
fn poll_next_unpin(&mut self, cx: &mut Context<'_>) -> Poll<Option<Self::Item>>where
Self: Unpin,
Stream::poll_next] on Unpin
stream types.§fn select_next_some(&mut self) -> SelectNextSome<'_, Self>where
Self: Unpin + FusedStream,
fn select_next_some(&mut self) -> SelectNextSome<'_, Self>where
Self: Unpin + FusedStream,
§impl<S, T, E> TryStream for S
impl<S, T, E> TryStream for S
§impl<S> TryStreamExt for Swhere
S: TryStream + ?Sized,
impl<S> TryStreamExt for Swhere
S: TryStream + ?Sized,
§fn err_into<E>(self) -> ErrInto<Self, E>
fn err_into<E>(self) -> ErrInto<Self, E>
§fn map_ok<T, F>(self, f: F) -> MapOk<Self, F>
fn map_ok<T, F>(self, f: F) -> MapOk<Self, F>
§fn map_err<E, F>(self, f: F) -> MapErr<Self, F>
fn map_err<E, F>(self, f: F) -> MapErr<Self, F>
§fn and_then<Fut, F>(self, f: F) -> AndThen<Self, Fut, F>
fn and_then<Fut, F>(self, f: F) -> AndThen<Self, Fut, F>
f. Read more§fn or_else<Fut, F>(self, f: F) -> OrElse<Self, Fut, F>
fn or_else<Fut, F>(self, f: F) -> OrElse<Self, Fut, F>
f. Read more§fn inspect_ok<F>(self, f: F) -> InspectOk<Self, F>
fn inspect_ok<F>(self, f: F) -> InspectOk<Self, F>
§fn inspect_err<F>(self, f: F) -> InspectErr<Self, F>
fn inspect_err<F>(self, f: F) -> InspectErr<Self, F>
§fn into_stream(self) -> IntoStream<Self>where
Self: Sized,
fn into_stream(self) -> IntoStream<Self>where
Self: Sized,
§fn try_next(&mut self) -> TryNext<'_, Self>where
Self: Unpin,
fn try_next(&mut self) -> TryNext<'_, Self>where
Self: Unpin,
§fn try_for_each<Fut, F>(self, f: F) -> TryForEach<Self, Fut, F>
fn try_for_each<Fut, F>(self, f: F) -> TryForEach<Self, Fut, F>
§fn try_skip_while<Fut, F>(self, f: F) -> TrySkipWhile<Self, Fut, F>
fn try_skip_while<Fut, F>(self, f: F) -> TrySkipWhile<Self, Fut, F>
true. Read more§fn try_take_while<Fut, F>(self, f: F) -> TryTakeWhile<Self, Fut, F>
fn try_take_while<Fut, F>(self, f: F) -> TryTakeWhile<Self, Fut, F>
true. Read more§fn try_for_each_concurrent<Fut, F>(
self,
limit: impl Into<Option<usize>>,
f: F,
) -> TryForEachConcurrent<Self, Fut, F>
fn try_for_each_concurrent<Fut, F>( self, limit: impl Into<Option<usize>>, f: F, ) -> TryForEachConcurrent<Self, Fut, F>
§fn try_collect<C>(self) -> TryCollect<Self, C>
fn try_collect<C>(self) -> TryCollect<Self, C>
§fn try_chunks(self, capacity: usize) -> TryChunks<Self>where
Self: Sized,
fn try_chunks(self, capacity: usize) -> TryChunks<Self>where
Self: Sized,
§fn try_ready_chunks(self, capacity: usize) -> TryReadyChunks<Self>where
Self: Sized,
fn try_ready_chunks(self, capacity: usize) -> TryReadyChunks<Self>where
Self: Sized,
§fn try_filter<Fut, F>(self, f: F) -> TryFilter<Self, Fut, F>
fn try_filter<Fut, F>(self, f: F) -> TryFilter<Self, Fut, F>
§fn try_filter_map<Fut, F, T>(self, f: F) -> TryFilterMap<Self, Fut, F>
fn try_filter_map<Fut, F, T>(self, f: F) -> TryFilterMap<Self, Fut, F>
§fn try_flatten_unordered(
self,
limit: impl Into<Option<usize>>,
) -> TryFlattenUnordered<Self>
fn try_flatten_unordered( self, limit: impl Into<Option<usize>>, ) -> TryFlattenUnordered<Self>
§fn try_flatten(self) -> TryFlatten<Self>
fn try_flatten(self) -> TryFlatten<Self>
§fn try_fold<T, Fut, F>(self, init: T, f: F) -> TryFold<Self, Fut, T, F>
fn try_fold<T, Fut, F>(self, init: T, f: F) -> TryFold<Self, Fut, T, F>
§fn try_concat(self) -> TryConcat<Self>
fn try_concat(self) -> TryConcat<Self>
§fn try_buffer_unordered(self, n: usize) -> TryBufferUnordered<Self>where
Self::Ok: TryFuture<Error = Self::Error>,
Self: Sized,
fn try_buffer_unordered(self, n: usize) -> TryBufferUnordered<Self>where
Self::Ok: TryFuture<Error = Self::Error>,
Self: Sized,
§fn try_buffered(self, n: usize) -> TryBuffered<Self>where
Self::Ok: TryFuture<Error = Self::Error>,
Self: Sized,
fn try_buffered(self, n: usize) -> TryBuffered<Self>where
Self::Ok: TryFuture<Error = Self::Error>,
Self: Sized,
§fn try_poll_next_unpin(
&mut self,
cx: &mut Context<'_>,
) -> Poll<Option<Result<Self::Ok, Self::Error>>>where
Self: Unpin,
fn try_poll_next_unpin(
&mut self,
cx: &mut Context<'_>,
) -> Poll<Option<Result<Self::Ok, Self::Error>>>where
Self: Unpin,
TryStream::try_poll_next] on Unpin
stream types.§fn into_async_read(self) -> IntoAsyncRead<Self>
fn into_async_read(self) -> IntoAsyncRead<Self>
AsyncBufRead. Read more§fn try_all<Fut, F>(self, f: F) -> TryAll<Self, Fut, F>
fn try_all<Fut, F>(self, f: F) -> TryAll<Self, Fut, F>
Err is encountered or if an Ok item is found
that does not satisfy the predicate. Read more