From a551f90b2f3fc139811c56ce83d073331f3b5fde Mon Sep 17 00:00:00 2001 From: Will Deng Date: Wed, 15 Nov 2023 02:45:55 -0500 Subject: [PATCH 1/6] added missing method to MetricAccessor protocol --- metricflow/model/semantics/metric_lookup.py | 11 +++-------- metricflow/protocols/semantics.py | 13 ++++++++++++- 2 files changed, 15 insertions(+), 9 deletions(-) diff --git a/metricflow/model/semantics/metric_lookup.py b/metricflow/model/semantics/metric_lookup.py index c8eb15e40e..7b5c125e22 100644 --- a/metricflow/model/semantics/metric_lookup.py +++ b/metricflow/model/semantics/metric_lookup.py @@ -104,14 +104,9 @@ def add_metric(self, metric: Metric) -> None: ) self._metrics[metric_reference] = metric - def configured_input_measure_for_metric(self, metric_reference: MetricReference) -> Optional[MetricInputMeasure]: - """Get input measure defined in the original metric config, if exists. - - When SemanticModel is constructed, input measures from input metrics are added to the list of input measures - for a metric. Here, use rules about metric types to determine which input measures were defined in the config: - - Simple & cumulative metrics require one input measure, and can't take any input metrics. - - Derived & ratio metrics take no input measures, only input metrics. - """ + def configured_input_measure_for_metric( # noqa: D + self, metric_reference: MetricReference + ) -> Optional[MetricInputMeasure]: metric = self.get_metric(metric_reference=metric_reference) if metric.type is MetricType.CUMULATIVE or metric.type is MetricType.SIMPLE: assert len(metric.input_measures) == 1, "Simple and cumulative metrics should have one input measure." diff --git a/metricflow/protocols/semantics.py b/metricflow/protocols/semantics.py index 72ae8c4c78..dc52dde60f 100644 --- a/metricflow/protocols/semantics.py +++ b/metricflow/protocols/semantics.py @@ -14,7 +14,7 @@ from dbt_semantic_interfaces.protocols.dimension import Dimension from dbt_semantic_interfaces.protocols.entity import Entity from dbt_semantic_interfaces.protocols.measure import Measure -from dbt_semantic_interfaces.protocols.metric import Metric +from dbt_semantic_interfaces.protocols.metric import Metric, MetricInputMeasure from dbt_semantic_interfaces.protocols.semantic_model import SemanticModel from dbt_semantic_interfaces.references import ( DimensionReference, @@ -177,3 +177,14 @@ def metric_input_specs_for_metric( ) -> Sequence[MetricSpec]: """Returns the metric input specs required to compute the metric.""" raise NotImplementedError + + @abstractmethod + def configured_input_measure_for_metric(self, metric_reference: MetricReference) -> Optional[MetricInputMeasure]: + """Get input measure defined in the original metric config, if exists. + + When SemanticModel is constructed, input measures from input metrics are added to the list of input measures + for a metric. Here, use rules about metric types to determine which input measures were defined in the config: + - Simple & cumulative metrics require one input measure, and can't take any input metrics. + - Derived & ratio metrics take no input measures, only input metrics. + """ + raise NotImplementedError From 4620f14daa15c0f09dbcd07611b6ce3a372c7a45 Mon Sep 17 00:00:00 2001 From: Will Deng Date: Wed, 15 Nov 2023 02:26:47 -0500 Subject: [PATCH 2/6] added fill_nulls_with property to MeasureSpec --- metricflow/plan_conversion/dataflow_to_sql.py | 5 +++ .../plan_conversion/instance_converters.py | 41 +++++++++++++++++++ metricflow/specs/specs.py | 2 + 3 files changed, 48 insertions(+) diff --git a/metricflow/plan_conversion/dataflow_to_sql.py b/metricflow/plan_conversion/dataflow_to_sql.py index f95ee2679b..0dfc8e0341 100644 --- a/metricflow/plan_conversion/dataflow_to_sql.py +++ b/metricflow/plan_conversion/dataflow_to_sql.py @@ -52,6 +52,7 @@ FilterLinkableInstancesWithLeadingLink, RemoveMeasures, RemoveMetrics, + UpdateMeasureFillNullsWith, create_select_columns_for_instance_sets, ) from metricflow.plan_conversion.select_column_gen import ( @@ -484,6 +485,10 @@ def visit_aggregate_measures_node(self, node: AggregateMeasuresNode) -> SqlDataS ChangeAssociatedColumns(self._column_association_resolver) ) + # Add fill null property to corresponding measure spec + aggregated_instance_set = aggregated_instance_set.transform( + UpdateMeasureFillNullsWith(metric_input_measure_specs=node.metric_input_measure_specs) + ) from_data_set_alias = self._next_unique_table_alias() # Convert the instance set into a set of select column statements with updated aliases diff --git a/metricflow/plan_conversion/instance_converters.py b/metricflow/plan_conversion/instance_converters.py index 45a874521c..dfe38b9cd5 100644 --- a/metricflow/plan_conversion/instance_converters.py +++ b/metricflow/plan_conversion/instance_converters.py @@ -586,6 +586,47 @@ def transform(self, instance_set: InstanceSet) -> InstanceSet: # noqa: D ) +class UpdateMeasureFillNullsWith(InstanceSetTransform[InstanceSet]): + """Returns a new instance set where all measures have been assigned the fill nulls with property.""" + + def __init__(self, metric_input_measure_specs: Sequence[MetricInputMeasureSpec]): + """Initializer stores the input specs, which contain the fill_nulls_with for each measure.""" + self.metric_input_measure_specs = metric_input_measure_specs + + def _update_fill_nulls_with(self, measure_instances: Tuple[MeasureInstance, ...]) -> Tuple[MeasureInstance, ...]: + """Update all measure instances with the corresponding fill_nulls_with value.""" + updated_instances: List[MeasureInstance] = [] + for instance in measure_instances: + matches = [spec for spec in self.metric_input_measure_specs if spec.measure_spec == instance.spec] + assert len(matches) == 1, f"Matched with {matches} for measure instance {instance}. " + "We should always have 1 direct match for each measure instance and input measure." + measure_spec = MeasureSpec( + element_name=instance.spec.element_name, + fill_nulls_with=matches[0].fill_nulls_with, + non_additive_dimension_spec=instance.spec.non_additive_dimension_spec, + ) + updated_instances.append( + MeasureInstance( + associated_columns=instance.associated_columns, + spec=measure_spec, + aggregation_state=instance.aggregation_state, + defined_from=instance.defined_from, + ) + ) + + return tuple(updated_instances) + + def transform(self, instance_set: InstanceSet) -> InstanceSet: # noqa: D + return InstanceSet( + measure_instances=self._update_fill_nulls_with(instance_set.measure_instances), + dimension_instances=instance_set.dimension_instances, + time_dimension_instances=instance_set.time_dimension_instances, + entity_instances=instance_set.entity_instances, + metric_instances=instance_set.metric_instances, + metadata_instances=instance_set.metadata_instances, + ) + + class AliasAggregatedMeasures(InstanceSetTransform[InstanceSet]): """Returns a new instance set where all measures have been assigned an alias spec.""" diff --git a/metricflow/specs/specs.py b/metricflow/specs/specs.py index 2c7f95e5ad..5883cba1c3 100644 --- a/metricflow/specs/specs.py +++ b/metricflow/specs/specs.py @@ -397,6 +397,7 @@ def __eq__(self, other: Any) -> bool: # type: ignore[misc] # noqa: D class MeasureSpec(InstanceSpec): # noqa: D element_name: str non_additive_dimension_spec: Optional[NonAdditiveDimensionSpec] = None + fill_nulls_with: Optional[int] = None @staticmethod def from_name(name: str) -> MeasureSpec: @@ -496,6 +497,7 @@ def post_aggregation_spec(self) -> MeasureSpec: return MeasureSpec( element_name=self.alias, non_additive_dimension_spec=self.measure_spec.non_additive_dimension_spec, + fill_nulls_with=self.fill_nulls_with, ) else: return self.measure_spec From 3d0628fc6e6f95cc1ec43b9e424b2a83c8a69a1a Mon Sep 17 00:00:00 2001 From: Will Deng Date: Wed, 15 Nov 2023 02:47:30 -0500 Subject: [PATCH 3/6] added ability to combine aggregated outputs nodes into CombineMetricsNode --- metricflow/plan_conversion/dataflow_to_sql.py | 84 ++++-------------- .../plan_conversion/instance_converters.py | 88 ++++++++++++++++++- 2 files changed, 101 insertions(+), 71 deletions(-) diff --git a/metricflow/plan_conversion/dataflow_to_sql.py b/metricflow/plan_conversion/dataflow_to_sql.py index 0dfc8e0341..1d85af7ad6 100644 --- a/metricflow/plan_conversion/dataflow_to_sql.py +++ b/metricflow/plan_conversion/dataflow_to_sql.py @@ -2,12 +2,11 @@ import logging from collections import OrderedDict -from typing import List, Optional, Sequence, Tuple, Union +from typing import List, Optional, Sequence, Union from dbt_semantic_interfaces.enum_extension import assert_values_exhausted from dbt_semantic_interfaces.protocols.metric import MetricInputMeasure, MetricType -from dbt_semantic_interfaces.references import MetricModelReference, MetricReference -from dbt_semantic_interfaces.type_enums.aggregation_type import AggregationType +from dbt_semantic_interfaces.references import MetricModelReference from metricflow.aggregation_properties import AggregationState from metricflow.dag.id_generation import IdGeneratorRegistry @@ -46,6 +45,7 @@ AliasAggregatedMeasures, ChangeAssociatedColumns, ChangeMeasureAggregationState, + CreateSelectColumnForCombineOutputNode, CreateSelectColumnsForInstances, CreateSelectColumnsWithMeasuresAggregated, FilterElements, @@ -786,62 +786,6 @@ def visit_where_constraint_node(self, node: WhereConstraintNode) -> SqlDataSet: ), ) - def _make_select_columns_for_multiple_metrics( - self, - table_alias_to_metric_instances: OrderedDict[str, Tuple[MetricInstance, ...]], - aggregation_type: Optional[AggregationType], - ) -> List[SqlSelectColumn]: - """Creates select columns that get the given metric using the given table alias. - - e.g. - - with table_alias_to_metric_instances = {"a": MetricSpec(element_name="bookings")} - - -> - - a.bookings AS bookings - """ - select_columns = [] - for table_alias, metric_instances in table_alias_to_metric_instances.items(): - for metric_instance in metric_instances: - metric_spec = metric_instance.spec - metric_column_name = self._column_association_resolver.resolve_spec(metric_spec).column_name - column_reference_expression = SqlColumnReferenceExpression( - col_ref=SqlColumnReference( - table_alias=table_alias, - column_name=metric_column_name, - ) - ) - if aggregation_type: - select_expression: SqlExpressionNode = SqlFunctionExpression.build_expression_from_aggregation_type( - aggregation_type=aggregation_type, sql_column_expression=column_reference_expression - ) - else: - select_expression = column_reference_expression - - # At this point, the MetricSpec might have the alias in place of the element name, so we need to look - # back at where it was defined from to get the metric element name. - metric_reference = MetricReference(element_name=metric_instance.defined_from.metric_name) - input_measure = self._metric_lookup.configured_input_measure_for_metric( - metric_reference=metric_reference - ) - if input_measure and input_measure.fill_nulls_with is not None: - select_expression = SqlAggregateFunctionExpression( - sql_function=SqlFunction.COALESCE, - sql_function_args=[ - select_expression, - SqlStringExpression(str(input_measure.fill_nulls_with)), - ], - ) - - select_columns.append( - SqlSelectColumn( - expr=select_expression, - column_alias=metric_column_name, - ) - ) - return select_columns - def visit_combine_metrics_node(self, node: CombineMetricsNode) -> SqlDataSet: """Join computed metric datasets together to return a single dataset containing all metrics. @@ -878,13 +822,13 @@ def visit_combine_metrics_node(self, node: CombineMetricsNode) -> SqlDataSet: ), "Shouldn't have a CombineMetricsNode in the dataflow plan if there's only 1 parent." parent_data_sets: List[AnnotatedSqlDataSet] = [] - table_alias_to_metric_instances: OrderedDict[str, Tuple[MetricInstance, ...]] = OrderedDict() + table_alias_to_instance_set: OrderedDict[str, InstanceSet] = OrderedDict() for parent_node in node.parent_nodes: parent_sql_data_set = parent_node.accept(self) table_alias = self._next_unique_table_alias() parent_data_sets.append(AnnotatedSqlDataSet(data_set=parent_sql_data_set, alias=table_alias)) - table_alias_to_metric_instances[table_alias] = parent_sql_data_set.instance_set.metric_instances + table_alias_to_instance_set[table_alias] = parent_sql_data_set.instance_set # When we create the components of the join that combines metrics it will be one of INNER, FULL OUTER, # or CROSS JOIN. Order doesn't matter for these join types, so we will use the first element in the FROM @@ -924,20 +868,24 @@ def visit_combine_metrics_node(self, node: CombineMetricsNode) -> SqlDataSet: output_instance_set = InstanceSet.merge([x.data_set.instance_set for x in parent_data_sets]) output_instance_set = output_instance_set.transform(ChangeAssociatedColumns(self._column_association_resolver)) - metric_aggregation_type = AggregationType.MAX - metric_select_column_set = SelectColumnSet( - metric_columns=self._make_select_columns_for_multiple_metrics( - table_alias_to_metric_instances=table_alias_to_metric_instances, - aggregation_type=metric_aggregation_type, + aggregated_select_columns = SelectColumnSet() + for table_alias, instance_set in table_alias_to_instance_set.items(): + aggregated_select_columns = aggregated_select_columns.merge( + instance_set.transform( + CreateSelectColumnForCombineOutputNode( + table_alias=table_alias, + column_resolver=self._column_association_resolver, + metric_lookup=self._metric_lookup, + ) + ) ) - ) linkable_select_column_set = linkable_spec_set.transform( CreateSelectCoalescedColumnsForLinkableSpecs( column_association_resolver=self._column_association_resolver, table_aliases=[x.alias for x in parent_data_sets], ) ) - combined_select_column_set = linkable_select_column_set.merge(metric_select_column_set) + combined_select_column_set = linkable_select_column_set.merge(aggregated_select_columns) return SqlDataSet( instance_set=output_instance_set, diff --git a/metricflow/plan_conversion/instance_converters.py b/metricflow/plan_conversion/instance_converters.py index dfe38b9cd5..598675f4dd 100644 --- a/metricflow/plan_conversion/instance_converters.py +++ b/metricflow/plan_conversion/instance_converters.py @@ -8,7 +8,8 @@ from itertools import chain from typing import Dict, List, Optional, Sequence, Tuple -from dbt_semantic_interfaces.references import SemanticModelReference +from dbt_semantic_interfaces.references import MetricReference, SemanticModelReference +from dbt_semantic_interfaces.type_enums.aggregation_type import AggregationType from dbt_semantic_interfaces.type_enums.date_part import DatePart from dbt_semantic_interfaces.type_enums.time_granularity import TimeGranularity from more_itertools import bucket @@ -28,7 +29,7 @@ TimeDimensionInstance, ) from metricflow.plan_conversion.select_column_gen import SelectColumnSet -from metricflow.protocols.semantics import SemanticModelAccessor +from metricflow.protocols.semantics import MetricAccessor, SemanticModelAccessor from metricflow.specs.column_assoc import ColumnAssociationResolver from metricflow.specs.specs import ( DimensionSpec, @@ -43,11 +44,17 @@ TimeDimensionSpec, ) from metricflow.sql.sql_exprs import ( + SqlAggregateFunctionExpression, SqlColumnReference, SqlColumnReferenceExpression, + SqlExpressionNode, + SqlFunction, SqlFunctionExpression, + SqlStringExpression, +) +from metricflow.sql.sql_plan import ( + SqlSelectColumn, ) -from metricflow.sql.sql_plan import SqlSelectColumn logger = logging.getLogger(__name__) @@ -762,6 +769,81 @@ def transform(self, instance_set: InstanceSet) -> Tuple[SqlColumnReferenceExpres ) +class CreateSelectColumnForCombineOutputNode(InstanceSetTransform[SelectColumnSet]): + """Create select column expressions for the instance for joining outputs. + + It assumes that the column names of the instances are represented by the supplied column association resolver and + come from the given table alias. + """ + + def __init__( # noqa: D + self, + table_alias: str, + column_resolver: ColumnAssociationResolver, + metric_lookup: MetricAccessor, + ) -> None: + self._table_alias = table_alias + self._column_resolver = column_resolver + self._metric_lookup = metric_lookup + + def _create_select_column(self, spec: InstanceSpec, fill_nulls_with: Optional[int] = None) -> SqlSelectColumn: + """Creates the select column for the given spec and the fill value.""" + column_name = self._column_resolver.resolve_spec(spec).column_name + column_reference_expression = SqlColumnReferenceExpression( + col_ref=SqlColumnReference( + table_alias=self._table_alias, + column_name=column_name, + ) + ) + select_expression: SqlExpressionNode = SqlFunctionExpression.build_expression_from_aggregation_type( + aggregation_type=AggregationType.MAX, sql_column_expression=column_reference_expression + ) + if fill_nulls_with is not None: + select_expression = SqlAggregateFunctionExpression( + sql_function=SqlFunction.COALESCE, + sql_function_args=[ + select_expression, + SqlStringExpression(str(fill_nulls_with)), + ], + ) + return SqlSelectColumn( + expr=select_expression, + column_alias=column_name, + ) + + def _create_select_columns_for_metrics( + self, metric_instances: Tuple[MetricInstance, ...] + ) -> List[SqlSelectColumn]: # noqa: D + select_columns: List[SqlSelectColumn] = [] + for metric_instance in metric_instances: + metric_reference = MetricReference(element_name=metric_instance.defined_from.metric_name) + input_measure = self._metric_lookup.configured_input_measure_for_metric(metric_reference=metric_reference) + fill_nulls_with: Optional[int] = None + if input_measure and input_measure.fill_nulls_with is not None: + fill_nulls_with = input_measure.fill_nulls_with + select_columns.append( + self._create_select_column(spec=metric_instance.spec, fill_nulls_with=fill_nulls_with) + ) + return select_columns + + def _create_select_columns_for_measures( # noqa: D + self, measure_instances: Tuple[MeasureInstance, ...] + ) -> List[SqlSelectColumn]: + select_columns: List[SqlSelectColumn] = [] + for measure_instance in measure_instances: + measure_spec = measure_instance.spec + select_columns.append( + self._create_select_column(spec=measure_spec, fill_nulls_with=measure_spec.fill_nulls_with) + ) + return select_columns + + def transform(self, instance_set: InstanceSet) -> SelectColumnSet: # noqa: D + return SelectColumnSet( + metric_columns=self._create_select_columns_for_metrics(instance_set.metric_instances), + measure_columns=self._create_select_columns_for_measures(instance_set.measure_instances), + ) + + class ChangeAssociatedColumns(InstanceSetTransform[InstanceSet]): """Change the columns associated with instances to the one specified by the resolver.""" From 9edee45bf0e7fa5d7a946786343ff4f2104e17c6 Mon Sep 17 00:00:00 2001 From: Will Deng Date: Thu, 16 Nov 2023 03:10:36 -0500 Subject: [PATCH 4/6] rename CombineMetricsNode -> CombineAggregatedOutputsNode --- metricflow/dag/id_generation.py | 2 +- .../dataflow/builder/dataflow_plan_builder.py | 4 +-- metricflow/dataflow/dataflow_plan.py | 16 +++++----- .../source_scan/cm_branch_combiner.py | 6 ++-- .../source_scan/source_scan_optimizer.py | 30 +++++++++++-------- metricflow/plan_conversion/dataflow_to_sql.py | 16 +++++----- .../plan_conversion/sql_join_builder.py | 4 +-- .../source_scan/test_source_scan_optimizer.py | 4 +-- 8 files changed, 44 insertions(+), 38 deletions(-) diff --git a/metricflow/dag/id_generation.py b/metricflow/dag/id_generation.py index 1fd5e03581..3fa362e6e5 100644 --- a/metricflow/dag/id_generation.py +++ b/metricflow/dag/id_generation.py @@ -18,7 +18,7 @@ DATAFLOW_NODE_WHERE_CONSTRAINT_ID_PREFIX = "wcc" DATAFLOW_NODE_WRITE_TO_RESULT_DATAFRAME_ID_PREFIX = "wrd" DATAFLOW_NODE_WRITE_TO_RESULT_TABLE_ID_PREFIX = "wrt" -DATAFLOW_NODE_COMBINE_METRICS_ID_PREFIX = "cbm" +DATAFLOW_NODE_COMBINE_AGGREGATED_OUTPUTS_ID_PREFIX = "cao" DATAFLOW_NODE_CONSTRAIN_TIME_RANGE_ID_PREFIX = "ctr" DATAFLOW_NODE_SET_MEASURE_AGGREGATION_TIME = "sma" DATAFLOW_NODE_SEMI_ADDITIVE_JOIN_ID_PREFIX = "saj" diff --git a/metricflow/dataflow/builder/dataflow_plan_builder.py b/metricflow/dataflow/builder/dataflow_plan_builder.py index 755f5c0358..a0d8247271 100644 --- a/metricflow/dataflow/builder/dataflow_plan_builder.py +++ b/metricflow/dataflow/builder/dataflow_plan_builder.py @@ -24,7 +24,7 @@ from metricflow.dataflow.dataflow_plan import ( AggregateMeasuresNode, BaseOutput, - CombineMetricsNode, + CombineAggregatedOutputsNode, ComputeMetricsNode, ConstrainTimeRangeNode, DataflowPlan, @@ -298,7 +298,7 @@ def _build_metrics_output_node( if len(output_nodes) == 1: return output_nodes[0] - return CombineMetricsNode(parent_nodes=output_nodes) + return CombineAggregatedOutputsNode(parent_nodes=output_nodes) def build_plan_for_distinct_values(self, query_spec: MetricFlowQuerySpec) -> DataflowPlan: """Generate a plan that would get the distinct values of a linkable instance. diff --git a/metricflow/dataflow/dataflow_plan.py b/metricflow/dataflow/dataflow_plan.py index 94404d662a..26704391ed 100644 --- a/metricflow/dataflow/dataflow_plan.py +++ b/metricflow/dataflow/dataflow_plan.py @@ -17,7 +17,7 @@ from metricflow.dag.id_generation import ( DATAFLOW_NODE_AGGREGATE_MEASURES_ID_PREFIX, - DATAFLOW_NODE_COMBINE_METRICS_ID_PREFIX, + DATAFLOW_NODE_COMBINE_AGGREGATED_OUTPUTS_ID_PREFIX, DATAFLOW_NODE_COMPUTE_METRICS_ID_PREFIX, DATAFLOW_NODE_CONSTRAIN_TIME_RANGE_ID_PREFIX, DATAFLOW_NODE_JOIN_SELF_OVER_TIME_RANGE_ID_PREFIX, @@ -150,7 +150,7 @@ def visit_pass_elements_filter_node(self, node: FilterElementsNode) -> VisitorOu pass @abstractmethod - def visit_combine_metrics_node(self, node: CombineMetricsNode) -> VisitorOutputT: # noqa: D + def visit_combine_aggregated_outputs_node(self, node: CombineAggregatedOutputsNode) -> VisitorOutputT: # noqa: D pass @abstractmethod @@ -1160,7 +1160,7 @@ def with_new_parents(self, new_parent_nodes: Sequence[BaseOutput]) -> WhereConst ) -class CombineMetricsNode(ComputedMetricsOutput): +class CombineAggregatedOutputsNode(ComputedMetricsOutput): """Combines metrics from different nodes into a single output.""" def __init__( # noqa: D @@ -1171,21 +1171,21 @@ def __init__( # noqa: D @classmethod def id_prefix(cls) -> str: # noqa: D - return DATAFLOW_NODE_COMBINE_METRICS_ID_PREFIX + return DATAFLOW_NODE_COMBINE_AGGREGATED_OUTPUTS_ID_PREFIX def accept(self, visitor: DataflowPlanNodeVisitor[VisitorOutputT]) -> VisitorOutputT: # noqa: D - return visitor.visit_combine_metrics_node(self) + return visitor.visit_combine_aggregated_outputs_node(self) @property def description(self) -> str: # noqa: D - return "Combine Metrics" + return "Combine Aggregated Outputs" def functionally_identical(self, other_node: DataflowPlanNode) -> bool: # noqa: D return isinstance(other_node, self.__class__) - def with_new_parents(self, new_parent_nodes: Sequence[BaseOutput]) -> CombineMetricsNode: # noqa: D + def with_new_parents(self, new_parent_nodes: Sequence[BaseOutput]) -> CombineAggregatedOutputsNode: # noqa: D assert len(new_parent_nodes) == 1 - return CombineMetricsNode(parent_nodes=new_parent_nodes) + return CombineAggregatedOutputsNode(parent_nodes=new_parent_nodes) class ConstrainTimeRangeNode(AggregatedMeasuresOutput, BaseOutput): diff --git a/metricflow/dataflow/optimizer/source_scan/cm_branch_combiner.py b/metricflow/dataflow/optimizer/source_scan/cm_branch_combiner.py index d14e7b1411..535221f826 100644 --- a/metricflow/dataflow/optimizer/source_scan/cm_branch_combiner.py +++ b/metricflow/dataflow/optimizer/source_scan/cm_branch_combiner.py @@ -7,7 +7,7 @@ from metricflow.dataflow.dataflow_plan import ( AggregateMeasuresNode, BaseOutput, - CombineMetricsNode, + CombineAggregatedOutputsNode, ComputeMetricsNode, ConstrainTimeRangeNode, DataflowPlanNode, @@ -368,7 +368,9 @@ def visit_pass_elements_filter_node( # noqa: D ) return ComputeMetricsBranchCombinerResult(combined_node) - def visit_combine_metrics_node(self, node: CombineMetricsNode) -> ComputeMetricsBranchCombinerResult: # noqa: D + def visit_combine_aggregated_outputs_node( # noqa: D + self, node: CombineAggregatedOutputsNode + ) -> ComputeMetricsBranchCombinerResult: self._log_visit_node_type(node) return self._handle_unsupported_node(node) diff --git a/metricflow/dataflow/optimizer/source_scan/source_scan_optimizer.py b/metricflow/dataflow/optimizer/source_scan/source_scan_optimizer.py index 1903ec9b0d..7a5f2c39fe 100644 --- a/metricflow/dataflow/optimizer/source_scan/source_scan_optimizer.py +++ b/metricflow/dataflow/optimizer/source_scan/source_scan_optimizer.py @@ -8,7 +8,7 @@ from metricflow.dataflow.dataflow_plan import ( AggregateMeasuresNode, BaseOutput, - CombineMetricsNode, + CombineAggregatedOutputsNode, ComputeMetricsNode, ConstrainTimeRangeNode, DataflowPlan, @@ -68,12 +68,12 @@ class SourceScanOptimizer( ): """Reduces the number of scans (ReadSqlSourceNodes) in a dataflow plan. - This attempts to reduce the number of scans by combining the parent nodes of CombineMetricsNode via the + This attempts to reduce the number of scans by combining the parent nodes of CombineAggregatedOutputsNode via the ComputeMetricsBranchCombiner. A plan with a structure similar to ... - + ... @@ -89,11 +89,11 @@ class SourceScanOptimizer( ... - + ... will be converted to ... - + ... @@ -104,11 +104,11 @@ class SourceScanOptimizer( ... - + ... when possible. - In cases where all ComputeMetricsNodes can be combined into a single one, the CombineMetricsNode may be removed as + In cases where all ComputeMetricsNodes can be combined into a single one, the CombineAggregatedOutputsNode may be removed as well. This traverses the dataflow plan using DFS. When visiting a node (current_node), it first runs the optimization @@ -229,9 +229,11 @@ def _combine_branches( ) return results - def visit_combine_metrics_node(self, node: CombineMetricsNode) -> OptimizeBranchResult: # noqa: D + def visit_combine_aggregated_outputs_node( # noqa: D + self, node: CombineAggregatedOutputsNode + ) -> OptimizeBranchResult: self._log_visit_node_type(node) - # The parent node of the CombineMetricsNode can be either ComputeMetricsNodes or CombineMetricsNodes + # The parent node of the CombineAggregatedOutputsNode can be either ComputeMetricsNodes or CombineAggregatedOutputsNodes # Stores the result of running this optimizer on each parent branch separately. optimized_parent_branches = [] @@ -248,7 +250,7 @@ def visit_combine_metrics_node(self, node: CombineMetricsNode) -> OptimizeBranch assert ( result.base_output_node is not None - ), f"Traversing the parents of a CombineMetricsNode should always produce a BaseOutput. Got: {result}" + ), f"Traversing the parents of a CombineAggregatedOutputsNode should always produce a BaseOutput. Got: {result}" optimized_parent_branches.append(result.base_output_node) # Try to combine (using ComputeMetricsBranchCombiner) as many parent branches as possible in a @@ -275,12 +277,14 @@ def visit_combine_metrics_node(self, node: CombineMetricsNode) -> OptimizeBranch logger.log(level=self._log_level, msg=f"Got {len(combined_parent_branches)} branches after combination") assert len(combined_parent_branches) > 0 - # If we were able to reduce the parent branches of the CombineMetricsNode into a single one, there's no need - # for a CombineMetricsNode. + # If we were able to reduce the parent branches of the CombineAggregatedOutputsNode into a single one, there's no need + # for a CombineAggregatedOutputsNode. if len(combined_parent_branches) == 1: return OptimizeBranchResult(base_output_node=combined_parent_branches[0]) - return OptimizeBranchResult(base_output_node=CombineMetricsNode(parent_nodes=combined_parent_branches)) + return OptimizeBranchResult( + base_output_node=CombineAggregatedOutputsNode(parent_nodes=combined_parent_branches) + ) def visit_constrain_time_range_node(self, node: ConstrainTimeRangeNode) -> OptimizeBranchResult: # noqa: D self._log_visit_node_type(node) diff --git a/metricflow/plan_conversion/dataflow_to_sql.py b/metricflow/plan_conversion/dataflow_to_sql.py index 1d85af7ad6..3893551625 100644 --- a/metricflow/plan_conversion/dataflow_to_sql.py +++ b/metricflow/plan_conversion/dataflow_to_sql.py @@ -13,7 +13,7 @@ from metricflow.dataflow.dataflow_plan import ( AggregateMeasuresNode, BaseOutput, - CombineMetricsNode, + CombineAggregatedOutputsNode, ComputedMetricsOutput, ComputeMetricsNode, ConstrainTimeRangeNode, @@ -786,12 +786,12 @@ def visit_where_constraint_node(self, node: WhereConstraintNode) -> SqlDataSet: ), ) - def visit_combine_metrics_node(self, node: CombineMetricsNode) -> SqlDataSet: - """Join computed metric datasets together to return a single dataset containing all metrics. + def visit_combine_aggregated_outputs_node(self, node: CombineAggregatedOutputsNode) -> SqlDataSet: + """Join aggregated output datasets together to return a single dataset containing all metrics/measures. - This node may exist in one of two situations: when metrics need to be combined in order to produce a single - dataset with all required inputs for a derived metric, or when metrics need to be combined in order to produce - a single dataset of output for downstream consumption by the end user. + This node may exist in one of two situations: when metrics/measures need to be combined in order to produce a single + dataset with all required inputs for a metric (ie., derived metric), or when metrics need to be combined in order to + produce a single dataset of output for downstream consumption by the end user. The join key will be a coalesced set of all previously seen dimension values. For example: FROM ( @@ -819,7 +819,7 @@ def visit_combine_metrics_node(self, node: CombineMetricsNode) -> SqlDataSet: """ assert ( len(node.parent_nodes) > 1 - ), "Shouldn't have a CombineMetricsNode in the dataflow plan if there's only 1 parent." + ), "Shouldn't have a CombineAggregatedOutputsNode in the dataflow plan if there's only 1 parent." parent_data_sets: List[AnnotatedSqlDataSet] = [] table_alias_to_instance_set: OrderedDict[str, InstanceSet] = OrderedDict() @@ -854,7 +854,7 @@ def visit_combine_metrics_node(self, node: CombineMetricsNode) -> SqlDataSet: aliases_seen = [from_data_set.alias] for join_data_set in join_data_sets: joins_descriptions.append( - SqlQueryPlanJoinBuilder.make_combine_metrics_join_description( + SqlQueryPlanJoinBuilder.make_join_description_for_combining_datasets( from_data_set=from_data_set, join_data_set=join_data_set, join_type=join_type, diff --git a/metricflow/plan_conversion/sql_join_builder.py b/metricflow/plan_conversion/sql_join_builder.py index 29a7b8002d..e31e26d2c7 100644 --- a/metricflow/plan_conversion/sql_join_builder.py +++ b/metricflow/plan_conversion/sql_join_builder.py @@ -305,14 +305,14 @@ def _make_time_window_join_condition( ) @staticmethod - def make_combine_metrics_join_description( + def make_join_description_for_combining_datasets( from_data_set: AnnotatedSqlDataSet, join_data_set: AnnotatedSqlDataSet, join_type: SqlJoinType, column_names: Sequence[str], table_aliases_for_coalesce: Sequence[str], ) -> SqlJoinDescription: - """Creates the sql join description for combining two separate metrics output datasets. + """Creates the sql join description for combining two separate output datasets. These might be combined in service of producing complete output for end user consumption, in which case the join type will be FULL OUTER in order to ensure all rows are included. In this case, the diff --git a/metricflow/test/dataflow/optimizer/source_scan/test_source_scan_optimizer.py b/metricflow/test/dataflow/optimizer/source_scan/test_source_scan_optimizer.py index 03ca1b9134..e97dbffacd 100644 --- a/metricflow/test/dataflow/optimizer/source_scan/test_source_scan_optimizer.py +++ b/metricflow/test/dataflow/optimizer/source_scan/test_source_scan_optimizer.py @@ -10,7 +10,7 @@ from metricflow.dataflow.builder.dataflow_plan_builder import DataflowPlanBuilder from metricflow.dataflow.dataflow_plan import ( AggregateMeasuresNode, - CombineMetricsNode, + CombineAggregatedOutputsNode, ComputeMetricsNode, ConstrainTimeRangeNode, DataflowPlan, @@ -79,7 +79,7 @@ def visit_write_to_result_table_node(self, node: WriteToResultTableNode) -> int: def visit_pass_elements_filter_node(self, node: FilterElementsNode) -> int: # noqa: D return self._sum_parents(node) - def visit_combine_metrics_node(self, node: CombineMetricsNode) -> int: # noqa: D + def visit_combine_aggregated_outputs_node(self, node: CombineAggregatedOutputsNode) -> int: # noqa: D return self._sum_parents(node) def visit_constrain_time_range_node(self, node: ConstrainTimeRangeNode) -> int: # noqa: D From 743cab10d73da52a603fae5070c1722504995b01 Mon Sep 17 00:00:00 2001 From: Will Deng Date: Thu, 16 Nov 2023 03:14:22 -0500 Subject: [PATCH 5/6] added new test and snapshot --- .../test_dataflow_to_sql_plan.py | 59 ++ .../test_combine_output_node__plan0.sql | 228 +++++ ...t_combine_output_node__plan0_optimized.sql | 46 + .../test_combine_output_node__plan0.xml | 843 ++++++++++++++++++ 4 files changed, 1176 insertions(+) create mode 100644 metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/DuckDB/test_combine_output_node__plan0.sql create mode 100644 metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/DuckDB/test_combine_output_node__plan0_optimized.sql create mode 100644 metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/test_combine_output_node__plan0.xml diff --git a/metricflow/test/plan_conversion/test_dataflow_to_sql_plan.py b/metricflow/test/plan_conversion/test_dataflow_to_sql_plan.py index 25c3521924..0c3680a13d 100644 --- a/metricflow/test/plan_conversion/test_dataflow_to_sql_plan.py +++ b/metricflow/test/plan_conversion/test_dataflow_to_sql_plan.py @@ -15,6 +15,7 @@ from metricflow.dataflow.dataflow_plan import ( AggregateMeasuresNode, BaseOutput, + CombineAggregatedOutputsNode, ComputeMetricsNode, ConstrainTimeRangeNode, DataflowPlan, @@ -1011,6 +1012,64 @@ def test_compute_metrics_node_ratio_from_multiple_semantic_models( ) +@pytest.mark.sql_engine_snapshot +def test_combine_output_node( # noqa: D + request: FixtureRequest, + mf_test_session_state: MetricFlowTestSessionState, + dataflow_to_sql_converter: DataflowToSqlQueryPlanConverter, + consistent_id_object_repository: ConsistentIdObjectRepository, + sql_client: SqlClient, +) -> None: + """Tests combining AggregateMeasuresNode.""" + sum_spec = MeasureSpec( + element_name="bookings", + ) + sum_boolean_spec = MeasureSpec( + element_name="instant_bookings", + ) + count_distinct_spec = MeasureSpec( + element_name="bookers", + ) + dimension_spec = DimensionSpec( + element_name="is_instant", + entity_links=(), + ) + measure_source_node = consistent_id_object_repository.simple_model_read_nodes["bookings_source"] + + # Build compute measures node + measure_specs: List[MeasureSpec] = [sum_spec] + filtered_measure_node = FilterElementsNode( + parent_node=measure_source_node, + include_specs=InstanceSpecSet(measure_specs=tuple(measure_specs), dimension_specs=(dimension_spec,)), + ) + aggregated_measure_node = AggregateMeasuresNode( + parent_node=filtered_measure_node, + metric_input_measure_specs=tuple(MetricInputMeasureSpec(measure_spec=x) for x in measure_specs), + ) + + # Build agg measures node + measure_specs_2 = [sum_boolean_spec, count_distinct_spec] + filtered_measure_node_2 = FilterElementsNode( + parent_node=measure_source_node, + include_specs=InstanceSpecSet(measure_specs=tuple(measure_specs_2), dimension_specs=(dimension_spec,)), + ) + aggregated_measure_node_2 = AggregateMeasuresNode( + parent_node=filtered_measure_node_2, + metric_input_measure_specs=tuple( + MetricInputMeasureSpec(measure_spec=x, fill_nulls_with=1) for x in measure_specs_2 + ), + ) + + combine_output_node = CombineAggregatedOutputsNode([aggregated_measure_node, aggregated_measure_node_2]) + convert_and_check( + request=request, + mf_test_session_state=mf_test_session_state, + dataflow_to_sql_converter=dataflow_to_sql_converter, + sql_client=sql_client, + node=combine_output_node, + ) + + @pytest.mark.sql_engine_snapshot def test_dimensions_requiring_join( request: FixtureRequest, diff --git a/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/DuckDB/test_combine_output_node__plan0.sql b/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/DuckDB/test_combine_output_node__plan0.sql new file mode 100644 index 0000000000..c800156c88 --- /dev/null +++ b/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/DuckDB/test_combine_output_node__plan0.sql @@ -0,0 +1,228 @@ +-- Combine Aggregated Outputs +SELECT + COALESCE(subq_2.is_instant, subq_5.is_instant) AS is_instant + , MAX(subq_2.bookings) AS bookings + , COALESCE(MAX(subq_5.instant_bookings), 1) AS instant_bookings + , COALESCE(MAX(subq_5.bookers), 1) AS bookers +FROM ( + -- Aggregate Measures + SELECT + subq_1.is_instant + , SUM(subq_1.bookings) AS bookings + FROM ( + -- Pass Only Elements: + -- ['bookings', 'is_instant'] + SELECT + subq_0.is_instant + , subq_0.bookings + FROM ( + -- Read Elements From Semantic Model 'bookings_source' + SELECT + 1 AS bookings + , CASE WHEN is_instant THEN 1 ELSE 0 END AS instant_bookings + , bookings_source_src_10001.booking_value + , bookings_source_src_10001.booking_value AS max_booking_value + , bookings_source_src_10001.booking_value AS min_booking_value + , bookings_source_src_10001.guest_id AS bookers + , bookings_source_src_10001.booking_value AS average_booking_value + , bookings_source_src_10001.booking_value AS booking_payments + , CASE WHEN referrer_id IS NOT NULL THEN 1 ELSE 0 END AS referred_bookings + , bookings_source_src_10001.booking_value AS median_booking_value + , bookings_source_src_10001.booking_value AS booking_value_p99 + , bookings_source_src_10001.booking_value AS discrete_booking_value_p99 + , bookings_source_src_10001.booking_value AS approximate_continuous_booking_value_p99 + , bookings_source_src_10001.booking_value AS approximate_discrete_booking_value_p99 + , bookings_source_src_10001.is_instant + , DATE_TRUNC('day', bookings_source_src_10001.ds) AS ds__day + , DATE_TRUNC('week', bookings_source_src_10001.ds) AS ds__week + , DATE_TRUNC('month', bookings_source_src_10001.ds) AS ds__month + , DATE_TRUNC('quarter', bookings_source_src_10001.ds) AS ds__quarter + , DATE_TRUNC('year', bookings_source_src_10001.ds) AS ds__year + , EXTRACT(year FROM bookings_source_src_10001.ds) AS ds__extract_year + , EXTRACT(quarter FROM bookings_source_src_10001.ds) AS ds__extract_quarter + , EXTRACT(month FROM bookings_source_src_10001.ds) AS ds__extract_month + , EXTRACT(day FROM bookings_source_src_10001.ds) AS ds__extract_day + , EXTRACT(isodow FROM bookings_source_src_10001.ds) AS ds__extract_dow + , EXTRACT(doy FROM bookings_source_src_10001.ds) AS ds__extract_doy + , DATE_TRUNC('day', bookings_source_src_10001.ds_partitioned) AS ds_partitioned__day + , DATE_TRUNC('week', bookings_source_src_10001.ds_partitioned) AS ds_partitioned__week + , DATE_TRUNC('month', bookings_source_src_10001.ds_partitioned) AS ds_partitioned__month + , DATE_TRUNC('quarter', bookings_source_src_10001.ds_partitioned) AS ds_partitioned__quarter + , DATE_TRUNC('year', bookings_source_src_10001.ds_partitioned) AS ds_partitioned__year + , EXTRACT(year FROM bookings_source_src_10001.ds_partitioned) AS ds_partitioned__extract_year + , EXTRACT(quarter FROM bookings_source_src_10001.ds_partitioned) AS ds_partitioned__extract_quarter + , EXTRACT(month FROM bookings_source_src_10001.ds_partitioned) AS ds_partitioned__extract_month + , EXTRACT(day FROM bookings_source_src_10001.ds_partitioned) AS ds_partitioned__extract_day + , EXTRACT(isodow FROM bookings_source_src_10001.ds_partitioned) AS ds_partitioned__extract_dow + , EXTRACT(doy FROM bookings_source_src_10001.ds_partitioned) AS ds_partitioned__extract_doy + , DATE_TRUNC('day', bookings_source_src_10001.paid_at) AS paid_at__day + , DATE_TRUNC('week', bookings_source_src_10001.paid_at) AS paid_at__week + , DATE_TRUNC('month', bookings_source_src_10001.paid_at) AS paid_at__month + , DATE_TRUNC('quarter', bookings_source_src_10001.paid_at) AS paid_at__quarter + , DATE_TRUNC('year', bookings_source_src_10001.paid_at) AS paid_at__year + , EXTRACT(year FROM bookings_source_src_10001.paid_at) AS paid_at__extract_year + , EXTRACT(quarter FROM bookings_source_src_10001.paid_at) AS paid_at__extract_quarter + , EXTRACT(month FROM bookings_source_src_10001.paid_at) AS paid_at__extract_month + , EXTRACT(day FROM bookings_source_src_10001.paid_at) AS paid_at__extract_day + , EXTRACT(isodow FROM bookings_source_src_10001.paid_at) AS paid_at__extract_dow + , EXTRACT(doy FROM bookings_source_src_10001.paid_at) AS paid_at__extract_doy + , bookings_source_src_10001.is_instant AS booking__is_instant + , DATE_TRUNC('day', bookings_source_src_10001.ds) AS booking__ds__day + , DATE_TRUNC('week', bookings_source_src_10001.ds) AS booking__ds__week + , DATE_TRUNC('month', bookings_source_src_10001.ds) AS booking__ds__month + , DATE_TRUNC('quarter', bookings_source_src_10001.ds) AS booking__ds__quarter + , DATE_TRUNC('year', bookings_source_src_10001.ds) AS booking__ds__year + , EXTRACT(year FROM bookings_source_src_10001.ds) AS booking__ds__extract_year + , EXTRACT(quarter FROM bookings_source_src_10001.ds) AS booking__ds__extract_quarter + , EXTRACT(month FROM bookings_source_src_10001.ds) AS booking__ds__extract_month + , EXTRACT(day FROM bookings_source_src_10001.ds) AS booking__ds__extract_day + , EXTRACT(isodow FROM bookings_source_src_10001.ds) AS booking__ds__extract_dow + , EXTRACT(doy FROM bookings_source_src_10001.ds) AS booking__ds__extract_doy + , DATE_TRUNC('day', bookings_source_src_10001.ds_partitioned) AS booking__ds_partitioned__day + , DATE_TRUNC('week', bookings_source_src_10001.ds_partitioned) AS booking__ds_partitioned__week + , DATE_TRUNC('month', bookings_source_src_10001.ds_partitioned) AS booking__ds_partitioned__month + , DATE_TRUNC('quarter', bookings_source_src_10001.ds_partitioned) AS booking__ds_partitioned__quarter + , DATE_TRUNC('year', bookings_source_src_10001.ds_partitioned) AS booking__ds_partitioned__year + , EXTRACT(year FROM bookings_source_src_10001.ds_partitioned) AS booking__ds_partitioned__extract_year + , EXTRACT(quarter FROM bookings_source_src_10001.ds_partitioned) AS booking__ds_partitioned__extract_quarter + , EXTRACT(month FROM bookings_source_src_10001.ds_partitioned) AS booking__ds_partitioned__extract_month + , EXTRACT(day FROM bookings_source_src_10001.ds_partitioned) AS booking__ds_partitioned__extract_day + , EXTRACT(isodow FROM bookings_source_src_10001.ds_partitioned) AS booking__ds_partitioned__extract_dow + , EXTRACT(doy FROM bookings_source_src_10001.ds_partitioned) AS booking__ds_partitioned__extract_doy + , DATE_TRUNC('day', bookings_source_src_10001.paid_at) AS booking__paid_at__day + , DATE_TRUNC('week', bookings_source_src_10001.paid_at) AS booking__paid_at__week + , DATE_TRUNC('month', bookings_source_src_10001.paid_at) AS booking__paid_at__month + , DATE_TRUNC('quarter', bookings_source_src_10001.paid_at) AS booking__paid_at__quarter + , DATE_TRUNC('year', bookings_source_src_10001.paid_at) AS booking__paid_at__year + , EXTRACT(year FROM bookings_source_src_10001.paid_at) AS booking__paid_at__extract_year + , EXTRACT(quarter FROM bookings_source_src_10001.paid_at) AS booking__paid_at__extract_quarter + , EXTRACT(month FROM bookings_source_src_10001.paid_at) AS booking__paid_at__extract_month + , EXTRACT(day FROM bookings_source_src_10001.paid_at) AS booking__paid_at__extract_day + , EXTRACT(isodow FROM bookings_source_src_10001.paid_at) AS booking__paid_at__extract_dow + , EXTRACT(doy FROM bookings_source_src_10001.paid_at) AS booking__paid_at__extract_doy + , bookings_source_src_10001.listing_id AS listing + , bookings_source_src_10001.guest_id AS guest + , bookings_source_src_10001.host_id AS host + , bookings_source_src_10001.listing_id AS booking__listing + , bookings_source_src_10001.guest_id AS booking__guest + , bookings_source_src_10001.host_id AS booking__host + FROM ***************************.fct_bookings bookings_source_src_10001 + ) subq_0 + ) subq_1 + GROUP BY + subq_1.is_instant +) subq_2 +FULL OUTER JOIN ( + -- Aggregate Measures + SELECT + subq_4.is_instant + , SUM(subq_4.instant_bookings) AS instant_bookings + , COUNT(DISTINCT subq_4.bookers) AS bookers + FROM ( + -- Pass Only Elements: + -- ['instant_bookings', 'bookers', 'is_instant'] + SELECT + subq_3.is_instant + , subq_3.instant_bookings + , subq_3.bookers + FROM ( + -- Read Elements From Semantic Model 'bookings_source' + SELECT + 1 AS bookings + , CASE WHEN is_instant THEN 1 ELSE 0 END AS instant_bookings + , bookings_source_src_10001.booking_value + , bookings_source_src_10001.booking_value AS max_booking_value + , bookings_source_src_10001.booking_value AS min_booking_value + , bookings_source_src_10001.guest_id AS bookers + , bookings_source_src_10001.booking_value AS average_booking_value + , bookings_source_src_10001.booking_value AS booking_payments + , CASE WHEN referrer_id IS NOT NULL THEN 1 ELSE 0 END AS referred_bookings + , bookings_source_src_10001.booking_value AS median_booking_value + , bookings_source_src_10001.booking_value AS booking_value_p99 + , bookings_source_src_10001.booking_value AS discrete_booking_value_p99 + , bookings_source_src_10001.booking_value AS approximate_continuous_booking_value_p99 + , bookings_source_src_10001.booking_value AS approximate_discrete_booking_value_p99 + , bookings_source_src_10001.is_instant + , DATE_TRUNC('day', bookings_source_src_10001.ds) AS ds__day + , DATE_TRUNC('week', bookings_source_src_10001.ds) AS ds__week + , DATE_TRUNC('month', bookings_source_src_10001.ds) AS ds__month + , DATE_TRUNC('quarter', bookings_source_src_10001.ds) AS ds__quarter + , DATE_TRUNC('year', bookings_source_src_10001.ds) AS ds__year + , EXTRACT(year FROM bookings_source_src_10001.ds) AS ds__extract_year + , EXTRACT(quarter FROM bookings_source_src_10001.ds) AS ds__extract_quarter + , EXTRACT(month FROM bookings_source_src_10001.ds) AS ds__extract_month + , EXTRACT(day FROM bookings_source_src_10001.ds) AS ds__extract_day + , EXTRACT(isodow FROM bookings_source_src_10001.ds) AS ds__extract_dow + , EXTRACT(doy FROM bookings_source_src_10001.ds) AS ds__extract_doy + , DATE_TRUNC('day', bookings_source_src_10001.ds_partitioned) AS ds_partitioned__day + , DATE_TRUNC('week', bookings_source_src_10001.ds_partitioned) AS ds_partitioned__week + , DATE_TRUNC('month', bookings_source_src_10001.ds_partitioned) AS ds_partitioned__month + , DATE_TRUNC('quarter', bookings_source_src_10001.ds_partitioned) AS ds_partitioned__quarter + , DATE_TRUNC('year', bookings_source_src_10001.ds_partitioned) AS ds_partitioned__year + , EXTRACT(year FROM bookings_source_src_10001.ds_partitioned) AS ds_partitioned__extract_year + , EXTRACT(quarter FROM bookings_source_src_10001.ds_partitioned) AS ds_partitioned__extract_quarter + , EXTRACT(month FROM bookings_source_src_10001.ds_partitioned) AS ds_partitioned__extract_month + , EXTRACT(day FROM bookings_source_src_10001.ds_partitioned) AS ds_partitioned__extract_day + , EXTRACT(isodow FROM bookings_source_src_10001.ds_partitioned) AS ds_partitioned__extract_dow + , EXTRACT(doy FROM bookings_source_src_10001.ds_partitioned) AS ds_partitioned__extract_doy + , DATE_TRUNC('day', bookings_source_src_10001.paid_at) AS paid_at__day + , DATE_TRUNC('week', bookings_source_src_10001.paid_at) AS paid_at__week + , DATE_TRUNC('month', bookings_source_src_10001.paid_at) AS paid_at__month + , DATE_TRUNC('quarter', bookings_source_src_10001.paid_at) AS paid_at__quarter + , DATE_TRUNC('year', bookings_source_src_10001.paid_at) AS paid_at__year + , EXTRACT(year FROM bookings_source_src_10001.paid_at) AS paid_at__extract_year + , EXTRACT(quarter FROM bookings_source_src_10001.paid_at) AS paid_at__extract_quarter + , EXTRACT(month FROM bookings_source_src_10001.paid_at) AS paid_at__extract_month + , EXTRACT(day FROM bookings_source_src_10001.paid_at) AS paid_at__extract_day + , EXTRACT(isodow FROM bookings_source_src_10001.paid_at) AS paid_at__extract_dow + , EXTRACT(doy FROM bookings_source_src_10001.paid_at) AS paid_at__extract_doy + , bookings_source_src_10001.is_instant AS booking__is_instant + , DATE_TRUNC('day', bookings_source_src_10001.ds) AS booking__ds__day + , DATE_TRUNC('week', bookings_source_src_10001.ds) AS booking__ds__week + , DATE_TRUNC('month', bookings_source_src_10001.ds) AS booking__ds__month + , DATE_TRUNC('quarter', bookings_source_src_10001.ds) AS booking__ds__quarter + , DATE_TRUNC('year', bookings_source_src_10001.ds) AS booking__ds__year + , EXTRACT(year FROM bookings_source_src_10001.ds) AS booking__ds__extract_year + , EXTRACT(quarter FROM bookings_source_src_10001.ds) AS booking__ds__extract_quarter + , EXTRACT(month FROM bookings_source_src_10001.ds) AS booking__ds__extract_month + , EXTRACT(day FROM bookings_source_src_10001.ds) AS booking__ds__extract_day + , EXTRACT(isodow FROM bookings_source_src_10001.ds) AS booking__ds__extract_dow + , EXTRACT(doy FROM bookings_source_src_10001.ds) AS booking__ds__extract_doy + , DATE_TRUNC('day', bookings_source_src_10001.ds_partitioned) AS booking__ds_partitioned__day + , DATE_TRUNC('week', bookings_source_src_10001.ds_partitioned) AS booking__ds_partitioned__week + , DATE_TRUNC('month', bookings_source_src_10001.ds_partitioned) AS booking__ds_partitioned__month + , DATE_TRUNC('quarter', bookings_source_src_10001.ds_partitioned) AS booking__ds_partitioned__quarter + , DATE_TRUNC('year', bookings_source_src_10001.ds_partitioned) AS booking__ds_partitioned__year + , EXTRACT(year FROM bookings_source_src_10001.ds_partitioned) AS booking__ds_partitioned__extract_year + , EXTRACT(quarter FROM bookings_source_src_10001.ds_partitioned) AS booking__ds_partitioned__extract_quarter + , EXTRACT(month FROM bookings_source_src_10001.ds_partitioned) AS booking__ds_partitioned__extract_month + , EXTRACT(day FROM bookings_source_src_10001.ds_partitioned) AS booking__ds_partitioned__extract_day + , EXTRACT(isodow FROM bookings_source_src_10001.ds_partitioned) AS booking__ds_partitioned__extract_dow + , EXTRACT(doy FROM bookings_source_src_10001.ds_partitioned) AS booking__ds_partitioned__extract_doy + , DATE_TRUNC('day', bookings_source_src_10001.paid_at) AS booking__paid_at__day + , DATE_TRUNC('week', bookings_source_src_10001.paid_at) AS booking__paid_at__week + , DATE_TRUNC('month', bookings_source_src_10001.paid_at) AS booking__paid_at__month + , DATE_TRUNC('quarter', bookings_source_src_10001.paid_at) AS booking__paid_at__quarter + , DATE_TRUNC('year', bookings_source_src_10001.paid_at) AS booking__paid_at__year + , EXTRACT(year FROM bookings_source_src_10001.paid_at) AS booking__paid_at__extract_year + , EXTRACT(quarter FROM bookings_source_src_10001.paid_at) AS booking__paid_at__extract_quarter + , EXTRACT(month FROM bookings_source_src_10001.paid_at) AS booking__paid_at__extract_month + , EXTRACT(day FROM bookings_source_src_10001.paid_at) AS booking__paid_at__extract_day + , EXTRACT(isodow FROM bookings_source_src_10001.paid_at) AS booking__paid_at__extract_dow + , EXTRACT(doy FROM bookings_source_src_10001.paid_at) AS booking__paid_at__extract_doy + , bookings_source_src_10001.listing_id AS listing + , bookings_source_src_10001.guest_id AS guest + , bookings_source_src_10001.host_id AS host + , bookings_source_src_10001.listing_id AS booking__listing + , bookings_source_src_10001.guest_id AS booking__guest + , bookings_source_src_10001.host_id AS booking__host + FROM ***************************.fct_bookings bookings_source_src_10001 + ) subq_3 + ) subq_4 + GROUP BY + subq_4.is_instant +) subq_5 +ON + subq_2.is_instant = subq_5.is_instant +GROUP BY + COALESCE(subq_2.is_instant, subq_5.is_instant) diff --git a/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/DuckDB/test_combine_output_node__plan0_optimized.sql b/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/DuckDB/test_combine_output_node__plan0_optimized.sql new file mode 100644 index 0000000000..62340aabcb --- /dev/null +++ b/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/DuckDB/test_combine_output_node__plan0_optimized.sql @@ -0,0 +1,46 @@ +-- Combine Aggregated Outputs +SELECT + COALESCE(subq_8.is_instant, subq_11.is_instant) AS is_instant + , MAX(subq_8.bookings) AS bookings + , COALESCE(MAX(subq_11.instant_bookings), 1) AS instant_bookings + , COALESCE(MAX(subq_11.bookers), 1) AS bookers +FROM ( + -- Aggregate Measures + SELECT + is_instant + , SUM(bookings) AS bookings + FROM ( + -- Read Elements From Semantic Model 'bookings_source' + -- Pass Only Elements: + -- ['bookings', 'is_instant'] + SELECT + is_instant + , 1 AS bookings + FROM ***************************.fct_bookings bookings_source_src_10001 + ) subq_7 + GROUP BY + is_instant +) subq_8 +FULL OUTER JOIN ( + -- Aggregate Measures + SELECT + is_instant + , SUM(instant_bookings) AS instant_bookings + , COUNT(DISTINCT bookers) AS bookers + FROM ( + -- Read Elements From Semantic Model 'bookings_source' + -- Pass Only Elements: + -- ['instant_bookings', 'bookers', 'is_instant'] + SELECT + is_instant + , CASE WHEN is_instant THEN 1 ELSE 0 END AS instant_bookings + , guest_id AS bookers + FROM ***************************.fct_bookings bookings_source_src_10001 + ) subq_10 + GROUP BY + is_instant +) subq_11 +ON + subq_8.is_instant = subq_11.is_instant +GROUP BY + COALESCE(subq_8.is_instant, subq_11.is_instant) diff --git a/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/test_combine_output_node__plan0.xml b/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/test_combine_output_node__plan0.xml new file mode 100644 index 0000000000..272a3b1630 --- /dev/null +++ b/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/test_combine_output_node__plan0.xml @@ -0,0 +1,843 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + From 7da7b495cc57fee429fb2eeb7d39bf18e27fd5f0 Mon Sep 17 00:00:00 2001 From: Will Deng Date: Thu, 16 Nov 2023 03:23:47 -0500 Subject: [PATCH 6/6] update snapshots --- .../test_filter_combination__dfpo_0.xml | 6 +- .../DataflowPlan/test_cyclic_join__dfp_0.xml | 6 +- .../test_common_semantic_model__dfp_0.xml | 20 +- ...indow_or_grain_with_metric_time__dfp_0.xml | 3 +- ...ow_or_grain_without_metric_time__dfp_0.xml | 3 +- ...t_cumulative_metric_with_window__dfp_0.xml | 3 +- ..._derived_metric_offset_to_grain__dfp_0.xml | 14 +- ...st_derived_metric_offset_window__dfp_0.xml | 3 +- ..._metric_offset_with_granularity__dfp_0.xml | 3 +- ...erived_offset_cumulative_metric__dfp_0.xml | 3 +- ..._if_no_time_dimension_requested__dfp_0.xml | 3 +- ...in_to_time_spine_derived_metric__dfp_0.xml | 14 +- ..._to_time_spine_with_metric_time__dfp_0.xml | 3 +- ...time_spine_with_non_metric_time__dfp_0.xml | 3 +- .../DataflowPlan/test_joined_plan__dfp_0.xml | 6 +- .../test_limit_rows_plan__dfp_0.xml | 3 +- .../test_measure_constraint_plan__dfp_0.xml | 29 ++- ...traint_with_reused_measure_plan__dfp_0.xml | 17 +- ...mantic_model_ratio_metrics_plan__dfp_0.xml | 20 +- .../test_multihop_join_plan__dfp_0.xml | 6 +- .../test_multiple_metrics_plan__dfp_0.xml | 14 +- .../test_order_by_plan__dfp_0.xml | 3 +- .../test_primary_entity_dimension__dfp_0.xml | 3 +- .../DataflowPlan/test_simple_plan__dfp_0.xml | 3 +- ...mantic_model_ratio_metrics_plan__dfp_0.xml | 20 +- .../test_where_constrained_plan__dfp_0.xml | 9 +- ...constrained_plan_time_dimension__dfp_0.xml | 6 +- ...ained_with_common_linkable_plan__dfp_0.xml | 6 +- .../test_combined_metrics_plan__ep_0.xml | 2 +- ...test_small_combined_metrics_plan__ep_0.xml | 2 +- .../test_combined_metrics_plan__ep_0.xml | 2 +- ...test_small_combined_metrics_plan__ep_0.xml | 2 +- .../test_combined_metrics_plan__ep_0.xml | 2 +- ...test_small_combined_metrics_plan__ep_0.xml | 2 +- .../test_combined_metrics_plan__ep_0.xml | 2 +- ...test_small_combined_metrics_plan__ep_0.xml | 2 +- .../test_combined_metrics_plan__ep_0.xml | 2 +- ...test_small_combined_metrics_plan__ep_0.xml | 2 +- .../test_combined_metrics_plan__ep_0.xml | 2 +- ...test_small_combined_metrics_plan__ep_0.xml | 2 +- ...ompute_metrics_node_simple_expr__plan0.xml | 3 +- ...spine_node_with_offset_to_grain__plan0.xml | 3 +- ...e_spine_node_with_offset_window__plan0.xml | 3 +- ..._time_spine_node_without_offset__plan0.xml | 3 +- .../test_combine_output_node__plan0.sql | 228 ++++++++++++++++++ ...t_combine_output_node__plan0_optimized.sql | 46 ++++ ...o_from_multiple_semantic_models__plan0.sql | 2 +- ...tiple_semantic_models__plan0_optimized.sql | 2 +- .../test_combine_output_node__plan0.sql | 228 ++++++++++++++++++ ...t_combine_output_node__plan0_optimized.sql | 46 ++++ ...o_from_multiple_semantic_models__plan0.sql | 2 +- ...tiple_semantic_models__plan0_optimized.sql | 2 +- ...o_from_multiple_semantic_models__plan0.sql | 2 +- ...tiple_semantic_models__plan0_optimized.sql | 2 +- .../test_combine_output_node__plan0.sql | 228 ++++++++++++++++++ ...t_combine_output_node__plan0_optimized.sql | 46 ++++ ...o_from_multiple_semantic_models__plan0.sql | 2 +- ...tiple_semantic_models__plan0_optimized.sql | 2 +- .../test_combine_output_node__plan0.sql | 228 ++++++++++++++++++ ...t_combine_output_node__plan0_optimized.sql | 46 ++++ ...o_from_multiple_semantic_models__plan0.sql | 2 +- ...tiple_semantic_models__plan0_optimized.sql | 2 +- .../test_combine_output_node__plan0.sql | 228 ++++++++++++++++++ ...t_combine_output_node__plan0_optimized.sql | 46 ++++ ...o_from_multiple_semantic_models__plan0.sql | 2 +- ...tiple_semantic_models__plan0_optimized.sql | 2 +- ...o_from_multiple_semantic_models__plan0.xml | 2 +- .../BigQuery/test_derived_metric__plan0.sql | 2 +- .../test_derived_metric__plan0_optimized.sql | 2 +- ...ved_metric_with_offset_to_grain__plan0.sql | 2 +- ..._with_offset_to_grain__plan0_optimized.sql | 2 +- ...offset_to_grain_and_granularity__plan0.sql | 2 +- ...grain_and_granularity__plan0_optimized.sql | 2 +- ...rived_metric_with_offset_window__plan0.sql | 2 +- ...ic_with_offset_window__plan0_optimized.sql | 2 +- ...h_offset_window_and_granularity__plan0.sql | 2 +- ...indow_and_granularity__plan0_optimized.sql | 2 +- ...fset_window_and_offset_to_grain__plan0.sql | 2 +- ...w_and_offset_to_grain__plan0_optimized.sql | 2 +- ...offset_to_grain_and_granularity__plan0.sql | 2 +- ...grain_and_granularity__plan0_optimized.sql | 2 +- ...h_offset_window_and_time_filter__plan0.sql | 2 +- ...indow_and_time_filter__plan0_optimized.sql | 2 +- .../test_nested_derived_metric__plan0.sql | 4 +- ...nested_derived_metric__plan0_optimized.sql | 4 +- .../Databricks/test_derived_metric__plan0.sql | 2 +- .../test_derived_metric__plan0_optimized.sql | 2 +- ...ved_metric_with_offset_to_grain__plan0.sql | 2 +- ..._with_offset_to_grain__plan0_optimized.sql | 2 +- ...offset_to_grain_and_granularity__plan0.sql | 2 +- ...grain_and_granularity__plan0_optimized.sql | 2 +- ...rived_metric_with_offset_window__plan0.sql | 2 +- ...ic_with_offset_window__plan0_optimized.sql | 2 +- ...h_offset_window_and_granularity__plan0.sql | 2 +- ...indow_and_granularity__plan0_optimized.sql | 2 +- ...fset_window_and_offset_to_grain__plan0.sql | 2 +- ...w_and_offset_to_grain__plan0_optimized.sql | 2 +- ...offset_to_grain_and_granularity__plan0.sql | 2 +- ...grain_and_granularity__plan0_optimized.sql | 2 +- ...h_offset_window_and_time_filter__plan0.sql | 2 +- ...indow_and_time_filter__plan0_optimized.sql | 2 +- .../test_nested_derived_metric__plan0.sql | 4 +- ...nested_derived_metric__plan0_optimized.sql | 4 +- .../DuckDB/test_derived_metric__plan0.sql | 2 +- .../test_derived_metric__plan0_optimized.sql | 2 +- ...ved_metric_with_offset_to_grain__plan0.sql | 2 +- ..._with_offset_to_grain__plan0_optimized.sql | 2 +- ...offset_to_grain_and_granularity__plan0.sql | 2 +- ...grain_and_granularity__plan0_optimized.sql | 2 +- ...rived_metric_with_offset_window__plan0.sql | 2 +- ...ic_with_offset_window__plan0_optimized.sql | 2 +- ...h_offset_window_and_granularity__plan0.sql | 2 +- ...indow_and_granularity__plan0_optimized.sql | 2 +- ...fset_window_and_offset_to_grain__plan0.sql | 2 +- ...w_and_offset_to_grain__plan0_optimized.sql | 2 +- ...offset_to_grain_and_granularity__plan0.sql | 2 +- ...grain_and_granularity__plan0_optimized.sql | 2 +- ...h_offset_window_and_time_filter__plan0.sql | 2 +- ...indow_and_time_filter__plan0_optimized.sql | 2 +- .../test_nested_derived_metric__plan0.sql | 4 +- ...nested_derived_metric__plan0_optimized.sql | 4 +- .../Postgres/test_derived_metric__plan0.sql | 2 +- .../test_derived_metric__plan0_optimized.sql | 2 +- ...ved_metric_with_offset_to_grain__plan0.sql | 2 +- ..._with_offset_to_grain__plan0_optimized.sql | 2 +- ...offset_to_grain_and_granularity__plan0.sql | 2 +- ...grain_and_granularity__plan0_optimized.sql | 2 +- ...rived_metric_with_offset_window__plan0.sql | 2 +- ...ic_with_offset_window__plan0_optimized.sql | 2 +- ...h_offset_window_and_granularity__plan0.sql | 2 +- ...indow_and_granularity__plan0_optimized.sql | 2 +- ...fset_window_and_offset_to_grain__plan0.sql | 2 +- ...w_and_offset_to_grain__plan0_optimized.sql | 2 +- ...offset_to_grain_and_granularity__plan0.sql | 2 +- ...grain_and_granularity__plan0_optimized.sql | 2 +- ...h_offset_window_and_time_filter__plan0.sql | 2 +- ...indow_and_time_filter__plan0_optimized.sql | 2 +- .../test_nested_derived_metric__plan0.sql | 4 +- ...nested_derived_metric__plan0_optimized.sql | 4 +- .../Redshift/test_derived_metric__plan0.sql | 2 +- .../test_derived_metric__plan0_optimized.sql | 2 +- ...ved_metric_with_offset_to_grain__plan0.sql | 2 +- ..._with_offset_to_grain__plan0_optimized.sql | 2 +- ...offset_to_grain_and_granularity__plan0.sql | 2 +- ...grain_and_granularity__plan0_optimized.sql | 2 +- ...rived_metric_with_offset_window__plan0.sql | 2 +- ...ic_with_offset_window__plan0_optimized.sql | 2 +- ...h_offset_window_and_granularity__plan0.sql | 2 +- ...indow_and_granularity__plan0_optimized.sql | 2 +- ...fset_window_and_offset_to_grain__plan0.sql | 2 +- ...w_and_offset_to_grain__plan0_optimized.sql | 2 +- ...offset_to_grain_and_granularity__plan0.sql | 2 +- ...grain_and_granularity__plan0_optimized.sql | 2 +- ...h_offset_window_and_time_filter__plan0.sql | 2 +- ...indow_and_time_filter__plan0_optimized.sql | 2 +- .../test_nested_derived_metric__plan0.sql | 4 +- ...nested_derived_metric__plan0_optimized.sql | 4 +- .../Snowflake/test_derived_metric__plan0.sql | 2 +- .../test_derived_metric__plan0_optimized.sql | 2 +- ...ved_metric_with_offset_to_grain__plan0.sql | 2 +- ..._with_offset_to_grain__plan0_optimized.sql | 2 +- ...offset_to_grain_and_granularity__plan0.sql | 2 +- ...grain_and_granularity__plan0_optimized.sql | 2 +- ...rived_metric_with_offset_window__plan0.sql | 2 +- ...ic_with_offset_window__plan0_optimized.sql | 2 +- ...h_offset_window_and_granularity__plan0.sql | 2 +- ...indow_and_granularity__plan0_optimized.sql | 2 +- ...fset_window_and_offset_to_grain__plan0.sql | 2 +- ...w_and_offset_to_grain__plan0_optimized.sql | 2 +- ...offset_to_grain_and_granularity__plan0.sql | 2 +- ...grain_and_granularity__plan0_optimized.sql | 2 +- ...h_offset_window_and_time_filter__plan0.sql | 2 +- ...indow_and_time_filter__plan0_optimized.sql | 2 +- .../test_nested_derived_metric__plan0.sql | 4 +- ...nested_derived_metric__plan0_optimized.sql | 4 +- ...fill_nulls_for_one_input_metric__plan0.sql | 2 +- ..._for_one_input_metric__plan0_optimized.sql | 2 +- ...fill_nulls_for_one_input_metric__plan0.sql | 2 +- ..._for_one_input_metric__plan0_optimized.sql | 2 +- ...fill_nulls_for_one_input_metric__plan0.sql | 2 +- ..._for_one_input_metric__plan0_optimized.sql | 2 +- ...fill_nulls_for_one_input_metric__plan0.sql | 2 +- ..._for_one_input_metric__plan0_optimized.sql | 2 +- ...fill_nulls_for_one_input_metric__plan0.sql | 2 +- ..._for_one_input_metric__plan0_optimized.sql | 2 +- ...fill_nulls_for_one_input_metric__plan0.sql | 2 +- ..._for_one_input_metric__plan0_optimized.sql | 2 +- ...st_offset_window_with_date_part__plan0.sql | 2 +- ...window_with_date_part__plan0_optimized.sql | 2 +- ...st_offset_window_with_date_part__plan0.sql | 2 +- ...window_with_date_part__plan0_optimized.sql | 2 +- ...st_offset_window_with_date_part__plan0.sql | 2 +- ...window_with_date_part__plan0_optimized.sql | 2 +- ...st_offset_window_with_date_part__plan0.sql | 2 +- ...window_with_date_part__plan0_optimized.sql | 2 +- ...st_offset_window_with_date_part__plan0.sql | 2 +- ...window_with_date_part__plan0_optimized.sql | 2 +- ...st_offset_window_with_date_part__plan0.sql | 2 +- ...window_with_date_part__plan0_optimized.sql | 2 +- ...uery_with_metric_time_dimension__plan0.sql | 2 +- ...metric_time_dimension__plan0_optimized.sql | 2 +- ...uery_with_metric_time_dimension__plan0.sql | 2 +- ...metric_time_dimension__plan0_optimized.sql | 2 +- ...uery_with_metric_time_dimension__plan0.sql | 2 +- ...metric_time_dimension__plan0_optimized.sql | 2 +- ...uery_with_metric_time_dimension__plan0.sql | 2 +- ...metric_time_dimension__plan0_optimized.sql | 2 +- ...uery_with_metric_time_dimension__plan0.sql | 2 +- ...metric_time_dimension__plan0_optimized.sql | 2 +- ...uery_with_metric_time_dimension__plan0.sql | 2 +- ...metric_time_dimension__plan0_optimized.sql | 2 +- ...uery_with_metric_time_dimension__plan0.xml | 2 +- .../test_common_semantic_model__plan0.sql | 2 +- ...common_semantic_model__plan0_optimized.sql | 2 +- .../test_measure_constraint__plan0.sql | 2 +- ...st_measure_constraint__plan0_optimized.sql | 2 +- ..._constraint_with_reused_measure__plan0.sql | 2 +- ...t_with_reused_measure__plan0_optimized.sql | 2 +- ..._multiple_sources_no_dimensions__plan0.sql | 2 +- ...sources_no_dimensions__plan0_optimized.sql | 2 +- ..._multiple_metrics_no_dimensions__plan0.sql | 2 +- ...metrics_no_dimensions__plan0_optimized.sql | 2 +- .../test_common_semantic_model__plan0.sql | 2 +- ...common_semantic_model__plan0_optimized.sql | 2 +- .../test_measure_constraint__plan0.sql | 2 +- ...st_measure_constraint__plan0_optimized.sql | 2 +- ..._constraint_with_reused_measure__plan0.sql | 2 +- ...t_with_reused_measure__plan0_optimized.sql | 2 +- ..._multiple_sources_no_dimensions__plan0.sql | 2 +- ...sources_no_dimensions__plan0_optimized.sql | 2 +- ..._multiple_metrics_no_dimensions__plan0.sql | 2 +- ...metrics_no_dimensions__plan0_optimized.sql | 2 +- .../test_common_semantic_model__plan0.sql | 2 +- ...common_semantic_model__plan0_optimized.sql | 2 +- .../DuckDB/test_measure_constraint__plan0.sql | 2 +- ...st_measure_constraint__plan0_optimized.sql | 2 +- ..._constraint_with_reused_measure__plan0.sql | 2 +- ...t_with_reused_measure__plan0_optimized.sql | 2 +- ..._multiple_sources_no_dimensions__plan0.sql | 2 +- ...sources_no_dimensions__plan0_optimized.sql | 2 +- ..._multiple_metrics_no_dimensions__plan0.sql | 2 +- ...metrics_no_dimensions__plan0_optimized.sql | 2 +- .../test_common_semantic_model__plan0.sql | 2 +- ...common_semantic_model__plan0_optimized.sql | 2 +- .../test_measure_constraint__plan0.sql | 2 +- ...st_measure_constraint__plan0_optimized.sql | 2 +- ..._constraint_with_reused_measure__plan0.sql | 2 +- ...t_with_reused_measure__plan0_optimized.sql | 2 +- ..._multiple_sources_no_dimensions__plan0.sql | 2 +- ...sources_no_dimensions__plan0_optimized.sql | 2 +- ..._multiple_metrics_no_dimensions__plan0.sql | 2 +- ...metrics_no_dimensions__plan0_optimized.sql | 2 +- .../test_common_semantic_model__plan0.sql | 2 +- ...common_semantic_model__plan0_optimized.sql | 2 +- .../test_measure_constraint__plan0.sql | 2 +- ...st_measure_constraint__plan0_optimized.sql | 2 +- ..._constraint_with_reused_measure__plan0.sql | 2 +- ...t_with_reused_measure__plan0_optimized.sql | 2 +- ..._multiple_sources_no_dimensions__plan0.sql | 2 +- ...sources_no_dimensions__plan0_optimized.sql | 2 +- ..._multiple_metrics_no_dimensions__plan0.sql | 2 +- ...metrics_no_dimensions__plan0_optimized.sql | 2 +- .../test_common_semantic_model__plan0.sql | 2 +- ...common_semantic_model__plan0_optimized.sql | 2 +- .../test_measure_constraint__plan0.sql | 2 +- ...st_measure_constraint__plan0_optimized.sql | 2 +- ..._constraint_with_reused_measure__plan0.sql | 2 +- ...t_with_reused_measure__plan0_optimized.sql | 2 +- ..._multiple_sources_no_dimensions__plan0.sql | 2 +- ...sources_no_dimensions__plan0_optimized.sql | 2 +- ..._multiple_metrics_no_dimensions__plan0.sql | 2 +- ...metrics_no_dimensions__plan0_optimized.sql | 2 +- ...2_metrics_from_1_semantic_model__dfp_0.xml | 20 +- ..._metrics_from_1_semantic_model__dfpo_0.xml | 12 +- ..._metrics_from_2_semantic_models__dfp_0.xml | 14 +- ...metrics_from_2_semantic_models__dfpo_0.xml | 14 +- ...o_metrics_from_1_semantic_model__dfp_0.xml | 36 +-- ..._metrics_from_1_semantic_model__dfpo_0.xml | 9 +- ..._metrics_from_2_semantic_models__dfp_0.xml | 17 +- ...metrics_from_2_semantic_models__dfpo_0.xml | 17 +- ...constrained_metric_not_combined__dfp_0.xml | 17 +- ...onstrained_metric_not_combined__dfpo_0.xml | 17 +- .../test_derived_metric__dfp_0.xml | 14 +- .../test_derived_metric__dfpo_0.xml | 6 +- ..._metric_with_non_derived_metric__dfp_0.xml | 25 +- ...metric_with_non_derived_metric__dfpo_0.xml | 17 +- .../test_nested_derived_metric__dfp_0.xml | 28 ++- .../test_nested_derived_metric__dfpo_0.xml | 20 +- 288 files changed, 1930 insertions(+), 451 deletions(-) create mode 100644 metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/BigQuery/test_combine_output_node__plan0.sql create mode 100644 metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/BigQuery/test_combine_output_node__plan0_optimized.sql create mode 100644 metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Databricks/test_combine_output_node__plan0.sql create mode 100644 metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Databricks/test_combine_output_node__plan0_optimized.sql create mode 100644 metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Postgres/test_combine_output_node__plan0.sql create mode 100644 metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Postgres/test_combine_output_node__plan0_optimized.sql create mode 100644 metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Redshift/test_combine_output_node__plan0.sql create mode 100644 metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Redshift/test_combine_output_node__plan0_optimized.sql create mode 100644 metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Snowflake/test_combine_output_node__plan0.sql create mode 100644 metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Snowflake/test_combine_output_node__plan0_optimized.sql diff --git a/metricflow/test/snapshots/test_cm_branch_combiner.py/DataflowPlan/test_filter_combination__dfpo_0.xml b/metricflow/test/snapshots/test_cm_branch_combiner.py/DataflowPlan/test_filter_combination__dfpo_0.xml index 8b44ecbdd0..c5408bce5b 100644 --- a/metricflow/test/snapshots/test_cm_branch_combiner.py/DataflowPlan/test_filter_combination__dfpo_0.xml +++ b/metricflow/test/snapshots/test_cm_branch_combiner.py/DataflowPlan/test_filter_combination__dfpo_0.xml @@ -10,11 +10,13 @@ - + + - + + diff --git a/metricflow/test/snapshots/test_cyclic_join.py/DataflowPlan/test_cyclic_join__dfp_0.xml b/metricflow/test/snapshots/test_cyclic_join.py/DataflowPlan/test_cyclic_join__dfp_0.xml index fe97bc8b8a..40e78e3dac 100644 --- a/metricflow/test/snapshots/test_cyclic_join.py/DataflowPlan/test_cyclic_join__dfp_0.xml +++ b/metricflow/test/snapshots/test_cyclic_join.py/DataflowPlan/test_cyclic_join__dfp_0.xml @@ -23,7 +23,8 @@ - + + @@ -51,7 +52,8 @@ - + + diff --git a/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_common_semantic_model__dfp_0.xml b/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_common_semantic_model__dfp_0.xml index 1f786abff0..a22f6531b8 100644 --- a/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_common_semantic_model__dfp_0.xml +++ b/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_common_semantic_model__dfp_0.xml @@ -2,9 +2,9 @@ - - - + + + @@ -26,7 +26,8 @@ - + + @@ -60,7 +61,8 @@ - + + @@ -130,7 +132,8 @@ - + + @@ -164,7 +167,8 @@ - + + @@ -213,6 +217,6 @@ - + diff --git a/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_cumulative_metric_no_window_or_grain_with_metric_time__dfp_0.xml b/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_cumulative_metric_no_window_or_grain_with_metric_time__dfp_0.xml index b25be80860..eec481fede 100644 --- a/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_cumulative_metric_no_window_or_grain_with_metric_time__dfp_0.xml +++ b/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_cumulative_metric_no_window_or_grain_with_metric_time__dfp_0.xml @@ -23,7 +23,8 @@ - + + diff --git a/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_cumulative_metric_no_window_or_grain_without_metric_time__dfp_0.xml b/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_cumulative_metric_no_window_or_grain_without_metric_time__dfp_0.xml index 3b58bd51b7..482446a4f0 100644 --- a/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_cumulative_metric_no_window_or_grain_without_metric_time__dfp_0.xml +++ b/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_cumulative_metric_no_window_or_grain_without_metric_time__dfp_0.xml @@ -23,7 +23,8 @@ - + + diff --git a/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_cumulative_metric_with_window__dfp_0.xml b/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_cumulative_metric_with_window__dfp_0.xml index 12efb1df0d..385ebf8855 100644 --- a/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_cumulative_metric_with_window__dfp_0.xml +++ b/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_cumulative_metric_with_window__dfp_0.xml @@ -23,7 +23,8 @@ - + + diff --git a/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_derived_metric_offset_to_grain__dfp_0.xml b/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_derived_metric_offset_to_grain__dfp_0.xml index 6e7a05c15d..6c9913097d 100644 --- a/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_derived_metric_offset_to_grain__dfp_0.xml +++ b/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_derived_metric_offset_to_grain__dfp_0.xml @@ -12,9 +12,9 @@ - - - + + + @@ -36,7 +36,8 @@ - + + @@ -81,7 +82,8 @@ - + + @@ -120,7 +122,7 @@ - + diff --git a/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_derived_metric_offset_window__dfp_0.xml b/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_derived_metric_offset_window__dfp_0.xml index a3ee6dc4f1..c124ff3b33 100644 --- a/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_derived_metric_offset_window__dfp_0.xml +++ b/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_derived_metric_offset_window__dfp_0.xml @@ -35,7 +35,8 @@ - + + diff --git a/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_derived_metric_offset_with_granularity__dfp_0.xml b/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_derived_metric_offset_with_granularity__dfp_0.xml index ff144a368d..3e4efaa3bc 100644 --- a/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_derived_metric_offset_with_granularity__dfp_0.xml +++ b/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_derived_metric_offset_with_granularity__dfp_0.xml @@ -35,7 +35,8 @@ - + + diff --git a/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_derived_offset_cumulative_metric__dfp_0.xml b/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_derived_offset_cumulative_metric__dfp_0.xml index e2968a3ac6..28e1ba6a80 100644 --- a/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_derived_offset_cumulative_metric__dfp_0.xml +++ b/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_derived_offset_cumulative_metric__dfp_0.xml @@ -35,7 +35,8 @@ - + + diff --git a/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_dont_join_to_time_spine_if_no_time_dimension_requested__dfp_0.xml b/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_dont_join_to_time_spine_if_no_time_dimension_requested__dfp_0.xml index c0e3abf921..f4fc83bc2b 100644 --- a/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_dont_join_to_time_spine_if_no_time_dimension_requested__dfp_0.xml +++ b/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_dont_join_to_time_spine_if_no_time_dimension_requested__dfp_0.xml @@ -23,7 +23,8 @@ - + + diff --git a/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_join_to_time_spine_derived_metric__dfp_0.xml b/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_join_to_time_spine_derived_metric__dfp_0.xml index ba0faa72af..ba96239d86 100644 --- a/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_join_to_time_spine_derived_metric__dfp_0.xml +++ b/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_join_to_time_spine_derived_metric__dfp_0.xml @@ -12,9 +12,9 @@ - - - + + + @@ -50,7 +50,8 @@ - + + @@ -112,7 +113,8 @@ - + + @@ -152,7 +154,7 @@ - + diff --git a/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_join_to_time_spine_with_metric_time__dfp_0.xml b/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_join_to_time_spine_with_metric_time__dfp_0.xml index a6b5cac816..77a4943244 100644 --- a/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_join_to_time_spine_with_metric_time__dfp_0.xml +++ b/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_join_to_time_spine_with_metric_time__dfp_0.xml @@ -37,7 +37,8 @@ - + + diff --git a/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_join_to_time_spine_with_non_metric_time__dfp_0.xml b/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_join_to_time_spine_with_non_metric_time__dfp_0.xml index fc88a85f3c..55b6fc1e28 100644 --- a/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_join_to_time_spine_with_non_metric_time__dfp_0.xml +++ b/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_join_to_time_spine_with_non_metric_time__dfp_0.xml @@ -23,7 +23,8 @@ - + + diff --git a/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_joined_plan__dfp_0.xml b/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_joined_plan__dfp_0.xml index e30c82f587..2e6ad5159b 100644 --- a/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_joined_plan__dfp_0.xml +++ b/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_joined_plan__dfp_0.xml @@ -23,7 +23,8 @@ - + + @@ -54,7 +55,8 @@ - + + diff --git a/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_limit_rows_plan__dfp_0.xml b/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_limit_rows_plan__dfp_0.xml index b4fd8b7143..193da08216 100644 --- a/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_limit_rows_plan__dfp_0.xml +++ b/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_limit_rows_plan__dfp_0.xml @@ -27,7 +27,8 @@ - + + diff --git a/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_measure_constraint_plan__dfp_0.xml b/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_measure_constraint_plan__dfp_0.xml index ff0aefadcc..749d39917d 100644 --- a/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_measure_constraint_plan__dfp_0.xml +++ b/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_measure_constraint_plan__dfp_0.xml @@ -12,9 +12,9 @@ - - - + + + @@ -46,7 +46,8 @@ - + + @@ -77,7 +78,8 @@ - + + @@ -111,7 +113,8 @@ - + + @@ -192,7 +195,8 @@ - + + @@ -223,7 +227,8 @@ - + + @@ -257,7 +262,8 @@ - + + @@ -328,7 +334,8 @@ - + + @@ -352,7 +359,7 @@ - + diff --git a/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_measure_constraint_with_reused_measure_plan__dfp_0.xml b/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_measure_constraint_with_reused_measure_plan__dfp_0.xml index 37e1a3123e..8d146e3041 100644 --- a/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_measure_constraint_with_reused_measure_plan__dfp_0.xml +++ b/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_measure_constraint_with_reused_measure_plan__dfp_0.xml @@ -12,9 +12,9 @@ - - - + + + @@ -46,7 +46,8 @@ - + + @@ -77,7 +78,8 @@ - + + @@ -128,7 +130,8 @@ - + + @@ -152,7 +155,7 @@ - + diff --git a/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_multi_semantic_model_ratio_metrics_plan__dfp_0.xml b/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_multi_semantic_model_ratio_metrics_plan__dfp_0.xml index 8bb478d6ca..ddf15aebc4 100644 --- a/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_multi_semantic_model_ratio_metrics_plan__dfp_0.xml +++ b/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_multi_semantic_model_ratio_metrics_plan__dfp_0.xml @@ -12,9 +12,9 @@ - - - + + + @@ -36,7 +36,8 @@ - + + @@ -70,7 +71,8 @@ - + + @@ -140,7 +142,8 @@ - + + @@ -174,7 +177,8 @@ - + + @@ -223,7 +227,7 @@ - + diff --git a/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_multihop_join_plan__dfp_0.xml b/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_multihop_join_plan__dfp_0.xml index 70d034e35b..c7808c613a 100644 --- a/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_multihop_join_plan__dfp_0.xml +++ b/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_multihop_join_plan__dfp_0.xml @@ -23,7 +23,8 @@ - + + @@ -63,7 +64,8 @@ - + + diff --git a/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_multiple_metrics_plan__dfp_0.xml b/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_multiple_metrics_plan__dfp_0.xml index 455487089d..1ff685a04e 100644 --- a/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_multiple_metrics_plan__dfp_0.xml +++ b/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_multiple_metrics_plan__dfp_0.xml @@ -2,9 +2,9 @@ - - - + + + @@ -26,7 +26,8 @@ - + + @@ -75,7 +76,8 @@ - + + @@ -103,6 +105,6 @@ - + diff --git a/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_order_by_plan__dfp_0.xml b/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_order_by_plan__dfp_0.xml index 86c6ccefde..ae588dbc5e 100644 --- a/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_order_by_plan__dfp_0.xml +++ b/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_order_by_plan__dfp_0.xml @@ -45,7 +45,8 @@ - + + diff --git a/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_primary_entity_dimension__dfp_0.xml b/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_primary_entity_dimension__dfp_0.xml index ccb2dd4669..a8d8373588 100644 --- a/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_primary_entity_dimension__dfp_0.xml +++ b/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_primary_entity_dimension__dfp_0.xml @@ -23,7 +23,8 @@ - + + diff --git a/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_simple_plan__dfp_0.xml b/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_simple_plan__dfp_0.xml index ccb2dd4669..a8d8373588 100644 --- a/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_simple_plan__dfp_0.xml +++ b/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_simple_plan__dfp_0.xml @@ -23,7 +23,8 @@ - + + diff --git a/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_single_semantic_model_ratio_metrics_plan__dfp_0.xml b/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_single_semantic_model_ratio_metrics_plan__dfp_0.xml index f9d0c651ae..ee7a40eceb 100644 --- a/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_single_semantic_model_ratio_metrics_plan__dfp_0.xml +++ b/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_single_semantic_model_ratio_metrics_plan__dfp_0.xml @@ -12,9 +12,9 @@ - - - + + + @@ -36,7 +36,8 @@ - + + @@ -70,7 +71,8 @@ - + + @@ -140,7 +142,8 @@ - + + @@ -174,7 +177,8 @@ - + + @@ -223,7 +227,7 @@ - + diff --git a/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_where_constrained_plan__dfp_0.xml b/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_where_constrained_plan__dfp_0.xml index e7d35170b0..14d24551f2 100644 --- a/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_where_constrained_plan__dfp_0.xml +++ b/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_where_constrained_plan__dfp_0.xml @@ -23,7 +23,8 @@ - + + @@ -51,7 +52,8 @@ - + + @@ -82,7 +84,8 @@ - + + diff --git a/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_where_constrained_plan_time_dimension__dfp_0.xml b/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_where_constrained_plan_time_dimension__dfp_0.xml index b8248be3ce..c9a38b5cb2 100644 --- a/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_where_constrained_plan_time_dimension__dfp_0.xml +++ b/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_where_constrained_plan_time_dimension__dfp_0.xml @@ -23,7 +23,8 @@ - + + @@ -53,7 +54,8 @@ - + + diff --git a/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_where_constrained_with_common_linkable_plan__dfp_0.xml b/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_where_constrained_with_common_linkable_plan__dfp_0.xml index 58ceffeb67..1fc2c1f0e1 100644 --- a/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_where_constrained_with_common_linkable_plan__dfp_0.xml +++ b/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_where_constrained_with_common_linkable_plan__dfp_0.xml @@ -37,7 +37,8 @@ - + + @@ -64,7 +65,8 @@ - + + diff --git a/metricflow/test/snapshots/test_dataflow_to_execution.py/ExecutionPlan/BigQuery/test_combined_metrics_plan__ep_0.xml b/metricflow/test/snapshots/test_dataflow_to_execution.py/ExecutionPlan/BigQuery/test_combined_metrics_plan__ep_0.xml index d9513dae5f..52dac71ac2 100644 --- a/metricflow/test/snapshots/test_dataflow_to_execution.py/ExecutionPlan/BigQuery/test_combined_metrics_plan__ep_0.xml +++ b/metricflow/test/snapshots/test_dataflow_to_execution.py/ExecutionPlan/BigQuery/test_combined_metrics_plan__ep_0.xml @@ -3,7 +3,7 @@ - + diff --git a/metricflow/test/snapshots/test_dataflow_to_execution.py/ExecutionPlan/BigQuery/test_small_combined_metrics_plan__ep_0.xml b/metricflow/test/snapshots/test_dataflow_to_execution.py/ExecutionPlan/BigQuery/test_small_combined_metrics_plan__ep_0.xml index 39598117e1..186902a81c 100644 --- a/metricflow/test/snapshots/test_dataflow_to_execution.py/ExecutionPlan/BigQuery/test_small_combined_metrics_plan__ep_0.xml +++ b/metricflow/test/snapshots/test_dataflow_to_execution.py/ExecutionPlan/BigQuery/test_small_combined_metrics_plan__ep_0.xml @@ -3,7 +3,7 @@ - + diff --git a/metricflow/test/snapshots/test_dataflow_to_execution.py/ExecutionPlan/Databricks/test_combined_metrics_plan__ep_0.xml b/metricflow/test/snapshots/test_dataflow_to_execution.py/ExecutionPlan/Databricks/test_combined_metrics_plan__ep_0.xml index 1768aeaecd..bdab0b0d91 100644 --- a/metricflow/test/snapshots/test_dataflow_to_execution.py/ExecutionPlan/Databricks/test_combined_metrics_plan__ep_0.xml +++ b/metricflow/test/snapshots/test_dataflow_to_execution.py/ExecutionPlan/Databricks/test_combined_metrics_plan__ep_0.xml @@ -3,7 +3,7 @@ - + diff --git a/metricflow/test/snapshots/test_dataflow_to_execution.py/ExecutionPlan/Databricks/test_small_combined_metrics_plan__ep_0.xml b/metricflow/test/snapshots/test_dataflow_to_execution.py/ExecutionPlan/Databricks/test_small_combined_metrics_plan__ep_0.xml index 53612723d0..57ca2f3d3c 100644 --- a/metricflow/test/snapshots/test_dataflow_to_execution.py/ExecutionPlan/Databricks/test_small_combined_metrics_plan__ep_0.xml +++ b/metricflow/test/snapshots/test_dataflow_to_execution.py/ExecutionPlan/Databricks/test_small_combined_metrics_plan__ep_0.xml @@ -3,7 +3,7 @@ - + diff --git a/metricflow/test/snapshots/test_dataflow_to_execution.py/ExecutionPlan/DuckDB/test_combined_metrics_plan__ep_0.xml b/metricflow/test/snapshots/test_dataflow_to_execution.py/ExecutionPlan/DuckDB/test_combined_metrics_plan__ep_0.xml index 1768aeaecd..bdab0b0d91 100644 --- a/metricflow/test/snapshots/test_dataflow_to_execution.py/ExecutionPlan/DuckDB/test_combined_metrics_plan__ep_0.xml +++ b/metricflow/test/snapshots/test_dataflow_to_execution.py/ExecutionPlan/DuckDB/test_combined_metrics_plan__ep_0.xml @@ -3,7 +3,7 @@ - + diff --git a/metricflow/test/snapshots/test_dataflow_to_execution.py/ExecutionPlan/DuckDB/test_small_combined_metrics_plan__ep_0.xml b/metricflow/test/snapshots/test_dataflow_to_execution.py/ExecutionPlan/DuckDB/test_small_combined_metrics_plan__ep_0.xml index 53612723d0..57ca2f3d3c 100644 --- a/metricflow/test/snapshots/test_dataflow_to_execution.py/ExecutionPlan/DuckDB/test_small_combined_metrics_plan__ep_0.xml +++ b/metricflow/test/snapshots/test_dataflow_to_execution.py/ExecutionPlan/DuckDB/test_small_combined_metrics_plan__ep_0.xml @@ -3,7 +3,7 @@ - + diff --git a/metricflow/test/snapshots/test_dataflow_to_execution.py/ExecutionPlan/Postgres/test_combined_metrics_plan__ep_0.xml b/metricflow/test/snapshots/test_dataflow_to_execution.py/ExecutionPlan/Postgres/test_combined_metrics_plan__ep_0.xml index 1768aeaecd..bdab0b0d91 100644 --- a/metricflow/test/snapshots/test_dataflow_to_execution.py/ExecutionPlan/Postgres/test_combined_metrics_plan__ep_0.xml +++ b/metricflow/test/snapshots/test_dataflow_to_execution.py/ExecutionPlan/Postgres/test_combined_metrics_plan__ep_0.xml @@ -3,7 +3,7 @@ - + diff --git a/metricflow/test/snapshots/test_dataflow_to_execution.py/ExecutionPlan/Postgres/test_small_combined_metrics_plan__ep_0.xml b/metricflow/test/snapshots/test_dataflow_to_execution.py/ExecutionPlan/Postgres/test_small_combined_metrics_plan__ep_0.xml index 53612723d0..57ca2f3d3c 100644 --- a/metricflow/test/snapshots/test_dataflow_to_execution.py/ExecutionPlan/Postgres/test_small_combined_metrics_plan__ep_0.xml +++ b/metricflow/test/snapshots/test_dataflow_to_execution.py/ExecutionPlan/Postgres/test_small_combined_metrics_plan__ep_0.xml @@ -3,7 +3,7 @@ - + diff --git a/metricflow/test/snapshots/test_dataflow_to_execution.py/ExecutionPlan/Redshift/test_combined_metrics_plan__ep_0.xml b/metricflow/test/snapshots/test_dataflow_to_execution.py/ExecutionPlan/Redshift/test_combined_metrics_plan__ep_0.xml index 1768aeaecd..bdab0b0d91 100644 --- a/metricflow/test/snapshots/test_dataflow_to_execution.py/ExecutionPlan/Redshift/test_combined_metrics_plan__ep_0.xml +++ b/metricflow/test/snapshots/test_dataflow_to_execution.py/ExecutionPlan/Redshift/test_combined_metrics_plan__ep_0.xml @@ -3,7 +3,7 @@ - + diff --git a/metricflow/test/snapshots/test_dataflow_to_execution.py/ExecutionPlan/Redshift/test_small_combined_metrics_plan__ep_0.xml b/metricflow/test/snapshots/test_dataflow_to_execution.py/ExecutionPlan/Redshift/test_small_combined_metrics_plan__ep_0.xml index 53612723d0..57ca2f3d3c 100644 --- a/metricflow/test/snapshots/test_dataflow_to_execution.py/ExecutionPlan/Redshift/test_small_combined_metrics_plan__ep_0.xml +++ b/metricflow/test/snapshots/test_dataflow_to_execution.py/ExecutionPlan/Redshift/test_small_combined_metrics_plan__ep_0.xml @@ -3,7 +3,7 @@ - + diff --git a/metricflow/test/snapshots/test_dataflow_to_execution.py/ExecutionPlan/Snowflake/test_combined_metrics_plan__ep_0.xml b/metricflow/test/snapshots/test_dataflow_to_execution.py/ExecutionPlan/Snowflake/test_combined_metrics_plan__ep_0.xml index 1768aeaecd..bdab0b0d91 100644 --- a/metricflow/test/snapshots/test_dataflow_to_execution.py/ExecutionPlan/Snowflake/test_combined_metrics_plan__ep_0.xml +++ b/metricflow/test/snapshots/test_dataflow_to_execution.py/ExecutionPlan/Snowflake/test_combined_metrics_plan__ep_0.xml @@ -3,7 +3,7 @@ - + diff --git a/metricflow/test/snapshots/test_dataflow_to_execution.py/ExecutionPlan/Snowflake/test_small_combined_metrics_plan__ep_0.xml b/metricflow/test/snapshots/test_dataflow_to_execution.py/ExecutionPlan/Snowflake/test_small_combined_metrics_plan__ep_0.xml index 53612723d0..57ca2f3d3c 100644 --- a/metricflow/test/snapshots/test_dataflow_to_execution.py/ExecutionPlan/Snowflake/test_small_combined_metrics_plan__ep_0.xml +++ b/metricflow/test/snapshots/test_dataflow_to_execution.py/ExecutionPlan/Snowflake/test_small_combined_metrics_plan__ep_0.xml @@ -3,7 +3,7 @@ - + diff --git a/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/DataflowPlan/test_compute_metrics_node_simple_expr__plan0.xml b/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/DataflowPlan/test_compute_metrics_node_simple_expr__plan0.xml index a03c45a687..c56df6a05c 100644 --- a/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/DataflowPlan/test_compute_metrics_node_simple_expr__plan0.xml +++ b/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/DataflowPlan/test_compute_metrics_node_simple_expr__plan0.xml @@ -36,7 +36,8 @@ - + + diff --git a/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/DataflowPlan/test_join_to_time_spine_node_with_offset_to_grain__plan0.xml b/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/DataflowPlan/test_join_to_time_spine_node_with_offset_to_grain__plan0.xml index 0d03e41f1c..856fc27383 100644 --- a/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/DataflowPlan/test_join_to_time_spine_node_with_offset_to_grain__plan0.xml +++ b/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/DataflowPlan/test_join_to_time_spine_node_with_offset_to_grain__plan0.xml @@ -37,7 +37,8 @@ - + + diff --git a/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/DataflowPlan/test_join_to_time_spine_node_with_offset_window__plan0.xml b/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/DataflowPlan/test_join_to_time_spine_node_with_offset_window__plan0.xml index 00cb173b2c..9bb30f61b6 100644 --- a/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/DataflowPlan/test_join_to_time_spine_node_with_offset_window__plan0.xml +++ b/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/DataflowPlan/test_join_to_time_spine_node_with_offset_window__plan0.xml @@ -37,7 +37,8 @@ - + + diff --git a/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/DataflowPlan/test_join_to_time_spine_node_without_offset__plan0.xml b/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/DataflowPlan/test_join_to_time_spine_node_without_offset__plan0.xml index 187ce156e4..a6a962c4eb 100644 --- a/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/DataflowPlan/test_join_to_time_spine_node_without_offset__plan0.xml +++ b/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/DataflowPlan/test_join_to_time_spine_node_without_offset__plan0.xml @@ -37,7 +37,8 @@ - + + diff --git a/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/BigQuery/test_combine_output_node__plan0.sql b/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/BigQuery/test_combine_output_node__plan0.sql new file mode 100644 index 0000000000..27ccba6239 --- /dev/null +++ b/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/BigQuery/test_combine_output_node__plan0.sql @@ -0,0 +1,228 @@ +-- Combine Aggregated Outputs +SELECT + COALESCE(subq_2.is_instant, subq_5.is_instant) AS is_instant + , MAX(subq_2.bookings) AS bookings + , COALESCE(MAX(subq_5.instant_bookings), 1) AS instant_bookings + , COALESCE(MAX(subq_5.bookers), 1) AS bookers +FROM ( + -- Aggregate Measures + SELECT + subq_1.is_instant + , SUM(subq_1.bookings) AS bookings + FROM ( + -- Pass Only Elements: + -- ['bookings', 'is_instant'] + SELECT + subq_0.is_instant + , subq_0.bookings + FROM ( + -- Read Elements From Semantic Model 'bookings_source' + SELECT + 1 AS bookings + , CASE WHEN is_instant THEN 1 ELSE 0 END AS instant_bookings + , bookings_source_src_10001.booking_value + , bookings_source_src_10001.booking_value AS max_booking_value + , bookings_source_src_10001.booking_value AS min_booking_value + , bookings_source_src_10001.guest_id AS bookers + , bookings_source_src_10001.booking_value AS average_booking_value + , bookings_source_src_10001.booking_value AS booking_payments + , CASE WHEN referrer_id IS NOT NULL THEN 1 ELSE 0 END AS referred_bookings + , bookings_source_src_10001.booking_value AS median_booking_value + , bookings_source_src_10001.booking_value AS booking_value_p99 + , bookings_source_src_10001.booking_value AS discrete_booking_value_p99 + , bookings_source_src_10001.booking_value AS approximate_continuous_booking_value_p99 + , bookings_source_src_10001.booking_value AS approximate_discrete_booking_value_p99 + , bookings_source_src_10001.is_instant + , DATE_TRUNC(bookings_source_src_10001.ds, day) AS ds__day + , DATE_TRUNC(bookings_source_src_10001.ds, isoweek) AS ds__week + , DATE_TRUNC(bookings_source_src_10001.ds, month) AS ds__month + , DATE_TRUNC(bookings_source_src_10001.ds, quarter) AS ds__quarter + , DATE_TRUNC(bookings_source_src_10001.ds, year) AS ds__year + , EXTRACT(year FROM bookings_source_src_10001.ds) AS ds__extract_year + , EXTRACT(quarter FROM bookings_source_src_10001.ds) AS ds__extract_quarter + , EXTRACT(month FROM bookings_source_src_10001.ds) AS ds__extract_month + , EXTRACT(day FROM bookings_source_src_10001.ds) AS ds__extract_day + , IF(EXTRACT(dayofweek FROM bookings_source_src_10001.ds) = 1, 7, EXTRACT(dayofweek FROM bookings_source_src_10001.ds) - 1) AS ds__extract_dow + , EXTRACT(dayofyear FROM bookings_source_src_10001.ds) AS ds__extract_doy + , DATE_TRUNC(bookings_source_src_10001.ds_partitioned, day) AS ds_partitioned__day + , DATE_TRUNC(bookings_source_src_10001.ds_partitioned, isoweek) AS ds_partitioned__week + , DATE_TRUNC(bookings_source_src_10001.ds_partitioned, month) AS ds_partitioned__month + , DATE_TRUNC(bookings_source_src_10001.ds_partitioned, quarter) AS ds_partitioned__quarter + , DATE_TRUNC(bookings_source_src_10001.ds_partitioned, year) AS ds_partitioned__year + , EXTRACT(year FROM bookings_source_src_10001.ds_partitioned) AS ds_partitioned__extract_year + , EXTRACT(quarter FROM bookings_source_src_10001.ds_partitioned) AS ds_partitioned__extract_quarter + , EXTRACT(month FROM bookings_source_src_10001.ds_partitioned) AS ds_partitioned__extract_month + , EXTRACT(day FROM bookings_source_src_10001.ds_partitioned) AS ds_partitioned__extract_day + , IF(EXTRACT(dayofweek FROM bookings_source_src_10001.ds_partitioned) = 1, 7, EXTRACT(dayofweek FROM bookings_source_src_10001.ds_partitioned) - 1) AS ds_partitioned__extract_dow + , EXTRACT(dayofyear FROM bookings_source_src_10001.ds_partitioned) AS ds_partitioned__extract_doy + , DATE_TRUNC(bookings_source_src_10001.paid_at, day) AS paid_at__day + , DATE_TRUNC(bookings_source_src_10001.paid_at, isoweek) AS paid_at__week + , DATE_TRUNC(bookings_source_src_10001.paid_at, month) AS paid_at__month + , DATE_TRUNC(bookings_source_src_10001.paid_at, quarter) AS paid_at__quarter + , DATE_TRUNC(bookings_source_src_10001.paid_at, year) AS paid_at__year + , EXTRACT(year FROM bookings_source_src_10001.paid_at) AS paid_at__extract_year + , EXTRACT(quarter FROM bookings_source_src_10001.paid_at) AS paid_at__extract_quarter + , EXTRACT(month FROM bookings_source_src_10001.paid_at) AS paid_at__extract_month + , EXTRACT(day FROM bookings_source_src_10001.paid_at) AS paid_at__extract_day + , IF(EXTRACT(dayofweek FROM bookings_source_src_10001.paid_at) = 1, 7, EXTRACT(dayofweek FROM bookings_source_src_10001.paid_at) - 1) AS paid_at__extract_dow + , EXTRACT(dayofyear FROM bookings_source_src_10001.paid_at) AS paid_at__extract_doy + , bookings_source_src_10001.is_instant AS booking__is_instant + , DATE_TRUNC(bookings_source_src_10001.ds, day) AS booking__ds__day + , DATE_TRUNC(bookings_source_src_10001.ds, isoweek) AS booking__ds__week + , DATE_TRUNC(bookings_source_src_10001.ds, month) AS booking__ds__month + , DATE_TRUNC(bookings_source_src_10001.ds, quarter) AS booking__ds__quarter + , DATE_TRUNC(bookings_source_src_10001.ds, year) AS booking__ds__year + , EXTRACT(year FROM bookings_source_src_10001.ds) AS booking__ds__extract_year + , EXTRACT(quarter FROM bookings_source_src_10001.ds) AS booking__ds__extract_quarter + , EXTRACT(month FROM bookings_source_src_10001.ds) AS booking__ds__extract_month + , EXTRACT(day FROM bookings_source_src_10001.ds) AS booking__ds__extract_day + , IF(EXTRACT(dayofweek FROM bookings_source_src_10001.ds) = 1, 7, EXTRACT(dayofweek FROM bookings_source_src_10001.ds) - 1) AS booking__ds__extract_dow + , EXTRACT(dayofyear FROM bookings_source_src_10001.ds) AS booking__ds__extract_doy + , DATE_TRUNC(bookings_source_src_10001.ds_partitioned, day) AS booking__ds_partitioned__day + , DATE_TRUNC(bookings_source_src_10001.ds_partitioned, isoweek) AS booking__ds_partitioned__week + , DATE_TRUNC(bookings_source_src_10001.ds_partitioned, month) AS booking__ds_partitioned__month + , DATE_TRUNC(bookings_source_src_10001.ds_partitioned, quarter) AS booking__ds_partitioned__quarter + , DATE_TRUNC(bookings_source_src_10001.ds_partitioned, year) AS booking__ds_partitioned__year + , EXTRACT(year FROM bookings_source_src_10001.ds_partitioned) AS booking__ds_partitioned__extract_year + , EXTRACT(quarter FROM bookings_source_src_10001.ds_partitioned) AS booking__ds_partitioned__extract_quarter + , EXTRACT(month FROM bookings_source_src_10001.ds_partitioned) AS booking__ds_partitioned__extract_month + , EXTRACT(day FROM bookings_source_src_10001.ds_partitioned) AS booking__ds_partitioned__extract_day + , IF(EXTRACT(dayofweek FROM bookings_source_src_10001.ds_partitioned) = 1, 7, EXTRACT(dayofweek FROM bookings_source_src_10001.ds_partitioned) - 1) AS booking__ds_partitioned__extract_dow + , EXTRACT(dayofyear FROM bookings_source_src_10001.ds_partitioned) AS booking__ds_partitioned__extract_doy + , DATE_TRUNC(bookings_source_src_10001.paid_at, day) AS booking__paid_at__day + , DATE_TRUNC(bookings_source_src_10001.paid_at, isoweek) AS booking__paid_at__week + , DATE_TRUNC(bookings_source_src_10001.paid_at, month) AS booking__paid_at__month + , DATE_TRUNC(bookings_source_src_10001.paid_at, quarter) AS booking__paid_at__quarter + , DATE_TRUNC(bookings_source_src_10001.paid_at, year) AS booking__paid_at__year + , EXTRACT(year FROM bookings_source_src_10001.paid_at) AS booking__paid_at__extract_year + , EXTRACT(quarter FROM bookings_source_src_10001.paid_at) AS booking__paid_at__extract_quarter + , EXTRACT(month FROM bookings_source_src_10001.paid_at) AS booking__paid_at__extract_month + , EXTRACT(day FROM bookings_source_src_10001.paid_at) AS booking__paid_at__extract_day + , IF(EXTRACT(dayofweek FROM bookings_source_src_10001.paid_at) = 1, 7, EXTRACT(dayofweek FROM bookings_source_src_10001.paid_at) - 1) AS booking__paid_at__extract_dow + , EXTRACT(dayofyear FROM bookings_source_src_10001.paid_at) AS booking__paid_at__extract_doy + , bookings_source_src_10001.listing_id AS listing + , bookings_source_src_10001.guest_id AS guest + , bookings_source_src_10001.host_id AS host + , bookings_source_src_10001.listing_id AS booking__listing + , bookings_source_src_10001.guest_id AS booking__guest + , bookings_source_src_10001.host_id AS booking__host + FROM ***************************.fct_bookings bookings_source_src_10001 + ) subq_0 + ) subq_1 + GROUP BY + is_instant +) subq_2 +FULL OUTER JOIN ( + -- Aggregate Measures + SELECT + subq_4.is_instant + , SUM(subq_4.instant_bookings) AS instant_bookings + , COUNT(DISTINCT subq_4.bookers) AS bookers + FROM ( + -- Pass Only Elements: + -- ['instant_bookings', 'bookers', 'is_instant'] + SELECT + subq_3.is_instant + , subq_3.instant_bookings + , subq_3.bookers + FROM ( + -- Read Elements From Semantic Model 'bookings_source' + SELECT + 1 AS bookings + , CASE WHEN is_instant THEN 1 ELSE 0 END AS instant_bookings + , bookings_source_src_10001.booking_value + , bookings_source_src_10001.booking_value AS max_booking_value + , bookings_source_src_10001.booking_value AS min_booking_value + , bookings_source_src_10001.guest_id AS bookers + , bookings_source_src_10001.booking_value AS average_booking_value + , bookings_source_src_10001.booking_value AS booking_payments + , CASE WHEN referrer_id IS NOT NULL THEN 1 ELSE 0 END AS referred_bookings + , bookings_source_src_10001.booking_value AS median_booking_value + , bookings_source_src_10001.booking_value AS booking_value_p99 + , bookings_source_src_10001.booking_value AS discrete_booking_value_p99 + , bookings_source_src_10001.booking_value AS approximate_continuous_booking_value_p99 + , bookings_source_src_10001.booking_value AS approximate_discrete_booking_value_p99 + , bookings_source_src_10001.is_instant + , DATE_TRUNC(bookings_source_src_10001.ds, day) AS ds__day + , DATE_TRUNC(bookings_source_src_10001.ds, isoweek) AS ds__week + , DATE_TRUNC(bookings_source_src_10001.ds, month) AS ds__month + , DATE_TRUNC(bookings_source_src_10001.ds, quarter) AS ds__quarter + , DATE_TRUNC(bookings_source_src_10001.ds, year) AS ds__year + , EXTRACT(year FROM bookings_source_src_10001.ds) AS ds__extract_year + , EXTRACT(quarter FROM bookings_source_src_10001.ds) AS ds__extract_quarter + , EXTRACT(month FROM bookings_source_src_10001.ds) AS ds__extract_month + , EXTRACT(day FROM bookings_source_src_10001.ds) AS ds__extract_day + , IF(EXTRACT(dayofweek FROM bookings_source_src_10001.ds) = 1, 7, EXTRACT(dayofweek FROM bookings_source_src_10001.ds) - 1) AS ds__extract_dow + , EXTRACT(dayofyear FROM bookings_source_src_10001.ds) AS ds__extract_doy + , DATE_TRUNC(bookings_source_src_10001.ds_partitioned, day) AS ds_partitioned__day + , DATE_TRUNC(bookings_source_src_10001.ds_partitioned, isoweek) AS ds_partitioned__week + , DATE_TRUNC(bookings_source_src_10001.ds_partitioned, month) AS ds_partitioned__month + , DATE_TRUNC(bookings_source_src_10001.ds_partitioned, quarter) AS ds_partitioned__quarter + , DATE_TRUNC(bookings_source_src_10001.ds_partitioned, year) AS ds_partitioned__year + , EXTRACT(year FROM bookings_source_src_10001.ds_partitioned) AS ds_partitioned__extract_year + , EXTRACT(quarter FROM bookings_source_src_10001.ds_partitioned) AS ds_partitioned__extract_quarter + , EXTRACT(month FROM bookings_source_src_10001.ds_partitioned) AS ds_partitioned__extract_month + , EXTRACT(day FROM bookings_source_src_10001.ds_partitioned) AS ds_partitioned__extract_day + , IF(EXTRACT(dayofweek FROM bookings_source_src_10001.ds_partitioned) = 1, 7, EXTRACT(dayofweek FROM bookings_source_src_10001.ds_partitioned) - 1) AS ds_partitioned__extract_dow + , EXTRACT(dayofyear FROM bookings_source_src_10001.ds_partitioned) AS ds_partitioned__extract_doy + , DATE_TRUNC(bookings_source_src_10001.paid_at, day) AS paid_at__day + , DATE_TRUNC(bookings_source_src_10001.paid_at, isoweek) AS paid_at__week + , DATE_TRUNC(bookings_source_src_10001.paid_at, month) AS paid_at__month + , DATE_TRUNC(bookings_source_src_10001.paid_at, quarter) AS paid_at__quarter + , DATE_TRUNC(bookings_source_src_10001.paid_at, year) AS paid_at__year + , EXTRACT(year FROM bookings_source_src_10001.paid_at) AS paid_at__extract_year + , EXTRACT(quarter FROM bookings_source_src_10001.paid_at) AS paid_at__extract_quarter + , EXTRACT(month FROM bookings_source_src_10001.paid_at) AS paid_at__extract_month + , EXTRACT(day FROM bookings_source_src_10001.paid_at) AS paid_at__extract_day + , IF(EXTRACT(dayofweek FROM bookings_source_src_10001.paid_at) = 1, 7, EXTRACT(dayofweek FROM bookings_source_src_10001.paid_at) - 1) AS paid_at__extract_dow + , EXTRACT(dayofyear FROM bookings_source_src_10001.paid_at) AS paid_at__extract_doy + , bookings_source_src_10001.is_instant AS booking__is_instant + , DATE_TRUNC(bookings_source_src_10001.ds, day) AS booking__ds__day + , DATE_TRUNC(bookings_source_src_10001.ds, isoweek) AS booking__ds__week + , DATE_TRUNC(bookings_source_src_10001.ds, month) AS booking__ds__month + , DATE_TRUNC(bookings_source_src_10001.ds, quarter) AS booking__ds__quarter + , DATE_TRUNC(bookings_source_src_10001.ds, year) AS booking__ds__year + , EXTRACT(year FROM bookings_source_src_10001.ds) AS booking__ds__extract_year + , EXTRACT(quarter FROM bookings_source_src_10001.ds) AS booking__ds__extract_quarter + , EXTRACT(month FROM bookings_source_src_10001.ds) AS booking__ds__extract_month + , EXTRACT(day FROM bookings_source_src_10001.ds) AS booking__ds__extract_day + , IF(EXTRACT(dayofweek FROM bookings_source_src_10001.ds) = 1, 7, EXTRACT(dayofweek FROM bookings_source_src_10001.ds) - 1) AS booking__ds__extract_dow + , EXTRACT(dayofyear FROM bookings_source_src_10001.ds) AS booking__ds__extract_doy + , DATE_TRUNC(bookings_source_src_10001.ds_partitioned, day) AS booking__ds_partitioned__day + , DATE_TRUNC(bookings_source_src_10001.ds_partitioned, isoweek) AS booking__ds_partitioned__week + , DATE_TRUNC(bookings_source_src_10001.ds_partitioned, month) AS booking__ds_partitioned__month + , DATE_TRUNC(bookings_source_src_10001.ds_partitioned, quarter) AS booking__ds_partitioned__quarter + , DATE_TRUNC(bookings_source_src_10001.ds_partitioned, year) AS booking__ds_partitioned__year + , EXTRACT(year FROM bookings_source_src_10001.ds_partitioned) AS booking__ds_partitioned__extract_year + , EXTRACT(quarter FROM bookings_source_src_10001.ds_partitioned) AS booking__ds_partitioned__extract_quarter + , EXTRACT(month FROM bookings_source_src_10001.ds_partitioned) AS booking__ds_partitioned__extract_month + , EXTRACT(day FROM bookings_source_src_10001.ds_partitioned) AS booking__ds_partitioned__extract_day + , IF(EXTRACT(dayofweek FROM bookings_source_src_10001.ds_partitioned) = 1, 7, EXTRACT(dayofweek FROM bookings_source_src_10001.ds_partitioned) - 1) AS booking__ds_partitioned__extract_dow + , EXTRACT(dayofyear FROM bookings_source_src_10001.ds_partitioned) AS booking__ds_partitioned__extract_doy + , DATE_TRUNC(bookings_source_src_10001.paid_at, day) AS booking__paid_at__day + , DATE_TRUNC(bookings_source_src_10001.paid_at, isoweek) AS booking__paid_at__week + , DATE_TRUNC(bookings_source_src_10001.paid_at, month) AS booking__paid_at__month + , DATE_TRUNC(bookings_source_src_10001.paid_at, quarter) AS booking__paid_at__quarter + , DATE_TRUNC(bookings_source_src_10001.paid_at, year) AS booking__paid_at__year + , EXTRACT(year FROM bookings_source_src_10001.paid_at) AS booking__paid_at__extract_year + , EXTRACT(quarter FROM bookings_source_src_10001.paid_at) AS booking__paid_at__extract_quarter + , EXTRACT(month FROM bookings_source_src_10001.paid_at) AS booking__paid_at__extract_month + , EXTRACT(day FROM bookings_source_src_10001.paid_at) AS booking__paid_at__extract_day + , IF(EXTRACT(dayofweek FROM bookings_source_src_10001.paid_at) = 1, 7, EXTRACT(dayofweek FROM bookings_source_src_10001.paid_at) - 1) AS booking__paid_at__extract_dow + , EXTRACT(dayofyear FROM bookings_source_src_10001.paid_at) AS booking__paid_at__extract_doy + , bookings_source_src_10001.listing_id AS listing + , bookings_source_src_10001.guest_id AS guest + , bookings_source_src_10001.host_id AS host + , bookings_source_src_10001.listing_id AS booking__listing + , bookings_source_src_10001.guest_id AS booking__guest + , bookings_source_src_10001.host_id AS booking__host + FROM ***************************.fct_bookings bookings_source_src_10001 + ) subq_3 + ) subq_4 + GROUP BY + is_instant +) subq_5 +ON + subq_2.is_instant = subq_5.is_instant +GROUP BY + is_instant diff --git a/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/BigQuery/test_combine_output_node__plan0_optimized.sql b/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/BigQuery/test_combine_output_node__plan0_optimized.sql new file mode 100644 index 0000000000..9d6f1d7241 --- /dev/null +++ b/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/BigQuery/test_combine_output_node__plan0_optimized.sql @@ -0,0 +1,46 @@ +-- Combine Aggregated Outputs +SELECT + COALESCE(subq_8.is_instant, subq_11.is_instant) AS is_instant + , MAX(subq_8.bookings) AS bookings + , COALESCE(MAX(subq_11.instant_bookings), 1) AS instant_bookings + , COALESCE(MAX(subq_11.bookers), 1) AS bookers +FROM ( + -- Aggregate Measures + SELECT + is_instant + , SUM(bookings) AS bookings + FROM ( + -- Read Elements From Semantic Model 'bookings_source' + -- Pass Only Elements: + -- ['bookings', 'is_instant'] + SELECT + is_instant + , 1 AS bookings + FROM ***************************.fct_bookings bookings_source_src_10001 + ) subq_7 + GROUP BY + is_instant +) subq_8 +FULL OUTER JOIN ( + -- Aggregate Measures + SELECT + is_instant + , SUM(instant_bookings) AS instant_bookings + , COUNT(DISTINCT bookers) AS bookers + FROM ( + -- Read Elements From Semantic Model 'bookings_source' + -- Pass Only Elements: + -- ['instant_bookings', 'bookers', 'is_instant'] + SELECT + is_instant + , CASE WHEN is_instant THEN 1 ELSE 0 END AS instant_bookings + , guest_id AS bookers + FROM ***************************.fct_bookings bookings_source_src_10001 + ) subq_10 + GROUP BY + is_instant +) subq_11 +ON + subq_8.is_instant = subq_11.is_instant +GROUP BY + is_instant diff --git a/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/BigQuery/test_compute_metrics_node_ratio_from_multiple_semantic_models__plan0.sql b/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/BigQuery/test_compute_metrics_node_ratio_from_multiple_semantic_models__plan0.sql index fc7317598e..76675b4f55 100644 --- a/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/BigQuery/test_compute_metrics_node_ratio_from_multiple_semantic_models__plan0.sql +++ b/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/BigQuery/test_compute_metrics_node_ratio_from_multiple_semantic_models__plan0.sql @@ -4,7 +4,7 @@ SELECT , subq_20.listing__country_latest , CAST(subq_20.bookings AS FLOAT64) / CAST(NULLIF(subq_20.views, 0) AS FLOAT64) AS bookings_per_view FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT COALESCE(subq_9.ds__day, subq_19.ds__day) AS ds__day , COALESCE(subq_9.listing__country_latest, subq_19.listing__country_latest) AS listing__country_latest diff --git a/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/BigQuery/test_compute_metrics_node_ratio_from_multiple_semantic_models__plan0_optimized.sql b/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/BigQuery/test_compute_metrics_node_ratio_from_multiple_semantic_models__plan0_optimized.sql index fb551dce52..c7350f6401 100644 --- a/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/BigQuery/test_compute_metrics_node_ratio_from_multiple_semantic_models__plan0_optimized.sql +++ b/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/BigQuery/test_compute_metrics_node_ratio_from_multiple_semantic_models__plan0_optimized.sql @@ -4,7 +4,7 @@ SELECT , listing__country_latest , CAST(bookings AS FLOAT64) / CAST(NULLIF(views, 0) AS FLOAT64) AS bookings_per_view FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT COALESCE(subq_30.ds__day, subq_40.ds__day) AS ds__day , COALESCE(subq_30.listing__country_latest, subq_40.listing__country_latest) AS listing__country_latest diff --git a/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Databricks/test_combine_output_node__plan0.sql b/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Databricks/test_combine_output_node__plan0.sql new file mode 100644 index 0000000000..3940ca4f3f --- /dev/null +++ b/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Databricks/test_combine_output_node__plan0.sql @@ -0,0 +1,228 @@ +-- Combine Aggregated Outputs +SELECT + COALESCE(subq_2.is_instant, subq_5.is_instant) AS is_instant + , MAX(subq_2.bookings) AS bookings + , COALESCE(MAX(subq_5.instant_bookings), 1) AS instant_bookings + , COALESCE(MAX(subq_5.bookers), 1) AS bookers +FROM ( + -- Aggregate Measures + SELECT + subq_1.is_instant + , SUM(subq_1.bookings) AS bookings + FROM ( + -- Pass Only Elements: + -- ['bookings', 'is_instant'] + SELECT + subq_0.is_instant + , subq_0.bookings + FROM ( + -- Read Elements From Semantic Model 'bookings_source' + SELECT + 1 AS bookings + , CASE WHEN is_instant THEN 1 ELSE 0 END AS instant_bookings + , bookings_source_src_10001.booking_value + , bookings_source_src_10001.booking_value AS max_booking_value + , bookings_source_src_10001.booking_value AS min_booking_value + , bookings_source_src_10001.guest_id AS bookers + , bookings_source_src_10001.booking_value AS average_booking_value + , bookings_source_src_10001.booking_value AS booking_payments + , CASE WHEN referrer_id IS NOT NULL THEN 1 ELSE 0 END AS referred_bookings + , bookings_source_src_10001.booking_value AS median_booking_value + , bookings_source_src_10001.booking_value AS booking_value_p99 + , bookings_source_src_10001.booking_value AS discrete_booking_value_p99 + , bookings_source_src_10001.booking_value AS approximate_continuous_booking_value_p99 + , bookings_source_src_10001.booking_value AS approximate_discrete_booking_value_p99 + , bookings_source_src_10001.is_instant + , DATE_TRUNC('day', bookings_source_src_10001.ds) AS ds__day + , DATE_TRUNC('week', bookings_source_src_10001.ds) AS ds__week + , DATE_TRUNC('month', bookings_source_src_10001.ds) AS ds__month + , DATE_TRUNC('quarter', bookings_source_src_10001.ds) AS ds__quarter + , DATE_TRUNC('year', bookings_source_src_10001.ds) AS ds__year + , EXTRACT(year FROM bookings_source_src_10001.ds) AS ds__extract_year + , EXTRACT(quarter FROM bookings_source_src_10001.ds) AS ds__extract_quarter + , EXTRACT(month FROM bookings_source_src_10001.ds) AS ds__extract_month + , EXTRACT(day FROM bookings_source_src_10001.ds) AS ds__extract_day + , EXTRACT(DAYOFWEEK_ISO FROM bookings_source_src_10001.ds) AS ds__extract_dow + , EXTRACT(doy FROM bookings_source_src_10001.ds) AS ds__extract_doy + , DATE_TRUNC('day', bookings_source_src_10001.ds_partitioned) AS ds_partitioned__day + , DATE_TRUNC('week', bookings_source_src_10001.ds_partitioned) AS ds_partitioned__week + , DATE_TRUNC('month', bookings_source_src_10001.ds_partitioned) AS ds_partitioned__month + , DATE_TRUNC('quarter', bookings_source_src_10001.ds_partitioned) AS ds_partitioned__quarter + , DATE_TRUNC('year', bookings_source_src_10001.ds_partitioned) AS ds_partitioned__year + , EXTRACT(year FROM bookings_source_src_10001.ds_partitioned) AS ds_partitioned__extract_year + , EXTRACT(quarter FROM bookings_source_src_10001.ds_partitioned) AS ds_partitioned__extract_quarter + , EXTRACT(month FROM bookings_source_src_10001.ds_partitioned) AS ds_partitioned__extract_month + , EXTRACT(day FROM bookings_source_src_10001.ds_partitioned) AS ds_partitioned__extract_day + , EXTRACT(DAYOFWEEK_ISO FROM bookings_source_src_10001.ds_partitioned) AS ds_partitioned__extract_dow + , EXTRACT(doy FROM bookings_source_src_10001.ds_partitioned) AS ds_partitioned__extract_doy + , DATE_TRUNC('day', bookings_source_src_10001.paid_at) AS paid_at__day + , DATE_TRUNC('week', bookings_source_src_10001.paid_at) AS paid_at__week + , DATE_TRUNC('month', bookings_source_src_10001.paid_at) AS paid_at__month + , DATE_TRUNC('quarter', bookings_source_src_10001.paid_at) AS paid_at__quarter + , DATE_TRUNC('year', bookings_source_src_10001.paid_at) AS paid_at__year + , EXTRACT(year FROM bookings_source_src_10001.paid_at) AS paid_at__extract_year + , EXTRACT(quarter FROM bookings_source_src_10001.paid_at) AS paid_at__extract_quarter + , EXTRACT(month FROM bookings_source_src_10001.paid_at) AS paid_at__extract_month + , EXTRACT(day FROM bookings_source_src_10001.paid_at) AS paid_at__extract_day + , EXTRACT(DAYOFWEEK_ISO FROM bookings_source_src_10001.paid_at) AS paid_at__extract_dow + , EXTRACT(doy FROM bookings_source_src_10001.paid_at) AS paid_at__extract_doy + , bookings_source_src_10001.is_instant AS booking__is_instant + , DATE_TRUNC('day', bookings_source_src_10001.ds) AS booking__ds__day + , DATE_TRUNC('week', bookings_source_src_10001.ds) AS booking__ds__week + , DATE_TRUNC('month', bookings_source_src_10001.ds) AS booking__ds__month + , DATE_TRUNC('quarter', bookings_source_src_10001.ds) AS booking__ds__quarter + , DATE_TRUNC('year', bookings_source_src_10001.ds) AS booking__ds__year + , EXTRACT(year FROM bookings_source_src_10001.ds) AS booking__ds__extract_year + , EXTRACT(quarter FROM bookings_source_src_10001.ds) AS booking__ds__extract_quarter + , EXTRACT(month FROM bookings_source_src_10001.ds) AS booking__ds__extract_month + , EXTRACT(day FROM bookings_source_src_10001.ds) AS booking__ds__extract_day + , EXTRACT(DAYOFWEEK_ISO FROM bookings_source_src_10001.ds) AS booking__ds__extract_dow + , EXTRACT(doy FROM bookings_source_src_10001.ds) AS booking__ds__extract_doy + , DATE_TRUNC('day', bookings_source_src_10001.ds_partitioned) AS booking__ds_partitioned__day + , DATE_TRUNC('week', bookings_source_src_10001.ds_partitioned) AS booking__ds_partitioned__week + , DATE_TRUNC('month', bookings_source_src_10001.ds_partitioned) AS booking__ds_partitioned__month + , DATE_TRUNC('quarter', bookings_source_src_10001.ds_partitioned) AS booking__ds_partitioned__quarter + , DATE_TRUNC('year', bookings_source_src_10001.ds_partitioned) AS booking__ds_partitioned__year + , EXTRACT(year FROM bookings_source_src_10001.ds_partitioned) AS booking__ds_partitioned__extract_year + , EXTRACT(quarter FROM bookings_source_src_10001.ds_partitioned) AS booking__ds_partitioned__extract_quarter + , EXTRACT(month FROM bookings_source_src_10001.ds_partitioned) AS booking__ds_partitioned__extract_month + , EXTRACT(day FROM bookings_source_src_10001.ds_partitioned) AS booking__ds_partitioned__extract_day + , EXTRACT(DAYOFWEEK_ISO FROM bookings_source_src_10001.ds_partitioned) AS booking__ds_partitioned__extract_dow + , EXTRACT(doy FROM bookings_source_src_10001.ds_partitioned) AS booking__ds_partitioned__extract_doy + , DATE_TRUNC('day', bookings_source_src_10001.paid_at) AS booking__paid_at__day + , DATE_TRUNC('week', bookings_source_src_10001.paid_at) AS booking__paid_at__week + , DATE_TRUNC('month', bookings_source_src_10001.paid_at) AS booking__paid_at__month + , DATE_TRUNC('quarter', bookings_source_src_10001.paid_at) AS booking__paid_at__quarter + , DATE_TRUNC('year', bookings_source_src_10001.paid_at) AS booking__paid_at__year + , EXTRACT(year FROM bookings_source_src_10001.paid_at) AS booking__paid_at__extract_year + , EXTRACT(quarter FROM bookings_source_src_10001.paid_at) AS booking__paid_at__extract_quarter + , EXTRACT(month FROM bookings_source_src_10001.paid_at) AS booking__paid_at__extract_month + , EXTRACT(day FROM bookings_source_src_10001.paid_at) AS booking__paid_at__extract_day + , EXTRACT(DAYOFWEEK_ISO FROM bookings_source_src_10001.paid_at) AS booking__paid_at__extract_dow + , EXTRACT(doy FROM bookings_source_src_10001.paid_at) AS booking__paid_at__extract_doy + , bookings_source_src_10001.listing_id AS listing + , bookings_source_src_10001.guest_id AS guest + , bookings_source_src_10001.host_id AS host + , bookings_source_src_10001.listing_id AS booking__listing + , bookings_source_src_10001.guest_id AS booking__guest + , bookings_source_src_10001.host_id AS booking__host + FROM ***************************.fct_bookings bookings_source_src_10001 + ) subq_0 + ) subq_1 + GROUP BY + subq_1.is_instant +) subq_2 +FULL OUTER JOIN ( + -- Aggregate Measures + SELECT + subq_4.is_instant + , SUM(subq_4.instant_bookings) AS instant_bookings + , COUNT(DISTINCT subq_4.bookers) AS bookers + FROM ( + -- Pass Only Elements: + -- ['instant_bookings', 'bookers', 'is_instant'] + SELECT + subq_3.is_instant + , subq_3.instant_bookings + , subq_3.bookers + FROM ( + -- Read Elements From Semantic Model 'bookings_source' + SELECT + 1 AS bookings + , CASE WHEN is_instant THEN 1 ELSE 0 END AS instant_bookings + , bookings_source_src_10001.booking_value + , bookings_source_src_10001.booking_value AS max_booking_value + , bookings_source_src_10001.booking_value AS min_booking_value + , bookings_source_src_10001.guest_id AS bookers + , bookings_source_src_10001.booking_value AS average_booking_value + , bookings_source_src_10001.booking_value AS booking_payments + , CASE WHEN referrer_id IS NOT NULL THEN 1 ELSE 0 END AS referred_bookings + , bookings_source_src_10001.booking_value AS median_booking_value + , bookings_source_src_10001.booking_value AS booking_value_p99 + , bookings_source_src_10001.booking_value AS discrete_booking_value_p99 + , bookings_source_src_10001.booking_value AS approximate_continuous_booking_value_p99 + , bookings_source_src_10001.booking_value AS approximate_discrete_booking_value_p99 + , bookings_source_src_10001.is_instant + , DATE_TRUNC('day', bookings_source_src_10001.ds) AS ds__day + , DATE_TRUNC('week', bookings_source_src_10001.ds) AS ds__week + , DATE_TRUNC('month', bookings_source_src_10001.ds) AS ds__month + , DATE_TRUNC('quarter', bookings_source_src_10001.ds) AS ds__quarter + , DATE_TRUNC('year', bookings_source_src_10001.ds) AS ds__year + , EXTRACT(year FROM bookings_source_src_10001.ds) AS ds__extract_year + , EXTRACT(quarter FROM bookings_source_src_10001.ds) AS ds__extract_quarter + , EXTRACT(month FROM bookings_source_src_10001.ds) AS ds__extract_month + , EXTRACT(day FROM bookings_source_src_10001.ds) AS ds__extract_day + , EXTRACT(DAYOFWEEK_ISO FROM bookings_source_src_10001.ds) AS ds__extract_dow + , EXTRACT(doy FROM bookings_source_src_10001.ds) AS ds__extract_doy + , DATE_TRUNC('day', bookings_source_src_10001.ds_partitioned) AS ds_partitioned__day + , DATE_TRUNC('week', bookings_source_src_10001.ds_partitioned) AS ds_partitioned__week + , DATE_TRUNC('month', bookings_source_src_10001.ds_partitioned) AS ds_partitioned__month + , DATE_TRUNC('quarter', bookings_source_src_10001.ds_partitioned) AS ds_partitioned__quarter + , DATE_TRUNC('year', bookings_source_src_10001.ds_partitioned) AS ds_partitioned__year + , EXTRACT(year FROM bookings_source_src_10001.ds_partitioned) AS ds_partitioned__extract_year + , EXTRACT(quarter FROM bookings_source_src_10001.ds_partitioned) AS ds_partitioned__extract_quarter + , EXTRACT(month FROM bookings_source_src_10001.ds_partitioned) AS ds_partitioned__extract_month + , EXTRACT(day FROM bookings_source_src_10001.ds_partitioned) AS ds_partitioned__extract_day + , EXTRACT(DAYOFWEEK_ISO FROM bookings_source_src_10001.ds_partitioned) AS ds_partitioned__extract_dow + , EXTRACT(doy FROM bookings_source_src_10001.ds_partitioned) AS ds_partitioned__extract_doy + , DATE_TRUNC('day', bookings_source_src_10001.paid_at) AS paid_at__day + , DATE_TRUNC('week', bookings_source_src_10001.paid_at) AS paid_at__week + , DATE_TRUNC('month', bookings_source_src_10001.paid_at) AS paid_at__month + , DATE_TRUNC('quarter', bookings_source_src_10001.paid_at) AS paid_at__quarter + , DATE_TRUNC('year', bookings_source_src_10001.paid_at) AS paid_at__year + , EXTRACT(year FROM bookings_source_src_10001.paid_at) AS paid_at__extract_year + , EXTRACT(quarter FROM bookings_source_src_10001.paid_at) AS paid_at__extract_quarter + , EXTRACT(month FROM bookings_source_src_10001.paid_at) AS paid_at__extract_month + , EXTRACT(day FROM bookings_source_src_10001.paid_at) AS paid_at__extract_day + , EXTRACT(DAYOFWEEK_ISO FROM bookings_source_src_10001.paid_at) AS paid_at__extract_dow + , EXTRACT(doy FROM bookings_source_src_10001.paid_at) AS paid_at__extract_doy + , bookings_source_src_10001.is_instant AS booking__is_instant + , DATE_TRUNC('day', bookings_source_src_10001.ds) AS booking__ds__day + , DATE_TRUNC('week', bookings_source_src_10001.ds) AS booking__ds__week + , DATE_TRUNC('month', bookings_source_src_10001.ds) AS booking__ds__month + , DATE_TRUNC('quarter', bookings_source_src_10001.ds) AS booking__ds__quarter + , DATE_TRUNC('year', bookings_source_src_10001.ds) AS booking__ds__year + , EXTRACT(year FROM bookings_source_src_10001.ds) AS booking__ds__extract_year + , EXTRACT(quarter FROM bookings_source_src_10001.ds) AS booking__ds__extract_quarter + , EXTRACT(month FROM bookings_source_src_10001.ds) AS booking__ds__extract_month + , EXTRACT(day FROM bookings_source_src_10001.ds) AS booking__ds__extract_day + , EXTRACT(DAYOFWEEK_ISO FROM bookings_source_src_10001.ds) AS booking__ds__extract_dow + , EXTRACT(doy FROM bookings_source_src_10001.ds) AS booking__ds__extract_doy + , DATE_TRUNC('day', bookings_source_src_10001.ds_partitioned) AS booking__ds_partitioned__day + , DATE_TRUNC('week', bookings_source_src_10001.ds_partitioned) AS booking__ds_partitioned__week + , DATE_TRUNC('month', bookings_source_src_10001.ds_partitioned) AS booking__ds_partitioned__month + , DATE_TRUNC('quarter', bookings_source_src_10001.ds_partitioned) AS booking__ds_partitioned__quarter + , DATE_TRUNC('year', bookings_source_src_10001.ds_partitioned) AS booking__ds_partitioned__year + , EXTRACT(year FROM bookings_source_src_10001.ds_partitioned) AS booking__ds_partitioned__extract_year + , EXTRACT(quarter FROM bookings_source_src_10001.ds_partitioned) AS booking__ds_partitioned__extract_quarter + , EXTRACT(month FROM bookings_source_src_10001.ds_partitioned) AS booking__ds_partitioned__extract_month + , EXTRACT(day FROM bookings_source_src_10001.ds_partitioned) AS booking__ds_partitioned__extract_day + , EXTRACT(DAYOFWEEK_ISO FROM bookings_source_src_10001.ds_partitioned) AS booking__ds_partitioned__extract_dow + , EXTRACT(doy FROM bookings_source_src_10001.ds_partitioned) AS booking__ds_partitioned__extract_doy + , DATE_TRUNC('day', bookings_source_src_10001.paid_at) AS booking__paid_at__day + , DATE_TRUNC('week', bookings_source_src_10001.paid_at) AS booking__paid_at__week + , DATE_TRUNC('month', bookings_source_src_10001.paid_at) AS booking__paid_at__month + , DATE_TRUNC('quarter', bookings_source_src_10001.paid_at) AS booking__paid_at__quarter + , DATE_TRUNC('year', bookings_source_src_10001.paid_at) AS booking__paid_at__year + , EXTRACT(year FROM bookings_source_src_10001.paid_at) AS booking__paid_at__extract_year + , EXTRACT(quarter FROM bookings_source_src_10001.paid_at) AS booking__paid_at__extract_quarter + , EXTRACT(month FROM bookings_source_src_10001.paid_at) AS booking__paid_at__extract_month + , EXTRACT(day FROM bookings_source_src_10001.paid_at) AS booking__paid_at__extract_day + , EXTRACT(DAYOFWEEK_ISO FROM bookings_source_src_10001.paid_at) AS booking__paid_at__extract_dow + , EXTRACT(doy FROM bookings_source_src_10001.paid_at) AS booking__paid_at__extract_doy + , bookings_source_src_10001.listing_id AS listing + , bookings_source_src_10001.guest_id AS guest + , bookings_source_src_10001.host_id AS host + , bookings_source_src_10001.listing_id AS booking__listing + , bookings_source_src_10001.guest_id AS booking__guest + , bookings_source_src_10001.host_id AS booking__host + FROM ***************************.fct_bookings bookings_source_src_10001 + ) subq_3 + ) subq_4 + GROUP BY + subq_4.is_instant +) subq_5 +ON + subq_2.is_instant = subq_5.is_instant +GROUP BY + COALESCE(subq_2.is_instant, subq_5.is_instant) diff --git a/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Databricks/test_combine_output_node__plan0_optimized.sql b/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Databricks/test_combine_output_node__plan0_optimized.sql new file mode 100644 index 0000000000..62340aabcb --- /dev/null +++ b/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Databricks/test_combine_output_node__plan0_optimized.sql @@ -0,0 +1,46 @@ +-- Combine Aggregated Outputs +SELECT + COALESCE(subq_8.is_instant, subq_11.is_instant) AS is_instant + , MAX(subq_8.bookings) AS bookings + , COALESCE(MAX(subq_11.instant_bookings), 1) AS instant_bookings + , COALESCE(MAX(subq_11.bookers), 1) AS bookers +FROM ( + -- Aggregate Measures + SELECT + is_instant + , SUM(bookings) AS bookings + FROM ( + -- Read Elements From Semantic Model 'bookings_source' + -- Pass Only Elements: + -- ['bookings', 'is_instant'] + SELECT + is_instant + , 1 AS bookings + FROM ***************************.fct_bookings bookings_source_src_10001 + ) subq_7 + GROUP BY + is_instant +) subq_8 +FULL OUTER JOIN ( + -- Aggregate Measures + SELECT + is_instant + , SUM(instant_bookings) AS instant_bookings + , COUNT(DISTINCT bookers) AS bookers + FROM ( + -- Read Elements From Semantic Model 'bookings_source' + -- Pass Only Elements: + -- ['instant_bookings', 'bookers', 'is_instant'] + SELECT + is_instant + , CASE WHEN is_instant THEN 1 ELSE 0 END AS instant_bookings + , guest_id AS bookers + FROM ***************************.fct_bookings bookings_source_src_10001 + ) subq_10 + GROUP BY + is_instant +) subq_11 +ON + subq_8.is_instant = subq_11.is_instant +GROUP BY + COALESCE(subq_8.is_instant, subq_11.is_instant) diff --git a/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Databricks/test_compute_metrics_node_ratio_from_multiple_semantic_models__plan0.sql b/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Databricks/test_compute_metrics_node_ratio_from_multiple_semantic_models__plan0.sql index b619d75d13..d2342c5daa 100644 --- a/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Databricks/test_compute_metrics_node_ratio_from_multiple_semantic_models__plan0.sql +++ b/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Databricks/test_compute_metrics_node_ratio_from_multiple_semantic_models__plan0.sql @@ -4,7 +4,7 @@ SELECT , subq_20.listing__country_latest , CAST(subq_20.bookings AS DOUBLE) / CAST(NULLIF(subq_20.views, 0) AS DOUBLE) AS bookings_per_view FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT COALESCE(subq_9.ds__day, subq_19.ds__day) AS ds__day , COALESCE(subq_9.listing__country_latest, subq_19.listing__country_latest) AS listing__country_latest diff --git a/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Databricks/test_compute_metrics_node_ratio_from_multiple_semantic_models__plan0_optimized.sql b/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Databricks/test_compute_metrics_node_ratio_from_multiple_semantic_models__plan0_optimized.sql index b53b1f06f4..635bfacdef 100644 --- a/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Databricks/test_compute_metrics_node_ratio_from_multiple_semantic_models__plan0_optimized.sql +++ b/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Databricks/test_compute_metrics_node_ratio_from_multiple_semantic_models__plan0_optimized.sql @@ -4,7 +4,7 @@ SELECT , listing__country_latest , CAST(bookings AS DOUBLE) / CAST(NULLIF(views, 0) AS DOUBLE) AS bookings_per_view FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT COALESCE(subq_30.ds__day, subq_40.ds__day) AS ds__day , COALESCE(subq_30.listing__country_latest, subq_40.listing__country_latest) AS listing__country_latest diff --git a/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/DuckDB/test_compute_metrics_node_ratio_from_multiple_semantic_models__plan0.sql b/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/DuckDB/test_compute_metrics_node_ratio_from_multiple_semantic_models__plan0.sql index e945f2d50c..dda45db51d 100644 --- a/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/DuckDB/test_compute_metrics_node_ratio_from_multiple_semantic_models__plan0.sql +++ b/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/DuckDB/test_compute_metrics_node_ratio_from_multiple_semantic_models__plan0.sql @@ -4,7 +4,7 @@ SELECT , subq_20.listing__country_latest , CAST(subq_20.bookings AS DOUBLE) / CAST(NULLIF(subq_20.views, 0) AS DOUBLE) AS bookings_per_view FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT COALESCE(subq_9.ds__day, subq_19.ds__day) AS ds__day , COALESCE(subq_9.listing__country_latest, subq_19.listing__country_latest) AS listing__country_latest diff --git a/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/DuckDB/test_compute_metrics_node_ratio_from_multiple_semantic_models__plan0_optimized.sql b/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/DuckDB/test_compute_metrics_node_ratio_from_multiple_semantic_models__plan0_optimized.sql index b53b1f06f4..635bfacdef 100644 --- a/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/DuckDB/test_compute_metrics_node_ratio_from_multiple_semantic_models__plan0_optimized.sql +++ b/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/DuckDB/test_compute_metrics_node_ratio_from_multiple_semantic_models__plan0_optimized.sql @@ -4,7 +4,7 @@ SELECT , listing__country_latest , CAST(bookings AS DOUBLE) / CAST(NULLIF(views, 0) AS DOUBLE) AS bookings_per_view FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT COALESCE(subq_30.ds__day, subq_40.ds__day) AS ds__day , COALESCE(subq_30.listing__country_latest, subq_40.listing__country_latest) AS listing__country_latest diff --git a/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Postgres/test_combine_output_node__plan0.sql b/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Postgres/test_combine_output_node__plan0.sql new file mode 100644 index 0000000000..c800156c88 --- /dev/null +++ b/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Postgres/test_combine_output_node__plan0.sql @@ -0,0 +1,228 @@ +-- Combine Aggregated Outputs +SELECT + COALESCE(subq_2.is_instant, subq_5.is_instant) AS is_instant + , MAX(subq_2.bookings) AS bookings + , COALESCE(MAX(subq_5.instant_bookings), 1) AS instant_bookings + , COALESCE(MAX(subq_5.bookers), 1) AS bookers +FROM ( + -- Aggregate Measures + SELECT + subq_1.is_instant + , SUM(subq_1.bookings) AS bookings + FROM ( + -- Pass Only Elements: + -- ['bookings', 'is_instant'] + SELECT + subq_0.is_instant + , subq_0.bookings + FROM ( + -- Read Elements From Semantic Model 'bookings_source' + SELECT + 1 AS bookings + , CASE WHEN is_instant THEN 1 ELSE 0 END AS instant_bookings + , bookings_source_src_10001.booking_value + , bookings_source_src_10001.booking_value AS max_booking_value + , bookings_source_src_10001.booking_value AS min_booking_value + , bookings_source_src_10001.guest_id AS bookers + , bookings_source_src_10001.booking_value AS average_booking_value + , bookings_source_src_10001.booking_value AS booking_payments + , CASE WHEN referrer_id IS NOT NULL THEN 1 ELSE 0 END AS referred_bookings + , bookings_source_src_10001.booking_value AS median_booking_value + , bookings_source_src_10001.booking_value AS booking_value_p99 + , bookings_source_src_10001.booking_value AS discrete_booking_value_p99 + , bookings_source_src_10001.booking_value AS approximate_continuous_booking_value_p99 + , bookings_source_src_10001.booking_value AS approximate_discrete_booking_value_p99 + , bookings_source_src_10001.is_instant + , DATE_TRUNC('day', bookings_source_src_10001.ds) AS ds__day + , DATE_TRUNC('week', bookings_source_src_10001.ds) AS ds__week + , DATE_TRUNC('month', bookings_source_src_10001.ds) AS ds__month + , DATE_TRUNC('quarter', bookings_source_src_10001.ds) AS ds__quarter + , DATE_TRUNC('year', bookings_source_src_10001.ds) AS ds__year + , EXTRACT(year FROM bookings_source_src_10001.ds) AS ds__extract_year + , EXTRACT(quarter FROM bookings_source_src_10001.ds) AS ds__extract_quarter + , EXTRACT(month FROM bookings_source_src_10001.ds) AS ds__extract_month + , EXTRACT(day FROM bookings_source_src_10001.ds) AS ds__extract_day + , EXTRACT(isodow FROM bookings_source_src_10001.ds) AS ds__extract_dow + , EXTRACT(doy FROM bookings_source_src_10001.ds) AS ds__extract_doy + , DATE_TRUNC('day', bookings_source_src_10001.ds_partitioned) AS ds_partitioned__day + , DATE_TRUNC('week', bookings_source_src_10001.ds_partitioned) AS ds_partitioned__week + , DATE_TRUNC('month', bookings_source_src_10001.ds_partitioned) AS ds_partitioned__month + , DATE_TRUNC('quarter', bookings_source_src_10001.ds_partitioned) AS ds_partitioned__quarter + , DATE_TRUNC('year', bookings_source_src_10001.ds_partitioned) AS ds_partitioned__year + , EXTRACT(year FROM bookings_source_src_10001.ds_partitioned) AS ds_partitioned__extract_year + , EXTRACT(quarter FROM bookings_source_src_10001.ds_partitioned) AS ds_partitioned__extract_quarter + , EXTRACT(month FROM bookings_source_src_10001.ds_partitioned) AS ds_partitioned__extract_month + , EXTRACT(day FROM bookings_source_src_10001.ds_partitioned) AS ds_partitioned__extract_day + , EXTRACT(isodow FROM bookings_source_src_10001.ds_partitioned) AS ds_partitioned__extract_dow + , EXTRACT(doy FROM bookings_source_src_10001.ds_partitioned) AS ds_partitioned__extract_doy + , DATE_TRUNC('day', bookings_source_src_10001.paid_at) AS paid_at__day + , DATE_TRUNC('week', bookings_source_src_10001.paid_at) AS paid_at__week + , DATE_TRUNC('month', bookings_source_src_10001.paid_at) AS paid_at__month + , DATE_TRUNC('quarter', bookings_source_src_10001.paid_at) AS paid_at__quarter + , DATE_TRUNC('year', bookings_source_src_10001.paid_at) AS paid_at__year + , EXTRACT(year FROM bookings_source_src_10001.paid_at) AS paid_at__extract_year + , EXTRACT(quarter FROM bookings_source_src_10001.paid_at) AS paid_at__extract_quarter + , EXTRACT(month FROM bookings_source_src_10001.paid_at) AS paid_at__extract_month + , EXTRACT(day FROM bookings_source_src_10001.paid_at) AS paid_at__extract_day + , EXTRACT(isodow FROM bookings_source_src_10001.paid_at) AS paid_at__extract_dow + , EXTRACT(doy FROM bookings_source_src_10001.paid_at) AS paid_at__extract_doy + , bookings_source_src_10001.is_instant AS booking__is_instant + , DATE_TRUNC('day', bookings_source_src_10001.ds) AS booking__ds__day + , DATE_TRUNC('week', bookings_source_src_10001.ds) AS booking__ds__week + , DATE_TRUNC('month', bookings_source_src_10001.ds) AS booking__ds__month + , DATE_TRUNC('quarter', bookings_source_src_10001.ds) AS booking__ds__quarter + , DATE_TRUNC('year', bookings_source_src_10001.ds) AS booking__ds__year + , EXTRACT(year FROM bookings_source_src_10001.ds) AS booking__ds__extract_year + , EXTRACT(quarter FROM bookings_source_src_10001.ds) AS booking__ds__extract_quarter + , EXTRACT(month FROM bookings_source_src_10001.ds) AS booking__ds__extract_month + , EXTRACT(day FROM bookings_source_src_10001.ds) AS booking__ds__extract_day + , EXTRACT(isodow FROM bookings_source_src_10001.ds) AS booking__ds__extract_dow + , EXTRACT(doy FROM bookings_source_src_10001.ds) AS booking__ds__extract_doy + , DATE_TRUNC('day', bookings_source_src_10001.ds_partitioned) AS booking__ds_partitioned__day + , DATE_TRUNC('week', bookings_source_src_10001.ds_partitioned) AS booking__ds_partitioned__week + , DATE_TRUNC('month', bookings_source_src_10001.ds_partitioned) AS booking__ds_partitioned__month + , DATE_TRUNC('quarter', bookings_source_src_10001.ds_partitioned) AS booking__ds_partitioned__quarter + , DATE_TRUNC('year', bookings_source_src_10001.ds_partitioned) AS booking__ds_partitioned__year + , EXTRACT(year FROM bookings_source_src_10001.ds_partitioned) AS booking__ds_partitioned__extract_year + , EXTRACT(quarter FROM bookings_source_src_10001.ds_partitioned) AS booking__ds_partitioned__extract_quarter + , EXTRACT(month FROM bookings_source_src_10001.ds_partitioned) AS booking__ds_partitioned__extract_month + , EXTRACT(day FROM bookings_source_src_10001.ds_partitioned) AS booking__ds_partitioned__extract_day + , EXTRACT(isodow FROM bookings_source_src_10001.ds_partitioned) AS booking__ds_partitioned__extract_dow + , EXTRACT(doy FROM bookings_source_src_10001.ds_partitioned) AS booking__ds_partitioned__extract_doy + , DATE_TRUNC('day', bookings_source_src_10001.paid_at) AS booking__paid_at__day + , DATE_TRUNC('week', bookings_source_src_10001.paid_at) AS booking__paid_at__week + , DATE_TRUNC('month', bookings_source_src_10001.paid_at) AS booking__paid_at__month + , DATE_TRUNC('quarter', bookings_source_src_10001.paid_at) AS booking__paid_at__quarter + , DATE_TRUNC('year', bookings_source_src_10001.paid_at) AS booking__paid_at__year + , EXTRACT(year FROM bookings_source_src_10001.paid_at) AS booking__paid_at__extract_year + , EXTRACT(quarter FROM bookings_source_src_10001.paid_at) AS booking__paid_at__extract_quarter + , EXTRACT(month FROM bookings_source_src_10001.paid_at) AS booking__paid_at__extract_month + , EXTRACT(day FROM bookings_source_src_10001.paid_at) AS booking__paid_at__extract_day + , EXTRACT(isodow FROM bookings_source_src_10001.paid_at) AS booking__paid_at__extract_dow + , EXTRACT(doy FROM bookings_source_src_10001.paid_at) AS booking__paid_at__extract_doy + , bookings_source_src_10001.listing_id AS listing + , bookings_source_src_10001.guest_id AS guest + , bookings_source_src_10001.host_id AS host + , bookings_source_src_10001.listing_id AS booking__listing + , bookings_source_src_10001.guest_id AS booking__guest + , bookings_source_src_10001.host_id AS booking__host + FROM ***************************.fct_bookings bookings_source_src_10001 + ) subq_0 + ) subq_1 + GROUP BY + subq_1.is_instant +) subq_2 +FULL OUTER JOIN ( + -- Aggregate Measures + SELECT + subq_4.is_instant + , SUM(subq_4.instant_bookings) AS instant_bookings + , COUNT(DISTINCT subq_4.bookers) AS bookers + FROM ( + -- Pass Only Elements: + -- ['instant_bookings', 'bookers', 'is_instant'] + SELECT + subq_3.is_instant + , subq_3.instant_bookings + , subq_3.bookers + FROM ( + -- Read Elements From Semantic Model 'bookings_source' + SELECT + 1 AS bookings + , CASE WHEN is_instant THEN 1 ELSE 0 END AS instant_bookings + , bookings_source_src_10001.booking_value + , bookings_source_src_10001.booking_value AS max_booking_value + , bookings_source_src_10001.booking_value AS min_booking_value + , bookings_source_src_10001.guest_id AS bookers + , bookings_source_src_10001.booking_value AS average_booking_value + , bookings_source_src_10001.booking_value AS booking_payments + , CASE WHEN referrer_id IS NOT NULL THEN 1 ELSE 0 END AS referred_bookings + , bookings_source_src_10001.booking_value AS median_booking_value + , bookings_source_src_10001.booking_value AS booking_value_p99 + , bookings_source_src_10001.booking_value AS discrete_booking_value_p99 + , bookings_source_src_10001.booking_value AS approximate_continuous_booking_value_p99 + , bookings_source_src_10001.booking_value AS approximate_discrete_booking_value_p99 + , bookings_source_src_10001.is_instant + , DATE_TRUNC('day', bookings_source_src_10001.ds) AS ds__day + , DATE_TRUNC('week', bookings_source_src_10001.ds) AS ds__week + , DATE_TRUNC('month', bookings_source_src_10001.ds) AS ds__month + , DATE_TRUNC('quarter', bookings_source_src_10001.ds) AS ds__quarter + , DATE_TRUNC('year', bookings_source_src_10001.ds) AS ds__year + , EXTRACT(year FROM bookings_source_src_10001.ds) AS ds__extract_year + , EXTRACT(quarter FROM bookings_source_src_10001.ds) AS ds__extract_quarter + , EXTRACT(month FROM bookings_source_src_10001.ds) AS ds__extract_month + , EXTRACT(day FROM bookings_source_src_10001.ds) AS ds__extract_day + , EXTRACT(isodow FROM bookings_source_src_10001.ds) AS ds__extract_dow + , EXTRACT(doy FROM bookings_source_src_10001.ds) AS ds__extract_doy + , DATE_TRUNC('day', bookings_source_src_10001.ds_partitioned) AS ds_partitioned__day + , DATE_TRUNC('week', bookings_source_src_10001.ds_partitioned) AS ds_partitioned__week + , DATE_TRUNC('month', bookings_source_src_10001.ds_partitioned) AS ds_partitioned__month + , DATE_TRUNC('quarter', bookings_source_src_10001.ds_partitioned) AS ds_partitioned__quarter + , DATE_TRUNC('year', bookings_source_src_10001.ds_partitioned) AS ds_partitioned__year + , EXTRACT(year FROM bookings_source_src_10001.ds_partitioned) AS ds_partitioned__extract_year + , EXTRACT(quarter FROM bookings_source_src_10001.ds_partitioned) AS ds_partitioned__extract_quarter + , EXTRACT(month FROM bookings_source_src_10001.ds_partitioned) AS ds_partitioned__extract_month + , EXTRACT(day FROM bookings_source_src_10001.ds_partitioned) AS ds_partitioned__extract_day + , EXTRACT(isodow FROM bookings_source_src_10001.ds_partitioned) AS ds_partitioned__extract_dow + , EXTRACT(doy FROM bookings_source_src_10001.ds_partitioned) AS ds_partitioned__extract_doy + , DATE_TRUNC('day', bookings_source_src_10001.paid_at) AS paid_at__day + , DATE_TRUNC('week', bookings_source_src_10001.paid_at) AS paid_at__week + , DATE_TRUNC('month', bookings_source_src_10001.paid_at) AS paid_at__month + , DATE_TRUNC('quarter', bookings_source_src_10001.paid_at) AS paid_at__quarter + , DATE_TRUNC('year', bookings_source_src_10001.paid_at) AS paid_at__year + , EXTRACT(year FROM bookings_source_src_10001.paid_at) AS paid_at__extract_year + , EXTRACT(quarter FROM bookings_source_src_10001.paid_at) AS paid_at__extract_quarter + , EXTRACT(month FROM bookings_source_src_10001.paid_at) AS paid_at__extract_month + , EXTRACT(day FROM bookings_source_src_10001.paid_at) AS paid_at__extract_day + , EXTRACT(isodow FROM bookings_source_src_10001.paid_at) AS paid_at__extract_dow + , EXTRACT(doy FROM bookings_source_src_10001.paid_at) AS paid_at__extract_doy + , bookings_source_src_10001.is_instant AS booking__is_instant + , DATE_TRUNC('day', bookings_source_src_10001.ds) AS booking__ds__day + , DATE_TRUNC('week', bookings_source_src_10001.ds) AS booking__ds__week + , DATE_TRUNC('month', bookings_source_src_10001.ds) AS booking__ds__month + , DATE_TRUNC('quarter', bookings_source_src_10001.ds) AS booking__ds__quarter + , DATE_TRUNC('year', bookings_source_src_10001.ds) AS booking__ds__year + , EXTRACT(year FROM bookings_source_src_10001.ds) AS booking__ds__extract_year + , EXTRACT(quarter FROM bookings_source_src_10001.ds) AS booking__ds__extract_quarter + , EXTRACT(month FROM bookings_source_src_10001.ds) AS booking__ds__extract_month + , EXTRACT(day FROM bookings_source_src_10001.ds) AS booking__ds__extract_day + , EXTRACT(isodow FROM bookings_source_src_10001.ds) AS booking__ds__extract_dow + , EXTRACT(doy FROM bookings_source_src_10001.ds) AS booking__ds__extract_doy + , DATE_TRUNC('day', bookings_source_src_10001.ds_partitioned) AS booking__ds_partitioned__day + , DATE_TRUNC('week', bookings_source_src_10001.ds_partitioned) AS booking__ds_partitioned__week + , DATE_TRUNC('month', bookings_source_src_10001.ds_partitioned) AS booking__ds_partitioned__month + , DATE_TRUNC('quarter', bookings_source_src_10001.ds_partitioned) AS booking__ds_partitioned__quarter + , DATE_TRUNC('year', bookings_source_src_10001.ds_partitioned) AS booking__ds_partitioned__year + , EXTRACT(year FROM bookings_source_src_10001.ds_partitioned) AS booking__ds_partitioned__extract_year + , EXTRACT(quarter FROM bookings_source_src_10001.ds_partitioned) AS booking__ds_partitioned__extract_quarter + , EXTRACT(month FROM bookings_source_src_10001.ds_partitioned) AS booking__ds_partitioned__extract_month + , EXTRACT(day FROM bookings_source_src_10001.ds_partitioned) AS booking__ds_partitioned__extract_day + , EXTRACT(isodow FROM bookings_source_src_10001.ds_partitioned) AS booking__ds_partitioned__extract_dow + , EXTRACT(doy FROM bookings_source_src_10001.ds_partitioned) AS booking__ds_partitioned__extract_doy + , DATE_TRUNC('day', bookings_source_src_10001.paid_at) AS booking__paid_at__day + , DATE_TRUNC('week', bookings_source_src_10001.paid_at) AS booking__paid_at__week + , DATE_TRUNC('month', bookings_source_src_10001.paid_at) AS booking__paid_at__month + , DATE_TRUNC('quarter', bookings_source_src_10001.paid_at) AS booking__paid_at__quarter + , DATE_TRUNC('year', bookings_source_src_10001.paid_at) AS booking__paid_at__year + , EXTRACT(year FROM bookings_source_src_10001.paid_at) AS booking__paid_at__extract_year + , EXTRACT(quarter FROM bookings_source_src_10001.paid_at) AS booking__paid_at__extract_quarter + , EXTRACT(month FROM bookings_source_src_10001.paid_at) AS booking__paid_at__extract_month + , EXTRACT(day FROM bookings_source_src_10001.paid_at) AS booking__paid_at__extract_day + , EXTRACT(isodow FROM bookings_source_src_10001.paid_at) AS booking__paid_at__extract_dow + , EXTRACT(doy FROM bookings_source_src_10001.paid_at) AS booking__paid_at__extract_doy + , bookings_source_src_10001.listing_id AS listing + , bookings_source_src_10001.guest_id AS guest + , bookings_source_src_10001.host_id AS host + , bookings_source_src_10001.listing_id AS booking__listing + , bookings_source_src_10001.guest_id AS booking__guest + , bookings_source_src_10001.host_id AS booking__host + FROM ***************************.fct_bookings bookings_source_src_10001 + ) subq_3 + ) subq_4 + GROUP BY + subq_4.is_instant +) subq_5 +ON + subq_2.is_instant = subq_5.is_instant +GROUP BY + COALESCE(subq_2.is_instant, subq_5.is_instant) diff --git a/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Postgres/test_combine_output_node__plan0_optimized.sql b/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Postgres/test_combine_output_node__plan0_optimized.sql new file mode 100644 index 0000000000..62340aabcb --- /dev/null +++ b/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Postgres/test_combine_output_node__plan0_optimized.sql @@ -0,0 +1,46 @@ +-- Combine Aggregated Outputs +SELECT + COALESCE(subq_8.is_instant, subq_11.is_instant) AS is_instant + , MAX(subq_8.bookings) AS bookings + , COALESCE(MAX(subq_11.instant_bookings), 1) AS instant_bookings + , COALESCE(MAX(subq_11.bookers), 1) AS bookers +FROM ( + -- Aggregate Measures + SELECT + is_instant + , SUM(bookings) AS bookings + FROM ( + -- Read Elements From Semantic Model 'bookings_source' + -- Pass Only Elements: + -- ['bookings', 'is_instant'] + SELECT + is_instant + , 1 AS bookings + FROM ***************************.fct_bookings bookings_source_src_10001 + ) subq_7 + GROUP BY + is_instant +) subq_8 +FULL OUTER JOIN ( + -- Aggregate Measures + SELECT + is_instant + , SUM(instant_bookings) AS instant_bookings + , COUNT(DISTINCT bookers) AS bookers + FROM ( + -- Read Elements From Semantic Model 'bookings_source' + -- Pass Only Elements: + -- ['instant_bookings', 'bookers', 'is_instant'] + SELECT + is_instant + , CASE WHEN is_instant THEN 1 ELSE 0 END AS instant_bookings + , guest_id AS bookers + FROM ***************************.fct_bookings bookings_source_src_10001 + ) subq_10 + GROUP BY + is_instant +) subq_11 +ON + subq_8.is_instant = subq_11.is_instant +GROUP BY + COALESCE(subq_8.is_instant, subq_11.is_instant) diff --git a/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Postgres/test_compute_metrics_node_ratio_from_multiple_semantic_models__plan0.sql b/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Postgres/test_compute_metrics_node_ratio_from_multiple_semantic_models__plan0.sql index 19b030239b..0bcc2c8877 100644 --- a/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Postgres/test_compute_metrics_node_ratio_from_multiple_semantic_models__plan0.sql +++ b/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Postgres/test_compute_metrics_node_ratio_from_multiple_semantic_models__plan0.sql @@ -4,7 +4,7 @@ SELECT , subq_20.listing__country_latest , CAST(subq_20.bookings AS DOUBLE PRECISION) / CAST(NULLIF(subq_20.views, 0) AS DOUBLE PRECISION) AS bookings_per_view FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT COALESCE(subq_9.ds__day, subq_19.ds__day) AS ds__day , COALESCE(subq_9.listing__country_latest, subq_19.listing__country_latest) AS listing__country_latest diff --git a/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Postgres/test_compute_metrics_node_ratio_from_multiple_semantic_models__plan0_optimized.sql b/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Postgres/test_compute_metrics_node_ratio_from_multiple_semantic_models__plan0_optimized.sql index 9526df6729..63f6522655 100644 --- a/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Postgres/test_compute_metrics_node_ratio_from_multiple_semantic_models__plan0_optimized.sql +++ b/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Postgres/test_compute_metrics_node_ratio_from_multiple_semantic_models__plan0_optimized.sql @@ -4,7 +4,7 @@ SELECT , listing__country_latest , CAST(bookings AS DOUBLE PRECISION) / CAST(NULLIF(views, 0) AS DOUBLE PRECISION) AS bookings_per_view FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT COALESCE(subq_30.ds__day, subq_40.ds__day) AS ds__day , COALESCE(subq_30.listing__country_latest, subq_40.listing__country_latest) AS listing__country_latest diff --git a/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Redshift/test_combine_output_node__plan0.sql b/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Redshift/test_combine_output_node__plan0.sql new file mode 100644 index 0000000000..80df7135b7 --- /dev/null +++ b/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Redshift/test_combine_output_node__plan0.sql @@ -0,0 +1,228 @@ +-- Combine Aggregated Outputs +SELECT + COALESCE(subq_2.is_instant, subq_5.is_instant) AS is_instant + , MAX(subq_2.bookings) AS bookings + , COALESCE(MAX(subq_5.instant_bookings), 1) AS instant_bookings + , COALESCE(MAX(subq_5.bookers), 1) AS bookers +FROM ( + -- Aggregate Measures + SELECT + subq_1.is_instant + , SUM(subq_1.bookings) AS bookings + FROM ( + -- Pass Only Elements: + -- ['bookings', 'is_instant'] + SELECT + subq_0.is_instant + , subq_0.bookings + FROM ( + -- Read Elements From Semantic Model 'bookings_source' + SELECT + 1 AS bookings + , CASE WHEN is_instant THEN 1 ELSE 0 END AS instant_bookings + , bookings_source_src_10001.booking_value + , bookings_source_src_10001.booking_value AS max_booking_value + , bookings_source_src_10001.booking_value AS min_booking_value + , bookings_source_src_10001.guest_id AS bookers + , bookings_source_src_10001.booking_value AS average_booking_value + , bookings_source_src_10001.booking_value AS booking_payments + , CASE WHEN referrer_id IS NOT NULL THEN 1 ELSE 0 END AS referred_bookings + , bookings_source_src_10001.booking_value AS median_booking_value + , bookings_source_src_10001.booking_value AS booking_value_p99 + , bookings_source_src_10001.booking_value AS discrete_booking_value_p99 + , bookings_source_src_10001.booking_value AS approximate_continuous_booking_value_p99 + , bookings_source_src_10001.booking_value AS approximate_discrete_booking_value_p99 + , bookings_source_src_10001.is_instant + , DATE_TRUNC('day', bookings_source_src_10001.ds) AS ds__day + , DATE_TRUNC('week', bookings_source_src_10001.ds) AS ds__week + , DATE_TRUNC('month', bookings_source_src_10001.ds) AS ds__month + , DATE_TRUNC('quarter', bookings_source_src_10001.ds) AS ds__quarter + , DATE_TRUNC('year', bookings_source_src_10001.ds) AS ds__year + , EXTRACT(year FROM bookings_source_src_10001.ds) AS ds__extract_year + , EXTRACT(quarter FROM bookings_source_src_10001.ds) AS ds__extract_quarter + , EXTRACT(month FROM bookings_source_src_10001.ds) AS ds__extract_month + , EXTRACT(day FROM bookings_source_src_10001.ds) AS ds__extract_day + , CASE WHEN EXTRACT(dow FROM bookings_source_src_10001.ds) = 0 THEN EXTRACT(dow FROM bookings_source_src_10001.ds) + 7 ELSE EXTRACT(dow FROM bookings_source_src_10001.ds) END AS ds__extract_dow + , EXTRACT(doy FROM bookings_source_src_10001.ds) AS ds__extract_doy + , DATE_TRUNC('day', bookings_source_src_10001.ds_partitioned) AS ds_partitioned__day + , DATE_TRUNC('week', bookings_source_src_10001.ds_partitioned) AS ds_partitioned__week + , DATE_TRUNC('month', bookings_source_src_10001.ds_partitioned) AS ds_partitioned__month + , DATE_TRUNC('quarter', bookings_source_src_10001.ds_partitioned) AS ds_partitioned__quarter + , DATE_TRUNC('year', bookings_source_src_10001.ds_partitioned) AS ds_partitioned__year + , EXTRACT(year FROM bookings_source_src_10001.ds_partitioned) AS ds_partitioned__extract_year + , EXTRACT(quarter FROM bookings_source_src_10001.ds_partitioned) AS ds_partitioned__extract_quarter + , EXTRACT(month FROM bookings_source_src_10001.ds_partitioned) AS ds_partitioned__extract_month + , EXTRACT(day FROM bookings_source_src_10001.ds_partitioned) AS ds_partitioned__extract_day + , CASE WHEN EXTRACT(dow FROM bookings_source_src_10001.ds_partitioned) = 0 THEN EXTRACT(dow FROM bookings_source_src_10001.ds_partitioned) + 7 ELSE EXTRACT(dow FROM bookings_source_src_10001.ds_partitioned) END AS ds_partitioned__extract_dow + , EXTRACT(doy FROM bookings_source_src_10001.ds_partitioned) AS ds_partitioned__extract_doy + , DATE_TRUNC('day', bookings_source_src_10001.paid_at) AS paid_at__day + , DATE_TRUNC('week', bookings_source_src_10001.paid_at) AS paid_at__week + , DATE_TRUNC('month', bookings_source_src_10001.paid_at) AS paid_at__month + , DATE_TRUNC('quarter', bookings_source_src_10001.paid_at) AS paid_at__quarter + , DATE_TRUNC('year', bookings_source_src_10001.paid_at) AS paid_at__year + , EXTRACT(year FROM bookings_source_src_10001.paid_at) AS paid_at__extract_year + , EXTRACT(quarter FROM bookings_source_src_10001.paid_at) AS paid_at__extract_quarter + , EXTRACT(month FROM bookings_source_src_10001.paid_at) AS paid_at__extract_month + , EXTRACT(day FROM bookings_source_src_10001.paid_at) AS paid_at__extract_day + , CASE WHEN EXTRACT(dow FROM bookings_source_src_10001.paid_at) = 0 THEN EXTRACT(dow FROM bookings_source_src_10001.paid_at) + 7 ELSE EXTRACT(dow FROM bookings_source_src_10001.paid_at) END AS paid_at__extract_dow + , EXTRACT(doy FROM bookings_source_src_10001.paid_at) AS paid_at__extract_doy + , bookings_source_src_10001.is_instant AS booking__is_instant + , DATE_TRUNC('day', bookings_source_src_10001.ds) AS booking__ds__day + , DATE_TRUNC('week', bookings_source_src_10001.ds) AS booking__ds__week + , DATE_TRUNC('month', bookings_source_src_10001.ds) AS booking__ds__month + , DATE_TRUNC('quarter', bookings_source_src_10001.ds) AS booking__ds__quarter + , DATE_TRUNC('year', bookings_source_src_10001.ds) AS booking__ds__year + , EXTRACT(year FROM bookings_source_src_10001.ds) AS booking__ds__extract_year + , EXTRACT(quarter FROM bookings_source_src_10001.ds) AS booking__ds__extract_quarter + , EXTRACT(month FROM bookings_source_src_10001.ds) AS booking__ds__extract_month + , EXTRACT(day FROM bookings_source_src_10001.ds) AS booking__ds__extract_day + , CASE WHEN EXTRACT(dow FROM bookings_source_src_10001.ds) = 0 THEN EXTRACT(dow FROM bookings_source_src_10001.ds) + 7 ELSE EXTRACT(dow FROM bookings_source_src_10001.ds) END AS booking__ds__extract_dow + , EXTRACT(doy FROM bookings_source_src_10001.ds) AS booking__ds__extract_doy + , DATE_TRUNC('day', bookings_source_src_10001.ds_partitioned) AS booking__ds_partitioned__day + , DATE_TRUNC('week', bookings_source_src_10001.ds_partitioned) AS booking__ds_partitioned__week + , DATE_TRUNC('month', bookings_source_src_10001.ds_partitioned) AS booking__ds_partitioned__month + , DATE_TRUNC('quarter', bookings_source_src_10001.ds_partitioned) AS booking__ds_partitioned__quarter + , DATE_TRUNC('year', bookings_source_src_10001.ds_partitioned) AS booking__ds_partitioned__year + , EXTRACT(year FROM bookings_source_src_10001.ds_partitioned) AS booking__ds_partitioned__extract_year + , EXTRACT(quarter FROM bookings_source_src_10001.ds_partitioned) AS booking__ds_partitioned__extract_quarter + , EXTRACT(month FROM bookings_source_src_10001.ds_partitioned) AS booking__ds_partitioned__extract_month + , EXTRACT(day FROM bookings_source_src_10001.ds_partitioned) AS booking__ds_partitioned__extract_day + , CASE WHEN EXTRACT(dow FROM bookings_source_src_10001.ds_partitioned) = 0 THEN EXTRACT(dow FROM bookings_source_src_10001.ds_partitioned) + 7 ELSE EXTRACT(dow FROM bookings_source_src_10001.ds_partitioned) END AS booking__ds_partitioned__extract_dow + , EXTRACT(doy FROM bookings_source_src_10001.ds_partitioned) AS booking__ds_partitioned__extract_doy + , DATE_TRUNC('day', bookings_source_src_10001.paid_at) AS booking__paid_at__day + , DATE_TRUNC('week', bookings_source_src_10001.paid_at) AS booking__paid_at__week + , DATE_TRUNC('month', bookings_source_src_10001.paid_at) AS booking__paid_at__month + , DATE_TRUNC('quarter', bookings_source_src_10001.paid_at) AS booking__paid_at__quarter + , DATE_TRUNC('year', bookings_source_src_10001.paid_at) AS booking__paid_at__year + , EXTRACT(year FROM bookings_source_src_10001.paid_at) AS booking__paid_at__extract_year + , EXTRACT(quarter FROM bookings_source_src_10001.paid_at) AS booking__paid_at__extract_quarter + , EXTRACT(month FROM bookings_source_src_10001.paid_at) AS booking__paid_at__extract_month + , EXTRACT(day FROM bookings_source_src_10001.paid_at) AS booking__paid_at__extract_day + , CASE WHEN EXTRACT(dow FROM bookings_source_src_10001.paid_at) = 0 THEN EXTRACT(dow FROM bookings_source_src_10001.paid_at) + 7 ELSE EXTRACT(dow FROM bookings_source_src_10001.paid_at) END AS booking__paid_at__extract_dow + , EXTRACT(doy FROM bookings_source_src_10001.paid_at) AS booking__paid_at__extract_doy + , bookings_source_src_10001.listing_id AS listing + , bookings_source_src_10001.guest_id AS guest + , bookings_source_src_10001.host_id AS host + , bookings_source_src_10001.listing_id AS booking__listing + , bookings_source_src_10001.guest_id AS booking__guest + , bookings_source_src_10001.host_id AS booking__host + FROM ***************************.fct_bookings bookings_source_src_10001 + ) subq_0 + ) subq_1 + GROUP BY + subq_1.is_instant +) subq_2 +FULL OUTER JOIN ( + -- Aggregate Measures + SELECT + subq_4.is_instant + , SUM(subq_4.instant_bookings) AS instant_bookings + , COUNT(DISTINCT subq_4.bookers) AS bookers + FROM ( + -- Pass Only Elements: + -- ['instant_bookings', 'bookers', 'is_instant'] + SELECT + subq_3.is_instant + , subq_3.instant_bookings + , subq_3.bookers + FROM ( + -- Read Elements From Semantic Model 'bookings_source' + SELECT + 1 AS bookings + , CASE WHEN is_instant THEN 1 ELSE 0 END AS instant_bookings + , bookings_source_src_10001.booking_value + , bookings_source_src_10001.booking_value AS max_booking_value + , bookings_source_src_10001.booking_value AS min_booking_value + , bookings_source_src_10001.guest_id AS bookers + , bookings_source_src_10001.booking_value AS average_booking_value + , bookings_source_src_10001.booking_value AS booking_payments + , CASE WHEN referrer_id IS NOT NULL THEN 1 ELSE 0 END AS referred_bookings + , bookings_source_src_10001.booking_value AS median_booking_value + , bookings_source_src_10001.booking_value AS booking_value_p99 + , bookings_source_src_10001.booking_value AS discrete_booking_value_p99 + , bookings_source_src_10001.booking_value AS approximate_continuous_booking_value_p99 + , bookings_source_src_10001.booking_value AS approximate_discrete_booking_value_p99 + , bookings_source_src_10001.is_instant + , DATE_TRUNC('day', bookings_source_src_10001.ds) AS ds__day + , DATE_TRUNC('week', bookings_source_src_10001.ds) AS ds__week + , DATE_TRUNC('month', bookings_source_src_10001.ds) AS ds__month + , DATE_TRUNC('quarter', bookings_source_src_10001.ds) AS ds__quarter + , DATE_TRUNC('year', bookings_source_src_10001.ds) AS ds__year + , EXTRACT(year FROM bookings_source_src_10001.ds) AS ds__extract_year + , EXTRACT(quarter FROM bookings_source_src_10001.ds) AS ds__extract_quarter + , EXTRACT(month FROM bookings_source_src_10001.ds) AS ds__extract_month + , EXTRACT(day FROM bookings_source_src_10001.ds) AS ds__extract_day + , CASE WHEN EXTRACT(dow FROM bookings_source_src_10001.ds) = 0 THEN EXTRACT(dow FROM bookings_source_src_10001.ds) + 7 ELSE EXTRACT(dow FROM bookings_source_src_10001.ds) END AS ds__extract_dow + , EXTRACT(doy FROM bookings_source_src_10001.ds) AS ds__extract_doy + , DATE_TRUNC('day', bookings_source_src_10001.ds_partitioned) AS ds_partitioned__day + , DATE_TRUNC('week', bookings_source_src_10001.ds_partitioned) AS ds_partitioned__week + , DATE_TRUNC('month', bookings_source_src_10001.ds_partitioned) AS ds_partitioned__month + , DATE_TRUNC('quarter', bookings_source_src_10001.ds_partitioned) AS ds_partitioned__quarter + , DATE_TRUNC('year', bookings_source_src_10001.ds_partitioned) AS ds_partitioned__year + , EXTRACT(year FROM bookings_source_src_10001.ds_partitioned) AS ds_partitioned__extract_year + , EXTRACT(quarter FROM bookings_source_src_10001.ds_partitioned) AS ds_partitioned__extract_quarter + , EXTRACT(month FROM bookings_source_src_10001.ds_partitioned) AS ds_partitioned__extract_month + , EXTRACT(day FROM bookings_source_src_10001.ds_partitioned) AS ds_partitioned__extract_day + , CASE WHEN EXTRACT(dow FROM bookings_source_src_10001.ds_partitioned) = 0 THEN EXTRACT(dow FROM bookings_source_src_10001.ds_partitioned) + 7 ELSE EXTRACT(dow FROM bookings_source_src_10001.ds_partitioned) END AS ds_partitioned__extract_dow + , EXTRACT(doy FROM bookings_source_src_10001.ds_partitioned) AS ds_partitioned__extract_doy + , DATE_TRUNC('day', bookings_source_src_10001.paid_at) AS paid_at__day + , DATE_TRUNC('week', bookings_source_src_10001.paid_at) AS paid_at__week + , DATE_TRUNC('month', bookings_source_src_10001.paid_at) AS paid_at__month + , DATE_TRUNC('quarter', bookings_source_src_10001.paid_at) AS paid_at__quarter + , DATE_TRUNC('year', bookings_source_src_10001.paid_at) AS paid_at__year + , EXTRACT(year FROM bookings_source_src_10001.paid_at) AS paid_at__extract_year + , EXTRACT(quarter FROM bookings_source_src_10001.paid_at) AS paid_at__extract_quarter + , EXTRACT(month FROM bookings_source_src_10001.paid_at) AS paid_at__extract_month + , EXTRACT(day FROM bookings_source_src_10001.paid_at) AS paid_at__extract_day + , CASE WHEN EXTRACT(dow FROM bookings_source_src_10001.paid_at) = 0 THEN EXTRACT(dow FROM bookings_source_src_10001.paid_at) + 7 ELSE EXTRACT(dow FROM bookings_source_src_10001.paid_at) END AS paid_at__extract_dow + , EXTRACT(doy FROM bookings_source_src_10001.paid_at) AS paid_at__extract_doy + , bookings_source_src_10001.is_instant AS booking__is_instant + , DATE_TRUNC('day', bookings_source_src_10001.ds) AS booking__ds__day + , DATE_TRUNC('week', bookings_source_src_10001.ds) AS booking__ds__week + , DATE_TRUNC('month', bookings_source_src_10001.ds) AS booking__ds__month + , DATE_TRUNC('quarter', bookings_source_src_10001.ds) AS booking__ds__quarter + , DATE_TRUNC('year', bookings_source_src_10001.ds) AS booking__ds__year + , EXTRACT(year FROM bookings_source_src_10001.ds) AS booking__ds__extract_year + , EXTRACT(quarter FROM bookings_source_src_10001.ds) AS booking__ds__extract_quarter + , EXTRACT(month FROM bookings_source_src_10001.ds) AS booking__ds__extract_month + , EXTRACT(day FROM bookings_source_src_10001.ds) AS booking__ds__extract_day + , CASE WHEN EXTRACT(dow FROM bookings_source_src_10001.ds) = 0 THEN EXTRACT(dow FROM bookings_source_src_10001.ds) + 7 ELSE EXTRACT(dow FROM bookings_source_src_10001.ds) END AS booking__ds__extract_dow + , EXTRACT(doy FROM bookings_source_src_10001.ds) AS booking__ds__extract_doy + , DATE_TRUNC('day', bookings_source_src_10001.ds_partitioned) AS booking__ds_partitioned__day + , DATE_TRUNC('week', bookings_source_src_10001.ds_partitioned) AS booking__ds_partitioned__week + , DATE_TRUNC('month', bookings_source_src_10001.ds_partitioned) AS booking__ds_partitioned__month + , DATE_TRUNC('quarter', bookings_source_src_10001.ds_partitioned) AS booking__ds_partitioned__quarter + , DATE_TRUNC('year', bookings_source_src_10001.ds_partitioned) AS booking__ds_partitioned__year + , EXTRACT(year FROM bookings_source_src_10001.ds_partitioned) AS booking__ds_partitioned__extract_year + , EXTRACT(quarter FROM bookings_source_src_10001.ds_partitioned) AS booking__ds_partitioned__extract_quarter + , EXTRACT(month FROM bookings_source_src_10001.ds_partitioned) AS booking__ds_partitioned__extract_month + , EXTRACT(day FROM bookings_source_src_10001.ds_partitioned) AS booking__ds_partitioned__extract_day + , CASE WHEN EXTRACT(dow FROM bookings_source_src_10001.ds_partitioned) = 0 THEN EXTRACT(dow FROM bookings_source_src_10001.ds_partitioned) + 7 ELSE EXTRACT(dow FROM bookings_source_src_10001.ds_partitioned) END AS booking__ds_partitioned__extract_dow + , EXTRACT(doy FROM bookings_source_src_10001.ds_partitioned) AS booking__ds_partitioned__extract_doy + , DATE_TRUNC('day', bookings_source_src_10001.paid_at) AS booking__paid_at__day + , DATE_TRUNC('week', bookings_source_src_10001.paid_at) AS booking__paid_at__week + , DATE_TRUNC('month', bookings_source_src_10001.paid_at) AS booking__paid_at__month + , DATE_TRUNC('quarter', bookings_source_src_10001.paid_at) AS booking__paid_at__quarter + , DATE_TRUNC('year', bookings_source_src_10001.paid_at) AS booking__paid_at__year + , EXTRACT(year FROM bookings_source_src_10001.paid_at) AS booking__paid_at__extract_year + , EXTRACT(quarter FROM bookings_source_src_10001.paid_at) AS booking__paid_at__extract_quarter + , EXTRACT(month FROM bookings_source_src_10001.paid_at) AS booking__paid_at__extract_month + , EXTRACT(day FROM bookings_source_src_10001.paid_at) AS booking__paid_at__extract_day + , CASE WHEN EXTRACT(dow FROM bookings_source_src_10001.paid_at) = 0 THEN EXTRACT(dow FROM bookings_source_src_10001.paid_at) + 7 ELSE EXTRACT(dow FROM bookings_source_src_10001.paid_at) END AS booking__paid_at__extract_dow + , EXTRACT(doy FROM bookings_source_src_10001.paid_at) AS booking__paid_at__extract_doy + , bookings_source_src_10001.listing_id AS listing + , bookings_source_src_10001.guest_id AS guest + , bookings_source_src_10001.host_id AS host + , bookings_source_src_10001.listing_id AS booking__listing + , bookings_source_src_10001.guest_id AS booking__guest + , bookings_source_src_10001.host_id AS booking__host + FROM ***************************.fct_bookings bookings_source_src_10001 + ) subq_3 + ) subq_4 + GROUP BY + subq_4.is_instant +) subq_5 +ON + subq_2.is_instant = subq_5.is_instant +GROUP BY + COALESCE(subq_2.is_instant, subq_5.is_instant) diff --git a/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Redshift/test_combine_output_node__plan0_optimized.sql b/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Redshift/test_combine_output_node__plan0_optimized.sql new file mode 100644 index 0000000000..62340aabcb --- /dev/null +++ b/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Redshift/test_combine_output_node__plan0_optimized.sql @@ -0,0 +1,46 @@ +-- Combine Aggregated Outputs +SELECT + COALESCE(subq_8.is_instant, subq_11.is_instant) AS is_instant + , MAX(subq_8.bookings) AS bookings + , COALESCE(MAX(subq_11.instant_bookings), 1) AS instant_bookings + , COALESCE(MAX(subq_11.bookers), 1) AS bookers +FROM ( + -- Aggregate Measures + SELECT + is_instant + , SUM(bookings) AS bookings + FROM ( + -- Read Elements From Semantic Model 'bookings_source' + -- Pass Only Elements: + -- ['bookings', 'is_instant'] + SELECT + is_instant + , 1 AS bookings + FROM ***************************.fct_bookings bookings_source_src_10001 + ) subq_7 + GROUP BY + is_instant +) subq_8 +FULL OUTER JOIN ( + -- Aggregate Measures + SELECT + is_instant + , SUM(instant_bookings) AS instant_bookings + , COUNT(DISTINCT bookers) AS bookers + FROM ( + -- Read Elements From Semantic Model 'bookings_source' + -- Pass Only Elements: + -- ['instant_bookings', 'bookers', 'is_instant'] + SELECT + is_instant + , CASE WHEN is_instant THEN 1 ELSE 0 END AS instant_bookings + , guest_id AS bookers + FROM ***************************.fct_bookings bookings_source_src_10001 + ) subq_10 + GROUP BY + is_instant +) subq_11 +ON + subq_8.is_instant = subq_11.is_instant +GROUP BY + COALESCE(subq_8.is_instant, subq_11.is_instant) diff --git a/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Redshift/test_compute_metrics_node_ratio_from_multiple_semantic_models__plan0.sql b/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Redshift/test_compute_metrics_node_ratio_from_multiple_semantic_models__plan0.sql index 8efb1d7083..c189d7644a 100644 --- a/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Redshift/test_compute_metrics_node_ratio_from_multiple_semantic_models__plan0.sql +++ b/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Redshift/test_compute_metrics_node_ratio_from_multiple_semantic_models__plan0.sql @@ -4,7 +4,7 @@ SELECT , subq_20.listing__country_latest , CAST(subq_20.bookings AS DOUBLE PRECISION) / CAST(NULLIF(subq_20.views, 0) AS DOUBLE PRECISION) AS bookings_per_view FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT COALESCE(subq_9.ds__day, subq_19.ds__day) AS ds__day , COALESCE(subq_9.listing__country_latest, subq_19.listing__country_latest) AS listing__country_latest diff --git a/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Redshift/test_compute_metrics_node_ratio_from_multiple_semantic_models__plan0_optimized.sql b/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Redshift/test_compute_metrics_node_ratio_from_multiple_semantic_models__plan0_optimized.sql index 9526df6729..63f6522655 100644 --- a/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Redshift/test_compute_metrics_node_ratio_from_multiple_semantic_models__plan0_optimized.sql +++ b/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Redshift/test_compute_metrics_node_ratio_from_multiple_semantic_models__plan0_optimized.sql @@ -4,7 +4,7 @@ SELECT , listing__country_latest , CAST(bookings AS DOUBLE PRECISION) / CAST(NULLIF(views, 0) AS DOUBLE PRECISION) AS bookings_per_view FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT COALESCE(subq_30.ds__day, subq_40.ds__day) AS ds__day , COALESCE(subq_30.listing__country_latest, subq_40.listing__country_latest) AS listing__country_latest diff --git a/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Snowflake/test_combine_output_node__plan0.sql b/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Snowflake/test_combine_output_node__plan0.sql new file mode 100644 index 0000000000..3f0303708d --- /dev/null +++ b/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Snowflake/test_combine_output_node__plan0.sql @@ -0,0 +1,228 @@ +-- Combine Aggregated Outputs +SELECT + COALESCE(subq_2.is_instant, subq_5.is_instant) AS is_instant + , MAX(subq_2.bookings) AS bookings + , COALESCE(MAX(subq_5.instant_bookings), 1) AS instant_bookings + , COALESCE(MAX(subq_5.bookers), 1) AS bookers +FROM ( + -- Aggregate Measures + SELECT + subq_1.is_instant + , SUM(subq_1.bookings) AS bookings + FROM ( + -- Pass Only Elements: + -- ['bookings', 'is_instant'] + SELECT + subq_0.is_instant + , subq_0.bookings + FROM ( + -- Read Elements From Semantic Model 'bookings_source' + SELECT + 1 AS bookings + , CASE WHEN is_instant THEN 1 ELSE 0 END AS instant_bookings + , bookings_source_src_10001.booking_value + , bookings_source_src_10001.booking_value AS max_booking_value + , bookings_source_src_10001.booking_value AS min_booking_value + , bookings_source_src_10001.guest_id AS bookers + , bookings_source_src_10001.booking_value AS average_booking_value + , bookings_source_src_10001.booking_value AS booking_payments + , CASE WHEN referrer_id IS NOT NULL THEN 1 ELSE 0 END AS referred_bookings + , bookings_source_src_10001.booking_value AS median_booking_value + , bookings_source_src_10001.booking_value AS booking_value_p99 + , bookings_source_src_10001.booking_value AS discrete_booking_value_p99 + , bookings_source_src_10001.booking_value AS approximate_continuous_booking_value_p99 + , bookings_source_src_10001.booking_value AS approximate_discrete_booking_value_p99 + , bookings_source_src_10001.is_instant + , DATE_TRUNC('day', bookings_source_src_10001.ds) AS ds__day + , DATE_TRUNC('week', bookings_source_src_10001.ds) AS ds__week + , DATE_TRUNC('month', bookings_source_src_10001.ds) AS ds__month + , DATE_TRUNC('quarter', bookings_source_src_10001.ds) AS ds__quarter + , DATE_TRUNC('year', bookings_source_src_10001.ds) AS ds__year + , EXTRACT(year FROM bookings_source_src_10001.ds) AS ds__extract_year + , EXTRACT(quarter FROM bookings_source_src_10001.ds) AS ds__extract_quarter + , EXTRACT(month FROM bookings_source_src_10001.ds) AS ds__extract_month + , EXTRACT(day FROM bookings_source_src_10001.ds) AS ds__extract_day + , EXTRACT(dayofweekiso FROM bookings_source_src_10001.ds) AS ds__extract_dow + , EXTRACT(doy FROM bookings_source_src_10001.ds) AS ds__extract_doy + , DATE_TRUNC('day', bookings_source_src_10001.ds_partitioned) AS ds_partitioned__day + , DATE_TRUNC('week', bookings_source_src_10001.ds_partitioned) AS ds_partitioned__week + , DATE_TRUNC('month', bookings_source_src_10001.ds_partitioned) AS ds_partitioned__month + , DATE_TRUNC('quarter', bookings_source_src_10001.ds_partitioned) AS ds_partitioned__quarter + , DATE_TRUNC('year', bookings_source_src_10001.ds_partitioned) AS ds_partitioned__year + , EXTRACT(year FROM bookings_source_src_10001.ds_partitioned) AS ds_partitioned__extract_year + , EXTRACT(quarter FROM bookings_source_src_10001.ds_partitioned) AS ds_partitioned__extract_quarter + , EXTRACT(month FROM bookings_source_src_10001.ds_partitioned) AS ds_partitioned__extract_month + , EXTRACT(day FROM bookings_source_src_10001.ds_partitioned) AS ds_partitioned__extract_day + , EXTRACT(dayofweekiso FROM bookings_source_src_10001.ds_partitioned) AS ds_partitioned__extract_dow + , EXTRACT(doy FROM bookings_source_src_10001.ds_partitioned) AS ds_partitioned__extract_doy + , DATE_TRUNC('day', bookings_source_src_10001.paid_at) AS paid_at__day + , DATE_TRUNC('week', bookings_source_src_10001.paid_at) AS paid_at__week + , DATE_TRUNC('month', bookings_source_src_10001.paid_at) AS paid_at__month + , DATE_TRUNC('quarter', bookings_source_src_10001.paid_at) AS paid_at__quarter + , DATE_TRUNC('year', bookings_source_src_10001.paid_at) AS paid_at__year + , EXTRACT(year FROM bookings_source_src_10001.paid_at) AS paid_at__extract_year + , EXTRACT(quarter FROM bookings_source_src_10001.paid_at) AS paid_at__extract_quarter + , EXTRACT(month FROM bookings_source_src_10001.paid_at) AS paid_at__extract_month + , EXTRACT(day FROM bookings_source_src_10001.paid_at) AS paid_at__extract_day + , EXTRACT(dayofweekiso FROM bookings_source_src_10001.paid_at) AS paid_at__extract_dow + , EXTRACT(doy FROM bookings_source_src_10001.paid_at) AS paid_at__extract_doy + , bookings_source_src_10001.is_instant AS booking__is_instant + , DATE_TRUNC('day', bookings_source_src_10001.ds) AS booking__ds__day + , DATE_TRUNC('week', bookings_source_src_10001.ds) AS booking__ds__week + , DATE_TRUNC('month', bookings_source_src_10001.ds) AS booking__ds__month + , DATE_TRUNC('quarter', bookings_source_src_10001.ds) AS booking__ds__quarter + , DATE_TRUNC('year', bookings_source_src_10001.ds) AS booking__ds__year + , EXTRACT(year FROM bookings_source_src_10001.ds) AS booking__ds__extract_year + , EXTRACT(quarter FROM bookings_source_src_10001.ds) AS booking__ds__extract_quarter + , EXTRACT(month FROM bookings_source_src_10001.ds) AS booking__ds__extract_month + , EXTRACT(day FROM bookings_source_src_10001.ds) AS booking__ds__extract_day + , EXTRACT(dayofweekiso FROM bookings_source_src_10001.ds) AS booking__ds__extract_dow + , EXTRACT(doy FROM bookings_source_src_10001.ds) AS booking__ds__extract_doy + , DATE_TRUNC('day', bookings_source_src_10001.ds_partitioned) AS booking__ds_partitioned__day + , DATE_TRUNC('week', bookings_source_src_10001.ds_partitioned) AS booking__ds_partitioned__week + , DATE_TRUNC('month', bookings_source_src_10001.ds_partitioned) AS booking__ds_partitioned__month + , DATE_TRUNC('quarter', bookings_source_src_10001.ds_partitioned) AS booking__ds_partitioned__quarter + , DATE_TRUNC('year', bookings_source_src_10001.ds_partitioned) AS booking__ds_partitioned__year + , EXTRACT(year FROM bookings_source_src_10001.ds_partitioned) AS booking__ds_partitioned__extract_year + , EXTRACT(quarter FROM bookings_source_src_10001.ds_partitioned) AS booking__ds_partitioned__extract_quarter + , EXTRACT(month FROM bookings_source_src_10001.ds_partitioned) AS booking__ds_partitioned__extract_month + , EXTRACT(day FROM bookings_source_src_10001.ds_partitioned) AS booking__ds_partitioned__extract_day + , EXTRACT(dayofweekiso FROM bookings_source_src_10001.ds_partitioned) AS booking__ds_partitioned__extract_dow + , EXTRACT(doy FROM bookings_source_src_10001.ds_partitioned) AS booking__ds_partitioned__extract_doy + , DATE_TRUNC('day', bookings_source_src_10001.paid_at) AS booking__paid_at__day + , DATE_TRUNC('week', bookings_source_src_10001.paid_at) AS booking__paid_at__week + , DATE_TRUNC('month', bookings_source_src_10001.paid_at) AS booking__paid_at__month + , DATE_TRUNC('quarter', bookings_source_src_10001.paid_at) AS booking__paid_at__quarter + , DATE_TRUNC('year', bookings_source_src_10001.paid_at) AS booking__paid_at__year + , EXTRACT(year FROM bookings_source_src_10001.paid_at) AS booking__paid_at__extract_year + , EXTRACT(quarter FROM bookings_source_src_10001.paid_at) AS booking__paid_at__extract_quarter + , EXTRACT(month FROM bookings_source_src_10001.paid_at) AS booking__paid_at__extract_month + , EXTRACT(day FROM bookings_source_src_10001.paid_at) AS booking__paid_at__extract_day + , EXTRACT(dayofweekiso FROM bookings_source_src_10001.paid_at) AS booking__paid_at__extract_dow + , EXTRACT(doy FROM bookings_source_src_10001.paid_at) AS booking__paid_at__extract_doy + , bookings_source_src_10001.listing_id AS listing + , bookings_source_src_10001.guest_id AS guest + , bookings_source_src_10001.host_id AS host + , bookings_source_src_10001.listing_id AS booking__listing + , bookings_source_src_10001.guest_id AS booking__guest + , bookings_source_src_10001.host_id AS booking__host + FROM ***************************.fct_bookings bookings_source_src_10001 + ) subq_0 + ) subq_1 + GROUP BY + subq_1.is_instant +) subq_2 +FULL OUTER JOIN ( + -- Aggregate Measures + SELECT + subq_4.is_instant + , SUM(subq_4.instant_bookings) AS instant_bookings + , COUNT(DISTINCT subq_4.bookers) AS bookers + FROM ( + -- Pass Only Elements: + -- ['instant_bookings', 'bookers', 'is_instant'] + SELECT + subq_3.is_instant + , subq_3.instant_bookings + , subq_3.bookers + FROM ( + -- Read Elements From Semantic Model 'bookings_source' + SELECT + 1 AS bookings + , CASE WHEN is_instant THEN 1 ELSE 0 END AS instant_bookings + , bookings_source_src_10001.booking_value + , bookings_source_src_10001.booking_value AS max_booking_value + , bookings_source_src_10001.booking_value AS min_booking_value + , bookings_source_src_10001.guest_id AS bookers + , bookings_source_src_10001.booking_value AS average_booking_value + , bookings_source_src_10001.booking_value AS booking_payments + , CASE WHEN referrer_id IS NOT NULL THEN 1 ELSE 0 END AS referred_bookings + , bookings_source_src_10001.booking_value AS median_booking_value + , bookings_source_src_10001.booking_value AS booking_value_p99 + , bookings_source_src_10001.booking_value AS discrete_booking_value_p99 + , bookings_source_src_10001.booking_value AS approximate_continuous_booking_value_p99 + , bookings_source_src_10001.booking_value AS approximate_discrete_booking_value_p99 + , bookings_source_src_10001.is_instant + , DATE_TRUNC('day', bookings_source_src_10001.ds) AS ds__day + , DATE_TRUNC('week', bookings_source_src_10001.ds) AS ds__week + , DATE_TRUNC('month', bookings_source_src_10001.ds) AS ds__month + , DATE_TRUNC('quarter', bookings_source_src_10001.ds) AS ds__quarter + , DATE_TRUNC('year', bookings_source_src_10001.ds) AS ds__year + , EXTRACT(year FROM bookings_source_src_10001.ds) AS ds__extract_year + , EXTRACT(quarter FROM bookings_source_src_10001.ds) AS ds__extract_quarter + , EXTRACT(month FROM bookings_source_src_10001.ds) AS ds__extract_month + , EXTRACT(day FROM bookings_source_src_10001.ds) AS ds__extract_day + , EXTRACT(dayofweekiso FROM bookings_source_src_10001.ds) AS ds__extract_dow + , EXTRACT(doy FROM bookings_source_src_10001.ds) AS ds__extract_doy + , DATE_TRUNC('day', bookings_source_src_10001.ds_partitioned) AS ds_partitioned__day + , DATE_TRUNC('week', bookings_source_src_10001.ds_partitioned) AS ds_partitioned__week + , DATE_TRUNC('month', bookings_source_src_10001.ds_partitioned) AS ds_partitioned__month + , DATE_TRUNC('quarter', bookings_source_src_10001.ds_partitioned) AS ds_partitioned__quarter + , DATE_TRUNC('year', bookings_source_src_10001.ds_partitioned) AS ds_partitioned__year + , EXTRACT(year FROM bookings_source_src_10001.ds_partitioned) AS ds_partitioned__extract_year + , EXTRACT(quarter FROM bookings_source_src_10001.ds_partitioned) AS ds_partitioned__extract_quarter + , EXTRACT(month FROM bookings_source_src_10001.ds_partitioned) AS ds_partitioned__extract_month + , EXTRACT(day FROM bookings_source_src_10001.ds_partitioned) AS ds_partitioned__extract_day + , EXTRACT(dayofweekiso FROM bookings_source_src_10001.ds_partitioned) AS ds_partitioned__extract_dow + , EXTRACT(doy FROM bookings_source_src_10001.ds_partitioned) AS ds_partitioned__extract_doy + , DATE_TRUNC('day', bookings_source_src_10001.paid_at) AS paid_at__day + , DATE_TRUNC('week', bookings_source_src_10001.paid_at) AS paid_at__week + , DATE_TRUNC('month', bookings_source_src_10001.paid_at) AS paid_at__month + , DATE_TRUNC('quarter', bookings_source_src_10001.paid_at) AS paid_at__quarter + , DATE_TRUNC('year', bookings_source_src_10001.paid_at) AS paid_at__year + , EXTRACT(year FROM bookings_source_src_10001.paid_at) AS paid_at__extract_year + , EXTRACT(quarter FROM bookings_source_src_10001.paid_at) AS paid_at__extract_quarter + , EXTRACT(month FROM bookings_source_src_10001.paid_at) AS paid_at__extract_month + , EXTRACT(day FROM bookings_source_src_10001.paid_at) AS paid_at__extract_day + , EXTRACT(dayofweekiso FROM bookings_source_src_10001.paid_at) AS paid_at__extract_dow + , EXTRACT(doy FROM bookings_source_src_10001.paid_at) AS paid_at__extract_doy + , bookings_source_src_10001.is_instant AS booking__is_instant + , DATE_TRUNC('day', bookings_source_src_10001.ds) AS booking__ds__day + , DATE_TRUNC('week', bookings_source_src_10001.ds) AS booking__ds__week + , DATE_TRUNC('month', bookings_source_src_10001.ds) AS booking__ds__month + , DATE_TRUNC('quarter', bookings_source_src_10001.ds) AS booking__ds__quarter + , DATE_TRUNC('year', bookings_source_src_10001.ds) AS booking__ds__year + , EXTRACT(year FROM bookings_source_src_10001.ds) AS booking__ds__extract_year + , EXTRACT(quarter FROM bookings_source_src_10001.ds) AS booking__ds__extract_quarter + , EXTRACT(month FROM bookings_source_src_10001.ds) AS booking__ds__extract_month + , EXTRACT(day FROM bookings_source_src_10001.ds) AS booking__ds__extract_day + , EXTRACT(dayofweekiso FROM bookings_source_src_10001.ds) AS booking__ds__extract_dow + , EXTRACT(doy FROM bookings_source_src_10001.ds) AS booking__ds__extract_doy + , DATE_TRUNC('day', bookings_source_src_10001.ds_partitioned) AS booking__ds_partitioned__day + , DATE_TRUNC('week', bookings_source_src_10001.ds_partitioned) AS booking__ds_partitioned__week + , DATE_TRUNC('month', bookings_source_src_10001.ds_partitioned) AS booking__ds_partitioned__month + , DATE_TRUNC('quarter', bookings_source_src_10001.ds_partitioned) AS booking__ds_partitioned__quarter + , DATE_TRUNC('year', bookings_source_src_10001.ds_partitioned) AS booking__ds_partitioned__year + , EXTRACT(year FROM bookings_source_src_10001.ds_partitioned) AS booking__ds_partitioned__extract_year + , EXTRACT(quarter FROM bookings_source_src_10001.ds_partitioned) AS booking__ds_partitioned__extract_quarter + , EXTRACT(month FROM bookings_source_src_10001.ds_partitioned) AS booking__ds_partitioned__extract_month + , EXTRACT(day FROM bookings_source_src_10001.ds_partitioned) AS booking__ds_partitioned__extract_day + , EXTRACT(dayofweekiso FROM bookings_source_src_10001.ds_partitioned) AS booking__ds_partitioned__extract_dow + , EXTRACT(doy FROM bookings_source_src_10001.ds_partitioned) AS booking__ds_partitioned__extract_doy + , DATE_TRUNC('day', bookings_source_src_10001.paid_at) AS booking__paid_at__day + , DATE_TRUNC('week', bookings_source_src_10001.paid_at) AS booking__paid_at__week + , DATE_TRUNC('month', bookings_source_src_10001.paid_at) AS booking__paid_at__month + , DATE_TRUNC('quarter', bookings_source_src_10001.paid_at) AS booking__paid_at__quarter + , DATE_TRUNC('year', bookings_source_src_10001.paid_at) AS booking__paid_at__year + , EXTRACT(year FROM bookings_source_src_10001.paid_at) AS booking__paid_at__extract_year + , EXTRACT(quarter FROM bookings_source_src_10001.paid_at) AS booking__paid_at__extract_quarter + , EXTRACT(month FROM bookings_source_src_10001.paid_at) AS booking__paid_at__extract_month + , EXTRACT(day FROM bookings_source_src_10001.paid_at) AS booking__paid_at__extract_day + , EXTRACT(dayofweekiso FROM bookings_source_src_10001.paid_at) AS booking__paid_at__extract_dow + , EXTRACT(doy FROM bookings_source_src_10001.paid_at) AS booking__paid_at__extract_doy + , bookings_source_src_10001.listing_id AS listing + , bookings_source_src_10001.guest_id AS guest + , bookings_source_src_10001.host_id AS host + , bookings_source_src_10001.listing_id AS booking__listing + , bookings_source_src_10001.guest_id AS booking__guest + , bookings_source_src_10001.host_id AS booking__host + FROM ***************************.fct_bookings bookings_source_src_10001 + ) subq_3 + ) subq_4 + GROUP BY + subq_4.is_instant +) subq_5 +ON + subq_2.is_instant = subq_5.is_instant +GROUP BY + COALESCE(subq_2.is_instant, subq_5.is_instant) diff --git a/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Snowflake/test_combine_output_node__plan0_optimized.sql b/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Snowflake/test_combine_output_node__plan0_optimized.sql new file mode 100644 index 0000000000..62340aabcb --- /dev/null +++ b/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Snowflake/test_combine_output_node__plan0_optimized.sql @@ -0,0 +1,46 @@ +-- Combine Aggregated Outputs +SELECT + COALESCE(subq_8.is_instant, subq_11.is_instant) AS is_instant + , MAX(subq_8.bookings) AS bookings + , COALESCE(MAX(subq_11.instant_bookings), 1) AS instant_bookings + , COALESCE(MAX(subq_11.bookers), 1) AS bookers +FROM ( + -- Aggregate Measures + SELECT + is_instant + , SUM(bookings) AS bookings + FROM ( + -- Read Elements From Semantic Model 'bookings_source' + -- Pass Only Elements: + -- ['bookings', 'is_instant'] + SELECT + is_instant + , 1 AS bookings + FROM ***************************.fct_bookings bookings_source_src_10001 + ) subq_7 + GROUP BY + is_instant +) subq_8 +FULL OUTER JOIN ( + -- Aggregate Measures + SELECT + is_instant + , SUM(instant_bookings) AS instant_bookings + , COUNT(DISTINCT bookers) AS bookers + FROM ( + -- Read Elements From Semantic Model 'bookings_source' + -- Pass Only Elements: + -- ['instant_bookings', 'bookers', 'is_instant'] + SELECT + is_instant + , CASE WHEN is_instant THEN 1 ELSE 0 END AS instant_bookings + , guest_id AS bookers + FROM ***************************.fct_bookings bookings_source_src_10001 + ) subq_10 + GROUP BY + is_instant +) subq_11 +ON + subq_8.is_instant = subq_11.is_instant +GROUP BY + COALESCE(subq_8.is_instant, subq_11.is_instant) diff --git a/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Snowflake/test_compute_metrics_node_ratio_from_multiple_semantic_models__plan0.sql b/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Snowflake/test_compute_metrics_node_ratio_from_multiple_semantic_models__plan0.sql index 81aee1e5e8..38473b92d1 100644 --- a/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Snowflake/test_compute_metrics_node_ratio_from_multiple_semantic_models__plan0.sql +++ b/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Snowflake/test_compute_metrics_node_ratio_from_multiple_semantic_models__plan0.sql @@ -4,7 +4,7 @@ SELECT , subq_20.listing__country_latest , CAST(subq_20.bookings AS DOUBLE) / CAST(NULLIF(subq_20.views, 0) AS DOUBLE) AS bookings_per_view FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT COALESCE(subq_9.ds__day, subq_19.ds__day) AS ds__day , COALESCE(subq_9.listing__country_latest, subq_19.listing__country_latest) AS listing__country_latest diff --git a/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Snowflake/test_compute_metrics_node_ratio_from_multiple_semantic_models__plan0_optimized.sql b/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Snowflake/test_compute_metrics_node_ratio_from_multiple_semantic_models__plan0_optimized.sql index b53b1f06f4..635bfacdef 100644 --- a/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Snowflake/test_compute_metrics_node_ratio_from_multiple_semantic_models__plan0_optimized.sql +++ b/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Snowflake/test_compute_metrics_node_ratio_from_multiple_semantic_models__plan0_optimized.sql @@ -4,7 +4,7 @@ SELECT , listing__country_latest , CAST(bookings AS DOUBLE) / CAST(NULLIF(views, 0) AS DOUBLE) AS bookings_per_view FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT COALESCE(subq_30.ds__day, subq_40.ds__day) AS ds__day , COALESCE(subq_30.listing__country_latest, subq_40.listing__country_latest) AS listing__country_latest diff --git a/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/test_compute_metrics_node_ratio_from_multiple_semantic_models__plan0.xml b/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/test_compute_metrics_node_ratio_from_multiple_semantic_models__plan0.xml index c7c329e83e..cc5132fbd1 100644 --- a/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/test_compute_metrics_node_ratio_from_multiple_semantic_models__plan0.xml +++ b/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/test_compute_metrics_node_ratio_from_multiple_semantic_models__plan0.xml @@ -18,7 +18,7 @@ - + diff --git a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/BigQuery/test_derived_metric__plan0.sql b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/BigQuery/test_derived_metric__plan0.sql index 601900bcd7..3124a0233d 100644 --- a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/BigQuery/test_derived_metric__plan0.sql +++ b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/BigQuery/test_derived_metric__plan0.sql @@ -3,7 +3,7 @@ SELECT subq_10.metric_time__day , (bookings - ref_bookings) * 1.0 / bookings AS non_referred_bookings_pct FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT COALESCE(subq_4.metric_time__day, subq_9.metric_time__day) AS metric_time__day , MAX(subq_4.ref_bookings) AS ref_bookings diff --git a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/BigQuery/test_derived_metric__plan0_optimized.sql b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/BigQuery/test_derived_metric__plan0_optimized.sql index b0d8d0b304..bb2f68ddf9 100644 --- a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/BigQuery/test_derived_metric__plan0_optimized.sql +++ b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/BigQuery/test_derived_metric__plan0_optimized.sql @@ -3,7 +3,7 @@ SELECT metric_time__day , (bookings - ref_bookings) * 1.0 / bookings AS non_referred_bookings_pct FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT COALESCE(subq_15.metric_time__day, subq_20.metric_time__day) AS metric_time__day , MAX(subq_15.ref_bookings) AS ref_bookings diff --git a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/BigQuery/test_derived_metric_with_offset_to_grain__plan0.sql b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/BigQuery/test_derived_metric_with_offset_to_grain__plan0.sql index f03d500002..b41db5191d 100644 --- a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/BigQuery/test_derived_metric_with_offset_to_grain__plan0.sql +++ b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/BigQuery/test_derived_metric_with_offset_to_grain__plan0.sql @@ -3,7 +3,7 @@ SELECT subq_13.metric_time__day , bookings - bookings_at_start_of_month AS bookings_growth_since_start_of_month FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT COALESCE(subq_4.metric_time__day, subq_12.metric_time__day) AS metric_time__day , MAX(subq_4.bookings) AS bookings diff --git a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/BigQuery/test_derived_metric_with_offset_to_grain__plan0_optimized.sql b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/BigQuery/test_derived_metric_with_offset_to_grain__plan0_optimized.sql index 7eef401bbe..99494f1d2b 100644 --- a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/BigQuery/test_derived_metric_with_offset_to_grain__plan0_optimized.sql +++ b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/BigQuery/test_derived_metric_with_offset_to_grain__plan0_optimized.sql @@ -3,7 +3,7 @@ SELECT metric_time__day , bookings - bookings_at_start_of_month AS bookings_growth_since_start_of_month FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT COALESCE(subq_18.metric_time__day, subq_26.metric_time__day) AS metric_time__day , MAX(subq_18.bookings) AS bookings diff --git a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/BigQuery/test_derived_metric_with_offset_to_grain_and_granularity__plan0.sql b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/BigQuery/test_derived_metric_with_offset_to_grain_and_granularity__plan0.sql index 44823a15e3..a1f2594d57 100644 --- a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/BigQuery/test_derived_metric_with_offset_to_grain_and_granularity__plan0.sql +++ b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/BigQuery/test_derived_metric_with_offset_to_grain_and_granularity__plan0.sql @@ -3,7 +3,7 @@ SELECT subq_13.metric_time__week , bookings - bookings_at_start_of_month AS bookings_growth_since_start_of_month FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT COALESCE(subq_4.metric_time__week, subq_12.metric_time__week) AS metric_time__week , MAX(subq_4.bookings) AS bookings diff --git a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/BigQuery/test_derived_metric_with_offset_to_grain_and_granularity__plan0_optimized.sql b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/BigQuery/test_derived_metric_with_offset_to_grain_and_granularity__plan0_optimized.sql index a2b43bfa72..6452810522 100644 --- a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/BigQuery/test_derived_metric_with_offset_to_grain_and_granularity__plan0_optimized.sql +++ b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/BigQuery/test_derived_metric_with_offset_to_grain_and_granularity__plan0_optimized.sql @@ -3,7 +3,7 @@ SELECT metric_time__week , bookings - bookings_at_start_of_month AS bookings_growth_since_start_of_month FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT COALESCE(subq_18.metric_time__week, subq_26.metric_time__week) AS metric_time__week , MAX(subq_18.bookings) AS bookings diff --git a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/BigQuery/test_derived_metric_with_offset_window__plan0.sql b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/BigQuery/test_derived_metric_with_offset_window__plan0.sql index d0fb01c279..92b172c295 100644 --- a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/BigQuery/test_derived_metric_with_offset_window__plan0.sql +++ b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/BigQuery/test_derived_metric_with_offset_window__plan0.sql @@ -3,7 +3,7 @@ SELECT subq_13.metric_time__day , bookings - bookings_2_weeks_ago AS bookings_growth_2_weeks FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT COALESCE(subq_4.metric_time__day, subq_12.metric_time__day) AS metric_time__day , MAX(subq_4.bookings) AS bookings diff --git a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/BigQuery/test_derived_metric_with_offset_window__plan0_optimized.sql b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/BigQuery/test_derived_metric_with_offset_window__plan0_optimized.sql index 397b1bd968..0908c851b4 100644 --- a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/BigQuery/test_derived_metric_with_offset_window__plan0_optimized.sql +++ b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/BigQuery/test_derived_metric_with_offset_window__plan0_optimized.sql @@ -3,7 +3,7 @@ SELECT metric_time__day , bookings - bookings_2_weeks_ago AS bookings_growth_2_weeks FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT COALESCE(subq_18.metric_time__day, subq_26.metric_time__day) AS metric_time__day , MAX(subq_18.bookings) AS bookings diff --git a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/BigQuery/test_derived_metric_with_offset_window_and_granularity__plan0.sql b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/BigQuery/test_derived_metric_with_offset_window_and_granularity__plan0.sql index 7f6a988e25..a59339e246 100644 --- a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/BigQuery/test_derived_metric_with_offset_window_and_granularity__plan0.sql +++ b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/BigQuery/test_derived_metric_with_offset_window_and_granularity__plan0.sql @@ -3,7 +3,7 @@ SELECT subq_13.metric_time__quarter , bookings - bookings_2_weeks_ago AS bookings_growth_2_weeks FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT COALESCE(subq_4.metric_time__quarter, subq_12.metric_time__quarter) AS metric_time__quarter , MAX(subq_4.bookings) AS bookings diff --git a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/BigQuery/test_derived_metric_with_offset_window_and_granularity__plan0_optimized.sql b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/BigQuery/test_derived_metric_with_offset_window_and_granularity__plan0_optimized.sql index 0b6326c173..b0a4a72502 100644 --- a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/BigQuery/test_derived_metric_with_offset_window_and_granularity__plan0_optimized.sql +++ b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/BigQuery/test_derived_metric_with_offset_window_and_granularity__plan0_optimized.sql @@ -3,7 +3,7 @@ SELECT metric_time__quarter , bookings - bookings_2_weeks_ago AS bookings_growth_2_weeks FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT COALESCE(subq_18.metric_time__quarter, subq_26.metric_time__quarter) AS metric_time__quarter , MAX(subq_18.bookings) AS bookings diff --git a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/BigQuery/test_derived_metric_with_offset_window_and_offset_to_grain__plan0.sql b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/BigQuery/test_derived_metric_with_offset_window_and_offset_to_grain__plan0.sql index 3c9cdf8bec..ceea092c4a 100644 --- a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/BigQuery/test_derived_metric_with_offset_window_and_offset_to_grain__plan0.sql +++ b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/BigQuery/test_derived_metric_with_offset_window_and_offset_to_grain__plan0.sql @@ -3,7 +3,7 @@ SELECT subq_16.metric_time__day , month_start_bookings - bookings_1_month_ago AS bookings_month_start_compared_to_1_month_prior FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT COALESCE(subq_7.metric_time__day, subq_15.metric_time__day) AS metric_time__day , MAX(subq_7.month_start_bookings) AS month_start_bookings diff --git a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/BigQuery/test_derived_metric_with_offset_window_and_offset_to_grain__plan0_optimized.sql b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/BigQuery/test_derived_metric_with_offset_window_and_offset_to_grain__plan0_optimized.sql index 503310614c..18f61dabf4 100644 --- a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/BigQuery/test_derived_metric_with_offset_window_and_offset_to_grain__plan0_optimized.sql +++ b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/BigQuery/test_derived_metric_with_offset_window_and_offset_to_grain__plan0_optimized.sql @@ -3,7 +3,7 @@ SELECT metric_time__day , month_start_bookings - bookings_1_month_ago AS bookings_month_start_compared_to_1_month_prior FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT COALESCE(subq_24.metric_time__day, subq_32.metric_time__day) AS metric_time__day , MAX(subq_24.month_start_bookings) AS month_start_bookings diff --git a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/BigQuery/test_derived_metric_with_offset_window_and_offset_to_grain_and_granularity__plan0.sql b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/BigQuery/test_derived_metric_with_offset_window_and_offset_to_grain_and_granularity__plan0.sql index d09a6065af..57f61c433a 100644 --- a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/BigQuery/test_derived_metric_with_offset_window_and_offset_to_grain_and_granularity__plan0.sql +++ b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/BigQuery/test_derived_metric_with_offset_window_and_offset_to_grain_and_granularity__plan0.sql @@ -3,7 +3,7 @@ SELECT subq_16.metric_time__year , month_start_bookings - bookings_1_month_ago AS bookings_month_start_compared_to_1_month_prior FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT COALESCE(subq_7.metric_time__year, subq_15.metric_time__year) AS metric_time__year , MAX(subq_7.month_start_bookings) AS month_start_bookings diff --git a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/BigQuery/test_derived_metric_with_offset_window_and_offset_to_grain_and_granularity__plan0_optimized.sql b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/BigQuery/test_derived_metric_with_offset_window_and_offset_to_grain_and_granularity__plan0_optimized.sql index 30d82361e0..ebc8a1a064 100644 --- a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/BigQuery/test_derived_metric_with_offset_window_and_offset_to_grain_and_granularity__plan0_optimized.sql +++ b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/BigQuery/test_derived_metric_with_offset_window_and_offset_to_grain_and_granularity__plan0_optimized.sql @@ -3,7 +3,7 @@ SELECT metric_time__year , month_start_bookings - bookings_1_month_ago AS bookings_month_start_compared_to_1_month_prior FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT COALESCE(subq_24.metric_time__year, subq_32.metric_time__year) AS metric_time__year , MAX(subq_24.month_start_bookings) AS month_start_bookings diff --git a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/BigQuery/test_derived_metric_with_offset_window_and_time_filter__plan0.sql b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/BigQuery/test_derived_metric_with_offset_window_and_time_filter__plan0.sql index 00f50dd62e..e24faae304 100644 --- a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/BigQuery/test_derived_metric_with_offset_window_and_time_filter__plan0.sql +++ b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/BigQuery/test_derived_metric_with_offset_window_and_time_filter__plan0.sql @@ -3,7 +3,7 @@ SELECT subq_15.metric_time__day , bookings - bookings_2_weeks_ago AS bookings_growth_2_weeks FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT COALESCE(subq_5.metric_time__day, subq_14.metric_time__day) AS metric_time__day , MAX(subq_5.bookings) AS bookings diff --git a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/BigQuery/test_derived_metric_with_offset_window_and_time_filter__plan0_optimized.sql b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/BigQuery/test_derived_metric_with_offset_window_and_time_filter__plan0_optimized.sql index 9a1b629882..31da750c43 100644 --- a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/BigQuery/test_derived_metric_with_offset_window_and_time_filter__plan0_optimized.sql +++ b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/BigQuery/test_derived_metric_with_offset_window_and_time_filter__plan0_optimized.sql @@ -3,7 +3,7 @@ SELECT metric_time__day , bookings - bookings_2_weeks_ago AS bookings_growth_2_weeks FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT COALESCE(subq_21.metric_time__day, subq_30.metric_time__day) AS metric_time__day , MAX(subq_21.bookings) AS bookings diff --git a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/BigQuery/test_nested_derived_metric__plan0.sql b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/BigQuery/test_nested_derived_metric__plan0.sql index 9432f82a2f..4455985d21 100644 --- a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/BigQuery/test_nested_derived_metric__plan0.sql +++ b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/BigQuery/test_nested_derived_metric__plan0.sql @@ -3,7 +3,7 @@ SELECT subq_22.metric_time__day , non_referred + (instant * 1.0 / bookings) AS instant_plus_non_referred_bookings_pct FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT COALESCE(subq_11.metric_time__day, subq_16.metric_time__day, subq_21.metric_time__day) AS metric_time__day , MAX(subq_11.non_referred) AS non_referred @@ -15,7 +15,7 @@ FROM ( subq_10.metric_time__day , (bookings - ref_bookings) * 1.0 / bookings AS non_referred FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT COALESCE(subq_4.metric_time__day, subq_9.metric_time__day) AS metric_time__day , MAX(subq_4.ref_bookings) AS ref_bookings diff --git a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/BigQuery/test_nested_derived_metric__plan0_optimized.sql b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/BigQuery/test_nested_derived_metric__plan0_optimized.sql index 9dd7154226..3544bef97c 100644 --- a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/BigQuery/test_nested_derived_metric__plan0_optimized.sql +++ b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/BigQuery/test_nested_derived_metric__plan0_optimized.sql @@ -3,7 +3,7 @@ SELECT metric_time__day , non_referred + (instant * 1.0 / bookings) AS instant_plus_non_referred_bookings_pct FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT COALESCE(subq_34.metric_time__day, subq_39.metric_time__day, subq_44.metric_time__day) AS metric_time__day , MAX(subq_34.non_referred) AS non_referred @@ -15,7 +15,7 @@ FROM ( metric_time__day , (bookings - ref_bookings) * 1.0 / bookings AS non_referred FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT COALESCE(subq_27.metric_time__day, subq_32.metric_time__day) AS metric_time__day , MAX(subq_27.ref_bookings) AS ref_bookings diff --git a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Databricks/test_derived_metric__plan0.sql b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Databricks/test_derived_metric__plan0.sql index 453868aaa2..6d56996d84 100644 --- a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Databricks/test_derived_metric__plan0.sql +++ b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Databricks/test_derived_metric__plan0.sql @@ -3,7 +3,7 @@ SELECT subq_10.metric_time__day , (bookings - ref_bookings) * 1.0 / bookings AS non_referred_bookings_pct FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT COALESCE(subq_4.metric_time__day, subq_9.metric_time__day) AS metric_time__day , MAX(subq_4.ref_bookings) AS ref_bookings diff --git a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Databricks/test_derived_metric__plan0_optimized.sql b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Databricks/test_derived_metric__plan0_optimized.sql index f7d37d45b3..b3e757c0c6 100644 --- a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Databricks/test_derived_metric__plan0_optimized.sql +++ b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Databricks/test_derived_metric__plan0_optimized.sql @@ -3,7 +3,7 @@ SELECT metric_time__day , (bookings - ref_bookings) * 1.0 / bookings AS non_referred_bookings_pct FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT COALESCE(subq_15.metric_time__day, subq_20.metric_time__day) AS metric_time__day , MAX(subq_15.ref_bookings) AS ref_bookings diff --git a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Databricks/test_derived_metric_with_offset_to_grain__plan0.sql b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Databricks/test_derived_metric_with_offset_to_grain__plan0.sql index 76852df7e0..009eac2dc6 100644 --- a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Databricks/test_derived_metric_with_offset_to_grain__plan0.sql +++ b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Databricks/test_derived_metric_with_offset_to_grain__plan0.sql @@ -3,7 +3,7 @@ SELECT subq_13.metric_time__day , bookings - bookings_at_start_of_month AS bookings_growth_since_start_of_month FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT COALESCE(subq_4.metric_time__day, subq_12.metric_time__day) AS metric_time__day , MAX(subq_4.bookings) AS bookings diff --git a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Databricks/test_derived_metric_with_offset_to_grain__plan0_optimized.sql b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Databricks/test_derived_metric_with_offset_to_grain__plan0_optimized.sql index 6d1b17b416..3763e526e9 100644 --- a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Databricks/test_derived_metric_with_offset_to_grain__plan0_optimized.sql +++ b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Databricks/test_derived_metric_with_offset_to_grain__plan0_optimized.sql @@ -3,7 +3,7 @@ SELECT metric_time__day , bookings - bookings_at_start_of_month AS bookings_growth_since_start_of_month FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT COALESCE(subq_18.metric_time__day, subq_26.metric_time__day) AS metric_time__day , MAX(subq_18.bookings) AS bookings diff --git a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Databricks/test_derived_metric_with_offset_to_grain_and_granularity__plan0.sql b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Databricks/test_derived_metric_with_offset_to_grain_and_granularity__plan0.sql index 4ca9c85eb2..e993841cde 100644 --- a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Databricks/test_derived_metric_with_offset_to_grain_and_granularity__plan0.sql +++ b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Databricks/test_derived_metric_with_offset_to_grain_and_granularity__plan0.sql @@ -3,7 +3,7 @@ SELECT subq_13.metric_time__week , bookings - bookings_at_start_of_month AS bookings_growth_since_start_of_month FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT COALESCE(subq_4.metric_time__week, subq_12.metric_time__week) AS metric_time__week , MAX(subq_4.bookings) AS bookings diff --git a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Databricks/test_derived_metric_with_offset_to_grain_and_granularity__plan0_optimized.sql b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Databricks/test_derived_metric_with_offset_to_grain_and_granularity__plan0_optimized.sql index 5b18fcfa5f..0ac0bf4ca1 100644 --- a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Databricks/test_derived_metric_with_offset_to_grain_and_granularity__plan0_optimized.sql +++ b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Databricks/test_derived_metric_with_offset_to_grain_and_granularity__plan0_optimized.sql @@ -3,7 +3,7 @@ SELECT metric_time__week , bookings - bookings_at_start_of_month AS bookings_growth_since_start_of_month FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT COALESCE(subq_18.metric_time__week, subq_26.metric_time__week) AS metric_time__week , MAX(subq_18.bookings) AS bookings diff --git a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Databricks/test_derived_metric_with_offset_window__plan0.sql b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Databricks/test_derived_metric_with_offset_window__plan0.sql index a08f661fd7..6d218b8b0d 100644 --- a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Databricks/test_derived_metric_with_offset_window__plan0.sql +++ b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Databricks/test_derived_metric_with_offset_window__plan0.sql @@ -3,7 +3,7 @@ SELECT subq_13.metric_time__day , bookings - bookings_2_weeks_ago AS bookings_growth_2_weeks FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT COALESCE(subq_4.metric_time__day, subq_12.metric_time__day) AS metric_time__day , MAX(subq_4.bookings) AS bookings diff --git a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Databricks/test_derived_metric_with_offset_window__plan0_optimized.sql b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Databricks/test_derived_metric_with_offset_window__plan0_optimized.sql index a01f18c65d..96cd7db009 100644 --- a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Databricks/test_derived_metric_with_offset_window__plan0_optimized.sql +++ b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Databricks/test_derived_metric_with_offset_window__plan0_optimized.sql @@ -3,7 +3,7 @@ SELECT metric_time__day , bookings - bookings_2_weeks_ago AS bookings_growth_2_weeks FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT COALESCE(subq_18.metric_time__day, subq_26.metric_time__day) AS metric_time__day , MAX(subq_18.bookings) AS bookings diff --git a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Databricks/test_derived_metric_with_offset_window_and_granularity__plan0.sql b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Databricks/test_derived_metric_with_offset_window_and_granularity__plan0.sql index 86edbbf9bc..376cf3b1db 100644 --- a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Databricks/test_derived_metric_with_offset_window_and_granularity__plan0.sql +++ b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Databricks/test_derived_metric_with_offset_window_and_granularity__plan0.sql @@ -3,7 +3,7 @@ SELECT subq_13.metric_time__quarter , bookings - bookings_2_weeks_ago AS bookings_growth_2_weeks FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT COALESCE(subq_4.metric_time__quarter, subq_12.metric_time__quarter) AS metric_time__quarter , MAX(subq_4.bookings) AS bookings diff --git a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Databricks/test_derived_metric_with_offset_window_and_granularity__plan0_optimized.sql b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Databricks/test_derived_metric_with_offset_window_and_granularity__plan0_optimized.sql index 10ccf3e20b..919cfd1274 100644 --- a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Databricks/test_derived_metric_with_offset_window_and_granularity__plan0_optimized.sql +++ b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Databricks/test_derived_metric_with_offset_window_and_granularity__plan0_optimized.sql @@ -3,7 +3,7 @@ SELECT metric_time__quarter , bookings - bookings_2_weeks_ago AS bookings_growth_2_weeks FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT COALESCE(subq_18.metric_time__quarter, subq_26.metric_time__quarter) AS metric_time__quarter , MAX(subq_18.bookings) AS bookings diff --git a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Databricks/test_derived_metric_with_offset_window_and_offset_to_grain__plan0.sql b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Databricks/test_derived_metric_with_offset_window_and_offset_to_grain__plan0.sql index 6230a73553..82260863dc 100644 --- a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Databricks/test_derived_metric_with_offset_window_and_offset_to_grain__plan0.sql +++ b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Databricks/test_derived_metric_with_offset_window_and_offset_to_grain__plan0.sql @@ -3,7 +3,7 @@ SELECT subq_16.metric_time__day , month_start_bookings - bookings_1_month_ago AS bookings_month_start_compared_to_1_month_prior FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT COALESCE(subq_7.metric_time__day, subq_15.metric_time__day) AS metric_time__day , MAX(subq_7.month_start_bookings) AS month_start_bookings diff --git a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Databricks/test_derived_metric_with_offset_window_and_offset_to_grain__plan0_optimized.sql b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Databricks/test_derived_metric_with_offset_window_and_offset_to_grain__plan0_optimized.sql index 91dc642b1b..33f71b11c3 100644 --- a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Databricks/test_derived_metric_with_offset_window_and_offset_to_grain__plan0_optimized.sql +++ b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Databricks/test_derived_metric_with_offset_window_and_offset_to_grain__plan0_optimized.sql @@ -3,7 +3,7 @@ SELECT metric_time__day , month_start_bookings - bookings_1_month_ago AS bookings_month_start_compared_to_1_month_prior FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT COALESCE(subq_24.metric_time__day, subq_32.metric_time__day) AS metric_time__day , MAX(subq_24.month_start_bookings) AS month_start_bookings diff --git a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Databricks/test_derived_metric_with_offset_window_and_offset_to_grain_and_granularity__plan0.sql b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Databricks/test_derived_metric_with_offset_window_and_offset_to_grain_and_granularity__plan0.sql index e28765e766..eb9cb1f964 100644 --- a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Databricks/test_derived_metric_with_offset_window_and_offset_to_grain_and_granularity__plan0.sql +++ b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Databricks/test_derived_metric_with_offset_window_and_offset_to_grain_and_granularity__plan0.sql @@ -3,7 +3,7 @@ SELECT subq_16.metric_time__year , month_start_bookings - bookings_1_month_ago AS bookings_month_start_compared_to_1_month_prior FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT COALESCE(subq_7.metric_time__year, subq_15.metric_time__year) AS metric_time__year , MAX(subq_7.month_start_bookings) AS month_start_bookings diff --git a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Databricks/test_derived_metric_with_offset_window_and_offset_to_grain_and_granularity__plan0_optimized.sql b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Databricks/test_derived_metric_with_offset_window_and_offset_to_grain_and_granularity__plan0_optimized.sql index f06da5a3a2..0d96f7f3f0 100644 --- a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Databricks/test_derived_metric_with_offset_window_and_offset_to_grain_and_granularity__plan0_optimized.sql +++ b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Databricks/test_derived_metric_with_offset_window_and_offset_to_grain_and_granularity__plan0_optimized.sql @@ -3,7 +3,7 @@ SELECT metric_time__year , month_start_bookings - bookings_1_month_ago AS bookings_month_start_compared_to_1_month_prior FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT COALESCE(subq_24.metric_time__year, subq_32.metric_time__year) AS metric_time__year , MAX(subq_24.month_start_bookings) AS month_start_bookings diff --git a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Databricks/test_derived_metric_with_offset_window_and_time_filter__plan0.sql b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Databricks/test_derived_metric_with_offset_window_and_time_filter__plan0.sql index 4b4ac46fe8..337698837b 100644 --- a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Databricks/test_derived_metric_with_offset_window_and_time_filter__plan0.sql +++ b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Databricks/test_derived_metric_with_offset_window_and_time_filter__plan0.sql @@ -3,7 +3,7 @@ SELECT subq_15.metric_time__day , bookings - bookings_2_weeks_ago AS bookings_growth_2_weeks FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT COALESCE(subq_5.metric_time__day, subq_14.metric_time__day) AS metric_time__day , MAX(subq_5.bookings) AS bookings diff --git a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Databricks/test_derived_metric_with_offset_window_and_time_filter__plan0_optimized.sql b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Databricks/test_derived_metric_with_offset_window_and_time_filter__plan0_optimized.sql index 032f104a2d..c2e865bddc 100644 --- a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Databricks/test_derived_metric_with_offset_window_and_time_filter__plan0_optimized.sql +++ b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Databricks/test_derived_metric_with_offset_window_and_time_filter__plan0_optimized.sql @@ -3,7 +3,7 @@ SELECT metric_time__day , bookings - bookings_2_weeks_ago AS bookings_growth_2_weeks FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT COALESCE(subq_21.metric_time__day, subq_30.metric_time__day) AS metric_time__day , MAX(subq_21.bookings) AS bookings diff --git a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Databricks/test_nested_derived_metric__plan0.sql b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Databricks/test_nested_derived_metric__plan0.sql index 061321a32e..56976bf146 100644 --- a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Databricks/test_nested_derived_metric__plan0.sql +++ b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Databricks/test_nested_derived_metric__plan0.sql @@ -3,7 +3,7 @@ SELECT subq_22.metric_time__day , non_referred + (instant * 1.0 / bookings) AS instant_plus_non_referred_bookings_pct FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT COALESCE(subq_11.metric_time__day, subq_16.metric_time__day, subq_21.metric_time__day) AS metric_time__day , MAX(subq_11.non_referred) AS non_referred @@ -15,7 +15,7 @@ FROM ( subq_10.metric_time__day , (bookings - ref_bookings) * 1.0 / bookings AS non_referred FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT COALESCE(subq_4.metric_time__day, subq_9.metric_time__day) AS metric_time__day , MAX(subq_4.ref_bookings) AS ref_bookings diff --git a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Databricks/test_nested_derived_metric__plan0_optimized.sql b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Databricks/test_nested_derived_metric__plan0_optimized.sql index 86124d9091..97c8543446 100644 --- a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Databricks/test_nested_derived_metric__plan0_optimized.sql +++ b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Databricks/test_nested_derived_metric__plan0_optimized.sql @@ -3,7 +3,7 @@ SELECT metric_time__day , non_referred + (instant * 1.0 / bookings) AS instant_plus_non_referred_bookings_pct FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT COALESCE(subq_34.metric_time__day, subq_39.metric_time__day, subq_44.metric_time__day) AS metric_time__day , MAX(subq_34.non_referred) AS non_referred @@ -15,7 +15,7 @@ FROM ( metric_time__day , (bookings - ref_bookings) * 1.0 / bookings AS non_referred FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT COALESCE(subq_27.metric_time__day, subq_32.metric_time__day) AS metric_time__day , MAX(subq_27.ref_bookings) AS ref_bookings diff --git a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/DuckDB/test_derived_metric__plan0.sql b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/DuckDB/test_derived_metric__plan0.sql index 85eee899d7..d6c134f2b2 100644 --- a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/DuckDB/test_derived_metric__plan0.sql +++ b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/DuckDB/test_derived_metric__plan0.sql @@ -3,7 +3,7 @@ SELECT subq_10.metric_time__day , (bookings - ref_bookings) * 1.0 / bookings AS non_referred_bookings_pct FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT COALESCE(subq_4.metric_time__day, subq_9.metric_time__day) AS metric_time__day , MAX(subq_4.ref_bookings) AS ref_bookings diff --git a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/DuckDB/test_derived_metric__plan0_optimized.sql b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/DuckDB/test_derived_metric__plan0_optimized.sql index f7d37d45b3..b3e757c0c6 100644 --- a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/DuckDB/test_derived_metric__plan0_optimized.sql +++ b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/DuckDB/test_derived_metric__plan0_optimized.sql @@ -3,7 +3,7 @@ SELECT metric_time__day , (bookings - ref_bookings) * 1.0 / bookings AS non_referred_bookings_pct FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT COALESCE(subq_15.metric_time__day, subq_20.metric_time__day) AS metric_time__day , MAX(subq_15.ref_bookings) AS ref_bookings diff --git a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/DuckDB/test_derived_metric_with_offset_to_grain__plan0.sql b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/DuckDB/test_derived_metric_with_offset_to_grain__plan0.sql index d8bb186c28..57c2a1b747 100644 --- a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/DuckDB/test_derived_metric_with_offset_to_grain__plan0.sql +++ b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/DuckDB/test_derived_metric_with_offset_to_grain__plan0.sql @@ -3,7 +3,7 @@ SELECT subq_13.metric_time__day , bookings - bookings_at_start_of_month AS bookings_growth_since_start_of_month FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT COALESCE(subq_4.metric_time__day, subq_12.metric_time__day) AS metric_time__day , MAX(subq_4.bookings) AS bookings diff --git a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/DuckDB/test_derived_metric_with_offset_to_grain__plan0_optimized.sql b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/DuckDB/test_derived_metric_with_offset_to_grain__plan0_optimized.sql index 6d1b17b416..3763e526e9 100644 --- a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/DuckDB/test_derived_metric_with_offset_to_grain__plan0_optimized.sql +++ b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/DuckDB/test_derived_metric_with_offset_to_grain__plan0_optimized.sql @@ -3,7 +3,7 @@ SELECT metric_time__day , bookings - bookings_at_start_of_month AS bookings_growth_since_start_of_month FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT COALESCE(subq_18.metric_time__day, subq_26.metric_time__day) AS metric_time__day , MAX(subq_18.bookings) AS bookings diff --git a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/DuckDB/test_derived_metric_with_offset_to_grain_and_granularity__plan0.sql b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/DuckDB/test_derived_metric_with_offset_to_grain_and_granularity__plan0.sql index 9e51107f26..8dab36ab0a 100644 --- a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/DuckDB/test_derived_metric_with_offset_to_grain_and_granularity__plan0.sql +++ b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/DuckDB/test_derived_metric_with_offset_to_grain_and_granularity__plan0.sql @@ -3,7 +3,7 @@ SELECT subq_13.metric_time__week , bookings - bookings_at_start_of_month AS bookings_growth_since_start_of_month FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT COALESCE(subq_4.metric_time__week, subq_12.metric_time__week) AS metric_time__week , MAX(subq_4.bookings) AS bookings diff --git a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/DuckDB/test_derived_metric_with_offset_to_grain_and_granularity__plan0_optimized.sql b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/DuckDB/test_derived_metric_with_offset_to_grain_and_granularity__plan0_optimized.sql index 5b18fcfa5f..0ac0bf4ca1 100644 --- a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/DuckDB/test_derived_metric_with_offset_to_grain_and_granularity__plan0_optimized.sql +++ b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/DuckDB/test_derived_metric_with_offset_to_grain_and_granularity__plan0_optimized.sql @@ -3,7 +3,7 @@ SELECT metric_time__week , bookings - bookings_at_start_of_month AS bookings_growth_since_start_of_month FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT COALESCE(subq_18.metric_time__week, subq_26.metric_time__week) AS metric_time__week , MAX(subq_18.bookings) AS bookings diff --git a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/DuckDB/test_derived_metric_with_offset_window__plan0.sql b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/DuckDB/test_derived_metric_with_offset_window__plan0.sql index ff86172aba..0cc8e55655 100644 --- a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/DuckDB/test_derived_metric_with_offset_window__plan0.sql +++ b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/DuckDB/test_derived_metric_with_offset_window__plan0.sql @@ -3,7 +3,7 @@ SELECT subq_13.metric_time__day , bookings - bookings_2_weeks_ago AS bookings_growth_2_weeks FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT COALESCE(subq_4.metric_time__day, subq_12.metric_time__day) AS metric_time__day , MAX(subq_4.bookings) AS bookings diff --git a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/DuckDB/test_derived_metric_with_offset_window__plan0_optimized.sql b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/DuckDB/test_derived_metric_with_offset_window__plan0_optimized.sql index 85492077fe..b079c124d4 100644 --- a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/DuckDB/test_derived_metric_with_offset_window__plan0_optimized.sql +++ b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/DuckDB/test_derived_metric_with_offset_window__plan0_optimized.sql @@ -3,7 +3,7 @@ SELECT metric_time__day , bookings - bookings_2_weeks_ago AS bookings_growth_2_weeks FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT COALESCE(subq_18.metric_time__day, subq_26.metric_time__day) AS metric_time__day , MAX(subq_18.bookings) AS bookings diff --git a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/DuckDB/test_derived_metric_with_offset_window_and_granularity__plan0.sql b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/DuckDB/test_derived_metric_with_offset_window_and_granularity__plan0.sql index b63165d712..5bf8952548 100644 --- a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/DuckDB/test_derived_metric_with_offset_window_and_granularity__plan0.sql +++ b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/DuckDB/test_derived_metric_with_offset_window_and_granularity__plan0.sql @@ -3,7 +3,7 @@ SELECT subq_13.metric_time__quarter , bookings - bookings_2_weeks_ago AS bookings_growth_2_weeks FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT COALESCE(subq_4.metric_time__quarter, subq_12.metric_time__quarter) AS metric_time__quarter , MAX(subq_4.bookings) AS bookings diff --git a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/DuckDB/test_derived_metric_with_offset_window_and_granularity__plan0_optimized.sql b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/DuckDB/test_derived_metric_with_offset_window_and_granularity__plan0_optimized.sql index babc99022a..9e34944fab 100644 --- a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/DuckDB/test_derived_metric_with_offset_window_and_granularity__plan0_optimized.sql +++ b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/DuckDB/test_derived_metric_with_offset_window_and_granularity__plan0_optimized.sql @@ -3,7 +3,7 @@ SELECT metric_time__quarter , bookings - bookings_2_weeks_ago AS bookings_growth_2_weeks FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT COALESCE(subq_18.metric_time__quarter, subq_26.metric_time__quarter) AS metric_time__quarter , MAX(subq_18.bookings) AS bookings diff --git a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/DuckDB/test_derived_metric_with_offset_window_and_offset_to_grain__plan0.sql b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/DuckDB/test_derived_metric_with_offset_window_and_offset_to_grain__plan0.sql index 81ba9df25d..4649fec5c4 100644 --- a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/DuckDB/test_derived_metric_with_offset_window_and_offset_to_grain__plan0.sql +++ b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/DuckDB/test_derived_metric_with_offset_window_and_offset_to_grain__plan0.sql @@ -3,7 +3,7 @@ SELECT subq_16.metric_time__day , month_start_bookings - bookings_1_month_ago AS bookings_month_start_compared_to_1_month_prior FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT COALESCE(subq_7.metric_time__day, subq_15.metric_time__day) AS metric_time__day , MAX(subq_7.month_start_bookings) AS month_start_bookings diff --git a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/DuckDB/test_derived_metric_with_offset_window_and_offset_to_grain__plan0_optimized.sql b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/DuckDB/test_derived_metric_with_offset_window_and_offset_to_grain__plan0_optimized.sql index 9127806db2..6cea345b6d 100644 --- a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/DuckDB/test_derived_metric_with_offset_window_and_offset_to_grain__plan0_optimized.sql +++ b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/DuckDB/test_derived_metric_with_offset_window_and_offset_to_grain__plan0_optimized.sql @@ -3,7 +3,7 @@ SELECT metric_time__day , month_start_bookings - bookings_1_month_ago AS bookings_month_start_compared_to_1_month_prior FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT COALESCE(subq_24.metric_time__day, subq_32.metric_time__day) AS metric_time__day , MAX(subq_24.month_start_bookings) AS month_start_bookings diff --git a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/DuckDB/test_derived_metric_with_offset_window_and_offset_to_grain_and_granularity__plan0.sql b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/DuckDB/test_derived_metric_with_offset_window_and_offset_to_grain_and_granularity__plan0.sql index 345740c7ad..0e316a2388 100644 --- a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/DuckDB/test_derived_metric_with_offset_window_and_offset_to_grain_and_granularity__plan0.sql +++ b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/DuckDB/test_derived_metric_with_offset_window_and_offset_to_grain_and_granularity__plan0.sql @@ -3,7 +3,7 @@ SELECT subq_16.metric_time__year , month_start_bookings - bookings_1_month_ago AS bookings_month_start_compared_to_1_month_prior FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT COALESCE(subq_7.metric_time__year, subq_15.metric_time__year) AS metric_time__year , MAX(subq_7.month_start_bookings) AS month_start_bookings diff --git a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/DuckDB/test_derived_metric_with_offset_window_and_offset_to_grain_and_granularity__plan0_optimized.sql b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/DuckDB/test_derived_metric_with_offset_window_and_offset_to_grain_and_granularity__plan0_optimized.sql index e0166fe3f3..86cb74ccbc 100644 --- a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/DuckDB/test_derived_metric_with_offset_window_and_offset_to_grain_and_granularity__plan0_optimized.sql +++ b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/DuckDB/test_derived_metric_with_offset_window_and_offset_to_grain_and_granularity__plan0_optimized.sql @@ -3,7 +3,7 @@ SELECT metric_time__year , month_start_bookings - bookings_1_month_ago AS bookings_month_start_compared_to_1_month_prior FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT COALESCE(subq_24.metric_time__year, subq_32.metric_time__year) AS metric_time__year , MAX(subq_24.month_start_bookings) AS month_start_bookings diff --git a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/DuckDB/test_derived_metric_with_offset_window_and_time_filter__plan0.sql b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/DuckDB/test_derived_metric_with_offset_window_and_time_filter__plan0.sql index f23803a5a7..c9555a94e8 100644 --- a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/DuckDB/test_derived_metric_with_offset_window_and_time_filter__plan0.sql +++ b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/DuckDB/test_derived_metric_with_offset_window_and_time_filter__plan0.sql @@ -3,7 +3,7 @@ SELECT subq_15.metric_time__day , bookings - bookings_2_weeks_ago AS bookings_growth_2_weeks FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT COALESCE(subq_5.metric_time__day, subq_14.metric_time__day) AS metric_time__day , MAX(subq_5.bookings) AS bookings diff --git a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/DuckDB/test_derived_metric_with_offset_window_and_time_filter__plan0_optimized.sql b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/DuckDB/test_derived_metric_with_offset_window_and_time_filter__plan0_optimized.sql index a0bd21160c..cb49133416 100644 --- a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/DuckDB/test_derived_metric_with_offset_window_and_time_filter__plan0_optimized.sql +++ b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/DuckDB/test_derived_metric_with_offset_window_and_time_filter__plan0_optimized.sql @@ -3,7 +3,7 @@ SELECT metric_time__day , bookings - bookings_2_weeks_ago AS bookings_growth_2_weeks FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT COALESCE(subq_21.metric_time__day, subq_30.metric_time__day) AS metric_time__day , MAX(subq_21.bookings) AS bookings diff --git a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/DuckDB/test_nested_derived_metric__plan0.sql b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/DuckDB/test_nested_derived_metric__plan0.sql index 0e23b840ec..0538b786c8 100644 --- a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/DuckDB/test_nested_derived_metric__plan0.sql +++ b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/DuckDB/test_nested_derived_metric__plan0.sql @@ -3,7 +3,7 @@ SELECT subq_22.metric_time__day , non_referred + (instant * 1.0 / bookings) AS instant_plus_non_referred_bookings_pct FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT COALESCE(subq_11.metric_time__day, subq_16.metric_time__day, subq_21.metric_time__day) AS metric_time__day , MAX(subq_11.non_referred) AS non_referred @@ -15,7 +15,7 @@ FROM ( subq_10.metric_time__day , (bookings - ref_bookings) * 1.0 / bookings AS non_referred FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT COALESCE(subq_4.metric_time__day, subq_9.metric_time__day) AS metric_time__day , MAX(subq_4.ref_bookings) AS ref_bookings diff --git a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/DuckDB/test_nested_derived_metric__plan0_optimized.sql b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/DuckDB/test_nested_derived_metric__plan0_optimized.sql index 86124d9091..97c8543446 100644 --- a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/DuckDB/test_nested_derived_metric__plan0_optimized.sql +++ b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/DuckDB/test_nested_derived_metric__plan0_optimized.sql @@ -3,7 +3,7 @@ SELECT metric_time__day , non_referred + (instant * 1.0 / bookings) AS instant_plus_non_referred_bookings_pct FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT COALESCE(subq_34.metric_time__day, subq_39.metric_time__day, subq_44.metric_time__day) AS metric_time__day , MAX(subq_34.non_referred) AS non_referred @@ -15,7 +15,7 @@ FROM ( metric_time__day , (bookings - ref_bookings) * 1.0 / bookings AS non_referred FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT COALESCE(subq_27.metric_time__day, subq_32.metric_time__day) AS metric_time__day , MAX(subq_27.ref_bookings) AS ref_bookings diff --git a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Postgres/test_derived_metric__plan0.sql b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Postgres/test_derived_metric__plan0.sql index 85eee899d7..d6c134f2b2 100644 --- a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Postgres/test_derived_metric__plan0.sql +++ b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Postgres/test_derived_metric__plan0.sql @@ -3,7 +3,7 @@ SELECT subq_10.metric_time__day , (bookings - ref_bookings) * 1.0 / bookings AS non_referred_bookings_pct FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT COALESCE(subq_4.metric_time__day, subq_9.metric_time__day) AS metric_time__day , MAX(subq_4.ref_bookings) AS ref_bookings diff --git a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Postgres/test_derived_metric__plan0_optimized.sql b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Postgres/test_derived_metric__plan0_optimized.sql index f7d37d45b3..b3e757c0c6 100644 --- a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Postgres/test_derived_metric__plan0_optimized.sql +++ b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Postgres/test_derived_metric__plan0_optimized.sql @@ -3,7 +3,7 @@ SELECT metric_time__day , (bookings - ref_bookings) * 1.0 / bookings AS non_referred_bookings_pct FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT COALESCE(subq_15.metric_time__day, subq_20.metric_time__day) AS metric_time__day , MAX(subq_15.ref_bookings) AS ref_bookings diff --git a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Postgres/test_derived_metric_with_offset_to_grain__plan0.sql b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Postgres/test_derived_metric_with_offset_to_grain__plan0.sql index d8bb186c28..57c2a1b747 100644 --- a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Postgres/test_derived_metric_with_offset_to_grain__plan0.sql +++ b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Postgres/test_derived_metric_with_offset_to_grain__plan0.sql @@ -3,7 +3,7 @@ SELECT subq_13.metric_time__day , bookings - bookings_at_start_of_month AS bookings_growth_since_start_of_month FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT COALESCE(subq_4.metric_time__day, subq_12.metric_time__day) AS metric_time__day , MAX(subq_4.bookings) AS bookings diff --git a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Postgres/test_derived_metric_with_offset_to_grain__plan0_optimized.sql b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Postgres/test_derived_metric_with_offset_to_grain__plan0_optimized.sql index 6d1b17b416..3763e526e9 100644 --- a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Postgres/test_derived_metric_with_offset_to_grain__plan0_optimized.sql +++ b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Postgres/test_derived_metric_with_offset_to_grain__plan0_optimized.sql @@ -3,7 +3,7 @@ SELECT metric_time__day , bookings - bookings_at_start_of_month AS bookings_growth_since_start_of_month FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT COALESCE(subq_18.metric_time__day, subq_26.metric_time__day) AS metric_time__day , MAX(subq_18.bookings) AS bookings diff --git a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Postgres/test_derived_metric_with_offset_to_grain_and_granularity__plan0.sql b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Postgres/test_derived_metric_with_offset_to_grain_and_granularity__plan0.sql index 9e51107f26..8dab36ab0a 100644 --- a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Postgres/test_derived_metric_with_offset_to_grain_and_granularity__plan0.sql +++ b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Postgres/test_derived_metric_with_offset_to_grain_and_granularity__plan0.sql @@ -3,7 +3,7 @@ SELECT subq_13.metric_time__week , bookings - bookings_at_start_of_month AS bookings_growth_since_start_of_month FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT COALESCE(subq_4.metric_time__week, subq_12.metric_time__week) AS metric_time__week , MAX(subq_4.bookings) AS bookings diff --git a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Postgres/test_derived_metric_with_offset_to_grain_and_granularity__plan0_optimized.sql b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Postgres/test_derived_metric_with_offset_to_grain_and_granularity__plan0_optimized.sql index 5b18fcfa5f..0ac0bf4ca1 100644 --- a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Postgres/test_derived_metric_with_offset_to_grain_and_granularity__plan0_optimized.sql +++ b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Postgres/test_derived_metric_with_offset_to_grain_and_granularity__plan0_optimized.sql @@ -3,7 +3,7 @@ SELECT metric_time__week , bookings - bookings_at_start_of_month AS bookings_growth_since_start_of_month FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT COALESCE(subq_18.metric_time__week, subq_26.metric_time__week) AS metric_time__week , MAX(subq_18.bookings) AS bookings diff --git a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Postgres/test_derived_metric_with_offset_window__plan0.sql b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Postgres/test_derived_metric_with_offset_window__plan0.sql index 69941e528e..30e11dd66e 100644 --- a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Postgres/test_derived_metric_with_offset_window__plan0.sql +++ b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Postgres/test_derived_metric_with_offset_window__plan0.sql @@ -3,7 +3,7 @@ SELECT subq_13.metric_time__day , bookings - bookings_2_weeks_ago AS bookings_growth_2_weeks FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT COALESCE(subq_4.metric_time__day, subq_12.metric_time__day) AS metric_time__day , MAX(subq_4.bookings) AS bookings diff --git a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Postgres/test_derived_metric_with_offset_window__plan0_optimized.sql b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Postgres/test_derived_metric_with_offset_window__plan0_optimized.sql index 7b84983917..bd95511378 100644 --- a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Postgres/test_derived_metric_with_offset_window__plan0_optimized.sql +++ b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Postgres/test_derived_metric_with_offset_window__plan0_optimized.sql @@ -3,7 +3,7 @@ SELECT metric_time__day , bookings - bookings_2_weeks_ago AS bookings_growth_2_weeks FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT COALESCE(subq_18.metric_time__day, subq_26.metric_time__day) AS metric_time__day , MAX(subq_18.bookings) AS bookings diff --git a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Postgres/test_derived_metric_with_offset_window_and_granularity__plan0.sql b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Postgres/test_derived_metric_with_offset_window_and_granularity__plan0.sql index f36d2d9cec..f39e8f9ddd 100644 --- a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Postgres/test_derived_metric_with_offset_window_and_granularity__plan0.sql +++ b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Postgres/test_derived_metric_with_offset_window_and_granularity__plan0.sql @@ -3,7 +3,7 @@ SELECT subq_13.metric_time__quarter , bookings - bookings_2_weeks_ago AS bookings_growth_2_weeks FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT COALESCE(subq_4.metric_time__quarter, subq_12.metric_time__quarter) AS metric_time__quarter , MAX(subq_4.bookings) AS bookings diff --git a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Postgres/test_derived_metric_with_offset_window_and_granularity__plan0_optimized.sql b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Postgres/test_derived_metric_with_offset_window_and_granularity__plan0_optimized.sql index a18d3b4423..78614de1b0 100644 --- a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Postgres/test_derived_metric_with_offset_window_and_granularity__plan0_optimized.sql +++ b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Postgres/test_derived_metric_with_offset_window_and_granularity__plan0_optimized.sql @@ -3,7 +3,7 @@ SELECT metric_time__quarter , bookings - bookings_2_weeks_ago AS bookings_growth_2_weeks FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT COALESCE(subq_18.metric_time__quarter, subq_26.metric_time__quarter) AS metric_time__quarter , MAX(subq_18.bookings) AS bookings diff --git a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Postgres/test_derived_metric_with_offset_window_and_offset_to_grain__plan0.sql b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Postgres/test_derived_metric_with_offset_window_and_offset_to_grain__plan0.sql index 04623e0e7c..40cebadb48 100644 --- a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Postgres/test_derived_metric_with_offset_window_and_offset_to_grain__plan0.sql +++ b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Postgres/test_derived_metric_with_offset_window_and_offset_to_grain__plan0.sql @@ -3,7 +3,7 @@ SELECT subq_16.metric_time__day , month_start_bookings - bookings_1_month_ago AS bookings_month_start_compared_to_1_month_prior FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT COALESCE(subq_7.metric_time__day, subq_15.metric_time__day) AS metric_time__day , MAX(subq_7.month_start_bookings) AS month_start_bookings diff --git a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Postgres/test_derived_metric_with_offset_window_and_offset_to_grain__plan0_optimized.sql b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Postgres/test_derived_metric_with_offset_window_and_offset_to_grain__plan0_optimized.sql index dcb2c47e2f..e08e12fb22 100644 --- a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Postgres/test_derived_metric_with_offset_window_and_offset_to_grain__plan0_optimized.sql +++ b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Postgres/test_derived_metric_with_offset_window_and_offset_to_grain__plan0_optimized.sql @@ -3,7 +3,7 @@ SELECT metric_time__day , month_start_bookings - bookings_1_month_ago AS bookings_month_start_compared_to_1_month_prior FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT COALESCE(subq_24.metric_time__day, subq_32.metric_time__day) AS metric_time__day , MAX(subq_24.month_start_bookings) AS month_start_bookings diff --git a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Postgres/test_derived_metric_with_offset_window_and_offset_to_grain_and_granularity__plan0.sql b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Postgres/test_derived_metric_with_offset_window_and_offset_to_grain_and_granularity__plan0.sql index ba6d756625..a513ea1d08 100644 --- a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Postgres/test_derived_metric_with_offset_window_and_offset_to_grain_and_granularity__plan0.sql +++ b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Postgres/test_derived_metric_with_offset_window_and_offset_to_grain_and_granularity__plan0.sql @@ -3,7 +3,7 @@ SELECT subq_16.metric_time__year , month_start_bookings - bookings_1_month_ago AS bookings_month_start_compared_to_1_month_prior FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT COALESCE(subq_7.metric_time__year, subq_15.metric_time__year) AS metric_time__year , MAX(subq_7.month_start_bookings) AS month_start_bookings diff --git a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Postgres/test_derived_metric_with_offset_window_and_offset_to_grain_and_granularity__plan0_optimized.sql b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Postgres/test_derived_metric_with_offset_window_and_offset_to_grain_and_granularity__plan0_optimized.sql index 6aa850f36a..5698c00ff6 100644 --- a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Postgres/test_derived_metric_with_offset_window_and_offset_to_grain_and_granularity__plan0_optimized.sql +++ b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Postgres/test_derived_metric_with_offset_window_and_offset_to_grain_and_granularity__plan0_optimized.sql @@ -3,7 +3,7 @@ SELECT metric_time__year , month_start_bookings - bookings_1_month_ago AS bookings_month_start_compared_to_1_month_prior FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT COALESCE(subq_24.metric_time__year, subq_32.metric_time__year) AS metric_time__year , MAX(subq_24.month_start_bookings) AS month_start_bookings diff --git a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Postgres/test_derived_metric_with_offset_window_and_time_filter__plan0.sql b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Postgres/test_derived_metric_with_offset_window_and_time_filter__plan0.sql index ca971f521b..ea5c014973 100644 --- a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Postgres/test_derived_metric_with_offset_window_and_time_filter__plan0.sql +++ b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Postgres/test_derived_metric_with_offset_window_and_time_filter__plan0.sql @@ -3,7 +3,7 @@ SELECT subq_15.metric_time__day , bookings - bookings_2_weeks_ago AS bookings_growth_2_weeks FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT COALESCE(subq_5.metric_time__day, subq_14.metric_time__day) AS metric_time__day , MAX(subq_5.bookings) AS bookings diff --git a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Postgres/test_derived_metric_with_offset_window_and_time_filter__plan0_optimized.sql b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Postgres/test_derived_metric_with_offset_window_and_time_filter__plan0_optimized.sql index 6b0512fa07..48b1098a8a 100644 --- a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Postgres/test_derived_metric_with_offset_window_and_time_filter__plan0_optimized.sql +++ b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Postgres/test_derived_metric_with_offset_window_and_time_filter__plan0_optimized.sql @@ -3,7 +3,7 @@ SELECT metric_time__day , bookings - bookings_2_weeks_ago AS bookings_growth_2_weeks FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT COALESCE(subq_21.metric_time__day, subq_30.metric_time__day) AS metric_time__day , MAX(subq_21.bookings) AS bookings diff --git a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Postgres/test_nested_derived_metric__plan0.sql b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Postgres/test_nested_derived_metric__plan0.sql index 0e23b840ec..0538b786c8 100644 --- a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Postgres/test_nested_derived_metric__plan0.sql +++ b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Postgres/test_nested_derived_metric__plan0.sql @@ -3,7 +3,7 @@ SELECT subq_22.metric_time__day , non_referred + (instant * 1.0 / bookings) AS instant_plus_non_referred_bookings_pct FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT COALESCE(subq_11.metric_time__day, subq_16.metric_time__day, subq_21.metric_time__day) AS metric_time__day , MAX(subq_11.non_referred) AS non_referred @@ -15,7 +15,7 @@ FROM ( subq_10.metric_time__day , (bookings - ref_bookings) * 1.0 / bookings AS non_referred FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT COALESCE(subq_4.metric_time__day, subq_9.metric_time__day) AS metric_time__day , MAX(subq_4.ref_bookings) AS ref_bookings diff --git a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Postgres/test_nested_derived_metric__plan0_optimized.sql b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Postgres/test_nested_derived_metric__plan0_optimized.sql index 86124d9091..97c8543446 100644 --- a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Postgres/test_nested_derived_metric__plan0_optimized.sql +++ b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Postgres/test_nested_derived_metric__plan0_optimized.sql @@ -3,7 +3,7 @@ SELECT metric_time__day , non_referred + (instant * 1.0 / bookings) AS instant_plus_non_referred_bookings_pct FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT COALESCE(subq_34.metric_time__day, subq_39.metric_time__day, subq_44.metric_time__day) AS metric_time__day , MAX(subq_34.non_referred) AS non_referred @@ -15,7 +15,7 @@ FROM ( metric_time__day , (bookings - ref_bookings) * 1.0 / bookings AS non_referred FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT COALESCE(subq_27.metric_time__day, subq_32.metric_time__day) AS metric_time__day , MAX(subq_27.ref_bookings) AS ref_bookings diff --git a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Redshift/test_derived_metric__plan0.sql b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Redshift/test_derived_metric__plan0.sql index 5534156eee..1a68266ad7 100644 --- a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Redshift/test_derived_metric__plan0.sql +++ b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Redshift/test_derived_metric__plan0.sql @@ -3,7 +3,7 @@ SELECT subq_10.metric_time__day , (bookings - ref_bookings) * 1.0 / bookings AS non_referred_bookings_pct FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT COALESCE(subq_4.metric_time__day, subq_9.metric_time__day) AS metric_time__day , MAX(subq_4.ref_bookings) AS ref_bookings diff --git a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Redshift/test_derived_metric__plan0_optimized.sql b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Redshift/test_derived_metric__plan0_optimized.sql index f7d37d45b3..b3e757c0c6 100644 --- a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Redshift/test_derived_metric__plan0_optimized.sql +++ b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Redshift/test_derived_metric__plan0_optimized.sql @@ -3,7 +3,7 @@ SELECT metric_time__day , (bookings - ref_bookings) * 1.0 / bookings AS non_referred_bookings_pct FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT COALESCE(subq_15.metric_time__day, subq_20.metric_time__day) AS metric_time__day , MAX(subq_15.ref_bookings) AS ref_bookings diff --git a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Redshift/test_derived_metric_with_offset_to_grain__plan0.sql b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Redshift/test_derived_metric_with_offset_to_grain__plan0.sql index 1e21685347..0cc5fa298b 100644 --- a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Redshift/test_derived_metric_with_offset_to_grain__plan0.sql +++ b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Redshift/test_derived_metric_with_offset_to_grain__plan0.sql @@ -3,7 +3,7 @@ SELECT subq_13.metric_time__day , bookings - bookings_at_start_of_month AS bookings_growth_since_start_of_month FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT COALESCE(subq_4.metric_time__day, subq_12.metric_time__day) AS metric_time__day , MAX(subq_4.bookings) AS bookings diff --git a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Redshift/test_derived_metric_with_offset_to_grain__plan0_optimized.sql b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Redshift/test_derived_metric_with_offset_to_grain__plan0_optimized.sql index 6d1b17b416..3763e526e9 100644 --- a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Redshift/test_derived_metric_with_offset_to_grain__plan0_optimized.sql +++ b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Redshift/test_derived_metric_with_offset_to_grain__plan0_optimized.sql @@ -3,7 +3,7 @@ SELECT metric_time__day , bookings - bookings_at_start_of_month AS bookings_growth_since_start_of_month FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT COALESCE(subq_18.metric_time__day, subq_26.metric_time__day) AS metric_time__day , MAX(subq_18.bookings) AS bookings diff --git a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Redshift/test_derived_metric_with_offset_to_grain_and_granularity__plan0.sql b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Redshift/test_derived_metric_with_offset_to_grain_and_granularity__plan0.sql index 9aa59d72d5..84d5fdee4e 100644 --- a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Redshift/test_derived_metric_with_offset_to_grain_and_granularity__plan0.sql +++ b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Redshift/test_derived_metric_with_offset_to_grain_and_granularity__plan0.sql @@ -3,7 +3,7 @@ SELECT subq_13.metric_time__week , bookings - bookings_at_start_of_month AS bookings_growth_since_start_of_month FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT COALESCE(subq_4.metric_time__week, subq_12.metric_time__week) AS metric_time__week , MAX(subq_4.bookings) AS bookings diff --git a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Redshift/test_derived_metric_with_offset_to_grain_and_granularity__plan0_optimized.sql b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Redshift/test_derived_metric_with_offset_to_grain_and_granularity__plan0_optimized.sql index 5b18fcfa5f..0ac0bf4ca1 100644 --- a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Redshift/test_derived_metric_with_offset_to_grain_and_granularity__plan0_optimized.sql +++ b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Redshift/test_derived_metric_with_offset_to_grain_and_granularity__plan0_optimized.sql @@ -3,7 +3,7 @@ SELECT metric_time__week , bookings - bookings_at_start_of_month AS bookings_growth_since_start_of_month FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT COALESCE(subq_18.metric_time__week, subq_26.metric_time__week) AS metric_time__week , MAX(subq_18.bookings) AS bookings diff --git a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Redshift/test_derived_metric_with_offset_window__plan0.sql b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Redshift/test_derived_metric_with_offset_window__plan0.sql index e650c49b60..b5b5d18eae 100644 --- a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Redshift/test_derived_metric_with_offset_window__plan0.sql +++ b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Redshift/test_derived_metric_with_offset_window__plan0.sql @@ -3,7 +3,7 @@ SELECT subq_13.metric_time__day , bookings - bookings_2_weeks_ago AS bookings_growth_2_weeks FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT COALESCE(subq_4.metric_time__day, subq_12.metric_time__day) AS metric_time__day , MAX(subq_4.bookings) AS bookings diff --git a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Redshift/test_derived_metric_with_offset_window__plan0_optimized.sql b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Redshift/test_derived_metric_with_offset_window__plan0_optimized.sql index a01f18c65d..96cd7db009 100644 --- a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Redshift/test_derived_metric_with_offset_window__plan0_optimized.sql +++ b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Redshift/test_derived_metric_with_offset_window__plan0_optimized.sql @@ -3,7 +3,7 @@ SELECT metric_time__day , bookings - bookings_2_weeks_ago AS bookings_growth_2_weeks FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT COALESCE(subq_18.metric_time__day, subq_26.metric_time__day) AS metric_time__day , MAX(subq_18.bookings) AS bookings diff --git a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Redshift/test_derived_metric_with_offset_window_and_granularity__plan0.sql b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Redshift/test_derived_metric_with_offset_window_and_granularity__plan0.sql index 832b087043..1743646847 100644 --- a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Redshift/test_derived_metric_with_offset_window_and_granularity__plan0.sql +++ b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Redshift/test_derived_metric_with_offset_window_and_granularity__plan0.sql @@ -3,7 +3,7 @@ SELECT subq_13.metric_time__quarter , bookings - bookings_2_weeks_ago AS bookings_growth_2_weeks FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT COALESCE(subq_4.metric_time__quarter, subq_12.metric_time__quarter) AS metric_time__quarter , MAX(subq_4.bookings) AS bookings diff --git a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Redshift/test_derived_metric_with_offset_window_and_granularity__plan0_optimized.sql b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Redshift/test_derived_metric_with_offset_window_and_granularity__plan0_optimized.sql index 10ccf3e20b..919cfd1274 100644 --- a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Redshift/test_derived_metric_with_offset_window_and_granularity__plan0_optimized.sql +++ b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Redshift/test_derived_metric_with_offset_window_and_granularity__plan0_optimized.sql @@ -3,7 +3,7 @@ SELECT metric_time__quarter , bookings - bookings_2_weeks_ago AS bookings_growth_2_weeks FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT COALESCE(subq_18.metric_time__quarter, subq_26.metric_time__quarter) AS metric_time__quarter , MAX(subq_18.bookings) AS bookings diff --git a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Redshift/test_derived_metric_with_offset_window_and_offset_to_grain__plan0.sql b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Redshift/test_derived_metric_with_offset_window_and_offset_to_grain__plan0.sql index a704ef1eab..e1a77d6005 100644 --- a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Redshift/test_derived_metric_with_offset_window_and_offset_to_grain__plan0.sql +++ b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Redshift/test_derived_metric_with_offset_window_and_offset_to_grain__plan0.sql @@ -3,7 +3,7 @@ SELECT subq_16.metric_time__day , month_start_bookings - bookings_1_month_ago AS bookings_month_start_compared_to_1_month_prior FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT COALESCE(subq_7.metric_time__day, subq_15.metric_time__day) AS metric_time__day , MAX(subq_7.month_start_bookings) AS month_start_bookings diff --git a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Redshift/test_derived_metric_with_offset_window_and_offset_to_grain__plan0_optimized.sql b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Redshift/test_derived_metric_with_offset_window_and_offset_to_grain__plan0_optimized.sql index 91dc642b1b..33f71b11c3 100644 --- a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Redshift/test_derived_metric_with_offset_window_and_offset_to_grain__plan0_optimized.sql +++ b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Redshift/test_derived_metric_with_offset_window_and_offset_to_grain__plan0_optimized.sql @@ -3,7 +3,7 @@ SELECT metric_time__day , month_start_bookings - bookings_1_month_ago AS bookings_month_start_compared_to_1_month_prior FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT COALESCE(subq_24.metric_time__day, subq_32.metric_time__day) AS metric_time__day , MAX(subq_24.month_start_bookings) AS month_start_bookings diff --git a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Redshift/test_derived_metric_with_offset_window_and_offset_to_grain_and_granularity__plan0.sql b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Redshift/test_derived_metric_with_offset_window_and_offset_to_grain_and_granularity__plan0.sql index e34bb8173a..c5ffb5ad46 100644 --- a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Redshift/test_derived_metric_with_offset_window_and_offset_to_grain_and_granularity__plan0.sql +++ b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Redshift/test_derived_metric_with_offset_window_and_offset_to_grain_and_granularity__plan0.sql @@ -3,7 +3,7 @@ SELECT subq_16.metric_time__year , month_start_bookings - bookings_1_month_ago AS bookings_month_start_compared_to_1_month_prior FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT COALESCE(subq_7.metric_time__year, subq_15.metric_time__year) AS metric_time__year , MAX(subq_7.month_start_bookings) AS month_start_bookings diff --git a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Redshift/test_derived_metric_with_offset_window_and_offset_to_grain_and_granularity__plan0_optimized.sql b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Redshift/test_derived_metric_with_offset_window_and_offset_to_grain_and_granularity__plan0_optimized.sql index f06da5a3a2..0d96f7f3f0 100644 --- a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Redshift/test_derived_metric_with_offset_window_and_offset_to_grain_and_granularity__plan0_optimized.sql +++ b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Redshift/test_derived_metric_with_offset_window_and_offset_to_grain_and_granularity__plan0_optimized.sql @@ -3,7 +3,7 @@ SELECT metric_time__year , month_start_bookings - bookings_1_month_ago AS bookings_month_start_compared_to_1_month_prior FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT COALESCE(subq_24.metric_time__year, subq_32.metric_time__year) AS metric_time__year , MAX(subq_24.month_start_bookings) AS month_start_bookings diff --git a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Redshift/test_derived_metric_with_offset_window_and_time_filter__plan0.sql b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Redshift/test_derived_metric_with_offset_window_and_time_filter__plan0.sql index 8c7fbcb4a5..3b05a07881 100644 --- a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Redshift/test_derived_metric_with_offset_window_and_time_filter__plan0.sql +++ b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Redshift/test_derived_metric_with_offset_window_and_time_filter__plan0.sql @@ -3,7 +3,7 @@ SELECT subq_15.metric_time__day , bookings - bookings_2_weeks_ago AS bookings_growth_2_weeks FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT COALESCE(subq_5.metric_time__day, subq_14.metric_time__day) AS metric_time__day , MAX(subq_5.bookings) AS bookings diff --git a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Redshift/test_derived_metric_with_offset_window_and_time_filter__plan0_optimized.sql b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Redshift/test_derived_metric_with_offset_window_and_time_filter__plan0_optimized.sql index 032f104a2d..c2e865bddc 100644 --- a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Redshift/test_derived_metric_with_offset_window_and_time_filter__plan0_optimized.sql +++ b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Redshift/test_derived_metric_with_offset_window_and_time_filter__plan0_optimized.sql @@ -3,7 +3,7 @@ SELECT metric_time__day , bookings - bookings_2_weeks_ago AS bookings_growth_2_weeks FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT COALESCE(subq_21.metric_time__day, subq_30.metric_time__day) AS metric_time__day , MAX(subq_21.bookings) AS bookings diff --git a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Redshift/test_nested_derived_metric__plan0.sql b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Redshift/test_nested_derived_metric__plan0.sql index a91d132652..2fbc67808b 100644 --- a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Redshift/test_nested_derived_metric__plan0.sql +++ b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Redshift/test_nested_derived_metric__plan0.sql @@ -3,7 +3,7 @@ SELECT subq_22.metric_time__day , non_referred + (instant * 1.0 / bookings) AS instant_plus_non_referred_bookings_pct FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT COALESCE(subq_11.metric_time__day, subq_16.metric_time__day, subq_21.metric_time__day) AS metric_time__day , MAX(subq_11.non_referred) AS non_referred @@ -15,7 +15,7 @@ FROM ( subq_10.metric_time__day , (bookings - ref_bookings) * 1.0 / bookings AS non_referred FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT COALESCE(subq_4.metric_time__day, subq_9.metric_time__day) AS metric_time__day , MAX(subq_4.ref_bookings) AS ref_bookings diff --git a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Redshift/test_nested_derived_metric__plan0_optimized.sql b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Redshift/test_nested_derived_metric__plan0_optimized.sql index 86124d9091..97c8543446 100644 --- a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Redshift/test_nested_derived_metric__plan0_optimized.sql +++ b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Redshift/test_nested_derived_metric__plan0_optimized.sql @@ -3,7 +3,7 @@ SELECT metric_time__day , non_referred + (instant * 1.0 / bookings) AS instant_plus_non_referred_bookings_pct FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT COALESCE(subq_34.metric_time__day, subq_39.metric_time__day, subq_44.metric_time__day) AS metric_time__day , MAX(subq_34.non_referred) AS non_referred @@ -15,7 +15,7 @@ FROM ( metric_time__day , (bookings - ref_bookings) * 1.0 / bookings AS non_referred FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT COALESCE(subq_27.metric_time__day, subq_32.metric_time__day) AS metric_time__day , MAX(subq_27.ref_bookings) AS ref_bookings diff --git a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Snowflake/test_derived_metric__plan0.sql b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Snowflake/test_derived_metric__plan0.sql index 210db4a0aa..874faf6ef3 100644 --- a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Snowflake/test_derived_metric__plan0.sql +++ b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Snowflake/test_derived_metric__plan0.sql @@ -3,7 +3,7 @@ SELECT subq_10.metric_time__day , (bookings - ref_bookings) * 1.0 / bookings AS non_referred_bookings_pct FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT COALESCE(subq_4.metric_time__day, subq_9.metric_time__day) AS metric_time__day , MAX(subq_4.ref_bookings) AS ref_bookings diff --git a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Snowflake/test_derived_metric__plan0_optimized.sql b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Snowflake/test_derived_metric__plan0_optimized.sql index f7d37d45b3..b3e757c0c6 100644 --- a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Snowflake/test_derived_metric__plan0_optimized.sql +++ b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Snowflake/test_derived_metric__plan0_optimized.sql @@ -3,7 +3,7 @@ SELECT metric_time__day , (bookings - ref_bookings) * 1.0 / bookings AS non_referred_bookings_pct FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT COALESCE(subq_15.metric_time__day, subq_20.metric_time__day) AS metric_time__day , MAX(subq_15.ref_bookings) AS ref_bookings diff --git a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Snowflake/test_derived_metric_with_offset_to_grain__plan0.sql b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Snowflake/test_derived_metric_with_offset_to_grain__plan0.sql index 35a7062093..2fa4788e24 100644 --- a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Snowflake/test_derived_metric_with_offset_to_grain__plan0.sql +++ b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Snowflake/test_derived_metric_with_offset_to_grain__plan0.sql @@ -3,7 +3,7 @@ SELECT subq_13.metric_time__day , bookings - bookings_at_start_of_month AS bookings_growth_since_start_of_month FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT COALESCE(subq_4.metric_time__day, subq_12.metric_time__day) AS metric_time__day , MAX(subq_4.bookings) AS bookings diff --git a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Snowflake/test_derived_metric_with_offset_to_grain__plan0_optimized.sql b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Snowflake/test_derived_metric_with_offset_to_grain__plan0_optimized.sql index 6d1b17b416..3763e526e9 100644 --- a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Snowflake/test_derived_metric_with_offset_to_grain__plan0_optimized.sql +++ b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Snowflake/test_derived_metric_with_offset_to_grain__plan0_optimized.sql @@ -3,7 +3,7 @@ SELECT metric_time__day , bookings - bookings_at_start_of_month AS bookings_growth_since_start_of_month FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT COALESCE(subq_18.metric_time__day, subq_26.metric_time__day) AS metric_time__day , MAX(subq_18.bookings) AS bookings diff --git a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Snowflake/test_derived_metric_with_offset_to_grain_and_granularity__plan0.sql b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Snowflake/test_derived_metric_with_offset_to_grain_and_granularity__plan0.sql index 56d35de19c..1f5faac889 100644 --- a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Snowflake/test_derived_metric_with_offset_to_grain_and_granularity__plan0.sql +++ b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Snowflake/test_derived_metric_with_offset_to_grain_and_granularity__plan0.sql @@ -3,7 +3,7 @@ SELECT subq_13.metric_time__week , bookings - bookings_at_start_of_month AS bookings_growth_since_start_of_month FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT COALESCE(subq_4.metric_time__week, subq_12.metric_time__week) AS metric_time__week , MAX(subq_4.bookings) AS bookings diff --git a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Snowflake/test_derived_metric_with_offset_to_grain_and_granularity__plan0_optimized.sql b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Snowflake/test_derived_metric_with_offset_to_grain_and_granularity__plan0_optimized.sql index 5b18fcfa5f..0ac0bf4ca1 100644 --- a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Snowflake/test_derived_metric_with_offset_to_grain_and_granularity__plan0_optimized.sql +++ b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Snowflake/test_derived_metric_with_offset_to_grain_and_granularity__plan0_optimized.sql @@ -3,7 +3,7 @@ SELECT metric_time__week , bookings - bookings_at_start_of_month AS bookings_growth_since_start_of_month FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT COALESCE(subq_18.metric_time__week, subq_26.metric_time__week) AS metric_time__week , MAX(subq_18.bookings) AS bookings diff --git a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Snowflake/test_derived_metric_with_offset_window__plan0.sql b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Snowflake/test_derived_metric_with_offset_window__plan0.sql index b79d2fadce..33f0dae6b5 100644 --- a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Snowflake/test_derived_metric_with_offset_window__plan0.sql +++ b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Snowflake/test_derived_metric_with_offset_window__plan0.sql @@ -3,7 +3,7 @@ SELECT subq_13.metric_time__day , bookings - bookings_2_weeks_ago AS bookings_growth_2_weeks FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT COALESCE(subq_4.metric_time__day, subq_12.metric_time__day) AS metric_time__day , MAX(subq_4.bookings) AS bookings diff --git a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Snowflake/test_derived_metric_with_offset_window__plan0_optimized.sql b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Snowflake/test_derived_metric_with_offset_window__plan0_optimized.sql index a01f18c65d..96cd7db009 100644 --- a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Snowflake/test_derived_metric_with_offset_window__plan0_optimized.sql +++ b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Snowflake/test_derived_metric_with_offset_window__plan0_optimized.sql @@ -3,7 +3,7 @@ SELECT metric_time__day , bookings - bookings_2_weeks_ago AS bookings_growth_2_weeks FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT COALESCE(subq_18.metric_time__day, subq_26.metric_time__day) AS metric_time__day , MAX(subq_18.bookings) AS bookings diff --git a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Snowflake/test_derived_metric_with_offset_window_and_granularity__plan0.sql b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Snowflake/test_derived_metric_with_offset_window_and_granularity__plan0.sql index 5e159a592a..9d7dceb8c5 100644 --- a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Snowflake/test_derived_metric_with_offset_window_and_granularity__plan0.sql +++ b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Snowflake/test_derived_metric_with_offset_window_and_granularity__plan0.sql @@ -3,7 +3,7 @@ SELECT subq_13.metric_time__quarter , bookings - bookings_2_weeks_ago AS bookings_growth_2_weeks FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT COALESCE(subq_4.metric_time__quarter, subq_12.metric_time__quarter) AS metric_time__quarter , MAX(subq_4.bookings) AS bookings diff --git a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Snowflake/test_derived_metric_with_offset_window_and_granularity__plan0_optimized.sql b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Snowflake/test_derived_metric_with_offset_window_and_granularity__plan0_optimized.sql index 10ccf3e20b..919cfd1274 100644 --- a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Snowflake/test_derived_metric_with_offset_window_and_granularity__plan0_optimized.sql +++ b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Snowflake/test_derived_metric_with_offset_window_and_granularity__plan0_optimized.sql @@ -3,7 +3,7 @@ SELECT metric_time__quarter , bookings - bookings_2_weeks_ago AS bookings_growth_2_weeks FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT COALESCE(subq_18.metric_time__quarter, subq_26.metric_time__quarter) AS metric_time__quarter , MAX(subq_18.bookings) AS bookings diff --git a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Snowflake/test_derived_metric_with_offset_window_and_offset_to_grain__plan0.sql b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Snowflake/test_derived_metric_with_offset_window_and_offset_to_grain__plan0.sql index d9b9bf4244..f4a5185146 100644 --- a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Snowflake/test_derived_metric_with_offset_window_and_offset_to_grain__plan0.sql +++ b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Snowflake/test_derived_metric_with_offset_window_and_offset_to_grain__plan0.sql @@ -3,7 +3,7 @@ SELECT subq_16.metric_time__day , month_start_bookings - bookings_1_month_ago AS bookings_month_start_compared_to_1_month_prior FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT COALESCE(subq_7.metric_time__day, subq_15.metric_time__day) AS metric_time__day , MAX(subq_7.month_start_bookings) AS month_start_bookings diff --git a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Snowflake/test_derived_metric_with_offset_window_and_offset_to_grain__plan0_optimized.sql b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Snowflake/test_derived_metric_with_offset_window_and_offset_to_grain__plan0_optimized.sql index 91dc642b1b..33f71b11c3 100644 --- a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Snowflake/test_derived_metric_with_offset_window_and_offset_to_grain__plan0_optimized.sql +++ b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Snowflake/test_derived_metric_with_offset_window_and_offset_to_grain__plan0_optimized.sql @@ -3,7 +3,7 @@ SELECT metric_time__day , month_start_bookings - bookings_1_month_ago AS bookings_month_start_compared_to_1_month_prior FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT COALESCE(subq_24.metric_time__day, subq_32.metric_time__day) AS metric_time__day , MAX(subq_24.month_start_bookings) AS month_start_bookings diff --git a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Snowflake/test_derived_metric_with_offset_window_and_offset_to_grain_and_granularity__plan0.sql b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Snowflake/test_derived_metric_with_offset_window_and_offset_to_grain_and_granularity__plan0.sql index 7416cc58f2..dee0333eac 100644 --- a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Snowflake/test_derived_metric_with_offset_window_and_offset_to_grain_and_granularity__plan0.sql +++ b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Snowflake/test_derived_metric_with_offset_window_and_offset_to_grain_and_granularity__plan0.sql @@ -3,7 +3,7 @@ SELECT subq_16.metric_time__year , month_start_bookings - bookings_1_month_ago AS bookings_month_start_compared_to_1_month_prior FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT COALESCE(subq_7.metric_time__year, subq_15.metric_time__year) AS metric_time__year , MAX(subq_7.month_start_bookings) AS month_start_bookings diff --git a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Snowflake/test_derived_metric_with_offset_window_and_offset_to_grain_and_granularity__plan0_optimized.sql b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Snowflake/test_derived_metric_with_offset_window_and_offset_to_grain_and_granularity__plan0_optimized.sql index f06da5a3a2..0d96f7f3f0 100644 --- a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Snowflake/test_derived_metric_with_offset_window_and_offset_to_grain_and_granularity__plan0_optimized.sql +++ b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Snowflake/test_derived_metric_with_offset_window_and_offset_to_grain_and_granularity__plan0_optimized.sql @@ -3,7 +3,7 @@ SELECT metric_time__year , month_start_bookings - bookings_1_month_ago AS bookings_month_start_compared_to_1_month_prior FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT COALESCE(subq_24.metric_time__year, subq_32.metric_time__year) AS metric_time__year , MAX(subq_24.month_start_bookings) AS month_start_bookings diff --git a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Snowflake/test_derived_metric_with_offset_window_and_time_filter__plan0.sql b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Snowflake/test_derived_metric_with_offset_window_and_time_filter__plan0.sql index 75d9444cd6..f54d065776 100644 --- a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Snowflake/test_derived_metric_with_offset_window_and_time_filter__plan0.sql +++ b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Snowflake/test_derived_metric_with_offset_window_and_time_filter__plan0.sql @@ -3,7 +3,7 @@ SELECT subq_15.metric_time__day , bookings - bookings_2_weeks_ago AS bookings_growth_2_weeks FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT COALESCE(subq_5.metric_time__day, subq_14.metric_time__day) AS metric_time__day , MAX(subq_5.bookings) AS bookings diff --git a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Snowflake/test_derived_metric_with_offset_window_and_time_filter__plan0_optimized.sql b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Snowflake/test_derived_metric_with_offset_window_and_time_filter__plan0_optimized.sql index 032f104a2d..c2e865bddc 100644 --- a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Snowflake/test_derived_metric_with_offset_window_and_time_filter__plan0_optimized.sql +++ b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Snowflake/test_derived_metric_with_offset_window_and_time_filter__plan0_optimized.sql @@ -3,7 +3,7 @@ SELECT metric_time__day , bookings - bookings_2_weeks_ago AS bookings_growth_2_weeks FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT COALESCE(subq_21.metric_time__day, subq_30.metric_time__day) AS metric_time__day , MAX(subq_21.bookings) AS bookings diff --git a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Snowflake/test_nested_derived_metric__plan0.sql b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Snowflake/test_nested_derived_metric__plan0.sql index 307275d7d4..fcc8376c0d 100644 --- a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Snowflake/test_nested_derived_metric__plan0.sql +++ b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Snowflake/test_nested_derived_metric__plan0.sql @@ -3,7 +3,7 @@ SELECT subq_22.metric_time__day , non_referred + (instant * 1.0 / bookings) AS instant_plus_non_referred_bookings_pct FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT COALESCE(subq_11.metric_time__day, subq_16.metric_time__day, subq_21.metric_time__day) AS metric_time__day , MAX(subq_11.non_referred) AS non_referred @@ -15,7 +15,7 @@ FROM ( subq_10.metric_time__day , (bookings - ref_bookings) * 1.0 / bookings AS non_referred FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT COALESCE(subq_4.metric_time__day, subq_9.metric_time__day) AS metric_time__day , MAX(subq_4.ref_bookings) AS ref_bookings diff --git a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Snowflake/test_nested_derived_metric__plan0_optimized.sql b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Snowflake/test_nested_derived_metric__plan0_optimized.sql index 86124d9091..97c8543446 100644 --- a/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Snowflake/test_nested_derived_metric__plan0_optimized.sql +++ b/metricflow/test/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Snowflake/test_nested_derived_metric__plan0_optimized.sql @@ -3,7 +3,7 @@ SELECT metric_time__day , non_referred + (instant * 1.0 / bookings) AS instant_plus_non_referred_bookings_pct FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT COALESCE(subq_34.metric_time__day, subq_39.metric_time__day, subq_44.metric_time__day) AS metric_time__day , MAX(subq_34.non_referred) AS non_referred @@ -15,7 +15,7 @@ FROM ( metric_time__day , (bookings - ref_bookings) * 1.0 / bookings AS non_referred FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT COALESCE(subq_27.metric_time__day, subq_32.metric_time__day) AS metric_time__day , MAX(subq_27.ref_bookings) AS ref_bookings diff --git a/metricflow/test/snapshots/test_fill_nulls_with_rendering.py/SqlQueryPlan/BigQuery/test_derived_fill_nulls_for_one_input_metric__plan0.sql b/metricflow/test/snapshots/test_fill_nulls_with_rendering.py/SqlQueryPlan/BigQuery/test_derived_fill_nulls_for_one_input_metric__plan0.sql index 21c9bba162..06d5eb037d 100644 --- a/metricflow/test/snapshots/test_fill_nulls_with_rendering.py/SqlQueryPlan/BigQuery/test_derived_fill_nulls_for_one_input_metric__plan0.sql +++ b/metricflow/test/snapshots/test_fill_nulls_with_rendering.py/SqlQueryPlan/BigQuery/test_derived_fill_nulls_for_one_input_metric__plan0.sql @@ -3,7 +3,7 @@ SELECT subq_16.metric_time__day , bookings_fill_nulls_with_0 - bookings_2_weeks_ago AS bookings_growth_2_weeks_fill_nulls_with_0_for_non_offset FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT COALESCE(subq_7.metric_time__day, subq_15.metric_time__day) AS metric_time__day , COALESCE(MAX(subq_7.bookings_fill_nulls_with_0), 0) AS bookings_fill_nulls_with_0 diff --git a/metricflow/test/snapshots/test_fill_nulls_with_rendering.py/SqlQueryPlan/BigQuery/test_derived_fill_nulls_for_one_input_metric__plan0_optimized.sql b/metricflow/test/snapshots/test_fill_nulls_with_rendering.py/SqlQueryPlan/BigQuery/test_derived_fill_nulls_for_one_input_metric__plan0_optimized.sql index fe0a20f6f9..7b71408995 100644 --- a/metricflow/test/snapshots/test_fill_nulls_with_rendering.py/SqlQueryPlan/BigQuery/test_derived_fill_nulls_for_one_input_metric__plan0_optimized.sql +++ b/metricflow/test/snapshots/test_fill_nulls_with_rendering.py/SqlQueryPlan/BigQuery/test_derived_fill_nulls_for_one_input_metric__plan0_optimized.sql @@ -3,7 +3,7 @@ SELECT metric_time__day , bookings_fill_nulls_with_0 - bookings_2_weeks_ago AS bookings_growth_2_weeks_fill_nulls_with_0_for_non_offset FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT COALESCE(subq_24.metric_time__day, subq_32.metric_time__day) AS metric_time__day , COALESCE(MAX(subq_24.bookings_fill_nulls_with_0), 0) AS bookings_fill_nulls_with_0 diff --git a/metricflow/test/snapshots/test_fill_nulls_with_rendering.py/SqlQueryPlan/Databricks/test_derived_fill_nulls_for_one_input_metric__plan0.sql b/metricflow/test/snapshots/test_fill_nulls_with_rendering.py/SqlQueryPlan/Databricks/test_derived_fill_nulls_for_one_input_metric__plan0.sql index 2e3efcf3fc..e6bcbfefe8 100644 --- a/metricflow/test/snapshots/test_fill_nulls_with_rendering.py/SqlQueryPlan/Databricks/test_derived_fill_nulls_for_one_input_metric__plan0.sql +++ b/metricflow/test/snapshots/test_fill_nulls_with_rendering.py/SqlQueryPlan/Databricks/test_derived_fill_nulls_for_one_input_metric__plan0.sql @@ -3,7 +3,7 @@ SELECT subq_16.metric_time__day , bookings_fill_nulls_with_0 - bookings_2_weeks_ago AS bookings_growth_2_weeks_fill_nulls_with_0_for_non_offset FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT COALESCE(subq_7.metric_time__day, subq_15.metric_time__day) AS metric_time__day , COALESCE(MAX(subq_7.bookings_fill_nulls_with_0), 0) AS bookings_fill_nulls_with_0 diff --git a/metricflow/test/snapshots/test_fill_nulls_with_rendering.py/SqlQueryPlan/Databricks/test_derived_fill_nulls_for_one_input_metric__plan0_optimized.sql b/metricflow/test/snapshots/test_fill_nulls_with_rendering.py/SqlQueryPlan/Databricks/test_derived_fill_nulls_for_one_input_metric__plan0_optimized.sql index 9381558d74..0bf4464932 100644 --- a/metricflow/test/snapshots/test_fill_nulls_with_rendering.py/SqlQueryPlan/Databricks/test_derived_fill_nulls_for_one_input_metric__plan0_optimized.sql +++ b/metricflow/test/snapshots/test_fill_nulls_with_rendering.py/SqlQueryPlan/Databricks/test_derived_fill_nulls_for_one_input_metric__plan0_optimized.sql @@ -3,7 +3,7 @@ SELECT metric_time__day , bookings_fill_nulls_with_0 - bookings_2_weeks_ago AS bookings_growth_2_weeks_fill_nulls_with_0_for_non_offset FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT COALESCE(subq_24.metric_time__day, subq_32.metric_time__day) AS metric_time__day , COALESCE(MAX(subq_24.bookings_fill_nulls_with_0), 0) AS bookings_fill_nulls_with_0 diff --git a/metricflow/test/snapshots/test_fill_nulls_with_rendering.py/SqlQueryPlan/DuckDB/test_derived_fill_nulls_for_one_input_metric__plan0.sql b/metricflow/test/snapshots/test_fill_nulls_with_rendering.py/SqlQueryPlan/DuckDB/test_derived_fill_nulls_for_one_input_metric__plan0.sql index 5c9c395244..0d75fd252c 100644 --- a/metricflow/test/snapshots/test_fill_nulls_with_rendering.py/SqlQueryPlan/DuckDB/test_derived_fill_nulls_for_one_input_metric__plan0.sql +++ b/metricflow/test/snapshots/test_fill_nulls_with_rendering.py/SqlQueryPlan/DuckDB/test_derived_fill_nulls_for_one_input_metric__plan0.sql @@ -3,7 +3,7 @@ SELECT subq_16.metric_time__day , bookings_fill_nulls_with_0 - bookings_2_weeks_ago AS bookings_growth_2_weeks_fill_nulls_with_0_for_non_offset FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT COALESCE(subq_7.metric_time__day, subq_15.metric_time__day) AS metric_time__day , COALESCE(MAX(subq_7.bookings_fill_nulls_with_0), 0) AS bookings_fill_nulls_with_0 diff --git a/metricflow/test/snapshots/test_fill_nulls_with_rendering.py/SqlQueryPlan/DuckDB/test_derived_fill_nulls_for_one_input_metric__plan0_optimized.sql b/metricflow/test/snapshots/test_fill_nulls_with_rendering.py/SqlQueryPlan/DuckDB/test_derived_fill_nulls_for_one_input_metric__plan0_optimized.sql index a47a257565..09ab0e3780 100644 --- a/metricflow/test/snapshots/test_fill_nulls_with_rendering.py/SqlQueryPlan/DuckDB/test_derived_fill_nulls_for_one_input_metric__plan0_optimized.sql +++ b/metricflow/test/snapshots/test_fill_nulls_with_rendering.py/SqlQueryPlan/DuckDB/test_derived_fill_nulls_for_one_input_metric__plan0_optimized.sql @@ -3,7 +3,7 @@ SELECT metric_time__day , bookings_fill_nulls_with_0 - bookings_2_weeks_ago AS bookings_growth_2_weeks_fill_nulls_with_0_for_non_offset FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT COALESCE(subq_24.metric_time__day, subq_32.metric_time__day) AS metric_time__day , COALESCE(MAX(subq_24.bookings_fill_nulls_with_0), 0) AS bookings_fill_nulls_with_0 diff --git a/metricflow/test/snapshots/test_fill_nulls_with_rendering.py/SqlQueryPlan/Postgres/test_derived_fill_nulls_for_one_input_metric__plan0.sql b/metricflow/test/snapshots/test_fill_nulls_with_rendering.py/SqlQueryPlan/Postgres/test_derived_fill_nulls_for_one_input_metric__plan0.sql index 039bc21b06..c56057b0d9 100644 --- a/metricflow/test/snapshots/test_fill_nulls_with_rendering.py/SqlQueryPlan/Postgres/test_derived_fill_nulls_for_one_input_metric__plan0.sql +++ b/metricflow/test/snapshots/test_fill_nulls_with_rendering.py/SqlQueryPlan/Postgres/test_derived_fill_nulls_for_one_input_metric__plan0.sql @@ -3,7 +3,7 @@ SELECT subq_16.metric_time__day , bookings_fill_nulls_with_0 - bookings_2_weeks_ago AS bookings_growth_2_weeks_fill_nulls_with_0_for_non_offset FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT COALESCE(subq_7.metric_time__day, subq_15.metric_time__day) AS metric_time__day , COALESCE(MAX(subq_7.bookings_fill_nulls_with_0), 0) AS bookings_fill_nulls_with_0 diff --git a/metricflow/test/snapshots/test_fill_nulls_with_rendering.py/SqlQueryPlan/Postgres/test_derived_fill_nulls_for_one_input_metric__plan0_optimized.sql b/metricflow/test/snapshots/test_fill_nulls_with_rendering.py/SqlQueryPlan/Postgres/test_derived_fill_nulls_for_one_input_metric__plan0_optimized.sql index 4229113c37..d76379936a 100644 --- a/metricflow/test/snapshots/test_fill_nulls_with_rendering.py/SqlQueryPlan/Postgres/test_derived_fill_nulls_for_one_input_metric__plan0_optimized.sql +++ b/metricflow/test/snapshots/test_fill_nulls_with_rendering.py/SqlQueryPlan/Postgres/test_derived_fill_nulls_for_one_input_metric__plan0_optimized.sql @@ -3,7 +3,7 @@ SELECT metric_time__day , bookings_fill_nulls_with_0 - bookings_2_weeks_ago AS bookings_growth_2_weeks_fill_nulls_with_0_for_non_offset FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT COALESCE(subq_24.metric_time__day, subq_32.metric_time__day) AS metric_time__day , COALESCE(MAX(subq_24.bookings_fill_nulls_with_0), 0) AS bookings_fill_nulls_with_0 diff --git a/metricflow/test/snapshots/test_fill_nulls_with_rendering.py/SqlQueryPlan/Redshift/test_derived_fill_nulls_for_one_input_metric__plan0.sql b/metricflow/test/snapshots/test_fill_nulls_with_rendering.py/SqlQueryPlan/Redshift/test_derived_fill_nulls_for_one_input_metric__plan0.sql index 8f3ebe8e9e..8d0505812c 100644 --- a/metricflow/test/snapshots/test_fill_nulls_with_rendering.py/SqlQueryPlan/Redshift/test_derived_fill_nulls_for_one_input_metric__plan0.sql +++ b/metricflow/test/snapshots/test_fill_nulls_with_rendering.py/SqlQueryPlan/Redshift/test_derived_fill_nulls_for_one_input_metric__plan0.sql @@ -3,7 +3,7 @@ SELECT subq_16.metric_time__day , bookings_fill_nulls_with_0 - bookings_2_weeks_ago AS bookings_growth_2_weeks_fill_nulls_with_0_for_non_offset FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT COALESCE(subq_7.metric_time__day, subq_15.metric_time__day) AS metric_time__day , COALESCE(MAX(subq_7.bookings_fill_nulls_with_0), 0) AS bookings_fill_nulls_with_0 diff --git a/metricflow/test/snapshots/test_fill_nulls_with_rendering.py/SqlQueryPlan/Redshift/test_derived_fill_nulls_for_one_input_metric__plan0_optimized.sql b/metricflow/test/snapshots/test_fill_nulls_with_rendering.py/SqlQueryPlan/Redshift/test_derived_fill_nulls_for_one_input_metric__plan0_optimized.sql index 9381558d74..0bf4464932 100644 --- a/metricflow/test/snapshots/test_fill_nulls_with_rendering.py/SqlQueryPlan/Redshift/test_derived_fill_nulls_for_one_input_metric__plan0_optimized.sql +++ b/metricflow/test/snapshots/test_fill_nulls_with_rendering.py/SqlQueryPlan/Redshift/test_derived_fill_nulls_for_one_input_metric__plan0_optimized.sql @@ -3,7 +3,7 @@ SELECT metric_time__day , bookings_fill_nulls_with_0 - bookings_2_weeks_ago AS bookings_growth_2_weeks_fill_nulls_with_0_for_non_offset FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT COALESCE(subq_24.metric_time__day, subq_32.metric_time__day) AS metric_time__day , COALESCE(MAX(subq_24.bookings_fill_nulls_with_0), 0) AS bookings_fill_nulls_with_0 diff --git a/metricflow/test/snapshots/test_fill_nulls_with_rendering.py/SqlQueryPlan/Snowflake/test_derived_fill_nulls_for_one_input_metric__plan0.sql b/metricflow/test/snapshots/test_fill_nulls_with_rendering.py/SqlQueryPlan/Snowflake/test_derived_fill_nulls_for_one_input_metric__plan0.sql index 77072714d4..7de1df40ee 100644 --- a/metricflow/test/snapshots/test_fill_nulls_with_rendering.py/SqlQueryPlan/Snowflake/test_derived_fill_nulls_for_one_input_metric__plan0.sql +++ b/metricflow/test/snapshots/test_fill_nulls_with_rendering.py/SqlQueryPlan/Snowflake/test_derived_fill_nulls_for_one_input_metric__plan0.sql @@ -3,7 +3,7 @@ SELECT subq_16.metric_time__day , bookings_fill_nulls_with_0 - bookings_2_weeks_ago AS bookings_growth_2_weeks_fill_nulls_with_0_for_non_offset FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT COALESCE(subq_7.metric_time__day, subq_15.metric_time__day) AS metric_time__day , COALESCE(MAX(subq_7.bookings_fill_nulls_with_0), 0) AS bookings_fill_nulls_with_0 diff --git a/metricflow/test/snapshots/test_fill_nulls_with_rendering.py/SqlQueryPlan/Snowflake/test_derived_fill_nulls_for_one_input_metric__plan0_optimized.sql b/metricflow/test/snapshots/test_fill_nulls_with_rendering.py/SqlQueryPlan/Snowflake/test_derived_fill_nulls_for_one_input_metric__plan0_optimized.sql index 9381558d74..0bf4464932 100644 --- a/metricflow/test/snapshots/test_fill_nulls_with_rendering.py/SqlQueryPlan/Snowflake/test_derived_fill_nulls_for_one_input_metric__plan0_optimized.sql +++ b/metricflow/test/snapshots/test_fill_nulls_with_rendering.py/SqlQueryPlan/Snowflake/test_derived_fill_nulls_for_one_input_metric__plan0_optimized.sql @@ -3,7 +3,7 @@ SELECT metric_time__day , bookings_fill_nulls_with_0 - bookings_2_weeks_ago AS bookings_growth_2_weeks_fill_nulls_with_0_for_non_offset FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT COALESCE(subq_24.metric_time__day, subq_32.metric_time__day) AS metric_time__day , COALESCE(MAX(subq_24.bookings_fill_nulls_with_0), 0) AS bookings_fill_nulls_with_0 diff --git a/metricflow/test/snapshots/test_granularity_date_part_rendering.py/SqlQueryPlan/BigQuery/test_offset_window_with_date_part__plan0.sql b/metricflow/test/snapshots/test_granularity_date_part_rendering.py/SqlQueryPlan/BigQuery/test_offset_window_with_date_part__plan0.sql index 4a09d351d9..a4d3c4b98e 100644 --- a/metricflow/test/snapshots/test_granularity_date_part_rendering.py/SqlQueryPlan/BigQuery/test_offset_window_with_date_part__plan0.sql +++ b/metricflow/test/snapshots/test_granularity_date_part_rendering.py/SqlQueryPlan/BigQuery/test_offset_window_with_date_part__plan0.sql @@ -3,7 +3,7 @@ SELECT subq_13.metric_time__extract_dow , bookings - bookings_2_weeks_ago AS bookings_growth_2_weeks FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT COALESCE(subq_4.metric_time__extract_dow, subq_12.metric_time__extract_dow) AS metric_time__extract_dow , MAX(subq_4.bookings) AS bookings diff --git a/metricflow/test/snapshots/test_granularity_date_part_rendering.py/SqlQueryPlan/BigQuery/test_offset_window_with_date_part__plan0_optimized.sql b/metricflow/test/snapshots/test_granularity_date_part_rendering.py/SqlQueryPlan/BigQuery/test_offset_window_with_date_part__plan0_optimized.sql index 1750a456ed..f3f322b3a0 100644 --- a/metricflow/test/snapshots/test_granularity_date_part_rendering.py/SqlQueryPlan/BigQuery/test_offset_window_with_date_part__plan0_optimized.sql +++ b/metricflow/test/snapshots/test_granularity_date_part_rendering.py/SqlQueryPlan/BigQuery/test_offset_window_with_date_part__plan0_optimized.sql @@ -3,7 +3,7 @@ SELECT metric_time__extract_dow , bookings - bookings_2_weeks_ago AS bookings_growth_2_weeks FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT COALESCE(subq_18.metric_time__extract_dow, subq_26.metric_time__extract_dow) AS metric_time__extract_dow , MAX(subq_18.bookings) AS bookings diff --git a/metricflow/test/snapshots/test_granularity_date_part_rendering.py/SqlQueryPlan/Databricks/test_offset_window_with_date_part__plan0.sql b/metricflow/test/snapshots/test_granularity_date_part_rendering.py/SqlQueryPlan/Databricks/test_offset_window_with_date_part__plan0.sql index ff637a4a8f..ab82a686b5 100644 --- a/metricflow/test/snapshots/test_granularity_date_part_rendering.py/SqlQueryPlan/Databricks/test_offset_window_with_date_part__plan0.sql +++ b/metricflow/test/snapshots/test_granularity_date_part_rendering.py/SqlQueryPlan/Databricks/test_offset_window_with_date_part__plan0.sql @@ -3,7 +3,7 @@ SELECT subq_13.metric_time__extract_dow , bookings - bookings_2_weeks_ago AS bookings_growth_2_weeks FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT COALESCE(subq_4.metric_time__extract_dow, subq_12.metric_time__extract_dow) AS metric_time__extract_dow , MAX(subq_4.bookings) AS bookings diff --git a/metricflow/test/snapshots/test_granularity_date_part_rendering.py/SqlQueryPlan/Databricks/test_offset_window_with_date_part__plan0_optimized.sql b/metricflow/test/snapshots/test_granularity_date_part_rendering.py/SqlQueryPlan/Databricks/test_offset_window_with_date_part__plan0_optimized.sql index 47d65c62a5..eb2dc1d204 100644 --- a/metricflow/test/snapshots/test_granularity_date_part_rendering.py/SqlQueryPlan/Databricks/test_offset_window_with_date_part__plan0_optimized.sql +++ b/metricflow/test/snapshots/test_granularity_date_part_rendering.py/SqlQueryPlan/Databricks/test_offset_window_with_date_part__plan0_optimized.sql @@ -3,7 +3,7 @@ SELECT metric_time__extract_dow , bookings - bookings_2_weeks_ago AS bookings_growth_2_weeks FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT COALESCE(subq_18.metric_time__extract_dow, subq_26.metric_time__extract_dow) AS metric_time__extract_dow , MAX(subq_18.bookings) AS bookings diff --git a/metricflow/test/snapshots/test_granularity_date_part_rendering.py/SqlQueryPlan/DuckDB/test_offset_window_with_date_part__plan0.sql b/metricflow/test/snapshots/test_granularity_date_part_rendering.py/SqlQueryPlan/DuckDB/test_offset_window_with_date_part__plan0.sql index 8e374bcc1b..29582a4df6 100644 --- a/metricflow/test/snapshots/test_granularity_date_part_rendering.py/SqlQueryPlan/DuckDB/test_offset_window_with_date_part__plan0.sql +++ b/metricflow/test/snapshots/test_granularity_date_part_rendering.py/SqlQueryPlan/DuckDB/test_offset_window_with_date_part__plan0.sql @@ -3,7 +3,7 @@ SELECT subq_13.metric_time__extract_dow , bookings - bookings_2_weeks_ago AS bookings_growth_2_weeks FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT COALESCE(subq_4.metric_time__extract_dow, subq_12.metric_time__extract_dow) AS metric_time__extract_dow , MAX(subq_4.bookings) AS bookings diff --git a/metricflow/test/snapshots/test_granularity_date_part_rendering.py/SqlQueryPlan/DuckDB/test_offset_window_with_date_part__plan0_optimized.sql b/metricflow/test/snapshots/test_granularity_date_part_rendering.py/SqlQueryPlan/DuckDB/test_offset_window_with_date_part__plan0_optimized.sql index d808676df4..7ec664448a 100644 --- a/metricflow/test/snapshots/test_granularity_date_part_rendering.py/SqlQueryPlan/DuckDB/test_offset_window_with_date_part__plan0_optimized.sql +++ b/metricflow/test/snapshots/test_granularity_date_part_rendering.py/SqlQueryPlan/DuckDB/test_offset_window_with_date_part__plan0_optimized.sql @@ -3,7 +3,7 @@ SELECT metric_time__extract_dow , bookings - bookings_2_weeks_ago AS bookings_growth_2_weeks FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT COALESCE(subq_18.metric_time__extract_dow, subq_26.metric_time__extract_dow) AS metric_time__extract_dow , MAX(subq_18.bookings) AS bookings diff --git a/metricflow/test/snapshots/test_granularity_date_part_rendering.py/SqlQueryPlan/Postgres/test_offset_window_with_date_part__plan0.sql b/metricflow/test/snapshots/test_granularity_date_part_rendering.py/SqlQueryPlan/Postgres/test_offset_window_with_date_part__plan0.sql index 5bb0010ee6..41c9ba2fc2 100644 --- a/metricflow/test/snapshots/test_granularity_date_part_rendering.py/SqlQueryPlan/Postgres/test_offset_window_with_date_part__plan0.sql +++ b/metricflow/test/snapshots/test_granularity_date_part_rendering.py/SqlQueryPlan/Postgres/test_offset_window_with_date_part__plan0.sql @@ -3,7 +3,7 @@ SELECT subq_13.metric_time__extract_dow , bookings - bookings_2_weeks_ago AS bookings_growth_2_weeks FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT COALESCE(subq_4.metric_time__extract_dow, subq_12.metric_time__extract_dow) AS metric_time__extract_dow , MAX(subq_4.bookings) AS bookings diff --git a/metricflow/test/snapshots/test_granularity_date_part_rendering.py/SqlQueryPlan/Postgres/test_offset_window_with_date_part__plan0_optimized.sql b/metricflow/test/snapshots/test_granularity_date_part_rendering.py/SqlQueryPlan/Postgres/test_offset_window_with_date_part__plan0_optimized.sql index b40bc359ae..05187fb87f 100644 --- a/metricflow/test/snapshots/test_granularity_date_part_rendering.py/SqlQueryPlan/Postgres/test_offset_window_with_date_part__plan0_optimized.sql +++ b/metricflow/test/snapshots/test_granularity_date_part_rendering.py/SqlQueryPlan/Postgres/test_offset_window_with_date_part__plan0_optimized.sql @@ -3,7 +3,7 @@ SELECT metric_time__extract_dow , bookings - bookings_2_weeks_ago AS bookings_growth_2_weeks FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT COALESCE(subq_18.metric_time__extract_dow, subq_26.metric_time__extract_dow) AS metric_time__extract_dow , MAX(subq_18.bookings) AS bookings diff --git a/metricflow/test/snapshots/test_granularity_date_part_rendering.py/SqlQueryPlan/Redshift/test_offset_window_with_date_part__plan0.sql b/metricflow/test/snapshots/test_granularity_date_part_rendering.py/SqlQueryPlan/Redshift/test_offset_window_with_date_part__plan0.sql index 3634349b84..b6828e2fb4 100644 --- a/metricflow/test/snapshots/test_granularity_date_part_rendering.py/SqlQueryPlan/Redshift/test_offset_window_with_date_part__plan0.sql +++ b/metricflow/test/snapshots/test_granularity_date_part_rendering.py/SqlQueryPlan/Redshift/test_offset_window_with_date_part__plan0.sql @@ -3,7 +3,7 @@ SELECT subq_13.metric_time__extract_dow , bookings - bookings_2_weeks_ago AS bookings_growth_2_weeks FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT COALESCE(subq_4.metric_time__extract_dow, subq_12.metric_time__extract_dow) AS metric_time__extract_dow , MAX(subq_4.bookings) AS bookings diff --git a/metricflow/test/snapshots/test_granularity_date_part_rendering.py/SqlQueryPlan/Redshift/test_offset_window_with_date_part__plan0_optimized.sql b/metricflow/test/snapshots/test_granularity_date_part_rendering.py/SqlQueryPlan/Redshift/test_offset_window_with_date_part__plan0_optimized.sql index d544c6c2a2..4604e1f54c 100644 --- a/metricflow/test/snapshots/test_granularity_date_part_rendering.py/SqlQueryPlan/Redshift/test_offset_window_with_date_part__plan0_optimized.sql +++ b/metricflow/test/snapshots/test_granularity_date_part_rendering.py/SqlQueryPlan/Redshift/test_offset_window_with_date_part__plan0_optimized.sql @@ -3,7 +3,7 @@ SELECT metric_time__extract_dow , bookings - bookings_2_weeks_ago AS bookings_growth_2_weeks FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT COALESCE(subq_18.metric_time__extract_dow, subq_26.metric_time__extract_dow) AS metric_time__extract_dow , MAX(subq_18.bookings) AS bookings diff --git a/metricflow/test/snapshots/test_granularity_date_part_rendering.py/SqlQueryPlan/Snowflake/test_offset_window_with_date_part__plan0.sql b/metricflow/test/snapshots/test_granularity_date_part_rendering.py/SqlQueryPlan/Snowflake/test_offset_window_with_date_part__plan0.sql index 4cd16e6523..8150930a5d 100644 --- a/metricflow/test/snapshots/test_granularity_date_part_rendering.py/SqlQueryPlan/Snowflake/test_offset_window_with_date_part__plan0.sql +++ b/metricflow/test/snapshots/test_granularity_date_part_rendering.py/SqlQueryPlan/Snowflake/test_offset_window_with_date_part__plan0.sql @@ -3,7 +3,7 @@ SELECT subq_13.metric_time__extract_dow , bookings - bookings_2_weeks_ago AS bookings_growth_2_weeks FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT COALESCE(subq_4.metric_time__extract_dow, subq_12.metric_time__extract_dow) AS metric_time__extract_dow , MAX(subq_4.bookings) AS bookings diff --git a/metricflow/test/snapshots/test_granularity_date_part_rendering.py/SqlQueryPlan/Snowflake/test_offset_window_with_date_part__plan0_optimized.sql b/metricflow/test/snapshots/test_granularity_date_part_rendering.py/SqlQueryPlan/Snowflake/test_offset_window_with_date_part__plan0_optimized.sql index 9daf56bc90..3b9fbb608d 100644 --- a/metricflow/test/snapshots/test_granularity_date_part_rendering.py/SqlQueryPlan/Snowflake/test_offset_window_with_date_part__plan0_optimized.sql +++ b/metricflow/test/snapshots/test_granularity_date_part_rendering.py/SqlQueryPlan/Snowflake/test_offset_window_with_date_part__plan0_optimized.sql @@ -3,7 +3,7 @@ SELECT metric_time__extract_dow , bookings - bookings_2_weeks_ago AS bookings_growth_2_weeks FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT COALESCE(subq_18.metric_time__extract_dow, subq_26.metric_time__extract_dow) AS metric_time__extract_dow , MAX(subq_18.bookings) AS bookings diff --git a/metricflow/test/snapshots/test_metric_time_dimension_to_sql.py/SqlQueryPlan/BigQuery/test_simple_query_with_metric_time_dimension__plan0.sql b/metricflow/test/snapshots/test_metric_time_dimension_to_sql.py/SqlQueryPlan/BigQuery/test_simple_query_with_metric_time_dimension__plan0.sql index af78758672..ca0cac4bc9 100644 --- a/metricflow/test/snapshots/test_metric_time_dimension_to_sql.py/SqlQueryPlan/BigQuery/test_simple_query_with_metric_time_dimension__plan0.sql +++ b/metricflow/test/snapshots/test_metric_time_dimension_to_sql.py/SqlQueryPlan/BigQuery/test_simple_query_with_metric_time_dimension__plan0.sql @@ -1,4 +1,4 @@ --- Combine Metrics +-- Combine Aggregated Outputs SELECT COALESCE(subq_4.metric_time__day, subq_9.metric_time__day) AS metric_time__day , MAX(subq_4.bookings) AS bookings diff --git a/metricflow/test/snapshots/test_metric_time_dimension_to_sql.py/SqlQueryPlan/BigQuery/test_simple_query_with_metric_time_dimension__plan0_optimized.sql b/metricflow/test/snapshots/test_metric_time_dimension_to_sql.py/SqlQueryPlan/BigQuery/test_simple_query_with_metric_time_dimension__plan0_optimized.sql index aa96d4e0fa..9aa45e7f5a 100644 --- a/metricflow/test/snapshots/test_metric_time_dimension_to_sql.py/SqlQueryPlan/BigQuery/test_simple_query_with_metric_time_dimension__plan0_optimized.sql +++ b/metricflow/test/snapshots/test_metric_time_dimension_to_sql.py/SqlQueryPlan/BigQuery/test_simple_query_with_metric_time_dimension__plan0_optimized.sql @@ -1,4 +1,4 @@ --- Combine Metrics +-- Combine Aggregated Outputs SELECT COALESCE(subq_14.metric_time__day, subq_19.metric_time__day) AS metric_time__day , MAX(subq_14.bookings) AS bookings diff --git a/metricflow/test/snapshots/test_metric_time_dimension_to_sql.py/SqlQueryPlan/Databricks/test_simple_query_with_metric_time_dimension__plan0.sql b/metricflow/test/snapshots/test_metric_time_dimension_to_sql.py/SqlQueryPlan/Databricks/test_simple_query_with_metric_time_dimension__plan0.sql index 328d7a0e4e..ea50da824f 100644 --- a/metricflow/test/snapshots/test_metric_time_dimension_to_sql.py/SqlQueryPlan/Databricks/test_simple_query_with_metric_time_dimension__plan0.sql +++ b/metricflow/test/snapshots/test_metric_time_dimension_to_sql.py/SqlQueryPlan/Databricks/test_simple_query_with_metric_time_dimension__plan0.sql @@ -1,4 +1,4 @@ --- Combine Metrics +-- Combine Aggregated Outputs SELECT COALESCE(subq_4.metric_time__day, subq_9.metric_time__day) AS metric_time__day , MAX(subq_4.bookings) AS bookings diff --git a/metricflow/test/snapshots/test_metric_time_dimension_to_sql.py/SqlQueryPlan/Databricks/test_simple_query_with_metric_time_dimension__plan0_optimized.sql b/metricflow/test/snapshots/test_metric_time_dimension_to_sql.py/SqlQueryPlan/Databricks/test_simple_query_with_metric_time_dimension__plan0_optimized.sql index 873beed5f6..2f965192ba 100644 --- a/metricflow/test/snapshots/test_metric_time_dimension_to_sql.py/SqlQueryPlan/Databricks/test_simple_query_with_metric_time_dimension__plan0_optimized.sql +++ b/metricflow/test/snapshots/test_metric_time_dimension_to_sql.py/SqlQueryPlan/Databricks/test_simple_query_with_metric_time_dimension__plan0_optimized.sql @@ -1,4 +1,4 @@ --- Combine Metrics +-- Combine Aggregated Outputs SELECT COALESCE(subq_14.metric_time__day, subq_19.metric_time__day) AS metric_time__day , MAX(subq_14.bookings) AS bookings diff --git a/metricflow/test/snapshots/test_metric_time_dimension_to_sql.py/SqlQueryPlan/DuckDB/test_simple_query_with_metric_time_dimension__plan0.sql b/metricflow/test/snapshots/test_metric_time_dimension_to_sql.py/SqlQueryPlan/DuckDB/test_simple_query_with_metric_time_dimension__plan0.sql index 41bb520478..91a3efaada 100644 --- a/metricflow/test/snapshots/test_metric_time_dimension_to_sql.py/SqlQueryPlan/DuckDB/test_simple_query_with_metric_time_dimension__plan0.sql +++ b/metricflow/test/snapshots/test_metric_time_dimension_to_sql.py/SqlQueryPlan/DuckDB/test_simple_query_with_metric_time_dimension__plan0.sql @@ -1,4 +1,4 @@ --- Combine Metrics +-- Combine Aggregated Outputs SELECT COALESCE(subq_4.metric_time__day, subq_9.metric_time__day) AS metric_time__day , MAX(subq_4.bookings) AS bookings diff --git a/metricflow/test/snapshots/test_metric_time_dimension_to_sql.py/SqlQueryPlan/DuckDB/test_simple_query_with_metric_time_dimension__plan0_optimized.sql b/metricflow/test/snapshots/test_metric_time_dimension_to_sql.py/SqlQueryPlan/DuckDB/test_simple_query_with_metric_time_dimension__plan0_optimized.sql index 873beed5f6..2f965192ba 100644 --- a/metricflow/test/snapshots/test_metric_time_dimension_to_sql.py/SqlQueryPlan/DuckDB/test_simple_query_with_metric_time_dimension__plan0_optimized.sql +++ b/metricflow/test/snapshots/test_metric_time_dimension_to_sql.py/SqlQueryPlan/DuckDB/test_simple_query_with_metric_time_dimension__plan0_optimized.sql @@ -1,4 +1,4 @@ --- Combine Metrics +-- Combine Aggregated Outputs SELECT COALESCE(subq_14.metric_time__day, subq_19.metric_time__day) AS metric_time__day , MAX(subq_14.bookings) AS bookings diff --git a/metricflow/test/snapshots/test_metric_time_dimension_to_sql.py/SqlQueryPlan/Postgres/test_simple_query_with_metric_time_dimension__plan0.sql b/metricflow/test/snapshots/test_metric_time_dimension_to_sql.py/SqlQueryPlan/Postgres/test_simple_query_with_metric_time_dimension__plan0.sql index 41bb520478..91a3efaada 100644 --- a/metricflow/test/snapshots/test_metric_time_dimension_to_sql.py/SqlQueryPlan/Postgres/test_simple_query_with_metric_time_dimension__plan0.sql +++ b/metricflow/test/snapshots/test_metric_time_dimension_to_sql.py/SqlQueryPlan/Postgres/test_simple_query_with_metric_time_dimension__plan0.sql @@ -1,4 +1,4 @@ --- Combine Metrics +-- Combine Aggregated Outputs SELECT COALESCE(subq_4.metric_time__day, subq_9.metric_time__day) AS metric_time__day , MAX(subq_4.bookings) AS bookings diff --git a/metricflow/test/snapshots/test_metric_time_dimension_to_sql.py/SqlQueryPlan/Postgres/test_simple_query_with_metric_time_dimension__plan0_optimized.sql b/metricflow/test/snapshots/test_metric_time_dimension_to_sql.py/SqlQueryPlan/Postgres/test_simple_query_with_metric_time_dimension__plan0_optimized.sql index 873beed5f6..2f965192ba 100644 --- a/metricflow/test/snapshots/test_metric_time_dimension_to_sql.py/SqlQueryPlan/Postgres/test_simple_query_with_metric_time_dimension__plan0_optimized.sql +++ b/metricflow/test/snapshots/test_metric_time_dimension_to_sql.py/SqlQueryPlan/Postgres/test_simple_query_with_metric_time_dimension__plan0_optimized.sql @@ -1,4 +1,4 @@ --- Combine Metrics +-- Combine Aggregated Outputs SELECT COALESCE(subq_14.metric_time__day, subq_19.metric_time__day) AS metric_time__day , MAX(subq_14.bookings) AS bookings diff --git a/metricflow/test/snapshots/test_metric_time_dimension_to_sql.py/SqlQueryPlan/Redshift/test_simple_query_with_metric_time_dimension__plan0.sql b/metricflow/test/snapshots/test_metric_time_dimension_to_sql.py/SqlQueryPlan/Redshift/test_simple_query_with_metric_time_dimension__plan0.sql index 2a37bd05bf..5fc2739d71 100644 --- a/metricflow/test/snapshots/test_metric_time_dimension_to_sql.py/SqlQueryPlan/Redshift/test_simple_query_with_metric_time_dimension__plan0.sql +++ b/metricflow/test/snapshots/test_metric_time_dimension_to_sql.py/SqlQueryPlan/Redshift/test_simple_query_with_metric_time_dimension__plan0.sql @@ -1,4 +1,4 @@ --- Combine Metrics +-- Combine Aggregated Outputs SELECT COALESCE(subq_4.metric_time__day, subq_9.metric_time__day) AS metric_time__day , MAX(subq_4.bookings) AS bookings diff --git a/metricflow/test/snapshots/test_metric_time_dimension_to_sql.py/SqlQueryPlan/Redshift/test_simple_query_with_metric_time_dimension__plan0_optimized.sql b/metricflow/test/snapshots/test_metric_time_dimension_to_sql.py/SqlQueryPlan/Redshift/test_simple_query_with_metric_time_dimension__plan0_optimized.sql index 873beed5f6..2f965192ba 100644 --- a/metricflow/test/snapshots/test_metric_time_dimension_to_sql.py/SqlQueryPlan/Redshift/test_simple_query_with_metric_time_dimension__plan0_optimized.sql +++ b/metricflow/test/snapshots/test_metric_time_dimension_to_sql.py/SqlQueryPlan/Redshift/test_simple_query_with_metric_time_dimension__plan0_optimized.sql @@ -1,4 +1,4 @@ --- Combine Metrics +-- Combine Aggregated Outputs SELECT COALESCE(subq_14.metric_time__day, subq_19.metric_time__day) AS metric_time__day , MAX(subq_14.bookings) AS bookings diff --git a/metricflow/test/snapshots/test_metric_time_dimension_to_sql.py/SqlQueryPlan/Snowflake/test_simple_query_with_metric_time_dimension__plan0.sql b/metricflow/test/snapshots/test_metric_time_dimension_to_sql.py/SqlQueryPlan/Snowflake/test_simple_query_with_metric_time_dimension__plan0.sql index 1dcb8db689..a08e5d9ef6 100644 --- a/metricflow/test/snapshots/test_metric_time_dimension_to_sql.py/SqlQueryPlan/Snowflake/test_simple_query_with_metric_time_dimension__plan0.sql +++ b/metricflow/test/snapshots/test_metric_time_dimension_to_sql.py/SqlQueryPlan/Snowflake/test_simple_query_with_metric_time_dimension__plan0.sql @@ -1,4 +1,4 @@ --- Combine Metrics +-- Combine Aggregated Outputs SELECT COALESCE(subq_4.metric_time__day, subq_9.metric_time__day) AS metric_time__day , MAX(subq_4.bookings) AS bookings diff --git a/metricflow/test/snapshots/test_metric_time_dimension_to_sql.py/SqlQueryPlan/Snowflake/test_simple_query_with_metric_time_dimension__plan0_optimized.sql b/metricflow/test/snapshots/test_metric_time_dimension_to_sql.py/SqlQueryPlan/Snowflake/test_simple_query_with_metric_time_dimension__plan0_optimized.sql index 873beed5f6..2f965192ba 100644 --- a/metricflow/test/snapshots/test_metric_time_dimension_to_sql.py/SqlQueryPlan/Snowflake/test_simple_query_with_metric_time_dimension__plan0_optimized.sql +++ b/metricflow/test/snapshots/test_metric_time_dimension_to_sql.py/SqlQueryPlan/Snowflake/test_simple_query_with_metric_time_dimension__plan0_optimized.sql @@ -1,4 +1,4 @@ --- Combine Metrics +-- Combine Aggregated Outputs SELECT COALESCE(subq_14.metric_time__day, subq_19.metric_time__day) AS metric_time__day , MAX(subq_14.bookings) AS bookings diff --git a/metricflow/test/snapshots/test_metric_time_dimension_to_sql.py/SqlQueryPlan/test_simple_query_with_metric_time_dimension__plan0.xml b/metricflow/test/snapshots/test_metric_time_dimension_to_sql.py/SqlQueryPlan/test_simple_query_with_metric_time_dimension__plan0.xml index 3ade70ad35..f10ffcaeb6 100644 --- a/metricflow/test/snapshots/test_metric_time_dimension_to_sql.py/SqlQueryPlan/test_simple_query_with_metric_time_dimension__plan0.xml +++ b/metricflow/test/snapshots/test_metric_time_dimension_to_sql.py/SqlQueryPlan/test_simple_query_with_metric_time_dimension__plan0.xml @@ -1,6 +1,6 @@ - + diff --git a/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/BigQuery/test_common_semantic_model__plan0.sql b/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/BigQuery/test_common_semantic_model__plan0.sql index c1cce0b30e..7023ca64aa 100644 --- a/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/BigQuery/test_common_semantic_model__plan0.sql +++ b/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/BigQuery/test_common_semantic_model__plan0.sql @@ -1,4 +1,4 @@ --- Combine Metrics +-- Combine Aggregated Outputs SELECT COALESCE(subq_4.metric_time__day, subq_9.metric_time__day) AS metric_time__day , MAX(subq_4.bookings) AS bookings diff --git a/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/BigQuery/test_common_semantic_model__plan0_optimized.sql b/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/BigQuery/test_common_semantic_model__plan0_optimized.sql index bb0ddce058..717b2bf81c 100644 --- a/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/BigQuery/test_common_semantic_model__plan0_optimized.sql +++ b/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/BigQuery/test_common_semantic_model__plan0_optimized.sql @@ -1,4 +1,4 @@ --- Combine Metrics +-- Combine Aggregated Outputs SELECT COALESCE(subq_14.metric_time__day, subq_19.metric_time__day) AS metric_time__day , MAX(subq_14.bookings) AS bookings diff --git a/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/BigQuery/test_measure_constraint__plan0.sql b/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/BigQuery/test_measure_constraint__plan0.sql index 4c176465fe..bda66f6049 100644 --- a/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/BigQuery/test_measure_constraint__plan0.sql +++ b/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/BigQuery/test_measure_constraint__plan0.sql @@ -3,7 +3,7 @@ SELECT subq_29.metric_time__day , average_booking_value * bookings / NULLIF(booking_value, 0) AS lux_booking_value_rate_expr FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT COALESCE(subq_11.metric_time__day, subq_23.metric_time__day, subq_28.metric_time__day) AS metric_time__day , MAX(subq_11.average_booking_value) AS average_booking_value diff --git a/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/BigQuery/test_measure_constraint__plan0_optimized.sql b/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/BigQuery/test_measure_constraint__plan0_optimized.sql index 7449730839..7c1744a24e 100644 --- a/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/BigQuery/test_measure_constraint__plan0_optimized.sql +++ b/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/BigQuery/test_measure_constraint__plan0_optimized.sql @@ -3,7 +3,7 @@ SELECT metric_time__day , average_booking_value * bookings / NULLIF(booking_value, 0) AS lux_booking_value_rate_expr FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT COALESCE(subq_41.metric_time__day, subq_53.metric_time__day, subq_58.metric_time__day) AS metric_time__day , MAX(subq_41.average_booking_value) AS average_booking_value diff --git a/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/BigQuery/test_measure_constraint_with_reused_measure__plan0.sql b/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/BigQuery/test_measure_constraint_with_reused_measure__plan0.sql index 2f6627ec7f..406d5c2478 100644 --- a/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/BigQuery/test_measure_constraint_with_reused_measure__plan0.sql +++ b/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/BigQuery/test_measure_constraint_with_reused_measure__plan0.sql @@ -3,7 +3,7 @@ SELECT subq_12.metric_time__day , CAST(subq_12.booking_value_with_is_instant_constraint AS FLOAT64) / CAST(NULLIF(subq_12.booking_value, 0) AS FLOAT64) AS instant_booking_value_ratio FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT COALESCE(subq_6.metric_time__day, subq_11.metric_time__day) AS metric_time__day , MAX(subq_6.booking_value_with_is_instant_constraint) AS booking_value_with_is_instant_constraint diff --git a/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/BigQuery/test_measure_constraint_with_reused_measure__plan0_optimized.sql b/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/BigQuery/test_measure_constraint_with_reused_measure__plan0_optimized.sql index 5b744a582e..b448fb33d4 100644 --- a/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/BigQuery/test_measure_constraint_with_reused_measure__plan0_optimized.sql +++ b/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/BigQuery/test_measure_constraint_with_reused_measure__plan0_optimized.sql @@ -3,7 +3,7 @@ SELECT metric_time__day , CAST(booking_value_with_is_instant_constraint AS FLOAT64) / CAST(NULLIF(booking_value, 0) AS FLOAT64) AS instant_booking_value_ratio FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT COALESCE(subq_19.metric_time__day, subq_24.metric_time__day) AS metric_time__day , MAX(subq_19.booking_value_with_is_instant_constraint) AS booking_value_with_is_instant_constraint diff --git a/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/BigQuery/test_metric_with_measures_from_multiple_sources_no_dimensions__plan0.sql b/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/BigQuery/test_metric_with_measures_from_multiple_sources_no_dimensions__plan0.sql index 4a2b12935f..a506e8dafb 100644 --- a/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/BigQuery/test_metric_with_measures_from_multiple_sources_no_dimensions__plan0.sql +++ b/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/BigQuery/test_metric_with_measures_from_multiple_sources_no_dimensions__plan0.sql @@ -2,7 +2,7 @@ SELECT CAST(subq_10.bookings AS FLOAT64) / CAST(NULLIF(subq_10.listings, 0) AS FLOAT64) AS bookings_per_listing FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT MAX(subq_4.bookings) AS bookings , MAX(subq_9.listings) AS listings diff --git a/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/BigQuery/test_metric_with_measures_from_multiple_sources_no_dimensions__plan0_optimized.sql b/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/BigQuery/test_metric_with_measures_from_multiple_sources_no_dimensions__plan0_optimized.sql index e66020c868..64aaaf1d47 100644 --- a/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/BigQuery/test_metric_with_measures_from_multiple_sources_no_dimensions__plan0_optimized.sql +++ b/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/BigQuery/test_metric_with_measures_from_multiple_sources_no_dimensions__plan0_optimized.sql @@ -1,4 +1,4 @@ --- Combine Metrics +-- Combine Aggregated Outputs -- Compute Metrics via Expressions SELECT CAST(MAX(subq_15.bookings) AS FLOAT64) / CAST(NULLIF(MAX(subq_20.listings), 0) AS FLOAT64) AS bookings_per_listing diff --git a/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/BigQuery/test_multiple_metrics_no_dimensions__plan0.sql b/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/BigQuery/test_multiple_metrics_no_dimensions__plan0.sql index 571f871e2a..6d823d91b4 100644 --- a/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/BigQuery/test_multiple_metrics_no_dimensions__plan0.sql +++ b/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/BigQuery/test_multiple_metrics_no_dimensions__plan0.sql @@ -1,4 +1,4 @@ --- Combine Metrics +-- Combine Aggregated Outputs SELECT MAX(subq_5.bookings) AS bookings , MAX(subq_11.listings) AS listings diff --git a/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/BigQuery/test_multiple_metrics_no_dimensions__plan0_optimized.sql b/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/BigQuery/test_multiple_metrics_no_dimensions__plan0_optimized.sql index 134eb00d16..a19c9a983b 100644 --- a/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/BigQuery/test_multiple_metrics_no_dimensions__plan0_optimized.sql +++ b/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/BigQuery/test_multiple_metrics_no_dimensions__plan0_optimized.sql @@ -1,4 +1,4 @@ --- Combine Metrics +-- Combine Aggregated Outputs SELECT MAX(subq_17.bookings) AS bookings , MAX(subq_23.listings) AS listings diff --git a/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/Databricks/test_common_semantic_model__plan0.sql b/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/Databricks/test_common_semantic_model__plan0.sql index 07dc51cee4..de42ab65c5 100644 --- a/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/Databricks/test_common_semantic_model__plan0.sql +++ b/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/Databricks/test_common_semantic_model__plan0.sql @@ -1,4 +1,4 @@ --- Combine Metrics +-- Combine Aggregated Outputs SELECT COALESCE(subq_4.metric_time__day, subq_9.metric_time__day) AS metric_time__day , MAX(subq_4.bookings) AS bookings diff --git a/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/Databricks/test_common_semantic_model__plan0_optimized.sql b/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/Databricks/test_common_semantic_model__plan0_optimized.sql index 552b4dc805..d2cebbcf57 100644 --- a/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/Databricks/test_common_semantic_model__plan0_optimized.sql +++ b/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/Databricks/test_common_semantic_model__plan0_optimized.sql @@ -1,4 +1,4 @@ --- Combine Metrics +-- Combine Aggregated Outputs SELECT COALESCE(subq_14.metric_time__day, subq_19.metric_time__day) AS metric_time__day , MAX(subq_14.bookings) AS bookings diff --git a/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/Databricks/test_measure_constraint__plan0.sql b/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/Databricks/test_measure_constraint__plan0.sql index 21cf8ec5e7..373cca1728 100644 --- a/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/Databricks/test_measure_constraint__plan0.sql +++ b/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/Databricks/test_measure_constraint__plan0.sql @@ -3,7 +3,7 @@ SELECT subq_29.metric_time__day , average_booking_value * bookings / NULLIF(booking_value, 0) AS lux_booking_value_rate_expr FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT COALESCE(subq_11.metric_time__day, subq_23.metric_time__day, subq_28.metric_time__day) AS metric_time__day , MAX(subq_11.average_booking_value) AS average_booking_value diff --git a/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/Databricks/test_measure_constraint__plan0_optimized.sql b/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/Databricks/test_measure_constraint__plan0_optimized.sql index 0b555b1430..3c0c641be9 100644 --- a/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/Databricks/test_measure_constraint__plan0_optimized.sql +++ b/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/Databricks/test_measure_constraint__plan0_optimized.sql @@ -3,7 +3,7 @@ SELECT metric_time__day , average_booking_value * bookings / NULLIF(booking_value, 0) AS lux_booking_value_rate_expr FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT COALESCE(subq_41.metric_time__day, subq_53.metric_time__day, subq_58.metric_time__day) AS metric_time__day , MAX(subq_41.average_booking_value) AS average_booking_value diff --git a/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/Databricks/test_measure_constraint_with_reused_measure__plan0.sql b/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/Databricks/test_measure_constraint_with_reused_measure__plan0.sql index 4609e7f48e..54a64092d3 100644 --- a/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/Databricks/test_measure_constraint_with_reused_measure__plan0.sql +++ b/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/Databricks/test_measure_constraint_with_reused_measure__plan0.sql @@ -3,7 +3,7 @@ SELECT subq_12.metric_time__day , CAST(subq_12.booking_value_with_is_instant_constraint AS DOUBLE) / CAST(NULLIF(subq_12.booking_value, 0) AS DOUBLE) AS instant_booking_value_ratio FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT COALESCE(subq_6.metric_time__day, subq_11.metric_time__day) AS metric_time__day , MAX(subq_6.booking_value_with_is_instant_constraint) AS booking_value_with_is_instant_constraint diff --git a/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/Databricks/test_measure_constraint_with_reused_measure__plan0_optimized.sql b/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/Databricks/test_measure_constraint_with_reused_measure__plan0_optimized.sql index 77f5d693df..2d2b5a142c 100644 --- a/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/Databricks/test_measure_constraint_with_reused_measure__plan0_optimized.sql +++ b/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/Databricks/test_measure_constraint_with_reused_measure__plan0_optimized.sql @@ -3,7 +3,7 @@ SELECT metric_time__day , CAST(booking_value_with_is_instant_constraint AS DOUBLE) / CAST(NULLIF(booking_value, 0) AS DOUBLE) AS instant_booking_value_ratio FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT COALESCE(subq_19.metric_time__day, subq_24.metric_time__day) AS metric_time__day , MAX(subq_19.booking_value_with_is_instant_constraint) AS booking_value_with_is_instant_constraint diff --git a/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/Databricks/test_metric_with_measures_from_multiple_sources_no_dimensions__plan0.sql b/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/Databricks/test_metric_with_measures_from_multiple_sources_no_dimensions__plan0.sql index e36b3d9819..12cf835a45 100644 --- a/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/Databricks/test_metric_with_measures_from_multiple_sources_no_dimensions__plan0.sql +++ b/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/Databricks/test_metric_with_measures_from_multiple_sources_no_dimensions__plan0.sql @@ -2,7 +2,7 @@ SELECT CAST(subq_10.bookings AS DOUBLE) / CAST(NULLIF(subq_10.listings, 0) AS DOUBLE) AS bookings_per_listing FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT MAX(subq_4.bookings) AS bookings , MAX(subq_9.listings) AS listings diff --git a/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/Databricks/test_metric_with_measures_from_multiple_sources_no_dimensions__plan0_optimized.sql b/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/Databricks/test_metric_with_measures_from_multiple_sources_no_dimensions__plan0_optimized.sql index 8a1096e90f..bd5b5c24ee 100644 --- a/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/Databricks/test_metric_with_measures_from_multiple_sources_no_dimensions__plan0_optimized.sql +++ b/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/Databricks/test_metric_with_measures_from_multiple_sources_no_dimensions__plan0_optimized.sql @@ -1,4 +1,4 @@ --- Combine Metrics +-- Combine Aggregated Outputs -- Compute Metrics via Expressions SELECT CAST(MAX(subq_15.bookings) AS DOUBLE) / CAST(NULLIF(MAX(subq_20.listings), 0) AS DOUBLE) AS bookings_per_listing diff --git a/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/Databricks/test_multiple_metrics_no_dimensions__plan0.sql b/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/Databricks/test_multiple_metrics_no_dimensions__plan0.sql index 0497fb5627..bb0ca2196f 100644 --- a/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/Databricks/test_multiple_metrics_no_dimensions__plan0.sql +++ b/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/Databricks/test_multiple_metrics_no_dimensions__plan0.sql @@ -1,4 +1,4 @@ --- Combine Metrics +-- Combine Aggregated Outputs SELECT MAX(subq_5.bookings) AS bookings , MAX(subq_11.listings) AS listings diff --git a/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/Databricks/test_multiple_metrics_no_dimensions__plan0_optimized.sql b/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/Databricks/test_multiple_metrics_no_dimensions__plan0_optimized.sql index 56554460d5..631546edf7 100644 --- a/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/Databricks/test_multiple_metrics_no_dimensions__plan0_optimized.sql +++ b/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/Databricks/test_multiple_metrics_no_dimensions__plan0_optimized.sql @@ -1,4 +1,4 @@ --- Combine Metrics +-- Combine Aggregated Outputs SELECT MAX(subq_17.bookings) AS bookings , MAX(subq_23.listings) AS listings diff --git a/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/DuckDB/test_common_semantic_model__plan0.sql b/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/DuckDB/test_common_semantic_model__plan0.sql index 1457dd33ee..02b3cbc31b 100644 --- a/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/DuckDB/test_common_semantic_model__plan0.sql +++ b/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/DuckDB/test_common_semantic_model__plan0.sql @@ -1,4 +1,4 @@ --- Combine Metrics +-- Combine Aggregated Outputs SELECT COALESCE(subq_4.metric_time__day, subq_9.metric_time__day) AS metric_time__day , MAX(subq_4.bookings) AS bookings diff --git a/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/DuckDB/test_common_semantic_model__plan0_optimized.sql b/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/DuckDB/test_common_semantic_model__plan0_optimized.sql index 552b4dc805..d2cebbcf57 100644 --- a/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/DuckDB/test_common_semantic_model__plan0_optimized.sql +++ b/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/DuckDB/test_common_semantic_model__plan0_optimized.sql @@ -1,4 +1,4 @@ --- Combine Metrics +-- Combine Aggregated Outputs SELECT COALESCE(subq_14.metric_time__day, subq_19.metric_time__day) AS metric_time__day , MAX(subq_14.bookings) AS bookings diff --git a/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/DuckDB/test_measure_constraint__plan0.sql b/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/DuckDB/test_measure_constraint__plan0.sql index ba7e10edd2..b314ea28d3 100644 --- a/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/DuckDB/test_measure_constraint__plan0.sql +++ b/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/DuckDB/test_measure_constraint__plan0.sql @@ -3,7 +3,7 @@ SELECT subq_29.metric_time__day , average_booking_value * bookings / NULLIF(booking_value, 0) AS lux_booking_value_rate_expr FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT COALESCE(subq_11.metric_time__day, subq_23.metric_time__day, subq_28.metric_time__day) AS metric_time__day , MAX(subq_11.average_booking_value) AS average_booking_value diff --git a/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/DuckDB/test_measure_constraint__plan0_optimized.sql b/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/DuckDB/test_measure_constraint__plan0_optimized.sql index 0b555b1430..3c0c641be9 100644 --- a/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/DuckDB/test_measure_constraint__plan0_optimized.sql +++ b/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/DuckDB/test_measure_constraint__plan0_optimized.sql @@ -3,7 +3,7 @@ SELECT metric_time__day , average_booking_value * bookings / NULLIF(booking_value, 0) AS lux_booking_value_rate_expr FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT COALESCE(subq_41.metric_time__day, subq_53.metric_time__day, subq_58.metric_time__day) AS metric_time__day , MAX(subq_41.average_booking_value) AS average_booking_value diff --git a/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/DuckDB/test_measure_constraint_with_reused_measure__plan0.sql b/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/DuckDB/test_measure_constraint_with_reused_measure__plan0.sql index c4e6b9b0d5..2b99daf59a 100644 --- a/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/DuckDB/test_measure_constraint_with_reused_measure__plan0.sql +++ b/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/DuckDB/test_measure_constraint_with_reused_measure__plan0.sql @@ -3,7 +3,7 @@ SELECT subq_12.metric_time__day , CAST(subq_12.booking_value_with_is_instant_constraint AS DOUBLE) / CAST(NULLIF(subq_12.booking_value, 0) AS DOUBLE) AS instant_booking_value_ratio FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT COALESCE(subq_6.metric_time__day, subq_11.metric_time__day) AS metric_time__day , MAX(subq_6.booking_value_with_is_instant_constraint) AS booking_value_with_is_instant_constraint diff --git a/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/DuckDB/test_measure_constraint_with_reused_measure__plan0_optimized.sql b/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/DuckDB/test_measure_constraint_with_reused_measure__plan0_optimized.sql index 77f5d693df..2d2b5a142c 100644 --- a/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/DuckDB/test_measure_constraint_with_reused_measure__plan0_optimized.sql +++ b/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/DuckDB/test_measure_constraint_with_reused_measure__plan0_optimized.sql @@ -3,7 +3,7 @@ SELECT metric_time__day , CAST(booking_value_with_is_instant_constraint AS DOUBLE) / CAST(NULLIF(booking_value, 0) AS DOUBLE) AS instant_booking_value_ratio FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT COALESCE(subq_19.metric_time__day, subq_24.metric_time__day) AS metric_time__day , MAX(subq_19.booking_value_with_is_instant_constraint) AS booking_value_with_is_instant_constraint diff --git a/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/DuckDB/test_metric_with_measures_from_multiple_sources_no_dimensions__plan0.sql b/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/DuckDB/test_metric_with_measures_from_multiple_sources_no_dimensions__plan0.sql index 01f9725d4d..1bbbe9b000 100644 --- a/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/DuckDB/test_metric_with_measures_from_multiple_sources_no_dimensions__plan0.sql +++ b/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/DuckDB/test_metric_with_measures_from_multiple_sources_no_dimensions__plan0.sql @@ -2,7 +2,7 @@ SELECT CAST(subq_10.bookings AS DOUBLE) / CAST(NULLIF(subq_10.listings, 0) AS DOUBLE) AS bookings_per_listing FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT MAX(subq_4.bookings) AS bookings , MAX(subq_9.listings) AS listings diff --git a/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/DuckDB/test_metric_with_measures_from_multiple_sources_no_dimensions__plan0_optimized.sql b/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/DuckDB/test_metric_with_measures_from_multiple_sources_no_dimensions__plan0_optimized.sql index 8a1096e90f..bd5b5c24ee 100644 --- a/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/DuckDB/test_metric_with_measures_from_multiple_sources_no_dimensions__plan0_optimized.sql +++ b/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/DuckDB/test_metric_with_measures_from_multiple_sources_no_dimensions__plan0_optimized.sql @@ -1,4 +1,4 @@ --- Combine Metrics +-- Combine Aggregated Outputs -- Compute Metrics via Expressions SELECT CAST(MAX(subq_15.bookings) AS DOUBLE) / CAST(NULLIF(MAX(subq_20.listings), 0) AS DOUBLE) AS bookings_per_listing diff --git a/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/DuckDB/test_multiple_metrics_no_dimensions__plan0.sql b/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/DuckDB/test_multiple_metrics_no_dimensions__plan0.sql index a9ee829c3e..7ad1c04247 100644 --- a/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/DuckDB/test_multiple_metrics_no_dimensions__plan0.sql +++ b/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/DuckDB/test_multiple_metrics_no_dimensions__plan0.sql @@ -1,4 +1,4 @@ --- Combine Metrics +-- Combine Aggregated Outputs SELECT MAX(subq_5.bookings) AS bookings , MAX(subq_11.listings) AS listings diff --git a/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/DuckDB/test_multiple_metrics_no_dimensions__plan0_optimized.sql b/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/DuckDB/test_multiple_metrics_no_dimensions__plan0_optimized.sql index 56554460d5..631546edf7 100644 --- a/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/DuckDB/test_multiple_metrics_no_dimensions__plan0_optimized.sql +++ b/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/DuckDB/test_multiple_metrics_no_dimensions__plan0_optimized.sql @@ -1,4 +1,4 @@ --- Combine Metrics +-- Combine Aggregated Outputs SELECT MAX(subq_17.bookings) AS bookings , MAX(subq_23.listings) AS listings diff --git a/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/Postgres/test_common_semantic_model__plan0.sql b/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/Postgres/test_common_semantic_model__plan0.sql index 1457dd33ee..02b3cbc31b 100644 --- a/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/Postgres/test_common_semantic_model__plan0.sql +++ b/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/Postgres/test_common_semantic_model__plan0.sql @@ -1,4 +1,4 @@ --- Combine Metrics +-- Combine Aggregated Outputs SELECT COALESCE(subq_4.metric_time__day, subq_9.metric_time__day) AS metric_time__day , MAX(subq_4.bookings) AS bookings diff --git a/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/Postgres/test_common_semantic_model__plan0_optimized.sql b/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/Postgres/test_common_semantic_model__plan0_optimized.sql index 552b4dc805..d2cebbcf57 100644 --- a/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/Postgres/test_common_semantic_model__plan0_optimized.sql +++ b/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/Postgres/test_common_semantic_model__plan0_optimized.sql @@ -1,4 +1,4 @@ --- Combine Metrics +-- Combine Aggregated Outputs SELECT COALESCE(subq_14.metric_time__day, subq_19.metric_time__day) AS metric_time__day , MAX(subq_14.bookings) AS bookings diff --git a/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/Postgres/test_measure_constraint__plan0.sql b/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/Postgres/test_measure_constraint__plan0.sql index ba7e10edd2..b314ea28d3 100644 --- a/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/Postgres/test_measure_constraint__plan0.sql +++ b/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/Postgres/test_measure_constraint__plan0.sql @@ -3,7 +3,7 @@ SELECT subq_29.metric_time__day , average_booking_value * bookings / NULLIF(booking_value, 0) AS lux_booking_value_rate_expr FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT COALESCE(subq_11.metric_time__day, subq_23.metric_time__day, subq_28.metric_time__day) AS metric_time__day , MAX(subq_11.average_booking_value) AS average_booking_value diff --git a/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/Postgres/test_measure_constraint__plan0_optimized.sql b/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/Postgres/test_measure_constraint__plan0_optimized.sql index 0b555b1430..3c0c641be9 100644 --- a/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/Postgres/test_measure_constraint__plan0_optimized.sql +++ b/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/Postgres/test_measure_constraint__plan0_optimized.sql @@ -3,7 +3,7 @@ SELECT metric_time__day , average_booking_value * bookings / NULLIF(booking_value, 0) AS lux_booking_value_rate_expr FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT COALESCE(subq_41.metric_time__day, subq_53.metric_time__day, subq_58.metric_time__day) AS metric_time__day , MAX(subq_41.average_booking_value) AS average_booking_value diff --git a/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/Postgres/test_measure_constraint_with_reused_measure__plan0.sql b/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/Postgres/test_measure_constraint_with_reused_measure__plan0.sql index d69116c39b..4cd6b2c6a9 100644 --- a/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/Postgres/test_measure_constraint_with_reused_measure__plan0.sql +++ b/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/Postgres/test_measure_constraint_with_reused_measure__plan0.sql @@ -3,7 +3,7 @@ SELECT subq_12.metric_time__day , CAST(subq_12.booking_value_with_is_instant_constraint AS DOUBLE PRECISION) / CAST(NULLIF(subq_12.booking_value, 0) AS DOUBLE PRECISION) AS instant_booking_value_ratio FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT COALESCE(subq_6.metric_time__day, subq_11.metric_time__day) AS metric_time__day , MAX(subq_6.booking_value_with_is_instant_constraint) AS booking_value_with_is_instant_constraint diff --git a/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/Postgres/test_measure_constraint_with_reused_measure__plan0_optimized.sql b/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/Postgres/test_measure_constraint_with_reused_measure__plan0_optimized.sql index dca28f5735..9d1c4deb21 100644 --- a/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/Postgres/test_measure_constraint_with_reused_measure__plan0_optimized.sql +++ b/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/Postgres/test_measure_constraint_with_reused_measure__plan0_optimized.sql @@ -3,7 +3,7 @@ SELECT metric_time__day , CAST(booking_value_with_is_instant_constraint AS DOUBLE PRECISION) / CAST(NULLIF(booking_value, 0) AS DOUBLE PRECISION) AS instant_booking_value_ratio FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT COALESCE(subq_19.metric_time__day, subq_24.metric_time__day) AS metric_time__day , MAX(subq_19.booking_value_with_is_instant_constraint) AS booking_value_with_is_instant_constraint diff --git a/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/Postgres/test_metric_with_measures_from_multiple_sources_no_dimensions__plan0.sql b/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/Postgres/test_metric_with_measures_from_multiple_sources_no_dimensions__plan0.sql index 20f37cdbd8..8222b0adff 100644 --- a/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/Postgres/test_metric_with_measures_from_multiple_sources_no_dimensions__plan0.sql +++ b/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/Postgres/test_metric_with_measures_from_multiple_sources_no_dimensions__plan0.sql @@ -2,7 +2,7 @@ SELECT CAST(subq_10.bookings AS DOUBLE PRECISION) / CAST(NULLIF(subq_10.listings, 0) AS DOUBLE PRECISION) AS bookings_per_listing FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT MAX(subq_4.bookings) AS bookings , MAX(subq_9.listings) AS listings diff --git a/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/Postgres/test_metric_with_measures_from_multiple_sources_no_dimensions__plan0_optimized.sql b/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/Postgres/test_metric_with_measures_from_multiple_sources_no_dimensions__plan0_optimized.sql index 31bce3d483..c48f7d8d21 100644 --- a/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/Postgres/test_metric_with_measures_from_multiple_sources_no_dimensions__plan0_optimized.sql +++ b/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/Postgres/test_metric_with_measures_from_multiple_sources_no_dimensions__plan0_optimized.sql @@ -1,4 +1,4 @@ --- Combine Metrics +-- Combine Aggregated Outputs -- Compute Metrics via Expressions SELECT CAST(MAX(subq_15.bookings) AS DOUBLE PRECISION) / CAST(NULLIF(MAX(subq_20.listings), 0) AS DOUBLE PRECISION) AS bookings_per_listing diff --git a/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/Postgres/test_multiple_metrics_no_dimensions__plan0.sql b/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/Postgres/test_multiple_metrics_no_dimensions__plan0.sql index a9ee829c3e..7ad1c04247 100644 --- a/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/Postgres/test_multiple_metrics_no_dimensions__plan0.sql +++ b/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/Postgres/test_multiple_metrics_no_dimensions__plan0.sql @@ -1,4 +1,4 @@ --- Combine Metrics +-- Combine Aggregated Outputs SELECT MAX(subq_5.bookings) AS bookings , MAX(subq_11.listings) AS listings diff --git a/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/Postgres/test_multiple_metrics_no_dimensions__plan0_optimized.sql b/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/Postgres/test_multiple_metrics_no_dimensions__plan0_optimized.sql index 56554460d5..631546edf7 100644 --- a/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/Postgres/test_multiple_metrics_no_dimensions__plan0_optimized.sql +++ b/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/Postgres/test_multiple_metrics_no_dimensions__plan0_optimized.sql @@ -1,4 +1,4 @@ --- Combine Metrics +-- Combine Aggregated Outputs SELECT MAX(subq_17.bookings) AS bookings , MAX(subq_23.listings) AS listings diff --git a/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/Redshift/test_common_semantic_model__plan0.sql b/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/Redshift/test_common_semantic_model__plan0.sql index c09ea751f8..32a9f1ca0b 100644 --- a/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/Redshift/test_common_semantic_model__plan0.sql +++ b/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/Redshift/test_common_semantic_model__plan0.sql @@ -1,4 +1,4 @@ --- Combine Metrics +-- Combine Aggregated Outputs SELECT COALESCE(subq_4.metric_time__day, subq_9.metric_time__day) AS metric_time__day , MAX(subq_4.bookings) AS bookings diff --git a/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/Redshift/test_common_semantic_model__plan0_optimized.sql b/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/Redshift/test_common_semantic_model__plan0_optimized.sql index 552b4dc805..d2cebbcf57 100644 --- a/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/Redshift/test_common_semantic_model__plan0_optimized.sql +++ b/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/Redshift/test_common_semantic_model__plan0_optimized.sql @@ -1,4 +1,4 @@ --- Combine Metrics +-- Combine Aggregated Outputs SELECT COALESCE(subq_14.metric_time__day, subq_19.metric_time__day) AS metric_time__day , MAX(subq_14.bookings) AS bookings diff --git a/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/Redshift/test_measure_constraint__plan0.sql b/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/Redshift/test_measure_constraint__plan0.sql index 53fc864739..0afe15717f 100644 --- a/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/Redshift/test_measure_constraint__plan0.sql +++ b/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/Redshift/test_measure_constraint__plan0.sql @@ -3,7 +3,7 @@ SELECT subq_29.metric_time__day , average_booking_value * bookings / NULLIF(booking_value, 0) AS lux_booking_value_rate_expr FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT COALESCE(subq_11.metric_time__day, subq_23.metric_time__day, subq_28.metric_time__day) AS metric_time__day , MAX(subq_11.average_booking_value) AS average_booking_value diff --git a/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/Redshift/test_measure_constraint__plan0_optimized.sql b/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/Redshift/test_measure_constraint__plan0_optimized.sql index 0b555b1430..3c0c641be9 100644 --- a/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/Redshift/test_measure_constraint__plan0_optimized.sql +++ b/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/Redshift/test_measure_constraint__plan0_optimized.sql @@ -3,7 +3,7 @@ SELECT metric_time__day , average_booking_value * bookings / NULLIF(booking_value, 0) AS lux_booking_value_rate_expr FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT COALESCE(subq_41.metric_time__day, subq_53.metric_time__day, subq_58.metric_time__day) AS metric_time__day , MAX(subq_41.average_booking_value) AS average_booking_value diff --git a/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/Redshift/test_measure_constraint_with_reused_measure__plan0.sql b/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/Redshift/test_measure_constraint_with_reused_measure__plan0.sql index 3ffd151578..dc10e020b0 100644 --- a/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/Redshift/test_measure_constraint_with_reused_measure__plan0.sql +++ b/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/Redshift/test_measure_constraint_with_reused_measure__plan0.sql @@ -3,7 +3,7 @@ SELECT subq_12.metric_time__day , CAST(subq_12.booking_value_with_is_instant_constraint AS DOUBLE PRECISION) / CAST(NULLIF(subq_12.booking_value, 0) AS DOUBLE PRECISION) AS instant_booking_value_ratio FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT COALESCE(subq_6.metric_time__day, subq_11.metric_time__day) AS metric_time__day , MAX(subq_6.booking_value_with_is_instant_constraint) AS booking_value_with_is_instant_constraint diff --git a/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/Redshift/test_measure_constraint_with_reused_measure__plan0_optimized.sql b/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/Redshift/test_measure_constraint_with_reused_measure__plan0_optimized.sql index dca28f5735..9d1c4deb21 100644 --- a/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/Redshift/test_measure_constraint_with_reused_measure__plan0_optimized.sql +++ b/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/Redshift/test_measure_constraint_with_reused_measure__plan0_optimized.sql @@ -3,7 +3,7 @@ SELECT metric_time__day , CAST(booking_value_with_is_instant_constraint AS DOUBLE PRECISION) / CAST(NULLIF(booking_value, 0) AS DOUBLE PRECISION) AS instant_booking_value_ratio FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT COALESCE(subq_19.metric_time__day, subq_24.metric_time__day) AS metric_time__day , MAX(subq_19.booking_value_with_is_instant_constraint) AS booking_value_with_is_instant_constraint diff --git a/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/Redshift/test_metric_with_measures_from_multiple_sources_no_dimensions__plan0.sql b/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/Redshift/test_metric_with_measures_from_multiple_sources_no_dimensions__plan0.sql index 45490517f3..5791199c8f 100644 --- a/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/Redshift/test_metric_with_measures_from_multiple_sources_no_dimensions__plan0.sql +++ b/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/Redshift/test_metric_with_measures_from_multiple_sources_no_dimensions__plan0.sql @@ -2,7 +2,7 @@ SELECT CAST(subq_10.bookings AS DOUBLE PRECISION) / CAST(NULLIF(subq_10.listings, 0) AS DOUBLE PRECISION) AS bookings_per_listing FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT MAX(subq_4.bookings) AS bookings , MAX(subq_9.listings) AS listings diff --git a/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/Redshift/test_metric_with_measures_from_multiple_sources_no_dimensions__plan0_optimized.sql b/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/Redshift/test_metric_with_measures_from_multiple_sources_no_dimensions__plan0_optimized.sql index 31bce3d483..c48f7d8d21 100644 --- a/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/Redshift/test_metric_with_measures_from_multiple_sources_no_dimensions__plan0_optimized.sql +++ b/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/Redshift/test_metric_with_measures_from_multiple_sources_no_dimensions__plan0_optimized.sql @@ -1,4 +1,4 @@ --- Combine Metrics +-- Combine Aggregated Outputs -- Compute Metrics via Expressions SELECT CAST(MAX(subq_15.bookings) AS DOUBLE PRECISION) / CAST(NULLIF(MAX(subq_20.listings), 0) AS DOUBLE PRECISION) AS bookings_per_listing diff --git a/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/Redshift/test_multiple_metrics_no_dimensions__plan0.sql b/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/Redshift/test_multiple_metrics_no_dimensions__plan0.sql index e4d19bf2a1..b245b7e6dc 100644 --- a/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/Redshift/test_multiple_metrics_no_dimensions__plan0.sql +++ b/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/Redshift/test_multiple_metrics_no_dimensions__plan0.sql @@ -1,4 +1,4 @@ --- Combine Metrics +-- Combine Aggregated Outputs SELECT MAX(subq_5.bookings) AS bookings , MAX(subq_11.listings) AS listings diff --git a/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/Redshift/test_multiple_metrics_no_dimensions__plan0_optimized.sql b/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/Redshift/test_multiple_metrics_no_dimensions__plan0_optimized.sql index 56554460d5..631546edf7 100644 --- a/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/Redshift/test_multiple_metrics_no_dimensions__plan0_optimized.sql +++ b/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/Redshift/test_multiple_metrics_no_dimensions__plan0_optimized.sql @@ -1,4 +1,4 @@ --- Combine Metrics +-- Combine Aggregated Outputs SELECT MAX(subq_17.bookings) AS bookings , MAX(subq_23.listings) AS listings diff --git a/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/Snowflake/test_common_semantic_model__plan0.sql b/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/Snowflake/test_common_semantic_model__plan0.sql index d67f49d085..25e23610de 100644 --- a/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/Snowflake/test_common_semantic_model__plan0.sql +++ b/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/Snowflake/test_common_semantic_model__plan0.sql @@ -1,4 +1,4 @@ --- Combine Metrics +-- Combine Aggregated Outputs SELECT COALESCE(subq_4.metric_time__day, subq_9.metric_time__day) AS metric_time__day , MAX(subq_4.bookings) AS bookings diff --git a/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/Snowflake/test_common_semantic_model__plan0_optimized.sql b/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/Snowflake/test_common_semantic_model__plan0_optimized.sql index 552b4dc805..d2cebbcf57 100644 --- a/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/Snowflake/test_common_semantic_model__plan0_optimized.sql +++ b/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/Snowflake/test_common_semantic_model__plan0_optimized.sql @@ -1,4 +1,4 @@ --- Combine Metrics +-- Combine Aggregated Outputs SELECT COALESCE(subq_14.metric_time__day, subq_19.metric_time__day) AS metric_time__day , MAX(subq_14.bookings) AS bookings diff --git a/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/Snowflake/test_measure_constraint__plan0.sql b/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/Snowflake/test_measure_constraint__plan0.sql index d5276887ae..df76282051 100644 --- a/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/Snowflake/test_measure_constraint__plan0.sql +++ b/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/Snowflake/test_measure_constraint__plan0.sql @@ -3,7 +3,7 @@ SELECT subq_29.metric_time__day , average_booking_value * bookings / NULLIF(booking_value, 0) AS lux_booking_value_rate_expr FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT COALESCE(subq_11.metric_time__day, subq_23.metric_time__day, subq_28.metric_time__day) AS metric_time__day , MAX(subq_11.average_booking_value) AS average_booking_value diff --git a/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/Snowflake/test_measure_constraint__plan0_optimized.sql b/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/Snowflake/test_measure_constraint__plan0_optimized.sql index 0b555b1430..3c0c641be9 100644 --- a/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/Snowflake/test_measure_constraint__plan0_optimized.sql +++ b/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/Snowflake/test_measure_constraint__plan0_optimized.sql @@ -3,7 +3,7 @@ SELECT metric_time__day , average_booking_value * bookings / NULLIF(booking_value, 0) AS lux_booking_value_rate_expr FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT COALESCE(subq_41.metric_time__day, subq_53.metric_time__day, subq_58.metric_time__day) AS metric_time__day , MAX(subq_41.average_booking_value) AS average_booking_value diff --git a/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/Snowflake/test_measure_constraint_with_reused_measure__plan0.sql b/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/Snowflake/test_measure_constraint_with_reused_measure__plan0.sql index 92e948277e..afb1ec0710 100644 --- a/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/Snowflake/test_measure_constraint_with_reused_measure__plan0.sql +++ b/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/Snowflake/test_measure_constraint_with_reused_measure__plan0.sql @@ -3,7 +3,7 @@ SELECT subq_12.metric_time__day , CAST(subq_12.booking_value_with_is_instant_constraint AS DOUBLE) / CAST(NULLIF(subq_12.booking_value, 0) AS DOUBLE) AS instant_booking_value_ratio FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT COALESCE(subq_6.metric_time__day, subq_11.metric_time__day) AS metric_time__day , MAX(subq_6.booking_value_with_is_instant_constraint) AS booking_value_with_is_instant_constraint diff --git a/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/Snowflake/test_measure_constraint_with_reused_measure__plan0_optimized.sql b/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/Snowflake/test_measure_constraint_with_reused_measure__plan0_optimized.sql index 77f5d693df..2d2b5a142c 100644 --- a/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/Snowflake/test_measure_constraint_with_reused_measure__plan0_optimized.sql +++ b/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/Snowflake/test_measure_constraint_with_reused_measure__plan0_optimized.sql @@ -3,7 +3,7 @@ SELECT metric_time__day , CAST(booking_value_with_is_instant_constraint AS DOUBLE) / CAST(NULLIF(booking_value, 0) AS DOUBLE) AS instant_booking_value_ratio FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT COALESCE(subq_19.metric_time__day, subq_24.metric_time__day) AS metric_time__day , MAX(subq_19.booking_value_with_is_instant_constraint) AS booking_value_with_is_instant_constraint diff --git a/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/Snowflake/test_metric_with_measures_from_multiple_sources_no_dimensions__plan0.sql b/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/Snowflake/test_metric_with_measures_from_multiple_sources_no_dimensions__plan0.sql index b7e32bf2f2..11a93dd97b 100644 --- a/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/Snowflake/test_metric_with_measures_from_multiple_sources_no_dimensions__plan0.sql +++ b/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/Snowflake/test_metric_with_measures_from_multiple_sources_no_dimensions__plan0.sql @@ -2,7 +2,7 @@ SELECT CAST(subq_10.bookings AS DOUBLE) / CAST(NULLIF(subq_10.listings, 0) AS DOUBLE) AS bookings_per_listing FROM ( - -- Combine Metrics + -- Combine Aggregated Outputs SELECT MAX(subq_4.bookings) AS bookings , MAX(subq_9.listings) AS listings diff --git a/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/Snowflake/test_metric_with_measures_from_multiple_sources_no_dimensions__plan0_optimized.sql b/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/Snowflake/test_metric_with_measures_from_multiple_sources_no_dimensions__plan0_optimized.sql index 8a1096e90f..bd5b5c24ee 100644 --- a/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/Snowflake/test_metric_with_measures_from_multiple_sources_no_dimensions__plan0_optimized.sql +++ b/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/Snowflake/test_metric_with_measures_from_multiple_sources_no_dimensions__plan0_optimized.sql @@ -1,4 +1,4 @@ --- Combine Metrics +-- Combine Aggregated Outputs -- Compute Metrics via Expressions SELECT CAST(MAX(subq_15.bookings) AS DOUBLE) / CAST(NULLIF(MAX(subq_20.listings), 0) AS DOUBLE) AS bookings_per_listing diff --git a/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/Snowflake/test_multiple_metrics_no_dimensions__plan0.sql b/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/Snowflake/test_multiple_metrics_no_dimensions__plan0.sql index 125313d7bb..ceda439945 100644 --- a/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/Snowflake/test_multiple_metrics_no_dimensions__plan0.sql +++ b/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/Snowflake/test_multiple_metrics_no_dimensions__plan0.sql @@ -1,4 +1,4 @@ --- Combine Metrics +-- Combine Aggregated Outputs SELECT MAX(subq_5.bookings) AS bookings , MAX(subq_11.listings) AS listings diff --git a/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/Snowflake/test_multiple_metrics_no_dimensions__plan0_optimized.sql b/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/Snowflake/test_multiple_metrics_no_dimensions__plan0_optimized.sql index 56554460d5..631546edf7 100644 --- a/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/Snowflake/test_multiple_metrics_no_dimensions__plan0_optimized.sql +++ b/metricflow/test/snapshots/test_query_rendering.py/SqlQueryPlan/Snowflake/test_multiple_metrics_no_dimensions__plan0_optimized.sql @@ -1,4 +1,4 @@ --- Combine Metrics +-- Combine Aggregated Outputs SELECT MAX(subq_17.bookings) AS bookings , MAX(subq_23.listings) AS listings diff --git a/metricflow/test/snapshots/test_source_scan_optimizer.py/DataflowPlan/test_2_metrics_from_1_semantic_model__dfp_0.xml b/metricflow/test/snapshots/test_source_scan_optimizer.py/DataflowPlan/test_2_metrics_from_1_semantic_model__dfp_0.xml index 1f786abff0..a22f6531b8 100644 --- a/metricflow/test/snapshots/test_source_scan_optimizer.py/DataflowPlan/test_2_metrics_from_1_semantic_model__dfp_0.xml +++ b/metricflow/test/snapshots/test_source_scan_optimizer.py/DataflowPlan/test_2_metrics_from_1_semantic_model__dfp_0.xml @@ -2,9 +2,9 @@ - - - + + + @@ -26,7 +26,8 @@ - + + @@ -60,7 +61,8 @@ - + + @@ -130,7 +132,8 @@ - + + @@ -164,7 +167,8 @@ - + + @@ -213,6 +217,6 @@ - + diff --git a/metricflow/test/snapshots/test_source_scan_optimizer.py/DataflowPlan/test_2_metrics_from_1_semantic_model__dfpo_0.xml b/metricflow/test/snapshots/test_source_scan_optimizer.py/DataflowPlan/test_2_metrics_from_1_semantic_model__dfpo_0.xml index 427c764d62..7f887a661a 100644 --- a/metricflow/test/snapshots/test_source_scan_optimizer.py/DataflowPlan/test_2_metrics_from_1_semantic_model__dfpo_0.xml +++ b/metricflow/test/snapshots/test_source_scan_optimizer.py/DataflowPlan/test_2_metrics_from_1_semantic_model__dfpo_0.xml @@ -30,11 +30,13 @@ - + + - + + @@ -68,11 +70,13 @@ - + + - + + diff --git a/metricflow/test/snapshots/test_source_scan_optimizer.py/DataflowPlan/test_2_metrics_from_2_semantic_models__dfp_0.xml b/metricflow/test/snapshots/test_source_scan_optimizer.py/DataflowPlan/test_2_metrics_from_2_semantic_models__dfp_0.xml index 4349bbc86d..85e26dc807 100644 --- a/metricflow/test/snapshots/test_source_scan_optimizer.py/DataflowPlan/test_2_metrics_from_2_semantic_models__dfp_0.xml +++ b/metricflow/test/snapshots/test_source_scan_optimizer.py/DataflowPlan/test_2_metrics_from_2_semantic_models__dfp_0.xml @@ -2,9 +2,9 @@ - - - + + + @@ -26,7 +26,8 @@ - + + @@ -71,7 +72,8 @@ - + + @@ -95,6 +97,6 @@ - + diff --git a/metricflow/test/snapshots/test_source_scan_optimizer.py/DataflowPlan/test_2_metrics_from_2_semantic_models__dfpo_0.xml b/metricflow/test/snapshots/test_source_scan_optimizer.py/DataflowPlan/test_2_metrics_from_2_semantic_models__dfpo_0.xml index ccbea73132..84c9e7f792 100644 --- a/metricflow/test/snapshots/test_source_scan_optimizer.py/DataflowPlan/test_2_metrics_from_2_semantic_models__dfpo_0.xml +++ b/metricflow/test/snapshots/test_source_scan_optimizer.py/DataflowPlan/test_2_metrics_from_2_semantic_models__dfpo_0.xml @@ -2,9 +2,9 @@ - - - + + + @@ -26,7 +26,8 @@ - + + @@ -71,7 +72,8 @@ - + + @@ -95,6 +97,6 @@ - + diff --git a/metricflow/test/snapshots/test_source_scan_optimizer.py/DataflowPlan/test_2_ratio_metrics_from_1_semantic_model__dfp_0.xml b/metricflow/test/snapshots/test_source_scan_optimizer.py/DataflowPlan/test_2_ratio_metrics_from_1_semantic_model__dfp_0.xml index b70b76fb45..fc92635b40 100644 --- a/metricflow/test/snapshots/test_source_scan_optimizer.py/DataflowPlan/test_2_ratio_metrics_from_1_semantic_model__dfp_0.xml +++ b/metricflow/test/snapshots/test_source_scan_optimizer.py/DataflowPlan/test_2_ratio_metrics_from_1_semantic_model__dfp_0.xml @@ -2,9 +2,9 @@ - - - + + + @@ -15,9 +15,9 @@ - - - + + + @@ -39,7 +39,8 @@ - + + @@ -84,7 +85,8 @@ - + + @@ -108,7 +110,7 @@ - + @@ -120,9 +122,9 @@ - - - + + + @@ -144,7 +146,8 @@ - + + @@ -189,7 +192,8 @@ - + + @@ -213,8 +217,8 @@ - + - + diff --git a/metricflow/test/snapshots/test_source_scan_optimizer.py/DataflowPlan/test_2_ratio_metrics_from_1_semantic_model__dfpo_0.xml b/metricflow/test/snapshots/test_source_scan_optimizer.py/DataflowPlan/test_2_ratio_metrics_from_1_semantic_model__dfpo_0.xml index 443bab0657..a25a07eaac 100644 --- a/metricflow/test/snapshots/test_source_scan_optimizer.py/DataflowPlan/test_2_ratio_metrics_from_1_semantic_model__dfpo_0.xml +++ b/metricflow/test/snapshots/test_source_scan_optimizer.py/DataflowPlan/test_2_ratio_metrics_from_1_semantic_model__dfpo_0.xml @@ -61,15 +61,18 @@ - + + - + + - + + diff --git a/metricflow/test/snapshots/test_source_scan_optimizer.py/DataflowPlan/test_3_metrics_from_2_semantic_models__dfp_0.xml b/metricflow/test/snapshots/test_source_scan_optimizer.py/DataflowPlan/test_3_metrics_from_2_semantic_models__dfp_0.xml index c577d49dac..b4deae8ae0 100644 --- a/metricflow/test/snapshots/test_source_scan_optimizer.py/DataflowPlan/test_3_metrics_from_2_semantic_models__dfp_0.xml +++ b/metricflow/test/snapshots/test_source_scan_optimizer.py/DataflowPlan/test_3_metrics_from_2_semantic_models__dfp_0.xml @@ -2,9 +2,9 @@ - - - + + + @@ -26,7 +26,8 @@ - + + @@ -71,7 +72,8 @@ - + + @@ -116,7 +118,8 @@ - + + @@ -140,6 +143,6 @@ - + diff --git a/metricflow/test/snapshots/test_source_scan_optimizer.py/DataflowPlan/test_3_metrics_from_2_semantic_models__dfpo_0.xml b/metricflow/test/snapshots/test_source_scan_optimizer.py/DataflowPlan/test_3_metrics_from_2_semantic_models__dfpo_0.xml index 4d3f2c07ad..cebfc1c9bb 100644 --- a/metricflow/test/snapshots/test_source_scan_optimizer.py/DataflowPlan/test_3_metrics_from_2_semantic_models__dfpo_0.xml +++ b/metricflow/test/snapshots/test_source_scan_optimizer.py/DataflowPlan/test_3_metrics_from_2_semantic_models__dfpo_0.xml @@ -2,9 +2,9 @@ - - - + + + @@ -33,11 +33,13 @@ - + + - + + @@ -82,7 +84,8 @@ - + + @@ -106,6 +109,6 @@ - + diff --git a/metricflow/test/snapshots/test_source_scan_optimizer.py/DataflowPlan/test_constrained_metric_not_combined__dfp_0.xml b/metricflow/test/snapshots/test_source_scan_optimizer.py/DataflowPlan/test_constrained_metric_not_combined__dfp_0.xml index 75c50ed00d..c418de8aab 100644 --- a/metricflow/test/snapshots/test_source_scan_optimizer.py/DataflowPlan/test_constrained_metric_not_combined__dfp_0.xml +++ b/metricflow/test/snapshots/test_source_scan_optimizer.py/DataflowPlan/test_constrained_metric_not_combined__dfp_0.xml @@ -2,9 +2,9 @@ - - - + + + @@ -26,7 +26,8 @@ - + + @@ -81,7 +82,8 @@ - + + @@ -112,7 +114,8 @@ - + + @@ -142,6 +145,6 @@ - + diff --git a/metricflow/test/snapshots/test_source_scan_optimizer.py/DataflowPlan/test_constrained_metric_not_combined__dfpo_0.xml b/metricflow/test/snapshots/test_source_scan_optimizer.py/DataflowPlan/test_constrained_metric_not_combined__dfpo_0.xml index 92b366667f..ff7f2e31d2 100644 --- a/metricflow/test/snapshots/test_source_scan_optimizer.py/DataflowPlan/test_constrained_metric_not_combined__dfpo_0.xml +++ b/metricflow/test/snapshots/test_source_scan_optimizer.py/DataflowPlan/test_constrained_metric_not_combined__dfpo_0.xml @@ -2,9 +2,9 @@ - - - + + + @@ -26,7 +26,8 @@ - + + @@ -81,7 +82,8 @@ - + + @@ -112,7 +114,8 @@ - + + @@ -142,6 +145,6 @@ - + diff --git a/metricflow/test/snapshots/test_source_scan_optimizer.py/DataflowPlan/test_derived_metric__dfp_0.xml b/metricflow/test/snapshots/test_source_scan_optimizer.py/DataflowPlan/test_derived_metric__dfp_0.xml index 29526bca43..03b6f242b8 100644 --- a/metricflow/test/snapshots/test_source_scan_optimizer.py/DataflowPlan/test_derived_metric__dfp_0.xml +++ b/metricflow/test/snapshots/test_source_scan_optimizer.py/DataflowPlan/test_derived_metric__dfp_0.xml @@ -12,9 +12,9 @@ - - - + + + @@ -36,7 +36,8 @@ - + + @@ -81,7 +82,8 @@ - + + @@ -105,7 +107,7 @@ - + diff --git a/metricflow/test/snapshots/test_source_scan_optimizer.py/DataflowPlan/test_derived_metric__dfpo_0.xml b/metricflow/test/snapshots/test_source_scan_optimizer.py/DataflowPlan/test_derived_metric__dfpo_0.xml index aa3dfef91a..03772b6ebe 100644 --- a/metricflow/test/snapshots/test_source_scan_optimizer.py/DataflowPlan/test_derived_metric__dfpo_0.xml +++ b/metricflow/test/snapshots/test_source_scan_optimizer.py/DataflowPlan/test_derived_metric__dfpo_0.xml @@ -40,11 +40,13 @@ - + + - + + diff --git a/metricflow/test/snapshots/test_source_scan_optimizer.py/DataflowPlan/test_derived_metric_with_non_derived_metric__dfp_0.xml b/metricflow/test/snapshots/test_source_scan_optimizer.py/DataflowPlan/test_derived_metric_with_non_derived_metric__dfp_0.xml index f9e04cae8b..ca91324ad9 100644 --- a/metricflow/test/snapshots/test_source_scan_optimizer.py/DataflowPlan/test_derived_metric_with_non_derived_metric__dfp_0.xml +++ b/metricflow/test/snapshots/test_source_scan_optimizer.py/DataflowPlan/test_derived_metric_with_non_derived_metric__dfp_0.xml @@ -2,9 +2,9 @@ - - - + + + @@ -26,7 +26,8 @@ - + + @@ -60,9 +61,9 @@ - - - + + + @@ -84,7 +85,8 @@ - + + @@ -129,7 +131,8 @@ - + + @@ -153,8 +156,8 @@ - + - + diff --git a/metricflow/test/snapshots/test_source_scan_optimizer.py/DataflowPlan/test_derived_metric_with_non_derived_metric__dfpo_0.xml b/metricflow/test/snapshots/test_source_scan_optimizer.py/DataflowPlan/test_derived_metric_with_non_derived_metric__dfpo_0.xml index 95b7f60916..9168186317 100644 --- a/metricflow/test/snapshots/test_source_scan_optimizer.py/DataflowPlan/test_derived_metric_with_non_derived_metric__dfpo_0.xml +++ b/metricflow/test/snapshots/test_source_scan_optimizer.py/DataflowPlan/test_derived_metric_with_non_derived_metric__dfpo_0.xml @@ -2,9 +2,9 @@ - - - + + + @@ -26,7 +26,8 @@ - + + @@ -88,11 +89,13 @@ - + + - + + @@ -117,6 +120,6 @@ - + diff --git a/metricflow/test/snapshots/test_source_scan_optimizer.py/DataflowPlan/test_nested_derived_metric__dfp_0.xml b/metricflow/test/snapshots/test_source_scan_optimizer.py/DataflowPlan/test_nested_derived_metric__dfp_0.xml index 1c4e8d0e42..79228ef5a8 100644 --- a/metricflow/test/snapshots/test_source_scan_optimizer.py/DataflowPlan/test_nested_derived_metric__dfp_0.xml +++ b/metricflow/test/snapshots/test_source_scan_optimizer.py/DataflowPlan/test_nested_derived_metric__dfp_0.xml @@ -12,9 +12,9 @@ - - - + + + @@ -25,9 +25,9 @@ - - - + + + @@ -49,7 +49,8 @@ - + + @@ -94,7 +95,8 @@ - + + @@ -118,7 +120,7 @@ - + @@ -141,7 +143,8 @@ - + + @@ -186,7 +189,8 @@ - + + @@ -210,7 +214,7 @@ - + diff --git a/metricflow/test/snapshots/test_source_scan_optimizer.py/DataflowPlan/test_nested_derived_metric__dfpo_0.xml b/metricflow/test/snapshots/test_source_scan_optimizer.py/DataflowPlan/test_nested_derived_metric__dfpo_0.xml index 09232c4bd3..3566eb57fb 100644 --- a/metricflow/test/snapshots/test_source_scan_optimizer.py/DataflowPlan/test_nested_derived_metric__dfpo_0.xml +++ b/metricflow/test/snapshots/test_source_scan_optimizer.py/DataflowPlan/test_nested_derived_metric__dfpo_0.xml @@ -12,9 +12,9 @@ - - - + + + @@ -53,11 +53,13 @@ - + + - + + @@ -110,11 +112,13 @@ - + + - + + @@ -138,7 +142,7 @@ - +