pub enum MirRelationExpr {
Show 15 variants Constant { rows: Result<Vec<(Row, Diff)>, EvalError>, typ: RelationType, }, Get { id: Id, typ: RelationType, access_strategy: AccessStrategy, }, Let { id: LocalId, value: Box<MirRelationExpr>, body: Box<MirRelationExpr>, }, LetRec { ids: Vec<LocalId>, values: Vec<MirRelationExpr>, limits: Vec<Option<LetRecLimit>>, body: Box<MirRelationExpr>, }, Project { input: Box<MirRelationExpr>, outputs: Vec<usize>, }, Map { input: Box<MirRelationExpr>, scalars: Vec<MirScalarExpr>, }, FlatMap { input: Box<MirRelationExpr>, func: TableFunc, exprs: Vec<MirScalarExpr>, }, Filter { input: Box<MirRelationExpr>, predicates: Vec<MirScalarExpr>, }, Join { inputs: Vec<MirRelationExpr>, equivalences: Vec<Vec<MirScalarExpr>>, implementation: JoinImplementation, }, Reduce { input: Box<MirRelationExpr>, group_key: Vec<MirScalarExpr>, aggregates: Vec<AggregateExpr>, monotonic: bool, expected_group_size: Option<u64>, }, TopK { input: Box<MirRelationExpr>, group_key: Vec<usize>, order_key: Vec<ColumnOrder>, limit: Option<MirScalarExpr>, offset: usize, monotonic: bool, expected_group_size: Option<u64>, }, Negate { input: Box<MirRelationExpr>, }, Threshold { input: Box<MirRelationExpr>, }, Union { base: Box<MirRelationExpr>, inputs: Vec<MirRelationExpr>, }, ArrangeBy { input: Box<MirRelationExpr>, keys: Vec<Vec<MirScalarExpr>>, },
}
Expand description

An abstract syntax tree which defines a collection.

The AST is meant to reflect the capabilities of the differential_dataflow::Collection type, written generically enough to avoid run-time compilation work.

Variants§

§

Constant

Fields

§rows: Result<Vec<(Row, Diff)>, EvalError>

Rows of the constant collection and their multiplicities.

§typ: RelationType

Schema of the collection.

A constant relation containing specified rows.

The runtime memory footprint of this operator is zero.

When you would like to pattern match on this, consider using MirRelationExpr::as_const instead, which looks behind ArrangeBys. You might want this matching behavior because constant folding doesn’t remove ArrangeBys.

§

Get

Fields

§id: Id

The identifier for the collection to load.

§typ: RelationType

Schema of the collection.

§access_strategy: AccessStrategy

If this is a global Get, this will indicate whether we are going to read from Persist or from an index, or from a different object in objects_to_build. If it’s an index, then how downstream dataflow operations will use this index is also recorded. This is filled by prune_and_annotate_dataflow_index_imports. Note that this is not used by the lowering to LIR, but is used only by EXPLAIN.

Get an existing dataflow.

The runtime memory footprint of this operator is zero.

§

Let

Fields

§id: LocalId

The identifier to be used in Get variants to retrieve value.

§value: Box<MirRelationExpr>

The collection to be bound to id.

§body: Box<MirRelationExpr>

The result of the Let, evaluated with id bound to value.

Introduce a temporary dataflow.

The runtime memory footprint of this operator is zero.

§

LetRec

Fields

§ids: Vec<LocalId>

The identifiers to be used in Get variants to retrieve each value.

§values: Vec<MirRelationExpr>

The collections to be bound to each id.

§limits: Vec<Option<LetRecLimit>>

Maximum number of iterations, after which we should artificially force a fixpoint. (We don’t error when reaching the limit, just return the current state as final result.) The per-LetRec limit that the user specified is initially copied to each binding to accommodate slicing and merging of LetRecs in MIR transforms (e.g., NormalizeLets).

§body: Box<MirRelationExpr>

The result of the Let, evaluated with id bound to value.

Introduce mutually recursive bindings.

Each LocalId is immediately bound to an initially empty collection with the type of its corresponding MirRelationExpr. Repeatedly, each binding is evaluated using the current contents of each other binding, and is refreshed to contain the new evaluation. This process continues through all bindings, and repeats as long as changes continue to occur.

The resulting value of the expression is body evaluated once in the context of the final iterates.

A zero-binding instance can be replaced by body. A single-binding instance is equivalent to MirRelationExpr::Let.

The runtime memory footprint of this operator is zero.

§

Project

Fields

§input: Box<MirRelationExpr>

The source collection.

§outputs: Vec<usize>

Indices of columns to retain.

Project out some columns from a dataflow

The runtime memory footprint of this operator is zero.

§

Map

Fields

§input: Box<MirRelationExpr>

The source collection.

§scalars: Vec<MirScalarExpr>

Expressions which determine values to append to each row. An expression may refer to columns in input or expressions defined earlier in the vector

Append new columns to a dataflow

The runtime memory footprint of this operator is zero.

§

FlatMap

Fields

§input: Box<MirRelationExpr>

The source collection

§func: TableFunc

The table func to apply

