Struct mz_compute_client::controller::instance::Instance

source ·
pub(super) struct Instance<T: ComputeControllerTimestamp> {
Show 25 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>)>, 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 ReadHolds 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.

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

source

fn collection( &self, id: GlobalId, ) -> Result<&CollectionState<T>, CollectionMissing>

Acquire a handle to the collection state associated with id.

source

fn collection_mut( &mut self, id: GlobalId, ) -> Result<&mut CollectionState<T>, CollectionMissing>

Acquire a mutable handle to the collection state associated with id.

source

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.

source

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.

source

fn collections_iter( &self, ) -> impl Iterator<Item = (GlobalId, &CollectionState<T>)>

source

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.

source

fn remove_collection(&mut self, id: GlobalId)

source

fn add_replica_state( &mut self, id: ReplicaId, client: ReplicaClient<T>, config: ReplicaConfig, epoch: ClusterStartupEpoch, )

source

fn deliver_response(&self, response: ComputeControllerResponse<T>)

Enqueue the given response for delivery to the controller clients.

source

fn deliver_introspection_updates( &self, type_: IntrospectionType, updates: Vec<(Row, Diff)>, )

Enqueue the given introspection updates for recording.

source

fn replica_exists(&self, id: ReplicaId) -> bool

Returns whether the identified replica exists.

source

fn peeks_targeting( &self, replica_id: ReplicaId, ) -> impl Iterator<Item = (Uuid, &PendingPeek<T>)>

Return the IDs of pending peeks targeting the specified replica.

source

fn subscribes_targeting( &self, replica_id: ReplicaId, ) -> impl Iterator<Item = GlobalId> + '_

Return the IDs of in-progress subscribes targeting the specified replica.

source

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.

source

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.

source

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.

source

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.

source

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.

source

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.

source

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.

source

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.

source

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:

  1. 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.
  2. 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.
  3. 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 ComputeResponses 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

pub fn dump(&self) -> Result<Value, Error>

Returns the state of the Instance formatted as JSON.

The returned value is not guaranteed to be stable and may change at any point in time.

source§

impl<T> Instance<T>

source

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

source

async fn run(self)

source

pub fn update_configuration(&mut self, config_params: ComputeParameters)

Update instance configuration.

source

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.

source

pub fn allow_writes(&mut self)

Allows this instance to affect writes to external systems (persist).

Calling this method repeatedly has no effect.

source

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.

source

fn send(&mut self, cmd: ComputeCommand<T>)

Sends a command to all replicas of this instance.

source

pub fn add_replica( &mut self, id: ReplicaId, config: ReplicaConfig, ) -> Result<(), ReplicaExists>

Add a new instance replica, by ID.

source

pub fn remove_replica(&mut self, id: ReplicaId) -> Result<(), ReplicaMissing>

Remove an existing instance replica, by ID.

source

fn rehydrate_replica(&mut self, id: ReplicaId)

Rehydrate the given instance replica.

§Panics

Panics if the specified replica does not exist.

source

fn rehydrate_failed_replicas(&mut self)

Rehydrate any failed replicas of this instance.

source

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.

source

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.

source

fn schedule_collections(&mut self)

Schedule any unscheduled collections that are ready.

source

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.

source

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.

source

pub fn cancel_peek(&mut self, uuid: Uuid, reason: PeekResponse)

Cancels an existing peek request.

source

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.

source

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.

source

fn apply_read_hold_changes(&mut self)

Apply collection read hold changes pending in read_holds_rx.

source

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 the ComputeCommandHistory to reduce away the corresponding Peek command.
  • Remove the read hold for this peek, unblocking compaction that might have waited on it.
source

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.

source

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.

source

fn handle_peek_response( &mut self, uuid: Uuid, response: PeekResponse, otel_ctx: OpenTelemetryContext, replica_id: ReplicaId, )

source

fn handle_copy_to_response( &mut self, sink_id: GlobalId, response: CopyToResponse, replica_id: ReplicaId, )

source

fn handle_subscribe_response( &mut self, subscribe_id: GlobalId, response: SubscribeResponse<T>, replica_id: ReplicaId, )

source

fn handle_status_response( &mut self, response: StatusResponse, replica_id: ReplicaId, )

source

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.

source

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

impl<T, U> CastInto<U> for T
where U: CastFrom<T>,

source§

fn cast_into(self) -> U

Performs the cast.
source§

