Struct mz_compute_client::controller::instance::Instance
source · pub(super) struct Instance<T: ComputeControllerTimestamp> {Show 26 fields
build_info: &'static BuildInfo,
storage_collections: Arc<dyn StorageCollections<Timestamp = T> + Send + Sync>,
initialized: bool,
read_only: bool,
replicas: BTreeMap<ReplicaId, ReplicaState<T>>,
collections: BTreeMap<GlobalId, CollectionState<T>>,
log_sources: BTreeMap<LogVariant, GlobalId>,
peeks: BTreeMap<Uuid, PendingPeek<T>>,
subscribes: BTreeMap<GlobalId, ActiveSubscribe<T>>,
copy_tos: BTreeSet<GlobalId>,
history: ComputeCommandHistory<UIntGauge, T>,
command_rx: UnboundedReceiver<Box<dyn FnOnce(&mut Instance<T>) + Send>>,
response_tx: UnboundedSender<ComputeControllerResponse<T>>,
introspection_tx: Sender<(IntrospectionType, Vec<(Row, Diff)>)>,
envd_epoch: NonZeroI64,
replica_epochs: BTreeMap<ReplicaId, u64>,
metrics: InstanceMetrics,
dyncfg: Arc<ConfigSet>,
now: NowFn,
wallclock_lag: WallclockLagFn<T>,
wallclock_lag_last_refresh: Instant,
read_holds_tx: UnboundedSender<(GlobalId, ChangeBatch<T>)>,
read_holds_rx: UnboundedReceiver<(GlobalId, ChangeBatch<T>)>,
stashed_read_hold_changes: BTreeMap<GlobalId, ChangeBatch<T>>,
replica_tx: InstrumentedUnboundedSender<(ReplicaId, u64, ComputeResponse<T>), DeleteOnDropCounter<'static, AtomicU64, Vec<String>>>,
replica_rx: InstrumentedUnboundedReceiver<(ReplicaId, u64, ComputeResponse<T>), DeleteOnDropCounter<'static, AtomicU64, Vec<String>>>,
}
Expand description
The state we keep for a compute instance.
Fields§
§build_info: &'static BuildInfo
Build info for spawning replicas
storage_collections: Arc<dyn StorageCollections<Timestamp = T> + Send + Sync>
A handle providing access to storage collections.
initialized: bool
Whether instance initialization has been completed.
read_only: bool
Whether or not this instance is in read-only mode.
When in read-only mode, neither the controller nor the instances controlled by it are allowed to affect changes to external systems (largely persist).
replicas: BTreeMap<ReplicaId, ReplicaState<T>>
The replicas of this compute instance.
collections: BTreeMap<GlobalId, CollectionState<T>>
Currently installed compute collections.
New entries are added for all collections exported from dataflows created through
Instance::create_dataflow
.
Entries are removed by Instance::cleanup_collections
. See that method’s documentation
about the conditions for removing collection state.
log_sources: BTreeMap<LogVariant, GlobalId>
IDs of log sources maintained by this compute instance.
peeks: BTreeMap<Uuid, PendingPeek<T>>
Currently outstanding peeks.
New entries are added for all peeks initiated through Instance::peek
.
The entry for a peek is only removed once all replicas have responded to the peek. This is currently required to ensure all replicas have stopped reading from the peeked collection’s inputs before we allow them to compact. database-issues#4822 tracks changing this so we only have to wait for the first peek response.
subscribes: BTreeMap<GlobalId, ActiveSubscribe<T>>
Currently in-progress subscribes.
New entries are added for all subscribes exported from dataflows created through
Instance::create_dataflow
.
The entry for a subscribe is removed once at least one replica has reported the subscribe to have advanced to the empty frontier or to have been dropped, implying that no further updates will be emitted for this subscribe.
Note that subscribes are tracked both in collections
and subscribes
. collections
keeps track of the subscribe’s upper and since frontiers and ensures appropriate read holds
on the subscribe’s input. subscribes
is only used to track which updates have been
emitted, to decide if new ones should be emitted or suppressed.
copy_tos: BTreeSet<GlobalId>
Tracks all in-progress COPY TOs.
New entries are added for all s3 oneshot sinks (corresponding to a COPY TO) exported from
dataflows created through Instance::create_dataflow
.
The entry for a copy to is removed once at least one replica has finished or the exporting collection is dropped.
history: ComputeCommandHistory<UIntGauge, T>
The command history, used when introducing new replicas or restarting existing replicas.
command_rx: UnboundedReceiver<Box<dyn FnOnce(&mut Instance<T>) + Send>>
Receiver for commands to be executed.
response_tx: UnboundedSender<ComputeControllerResponse<T>>
Sender for responses to be delivered.
introspection_tx: Sender<(IntrospectionType, Vec<(Row, Diff)>)>
Sender for introspection updates to be recorded.
envd_epoch: NonZeroI64
A number that increases with each restart of environmentd
.
replica_epochs: BTreeMap<ReplicaId, u64>
Numbers that increase with each restart of a replica.
metrics: InstanceMetrics
The registry the controller uses to report metrics.
dyncfg: Arc<ConfigSet>
Dynamic system configuration.
now: NowFn
A function that produces the current wallclock time.
wallclock_lag: WallclockLagFn<T>
A function that computes the lag between the given time and wallclock time.
wallclock_lag_last_refresh: Instant
The last time wallclock lag introspection was refreshed.
read_holds_tx: UnboundedSender<(GlobalId, ChangeBatch<T>)>
Sender for updates to collection read holds.
Copies of this sender are given to ReadHold
s that are created in
CollectionState::new
.
read_holds_rx: UnboundedReceiver<(GlobalId, ChangeBatch<T>)>
Receiver for updates to collection read holds.
Received updates are applied by Instance::apply_read_hold_changes
.
stashed_read_hold_changes: BTreeMap<GlobalId, ChangeBatch<T>>
Stashed read hold changes.
Used by Instance::apply_read_hold_changes
to stash read hold changes that cannot be
applied immediately until they can be applied.
replica_tx: InstrumentedUnboundedSender<(ReplicaId, u64, ComputeResponse<T>), DeleteOnDropCounter<'static, AtomicU64, Vec<String>>>
A sender for responses from replicas.
replica_rx: InstrumentedUnboundedReceiver<(ReplicaId, u64, ComputeResponse<T>), DeleteOnDropCounter<'static, AtomicU64, Vec<String>>>
A receiver for responses from replicas.
Implementations§
source§impl<T: ComputeControllerTimestamp> Instance<T>
impl<T: ComputeControllerTimestamp> Instance<T>
sourcefn collection(
&self,
id: GlobalId,
) -> Result<&CollectionState<T>, CollectionMissing>
fn collection( &self, id: GlobalId, ) -> Result<&CollectionState<T>, CollectionMissing>
Acquire a handle to the collection state associated with id
.
sourcefn collection_mut(
&mut self,
id: GlobalId,
) -> Result<&mut CollectionState<T>, CollectionMissing>
fn collection_mut( &mut self, id: GlobalId, ) -> Result<&mut CollectionState<T>, CollectionMissing>
Acquire a mutable handle to the collection state associated with id
.
sourcefn expect_collection(&self, id: GlobalId) -> &CollectionState<T>
fn expect_collection(&self, id: GlobalId) -> &CollectionState<T>
Acquire a handle to the collection state associated with id
.
§Panics
Panics if the identified collection does not exist.
sourcefn expect_collection_mut(&mut self, id: GlobalId) -> &mut CollectionState<T>
fn expect_collection_mut(&mut self, id: GlobalId) -> &mut CollectionState<T>
Acquire a mutable handle to the collection state associated with id
.
§Panics
Panics if the identified collection does not exist.
fn collections_iter( &self, ) -> impl Iterator<Item = (GlobalId, &CollectionState<T>)>
sourcefn add_collection(
&mut self,
id: GlobalId,
as_of: Antichain<T>,
shared: SharedCollectionState<T>,
storage_dependencies: BTreeMap<GlobalId, ReadHold<T>>,
compute_dependencies: BTreeMap<GlobalId, ReadHold<T>>,
replica_input_read_holds: Vec<ReadHold<T>>,
write_only: bool,
storage_sink: bool,
initial_as_of: Option<Antichain<T>>,
refresh_schedule: Option<RefreshSchedule>,
)
fn add_collection( &mut self, id: GlobalId, as_of: Antichain<T>, shared: SharedCollectionState<T>, storage_dependencies: BTreeMap<GlobalId, ReadHold<T>>, compute_dependencies: BTreeMap<GlobalId, ReadHold<T>>, replica_input_read_holds: Vec<ReadHold<T>>, write_only: bool, storage_sink: bool, initial_as_of: Option<Antichain<T>>, refresh_schedule: Option<RefreshSchedule>, )
Add a collection to the instance state.
§Panics
Panics if a collection with the same ID exists already.
fn remove_collection(&mut self, id: GlobalId)
fn add_replica_state( &mut self, id: ReplicaId, client: ReplicaClient<T>, config: ReplicaConfig, epoch: ClusterStartupEpoch, )
sourcefn deliver_response(&self, response: ComputeControllerResponse<T>)
fn deliver_response(&self, response: ComputeControllerResponse<T>)
Enqueue the given response for delivery to the controller clients.
sourcefn deliver_introspection_updates(
&self,
type_: IntrospectionType,
updates: Vec<(Row, Diff)>,
)
fn deliver_introspection_updates( &self, type_: IntrospectionType, updates: Vec<(Row, Diff)>, )
Enqueue the given introspection updates for recording.
sourcefn replica_exists(&self, id: ReplicaId) -> bool
fn replica_exists(&self, id: ReplicaId) -> bool
Returns whether the identified replica exists.
sourcefn peeks_targeting(
&self,
replica_id: ReplicaId,
) -> impl Iterator<Item = (Uuid, &PendingPeek<T>)>
fn peeks_targeting( &self, replica_id: ReplicaId, ) -> impl Iterator<Item = (Uuid, &PendingPeek<T>)>
Return the IDs of pending peeks targeting the specified replica.
sourcefn subscribes_targeting(
&self,
replica_id: ReplicaId,
) -> impl Iterator<Item = GlobalId> + '_
fn subscribes_targeting( &self, replica_id: ReplicaId, ) -> impl Iterator<Item = GlobalId> + '_
Return the IDs of in-progress subscribes targeting the specified replica.
sourcefn update_frontier_introspection(&mut self)
fn update_frontier_introspection(&mut self)
Update introspection with the current collection frontiers.
We could also do this directly in response to frontier changes, but doing it periodically lets us avoid emitting some introspection updates that can be consolidated (e.g. a write frontier updated immediately followed by a read frontier update).
This method is invoked by ComputeController::maintain
, which we expect to be called once
per second during normal operation.
sourcefn refresh_state_metrics(&self)
fn refresh_state_metrics(&self)
Refresh the controller state metrics for this instance.
We could also do state metric updates directly in response to state changes, but that would mean littering the code with metric update calls. Encapsulating state metric maintenance in a single method is less noisy.
This method is invoked by ComputeController::maintain
, which we expect to be called once
per second during normal operation.
sourcefn refresh_wallclock_lag(&mut self)
fn refresh_wallclock_lag(&mut self)
Refresh the WallclockLagHistory
introspection and the wallclock_lag_*_seconds
metrics
with the current lag values.
This method is invoked by ComputeController::maintain
, which we expect to be called once
per second during normal operation.
sourcefn report_dependency_updates(&self, id: GlobalId, diff: i64)
fn report_dependency_updates(&self, id: GlobalId, diff: i64)
Report updates (inserts or retractions) to the identified collection’s dependencies.
§Panics
Panics if the identified collection does not exist.
sourcefn update_operator_hydration_status(
&mut self,
replica_id: ReplicaId,
status: OperatorHydrationStatus,
)
fn update_operator_hydration_status( &mut self, replica_id: ReplicaId, status: OperatorHydrationStatus, )
Update the tracked hydration status for an operator according to a received status update.
sourcepub fn collection_hydrated(
&self,
collection_id: GlobalId,
) -> Result<bool, CollectionLookupError>
pub fn collection_hydrated( &self, collection_id: GlobalId, ) -> Result<bool, CollectionLookupError>
Returns true
if the given collection is hydrated on at least one
replica.
This also returns true
in case this cluster does not have any
replicas.
sourcepub fn collections_hydrated_on_replicas(
&self,
target_replica_ids: Option<Vec<ReplicaId>>,
exclude_collections: &BTreeSet<GlobalId>,
) -> Result<bool, HydrationCheckBadTarget>
pub fn collections_hydrated_on_replicas( &self, target_replica_ids: Option<Vec<ReplicaId>>, exclude_collections: &BTreeSet<GlobalId>, ) -> Result<bool, HydrationCheckBadTarget>
Returns true
if each non-transient, non-excluded collection is hydrated on at
least one replica.
This also returns true
in case this cluster does not have any
replicas.
sourcepub fn collections_hydrated(
&self,
exclude_collections: &BTreeSet<GlobalId>,
) -> bool
pub fn collections_hydrated( &self, exclude_collections: &BTreeSet<GlobalId>, ) -> bool
Returns true
if all non-transient, non-excluded collections are hydrated on at least one
replica.
This also returns true
in case this cluster does not have any
replicas.
sourcefn cleanup_collections(&mut self)
fn cleanup_collections(&mut self)
Clean up collection state that is not needed anymore.
Three conditions need to be true before we can remove state for a collection:
- A client must have explicitly dropped the collection. If that is not the case, clients can still reasonably assume that the controller knows about the collection and can answer queries about it.
- There must be no outstanding read capabilities on the collection. As long as someone still holds read capabilities on a collection, we need to keep it around to be able to properly handle downgrading of said capabilities.
- All replica frontiers for the collection must have advanced to the empty frontier.
Advancement to the empty frontiers signals that replicas are done computing the
collection and that they won’t send more
ComputeResponse
s for it. As long as we might receive responses for a collection we want to keep it around to be able to validate and handle these responses.
source§impl<T> Instance<T>
impl<T> Instance<T>
fn new( build_info: &'static BuildInfo, storage: Arc<dyn StorageCollections<Timestamp = T> + Send + Sync>, arranged_logs: Vec<(LogVariant, GlobalId, SharedCollectionState<T>)>, envd_epoch: NonZeroI64, metrics: InstanceMetrics, now: NowFn, wallclock_lag: WallclockLagFn<T>, dyncfg: Arc<ConfigSet>, command_rx: UnboundedReceiver<Box<dyn FnOnce(&mut Instance<T>) + Send>>, response_tx: UnboundedSender<ComputeControllerResponse<T>>, introspection_tx: Sender<(IntrospectionType, Vec<(Row, Diff)>)>, read_holds_tx: UnboundedSender<(GlobalId, ChangeBatch<T>)>, read_holds_rx: UnboundedReceiver<(GlobalId, ChangeBatch<T>)>, ) -> Self
async fn run(self)
sourcepub fn update_configuration(&mut self, config_params: ComputeParameters)
pub fn update_configuration(&mut self, config_params: ComputeParameters)
Update instance configuration.
sourcepub fn initialization_complete(&mut self)
pub fn initialization_complete(&mut self)
Marks the end of any initialization commands.
Intended to be called by Controller
, rather than by other code.
Calling this method repeatedly has no effect.
sourcepub fn allow_writes(&mut self)
pub fn allow_writes(&mut self)
Allows this instance to affect writes to external systems (persist).
Calling this method repeatedly has no effect.
sourcepub fn check_empty(&mut self)
pub fn check_empty(&mut self)
Check that the current instance is empty.
This method exists to help us find bugs where the client drops a compute instance that still has replicas or collections installed, and later assumes that said replicas/collections still exists.
§Panics
Panics if the compute instance still has active replicas. Panics if the compute instance still has collections installed.
sourcefn send(&mut self, cmd: ComputeCommand<T>)
fn send(&mut self, cmd: ComputeCommand<T>)
Sends a command to all replicas of this instance.
sourcepub fn add_replica(
&mut self,
id: ReplicaId,
config: ReplicaConfig,
) -> Result<(), ReplicaExists>
pub fn add_replica( &mut self, id: ReplicaId, config: ReplicaConfig, ) -> Result<(), ReplicaExists>
Add a new instance replica, by ID.
sourcepub fn remove_replica(&mut self, id: ReplicaId) -> Result<(), ReplicaMissing>
pub fn remove_replica(&mut self, id: ReplicaId) -> Result<(), ReplicaMissing>
Remove an existing instance replica, by ID.
sourcefn rehydrate_replica(&mut self, id: ReplicaId)
fn rehydrate_replica(&mut self, id: ReplicaId)
sourcefn rehydrate_failed_replicas(&mut self)
fn rehydrate_failed_replicas(&mut self)
Rehydrate any failed replicas of this instance.
sourcepub fn create_dataflow(
&mut self,
dataflow: DataflowDescription<Plan<T>, (), T>,
import_read_holds: Vec<ReadHold<T>>,
subscribe_target_replica: Option<ReplicaId>,
shared_collection_state: BTreeMap<GlobalId, SharedCollectionState<T>>,
) -> Result<(), DataflowCreationError>
pub fn create_dataflow( &mut self, dataflow: DataflowDescription<Plan<T>, (), T>, import_read_holds: Vec<ReadHold<T>>, subscribe_target_replica: Option<ReplicaId>, shared_collection_state: BTreeMap<GlobalId, SharedCollectionState<T>>, ) -> Result<(), DataflowCreationError>
Creates the described dataflow and initializes state for its output.
This method expects a DataflowDescription
with an as_of
frontier specified, as well as
for each imported collection a read hold in import_read_holds
at at least the as_of
.
If a subscribe_target_replica
is given, any subscribes exported by the dataflow are
configured to target that replica, i.e., only subscribe responses sent by that replica are
considered.
sourcefn maybe_schedule_collection(&mut self, id: GlobalId)
fn maybe_schedule_collection(&mut self, id: GlobalId)
Schedule the identified collection if all its inputs are available.
§Panics
Panics if the identified collection does not exist.
sourcefn schedule_collections(&mut self)
fn schedule_collections(&mut self)
Schedule any unscheduled collections that are ready.
sourcepub fn drop_collections(
&mut self,
ids: Vec<GlobalId>,
) -> Result<(), CollectionMissing>
pub fn drop_collections( &mut self, ids: Vec<GlobalId>, ) -> Result<(), CollectionMissing>
Drops the read capability for the given collections and allows their resources to be reclaimed.
sourcepub fn peek(
&mut self,
peek_target: PeekTarget,
literal_constraints: Option<Vec<Row>>,
uuid: Uuid,
timestamp: T,
finishing: RowSetFinishing,
map_filter_project: SafeMfpPlan,
read_hold: ReadHold<T>,
target_replica: Option<ReplicaId>,
peek_response_tx: Sender<PeekResponse>,
) -> Result<(), PeekError>
pub fn peek( &mut self, peek_target: PeekTarget, literal_constraints: Option<Vec<Row>>, uuid: Uuid, timestamp: T, finishing: RowSetFinishing, map_filter_project: SafeMfpPlan, read_hold: ReadHold<T>, target_replica: Option<ReplicaId>, peek_response_tx: Sender<PeekResponse>, ) -> Result<(), PeekError>
Initiate a peek request for the contents of id
at timestamp
.
sourcepub fn cancel_peek(&mut self, uuid: Uuid, reason: PeekResponse)
pub fn cancel_peek(&mut self, uuid: Uuid, reason: PeekResponse)
Cancels an existing peek request.
sourcepub fn set_read_policy(
&mut self,
policies: Vec<(GlobalId, ReadPolicy<T>)>,
) -> Result<(), ReadPolicyError>
pub fn set_read_policy( &mut self, policies: Vec<(GlobalId, ReadPolicy<T>)>, ) -> Result<(), ReadPolicyError>
Assigns a read policy to specific identifiers.
The policies are assigned in the order presented, and repeated identifiers should conclude with the last policy. Changing a policy will immediately downgrade the read capability if appropriate, but it will not “recover” the read capability if the prior capability is already ahead of it.
Identifiers not present in policies
retain their existing read policies.
It is an error to attempt to set a read policy for a collection that is not readable in the context of compute. At this time, only indexes are readable compute collections.
sourcefn maybe_update_global_write_frontier(
&mut self,
id: GlobalId,
new_frontier: Antichain<T>,
)
fn maybe_update_global_write_frontier( &mut self, id: GlobalId, new_frontier: Antichain<T>, )
Advance the global write frontier of the given collection.
Frontier regressions are gracefully ignored.
§Panics
Panics if the identified collection does not exist.
sourcefn apply_read_hold_changes(&mut self)
fn apply_read_hold_changes(&mut self)
Apply collection read hold changes pending in read_holds_rx
.
sourcefn finish_peek(&mut self, uuid: Uuid, response: PeekResponse)
fn finish_peek(&mut self, uuid: Uuid, response: PeekResponse)
Fulfills a registered peek and cleans up associated state.
As part of this we:
- Send a
PeekResponse
through the peek’s response channel. - Emit a
CancelPeek
command to instruct replicas to stop spending resources on this peek, and to allow theComputeCommandHistory
to reduce away the correspondingPeek
command. - Remove the read hold for this peek, unblocking compaction that might have waited on it.
sourcefn handle_response(
&mut self,
(replica_id, incarnation, response): (ReplicaId, u64, ComputeResponse<T>),
)
fn handle_response( &mut self, (replica_id, incarnation, response): (ReplicaId, u64, ComputeResponse<T>), )
Handles a response from a replica. Replica IDs are re-used across replica restarts, so we use the replica incarnation to drop stale responses.
sourcefn handle_frontiers_response(
&mut self,
id: GlobalId,
frontiers: FrontiersResponse<T>,
replica_id: ReplicaId,
)
fn handle_frontiers_response( &mut self, id: GlobalId, frontiers: FrontiersResponse<T>, replica_id: ReplicaId, )
Handle new frontiers, returning any compute response that needs to be sent to the client.
fn handle_peek_response( &mut self, uuid: Uuid, response: PeekResponse, otel_ctx: OpenTelemetryContext, replica_id: ReplicaId, )
fn handle_copy_to_response( &mut self, sink_id: GlobalId, response: CopyToResponse, replica_id: ReplicaId, )
fn handle_subscribe_response( &mut self, subscribe_id: GlobalId, response: SubscribeResponse<T>, replica_id: ReplicaId, )
fn handle_status_response( &mut self, response: StatusResponse, replica_id: ReplicaId, )
sourcefn downgrade_warmup_capabilities(&mut self)
fn downgrade_warmup_capabilities(&mut self)
Downgrade the warmup capabilities of collections as much as possible.
The only requirement we have for a collection’s warmup capability is that it is for a time
that is available in all of the collection’s inputs. For each input the latest time that is
the case for is write_frontier - 1
. So the farthest we can downgrade a collection’s
warmup capability is the minimum of write_frontier - 1
of all its inputs.
This method expects to be periodically called as part of instance maintenance work. We would like to instead update the warmup capabilities synchronously in response to frontier updates of dependency collections, but that is not generally possible because we don’t learn about frontier updates of storage collections synchronously. We could do synchronous updates for compute dependencies, but we refrain from doing for simplicity.
sourcepub fn maintain(&mut self)
pub fn maintain(&mut self)
Process pending maintenance work.
This method is invoked periodically by the global controller. It is a good place to perform maintenance work that arises from various controller state changes and that cannot conveniently be handled synchronously with those state changes.
Auto Trait Implementations§
impl<T> Freeze for Instance<T>
impl<T> !RefUnwindSafe for Instance<T>
impl<T> Send for Instance<T>
impl<T> Sync for Instance<T>
impl<T> Unpin for Instance<T>where
T: Unpin,
impl<T> !UnwindSafe for Instance<T>
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> FmtForward for T
impl<T> FmtForward for T
source§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.source§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.source§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.source§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.source§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.source§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.source§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.source§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.source§impl<T> FutureExt for T
impl<T> FutureExt for T
source§fn with_context(self, otel_cx: Context) -> WithContext<Self>
fn with_context(self, otel_cx: Context) -> WithContext<Self>
source§fn with_current_context(self) -> WithContext<Self>
fn with_current_context(self) -> WithContext<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> 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
source§impl<T, U> OverrideFrom<Option<&T>> for Uwhere
U: OverrideFrom<T>,
impl<T, U> OverrideFrom<Option<&T>> for Uwhere
U: OverrideFrom<T>,
source§impl<T> Pipe for Twhere
T: ?Sized,
impl<T> Pipe for Twhere
T: ?Sized,
source§fn pipe<R>(self, func: impl FnOnce(Self) -> R) -> Rwhere
Self: Sized,
fn pipe<R>(self, func: impl FnOnce(Self) -> R) -> Rwhere
Self: Sized,
source§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 moresource§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 moresource§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
source§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
source§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.source§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.source§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.source§impl<T> Pointable for T
impl<T> Pointable for T
source§impl<P, R> ProtoType<R> for Pwhere
R: RustType<P>,
impl<P, R> ProtoType<R> for Pwhere
R: RustType<P>,
source§fn into_rust(self) -> Result<R, TryFromProtoError>
fn into_rust(self) -> Result<R, TryFromProtoError>
RustType::from_proto
.source§fn from_rust(rust: &R) -> P
fn from_rust(rust: &R) -> P
RustType::into_proto
.source§impl<'a, S, T> Semigroup<&'a S> for Twhere
T: Semigroup<S>,
impl<'a, S, T> Semigroup<&'a S> for Twhere
T: Semigroup<S>,
source§fn plus_equals(&mut self, rhs: &&'a S)
fn plus_equals(&mut self, rhs: &&'a S)
std::ops::AddAssign
, for types that do not implement AddAssign
.source§impl<T> Tap for T
impl<T> Tap for T
source§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 moresource§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 moresource§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 moresource§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 moresource§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 moresource§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 moresource§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.source§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.source§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.source§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.source§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.source§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.source§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.