§exprs: Vec<MirScalarExpr>

The argument to the table func

Like Map, but yields zero-or-more output rows per input row

The runtime memory footprint of this operator is zero.

§

Filter

Fields

§input: Box<MirRelationExpr>

The source collection.

§predicates: Vec<MirScalarExpr>

Predicates, each of which must be true.

Keep rows from a dataflow where all the predicates are true

The runtime memory footprint of this operator is zero.

§

Join

Fields

§inputs: Vec<MirRelationExpr>

A sequence of input relations.

§equivalences: Vec<Vec<MirScalarExpr>>

A sequence of equivalence classes of expressions on the cross product of inputs.

Each equivalence class is a list of scalar expressions, where for each class the intended interpretation is that all evaluated expressions should be equal.

Each scalar expression is to be evaluated over the cross-product of all records from all inputs. In many cases this may just be column selection from specific inputs, but more general cases exist (e.g. complex functions of multiple columns from multiple inputs, or just constant literals).

§implementation: JoinImplementation

Join implementation information.

Join several collections, where some columns must be equal.

For further details consult the documentation for MirRelationExpr::join.

The runtime memory footprint of this operator can be proportional to the sizes of all inputs and the size of all joins of prefixes. This may be reduced due to arrangements available at rendering time.

§

Reduce

Fields

§input: Box<MirRelationExpr>

The source collection.

§group_key: Vec<MirScalarExpr>

Column indices used to form groups.

§aggregates: Vec<AggregateExpr>

Expressions which determine values to append to each row, after the group keys.

§monotonic: bool

True iff the input is known to monotonically increase (only addition of records).

§expected_group_size: Option<u64>

User hint: expected number of values per group key. Used to optimize physical rendering.

Group a dataflow by some columns and aggregate over each group

The runtime memory footprint of this operator is at most proportional to the number of distinct records in the input and output. The actual requirements can be less: the number of distinct inputs to each aggregate, summed across each aggregate, plus the output size. For more details consult the code that builds the associated dataflow.

§

TopK

Fields

§input: Box<MirRelationExpr>

The source collection.

§group_key: Vec<usize>

Column indices used to form groups.

§order_key: Vec<ColumnOrder>

Column indices used to order rows within groups.

§limit: Option<MirScalarExpr>

Number of records to retain

§offset: usize

Number of records to skip

§monotonic: bool

True iff the input is known to monotonically increase (only addition of records).

§expected_group_size: Option<u64>

User-supplied hint: how many rows will have the same group key.

Groups and orders within each group, limiting output.

The runtime memory footprint of this operator is proportional to its input and output.

§

Negate

Fields

§input: Box<MirRelationExpr>

The source collection.

Return a dataflow where the row counts are negated

The runtime memory footprint of this operator is zero.

§

Threshold

Fields

§input: Box<MirRelationExpr>

The source collection.

Keep rows from a dataflow where the row counts are positive

The runtime memory footprint of this operator is proportional to its input and output.

§

Union

Fields

§base: Box<MirRelationExpr>

A source collection.

§inputs: Vec<MirRelationExpr>

Source collections to union.

Adds the frequencies of elements in contained sets.

The runtime memory footprint of this operator is zero.

§

ArrangeBy

Fields

§input: Box<MirRelationExpr>

The source collection

§keys: Vec<Vec<MirScalarExpr>>

Columns to arrange input by, in order of decreasing primacy

Technically a no-op. Used to render an index. Will be used to optimize queries on finer grain. Each keys item represents a different index that should be produced from the keys.

The runtime memory footprint of this operator is proportional to its input.

Implementations§

source§

impl MirRelationExpr

source

pub fn typ(&self) -> RelationType

Reports the schema of the relation.

This method determines the type through recursive traversal of the relation expression, drawing from the types of base collections. As such, this is not an especially cheap method, and should be used judiciously.

The relation type is computed incrementally with a recursive post-order traversal, that accumulates the input types for the relations yet to be visited in type_stack.

source

pub fn typ_with_input_types(&self, input_types: &[RelationType]) -> RelationType

Reports the schema of the relation given the schema of the input relations.

input_types is required to contain the schemas for the input relations of the current relation in the same order as they are visited by try_visit_children method, even though not all may be used for computing the schema of the current relation. For example, Let expects two input types, one for the value relation and one for the body, in that order, but only the one for the body is used to determine the type of the Let relation.

It is meant to be used during post-order traversals to compute relation schemas incrementally.

source

pub fn col_with_input_cols<'a, I>(&self, input_types: I) -> Vec<ColumnType>
where I: Iterator<Item = &'a Vec<ColumnType>>,

Reports the column types of the relation given the column types of the input relations.

This method delegates to try_col_with_input_cols, panicing if an Err variant is returned.

source

pub fn try_col_with_input_cols<'a, I>( &self, input_types: I ) -> Result<Vec<ColumnType>, String>
where I: Iterator<Item = &'a Vec<ColumnType>>,

Reports the column types of the relation given the column types of the input relations.