impl<T> Conv for T

source§

fn conv<T>(self) -> T
where Self: Into<T>,

Converts self into T using Into<T>. Read more
source§

impl<T> FmtForward for T

source§

fn fmt_binary(self) -> FmtBinary<Self>
where Self: Binary,

Causes self to use its Binary implementation when Debug-formatted.
source§

fn fmt_display(self) -> FmtDisplay<Self>
where Self: Display,

Causes self to use its Display implementation when Debug-formatted.
source§

fn fmt_lower_exp(self) -> FmtLowerExp<Self>
where Self: LowerExp,

Causes self to use its LowerExp implementation when Debug-formatted.
source§

fn fmt_lower_hex(self) -> FmtLowerHex<Self>
where Self: LowerHex,

Causes self to use its LowerHex implementation when Debug-formatted.
source§

fn fmt_octal(self) -> FmtOctal<Self>
where Self: Octal,

Causes self to use its Octal implementation when Debug-formatted.
source§

fn fmt_pointer(self) -> FmtPointer<Self>
where Self: Pointer,

Causes self to use its Pointer implementation when Debug-formatted.
source§

fn fmt_upper_exp(self) -> FmtUpperExp<Self>
where Self: UpperExp,

Causes self to use its UpperExp implementation when Debug-formatted.
source§

fn fmt_upper_hex(self) -> FmtUpperHex<Self>
where Self: UpperHex,

Causes self to use its UpperHex implementation when Debug-formatted.
source§

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.

source§

impl<T> FutureExt for T

source§

fn with_context(self, otel_cx: Context) -> WithContext<Self>

Attaches the provided Context to this type, returning a WithContext wrapper. Read more
source§

fn with_current_context(self) -> WithContext<Self>

Attaches the current Context to this type, returning a WithContext 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> IntoRequest<T> for T

source§

fn into_request(self) -> Request<T>

Wrap the input message T in a tonic::Request
source§

impl<Unshared, Shared> IntoShared<Shared> for Unshared
where Shared: FromUnshared<Unshared>,

source§

fn into_shared(self) -> Shared

Creates a shared type from an unshared type.
source§

impl<T, U> OverrideFrom<Option<&T>> for U
where U: OverrideFrom<T>,

source§

fn override_from(self, layer: &Option<&T>) -> U

Override the configuration represented by Self with values from the given layer.
source§

impl<T> Pipe for T
where T: ?Sized,

source§

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

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

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

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

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

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.
source§

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.
source§

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.
source§

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.
source§

impl<T> Pointable for T

source§

const ALIGN: usize = _

The alignment of pointer.
§

type Init = T

The type for initializers.
source§

unsafe fn init(init: <T as Pointable>::Init) -> usize

Initializes a with the given initializer. Read more
source§

unsafe fn deref<'a>(ptr: usize) -> &'a T

Dereferences the given pointer. Read more
source§

unsafe fn deref_mut<'a>(ptr: usize) -> &'a mut T

Mutably dereferences the given pointer. Read more
source§

unsafe fn drop(ptr: usize)

Drops the object pointed to by the given pointer. Read more
source§

impl<P, R> ProtoType<R> for P
where R: RustType<P>,

source§

impl<T> Same for T

§

type Output = T

Should always be Self
source§

impl<'a, S, T> Semigroup<&'a S> for T
where T: Semigroup<S>,

source§

fn plus_equals(&mut self, rhs: &&'a S)

The method of std::ops::AddAssign, for types that do not implement AddAssign.
source§

impl<T> Tap for T

source§

fn tap(self, func: impl FnOnce(&Self)) -> Self

Immutable access to a value. Read more
source§

fn tap_mut(self, func: impl FnOnce(&mut Self)) -> Self

Mutable access to a value. Read more
source§

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

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

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

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

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

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

fn tap_dbg(self, func: impl FnOnce(&Self)) -> Self

Calls .tap() only in debug builds, and is erased in release builds.
source§

fn tap_mut_dbg(self, func: impl FnOnce(&mut Self)) -> Self

Calls .tap_mut() only in debug builds, and is erased in release builds.
source§

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.
source§

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.
source§

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.
source§

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.
source§

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.
source§

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.
source§

impl<T> TryConv for T

source§

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

§

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

§

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.
source§

impl<V, T> VZip<V> for T
where V: MultiLane<T>,

source§

fn vzip(self) -> V

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