input_types is required to contain the column types for the input relations of the current relation in the same order as they are visited by try_visit_children method, even though not all may be used for computing the schema of the current relation. For example, Let expects two input types, one for the value relation and one for the body, in that order, but only the one for the body is used to determine the type of the Let relation.

It is meant to be used during post-order traversals to compute column types incrementally.

source

pub fn keys_with_input_keys<'a, I, J>( &self, input_arities: I, input_keys: J ) -> Vec<Vec<usize>>
where I: Iterator<Item = usize>, J: Iterator<Item = &'a Vec<Vec<usize>>>,

Reports the unique keys of the relation given the arities and the unique keys of the input relations.

input_arities and input_keys are required to contain the corresponding info for the input relations of the current relation in the same order as they are visited by try_visit_children method, even though not all may be used for computing the schema of the current relation. For example, Let expects two input types, one for the value relation and one for the body, in that order, but only the one for the body is used to determine the type of the Let relation.

It is meant to be used during post-order traversals to compute unique keys incrementally.

source

pub fn arity(&self) -> usize

The number of columns in the relation.

This number is determined from the type, which is determined recursively at non-trivial cost.

The arity is computed incrementally with a recursive post-order traversal, that accumulates the arities for the relations yet to be visited in arity_stack.

source

pub fn arity_with_input_arities<I>(&self, input_arities: I) -> usize
where I: Iterator<Item = usize>,

Reports the arity of the relation given the schema of the input relations.

input_arities is required to contain the arities for the input relations of the current relation in the same order as they are visited by try_visit_children method, even though not all may be used for computing the schema of the current relation. For example, Let expects two input types, one for the value relation and one for the body, in that order, but only the one for the body is used to determine the type of the Let relation.

It is meant to be used during post-order traversals to compute arities incrementally.

source

pub fn num_inputs(&self) -> usize

The number of child relations this relation has.

source

pub fn constant(rows: Vec<Vec<Datum<'_>>>, typ: RelationType) -> Self

Constructs a constant collection from specific rows and schema, where each row will have a multiplicity of one.

source

pub fn constant_diff( rows: Vec<(Vec<Datum<'_>>, Diff)>, typ: RelationType ) -> Self

Constructs a constant collection from specific rows and schema, where each row can have an arbitrary multiplicity.

source

pub fn as_const( &self ) -> Option<(&Result<Vec<(Row, Diff)>, EvalError>, &RelationType)>

If self is a constant, return the value and the type, otherwise None. Looks behind ArrangeBys.

source

pub fn as_const_mut( &mut self ) -> Option<(&mut Result<Vec<(Row, Diff)>, EvalError>, &mut RelationType)>

If self is a constant, mutably return the value and the type, otherwise None. Looks behind ArrangeBys.

source

pub fn as_const_err(&self) -> Option<&EvalError>

If self is a constant error, return the error, otherwise None. Looks behind ArrangeBys.

source

pub fn is_constant_singleton(&self) -> bool

Checks if self is the single element collection with no columns.

source

pub fn local_get(id: LocalId, typ: RelationType) -> Self

Constructs the expression for getting a local collection.

source

pub fn global_get(id: GlobalId, typ: RelationType) -> Self

Constructs the expression for getting a global collection

source

pub fn project(self, outputs: Vec<usize>) -> Self

Retains only the columns specified by output.

source

pub fn map(self, scalars: Vec<MirScalarExpr>) -> Self

Append to each row the results of applying elements of scalar.

source

pub fn map_one(self, scalar: MirScalarExpr) -> Self

Append to each row a single scalar.

source

pub fn flat_map(self, func: TableFunc, exprs: Vec<MirScalarExpr>) -> Self

Like map, but yields zero-or-more output rows per input row

source

pub fn filter<I>(self, predicates: I) -> Self
where I: IntoIterator<Item = MirScalarExpr>,

Retain only the rows satisfying each of several predicates.

source

pub fn product(self, right: Self) -> Self

Form the Cartesian outer-product of rows in both inputs.

source

pub fn join( inputs: Vec<MirRelationExpr>, variables: Vec<Vec<(usize, usize)>> ) -> Self

Performs a relational equijoin among the input collections.

The sequence inputs each describe different input collections, and the sequence variables describes equality constraints that some of their columns must satisfy. Each element in variable describes a set of pairs (input_index, column_index) where every value described by that set must be equal.

For example, the pair (input, column) indexes into inputs[input][column], extracting the inputth input collection and for each row examining its columnth column.

Example
use mz_repr::{Datum, ColumnType, RelationType, ScalarType};
use mz_expr::MirRelationExpr;

// A common schema for each input.
let schema = RelationType::new(vec![
    ScalarType::Int32.nullable(false),
    ScalarType::Int32.nullable(false),
]);

// the specific data are not important here.
let data = vec![Datum::Int32(0), Datum::Int32(1)];

// Three collections that could have been different.
let input0 = MirRelationExpr::constant(vec![data.clone()], schema.clone());
let input1 = MirRelationExpr::constant(vec![data.clone()], schema.clone());
let input2 = MirRelationExpr::constant(vec![data.clone()], schema.clone());

// Join the three relations looking for triangles, like so.
//
//     Output(A,B,C) := Input0(A,B), Input1(B,C), Input2(A,C)
let joined = MirRelationExpr::join(
    vec![input0, input1, input2],
    vec![
        vec![(0,0), (2,0)], // fields A of inputs 0 and 2.
        vec![(0,1), (1,0)], // fields B of inputs 0 and 1.
        vec![(1,1), (2,1)], // fields C of inputs 1 and 2.
    ],
);

// Technically the above produces `Output(A,B,B,C,A,C)` because the columns are concatenated.
// A projection resolves this and produces the correct output.
let result = joined.project(vec![0, 1, 3]);
source

pub fn join_scalars( inputs: Vec<MirRelationExpr>, equivalences: Vec<Vec<MirScalarExpr>> ) -> Self

Constructs a join operator from inputs and required-equal scalar expressions.

source

pub fn reduce( self, group_key: Vec<usize>, aggregates: Vec<AggregateExpr>, expected_group_size: Option<u64> ) -> Self

Perform a key-wise reduction / aggregation.

The group_key argument indicates columns in the input collection that should be grouped, and aggregates lists aggregation functions each of which produces one output column in addition to the keys.

source

pub fn top_k( self, group_key: Vec<usize>, order_key: Vec<ColumnOrder>, limit: Option<MirScalarExpr>, offset: usize, expected_group_size: Option<u64> ) -> Self

Perform a key-wise reduction order by and limit.

The group_key argument indicates columns in the input collection that should be grouped, the order_key argument indicates columns that should be further used to order records within groups, and the limit argument constrains the total number of records that should be produced in each group.

source

pub fn negate(self) -> Self

Negates the occurrences of each row.

source

pub fn distinct(self) -> Self

Removes all but the first occurrence of each row.

source

pub fn distinct_by(self, group_key: Vec<usize>) -> Self

Removes all but the first occurrence of each key. Columns not included in the group_key are discarded.

source

pub fn threshold(self) -> Self

Discards rows with a negative frequency.

source

pub fn union_many(inputs: Vec<Self>, typ: RelationType) -> Self

Unions together any number inputs.

If inputs is empty, then an empty relation of type typ is constructed.

source

pub fn union(self, other: Self) -> Self

Produces one collection where each row is present with the sum of its frequencies in each input.

source

pub fn arrange_by(self, keys: &[Vec<MirScalarExpr>]) -> Self

Arranges the collection by the specified columns

source

pub fn is_empty(&self) -> bool

Indicates if this is a constant empty collection.

A false value does not mean the collection is known to be non-empty, only that we cannot currently determine that it is statically empty.

source

pub fn is_negated_project(&self) -> Option<(&MirRelationExpr, &[usize])>

If the expression is a negated project, return the input and the projection.

source

pub fn pretty(&self) -> String

Pretty-print this MirRelationExpr to a string.

source

pub fn explain( &self, config: &ExplainConfig, humanizer: Option<&dyn ExprHumanizer> ) -> String

Pretty-print this MirRelationExpr to a string using a custom ExplainConfig and an optionally provided ExprHumanizer.

source

pub fn take_safely(&mut self) -> MirRelationExpr

Take ownership of self, leaving an empty MirRelationExpr::Constant with the correct type.

source

pub fn take_dangerous(&mut self) -> MirRelationExpr

Take ownership of self, leaving an empty MirRelationExpr::Constant with an incorrect type.

This should only be used if self is about to be dropped or otherwise overwritten.

source

pub fn replace_using<F>(&mut self, logic: F)

Replaces self with some logic applied to self.

source

pub fn let_in<Body>(self, id_gen: &mut IdGen, body: Body) -> MirRelationExpr

Store self in a Let and pass the corresponding Get to body

source

pub fn let_in_fallible<Body, E>( self, id_gen: &mut IdGen, body: Body ) -> Result<MirRelationExpr, E>

Like MirRelationExpr::let_in, but with a fallible return type.

source

pub fn anti_lookup( self, id_gen: &mut IdGen, keys_and_values: MirRelationExpr, default: Vec<(Datum<'_>, ScalarType)> ) -> MirRelationExpr

Return every row in self that does not have a matching row in the first columns of keys_and_values, using default to fill in the remaining columns (If default is a row of nulls, this is the ‘outer’ part of LEFT OUTER JOIN)

source

pub fn lookup( self, id_gen: &mut IdGen, keys_and_values: MirRelationExpr, default: Vec<(Datum<'static>, ScalarType)> ) -> MirRelationExpr

Return:

  • every row in keys_and_values
  • every row in self that does not have a matching row in the first columns of keys_and_values, using default to fill in the remaining columns (This is LEFT OUTER JOIN if:
  1. default is a row of null
  2. matching rows in keys_and_values and self have the same multiplicity.)
source

pub fn contains_temporal(&self) -> bool

True iff the expression contains a NullaryFunc::MzLogicalTimestamp.

source

pub fn try_visit_scalars_mut1<F, E>(&mut self, f: &mut F) -> Result<(), E>
where F: FnMut(&mut MirScalarExpr) -> Result<(), E>,

Fallible visitor for the MirScalarExprs directly owned by this relation expression.

The f visitor should not recursively descend into owned MirRelationExprs.

source

pub fn try_visit_scalars_mut<F, E>(&mut self, f: &mut F) -> Result<(), E>
where F: FnMut(&mut MirScalarExpr) -> Result<(), E>, E: From<RecursionLimitError>,

Fallible mutable visitor for the MirScalarExprs in the MirRelationExpr subtree rooted at self.

Note that this does not recurse into MirRelationExpr subtrees within MirScalarExpr nodes.

source

pub fn visit_scalars_mut<F>(&mut self, f: &mut F)
where F: FnMut(&mut MirScalarExpr),

Infallible mutable visitor for the MirScalarExprs in the MirRelationExpr subtree rooted at at self.

Note that this does not recurse into MirRelationExpr subtrees within MirScalarExpr nodes.

source

pub fn try_visit_scalars_1<F, E>(&self, f: &mut F) -> Result<(), E>
where F: FnMut(&MirScalarExpr) -> Result<(), E>,

Fallible visitor for the MirScalarExprs directly owned by this relation expression.

The f visitor should not recursively descend into owned MirRelationExprs.

source

pub fn try_visit_scalars<F, E>(&self, f: &mut F) -> Result<(), E>

Fallible immutable visitor for the MirScalarExprs in the MirRelationExpr subtree rooted at self.

Note that this does not recurse into MirRelationExpr subtrees within MirScalarExpr nodes.

source

pub fn visit_scalars<F>(&self, f: &mut F)
where F: FnMut(&MirScalarExpr),

Infallible immutable visitor for the MirScalarExprs in the MirRelationExpr subtree rooted at at self.

Note that this does not recurse into MirRelationExpr subtrees within MirScalarExpr nodes.

source

pub fn destroy_carefully(&mut self)

Clears the contents of self even if it’s so deep that simply dropping it would cause a stack overflow in drop_in_place.

Leaves self in an unusable state, so this should only be used if self is about to be dropped or otherwise overwritten.

source

pub fn debug_size_and_depth(&self) -> (usize, usize)

Computes the size (total number of nodes) and maximum depth of a MirRelationExpr for debug printing purposes.

source

pub fn could_run_expensive_function(&self) -> bool

The MirRelationExpr is considered potentially expensive if and only if at least one of the following conditions is true:

  • It contains at least one FlatMap or a Reduce operator.
  • It contains at least one MirScalarExpr with a function call.

!!!WARNING!!!: this method has an HirRelationExpr counterpart. The two should be kept in sync w.r.t. HIR ⇒ MIR lowering!

source§

impl MirRelationExpr

source

pub fn is_recursive(self: &MirRelationExpr) -> bool

True when expr contains a LetRec AST node.

source

pub fn size(&self) -> usize

Return the number of sub-expressions in the tree (including self).

source

pub fn recursive_ids( ids: &[LocalId], values: &[MirRelationExpr] ) -> BTreeSet<LocalId>

Given the ids and values of a LetRec, it computes the subset of ids that are used across iterations. These are those ids that have a reference before they are defined, when reading all the bindings in order.

For example:

WITH MUTUALLY RECURSIVE
    x(...) AS f(z),
    y(...) AS g(x),
    z(...) AS h(y)
...;

Here, only z is returned, because x and y are referenced only within the same iteration.

Note that if a binding references itself, that is also returned.

source

pub fn make_nonrecursive(self: &mut MirRelationExpr)

Replaces LetRec nodes with a stack of Let nodes.

In each Let binding, uses of Get in value that are not at strictly greater identifiers are rewritten to be the constant collection. This makes the computation perform exactly “one” iteration.

This was used only temporarily while developing LetRec.

source

pub fn collect_expirations( id: LocalId, expr: &MirRelationExpr, expire_whens: &mut BTreeMap<LocalId, Vec<LocalId>> )

For each Id id' referenced in expr, if it is larger or equal than id, then record in expire_whens that when id' is redefined, then we should expire the information that we are holding about id. Call do_expirations with expire_whens at each Id redefinition.

IMPORTANT: Relies on the numbering of Ids to be what renumber_bindings gives.

source

pub fn do_expirations<I>( redefined_id: LocalId, expire_whens: &mut BTreeMap<LocalId, Vec<LocalId>>, id_infos: &mut BTreeMap<LocalId, I> ) -> Vec<(LocalId, I)>

Call this function when id is redefined. It modifies id_infos by removing information about such Ids whose information depended on the earlier definition of id, according to expire_whens. Also modifies expire_whens: it removes the currently processed entry.

source§

impl MirRelationExpr

source

pub fn children(&self) -> impl DoubleEndedIterator<Item = &Self>

Iterates through references to child expressions.

source

pub fn children_mut(&mut self) -> impl DoubleEndedIterator<Item = &mut Self>

Iterates through mutable references to child expressions.

source

pub fn visit_pre<'a, F: FnMut(&'a Self)>(&'a self, f: F)

Iterative pre-order visitor.

source

pub fn visit_pre_mut<F: FnMut(&mut Self)>(&mut self, f: F)

Iterative pre-order visitor.

source

pub fn post_order_vec(&self) -> Vec<&Self>

Return a vector of references to the subtrees of this expression in post-visit order (the last element is &self).

source§

impl MirRelationExpr

source

pub fn fmt_indexed_filter<'a, T>( f: &mut Formatter<'_>, ctx: &mut PlanRenderingContext<'a, T>, coll_id: &GlobalId, idx_id: &GlobalId, constants: Option<Vec<Row>>, cse_id: Option<&LocalId> ) -> Result

Trait Implementations§

source§

impl Clone for MirRelationExpr

source§

fn clone(&self) -> MirRelationExpr

Returns a copy of the value. Read more
1.0.0 · source§

fn clone_from(&mut self, source: &Self)

Performs copy-assignment from source. Read more
source§

impl CollectionPlan for MirRelationExpr

source§

fn depends_on_into(&self, out: &mut BTreeSet<GlobalId>)

Collects the set of global identifiers from dataflows referenced in Get.
source§

fn depends_on(&self) -> BTreeSet<GlobalId>

Returns the set of global identifiers from dataflows referenced in Get. Read more
source§

impl Debug for MirRelationExpr

source§

fn fmt(&self, f: &mut Formatter<'_>) -> Result

Formats the value using the given formatter. Read more
source§

impl<'de> Deserialize<'de> for MirRelationExpr

source§

fn deserialize<__D>(__deserializer: __D) -> Result<Self, __D::Error>
where __D: Deserializer<'de>,

Deserialize this value from the given Serde deserializer. Read more
source§

impl DisplayText<PlanRenderingContext<'_, MirRelationExpr>> for MirRelationExpr

EXPLAIN ... AS TEXT support for MirRelationExpr.

The format adheres to the following conventions:

  1. In general, every line corresponds to an MirRelationExpr node in the plan.
  2. Non-recursive parameters of each sub-plan are written as $key=$val pairs on the same line.
  3. A single non-recursive parameter can be written just as $val.
  4. Exceptions in (1) can be made when virtual syntax is requested (done by default, can be turned off with WITH(raw_syntax)).
  5. Exceptions in (2) can be made when join implementations are rendered explicitly WITH(join_impls).
source§

fn fmt_text( &self, f: &mut Formatter<'_>, ctx: &mut PlanRenderingContext<'_, MirRelationExpr> ) -> Result

source§

impl<'a> Explain<'a> for MirRelationExpr

§

type Context = ExplainContext<'a>

The type of the immutable context in which the explanation will be rendered.
§

type Text = ExplainSinglePlan<'a, MirRelationExpr>

The explanation type produced by a successful Explain::explain_text call.
§

type Json = ExplainSinglePlan<'a, MirRelationExpr>

The explanation type produced by a successful Explain::explain_json call.
§

type Dot = UnsupportedFormat

The explanation type produced by a successful Explain::explain_json call.
source§

fn explain_text( &'a mut self, context: &'a Self::Context ) -> Result<Self::Text, ExplainError>

Construct a Result::Ok of the Explain::Text format from the config and the context. Read more
source§

fn explain_json( &'a mut self, context: &'a Self::Context ) -> Result<Self::Json, ExplainError>

Construct a Result::Ok of the Explain::Json format from the config and the context. Read more
source§

fn explain( &'a mut self, format: &'a ExplainFormat, context: &'a Self::Context ) -> Result<String, ExplainError>

Explain an instance of Self within the given Explain::Context. Read more
source§

fn explain_dot( &'a mut self, context: &'a Self::Context ) -> Result<Self::Dot, ExplainError>

Construct a Result::Ok of the Explain::Dot format from the config and the context. Read more
source§

impl Hash for MirRelationExpr

source§

fn hash<__H: Hasher>(&self, state: &mut __H)

Feeds this value into the given Hasher. Read more
1.3.0 · source§

fn hash_slice<H>(data: &[Self], state: &mut H)
where H: Hasher, Self: Sized,

Feeds a slice of this type into the given Hasher. Read more
source§

impl MzReflect for MirRelationExpr

source§

fn add_to_reflected_type_info(rti: &mut ReflectedTypeInfo)

Adds names and types of the fields of the struct or enum to rti. Read more
source§

impl Ord for MirRelationExpr

source§

fn cmp(&self, other: &MirRelationExpr) -> Ordering

This method returns an Ordering between self and other. Read more
1.21.0 · source§

fn max(self, other: Self) -> Self
where Self: Sized,

Compares and returns the maximum of two values. Read more
1.21.0 · source§

fn min(self, other: Self) -> Self
where Self: Sized,

Compares and returns the minimum of two values. Read more
1.50.0 · source§

fn clamp(self, min: Self, max: Self) -> Self
where Self: Sized + PartialOrd,

Restrict a value to a certain interval. Read more
source§

impl PartialEq for MirRelationExpr

source§

fn eq(&self, other: &MirRelationExpr) -> bool

This method tests for self and other values to be equal, and is used by ==.
1.0.0 · source§

fn ne(&self, other: &Rhs) -> bool

This method tests for !=. The default implementation is almost always sufficient, and should not be overridden without very good reason.
source§

impl PartialOrd for MirRelationExpr

source§

fn partial_cmp(&self, other: &MirRelationExpr) -> Option<Ordering>

This method returns an ordering between self and other values if one exists. Read more
1.0.0 · source§

fn lt(&self, other: &Rhs) -> bool

This method tests less than (for self and other) and is used by the < operator. Read more
1.0.0 · source§

fn le(&self, other: &Rhs) -> bool

This method tests less than or equal to (for self and other) and is used by the <= operator. Read more
1.0.0 · source§

fn gt(&self, other: &Rhs) -> bool

This method tests greater than (for self and other) and is used by the > operator. Read more
1.0.0 · source§

fn ge(&self, other: &Rhs) -> bool

This method tests greater than or equal to (for self and other) and is used by the >= operator. Read more
source§

impl Serialize for MirRelationExpr

source§

fn serialize<__S>(&self, __serializer: __S) -> Result<__S::Ok, __S::Error>
where __S: Serializer,

Serialize this value into the given Serde serializer. Read more
source§

impl VisitChildren<MirRelationExpr> for MirRelationExpr

source§

fn visit_children<F>(&self, f: F)
where F: FnMut(&Self),

Apply an infallible immutable function f to each direct child.
source§

fn visit_mut_children<F>(&mut self, f: F)
where F: FnMut(&mut Self),

Apply an infallible mutable function f to each direct child.
source§

fn try_visit_children<F, E>(&self, f: F) -> Result<(), E>
where F: FnMut(&Self) -> Result<(), E>, E: From<RecursionLimitError>,

Apply a fallible immutable function f to each direct child. Read more
source§

fn try_visit_mut_children<F, E>(&mut self, f: F) -> Result<(), E>

Apply a fallible mutable function f to each direct child. Read more
source§

impl Eq for MirRelationExpr

source§

impl StructuralEq for MirRelationExpr

source§

impl StructuralPartialEq for MirRelationExpr

Auto Trait Implementations§

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<Q, K> Comparable<K> for Q
where Q: Ord + ?Sized, K: Borrow<Q> + ?Sized,

source§

fn compare(&self, key: &K) -> Ordering

Compare self to key and return their ordering.
source§

impl<T> DynClone for T
where T: Clone,

source§

fn __clone_box(&self, _: Private) -> *mut ()

source§

impl<Q, K> Equivalent<K> for Q
where Q: Eq + ?Sized, K: Borrow<Q> + ?Sized,

source§

fn equivalent(&self, key: &K) -> bool

Compare self to key and return true if they are equal.
source§

impl<Q, K> Equivalent<K> for Q
where Q: Eq + ?Sized, K: Borrow<Q> + ?Sized,

source§

fn equivalent(&self, key: &K) -> bool

Checks if this value is equivalent to the given key. Read more
source§

impl<Q, K> Equivalent<K> for Q
where Q: Eq + ?Sized, K: Borrow<Q> + ?Sized,

source§

fn equivalent(&self, key: &K) -> bool

Compare self to key and return true if they are equal.
source§

impl<T> From<T> for T

source§

fn from(t: T) -> T

Returns the argument unchanged.

source§

impl<T> FromRef<T> for T
where T: Clone,

source§

fn from_ref(input: &T) -> T

Converts to this type from a reference to the input type.
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> Hashable for T
where T: Hash,

§

type Output = u64

The type of the output value.
source§

fn hashed(&self) -> u64

A well-distributed integer derived from the data.
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<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<T> PreferredContainer for T
where T: Ord + Clone + 'static,

§

type Container = Vec<T>

The preferred container for the type.
source§

impl<T> ProgressEventTimestamp for T
where T: Data + Debug + Any,

source§

fn as_any(&self) -> &(dyn Any + 'static)

Upcasts this ProgressEventTimestamp to Any. Read more
source§

fn type_name(&self) -> &'static str

Returns the name of the concrete type of this object. Read more
source§

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

source§

impl<T> PushInto<Vec<T>> for T

source§

fn push_into(self, target: &mut Vec<T>)

Push self into the target container.
source§

impl<T> Same for T

§

type Output = T

Should always be Self
source§

impl<T> ToOwned for T
where T: Clone,

§

type Owned = T

The resulting type after obtaining ownership.
source§

fn to_owned(&self) -> T

Creates owned data from borrowed data, usually by cloning. Read more
source§

fn clone_into(&self, target: &mut T)

Uses borrowed data to replace owned data, usually by cloning. 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> Visit for T
where T: VisitChildren<T>,

source§

fn visit_post<F>(&self, f: &mut F) -> Result<(), RecursionLimitError>
where F: FnMut(&T),

Post-order immutable infallible visitor for self.
source§

fn visit_post_nolimit<F>(&self, f: &mut F)
where F: FnMut(&T),

👎Deprecated: Use visit_post instead.
Post-order immutable infallible visitor for self. Does not enforce a recursion limit.
source§

fn visit_mut_post<F>(&mut self, f: &mut F) -> Result<(), RecursionLimitError>
where F: FnMut(&mut T),

Post-order mutable infallible visitor for self.
source§

fn visit_mut_post_nolimit<F>(&mut self, f: &mut F)
where F: FnMut(&mut T),

👎Deprecated: Use visit_mut_post instead.
Post-order mutable infallible visitor for self. Does not enforce a recursion limit.
source§

fn try_visit_post<F, E>(&self, f: &mut F) -> Result<(), E>
where F: FnMut(&T) -> Result<(), E>, E: From<RecursionLimitError>,

Post-order immutable fallible visitor for self.
source§

fn try_visit_mut_post<F, E>(&mut self, f: &mut F) -> Result<(), E>
where F: FnMut(&mut T) -> Result<(), E>, E: From<RecursionLimitError>,

Post-order mutable fallible visitor for self.
source§

fn visit_pre<F>(&self, f: &mut F) -> Result<(), RecursionLimitError>
where F: FnMut(&T),

Pre-order immutable infallible visitor for self.
source§

fn visit_pre_with_context<Context, AccFun, Visitor>( &self, init: Context, acc_fun: &mut AccFun, visitor: &mut Visitor ) -> Result<(), RecursionLimitError>
where Context: Clone, AccFun: FnMut(Context, &T) -> Context, Visitor: FnMut(&Context, &T),

Pre-order immutable infallible visitor for self, which also accumulates context information along the path from the root to the current node’s parent. acc_fun is a similar closure as in fold. The accumulated context is passed to the visitor, along with the current node. Read more
source§

fn visit_pre_nolimit<F>(&self, f: &mut F)
where F: FnMut(&T),

👎Deprecated: Use visit_pre instead.
Pre-order immutable infallible visitor for self. Does not enforce a recursion limit.
source§

fn visit_mut_pre<F>(&mut self, f: &mut F) -> Result<(), RecursionLimitError>
where F: FnMut(&mut T),

Pre-order mutable infallible visitor for self.
source§

fn visit_mut_pre_nolimit<F>(&mut self, f: &mut F)
where F: FnMut(&mut T),

👎Deprecated: Use visit_mut_pre instead.
Pre-order mutable infallible visitor for self. Does not enforce a recursion limit.
source§

fn try_visit_pre<F, E>(&self, f: &mut F) -> Result<(), E>
where F: FnMut(&T) -> Result<(), E>, E: From<RecursionLimitError>,

Pre-order immutable fallible visitor for self.
source§

fn try_visit_mut_pre<F, E>(&mut self, f: &mut F) -> Result<(), E>
where F: FnMut(&mut T) -> Result<(), E>, E: From<RecursionLimitError>,

Pre-order mutable fallible visitor for self.
source§

fn visit_pre_post<F1, F2>( &self, pre: &mut F1, post: &mut F2 ) -> Result<(), RecursionLimitError>
where F1: FnMut(&T) -> Option<Vec<&T>>, F2: FnMut(&T),

source§

fn visit_pre_post_nolimit<F1, F2>(&self, pre: &mut F1, post: &mut F2)
where F1: FnMut(&T) -> Option<Vec<&T>>, F2: FnMut(&T),

👎Deprecated: Use visit instead.
A generalization of Visit::visit_pre and Visit::visit_post. Does not enforce a recursion limit. Read more
source§

fn visit_mut_pre_post<F1, F2>( &mut self, pre: &mut F1, post: &mut F2 ) -> Result<(), RecursionLimitError>
where F1: FnMut(&mut T) -> Option<Vec<&mut T>>, F2: FnMut(&mut T),

👎Deprecated: Use visit_mut instead.
source§

fn visit_mut_pre_post_nolimit<F1, F2>(&mut self, pre: &mut F1, post: &mut F2)
where F1: FnMut(&mut T) -> Option<Vec<&mut T>>, F2: FnMut(&mut T),

👎Deprecated: Use visit_mut_pre_post instead.
A generalization of Visit::visit_mut_pre and Visit::visit_mut_post. Does not enforce a recursion limit. Read more
source§

fn visit<V>(&self, visitor: &mut V) -> Result<(), RecursionLimitError>
where V: Visitor<T>,

source§

fn visit_mut<V>(&mut self, visitor: &mut V) -> Result<(), RecursionLimitError>
where V: VisitorMut<T>,

source§

fn try_visit<V, E>(&self, visitor: &mut V) -> Result<(), E>
where V: TryVisitor<T, E>, E: From<RecursionLimitError>,

source§

fn try_visit_mut<V, E>(&mut self, visitor: &mut V) -> Result<(), E>

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

impl<T> Data for T
where T: Clone + 'static,

source§

impl<T> Data for T
where T: Send + Sync + Any + Serialize + for<'a> Deserialize<'a> + 'static,

source§

impl<T> Data for T
where T: Data + Ord + Debug,

source§

impl<T> DeserializeOwned for T
where T: for<'de> Deserialize<'de>,

source§

impl<T> ExchangeData for T
where T: Data + Data,

source§

impl<T> ExchangeData for T
where T: ExchangeData + Ord + Debug,

source§

impl<T> Sequence for T
where T: Eq + Hash,