diff --git a/Makefile b/Makefile index fd74fd79d3..519be637eb 100644 --- a/Makefile +++ b/Makefile @@ -98,3 +98,11 @@ populate-persistent-source-schemas: .PHONY: test-snap test-snap: make test ADDITIONAL_PYTEST_OPTIONS=--overwrite-snapshots + +.PHONY: testx +testx: + make test ADDITIONAL_PYTEST_OPTIONS=-x + +.PHONY: testx-snap +testx-snap: + make test ADDITIONAL_PYTEST_OPTIONS='-x --overwrite-snapshots' diff --git a/metricflow-semantics/metricflow_semantics/instances.py b/metricflow-semantics/metricflow_semantics/instances.py index 155fa3667c..779a2c879e 100644 --- a/metricflow-semantics/metricflow_semantics/instances.py +++ b/metricflow-semantics/metricflow_semantics/instances.py @@ -3,14 +3,15 @@ from __future__ import annotations from abc import ABC, abstractmethod -from dataclasses import dataclass -from typing import Generic, List, Tuple, TypeVar +from dataclasses import dataclass, field +from typing import Generic, List, Sequence, Tuple, TypeVar from dbt_semantic_interfaces.dataclass_serialization import SerializableDataclass -from dbt_semantic_interfaces.references import MetricModelReference, SemanticModelElementReference +from dbt_semantic_interfaces.references import EntityReference, MetricModelReference, SemanticModelElementReference +from typing_extensions import override from metricflow_semantics.aggregation_properties import AggregationState -from metricflow_semantics.specs.column_assoc import ColumnAssociation +from metricflow_semantics.specs.column_assoc import ColumnAssociation, ColumnAssociationResolver from metricflow_semantics.specs.dimension_spec import DimensionSpec from metricflow_semantics.specs.entity_spec import EntitySpec from metricflow_semantics.specs.group_by_metric_spec import GroupByMetricSpec @@ -20,6 +21,7 @@ from metricflow_semantics.specs.metric_spec import MetricSpec from metricflow_semantics.specs.spec_set import InstanceSpecSet from metricflow_semantics.specs.time_dimension_spec import TimeDimensionSpec +from metricflow_semantics.visitor import VisitorOutputT # Type for the specification used in the instance. SpecT = TypeVar("SpecT", bound=InstanceSpec) @@ -38,6 +40,8 @@ class MdoInstance(ABC, Generic[SpecT]): """ # The columns associated with this instance. + # TODO: if poss, remove this and instead add a method that resolves this from the spec + column association resolver + # (ensure we're using consistent logic everywhere so this bug doesn't happen again) associated_columns: Tuple[ColumnAssociation, ...] # The spec that describes this instance. spec: SpecT @@ -48,6 +52,20 @@ def associated_column(self) -> ColumnAssociation: assert len(self.associated_columns) == 1 return self.associated_columns[0] + def accept(self, visitor: InstanceVisitor[VisitorOutputT]) -> VisitorOutputT: + """See Visitable.""" + raise NotImplementedError() + + +class LinkableInstance(MdoInstance, Generic[SpecT]): + """An MdoInstance whose spec is linkable (i.e., it can have entity links).""" + + def with_entity_prefix( + self, entity_prefix: EntityReference, column_association_resolver: ColumnAssociationResolver + ) -> MdoInstance: + """Add entity link to the underlying spec and associated column.""" + raise NotImplementedError() + # Instances for the major metric object types @@ -82,31 +100,90 @@ class MeasureInstance(MdoInstance[MeasureSpec], SemanticModelElementInstance): spec: MeasureSpec aggregation_state: AggregationState + def accept(self, visitor: InstanceVisitor[VisitorOutputT]) -> VisitorOutputT: # noqa: D102 + return visitor.visit_measure_instance(self) + @dataclass(frozen=True) -class DimensionInstance(MdoInstance[DimensionSpec], SemanticModelElementInstance): # noqa: D101 +class DimensionInstance(LinkableInstance[DimensionSpec], SemanticModelElementInstance): # noqa: D101 associated_columns: Tuple[ColumnAssociation, ...] spec: DimensionSpec + def accept(self, visitor: InstanceVisitor[VisitorOutputT]) -> VisitorOutputT: # noqa: D102 + return visitor.visit_dimension_instance(self) + + def with_entity_prefix( + self, entity_prefix: EntityReference, column_association_resolver: ColumnAssociationResolver + ) -> DimensionInstance: + """Returns a new instance with the entity prefix added to the entity links.""" + transformed_spec = self.spec.with_entity_prefix(entity_prefix) + return DimensionInstance( + associated_columns=(column_association_resolver.resolve_spec(transformed_spec),), + defined_from=self.defined_from, + spec=transformed_spec, + ) + @dataclass(frozen=True) -class TimeDimensionInstance(MdoInstance[TimeDimensionSpec], SemanticModelElementInstance): # noqa: D101 +class TimeDimensionInstance(LinkableInstance[TimeDimensionSpec], SemanticModelElementInstance): # noqa: D101 associated_columns: Tuple[ColumnAssociation, ...] spec: TimeDimensionSpec + def accept(self, visitor: InstanceVisitor[VisitorOutputT]) -> VisitorOutputT: # noqa: D102 + return visitor.visit_time_dimension_instance(self) + + def with_entity_prefix( + self, entity_prefix: EntityReference, column_association_resolver: ColumnAssociationResolver + ) -> TimeDimensionInstance: + """Returns a new instance with the entity prefix added to the entity links.""" + transformed_spec = self.spec.with_entity_prefix(entity_prefix) + return TimeDimensionInstance( + associated_columns=(column_association_resolver.resolve_spec(transformed_spec),), + defined_from=self.defined_from, + spec=transformed_spec, + ) + @dataclass(frozen=True) -class EntityInstance(MdoInstance[EntitySpec], SemanticModelElementInstance): # noqa: D101 +class EntityInstance(LinkableInstance[EntitySpec], SemanticModelElementInstance): # noqa: D101 associated_columns: Tuple[ColumnAssociation, ...] spec: EntitySpec + def accept(self, visitor: InstanceVisitor[VisitorOutputT]) -> VisitorOutputT: # noqa: D102 + return visitor.visit_entity_instance(self) + + def with_entity_prefix( + self, entity_prefix: EntityReference, column_association_resolver: ColumnAssociationResolver + ) -> EntityInstance: + """Returns a new instance with the entity prefix added to the entity links.""" + transformed_spec = self.spec.with_entity_prefix(entity_prefix) + return EntityInstance( + associated_columns=(column_association_resolver.resolve_spec(transformed_spec),), + defined_from=self.defined_from, + spec=transformed_spec, + ) + @dataclass(frozen=True) -class GroupByMetricInstance(MdoInstance[GroupByMetricSpec], SerializableDataclass): # noqa: D101 +class GroupByMetricInstance(LinkableInstance[GroupByMetricSpec], SerializableDataclass): # noqa: D101 associated_columns: Tuple[ColumnAssociation, ...] spec: GroupByMetricSpec defined_from: MetricModelReference + def accept(self, visitor: InstanceVisitor[VisitorOutputT]) -> VisitorOutputT: # noqa: D102 + return visitor.visit_group_by_metric_instance(self) + + def with_entity_prefix( + self, entity_prefix: EntityReference, column_association_resolver: ColumnAssociationResolver + ) -> GroupByMetricInstance: + """Returns a new instance with the entity prefix added to the entity links.""" + transformed_spec = self.spec.with_entity_prefix(entity_prefix) + return GroupByMetricInstance( + associated_columns=(column_association_resolver.resolve_spec(transformed_spec),), + defined_from=self.defined_from, + spec=transformed_spec, + ) + @dataclass(frozen=True) class MetricInstance(MdoInstance[MetricSpec], SerializableDataclass): # noqa: D101 @@ -114,12 +191,18 @@ class MetricInstance(MdoInstance[MetricSpec], SerializableDataclass): # noqa: D spec: MetricSpec defined_from: MetricModelReference + def accept(self, visitor: InstanceVisitor[VisitorOutputT]) -> VisitorOutputT: # noqa: D102 + return visitor.visit_metric_instance(self) + @dataclass(frozen=True) class MetadataInstance(MdoInstance[MetadataSpec], SerializableDataclass): # noqa: D101 associated_columns: Tuple[ColumnAssociation, ...] spec: MetadataSpec + def accept(self, visitor: InstanceVisitor[VisitorOutputT]) -> VisitorOutputT: # noqa: D102 + return visitor.visit_metadata_instance(self) + # Output type of transform function TransformOutputT = TypeVar("TransformOutputT") @@ -226,3 +309,106 @@ def as_tuple(self) -> Tuple[MdoInstance, ...]: # noqa: D102 + self.metric_instances + self.metadata_instances ) + + @property + def linkable_instances(self) -> Tuple[LinkableInstance, ...]: # noqa: D102 + return ( + self.dimension_instances + + self.time_dimension_instances + + self.entity_instances + + self.group_by_metric_instances + ) + + +class InstanceVisitor(Generic[VisitorOutputT], ABC): + """Visitor for the Instance classes.""" + + @abstractmethod + def visit_measure_instance(self, measure_instance: MeasureInstance) -> VisitorOutputT: # noqa: D102 + raise NotImplementedError + + @abstractmethod + def visit_dimension_instance(self, dimension_instance: DimensionInstance) -> VisitorOutputT: # noqa: D102 + raise NotImplementedError + + @abstractmethod + def visit_time_dimension_instance( # noqa: D102 + self, time_dimension_instance: TimeDimensionInstance + ) -> VisitorOutputT: + raise NotImplementedError + + @abstractmethod + def visit_entity_instance(self, entity_instance: EntityInstance) -> VisitorOutputT: # noqa: D102 + raise NotImplementedError + + @abstractmethod + def visit_group_by_metric_instance( # noqa: D102 + self, group_by_metric_instance: GroupByMetricInstance + ) -> VisitorOutputT: + raise NotImplementedError + + @abstractmethod + def visit_metric_instance(self, metric_instance: MetricInstance) -> VisitorOutputT: # noqa: D102 + raise NotImplementedError + + @abstractmethod + def visit_metadata_instance(self, metadata_instance: MetadataInstance) -> VisitorOutputT: # noqa: D102 + raise NotImplementedError + + +@dataclass +class _GroupInstanceByTypeVisitor(InstanceVisitor[None]): + """Group instances by type into an `InstanceSet`.""" + + metric_instances: List[MetricInstance] = field(default_factory=list) + measure_instances: List[MeasureInstance] = field(default_factory=list) + dimension_instances: List[DimensionInstance] = field(default_factory=list) + entity_instances: List[EntityInstance] = field(default_factory=list) + time_dimension_instances: List[TimeDimensionInstance] = field(default_factory=list) + group_by_metric_instances: List[GroupByMetricInstance] = field(default_factory=list) + metadata_instances: List[MetadataInstance] = field(default_factory=list) + + @override + def visit_measure_instance(self, measure_instance: MeasureInstance) -> None: + self.measure_instances.append(measure_instance) + + @override + def visit_dimension_instance(self, dimension_instance: DimensionInstance) -> None: + self.dimension_instances.append(dimension_instance) + + @override + def visit_time_dimension_instance(self, time_dimension_instance: TimeDimensionInstance) -> None: + self.time_dimension_instances.append(time_dimension_instance) + + @override + def visit_entity_instance(self, entity_instance: EntityInstance) -> None: + self.entity_instances.append(entity_instance) + + @override + def visit_group_by_metric_instance(self, group_by_metric_instance: GroupByMetricInstance) -> None: + self.group_by_metric_instances.append(group_by_metric_instance) + + @override + def visit_metric_instance(self, metric_instance: MetricInstance) -> None: + self.metric_instances.append(metric_instance) + + @override + def visit_metadata_instance(self, metadata_instance: MetadataInstance) -> None: + self.metadata_instances.append(metadata_instance) + + +def group_instances_by_type(instances: Sequence[MdoInstance]) -> InstanceSet: + """Groups a sequence of instances by type.""" + grouper = _GroupInstanceByTypeVisitor() + for instance in instances: + instance.accept(grouper) + + return InstanceSet( + metric_instances=tuple(grouper.metric_instances), + measure_instances=tuple(grouper.measure_instances), + dimension_instances=tuple(grouper.dimension_instances), + entity_instances=tuple(grouper.entity_instances), + time_dimension_instances=tuple(grouper.time_dimension_instances), + group_by_metric_instances=tuple(grouper.group_by_metric_instances), + metadata_instances=tuple(grouper.metadata_instances), + ) diff --git a/metricflow-semantics/metricflow_semantics/specs/dimension_spec.py b/metricflow-semantics/metricflow_semantics/specs/dimension_spec.py index 7b20a8c2c9..43b4b6a010 100644 --- a/metricflow-semantics/metricflow_semantics/specs/dimension_spec.py +++ b/metricflow-semantics/metricflow_semantics/specs/dimension_spec.py @@ -43,3 +43,6 @@ def element_path_key(self) -> ElementPathKey: return ElementPathKey( element_name=self.element_name, element_type=LinkableElementType.DIMENSION, entity_links=self.entity_links ) + + def with_entity_prefix(self, entity_prefix: EntityReference) -> DimensionSpec: # noqa: D102 + return DimensionSpec(element_name=self.element_name, entity_links=(entity_prefix,) + self.entity_links) diff --git a/metricflow-semantics/metricflow_semantics/specs/entity_spec.py b/metricflow-semantics/metricflow_semantics/specs/entity_spec.py index 18100e8551..0ffeb81dc5 100644 --- a/metricflow-semantics/metricflow_semantics/specs/entity_spec.py +++ b/metricflow-semantics/metricflow_semantics/specs/entity_spec.py @@ -53,6 +53,9 @@ def element_path_key(self) -> ElementPathKey: element_name=self.element_name, element_type=LinkableElementType.ENTITY, entity_links=self.entity_links ) + def with_entity_prefix(self, entity_prefix: EntityReference) -> EntitySpec: # noqa: D102 + return EntitySpec(element_name=self.element_name, entity_links=(entity_prefix,) + self.entity_links) + @dataclass(frozen=True) class LinklessEntitySpec(EntitySpec, SerializableDataclass): diff --git a/metricflow-semantics/metricflow_semantics/specs/group_by_metric_spec.py b/metricflow-semantics/metricflow_semantics/specs/group_by_metric_spec.py index bfb6d7629a..1fbbc6562a 100644 --- a/metricflow-semantics/metricflow_semantics/specs/group_by_metric_spec.py +++ b/metricflow-semantics/metricflow_semantics/specs/group_by_metric_spec.py @@ -111,3 +111,10 @@ def element_path_key(self) -> ElementPathKey: return ElementPathKey( element_name=self.element_name, element_type=LinkableElementType.METRIC, entity_links=self.entity_links ) + + def with_entity_prefix(self, entity_prefix: EntityReference) -> GroupByMetricSpec: # noqa: D102 + return GroupByMetricSpec( + element_name=self.element_name, + entity_links=(entity_prefix,) + self.entity_links, + metric_subquery_entity_links=self.metric_subquery_entity_links, + ) diff --git a/metricflow-semantics/metricflow_semantics/specs/instance_spec.py b/metricflow-semantics/metricflow_semantics/specs/instance_spec.py index 65d36c9aba..9cb04b3e8f 100644 --- a/metricflow-semantics/metricflow_semantics/specs/instance_spec.py +++ b/metricflow-semantics/metricflow_semantics/specs/instance_spec.py @@ -136,5 +136,10 @@ def element_path_key(self) -> ElementPathKey: """Return the ElementPathKey representation of the LinkableInstanceSpec subtype.""" raise NotImplementedError() + @abstractmethod + def with_entity_prefix(self, entity_prefix: EntityReference) -> LinkableInstanceSpec: + """Add the selected entity prefix to the start of the entity links.""" + raise NotImplementedError() + SelfTypeT = TypeVar("SelfTypeT", bound="LinkableInstanceSpec") diff --git a/metricflow-semantics/metricflow_semantics/specs/time_dimension_spec.py b/metricflow-semantics/metricflow_semantics/specs/time_dimension_spec.py index 48f8b3f7eb..5dc90f842d 100644 --- a/metricflow-semantics/metricflow_semantics/specs/time_dimension_spec.py +++ b/metricflow-semantics/metricflow_semantics/specs/time_dimension_spec.py @@ -235,3 +235,12 @@ def generate_possible_specs_for_time_dimension( @property def is_metric_time(self) -> bool: # noqa: D102 return self.element_name == METRIC_TIME_ELEMENT_NAME + + def with_entity_prefix(self, entity_prefix: EntityReference) -> TimeDimensionSpec: # noqa: D102 + return TimeDimensionSpec( + element_name=self.element_name, + entity_links=(entity_prefix,) + self.entity_links, + time_granularity=self.time_granularity, + date_part=self.date_part, + aggregation_state=self.aggregation_state, + ) diff --git a/metricflow/dataflow/builder/dataflow_plan_builder.py b/metricflow/dataflow/builder/dataflow_plan_builder.py index 507a566525..1bb7159a28 100644 --- a/metricflow/dataflow/builder/dataflow_plan_builder.py +++ b/metricflow/dataflow/builder/dataflow_plan_builder.py @@ -221,12 +221,12 @@ def _build_plan( def _optimize_plan(self, plan: DataflowPlan, optimizations: FrozenSet[DataflowPlanOptimization]) -> DataflowPlan: optimizer_factory = DataflowPlanOptimizerFactory(self._node_data_set_resolver) for optimizer in optimizer_factory.get_optimizers(optimizations): - logger.debug(LazyFormat(lambda: f"Applying {optimizer.__class__.__name__}")) + logger.debug(LazyFormat(lambda: f"Applying optimizer: {optimizer.__class__.__name__}")) try: plan = optimizer.optimize(plan) logger.debug( LazyFormat( - lambda: f"After applying {optimizer.__class__.__name__}, the dataflow plan is:\n" + lambda: f"After applying optimizer {optimizer.__class__.__name__}, the dataflow plan is:\n" f"{indent(plan.structure_text())}" ) ) diff --git a/metricflow/plan_conversion/dataflow_to_sql.py b/metricflow/plan_conversion/dataflow_to_sql.py index c2016fa1b5..8fc3465a40 100644 --- a/metricflow/plan_conversion/dataflow_to_sql.py +++ b/metricflow/plan_conversion/dataflow_to_sql.py @@ -25,13 +25,12 @@ MetadataInstance, MetricInstance, TimeDimensionInstance, + group_instances_by_type, ) from metricflow_semantics.mf_logging.formatting import indent from metricflow_semantics.mf_logging.lazy_formattable import LazyFormat from metricflow_semantics.model.semantic_manifest_lookup import SemanticManifestLookup -from metricflow_semantics.specs.column_assoc import ( - ColumnAssociationResolver, -) +from metricflow_semantics.specs.column_assoc import ColumnAssociationResolver from metricflow_semantics.specs.group_by_metric_spec import GroupByMetricSpec from metricflow_semantics.specs.instance_spec import InstanceSpec from metricflow_semantics.specs.measure_spec import MeasureSpec @@ -72,7 +71,6 @@ from metricflow.plan_conversion.convert_to_sql_plan import ConvertToSqlPlanResult from metricflow.plan_conversion.instance_converters import ( AddGroupByMetric, - AddLinkToLinkableElements, AddMetadata, AddMetrics, AliasAggregatedMeasures, @@ -89,7 +87,7 @@ RemoveMeasures, RemoveMetrics, UpdateMeasureFillNullsWith, - create_select_columns_for_instance_sets, + create_simple_select_columns_for_instance_sets, ) from metricflow.plan_conversion.select_column_gen import ( SelectColumnSet, @@ -229,7 +227,7 @@ def convert_to_sql_query_plan( sql_node = optimizer.optimize(sql_node) logger.debug( LazyFormat( - lambda: f"After applying {optimizer.__class__.__name__}, the SQL query plan is:\n" + lambda: f"After applying optimizer {optimizer.__class__.__name__}, the SQL query plan is:\n" f"{indent(sql_node.structure_text())}" ) ) @@ -347,7 +345,7 @@ def _make_time_spine_data_set( complete_outer_where_filter = where_constraint_exprs[0] outer_query_output_instance_set = InstanceSet(time_dimension_instances=agg_time_dimension_instances) - outer_query_select_columns = create_select_columns_for_instance_sets( + outer_query_select_columns = create_simple_select_columns_for_instance_sets( column_resolver=self._column_association_resolver, table_alias_to_instance_set=OrderedDict({inner_query_alias: outer_query_output_instance_set}), ) @@ -446,7 +444,7 @@ def visit_join_over_time_range_node(self, node: JoinOverTimeRangeNode) -> SqlDat instance_set=output_instance_set, sql_select_node=SqlSelectStatementNode.create( description=node.description, - select_columns=create_select_columns_for_instance_sets( + select_columns=create_simple_select_columns_for_instance_sets( self._column_association_resolver, table_alias_to_instance_set ), from_source=time_spine_data_set.checked_sql_select_node, @@ -456,31 +454,40 @@ def visit_join_over_time_range_node(self, node: JoinOverTimeRangeNode) -> SqlDat ) def visit_join_on_entities_node(self, node: JoinOnEntitiesNode) -> SqlDataSet: - """Generates the query that realizes the behavior of the JoinToStandardOutputNode.""" - # Keep a mapping between the table aliases that would be used in the query and the MDO instances in that source. - # e.g. when building "FROM from_table a JOIN right_table b", the value for key "a" would be the instances in - # "from_table" - table_alias_to_instance_set: OrderedDict[str, InstanceSet] = OrderedDict() - - # Convert the dataflow from the left node to a DataSet and add context for it to table_alias_to_instance_set - # A DataSet is a bundle of the SQL query (in object form) and the MDO instances that the SQL query contains. + """Generates the query that realizes the behavior of the JoinOnEntitiesNode.""" from_data_set = node.left_node.accept(self) from_data_set_alias = self._next_unique_table_alias() - table_alias_to_instance_set[from_data_set_alias] = from_data_set.instance_set - # Build the join descriptions for the SqlQueryPlan - different from node.join_descriptions which are the join - # descriptions from the dataflow plan. - sql_join_descs: List[SqlJoinDescription] = [] + # Change the aggregation state for the measures to be partially aggregated if it was previously aggregated + # since we removed the entities and added the dimensions. The dimensions could have the same value for + # multiple rows, so we'll need to re-aggregate. + from_data_set_output_instance_set = from_data_set.instance_set.transform( + # TODO: is this filter doing anything? seems like no? + FilterElements(include_specs=from_data_set.instance_set.spec_set) + ).transform( + ChangeMeasureAggregationState( + { + AggregationState.NON_AGGREGATED: AggregationState.NON_AGGREGATED, + AggregationState.COMPLETE: AggregationState.PARTIAL, + AggregationState.PARTIAL: AggregationState.PARTIAL, + } + ) + ) + instances_to_build_simple_select_columns_for = OrderedDict( + {from_data_set_alias: from_data_set_output_instance_set} + ) - # The dataflow plan describes how the data sets coming from the parent nodes should be joined together. Use - # those descriptions to convert them to join descriptions for the SQL query plan. + # Build SQL join description, instance set, and select columns for each join target. + output_instance_set = from_data_set_output_instance_set + select_columns: Tuple[SqlSelectColumn, ...] = () + sql_join_descs: List[SqlJoinDescription] = [] for join_description in node.join_targets: join_on_entity = join_description.join_on_entity - - right_node_to_join: DataflowPlanNode = join_description.join_node + right_node_to_join = join_description.join_node right_data_set: SqlDataSet = right_node_to_join.accept(self) right_data_set_alias = self._next_unique_table_alias() + # Build join description. sql_join_desc = SqlQueryPlanJoinBuilder.make_base_output_join_description( left_data_set=AnnotatedSqlDataSet(data_set=from_data_set, alias=from_data_set_alias), right_data_set=AnnotatedSqlDataSet(data_set=right_data_set, alias=right_data_set_alias), @@ -489,56 +496,46 @@ def visit_join_on_entities_node(self, node: JoinOnEntitiesNode) -> SqlDataSet: sql_join_descs.append(sql_join_desc) if join_on_entity: - # Remove the linkable instances with the join_on_entity as the leading link as the next step adds the - # link. This is to avoid cases where there is a primary entity and a dimension in the data set, and we - # create an instance in the next step that has the same entity link. - # e.g. a data set has the dimension "listing__country_latest" and "listing" is a primary entity in the - # data set. The next step would create an instance like "listing__listing__country_latest" without this - # filter. - right_data_set_instance_set_filtered = FilterLinkableInstancesWithLeadingLink( - entity_link=join_on_entity, + # Remove any instances that already have the join_on_entity as the leading link. This will prevent a duplicate + # entity link when we add it in the next step. + right_instance_set_filtered = FilterLinkableInstancesWithLeadingLink( + join_on_entity.reference ).transform(right_data_set.instance_set) - # After the right data set is joined to the "from" data set, we need to change the links for some of the - # instances that represent the right data set. For example, if the "from" data set contains the "bookings" - # measure instance and the right dataset contains the "country" dimension instance, then after the join, - # the output data set should have the "country" dimension instance with the "user_id" entity link - # (if "user_id" equality was the join condition). "country" -> "user_id__country" - right_data_set_instance_set_after_join = right_data_set_instance_set_filtered.transform( - AddLinkToLinkableElements(join_on_entity=join_on_entity) - ) + # After the right data set is joined, update the entity links to indicate that joining on the entity was + # required to reach the spec. If the "country" dimension was joined and "user_id" is the join_on_entity, + # then the joined data set should have the "user__country" dimension. + new_instances: Tuple[MdoInstance, ...] = () + for original_instance in right_instance_set_filtered.linkable_instances: + new_instance = original_instance.with_entity_prefix( + join_on_entity.reference, column_association_resolver=self._column_association_resolver + ) + # Build new select column using the old column name as the expr and the new column name as the alias. + select_column = SqlSelectColumn( + expr=SqlColumnReferenceExpression.from_table_and_column_names( + table_alias=right_data_set_alias, + column_name=original_instance.associated_column.column_name, + ), + column_alias=new_instance.associated_column.column_name, + ) + new_instances += (new_instance,) + select_columns += (select_column,) + right_instance_set_after_join = group_instances_by_type(new_instances) else: - right_data_set_instance_set_after_join = right_data_set.instance_set - table_alias_to_instance_set[right_data_set_alias] = right_data_set_instance_set_after_join + right_instance_set_after_join = right_data_set.instance_set + instances_to_build_simple_select_columns_for[right_data_set_alias] = right_instance_set_after_join - from_data_set_output_instance_set = from_data_set.instance_set.transform( - FilterElements(include_specs=from_data_set.instance_set.spec_set) - ) + output_instance_set = InstanceSet.merge([output_instance_set, right_instance_set_after_join]) - # Change the aggregation state for the measures to be partially aggregated if it was previously aggregated - # since we removed the entities and added the dimensions. The dimensions could have the same value for - # multiple rows, so we'll need to re-aggregate. - from_data_set_output_instance_set = from_data_set_output_instance_set.transform( - ChangeMeasureAggregationState( - { - AggregationState.NON_AGGREGATED: AggregationState.NON_AGGREGATED, - AggregationState.COMPLETE: AggregationState.PARTIAL, - AggregationState.PARTIAL: AggregationState.PARTIAL, - } - ) + select_columns += create_simple_select_columns_for_instance_sets( + column_resolver=self._column_association_resolver, + table_alias_to_instance_set=instances_to_build_simple_select_columns_for, ) - - table_alias_to_instance_set[from_data_set_alias] = from_data_set_output_instance_set - - # Construct the data set that contains the updated instances and the SQL nodes that should go in the various - # clauses. return SqlDataSet( - instance_set=InstanceSet.merge(list(table_alias_to_instance_set.values())), + instance_set=output_instance_set, sql_select_node=SqlSelectStatementNode.create( description=node.description, - select_columns=create_select_columns_for_instance_sets( - self._column_association_resolver, table_alias_to_instance_set - ), + select_columns=select_columns, from_source=from_data_set.checked_sql_select_node, from_source_alias=from_data_set_alias, join_descs=tuple(sql_join_descs), @@ -1385,7 +1382,7 @@ def visit_join_to_time_spine_node(self, node: JoinToTimeSpineNode) -> SqlDataSet metric_instances=parent_data_set.instance_set.metric_instances, metadata_instances=parent_data_set.instance_set.metadata_instances, ) - parent_select_columns = create_select_columns_for_instance_sets( + parent_select_columns = create_simple_select_columns_for_instance_sets( self._column_association_resolver, OrderedDict({parent_alias: parent_instance_set}) ) diff --git a/metricflow/plan_conversion/instance_converters.py b/metricflow/plan_conversion/instance_converters.py index d49457a637..c6e25835b5 100644 --- a/metricflow/plan_conversion/instance_converters.py +++ b/metricflow/plan_conversion/instance_converters.py @@ -29,13 +29,9 @@ from metricflow_semantics.model.semantics.metric_lookup import MetricLookup from metricflow_semantics.model.semantics.semantic_model_lookup import SemanticModelLookup from metricflow_semantics.specs.column_assoc import ColumnAssociationResolver -from metricflow_semantics.specs.dimension_spec import DimensionSpec -from metricflow_semantics.specs.entity_spec import EntitySpec, LinklessEntitySpec -from metricflow_semantics.specs.group_by_metric_spec import GroupByMetricSpec from metricflow_semantics.specs.instance_spec import InstanceSpec, LinkableInstanceSpec from metricflow_semantics.specs.measure_spec import MeasureSpec, MetricInputMeasureSpec from metricflow_semantics.specs.spec_set import InstanceSpecSet -from metricflow_semantics.specs.time_dimension_spec import TimeDimensionSpec from more_itertools import bucket from metricflow.dataflow.nodes.join_to_base import ValidityWindowJoinDescription @@ -387,125 +383,20 @@ def transform(self, instance_set: InstanceSet) -> Optional[ValidityWindowJoinDes return None -class AddLinkToLinkableElements(InstanceSetTransform[InstanceSet]): - """Return a new instance set where the all linkable elements in the set have a new link added. - - e.g. "country" -> "user_id__country" after a data set has been joined by entity. - """ - - def __init__(self, join_on_entity: LinklessEntitySpec) -> None: # noqa: D107 - self._join_on_entity = join_on_entity - - def transform(self, instance_set: InstanceSet) -> InstanceSet: # noqa: D102 - assert len(instance_set.metric_instances) == 0, "Can't add links to instance sets with metrics" - assert len(instance_set.measure_instances) == 0, "Can't add links to instance sets with measures" - - # Handle dimension instances - dimension_instances_with_additional_link = [] - for dimension_instance in instance_set.dimension_instances: - # The new dimension spec should include the join on entity. - transformed_dimension_spec_from_right = DimensionSpec( - element_name=dimension_instance.spec.element_name, - entity_links=self._join_on_entity.as_linkless_prefix + dimension_instance.spec.entity_links, - ) - dimension_instances_with_additional_link.append( - DimensionInstance( - associated_columns=dimension_instance.associated_columns, - defined_from=dimension_instance.defined_from, - spec=transformed_dimension_spec_from_right, - ) - ) - - # Handle time dimension instances - time_dimension_instances_with_additional_link = [] - for time_dimension_instance in instance_set.time_dimension_instances: - # The new dimension spec should include the join on entity. - transformed_time_dimension_spec_from_right = TimeDimensionSpec( - element_name=time_dimension_instance.spec.element_name, - entity_links=( - (EntityReference(element_name=self._join_on_entity.element_name),) - + time_dimension_instance.spec.entity_links - ), - time_granularity=time_dimension_instance.spec.time_granularity, - date_part=time_dimension_instance.spec.date_part, - ) - time_dimension_instances_with_additional_link.append( - TimeDimensionInstance( - associated_columns=time_dimension_instance.associated_columns, - defined_from=time_dimension_instance.defined_from, - spec=transformed_time_dimension_spec_from_right, - ) - ) - - # Handle entity instances - entity_instances_with_additional_link = [] - for entity_instance in instance_set.entity_instances: - # Don't include adding the entity link to the same entity. - # Otherwise, you would create "user_id__user_id", which is confusing. - if entity_instance.spec == self._join_on_entity: - continue - # The new entity spec should include the join on entity. - transformed_entity_spec_from_right = EntitySpec( - element_name=entity_instance.spec.element_name, - entity_links=self._join_on_entity.as_linkless_prefix + entity_instance.spec.entity_links, - ) - entity_instances_with_additional_link.append( - EntityInstance( - associated_columns=entity_instance.associated_columns, - defined_from=entity_instance.defined_from, - spec=transformed_entity_spec_from_right, - ) - ) - - # Handle group by metric instances - group_by_metric_instances_with_additional_link = [] - for group_by_metric_instance in instance_set.group_by_metric_instances: - transformed_group_by_metric_spec_from_right = GroupByMetricSpec( - element_name=group_by_metric_instance.spec.element_name, - entity_links=self._join_on_entity.as_linkless_prefix + group_by_metric_instance.spec.entity_links, - metric_subquery_entity_links=group_by_metric_instance.spec.metric_subquery_entity_links, - ) - group_by_metric_instances_with_additional_link.append( - GroupByMetricInstance( - associated_columns=group_by_metric_instance.associated_columns, - defined_from=group_by_metric_instance.defined_from, - spec=transformed_group_by_metric_spec_from_right, - ) - ) - - return InstanceSet( - measure_instances=(), - dimension_instances=tuple(dimension_instances_with_additional_link), - time_dimension_instances=tuple(time_dimension_instances_with_additional_link), - entity_instances=tuple(entity_instances_with_additional_link), - group_by_metric_instances=tuple(group_by_metric_instances_with_additional_link), - metric_instances=(), - metadata_instances=(), - ) - - class FilterLinkableInstancesWithLeadingLink(InstanceSetTransform[InstanceSet]): """Return an instance set with the elements that have a specified leading link removed. e.g. Remove "listing__country" if the specified link is "listing". """ - def __init__( - self, - entity_link: LinklessEntitySpec, - ) -> None: - """Constructor. - - Args: - entity_link: Remove elements with this link as the first element in "entity_links" - """ + def __init__(self, entity_link: EntityReference) -> None: + """Remove elements with this link as the first element in "entity_links".""" self._entity_link = entity_link def _should_pass(self, linkable_spec: LinkableInstanceSpec) -> bool: - return ( - len(linkable_spec.entity_links) == 0 - or LinklessEntitySpec.from_reference(linkable_spec.entity_links[0]) != self._entity_link - ) + if len(linkable_spec.entity_links) == 0: + return not linkable_spec.reference == self._entity_link + return linkable_spec.entity_links[0] != self._entity_link def transform(self, instance_set: InstanceSet) -> InstanceSet: # noqa: D102 # Normal to not filter anything if the instance set has no instances with links. @@ -1042,7 +933,7 @@ def transform(self, instance_set: InstanceSet) -> InstanceSet: # noqa: D102 ) -def create_select_columns_for_instance_sets( +def create_simple_select_columns_for_instance_sets( column_resolver: ColumnAssociationResolver, table_alias_to_instance_set: OrderedDict[str, InstanceSet], ) -> Tuple[SqlSelectColumn, ...]: diff --git a/metricflow/sql/optimizer/column_pruner.py b/metricflow/sql/optimizer/column_pruner.py index 79e6d8250a..96b0e6e52e 100644 --- a/metricflow/sql/optimizer/column_pruner.py +++ b/metricflow/sql/optimizer/column_pruner.py @@ -133,7 +133,13 @@ def visit_select_statement_node(self, node: SqlSelectStatementNode) -> SqlQueryP # SqlExpressionNode. if len(pruned_select_columns) == 0: - raise RuntimeError("All columns have been pruned - this indicates an bug in the pruner or in the inputs.") + raise RuntimeError( + "All columns have been removed - this indicates an bug in the pruner or in the inputs.\n" + f"Original column aliases: {[col.column_alias for col in node.select_columns]}\n" + f"Required column aliases: {self._required_column_aliases}\n" + f"Group bys: {node.group_bys}\n" + f"Distinct: {node.distinct}" + ) # Based on the expressions in this select statement, figure out what column aliases are needed in the sources of # this query (i.e. tables or sub-queries in the FROM or JOIN clauses). diff --git a/tests_metricflow/snapshots/test_custom_granularity.py/SqlQueryPlan/BigQuery/test_simple_metric_with_multi_hop_custom_granularity__plan0.sql b/tests_metricflow/snapshots/test_custom_granularity.py/SqlQueryPlan/BigQuery/test_simple_metric_with_multi_hop_custom_granularity__plan0.sql index 8df1df5378..36a7279e19 100644 --- a/tests_metricflow/snapshots/test_custom_granularity.py/SqlQueryPlan/BigQuery/test_simple_metric_with_multi_hop_custom_granularity__plan0.sql +++ b/tests_metricflow/snapshots/test_custom_granularity.py/SqlQueryPlan/BigQuery/test_simple_metric_with_multi_hop_custom_granularity__plan0.sql @@ -22,9 +22,9 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.ds_partitioned__day AS ds_partitioned__day - , subq_9.user__ds__day AS listing__user__ds__day + subq_9.user__ds__day AS listing__user__ds__day , subq_9.user__ds_partitioned__day AS listing__user__ds_partitioned__day + , subq_2.ds_partitioned__day AS ds_partitioned__day , subq_2.listing AS listing , subq_2.bookings AS bookings FROM ( @@ -238,61 +238,7 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_4.ds__day AS ds__day - , subq_4.ds__week AS ds__week - , subq_4.ds__month AS ds__month - , subq_4.ds__quarter AS ds__quarter - , subq_4.ds__year AS ds__year - , subq_4.ds__extract_year AS ds__extract_year - , subq_4.ds__extract_quarter AS ds__extract_quarter - , subq_4.ds__extract_month AS ds__extract_month - , subq_4.ds__extract_day AS ds__extract_day - , subq_4.ds__extract_dow AS ds__extract_dow - , subq_4.ds__extract_doy AS ds__extract_doy - , subq_4.created_at__day AS created_at__day - , subq_4.created_at__week AS created_at__week - , subq_4.created_at__month AS created_at__month - , subq_4.created_at__quarter AS created_at__quarter - , subq_4.created_at__year AS created_at__year - , subq_4.created_at__extract_year AS created_at__extract_year - , subq_4.created_at__extract_quarter AS created_at__extract_quarter - , subq_4.created_at__extract_month AS created_at__extract_month - , subq_4.created_at__extract_day AS created_at__extract_day - , subq_4.created_at__extract_dow AS created_at__extract_dow - , subq_4.created_at__extract_doy AS created_at__extract_doy - , subq_4.listing__ds__day AS listing__ds__day - , subq_4.listing__ds__week AS listing__ds__week - , subq_4.listing__ds__month AS listing__ds__month - , subq_4.listing__ds__quarter AS listing__ds__quarter - , subq_4.listing__ds__year AS listing__ds__year - , subq_4.listing__ds__extract_year AS listing__ds__extract_year - , subq_4.listing__ds__extract_quarter AS listing__ds__extract_quarter - , subq_4.listing__ds__extract_month AS listing__ds__extract_month - , subq_4.listing__ds__extract_day AS listing__ds__extract_day - , subq_4.listing__ds__extract_dow AS listing__ds__extract_dow - , subq_4.listing__ds__extract_doy AS listing__ds__extract_doy - , subq_4.listing__created_at__day AS listing__created_at__day - , subq_4.listing__created_at__week AS listing__created_at__week - , subq_4.listing__created_at__month AS listing__created_at__month - , subq_4.listing__created_at__quarter AS listing__created_at__quarter - , subq_4.listing__created_at__year AS listing__created_at__year - , subq_4.listing__created_at__extract_year AS listing__created_at__extract_year - , subq_4.listing__created_at__extract_quarter AS listing__created_at__extract_quarter - , subq_4.listing__created_at__extract_month AS listing__created_at__extract_month - , subq_4.listing__created_at__extract_day AS listing__created_at__extract_day - , subq_4.listing__created_at__extract_dow AS listing__created_at__extract_dow - , subq_4.listing__created_at__extract_doy AS listing__created_at__extract_doy - , subq_4.metric_time__day AS metric_time__day - , subq_4.metric_time__week AS metric_time__week - , subq_4.metric_time__month AS metric_time__month - , subq_4.metric_time__quarter AS metric_time__quarter - , subq_4.metric_time__year AS metric_time__year - , subq_4.metric_time__extract_year AS metric_time__extract_year - , subq_4.metric_time__extract_quarter AS metric_time__extract_quarter - , subq_4.metric_time__extract_month AS metric_time__extract_month - , subq_4.metric_time__extract_day AS metric_time__extract_day - , subq_4.metric_time__extract_dow AS metric_time__extract_dow - , subq_4.metric_time__extract_doy AS metric_time__extract_doy + subq_7.home_state AS user__home_state , subq_7.ds__day AS user__ds__day , subq_7.ds__week AS user__ds__week , subq_7.ds__month AS user__ds__month @@ -391,6 +337,61 @@ FROM ( , subq_7.metric_time__extract_day AS user__metric_time__extract_day , subq_7.metric_time__extract_dow AS user__metric_time__extract_dow , subq_7.metric_time__extract_doy AS user__metric_time__extract_doy + , subq_4.ds__day AS ds__day + , subq_4.ds__week AS ds__week + , subq_4.ds__month AS ds__month + , subq_4.ds__quarter AS ds__quarter + , subq_4.ds__year AS ds__year + , subq_4.ds__extract_year AS ds__extract_year + , subq_4.ds__extract_quarter AS ds__extract_quarter + , subq_4.ds__extract_month AS ds__extract_month + , subq_4.ds__extract_day AS ds__extract_day + , subq_4.ds__extract_dow AS ds__extract_dow + , subq_4.ds__extract_doy AS ds__extract_doy + , subq_4.created_at__day AS created_at__day + , subq_4.created_at__week AS created_at__week + , subq_4.created_at__month AS created_at__month + , subq_4.created_at__quarter AS created_at__quarter + , subq_4.created_at__year AS created_at__year + , subq_4.created_at__extract_year AS created_at__extract_year + , subq_4.created_at__extract_quarter AS created_at__extract_quarter + , subq_4.created_at__extract_month AS created_at__extract_month + , subq_4.created_at__extract_day AS created_at__extract_day + , subq_4.created_at__extract_dow AS created_at__extract_dow + , subq_4.created_at__extract_doy AS created_at__extract_doy + , subq_4.listing__ds__day AS listing__ds__day + , subq_4.listing__ds__week AS listing__ds__week + , subq_4.listing__ds__month AS listing__ds__month + , subq_4.listing__ds__quarter AS listing__ds__quarter + , subq_4.listing__ds__year AS listing__ds__year + , subq_4.listing__ds__extract_year AS listing__ds__extract_year + , subq_4.listing__ds__extract_quarter AS listing__ds__extract_quarter + , subq_4.listing__ds__extract_month AS listing__ds__extract_month + , subq_4.listing__ds__extract_day AS listing__ds__extract_day + , subq_4.listing__ds__extract_dow AS listing__ds__extract_dow + , subq_4.listing__ds__extract_doy AS listing__ds__extract_doy + , subq_4.listing__created_at__day AS listing__created_at__day + , subq_4.listing__created_at__week AS listing__created_at__week + , subq_4.listing__created_at__month AS listing__created_at__month + , subq_4.listing__created_at__quarter AS listing__created_at__quarter + , subq_4.listing__created_at__year AS listing__created_at__year + , subq_4.listing__created_at__extract_year AS listing__created_at__extract_year + , subq_4.listing__created_at__extract_quarter AS listing__created_at__extract_quarter + , subq_4.listing__created_at__extract_month AS listing__created_at__extract_month + , subq_4.listing__created_at__extract_day AS listing__created_at__extract_day + , subq_4.listing__created_at__extract_dow AS listing__created_at__extract_dow + , subq_4.listing__created_at__extract_doy AS listing__created_at__extract_doy + , subq_4.metric_time__day AS metric_time__day + , subq_4.metric_time__week AS metric_time__week + , subq_4.metric_time__month AS metric_time__month + , subq_4.metric_time__quarter AS metric_time__quarter + , subq_4.metric_time__year AS metric_time__year + , subq_4.metric_time__extract_year AS metric_time__extract_year + , subq_4.metric_time__extract_quarter AS metric_time__extract_quarter + , subq_4.metric_time__extract_month AS metric_time__extract_month + , subq_4.metric_time__extract_day AS metric_time__extract_day + , subq_4.metric_time__extract_dow AS metric_time__extract_dow + , subq_4.metric_time__extract_doy AS metric_time__extract_doy , subq_4.listing AS listing , subq_4.user AS user , subq_4.listing__user AS listing__user @@ -400,7 +401,6 @@ FROM ( , subq_4.listing__country_latest AS listing__country_latest , subq_4.listing__is_lux_latest AS listing__is_lux_latest , subq_4.listing__capacity_latest AS listing__capacity_latest - , subq_7.home_state AS user__home_state , subq_4.listings AS listings , subq_4.largest_listing AS largest_listing , subq_4.smallest_listing AS smallest_listing diff --git a/tests_metricflow/snapshots/test_custom_granularity.py/SqlQueryPlan/Databricks/test_simple_metric_with_multi_hop_custom_granularity__plan0.sql b/tests_metricflow/snapshots/test_custom_granularity.py/SqlQueryPlan/Databricks/test_simple_metric_with_multi_hop_custom_granularity__plan0.sql index 8af1554119..32c775ba75 100644 --- a/tests_metricflow/snapshots/test_custom_granularity.py/SqlQueryPlan/Databricks/test_simple_metric_with_multi_hop_custom_granularity__plan0.sql +++ b/tests_metricflow/snapshots/test_custom_granularity.py/SqlQueryPlan/Databricks/test_simple_metric_with_multi_hop_custom_granularity__plan0.sql @@ -22,9 +22,9 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.ds_partitioned__day AS ds_partitioned__day - , subq_9.user__ds__day AS listing__user__ds__day + subq_9.user__ds__day AS listing__user__ds__day , subq_9.user__ds_partitioned__day AS listing__user__ds_partitioned__day + , subq_2.ds_partitioned__day AS ds_partitioned__day , subq_2.listing AS listing , subq_2.bookings AS bookings FROM ( @@ -238,61 +238,7 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_4.ds__day AS ds__day - , subq_4.ds__week AS ds__week - , subq_4.ds__month AS ds__month - , subq_4.ds__quarter AS ds__quarter - , subq_4.ds__year AS ds__year - , subq_4.ds__extract_year AS ds__extract_year - , subq_4.ds__extract_quarter AS ds__extract_quarter - , subq_4.ds__extract_month AS ds__extract_month - , subq_4.ds__extract_day AS ds__extract_day - , subq_4.ds__extract_dow AS ds__extract_dow - , subq_4.ds__extract_doy AS ds__extract_doy - , subq_4.created_at__day AS created_at__day - , subq_4.created_at__week AS created_at__week - , subq_4.created_at__month AS created_at__month - , subq_4.created_at__quarter AS created_at__quarter - , subq_4.created_at__year AS created_at__year - , subq_4.created_at__extract_year AS created_at__extract_year - , subq_4.created_at__extract_quarter AS created_at__extract_quarter - , subq_4.created_at__extract_month AS created_at__extract_month - , subq_4.created_at__extract_day AS created_at__extract_day - , subq_4.created_at__extract_dow AS created_at__extract_dow - , subq_4.created_at__extract_doy AS created_at__extract_doy - , subq_4.listing__ds__day AS listing__ds__day - , subq_4.listing__ds__week AS listing__ds__week - , subq_4.listing__ds__month AS listing__ds__month - , subq_4.listing__ds__quarter AS listing__ds__quarter - , subq_4.listing__ds__year AS listing__ds__year - , subq_4.listing__ds__extract_year AS listing__ds__extract_year - , subq_4.listing__ds__extract_quarter AS listing__ds__extract_quarter - , subq_4.listing__ds__extract_month AS listing__ds__extract_month - , subq_4.listing__ds__extract_day AS listing__ds__extract_day - , subq_4.listing__ds__extract_dow AS listing__ds__extract_dow - , subq_4.listing__ds__extract_doy AS listing__ds__extract_doy - , subq_4.listing__created_at__day AS listing__created_at__day - , subq_4.listing__created_at__week AS listing__created_at__week - , subq_4.listing__created_at__month AS listing__created_at__month - , subq_4.listing__created_at__quarter AS listing__created_at__quarter - , subq_4.listing__created_at__year AS listing__created_at__year - , subq_4.listing__created_at__extract_year AS listing__created_at__extract_year - , subq_4.listing__created_at__extract_quarter AS listing__created_at__extract_quarter - , subq_4.listing__created_at__extract_month AS listing__created_at__extract_month - , subq_4.listing__created_at__extract_day AS listing__created_at__extract_day - , subq_4.listing__created_at__extract_dow AS listing__created_at__extract_dow - , subq_4.listing__created_at__extract_doy AS listing__created_at__extract_doy - , subq_4.metric_time__day AS metric_time__day - , subq_4.metric_time__week AS metric_time__week - , subq_4.metric_time__month AS metric_time__month - , subq_4.metric_time__quarter AS metric_time__quarter - , subq_4.metric_time__year AS metric_time__year - , subq_4.metric_time__extract_year AS metric_time__extract_year - , subq_4.metric_time__extract_quarter AS metric_time__extract_quarter - , subq_4.metric_time__extract_month AS metric_time__extract_month - , subq_4.metric_time__extract_day AS metric_time__extract_day - , subq_4.metric_time__extract_dow AS metric_time__extract_dow - , subq_4.metric_time__extract_doy AS metric_time__extract_doy + subq_7.home_state AS user__home_state , subq_7.ds__day AS user__ds__day , subq_7.ds__week AS user__ds__week , subq_7.ds__month AS user__ds__month @@ -391,6 +337,61 @@ FROM ( , subq_7.metric_time__extract_day AS user__metric_time__extract_day , subq_7.metric_time__extract_dow AS user__metric_time__extract_dow , subq_7.metric_time__extract_doy AS user__metric_time__extract_doy + , subq_4.ds__day AS ds__day + , subq_4.ds__week AS ds__week + , subq_4.ds__month AS ds__month + , subq_4.ds__quarter AS ds__quarter + , subq_4.ds__year AS ds__year + , subq_4.ds__extract_year AS ds__extract_year + , subq_4.ds__extract_quarter AS ds__extract_quarter + , subq_4.ds__extract_month AS ds__extract_month + , subq_4.ds__extract_day AS ds__extract_day + , subq_4.ds__extract_dow AS ds__extract_dow + , subq_4.ds__extract_doy AS ds__extract_doy + , subq_4.created_at__day AS created_at__day + , subq_4.created_at__week AS created_at__week + , subq_4.created_at__month AS created_at__month + , subq_4.created_at__quarter AS created_at__quarter + , subq_4.created_at__year AS created_at__year + , subq_4.created_at__extract_year AS created_at__extract_year + , subq_4.created_at__extract_quarter AS created_at__extract_quarter + , subq_4.created_at__extract_month AS created_at__extract_month + , subq_4.created_at__extract_day AS created_at__extract_day + , subq_4.created_at__extract_dow AS created_at__extract_dow + , subq_4.created_at__extract_doy AS created_at__extract_doy + , subq_4.listing__ds__day AS listing__ds__day + , subq_4.listing__ds__week AS listing__ds__week + , subq_4.listing__ds__month AS listing__ds__month + , subq_4.listing__ds__quarter AS listing__ds__quarter + , subq_4.listing__ds__year AS listing__ds__year + , subq_4.listing__ds__extract_year AS listing__ds__extract_year + , subq_4.listing__ds__extract_quarter AS listing__ds__extract_quarter + , subq_4.listing__ds__extract_month AS listing__ds__extract_month + , subq_4.listing__ds__extract_day AS listing__ds__extract_day + , subq_4.listing__ds__extract_dow AS listing__ds__extract_dow + , subq_4.listing__ds__extract_doy AS listing__ds__extract_doy + , subq_4.listing__created_at__day AS listing__created_at__day + , subq_4.listing__created_at__week AS listing__created_at__week + , subq_4.listing__created_at__month AS listing__created_at__month + , subq_4.listing__created_at__quarter AS listing__created_at__quarter + , subq_4.listing__created_at__year AS listing__created_at__year + , subq_4.listing__created_at__extract_year AS listing__created_at__extract_year + , subq_4.listing__created_at__extract_quarter AS listing__created_at__extract_quarter + , subq_4.listing__created_at__extract_month AS listing__created_at__extract_month + , subq_4.listing__created_at__extract_day AS listing__created_at__extract_day + , subq_4.listing__created_at__extract_dow AS listing__created_at__extract_dow + , subq_4.listing__created_at__extract_doy AS listing__created_at__extract_doy + , subq_4.metric_time__day AS metric_time__day + , subq_4.metric_time__week AS metric_time__week + , subq_4.metric_time__month AS metric_time__month + , subq_4.metric_time__quarter AS metric_time__quarter + , subq_4.metric_time__year AS metric_time__year + , subq_4.metric_time__extract_year AS metric_time__extract_year + , subq_4.metric_time__extract_quarter AS metric_time__extract_quarter + , subq_4.metric_time__extract_month AS metric_time__extract_month + , subq_4.metric_time__extract_day AS metric_time__extract_day + , subq_4.metric_time__extract_dow AS metric_time__extract_dow + , subq_4.metric_time__extract_doy AS metric_time__extract_doy , subq_4.listing AS listing , subq_4.user AS user , subq_4.listing__user AS listing__user @@ -400,7 +401,6 @@ FROM ( , subq_4.listing__country_latest AS listing__country_latest , subq_4.listing__is_lux_latest AS listing__is_lux_latest , subq_4.listing__capacity_latest AS listing__capacity_latest - , subq_7.home_state AS user__home_state , subq_4.listings AS listings , subq_4.largest_listing AS largest_listing , subq_4.smallest_listing AS smallest_listing diff --git a/tests_metricflow/snapshots/test_custom_granularity.py/SqlQueryPlan/DuckDB/test_simple_metric_with_multi_hop_custom_granularity__plan0.sql b/tests_metricflow/snapshots/test_custom_granularity.py/SqlQueryPlan/DuckDB/test_simple_metric_with_multi_hop_custom_granularity__plan0.sql index dc337169ee..fb56a0ee0b 100644 --- a/tests_metricflow/snapshots/test_custom_granularity.py/SqlQueryPlan/DuckDB/test_simple_metric_with_multi_hop_custom_granularity__plan0.sql +++ b/tests_metricflow/snapshots/test_custom_granularity.py/SqlQueryPlan/DuckDB/test_simple_metric_with_multi_hop_custom_granularity__plan0.sql @@ -22,9 +22,9 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.ds_partitioned__day AS ds_partitioned__day - , subq_9.user__ds__day AS listing__user__ds__day + subq_9.user__ds__day AS listing__user__ds__day , subq_9.user__ds_partitioned__day AS listing__user__ds_partitioned__day + , subq_2.ds_partitioned__day AS ds_partitioned__day , subq_2.listing AS listing , subq_2.bookings AS bookings FROM ( @@ -238,61 +238,7 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_4.ds__day AS ds__day - , subq_4.ds__week AS ds__week - , subq_4.ds__month AS ds__month - , subq_4.ds__quarter AS ds__quarter - , subq_4.ds__year AS ds__year - , subq_4.ds__extract_year AS ds__extract_year - , subq_4.ds__extract_quarter AS ds__extract_quarter - , subq_4.ds__extract_month AS ds__extract_month - , subq_4.ds__extract_day AS ds__extract_day - , subq_4.ds__extract_dow AS ds__extract_dow - , subq_4.ds__extract_doy AS ds__extract_doy - , subq_4.created_at__day AS created_at__day - , subq_4.created_at__week AS created_at__week - , subq_4.created_at__month AS created_at__month - , subq_4.created_at__quarter AS created_at__quarter - , subq_4.created_at__year AS created_at__year - , subq_4.created_at__extract_year AS created_at__extract_year - , subq_4.created_at__extract_quarter AS created_at__extract_quarter - , subq_4.created_at__extract_month AS created_at__extract_month - , subq_4.created_at__extract_day AS created_at__extract_day - , subq_4.created_at__extract_dow AS created_at__extract_dow - , subq_4.created_at__extract_doy AS created_at__extract_doy - , subq_4.listing__ds__day AS listing__ds__day - , subq_4.listing__ds__week AS listing__ds__week - , subq_4.listing__ds__month AS listing__ds__month - , subq_4.listing__ds__quarter AS listing__ds__quarter - , subq_4.listing__ds__year AS listing__ds__year - , subq_4.listing__ds__extract_year AS listing__ds__extract_year - , subq_4.listing__ds__extract_quarter AS listing__ds__extract_quarter - , subq_4.listing__ds__extract_month AS listing__ds__extract_month - , subq_4.listing__ds__extract_day AS listing__ds__extract_day - , subq_4.listing__ds__extract_dow AS listing__ds__extract_dow - , subq_4.listing__ds__extract_doy AS listing__ds__extract_doy - , subq_4.listing__created_at__day AS listing__created_at__day - , subq_4.listing__created_at__week AS listing__created_at__week - , subq_4.listing__created_at__month AS listing__created_at__month - , subq_4.listing__created_at__quarter AS listing__created_at__quarter - , subq_4.listing__created_at__year AS listing__created_at__year - , subq_4.listing__created_at__extract_year AS listing__created_at__extract_year - , subq_4.listing__created_at__extract_quarter AS listing__created_at__extract_quarter - , subq_4.listing__created_at__extract_month AS listing__created_at__extract_month - , subq_4.listing__created_at__extract_day AS listing__created_at__extract_day - , subq_4.listing__created_at__extract_dow AS listing__created_at__extract_dow - , subq_4.listing__created_at__extract_doy AS listing__created_at__extract_doy - , subq_4.metric_time__day AS metric_time__day - , subq_4.metric_time__week AS metric_time__week - , subq_4.metric_time__month AS metric_time__month - , subq_4.metric_time__quarter AS metric_time__quarter - , subq_4.metric_time__year AS metric_time__year - , subq_4.metric_time__extract_year AS metric_time__extract_year - , subq_4.metric_time__extract_quarter AS metric_time__extract_quarter - , subq_4.metric_time__extract_month AS metric_time__extract_month - , subq_4.metric_time__extract_day AS metric_time__extract_day - , subq_4.metric_time__extract_dow AS metric_time__extract_dow - , subq_4.metric_time__extract_doy AS metric_time__extract_doy + subq_7.home_state AS user__home_state , subq_7.ds__day AS user__ds__day , subq_7.ds__week AS user__ds__week , subq_7.ds__month AS user__ds__month @@ -391,6 +337,61 @@ FROM ( , subq_7.metric_time__extract_day AS user__metric_time__extract_day , subq_7.metric_time__extract_dow AS user__metric_time__extract_dow , subq_7.metric_time__extract_doy AS user__metric_time__extract_doy + , subq_4.ds__day AS ds__day + , subq_4.ds__week AS ds__week + , subq_4.ds__month AS ds__month + , subq_4.ds__quarter AS ds__quarter + , subq_4.ds__year AS ds__year + , subq_4.ds__extract_year AS ds__extract_year + , subq_4.ds__extract_quarter AS ds__extract_quarter + , subq_4.ds__extract_month AS ds__extract_month + , subq_4.ds__extract_day AS ds__extract_day + , subq_4.ds__extract_dow AS ds__extract_dow + , subq_4.ds__extract_doy AS ds__extract_doy + , subq_4.created_at__day AS created_at__day + , subq_4.created_at__week AS created_at__week + , subq_4.created_at__month AS created_at__month + , subq_4.created_at__quarter AS created_at__quarter + , subq_4.created_at__year AS created_at__year + , subq_4.created_at__extract_year AS created_at__extract_year + , subq_4.created_at__extract_quarter AS created_at__extract_quarter + , subq_4.created_at__extract_month AS created_at__extract_month + , subq_4.created_at__extract_day AS created_at__extract_day + , subq_4.created_at__extract_dow AS created_at__extract_dow + , subq_4.created_at__extract_doy AS created_at__extract_doy + , subq_4.listing__ds__day AS listing__ds__day + , subq_4.listing__ds__week AS listing__ds__week + , subq_4.listing__ds__month AS listing__ds__month + , subq_4.listing__ds__quarter AS listing__ds__quarter + , subq_4.listing__ds__year AS listing__ds__year + , subq_4.listing__ds__extract_year AS listing__ds__extract_year + , subq_4.listing__ds__extract_quarter AS listing__ds__extract_quarter + , subq_4.listing__ds__extract_month AS listing__ds__extract_month + , subq_4.listing__ds__extract_day AS listing__ds__extract_day + , subq_4.listing__ds__extract_dow AS listing__ds__extract_dow + , subq_4.listing__ds__extract_doy AS listing__ds__extract_doy + , subq_4.listing__created_at__day AS listing__created_at__day + , subq_4.listing__created_at__week AS listing__created_at__week + , subq_4.listing__created_at__month AS listing__created_at__month + , subq_4.listing__created_at__quarter AS listing__created_at__quarter + , subq_4.listing__created_at__year AS listing__created_at__year + , subq_4.listing__created_at__extract_year AS listing__created_at__extract_year + , subq_4.listing__created_at__extract_quarter AS listing__created_at__extract_quarter + , subq_4.listing__created_at__extract_month AS listing__created_at__extract_month + , subq_4.listing__created_at__extract_day AS listing__created_at__extract_day + , subq_4.listing__created_at__extract_dow AS listing__created_at__extract_dow + , subq_4.listing__created_at__extract_doy AS listing__created_at__extract_doy + , subq_4.metric_time__day AS metric_time__day + , subq_4.metric_time__week AS metric_time__week + , subq_4.metric_time__month AS metric_time__month + , subq_4.metric_time__quarter AS metric_time__quarter + , subq_4.metric_time__year AS metric_time__year + , subq_4.metric_time__extract_year AS metric_time__extract_year + , subq_4.metric_time__extract_quarter AS metric_time__extract_quarter + , subq_4.metric_time__extract_month AS metric_time__extract_month + , subq_4.metric_time__extract_day AS metric_time__extract_day + , subq_4.metric_time__extract_dow AS metric_time__extract_dow + , subq_4.metric_time__extract_doy AS metric_time__extract_doy , subq_4.listing AS listing , subq_4.user AS user , subq_4.listing__user AS listing__user @@ -400,7 +401,6 @@ FROM ( , subq_4.listing__country_latest AS listing__country_latest , subq_4.listing__is_lux_latest AS listing__is_lux_latest , subq_4.listing__capacity_latest AS listing__capacity_latest - , subq_7.home_state AS user__home_state , subq_4.listings AS listings , subq_4.largest_listing AS largest_listing , subq_4.smallest_listing AS smallest_listing diff --git a/tests_metricflow/snapshots/test_custom_granularity.py/SqlQueryPlan/Postgres/test_simple_metric_with_multi_hop_custom_granularity__plan0.sql b/tests_metricflow/snapshots/test_custom_granularity.py/SqlQueryPlan/Postgres/test_simple_metric_with_multi_hop_custom_granularity__plan0.sql index dc337169ee..fb56a0ee0b 100644 --- a/tests_metricflow/snapshots/test_custom_granularity.py/SqlQueryPlan/Postgres/test_simple_metric_with_multi_hop_custom_granularity__plan0.sql +++ b/tests_metricflow/snapshots/test_custom_granularity.py/SqlQueryPlan/Postgres/test_simple_metric_with_multi_hop_custom_granularity__plan0.sql @@ -22,9 +22,9 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.ds_partitioned__day AS ds_partitioned__day - , subq_9.user__ds__day AS listing__user__ds__day + subq_9.user__ds__day AS listing__user__ds__day , subq_9.user__ds_partitioned__day AS listing__user__ds_partitioned__day + , subq_2.ds_partitioned__day AS ds_partitioned__day , subq_2.listing AS listing , subq_2.bookings AS bookings FROM ( @@ -238,61 +238,7 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_4.ds__day AS ds__day - , subq_4.ds__week AS ds__week - , subq_4.ds__month AS ds__month - , subq_4.ds__quarter AS ds__quarter - , subq_4.ds__year AS ds__year - , subq_4.ds__extract_year AS ds__extract_year - , subq_4.ds__extract_quarter AS ds__extract_quarter - , subq_4.ds__extract_month AS ds__extract_month - , subq_4.ds__extract_day AS ds__extract_day - , subq_4.ds__extract_dow AS ds__extract_dow - , subq_4.ds__extract_doy AS ds__extract_doy - , subq_4.created_at__day AS created_at__day - , subq_4.created_at__week AS created_at__week - , subq_4.created_at__month AS created_at__month - , subq_4.created_at__quarter AS created_at__quarter - , subq_4.created_at__year AS created_at__year - , subq_4.created_at__extract_year AS created_at__extract_year - , subq_4.created_at__extract_quarter AS created_at__extract_quarter - , subq_4.created_at__extract_month AS created_at__extract_month - , subq_4.created_at__extract_day AS created_at__extract_day - , subq_4.created_at__extract_dow AS created_at__extract_dow - , subq_4.created_at__extract_doy AS created_at__extract_doy - , subq_4.listing__ds__day AS listing__ds__day - , subq_4.listing__ds__week AS listing__ds__week - , subq_4.listing__ds__month AS listing__ds__month - , subq_4.listing__ds__quarter AS listing__ds__quarter - , subq_4.listing__ds__year AS listing__ds__year - , subq_4.listing__ds__extract_year AS listing__ds__extract_year - , subq_4.listing__ds__extract_quarter AS listing__ds__extract_quarter - , subq_4.listing__ds__extract_month AS listing__ds__extract_month - , subq_4.listing__ds__extract_day AS listing__ds__extract_day - , subq_4.listing__ds__extract_dow AS listing__ds__extract_dow - , subq_4.listing__ds__extract_doy AS listing__ds__extract_doy - , subq_4.listing__created_at__day AS listing__created_at__day - , subq_4.listing__created_at__week AS listing__created_at__week - , subq_4.listing__created_at__month AS listing__created_at__month - , subq_4.listing__created_at__quarter AS listing__created_at__quarter - , subq_4.listing__created_at__year AS listing__created_at__year - , subq_4.listing__created_at__extract_year AS listing__created_at__extract_year - , subq_4.listing__created_at__extract_quarter AS listing__created_at__extract_quarter - , subq_4.listing__created_at__extract_month AS listing__created_at__extract_month - , subq_4.listing__created_at__extract_day AS listing__created_at__extract_day - , subq_4.listing__created_at__extract_dow AS listing__created_at__extract_dow - , subq_4.listing__created_at__extract_doy AS listing__created_at__extract_doy - , subq_4.metric_time__day AS metric_time__day - , subq_4.metric_time__week AS metric_time__week - , subq_4.metric_time__month AS metric_time__month - , subq_4.metric_time__quarter AS metric_time__quarter - , subq_4.metric_time__year AS metric_time__year - , subq_4.metric_time__extract_year AS metric_time__extract_year - , subq_4.metric_time__extract_quarter AS metric_time__extract_quarter - , subq_4.metric_time__extract_month AS metric_time__extract_month - , subq_4.metric_time__extract_day AS metric_time__extract_day - , subq_4.metric_time__extract_dow AS metric_time__extract_dow - , subq_4.metric_time__extract_doy AS metric_time__extract_doy + subq_7.home_state AS user__home_state , subq_7.ds__day AS user__ds__day , subq_7.ds__week AS user__ds__week , subq_7.ds__month AS user__ds__month @@ -391,6 +337,61 @@ FROM ( , subq_7.metric_time__extract_day AS user__metric_time__extract_day , subq_7.metric_time__extract_dow AS user__metric_time__extract_dow , subq_7.metric_time__extract_doy AS user__metric_time__extract_doy + , subq_4.ds__day AS ds__day + , subq_4.ds__week AS ds__week + , subq_4.ds__month AS ds__month + , subq_4.ds__quarter AS ds__quarter + , subq_4.ds__year AS ds__year + , subq_4.ds__extract_year AS ds__extract_year + , subq_4.ds__extract_quarter AS ds__extract_quarter + , subq_4.ds__extract_month AS ds__extract_month + , subq_4.ds__extract_day AS ds__extract_day + , subq_4.ds__extract_dow AS ds__extract_dow + , subq_4.ds__extract_doy AS ds__extract_doy + , subq_4.created_at__day AS created_at__day + , subq_4.created_at__week AS created_at__week + , subq_4.created_at__month AS created_at__month + , subq_4.created_at__quarter AS created_at__quarter + , subq_4.created_at__year AS created_at__year + , subq_4.created_at__extract_year AS created_at__extract_year + , subq_4.created_at__extract_quarter AS created_at__extract_quarter + , subq_4.created_at__extract_month AS created_at__extract_month + , subq_4.created_at__extract_day AS created_at__extract_day + , subq_4.created_at__extract_dow AS created_at__extract_dow + , subq_4.created_at__extract_doy AS created_at__extract_doy + , subq_4.listing__ds__day AS listing__ds__day + , subq_4.listing__ds__week AS listing__ds__week + , subq_4.listing__ds__month AS listing__ds__month + , subq_4.listing__ds__quarter AS listing__ds__quarter + , subq_4.listing__ds__year AS listing__ds__year + , subq_4.listing__ds__extract_year AS listing__ds__extract_year + , subq_4.listing__ds__extract_quarter AS listing__ds__extract_quarter + , subq_4.listing__ds__extract_month AS listing__ds__extract_month + , subq_4.listing__ds__extract_day AS listing__ds__extract_day + , subq_4.listing__ds__extract_dow AS listing__ds__extract_dow + , subq_4.listing__ds__extract_doy AS listing__ds__extract_doy + , subq_4.listing__created_at__day AS listing__created_at__day + , subq_4.listing__created_at__week AS listing__created_at__week + , subq_4.listing__created_at__month AS listing__created_at__month + , subq_4.listing__created_at__quarter AS listing__created_at__quarter + , subq_4.listing__created_at__year AS listing__created_at__year + , subq_4.listing__created_at__extract_year AS listing__created_at__extract_year + , subq_4.listing__created_at__extract_quarter AS listing__created_at__extract_quarter + , subq_4.listing__created_at__extract_month AS listing__created_at__extract_month + , subq_4.listing__created_at__extract_day AS listing__created_at__extract_day + , subq_4.listing__created_at__extract_dow AS listing__created_at__extract_dow + , subq_4.listing__created_at__extract_doy AS listing__created_at__extract_doy + , subq_4.metric_time__day AS metric_time__day + , subq_4.metric_time__week AS metric_time__week + , subq_4.metric_time__month AS metric_time__month + , subq_4.metric_time__quarter AS metric_time__quarter + , subq_4.metric_time__year AS metric_time__year + , subq_4.metric_time__extract_year AS metric_time__extract_year + , subq_4.metric_time__extract_quarter AS metric_time__extract_quarter + , subq_4.metric_time__extract_month AS metric_time__extract_month + , subq_4.metric_time__extract_day AS metric_time__extract_day + , subq_4.metric_time__extract_dow AS metric_time__extract_dow + , subq_4.metric_time__extract_doy AS metric_time__extract_doy , subq_4.listing AS listing , subq_4.user AS user , subq_4.listing__user AS listing__user @@ -400,7 +401,6 @@ FROM ( , subq_4.listing__country_latest AS listing__country_latest , subq_4.listing__is_lux_latest AS listing__is_lux_latest , subq_4.listing__capacity_latest AS listing__capacity_latest - , subq_7.home_state AS user__home_state , subq_4.listings AS listings , subq_4.largest_listing AS largest_listing , subq_4.smallest_listing AS smallest_listing diff --git a/tests_metricflow/snapshots/test_custom_granularity.py/SqlQueryPlan/Redshift/test_simple_metric_with_multi_hop_custom_granularity__plan0.sql b/tests_metricflow/snapshots/test_custom_granularity.py/SqlQueryPlan/Redshift/test_simple_metric_with_multi_hop_custom_granularity__plan0.sql index 96bb37ce6c..0b30358d05 100644 --- a/tests_metricflow/snapshots/test_custom_granularity.py/SqlQueryPlan/Redshift/test_simple_metric_with_multi_hop_custom_granularity__plan0.sql +++ b/tests_metricflow/snapshots/test_custom_granularity.py/SqlQueryPlan/Redshift/test_simple_metric_with_multi_hop_custom_granularity__plan0.sql @@ -22,9 +22,9 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.ds_partitioned__day AS ds_partitioned__day - , subq_9.user__ds__day AS listing__user__ds__day + subq_9.user__ds__day AS listing__user__ds__day , subq_9.user__ds_partitioned__day AS listing__user__ds_partitioned__day + , subq_2.ds_partitioned__day AS ds_partitioned__day , subq_2.listing AS listing , subq_2.bookings AS bookings FROM ( @@ -238,61 +238,7 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_4.ds__day AS ds__day - , subq_4.ds__week AS ds__week - , subq_4.ds__month AS ds__month - , subq_4.ds__quarter AS ds__quarter - , subq_4.ds__year AS ds__year - , subq_4.ds__extract_year AS ds__extract_year - , subq_4.ds__extract_quarter AS ds__extract_quarter - , subq_4.ds__extract_month AS ds__extract_month - , subq_4.ds__extract_day AS ds__extract_day - , subq_4.ds__extract_dow AS ds__extract_dow - , subq_4.ds__extract_doy AS ds__extract_doy - , subq_4.created_at__day AS created_at__day - , subq_4.created_at__week AS created_at__week - , subq_4.created_at__month AS created_at__month - , subq_4.created_at__quarter AS created_at__quarter - , subq_4.created_at__year AS created_at__year - , subq_4.created_at__extract_year AS created_at__extract_year - , subq_4.created_at__extract_quarter AS created_at__extract_quarter - , subq_4.created_at__extract_month AS created_at__extract_month - , subq_4.created_at__extract_day AS created_at__extract_day - , subq_4.created_at__extract_dow AS created_at__extract_dow - , subq_4.created_at__extract_doy AS created_at__extract_doy - , subq_4.listing__ds__day AS listing__ds__day - , subq_4.listing__ds__week AS listing__ds__week - , subq_4.listing__ds__month AS listing__ds__month - , subq_4.listing__ds__quarter AS listing__ds__quarter - , subq_4.listing__ds__year AS listing__ds__year - , subq_4.listing__ds__extract_year AS listing__ds__extract_year - , subq_4.listing__ds__extract_quarter AS listing__ds__extract_quarter - , subq_4.listing__ds__extract_month AS listing__ds__extract_month - , subq_4.listing__ds__extract_day AS listing__ds__extract_day - , subq_4.listing__ds__extract_dow AS listing__ds__extract_dow - , subq_4.listing__ds__extract_doy AS listing__ds__extract_doy - , subq_4.listing__created_at__day AS listing__created_at__day - , subq_4.listing__created_at__week AS listing__created_at__week - , subq_4.listing__created_at__month AS listing__created_at__month - , subq_4.listing__created_at__quarter AS listing__created_at__quarter - , subq_4.listing__created_at__year AS listing__created_at__year - , subq_4.listing__created_at__extract_year AS listing__created_at__extract_year - , subq_4.listing__created_at__extract_quarter AS listing__created_at__extract_quarter - , subq_4.listing__created_at__extract_month AS listing__created_at__extract_month - , subq_4.listing__created_at__extract_day AS listing__created_at__extract_day - , subq_4.listing__created_at__extract_dow AS listing__created_at__extract_dow - , subq_4.listing__created_at__extract_doy AS listing__created_at__extract_doy - , subq_4.metric_time__day AS metric_time__day - , subq_4.metric_time__week AS metric_time__week - , subq_4.metric_time__month AS metric_time__month - , subq_4.metric_time__quarter AS metric_time__quarter - , subq_4.metric_time__year AS metric_time__year - , subq_4.metric_time__extract_year AS metric_time__extract_year - , subq_4.metric_time__extract_quarter AS metric_time__extract_quarter - , subq_4.metric_time__extract_month AS metric_time__extract_month - , subq_4.metric_time__extract_day AS metric_time__extract_day - , subq_4.metric_time__extract_dow AS metric_time__extract_dow - , subq_4.metric_time__extract_doy AS metric_time__extract_doy + subq_7.home_state AS user__home_state , subq_7.ds__day AS user__ds__day , subq_7.ds__week AS user__ds__week , subq_7.ds__month AS user__ds__month @@ -391,6 +337,61 @@ FROM ( , subq_7.metric_time__extract_day AS user__metric_time__extract_day , subq_7.metric_time__extract_dow AS user__metric_time__extract_dow , subq_7.metric_time__extract_doy AS user__metric_time__extract_doy + , subq_4.ds__day AS ds__day + , subq_4.ds__week AS ds__week + , subq_4.ds__month AS ds__month + , subq_4.ds__quarter AS ds__quarter + , subq_4.ds__year AS ds__year + , subq_4.ds__extract_year AS ds__extract_year + , subq_4.ds__extract_quarter AS ds__extract_quarter + , subq_4.ds__extract_month AS ds__extract_month + , subq_4.ds__extract_day AS ds__extract_day + , subq_4.ds__extract_dow AS ds__extract_dow + , subq_4.ds__extract_doy AS ds__extract_doy + , subq_4.created_at__day AS created_at__day + , subq_4.created_at__week AS created_at__week + , subq_4.created_at__month AS created_at__month + , subq_4.created_at__quarter AS created_at__quarter + , subq_4.created_at__year AS created_at__year + , subq_4.created_at__extract_year AS created_at__extract_year + , subq_4.created_at__extract_quarter AS created_at__extract_quarter + , subq_4.created_at__extract_month AS created_at__extract_month + , subq_4.created_at__extract_day AS created_at__extract_day + , subq_4.created_at__extract_dow AS created_at__extract_dow + , subq_4.created_at__extract_doy AS created_at__extract_doy + , subq_4.listing__ds__day AS listing__ds__day + , subq_4.listing__ds__week AS listing__ds__week + , subq_4.listing__ds__month AS listing__ds__month + , subq_4.listing__ds__quarter AS listing__ds__quarter + , subq_4.listing__ds__year AS listing__ds__year + , subq_4.listing__ds__extract_year AS listing__ds__extract_year + , subq_4.listing__ds__extract_quarter AS listing__ds__extract_quarter + , subq_4.listing__ds__extract_month AS listing__ds__extract_month + , subq_4.listing__ds__extract_day AS listing__ds__extract_day + , subq_4.listing__ds__extract_dow AS listing__ds__extract_dow + , subq_4.listing__ds__extract_doy AS listing__ds__extract_doy + , subq_4.listing__created_at__day AS listing__created_at__day + , subq_4.listing__created_at__week AS listing__created_at__week + , subq_4.listing__created_at__month AS listing__created_at__month + , subq_4.listing__created_at__quarter AS listing__created_at__quarter + , subq_4.listing__created_at__year AS listing__created_at__year + , subq_4.listing__created_at__extract_year AS listing__created_at__extract_year + , subq_4.listing__created_at__extract_quarter AS listing__created_at__extract_quarter + , subq_4.listing__created_at__extract_month AS listing__created_at__extract_month + , subq_4.listing__created_at__extract_day AS listing__created_at__extract_day + , subq_4.listing__created_at__extract_dow AS listing__created_at__extract_dow + , subq_4.listing__created_at__extract_doy AS listing__created_at__extract_doy + , subq_4.metric_time__day AS metric_time__day + , subq_4.metric_time__week AS metric_time__week + , subq_4.metric_time__month AS metric_time__month + , subq_4.metric_time__quarter AS metric_time__quarter + , subq_4.metric_time__year AS metric_time__year + , subq_4.metric_time__extract_year AS metric_time__extract_year + , subq_4.metric_time__extract_quarter AS metric_time__extract_quarter + , subq_4.metric_time__extract_month AS metric_time__extract_month + , subq_4.metric_time__extract_day AS metric_time__extract_day + , subq_4.metric_time__extract_dow AS metric_time__extract_dow + , subq_4.metric_time__extract_doy AS metric_time__extract_doy , subq_4.listing AS listing , subq_4.user AS user , subq_4.listing__user AS listing__user @@ -400,7 +401,6 @@ FROM ( , subq_4.listing__country_latest AS listing__country_latest , subq_4.listing__is_lux_latest AS listing__is_lux_latest , subq_4.listing__capacity_latest AS listing__capacity_latest - , subq_7.home_state AS user__home_state , subq_4.listings AS listings , subq_4.largest_listing AS largest_listing , subq_4.smallest_listing AS smallest_listing diff --git a/tests_metricflow/snapshots/test_custom_granularity.py/SqlQueryPlan/Snowflake/test_simple_metric_with_multi_hop_custom_granularity__plan0.sql b/tests_metricflow/snapshots/test_custom_granularity.py/SqlQueryPlan/Snowflake/test_simple_metric_with_multi_hop_custom_granularity__plan0.sql index 44b455724b..0b58d7018f 100644 --- a/tests_metricflow/snapshots/test_custom_granularity.py/SqlQueryPlan/Snowflake/test_simple_metric_with_multi_hop_custom_granularity__plan0.sql +++ b/tests_metricflow/snapshots/test_custom_granularity.py/SqlQueryPlan/Snowflake/test_simple_metric_with_multi_hop_custom_granularity__plan0.sql @@ -22,9 +22,9 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.ds_partitioned__day AS ds_partitioned__day - , subq_9.user__ds__day AS listing__user__ds__day + subq_9.user__ds__day AS listing__user__ds__day , subq_9.user__ds_partitioned__day AS listing__user__ds_partitioned__day + , subq_2.ds_partitioned__day AS ds_partitioned__day , subq_2.listing AS listing , subq_2.bookings AS bookings FROM ( @@ -238,61 +238,7 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_4.ds__day AS ds__day - , subq_4.ds__week AS ds__week - , subq_4.ds__month AS ds__month - , subq_4.ds__quarter AS ds__quarter - , subq_4.ds__year AS ds__year - , subq_4.ds__extract_year AS ds__extract_year - , subq_4.ds__extract_quarter AS ds__extract_quarter - , subq_4.ds__extract_month AS ds__extract_month - , subq_4.ds__extract_day AS ds__extract_day - , subq_4.ds__extract_dow AS ds__extract_dow - , subq_4.ds__extract_doy AS ds__extract_doy - , subq_4.created_at__day AS created_at__day - , subq_4.created_at__week AS created_at__week - , subq_4.created_at__month AS created_at__month - , subq_4.created_at__quarter AS created_at__quarter - , subq_4.created_at__year AS created_at__year - , subq_4.created_at__extract_year AS created_at__extract_year - , subq_4.created_at__extract_quarter AS created_at__extract_quarter - , subq_4.created_at__extract_month AS created_at__extract_month - , subq_4.created_at__extract_day AS created_at__extract_day - , subq_4.created_at__extract_dow AS created_at__extract_dow - , subq_4.created_at__extract_doy AS created_at__extract_doy - , subq_4.listing__ds__day AS listing__ds__day - , subq_4.listing__ds__week AS listing__ds__week - , subq_4.listing__ds__month AS listing__ds__month - , subq_4.listing__ds__quarter AS listing__ds__quarter - , subq_4.listing__ds__year AS listing__ds__year - , subq_4.listing__ds__extract_year AS listing__ds__extract_year - , subq_4.listing__ds__extract_quarter AS listing__ds__extract_quarter - , subq_4.listing__ds__extract_month AS listing__ds__extract_month - , subq_4.listing__ds__extract_day AS listing__ds__extract_day - , subq_4.listing__ds__extract_dow AS listing__ds__extract_dow - , subq_4.listing__ds__extract_doy AS listing__ds__extract_doy - , subq_4.listing__created_at__day AS listing__created_at__day - , subq_4.listing__created_at__week AS listing__created_at__week - , subq_4.listing__created_at__month AS listing__created_at__month - , subq_4.listing__created_at__quarter AS listing__created_at__quarter - , subq_4.listing__created_at__year AS listing__created_at__year - , subq_4.listing__created_at__extract_year AS listing__created_at__extract_year - , subq_4.listing__created_at__extract_quarter AS listing__created_at__extract_quarter - , subq_4.listing__created_at__extract_month AS listing__created_at__extract_month - , subq_4.listing__created_at__extract_day AS listing__created_at__extract_day - , subq_4.listing__created_at__extract_dow AS listing__created_at__extract_dow - , subq_4.listing__created_at__extract_doy AS listing__created_at__extract_doy - , subq_4.metric_time__day AS metric_time__day - , subq_4.metric_time__week AS metric_time__week - , subq_4.metric_time__month AS metric_time__month - , subq_4.metric_time__quarter AS metric_time__quarter - , subq_4.metric_time__year AS metric_time__year - , subq_4.metric_time__extract_year AS metric_time__extract_year - , subq_4.metric_time__extract_quarter AS metric_time__extract_quarter - , subq_4.metric_time__extract_month AS metric_time__extract_month - , subq_4.metric_time__extract_day AS metric_time__extract_day - , subq_4.metric_time__extract_dow AS metric_time__extract_dow - , subq_4.metric_time__extract_doy AS metric_time__extract_doy + subq_7.home_state AS user__home_state , subq_7.ds__day AS user__ds__day , subq_7.ds__week AS user__ds__week , subq_7.ds__month AS user__ds__month @@ -391,6 +337,61 @@ FROM ( , subq_7.metric_time__extract_day AS user__metric_time__extract_day , subq_7.metric_time__extract_dow AS user__metric_time__extract_dow , subq_7.metric_time__extract_doy AS user__metric_time__extract_doy + , subq_4.ds__day AS ds__day + , subq_4.ds__week AS ds__week + , subq_4.ds__month AS ds__month + , subq_4.ds__quarter AS ds__quarter + , subq_4.ds__year AS ds__year + , subq_4.ds__extract_year AS ds__extract_year + , subq_4.ds__extract_quarter AS ds__extract_quarter + , subq_4.ds__extract_month AS ds__extract_month + , subq_4.ds__extract_day AS ds__extract_day + , subq_4.ds__extract_dow AS ds__extract_dow + , subq_4.ds__extract_doy AS ds__extract_doy + , subq_4.created_at__day AS created_at__day + , subq_4.created_at__week AS created_at__week + , subq_4.created_at__month AS created_at__month + , subq_4.created_at__quarter AS created_at__quarter + , subq_4.created_at__year AS created_at__year + , subq_4.created_at__extract_year AS created_at__extract_year + , subq_4.created_at__extract_quarter AS created_at__extract_quarter + , subq_4.created_at__extract_month AS created_at__extract_month + , subq_4.created_at__extract_day AS created_at__extract_day + , subq_4.created_at__extract_dow AS created_at__extract_dow + , subq_4.created_at__extract_doy AS created_at__extract_doy + , subq_4.listing__ds__day AS listing__ds__day + , subq_4.listing__ds__week AS listing__ds__week + , subq_4.listing__ds__month AS listing__ds__month + , subq_4.listing__ds__quarter AS listing__ds__quarter + , subq_4.listing__ds__year AS listing__ds__year + , subq_4.listing__ds__extract_year AS listing__ds__extract_year + , subq_4.listing__ds__extract_quarter AS listing__ds__extract_quarter + , subq_4.listing__ds__extract_month AS listing__ds__extract_month + , subq_4.listing__ds__extract_day AS listing__ds__extract_day + , subq_4.listing__ds__extract_dow AS listing__ds__extract_dow + , subq_4.listing__ds__extract_doy AS listing__ds__extract_doy + , subq_4.listing__created_at__day AS listing__created_at__day + , subq_4.listing__created_at__week AS listing__created_at__week + , subq_4.listing__created_at__month AS listing__created_at__month + , subq_4.listing__created_at__quarter AS listing__created_at__quarter + , subq_4.listing__created_at__year AS listing__created_at__year + , subq_4.listing__created_at__extract_year AS listing__created_at__extract_year + , subq_4.listing__created_at__extract_quarter AS listing__created_at__extract_quarter + , subq_4.listing__created_at__extract_month AS listing__created_at__extract_month + , subq_4.listing__created_at__extract_day AS listing__created_at__extract_day + , subq_4.listing__created_at__extract_dow AS listing__created_at__extract_dow + , subq_4.listing__created_at__extract_doy AS listing__created_at__extract_doy + , subq_4.metric_time__day AS metric_time__day + , subq_4.metric_time__week AS metric_time__week + , subq_4.metric_time__month AS metric_time__month + , subq_4.metric_time__quarter AS metric_time__quarter + , subq_4.metric_time__year AS metric_time__year + , subq_4.metric_time__extract_year AS metric_time__extract_year + , subq_4.metric_time__extract_quarter AS metric_time__extract_quarter + , subq_4.metric_time__extract_month AS metric_time__extract_month + , subq_4.metric_time__extract_day AS metric_time__extract_day + , subq_4.metric_time__extract_dow AS metric_time__extract_dow + , subq_4.metric_time__extract_doy AS metric_time__extract_doy , subq_4.listing AS listing , subq_4.user AS user , subq_4.listing__user AS listing__user @@ -400,7 +401,6 @@ FROM ( , subq_4.listing__country_latest AS listing__country_latest , subq_4.listing__is_lux_latest AS listing__is_lux_latest , subq_4.listing__capacity_latest AS listing__capacity_latest - , subq_7.home_state AS user__home_state , subq_4.listings AS listings , subq_4.largest_listing AS largest_listing , subq_4.smallest_listing AS smallest_listing diff --git a/tests_metricflow/snapshots/test_custom_granularity.py/SqlQueryPlan/Trino/test_simple_metric_with_multi_hop_custom_granularity__plan0.sql b/tests_metricflow/snapshots/test_custom_granularity.py/SqlQueryPlan/Trino/test_simple_metric_with_multi_hop_custom_granularity__plan0.sql index 7f43f0da79..780f8655ae 100644 --- a/tests_metricflow/snapshots/test_custom_granularity.py/SqlQueryPlan/Trino/test_simple_metric_with_multi_hop_custom_granularity__plan0.sql +++ b/tests_metricflow/snapshots/test_custom_granularity.py/SqlQueryPlan/Trino/test_simple_metric_with_multi_hop_custom_granularity__plan0.sql @@ -22,9 +22,9 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.ds_partitioned__day AS ds_partitioned__day - , subq_9.user__ds__day AS listing__user__ds__day + subq_9.user__ds__day AS listing__user__ds__day , subq_9.user__ds_partitioned__day AS listing__user__ds_partitioned__day + , subq_2.ds_partitioned__day AS ds_partitioned__day , subq_2.listing AS listing , subq_2.bookings AS bookings FROM ( @@ -238,61 +238,7 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_4.ds__day AS ds__day - , subq_4.ds__week AS ds__week - , subq_4.ds__month AS ds__month - , subq_4.ds__quarter AS ds__quarter - , subq_4.ds__year AS ds__year - , subq_4.ds__extract_year AS ds__extract_year - , subq_4.ds__extract_quarter AS ds__extract_quarter - , subq_4.ds__extract_month AS ds__extract_month - , subq_4.ds__extract_day AS ds__extract_day - , subq_4.ds__extract_dow AS ds__extract_dow - , subq_4.ds__extract_doy AS ds__extract_doy - , subq_4.created_at__day AS created_at__day - , subq_4.created_at__week AS created_at__week - , subq_4.created_at__month AS created_at__month - , subq_4.created_at__quarter AS created_at__quarter - , subq_4.created_at__year AS created_at__year - , subq_4.created_at__extract_year AS created_at__extract_year - , subq_4.created_at__extract_quarter AS created_at__extract_quarter - , subq_4.created_at__extract_month AS created_at__extract_month - , subq_4.created_at__extract_day AS created_at__extract_day - , subq_4.created_at__extract_dow AS created_at__extract_dow - , subq_4.created_at__extract_doy AS created_at__extract_doy - , subq_4.listing__ds__day AS listing__ds__day - , subq_4.listing__ds__week AS listing__ds__week - , subq_4.listing__ds__month AS listing__ds__month - , subq_4.listing__ds__quarter AS listing__ds__quarter - , subq_4.listing__ds__year AS listing__ds__year - , subq_4.listing__ds__extract_year AS listing__ds__extract_year - , subq_4.listing__ds__extract_quarter AS listing__ds__extract_quarter - , subq_4.listing__ds__extract_month AS listing__ds__extract_month - , subq_4.listing__ds__extract_day AS listing__ds__extract_day - , subq_4.listing__ds__extract_dow AS listing__ds__extract_dow - , subq_4.listing__ds__extract_doy AS listing__ds__extract_doy - , subq_4.listing__created_at__day AS listing__created_at__day - , subq_4.listing__created_at__week AS listing__created_at__week - , subq_4.listing__created_at__month AS listing__created_at__month - , subq_4.listing__created_at__quarter AS listing__created_at__quarter - , subq_4.listing__created_at__year AS listing__created_at__year - , subq_4.listing__created_at__extract_year AS listing__created_at__extract_year - , subq_4.listing__created_at__extract_quarter AS listing__created_at__extract_quarter - , subq_4.listing__created_at__extract_month AS listing__created_at__extract_month - , subq_4.listing__created_at__extract_day AS listing__created_at__extract_day - , subq_4.listing__created_at__extract_dow AS listing__created_at__extract_dow - , subq_4.listing__created_at__extract_doy AS listing__created_at__extract_doy - , subq_4.metric_time__day AS metric_time__day - , subq_4.metric_time__week AS metric_time__week - , subq_4.metric_time__month AS metric_time__month - , subq_4.metric_time__quarter AS metric_time__quarter - , subq_4.metric_time__year AS metric_time__year - , subq_4.metric_time__extract_year AS metric_time__extract_year - , subq_4.metric_time__extract_quarter AS metric_time__extract_quarter - , subq_4.metric_time__extract_month AS metric_time__extract_month - , subq_4.metric_time__extract_day AS metric_time__extract_day - , subq_4.metric_time__extract_dow AS metric_time__extract_dow - , subq_4.metric_time__extract_doy AS metric_time__extract_doy + subq_7.home_state AS user__home_state , subq_7.ds__day AS user__ds__day , subq_7.ds__week AS user__ds__week , subq_7.ds__month AS user__ds__month @@ -391,6 +337,61 @@ FROM ( , subq_7.metric_time__extract_day AS user__metric_time__extract_day , subq_7.metric_time__extract_dow AS user__metric_time__extract_dow , subq_7.metric_time__extract_doy AS user__metric_time__extract_doy + , subq_4.ds__day AS ds__day + , subq_4.ds__week AS ds__week + , subq_4.ds__month AS ds__month + , subq_4.ds__quarter AS ds__quarter + , subq_4.ds__year AS ds__year + , subq_4.ds__extract_year AS ds__extract_year + , subq_4.ds__extract_quarter AS ds__extract_quarter + , subq_4.ds__extract_month AS ds__extract_month + , subq_4.ds__extract_day AS ds__extract_day + , subq_4.ds__extract_dow AS ds__extract_dow + , subq_4.ds__extract_doy AS ds__extract_doy + , subq_4.created_at__day AS created_at__day + , subq_4.created_at__week AS created_at__week + , subq_4.created_at__month AS created_at__month + , subq_4.created_at__quarter AS created_at__quarter + , subq_4.created_at__year AS created_at__year + , subq_4.created_at__extract_year AS created_at__extract_year + , subq_4.created_at__extract_quarter AS created_at__extract_quarter + , subq_4.created_at__extract_month AS created_at__extract_month + , subq_4.created_at__extract_day AS created_at__extract_day + , subq_4.created_at__extract_dow AS created_at__extract_dow + , subq_4.created_at__extract_doy AS created_at__extract_doy + , subq_4.listing__ds__day AS listing__ds__day + , subq_4.listing__ds__week AS listing__ds__week + , subq_4.listing__ds__month AS listing__ds__month + , subq_4.listing__ds__quarter AS listing__ds__quarter + , subq_4.listing__ds__year AS listing__ds__year + , subq_4.listing__ds__extract_year AS listing__ds__extract_year + , subq_4.listing__ds__extract_quarter AS listing__ds__extract_quarter + , subq_4.listing__ds__extract_month AS listing__ds__extract_month + , subq_4.listing__ds__extract_day AS listing__ds__extract_day + , subq_4.listing__ds__extract_dow AS listing__ds__extract_dow + , subq_4.listing__ds__extract_doy AS listing__ds__extract_doy + , subq_4.listing__created_at__day AS listing__created_at__day + , subq_4.listing__created_at__week AS listing__created_at__week + , subq_4.listing__created_at__month AS listing__created_at__month + , subq_4.listing__created_at__quarter AS listing__created_at__quarter + , subq_4.listing__created_at__year AS listing__created_at__year + , subq_4.listing__created_at__extract_year AS listing__created_at__extract_year + , subq_4.listing__created_at__extract_quarter AS listing__created_at__extract_quarter + , subq_4.listing__created_at__extract_month AS listing__created_at__extract_month + , subq_4.listing__created_at__extract_day AS listing__created_at__extract_day + , subq_4.listing__created_at__extract_dow AS listing__created_at__extract_dow + , subq_4.listing__created_at__extract_doy AS listing__created_at__extract_doy + , subq_4.metric_time__day AS metric_time__day + , subq_4.metric_time__week AS metric_time__week + , subq_4.metric_time__month AS metric_time__month + , subq_4.metric_time__quarter AS metric_time__quarter + , subq_4.metric_time__year AS metric_time__year + , subq_4.metric_time__extract_year AS metric_time__extract_year + , subq_4.metric_time__extract_quarter AS metric_time__extract_quarter + , subq_4.metric_time__extract_month AS metric_time__extract_month + , subq_4.metric_time__extract_day AS metric_time__extract_day + , subq_4.metric_time__extract_dow AS metric_time__extract_dow + , subq_4.metric_time__extract_doy AS metric_time__extract_doy , subq_4.listing AS listing , subq_4.user AS user , subq_4.listing__user AS listing__user @@ -400,7 +401,6 @@ FROM ( , subq_4.listing__country_latest AS listing__country_latest , subq_4.listing__is_lux_latest AS listing__is_lux_latest , subq_4.listing__capacity_latest AS listing__capacity_latest - , subq_7.home_state AS user__home_state , subq_4.listings AS listings , subq_4.largest_listing AS largest_listing , subq_4.smallest_listing AS smallest_listing diff --git a/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/BigQuery/test_compute_metrics_node__plan0.sql b/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/BigQuery/test_compute_metrics_node__plan0.sql index 3ea6e9c003..a78a40c79d 100644 --- a/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/BigQuery/test_compute_metrics_node__plan0.sql +++ b/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/BigQuery/test_compute_metrics_node__plan0.sql @@ -12,8 +12,8 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_1.listing AS listing - , subq_3.country_latest AS listing__country_latest + subq_3.country_latest AS listing__country_latest + , subq_1.listing AS listing , subq_1.bookings AS bookings FROM ( -- Pass Only Elements: ['bookings', 'listing'] diff --git a/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/BigQuery/test_compute_metrics_node_ratio_from_multiple_semantic_models__plan0.sql b/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/BigQuery/test_compute_metrics_node_ratio_from_multiple_semantic_models__plan0.sql index 91f5187a93..c643d94092 100644 --- a/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/BigQuery/test_compute_metrics_node_ratio_from_multiple_semantic_models__plan0.sql +++ b/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/BigQuery/test_compute_metrics_node_ratio_from_multiple_semantic_models__plan0.sql @@ -31,9 +31,9 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.ds__day AS ds__day + subq_5.country_latest AS listing__country_latest + , subq_2.ds__day AS ds__day , subq_2.listing AS listing - , subq_5.country_latest AS listing__country_latest , subq_2.bookings AS bookings FROM ( -- Pass Only Elements: ['bookings', 'ds__day', 'listing'] @@ -405,9 +405,9 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_12.ds__day AS ds__day + subq_15.country_latest AS listing__country_latest + , subq_12.ds__day AS ds__day , subq_12.listing AS listing - , subq_15.country_latest AS listing__country_latest , subq_12.views AS views FROM ( -- Pass Only Elements: ['views', 'ds__day', 'listing'] diff --git a/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/BigQuery/test_compute_metrics_node_ratio_from_single_semantic_model__plan0.sql b/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/BigQuery/test_compute_metrics_node_ratio_from_single_semantic_model__plan0.sql index 780bfddd2f..1bdd606db3 100644 --- a/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/BigQuery/test_compute_metrics_node_ratio_from_single_semantic_model__plan0.sql +++ b/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/BigQuery/test_compute_metrics_node_ratio_from_single_semantic_model__plan0.sql @@ -13,8 +13,8 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_1.listing AS listing - , subq_3.country_latest AS listing__country_latest + subq_3.country_latest AS listing__country_latest + , subq_1.listing AS listing , subq_1.bookings AS bookings , subq_1.bookers AS bookers FROM ( diff --git a/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/BigQuery/test_compute_metrics_node_simple_expr__plan0.sql b/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/BigQuery/test_compute_metrics_node_simple_expr__plan0.sql index 6240b480e5..53f2e34c4b 100644 --- a/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/BigQuery/test_compute_metrics_node_simple_expr__plan0.sql +++ b/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/BigQuery/test_compute_metrics_node_simple_expr__plan0.sql @@ -12,8 +12,8 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_1.listing AS listing - , subq_3.country_latest AS listing__country_latest + subq_3.country_latest AS listing__country_latest + , subq_1.listing AS listing , subq_1.booking_value AS booking_value FROM ( -- Pass Only Elements: ['booking_value', 'listing'] diff --git a/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/BigQuery/test_dimension_with_joined_where_constraint__plan0.sql b/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/BigQuery/test_dimension_with_joined_where_constraint__plan0.sql index 68a23dbf9e..153b3f9a4a 100644 --- a/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/BigQuery/test_dimension_with_joined_where_constraint__plan0.sql +++ b/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/BigQuery/test_dimension_with_joined_where_constraint__plan0.sql @@ -64,7 +64,8 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_0.ds__day AS ds__day + subq_2.home_state_latest AS user__home_state_latest + , subq_0.ds__day AS ds__day , subq_0.ds__week AS ds__week , subq_0.ds__month AS ds__month , subq_0.ds__quarter AS ds__quarter @@ -117,7 +118,6 @@ FROM ( , subq_0.listing__country_latest AS listing__country_latest , subq_0.listing__is_lux_latest AS listing__is_lux_latest , subq_0.listing__capacity_latest AS listing__capacity_latest - , subq_2.home_state_latest AS user__home_state_latest , subq_0.listings AS listings , subq_0.largest_listing AS largest_listing , subq_0.smallest_listing AS smallest_listing diff --git a/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/BigQuery/test_dimension_with_joined_where_constraint__plan0_optimized.sql b/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/BigQuery/test_dimension_with_joined_where_constraint__plan0_optimized.sql index c4e57d3c86..08f5aa9a14 100644 --- a/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/BigQuery/test_dimension_with_joined_where_constraint__plan0_optimized.sql +++ b/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/BigQuery/test_dimension_with_joined_where_constraint__plan0_optimized.sql @@ -5,8 +5,8 @@ SELECT FROM ( -- Join Standard Outputs SELECT - listings_latest_src_28000.country AS listing__country_latest - , users_latest_src_28000.home_state_latest AS user__home_state_latest + users_latest_src_28000.home_state_latest AS user__home_state_latest + , listings_latest_src_28000.country AS listing__country_latest FROM ***************************.dim_listings_latest listings_latest_src_28000 FULL OUTER JOIN ***************************.dim_users_latest users_latest_src_28000 diff --git a/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/BigQuery/test_dimensions_requiring_join__plan0.sql b/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/BigQuery/test_dimensions_requiring_join__plan0.sql index f112ad4699..63f88a3bf1 100644 --- a/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/BigQuery/test_dimensions_requiring_join__plan0.sql +++ b/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/BigQuery/test_dimensions_requiring_join__plan0.sql @@ -5,7 +5,8 @@ SELECT FROM ( -- Join Standard Outputs SELECT - subq_0.ds__day AS ds__day + subq_2.home_state_latest AS user__home_state_latest + , subq_0.ds__day AS ds__day , subq_0.ds__week AS ds__week , subq_0.ds__month AS ds__month , subq_0.ds__quarter AS ds__quarter @@ -58,7 +59,6 @@ FROM ( , subq_0.listing__country_latest AS listing__country_latest , subq_0.listing__is_lux_latest AS listing__is_lux_latest , subq_0.listing__capacity_latest AS listing__capacity_latest - , subq_2.home_state_latest AS user__home_state_latest , subq_0.listings AS listings , subq_0.largest_listing AS largest_listing , subq_0.smallest_listing AS smallest_listing diff --git a/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/BigQuery/test_multi_join_node__plan0.sql b/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/BigQuery/test_multi_join_node__plan0.sql index e6aedd6c08..86e179aee0 100644 --- a/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/BigQuery/test_multi_join_node__plan0.sql +++ b/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/BigQuery/test_multi_join_node__plan0.sql @@ -1,8 +1,8 @@ -- Join Standard Outputs SELECT - subq_1.listing AS listing - , subq_3.country_latest AS listing__country_latest + subq_3.country_latest AS listing__country_latest , subq_5.country_latest AS listing__country_latest + , subq_1.listing AS listing , subq_1.bookings AS bookings FROM ( -- Pass Only Elements: ['bookings', 'listing'] diff --git a/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/BigQuery/test_multi_join_node__plan0_optimized.sql b/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/BigQuery/test_multi_join_node__plan0_optimized.sql index 7d9c669f54..a880de7068 100644 --- a/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/BigQuery/test_multi_join_node__plan0_optimized.sql +++ b/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/BigQuery/test_multi_join_node__plan0_optimized.sql @@ -1,8 +1,8 @@ -- Join Standard Outputs SELECT - subq_7.listing AS listing - , subq_9.country_latest AS listing__country_latest + subq_9.country_latest AS listing__country_latest , subq_11.country_latest AS listing__country_latest + , subq_7.listing AS listing , subq_7.bookings AS bookings FROM ( -- Read Elements From Semantic Model 'bookings_source' diff --git a/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Databricks/test_compute_metrics_node__plan0.sql b/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Databricks/test_compute_metrics_node__plan0.sql index b6d83b8cf4..520c9ac9fa 100644 --- a/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Databricks/test_compute_metrics_node__plan0.sql +++ b/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Databricks/test_compute_metrics_node__plan0.sql @@ -12,8 +12,8 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_1.listing AS listing - , subq_3.country_latest AS listing__country_latest + subq_3.country_latest AS listing__country_latest + , subq_1.listing AS listing , subq_1.bookings AS bookings FROM ( -- Pass Only Elements: ['bookings', 'listing'] diff --git a/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Databricks/test_compute_metrics_node_ratio_from_multiple_semantic_models__plan0.sql b/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Databricks/test_compute_metrics_node_ratio_from_multiple_semantic_models__plan0.sql index 5c713dedac..c59a914110 100644 --- a/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Databricks/test_compute_metrics_node_ratio_from_multiple_semantic_models__plan0.sql +++ b/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Databricks/test_compute_metrics_node_ratio_from_multiple_semantic_models__plan0.sql @@ -31,9 +31,9 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.ds__day AS ds__day + subq_5.country_latest AS listing__country_latest + , subq_2.ds__day AS ds__day , subq_2.listing AS listing - , subq_5.country_latest AS listing__country_latest , subq_2.bookings AS bookings FROM ( -- Pass Only Elements: ['bookings', 'ds__day', 'listing'] @@ -405,9 +405,9 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_12.ds__day AS ds__day + subq_15.country_latest AS listing__country_latest + , subq_12.ds__day AS ds__day , subq_12.listing AS listing - , subq_15.country_latest AS listing__country_latest , subq_12.views AS views FROM ( -- Pass Only Elements: ['views', 'ds__day', 'listing'] diff --git a/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Databricks/test_compute_metrics_node_ratio_from_single_semantic_model__plan0.sql b/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Databricks/test_compute_metrics_node_ratio_from_single_semantic_model__plan0.sql index f5b5ee6cc2..d02d3dc0f8 100644 --- a/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Databricks/test_compute_metrics_node_ratio_from_single_semantic_model__plan0.sql +++ b/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Databricks/test_compute_metrics_node_ratio_from_single_semantic_model__plan0.sql @@ -13,8 +13,8 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_1.listing AS listing - , subq_3.country_latest AS listing__country_latest + subq_3.country_latest AS listing__country_latest + , subq_1.listing AS listing , subq_1.bookings AS bookings , subq_1.bookers AS bookers FROM ( diff --git a/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Databricks/test_compute_metrics_node_simple_expr__plan0.sql b/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Databricks/test_compute_metrics_node_simple_expr__plan0.sql index e0efb20972..d857fd72d4 100644 --- a/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Databricks/test_compute_metrics_node_simple_expr__plan0.sql +++ b/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Databricks/test_compute_metrics_node_simple_expr__plan0.sql @@ -12,8 +12,8 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_1.listing AS listing - , subq_3.country_latest AS listing__country_latest + subq_3.country_latest AS listing__country_latest + , subq_1.listing AS listing , subq_1.booking_value AS booking_value FROM ( -- Pass Only Elements: ['booking_value', 'listing'] diff --git a/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Databricks/test_dimension_with_joined_where_constraint__plan0.sql b/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Databricks/test_dimension_with_joined_where_constraint__plan0.sql index 1d8a2bddf2..7c26173807 100644 --- a/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Databricks/test_dimension_with_joined_where_constraint__plan0.sql +++ b/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Databricks/test_dimension_with_joined_where_constraint__plan0.sql @@ -64,7 +64,8 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_0.ds__day AS ds__day + subq_2.home_state_latest AS user__home_state_latest + , subq_0.ds__day AS ds__day , subq_0.ds__week AS ds__week , subq_0.ds__month AS ds__month , subq_0.ds__quarter AS ds__quarter @@ -117,7 +118,6 @@ FROM ( , subq_0.listing__country_latest AS listing__country_latest , subq_0.listing__is_lux_latest AS listing__is_lux_latest , subq_0.listing__capacity_latest AS listing__capacity_latest - , subq_2.home_state_latest AS user__home_state_latest , subq_0.listings AS listings , subq_0.largest_listing AS largest_listing , subq_0.smallest_listing AS smallest_listing diff --git a/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Databricks/test_dimension_with_joined_where_constraint__plan0_optimized.sql b/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Databricks/test_dimension_with_joined_where_constraint__plan0_optimized.sql index c4e57d3c86..08f5aa9a14 100644 --- a/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Databricks/test_dimension_with_joined_where_constraint__plan0_optimized.sql +++ b/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Databricks/test_dimension_with_joined_where_constraint__plan0_optimized.sql @@ -5,8 +5,8 @@ SELECT FROM ( -- Join Standard Outputs SELECT - listings_latest_src_28000.country AS listing__country_latest - , users_latest_src_28000.home_state_latest AS user__home_state_latest + users_latest_src_28000.home_state_latest AS user__home_state_latest + , listings_latest_src_28000.country AS listing__country_latest FROM ***************************.dim_listings_latest listings_latest_src_28000 FULL OUTER JOIN ***************************.dim_users_latest users_latest_src_28000 diff --git a/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Databricks/test_dimensions_requiring_join__plan0.sql b/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Databricks/test_dimensions_requiring_join__plan0.sql index d66932d7d4..5907829912 100644 --- a/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Databricks/test_dimensions_requiring_join__plan0.sql +++ b/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Databricks/test_dimensions_requiring_join__plan0.sql @@ -5,7 +5,8 @@ SELECT FROM ( -- Join Standard Outputs SELECT - subq_0.ds__day AS ds__day + subq_2.home_state_latest AS user__home_state_latest + , subq_0.ds__day AS ds__day , subq_0.ds__week AS ds__week , subq_0.ds__month AS ds__month , subq_0.ds__quarter AS ds__quarter @@ -58,7 +59,6 @@ FROM ( , subq_0.listing__country_latest AS listing__country_latest , subq_0.listing__is_lux_latest AS listing__is_lux_latest , subq_0.listing__capacity_latest AS listing__capacity_latest - , subq_2.home_state_latest AS user__home_state_latest , subq_0.listings AS listings , subq_0.largest_listing AS largest_listing , subq_0.smallest_listing AS smallest_listing diff --git a/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Databricks/test_multi_join_node__plan0.sql b/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Databricks/test_multi_join_node__plan0.sql index 27a7f61bce..2cd623dc07 100644 --- a/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Databricks/test_multi_join_node__plan0.sql +++ b/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Databricks/test_multi_join_node__plan0.sql @@ -1,8 +1,8 @@ -- Join Standard Outputs SELECT - subq_1.listing AS listing - , subq_3.country_latest AS listing__country_latest + subq_3.country_latest AS listing__country_latest , subq_5.country_latest AS listing__country_latest + , subq_1.listing AS listing , subq_1.bookings AS bookings FROM ( -- Pass Only Elements: ['bookings', 'listing'] diff --git a/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Databricks/test_multi_join_node__plan0_optimized.sql b/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Databricks/test_multi_join_node__plan0_optimized.sql index 7d9c669f54..a880de7068 100644 --- a/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Databricks/test_multi_join_node__plan0_optimized.sql +++ b/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Databricks/test_multi_join_node__plan0_optimized.sql @@ -1,8 +1,8 @@ -- Join Standard Outputs SELECT - subq_7.listing AS listing - , subq_9.country_latest AS listing__country_latest + subq_9.country_latest AS listing__country_latest , subq_11.country_latest AS listing__country_latest + , subq_7.listing AS listing , subq_7.bookings AS bookings FROM ( -- Read Elements From Semantic Model 'bookings_source' diff --git a/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/DuckDB/test_compute_metrics_node__plan0.sql b/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/DuckDB/test_compute_metrics_node__plan0.sql index 08fdfa6034..518be50ca5 100644 --- a/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/DuckDB/test_compute_metrics_node__plan0.sql +++ b/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/DuckDB/test_compute_metrics_node__plan0.sql @@ -12,8 +12,8 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_1.listing AS listing - , subq_3.country_latest AS listing__country_latest + subq_3.country_latest AS listing__country_latest + , subq_1.listing AS listing , subq_1.bookings AS bookings FROM ( -- Pass Only Elements: ['bookings', 'listing'] diff --git a/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/DuckDB/test_compute_metrics_node_ratio_from_multiple_semantic_models__plan0.sql b/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/DuckDB/test_compute_metrics_node_ratio_from_multiple_semantic_models__plan0.sql index d37cd06bd3..87475628d3 100644 --- a/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/DuckDB/test_compute_metrics_node_ratio_from_multiple_semantic_models__plan0.sql +++ b/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/DuckDB/test_compute_metrics_node_ratio_from_multiple_semantic_models__plan0.sql @@ -31,9 +31,9 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.ds__day AS ds__day + subq_5.country_latest AS listing__country_latest + , subq_2.ds__day AS ds__day , subq_2.listing AS listing - , subq_5.country_latest AS listing__country_latest , subq_2.bookings AS bookings FROM ( -- Pass Only Elements: ['bookings', 'ds__day', 'listing'] @@ -405,9 +405,9 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_12.ds__day AS ds__day + subq_15.country_latest AS listing__country_latest + , subq_12.ds__day AS ds__day , subq_12.listing AS listing - , subq_15.country_latest AS listing__country_latest , subq_12.views AS views FROM ( -- Pass Only Elements: ['views', 'ds__day', 'listing'] diff --git a/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/DuckDB/test_compute_metrics_node_ratio_from_single_semantic_model__plan0.sql b/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/DuckDB/test_compute_metrics_node_ratio_from_single_semantic_model__plan0.sql index c7aabe5705..868e41e121 100644 --- a/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/DuckDB/test_compute_metrics_node_ratio_from_single_semantic_model__plan0.sql +++ b/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/DuckDB/test_compute_metrics_node_ratio_from_single_semantic_model__plan0.sql @@ -13,8 +13,8 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_1.listing AS listing - , subq_3.country_latest AS listing__country_latest + subq_3.country_latest AS listing__country_latest + , subq_1.listing AS listing , subq_1.bookings AS bookings , subq_1.bookers AS bookers FROM ( diff --git a/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/DuckDB/test_compute_metrics_node_simple_expr__plan0.sql b/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/DuckDB/test_compute_metrics_node_simple_expr__plan0.sql index 069df1629d..dfbf89867b 100644 --- a/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/DuckDB/test_compute_metrics_node_simple_expr__plan0.sql +++ b/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/DuckDB/test_compute_metrics_node_simple_expr__plan0.sql @@ -12,8 +12,8 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_1.listing AS listing - , subq_3.country_latest AS listing__country_latest + subq_3.country_latest AS listing__country_latest + , subq_1.listing AS listing , subq_1.booking_value AS booking_value FROM ( -- Pass Only Elements: ['booking_value', 'listing'] diff --git a/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/DuckDB/test_dimension_with_joined_where_constraint__plan0.sql b/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/DuckDB/test_dimension_with_joined_where_constraint__plan0.sql index ed1028fb88..a254ce28b8 100644 --- a/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/DuckDB/test_dimension_with_joined_where_constraint__plan0.sql +++ b/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/DuckDB/test_dimension_with_joined_where_constraint__plan0.sql @@ -64,7 +64,8 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_0.ds__day AS ds__day + subq_2.home_state_latest AS user__home_state_latest + , subq_0.ds__day AS ds__day , subq_0.ds__week AS ds__week , subq_0.ds__month AS ds__month , subq_0.ds__quarter AS ds__quarter @@ -117,7 +118,6 @@ FROM ( , subq_0.listing__country_latest AS listing__country_latest , subq_0.listing__is_lux_latest AS listing__is_lux_latest , subq_0.listing__capacity_latest AS listing__capacity_latest - , subq_2.home_state_latest AS user__home_state_latest , subq_0.listings AS listings , subq_0.largest_listing AS largest_listing , subq_0.smallest_listing AS smallest_listing diff --git a/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/DuckDB/test_dimension_with_joined_where_constraint__plan0_optimized.sql b/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/DuckDB/test_dimension_with_joined_where_constraint__plan0_optimized.sql index c4e57d3c86..08f5aa9a14 100644 --- a/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/DuckDB/test_dimension_with_joined_where_constraint__plan0_optimized.sql +++ b/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/DuckDB/test_dimension_with_joined_where_constraint__plan0_optimized.sql @@ -5,8 +5,8 @@ SELECT FROM ( -- Join Standard Outputs SELECT - listings_latest_src_28000.country AS listing__country_latest - , users_latest_src_28000.home_state_latest AS user__home_state_latest + users_latest_src_28000.home_state_latest AS user__home_state_latest + , listings_latest_src_28000.country AS listing__country_latest FROM ***************************.dim_listings_latest listings_latest_src_28000 FULL OUTER JOIN ***************************.dim_users_latest users_latest_src_28000 diff --git a/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/DuckDB/test_dimensions_requiring_join__plan0.sql b/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/DuckDB/test_dimensions_requiring_join__plan0.sql index 0962aa9958..e4d9c4cec1 100644 --- a/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/DuckDB/test_dimensions_requiring_join__plan0.sql +++ b/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/DuckDB/test_dimensions_requiring_join__plan0.sql @@ -5,7 +5,8 @@ SELECT FROM ( -- Join Standard Outputs SELECT - subq_0.ds__day AS ds__day + subq_2.home_state_latest AS user__home_state_latest + , subq_0.ds__day AS ds__day , subq_0.ds__week AS ds__week , subq_0.ds__month AS ds__month , subq_0.ds__quarter AS ds__quarter @@ -58,7 +59,6 @@ FROM ( , subq_0.listing__country_latest AS listing__country_latest , subq_0.listing__is_lux_latest AS listing__is_lux_latest , subq_0.listing__capacity_latest AS listing__capacity_latest - , subq_2.home_state_latest AS user__home_state_latest , subq_0.listings AS listings , subq_0.largest_listing AS largest_listing , subq_0.smallest_listing AS smallest_listing diff --git a/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/DuckDB/test_multi_join_node__plan0.sql b/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/DuckDB/test_multi_join_node__plan0.sql index f3ed9fa304..9607966cf1 100644 --- a/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/DuckDB/test_multi_join_node__plan0.sql +++ b/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/DuckDB/test_multi_join_node__plan0.sql @@ -1,8 +1,8 @@ -- Join Standard Outputs SELECT - subq_1.listing AS listing - , subq_3.country_latest AS listing__country_latest + subq_3.country_latest AS listing__country_latest , subq_5.country_latest AS listing__country_latest + , subq_1.listing AS listing , subq_1.bookings AS bookings FROM ( -- Pass Only Elements: ['bookings', 'listing'] diff --git a/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/DuckDB/test_multi_join_node__plan0_optimized.sql b/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/DuckDB/test_multi_join_node__plan0_optimized.sql index 7d9c669f54..a880de7068 100644 --- a/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/DuckDB/test_multi_join_node__plan0_optimized.sql +++ b/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/DuckDB/test_multi_join_node__plan0_optimized.sql @@ -1,8 +1,8 @@ -- Join Standard Outputs SELECT - subq_7.listing AS listing - , subq_9.country_latest AS listing__country_latest + subq_9.country_latest AS listing__country_latest , subq_11.country_latest AS listing__country_latest + , subq_7.listing AS listing , subq_7.bookings AS bookings FROM ( -- Read Elements From Semantic Model 'bookings_source' diff --git a/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Postgres/test_compute_metrics_node__plan0.sql b/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Postgres/test_compute_metrics_node__plan0.sql index 08fdfa6034..518be50ca5 100644 --- a/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Postgres/test_compute_metrics_node__plan0.sql +++ b/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Postgres/test_compute_metrics_node__plan0.sql @@ -12,8 +12,8 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_1.listing AS listing - , subq_3.country_latest AS listing__country_latest + subq_3.country_latest AS listing__country_latest + , subq_1.listing AS listing , subq_1.bookings AS bookings FROM ( -- Pass Only Elements: ['bookings', 'listing'] diff --git a/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Postgres/test_compute_metrics_node_ratio_from_multiple_semantic_models__plan0.sql b/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Postgres/test_compute_metrics_node_ratio_from_multiple_semantic_models__plan0.sql index e3b7fa954b..0faa937f47 100644 --- a/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Postgres/test_compute_metrics_node_ratio_from_multiple_semantic_models__plan0.sql +++ b/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Postgres/test_compute_metrics_node_ratio_from_multiple_semantic_models__plan0.sql @@ -31,9 +31,9 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.ds__day AS ds__day + subq_5.country_latest AS listing__country_latest + , subq_2.ds__day AS ds__day , subq_2.listing AS listing - , subq_5.country_latest AS listing__country_latest , subq_2.bookings AS bookings FROM ( -- Pass Only Elements: ['bookings', 'ds__day', 'listing'] @@ -405,9 +405,9 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_12.ds__day AS ds__day + subq_15.country_latest AS listing__country_latest + , subq_12.ds__day AS ds__day , subq_12.listing AS listing - , subq_15.country_latest AS listing__country_latest , subq_12.views AS views FROM ( -- Pass Only Elements: ['views', 'ds__day', 'listing'] diff --git a/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Postgres/test_compute_metrics_node_ratio_from_single_semantic_model__plan0.sql b/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Postgres/test_compute_metrics_node_ratio_from_single_semantic_model__plan0.sql index 01a2b684f3..86723995a1 100644 --- a/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Postgres/test_compute_metrics_node_ratio_from_single_semantic_model__plan0.sql +++ b/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Postgres/test_compute_metrics_node_ratio_from_single_semantic_model__plan0.sql @@ -13,8 +13,8 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_1.listing AS listing - , subq_3.country_latest AS listing__country_latest + subq_3.country_latest AS listing__country_latest + , subq_1.listing AS listing , subq_1.bookings AS bookings , subq_1.bookers AS bookers FROM ( diff --git a/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Postgres/test_compute_metrics_node_simple_expr__plan0.sql b/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Postgres/test_compute_metrics_node_simple_expr__plan0.sql index 069df1629d..dfbf89867b 100644 --- a/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Postgres/test_compute_metrics_node_simple_expr__plan0.sql +++ b/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Postgres/test_compute_metrics_node_simple_expr__plan0.sql @@ -12,8 +12,8 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_1.listing AS listing - , subq_3.country_latest AS listing__country_latest + subq_3.country_latest AS listing__country_latest + , subq_1.listing AS listing , subq_1.booking_value AS booking_value FROM ( -- Pass Only Elements: ['booking_value', 'listing'] diff --git a/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Postgres/test_dimension_with_joined_where_constraint__plan0.sql b/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Postgres/test_dimension_with_joined_where_constraint__plan0.sql index ed1028fb88..a254ce28b8 100644 --- a/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Postgres/test_dimension_with_joined_where_constraint__plan0.sql +++ b/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Postgres/test_dimension_with_joined_where_constraint__plan0.sql @@ -64,7 +64,8 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_0.ds__day AS ds__day + subq_2.home_state_latest AS user__home_state_latest + , subq_0.ds__day AS ds__day , subq_0.ds__week AS ds__week , subq_0.ds__month AS ds__month , subq_0.ds__quarter AS ds__quarter @@ -117,7 +118,6 @@ FROM ( , subq_0.listing__country_latest AS listing__country_latest , subq_0.listing__is_lux_latest AS listing__is_lux_latest , subq_0.listing__capacity_latest AS listing__capacity_latest - , subq_2.home_state_latest AS user__home_state_latest , subq_0.listings AS listings , subq_0.largest_listing AS largest_listing , subq_0.smallest_listing AS smallest_listing diff --git a/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Postgres/test_dimension_with_joined_where_constraint__plan0_optimized.sql b/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Postgres/test_dimension_with_joined_where_constraint__plan0_optimized.sql index c4e57d3c86..08f5aa9a14 100644 --- a/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Postgres/test_dimension_with_joined_where_constraint__plan0_optimized.sql +++ b/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Postgres/test_dimension_with_joined_where_constraint__plan0_optimized.sql @@ -5,8 +5,8 @@ SELECT FROM ( -- Join Standard Outputs SELECT - listings_latest_src_28000.country AS listing__country_latest - , users_latest_src_28000.home_state_latest AS user__home_state_latest + users_latest_src_28000.home_state_latest AS user__home_state_latest + , listings_latest_src_28000.country AS listing__country_latest FROM ***************************.dim_listings_latest listings_latest_src_28000 FULL OUTER JOIN ***************************.dim_users_latest users_latest_src_28000 diff --git a/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Postgres/test_dimensions_requiring_join__plan0.sql b/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Postgres/test_dimensions_requiring_join__plan0.sql index 0962aa9958..e4d9c4cec1 100644 --- a/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Postgres/test_dimensions_requiring_join__plan0.sql +++ b/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Postgres/test_dimensions_requiring_join__plan0.sql @@ -5,7 +5,8 @@ SELECT FROM ( -- Join Standard Outputs SELECT - subq_0.ds__day AS ds__day + subq_2.home_state_latest AS user__home_state_latest + , subq_0.ds__day AS ds__day , subq_0.ds__week AS ds__week , subq_0.ds__month AS ds__month , subq_0.ds__quarter AS ds__quarter @@ -58,7 +59,6 @@ FROM ( , subq_0.listing__country_latest AS listing__country_latest , subq_0.listing__is_lux_latest AS listing__is_lux_latest , subq_0.listing__capacity_latest AS listing__capacity_latest - , subq_2.home_state_latest AS user__home_state_latest , subq_0.listings AS listings , subq_0.largest_listing AS largest_listing , subq_0.smallest_listing AS smallest_listing diff --git a/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Postgres/test_multi_join_node__plan0.sql b/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Postgres/test_multi_join_node__plan0.sql index f3ed9fa304..9607966cf1 100644 --- a/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Postgres/test_multi_join_node__plan0.sql +++ b/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Postgres/test_multi_join_node__plan0.sql @@ -1,8 +1,8 @@ -- Join Standard Outputs SELECT - subq_1.listing AS listing - , subq_3.country_latest AS listing__country_latest + subq_3.country_latest AS listing__country_latest , subq_5.country_latest AS listing__country_latest + , subq_1.listing AS listing , subq_1.bookings AS bookings FROM ( -- Pass Only Elements: ['bookings', 'listing'] diff --git a/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Postgres/test_multi_join_node__plan0_optimized.sql b/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Postgres/test_multi_join_node__plan0_optimized.sql index 7d9c669f54..a880de7068 100644 --- a/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Postgres/test_multi_join_node__plan0_optimized.sql +++ b/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Postgres/test_multi_join_node__plan0_optimized.sql @@ -1,8 +1,8 @@ -- Join Standard Outputs SELECT - subq_7.listing AS listing - , subq_9.country_latest AS listing__country_latest + subq_9.country_latest AS listing__country_latest , subq_11.country_latest AS listing__country_latest + , subq_7.listing AS listing , subq_7.bookings AS bookings FROM ( -- Read Elements From Semantic Model 'bookings_source' diff --git a/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Redshift/test_compute_metrics_node__plan0.sql b/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Redshift/test_compute_metrics_node__plan0.sql index a94865f00e..61ccf9b206 100644 --- a/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Redshift/test_compute_metrics_node__plan0.sql +++ b/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Redshift/test_compute_metrics_node__plan0.sql @@ -12,8 +12,8 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_1.listing AS listing - , subq_3.country_latest AS listing__country_latest + subq_3.country_latest AS listing__country_latest + , subq_1.listing AS listing , subq_1.bookings AS bookings FROM ( -- Pass Only Elements: ['bookings', 'listing'] diff --git a/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Redshift/test_compute_metrics_node_ratio_from_multiple_semantic_models__plan0.sql b/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Redshift/test_compute_metrics_node_ratio_from_multiple_semantic_models__plan0.sql index 60ca0d5196..f5832b8060 100644 --- a/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Redshift/test_compute_metrics_node_ratio_from_multiple_semantic_models__plan0.sql +++ b/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Redshift/test_compute_metrics_node_ratio_from_multiple_semantic_models__plan0.sql @@ -31,9 +31,9 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.ds__day AS ds__day + subq_5.country_latest AS listing__country_latest + , subq_2.ds__day AS ds__day , subq_2.listing AS listing - , subq_5.country_latest AS listing__country_latest , subq_2.bookings AS bookings FROM ( -- Pass Only Elements: ['bookings', 'ds__day', 'listing'] @@ -405,9 +405,9 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_12.ds__day AS ds__day + subq_15.country_latest AS listing__country_latest + , subq_12.ds__day AS ds__day , subq_12.listing AS listing - , subq_15.country_latest AS listing__country_latest , subq_12.views AS views FROM ( -- Pass Only Elements: ['views', 'ds__day', 'listing'] diff --git a/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Redshift/test_compute_metrics_node_ratio_from_single_semantic_model__plan0.sql b/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Redshift/test_compute_metrics_node_ratio_from_single_semantic_model__plan0.sql index c8f22b47b7..1b15e0c4b9 100644 --- a/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Redshift/test_compute_metrics_node_ratio_from_single_semantic_model__plan0.sql +++ b/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Redshift/test_compute_metrics_node_ratio_from_single_semantic_model__plan0.sql @@ -13,8 +13,8 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_1.listing AS listing - , subq_3.country_latest AS listing__country_latest + subq_3.country_latest AS listing__country_latest + , subq_1.listing AS listing , subq_1.bookings AS bookings , subq_1.bookers AS bookers FROM ( diff --git a/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Redshift/test_compute_metrics_node_simple_expr__plan0.sql b/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Redshift/test_compute_metrics_node_simple_expr__plan0.sql index a332bba687..49d0e9dc56 100644 --- a/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Redshift/test_compute_metrics_node_simple_expr__plan0.sql +++ b/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Redshift/test_compute_metrics_node_simple_expr__plan0.sql @@ -12,8 +12,8 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_1.listing AS listing - , subq_3.country_latest AS listing__country_latest + subq_3.country_latest AS listing__country_latest + , subq_1.listing AS listing , subq_1.booking_value AS booking_value FROM ( -- Pass Only Elements: ['booking_value', 'listing'] diff --git a/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Redshift/test_dimension_with_joined_where_constraint__plan0.sql b/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Redshift/test_dimension_with_joined_where_constraint__plan0.sql index c59a5e39af..95966da9a0 100644 --- a/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Redshift/test_dimension_with_joined_where_constraint__plan0.sql +++ b/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Redshift/test_dimension_with_joined_where_constraint__plan0.sql @@ -64,7 +64,8 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_0.ds__day AS ds__day + subq_2.home_state_latest AS user__home_state_latest + , subq_0.ds__day AS ds__day , subq_0.ds__week AS ds__week , subq_0.ds__month AS ds__month , subq_0.ds__quarter AS ds__quarter @@ -117,7 +118,6 @@ FROM ( , subq_0.listing__country_latest AS listing__country_latest , subq_0.listing__is_lux_latest AS listing__is_lux_latest , subq_0.listing__capacity_latest AS listing__capacity_latest - , subq_2.home_state_latest AS user__home_state_latest , subq_0.listings AS listings , subq_0.largest_listing AS largest_listing , subq_0.smallest_listing AS smallest_listing diff --git a/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Redshift/test_dimension_with_joined_where_constraint__plan0_optimized.sql b/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Redshift/test_dimension_with_joined_where_constraint__plan0_optimized.sql index c4e57d3c86..08f5aa9a14 100644 --- a/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Redshift/test_dimension_with_joined_where_constraint__plan0_optimized.sql +++ b/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Redshift/test_dimension_with_joined_where_constraint__plan0_optimized.sql @@ -5,8 +5,8 @@ SELECT FROM ( -- Join Standard Outputs SELECT - listings_latest_src_28000.country AS listing__country_latest - , users_latest_src_28000.home_state_latest AS user__home_state_latest + users_latest_src_28000.home_state_latest AS user__home_state_latest + , listings_latest_src_28000.country AS listing__country_latest FROM ***************************.dim_listings_latest listings_latest_src_28000 FULL OUTER JOIN ***************************.dim_users_latest users_latest_src_28000 diff --git a/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Redshift/test_dimensions_requiring_join__plan0.sql b/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Redshift/test_dimensions_requiring_join__plan0.sql index 6ada21e5f4..11790f4d9c 100644 --- a/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Redshift/test_dimensions_requiring_join__plan0.sql +++ b/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Redshift/test_dimensions_requiring_join__plan0.sql @@ -5,7 +5,8 @@ SELECT FROM ( -- Join Standard Outputs SELECT - subq_0.ds__day AS ds__day + subq_2.home_state_latest AS user__home_state_latest + , subq_0.ds__day AS ds__day , subq_0.ds__week AS ds__week , subq_0.ds__month AS ds__month , subq_0.ds__quarter AS ds__quarter @@ -58,7 +59,6 @@ FROM ( , subq_0.listing__country_latest AS listing__country_latest , subq_0.listing__is_lux_latest AS listing__is_lux_latest , subq_0.listing__capacity_latest AS listing__capacity_latest - , subq_2.home_state_latest AS user__home_state_latest , subq_0.listings AS listings , subq_0.largest_listing AS largest_listing , subq_0.smallest_listing AS smallest_listing diff --git a/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Redshift/test_multi_join_node__plan0.sql b/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Redshift/test_multi_join_node__plan0.sql index 7c1221217a..b44d0eea85 100644 --- a/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Redshift/test_multi_join_node__plan0.sql +++ b/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Redshift/test_multi_join_node__plan0.sql @@ -1,8 +1,8 @@ -- Join Standard Outputs SELECT - subq_1.listing AS listing - , subq_3.country_latest AS listing__country_latest + subq_3.country_latest AS listing__country_latest , subq_5.country_latest AS listing__country_latest + , subq_1.listing AS listing , subq_1.bookings AS bookings FROM ( -- Pass Only Elements: ['bookings', 'listing'] diff --git a/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Redshift/test_multi_join_node__plan0_optimized.sql b/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Redshift/test_multi_join_node__plan0_optimized.sql index 7d9c669f54..a880de7068 100644 --- a/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Redshift/test_multi_join_node__plan0_optimized.sql +++ b/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Redshift/test_multi_join_node__plan0_optimized.sql @@ -1,8 +1,8 @@ -- Join Standard Outputs SELECT - subq_7.listing AS listing - , subq_9.country_latest AS listing__country_latest + subq_9.country_latest AS listing__country_latest , subq_11.country_latest AS listing__country_latest + , subq_7.listing AS listing , subq_7.bookings AS bookings FROM ( -- Read Elements From Semantic Model 'bookings_source' diff --git a/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Snowflake/test_compute_metrics_node__plan0.sql b/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Snowflake/test_compute_metrics_node__plan0.sql index 323ea29003..2a4bb76263 100644 --- a/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Snowflake/test_compute_metrics_node__plan0.sql +++ b/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Snowflake/test_compute_metrics_node__plan0.sql @@ -12,8 +12,8 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_1.listing AS listing - , subq_3.country_latest AS listing__country_latest + subq_3.country_latest AS listing__country_latest + , subq_1.listing AS listing , subq_1.bookings AS bookings FROM ( -- Pass Only Elements: ['bookings', 'listing'] diff --git a/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Snowflake/test_compute_metrics_node_ratio_from_multiple_semantic_models__plan0.sql b/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Snowflake/test_compute_metrics_node_ratio_from_multiple_semantic_models__plan0.sql index 35db359fef..cfc49a14d2 100644 --- a/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Snowflake/test_compute_metrics_node_ratio_from_multiple_semantic_models__plan0.sql +++ b/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Snowflake/test_compute_metrics_node_ratio_from_multiple_semantic_models__plan0.sql @@ -31,9 +31,9 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.ds__day AS ds__day + subq_5.country_latest AS listing__country_latest + , subq_2.ds__day AS ds__day , subq_2.listing AS listing - , subq_5.country_latest AS listing__country_latest , subq_2.bookings AS bookings FROM ( -- Pass Only Elements: ['bookings', 'ds__day', 'listing'] @@ -405,9 +405,9 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_12.ds__day AS ds__day + subq_15.country_latest AS listing__country_latest + , subq_12.ds__day AS ds__day , subq_12.listing AS listing - , subq_15.country_latest AS listing__country_latest , subq_12.views AS views FROM ( -- Pass Only Elements: ['views', 'ds__day', 'listing'] diff --git a/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Snowflake/test_compute_metrics_node_ratio_from_single_semantic_model__plan0.sql b/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Snowflake/test_compute_metrics_node_ratio_from_single_semantic_model__plan0.sql index 390392dacf..dad00e3fda 100644 --- a/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Snowflake/test_compute_metrics_node_ratio_from_single_semantic_model__plan0.sql +++ b/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Snowflake/test_compute_metrics_node_ratio_from_single_semantic_model__plan0.sql @@ -13,8 +13,8 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_1.listing AS listing - , subq_3.country_latest AS listing__country_latest + subq_3.country_latest AS listing__country_latest + , subq_1.listing AS listing , subq_1.bookings AS bookings , subq_1.bookers AS bookers FROM ( diff --git a/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Snowflake/test_compute_metrics_node_simple_expr__plan0.sql b/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Snowflake/test_compute_metrics_node_simple_expr__plan0.sql index 038022f027..6b8810b2f6 100644 --- a/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Snowflake/test_compute_metrics_node_simple_expr__plan0.sql +++ b/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Snowflake/test_compute_metrics_node_simple_expr__plan0.sql @@ -12,8 +12,8 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_1.listing AS listing - , subq_3.country_latest AS listing__country_latest + subq_3.country_latest AS listing__country_latest + , subq_1.listing AS listing , subq_1.booking_value AS booking_value FROM ( -- Pass Only Elements: ['booking_value', 'listing'] diff --git a/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Snowflake/test_dimension_with_joined_where_constraint__plan0.sql b/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Snowflake/test_dimension_with_joined_where_constraint__plan0.sql index d1dedc82a3..0adb71aabf 100644 --- a/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Snowflake/test_dimension_with_joined_where_constraint__plan0.sql +++ b/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Snowflake/test_dimension_with_joined_where_constraint__plan0.sql @@ -64,7 +64,8 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_0.ds__day AS ds__day + subq_2.home_state_latest AS user__home_state_latest + , subq_0.ds__day AS ds__day , subq_0.ds__week AS ds__week , subq_0.ds__month AS ds__month , subq_0.ds__quarter AS ds__quarter @@ -117,7 +118,6 @@ FROM ( , subq_0.listing__country_latest AS listing__country_latest , subq_0.listing__is_lux_latest AS listing__is_lux_latest , subq_0.listing__capacity_latest AS listing__capacity_latest - , subq_2.home_state_latest AS user__home_state_latest , subq_0.listings AS listings , subq_0.largest_listing AS largest_listing , subq_0.smallest_listing AS smallest_listing diff --git a/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Snowflake/test_dimension_with_joined_where_constraint__plan0_optimized.sql b/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Snowflake/test_dimension_with_joined_where_constraint__plan0_optimized.sql index c4e57d3c86..08f5aa9a14 100644 --- a/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Snowflake/test_dimension_with_joined_where_constraint__plan0_optimized.sql +++ b/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Snowflake/test_dimension_with_joined_where_constraint__plan0_optimized.sql @@ -5,8 +5,8 @@ SELECT FROM ( -- Join Standard Outputs SELECT - listings_latest_src_28000.country AS listing__country_latest - , users_latest_src_28000.home_state_latest AS user__home_state_latest + users_latest_src_28000.home_state_latest AS user__home_state_latest + , listings_latest_src_28000.country AS listing__country_latest FROM ***************************.dim_listings_latest listings_latest_src_28000 FULL OUTER JOIN ***************************.dim_users_latest users_latest_src_28000 diff --git a/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Snowflake/test_dimensions_requiring_join__plan0.sql b/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Snowflake/test_dimensions_requiring_join__plan0.sql index 651824a182..cd3786df0e 100644 --- a/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Snowflake/test_dimensions_requiring_join__plan0.sql +++ b/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Snowflake/test_dimensions_requiring_join__plan0.sql @@ -5,7 +5,8 @@ SELECT FROM ( -- Join Standard Outputs SELECT - subq_0.ds__day AS ds__day + subq_2.home_state_latest AS user__home_state_latest + , subq_0.ds__day AS ds__day , subq_0.ds__week AS ds__week , subq_0.ds__month AS ds__month , subq_0.ds__quarter AS ds__quarter @@ -58,7 +59,6 @@ FROM ( , subq_0.listing__country_latest AS listing__country_latest , subq_0.listing__is_lux_latest AS listing__is_lux_latest , subq_0.listing__capacity_latest AS listing__capacity_latest - , subq_2.home_state_latest AS user__home_state_latest , subq_0.listings AS listings , subq_0.largest_listing AS largest_listing , subq_0.smallest_listing AS smallest_listing diff --git a/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Snowflake/test_multi_join_node__plan0.sql b/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Snowflake/test_multi_join_node__plan0.sql index 4e470ff794..7a3efa023f 100644 --- a/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Snowflake/test_multi_join_node__plan0.sql +++ b/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Snowflake/test_multi_join_node__plan0.sql @@ -1,8 +1,8 @@ -- Join Standard Outputs SELECT - subq_1.listing AS listing - , subq_3.country_latest AS listing__country_latest + subq_3.country_latest AS listing__country_latest , subq_5.country_latest AS listing__country_latest + , subq_1.listing AS listing , subq_1.bookings AS bookings FROM ( -- Pass Only Elements: ['bookings', 'listing'] diff --git a/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Snowflake/test_multi_join_node__plan0_optimized.sql b/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Snowflake/test_multi_join_node__plan0_optimized.sql index 7d9c669f54..a880de7068 100644 --- a/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Snowflake/test_multi_join_node__plan0_optimized.sql +++ b/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Snowflake/test_multi_join_node__plan0_optimized.sql @@ -1,8 +1,8 @@ -- Join Standard Outputs SELECT - subq_7.listing AS listing - , subq_9.country_latest AS listing__country_latest + subq_9.country_latest AS listing__country_latest , subq_11.country_latest AS listing__country_latest + , subq_7.listing AS listing , subq_7.bookings AS bookings FROM ( -- Read Elements From Semantic Model 'bookings_source' diff --git a/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Trino/test_compute_metrics_node__plan0.sql b/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Trino/test_compute_metrics_node__plan0.sql index 169a72688a..c5555e9030 100644 --- a/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Trino/test_compute_metrics_node__plan0.sql +++ b/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Trino/test_compute_metrics_node__plan0.sql @@ -12,8 +12,8 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_1.listing AS listing - , subq_3.country_latest AS listing__country_latest + subq_3.country_latest AS listing__country_latest + , subq_1.listing AS listing , subq_1.bookings AS bookings FROM ( -- Pass Only Elements: ['bookings', 'listing'] diff --git a/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Trino/test_compute_metrics_node_ratio_from_multiple_semantic_models__plan0.sql b/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Trino/test_compute_metrics_node_ratio_from_multiple_semantic_models__plan0.sql index 66a79bed4c..a00d94921e 100644 --- a/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Trino/test_compute_metrics_node_ratio_from_multiple_semantic_models__plan0.sql +++ b/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Trino/test_compute_metrics_node_ratio_from_multiple_semantic_models__plan0.sql @@ -31,9 +31,9 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.ds__day AS ds__day + subq_5.country_latest AS listing__country_latest + , subq_2.ds__day AS ds__day , subq_2.listing AS listing - , subq_5.country_latest AS listing__country_latest , subq_2.bookings AS bookings FROM ( -- Pass Only Elements: ['bookings', 'ds__day', 'listing'] @@ -405,9 +405,9 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_12.ds__day AS ds__day + subq_15.country_latest AS listing__country_latest + , subq_12.ds__day AS ds__day , subq_12.listing AS listing - , subq_15.country_latest AS listing__country_latest , subq_12.views AS views FROM ( -- Pass Only Elements: ['views', 'ds__day', 'listing'] diff --git a/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Trino/test_compute_metrics_node_ratio_from_single_semantic_model__plan0.sql b/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Trino/test_compute_metrics_node_ratio_from_single_semantic_model__plan0.sql index c2f25848d4..da397798f1 100644 --- a/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Trino/test_compute_metrics_node_ratio_from_single_semantic_model__plan0.sql +++ b/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Trino/test_compute_metrics_node_ratio_from_single_semantic_model__plan0.sql @@ -13,8 +13,8 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_1.listing AS listing - , subq_3.country_latest AS listing__country_latest + subq_3.country_latest AS listing__country_latest + , subq_1.listing AS listing , subq_1.bookings AS bookings , subq_1.bookers AS bookers FROM ( diff --git a/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Trino/test_compute_metrics_node_simple_expr__plan0.sql b/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Trino/test_compute_metrics_node_simple_expr__plan0.sql index 1d72a8ab11..6ad2ff6da7 100644 --- a/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Trino/test_compute_metrics_node_simple_expr__plan0.sql +++ b/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Trino/test_compute_metrics_node_simple_expr__plan0.sql @@ -12,8 +12,8 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_1.listing AS listing - , subq_3.country_latest AS listing__country_latest + subq_3.country_latest AS listing__country_latest + , subq_1.listing AS listing , subq_1.booking_value AS booking_value FROM ( -- Pass Only Elements: ['booking_value', 'listing'] diff --git a/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Trino/test_dimension_with_joined_where_constraint__plan0.sql b/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Trino/test_dimension_with_joined_where_constraint__plan0.sql index c30f2c64e7..a53b66b1c1 100644 --- a/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Trino/test_dimension_with_joined_where_constraint__plan0.sql +++ b/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Trino/test_dimension_with_joined_where_constraint__plan0.sql @@ -64,7 +64,8 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_0.ds__day AS ds__day + subq_2.home_state_latest AS user__home_state_latest + , subq_0.ds__day AS ds__day , subq_0.ds__week AS ds__week , subq_0.ds__month AS ds__month , subq_0.ds__quarter AS ds__quarter @@ -117,7 +118,6 @@ FROM ( , subq_0.listing__country_latest AS listing__country_latest , subq_0.listing__is_lux_latest AS listing__is_lux_latest , subq_0.listing__capacity_latest AS listing__capacity_latest - , subq_2.home_state_latest AS user__home_state_latest , subq_0.listings AS listings , subq_0.largest_listing AS largest_listing , subq_0.smallest_listing AS smallest_listing diff --git a/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Trino/test_dimension_with_joined_where_constraint__plan0_optimized.sql b/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Trino/test_dimension_with_joined_where_constraint__plan0_optimized.sql index c4e57d3c86..08f5aa9a14 100644 --- a/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Trino/test_dimension_with_joined_where_constraint__plan0_optimized.sql +++ b/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Trino/test_dimension_with_joined_where_constraint__plan0_optimized.sql @@ -5,8 +5,8 @@ SELECT FROM ( -- Join Standard Outputs SELECT - listings_latest_src_28000.country AS listing__country_latest - , users_latest_src_28000.home_state_latest AS user__home_state_latest + users_latest_src_28000.home_state_latest AS user__home_state_latest + , listings_latest_src_28000.country AS listing__country_latest FROM ***************************.dim_listings_latest listings_latest_src_28000 FULL OUTER JOIN ***************************.dim_users_latest users_latest_src_28000 diff --git a/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Trino/test_dimensions_requiring_join__plan0.sql b/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Trino/test_dimensions_requiring_join__plan0.sql index 28f04502f2..007d0aac57 100644 --- a/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Trino/test_dimensions_requiring_join__plan0.sql +++ b/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Trino/test_dimensions_requiring_join__plan0.sql @@ -5,7 +5,8 @@ SELECT FROM ( -- Join Standard Outputs SELECT - subq_0.ds__day AS ds__day + subq_2.home_state_latest AS user__home_state_latest + , subq_0.ds__day AS ds__day , subq_0.ds__week AS ds__week , subq_0.ds__month AS ds__month , subq_0.ds__quarter AS ds__quarter @@ -58,7 +59,6 @@ FROM ( , subq_0.listing__country_latest AS listing__country_latest , subq_0.listing__is_lux_latest AS listing__is_lux_latest , subq_0.listing__capacity_latest AS listing__capacity_latest - , subq_2.home_state_latest AS user__home_state_latest , subq_0.listings AS listings , subq_0.largest_listing AS largest_listing , subq_0.smallest_listing AS smallest_listing diff --git a/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Trino/test_multi_join_node__plan0.sql b/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Trino/test_multi_join_node__plan0.sql index bf4d3b9719..74288b057c 100644 --- a/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Trino/test_multi_join_node__plan0.sql +++ b/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Trino/test_multi_join_node__plan0.sql @@ -1,8 +1,8 @@ -- Join Standard Outputs SELECT - subq_1.listing AS listing - , subq_3.country_latest AS listing__country_latest + subq_3.country_latest AS listing__country_latest , subq_5.country_latest AS listing__country_latest + , subq_1.listing AS listing , subq_1.bookings AS bookings FROM ( -- Pass Only Elements: ['bookings', 'listing'] diff --git a/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Trino/test_multi_join_node__plan0_optimized.sql b/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Trino/test_multi_join_node__plan0_optimized.sql index 7d9c669f54..a880de7068 100644 --- a/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Trino/test_multi_join_node__plan0_optimized.sql +++ b/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Trino/test_multi_join_node__plan0_optimized.sql @@ -1,8 +1,8 @@ -- Join Standard Outputs SELECT - subq_7.listing AS listing - , subq_9.country_latest AS listing__country_latest + subq_9.country_latest AS listing__country_latest , subq_11.country_latest AS listing__country_latest + , subq_7.listing AS listing , subq_7.bookings AS bookings FROM ( -- Read Elements From Semantic Model 'bookings_source' diff --git a/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/test_compute_metrics_node__plan0.xml b/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/test_compute_metrics_node__plan0.xml index 41a0d7fadf..0977ce5781 100644 --- a/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/test_compute_metrics_node__plan0.xml +++ b/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/test_compute_metrics_node__plan0.xml @@ -35,13 +35,13 @@ - - + - + - + + diff --git a/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/test_compute_metrics_node_ratio_from_multiple_semantic_models__plan0.xml b/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/test_compute_metrics_node_ratio_from_multiple_semantic_models__plan0.xml index 9689428e69..d46d963d11 100644 --- a/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/test_compute_metrics_node_ratio_from_multiple_semantic_models__plan0.xml +++ b/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/test_compute_metrics_node_ratio_from_multiple_semantic_models__plan0.xml @@ -110,22 +110,22 @@ - - + + - + - + - + @@ -1823,22 +1823,22 @@ - - + + - + - + - + diff --git a/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/test_compute_metrics_node_ratio_from_single_semantic_model__plan0.xml b/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/test_compute_metrics_node_ratio_from_single_semantic_model__plan0.xml index b7e81ad415..224fa33647 100644 --- a/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/test_compute_metrics_node_ratio_from_single_semantic_model__plan0.xml +++ b/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/test_compute_metrics_node_ratio_from_single_semantic_model__plan0.xml @@ -41,14 +41,14 @@ - - - - - - - - + + + + + + + + diff --git a/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/test_compute_metrics_node_simple_expr__plan0.xml b/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/test_compute_metrics_node_simple_expr__plan0.xml index c4e2b19ee7..f693d083f3 100644 --- a/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/test_compute_metrics_node_simple_expr__plan0.xml +++ b/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/test_compute_metrics_node_simple_expr__plan0.xml @@ -39,14 +39,14 @@ - - + - + + - + diff --git a/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/test_dimension_with_joined_where_constraint__plan0.xml b/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/test_dimension_with_joined_where_constraint__plan0.xml index 666304faf3..074fb50317 100644 --- a/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/test_dimension_with_joined_where_constraint__plan0.xml +++ b/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/test_dimension_with_joined_where_constraint__plan0.xml @@ -204,206 +204,206 @@ - - - - - - - - - + + + + + + + + + + + + + + - + - - - - + + - + - + - - - - - - + + + + + + + + - + - - - - - - - + + - + - + - + - + - + - - - - - + - - + + + + + + - + - - - + - - + + + + - + - + - + - + - + - + - + - + - + - + + + + + + - + - + - + - + - - - - - - + - + - - - - - - - - - - - + - - + + + + + + + + + + + + - + - + - - + + - + - + - + diff --git a/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/test_dimensions_requiring_join__plan0.xml b/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/test_dimensions_requiring_join__plan0.xml index 9c1b393a33..2d8e288ee1 100644 --- a/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/test_dimensions_requiring_join__plan0.xml +++ b/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/test_dimensions_requiring_join__plan0.xml @@ -16,190 +16,190 @@ - - - - - - - - - - - - - + + + + + + + + + + + + + + + + - + - + - - - - - - - - - - - - - - + + + + + + + + + + + - + - + - + - + - + - - - - - - - - - - - + - - + + + + + + + + + + + + - + - + - + - + - + - + - + - + - + - + + + + + + - + - + - + - + - - - - - - + - + - - - - - - - - + - - + + + + + + + + + - + - + - - + + - + - + - + diff --git a/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/test_multi_join_node__plan0.xml b/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/test_multi_join_node__plan0.xml index 04ff272c2e..b57d69bf27 100644 --- a/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/test_multi_join_node__plan0.xml +++ b/tests_metricflow/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/test_multi_join_node__plan0.xml @@ -2,12 +2,12 @@ - + + - - - - + + + @@ -580,8 +580,8 @@ - - + + diff --git a/tests_metricflow/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/BigQuery/test_nested_filters__plan0.sql b/tests_metricflow/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/BigQuery/test_nested_filters__plan0.sql index 510d1145cd..1a77e87eff 100644 --- a/tests_metricflow/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/BigQuery/test_nested_filters__plan0.sql +++ b/tests_metricflow/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/BigQuery/test_nested_filters__plan0.sql @@ -38,9 +38,9 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.listing AS listing + subq_5.is_lux_latest AS listing__is_lux_latest + , subq_2.listing AS listing , subq_2.booking__is_instant AS booking__is_instant - , subq_5.is_lux_latest AS listing__is_lux_latest , subq_2.average_booking_value AS average_booking_value FROM ( -- Pass Only Elements: ['average_booking_value', 'booking__is_instant', 'listing'] @@ -418,9 +418,9 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_14.listing AS listing + subq_17.is_lux_latest AS listing__is_lux_latest + , subq_14.listing AS listing , subq_14.booking__is_instant AS booking__is_instant - , subq_17.is_lux_latest AS listing__is_lux_latest , subq_14.bookings AS bookings FROM ( -- Pass Only Elements: ['bookings', 'booking__is_instant', 'listing'] diff --git a/tests_metricflow/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Databricks/test_nested_filters__plan0.sql b/tests_metricflow/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Databricks/test_nested_filters__plan0.sql index 94ca50b1fd..37c4e4c813 100644 --- a/tests_metricflow/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Databricks/test_nested_filters__plan0.sql +++ b/tests_metricflow/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Databricks/test_nested_filters__plan0.sql @@ -38,9 +38,9 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.listing AS listing + subq_5.is_lux_latest AS listing__is_lux_latest + , subq_2.listing AS listing , subq_2.booking__is_instant AS booking__is_instant - , subq_5.is_lux_latest AS listing__is_lux_latest , subq_2.average_booking_value AS average_booking_value FROM ( -- Pass Only Elements: ['average_booking_value', 'booking__is_instant', 'listing'] @@ -418,9 +418,9 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_14.listing AS listing + subq_17.is_lux_latest AS listing__is_lux_latest + , subq_14.listing AS listing , subq_14.booking__is_instant AS booking__is_instant - , subq_17.is_lux_latest AS listing__is_lux_latest , subq_14.bookings AS bookings FROM ( -- Pass Only Elements: ['bookings', 'booking__is_instant', 'listing'] diff --git a/tests_metricflow/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/DuckDB/test_nested_filters__plan0.sql b/tests_metricflow/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/DuckDB/test_nested_filters__plan0.sql index 4569d672d5..1cb582d6b5 100644 --- a/tests_metricflow/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/DuckDB/test_nested_filters__plan0.sql +++ b/tests_metricflow/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/DuckDB/test_nested_filters__plan0.sql @@ -38,9 +38,9 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.listing AS listing + subq_5.is_lux_latest AS listing__is_lux_latest + , subq_2.listing AS listing , subq_2.booking__is_instant AS booking__is_instant - , subq_5.is_lux_latest AS listing__is_lux_latest , subq_2.average_booking_value AS average_booking_value FROM ( -- Pass Only Elements: ['average_booking_value', 'booking__is_instant', 'listing'] @@ -418,9 +418,9 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_14.listing AS listing + subq_17.is_lux_latest AS listing__is_lux_latest + , subq_14.listing AS listing , subq_14.booking__is_instant AS booking__is_instant - , subq_17.is_lux_latest AS listing__is_lux_latest , subq_14.bookings AS bookings FROM ( -- Pass Only Elements: ['bookings', 'booking__is_instant', 'listing'] diff --git a/tests_metricflow/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Postgres/test_nested_filters__plan0.sql b/tests_metricflow/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Postgres/test_nested_filters__plan0.sql index 4569d672d5..1cb582d6b5 100644 --- a/tests_metricflow/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Postgres/test_nested_filters__plan0.sql +++ b/tests_metricflow/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Postgres/test_nested_filters__plan0.sql @@ -38,9 +38,9 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.listing AS listing + subq_5.is_lux_latest AS listing__is_lux_latest + , subq_2.listing AS listing , subq_2.booking__is_instant AS booking__is_instant - , subq_5.is_lux_latest AS listing__is_lux_latest , subq_2.average_booking_value AS average_booking_value FROM ( -- Pass Only Elements: ['average_booking_value', 'booking__is_instant', 'listing'] @@ -418,9 +418,9 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_14.listing AS listing + subq_17.is_lux_latest AS listing__is_lux_latest + , subq_14.listing AS listing , subq_14.booking__is_instant AS booking__is_instant - , subq_17.is_lux_latest AS listing__is_lux_latest , subq_14.bookings AS bookings FROM ( -- Pass Only Elements: ['bookings', 'booking__is_instant', 'listing'] diff --git a/tests_metricflow/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Redshift/test_nested_filters__plan0.sql b/tests_metricflow/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Redshift/test_nested_filters__plan0.sql index 5db13f9c09..57d207544d 100644 --- a/tests_metricflow/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Redshift/test_nested_filters__plan0.sql +++ b/tests_metricflow/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Redshift/test_nested_filters__plan0.sql @@ -38,9 +38,9 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.listing AS listing + subq_5.is_lux_latest AS listing__is_lux_latest + , subq_2.listing AS listing , subq_2.booking__is_instant AS booking__is_instant - , subq_5.is_lux_latest AS listing__is_lux_latest , subq_2.average_booking_value AS average_booking_value FROM ( -- Pass Only Elements: ['average_booking_value', 'booking__is_instant', 'listing'] @@ -418,9 +418,9 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_14.listing AS listing + subq_17.is_lux_latest AS listing__is_lux_latest + , subq_14.listing AS listing , subq_14.booking__is_instant AS booking__is_instant - , subq_17.is_lux_latest AS listing__is_lux_latest , subq_14.bookings AS bookings FROM ( -- Pass Only Elements: ['bookings', 'booking__is_instant', 'listing'] diff --git a/tests_metricflow/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Snowflake/test_nested_filters__plan0.sql b/tests_metricflow/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Snowflake/test_nested_filters__plan0.sql index ed681431ce..4bd8517da5 100644 --- a/tests_metricflow/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Snowflake/test_nested_filters__plan0.sql +++ b/tests_metricflow/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Snowflake/test_nested_filters__plan0.sql @@ -38,9 +38,9 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.listing AS listing + subq_5.is_lux_latest AS listing__is_lux_latest + , subq_2.listing AS listing , subq_2.booking__is_instant AS booking__is_instant - , subq_5.is_lux_latest AS listing__is_lux_latest , subq_2.average_booking_value AS average_booking_value FROM ( -- Pass Only Elements: ['average_booking_value', 'booking__is_instant', 'listing'] @@ -418,9 +418,9 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_14.listing AS listing + subq_17.is_lux_latest AS listing__is_lux_latest + , subq_14.listing AS listing , subq_14.booking__is_instant AS booking__is_instant - , subq_17.is_lux_latest AS listing__is_lux_latest , subq_14.bookings AS bookings FROM ( -- Pass Only Elements: ['bookings', 'booking__is_instant', 'listing'] diff --git a/tests_metricflow/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Trino/test_nested_filters__plan0.sql b/tests_metricflow/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Trino/test_nested_filters__plan0.sql index 9edb0371c0..c903d27c87 100644 --- a/tests_metricflow/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Trino/test_nested_filters__plan0.sql +++ b/tests_metricflow/snapshots/test_derived_metric_rendering.py/SqlQueryPlan/Trino/test_nested_filters__plan0.sql @@ -38,9 +38,9 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.listing AS listing + subq_5.is_lux_latest AS listing__is_lux_latest + , subq_2.listing AS listing , subq_2.booking__is_instant AS booking__is_instant - , subq_5.is_lux_latest AS listing__is_lux_latest , subq_2.average_booking_value AS average_booking_value FROM ( -- Pass Only Elements: ['average_booking_value', 'booking__is_instant', 'listing'] @@ -418,9 +418,9 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_14.listing AS listing + subq_17.is_lux_latest AS listing__is_lux_latest + , subq_14.listing AS listing , subq_14.booking__is_instant AS booking__is_instant - , subq_17.is_lux_latest AS listing__is_lux_latest , subq_14.bookings AS bookings FROM ( -- Pass Only Elements: ['bookings', 'booking__is_instant', 'listing'] diff --git a/tests_metricflow/snapshots/test_distinct_values_to_sql.py/SqlQueryPlan/BigQuery/test_dimension_values_with_a_join_and_a_filter__plan0.sql b/tests_metricflow/snapshots/test_distinct_values_to_sql.py/SqlQueryPlan/BigQuery/test_dimension_values_with_a_join_and_a_filter__plan0.sql index 0fa51d4648..d2d2f5e38e 100644 --- a/tests_metricflow/snapshots/test_distinct_values_to_sql.py/SqlQueryPlan/BigQuery/test_dimension_values_with_a_join_and_a_filter__plan0.sql +++ b/tests_metricflow/snapshots/test_distinct_values_to_sql.py/SqlQueryPlan/BigQuery/test_dimension_values_with_a_join_and_a_filter__plan0.sql @@ -65,7 +65,8 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_0.ds__day AS ds__day + subq_2.home_state_latest AS user__home_state_latest + , subq_0.ds__day AS ds__day , subq_0.ds__week AS ds__week , subq_0.ds__month AS ds__month , subq_0.ds__quarter AS ds__quarter @@ -118,7 +119,6 @@ FROM ( , subq_0.listing__country_latest AS listing__country_latest , subq_0.listing__is_lux_latest AS listing__is_lux_latest , subq_0.listing__capacity_latest AS listing__capacity_latest - , subq_2.home_state_latest AS user__home_state_latest , subq_0.listings AS listings , subq_0.largest_listing AS largest_listing , subq_0.smallest_listing AS smallest_listing diff --git a/tests_metricflow/snapshots/test_distinct_values_to_sql.py/SqlQueryPlan/BigQuery/test_dimension_values_with_a_join_and_a_filter__plan0_optimized.sql b/tests_metricflow/snapshots/test_distinct_values_to_sql.py/SqlQueryPlan/BigQuery/test_dimension_values_with_a_join_and_a_filter__plan0_optimized.sql index e54eaffeb6..8710b3d2f6 100644 --- a/tests_metricflow/snapshots/test_distinct_values_to_sql.py/SqlQueryPlan/BigQuery/test_dimension_values_with_a_join_and_a_filter__plan0_optimized.sql +++ b/tests_metricflow/snapshots/test_distinct_values_to_sql.py/SqlQueryPlan/BigQuery/test_dimension_values_with_a_join_and_a_filter__plan0_optimized.sql @@ -6,8 +6,8 @@ SELECT FROM ( -- Join Standard Outputs SELECT - listings_latest_src_28000.is_lux AS listing__is_lux_latest - , users_latest_src_28000.home_state_latest AS user__home_state_latest + users_latest_src_28000.home_state_latest AS user__home_state_latest + , listings_latest_src_28000.is_lux AS listing__is_lux_latest FROM ***************************.dim_listings_latest listings_latest_src_28000 FULL OUTER JOIN ***************************.dim_users_latest users_latest_src_28000 diff --git a/tests_metricflow/snapshots/test_distinct_values_to_sql.py/SqlQueryPlan/BigQuery/test_dimensions_requiring_join__plan0.sql b/tests_metricflow/snapshots/test_distinct_values_to_sql.py/SqlQueryPlan/BigQuery/test_dimensions_requiring_join__plan0.sql index f112ad4699..63f88a3bf1 100644 --- a/tests_metricflow/snapshots/test_distinct_values_to_sql.py/SqlQueryPlan/BigQuery/test_dimensions_requiring_join__plan0.sql +++ b/tests_metricflow/snapshots/test_distinct_values_to_sql.py/SqlQueryPlan/BigQuery/test_dimensions_requiring_join__plan0.sql @@ -5,7 +5,8 @@ SELECT FROM ( -- Join Standard Outputs SELECT - subq_0.ds__day AS ds__day + subq_2.home_state_latest AS user__home_state_latest + , subq_0.ds__day AS ds__day , subq_0.ds__week AS ds__week , subq_0.ds__month AS ds__month , subq_0.ds__quarter AS ds__quarter @@ -58,7 +59,6 @@ FROM ( , subq_0.listing__country_latest AS listing__country_latest , subq_0.listing__is_lux_latest AS listing__is_lux_latest , subq_0.listing__capacity_latest AS listing__capacity_latest - , subq_2.home_state_latest AS user__home_state_latest , subq_0.listings AS listings , subq_0.largest_listing AS largest_listing , subq_0.smallest_listing AS smallest_listing diff --git a/tests_metricflow/snapshots/test_distinct_values_to_sql.py/SqlQueryPlan/Databricks/test_dimension_values_with_a_join_and_a_filter__plan0.sql b/tests_metricflow/snapshots/test_distinct_values_to_sql.py/SqlQueryPlan/Databricks/test_dimension_values_with_a_join_and_a_filter__plan0.sql index 90b66576dd..f0b84e8e19 100644 --- a/tests_metricflow/snapshots/test_distinct_values_to_sql.py/SqlQueryPlan/Databricks/test_dimension_values_with_a_join_and_a_filter__plan0.sql +++ b/tests_metricflow/snapshots/test_distinct_values_to_sql.py/SqlQueryPlan/Databricks/test_dimension_values_with_a_join_and_a_filter__plan0.sql @@ -65,7 +65,8 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_0.ds__day AS ds__day + subq_2.home_state_latest AS user__home_state_latest + , subq_0.ds__day AS ds__day , subq_0.ds__week AS ds__week , subq_0.ds__month AS ds__month , subq_0.ds__quarter AS ds__quarter @@ -118,7 +119,6 @@ FROM ( , subq_0.listing__country_latest AS listing__country_latest , subq_0.listing__is_lux_latest AS listing__is_lux_latest , subq_0.listing__capacity_latest AS listing__capacity_latest - , subq_2.home_state_latest AS user__home_state_latest , subq_0.listings AS listings , subq_0.largest_listing AS largest_listing , subq_0.smallest_listing AS smallest_listing diff --git a/tests_metricflow/snapshots/test_distinct_values_to_sql.py/SqlQueryPlan/Databricks/test_dimension_values_with_a_join_and_a_filter__plan0_optimized.sql b/tests_metricflow/snapshots/test_distinct_values_to_sql.py/SqlQueryPlan/Databricks/test_dimension_values_with_a_join_and_a_filter__plan0_optimized.sql index e54eaffeb6..8710b3d2f6 100644 --- a/tests_metricflow/snapshots/test_distinct_values_to_sql.py/SqlQueryPlan/Databricks/test_dimension_values_with_a_join_and_a_filter__plan0_optimized.sql +++ b/tests_metricflow/snapshots/test_distinct_values_to_sql.py/SqlQueryPlan/Databricks/test_dimension_values_with_a_join_and_a_filter__plan0_optimized.sql @@ -6,8 +6,8 @@ SELECT FROM ( -- Join Standard Outputs SELECT - listings_latest_src_28000.is_lux AS listing__is_lux_latest - , users_latest_src_28000.home_state_latest AS user__home_state_latest + users_latest_src_28000.home_state_latest AS user__home_state_latest + , listings_latest_src_28000.is_lux AS listing__is_lux_latest FROM ***************************.dim_listings_latest listings_latest_src_28000 FULL OUTER JOIN ***************************.dim_users_latest users_latest_src_28000 diff --git a/tests_metricflow/snapshots/test_distinct_values_to_sql.py/SqlQueryPlan/Databricks/test_dimensions_requiring_join__plan0.sql b/tests_metricflow/snapshots/test_distinct_values_to_sql.py/SqlQueryPlan/Databricks/test_dimensions_requiring_join__plan0.sql index d66932d7d4..5907829912 100644 --- a/tests_metricflow/snapshots/test_distinct_values_to_sql.py/SqlQueryPlan/Databricks/test_dimensions_requiring_join__plan0.sql +++ b/tests_metricflow/snapshots/test_distinct_values_to_sql.py/SqlQueryPlan/Databricks/test_dimensions_requiring_join__plan0.sql @@ -5,7 +5,8 @@ SELECT FROM ( -- Join Standard Outputs SELECT - subq_0.ds__day AS ds__day + subq_2.home_state_latest AS user__home_state_latest + , subq_0.ds__day AS ds__day , subq_0.ds__week AS ds__week , subq_0.ds__month AS ds__month , subq_0.ds__quarter AS ds__quarter @@ -58,7 +59,6 @@ FROM ( , subq_0.listing__country_latest AS listing__country_latest , subq_0.listing__is_lux_latest AS listing__is_lux_latest , subq_0.listing__capacity_latest AS listing__capacity_latest - , subq_2.home_state_latest AS user__home_state_latest , subq_0.listings AS listings , subq_0.largest_listing AS largest_listing , subq_0.smallest_listing AS smallest_listing diff --git a/tests_metricflow/snapshots/test_distinct_values_to_sql.py/SqlQueryPlan/DuckDB/test_dimension_values_with_a_join_and_a_filter__plan0.sql b/tests_metricflow/snapshots/test_distinct_values_to_sql.py/SqlQueryPlan/DuckDB/test_dimension_values_with_a_join_and_a_filter__plan0.sql index d736ad0d03..d1d3afb12c 100644 --- a/tests_metricflow/snapshots/test_distinct_values_to_sql.py/SqlQueryPlan/DuckDB/test_dimension_values_with_a_join_and_a_filter__plan0.sql +++ b/tests_metricflow/snapshots/test_distinct_values_to_sql.py/SqlQueryPlan/DuckDB/test_dimension_values_with_a_join_and_a_filter__plan0.sql @@ -65,7 +65,8 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_0.ds__day AS ds__day + subq_2.home_state_latest AS user__home_state_latest + , subq_0.ds__day AS ds__day , subq_0.ds__week AS ds__week , subq_0.ds__month AS ds__month , subq_0.ds__quarter AS ds__quarter @@ -118,7 +119,6 @@ FROM ( , subq_0.listing__country_latest AS listing__country_latest , subq_0.listing__is_lux_latest AS listing__is_lux_latest , subq_0.listing__capacity_latest AS listing__capacity_latest - , subq_2.home_state_latest AS user__home_state_latest , subq_0.listings AS listings , subq_0.largest_listing AS largest_listing , subq_0.smallest_listing AS smallest_listing diff --git a/tests_metricflow/snapshots/test_distinct_values_to_sql.py/SqlQueryPlan/DuckDB/test_dimension_values_with_a_join_and_a_filter__plan0_optimized.sql b/tests_metricflow/snapshots/test_distinct_values_to_sql.py/SqlQueryPlan/DuckDB/test_dimension_values_with_a_join_and_a_filter__plan0_optimized.sql index e54eaffeb6..8710b3d2f6 100644 --- a/tests_metricflow/snapshots/test_distinct_values_to_sql.py/SqlQueryPlan/DuckDB/test_dimension_values_with_a_join_and_a_filter__plan0_optimized.sql +++ b/tests_metricflow/snapshots/test_distinct_values_to_sql.py/SqlQueryPlan/DuckDB/test_dimension_values_with_a_join_and_a_filter__plan0_optimized.sql @@ -6,8 +6,8 @@ SELECT FROM ( -- Join Standard Outputs SELECT - listings_latest_src_28000.is_lux AS listing__is_lux_latest - , users_latest_src_28000.home_state_latest AS user__home_state_latest + users_latest_src_28000.home_state_latest AS user__home_state_latest + , listings_latest_src_28000.is_lux AS listing__is_lux_latest FROM ***************************.dim_listings_latest listings_latest_src_28000 FULL OUTER JOIN ***************************.dim_users_latest users_latest_src_28000 diff --git a/tests_metricflow/snapshots/test_distinct_values_to_sql.py/SqlQueryPlan/DuckDB/test_dimensions_requiring_join__plan0.sql b/tests_metricflow/snapshots/test_distinct_values_to_sql.py/SqlQueryPlan/DuckDB/test_dimensions_requiring_join__plan0.sql index 0962aa9958..e4d9c4cec1 100644 --- a/tests_metricflow/snapshots/test_distinct_values_to_sql.py/SqlQueryPlan/DuckDB/test_dimensions_requiring_join__plan0.sql +++ b/tests_metricflow/snapshots/test_distinct_values_to_sql.py/SqlQueryPlan/DuckDB/test_dimensions_requiring_join__plan0.sql @@ -5,7 +5,8 @@ SELECT FROM ( -- Join Standard Outputs SELECT - subq_0.ds__day AS ds__day + subq_2.home_state_latest AS user__home_state_latest + , subq_0.ds__day AS ds__day , subq_0.ds__week AS ds__week , subq_0.ds__month AS ds__month , subq_0.ds__quarter AS ds__quarter @@ -58,7 +59,6 @@ FROM ( , subq_0.listing__country_latest AS listing__country_latest , subq_0.listing__is_lux_latest AS listing__is_lux_latest , subq_0.listing__capacity_latest AS listing__capacity_latest - , subq_2.home_state_latest AS user__home_state_latest , subq_0.listings AS listings , subq_0.largest_listing AS largest_listing , subq_0.smallest_listing AS smallest_listing diff --git a/tests_metricflow/snapshots/test_distinct_values_to_sql.py/SqlQueryPlan/Postgres/test_dimension_values_with_a_join_and_a_filter__plan0.sql b/tests_metricflow/snapshots/test_distinct_values_to_sql.py/SqlQueryPlan/Postgres/test_dimension_values_with_a_join_and_a_filter__plan0.sql index d736ad0d03..d1d3afb12c 100644 --- a/tests_metricflow/snapshots/test_distinct_values_to_sql.py/SqlQueryPlan/Postgres/test_dimension_values_with_a_join_and_a_filter__plan0.sql +++ b/tests_metricflow/snapshots/test_distinct_values_to_sql.py/SqlQueryPlan/Postgres/test_dimension_values_with_a_join_and_a_filter__plan0.sql @@ -65,7 +65,8 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_0.ds__day AS ds__day + subq_2.home_state_latest AS user__home_state_latest + , subq_0.ds__day AS ds__day , subq_0.ds__week AS ds__week , subq_0.ds__month AS ds__month , subq_0.ds__quarter AS ds__quarter @@ -118,7 +119,6 @@ FROM ( , subq_0.listing__country_latest AS listing__country_latest , subq_0.listing__is_lux_latest AS listing__is_lux_latest , subq_0.listing__capacity_latest AS listing__capacity_latest - , subq_2.home_state_latest AS user__home_state_latest , subq_0.listings AS listings , subq_0.largest_listing AS largest_listing , subq_0.smallest_listing AS smallest_listing diff --git a/tests_metricflow/snapshots/test_distinct_values_to_sql.py/SqlQueryPlan/Postgres/test_dimension_values_with_a_join_and_a_filter__plan0_optimized.sql b/tests_metricflow/snapshots/test_distinct_values_to_sql.py/SqlQueryPlan/Postgres/test_dimension_values_with_a_join_and_a_filter__plan0_optimized.sql index e54eaffeb6..8710b3d2f6 100644 --- a/tests_metricflow/snapshots/test_distinct_values_to_sql.py/SqlQueryPlan/Postgres/test_dimension_values_with_a_join_and_a_filter__plan0_optimized.sql +++ b/tests_metricflow/snapshots/test_distinct_values_to_sql.py/SqlQueryPlan/Postgres/test_dimension_values_with_a_join_and_a_filter__plan0_optimized.sql @@ -6,8 +6,8 @@ SELECT FROM ( -- Join Standard Outputs SELECT - listings_latest_src_28000.is_lux AS listing__is_lux_latest - , users_latest_src_28000.home_state_latest AS user__home_state_latest + users_latest_src_28000.home_state_latest AS user__home_state_latest + , listings_latest_src_28000.is_lux AS listing__is_lux_latest FROM ***************************.dim_listings_latest listings_latest_src_28000 FULL OUTER JOIN ***************************.dim_users_latest users_latest_src_28000 diff --git a/tests_metricflow/snapshots/test_distinct_values_to_sql.py/SqlQueryPlan/Postgres/test_dimensions_requiring_join__plan0.sql b/tests_metricflow/snapshots/test_distinct_values_to_sql.py/SqlQueryPlan/Postgres/test_dimensions_requiring_join__plan0.sql index 0962aa9958..e4d9c4cec1 100644 --- a/tests_metricflow/snapshots/test_distinct_values_to_sql.py/SqlQueryPlan/Postgres/test_dimensions_requiring_join__plan0.sql +++ b/tests_metricflow/snapshots/test_distinct_values_to_sql.py/SqlQueryPlan/Postgres/test_dimensions_requiring_join__plan0.sql @@ -5,7 +5,8 @@ SELECT FROM ( -- Join Standard Outputs SELECT - subq_0.ds__day AS ds__day + subq_2.home_state_latest AS user__home_state_latest + , subq_0.ds__day AS ds__day , subq_0.ds__week AS ds__week , subq_0.ds__month AS ds__month , subq_0.ds__quarter AS ds__quarter @@ -58,7 +59,6 @@ FROM ( , subq_0.listing__country_latest AS listing__country_latest , subq_0.listing__is_lux_latest AS listing__is_lux_latest , subq_0.listing__capacity_latest AS listing__capacity_latest - , subq_2.home_state_latest AS user__home_state_latest , subq_0.listings AS listings , subq_0.largest_listing AS largest_listing , subq_0.smallest_listing AS smallest_listing diff --git a/tests_metricflow/snapshots/test_distinct_values_to_sql.py/SqlQueryPlan/Redshift/test_dimension_values_with_a_join_and_a_filter__plan0.sql b/tests_metricflow/snapshots/test_distinct_values_to_sql.py/SqlQueryPlan/Redshift/test_dimension_values_with_a_join_and_a_filter__plan0.sql index 2444ef1956..8645ee35ba 100644 --- a/tests_metricflow/snapshots/test_distinct_values_to_sql.py/SqlQueryPlan/Redshift/test_dimension_values_with_a_join_and_a_filter__plan0.sql +++ b/tests_metricflow/snapshots/test_distinct_values_to_sql.py/SqlQueryPlan/Redshift/test_dimension_values_with_a_join_and_a_filter__plan0.sql @@ -65,7 +65,8 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_0.ds__day AS ds__day + subq_2.home_state_latest AS user__home_state_latest + , subq_0.ds__day AS ds__day , subq_0.ds__week AS ds__week , subq_0.ds__month AS ds__month , subq_0.ds__quarter AS ds__quarter @@ -118,7 +119,6 @@ FROM ( , subq_0.listing__country_latest AS listing__country_latest , subq_0.listing__is_lux_latest AS listing__is_lux_latest , subq_0.listing__capacity_latest AS listing__capacity_latest - , subq_2.home_state_latest AS user__home_state_latest , subq_0.listings AS listings , subq_0.largest_listing AS largest_listing , subq_0.smallest_listing AS smallest_listing diff --git a/tests_metricflow/snapshots/test_distinct_values_to_sql.py/SqlQueryPlan/Redshift/test_dimension_values_with_a_join_and_a_filter__plan0_optimized.sql b/tests_metricflow/snapshots/test_distinct_values_to_sql.py/SqlQueryPlan/Redshift/test_dimension_values_with_a_join_and_a_filter__plan0_optimized.sql index e54eaffeb6..8710b3d2f6 100644 --- a/tests_metricflow/snapshots/test_distinct_values_to_sql.py/SqlQueryPlan/Redshift/test_dimension_values_with_a_join_and_a_filter__plan0_optimized.sql +++ b/tests_metricflow/snapshots/test_distinct_values_to_sql.py/SqlQueryPlan/Redshift/test_dimension_values_with_a_join_and_a_filter__plan0_optimized.sql @@ -6,8 +6,8 @@ SELECT FROM ( -- Join Standard Outputs SELECT - listings_latest_src_28000.is_lux AS listing__is_lux_latest - , users_latest_src_28000.home_state_latest AS user__home_state_latest + users_latest_src_28000.home_state_latest AS user__home_state_latest + , listings_latest_src_28000.is_lux AS listing__is_lux_latest FROM ***************************.dim_listings_latest listings_latest_src_28000 FULL OUTER JOIN ***************************.dim_users_latest users_latest_src_28000 diff --git a/tests_metricflow/snapshots/test_distinct_values_to_sql.py/SqlQueryPlan/Redshift/test_dimensions_requiring_join__plan0.sql b/tests_metricflow/snapshots/test_distinct_values_to_sql.py/SqlQueryPlan/Redshift/test_dimensions_requiring_join__plan0.sql index 6ada21e5f4..11790f4d9c 100644 --- a/tests_metricflow/snapshots/test_distinct_values_to_sql.py/SqlQueryPlan/Redshift/test_dimensions_requiring_join__plan0.sql +++ b/tests_metricflow/snapshots/test_distinct_values_to_sql.py/SqlQueryPlan/Redshift/test_dimensions_requiring_join__plan0.sql @@ -5,7 +5,8 @@ SELECT FROM ( -- Join Standard Outputs SELECT - subq_0.ds__day AS ds__day + subq_2.home_state_latest AS user__home_state_latest + , subq_0.ds__day AS ds__day , subq_0.ds__week AS ds__week , subq_0.ds__month AS ds__month , subq_0.ds__quarter AS ds__quarter @@ -58,7 +59,6 @@ FROM ( , subq_0.listing__country_latest AS listing__country_latest , subq_0.listing__is_lux_latest AS listing__is_lux_latest , subq_0.listing__capacity_latest AS listing__capacity_latest - , subq_2.home_state_latest AS user__home_state_latest , subq_0.listings AS listings , subq_0.largest_listing AS largest_listing , subq_0.smallest_listing AS smallest_listing diff --git a/tests_metricflow/snapshots/test_distinct_values_to_sql.py/SqlQueryPlan/Snowflake/test_dimension_values_with_a_join_and_a_filter__plan0.sql b/tests_metricflow/snapshots/test_distinct_values_to_sql.py/SqlQueryPlan/Snowflake/test_dimension_values_with_a_join_and_a_filter__plan0.sql index fe092394e6..e708f32d57 100644 --- a/tests_metricflow/snapshots/test_distinct_values_to_sql.py/SqlQueryPlan/Snowflake/test_dimension_values_with_a_join_and_a_filter__plan0.sql +++ b/tests_metricflow/snapshots/test_distinct_values_to_sql.py/SqlQueryPlan/Snowflake/test_dimension_values_with_a_join_and_a_filter__plan0.sql @@ -65,7 +65,8 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_0.ds__day AS ds__day + subq_2.home_state_latest AS user__home_state_latest + , subq_0.ds__day AS ds__day , subq_0.ds__week AS ds__week , subq_0.ds__month AS ds__month , subq_0.ds__quarter AS ds__quarter @@ -118,7 +119,6 @@ FROM ( , subq_0.listing__country_latest AS listing__country_latest , subq_0.listing__is_lux_latest AS listing__is_lux_latest , subq_0.listing__capacity_latest AS listing__capacity_latest - , subq_2.home_state_latest AS user__home_state_latest , subq_0.listings AS listings , subq_0.largest_listing AS largest_listing , subq_0.smallest_listing AS smallest_listing diff --git a/tests_metricflow/snapshots/test_distinct_values_to_sql.py/SqlQueryPlan/Snowflake/test_dimension_values_with_a_join_and_a_filter__plan0_optimized.sql b/tests_metricflow/snapshots/test_distinct_values_to_sql.py/SqlQueryPlan/Snowflake/test_dimension_values_with_a_join_and_a_filter__plan0_optimized.sql index e54eaffeb6..8710b3d2f6 100644 --- a/tests_metricflow/snapshots/test_distinct_values_to_sql.py/SqlQueryPlan/Snowflake/test_dimension_values_with_a_join_and_a_filter__plan0_optimized.sql +++ b/tests_metricflow/snapshots/test_distinct_values_to_sql.py/SqlQueryPlan/Snowflake/test_dimension_values_with_a_join_and_a_filter__plan0_optimized.sql @@ -6,8 +6,8 @@ SELECT FROM ( -- Join Standard Outputs SELECT - listings_latest_src_28000.is_lux AS listing__is_lux_latest - , users_latest_src_28000.home_state_latest AS user__home_state_latest + users_latest_src_28000.home_state_latest AS user__home_state_latest + , listings_latest_src_28000.is_lux AS listing__is_lux_latest FROM ***************************.dim_listings_latest listings_latest_src_28000 FULL OUTER JOIN ***************************.dim_users_latest users_latest_src_28000 diff --git a/tests_metricflow/snapshots/test_distinct_values_to_sql.py/SqlQueryPlan/Snowflake/test_dimensions_requiring_join__plan0.sql b/tests_metricflow/snapshots/test_distinct_values_to_sql.py/SqlQueryPlan/Snowflake/test_dimensions_requiring_join__plan0.sql index 651824a182..cd3786df0e 100644 --- a/tests_metricflow/snapshots/test_distinct_values_to_sql.py/SqlQueryPlan/Snowflake/test_dimensions_requiring_join__plan0.sql +++ b/tests_metricflow/snapshots/test_distinct_values_to_sql.py/SqlQueryPlan/Snowflake/test_dimensions_requiring_join__plan0.sql @@ -5,7 +5,8 @@ SELECT FROM ( -- Join Standard Outputs SELECT - subq_0.ds__day AS ds__day + subq_2.home_state_latest AS user__home_state_latest + , subq_0.ds__day AS ds__day , subq_0.ds__week AS ds__week , subq_0.ds__month AS ds__month , subq_0.ds__quarter AS ds__quarter @@ -58,7 +59,6 @@ FROM ( , subq_0.listing__country_latest AS listing__country_latest , subq_0.listing__is_lux_latest AS listing__is_lux_latest , subq_0.listing__capacity_latest AS listing__capacity_latest - , subq_2.home_state_latest AS user__home_state_latest , subq_0.listings AS listings , subq_0.largest_listing AS largest_listing , subq_0.smallest_listing AS smallest_listing diff --git a/tests_metricflow/snapshots/test_distinct_values_to_sql.py/SqlQueryPlan/Trino/test_dimension_values_with_a_join_and_a_filter__plan0.sql b/tests_metricflow/snapshots/test_distinct_values_to_sql.py/SqlQueryPlan/Trino/test_dimension_values_with_a_join_and_a_filter__plan0.sql index 478b5d81be..47814b943d 100644 --- a/tests_metricflow/snapshots/test_distinct_values_to_sql.py/SqlQueryPlan/Trino/test_dimension_values_with_a_join_and_a_filter__plan0.sql +++ b/tests_metricflow/snapshots/test_distinct_values_to_sql.py/SqlQueryPlan/Trino/test_dimension_values_with_a_join_and_a_filter__plan0.sql @@ -65,7 +65,8 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_0.ds__day AS ds__day + subq_2.home_state_latest AS user__home_state_latest + , subq_0.ds__day AS ds__day , subq_0.ds__week AS ds__week , subq_0.ds__month AS ds__month , subq_0.ds__quarter AS ds__quarter @@ -118,7 +119,6 @@ FROM ( , subq_0.listing__country_latest AS listing__country_latest , subq_0.listing__is_lux_latest AS listing__is_lux_latest , subq_0.listing__capacity_latest AS listing__capacity_latest - , subq_2.home_state_latest AS user__home_state_latest , subq_0.listings AS listings , subq_0.largest_listing AS largest_listing , subq_0.smallest_listing AS smallest_listing diff --git a/tests_metricflow/snapshots/test_distinct_values_to_sql.py/SqlQueryPlan/Trino/test_dimension_values_with_a_join_and_a_filter__plan0_optimized.sql b/tests_metricflow/snapshots/test_distinct_values_to_sql.py/SqlQueryPlan/Trino/test_dimension_values_with_a_join_and_a_filter__plan0_optimized.sql index e54eaffeb6..8710b3d2f6 100644 --- a/tests_metricflow/snapshots/test_distinct_values_to_sql.py/SqlQueryPlan/Trino/test_dimension_values_with_a_join_and_a_filter__plan0_optimized.sql +++ b/tests_metricflow/snapshots/test_distinct_values_to_sql.py/SqlQueryPlan/Trino/test_dimension_values_with_a_join_and_a_filter__plan0_optimized.sql @@ -6,8 +6,8 @@ SELECT FROM ( -- Join Standard Outputs SELECT - listings_latest_src_28000.is_lux AS listing__is_lux_latest - , users_latest_src_28000.home_state_latest AS user__home_state_latest + users_latest_src_28000.home_state_latest AS user__home_state_latest + , listings_latest_src_28000.is_lux AS listing__is_lux_latest FROM ***************************.dim_listings_latest listings_latest_src_28000 FULL OUTER JOIN ***************************.dim_users_latest users_latest_src_28000 diff --git a/tests_metricflow/snapshots/test_distinct_values_to_sql.py/SqlQueryPlan/Trino/test_dimensions_requiring_join__plan0.sql b/tests_metricflow/snapshots/test_distinct_values_to_sql.py/SqlQueryPlan/Trino/test_dimensions_requiring_join__plan0.sql index 28f04502f2..007d0aac57 100644 --- a/tests_metricflow/snapshots/test_distinct_values_to_sql.py/SqlQueryPlan/Trino/test_dimensions_requiring_join__plan0.sql +++ b/tests_metricflow/snapshots/test_distinct_values_to_sql.py/SqlQueryPlan/Trino/test_dimensions_requiring_join__plan0.sql @@ -5,7 +5,8 @@ SELECT FROM ( -- Join Standard Outputs SELECT - subq_0.ds__day AS ds__day + subq_2.home_state_latest AS user__home_state_latest + , subq_0.ds__day AS ds__day , subq_0.ds__week AS ds__week , subq_0.ds__month AS ds__month , subq_0.ds__quarter AS ds__quarter @@ -58,7 +59,6 @@ FROM ( , subq_0.listing__country_latest AS listing__country_latest , subq_0.listing__is_lux_latest AS listing__is_lux_latest , subq_0.listing__capacity_latest AS listing__capacity_latest - , subq_2.home_state_latest AS user__home_state_latest , subq_0.listings AS listings , subq_0.largest_listing AS largest_listing , subq_0.smallest_listing AS smallest_listing diff --git a/tests_metricflow/snapshots/test_distinct_values_to_sql.py/SqlQueryPlan/test_dimension_values_with_a_join_and_a_filter__plan0.xml b/tests_metricflow/snapshots/test_distinct_values_to_sql.py/SqlQueryPlan/test_dimension_values_with_a_join_and_a_filter__plan0.xml index 2c33124008..31ec4e8cdb 100644 --- a/tests_metricflow/snapshots/test_distinct_values_to_sql.py/SqlQueryPlan/test_dimension_values_with_a_join_and_a_filter__plan0.xml +++ b/tests_metricflow/snapshots/test_distinct_values_to_sql.py/SqlQueryPlan/test_dimension_values_with_a_join_and_a_filter__plan0.xml @@ -208,206 +208,206 @@ - - - - - - - - - + + + + + + + + + + + + + + - + - - - - + + - + - + - - - - - - + + + + + + + + - + - - - - - - - + + - + - + - + - + - + - - - - - + - - + + + + + + - + - - - + - - + + + + - + - + - + - + - + - + - + - + - + - + + + + + + - + - + - + - + - - - - - - + - + - - - - - - - - - - - + - - + + + + + + + + + + + + - + - + - - + + - + - + - + diff --git a/tests_metricflow/snapshots/test_distinct_values_to_sql.py/SqlQueryPlan/test_dimensions_requiring_join__plan0.xml b/tests_metricflow/snapshots/test_distinct_values_to_sql.py/SqlQueryPlan/test_dimensions_requiring_join__plan0.xml index 9c1b393a33..2d8e288ee1 100644 --- a/tests_metricflow/snapshots/test_distinct_values_to_sql.py/SqlQueryPlan/test_dimensions_requiring_join__plan0.xml +++ b/tests_metricflow/snapshots/test_distinct_values_to_sql.py/SqlQueryPlan/test_dimensions_requiring_join__plan0.xml @@ -16,190 +16,190 @@ - - - - - - - - - - - - - + + + + + + + + + + + + + + + + - + - + - - - - - - - - - - - - - - + + + + + + + + + + + - + - + - + - + - + - - - - - - - - - - - + - - + + + + + + + + + + + + - + - + - + - + - + - + - + - + - + - + + + + + + - + - + - + - + - - - - - - + - + - - - - - - - - + - - + + + + + + + + + - + - + - - + + - + - + - + diff --git a/tests_metricflow/snapshots/test_granularity_date_part_rendering.py/SqlQueryPlan/BigQuery/test_simple_metric_with_joined_sub_daily_dimension__plan0.sql b/tests_metricflow/snapshots/test_granularity_date_part_rendering.py/SqlQueryPlan/BigQuery/test_simple_metric_with_joined_sub_daily_dimension__plan0.sql index 064320f388..6163da77c9 100644 --- a/tests_metricflow/snapshots/test_granularity_date_part_rendering.py/SqlQueryPlan/BigQuery/test_simple_metric_with_joined_sub_daily_dimension__plan0.sql +++ b/tests_metricflow/snapshots/test_granularity_date_part_rendering.py/SqlQueryPlan/BigQuery/test_simple_metric_with_joined_sub_daily_dimension__plan0.sql @@ -15,9 +15,9 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.ds_partitioned__day AS ds_partitioned__day - , subq_9.user__ds_partitioned__day AS listing__user__ds_partitioned__day + subq_9.user__ds_partitioned__day AS listing__user__ds_partitioned__day , subq_9.user__bio_added_ts__minute AS listing__user__bio_added_ts__minute + , subq_2.ds_partitioned__day AS ds_partitioned__day , subq_2.listing AS listing , subq_2.bookings AS bookings FROM ( @@ -231,61 +231,7 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_4.ds__day AS ds__day - , subq_4.ds__week AS ds__week - , subq_4.ds__month AS ds__month - , subq_4.ds__quarter AS ds__quarter - , subq_4.ds__year AS ds__year - , subq_4.ds__extract_year AS ds__extract_year - , subq_4.ds__extract_quarter AS ds__extract_quarter - , subq_4.ds__extract_month AS ds__extract_month - , subq_4.ds__extract_day AS ds__extract_day - , subq_4.ds__extract_dow AS ds__extract_dow - , subq_4.ds__extract_doy AS ds__extract_doy - , subq_4.created_at__day AS created_at__day - , subq_4.created_at__week AS created_at__week - , subq_4.created_at__month AS created_at__month - , subq_4.created_at__quarter AS created_at__quarter - , subq_4.created_at__year AS created_at__year - , subq_4.created_at__extract_year AS created_at__extract_year - , subq_4.created_at__extract_quarter AS created_at__extract_quarter - , subq_4.created_at__extract_month AS created_at__extract_month - , subq_4.created_at__extract_day AS created_at__extract_day - , subq_4.created_at__extract_dow AS created_at__extract_dow - , subq_4.created_at__extract_doy AS created_at__extract_doy - , subq_4.listing__ds__day AS listing__ds__day - , subq_4.listing__ds__week AS listing__ds__week - , subq_4.listing__ds__month AS listing__ds__month - , subq_4.listing__ds__quarter AS listing__ds__quarter - , subq_4.listing__ds__year AS listing__ds__year - , subq_4.listing__ds__extract_year AS listing__ds__extract_year - , subq_4.listing__ds__extract_quarter AS listing__ds__extract_quarter - , subq_4.listing__ds__extract_month AS listing__ds__extract_month - , subq_4.listing__ds__extract_day AS listing__ds__extract_day - , subq_4.listing__ds__extract_dow AS listing__ds__extract_dow - , subq_4.listing__ds__extract_doy AS listing__ds__extract_doy - , subq_4.listing__created_at__day AS listing__created_at__day - , subq_4.listing__created_at__week AS listing__created_at__week - , subq_4.listing__created_at__month AS listing__created_at__month - , subq_4.listing__created_at__quarter AS listing__created_at__quarter - , subq_4.listing__created_at__year AS listing__created_at__year - , subq_4.listing__created_at__extract_year AS listing__created_at__extract_year - , subq_4.listing__created_at__extract_quarter AS listing__created_at__extract_quarter - , subq_4.listing__created_at__extract_month AS listing__created_at__extract_month - , subq_4.listing__created_at__extract_day AS listing__created_at__extract_day - , subq_4.listing__created_at__extract_dow AS listing__created_at__extract_dow - , subq_4.listing__created_at__extract_doy AS listing__created_at__extract_doy - , subq_4.metric_time__day AS metric_time__day - , subq_4.metric_time__week AS metric_time__week - , subq_4.metric_time__month AS metric_time__month - , subq_4.metric_time__quarter AS metric_time__quarter - , subq_4.metric_time__year AS metric_time__year - , subq_4.metric_time__extract_year AS metric_time__extract_year - , subq_4.metric_time__extract_quarter AS metric_time__extract_quarter - , subq_4.metric_time__extract_month AS metric_time__extract_month - , subq_4.metric_time__extract_day AS metric_time__extract_day - , subq_4.metric_time__extract_dow AS metric_time__extract_dow - , subq_4.metric_time__extract_doy AS metric_time__extract_doy + subq_7.home_state AS user__home_state , subq_7.ds__day AS user__ds__day , subq_7.ds__week AS user__ds__week , subq_7.ds__month AS user__ds__month @@ -384,6 +330,61 @@ FROM ( , subq_7.metric_time__extract_day AS user__metric_time__extract_day , subq_7.metric_time__extract_dow AS user__metric_time__extract_dow , subq_7.metric_time__extract_doy AS user__metric_time__extract_doy + , subq_4.ds__day AS ds__day + , subq_4.ds__week AS ds__week + , subq_4.ds__month AS ds__month + , subq_4.ds__quarter AS ds__quarter + , subq_4.ds__year AS ds__year + , subq_4.ds__extract_year AS ds__extract_year + , subq_4.ds__extract_quarter AS ds__extract_quarter + , subq_4.ds__extract_month AS ds__extract_month + , subq_4.ds__extract_day AS ds__extract_day + , subq_4.ds__extract_dow AS ds__extract_dow + , subq_4.ds__extract_doy AS ds__extract_doy + , subq_4.created_at__day AS created_at__day + , subq_4.created_at__week AS created_at__week + , subq_4.created_at__month AS created_at__month + , subq_4.created_at__quarter AS created_at__quarter + , subq_4.created_at__year AS created_at__year + , subq_4.created_at__extract_year AS created_at__extract_year + , subq_4.created_at__extract_quarter AS created_at__extract_quarter + , subq_4.created_at__extract_month AS created_at__extract_month + , subq_4.created_at__extract_day AS created_at__extract_day + , subq_4.created_at__extract_dow AS created_at__extract_dow + , subq_4.created_at__extract_doy AS created_at__extract_doy + , subq_4.listing__ds__day AS listing__ds__day + , subq_4.listing__ds__week AS listing__ds__week + , subq_4.listing__ds__month AS listing__ds__month + , subq_4.listing__ds__quarter AS listing__ds__quarter + , subq_4.listing__ds__year AS listing__ds__year + , subq_4.listing__ds__extract_year AS listing__ds__extract_year + , subq_4.listing__ds__extract_quarter AS listing__ds__extract_quarter + , subq_4.listing__ds__extract_month AS listing__ds__extract_month + , subq_4.listing__ds__extract_day AS listing__ds__extract_day + , subq_4.listing__ds__extract_dow AS listing__ds__extract_dow + , subq_4.listing__ds__extract_doy AS listing__ds__extract_doy + , subq_4.listing__created_at__day AS listing__created_at__day + , subq_4.listing__created_at__week AS listing__created_at__week + , subq_4.listing__created_at__month AS listing__created_at__month + , subq_4.listing__created_at__quarter AS listing__created_at__quarter + , subq_4.listing__created_at__year AS listing__created_at__year + , subq_4.listing__created_at__extract_year AS listing__created_at__extract_year + , subq_4.listing__created_at__extract_quarter AS listing__created_at__extract_quarter + , subq_4.listing__created_at__extract_month AS listing__created_at__extract_month + , subq_4.listing__created_at__extract_day AS listing__created_at__extract_day + , subq_4.listing__created_at__extract_dow AS listing__created_at__extract_dow + , subq_4.listing__created_at__extract_doy AS listing__created_at__extract_doy + , subq_4.metric_time__day AS metric_time__day + , subq_4.metric_time__week AS metric_time__week + , subq_4.metric_time__month AS metric_time__month + , subq_4.metric_time__quarter AS metric_time__quarter + , subq_4.metric_time__year AS metric_time__year + , subq_4.metric_time__extract_year AS metric_time__extract_year + , subq_4.metric_time__extract_quarter AS metric_time__extract_quarter + , subq_4.metric_time__extract_month AS metric_time__extract_month + , subq_4.metric_time__extract_day AS metric_time__extract_day + , subq_4.metric_time__extract_dow AS metric_time__extract_dow + , subq_4.metric_time__extract_doy AS metric_time__extract_doy , subq_4.listing AS listing , subq_4.user AS user , subq_4.listing__user AS listing__user @@ -393,7 +394,6 @@ FROM ( , subq_4.listing__country_latest AS listing__country_latest , subq_4.listing__is_lux_latest AS listing__is_lux_latest , subq_4.listing__capacity_latest AS listing__capacity_latest - , subq_7.home_state AS user__home_state , subq_4.listings AS listings , subq_4.largest_listing AS largest_listing , subq_4.smallest_listing AS smallest_listing diff --git a/tests_metricflow/snapshots/test_granularity_date_part_rendering.py/SqlQueryPlan/Databricks/test_simple_metric_with_joined_sub_daily_dimension__plan0.sql b/tests_metricflow/snapshots/test_granularity_date_part_rendering.py/SqlQueryPlan/Databricks/test_simple_metric_with_joined_sub_daily_dimension__plan0.sql index 080cda4587..4e6c3b98c4 100644 --- a/tests_metricflow/snapshots/test_granularity_date_part_rendering.py/SqlQueryPlan/Databricks/test_simple_metric_with_joined_sub_daily_dimension__plan0.sql +++ b/tests_metricflow/snapshots/test_granularity_date_part_rendering.py/SqlQueryPlan/Databricks/test_simple_metric_with_joined_sub_daily_dimension__plan0.sql @@ -15,9 +15,9 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.ds_partitioned__day AS ds_partitioned__day - , subq_9.user__ds_partitioned__day AS listing__user__ds_partitioned__day + subq_9.user__ds_partitioned__day AS listing__user__ds_partitioned__day , subq_9.user__bio_added_ts__minute AS listing__user__bio_added_ts__minute + , subq_2.ds_partitioned__day AS ds_partitioned__day , subq_2.listing AS listing , subq_2.bookings AS bookings FROM ( @@ -231,61 +231,7 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_4.ds__day AS ds__day - , subq_4.ds__week AS ds__week - , subq_4.ds__month AS ds__month - , subq_4.ds__quarter AS ds__quarter - , subq_4.ds__year AS ds__year - , subq_4.ds__extract_year AS ds__extract_year - , subq_4.ds__extract_quarter AS ds__extract_quarter - , subq_4.ds__extract_month AS ds__extract_month - , subq_4.ds__extract_day AS ds__extract_day - , subq_4.ds__extract_dow AS ds__extract_dow - , subq_4.ds__extract_doy AS ds__extract_doy - , subq_4.created_at__day AS created_at__day - , subq_4.created_at__week AS created_at__week - , subq_4.created_at__month AS created_at__month - , subq_4.created_at__quarter AS created_at__quarter - , subq_4.created_at__year AS created_at__year - , subq_4.created_at__extract_year AS created_at__extract_year - , subq_4.created_at__extract_quarter AS created_at__extract_quarter - , subq_4.created_at__extract_month AS created_at__extract_month - , subq_4.created_at__extract_day AS created_at__extract_day - , subq_4.created_at__extract_dow AS created_at__extract_dow - , subq_4.created_at__extract_doy AS created_at__extract_doy - , subq_4.listing__ds__day AS listing__ds__day - , subq_4.listing__ds__week AS listing__ds__week - , subq_4.listing__ds__month AS listing__ds__month - , subq_4.listing__ds__quarter AS listing__ds__quarter - , subq_4.listing__ds__year AS listing__ds__year - , subq_4.listing__ds__extract_year AS listing__ds__extract_year - , subq_4.listing__ds__extract_quarter AS listing__ds__extract_quarter - , subq_4.listing__ds__extract_month AS listing__ds__extract_month - , subq_4.listing__ds__extract_day AS listing__ds__extract_day - , subq_4.listing__ds__extract_dow AS listing__ds__extract_dow - , subq_4.listing__ds__extract_doy AS listing__ds__extract_doy - , subq_4.listing__created_at__day AS listing__created_at__day - , subq_4.listing__created_at__week AS listing__created_at__week - , subq_4.listing__created_at__month AS listing__created_at__month - , subq_4.listing__created_at__quarter AS listing__created_at__quarter - , subq_4.listing__created_at__year AS listing__created_at__year - , subq_4.listing__created_at__extract_year AS listing__created_at__extract_year - , subq_4.listing__created_at__extract_quarter AS listing__created_at__extract_quarter - , subq_4.listing__created_at__extract_month AS listing__created_at__extract_month - , subq_4.listing__created_at__extract_day AS listing__created_at__extract_day - , subq_4.listing__created_at__extract_dow AS listing__created_at__extract_dow - , subq_4.listing__created_at__extract_doy AS listing__created_at__extract_doy - , subq_4.metric_time__day AS metric_time__day - , subq_4.metric_time__week AS metric_time__week - , subq_4.metric_time__month AS metric_time__month - , subq_4.metric_time__quarter AS metric_time__quarter - , subq_4.metric_time__year AS metric_time__year - , subq_4.metric_time__extract_year AS metric_time__extract_year - , subq_4.metric_time__extract_quarter AS metric_time__extract_quarter - , subq_4.metric_time__extract_month AS metric_time__extract_month - , subq_4.metric_time__extract_day AS metric_time__extract_day - , subq_4.metric_time__extract_dow AS metric_time__extract_dow - , subq_4.metric_time__extract_doy AS metric_time__extract_doy + subq_7.home_state AS user__home_state , subq_7.ds__day AS user__ds__day , subq_7.ds__week AS user__ds__week , subq_7.ds__month AS user__ds__month @@ -384,6 +330,61 @@ FROM ( , subq_7.metric_time__extract_day AS user__metric_time__extract_day , subq_7.metric_time__extract_dow AS user__metric_time__extract_dow , subq_7.metric_time__extract_doy AS user__metric_time__extract_doy + , subq_4.ds__day AS ds__day + , subq_4.ds__week AS ds__week + , subq_4.ds__month AS ds__month + , subq_4.ds__quarter AS ds__quarter + , subq_4.ds__year AS ds__year + , subq_4.ds__extract_year AS ds__extract_year + , subq_4.ds__extract_quarter AS ds__extract_quarter + , subq_4.ds__extract_month AS ds__extract_month + , subq_4.ds__extract_day AS ds__extract_day + , subq_4.ds__extract_dow AS ds__extract_dow + , subq_4.ds__extract_doy AS ds__extract_doy + , subq_4.created_at__day AS created_at__day + , subq_4.created_at__week AS created_at__week + , subq_4.created_at__month AS created_at__month + , subq_4.created_at__quarter AS created_at__quarter + , subq_4.created_at__year AS created_at__year + , subq_4.created_at__extract_year AS created_at__extract_year + , subq_4.created_at__extract_quarter AS created_at__extract_quarter + , subq_4.created_at__extract_month AS created_at__extract_month + , subq_4.created_at__extract_day AS created_at__extract_day + , subq_4.created_at__extract_dow AS created_at__extract_dow + , subq_4.created_at__extract_doy AS created_at__extract_doy + , subq_4.listing__ds__day AS listing__ds__day + , subq_4.listing__ds__week AS listing__ds__week + , subq_4.listing__ds__month AS listing__ds__month + , subq_4.listing__ds__quarter AS listing__ds__quarter + , subq_4.listing__ds__year AS listing__ds__year + , subq_4.listing__ds__extract_year AS listing__ds__extract_year + , subq_4.listing__ds__extract_quarter AS listing__ds__extract_quarter + , subq_4.listing__ds__extract_month AS listing__ds__extract_month + , subq_4.listing__ds__extract_day AS listing__ds__extract_day + , subq_4.listing__ds__extract_dow AS listing__ds__extract_dow + , subq_4.listing__ds__extract_doy AS listing__ds__extract_doy + , subq_4.listing__created_at__day AS listing__created_at__day + , subq_4.listing__created_at__week AS listing__created_at__week + , subq_4.listing__created_at__month AS listing__created_at__month + , subq_4.listing__created_at__quarter AS listing__created_at__quarter + , subq_4.listing__created_at__year AS listing__created_at__year + , subq_4.listing__created_at__extract_year AS listing__created_at__extract_year + , subq_4.listing__created_at__extract_quarter AS listing__created_at__extract_quarter + , subq_4.listing__created_at__extract_month AS listing__created_at__extract_month + , subq_4.listing__created_at__extract_day AS listing__created_at__extract_day + , subq_4.listing__created_at__extract_dow AS listing__created_at__extract_dow + , subq_4.listing__created_at__extract_doy AS listing__created_at__extract_doy + , subq_4.metric_time__day AS metric_time__day + , subq_4.metric_time__week AS metric_time__week + , subq_4.metric_time__month AS metric_time__month + , subq_4.metric_time__quarter AS metric_time__quarter + , subq_4.metric_time__year AS metric_time__year + , subq_4.metric_time__extract_year AS metric_time__extract_year + , subq_4.metric_time__extract_quarter AS metric_time__extract_quarter + , subq_4.metric_time__extract_month AS metric_time__extract_month + , subq_4.metric_time__extract_day AS metric_time__extract_day + , subq_4.metric_time__extract_dow AS metric_time__extract_dow + , subq_4.metric_time__extract_doy AS metric_time__extract_doy , subq_4.listing AS listing , subq_4.user AS user , subq_4.listing__user AS listing__user @@ -393,7 +394,6 @@ FROM ( , subq_4.listing__country_latest AS listing__country_latest , subq_4.listing__is_lux_latest AS listing__is_lux_latest , subq_4.listing__capacity_latest AS listing__capacity_latest - , subq_7.home_state AS user__home_state , subq_4.listings AS listings , subq_4.largest_listing AS largest_listing , subq_4.smallest_listing AS smallest_listing diff --git a/tests_metricflow/snapshots/test_granularity_date_part_rendering.py/SqlQueryPlan/DuckDB/test_simple_metric_with_joined_sub_daily_dimension__plan0.sql b/tests_metricflow/snapshots/test_granularity_date_part_rendering.py/SqlQueryPlan/DuckDB/test_simple_metric_with_joined_sub_daily_dimension__plan0.sql index c2505892cd..1f119872ca 100644 --- a/tests_metricflow/snapshots/test_granularity_date_part_rendering.py/SqlQueryPlan/DuckDB/test_simple_metric_with_joined_sub_daily_dimension__plan0.sql +++ b/tests_metricflow/snapshots/test_granularity_date_part_rendering.py/SqlQueryPlan/DuckDB/test_simple_metric_with_joined_sub_daily_dimension__plan0.sql @@ -15,9 +15,9 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.ds_partitioned__day AS ds_partitioned__day - , subq_9.user__ds_partitioned__day AS listing__user__ds_partitioned__day + subq_9.user__ds_partitioned__day AS listing__user__ds_partitioned__day , subq_9.user__bio_added_ts__minute AS listing__user__bio_added_ts__minute + , subq_2.ds_partitioned__day AS ds_partitioned__day , subq_2.listing AS listing , subq_2.bookings AS bookings FROM ( @@ -231,61 +231,7 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_4.ds__day AS ds__day - , subq_4.ds__week AS ds__week - , subq_4.ds__month AS ds__month - , subq_4.ds__quarter AS ds__quarter - , subq_4.ds__year AS ds__year - , subq_4.ds__extract_year AS ds__extract_year - , subq_4.ds__extract_quarter AS ds__extract_quarter - , subq_4.ds__extract_month AS ds__extract_month - , subq_4.ds__extract_day AS ds__extract_day - , subq_4.ds__extract_dow AS ds__extract_dow - , subq_4.ds__extract_doy AS ds__extract_doy - , subq_4.created_at__day AS created_at__day - , subq_4.created_at__week AS created_at__week - , subq_4.created_at__month AS created_at__month - , subq_4.created_at__quarter AS created_at__quarter - , subq_4.created_at__year AS created_at__year - , subq_4.created_at__extract_year AS created_at__extract_year - , subq_4.created_at__extract_quarter AS created_at__extract_quarter - , subq_4.created_at__extract_month AS created_at__extract_month - , subq_4.created_at__extract_day AS created_at__extract_day - , subq_4.created_at__extract_dow AS created_at__extract_dow - , subq_4.created_at__extract_doy AS created_at__extract_doy - , subq_4.listing__ds__day AS listing__ds__day - , subq_4.listing__ds__week AS listing__ds__week - , subq_4.listing__ds__month AS listing__ds__month - , subq_4.listing__ds__quarter AS listing__ds__quarter - , subq_4.listing__ds__year AS listing__ds__year - , subq_4.listing__ds__extract_year AS listing__ds__extract_year - , subq_4.listing__ds__extract_quarter AS listing__ds__extract_quarter - , subq_4.listing__ds__extract_month AS listing__ds__extract_month - , subq_4.listing__ds__extract_day AS listing__ds__extract_day - , subq_4.listing__ds__extract_dow AS listing__ds__extract_dow - , subq_4.listing__ds__extract_doy AS listing__ds__extract_doy - , subq_4.listing__created_at__day AS listing__created_at__day - , subq_4.listing__created_at__week AS listing__created_at__week - , subq_4.listing__created_at__month AS listing__created_at__month - , subq_4.listing__created_at__quarter AS listing__created_at__quarter - , subq_4.listing__created_at__year AS listing__created_at__year - , subq_4.listing__created_at__extract_year AS listing__created_at__extract_year - , subq_4.listing__created_at__extract_quarter AS listing__created_at__extract_quarter - , subq_4.listing__created_at__extract_month AS listing__created_at__extract_month - , subq_4.listing__created_at__extract_day AS listing__created_at__extract_day - , subq_4.listing__created_at__extract_dow AS listing__created_at__extract_dow - , subq_4.listing__created_at__extract_doy AS listing__created_at__extract_doy - , subq_4.metric_time__day AS metric_time__day - , subq_4.metric_time__week AS metric_time__week - , subq_4.metric_time__month AS metric_time__month - , subq_4.metric_time__quarter AS metric_time__quarter - , subq_4.metric_time__year AS metric_time__year - , subq_4.metric_time__extract_year AS metric_time__extract_year - , subq_4.metric_time__extract_quarter AS metric_time__extract_quarter - , subq_4.metric_time__extract_month AS metric_time__extract_month - , subq_4.metric_time__extract_day AS metric_time__extract_day - , subq_4.metric_time__extract_dow AS metric_time__extract_dow - , subq_4.metric_time__extract_doy AS metric_time__extract_doy + subq_7.home_state AS user__home_state , subq_7.ds__day AS user__ds__day , subq_7.ds__week AS user__ds__week , subq_7.ds__month AS user__ds__month @@ -384,6 +330,61 @@ FROM ( , subq_7.metric_time__extract_day AS user__metric_time__extract_day , subq_7.metric_time__extract_dow AS user__metric_time__extract_dow , subq_7.metric_time__extract_doy AS user__metric_time__extract_doy + , subq_4.ds__day AS ds__day + , subq_4.ds__week AS ds__week + , subq_4.ds__month AS ds__month + , subq_4.ds__quarter AS ds__quarter + , subq_4.ds__year AS ds__year + , subq_4.ds__extract_year AS ds__extract_year + , subq_4.ds__extract_quarter AS ds__extract_quarter + , subq_4.ds__extract_month AS ds__extract_month + , subq_4.ds__extract_day AS ds__extract_day + , subq_4.ds__extract_dow AS ds__extract_dow + , subq_4.ds__extract_doy AS ds__extract_doy + , subq_4.created_at__day AS created_at__day + , subq_4.created_at__week AS created_at__week + , subq_4.created_at__month AS created_at__month + , subq_4.created_at__quarter AS created_at__quarter + , subq_4.created_at__year AS created_at__year + , subq_4.created_at__extract_year AS created_at__extract_year + , subq_4.created_at__extract_quarter AS created_at__extract_quarter + , subq_4.created_at__extract_month AS created_at__extract_month + , subq_4.created_at__extract_day AS created_at__extract_day + , subq_4.created_at__extract_dow AS created_at__extract_dow + , subq_4.created_at__extract_doy AS created_at__extract_doy + , subq_4.listing__ds__day AS listing__ds__day + , subq_4.listing__ds__week AS listing__ds__week + , subq_4.listing__ds__month AS listing__ds__month + , subq_4.listing__ds__quarter AS listing__ds__quarter + , subq_4.listing__ds__year AS listing__ds__year + , subq_4.listing__ds__extract_year AS listing__ds__extract_year + , subq_4.listing__ds__extract_quarter AS listing__ds__extract_quarter + , subq_4.listing__ds__extract_month AS listing__ds__extract_month + , subq_4.listing__ds__extract_day AS listing__ds__extract_day + , subq_4.listing__ds__extract_dow AS listing__ds__extract_dow + , subq_4.listing__ds__extract_doy AS listing__ds__extract_doy + , subq_4.listing__created_at__day AS listing__created_at__day + , subq_4.listing__created_at__week AS listing__created_at__week + , subq_4.listing__created_at__month AS listing__created_at__month + , subq_4.listing__created_at__quarter AS listing__created_at__quarter + , subq_4.listing__created_at__year AS listing__created_at__year + , subq_4.listing__created_at__extract_year AS listing__created_at__extract_year + , subq_4.listing__created_at__extract_quarter AS listing__created_at__extract_quarter + , subq_4.listing__created_at__extract_month AS listing__created_at__extract_month + , subq_4.listing__created_at__extract_day AS listing__created_at__extract_day + , subq_4.listing__created_at__extract_dow AS listing__created_at__extract_dow + , subq_4.listing__created_at__extract_doy AS listing__created_at__extract_doy + , subq_4.metric_time__day AS metric_time__day + , subq_4.metric_time__week AS metric_time__week + , subq_4.metric_time__month AS metric_time__month + , subq_4.metric_time__quarter AS metric_time__quarter + , subq_4.metric_time__year AS metric_time__year + , subq_4.metric_time__extract_year AS metric_time__extract_year + , subq_4.metric_time__extract_quarter AS metric_time__extract_quarter + , subq_4.metric_time__extract_month AS metric_time__extract_month + , subq_4.metric_time__extract_day AS metric_time__extract_day + , subq_4.metric_time__extract_dow AS metric_time__extract_dow + , subq_4.metric_time__extract_doy AS metric_time__extract_doy , subq_4.listing AS listing , subq_4.user AS user , subq_4.listing__user AS listing__user @@ -393,7 +394,6 @@ FROM ( , subq_4.listing__country_latest AS listing__country_latest , subq_4.listing__is_lux_latest AS listing__is_lux_latest , subq_4.listing__capacity_latest AS listing__capacity_latest - , subq_7.home_state AS user__home_state , subq_4.listings AS listings , subq_4.largest_listing AS largest_listing , subq_4.smallest_listing AS smallest_listing diff --git a/tests_metricflow/snapshots/test_granularity_date_part_rendering.py/SqlQueryPlan/Postgres/test_simple_metric_with_joined_sub_daily_dimension__plan0.sql b/tests_metricflow/snapshots/test_granularity_date_part_rendering.py/SqlQueryPlan/Postgres/test_simple_metric_with_joined_sub_daily_dimension__plan0.sql index c2505892cd..1f119872ca 100644 --- a/tests_metricflow/snapshots/test_granularity_date_part_rendering.py/SqlQueryPlan/Postgres/test_simple_metric_with_joined_sub_daily_dimension__plan0.sql +++ b/tests_metricflow/snapshots/test_granularity_date_part_rendering.py/SqlQueryPlan/Postgres/test_simple_metric_with_joined_sub_daily_dimension__plan0.sql @@ -15,9 +15,9 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.ds_partitioned__day AS ds_partitioned__day - , subq_9.user__ds_partitioned__day AS listing__user__ds_partitioned__day + subq_9.user__ds_partitioned__day AS listing__user__ds_partitioned__day , subq_9.user__bio_added_ts__minute AS listing__user__bio_added_ts__minute + , subq_2.ds_partitioned__day AS ds_partitioned__day , subq_2.listing AS listing , subq_2.bookings AS bookings FROM ( @@ -231,61 +231,7 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_4.ds__day AS ds__day - , subq_4.ds__week AS ds__week - , subq_4.ds__month AS ds__month - , subq_4.ds__quarter AS ds__quarter - , subq_4.ds__year AS ds__year - , subq_4.ds__extract_year AS ds__extract_year - , subq_4.ds__extract_quarter AS ds__extract_quarter - , subq_4.ds__extract_month AS ds__extract_month - , subq_4.ds__extract_day AS ds__extract_day - , subq_4.ds__extract_dow AS ds__extract_dow - , subq_4.ds__extract_doy AS ds__extract_doy - , subq_4.created_at__day AS created_at__day - , subq_4.created_at__week AS created_at__week - , subq_4.created_at__month AS created_at__month - , subq_4.created_at__quarter AS created_at__quarter - , subq_4.created_at__year AS created_at__year - , subq_4.created_at__extract_year AS created_at__extract_year - , subq_4.created_at__extract_quarter AS created_at__extract_quarter - , subq_4.created_at__extract_month AS created_at__extract_month - , subq_4.created_at__extract_day AS created_at__extract_day - , subq_4.created_at__extract_dow AS created_at__extract_dow - , subq_4.created_at__extract_doy AS created_at__extract_doy - , subq_4.listing__ds__day AS listing__ds__day - , subq_4.listing__ds__week AS listing__ds__week - , subq_4.listing__ds__month AS listing__ds__month - , subq_4.listing__ds__quarter AS listing__ds__quarter - , subq_4.listing__ds__year AS listing__ds__year - , subq_4.listing__ds__extract_year AS listing__ds__extract_year - , subq_4.listing__ds__extract_quarter AS listing__ds__extract_quarter - , subq_4.listing__ds__extract_month AS listing__ds__extract_month - , subq_4.listing__ds__extract_day AS listing__ds__extract_day - , subq_4.listing__ds__extract_dow AS listing__ds__extract_dow - , subq_4.listing__ds__extract_doy AS listing__ds__extract_doy - , subq_4.listing__created_at__day AS listing__created_at__day - , subq_4.listing__created_at__week AS listing__created_at__week - , subq_4.listing__created_at__month AS listing__created_at__month - , subq_4.listing__created_at__quarter AS listing__created_at__quarter - , subq_4.listing__created_at__year AS listing__created_at__year - , subq_4.listing__created_at__extract_year AS listing__created_at__extract_year - , subq_4.listing__created_at__extract_quarter AS listing__created_at__extract_quarter - , subq_4.listing__created_at__extract_month AS listing__created_at__extract_month - , subq_4.listing__created_at__extract_day AS listing__created_at__extract_day - , subq_4.listing__created_at__extract_dow AS listing__created_at__extract_dow - , subq_4.listing__created_at__extract_doy AS listing__created_at__extract_doy - , subq_4.metric_time__day AS metric_time__day - , subq_4.metric_time__week AS metric_time__week - , subq_4.metric_time__month AS metric_time__month - , subq_4.metric_time__quarter AS metric_time__quarter - , subq_4.metric_time__year AS metric_time__year - , subq_4.metric_time__extract_year AS metric_time__extract_year - , subq_4.metric_time__extract_quarter AS metric_time__extract_quarter - , subq_4.metric_time__extract_month AS metric_time__extract_month - , subq_4.metric_time__extract_day AS metric_time__extract_day - , subq_4.metric_time__extract_dow AS metric_time__extract_dow - , subq_4.metric_time__extract_doy AS metric_time__extract_doy + subq_7.home_state AS user__home_state , subq_7.ds__day AS user__ds__day , subq_7.ds__week AS user__ds__week , subq_7.ds__month AS user__ds__month @@ -384,6 +330,61 @@ FROM ( , subq_7.metric_time__extract_day AS user__metric_time__extract_day , subq_7.metric_time__extract_dow AS user__metric_time__extract_dow , subq_7.metric_time__extract_doy AS user__metric_time__extract_doy + , subq_4.ds__day AS ds__day + , subq_4.ds__week AS ds__week + , subq_4.ds__month AS ds__month + , subq_4.ds__quarter AS ds__quarter + , subq_4.ds__year AS ds__year + , subq_4.ds__extract_year AS ds__extract_year + , subq_4.ds__extract_quarter AS ds__extract_quarter + , subq_4.ds__extract_month AS ds__extract_month + , subq_4.ds__extract_day AS ds__extract_day + , subq_4.ds__extract_dow AS ds__extract_dow + , subq_4.ds__extract_doy AS ds__extract_doy + , subq_4.created_at__day AS created_at__day + , subq_4.created_at__week AS created_at__week + , subq_4.created_at__month AS created_at__month + , subq_4.created_at__quarter AS created_at__quarter + , subq_4.created_at__year AS created_at__year + , subq_4.created_at__extract_year AS created_at__extract_year + , subq_4.created_at__extract_quarter AS created_at__extract_quarter + , subq_4.created_at__extract_month AS created_at__extract_month + , subq_4.created_at__extract_day AS created_at__extract_day + , subq_4.created_at__extract_dow AS created_at__extract_dow + , subq_4.created_at__extract_doy AS created_at__extract_doy + , subq_4.listing__ds__day AS listing__ds__day + , subq_4.listing__ds__week AS listing__ds__week + , subq_4.listing__ds__month AS listing__ds__month + , subq_4.listing__ds__quarter AS listing__ds__quarter + , subq_4.listing__ds__year AS listing__ds__year + , subq_4.listing__ds__extract_year AS listing__ds__extract_year + , subq_4.listing__ds__extract_quarter AS listing__ds__extract_quarter + , subq_4.listing__ds__extract_month AS listing__ds__extract_month + , subq_4.listing__ds__extract_day AS listing__ds__extract_day + , subq_4.listing__ds__extract_dow AS listing__ds__extract_dow + , subq_4.listing__ds__extract_doy AS listing__ds__extract_doy + , subq_4.listing__created_at__day AS listing__created_at__day + , subq_4.listing__created_at__week AS listing__created_at__week + , subq_4.listing__created_at__month AS listing__created_at__month + , subq_4.listing__created_at__quarter AS listing__created_at__quarter + , subq_4.listing__created_at__year AS listing__created_at__year + , subq_4.listing__created_at__extract_year AS listing__created_at__extract_year + , subq_4.listing__created_at__extract_quarter AS listing__created_at__extract_quarter + , subq_4.listing__created_at__extract_month AS listing__created_at__extract_month + , subq_4.listing__created_at__extract_day AS listing__created_at__extract_day + , subq_4.listing__created_at__extract_dow AS listing__created_at__extract_dow + , subq_4.listing__created_at__extract_doy AS listing__created_at__extract_doy + , subq_4.metric_time__day AS metric_time__day + , subq_4.metric_time__week AS metric_time__week + , subq_4.metric_time__month AS metric_time__month + , subq_4.metric_time__quarter AS metric_time__quarter + , subq_4.metric_time__year AS metric_time__year + , subq_4.metric_time__extract_year AS metric_time__extract_year + , subq_4.metric_time__extract_quarter AS metric_time__extract_quarter + , subq_4.metric_time__extract_month AS metric_time__extract_month + , subq_4.metric_time__extract_day AS metric_time__extract_day + , subq_4.metric_time__extract_dow AS metric_time__extract_dow + , subq_4.metric_time__extract_doy AS metric_time__extract_doy , subq_4.listing AS listing , subq_4.user AS user , subq_4.listing__user AS listing__user @@ -393,7 +394,6 @@ FROM ( , subq_4.listing__country_latest AS listing__country_latest , subq_4.listing__is_lux_latest AS listing__is_lux_latest , subq_4.listing__capacity_latest AS listing__capacity_latest - , subq_7.home_state AS user__home_state , subq_4.listings AS listings , subq_4.largest_listing AS largest_listing , subq_4.smallest_listing AS smallest_listing diff --git a/tests_metricflow/snapshots/test_granularity_date_part_rendering.py/SqlQueryPlan/Redshift/test_simple_metric_with_joined_sub_daily_dimension__plan0.sql b/tests_metricflow/snapshots/test_granularity_date_part_rendering.py/SqlQueryPlan/Redshift/test_simple_metric_with_joined_sub_daily_dimension__plan0.sql index 9982f51ad0..565b27172d 100644 --- a/tests_metricflow/snapshots/test_granularity_date_part_rendering.py/SqlQueryPlan/Redshift/test_simple_metric_with_joined_sub_daily_dimension__plan0.sql +++ b/tests_metricflow/snapshots/test_granularity_date_part_rendering.py/SqlQueryPlan/Redshift/test_simple_metric_with_joined_sub_daily_dimension__plan0.sql @@ -15,9 +15,9 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.ds_partitioned__day AS ds_partitioned__day - , subq_9.user__ds_partitioned__day AS listing__user__ds_partitioned__day + subq_9.user__ds_partitioned__day AS listing__user__ds_partitioned__day , subq_9.user__bio_added_ts__minute AS listing__user__bio_added_ts__minute + , subq_2.ds_partitioned__day AS ds_partitioned__day , subq_2.listing AS listing , subq_2.bookings AS bookings FROM ( @@ -231,61 +231,7 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_4.ds__day AS ds__day - , subq_4.ds__week AS ds__week - , subq_4.ds__month AS ds__month - , subq_4.ds__quarter AS ds__quarter - , subq_4.ds__year AS ds__year - , subq_4.ds__extract_year AS ds__extract_year - , subq_4.ds__extract_quarter AS ds__extract_quarter - , subq_4.ds__extract_month AS ds__extract_month - , subq_4.ds__extract_day AS ds__extract_day - , subq_4.ds__extract_dow AS ds__extract_dow - , subq_4.ds__extract_doy AS ds__extract_doy - , subq_4.created_at__day AS created_at__day - , subq_4.created_at__week AS created_at__week - , subq_4.created_at__month AS created_at__month - , subq_4.created_at__quarter AS created_at__quarter - , subq_4.created_at__year AS created_at__year - , subq_4.created_at__extract_year AS created_at__extract_year - , subq_4.created_at__extract_quarter AS created_at__extract_quarter - , subq_4.created_at__extract_month AS created_at__extract_month - , subq_4.created_at__extract_day AS created_at__extract_day - , subq_4.created_at__extract_dow AS created_at__extract_dow - , subq_4.created_at__extract_doy AS created_at__extract_doy - , subq_4.listing__ds__day AS listing__ds__day - , subq_4.listing__ds__week AS listing__ds__week - , subq_4.listing__ds__month AS listing__ds__month - , subq_4.listing__ds__quarter AS listing__ds__quarter - , subq_4.listing__ds__year AS listing__ds__year - , subq_4.listing__ds__extract_year AS listing__ds__extract_year - , subq_4.listing__ds__extract_quarter AS listing__ds__extract_quarter - , subq_4.listing__ds__extract_month AS listing__ds__extract_month - , subq_4.listing__ds__extract_day AS listing__ds__extract_day - , subq_4.listing__ds__extract_dow AS listing__ds__extract_dow - , subq_4.listing__ds__extract_doy AS listing__ds__extract_doy - , subq_4.listing__created_at__day AS listing__created_at__day - , subq_4.listing__created_at__week AS listing__created_at__week - , subq_4.listing__created_at__month AS listing__created_at__month - , subq_4.listing__created_at__quarter AS listing__created_at__quarter - , subq_4.listing__created_at__year AS listing__created_at__year - , subq_4.listing__created_at__extract_year AS listing__created_at__extract_year - , subq_4.listing__created_at__extract_quarter AS listing__created_at__extract_quarter - , subq_4.listing__created_at__extract_month AS listing__created_at__extract_month - , subq_4.listing__created_at__extract_day AS listing__created_at__extract_day - , subq_4.listing__created_at__extract_dow AS listing__created_at__extract_dow - , subq_4.listing__created_at__extract_doy AS listing__created_at__extract_doy - , subq_4.metric_time__day AS metric_time__day - , subq_4.metric_time__week AS metric_time__week - , subq_4.metric_time__month AS metric_time__month - , subq_4.metric_time__quarter AS metric_time__quarter - , subq_4.metric_time__year AS metric_time__year - , subq_4.metric_time__extract_year AS metric_time__extract_year - , subq_4.metric_time__extract_quarter AS metric_time__extract_quarter - , subq_4.metric_time__extract_month AS metric_time__extract_month - , subq_4.metric_time__extract_day AS metric_time__extract_day - , subq_4.metric_time__extract_dow AS metric_time__extract_dow - , subq_4.metric_time__extract_doy AS metric_time__extract_doy + subq_7.home_state AS user__home_state , subq_7.ds__day AS user__ds__day , subq_7.ds__week AS user__ds__week , subq_7.ds__month AS user__ds__month @@ -384,6 +330,61 @@ FROM ( , subq_7.metric_time__extract_day AS user__metric_time__extract_day , subq_7.metric_time__extract_dow AS user__metric_time__extract_dow , subq_7.metric_time__extract_doy AS user__metric_time__extract_doy + , subq_4.ds__day AS ds__day + , subq_4.ds__week AS ds__week + , subq_4.ds__month AS ds__month + , subq_4.ds__quarter AS ds__quarter + , subq_4.ds__year AS ds__year + , subq_4.ds__extract_year AS ds__extract_year + , subq_4.ds__extract_quarter AS ds__extract_quarter + , subq_4.ds__extract_month AS ds__extract_month + , subq_4.ds__extract_day AS ds__extract_day + , subq_4.ds__extract_dow AS ds__extract_dow + , subq_4.ds__extract_doy AS ds__extract_doy + , subq_4.created_at__day AS created_at__day + , subq_4.created_at__week AS created_at__week + , subq_4.created_at__month AS created_at__month + , subq_4.created_at__quarter AS created_at__quarter + , subq_4.created_at__year AS created_at__year + , subq_4.created_at__extract_year AS created_at__extract_year + , subq_4.created_at__extract_quarter AS created_at__extract_quarter + , subq_4.created_at__extract_month AS created_at__extract_month + , subq_4.created_at__extract_day AS created_at__extract_day + , subq_4.created_at__extract_dow AS created_at__extract_dow + , subq_4.created_at__extract_doy AS created_at__extract_doy + , subq_4.listing__ds__day AS listing__ds__day + , subq_4.listing__ds__week AS listing__ds__week + , subq_4.listing__ds__month AS listing__ds__month + , subq_4.listing__ds__quarter AS listing__ds__quarter + , subq_4.listing__ds__year AS listing__ds__year + , subq_4.listing__ds__extract_year AS listing__ds__extract_year + , subq_4.listing__ds__extract_quarter AS listing__ds__extract_quarter + , subq_4.listing__ds__extract_month AS listing__ds__extract_month + , subq_4.listing__ds__extract_day AS listing__ds__extract_day + , subq_4.listing__ds__extract_dow AS listing__ds__extract_dow + , subq_4.listing__ds__extract_doy AS listing__ds__extract_doy + , subq_4.listing__created_at__day AS listing__created_at__day + , subq_4.listing__created_at__week AS listing__created_at__week + , subq_4.listing__created_at__month AS listing__created_at__month + , subq_4.listing__created_at__quarter AS listing__created_at__quarter + , subq_4.listing__created_at__year AS listing__created_at__year + , subq_4.listing__created_at__extract_year AS listing__created_at__extract_year + , subq_4.listing__created_at__extract_quarter AS listing__created_at__extract_quarter + , subq_4.listing__created_at__extract_month AS listing__created_at__extract_month + , subq_4.listing__created_at__extract_day AS listing__created_at__extract_day + , subq_4.listing__created_at__extract_dow AS listing__created_at__extract_dow + , subq_4.listing__created_at__extract_doy AS listing__created_at__extract_doy + , subq_4.metric_time__day AS metric_time__day + , subq_4.metric_time__week AS metric_time__week + , subq_4.metric_time__month AS metric_time__month + , subq_4.metric_time__quarter AS metric_time__quarter + , subq_4.metric_time__year AS metric_time__year + , subq_4.metric_time__extract_year AS metric_time__extract_year + , subq_4.metric_time__extract_quarter AS metric_time__extract_quarter + , subq_4.metric_time__extract_month AS metric_time__extract_month + , subq_4.metric_time__extract_day AS metric_time__extract_day + , subq_4.metric_time__extract_dow AS metric_time__extract_dow + , subq_4.metric_time__extract_doy AS metric_time__extract_doy , subq_4.listing AS listing , subq_4.user AS user , subq_4.listing__user AS listing__user @@ -393,7 +394,6 @@ FROM ( , subq_4.listing__country_latest AS listing__country_latest , subq_4.listing__is_lux_latest AS listing__is_lux_latest , subq_4.listing__capacity_latest AS listing__capacity_latest - , subq_7.home_state AS user__home_state , subq_4.listings AS listings , subq_4.largest_listing AS largest_listing , subq_4.smallest_listing AS smallest_listing diff --git a/tests_metricflow/snapshots/test_granularity_date_part_rendering.py/SqlQueryPlan/Snowflake/test_simple_metric_with_joined_sub_daily_dimension__plan0.sql b/tests_metricflow/snapshots/test_granularity_date_part_rendering.py/SqlQueryPlan/Snowflake/test_simple_metric_with_joined_sub_daily_dimension__plan0.sql index 06117ebe55..045f91b040 100644 --- a/tests_metricflow/snapshots/test_granularity_date_part_rendering.py/SqlQueryPlan/Snowflake/test_simple_metric_with_joined_sub_daily_dimension__plan0.sql +++ b/tests_metricflow/snapshots/test_granularity_date_part_rendering.py/SqlQueryPlan/Snowflake/test_simple_metric_with_joined_sub_daily_dimension__plan0.sql @@ -15,9 +15,9 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.ds_partitioned__day AS ds_partitioned__day - , subq_9.user__ds_partitioned__day AS listing__user__ds_partitioned__day + subq_9.user__ds_partitioned__day AS listing__user__ds_partitioned__day , subq_9.user__bio_added_ts__minute AS listing__user__bio_added_ts__minute + , subq_2.ds_partitioned__day AS ds_partitioned__day , subq_2.listing AS listing , subq_2.bookings AS bookings FROM ( @@ -231,61 +231,7 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_4.ds__day AS ds__day - , subq_4.ds__week AS ds__week - , subq_4.ds__month AS ds__month - , subq_4.ds__quarter AS ds__quarter - , subq_4.ds__year AS ds__year - , subq_4.ds__extract_year AS ds__extract_year - , subq_4.ds__extract_quarter AS ds__extract_quarter - , subq_4.ds__extract_month AS ds__extract_month - , subq_4.ds__extract_day AS ds__extract_day - , subq_4.ds__extract_dow AS ds__extract_dow - , subq_4.ds__extract_doy AS ds__extract_doy - , subq_4.created_at__day AS created_at__day - , subq_4.created_at__week AS created_at__week - , subq_4.created_at__month AS created_at__month - , subq_4.created_at__quarter AS created_at__quarter - , subq_4.created_at__year AS created_at__year - , subq_4.created_at__extract_year AS created_at__extract_year - , subq_4.created_at__extract_quarter AS created_at__extract_quarter - , subq_4.created_at__extract_month AS created_at__extract_month - , subq_4.created_at__extract_day AS created_at__extract_day - , subq_4.created_at__extract_dow AS created_at__extract_dow - , subq_4.created_at__extract_doy AS created_at__extract_doy - , subq_4.listing__ds__day AS listing__ds__day - , subq_4.listing__ds__week AS listing__ds__week - , subq_4.listing__ds__month AS listing__ds__month - , subq_4.listing__ds__quarter AS listing__ds__quarter - , subq_4.listing__ds__year AS listing__ds__year - , subq_4.listing__ds__extract_year AS listing__ds__extract_year - , subq_4.listing__ds__extract_quarter AS listing__ds__extract_quarter - , subq_4.listing__ds__extract_month AS listing__ds__extract_month - , subq_4.listing__ds__extract_day AS listing__ds__extract_day - , subq_4.listing__ds__extract_dow AS listing__ds__extract_dow - , subq_4.listing__ds__extract_doy AS listing__ds__extract_doy - , subq_4.listing__created_at__day AS listing__created_at__day - , subq_4.listing__created_at__week AS listing__created_at__week - , subq_4.listing__created_at__month AS listing__created_at__month - , subq_4.listing__created_at__quarter AS listing__created_at__quarter - , subq_4.listing__created_at__year AS listing__created_at__year - , subq_4.listing__created_at__extract_year AS listing__created_at__extract_year - , subq_4.listing__created_at__extract_quarter AS listing__created_at__extract_quarter - , subq_4.listing__created_at__extract_month AS listing__created_at__extract_month - , subq_4.listing__created_at__extract_day AS listing__created_at__extract_day - , subq_4.listing__created_at__extract_dow AS listing__created_at__extract_dow - , subq_4.listing__created_at__extract_doy AS listing__created_at__extract_doy - , subq_4.metric_time__day AS metric_time__day - , subq_4.metric_time__week AS metric_time__week - , subq_4.metric_time__month AS metric_time__month - , subq_4.metric_time__quarter AS metric_time__quarter - , subq_4.metric_time__year AS metric_time__year - , subq_4.metric_time__extract_year AS metric_time__extract_year - , subq_4.metric_time__extract_quarter AS metric_time__extract_quarter - , subq_4.metric_time__extract_month AS metric_time__extract_month - , subq_4.metric_time__extract_day AS metric_time__extract_day - , subq_4.metric_time__extract_dow AS metric_time__extract_dow - , subq_4.metric_time__extract_doy AS metric_time__extract_doy + subq_7.home_state AS user__home_state , subq_7.ds__day AS user__ds__day , subq_7.ds__week AS user__ds__week , subq_7.ds__month AS user__ds__month @@ -384,6 +330,61 @@ FROM ( , subq_7.metric_time__extract_day AS user__metric_time__extract_day , subq_7.metric_time__extract_dow AS user__metric_time__extract_dow , subq_7.metric_time__extract_doy AS user__metric_time__extract_doy + , subq_4.ds__day AS ds__day + , subq_4.ds__week AS ds__week + , subq_4.ds__month AS ds__month + , subq_4.ds__quarter AS ds__quarter + , subq_4.ds__year AS ds__year + , subq_4.ds__extract_year AS ds__extract_year + , subq_4.ds__extract_quarter AS ds__extract_quarter + , subq_4.ds__extract_month AS ds__extract_month + , subq_4.ds__extract_day AS ds__extract_day + , subq_4.ds__extract_dow AS ds__extract_dow + , subq_4.ds__extract_doy AS ds__extract_doy + , subq_4.created_at__day AS created_at__day + , subq_4.created_at__week AS created_at__week + , subq_4.created_at__month AS created_at__month + , subq_4.created_at__quarter AS created_at__quarter + , subq_4.created_at__year AS created_at__year + , subq_4.created_at__extract_year AS created_at__extract_year + , subq_4.created_at__extract_quarter AS created_at__extract_quarter + , subq_4.created_at__extract_month AS created_at__extract_month + , subq_4.created_at__extract_day AS created_at__extract_day + , subq_4.created_at__extract_dow AS created_at__extract_dow + , subq_4.created_at__extract_doy AS created_at__extract_doy + , subq_4.listing__ds__day AS listing__ds__day + , subq_4.listing__ds__week AS listing__ds__week + , subq_4.listing__ds__month AS listing__ds__month + , subq_4.listing__ds__quarter AS listing__ds__quarter + , subq_4.listing__ds__year AS listing__ds__year + , subq_4.listing__ds__extract_year AS listing__ds__extract_year + , subq_4.listing__ds__extract_quarter AS listing__ds__extract_quarter + , subq_4.listing__ds__extract_month AS listing__ds__extract_month + , subq_4.listing__ds__extract_day AS listing__ds__extract_day + , subq_4.listing__ds__extract_dow AS listing__ds__extract_dow + , subq_4.listing__ds__extract_doy AS listing__ds__extract_doy + , subq_4.listing__created_at__day AS listing__created_at__day + , subq_4.listing__created_at__week AS listing__created_at__week + , subq_4.listing__created_at__month AS listing__created_at__month + , subq_4.listing__created_at__quarter AS listing__created_at__quarter + , subq_4.listing__created_at__year AS listing__created_at__year + , subq_4.listing__created_at__extract_year AS listing__created_at__extract_year + , subq_4.listing__created_at__extract_quarter AS listing__created_at__extract_quarter + , subq_4.listing__created_at__extract_month AS listing__created_at__extract_month + , subq_4.listing__created_at__extract_day AS listing__created_at__extract_day + , subq_4.listing__created_at__extract_dow AS listing__created_at__extract_dow + , subq_4.listing__created_at__extract_doy AS listing__created_at__extract_doy + , subq_4.metric_time__day AS metric_time__day + , subq_4.metric_time__week AS metric_time__week + , subq_4.metric_time__month AS metric_time__month + , subq_4.metric_time__quarter AS metric_time__quarter + , subq_4.metric_time__year AS metric_time__year + , subq_4.metric_time__extract_year AS metric_time__extract_year + , subq_4.metric_time__extract_quarter AS metric_time__extract_quarter + , subq_4.metric_time__extract_month AS metric_time__extract_month + , subq_4.metric_time__extract_day AS metric_time__extract_day + , subq_4.metric_time__extract_dow AS metric_time__extract_dow + , subq_4.metric_time__extract_doy AS metric_time__extract_doy , subq_4.listing AS listing , subq_4.user AS user , subq_4.listing__user AS listing__user @@ -393,7 +394,6 @@ FROM ( , subq_4.listing__country_latest AS listing__country_latest , subq_4.listing__is_lux_latest AS listing__is_lux_latest , subq_4.listing__capacity_latest AS listing__capacity_latest - , subq_7.home_state AS user__home_state , subq_4.listings AS listings , subq_4.largest_listing AS largest_listing , subq_4.smallest_listing AS smallest_listing diff --git a/tests_metricflow/snapshots/test_granularity_date_part_rendering.py/SqlQueryPlan/Trino/test_simple_metric_with_joined_sub_daily_dimension__plan0.sql b/tests_metricflow/snapshots/test_granularity_date_part_rendering.py/SqlQueryPlan/Trino/test_simple_metric_with_joined_sub_daily_dimension__plan0.sql index 667fbfe1e6..1e33b6dfe2 100644 --- a/tests_metricflow/snapshots/test_granularity_date_part_rendering.py/SqlQueryPlan/Trino/test_simple_metric_with_joined_sub_daily_dimension__plan0.sql +++ b/tests_metricflow/snapshots/test_granularity_date_part_rendering.py/SqlQueryPlan/Trino/test_simple_metric_with_joined_sub_daily_dimension__plan0.sql @@ -15,9 +15,9 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.ds_partitioned__day AS ds_partitioned__day - , subq_9.user__ds_partitioned__day AS listing__user__ds_partitioned__day + subq_9.user__ds_partitioned__day AS listing__user__ds_partitioned__day , subq_9.user__bio_added_ts__minute AS listing__user__bio_added_ts__minute + , subq_2.ds_partitioned__day AS ds_partitioned__day , subq_2.listing AS listing , subq_2.bookings AS bookings FROM ( @@ -231,61 +231,7 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_4.ds__day AS ds__day - , subq_4.ds__week AS ds__week - , subq_4.ds__month AS ds__month - , subq_4.ds__quarter AS ds__quarter - , subq_4.ds__year AS ds__year - , subq_4.ds__extract_year AS ds__extract_year - , subq_4.ds__extract_quarter AS ds__extract_quarter - , subq_4.ds__extract_month AS ds__extract_month - , subq_4.ds__extract_day AS ds__extract_day - , subq_4.ds__extract_dow AS ds__extract_dow - , subq_4.ds__extract_doy AS ds__extract_doy - , subq_4.created_at__day AS created_at__day - , subq_4.created_at__week AS created_at__week - , subq_4.created_at__month AS created_at__month - , subq_4.created_at__quarter AS created_at__quarter - , subq_4.created_at__year AS created_at__year - , subq_4.created_at__extract_year AS created_at__extract_year - , subq_4.created_at__extract_quarter AS created_at__extract_quarter - , subq_4.created_at__extract_month AS created_at__extract_month - , subq_4.created_at__extract_day AS created_at__extract_day - , subq_4.created_at__extract_dow AS created_at__extract_dow - , subq_4.created_at__extract_doy AS created_at__extract_doy - , subq_4.listing__ds__day AS listing__ds__day - , subq_4.listing__ds__week AS listing__ds__week - , subq_4.listing__ds__month AS listing__ds__month - , subq_4.listing__ds__quarter AS listing__ds__quarter - , subq_4.listing__ds__year AS listing__ds__year - , subq_4.listing__ds__extract_year AS listing__ds__extract_year - , subq_4.listing__ds__extract_quarter AS listing__ds__extract_quarter - , subq_4.listing__ds__extract_month AS listing__ds__extract_month - , subq_4.listing__ds__extract_day AS listing__ds__extract_day - , subq_4.listing__ds__extract_dow AS listing__ds__extract_dow - , subq_4.listing__ds__extract_doy AS listing__ds__extract_doy - , subq_4.listing__created_at__day AS listing__created_at__day - , subq_4.listing__created_at__week AS listing__created_at__week - , subq_4.listing__created_at__month AS listing__created_at__month - , subq_4.listing__created_at__quarter AS listing__created_at__quarter - , subq_4.listing__created_at__year AS listing__created_at__year - , subq_4.listing__created_at__extract_year AS listing__created_at__extract_year - , subq_4.listing__created_at__extract_quarter AS listing__created_at__extract_quarter - , subq_4.listing__created_at__extract_month AS listing__created_at__extract_month - , subq_4.listing__created_at__extract_day AS listing__created_at__extract_day - , subq_4.listing__created_at__extract_dow AS listing__created_at__extract_dow - , subq_4.listing__created_at__extract_doy AS listing__created_at__extract_doy - , subq_4.metric_time__day AS metric_time__day - , subq_4.metric_time__week AS metric_time__week - , subq_4.metric_time__month AS metric_time__month - , subq_4.metric_time__quarter AS metric_time__quarter - , subq_4.metric_time__year AS metric_time__year - , subq_4.metric_time__extract_year AS metric_time__extract_year - , subq_4.metric_time__extract_quarter AS metric_time__extract_quarter - , subq_4.metric_time__extract_month AS metric_time__extract_month - , subq_4.metric_time__extract_day AS metric_time__extract_day - , subq_4.metric_time__extract_dow AS metric_time__extract_dow - , subq_4.metric_time__extract_doy AS metric_time__extract_doy + subq_7.home_state AS user__home_state , subq_7.ds__day AS user__ds__day , subq_7.ds__week AS user__ds__week , subq_7.ds__month AS user__ds__month @@ -384,6 +330,61 @@ FROM ( , subq_7.metric_time__extract_day AS user__metric_time__extract_day , subq_7.metric_time__extract_dow AS user__metric_time__extract_dow , subq_7.metric_time__extract_doy AS user__metric_time__extract_doy + , subq_4.ds__day AS ds__day + , subq_4.ds__week AS ds__week + , subq_4.ds__month AS ds__month + , subq_4.ds__quarter AS ds__quarter + , subq_4.ds__year AS ds__year + , subq_4.ds__extract_year AS ds__extract_year + , subq_4.ds__extract_quarter AS ds__extract_quarter + , subq_4.ds__extract_month AS ds__extract_month + , subq_4.ds__extract_day AS ds__extract_day + , subq_4.ds__extract_dow AS ds__extract_dow + , subq_4.ds__extract_doy AS ds__extract_doy + , subq_4.created_at__day AS created_at__day + , subq_4.created_at__week AS created_at__week + , subq_4.created_at__month AS created_at__month + , subq_4.created_at__quarter AS created_at__quarter + , subq_4.created_at__year AS created_at__year + , subq_4.created_at__extract_year AS created_at__extract_year + , subq_4.created_at__extract_quarter AS created_at__extract_quarter + , subq_4.created_at__extract_month AS created_at__extract_month + , subq_4.created_at__extract_day AS created_at__extract_day + , subq_4.created_at__extract_dow AS created_at__extract_dow + , subq_4.created_at__extract_doy AS created_at__extract_doy + , subq_4.listing__ds__day AS listing__ds__day + , subq_4.listing__ds__week AS listing__ds__week + , subq_4.listing__ds__month AS listing__ds__month + , subq_4.listing__ds__quarter AS listing__ds__quarter + , subq_4.listing__ds__year AS listing__ds__year + , subq_4.listing__ds__extract_year AS listing__ds__extract_year + , subq_4.listing__ds__extract_quarter AS listing__ds__extract_quarter + , subq_4.listing__ds__extract_month AS listing__ds__extract_month + , subq_4.listing__ds__extract_day AS listing__ds__extract_day + , subq_4.listing__ds__extract_dow AS listing__ds__extract_dow + , subq_4.listing__ds__extract_doy AS listing__ds__extract_doy + , subq_4.listing__created_at__day AS listing__created_at__day + , subq_4.listing__created_at__week AS listing__created_at__week + , subq_4.listing__created_at__month AS listing__created_at__month + , subq_4.listing__created_at__quarter AS listing__created_at__quarter + , subq_4.listing__created_at__year AS listing__created_at__year + , subq_4.listing__created_at__extract_year AS listing__created_at__extract_year + , subq_4.listing__created_at__extract_quarter AS listing__created_at__extract_quarter + , subq_4.listing__created_at__extract_month AS listing__created_at__extract_month + , subq_4.listing__created_at__extract_day AS listing__created_at__extract_day + , subq_4.listing__created_at__extract_dow AS listing__created_at__extract_dow + , subq_4.listing__created_at__extract_doy AS listing__created_at__extract_doy + , subq_4.metric_time__day AS metric_time__day + , subq_4.metric_time__week AS metric_time__week + , subq_4.metric_time__month AS metric_time__month + , subq_4.metric_time__quarter AS metric_time__quarter + , subq_4.metric_time__year AS metric_time__year + , subq_4.metric_time__extract_year AS metric_time__extract_year + , subq_4.metric_time__extract_quarter AS metric_time__extract_quarter + , subq_4.metric_time__extract_month AS metric_time__extract_month + , subq_4.metric_time__extract_day AS metric_time__extract_day + , subq_4.metric_time__extract_dow AS metric_time__extract_dow + , subq_4.metric_time__extract_doy AS metric_time__extract_doy , subq_4.listing AS listing , subq_4.user AS user , subq_4.listing__user AS listing__user @@ -393,7 +394,6 @@ FROM ( , subq_4.listing__country_latest AS listing__country_latest , subq_4.listing__is_lux_latest AS listing__is_lux_latest , subq_4.listing__capacity_latest AS listing__capacity_latest - , subq_7.home_state AS user__home_state , subq_4.listings AS listings , subq_4.largest_listing AS largest_listing , subq_4.smallest_listing AS smallest_listing diff --git a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/BigQuery/test_distinct_values_query_with_metric_filter__plan0.sql b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/BigQuery/test_distinct_values_query_with_metric_filter__plan0.sql index 7076519bc5..5fca49a30c 100644 --- a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/BigQuery/test_distinct_values_query_with_metric_filter__plan0.sql +++ b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/BigQuery/test_distinct_values_query_with_metric_filter__plan0.sql @@ -11,10 +11,10 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_0.listing AS listing + subq_6.listing__bookings AS listing__bookings + , subq_0.listing AS listing , subq_0.lux_listing AS lux_listing , subq_0.listing__lux_listing AS listing__lux_listing - , subq_6.listing__bookings AS listing__bookings FROM ( -- Read Elements From Semantic Model 'lux_listing_mapping' SELECT diff --git a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/BigQuery/test_distinct_values_query_with_metric_filter__plan0_optimized.sql b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/BigQuery/test_distinct_values_query_with_metric_filter__plan0_optimized.sql index d79ef0aa09..c51da11fab 100644 --- a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/BigQuery/test_distinct_values_query_with_metric_filter__plan0_optimized.sql +++ b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/BigQuery/test_distinct_values_query_with_metric_filter__plan0_optimized.sql @@ -5,8 +5,8 @@ SELECT FROM ( -- Join Standard Outputs SELECT - lux_listing_mapping_src_28000.listing_id AS listing - , subq_15.listing__bookings AS listing__bookings + subq_15.listing__bookings AS listing__bookings + , lux_listing_mapping_src_28000.listing_id AS listing FROM ***************************.dim_lux_listing_id_mapping lux_listing_mapping_src_28000 FULL OUTER JOIN ( -- Aggregate Measures diff --git a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/BigQuery/test_filter_by_metric_in_same_semantic_model_as_queried_metric__plan0.sql b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/BigQuery/test_filter_by_metric_in_same_semantic_model_as_queried_metric__plan0.sql index 6e6328a698..35c3eed167 100644 --- a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/BigQuery/test_filter_by_metric_in_same_semantic_model_as_queried_metric__plan0.sql +++ b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/BigQuery/test_filter_by_metric_in_same_semantic_model_as_queried_metric__plan0.sql @@ -22,8 +22,8 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.guest AS guest - , subq_8.guest__booking_value AS guest__booking_value + subq_8.guest__booking_value AS guest__booking_value + , subq_2.guest AS guest , subq_2.bookers AS bookers FROM ( -- Pass Only Elements: ['bookers', 'guest'] diff --git a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/BigQuery/test_filter_with_conversion_metric__plan0.sql b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/BigQuery/test_filter_with_conversion_metric__plan0.sql index b92430cc03..7e751ecf4f 100644 --- a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/BigQuery/test_filter_with_conversion_metric__plan0.sql +++ b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/BigQuery/test_filter_with_conversion_metric__plan0.sql @@ -22,8 +22,8 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.user AS user - , subq_19.user__visit_buy_conversion_rate AS user__visit_buy_conversion_rate + subq_19.user__visit_buy_conversion_rate AS user__visit_buy_conversion_rate + , subq_2.user AS user , subq_2.listings AS listings FROM ( -- Pass Only Elements: ['listings', 'user'] diff --git a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/BigQuery/test_group_by_has_local_entity_prefix__plan0.sql b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/BigQuery/test_group_by_has_local_entity_prefix__plan0.sql index e57b6e40eb..155981c2ff 100644 --- a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/BigQuery/test_group_by_has_local_entity_prefix__plan0.sql +++ b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/BigQuery/test_group_by_has_local_entity_prefix__plan0.sql @@ -22,9 +22,9 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.user AS user - , subq_13.listing__user AS user__listing__user + subq_13.listing__user AS user__listing__user , subq_13.listing__user__average_booking_value AS user__listing__user__average_booking_value + , subq_2.user AS user , subq_2.listings AS listings FROM ( -- Pass Only Elements: ['listings', 'user'] @@ -187,8 +187,8 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_5.listing AS listing - , subq_8.user AS listing__user + subq_8.user AS listing__user + , subq_5.listing AS listing , subq_5.average_booking_value AS average_booking_value FROM ( -- Pass Only Elements: ['average_booking_value', 'listing'] diff --git a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/BigQuery/test_inner_query_multi_hop__plan0.sql b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/BigQuery/test_inner_query_multi_hop__plan0.sql index 23c478367d..f743c3ea12 100644 --- a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/BigQuery/test_inner_query_multi_hop__plan0.sql +++ b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/BigQuery/test_inner_query_multi_hop__plan0.sql @@ -22,9 +22,9 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.customer_third_hop_id AS customer_third_hop_id - , subq_17.account_id__customer_id__customer_third_hop_id AS customer_third_hop_id__account_id__customer_id__customer_third_hop_id + subq_17.account_id__customer_id__customer_third_hop_id AS customer_third_hop_id__account_id__customer_id__customer_third_hop_id , subq_17.account_id__customer_id__customer_third_hop_id__txn_count AS customer_third_hop_id__account_id__customer_id__customer_third_hop_id__txn_count + , subq_2.customer_third_hop_id AS customer_third_hop_id , subq_2.third_hop_count AS third_hop_count FROM ( -- Pass Only Elements: ['third_hop_count', 'customer_third_hop_id'] @@ -127,10 +127,10 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_5.ds_partitioned__day AS ds_partitioned__day - , subq_12.ds_partitioned__day AS account_id__ds_partitioned__day - , subq_5.account_id AS account_id + subq_12.ds_partitioned__day AS account_id__ds_partitioned__day , subq_12.customer_id__customer_third_hop_id AS account_id__customer_id__customer_third_hop_id + , subq_5.ds_partitioned__day AS ds_partitioned__day + , subq_5.account_id AS account_id , subq_5.txn_count AS txn_count FROM ( -- Pass Only Elements: ['txn_count', 'ds_partitioned__day', 'account_id'] @@ -264,7 +264,44 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_7.ds_partitioned__day AS ds_partitioned__day + subq_10.country AS customer_id__country + , subq_10.customer_third_hop_id__country AS customer_id__customer_third_hop_id__country + , subq_10.acquired_ds__day AS customer_id__acquired_ds__day + , subq_10.acquired_ds__week AS customer_id__acquired_ds__week + , subq_10.acquired_ds__month AS customer_id__acquired_ds__month + , subq_10.acquired_ds__quarter AS customer_id__acquired_ds__quarter + , subq_10.acquired_ds__year AS customer_id__acquired_ds__year + , subq_10.acquired_ds__extract_year AS customer_id__acquired_ds__extract_year + , subq_10.acquired_ds__extract_quarter AS customer_id__acquired_ds__extract_quarter + , subq_10.acquired_ds__extract_month AS customer_id__acquired_ds__extract_month + , subq_10.acquired_ds__extract_day AS customer_id__acquired_ds__extract_day + , subq_10.acquired_ds__extract_dow AS customer_id__acquired_ds__extract_dow + , subq_10.acquired_ds__extract_doy AS customer_id__acquired_ds__extract_doy + , subq_10.customer_third_hop_id__acquired_ds__day AS customer_id__customer_third_hop_id__acquired_ds__day + , subq_10.customer_third_hop_id__acquired_ds__week AS customer_id__customer_third_hop_id__acquired_ds__week + , subq_10.customer_third_hop_id__acquired_ds__month AS customer_id__customer_third_hop_id__acquired_ds__month + , subq_10.customer_third_hop_id__acquired_ds__quarter AS customer_id__customer_third_hop_id__acquired_ds__quarter + , subq_10.customer_third_hop_id__acquired_ds__year AS customer_id__customer_third_hop_id__acquired_ds__year + , subq_10.customer_third_hop_id__acquired_ds__extract_year AS customer_id__customer_third_hop_id__acquired_ds__extract_year + , subq_10.customer_third_hop_id__acquired_ds__extract_quarter AS customer_id__customer_third_hop_id__acquired_ds__extract_quarter + , subq_10.customer_third_hop_id__acquired_ds__extract_month AS customer_id__customer_third_hop_id__acquired_ds__extract_month + , subq_10.customer_third_hop_id__acquired_ds__extract_day AS customer_id__customer_third_hop_id__acquired_ds__extract_day + , subq_10.customer_third_hop_id__acquired_ds__extract_dow AS customer_id__customer_third_hop_id__acquired_ds__extract_dow + , subq_10.customer_third_hop_id__acquired_ds__extract_doy AS customer_id__customer_third_hop_id__acquired_ds__extract_doy + , subq_10.metric_time__day AS customer_id__metric_time__day + , subq_10.metric_time__week AS customer_id__metric_time__week + , subq_10.metric_time__month AS customer_id__metric_time__month + , subq_10.metric_time__quarter AS customer_id__metric_time__quarter + , subq_10.metric_time__year AS customer_id__metric_time__year + , subq_10.metric_time__extract_year AS customer_id__metric_time__extract_year + , subq_10.metric_time__extract_quarter AS customer_id__metric_time__extract_quarter + , subq_10.metric_time__extract_month AS customer_id__metric_time__extract_month + , subq_10.metric_time__extract_day AS customer_id__metric_time__extract_day + , subq_10.metric_time__extract_dow AS customer_id__metric_time__extract_dow + , subq_10.metric_time__extract_doy AS customer_id__metric_time__extract_doy + , subq_10.customer_third_hop_id AS customer_id__customer_third_hop_id + , subq_10.customer_third_hop_id__customer_id AS customer_id__customer_third_hop_id__customer_id + , subq_7.ds_partitioned__day AS ds_partitioned__day , subq_7.ds_partitioned__week AS ds_partitioned__week , subq_7.ds_partitioned__month AS ds_partitioned__month , subq_7.ds_partitioned__quarter AS ds_partitioned__quarter @@ -308,51 +345,14 @@ FROM ( , subq_7.metric_time__extract_day AS metric_time__extract_day , subq_7.metric_time__extract_dow AS metric_time__extract_dow , subq_7.metric_time__extract_doy AS metric_time__extract_doy - , subq_10.acquired_ds__day AS customer_id__acquired_ds__day - , subq_10.acquired_ds__week AS customer_id__acquired_ds__week - , subq_10.acquired_ds__month AS customer_id__acquired_ds__month - , subq_10.acquired_ds__quarter AS customer_id__acquired_ds__quarter - , subq_10.acquired_ds__year AS customer_id__acquired_ds__year - , subq_10.acquired_ds__extract_year AS customer_id__acquired_ds__extract_year - , subq_10.acquired_ds__extract_quarter AS customer_id__acquired_ds__extract_quarter - , subq_10.acquired_ds__extract_month AS customer_id__acquired_ds__extract_month - , subq_10.acquired_ds__extract_day AS customer_id__acquired_ds__extract_day - , subq_10.acquired_ds__extract_dow AS customer_id__acquired_ds__extract_dow - , subq_10.acquired_ds__extract_doy AS customer_id__acquired_ds__extract_doy - , subq_10.customer_third_hop_id__acquired_ds__day AS customer_id__customer_third_hop_id__acquired_ds__day - , subq_10.customer_third_hop_id__acquired_ds__week AS customer_id__customer_third_hop_id__acquired_ds__week - , subq_10.customer_third_hop_id__acquired_ds__month AS customer_id__customer_third_hop_id__acquired_ds__month - , subq_10.customer_third_hop_id__acquired_ds__quarter AS customer_id__customer_third_hop_id__acquired_ds__quarter - , subq_10.customer_third_hop_id__acquired_ds__year AS customer_id__customer_third_hop_id__acquired_ds__year - , subq_10.customer_third_hop_id__acquired_ds__extract_year AS customer_id__customer_third_hop_id__acquired_ds__extract_year - , subq_10.customer_third_hop_id__acquired_ds__extract_quarter AS customer_id__customer_third_hop_id__acquired_ds__extract_quarter - , subq_10.customer_third_hop_id__acquired_ds__extract_month AS customer_id__customer_third_hop_id__acquired_ds__extract_month - , subq_10.customer_third_hop_id__acquired_ds__extract_day AS customer_id__customer_third_hop_id__acquired_ds__extract_day - , subq_10.customer_third_hop_id__acquired_ds__extract_dow AS customer_id__customer_third_hop_id__acquired_ds__extract_dow - , subq_10.customer_third_hop_id__acquired_ds__extract_doy AS customer_id__customer_third_hop_id__acquired_ds__extract_doy - , subq_10.metric_time__day AS customer_id__metric_time__day - , subq_10.metric_time__week AS customer_id__metric_time__week - , subq_10.metric_time__month AS customer_id__metric_time__month - , subq_10.metric_time__quarter AS customer_id__metric_time__quarter - , subq_10.metric_time__year AS customer_id__metric_time__year - , subq_10.metric_time__extract_year AS customer_id__metric_time__extract_year - , subq_10.metric_time__extract_quarter AS customer_id__metric_time__extract_quarter - , subq_10.metric_time__extract_month AS customer_id__metric_time__extract_month - , subq_10.metric_time__extract_day AS customer_id__metric_time__extract_day - , subq_10.metric_time__extract_dow AS customer_id__metric_time__extract_dow - , subq_10.metric_time__extract_doy AS customer_id__metric_time__extract_doy , subq_7.account_id AS account_id , subq_7.customer_id AS customer_id , subq_7.account_id__customer_id AS account_id__customer_id , subq_7.bridge_account__account_id AS bridge_account__account_id , subq_7.bridge_account__customer_id AS bridge_account__customer_id - , subq_10.customer_third_hop_id AS customer_id__customer_third_hop_id - , subq_10.customer_third_hop_id__customer_id AS customer_id__customer_third_hop_id__customer_id , subq_7.extra_dim AS extra_dim , subq_7.account_id__extra_dim AS account_id__extra_dim , subq_7.bridge_account__extra_dim AS bridge_account__extra_dim - , subq_10.country AS customer_id__country - , subq_10.customer_third_hop_id__country AS customer_id__customer_third_hop_id__country , subq_7.account_customer_combos AS account_customer_combos FROM ( -- Metric Time Dimension 'ds_partitioned' diff --git a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/BigQuery/test_inner_query_single_hop__plan0.sql b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/BigQuery/test_inner_query_single_hop__plan0.sql index 772b715853..f9bd59a56c 100644 --- a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/BigQuery/test_inner_query_single_hop__plan0.sql +++ b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/BigQuery/test_inner_query_single_hop__plan0.sql @@ -22,9 +22,9 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.customer_third_hop_id AS customer_third_hop_id - , subq_10.customer_id__customer_third_hop_id AS customer_third_hop_id__customer_id__customer_third_hop_id + subq_10.customer_id__customer_third_hop_id AS customer_third_hop_id__customer_id__customer_third_hop_id , subq_10.customer_id__customer_third_hop_id__paraguayan_customers AS customer_third_hop_id__customer_id__customer_third_hop_id__paraguayan_customers + , subq_2.customer_third_hop_id AS customer_third_hop_id , subq_2.third_hop_count AS third_hop_count FROM ( -- Pass Only Elements: ['third_hop_count', 'customer_third_hop_id'] diff --git a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/BigQuery/test_metric_filtered_by_itself__plan0.sql b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/BigQuery/test_metric_filtered_by_itself__plan0.sql index 81433f10c2..ae7d821d26 100644 --- a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/BigQuery/test_metric_filtered_by_itself__plan0.sql +++ b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/BigQuery/test_metric_filtered_by_itself__plan0.sql @@ -22,8 +22,8 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.listing AS listing - , subq_8.listing__bookers AS listing__bookers + subq_8.listing__bookers AS listing__bookers + , subq_2.listing AS listing , subq_2.bookers AS bookers FROM ( -- Pass Only Elements: ['bookers', 'listing'] diff --git a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/BigQuery/test_metric_with_metric_in_where_filter__plan0.sql b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/BigQuery/test_metric_with_metric_in_where_filter__plan0.sql index 2341f5bd11..34fe1f900f 100644 --- a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/BigQuery/test_metric_with_metric_in_where_filter__plan0.sql +++ b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/BigQuery/test_metric_with_metric_in_where_filter__plan0.sql @@ -27,9 +27,9 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.metric_time__day AS metric_time__day + subq_8.listing__bookings AS listing__bookings + , subq_2.metric_time__day AS metric_time__day , subq_2.listing AS listing - , subq_8.listing__bookings AS listing__bookings , subq_2.listings AS listings FROM ( -- Pass Only Elements: ['listings', 'metric_time__day', 'listing'] diff --git a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/BigQuery/test_query_with_cumulative_metric_in_where_filter__plan0.sql b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/BigQuery/test_query_with_cumulative_metric_in_where_filter__plan0.sql index 5794f57594..bceb3143d0 100644 --- a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/BigQuery/test_query_with_cumulative_metric_in_where_filter__plan0.sql +++ b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/BigQuery/test_query_with_cumulative_metric_in_where_filter__plan0.sql @@ -22,8 +22,8 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.user AS user - , subq_8.user__revenue_all_time AS user__revenue_all_time + subq_8.user__revenue_all_time AS user__revenue_all_time + , subq_2.user AS user , subq_2.listings AS listings FROM ( -- Pass Only Elements: ['listings', 'user'] diff --git a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/BigQuery/test_query_with_derived_metric_in_where_filter__plan0.sql b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/BigQuery/test_query_with_derived_metric_in_where_filter__plan0.sql index 92c1f08572..049db657ff 100644 --- a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/BigQuery/test_query_with_derived_metric_in_where_filter__plan0.sql +++ b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/BigQuery/test_query_with_derived_metric_in_where_filter__plan0.sql @@ -22,8 +22,8 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.listing AS listing - , subq_15.listing__views_times_booking_value AS listing__views_times_booking_value + subq_15.listing__views_times_booking_value AS listing__views_times_booking_value + , subq_2.listing AS listing , subq_2.listings AS listings FROM ( -- Pass Only Elements: ['listings', 'listing'] diff --git a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/BigQuery/test_query_with_multiple_metrics_in_filter__plan0.sql b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/BigQuery/test_query_with_multiple_metrics_in_filter__plan0.sql index f48b93b471..3f2383b230 100644 --- a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/BigQuery/test_query_with_multiple_metrics_in_filter__plan0.sql +++ b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/BigQuery/test_query_with_multiple_metrics_in_filter__plan0.sql @@ -24,9 +24,9 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.listing AS listing - , subq_8.listing__bookings AS listing__bookings + subq_8.listing__bookings AS listing__bookings , subq_14.listing__bookers AS listing__bookers + , subq_2.listing AS listing , subq_2.listings AS listings FROM ( -- Pass Only Elements: ['listings', 'listing', 'listing'] diff --git a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/BigQuery/test_query_with_ratio_metric_in_where_filter__plan0.sql b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/BigQuery/test_query_with_ratio_metric_in_where_filter__plan0.sql index 01371f26dc..712702655f 100644 --- a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/BigQuery/test_query_with_ratio_metric_in_where_filter__plan0.sql +++ b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/BigQuery/test_query_with_ratio_metric_in_where_filter__plan0.sql @@ -22,8 +22,8 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.listing AS listing - , subq_15.listing__bookings_per_booker AS listing__bookings_per_booker + subq_15.listing__bookings_per_booker AS listing__bookings_per_booker + , subq_2.listing AS listing , subq_2.listings AS listings FROM ( -- Pass Only Elements: ['listings', 'listing'] diff --git a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/BigQuery/test_query_with_simple_metric_in_where_filter__plan0.sql b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/BigQuery/test_query_with_simple_metric_in_where_filter__plan0.sql index 6f8291f37d..b81eb9a775 100644 --- a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/BigQuery/test_query_with_simple_metric_in_where_filter__plan0.sql +++ b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/BigQuery/test_query_with_simple_metric_in_where_filter__plan0.sql @@ -22,8 +22,8 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.listing AS listing - , subq_8.listing__bookings AS listing__bookings + subq_8.listing__bookings AS listing__bookings + , subq_2.listing AS listing , subq_2.listings AS listings FROM ( -- Pass Only Elements: ['listings', 'listing'] diff --git a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Databricks/test_distinct_values_query_with_metric_filter__plan0.sql b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Databricks/test_distinct_values_query_with_metric_filter__plan0.sql index bbde80db86..6efb1cdee0 100644 --- a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Databricks/test_distinct_values_query_with_metric_filter__plan0.sql +++ b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Databricks/test_distinct_values_query_with_metric_filter__plan0.sql @@ -11,10 +11,10 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_0.listing AS listing + subq_6.listing__bookings AS listing__bookings + , subq_0.listing AS listing , subq_0.lux_listing AS lux_listing , subq_0.listing__lux_listing AS listing__lux_listing - , subq_6.listing__bookings AS listing__bookings FROM ( -- Read Elements From Semantic Model 'lux_listing_mapping' SELECT diff --git a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Databricks/test_distinct_values_query_with_metric_filter__plan0_optimized.sql b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Databricks/test_distinct_values_query_with_metric_filter__plan0_optimized.sql index d79ef0aa09..c51da11fab 100644 --- a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Databricks/test_distinct_values_query_with_metric_filter__plan0_optimized.sql +++ b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Databricks/test_distinct_values_query_with_metric_filter__plan0_optimized.sql @@ -5,8 +5,8 @@ SELECT FROM ( -- Join Standard Outputs SELECT - lux_listing_mapping_src_28000.listing_id AS listing - , subq_15.listing__bookings AS listing__bookings + subq_15.listing__bookings AS listing__bookings + , lux_listing_mapping_src_28000.listing_id AS listing FROM ***************************.dim_lux_listing_id_mapping lux_listing_mapping_src_28000 FULL OUTER JOIN ( -- Aggregate Measures diff --git a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Databricks/test_filter_by_metric_in_same_semantic_model_as_queried_metric__plan0.sql b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Databricks/test_filter_by_metric_in_same_semantic_model_as_queried_metric__plan0.sql index c3c28041e7..52c7168b62 100644 --- a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Databricks/test_filter_by_metric_in_same_semantic_model_as_queried_metric__plan0.sql +++ b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Databricks/test_filter_by_metric_in_same_semantic_model_as_queried_metric__plan0.sql @@ -22,8 +22,8 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.guest AS guest - , subq_8.guest__booking_value AS guest__booking_value + subq_8.guest__booking_value AS guest__booking_value + , subq_2.guest AS guest , subq_2.bookers AS bookers FROM ( -- Pass Only Elements: ['bookers', 'guest'] diff --git a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Databricks/test_filter_with_conversion_metric__plan0.sql b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Databricks/test_filter_with_conversion_metric__plan0.sql index 595fe04f38..064154fa99 100644 --- a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Databricks/test_filter_with_conversion_metric__plan0.sql +++ b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Databricks/test_filter_with_conversion_metric__plan0.sql @@ -22,8 +22,8 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.user AS user - , subq_19.user__visit_buy_conversion_rate AS user__visit_buy_conversion_rate + subq_19.user__visit_buy_conversion_rate AS user__visit_buy_conversion_rate + , subq_2.user AS user , subq_2.listings AS listings FROM ( -- Pass Only Elements: ['listings', 'user'] diff --git a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Databricks/test_group_by_has_local_entity_prefix__plan0.sql b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Databricks/test_group_by_has_local_entity_prefix__plan0.sql index 331490f28b..923c6ef78b 100644 --- a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Databricks/test_group_by_has_local_entity_prefix__plan0.sql +++ b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Databricks/test_group_by_has_local_entity_prefix__plan0.sql @@ -22,9 +22,9 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.user AS user - , subq_13.listing__user AS user__listing__user + subq_13.listing__user AS user__listing__user , subq_13.listing__user__average_booking_value AS user__listing__user__average_booking_value + , subq_2.user AS user , subq_2.listings AS listings FROM ( -- Pass Only Elements: ['listings', 'user'] @@ -187,8 +187,8 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_5.listing AS listing - , subq_8.user AS listing__user + subq_8.user AS listing__user + , subq_5.listing AS listing , subq_5.average_booking_value AS average_booking_value FROM ( -- Pass Only Elements: ['average_booking_value', 'listing'] diff --git a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Databricks/test_inner_query_multi_hop__plan0.sql b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Databricks/test_inner_query_multi_hop__plan0.sql index 0c8a0cdd59..43f4771e88 100644 --- a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Databricks/test_inner_query_multi_hop__plan0.sql +++ b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Databricks/test_inner_query_multi_hop__plan0.sql @@ -22,9 +22,9 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.customer_third_hop_id AS customer_third_hop_id - , subq_17.account_id__customer_id__customer_third_hop_id AS customer_third_hop_id__account_id__customer_id__customer_third_hop_id + subq_17.account_id__customer_id__customer_third_hop_id AS customer_third_hop_id__account_id__customer_id__customer_third_hop_id , subq_17.account_id__customer_id__customer_third_hop_id__txn_count AS customer_third_hop_id__account_id__customer_id__customer_third_hop_id__txn_count + , subq_2.customer_third_hop_id AS customer_third_hop_id , subq_2.third_hop_count AS third_hop_count FROM ( -- Pass Only Elements: ['third_hop_count', 'customer_third_hop_id'] @@ -127,10 +127,10 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_5.ds_partitioned__day AS ds_partitioned__day - , subq_12.ds_partitioned__day AS account_id__ds_partitioned__day - , subq_5.account_id AS account_id + subq_12.ds_partitioned__day AS account_id__ds_partitioned__day , subq_12.customer_id__customer_third_hop_id AS account_id__customer_id__customer_third_hop_id + , subq_5.ds_partitioned__day AS ds_partitioned__day + , subq_5.account_id AS account_id , subq_5.txn_count AS txn_count FROM ( -- Pass Only Elements: ['txn_count', 'ds_partitioned__day', 'account_id'] @@ -264,7 +264,44 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_7.ds_partitioned__day AS ds_partitioned__day + subq_10.country AS customer_id__country + , subq_10.customer_third_hop_id__country AS customer_id__customer_third_hop_id__country + , subq_10.acquired_ds__day AS customer_id__acquired_ds__day + , subq_10.acquired_ds__week AS customer_id__acquired_ds__week + , subq_10.acquired_ds__month AS customer_id__acquired_ds__month + , subq_10.acquired_ds__quarter AS customer_id__acquired_ds__quarter + , subq_10.acquired_ds__year AS customer_id__acquired_ds__year + , subq_10.acquired_ds__extract_year AS customer_id__acquired_ds__extract_year + , subq_10.acquired_ds__extract_quarter AS customer_id__acquired_ds__extract_quarter + , subq_10.acquired_ds__extract_month AS customer_id__acquired_ds__extract_month + , subq_10.acquired_ds__extract_day AS customer_id__acquired_ds__extract_day + , subq_10.acquired_ds__extract_dow AS customer_id__acquired_ds__extract_dow + , subq_10.acquired_ds__extract_doy AS customer_id__acquired_ds__extract_doy + , subq_10.customer_third_hop_id__acquired_ds__day AS customer_id__customer_third_hop_id__acquired_ds__day + , subq_10.customer_third_hop_id__acquired_ds__week AS customer_id__customer_third_hop_id__acquired_ds__week + , subq_10.customer_third_hop_id__acquired_ds__month AS customer_id__customer_third_hop_id__acquired_ds__month + , subq_10.customer_third_hop_id__acquired_ds__quarter AS customer_id__customer_third_hop_id__acquired_ds__quarter + , subq_10.customer_third_hop_id__acquired_ds__year AS customer_id__customer_third_hop_id__acquired_ds__year + , subq_10.customer_third_hop_id__acquired_ds__extract_year AS customer_id__customer_third_hop_id__acquired_ds__extract_year + , subq_10.customer_third_hop_id__acquired_ds__extract_quarter AS customer_id__customer_third_hop_id__acquired_ds__extract_quarter + , subq_10.customer_third_hop_id__acquired_ds__extract_month AS customer_id__customer_third_hop_id__acquired_ds__extract_month + , subq_10.customer_third_hop_id__acquired_ds__extract_day AS customer_id__customer_third_hop_id__acquired_ds__extract_day + , subq_10.customer_third_hop_id__acquired_ds__extract_dow AS customer_id__customer_third_hop_id__acquired_ds__extract_dow + , subq_10.customer_third_hop_id__acquired_ds__extract_doy AS customer_id__customer_third_hop_id__acquired_ds__extract_doy + , subq_10.metric_time__day AS customer_id__metric_time__day + , subq_10.metric_time__week AS customer_id__metric_time__week + , subq_10.metric_time__month AS customer_id__metric_time__month + , subq_10.metric_time__quarter AS customer_id__metric_time__quarter + , subq_10.metric_time__year AS customer_id__metric_time__year + , subq_10.metric_time__extract_year AS customer_id__metric_time__extract_year + , subq_10.metric_time__extract_quarter AS customer_id__metric_time__extract_quarter + , subq_10.metric_time__extract_month AS customer_id__metric_time__extract_month + , subq_10.metric_time__extract_day AS customer_id__metric_time__extract_day + , subq_10.metric_time__extract_dow AS customer_id__metric_time__extract_dow + , subq_10.metric_time__extract_doy AS customer_id__metric_time__extract_doy + , subq_10.customer_third_hop_id AS customer_id__customer_third_hop_id + , subq_10.customer_third_hop_id__customer_id AS customer_id__customer_third_hop_id__customer_id + , subq_7.ds_partitioned__day AS ds_partitioned__day , subq_7.ds_partitioned__week AS ds_partitioned__week , subq_7.ds_partitioned__month AS ds_partitioned__month , subq_7.ds_partitioned__quarter AS ds_partitioned__quarter @@ -308,51 +345,14 @@ FROM ( , subq_7.metric_time__extract_day AS metric_time__extract_day , subq_7.metric_time__extract_dow AS metric_time__extract_dow , subq_7.metric_time__extract_doy AS metric_time__extract_doy - , subq_10.acquired_ds__day AS customer_id__acquired_ds__day - , subq_10.acquired_ds__week AS customer_id__acquired_ds__week - , subq_10.acquired_ds__month AS customer_id__acquired_ds__month - , subq_10.acquired_ds__quarter AS customer_id__acquired_ds__quarter - , subq_10.acquired_ds__year AS customer_id__acquired_ds__year - , subq_10.acquired_ds__extract_year AS customer_id__acquired_ds__extract_year - , subq_10.acquired_ds__extract_quarter AS customer_id__acquired_ds__extract_quarter - , subq_10.acquired_ds__extract_month AS customer_id__acquired_ds__extract_month - , subq_10.acquired_ds__extract_day AS customer_id__acquired_ds__extract_day - , subq_10.acquired_ds__extract_dow AS customer_id__acquired_ds__extract_dow - , subq_10.acquired_ds__extract_doy AS customer_id__acquired_ds__extract_doy - , subq_10.customer_third_hop_id__acquired_ds__day AS customer_id__customer_third_hop_id__acquired_ds__day - , subq_10.customer_third_hop_id__acquired_ds__week AS customer_id__customer_third_hop_id__acquired_ds__week - , subq_10.customer_third_hop_id__acquired_ds__month AS customer_id__customer_third_hop_id__acquired_ds__month - , subq_10.customer_third_hop_id__acquired_ds__quarter AS customer_id__customer_third_hop_id__acquired_ds__quarter - , subq_10.customer_third_hop_id__acquired_ds__year AS customer_id__customer_third_hop_id__acquired_ds__year - , subq_10.customer_third_hop_id__acquired_ds__extract_year AS customer_id__customer_third_hop_id__acquired_ds__extract_year - , subq_10.customer_third_hop_id__acquired_ds__extract_quarter AS customer_id__customer_third_hop_id__acquired_ds__extract_quarter - , subq_10.customer_third_hop_id__acquired_ds__extract_month AS customer_id__customer_third_hop_id__acquired_ds__extract_month - , subq_10.customer_third_hop_id__acquired_ds__extract_day AS customer_id__customer_third_hop_id__acquired_ds__extract_day - , subq_10.customer_third_hop_id__acquired_ds__extract_dow AS customer_id__customer_third_hop_id__acquired_ds__extract_dow - , subq_10.customer_third_hop_id__acquired_ds__extract_doy AS customer_id__customer_third_hop_id__acquired_ds__extract_doy - , subq_10.metric_time__day AS customer_id__metric_time__day - , subq_10.metric_time__week AS customer_id__metric_time__week - , subq_10.metric_time__month AS customer_id__metric_time__month - , subq_10.metric_time__quarter AS customer_id__metric_time__quarter - , subq_10.metric_time__year AS customer_id__metric_time__year - , subq_10.metric_time__extract_year AS customer_id__metric_time__extract_year - , subq_10.metric_time__extract_quarter AS customer_id__metric_time__extract_quarter - , subq_10.metric_time__extract_month AS customer_id__metric_time__extract_month - , subq_10.metric_time__extract_day AS customer_id__metric_time__extract_day - , subq_10.metric_time__extract_dow AS customer_id__metric_time__extract_dow - , subq_10.metric_time__extract_doy AS customer_id__metric_time__extract_doy , subq_7.account_id AS account_id , subq_7.customer_id AS customer_id , subq_7.account_id__customer_id AS account_id__customer_id , subq_7.bridge_account__account_id AS bridge_account__account_id , subq_7.bridge_account__customer_id AS bridge_account__customer_id - , subq_10.customer_third_hop_id AS customer_id__customer_third_hop_id - , subq_10.customer_third_hop_id__customer_id AS customer_id__customer_third_hop_id__customer_id , subq_7.extra_dim AS extra_dim , subq_7.account_id__extra_dim AS account_id__extra_dim , subq_7.bridge_account__extra_dim AS bridge_account__extra_dim - , subq_10.country AS customer_id__country - , subq_10.customer_third_hop_id__country AS customer_id__customer_third_hop_id__country , subq_7.account_customer_combos AS account_customer_combos FROM ( -- Metric Time Dimension 'ds_partitioned' diff --git a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Databricks/test_inner_query_single_hop__plan0.sql b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Databricks/test_inner_query_single_hop__plan0.sql index 52ccb85d6f..84b93f01f6 100644 --- a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Databricks/test_inner_query_single_hop__plan0.sql +++ b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Databricks/test_inner_query_single_hop__plan0.sql @@ -22,9 +22,9 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.customer_third_hop_id AS customer_third_hop_id - , subq_10.customer_id__customer_third_hop_id AS customer_third_hop_id__customer_id__customer_third_hop_id + subq_10.customer_id__customer_third_hop_id AS customer_third_hop_id__customer_id__customer_third_hop_id , subq_10.customer_id__customer_third_hop_id__paraguayan_customers AS customer_third_hop_id__customer_id__customer_third_hop_id__paraguayan_customers + , subq_2.customer_third_hop_id AS customer_third_hop_id , subq_2.third_hop_count AS third_hop_count FROM ( -- Pass Only Elements: ['third_hop_count', 'customer_third_hop_id'] diff --git a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Databricks/test_metric_filtered_by_itself__plan0.sql b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Databricks/test_metric_filtered_by_itself__plan0.sql index b787909bd7..690c3af2fa 100644 --- a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Databricks/test_metric_filtered_by_itself__plan0.sql +++ b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Databricks/test_metric_filtered_by_itself__plan0.sql @@ -22,8 +22,8 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.listing AS listing - , subq_8.listing__bookers AS listing__bookers + subq_8.listing__bookers AS listing__bookers + , subq_2.listing AS listing , subq_2.bookers AS bookers FROM ( -- Pass Only Elements: ['bookers', 'listing'] diff --git a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Databricks/test_metric_with_metric_in_where_filter__plan0.sql b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Databricks/test_metric_with_metric_in_where_filter__plan0.sql index e66cb5862b..91e5941c55 100644 --- a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Databricks/test_metric_with_metric_in_where_filter__plan0.sql +++ b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Databricks/test_metric_with_metric_in_where_filter__plan0.sql @@ -27,9 +27,9 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.metric_time__day AS metric_time__day + subq_8.listing__bookings AS listing__bookings + , subq_2.metric_time__day AS metric_time__day , subq_2.listing AS listing - , subq_8.listing__bookings AS listing__bookings , subq_2.listings AS listings FROM ( -- Pass Only Elements: ['listings', 'metric_time__day', 'listing'] diff --git a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Databricks/test_query_with_cumulative_metric_in_where_filter__plan0.sql b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Databricks/test_query_with_cumulative_metric_in_where_filter__plan0.sql index 7286c555a4..740aeef1ab 100644 --- a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Databricks/test_query_with_cumulative_metric_in_where_filter__plan0.sql +++ b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Databricks/test_query_with_cumulative_metric_in_where_filter__plan0.sql @@ -22,8 +22,8 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.user AS user - , subq_8.user__revenue_all_time AS user__revenue_all_time + subq_8.user__revenue_all_time AS user__revenue_all_time + , subq_2.user AS user , subq_2.listings AS listings FROM ( -- Pass Only Elements: ['listings', 'user'] diff --git a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Databricks/test_query_with_derived_metric_in_where_filter__plan0.sql b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Databricks/test_query_with_derived_metric_in_where_filter__plan0.sql index 35cc6163cc..b44efcd0dc 100644 --- a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Databricks/test_query_with_derived_metric_in_where_filter__plan0.sql +++ b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Databricks/test_query_with_derived_metric_in_where_filter__plan0.sql @@ -22,8 +22,8 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.listing AS listing - , subq_15.listing__views_times_booking_value AS listing__views_times_booking_value + subq_15.listing__views_times_booking_value AS listing__views_times_booking_value + , subq_2.listing AS listing , subq_2.listings AS listings FROM ( -- Pass Only Elements: ['listings', 'listing'] diff --git a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Databricks/test_query_with_multiple_metrics_in_filter__plan0.sql b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Databricks/test_query_with_multiple_metrics_in_filter__plan0.sql index eeae43d647..b5c9b3dc42 100644 --- a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Databricks/test_query_with_multiple_metrics_in_filter__plan0.sql +++ b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Databricks/test_query_with_multiple_metrics_in_filter__plan0.sql @@ -24,9 +24,9 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.listing AS listing - , subq_8.listing__bookings AS listing__bookings + subq_8.listing__bookings AS listing__bookings , subq_14.listing__bookers AS listing__bookers + , subq_2.listing AS listing , subq_2.listings AS listings FROM ( -- Pass Only Elements: ['listings', 'listing', 'listing'] diff --git a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Databricks/test_query_with_ratio_metric_in_where_filter__plan0.sql b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Databricks/test_query_with_ratio_metric_in_where_filter__plan0.sql index 31096cf73a..a6bebe31f3 100644 --- a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Databricks/test_query_with_ratio_metric_in_where_filter__plan0.sql +++ b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Databricks/test_query_with_ratio_metric_in_where_filter__plan0.sql @@ -22,8 +22,8 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.listing AS listing - , subq_15.listing__bookings_per_booker AS listing__bookings_per_booker + subq_15.listing__bookings_per_booker AS listing__bookings_per_booker + , subq_2.listing AS listing , subq_2.listings AS listings FROM ( -- Pass Only Elements: ['listings', 'listing'] diff --git a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Databricks/test_query_with_simple_metric_in_where_filter__plan0.sql b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Databricks/test_query_with_simple_metric_in_where_filter__plan0.sql index d3b41f9913..3c41302368 100644 --- a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Databricks/test_query_with_simple_metric_in_where_filter__plan0.sql +++ b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Databricks/test_query_with_simple_metric_in_where_filter__plan0.sql @@ -22,8 +22,8 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.listing AS listing - , subq_8.listing__bookings AS listing__bookings + subq_8.listing__bookings AS listing__bookings + , subq_2.listing AS listing , subq_2.listings AS listings FROM ( -- Pass Only Elements: ['listings', 'listing'] diff --git a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/DuckDB/test_distinct_values_query_with_metric_filter__plan0.sql b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/DuckDB/test_distinct_values_query_with_metric_filter__plan0.sql index 799e437abe..d89cbe19cb 100644 --- a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/DuckDB/test_distinct_values_query_with_metric_filter__plan0.sql +++ b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/DuckDB/test_distinct_values_query_with_metric_filter__plan0.sql @@ -11,10 +11,10 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_0.listing AS listing + subq_6.listing__bookings AS listing__bookings + , subq_0.listing AS listing , subq_0.lux_listing AS lux_listing , subq_0.listing__lux_listing AS listing__lux_listing - , subq_6.listing__bookings AS listing__bookings FROM ( -- Read Elements From Semantic Model 'lux_listing_mapping' SELECT diff --git a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/DuckDB/test_distinct_values_query_with_metric_filter__plan0_optimized.sql b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/DuckDB/test_distinct_values_query_with_metric_filter__plan0_optimized.sql index d79ef0aa09..c51da11fab 100644 --- a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/DuckDB/test_distinct_values_query_with_metric_filter__plan0_optimized.sql +++ b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/DuckDB/test_distinct_values_query_with_metric_filter__plan0_optimized.sql @@ -5,8 +5,8 @@ SELECT FROM ( -- Join Standard Outputs SELECT - lux_listing_mapping_src_28000.listing_id AS listing - , subq_15.listing__bookings AS listing__bookings + subq_15.listing__bookings AS listing__bookings + , lux_listing_mapping_src_28000.listing_id AS listing FROM ***************************.dim_lux_listing_id_mapping lux_listing_mapping_src_28000 FULL OUTER JOIN ( -- Aggregate Measures diff --git a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/DuckDB/test_filter_by_metric_in_same_semantic_model_as_queried_metric__plan0.sql b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/DuckDB/test_filter_by_metric_in_same_semantic_model_as_queried_metric__plan0.sql index e5dc490b67..a0a48965f4 100644 --- a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/DuckDB/test_filter_by_metric_in_same_semantic_model_as_queried_metric__plan0.sql +++ b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/DuckDB/test_filter_by_metric_in_same_semantic_model_as_queried_metric__plan0.sql @@ -22,8 +22,8 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.guest AS guest - , subq_8.guest__booking_value AS guest__booking_value + subq_8.guest__booking_value AS guest__booking_value + , subq_2.guest AS guest , subq_2.bookers AS bookers FROM ( -- Pass Only Elements: ['bookers', 'guest'] diff --git a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/DuckDB/test_filter_with_conversion_metric__plan0.sql b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/DuckDB/test_filter_with_conversion_metric__plan0.sql index 648d83f1db..88183f840e 100644 --- a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/DuckDB/test_filter_with_conversion_metric__plan0.sql +++ b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/DuckDB/test_filter_with_conversion_metric__plan0.sql @@ -22,8 +22,8 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.user AS user - , subq_19.user__visit_buy_conversion_rate AS user__visit_buy_conversion_rate + subq_19.user__visit_buy_conversion_rate AS user__visit_buy_conversion_rate + , subq_2.user AS user , subq_2.listings AS listings FROM ( -- Pass Only Elements: ['listings', 'user'] diff --git a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/DuckDB/test_group_by_has_local_entity_prefix__plan0.sql b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/DuckDB/test_group_by_has_local_entity_prefix__plan0.sql index 1b6946e2c8..646daaaf6e 100644 --- a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/DuckDB/test_group_by_has_local_entity_prefix__plan0.sql +++ b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/DuckDB/test_group_by_has_local_entity_prefix__plan0.sql @@ -22,9 +22,9 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.user AS user - , subq_13.listing__user AS user__listing__user + subq_13.listing__user AS user__listing__user , subq_13.listing__user__average_booking_value AS user__listing__user__average_booking_value + , subq_2.user AS user , subq_2.listings AS listings FROM ( -- Pass Only Elements: ['listings', 'user'] @@ -187,8 +187,8 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_5.listing AS listing - , subq_8.user AS listing__user + subq_8.user AS listing__user + , subq_5.listing AS listing , subq_5.average_booking_value AS average_booking_value FROM ( -- Pass Only Elements: ['average_booking_value', 'listing'] diff --git a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/DuckDB/test_inner_query_multi_hop__plan0.sql b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/DuckDB/test_inner_query_multi_hop__plan0.sql index 3146c0bc2f..5ce70b6587 100644 --- a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/DuckDB/test_inner_query_multi_hop__plan0.sql +++ b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/DuckDB/test_inner_query_multi_hop__plan0.sql @@ -22,9 +22,9 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.customer_third_hop_id AS customer_third_hop_id - , subq_17.account_id__customer_id__customer_third_hop_id AS customer_third_hop_id__account_id__customer_id__customer_third_hop_id + subq_17.account_id__customer_id__customer_third_hop_id AS customer_third_hop_id__account_id__customer_id__customer_third_hop_id , subq_17.account_id__customer_id__customer_third_hop_id__txn_count AS customer_third_hop_id__account_id__customer_id__customer_third_hop_id__txn_count + , subq_2.customer_third_hop_id AS customer_third_hop_id , subq_2.third_hop_count AS third_hop_count FROM ( -- Pass Only Elements: ['third_hop_count', 'customer_third_hop_id'] @@ -127,10 +127,10 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_5.ds_partitioned__day AS ds_partitioned__day - , subq_12.ds_partitioned__day AS account_id__ds_partitioned__day - , subq_5.account_id AS account_id + subq_12.ds_partitioned__day AS account_id__ds_partitioned__day , subq_12.customer_id__customer_third_hop_id AS account_id__customer_id__customer_third_hop_id + , subq_5.ds_partitioned__day AS ds_partitioned__day + , subq_5.account_id AS account_id , subq_5.txn_count AS txn_count FROM ( -- Pass Only Elements: ['txn_count', 'ds_partitioned__day', 'account_id'] @@ -264,7 +264,44 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_7.ds_partitioned__day AS ds_partitioned__day + subq_10.country AS customer_id__country + , subq_10.customer_third_hop_id__country AS customer_id__customer_third_hop_id__country + , subq_10.acquired_ds__day AS customer_id__acquired_ds__day + , subq_10.acquired_ds__week AS customer_id__acquired_ds__week + , subq_10.acquired_ds__month AS customer_id__acquired_ds__month + , subq_10.acquired_ds__quarter AS customer_id__acquired_ds__quarter + , subq_10.acquired_ds__year AS customer_id__acquired_ds__year + , subq_10.acquired_ds__extract_year AS customer_id__acquired_ds__extract_year + , subq_10.acquired_ds__extract_quarter AS customer_id__acquired_ds__extract_quarter + , subq_10.acquired_ds__extract_month AS customer_id__acquired_ds__extract_month + , subq_10.acquired_ds__extract_day AS customer_id__acquired_ds__extract_day + , subq_10.acquired_ds__extract_dow AS customer_id__acquired_ds__extract_dow + , subq_10.acquired_ds__extract_doy AS customer_id__acquired_ds__extract_doy + , subq_10.customer_third_hop_id__acquired_ds__day AS customer_id__customer_third_hop_id__acquired_ds__day + , subq_10.customer_third_hop_id__acquired_ds__week AS customer_id__customer_third_hop_id__acquired_ds__week + , subq_10.customer_third_hop_id__acquired_ds__month AS customer_id__customer_third_hop_id__acquired_ds__month + , subq_10.customer_third_hop_id__acquired_ds__quarter AS customer_id__customer_third_hop_id__acquired_ds__quarter + , subq_10.customer_third_hop_id__acquired_ds__year AS customer_id__customer_third_hop_id__acquired_ds__year + , subq_10.customer_third_hop_id__acquired_ds__extract_year AS customer_id__customer_third_hop_id__acquired_ds__extract_year + , subq_10.customer_third_hop_id__acquired_ds__extract_quarter AS customer_id__customer_third_hop_id__acquired_ds__extract_quarter + , subq_10.customer_third_hop_id__acquired_ds__extract_month AS customer_id__customer_third_hop_id__acquired_ds__extract_month + , subq_10.customer_third_hop_id__acquired_ds__extract_day AS customer_id__customer_third_hop_id__acquired_ds__extract_day + , subq_10.customer_third_hop_id__acquired_ds__extract_dow AS customer_id__customer_third_hop_id__acquired_ds__extract_dow + , subq_10.customer_third_hop_id__acquired_ds__extract_doy AS customer_id__customer_third_hop_id__acquired_ds__extract_doy + , subq_10.metric_time__day AS customer_id__metric_time__day + , subq_10.metric_time__week AS customer_id__metric_time__week + , subq_10.metric_time__month AS customer_id__metric_time__month + , subq_10.metric_time__quarter AS customer_id__metric_time__quarter + , subq_10.metric_time__year AS customer_id__metric_time__year + , subq_10.metric_time__extract_year AS customer_id__metric_time__extract_year + , subq_10.metric_time__extract_quarter AS customer_id__metric_time__extract_quarter + , subq_10.metric_time__extract_month AS customer_id__metric_time__extract_month + , subq_10.metric_time__extract_day AS customer_id__metric_time__extract_day + , subq_10.metric_time__extract_dow AS customer_id__metric_time__extract_dow + , subq_10.metric_time__extract_doy AS customer_id__metric_time__extract_doy + , subq_10.customer_third_hop_id AS customer_id__customer_third_hop_id + , subq_10.customer_third_hop_id__customer_id AS customer_id__customer_third_hop_id__customer_id + , subq_7.ds_partitioned__day AS ds_partitioned__day , subq_7.ds_partitioned__week AS ds_partitioned__week , subq_7.ds_partitioned__month AS ds_partitioned__month , subq_7.ds_partitioned__quarter AS ds_partitioned__quarter @@ -308,51 +345,14 @@ FROM ( , subq_7.metric_time__extract_day AS metric_time__extract_day , subq_7.metric_time__extract_dow AS metric_time__extract_dow , subq_7.metric_time__extract_doy AS metric_time__extract_doy - , subq_10.acquired_ds__day AS customer_id__acquired_ds__day - , subq_10.acquired_ds__week AS customer_id__acquired_ds__week - , subq_10.acquired_ds__month AS customer_id__acquired_ds__month - , subq_10.acquired_ds__quarter AS customer_id__acquired_ds__quarter - , subq_10.acquired_ds__year AS customer_id__acquired_ds__year - , subq_10.acquired_ds__extract_year AS customer_id__acquired_ds__extract_year - , subq_10.acquired_ds__extract_quarter AS customer_id__acquired_ds__extract_quarter - , subq_10.acquired_ds__extract_month AS customer_id__acquired_ds__extract_month - , subq_10.acquired_ds__extract_day AS customer_id__acquired_ds__extract_day - , subq_10.acquired_ds__extract_dow AS customer_id__acquired_ds__extract_dow - , subq_10.acquired_ds__extract_doy AS customer_id__acquired_ds__extract_doy - , subq_10.customer_third_hop_id__acquired_ds__day AS customer_id__customer_third_hop_id__acquired_ds__day - , subq_10.customer_third_hop_id__acquired_ds__week AS customer_id__customer_third_hop_id__acquired_ds__week - , subq_10.customer_third_hop_id__acquired_ds__month AS customer_id__customer_third_hop_id__acquired_ds__month - , subq_10.customer_third_hop_id__acquired_ds__quarter AS customer_id__customer_third_hop_id__acquired_ds__quarter - , subq_10.customer_third_hop_id__acquired_ds__year AS customer_id__customer_third_hop_id__acquired_ds__year - , subq_10.customer_third_hop_id__acquired_ds__extract_year AS customer_id__customer_third_hop_id__acquired_ds__extract_year - , subq_10.customer_third_hop_id__acquired_ds__extract_quarter AS customer_id__customer_third_hop_id__acquired_ds__extract_quarter - , subq_10.customer_third_hop_id__acquired_ds__extract_month AS customer_id__customer_third_hop_id__acquired_ds__extract_month - , subq_10.customer_third_hop_id__acquired_ds__extract_day AS customer_id__customer_third_hop_id__acquired_ds__extract_day - , subq_10.customer_third_hop_id__acquired_ds__extract_dow AS customer_id__customer_third_hop_id__acquired_ds__extract_dow - , subq_10.customer_third_hop_id__acquired_ds__extract_doy AS customer_id__customer_third_hop_id__acquired_ds__extract_doy - , subq_10.metric_time__day AS customer_id__metric_time__day - , subq_10.metric_time__week AS customer_id__metric_time__week - , subq_10.metric_time__month AS customer_id__metric_time__month - , subq_10.metric_time__quarter AS customer_id__metric_time__quarter - , subq_10.metric_time__year AS customer_id__metric_time__year - , subq_10.metric_time__extract_year AS customer_id__metric_time__extract_year - , subq_10.metric_time__extract_quarter AS customer_id__metric_time__extract_quarter - , subq_10.metric_time__extract_month AS customer_id__metric_time__extract_month - , subq_10.metric_time__extract_day AS customer_id__metric_time__extract_day - , subq_10.metric_time__extract_dow AS customer_id__metric_time__extract_dow - , subq_10.metric_time__extract_doy AS customer_id__metric_time__extract_doy , subq_7.account_id AS account_id , subq_7.customer_id AS customer_id , subq_7.account_id__customer_id AS account_id__customer_id , subq_7.bridge_account__account_id AS bridge_account__account_id , subq_7.bridge_account__customer_id AS bridge_account__customer_id - , subq_10.customer_third_hop_id AS customer_id__customer_third_hop_id - , subq_10.customer_third_hop_id__customer_id AS customer_id__customer_third_hop_id__customer_id , subq_7.extra_dim AS extra_dim , subq_7.account_id__extra_dim AS account_id__extra_dim , subq_7.bridge_account__extra_dim AS bridge_account__extra_dim - , subq_10.country AS customer_id__country - , subq_10.customer_third_hop_id__country AS customer_id__customer_third_hop_id__country , subq_7.account_customer_combos AS account_customer_combos FROM ( -- Metric Time Dimension 'ds_partitioned' diff --git a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/DuckDB/test_inner_query_single_hop__plan0.sql b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/DuckDB/test_inner_query_single_hop__plan0.sql index aa1fbcf12d..a57e5675da 100644 --- a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/DuckDB/test_inner_query_single_hop__plan0.sql +++ b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/DuckDB/test_inner_query_single_hop__plan0.sql @@ -22,9 +22,9 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.customer_third_hop_id AS customer_third_hop_id - , subq_10.customer_id__customer_third_hop_id AS customer_third_hop_id__customer_id__customer_third_hop_id + subq_10.customer_id__customer_third_hop_id AS customer_third_hop_id__customer_id__customer_third_hop_id , subq_10.customer_id__customer_third_hop_id__paraguayan_customers AS customer_third_hop_id__customer_id__customer_third_hop_id__paraguayan_customers + , subq_2.customer_third_hop_id AS customer_third_hop_id , subq_2.third_hop_count AS third_hop_count FROM ( -- Pass Only Elements: ['third_hop_count', 'customer_third_hop_id'] diff --git a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/DuckDB/test_metric_filtered_by_itself__plan0.sql b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/DuckDB/test_metric_filtered_by_itself__plan0.sql index c22561e2a0..b10efd3e3b 100644 --- a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/DuckDB/test_metric_filtered_by_itself__plan0.sql +++ b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/DuckDB/test_metric_filtered_by_itself__plan0.sql @@ -22,8 +22,8 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.listing AS listing - , subq_8.listing__bookers AS listing__bookers + subq_8.listing__bookers AS listing__bookers + , subq_2.listing AS listing , subq_2.bookers AS bookers FROM ( -- Pass Only Elements: ['bookers', 'listing'] diff --git a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/DuckDB/test_metric_with_metric_in_where_filter__plan0.sql b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/DuckDB/test_metric_with_metric_in_where_filter__plan0.sql index 35bfbe44c8..3e41750e11 100644 --- a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/DuckDB/test_metric_with_metric_in_where_filter__plan0.sql +++ b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/DuckDB/test_metric_with_metric_in_where_filter__plan0.sql @@ -27,9 +27,9 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.metric_time__day AS metric_time__day + subq_8.listing__bookings AS listing__bookings + , subq_2.metric_time__day AS metric_time__day , subq_2.listing AS listing - , subq_8.listing__bookings AS listing__bookings , subq_2.listings AS listings FROM ( -- Pass Only Elements: ['listings', 'metric_time__day', 'listing'] diff --git a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/DuckDB/test_query_with_cumulative_metric_in_where_filter__plan0.sql b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/DuckDB/test_query_with_cumulative_metric_in_where_filter__plan0.sql index 5e05e63f8a..d7686e6bf8 100644 --- a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/DuckDB/test_query_with_cumulative_metric_in_where_filter__plan0.sql +++ b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/DuckDB/test_query_with_cumulative_metric_in_where_filter__plan0.sql @@ -22,8 +22,8 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.user AS user - , subq_8.user__revenue_all_time AS user__revenue_all_time + subq_8.user__revenue_all_time AS user__revenue_all_time + , subq_2.user AS user , subq_2.listings AS listings FROM ( -- Pass Only Elements: ['listings', 'user'] diff --git a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/DuckDB/test_query_with_derived_metric_in_where_filter__plan0.sql b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/DuckDB/test_query_with_derived_metric_in_where_filter__plan0.sql index b4b77786c8..cfac984d0b 100644 --- a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/DuckDB/test_query_with_derived_metric_in_where_filter__plan0.sql +++ b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/DuckDB/test_query_with_derived_metric_in_where_filter__plan0.sql @@ -22,8 +22,8 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.listing AS listing - , subq_15.listing__views_times_booking_value AS listing__views_times_booking_value + subq_15.listing__views_times_booking_value AS listing__views_times_booking_value + , subq_2.listing AS listing , subq_2.listings AS listings FROM ( -- Pass Only Elements: ['listings', 'listing'] diff --git a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/DuckDB/test_query_with_multiple_metrics_in_filter__plan0.sql b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/DuckDB/test_query_with_multiple_metrics_in_filter__plan0.sql index 076b390821..7df39950a3 100644 --- a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/DuckDB/test_query_with_multiple_metrics_in_filter__plan0.sql +++ b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/DuckDB/test_query_with_multiple_metrics_in_filter__plan0.sql @@ -24,9 +24,9 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.listing AS listing - , subq_8.listing__bookings AS listing__bookings + subq_8.listing__bookings AS listing__bookings , subq_14.listing__bookers AS listing__bookers + , subq_2.listing AS listing , subq_2.listings AS listings FROM ( -- Pass Only Elements: ['listings', 'listing', 'listing'] diff --git a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/DuckDB/test_query_with_ratio_metric_in_where_filter__plan0.sql b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/DuckDB/test_query_with_ratio_metric_in_where_filter__plan0.sql index 6d407fb8ca..225b6d2a32 100644 --- a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/DuckDB/test_query_with_ratio_metric_in_where_filter__plan0.sql +++ b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/DuckDB/test_query_with_ratio_metric_in_where_filter__plan0.sql @@ -22,8 +22,8 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.listing AS listing - , subq_15.listing__bookings_per_booker AS listing__bookings_per_booker + subq_15.listing__bookings_per_booker AS listing__bookings_per_booker + , subq_2.listing AS listing , subq_2.listings AS listings FROM ( -- Pass Only Elements: ['listings', 'listing'] diff --git a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/DuckDB/test_query_with_simple_metric_in_where_filter__plan0.sql b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/DuckDB/test_query_with_simple_metric_in_where_filter__plan0.sql index d42f3e3875..97ec04b930 100644 --- a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/DuckDB/test_query_with_simple_metric_in_where_filter__plan0.sql +++ b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/DuckDB/test_query_with_simple_metric_in_where_filter__plan0.sql @@ -22,8 +22,8 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.listing AS listing - , subq_8.listing__bookings AS listing__bookings + subq_8.listing__bookings AS listing__bookings + , subq_2.listing AS listing , subq_2.listings AS listings FROM ( -- Pass Only Elements: ['listings', 'listing'] diff --git a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Postgres/test_distinct_values_query_with_metric_filter__plan0.sql b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Postgres/test_distinct_values_query_with_metric_filter__plan0.sql index 799e437abe..d89cbe19cb 100644 --- a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Postgres/test_distinct_values_query_with_metric_filter__plan0.sql +++ b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Postgres/test_distinct_values_query_with_metric_filter__plan0.sql @@ -11,10 +11,10 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_0.listing AS listing + subq_6.listing__bookings AS listing__bookings + , subq_0.listing AS listing , subq_0.lux_listing AS lux_listing , subq_0.listing__lux_listing AS listing__lux_listing - , subq_6.listing__bookings AS listing__bookings FROM ( -- Read Elements From Semantic Model 'lux_listing_mapping' SELECT diff --git a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Postgres/test_distinct_values_query_with_metric_filter__plan0_optimized.sql b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Postgres/test_distinct_values_query_with_metric_filter__plan0_optimized.sql index d79ef0aa09..c51da11fab 100644 --- a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Postgres/test_distinct_values_query_with_metric_filter__plan0_optimized.sql +++ b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Postgres/test_distinct_values_query_with_metric_filter__plan0_optimized.sql @@ -5,8 +5,8 @@ SELECT FROM ( -- Join Standard Outputs SELECT - lux_listing_mapping_src_28000.listing_id AS listing - , subq_15.listing__bookings AS listing__bookings + subq_15.listing__bookings AS listing__bookings + , lux_listing_mapping_src_28000.listing_id AS listing FROM ***************************.dim_lux_listing_id_mapping lux_listing_mapping_src_28000 FULL OUTER JOIN ( -- Aggregate Measures diff --git a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Postgres/test_filter_by_metric_in_same_semantic_model_as_queried_metric__plan0.sql b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Postgres/test_filter_by_metric_in_same_semantic_model_as_queried_metric__plan0.sql index e5dc490b67..a0a48965f4 100644 --- a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Postgres/test_filter_by_metric_in_same_semantic_model_as_queried_metric__plan0.sql +++ b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Postgres/test_filter_by_metric_in_same_semantic_model_as_queried_metric__plan0.sql @@ -22,8 +22,8 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.guest AS guest - , subq_8.guest__booking_value AS guest__booking_value + subq_8.guest__booking_value AS guest__booking_value + , subq_2.guest AS guest , subq_2.bookers AS bookers FROM ( -- Pass Only Elements: ['bookers', 'guest'] diff --git a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Postgres/test_filter_with_conversion_metric__plan0.sql b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Postgres/test_filter_with_conversion_metric__plan0.sql index 3ed6c979b3..40436a2c9d 100644 --- a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Postgres/test_filter_with_conversion_metric__plan0.sql +++ b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Postgres/test_filter_with_conversion_metric__plan0.sql @@ -22,8 +22,8 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.user AS user - , subq_19.user__visit_buy_conversion_rate AS user__visit_buy_conversion_rate + subq_19.user__visit_buy_conversion_rate AS user__visit_buy_conversion_rate + , subq_2.user AS user , subq_2.listings AS listings FROM ( -- Pass Only Elements: ['listings', 'user'] diff --git a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Postgres/test_group_by_has_local_entity_prefix__plan0.sql b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Postgres/test_group_by_has_local_entity_prefix__plan0.sql index 1b6946e2c8..646daaaf6e 100644 --- a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Postgres/test_group_by_has_local_entity_prefix__plan0.sql +++ b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Postgres/test_group_by_has_local_entity_prefix__plan0.sql @@ -22,9 +22,9 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.user AS user - , subq_13.listing__user AS user__listing__user + subq_13.listing__user AS user__listing__user , subq_13.listing__user__average_booking_value AS user__listing__user__average_booking_value + , subq_2.user AS user , subq_2.listings AS listings FROM ( -- Pass Only Elements: ['listings', 'user'] @@ -187,8 +187,8 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_5.listing AS listing - , subq_8.user AS listing__user + subq_8.user AS listing__user + , subq_5.listing AS listing , subq_5.average_booking_value AS average_booking_value FROM ( -- Pass Only Elements: ['average_booking_value', 'listing'] diff --git a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Postgres/test_inner_query_multi_hop__plan0.sql b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Postgres/test_inner_query_multi_hop__plan0.sql index 3146c0bc2f..5ce70b6587 100644 --- a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Postgres/test_inner_query_multi_hop__plan0.sql +++ b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Postgres/test_inner_query_multi_hop__plan0.sql @@ -22,9 +22,9 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.customer_third_hop_id AS customer_third_hop_id - , subq_17.account_id__customer_id__customer_third_hop_id AS customer_third_hop_id__account_id__customer_id__customer_third_hop_id + subq_17.account_id__customer_id__customer_third_hop_id AS customer_third_hop_id__account_id__customer_id__customer_third_hop_id , subq_17.account_id__customer_id__customer_third_hop_id__txn_count AS customer_third_hop_id__account_id__customer_id__customer_third_hop_id__txn_count + , subq_2.customer_third_hop_id AS customer_third_hop_id , subq_2.third_hop_count AS third_hop_count FROM ( -- Pass Only Elements: ['third_hop_count', 'customer_third_hop_id'] @@ -127,10 +127,10 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_5.ds_partitioned__day AS ds_partitioned__day - , subq_12.ds_partitioned__day AS account_id__ds_partitioned__day - , subq_5.account_id AS account_id + subq_12.ds_partitioned__day AS account_id__ds_partitioned__day , subq_12.customer_id__customer_third_hop_id AS account_id__customer_id__customer_third_hop_id + , subq_5.ds_partitioned__day AS ds_partitioned__day + , subq_5.account_id AS account_id , subq_5.txn_count AS txn_count FROM ( -- Pass Only Elements: ['txn_count', 'ds_partitioned__day', 'account_id'] @@ -264,7 +264,44 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_7.ds_partitioned__day AS ds_partitioned__day + subq_10.country AS customer_id__country + , subq_10.customer_third_hop_id__country AS customer_id__customer_third_hop_id__country + , subq_10.acquired_ds__day AS customer_id__acquired_ds__day + , subq_10.acquired_ds__week AS customer_id__acquired_ds__week + , subq_10.acquired_ds__month AS customer_id__acquired_ds__month + , subq_10.acquired_ds__quarter AS customer_id__acquired_ds__quarter + , subq_10.acquired_ds__year AS customer_id__acquired_ds__year + , subq_10.acquired_ds__extract_year AS customer_id__acquired_ds__extract_year + , subq_10.acquired_ds__extract_quarter AS customer_id__acquired_ds__extract_quarter + , subq_10.acquired_ds__extract_month AS customer_id__acquired_ds__extract_month + , subq_10.acquired_ds__extract_day AS customer_id__acquired_ds__extract_day + , subq_10.acquired_ds__extract_dow AS customer_id__acquired_ds__extract_dow + , subq_10.acquired_ds__extract_doy AS customer_id__acquired_ds__extract_doy + , subq_10.customer_third_hop_id__acquired_ds__day AS customer_id__customer_third_hop_id__acquired_ds__day + , subq_10.customer_third_hop_id__acquired_ds__week AS customer_id__customer_third_hop_id__acquired_ds__week + , subq_10.customer_third_hop_id__acquired_ds__month AS customer_id__customer_third_hop_id__acquired_ds__month + , subq_10.customer_third_hop_id__acquired_ds__quarter AS customer_id__customer_third_hop_id__acquired_ds__quarter + , subq_10.customer_third_hop_id__acquired_ds__year AS customer_id__customer_third_hop_id__acquired_ds__year + , subq_10.customer_third_hop_id__acquired_ds__extract_year AS customer_id__customer_third_hop_id__acquired_ds__extract_year + , subq_10.customer_third_hop_id__acquired_ds__extract_quarter AS customer_id__customer_third_hop_id__acquired_ds__extract_quarter + , subq_10.customer_third_hop_id__acquired_ds__extract_month AS customer_id__customer_third_hop_id__acquired_ds__extract_month + , subq_10.customer_third_hop_id__acquired_ds__extract_day AS customer_id__customer_third_hop_id__acquired_ds__extract_day + , subq_10.customer_third_hop_id__acquired_ds__extract_dow AS customer_id__customer_third_hop_id__acquired_ds__extract_dow + , subq_10.customer_third_hop_id__acquired_ds__extract_doy AS customer_id__customer_third_hop_id__acquired_ds__extract_doy + , subq_10.metric_time__day AS customer_id__metric_time__day + , subq_10.metric_time__week AS customer_id__metric_time__week + , subq_10.metric_time__month AS customer_id__metric_time__month + , subq_10.metric_time__quarter AS customer_id__metric_time__quarter + , subq_10.metric_time__year AS customer_id__metric_time__year + , subq_10.metric_time__extract_year AS customer_id__metric_time__extract_year + , subq_10.metric_time__extract_quarter AS customer_id__metric_time__extract_quarter + , subq_10.metric_time__extract_month AS customer_id__metric_time__extract_month + , subq_10.metric_time__extract_day AS customer_id__metric_time__extract_day + , subq_10.metric_time__extract_dow AS customer_id__metric_time__extract_dow + , subq_10.metric_time__extract_doy AS customer_id__metric_time__extract_doy + , subq_10.customer_third_hop_id AS customer_id__customer_third_hop_id + , subq_10.customer_third_hop_id__customer_id AS customer_id__customer_third_hop_id__customer_id + , subq_7.ds_partitioned__day AS ds_partitioned__day , subq_7.ds_partitioned__week AS ds_partitioned__week , subq_7.ds_partitioned__month AS ds_partitioned__month , subq_7.ds_partitioned__quarter AS ds_partitioned__quarter @@ -308,51 +345,14 @@ FROM ( , subq_7.metric_time__extract_day AS metric_time__extract_day , subq_7.metric_time__extract_dow AS metric_time__extract_dow , subq_7.metric_time__extract_doy AS metric_time__extract_doy - , subq_10.acquired_ds__day AS customer_id__acquired_ds__day - , subq_10.acquired_ds__week AS customer_id__acquired_ds__week - , subq_10.acquired_ds__month AS customer_id__acquired_ds__month - , subq_10.acquired_ds__quarter AS customer_id__acquired_ds__quarter - , subq_10.acquired_ds__year AS customer_id__acquired_ds__year - , subq_10.acquired_ds__extract_year AS customer_id__acquired_ds__extract_year - , subq_10.acquired_ds__extract_quarter AS customer_id__acquired_ds__extract_quarter - , subq_10.acquired_ds__extract_month AS customer_id__acquired_ds__extract_month - , subq_10.acquired_ds__extract_day AS customer_id__acquired_ds__extract_day - , subq_10.acquired_ds__extract_dow AS customer_id__acquired_ds__extract_dow - , subq_10.acquired_ds__extract_doy AS customer_id__acquired_ds__extract_doy - , subq_10.customer_third_hop_id__acquired_ds__day AS customer_id__customer_third_hop_id__acquired_ds__day - , subq_10.customer_third_hop_id__acquired_ds__week AS customer_id__customer_third_hop_id__acquired_ds__week - , subq_10.customer_third_hop_id__acquired_ds__month AS customer_id__customer_third_hop_id__acquired_ds__month - , subq_10.customer_third_hop_id__acquired_ds__quarter AS customer_id__customer_third_hop_id__acquired_ds__quarter - , subq_10.customer_third_hop_id__acquired_ds__year AS customer_id__customer_third_hop_id__acquired_ds__year - , subq_10.customer_third_hop_id__acquired_ds__extract_year AS customer_id__customer_third_hop_id__acquired_ds__extract_year - , subq_10.customer_third_hop_id__acquired_ds__extract_quarter AS customer_id__customer_third_hop_id__acquired_ds__extract_quarter - , subq_10.customer_third_hop_id__acquired_ds__extract_month AS customer_id__customer_third_hop_id__acquired_ds__extract_month - , subq_10.customer_third_hop_id__acquired_ds__extract_day AS customer_id__customer_third_hop_id__acquired_ds__extract_day - , subq_10.customer_third_hop_id__acquired_ds__extract_dow AS customer_id__customer_third_hop_id__acquired_ds__extract_dow - , subq_10.customer_third_hop_id__acquired_ds__extract_doy AS customer_id__customer_third_hop_id__acquired_ds__extract_doy - , subq_10.metric_time__day AS customer_id__metric_time__day - , subq_10.metric_time__week AS customer_id__metric_time__week - , subq_10.metric_time__month AS customer_id__metric_time__month - , subq_10.metric_time__quarter AS customer_id__metric_time__quarter - , subq_10.metric_time__year AS customer_id__metric_time__year - , subq_10.metric_time__extract_year AS customer_id__metric_time__extract_year - , subq_10.metric_time__extract_quarter AS customer_id__metric_time__extract_quarter - , subq_10.metric_time__extract_month AS customer_id__metric_time__extract_month - , subq_10.metric_time__extract_day AS customer_id__metric_time__extract_day - , subq_10.metric_time__extract_dow AS customer_id__metric_time__extract_dow - , subq_10.metric_time__extract_doy AS customer_id__metric_time__extract_doy , subq_7.account_id AS account_id , subq_7.customer_id AS customer_id , subq_7.account_id__customer_id AS account_id__customer_id , subq_7.bridge_account__account_id AS bridge_account__account_id , subq_7.bridge_account__customer_id AS bridge_account__customer_id - , subq_10.customer_third_hop_id AS customer_id__customer_third_hop_id - , subq_10.customer_third_hop_id__customer_id AS customer_id__customer_third_hop_id__customer_id , subq_7.extra_dim AS extra_dim , subq_7.account_id__extra_dim AS account_id__extra_dim , subq_7.bridge_account__extra_dim AS bridge_account__extra_dim - , subq_10.country AS customer_id__country - , subq_10.customer_third_hop_id__country AS customer_id__customer_third_hop_id__country , subq_7.account_customer_combos AS account_customer_combos FROM ( -- Metric Time Dimension 'ds_partitioned' diff --git a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Postgres/test_inner_query_single_hop__plan0.sql b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Postgres/test_inner_query_single_hop__plan0.sql index aa1fbcf12d..a57e5675da 100644 --- a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Postgres/test_inner_query_single_hop__plan0.sql +++ b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Postgres/test_inner_query_single_hop__plan0.sql @@ -22,9 +22,9 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.customer_third_hop_id AS customer_third_hop_id - , subq_10.customer_id__customer_third_hop_id AS customer_third_hop_id__customer_id__customer_third_hop_id + subq_10.customer_id__customer_third_hop_id AS customer_third_hop_id__customer_id__customer_third_hop_id , subq_10.customer_id__customer_third_hop_id__paraguayan_customers AS customer_third_hop_id__customer_id__customer_third_hop_id__paraguayan_customers + , subq_2.customer_third_hop_id AS customer_third_hop_id , subq_2.third_hop_count AS third_hop_count FROM ( -- Pass Only Elements: ['third_hop_count', 'customer_third_hop_id'] diff --git a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Postgres/test_metric_filtered_by_itself__plan0.sql b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Postgres/test_metric_filtered_by_itself__plan0.sql index c22561e2a0..b10efd3e3b 100644 --- a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Postgres/test_metric_filtered_by_itself__plan0.sql +++ b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Postgres/test_metric_filtered_by_itself__plan0.sql @@ -22,8 +22,8 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.listing AS listing - , subq_8.listing__bookers AS listing__bookers + subq_8.listing__bookers AS listing__bookers + , subq_2.listing AS listing , subq_2.bookers AS bookers FROM ( -- Pass Only Elements: ['bookers', 'listing'] diff --git a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Postgres/test_metric_with_metric_in_where_filter__plan0.sql b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Postgres/test_metric_with_metric_in_where_filter__plan0.sql index 35bfbe44c8..3e41750e11 100644 --- a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Postgres/test_metric_with_metric_in_where_filter__plan0.sql +++ b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Postgres/test_metric_with_metric_in_where_filter__plan0.sql @@ -27,9 +27,9 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.metric_time__day AS metric_time__day + subq_8.listing__bookings AS listing__bookings + , subq_2.metric_time__day AS metric_time__day , subq_2.listing AS listing - , subq_8.listing__bookings AS listing__bookings , subq_2.listings AS listings FROM ( -- Pass Only Elements: ['listings', 'metric_time__day', 'listing'] diff --git a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Postgres/test_query_with_cumulative_metric_in_where_filter__plan0.sql b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Postgres/test_query_with_cumulative_metric_in_where_filter__plan0.sql index 5e05e63f8a..d7686e6bf8 100644 --- a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Postgres/test_query_with_cumulative_metric_in_where_filter__plan0.sql +++ b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Postgres/test_query_with_cumulative_metric_in_where_filter__plan0.sql @@ -22,8 +22,8 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.user AS user - , subq_8.user__revenue_all_time AS user__revenue_all_time + subq_8.user__revenue_all_time AS user__revenue_all_time + , subq_2.user AS user , subq_2.listings AS listings FROM ( -- Pass Only Elements: ['listings', 'user'] diff --git a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Postgres/test_query_with_derived_metric_in_where_filter__plan0.sql b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Postgres/test_query_with_derived_metric_in_where_filter__plan0.sql index b4b77786c8..cfac984d0b 100644 --- a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Postgres/test_query_with_derived_metric_in_where_filter__plan0.sql +++ b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Postgres/test_query_with_derived_metric_in_where_filter__plan0.sql @@ -22,8 +22,8 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.listing AS listing - , subq_15.listing__views_times_booking_value AS listing__views_times_booking_value + subq_15.listing__views_times_booking_value AS listing__views_times_booking_value + , subq_2.listing AS listing , subq_2.listings AS listings FROM ( -- Pass Only Elements: ['listings', 'listing'] diff --git a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Postgres/test_query_with_multiple_metrics_in_filter__plan0.sql b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Postgres/test_query_with_multiple_metrics_in_filter__plan0.sql index 076b390821..7df39950a3 100644 --- a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Postgres/test_query_with_multiple_metrics_in_filter__plan0.sql +++ b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Postgres/test_query_with_multiple_metrics_in_filter__plan0.sql @@ -24,9 +24,9 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.listing AS listing - , subq_8.listing__bookings AS listing__bookings + subq_8.listing__bookings AS listing__bookings , subq_14.listing__bookers AS listing__bookers + , subq_2.listing AS listing , subq_2.listings AS listings FROM ( -- Pass Only Elements: ['listings', 'listing', 'listing'] diff --git a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Postgres/test_query_with_ratio_metric_in_where_filter__plan0.sql b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Postgres/test_query_with_ratio_metric_in_where_filter__plan0.sql index de9c83785e..ba88cd95f5 100644 --- a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Postgres/test_query_with_ratio_metric_in_where_filter__plan0.sql +++ b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Postgres/test_query_with_ratio_metric_in_where_filter__plan0.sql @@ -22,8 +22,8 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.listing AS listing - , subq_15.listing__bookings_per_booker AS listing__bookings_per_booker + subq_15.listing__bookings_per_booker AS listing__bookings_per_booker + , subq_2.listing AS listing , subq_2.listings AS listings FROM ( -- Pass Only Elements: ['listings', 'listing'] diff --git a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Postgres/test_query_with_simple_metric_in_where_filter__plan0.sql b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Postgres/test_query_with_simple_metric_in_where_filter__plan0.sql index d42f3e3875..97ec04b930 100644 --- a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Postgres/test_query_with_simple_metric_in_where_filter__plan0.sql +++ b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Postgres/test_query_with_simple_metric_in_where_filter__plan0.sql @@ -22,8 +22,8 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.listing AS listing - , subq_8.listing__bookings AS listing__bookings + subq_8.listing__bookings AS listing__bookings + , subq_2.listing AS listing , subq_2.listings AS listings FROM ( -- Pass Only Elements: ['listings', 'listing'] diff --git a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Redshift/test_distinct_values_query_with_metric_filter__plan0.sql b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Redshift/test_distinct_values_query_with_metric_filter__plan0.sql index 21bba933b5..30a4555b9e 100644 --- a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Redshift/test_distinct_values_query_with_metric_filter__plan0.sql +++ b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Redshift/test_distinct_values_query_with_metric_filter__plan0.sql @@ -11,10 +11,10 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_0.listing AS listing + subq_6.listing__bookings AS listing__bookings + , subq_0.listing AS listing , subq_0.lux_listing AS lux_listing , subq_0.listing__lux_listing AS listing__lux_listing - , subq_6.listing__bookings AS listing__bookings FROM ( -- Read Elements From Semantic Model 'lux_listing_mapping' SELECT diff --git a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Redshift/test_distinct_values_query_with_metric_filter__plan0_optimized.sql b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Redshift/test_distinct_values_query_with_metric_filter__plan0_optimized.sql index d79ef0aa09..c51da11fab 100644 --- a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Redshift/test_distinct_values_query_with_metric_filter__plan0_optimized.sql +++ b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Redshift/test_distinct_values_query_with_metric_filter__plan0_optimized.sql @@ -5,8 +5,8 @@ SELECT FROM ( -- Join Standard Outputs SELECT - lux_listing_mapping_src_28000.listing_id AS listing - , subq_15.listing__bookings AS listing__bookings + subq_15.listing__bookings AS listing__bookings + , lux_listing_mapping_src_28000.listing_id AS listing FROM ***************************.dim_lux_listing_id_mapping lux_listing_mapping_src_28000 FULL OUTER JOIN ( -- Aggregate Measures diff --git a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Redshift/test_filter_by_metric_in_same_semantic_model_as_queried_metric__plan0.sql b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Redshift/test_filter_by_metric_in_same_semantic_model_as_queried_metric__plan0.sql index f2b0169f9d..819212083f 100644 --- a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Redshift/test_filter_by_metric_in_same_semantic_model_as_queried_metric__plan0.sql +++ b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Redshift/test_filter_by_metric_in_same_semantic_model_as_queried_metric__plan0.sql @@ -22,8 +22,8 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.guest AS guest - , subq_8.guest__booking_value AS guest__booking_value + subq_8.guest__booking_value AS guest__booking_value + , subq_2.guest AS guest , subq_2.bookers AS bookers FROM ( -- Pass Only Elements: ['bookers', 'guest'] diff --git a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Redshift/test_filter_with_conversion_metric__plan0.sql b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Redshift/test_filter_with_conversion_metric__plan0.sql index 368ebe81d3..9a08a748f2 100644 --- a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Redshift/test_filter_with_conversion_metric__plan0.sql +++ b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Redshift/test_filter_with_conversion_metric__plan0.sql @@ -22,8 +22,8 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.user AS user - , subq_19.user__visit_buy_conversion_rate AS user__visit_buy_conversion_rate + subq_19.user__visit_buy_conversion_rate AS user__visit_buy_conversion_rate + , subq_2.user AS user , subq_2.listings AS listings FROM ( -- Pass Only Elements: ['listings', 'user'] diff --git a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Redshift/test_group_by_has_local_entity_prefix__plan0.sql b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Redshift/test_group_by_has_local_entity_prefix__plan0.sql index 4077c6d0b8..e9cf44d859 100644 --- a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Redshift/test_group_by_has_local_entity_prefix__plan0.sql +++ b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Redshift/test_group_by_has_local_entity_prefix__plan0.sql @@ -22,9 +22,9 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.user AS user - , subq_13.listing__user AS user__listing__user + subq_13.listing__user AS user__listing__user , subq_13.listing__user__average_booking_value AS user__listing__user__average_booking_value + , subq_2.user AS user , subq_2.listings AS listings FROM ( -- Pass Only Elements: ['listings', 'user'] @@ -187,8 +187,8 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_5.listing AS listing - , subq_8.user AS listing__user + subq_8.user AS listing__user + , subq_5.listing AS listing , subq_5.average_booking_value AS average_booking_value FROM ( -- Pass Only Elements: ['average_booking_value', 'listing'] diff --git a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Redshift/test_inner_query_multi_hop__plan0.sql b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Redshift/test_inner_query_multi_hop__plan0.sql index 4e4fa8d309..a77e86ef93 100644 --- a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Redshift/test_inner_query_multi_hop__plan0.sql +++ b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Redshift/test_inner_query_multi_hop__plan0.sql @@ -22,9 +22,9 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.customer_third_hop_id AS customer_third_hop_id - , subq_17.account_id__customer_id__customer_third_hop_id AS customer_third_hop_id__account_id__customer_id__customer_third_hop_id + subq_17.account_id__customer_id__customer_third_hop_id AS customer_third_hop_id__account_id__customer_id__customer_third_hop_id , subq_17.account_id__customer_id__customer_third_hop_id__txn_count AS customer_third_hop_id__account_id__customer_id__customer_third_hop_id__txn_count + , subq_2.customer_third_hop_id AS customer_third_hop_id , subq_2.third_hop_count AS third_hop_count FROM ( -- Pass Only Elements: ['third_hop_count', 'customer_third_hop_id'] @@ -127,10 +127,10 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_5.ds_partitioned__day AS ds_partitioned__day - , subq_12.ds_partitioned__day AS account_id__ds_partitioned__day - , subq_5.account_id AS account_id + subq_12.ds_partitioned__day AS account_id__ds_partitioned__day , subq_12.customer_id__customer_third_hop_id AS account_id__customer_id__customer_third_hop_id + , subq_5.ds_partitioned__day AS ds_partitioned__day + , subq_5.account_id AS account_id , subq_5.txn_count AS txn_count FROM ( -- Pass Only Elements: ['txn_count', 'ds_partitioned__day', 'account_id'] @@ -264,7 +264,44 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_7.ds_partitioned__day AS ds_partitioned__day + subq_10.country AS customer_id__country + , subq_10.customer_third_hop_id__country AS customer_id__customer_third_hop_id__country + , subq_10.acquired_ds__day AS customer_id__acquired_ds__day + , subq_10.acquired_ds__week AS customer_id__acquired_ds__week + , subq_10.acquired_ds__month AS customer_id__acquired_ds__month + , subq_10.acquired_ds__quarter AS customer_id__acquired_ds__quarter + , subq_10.acquired_ds__year AS customer_id__acquired_ds__year + , subq_10.acquired_ds__extract_year AS customer_id__acquired_ds__extract_year + , subq_10.acquired_ds__extract_quarter AS customer_id__acquired_ds__extract_quarter + , subq_10.acquired_ds__extract_month AS customer_id__acquired_ds__extract_month + , subq_10.acquired_ds__extract_day AS customer_id__acquired_ds__extract_day + , subq_10.acquired_ds__extract_dow AS customer_id__acquired_ds__extract_dow + , subq_10.acquired_ds__extract_doy AS customer_id__acquired_ds__extract_doy + , subq_10.customer_third_hop_id__acquired_ds__day AS customer_id__customer_third_hop_id__acquired_ds__day + , subq_10.customer_third_hop_id__acquired_ds__week AS customer_id__customer_third_hop_id__acquired_ds__week + , subq_10.customer_third_hop_id__acquired_ds__month AS customer_id__customer_third_hop_id__acquired_ds__month + , subq_10.customer_third_hop_id__acquired_ds__quarter AS customer_id__customer_third_hop_id__acquired_ds__quarter + , subq_10.customer_third_hop_id__acquired_ds__year AS customer_id__customer_third_hop_id__acquired_ds__year + , subq_10.customer_third_hop_id__acquired_ds__extract_year AS customer_id__customer_third_hop_id__acquired_ds__extract_year + , subq_10.customer_third_hop_id__acquired_ds__extract_quarter AS customer_id__customer_third_hop_id__acquired_ds__extract_quarter + , subq_10.customer_third_hop_id__acquired_ds__extract_month AS customer_id__customer_third_hop_id__acquired_ds__extract_month + , subq_10.customer_third_hop_id__acquired_ds__extract_day AS customer_id__customer_third_hop_id__acquired_ds__extract_day + , subq_10.customer_third_hop_id__acquired_ds__extract_dow AS customer_id__customer_third_hop_id__acquired_ds__extract_dow + , subq_10.customer_third_hop_id__acquired_ds__extract_doy AS customer_id__customer_third_hop_id__acquired_ds__extract_doy + , subq_10.metric_time__day AS customer_id__metric_time__day + , subq_10.metric_time__week AS customer_id__metric_time__week + , subq_10.metric_time__month AS customer_id__metric_time__month + , subq_10.metric_time__quarter AS customer_id__metric_time__quarter + , subq_10.metric_time__year AS customer_id__metric_time__year + , subq_10.metric_time__extract_year AS customer_id__metric_time__extract_year + , subq_10.metric_time__extract_quarter AS customer_id__metric_time__extract_quarter + , subq_10.metric_time__extract_month AS customer_id__metric_time__extract_month + , subq_10.metric_time__extract_day AS customer_id__metric_time__extract_day + , subq_10.metric_time__extract_dow AS customer_id__metric_time__extract_dow + , subq_10.metric_time__extract_doy AS customer_id__metric_time__extract_doy + , subq_10.customer_third_hop_id AS customer_id__customer_third_hop_id + , subq_10.customer_third_hop_id__customer_id AS customer_id__customer_third_hop_id__customer_id + , subq_7.ds_partitioned__day AS ds_partitioned__day , subq_7.ds_partitioned__week AS ds_partitioned__week , subq_7.ds_partitioned__month AS ds_partitioned__month , subq_7.ds_partitioned__quarter AS ds_partitioned__quarter @@ -308,51 +345,14 @@ FROM ( , subq_7.metric_time__extract_day AS metric_time__extract_day , subq_7.metric_time__extract_dow AS metric_time__extract_dow , subq_7.metric_time__extract_doy AS metric_time__extract_doy - , subq_10.acquired_ds__day AS customer_id__acquired_ds__day - , subq_10.acquired_ds__week AS customer_id__acquired_ds__week - , subq_10.acquired_ds__month AS customer_id__acquired_ds__month - , subq_10.acquired_ds__quarter AS customer_id__acquired_ds__quarter - , subq_10.acquired_ds__year AS customer_id__acquired_ds__year - , subq_10.acquired_ds__extract_year AS customer_id__acquired_ds__extract_year - , subq_10.acquired_ds__extract_quarter AS customer_id__acquired_ds__extract_quarter - , subq_10.acquired_ds__extract_month AS customer_id__acquired_ds__extract_month - , subq_10.acquired_ds__extract_day AS customer_id__acquired_ds__extract_day - , subq_10.acquired_ds__extract_dow AS customer_id__acquired_ds__extract_dow - , subq_10.acquired_ds__extract_doy AS customer_id__acquired_ds__extract_doy - , subq_10.customer_third_hop_id__acquired_ds__day AS customer_id__customer_third_hop_id__acquired_ds__day - , subq_10.customer_third_hop_id__acquired_ds__week AS customer_id__customer_third_hop_id__acquired_ds__week - , subq_10.customer_third_hop_id__acquired_ds__month AS customer_id__customer_third_hop_id__acquired_ds__month - , subq_10.customer_third_hop_id__acquired_ds__quarter AS customer_id__customer_third_hop_id__acquired_ds__quarter - , subq_10.customer_third_hop_id__acquired_ds__year AS customer_id__customer_third_hop_id__acquired_ds__year - , subq_10.customer_third_hop_id__acquired_ds__extract_year AS customer_id__customer_third_hop_id__acquired_ds__extract_year - , subq_10.customer_third_hop_id__acquired_ds__extract_quarter AS customer_id__customer_third_hop_id__acquired_ds__extract_quarter - , subq_10.customer_third_hop_id__acquired_ds__extract_month AS customer_id__customer_third_hop_id__acquired_ds__extract_month - , subq_10.customer_third_hop_id__acquired_ds__extract_day AS customer_id__customer_third_hop_id__acquired_ds__extract_day - , subq_10.customer_third_hop_id__acquired_ds__extract_dow AS customer_id__customer_third_hop_id__acquired_ds__extract_dow - , subq_10.customer_third_hop_id__acquired_ds__extract_doy AS customer_id__customer_third_hop_id__acquired_ds__extract_doy - , subq_10.metric_time__day AS customer_id__metric_time__day - , subq_10.metric_time__week AS customer_id__metric_time__week - , subq_10.metric_time__month AS customer_id__metric_time__month - , subq_10.metric_time__quarter AS customer_id__metric_time__quarter - , subq_10.metric_time__year AS customer_id__metric_time__year - , subq_10.metric_time__extract_year AS customer_id__metric_time__extract_year - , subq_10.metric_time__extract_quarter AS customer_id__metric_time__extract_quarter - , subq_10.metric_time__extract_month AS customer_id__metric_time__extract_month - , subq_10.metric_time__extract_day AS customer_id__metric_time__extract_day - , subq_10.metric_time__extract_dow AS customer_id__metric_time__extract_dow - , subq_10.metric_time__extract_doy AS customer_id__metric_time__extract_doy , subq_7.account_id AS account_id , subq_7.customer_id AS customer_id , subq_7.account_id__customer_id AS account_id__customer_id , subq_7.bridge_account__account_id AS bridge_account__account_id , subq_7.bridge_account__customer_id AS bridge_account__customer_id - , subq_10.customer_third_hop_id AS customer_id__customer_third_hop_id - , subq_10.customer_third_hop_id__customer_id AS customer_id__customer_third_hop_id__customer_id , subq_7.extra_dim AS extra_dim , subq_7.account_id__extra_dim AS account_id__extra_dim , subq_7.bridge_account__extra_dim AS bridge_account__extra_dim - , subq_10.country AS customer_id__country - , subq_10.customer_third_hop_id__country AS customer_id__customer_third_hop_id__country , subq_7.account_customer_combos AS account_customer_combos FROM ( -- Metric Time Dimension 'ds_partitioned' diff --git a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Redshift/test_inner_query_single_hop__plan0.sql b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Redshift/test_inner_query_single_hop__plan0.sql index 949979d2bb..8a031c83ea 100644 --- a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Redshift/test_inner_query_single_hop__plan0.sql +++ b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Redshift/test_inner_query_single_hop__plan0.sql @@ -22,9 +22,9 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.customer_third_hop_id AS customer_third_hop_id - , subq_10.customer_id__customer_third_hop_id AS customer_third_hop_id__customer_id__customer_third_hop_id + subq_10.customer_id__customer_third_hop_id AS customer_third_hop_id__customer_id__customer_third_hop_id , subq_10.customer_id__customer_third_hop_id__paraguayan_customers AS customer_third_hop_id__customer_id__customer_third_hop_id__paraguayan_customers + , subq_2.customer_third_hop_id AS customer_third_hop_id , subq_2.third_hop_count AS third_hop_count FROM ( -- Pass Only Elements: ['third_hop_count', 'customer_third_hop_id'] diff --git a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Redshift/test_metric_filtered_by_itself__plan0.sql b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Redshift/test_metric_filtered_by_itself__plan0.sql index 52e3493f58..be6c880698 100644 --- a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Redshift/test_metric_filtered_by_itself__plan0.sql +++ b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Redshift/test_metric_filtered_by_itself__plan0.sql @@ -22,8 +22,8 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.listing AS listing - , subq_8.listing__bookers AS listing__bookers + subq_8.listing__bookers AS listing__bookers + , subq_2.listing AS listing , subq_2.bookers AS bookers FROM ( -- Pass Only Elements: ['bookers', 'listing'] diff --git a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Redshift/test_metric_with_metric_in_where_filter__plan0.sql b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Redshift/test_metric_with_metric_in_where_filter__plan0.sql index 6cb487ea46..76198dc32c 100644 --- a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Redshift/test_metric_with_metric_in_where_filter__plan0.sql +++ b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Redshift/test_metric_with_metric_in_where_filter__plan0.sql @@ -27,9 +27,9 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.metric_time__day AS metric_time__day + subq_8.listing__bookings AS listing__bookings + , subq_2.metric_time__day AS metric_time__day , subq_2.listing AS listing - , subq_8.listing__bookings AS listing__bookings , subq_2.listings AS listings FROM ( -- Pass Only Elements: ['listings', 'metric_time__day', 'listing'] diff --git a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Redshift/test_query_with_cumulative_metric_in_where_filter__plan0.sql b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Redshift/test_query_with_cumulative_metric_in_where_filter__plan0.sql index 55ff637b8f..2ff47a9733 100644 --- a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Redshift/test_query_with_cumulative_metric_in_where_filter__plan0.sql +++ b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Redshift/test_query_with_cumulative_metric_in_where_filter__plan0.sql @@ -22,8 +22,8 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.user AS user - , subq_8.user__revenue_all_time AS user__revenue_all_time + subq_8.user__revenue_all_time AS user__revenue_all_time + , subq_2.user AS user , subq_2.listings AS listings FROM ( -- Pass Only Elements: ['listings', 'user'] diff --git a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Redshift/test_query_with_derived_metric_in_where_filter__plan0.sql b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Redshift/test_query_with_derived_metric_in_where_filter__plan0.sql index 670d846141..e4fa9aadce 100644 --- a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Redshift/test_query_with_derived_metric_in_where_filter__plan0.sql +++ b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Redshift/test_query_with_derived_metric_in_where_filter__plan0.sql @@ -22,8 +22,8 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.listing AS listing - , subq_15.listing__views_times_booking_value AS listing__views_times_booking_value + subq_15.listing__views_times_booking_value AS listing__views_times_booking_value + , subq_2.listing AS listing , subq_2.listings AS listings FROM ( -- Pass Only Elements: ['listings', 'listing'] diff --git a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Redshift/test_query_with_multiple_metrics_in_filter__plan0.sql b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Redshift/test_query_with_multiple_metrics_in_filter__plan0.sql index 5e03e7bb6b..e2d1dad354 100644 --- a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Redshift/test_query_with_multiple_metrics_in_filter__plan0.sql +++ b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Redshift/test_query_with_multiple_metrics_in_filter__plan0.sql @@ -24,9 +24,9 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.listing AS listing - , subq_8.listing__bookings AS listing__bookings + subq_8.listing__bookings AS listing__bookings , subq_14.listing__bookers AS listing__bookers + , subq_2.listing AS listing , subq_2.listings AS listings FROM ( -- Pass Only Elements: ['listings', 'listing', 'listing'] diff --git a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Redshift/test_query_with_ratio_metric_in_where_filter__plan0.sql b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Redshift/test_query_with_ratio_metric_in_where_filter__plan0.sql index 9d9e2daf1b..3efd088ba3 100644 --- a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Redshift/test_query_with_ratio_metric_in_where_filter__plan0.sql +++ b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Redshift/test_query_with_ratio_metric_in_where_filter__plan0.sql @@ -22,8 +22,8 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.listing AS listing - , subq_15.listing__bookings_per_booker AS listing__bookings_per_booker + subq_15.listing__bookings_per_booker AS listing__bookings_per_booker + , subq_2.listing AS listing , subq_2.listings AS listings FROM ( -- Pass Only Elements: ['listings', 'listing'] diff --git a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Redshift/test_query_with_simple_metric_in_where_filter__plan0.sql b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Redshift/test_query_with_simple_metric_in_where_filter__plan0.sql index 4fadfb17c8..a60ecf02a8 100644 --- a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Redshift/test_query_with_simple_metric_in_where_filter__plan0.sql +++ b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Redshift/test_query_with_simple_metric_in_where_filter__plan0.sql @@ -22,8 +22,8 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.listing AS listing - , subq_8.listing__bookings AS listing__bookings + subq_8.listing__bookings AS listing__bookings + , subq_2.listing AS listing , subq_2.listings AS listings FROM ( -- Pass Only Elements: ['listings', 'listing'] diff --git a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Snowflake/test_distinct_values_query_with_metric_filter__plan0.sql b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Snowflake/test_distinct_values_query_with_metric_filter__plan0.sql index 5c91195c37..105fc0d592 100644 --- a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Snowflake/test_distinct_values_query_with_metric_filter__plan0.sql +++ b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Snowflake/test_distinct_values_query_with_metric_filter__plan0.sql @@ -11,10 +11,10 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_0.listing AS listing + subq_6.listing__bookings AS listing__bookings + , subq_0.listing AS listing , subq_0.lux_listing AS lux_listing , subq_0.listing__lux_listing AS listing__lux_listing - , subq_6.listing__bookings AS listing__bookings FROM ( -- Read Elements From Semantic Model 'lux_listing_mapping' SELECT diff --git a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Snowflake/test_distinct_values_query_with_metric_filter__plan0_optimized.sql b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Snowflake/test_distinct_values_query_with_metric_filter__plan0_optimized.sql index d79ef0aa09..c51da11fab 100644 --- a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Snowflake/test_distinct_values_query_with_metric_filter__plan0_optimized.sql +++ b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Snowflake/test_distinct_values_query_with_metric_filter__plan0_optimized.sql @@ -5,8 +5,8 @@ SELECT FROM ( -- Join Standard Outputs SELECT - lux_listing_mapping_src_28000.listing_id AS listing - , subq_15.listing__bookings AS listing__bookings + subq_15.listing__bookings AS listing__bookings + , lux_listing_mapping_src_28000.listing_id AS listing FROM ***************************.dim_lux_listing_id_mapping lux_listing_mapping_src_28000 FULL OUTER JOIN ( -- Aggregate Measures diff --git a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Snowflake/test_filter_by_metric_in_same_semantic_model_as_queried_metric__plan0.sql b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Snowflake/test_filter_by_metric_in_same_semantic_model_as_queried_metric__plan0.sql index 34d6221739..a81183bee9 100644 --- a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Snowflake/test_filter_by_metric_in_same_semantic_model_as_queried_metric__plan0.sql +++ b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Snowflake/test_filter_by_metric_in_same_semantic_model_as_queried_metric__plan0.sql @@ -22,8 +22,8 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.guest AS guest - , subq_8.guest__booking_value AS guest__booking_value + subq_8.guest__booking_value AS guest__booking_value + , subq_2.guest AS guest , subq_2.bookers AS bookers FROM ( -- Pass Only Elements: ['bookers', 'guest'] diff --git a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Snowflake/test_filter_with_conversion_metric__plan0.sql b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Snowflake/test_filter_with_conversion_metric__plan0.sql index c336568c52..bc498aa0c6 100644 --- a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Snowflake/test_filter_with_conversion_metric__plan0.sql +++ b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Snowflake/test_filter_with_conversion_metric__plan0.sql @@ -22,8 +22,8 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.user AS user - , subq_19.user__visit_buy_conversion_rate AS user__visit_buy_conversion_rate + subq_19.user__visit_buy_conversion_rate AS user__visit_buy_conversion_rate + , subq_2.user AS user , subq_2.listings AS listings FROM ( -- Pass Only Elements: ['listings', 'user'] diff --git a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Snowflake/test_group_by_has_local_entity_prefix__plan0.sql b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Snowflake/test_group_by_has_local_entity_prefix__plan0.sql index b72141800b..b07fd00394 100644 --- a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Snowflake/test_group_by_has_local_entity_prefix__plan0.sql +++ b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Snowflake/test_group_by_has_local_entity_prefix__plan0.sql @@ -22,9 +22,9 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.user AS user - , subq_13.listing__user AS user__listing__user + subq_13.listing__user AS user__listing__user , subq_13.listing__user__average_booking_value AS user__listing__user__average_booking_value + , subq_2.user AS user , subq_2.listings AS listings FROM ( -- Pass Only Elements: ['listings', 'user'] @@ -187,8 +187,8 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_5.listing AS listing - , subq_8.user AS listing__user + subq_8.user AS listing__user + , subq_5.listing AS listing , subq_5.average_booking_value AS average_booking_value FROM ( -- Pass Only Elements: ['average_booking_value', 'listing'] diff --git a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Snowflake/test_inner_query_multi_hop__plan0.sql b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Snowflake/test_inner_query_multi_hop__plan0.sql index 57cfc5f054..7a335a0428 100644 --- a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Snowflake/test_inner_query_multi_hop__plan0.sql +++ b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Snowflake/test_inner_query_multi_hop__plan0.sql @@ -22,9 +22,9 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.customer_third_hop_id AS customer_third_hop_id - , subq_17.account_id__customer_id__customer_third_hop_id AS customer_third_hop_id__account_id__customer_id__customer_third_hop_id + subq_17.account_id__customer_id__customer_third_hop_id AS customer_third_hop_id__account_id__customer_id__customer_third_hop_id , subq_17.account_id__customer_id__customer_third_hop_id__txn_count AS customer_third_hop_id__account_id__customer_id__customer_third_hop_id__txn_count + , subq_2.customer_third_hop_id AS customer_third_hop_id , subq_2.third_hop_count AS third_hop_count FROM ( -- Pass Only Elements: ['third_hop_count', 'customer_third_hop_id'] @@ -127,10 +127,10 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_5.ds_partitioned__day AS ds_partitioned__day - , subq_12.ds_partitioned__day AS account_id__ds_partitioned__day - , subq_5.account_id AS account_id + subq_12.ds_partitioned__day AS account_id__ds_partitioned__day , subq_12.customer_id__customer_third_hop_id AS account_id__customer_id__customer_third_hop_id + , subq_5.ds_partitioned__day AS ds_partitioned__day + , subq_5.account_id AS account_id , subq_5.txn_count AS txn_count FROM ( -- Pass Only Elements: ['txn_count', 'ds_partitioned__day', 'account_id'] @@ -264,7 +264,44 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_7.ds_partitioned__day AS ds_partitioned__day + subq_10.country AS customer_id__country + , subq_10.customer_third_hop_id__country AS customer_id__customer_third_hop_id__country + , subq_10.acquired_ds__day AS customer_id__acquired_ds__day + , subq_10.acquired_ds__week AS customer_id__acquired_ds__week + , subq_10.acquired_ds__month AS customer_id__acquired_ds__month + , subq_10.acquired_ds__quarter AS customer_id__acquired_ds__quarter + , subq_10.acquired_ds__year AS customer_id__acquired_ds__year + , subq_10.acquired_ds__extract_year AS customer_id__acquired_ds__extract_year + , subq_10.acquired_ds__extract_quarter AS customer_id__acquired_ds__extract_quarter + , subq_10.acquired_ds__extract_month AS customer_id__acquired_ds__extract_month + , subq_10.acquired_ds__extract_day AS customer_id__acquired_ds__extract_day + , subq_10.acquired_ds__extract_dow AS customer_id__acquired_ds__extract_dow + , subq_10.acquired_ds__extract_doy AS customer_id__acquired_ds__extract_doy + , subq_10.customer_third_hop_id__acquired_ds__day AS customer_id__customer_third_hop_id__acquired_ds__day + , subq_10.customer_third_hop_id__acquired_ds__week AS customer_id__customer_third_hop_id__acquired_ds__week + , subq_10.customer_third_hop_id__acquired_ds__month AS customer_id__customer_third_hop_id__acquired_ds__month + , subq_10.customer_third_hop_id__acquired_ds__quarter AS customer_id__customer_third_hop_id__acquired_ds__quarter + , subq_10.customer_third_hop_id__acquired_ds__year AS customer_id__customer_third_hop_id__acquired_ds__year + , subq_10.customer_third_hop_id__acquired_ds__extract_year AS customer_id__customer_third_hop_id__acquired_ds__extract_year + , subq_10.customer_third_hop_id__acquired_ds__extract_quarter AS customer_id__customer_third_hop_id__acquired_ds__extract_quarter + , subq_10.customer_third_hop_id__acquired_ds__extract_month AS customer_id__customer_third_hop_id__acquired_ds__extract_month + , subq_10.customer_third_hop_id__acquired_ds__extract_day AS customer_id__customer_third_hop_id__acquired_ds__extract_day + , subq_10.customer_third_hop_id__acquired_ds__extract_dow AS customer_id__customer_third_hop_id__acquired_ds__extract_dow + , subq_10.customer_third_hop_id__acquired_ds__extract_doy AS customer_id__customer_third_hop_id__acquired_ds__extract_doy + , subq_10.metric_time__day AS customer_id__metric_time__day + , subq_10.metric_time__week AS customer_id__metric_time__week + , subq_10.metric_time__month AS customer_id__metric_time__month + , subq_10.metric_time__quarter AS customer_id__metric_time__quarter + , subq_10.metric_time__year AS customer_id__metric_time__year + , subq_10.metric_time__extract_year AS customer_id__metric_time__extract_year + , subq_10.metric_time__extract_quarter AS customer_id__metric_time__extract_quarter + , subq_10.metric_time__extract_month AS customer_id__metric_time__extract_month + , subq_10.metric_time__extract_day AS customer_id__metric_time__extract_day + , subq_10.metric_time__extract_dow AS customer_id__metric_time__extract_dow + , subq_10.metric_time__extract_doy AS customer_id__metric_time__extract_doy + , subq_10.customer_third_hop_id AS customer_id__customer_third_hop_id + , subq_10.customer_third_hop_id__customer_id AS customer_id__customer_third_hop_id__customer_id + , subq_7.ds_partitioned__day AS ds_partitioned__day , subq_7.ds_partitioned__week AS ds_partitioned__week , subq_7.ds_partitioned__month AS ds_partitioned__month , subq_7.ds_partitioned__quarter AS ds_partitioned__quarter @@ -308,51 +345,14 @@ FROM ( , subq_7.metric_time__extract_day AS metric_time__extract_day , subq_7.metric_time__extract_dow AS metric_time__extract_dow , subq_7.metric_time__extract_doy AS metric_time__extract_doy - , subq_10.acquired_ds__day AS customer_id__acquired_ds__day - , subq_10.acquired_ds__week AS customer_id__acquired_ds__week - , subq_10.acquired_ds__month AS customer_id__acquired_ds__month - , subq_10.acquired_ds__quarter AS customer_id__acquired_ds__quarter - , subq_10.acquired_ds__year AS customer_id__acquired_ds__year - , subq_10.acquired_ds__extract_year AS customer_id__acquired_ds__extract_year - , subq_10.acquired_ds__extract_quarter AS customer_id__acquired_ds__extract_quarter - , subq_10.acquired_ds__extract_month AS customer_id__acquired_ds__extract_month - , subq_10.acquired_ds__extract_day AS customer_id__acquired_ds__extract_day - , subq_10.acquired_ds__extract_dow AS customer_id__acquired_ds__extract_dow - , subq_10.acquired_ds__extract_doy AS customer_id__acquired_ds__extract_doy - , subq_10.customer_third_hop_id__acquired_ds__day AS customer_id__customer_third_hop_id__acquired_ds__day - , subq_10.customer_third_hop_id__acquired_ds__week AS customer_id__customer_third_hop_id__acquired_ds__week - , subq_10.customer_third_hop_id__acquired_ds__month AS customer_id__customer_third_hop_id__acquired_ds__month - , subq_10.customer_third_hop_id__acquired_ds__quarter AS customer_id__customer_third_hop_id__acquired_ds__quarter - , subq_10.customer_third_hop_id__acquired_ds__year AS customer_id__customer_third_hop_id__acquired_ds__year - , subq_10.customer_third_hop_id__acquired_ds__extract_year AS customer_id__customer_third_hop_id__acquired_ds__extract_year - , subq_10.customer_third_hop_id__acquired_ds__extract_quarter AS customer_id__customer_third_hop_id__acquired_ds__extract_quarter - , subq_10.customer_third_hop_id__acquired_ds__extract_month AS customer_id__customer_third_hop_id__acquired_ds__extract_month - , subq_10.customer_third_hop_id__acquired_ds__extract_day AS customer_id__customer_third_hop_id__acquired_ds__extract_day - , subq_10.customer_third_hop_id__acquired_ds__extract_dow AS customer_id__customer_third_hop_id__acquired_ds__extract_dow - , subq_10.customer_third_hop_id__acquired_ds__extract_doy AS customer_id__customer_third_hop_id__acquired_ds__extract_doy - , subq_10.metric_time__day AS customer_id__metric_time__day - , subq_10.metric_time__week AS customer_id__metric_time__week - , subq_10.metric_time__month AS customer_id__metric_time__month - , subq_10.metric_time__quarter AS customer_id__metric_time__quarter - , subq_10.metric_time__year AS customer_id__metric_time__year - , subq_10.metric_time__extract_year AS customer_id__metric_time__extract_year - , subq_10.metric_time__extract_quarter AS customer_id__metric_time__extract_quarter - , subq_10.metric_time__extract_month AS customer_id__metric_time__extract_month - , subq_10.metric_time__extract_day AS customer_id__metric_time__extract_day - , subq_10.metric_time__extract_dow AS customer_id__metric_time__extract_dow - , subq_10.metric_time__extract_doy AS customer_id__metric_time__extract_doy , subq_7.account_id AS account_id , subq_7.customer_id AS customer_id , subq_7.account_id__customer_id AS account_id__customer_id , subq_7.bridge_account__account_id AS bridge_account__account_id , subq_7.bridge_account__customer_id AS bridge_account__customer_id - , subq_10.customer_third_hop_id AS customer_id__customer_third_hop_id - , subq_10.customer_third_hop_id__customer_id AS customer_id__customer_third_hop_id__customer_id , subq_7.extra_dim AS extra_dim , subq_7.account_id__extra_dim AS account_id__extra_dim , subq_7.bridge_account__extra_dim AS bridge_account__extra_dim - , subq_10.country AS customer_id__country - , subq_10.customer_third_hop_id__country AS customer_id__customer_third_hop_id__country , subq_7.account_customer_combos AS account_customer_combos FROM ( -- Metric Time Dimension 'ds_partitioned' diff --git a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Snowflake/test_inner_query_single_hop__plan0.sql b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Snowflake/test_inner_query_single_hop__plan0.sql index 9f1cb0c597..50b5010e98 100644 --- a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Snowflake/test_inner_query_single_hop__plan0.sql +++ b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Snowflake/test_inner_query_single_hop__plan0.sql @@ -22,9 +22,9 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.customer_third_hop_id AS customer_third_hop_id - , subq_10.customer_id__customer_third_hop_id AS customer_third_hop_id__customer_id__customer_third_hop_id + subq_10.customer_id__customer_third_hop_id AS customer_third_hop_id__customer_id__customer_third_hop_id , subq_10.customer_id__customer_third_hop_id__paraguayan_customers AS customer_third_hop_id__customer_id__customer_third_hop_id__paraguayan_customers + , subq_2.customer_third_hop_id AS customer_third_hop_id , subq_2.third_hop_count AS third_hop_count FROM ( -- Pass Only Elements: ['third_hop_count', 'customer_third_hop_id'] diff --git a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Snowflake/test_metric_filtered_by_itself__plan0.sql b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Snowflake/test_metric_filtered_by_itself__plan0.sql index f96226cbe1..f00d017f9a 100644 --- a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Snowflake/test_metric_filtered_by_itself__plan0.sql +++ b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Snowflake/test_metric_filtered_by_itself__plan0.sql @@ -22,8 +22,8 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.listing AS listing - , subq_8.listing__bookers AS listing__bookers + subq_8.listing__bookers AS listing__bookers + , subq_2.listing AS listing , subq_2.bookers AS bookers FROM ( -- Pass Only Elements: ['bookers', 'listing'] diff --git a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Snowflake/test_metric_with_metric_in_where_filter__plan0.sql b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Snowflake/test_metric_with_metric_in_where_filter__plan0.sql index 5123357f47..8f424b5bee 100644 --- a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Snowflake/test_metric_with_metric_in_where_filter__plan0.sql +++ b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Snowflake/test_metric_with_metric_in_where_filter__plan0.sql @@ -27,9 +27,9 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.metric_time__day AS metric_time__day + subq_8.listing__bookings AS listing__bookings + , subq_2.metric_time__day AS metric_time__day , subq_2.listing AS listing - , subq_8.listing__bookings AS listing__bookings , subq_2.listings AS listings FROM ( -- Pass Only Elements: ['listings', 'metric_time__day', 'listing'] diff --git a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Snowflake/test_query_with_cumulative_metric_in_where_filter__plan0.sql b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Snowflake/test_query_with_cumulative_metric_in_where_filter__plan0.sql index 96442754af..1379c1c1c9 100644 --- a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Snowflake/test_query_with_cumulative_metric_in_where_filter__plan0.sql +++ b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Snowflake/test_query_with_cumulative_metric_in_where_filter__plan0.sql @@ -22,8 +22,8 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.user AS user - , subq_8.user__revenue_all_time AS user__revenue_all_time + subq_8.user__revenue_all_time AS user__revenue_all_time + , subq_2.user AS user , subq_2.listings AS listings FROM ( -- Pass Only Elements: ['listings', 'user'] diff --git a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Snowflake/test_query_with_derived_metric_in_where_filter__plan0.sql b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Snowflake/test_query_with_derived_metric_in_where_filter__plan0.sql index ab98e985f0..e96c04b505 100644 --- a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Snowflake/test_query_with_derived_metric_in_where_filter__plan0.sql +++ b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Snowflake/test_query_with_derived_metric_in_where_filter__plan0.sql @@ -22,8 +22,8 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.listing AS listing - , subq_15.listing__views_times_booking_value AS listing__views_times_booking_value + subq_15.listing__views_times_booking_value AS listing__views_times_booking_value + , subq_2.listing AS listing , subq_2.listings AS listings FROM ( -- Pass Only Elements: ['listings', 'listing'] diff --git a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Snowflake/test_query_with_multiple_metrics_in_filter__plan0.sql b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Snowflake/test_query_with_multiple_metrics_in_filter__plan0.sql index 5c032234bd..84eafa76d9 100644 --- a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Snowflake/test_query_with_multiple_metrics_in_filter__plan0.sql +++ b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Snowflake/test_query_with_multiple_metrics_in_filter__plan0.sql @@ -24,9 +24,9 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.listing AS listing - , subq_8.listing__bookings AS listing__bookings + subq_8.listing__bookings AS listing__bookings , subq_14.listing__bookers AS listing__bookers + , subq_2.listing AS listing , subq_2.listings AS listings FROM ( -- Pass Only Elements: ['listings', 'listing', 'listing'] diff --git a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Snowflake/test_query_with_ratio_metric_in_where_filter__plan0.sql b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Snowflake/test_query_with_ratio_metric_in_where_filter__plan0.sql index 559ca2799b..406289ee16 100644 --- a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Snowflake/test_query_with_ratio_metric_in_where_filter__plan0.sql +++ b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Snowflake/test_query_with_ratio_metric_in_where_filter__plan0.sql @@ -22,8 +22,8 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.listing AS listing - , subq_15.listing__bookings_per_booker AS listing__bookings_per_booker + subq_15.listing__bookings_per_booker AS listing__bookings_per_booker + , subq_2.listing AS listing , subq_2.listings AS listings FROM ( -- Pass Only Elements: ['listings', 'listing'] diff --git a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Snowflake/test_query_with_simple_metric_in_where_filter__plan0.sql b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Snowflake/test_query_with_simple_metric_in_where_filter__plan0.sql index 2dec46f789..53f7e6dfa0 100644 --- a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Snowflake/test_query_with_simple_metric_in_where_filter__plan0.sql +++ b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Snowflake/test_query_with_simple_metric_in_where_filter__plan0.sql @@ -22,8 +22,8 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.listing AS listing - , subq_8.listing__bookings AS listing__bookings + subq_8.listing__bookings AS listing__bookings + , subq_2.listing AS listing , subq_2.listings AS listings FROM ( -- Pass Only Elements: ['listings', 'listing'] diff --git a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Trino/test_distinct_values_query_with_metric_filter__plan0.sql b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Trino/test_distinct_values_query_with_metric_filter__plan0.sql index dbbd3a25db..a98d339aba 100644 --- a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Trino/test_distinct_values_query_with_metric_filter__plan0.sql +++ b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Trino/test_distinct_values_query_with_metric_filter__plan0.sql @@ -11,10 +11,10 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_0.listing AS listing + subq_6.listing__bookings AS listing__bookings + , subq_0.listing AS listing , subq_0.lux_listing AS lux_listing , subq_0.listing__lux_listing AS listing__lux_listing - , subq_6.listing__bookings AS listing__bookings FROM ( -- Read Elements From Semantic Model 'lux_listing_mapping' SELECT diff --git a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Trino/test_distinct_values_query_with_metric_filter__plan0_optimized.sql b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Trino/test_distinct_values_query_with_metric_filter__plan0_optimized.sql index d79ef0aa09..c51da11fab 100644 --- a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Trino/test_distinct_values_query_with_metric_filter__plan0_optimized.sql +++ b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Trino/test_distinct_values_query_with_metric_filter__plan0_optimized.sql @@ -5,8 +5,8 @@ SELECT FROM ( -- Join Standard Outputs SELECT - lux_listing_mapping_src_28000.listing_id AS listing - , subq_15.listing__bookings AS listing__bookings + subq_15.listing__bookings AS listing__bookings + , lux_listing_mapping_src_28000.listing_id AS listing FROM ***************************.dim_lux_listing_id_mapping lux_listing_mapping_src_28000 FULL OUTER JOIN ( -- Aggregate Measures diff --git a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Trino/test_filter_by_metric_in_same_semantic_model_as_queried_metric__plan0.sql b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Trino/test_filter_by_metric_in_same_semantic_model_as_queried_metric__plan0.sql index 65c15f1222..3234da955f 100644 --- a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Trino/test_filter_by_metric_in_same_semantic_model_as_queried_metric__plan0.sql +++ b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Trino/test_filter_by_metric_in_same_semantic_model_as_queried_metric__plan0.sql @@ -22,8 +22,8 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.guest AS guest - , subq_8.guest__booking_value AS guest__booking_value + subq_8.guest__booking_value AS guest__booking_value + , subq_2.guest AS guest , subq_2.bookers AS bookers FROM ( -- Pass Only Elements: ['bookers', 'guest'] diff --git a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Trino/test_filter_with_conversion_metric__plan0.sql b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Trino/test_filter_with_conversion_metric__plan0.sql index ae3eae985b..c1da875146 100644 --- a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Trino/test_filter_with_conversion_metric__plan0.sql +++ b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Trino/test_filter_with_conversion_metric__plan0.sql @@ -22,8 +22,8 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.user AS user - , subq_19.user__visit_buy_conversion_rate AS user__visit_buy_conversion_rate + subq_19.user__visit_buy_conversion_rate AS user__visit_buy_conversion_rate + , subq_2.user AS user , subq_2.listings AS listings FROM ( -- Pass Only Elements: ['listings', 'user'] diff --git a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Trino/test_group_by_has_local_entity_prefix__plan0.sql b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Trino/test_group_by_has_local_entity_prefix__plan0.sql index 3bf96a545f..34aa606047 100644 --- a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Trino/test_group_by_has_local_entity_prefix__plan0.sql +++ b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Trino/test_group_by_has_local_entity_prefix__plan0.sql @@ -22,9 +22,9 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.user AS user - , subq_13.listing__user AS user__listing__user + subq_13.listing__user AS user__listing__user , subq_13.listing__user__average_booking_value AS user__listing__user__average_booking_value + , subq_2.user AS user , subq_2.listings AS listings FROM ( -- Pass Only Elements: ['listings', 'user'] @@ -187,8 +187,8 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_5.listing AS listing - , subq_8.user AS listing__user + subq_8.user AS listing__user + , subq_5.listing AS listing , subq_5.average_booking_value AS average_booking_value FROM ( -- Pass Only Elements: ['average_booking_value', 'listing'] diff --git a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Trino/test_inner_query_multi_hop__plan0.sql b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Trino/test_inner_query_multi_hop__plan0.sql index 0ac729d058..7221218118 100644 --- a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Trino/test_inner_query_multi_hop__plan0.sql +++ b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Trino/test_inner_query_multi_hop__plan0.sql @@ -22,9 +22,9 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.customer_third_hop_id AS customer_third_hop_id - , subq_17.account_id__customer_id__customer_third_hop_id AS customer_third_hop_id__account_id__customer_id__customer_third_hop_id + subq_17.account_id__customer_id__customer_third_hop_id AS customer_third_hop_id__account_id__customer_id__customer_third_hop_id , subq_17.account_id__customer_id__customer_third_hop_id__txn_count AS customer_third_hop_id__account_id__customer_id__customer_third_hop_id__txn_count + , subq_2.customer_third_hop_id AS customer_third_hop_id , subq_2.third_hop_count AS third_hop_count FROM ( -- Pass Only Elements: ['third_hop_count', 'customer_third_hop_id'] @@ -127,10 +127,10 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_5.ds_partitioned__day AS ds_partitioned__day - , subq_12.ds_partitioned__day AS account_id__ds_partitioned__day - , subq_5.account_id AS account_id + subq_12.ds_partitioned__day AS account_id__ds_partitioned__day , subq_12.customer_id__customer_third_hop_id AS account_id__customer_id__customer_third_hop_id + , subq_5.ds_partitioned__day AS ds_partitioned__day + , subq_5.account_id AS account_id , subq_5.txn_count AS txn_count FROM ( -- Pass Only Elements: ['txn_count', 'ds_partitioned__day', 'account_id'] @@ -264,7 +264,44 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_7.ds_partitioned__day AS ds_partitioned__day + subq_10.country AS customer_id__country + , subq_10.customer_third_hop_id__country AS customer_id__customer_third_hop_id__country + , subq_10.acquired_ds__day AS customer_id__acquired_ds__day + , subq_10.acquired_ds__week AS customer_id__acquired_ds__week + , subq_10.acquired_ds__month AS customer_id__acquired_ds__month + , subq_10.acquired_ds__quarter AS customer_id__acquired_ds__quarter + , subq_10.acquired_ds__year AS customer_id__acquired_ds__year + , subq_10.acquired_ds__extract_year AS customer_id__acquired_ds__extract_year + , subq_10.acquired_ds__extract_quarter AS customer_id__acquired_ds__extract_quarter + , subq_10.acquired_ds__extract_month AS customer_id__acquired_ds__extract_month + , subq_10.acquired_ds__extract_day AS customer_id__acquired_ds__extract_day + , subq_10.acquired_ds__extract_dow AS customer_id__acquired_ds__extract_dow + , subq_10.acquired_ds__extract_doy AS customer_id__acquired_ds__extract_doy + , subq_10.customer_third_hop_id__acquired_ds__day AS customer_id__customer_third_hop_id__acquired_ds__day + , subq_10.customer_third_hop_id__acquired_ds__week AS customer_id__customer_third_hop_id__acquired_ds__week + , subq_10.customer_third_hop_id__acquired_ds__month AS customer_id__customer_third_hop_id__acquired_ds__month + , subq_10.customer_third_hop_id__acquired_ds__quarter AS customer_id__customer_third_hop_id__acquired_ds__quarter + , subq_10.customer_third_hop_id__acquired_ds__year AS customer_id__customer_third_hop_id__acquired_ds__year + , subq_10.customer_third_hop_id__acquired_ds__extract_year AS customer_id__customer_third_hop_id__acquired_ds__extract_year + , subq_10.customer_third_hop_id__acquired_ds__extract_quarter AS customer_id__customer_third_hop_id__acquired_ds__extract_quarter + , subq_10.customer_third_hop_id__acquired_ds__extract_month AS customer_id__customer_third_hop_id__acquired_ds__extract_month + , subq_10.customer_third_hop_id__acquired_ds__extract_day AS customer_id__customer_third_hop_id__acquired_ds__extract_day + , subq_10.customer_third_hop_id__acquired_ds__extract_dow AS customer_id__customer_third_hop_id__acquired_ds__extract_dow + , subq_10.customer_third_hop_id__acquired_ds__extract_doy AS customer_id__customer_third_hop_id__acquired_ds__extract_doy + , subq_10.metric_time__day AS customer_id__metric_time__day + , subq_10.metric_time__week AS customer_id__metric_time__week + , subq_10.metric_time__month AS customer_id__metric_time__month + , subq_10.metric_time__quarter AS customer_id__metric_time__quarter + , subq_10.metric_time__year AS customer_id__metric_time__year + , subq_10.metric_time__extract_year AS customer_id__metric_time__extract_year + , subq_10.metric_time__extract_quarter AS customer_id__metric_time__extract_quarter + , subq_10.metric_time__extract_month AS customer_id__metric_time__extract_month + , subq_10.metric_time__extract_day AS customer_id__metric_time__extract_day + , subq_10.metric_time__extract_dow AS customer_id__metric_time__extract_dow + , subq_10.metric_time__extract_doy AS customer_id__metric_time__extract_doy + , subq_10.customer_third_hop_id AS customer_id__customer_third_hop_id + , subq_10.customer_third_hop_id__customer_id AS customer_id__customer_third_hop_id__customer_id + , subq_7.ds_partitioned__day AS ds_partitioned__day , subq_7.ds_partitioned__week AS ds_partitioned__week , subq_7.ds_partitioned__month AS ds_partitioned__month , subq_7.ds_partitioned__quarter AS ds_partitioned__quarter @@ -308,51 +345,14 @@ FROM ( , subq_7.metric_time__extract_day AS metric_time__extract_day , subq_7.metric_time__extract_dow AS metric_time__extract_dow , subq_7.metric_time__extract_doy AS metric_time__extract_doy - , subq_10.acquired_ds__day AS customer_id__acquired_ds__day - , subq_10.acquired_ds__week AS customer_id__acquired_ds__week - , subq_10.acquired_ds__month AS customer_id__acquired_ds__month - , subq_10.acquired_ds__quarter AS customer_id__acquired_ds__quarter - , subq_10.acquired_ds__year AS customer_id__acquired_ds__year - , subq_10.acquired_ds__extract_year AS customer_id__acquired_ds__extract_year - , subq_10.acquired_ds__extract_quarter AS customer_id__acquired_ds__extract_quarter - , subq_10.acquired_ds__extract_month AS customer_id__acquired_ds__extract_month - , subq_10.acquired_ds__extract_day AS customer_id__acquired_ds__extract_day - , subq_10.acquired_ds__extract_dow AS customer_id__acquired_ds__extract_dow - , subq_10.acquired_ds__extract_doy AS customer_id__acquired_ds__extract_doy - , subq_10.customer_third_hop_id__acquired_ds__day AS customer_id__customer_third_hop_id__acquired_ds__day - , subq_10.customer_third_hop_id__acquired_ds__week AS customer_id__customer_third_hop_id__acquired_ds__week - , subq_10.customer_third_hop_id__acquired_ds__month AS customer_id__customer_third_hop_id__acquired_ds__month - , subq_10.customer_third_hop_id__acquired_ds__quarter AS customer_id__customer_third_hop_id__acquired_ds__quarter - , subq_10.customer_third_hop_id__acquired_ds__year AS customer_id__customer_third_hop_id__acquired_ds__year - , subq_10.customer_third_hop_id__acquired_ds__extract_year AS customer_id__customer_third_hop_id__acquired_ds__extract_year - , subq_10.customer_third_hop_id__acquired_ds__extract_quarter AS customer_id__customer_third_hop_id__acquired_ds__extract_quarter - , subq_10.customer_third_hop_id__acquired_ds__extract_month AS customer_id__customer_third_hop_id__acquired_ds__extract_month - , subq_10.customer_third_hop_id__acquired_ds__extract_day AS customer_id__customer_third_hop_id__acquired_ds__extract_day - , subq_10.customer_third_hop_id__acquired_ds__extract_dow AS customer_id__customer_third_hop_id__acquired_ds__extract_dow - , subq_10.customer_third_hop_id__acquired_ds__extract_doy AS customer_id__customer_third_hop_id__acquired_ds__extract_doy - , subq_10.metric_time__day AS customer_id__metric_time__day - , subq_10.metric_time__week AS customer_id__metric_time__week - , subq_10.metric_time__month AS customer_id__metric_time__month - , subq_10.metric_time__quarter AS customer_id__metric_time__quarter - , subq_10.metric_time__year AS customer_id__metric_time__year - , subq_10.metric_time__extract_year AS customer_id__metric_time__extract_year - , subq_10.metric_time__extract_quarter AS customer_id__metric_time__extract_quarter - , subq_10.metric_time__extract_month AS customer_id__metric_time__extract_month - , subq_10.metric_time__extract_day AS customer_id__metric_time__extract_day - , subq_10.metric_time__extract_dow AS customer_id__metric_time__extract_dow - , subq_10.metric_time__extract_doy AS customer_id__metric_time__extract_doy , subq_7.account_id AS account_id , subq_7.customer_id AS customer_id , subq_7.account_id__customer_id AS account_id__customer_id , subq_7.bridge_account__account_id AS bridge_account__account_id , subq_7.bridge_account__customer_id AS bridge_account__customer_id - , subq_10.customer_third_hop_id AS customer_id__customer_third_hop_id - , subq_10.customer_third_hop_id__customer_id AS customer_id__customer_third_hop_id__customer_id , subq_7.extra_dim AS extra_dim , subq_7.account_id__extra_dim AS account_id__extra_dim , subq_7.bridge_account__extra_dim AS bridge_account__extra_dim - , subq_10.country AS customer_id__country - , subq_10.customer_third_hop_id__country AS customer_id__customer_third_hop_id__country , subq_7.account_customer_combos AS account_customer_combos FROM ( -- Metric Time Dimension 'ds_partitioned' diff --git a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Trino/test_inner_query_single_hop__plan0.sql b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Trino/test_inner_query_single_hop__plan0.sql index 5267b14e69..ad3df23fab 100644 --- a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Trino/test_inner_query_single_hop__plan0.sql +++ b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Trino/test_inner_query_single_hop__plan0.sql @@ -22,9 +22,9 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.customer_third_hop_id AS customer_third_hop_id - , subq_10.customer_id__customer_third_hop_id AS customer_third_hop_id__customer_id__customer_third_hop_id + subq_10.customer_id__customer_third_hop_id AS customer_third_hop_id__customer_id__customer_third_hop_id , subq_10.customer_id__customer_third_hop_id__paraguayan_customers AS customer_third_hop_id__customer_id__customer_third_hop_id__paraguayan_customers + , subq_2.customer_third_hop_id AS customer_third_hop_id , subq_2.third_hop_count AS third_hop_count FROM ( -- Pass Only Elements: ['third_hop_count', 'customer_third_hop_id'] diff --git a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Trino/test_metric_filtered_by_itself__plan0.sql b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Trino/test_metric_filtered_by_itself__plan0.sql index d0dc709959..8a5e43d716 100644 --- a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Trino/test_metric_filtered_by_itself__plan0.sql +++ b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Trino/test_metric_filtered_by_itself__plan0.sql @@ -22,8 +22,8 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.listing AS listing - , subq_8.listing__bookers AS listing__bookers + subq_8.listing__bookers AS listing__bookers + , subq_2.listing AS listing , subq_2.bookers AS bookers FROM ( -- Pass Only Elements: ['bookers', 'listing'] diff --git a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Trino/test_metric_with_metric_in_where_filter__plan0.sql b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Trino/test_metric_with_metric_in_where_filter__plan0.sql index 2af39223a4..d559ec3bc4 100644 --- a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Trino/test_metric_with_metric_in_where_filter__plan0.sql +++ b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Trino/test_metric_with_metric_in_where_filter__plan0.sql @@ -27,9 +27,9 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.metric_time__day AS metric_time__day + subq_8.listing__bookings AS listing__bookings + , subq_2.metric_time__day AS metric_time__day , subq_2.listing AS listing - , subq_8.listing__bookings AS listing__bookings , subq_2.listings AS listings FROM ( -- Pass Only Elements: ['listings', 'metric_time__day', 'listing'] diff --git a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Trino/test_query_with_cumulative_metric_in_where_filter__plan0.sql b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Trino/test_query_with_cumulative_metric_in_where_filter__plan0.sql index 60cff44870..3a2c363bc9 100644 --- a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Trino/test_query_with_cumulative_metric_in_where_filter__plan0.sql +++ b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Trino/test_query_with_cumulative_metric_in_where_filter__plan0.sql @@ -22,8 +22,8 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.user AS user - , subq_8.user__revenue_all_time AS user__revenue_all_time + subq_8.user__revenue_all_time AS user__revenue_all_time + , subq_2.user AS user , subq_2.listings AS listings FROM ( -- Pass Only Elements: ['listings', 'user'] diff --git a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Trino/test_query_with_derived_metric_in_where_filter__plan0.sql b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Trino/test_query_with_derived_metric_in_where_filter__plan0.sql index d293da2f40..1904d4dfc7 100644 --- a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Trino/test_query_with_derived_metric_in_where_filter__plan0.sql +++ b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Trino/test_query_with_derived_metric_in_where_filter__plan0.sql @@ -22,8 +22,8 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.listing AS listing - , subq_15.listing__views_times_booking_value AS listing__views_times_booking_value + subq_15.listing__views_times_booking_value AS listing__views_times_booking_value + , subq_2.listing AS listing , subq_2.listings AS listings FROM ( -- Pass Only Elements: ['listings', 'listing'] diff --git a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Trino/test_query_with_multiple_metrics_in_filter__plan0.sql b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Trino/test_query_with_multiple_metrics_in_filter__plan0.sql index c7e55c7f64..7781a7d54c 100644 --- a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Trino/test_query_with_multiple_metrics_in_filter__plan0.sql +++ b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Trino/test_query_with_multiple_metrics_in_filter__plan0.sql @@ -24,9 +24,9 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.listing AS listing - , subq_8.listing__bookings AS listing__bookings + subq_8.listing__bookings AS listing__bookings , subq_14.listing__bookers AS listing__bookers + , subq_2.listing AS listing , subq_2.listings AS listings FROM ( -- Pass Only Elements: ['listings', 'listing', 'listing'] diff --git a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Trino/test_query_with_ratio_metric_in_where_filter__plan0.sql b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Trino/test_query_with_ratio_metric_in_where_filter__plan0.sql index ff5fde9d58..40bb3f4b49 100644 --- a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Trino/test_query_with_ratio_metric_in_where_filter__plan0.sql +++ b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Trino/test_query_with_ratio_metric_in_where_filter__plan0.sql @@ -22,8 +22,8 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.listing AS listing - , subq_15.listing__bookings_per_booker AS listing__bookings_per_booker + subq_15.listing__bookings_per_booker AS listing__bookings_per_booker + , subq_2.listing AS listing , subq_2.listings AS listings FROM ( -- Pass Only Elements: ['listings', 'listing'] diff --git a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Trino/test_query_with_simple_metric_in_where_filter__plan0.sql b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Trino/test_query_with_simple_metric_in_where_filter__plan0.sql index df04d2c9e4..56c887c51b 100644 --- a/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Trino/test_query_with_simple_metric_in_where_filter__plan0.sql +++ b/tests_metricflow/snapshots/test_metric_filter_rendering.py/SqlQueryPlan/Trino/test_query_with_simple_metric_in_where_filter__plan0.sql @@ -22,8 +22,8 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.listing AS listing - , subq_8.listing__bookings AS listing__bookings + subq_8.listing__bookings AS listing__bookings + , subq_2.listing AS listing , subq_2.listings AS listings FROM ( -- Pass Only Elements: ['listings', 'listing'] diff --git a/tests_metricflow/snapshots/test_metric_time_without_metrics.py/SqlQueryPlan/BigQuery/test_dimensions_with_time_constraint__plan0.sql b/tests_metricflow/snapshots/test_metric_time_without_metrics.py/SqlQueryPlan/BigQuery/test_dimensions_with_time_constraint__plan0.sql index e8d33642c7..4387ca064f 100644 --- a/tests_metricflow/snapshots/test_metric_time_without_metrics.py/SqlQueryPlan/BigQuery/test_dimensions_with_time_constraint__plan0.sql +++ b/tests_metricflow/snapshots/test_metric_time_without_metrics.py/SqlQueryPlan/BigQuery/test_dimensions_with_time_constraint__plan0.sql @@ -67,7 +67,8 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_0.ds__day AS ds__day + subq_5.home_state_latest AS user__home_state_latest + , subq_0.ds__day AS ds__day , subq_0.ds__week AS ds__week , subq_0.ds__month AS ds__month , subq_0.ds__quarter AS ds__quarter @@ -121,7 +122,6 @@ FROM ( , subq_0.listing__country_latest AS listing__country_latest , subq_0.listing__is_lux_latest AS listing__is_lux_latest , subq_0.listing__capacity_latest AS listing__capacity_latest - , subq_5.home_state_latest AS user__home_state_latest , subq_0.listings AS listings , subq_0.largest_listing AS largest_listing , subq_0.smallest_listing AS smallest_listing diff --git a/tests_metricflow/snapshots/test_metric_time_without_metrics.py/SqlQueryPlan/BigQuery/test_metric_time_with_other_dimensions__plan0.sql b/tests_metricflow/snapshots/test_metric_time_without_metrics.py/SqlQueryPlan/BigQuery/test_metric_time_with_other_dimensions__plan0.sql index ceb40208b9..1fd8948dea 100644 --- a/tests_metricflow/snapshots/test_metric_time_without_metrics.py/SqlQueryPlan/BigQuery/test_metric_time_with_other_dimensions__plan0.sql +++ b/tests_metricflow/snapshots/test_metric_time_without_metrics.py/SqlQueryPlan/BigQuery/test_metric_time_with_other_dimensions__plan0.sql @@ -6,7 +6,8 @@ SELECT FROM ( -- Join Standard Outputs SELECT - subq_0.ds__day AS ds__day + subq_5.home_state_latest AS user__home_state_latest + , subq_0.ds__day AS ds__day , subq_0.ds__week AS ds__week , subq_0.ds__month AS ds__month , subq_0.ds__quarter AS ds__quarter @@ -60,7 +61,6 @@ FROM ( , subq_0.listing__country_latest AS listing__country_latest , subq_0.listing__is_lux_latest AS listing__is_lux_latest , subq_0.listing__capacity_latest AS listing__capacity_latest - , subq_5.home_state_latest AS user__home_state_latest , subq_0.listings AS listings , subq_0.largest_listing AS largest_listing , subq_0.smallest_listing AS smallest_listing diff --git a/tests_metricflow/snapshots/test_metric_time_without_metrics.py/SqlQueryPlan/Databricks/test_dimensions_with_time_constraint__plan0.sql b/tests_metricflow/snapshots/test_metric_time_without_metrics.py/SqlQueryPlan/Databricks/test_dimensions_with_time_constraint__plan0.sql index e382a4b245..a0ff1aa632 100644 --- a/tests_metricflow/snapshots/test_metric_time_without_metrics.py/SqlQueryPlan/Databricks/test_dimensions_with_time_constraint__plan0.sql +++ b/tests_metricflow/snapshots/test_metric_time_without_metrics.py/SqlQueryPlan/Databricks/test_dimensions_with_time_constraint__plan0.sql @@ -67,7 +67,8 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_0.ds__day AS ds__day + subq_5.home_state_latest AS user__home_state_latest + , subq_0.ds__day AS ds__day , subq_0.ds__week AS ds__week , subq_0.ds__month AS ds__month , subq_0.ds__quarter AS ds__quarter @@ -121,7 +122,6 @@ FROM ( , subq_0.listing__country_latest AS listing__country_latest , subq_0.listing__is_lux_latest AS listing__is_lux_latest , subq_0.listing__capacity_latest AS listing__capacity_latest - , subq_5.home_state_latest AS user__home_state_latest , subq_0.listings AS listings , subq_0.largest_listing AS largest_listing , subq_0.smallest_listing AS smallest_listing diff --git a/tests_metricflow/snapshots/test_metric_time_without_metrics.py/SqlQueryPlan/Databricks/test_metric_time_with_other_dimensions__plan0.sql b/tests_metricflow/snapshots/test_metric_time_without_metrics.py/SqlQueryPlan/Databricks/test_metric_time_with_other_dimensions__plan0.sql index f6d0f5b82f..8b126ffbc9 100644 --- a/tests_metricflow/snapshots/test_metric_time_without_metrics.py/SqlQueryPlan/Databricks/test_metric_time_with_other_dimensions__plan0.sql +++ b/tests_metricflow/snapshots/test_metric_time_without_metrics.py/SqlQueryPlan/Databricks/test_metric_time_with_other_dimensions__plan0.sql @@ -6,7 +6,8 @@ SELECT FROM ( -- Join Standard Outputs SELECT - subq_0.ds__day AS ds__day + subq_5.home_state_latest AS user__home_state_latest + , subq_0.ds__day AS ds__day , subq_0.ds__week AS ds__week , subq_0.ds__month AS ds__month , subq_0.ds__quarter AS ds__quarter @@ -60,7 +61,6 @@ FROM ( , subq_0.listing__country_latest AS listing__country_latest , subq_0.listing__is_lux_latest AS listing__is_lux_latest , subq_0.listing__capacity_latest AS listing__capacity_latest - , subq_5.home_state_latest AS user__home_state_latest , subq_0.listings AS listings , subq_0.largest_listing AS largest_listing , subq_0.smallest_listing AS smallest_listing diff --git a/tests_metricflow/snapshots/test_metric_time_without_metrics.py/SqlQueryPlan/DuckDB/test_dimensions_with_time_constraint__plan0.sql b/tests_metricflow/snapshots/test_metric_time_without_metrics.py/SqlQueryPlan/DuckDB/test_dimensions_with_time_constraint__plan0.sql index 924b094be0..a38763a78d 100644 --- a/tests_metricflow/snapshots/test_metric_time_without_metrics.py/SqlQueryPlan/DuckDB/test_dimensions_with_time_constraint__plan0.sql +++ b/tests_metricflow/snapshots/test_metric_time_without_metrics.py/SqlQueryPlan/DuckDB/test_dimensions_with_time_constraint__plan0.sql @@ -67,7 +67,8 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_0.ds__day AS ds__day + subq_5.home_state_latest AS user__home_state_latest + , subq_0.ds__day AS ds__day , subq_0.ds__week AS ds__week , subq_0.ds__month AS ds__month , subq_0.ds__quarter AS ds__quarter @@ -121,7 +122,6 @@ FROM ( , subq_0.listing__country_latest AS listing__country_latest , subq_0.listing__is_lux_latest AS listing__is_lux_latest , subq_0.listing__capacity_latest AS listing__capacity_latest - , subq_5.home_state_latest AS user__home_state_latest , subq_0.listings AS listings , subq_0.largest_listing AS largest_listing , subq_0.smallest_listing AS smallest_listing diff --git a/tests_metricflow/snapshots/test_metric_time_without_metrics.py/SqlQueryPlan/DuckDB/test_metric_time_with_other_dimensions__plan0.sql b/tests_metricflow/snapshots/test_metric_time_without_metrics.py/SqlQueryPlan/DuckDB/test_metric_time_with_other_dimensions__plan0.sql index 9997adfc98..95837e5a9e 100644 --- a/tests_metricflow/snapshots/test_metric_time_without_metrics.py/SqlQueryPlan/DuckDB/test_metric_time_with_other_dimensions__plan0.sql +++ b/tests_metricflow/snapshots/test_metric_time_without_metrics.py/SqlQueryPlan/DuckDB/test_metric_time_with_other_dimensions__plan0.sql @@ -6,7 +6,8 @@ SELECT FROM ( -- Join Standard Outputs SELECT - subq_0.ds__day AS ds__day + subq_5.home_state_latest AS user__home_state_latest + , subq_0.ds__day AS ds__day , subq_0.ds__week AS ds__week , subq_0.ds__month AS ds__month , subq_0.ds__quarter AS ds__quarter @@ -60,7 +61,6 @@ FROM ( , subq_0.listing__country_latest AS listing__country_latest , subq_0.listing__is_lux_latest AS listing__is_lux_latest , subq_0.listing__capacity_latest AS listing__capacity_latest - , subq_5.home_state_latest AS user__home_state_latest , subq_0.listings AS listings , subq_0.largest_listing AS largest_listing , subq_0.smallest_listing AS smallest_listing diff --git a/tests_metricflow/snapshots/test_metric_time_without_metrics.py/SqlQueryPlan/Postgres/test_dimensions_with_time_constraint__plan0.sql b/tests_metricflow/snapshots/test_metric_time_without_metrics.py/SqlQueryPlan/Postgres/test_dimensions_with_time_constraint__plan0.sql index 924b094be0..a38763a78d 100644 --- a/tests_metricflow/snapshots/test_metric_time_without_metrics.py/SqlQueryPlan/Postgres/test_dimensions_with_time_constraint__plan0.sql +++ b/tests_metricflow/snapshots/test_metric_time_without_metrics.py/SqlQueryPlan/Postgres/test_dimensions_with_time_constraint__plan0.sql @@ -67,7 +67,8 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_0.ds__day AS ds__day + subq_5.home_state_latest AS user__home_state_latest + , subq_0.ds__day AS ds__day , subq_0.ds__week AS ds__week , subq_0.ds__month AS ds__month , subq_0.ds__quarter AS ds__quarter @@ -121,7 +122,6 @@ FROM ( , subq_0.listing__country_latest AS listing__country_latest , subq_0.listing__is_lux_latest AS listing__is_lux_latest , subq_0.listing__capacity_latest AS listing__capacity_latest - , subq_5.home_state_latest AS user__home_state_latest , subq_0.listings AS listings , subq_0.largest_listing AS largest_listing , subq_0.smallest_listing AS smallest_listing diff --git a/tests_metricflow/snapshots/test_metric_time_without_metrics.py/SqlQueryPlan/Postgres/test_metric_time_with_other_dimensions__plan0.sql b/tests_metricflow/snapshots/test_metric_time_without_metrics.py/SqlQueryPlan/Postgres/test_metric_time_with_other_dimensions__plan0.sql index 9997adfc98..95837e5a9e 100644 --- a/tests_metricflow/snapshots/test_metric_time_without_metrics.py/SqlQueryPlan/Postgres/test_metric_time_with_other_dimensions__plan0.sql +++ b/tests_metricflow/snapshots/test_metric_time_without_metrics.py/SqlQueryPlan/Postgres/test_metric_time_with_other_dimensions__plan0.sql @@ -6,7 +6,8 @@ SELECT FROM ( -- Join Standard Outputs SELECT - subq_0.ds__day AS ds__day + subq_5.home_state_latest AS user__home_state_latest + , subq_0.ds__day AS ds__day , subq_0.ds__week AS ds__week , subq_0.ds__month AS ds__month , subq_0.ds__quarter AS ds__quarter @@ -60,7 +61,6 @@ FROM ( , subq_0.listing__country_latest AS listing__country_latest , subq_0.listing__is_lux_latest AS listing__is_lux_latest , subq_0.listing__capacity_latest AS listing__capacity_latest - , subq_5.home_state_latest AS user__home_state_latest , subq_0.listings AS listings , subq_0.largest_listing AS largest_listing , subq_0.smallest_listing AS smallest_listing diff --git a/tests_metricflow/snapshots/test_metric_time_without_metrics.py/SqlQueryPlan/Redshift/test_dimensions_with_time_constraint__plan0.sql b/tests_metricflow/snapshots/test_metric_time_without_metrics.py/SqlQueryPlan/Redshift/test_dimensions_with_time_constraint__plan0.sql index 377b774a6d..efea7cead8 100644 --- a/tests_metricflow/snapshots/test_metric_time_without_metrics.py/SqlQueryPlan/Redshift/test_dimensions_with_time_constraint__plan0.sql +++ b/tests_metricflow/snapshots/test_metric_time_without_metrics.py/SqlQueryPlan/Redshift/test_dimensions_with_time_constraint__plan0.sql @@ -67,7 +67,8 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_0.ds__day AS ds__day + subq_5.home_state_latest AS user__home_state_latest + , subq_0.ds__day AS ds__day , subq_0.ds__week AS ds__week , subq_0.ds__month AS ds__month , subq_0.ds__quarter AS ds__quarter @@ -121,7 +122,6 @@ FROM ( , subq_0.listing__country_latest AS listing__country_latest , subq_0.listing__is_lux_latest AS listing__is_lux_latest , subq_0.listing__capacity_latest AS listing__capacity_latest - , subq_5.home_state_latest AS user__home_state_latest , subq_0.listings AS listings , subq_0.largest_listing AS largest_listing , subq_0.smallest_listing AS smallest_listing diff --git a/tests_metricflow/snapshots/test_metric_time_without_metrics.py/SqlQueryPlan/Redshift/test_metric_time_with_other_dimensions__plan0.sql b/tests_metricflow/snapshots/test_metric_time_without_metrics.py/SqlQueryPlan/Redshift/test_metric_time_with_other_dimensions__plan0.sql index 5c33d96ef3..8aad8892d7 100644 --- a/tests_metricflow/snapshots/test_metric_time_without_metrics.py/SqlQueryPlan/Redshift/test_metric_time_with_other_dimensions__plan0.sql +++ b/tests_metricflow/snapshots/test_metric_time_without_metrics.py/SqlQueryPlan/Redshift/test_metric_time_with_other_dimensions__plan0.sql @@ -6,7 +6,8 @@ SELECT FROM ( -- Join Standard Outputs SELECT - subq_0.ds__day AS ds__day + subq_5.home_state_latest AS user__home_state_latest + , subq_0.ds__day AS ds__day , subq_0.ds__week AS ds__week , subq_0.ds__month AS ds__month , subq_0.ds__quarter AS ds__quarter @@ -60,7 +61,6 @@ FROM ( , subq_0.listing__country_latest AS listing__country_latest , subq_0.listing__is_lux_latest AS listing__is_lux_latest , subq_0.listing__capacity_latest AS listing__capacity_latest - , subq_5.home_state_latest AS user__home_state_latest , subq_0.listings AS listings , subq_0.largest_listing AS largest_listing , subq_0.smallest_listing AS smallest_listing diff --git a/tests_metricflow/snapshots/test_metric_time_without_metrics.py/SqlQueryPlan/Snowflake/test_dimensions_with_time_constraint__plan0.sql b/tests_metricflow/snapshots/test_metric_time_without_metrics.py/SqlQueryPlan/Snowflake/test_dimensions_with_time_constraint__plan0.sql index f616ddf62e..b16fc5052e 100644 --- a/tests_metricflow/snapshots/test_metric_time_without_metrics.py/SqlQueryPlan/Snowflake/test_dimensions_with_time_constraint__plan0.sql +++ b/tests_metricflow/snapshots/test_metric_time_without_metrics.py/SqlQueryPlan/Snowflake/test_dimensions_with_time_constraint__plan0.sql @@ -67,7 +67,8 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_0.ds__day AS ds__day + subq_5.home_state_latest AS user__home_state_latest + , subq_0.ds__day AS ds__day , subq_0.ds__week AS ds__week , subq_0.ds__month AS ds__month , subq_0.ds__quarter AS ds__quarter @@ -121,7 +122,6 @@ FROM ( , subq_0.listing__country_latest AS listing__country_latest , subq_0.listing__is_lux_latest AS listing__is_lux_latest , subq_0.listing__capacity_latest AS listing__capacity_latest - , subq_5.home_state_latest AS user__home_state_latest , subq_0.listings AS listings , subq_0.largest_listing AS largest_listing , subq_0.smallest_listing AS smallest_listing diff --git a/tests_metricflow/snapshots/test_metric_time_without_metrics.py/SqlQueryPlan/Snowflake/test_metric_time_with_other_dimensions__plan0.sql b/tests_metricflow/snapshots/test_metric_time_without_metrics.py/SqlQueryPlan/Snowflake/test_metric_time_with_other_dimensions__plan0.sql index 1c71cafc3e..a61dba85b0 100644 --- a/tests_metricflow/snapshots/test_metric_time_without_metrics.py/SqlQueryPlan/Snowflake/test_metric_time_with_other_dimensions__plan0.sql +++ b/tests_metricflow/snapshots/test_metric_time_without_metrics.py/SqlQueryPlan/Snowflake/test_metric_time_with_other_dimensions__plan0.sql @@ -6,7 +6,8 @@ SELECT FROM ( -- Join Standard Outputs SELECT - subq_0.ds__day AS ds__day + subq_5.home_state_latest AS user__home_state_latest + , subq_0.ds__day AS ds__day , subq_0.ds__week AS ds__week , subq_0.ds__month AS ds__month , subq_0.ds__quarter AS ds__quarter @@ -60,7 +61,6 @@ FROM ( , subq_0.listing__country_latest AS listing__country_latest , subq_0.listing__is_lux_latest AS listing__is_lux_latest , subq_0.listing__capacity_latest AS listing__capacity_latest - , subq_5.home_state_latest AS user__home_state_latest , subq_0.listings AS listings , subq_0.largest_listing AS largest_listing , subq_0.smallest_listing AS smallest_listing diff --git a/tests_metricflow/snapshots/test_metric_time_without_metrics.py/SqlQueryPlan/Trino/test_dimensions_with_time_constraint__plan0.sql b/tests_metricflow/snapshots/test_metric_time_without_metrics.py/SqlQueryPlan/Trino/test_dimensions_with_time_constraint__plan0.sql index 899b565524..26e9563b2e 100644 --- a/tests_metricflow/snapshots/test_metric_time_without_metrics.py/SqlQueryPlan/Trino/test_dimensions_with_time_constraint__plan0.sql +++ b/tests_metricflow/snapshots/test_metric_time_without_metrics.py/SqlQueryPlan/Trino/test_dimensions_with_time_constraint__plan0.sql @@ -67,7 +67,8 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_0.ds__day AS ds__day + subq_5.home_state_latest AS user__home_state_latest + , subq_0.ds__day AS ds__day , subq_0.ds__week AS ds__week , subq_0.ds__month AS ds__month , subq_0.ds__quarter AS ds__quarter @@ -121,7 +122,6 @@ FROM ( , subq_0.listing__country_latest AS listing__country_latest , subq_0.listing__is_lux_latest AS listing__is_lux_latest , subq_0.listing__capacity_latest AS listing__capacity_latest - , subq_5.home_state_latest AS user__home_state_latest , subq_0.listings AS listings , subq_0.largest_listing AS largest_listing , subq_0.smallest_listing AS smallest_listing diff --git a/tests_metricflow/snapshots/test_metric_time_without_metrics.py/SqlQueryPlan/Trino/test_metric_time_with_other_dimensions__plan0.sql b/tests_metricflow/snapshots/test_metric_time_without_metrics.py/SqlQueryPlan/Trino/test_metric_time_with_other_dimensions__plan0.sql index e8b4477546..84e2cf6d8e 100644 --- a/tests_metricflow/snapshots/test_metric_time_without_metrics.py/SqlQueryPlan/Trino/test_metric_time_with_other_dimensions__plan0.sql +++ b/tests_metricflow/snapshots/test_metric_time_without_metrics.py/SqlQueryPlan/Trino/test_metric_time_with_other_dimensions__plan0.sql @@ -6,7 +6,8 @@ SELECT FROM ( -- Join Standard Outputs SELECT - subq_0.ds__day AS ds__day + subq_5.home_state_latest AS user__home_state_latest + , subq_0.ds__day AS ds__day , subq_0.ds__week AS ds__week , subq_0.ds__month AS ds__month , subq_0.ds__quarter AS ds__quarter @@ -60,7 +61,6 @@ FROM ( , subq_0.listing__country_latest AS listing__country_latest , subq_0.listing__is_lux_latest AS listing__is_lux_latest , subq_0.listing__capacity_latest AS listing__capacity_latest - , subq_5.home_state_latest AS user__home_state_latest , subq_0.listings AS listings , subq_0.largest_listing AS largest_listing , subq_0.smallest_listing AS smallest_listing diff --git a/tests_metricflow/snapshots/test_metric_time_without_metrics.py/SqlQueryPlan/test_dimensions_with_time_constraint__plan0.xml b/tests_metricflow/snapshots/test_metric_time_without_metrics.py/SqlQueryPlan/test_dimensions_with_time_constraint__plan0.xml index 59f8d1b5ec..c969ff9725 100644 --- a/tests_metricflow/snapshots/test_metric_time_without_metrics.py/SqlQueryPlan/test_dimensions_with_time_constraint__plan0.xml +++ b/tests_metricflow/snapshots/test_metric_time_without_metrics.py/SqlQueryPlan/test_dimensions_with_time_constraint__plan0.xml @@ -214,208 +214,208 @@ - - - - - - - - - + - + + + + + + + + + + + + + + - - - - + + - + - + - - - - - - + + + + + + + + - + - - - - - - - + + - + - + - + - + - + - - - - - + - - + + + + + + - + - - - + - - + + + + - + - + - + - + - + - + - + - + - + - + + + + + + - + - + - + - + - - - - - - + - + - - - - - - - - - - - - - + - - + + + + + + + + + + + + + + - + - + - - + + - + - + - + diff --git a/tests_metricflow/snapshots/test_metric_time_without_metrics.py/SqlQueryPlan/test_metric_time_with_other_dimensions__plan0.xml b/tests_metricflow/snapshots/test_metric_time_without_metrics.py/SqlQueryPlan/test_metric_time_with_other_dimensions__plan0.xml index aa697614b3..356fbd4c20 100644 --- a/tests_metricflow/snapshots/test_metric_time_without_metrics.py/SqlQueryPlan/test_metric_time_with_other_dimensions__plan0.xml +++ b/tests_metricflow/snapshots/test_metric_time_without_metrics.py/SqlQueryPlan/test_metric_time_with_other_dimensions__plan0.xml @@ -20,193 +20,193 @@ - - - - - - - - - - - - - + + + + + + + + + + + + + + + + - + - + - - - - - - - - - - - - - - + + + + + + + + + + + - + - + - + - + - + - - - - - - - - - - - + - - + + + + + + + + + + + + - + - + - + - + - + - + - + - + - + - + + + + + + - + - + - + - + - - - - - - + - + - - - - - - - - - - - + - - + + + + + + + + + + + + - + - + - - + + - + - + - + diff --git a/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/BigQuery/test_conversion_metric_query_filters__plan0.sql b/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/BigQuery/test_conversion_metric_query_filters__plan0.sql index 0237001154..152fa655b0 100644 --- a/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/BigQuery/test_conversion_metric_query_filters__plan0.sql +++ b/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/BigQuery/test_conversion_metric_query_filters__plan0.sql @@ -39,10 +39,10 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.metric_time__day AS metric_time__day + subq_4.home_state_latest AS user__home_state_latest + , subq_2.metric_time__day AS metric_time__day , subq_2.user AS user , subq_2.visit__referrer_id AS visit__referrer_id - , subq_4.home_state_latest AS user__home_state_latest , subq_2.visits AS visits FROM ( -- Pass Only Elements: ['visits', 'visit__referrer_id', 'metric_time__day', 'user'] @@ -301,7 +301,8 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_11.ds__day AS ds__day + subq_13.home_state_latest AS user__home_state_latest + , subq_11.ds__day AS ds__day , subq_11.ds__week AS ds__week , subq_11.ds__month AS ds__month , subq_11.ds__quarter AS ds__quarter @@ -340,7 +341,6 @@ FROM ( , subq_11.visit__session AS visit__session , subq_11.referrer_id AS referrer_id , subq_11.visit__referrer_id AS visit__referrer_id - , subq_13.home_state_latest AS user__home_state_latest , subq_11.visits AS visits , subq_11.visitors AS visitors FROM ( diff --git a/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/BigQuery/test_conversion_metric_query_filters__plan0_optimized.sql b/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/BigQuery/test_conversion_metric_query_filters__plan0_optimized.sql index 57331e9f05..fac0a1caaf 100644 --- a/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/BigQuery/test_conversion_metric_query_filters__plan0_optimized.sql +++ b/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/BigQuery/test_conversion_metric_query_filters__plan0_optimized.sql @@ -112,10 +112,10 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_36.metric_time__day AS metric_time__day + users_latest_src_28000.home_state_latest AS user__home_state_latest + , subq_36.metric_time__day AS metric_time__day , subq_36.user AS user , subq_36.visit__referrer_id AS visit__referrer_id - , users_latest_src_28000.home_state_latest AS user__home_state_latest , subq_36.visits AS visits FROM ( -- Read Elements From Semantic Model 'visits_source' diff --git a/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/BigQuery/test_cumulative_metric_with_query_time_filters__plan0.sql b/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/BigQuery/test_cumulative_metric_with_query_time_filters__plan0.sql index b14b3141df..d9951a2c4a 100644 --- a/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/BigQuery/test_cumulative_metric_with_query_time_filters__plan0.sql +++ b/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/BigQuery/test_cumulative_metric_with_query_time_filters__plan0.sql @@ -32,10 +32,10 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_5.metric_time__day AS metric_time__day + subq_8.country_latest AS listing__country_latest + , subq_5.metric_time__day AS metric_time__day , subq_5.listing AS listing , subq_5.booking__is_instant AS booking__is_instant - , subq_8.country_latest AS listing__country_latest , subq_5.bookers AS bookers FROM ( -- Pass Only Elements: ['bookers', 'booking__is_instant', 'metric_time__day', 'listing'] diff --git a/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/BigQuery/test_fill_nulls_time_spine_metric_predicate_pushdown__plan0.sql b/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/BigQuery/test_fill_nulls_time_spine_metric_predicate_pushdown__plan0.sql index 76e3e3bf5d..dd014f8c4e 100644 --- a/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/BigQuery/test_fill_nulls_time_spine_metric_predicate_pushdown__plan0.sql +++ b/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/BigQuery/test_fill_nulls_time_spine_metric_predicate_pushdown__plan0.sql @@ -57,10 +57,10 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.metric_time__day AS metric_time__day + subq_5.country_latest AS listing__country_latest + , subq_2.metric_time__day AS metric_time__day , subq_2.listing AS listing , subq_2.booking__is_instant AS booking__is_instant - , subq_5.country_latest AS listing__country_latest , subq_2.bookings AS bookings FROM ( -- Pass Only Elements: ['bookings', 'booking__is_instant', 'metric_time__day', 'listing'] @@ -465,10 +465,10 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_20.metric_time__day AS metric_time__day + subq_23.country_latest AS listing__country_latest + , subq_20.metric_time__day AS metric_time__day , subq_20.listing AS listing , subq_20.booking__is_instant AS booking__is_instant - , subq_23.country_latest AS listing__country_latest , subq_20.bookings AS bookings FROM ( -- Pass Only Elements: ['bookings', 'booking__is_instant', 'metric_time__day', 'listing'] diff --git a/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/BigQuery/test_metric_time_filter_with_two_targets__plan0.sql b/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/BigQuery/test_metric_time_filter_with_two_targets__plan0.sql index c5fbf342aa..73bd3e4430 100644 --- a/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/BigQuery/test_metric_time_filter_with_two_targets__plan0.sql +++ b/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/BigQuery/test_metric_time_filter_with_two_targets__plan0.sql @@ -27,9 +27,9 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.metric_time__day AS metric_time__day + subq_5.country_latest AS listing__country_latest + , subq_2.metric_time__day AS metric_time__day , subq_2.listing AS listing - , subq_5.country_latest AS listing__country_latest , subq_2.bookings AS bookings FROM ( -- Pass Only Elements: ['bookings', 'metric_time__day', 'listing'] diff --git a/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/BigQuery/test_multiple_categorical_dimension_pushdown__plan0.sql b/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/BigQuery/test_multiple_categorical_dimension_pushdown__plan0.sql index f389bc4063..ec163152c8 100644 --- a/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/BigQuery/test_multiple_categorical_dimension_pushdown__plan0.sql +++ b/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/BigQuery/test_multiple_categorical_dimension_pushdown__plan0.sql @@ -29,10 +29,10 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.user AS user + subq_4.home_state_latest AS user__home_state_latest + , subq_2.user AS user , subq_2.listing__is_lux_latest AS listing__is_lux_latest , subq_2.listing__capacity_latest AS listing__capacity_latest - , subq_4.home_state_latest AS user__home_state_latest , subq_2.listings AS listings FROM ( -- Pass Only Elements: ['listings', 'listing__is_lux_latest', 'listing__capacity_latest', 'user'] diff --git a/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/BigQuery/test_offset_metric_with_query_time_filters__plan0.sql b/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/BigQuery/test_offset_metric_with_query_time_filters__plan0.sql index ea15614ed2..d7984aaa46 100644 --- a/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/BigQuery/test_offset_metric_with_query_time_filters__plan0.sql +++ b/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/BigQuery/test_offset_metric_with_query_time_filters__plan0.sql @@ -45,10 +45,10 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.metric_time__day AS metric_time__day + subq_5.country_latest AS listing__country_latest + , subq_2.metric_time__day AS metric_time__day , subq_2.listing AS listing , subq_2.booking__is_instant AS booking__is_instant - , subq_5.country_latest AS listing__country_latest , subq_2.bookings AS bookings FROM ( -- Pass Only Elements: ['bookings', 'booking__is_instant', 'metric_time__day', 'listing'] @@ -438,10 +438,10 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_17.metric_time__day AS metric_time__day + subq_20.country_latest AS listing__country_latest + , subq_17.metric_time__day AS metric_time__day , subq_17.listing AS listing , subq_17.booking__is_instant AS booking__is_instant - , subq_20.country_latest AS listing__country_latest , subq_17.bookings AS bookings FROM ( -- Pass Only Elements: ['bookings', 'booking__is_instant', 'metric_time__day', 'listing'] diff --git a/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/BigQuery/test_saved_query_with_metric_joins_and_filter__plan0.sql b/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/BigQuery/test_saved_query_with_metric_joins_and_filter__plan0.sql index 285de52878..3811f60886 100644 --- a/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/BigQuery/test_saved_query_with_metric_joins_and_filter__plan0.sql +++ b/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/BigQuery/test_saved_query_with_metric_joins_and_filter__plan0.sql @@ -36,10 +36,10 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.metric_time__day AS metric_time__day - , subq_2.listing AS listing - , subq_5.is_lux_latest AS listing__is_lux_latest + subq_5.is_lux_latest AS listing__is_lux_latest , subq_5.capacity_latest AS listing__capacity_latest + , subq_2.metric_time__day AS metric_time__day + , subq_2.listing AS listing , subq_2.bookings AS bookings FROM ( -- Pass Only Elements: ['bookings', 'metric_time__day', 'listing'] @@ -425,10 +425,10 @@ FULL OUTER JOIN ( FROM ( -- Join Standard Outputs SELECT - subq_14.metric_time__day AS metric_time__day - , subq_14.listing AS listing - , subq_17.is_lux_latest AS listing__is_lux_latest + subq_17.is_lux_latest AS listing__is_lux_latest , subq_17.capacity_latest AS listing__capacity_latest + , subq_14.metric_time__day AS metric_time__day + , subq_14.listing AS listing , subq_14.views AS views FROM ( -- Pass Only Elements: ['views', 'metric_time__day', 'listing'] @@ -750,10 +750,10 @@ FULL OUTER JOIN ( FROM ( -- Join Standard Outputs SELECT - subq_26.metric_time__day AS metric_time__day - , subq_26.listing AS listing - , subq_29.is_lux_latest AS listing__is_lux_latest + subq_29.is_lux_latest AS listing__is_lux_latest , subq_29.capacity_latest AS listing__capacity_latest + , subq_26.metric_time__day AS metric_time__day + , subq_26.listing AS listing , subq_26.bookings AS bookings FROM ( -- Pass Only Elements: ['bookings', 'metric_time__day', 'listing'] @@ -1139,10 +1139,10 @@ FULL OUTER JOIN ( FROM ( -- Join Standard Outputs SELECT - subq_38.metric_time__day AS metric_time__day - , subq_38.listing AS listing - , subq_41.is_lux_latest AS listing__is_lux_latest + subq_41.is_lux_latest AS listing__is_lux_latest , subq_41.capacity_latest AS listing__capacity_latest + , subq_38.metric_time__day AS metric_time__day + , subq_38.listing AS listing , subq_38.views AS views FROM ( -- Pass Only Elements: ['views', 'metric_time__day', 'listing'] diff --git a/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/BigQuery/test_single_categorical_dimension_pushdown__plan0.sql b/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/BigQuery/test_single_categorical_dimension_pushdown__plan0.sql index b13876e3aa..824f5e2479 100644 --- a/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/BigQuery/test_single_categorical_dimension_pushdown__plan0.sql +++ b/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/BigQuery/test_single_categorical_dimension_pushdown__plan0.sql @@ -27,9 +27,9 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.listing AS listing + subq_5.country_latest AS listing__country_latest + , subq_2.listing AS listing , subq_2.booking__is_instant AS booking__is_instant - , subq_5.country_latest AS listing__country_latest , subq_2.bookings AS bookings FROM ( -- Pass Only Elements: ['bookings', 'booking__is_instant', 'listing'] diff --git a/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/BigQuery/test_skipped_pushdown__plan0.sql b/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/BigQuery/test_skipped_pushdown__plan0.sql index 945c329d57..b86b8a5a54 100644 --- a/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/BigQuery/test_skipped_pushdown__plan0.sql +++ b/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/BigQuery/test_skipped_pushdown__plan0.sql @@ -29,10 +29,10 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.listing AS listing - , subq_2.booking__is_instant AS booking__is_instant - , subq_5.country_latest AS listing__country_latest + subq_5.country_latest AS listing__country_latest , subq_5.is_lux_latest AS listing__is_lux_latest + , subq_2.listing AS listing + , subq_2.booking__is_instant AS booking__is_instant , subq_2.bookings AS bookings FROM ( -- Pass Only Elements: ['bookings', 'booking__is_instant', 'listing'] diff --git a/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Databricks/test_conversion_metric_query_filters__plan0.sql b/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Databricks/test_conversion_metric_query_filters__plan0.sql index 033b95213c..4821515888 100644 --- a/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Databricks/test_conversion_metric_query_filters__plan0.sql +++ b/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Databricks/test_conversion_metric_query_filters__plan0.sql @@ -39,10 +39,10 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.metric_time__day AS metric_time__day + subq_4.home_state_latest AS user__home_state_latest + , subq_2.metric_time__day AS metric_time__day , subq_2.user AS user , subq_2.visit__referrer_id AS visit__referrer_id - , subq_4.home_state_latest AS user__home_state_latest , subq_2.visits AS visits FROM ( -- Pass Only Elements: ['visits', 'visit__referrer_id', 'metric_time__day', 'user'] @@ -301,7 +301,8 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_11.ds__day AS ds__day + subq_13.home_state_latest AS user__home_state_latest + , subq_11.ds__day AS ds__day , subq_11.ds__week AS ds__week , subq_11.ds__month AS ds__month , subq_11.ds__quarter AS ds__quarter @@ -340,7 +341,6 @@ FROM ( , subq_11.visit__session AS visit__session , subq_11.referrer_id AS referrer_id , subq_11.visit__referrer_id AS visit__referrer_id - , subq_13.home_state_latest AS user__home_state_latest , subq_11.visits AS visits , subq_11.visitors AS visitors FROM ( diff --git a/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Databricks/test_conversion_metric_query_filters__plan0_optimized.sql b/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Databricks/test_conversion_metric_query_filters__plan0_optimized.sql index ba92724c0d..4c666da226 100644 --- a/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Databricks/test_conversion_metric_query_filters__plan0_optimized.sql +++ b/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Databricks/test_conversion_metric_query_filters__plan0_optimized.sql @@ -112,10 +112,10 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_36.metric_time__day AS metric_time__day + users_latest_src_28000.home_state_latest AS user__home_state_latest + , subq_36.metric_time__day AS metric_time__day , subq_36.user AS user , subq_36.visit__referrer_id AS visit__referrer_id - , users_latest_src_28000.home_state_latest AS user__home_state_latest , subq_36.visits AS visits FROM ( -- Read Elements From Semantic Model 'visits_source' diff --git a/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Databricks/test_cumulative_metric_with_query_time_filters__plan0.sql b/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Databricks/test_cumulative_metric_with_query_time_filters__plan0.sql index 3f45fb8f0f..60823e39c5 100644 --- a/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Databricks/test_cumulative_metric_with_query_time_filters__plan0.sql +++ b/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Databricks/test_cumulative_metric_with_query_time_filters__plan0.sql @@ -32,10 +32,10 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_5.metric_time__day AS metric_time__day + subq_8.country_latest AS listing__country_latest + , subq_5.metric_time__day AS metric_time__day , subq_5.listing AS listing , subq_5.booking__is_instant AS booking__is_instant - , subq_8.country_latest AS listing__country_latest , subq_5.bookers AS bookers FROM ( -- Pass Only Elements: ['bookers', 'booking__is_instant', 'metric_time__day', 'listing'] diff --git a/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Databricks/test_fill_nulls_time_spine_metric_predicate_pushdown__plan0.sql b/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Databricks/test_fill_nulls_time_spine_metric_predicate_pushdown__plan0.sql index ff2a2c2169..f5b25b0aba 100644 --- a/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Databricks/test_fill_nulls_time_spine_metric_predicate_pushdown__plan0.sql +++ b/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Databricks/test_fill_nulls_time_spine_metric_predicate_pushdown__plan0.sql @@ -57,10 +57,10 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.metric_time__day AS metric_time__day + subq_5.country_latest AS listing__country_latest + , subq_2.metric_time__day AS metric_time__day , subq_2.listing AS listing , subq_2.booking__is_instant AS booking__is_instant - , subq_5.country_latest AS listing__country_latest , subq_2.bookings AS bookings FROM ( -- Pass Only Elements: ['bookings', 'booking__is_instant', 'metric_time__day', 'listing'] @@ -465,10 +465,10 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_20.metric_time__day AS metric_time__day + subq_23.country_latest AS listing__country_latest + , subq_20.metric_time__day AS metric_time__day , subq_20.listing AS listing , subq_20.booking__is_instant AS booking__is_instant - , subq_23.country_latest AS listing__country_latest , subq_20.bookings AS bookings FROM ( -- Pass Only Elements: ['bookings', 'booking__is_instant', 'metric_time__day', 'listing'] diff --git a/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Databricks/test_metric_time_filter_with_two_targets__plan0.sql b/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Databricks/test_metric_time_filter_with_two_targets__plan0.sql index de41d36fee..e46f00634c 100644 --- a/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Databricks/test_metric_time_filter_with_two_targets__plan0.sql +++ b/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Databricks/test_metric_time_filter_with_two_targets__plan0.sql @@ -27,9 +27,9 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.metric_time__day AS metric_time__day + subq_5.country_latest AS listing__country_latest + , subq_2.metric_time__day AS metric_time__day , subq_2.listing AS listing - , subq_5.country_latest AS listing__country_latest , subq_2.bookings AS bookings FROM ( -- Pass Only Elements: ['bookings', 'metric_time__day', 'listing'] diff --git a/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Databricks/test_multiple_categorical_dimension_pushdown__plan0.sql b/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Databricks/test_multiple_categorical_dimension_pushdown__plan0.sql index c0081fcc66..a7891018d6 100644 --- a/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Databricks/test_multiple_categorical_dimension_pushdown__plan0.sql +++ b/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Databricks/test_multiple_categorical_dimension_pushdown__plan0.sql @@ -29,10 +29,10 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.user AS user + subq_4.home_state_latest AS user__home_state_latest + , subq_2.user AS user , subq_2.listing__is_lux_latest AS listing__is_lux_latest , subq_2.listing__capacity_latest AS listing__capacity_latest - , subq_4.home_state_latest AS user__home_state_latest , subq_2.listings AS listings FROM ( -- Pass Only Elements: ['listings', 'listing__is_lux_latest', 'listing__capacity_latest', 'user'] diff --git a/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Databricks/test_offset_metric_with_query_time_filters__plan0.sql b/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Databricks/test_offset_metric_with_query_time_filters__plan0.sql index 031cbe720b..3d647dff0b 100644 --- a/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Databricks/test_offset_metric_with_query_time_filters__plan0.sql +++ b/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Databricks/test_offset_metric_with_query_time_filters__plan0.sql @@ -45,10 +45,10 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.metric_time__day AS metric_time__day + subq_5.country_latest AS listing__country_latest + , subq_2.metric_time__day AS metric_time__day , subq_2.listing AS listing , subq_2.booking__is_instant AS booking__is_instant - , subq_5.country_latest AS listing__country_latest , subq_2.bookings AS bookings FROM ( -- Pass Only Elements: ['bookings', 'booking__is_instant', 'metric_time__day', 'listing'] @@ -438,10 +438,10 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_17.metric_time__day AS metric_time__day + subq_20.country_latest AS listing__country_latest + , subq_17.metric_time__day AS metric_time__day , subq_17.listing AS listing , subq_17.booking__is_instant AS booking__is_instant - , subq_20.country_latest AS listing__country_latest , subq_17.bookings AS bookings FROM ( -- Pass Only Elements: ['bookings', 'booking__is_instant', 'metric_time__day', 'listing'] diff --git a/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Databricks/test_saved_query_with_metric_joins_and_filter__plan0.sql b/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Databricks/test_saved_query_with_metric_joins_and_filter__plan0.sql index c8655a1c38..435b246d65 100644 --- a/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Databricks/test_saved_query_with_metric_joins_and_filter__plan0.sql +++ b/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Databricks/test_saved_query_with_metric_joins_and_filter__plan0.sql @@ -36,10 +36,10 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.metric_time__day AS metric_time__day - , subq_2.listing AS listing - , subq_5.is_lux_latest AS listing__is_lux_latest + subq_5.is_lux_latest AS listing__is_lux_latest , subq_5.capacity_latest AS listing__capacity_latest + , subq_2.metric_time__day AS metric_time__day + , subq_2.listing AS listing , subq_2.bookings AS bookings FROM ( -- Pass Only Elements: ['bookings', 'metric_time__day', 'listing'] @@ -425,10 +425,10 @@ FULL OUTER JOIN ( FROM ( -- Join Standard Outputs SELECT - subq_14.metric_time__day AS metric_time__day - , subq_14.listing AS listing - , subq_17.is_lux_latest AS listing__is_lux_latest + subq_17.is_lux_latest AS listing__is_lux_latest , subq_17.capacity_latest AS listing__capacity_latest + , subq_14.metric_time__day AS metric_time__day + , subq_14.listing AS listing , subq_14.views AS views FROM ( -- Pass Only Elements: ['views', 'metric_time__day', 'listing'] @@ -750,10 +750,10 @@ FULL OUTER JOIN ( FROM ( -- Join Standard Outputs SELECT - subq_26.metric_time__day AS metric_time__day - , subq_26.listing AS listing - , subq_29.is_lux_latest AS listing__is_lux_latest + subq_29.is_lux_latest AS listing__is_lux_latest , subq_29.capacity_latest AS listing__capacity_latest + , subq_26.metric_time__day AS metric_time__day + , subq_26.listing AS listing , subq_26.bookings AS bookings FROM ( -- Pass Only Elements: ['bookings', 'metric_time__day', 'listing'] @@ -1139,10 +1139,10 @@ FULL OUTER JOIN ( FROM ( -- Join Standard Outputs SELECT - subq_38.metric_time__day AS metric_time__day - , subq_38.listing AS listing - , subq_41.is_lux_latest AS listing__is_lux_latest + subq_41.is_lux_latest AS listing__is_lux_latest , subq_41.capacity_latest AS listing__capacity_latest + , subq_38.metric_time__day AS metric_time__day + , subq_38.listing AS listing , subq_38.views AS views FROM ( -- Pass Only Elements: ['views', 'metric_time__day', 'listing'] diff --git a/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Databricks/test_single_categorical_dimension_pushdown__plan0.sql b/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Databricks/test_single_categorical_dimension_pushdown__plan0.sql index 137b07b34f..01ed482fcc 100644 --- a/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Databricks/test_single_categorical_dimension_pushdown__plan0.sql +++ b/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Databricks/test_single_categorical_dimension_pushdown__plan0.sql @@ -27,9 +27,9 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.listing AS listing + subq_5.country_latest AS listing__country_latest + , subq_2.listing AS listing , subq_2.booking__is_instant AS booking__is_instant - , subq_5.country_latest AS listing__country_latest , subq_2.bookings AS bookings FROM ( -- Pass Only Elements: ['bookings', 'booking__is_instant', 'listing'] diff --git a/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Databricks/test_skipped_pushdown__plan0.sql b/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Databricks/test_skipped_pushdown__plan0.sql index 968378bfc3..30e99fa656 100644 --- a/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Databricks/test_skipped_pushdown__plan0.sql +++ b/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Databricks/test_skipped_pushdown__plan0.sql @@ -29,10 +29,10 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.listing AS listing - , subq_2.booking__is_instant AS booking__is_instant - , subq_5.country_latest AS listing__country_latest + subq_5.country_latest AS listing__country_latest , subq_5.is_lux_latest AS listing__is_lux_latest + , subq_2.listing AS listing + , subq_2.booking__is_instant AS booking__is_instant , subq_2.bookings AS bookings FROM ( -- Pass Only Elements: ['bookings', 'booking__is_instant', 'listing'] diff --git a/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/DuckDB/test_conversion_metric_query_filters__plan0.sql b/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/DuckDB/test_conversion_metric_query_filters__plan0.sql index 2337d364dc..5c8a4f3eb1 100644 --- a/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/DuckDB/test_conversion_metric_query_filters__plan0.sql +++ b/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/DuckDB/test_conversion_metric_query_filters__plan0.sql @@ -39,10 +39,10 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.metric_time__day AS metric_time__day + subq_4.home_state_latest AS user__home_state_latest + , subq_2.metric_time__day AS metric_time__day , subq_2.user AS user , subq_2.visit__referrer_id AS visit__referrer_id - , subq_4.home_state_latest AS user__home_state_latest , subq_2.visits AS visits FROM ( -- Pass Only Elements: ['visits', 'visit__referrer_id', 'metric_time__day', 'user'] @@ -301,7 +301,8 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_11.ds__day AS ds__day + subq_13.home_state_latest AS user__home_state_latest + , subq_11.ds__day AS ds__day , subq_11.ds__week AS ds__week , subq_11.ds__month AS ds__month , subq_11.ds__quarter AS ds__quarter @@ -340,7 +341,6 @@ FROM ( , subq_11.visit__session AS visit__session , subq_11.referrer_id AS referrer_id , subq_11.visit__referrer_id AS visit__referrer_id - , subq_13.home_state_latest AS user__home_state_latest , subq_11.visits AS visits , subq_11.visitors AS visitors FROM ( diff --git a/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/DuckDB/test_conversion_metric_query_filters__plan0_optimized.sql b/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/DuckDB/test_conversion_metric_query_filters__plan0_optimized.sql index 4af834567c..0cfc5f2966 100644 --- a/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/DuckDB/test_conversion_metric_query_filters__plan0_optimized.sql +++ b/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/DuckDB/test_conversion_metric_query_filters__plan0_optimized.sql @@ -112,10 +112,10 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_36.metric_time__day AS metric_time__day + users_latest_src_28000.home_state_latest AS user__home_state_latest + , subq_36.metric_time__day AS metric_time__day , subq_36.user AS user , subq_36.visit__referrer_id AS visit__referrer_id - , users_latest_src_28000.home_state_latest AS user__home_state_latest , subq_36.visits AS visits FROM ( -- Read Elements From Semantic Model 'visits_source' diff --git a/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/DuckDB/test_cumulative_metric_with_query_time_filters__plan0.sql b/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/DuckDB/test_cumulative_metric_with_query_time_filters__plan0.sql index 48a9f31b84..6f79ec6c8c 100644 --- a/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/DuckDB/test_cumulative_metric_with_query_time_filters__plan0.sql +++ b/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/DuckDB/test_cumulative_metric_with_query_time_filters__plan0.sql @@ -32,10 +32,10 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_5.metric_time__day AS metric_time__day + subq_8.country_latest AS listing__country_latest + , subq_5.metric_time__day AS metric_time__day , subq_5.listing AS listing , subq_5.booking__is_instant AS booking__is_instant - , subq_8.country_latest AS listing__country_latest , subq_5.bookers AS bookers FROM ( -- Pass Only Elements: ['bookers', 'booking__is_instant', 'metric_time__day', 'listing'] diff --git a/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/DuckDB/test_fill_nulls_time_spine_metric_predicate_pushdown__plan0.sql b/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/DuckDB/test_fill_nulls_time_spine_metric_predicate_pushdown__plan0.sql index 620302ced3..bff625f5b7 100644 --- a/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/DuckDB/test_fill_nulls_time_spine_metric_predicate_pushdown__plan0.sql +++ b/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/DuckDB/test_fill_nulls_time_spine_metric_predicate_pushdown__plan0.sql @@ -57,10 +57,10 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.metric_time__day AS metric_time__day + subq_5.country_latest AS listing__country_latest + , subq_2.metric_time__day AS metric_time__day , subq_2.listing AS listing , subq_2.booking__is_instant AS booking__is_instant - , subq_5.country_latest AS listing__country_latest , subq_2.bookings AS bookings FROM ( -- Pass Only Elements: ['bookings', 'booking__is_instant', 'metric_time__day', 'listing'] @@ -465,10 +465,10 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_20.metric_time__day AS metric_time__day + subq_23.country_latest AS listing__country_latest + , subq_20.metric_time__day AS metric_time__day , subq_20.listing AS listing , subq_20.booking__is_instant AS booking__is_instant - , subq_23.country_latest AS listing__country_latest , subq_20.bookings AS bookings FROM ( -- Pass Only Elements: ['bookings', 'booking__is_instant', 'metric_time__day', 'listing'] diff --git a/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/DuckDB/test_metric_time_filter_with_two_targets__plan0.sql b/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/DuckDB/test_metric_time_filter_with_two_targets__plan0.sql index 4107fcf148..a15d93b916 100644 --- a/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/DuckDB/test_metric_time_filter_with_two_targets__plan0.sql +++ b/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/DuckDB/test_metric_time_filter_with_two_targets__plan0.sql @@ -27,9 +27,9 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.metric_time__day AS metric_time__day + subq_5.country_latest AS listing__country_latest + , subq_2.metric_time__day AS metric_time__day , subq_2.listing AS listing - , subq_5.country_latest AS listing__country_latest , subq_2.bookings AS bookings FROM ( -- Pass Only Elements: ['bookings', 'metric_time__day', 'listing'] diff --git a/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/DuckDB/test_multiple_categorical_dimension_pushdown__plan0.sql b/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/DuckDB/test_multiple_categorical_dimension_pushdown__plan0.sql index 70f27e867b..9dd9bdab7c 100644 --- a/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/DuckDB/test_multiple_categorical_dimension_pushdown__plan0.sql +++ b/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/DuckDB/test_multiple_categorical_dimension_pushdown__plan0.sql @@ -29,10 +29,10 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.user AS user + subq_4.home_state_latest AS user__home_state_latest + , subq_2.user AS user , subq_2.listing__is_lux_latest AS listing__is_lux_latest , subq_2.listing__capacity_latest AS listing__capacity_latest - , subq_4.home_state_latest AS user__home_state_latest , subq_2.listings AS listings FROM ( -- Pass Only Elements: ['listings', 'listing__is_lux_latest', 'listing__capacity_latest', 'user'] diff --git a/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/DuckDB/test_offset_metric_with_query_time_filters__plan0.sql b/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/DuckDB/test_offset_metric_with_query_time_filters__plan0.sql index 2bbf738849..029b248c52 100644 --- a/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/DuckDB/test_offset_metric_with_query_time_filters__plan0.sql +++ b/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/DuckDB/test_offset_metric_with_query_time_filters__plan0.sql @@ -45,10 +45,10 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.metric_time__day AS metric_time__day + subq_5.country_latest AS listing__country_latest + , subq_2.metric_time__day AS metric_time__day , subq_2.listing AS listing , subq_2.booking__is_instant AS booking__is_instant - , subq_5.country_latest AS listing__country_latest , subq_2.bookings AS bookings FROM ( -- Pass Only Elements: ['bookings', 'booking__is_instant', 'metric_time__day', 'listing'] @@ -438,10 +438,10 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_17.metric_time__day AS metric_time__day + subq_20.country_latest AS listing__country_latest + , subq_17.metric_time__day AS metric_time__day , subq_17.listing AS listing , subq_17.booking__is_instant AS booking__is_instant - , subq_20.country_latest AS listing__country_latest , subq_17.bookings AS bookings FROM ( -- Pass Only Elements: ['bookings', 'booking__is_instant', 'metric_time__day', 'listing'] diff --git a/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/DuckDB/test_saved_query_with_metric_joins_and_filter__plan0.sql b/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/DuckDB/test_saved_query_with_metric_joins_and_filter__plan0.sql index 25af00dd94..3f78f52b66 100644 --- a/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/DuckDB/test_saved_query_with_metric_joins_and_filter__plan0.sql +++ b/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/DuckDB/test_saved_query_with_metric_joins_and_filter__plan0.sql @@ -36,10 +36,10 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.metric_time__day AS metric_time__day - , subq_2.listing AS listing - , subq_5.is_lux_latest AS listing__is_lux_latest + subq_5.is_lux_latest AS listing__is_lux_latest , subq_5.capacity_latest AS listing__capacity_latest + , subq_2.metric_time__day AS metric_time__day + , subq_2.listing AS listing , subq_2.bookings AS bookings FROM ( -- Pass Only Elements: ['bookings', 'metric_time__day', 'listing'] @@ -425,10 +425,10 @@ FULL OUTER JOIN ( FROM ( -- Join Standard Outputs SELECT - subq_14.metric_time__day AS metric_time__day - , subq_14.listing AS listing - , subq_17.is_lux_latest AS listing__is_lux_latest + subq_17.is_lux_latest AS listing__is_lux_latest , subq_17.capacity_latest AS listing__capacity_latest + , subq_14.metric_time__day AS metric_time__day + , subq_14.listing AS listing , subq_14.views AS views FROM ( -- Pass Only Elements: ['views', 'metric_time__day', 'listing'] @@ -750,10 +750,10 @@ FULL OUTER JOIN ( FROM ( -- Join Standard Outputs SELECT - subq_26.metric_time__day AS metric_time__day - , subq_26.listing AS listing - , subq_29.is_lux_latest AS listing__is_lux_latest + subq_29.is_lux_latest AS listing__is_lux_latest , subq_29.capacity_latest AS listing__capacity_latest + , subq_26.metric_time__day AS metric_time__day + , subq_26.listing AS listing , subq_26.bookings AS bookings FROM ( -- Pass Only Elements: ['bookings', 'metric_time__day', 'listing'] @@ -1139,10 +1139,10 @@ FULL OUTER JOIN ( FROM ( -- Join Standard Outputs SELECT - subq_38.metric_time__day AS metric_time__day - , subq_38.listing AS listing - , subq_41.is_lux_latest AS listing__is_lux_latest + subq_41.is_lux_latest AS listing__is_lux_latest , subq_41.capacity_latest AS listing__capacity_latest + , subq_38.metric_time__day AS metric_time__day + , subq_38.listing AS listing , subq_38.views AS views FROM ( -- Pass Only Elements: ['views', 'metric_time__day', 'listing'] diff --git a/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/DuckDB/test_single_categorical_dimension_pushdown__plan0.sql b/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/DuckDB/test_single_categorical_dimension_pushdown__plan0.sql index 14aba1afb1..5fc23ef4c0 100644 --- a/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/DuckDB/test_single_categorical_dimension_pushdown__plan0.sql +++ b/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/DuckDB/test_single_categorical_dimension_pushdown__plan0.sql @@ -27,9 +27,9 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.listing AS listing + subq_5.country_latest AS listing__country_latest + , subq_2.listing AS listing , subq_2.booking__is_instant AS booking__is_instant - , subq_5.country_latest AS listing__country_latest , subq_2.bookings AS bookings FROM ( -- Pass Only Elements: ['bookings', 'booking__is_instant', 'listing'] diff --git a/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/DuckDB/test_skipped_pushdown__plan0.sql b/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/DuckDB/test_skipped_pushdown__plan0.sql index a62b838b8c..8c638d3b5e 100644 --- a/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/DuckDB/test_skipped_pushdown__plan0.sql +++ b/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/DuckDB/test_skipped_pushdown__plan0.sql @@ -29,10 +29,10 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.listing AS listing - , subq_2.booking__is_instant AS booking__is_instant - , subq_5.country_latest AS listing__country_latest + subq_5.country_latest AS listing__country_latest , subq_5.is_lux_latest AS listing__is_lux_latest + , subq_2.listing AS listing + , subq_2.booking__is_instant AS booking__is_instant , subq_2.bookings AS bookings FROM ( -- Pass Only Elements: ['bookings', 'booking__is_instant', 'listing'] diff --git a/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Postgres/test_conversion_metric_query_filters__plan0.sql b/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Postgres/test_conversion_metric_query_filters__plan0.sql index 54bda832b8..664dc7ad65 100644 --- a/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Postgres/test_conversion_metric_query_filters__plan0.sql +++ b/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Postgres/test_conversion_metric_query_filters__plan0.sql @@ -39,10 +39,10 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.metric_time__day AS metric_time__day + subq_4.home_state_latest AS user__home_state_latest + , subq_2.metric_time__day AS metric_time__day , subq_2.user AS user , subq_2.visit__referrer_id AS visit__referrer_id - , subq_4.home_state_latest AS user__home_state_latest , subq_2.visits AS visits FROM ( -- Pass Only Elements: ['visits', 'visit__referrer_id', 'metric_time__day', 'user'] @@ -301,7 +301,8 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_11.ds__day AS ds__day + subq_13.home_state_latest AS user__home_state_latest + , subq_11.ds__day AS ds__day , subq_11.ds__week AS ds__week , subq_11.ds__month AS ds__month , subq_11.ds__quarter AS ds__quarter @@ -340,7 +341,6 @@ FROM ( , subq_11.visit__session AS visit__session , subq_11.referrer_id AS referrer_id , subq_11.visit__referrer_id AS visit__referrer_id - , subq_13.home_state_latest AS user__home_state_latest , subq_11.visits AS visits , subq_11.visitors AS visitors FROM ( diff --git a/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Postgres/test_conversion_metric_query_filters__plan0_optimized.sql b/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Postgres/test_conversion_metric_query_filters__plan0_optimized.sql index 05652bcac5..caa1575854 100644 --- a/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Postgres/test_conversion_metric_query_filters__plan0_optimized.sql +++ b/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Postgres/test_conversion_metric_query_filters__plan0_optimized.sql @@ -112,10 +112,10 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_36.metric_time__day AS metric_time__day + users_latest_src_28000.home_state_latest AS user__home_state_latest + , subq_36.metric_time__day AS metric_time__day , subq_36.user AS user , subq_36.visit__referrer_id AS visit__referrer_id - , users_latest_src_28000.home_state_latest AS user__home_state_latest , subq_36.visits AS visits FROM ( -- Read Elements From Semantic Model 'visits_source' diff --git a/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Postgres/test_cumulative_metric_with_query_time_filters__plan0.sql b/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Postgres/test_cumulative_metric_with_query_time_filters__plan0.sql index 83de7ff3ef..4b3816415d 100644 --- a/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Postgres/test_cumulative_metric_with_query_time_filters__plan0.sql +++ b/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Postgres/test_cumulative_metric_with_query_time_filters__plan0.sql @@ -32,10 +32,10 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_5.metric_time__day AS metric_time__day + subq_8.country_latest AS listing__country_latest + , subq_5.metric_time__day AS metric_time__day , subq_5.listing AS listing , subq_5.booking__is_instant AS booking__is_instant - , subq_8.country_latest AS listing__country_latest , subq_5.bookers AS bookers FROM ( -- Pass Only Elements: ['bookers', 'booking__is_instant', 'metric_time__day', 'listing'] diff --git a/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Postgres/test_fill_nulls_time_spine_metric_predicate_pushdown__plan0.sql b/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Postgres/test_fill_nulls_time_spine_metric_predicate_pushdown__plan0.sql index 50cdc52ebe..642229ed86 100644 --- a/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Postgres/test_fill_nulls_time_spine_metric_predicate_pushdown__plan0.sql +++ b/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Postgres/test_fill_nulls_time_spine_metric_predicate_pushdown__plan0.sql @@ -57,10 +57,10 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.metric_time__day AS metric_time__day + subq_5.country_latest AS listing__country_latest + , subq_2.metric_time__day AS metric_time__day , subq_2.listing AS listing , subq_2.booking__is_instant AS booking__is_instant - , subq_5.country_latest AS listing__country_latest , subq_2.bookings AS bookings FROM ( -- Pass Only Elements: ['bookings', 'booking__is_instant', 'metric_time__day', 'listing'] @@ -465,10 +465,10 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_20.metric_time__day AS metric_time__day + subq_23.country_latest AS listing__country_latest + , subq_20.metric_time__day AS metric_time__day , subq_20.listing AS listing , subq_20.booking__is_instant AS booking__is_instant - , subq_23.country_latest AS listing__country_latest , subq_20.bookings AS bookings FROM ( -- Pass Only Elements: ['bookings', 'booking__is_instant', 'metric_time__day', 'listing'] diff --git a/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Postgres/test_metric_time_filter_with_two_targets__plan0.sql b/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Postgres/test_metric_time_filter_with_two_targets__plan0.sql index 4107fcf148..a15d93b916 100644 --- a/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Postgres/test_metric_time_filter_with_two_targets__plan0.sql +++ b/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Postgres/test_metric_time_filter_with_two_targets__plan0.sql @@ -27,9 +27,9 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.metric_time__day AS metric_time__day + subq_5.country_latest AS listing__country_latest + , subq_2.metric_time__day AS metric_time__day , subq_2.listing AS listing - , subq_5.country_latest AS listing__country_latest , subq_2.bookings AS bookings FROM ( -- Pass Only Elements: ['bookings', 'metric_time__day', 'listing'] diff --git a/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Postgres/test_multiple_categorical_dimension_pushdown__plan0.sql b/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Postgres/test_multiple_categorical_dimension_pushdown__plan0.sql index 70f27e867b..9dd9bdab7c 100644 --- a/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Postgres/test_multiple_categorical_dimension_pushdown__plan0.sql +++ b/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Postgres/test_multiple_categorical_dimension_pushdown__plan0.sql @@ -29,10 +29,10 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.user AS user + subq_4.home_state_latest AS user__home_state_latest + , subq_2.user AS user , subq_2.listing__is_lux_latest AS listing__is_lux_latest , subq_2.listing__capacity_latest AS listing__capacity_latest - , subq_4.home_state_latest AS user__home_state_latest , subq_2.listings AS listings FROM ( -- Pass Only Elements: ['listings', 'listing__is_lux_latest', 'listing__capacity_latest', 'user'] diff --git a/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Postgres/test_offset_metric_with_query_time_filters__plan0.sql b/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Postgres/test_offset_metric_with_query_time_filters__plan0.sql index bb2bedb198..d81e5cd4d7 100644 --- a/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Postgres/test_offset_metric_with_query_time_filters__plan0.sql +++ b/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Postgres/test_offset_metric_with_query_time_filters__plan0.sql @@ -45,10 +45,10 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.metric_time__day AS metric_time__day + subq_5.country_latest AS listing__country_latest + , subq_2.metric_time__day AS metric_time__day , subq_2.listing AS listing , subq_2.booking__is_instant AS booking__is_instant - , subq_5.country_latest AS listing__country_latest , subq_2.bookings AS bookings FROM ( -- Pass Only Elements: ['bookings', 'booking__is_instant', 'metric_time__day', 'listing'] @@ -438,10 +438,10 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_17.metric_time__day AS metric_time__day + subq_20.country_latest AS listing__country_latest + , subq_17.metric_time__day AS metric_time__day , subq_17.listing AS listing , subq_17.booking__is_instant AS booking__is_instant - , subq_20.country_latest AS listing__country_latest , subq_17.bookings AS bookings FROM ( -- Pass Only Elements: ['bookings', 'booking__is_instant', 'metric_time__day', 'listing'] diff --git a/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Postgres/test_saved_query_with_metric_joins_and_filter__plan0.sql b/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Postgres/test_saved_query_with_metric_joins_and_filter__plan0.sql index 43ff1fdc74..4186e8bfd4 100644 --- a/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Postgres/test_saved_query_with_metric_joins_and_filter__plan0.sql +++ b/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Postgres/test_saved_query_with_metric_joins_and_filter__plan0.sql @@ -36,10 +36,10 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.metric_time__day AS metric_time__day - , subq_2.listing AS listing - , subq_5.is_lux_latest AS listing__is_lux_latest + subq_5.is_lux_latest AS listing__is_lux_latest , subq_5.capacity_latest AS listing__capacity_latest + , subq_2.metric_time__day AS metric_time__day + , subq_2.listing AS listing , subq_2.bookings AS bookings FROM ( -- Pass Only Elements: ['bookings', 'metric_time__day', 'listing'] @@ -425,10 +425,10 @@ FULL OUTER JOIN ( FROM ( -- Join Standard Outputs SELECT - subq_14.metric_time__day AS metric_time__day - , subq_14.listing AS listing - , subq_17.is_lux_latest AS listing__is_lux_latest + subq_17.is_lux_latest AS listing__is_lux_latest , subq_17.capacity_latest AS listing__capacity_latest + , subq_14.metric_time__day AS metric_time__day + , subq_14.listing AS listing , subq_14.views AS views FROM ( -- Pass Only Elements: ['views', 'metric_time__day', 'listing'] @@ -750,10 +750,10 @@ FULL OUTER JOIN ( FROM ( -- Join Standard Outputs SELECT - subq_26.metric_time__day AS metric_time__day - , subq_26.listing AS listing - , subq_29.is_lux_latest AS listing__is_lux_latest + subq_29.is_lux_latest AS listing__is_lux_latest , subq_29.capacity_latest AS listing__capacity_latest + , subq_26.metric_time__day AS metric_time__day + , subq_26.listing AS listing , subq_26.bookings AS bookings FROM ( -- Pass Only Elements: ['bookings', 'metric_time__day', 'listing'] @@ -1139,10 +1139,10 @@ FULL OUTER JOIN ( FROM ( -- Join Standard Outputs SELECT - subq_38.metric_time__day AS metric_time__day - , subq_38.listing AS listing - , subq_41.is_lux_latest AS listing__is_lux_latest + subq_41.is_lux_latest AS listing__is_lux_latest , subq_41.capacity_latest AS listing__capacity_latest + , subq_38.metric_time__day AS metric_time__day + , subq_38.listing AS listing , subq_38.views AS views FROM ( -- Pass Only Elements: ['views', 'metric_time__day', 'listing'] diff --git a/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Postgres/test_single_categorical_dimension_pushdown__plan0.sql b/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Postgres/test_single_categorical_dimension_pushdown__plan0.sql index 14aba1afb1..5fc23ef4c0 100644 --- a/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Postgres/test_single_categorical_dimension_pushdown__plan0.sql +++ b/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Postgres/test_single_categorical_dimension_pushdown__plan0.sql @@ -27,9 +27,9 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.listing AS listing + subq_5.country_latest AS listing__country_latest + , subq_2.listing AS listing , subq_2.booking__is_instant AS booking__is_instant - , subq_5.country_latest AS listing__country_latest , subq_2.bookings AS bookings FROM ( -- Pass Only Elements: ['bookings', 'booking__is_instant', 'listing'] diff --git a/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Postgres/test_skipped_pushdown__plan0.sql b/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Postgres/test_skipped_pushdown__plan0.sql index a62b838b8c..8c638d3b5e 100644 --- a/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Postgres/test_skipped_pushdown__plan0.sql +++ b/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Postgres/test_skipped_pushdown__plan0.sql @@ -29,10 +29,10 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.listing AS listing - , subq_2.booking__is_instant AS booking__is_instant - , subq_5.country_latest AS listing__country_latest + subq_5.country_latest AS listing__country_latest , subq_5.is_lux_latest AS listing__is_lux_latest + , subq_2.listing AS listing + , subq_2.booking__is_instant AS booking__is_instant , subq_2.bookings AS bookings FROM ( -- Pass Only Elements: ['bookings', 'booking__is_instant', 'listing'] diff --git a/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Redshift/test_conversion_metric_query_filters__plan0.sql b/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Redshift/test_conversion_metric_query_filters__plan0.sql index 74836e8644..7a70602ed6 100644 --- a/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Redshift/test_conversion_metric_query_filters__plan0.sql +++ b/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Redshift/test_conversion_metric_query_filters__plan0.sql @@ -39,10 +39,10 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.metric_time__day AS metric_time__day + subq_4.home_state_latest AS user__home_state_latest + , subq_2.metric_time__day AS metric_time__day , subq_2.user AS user , subq_2.visit__referrer_id AS visit__referrer_id - , subq_4.home_state_latest AS user__home_state_latest , subq_2.visits AS visits FROM ( -- Pass Only Elements: ['visits', 'visit__referrer_id', 'metric_time__day', 'user'] @@ -301,7 +301,8 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_11.ds__day AS ds__day + subq_13.home_state_latest AS user__home_state_latest + , subq_11.ds__day AS ds__day , subq_11.ds__week AS ds__week , subq_11.ds__month AS ds__month , subq_11.ds__quarter AS ds__quarter @@ -340,7 +341,6 @@ FROM ( , subq_11.visit__session AS visit__session , subq_11.referrer_id AS referrer_id , subq_11.visit__referrer_id AS visit__referrer_id - , subq_13.home_state_latest AS user__home_state_latest , subq_11.visits AS visits , subq_11.visitors AS visitors FROM ( diff --git a/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Redshift/test_conversion_metric_query_filters__plan0_optimized.sql b/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Redshift/test_conversion_metric_query_filters__plan0_optimized.sql index 91da02375d..567729e351 100644 --- a/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Redshift/test_conversion_metric_query_filters__plan0_optimized.sql +++ b/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Redshift/test_conversion_metric_query_filters__plan0_optimized.sql @@ -112,10 +112,10 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_36.metric_time__day AS metric_time__day + users_latest_src_28000.home_state_latest AS user__home_state_latest + , subq_36.metric_time__day AS metric_time__day , subq_36.user AS user , subq_36.visit__referrer_id AS visit__referrer_id - , users_latest_src_28000.home_state_latest AS user__home_state_latest , subq_36.visits AS visits FROM ( -- Read Elements From Semantic Model 'visits_source' diff --git a/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Redshift/test_cumulative_metric_with_query_time_filters__plan0.sql b/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Redshift/test_cumulative_metric_with_query_time_filters__plan0.sql index ed75cd86b6..503211f4ba 100644 --- a/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Redshift/test_cumulative_metric_with_query_time_filters__plan0.sql +++ b/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Redshift/test_cumulative_metric_with_query_time_filters__plan0.sql @@ -32,10 +32,10 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_5.metric_time__day AS metric_time__day + subq_8.country_latest AS listing__country_latest + , subq_5.metric_time__day AS metric_time__day , subq_5.listing AS listing , subq_5.booking__is_instant AS booking__is_instant - , subq_8.country_latest AS listing__country_latest , subq_5.bookers AS bookers FROM ( -- Pass Only Elements: ['bookers', 'booking__is_instant', 'metric_time__day', 'listing'] diff --git a/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Redshift/test_fill_nulls_time_spine_metric_predicate_pushdown__plan0.sql b/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Redshift/test_fill_nulls_time_spine_metric_predicate_pushdown__plan0.sql index ddff072fa4..5122c0b31e 100644 --- a/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Redshift/test_fill_nulls_time_spine_metric_predicate_pushdown__plan0.sql +++ b/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Redshift/test_fill_nulls_time_spine_metric_predicate_pushdown__plan0.sql @@ -57,10 +57,10 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.metric_time__day AS metric_time__day + subq_5.country_latest AS listing__country_latest + , subq_2.metric_time__day AS metric_time__day , subq_2.listing AS listing , subq_2.booking__is_instant AS booking__is_instant - , subq_5.country_latest AS listing__country_latest , subq_2.bookings AS bookings FROM ( -- Pass Only Elements: ['bookings', 'booking__is_instant', 'metric_time__day', 'listing'] @@ -465,10 +465,10 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_20.metric_time__day AS metric_time__day + subq_23.country_latest AS listing__country_latest + , subq_20.metric_time__day AS metric_time__day , subq_20.listing AS listing , subq_20.booking__is_instant AS booking__is_instant - , subq_23.country_latest AS listing__country_latest , subq_20.bookings AS bookings FROM ( -- Pass Only Elements: ['bookings', 'booking__is_instant', 'metric_time__day', 'listing'] diff --git a/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Redshift/test_metric_time_filter_with_two_targets__plan0.sql b/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Redshift/test_metric_time_filter_with_two_targets__plan0.sql index 220689a85c..2f152299b1 100644 --- a/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Redshift/test_metric_time_filter_with_two_targets__plan0.sql +++ b/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Redshift/test_metric_time_filter_with_two_targets__plan0.sql @@ -27,9 +27,9 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.metric_time__day AS metric_time__day + subq_5.country_latest AS listing__country_latest + , subq_2.metric_time__day AS metric_time__day , subq_2.listing AS listing - , subq_5.country_latest AS listing__country_latest , subq_2.bookings AS bookings FROM ( -- Pass Only Elements: ['bookings', 'metric_time__day', 'listing'] diff --git a/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Redshift/test_multiple_categorical_dimension_pushdown__plan0.sql b/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Redshift/test_multiple_categorical_dimension_pushdown__plan0.sql index 05076323af..b65ad2b95b 100644 --- a/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Redshift/test_multiple_categorical_dimension_pushdown__plan0.sql +++ b/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Redshift/test_multiple_categorical_dimension_pushdown__plan0.sql @@ -29,10 +29,10 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.user AS user + subq_4.home_state_latest AS user__home_state_latest + , subq_2.user AS user , subq_2.listing__is_lux_latest AS listing__is_lux_latest , subq_2.listing__capacity_latest AS listing__capacity_latest - , subq_4.home_state_latest AS user__home_state_latest , subq_2.listings AS listings FROM ( -- Pass Only Elements: ['listings', 'listing__is_lux_latest', 'listing__capacity_latest', 'user'] diff --git a/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Redshift/test_offset_metric_with_query_time_filters__plan0.sql b/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Redshift/test_offset_metric_with_query_time_filters__plan0.sql index 9ddee043d2..148e44428a 100644 --- a/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Redshift/test_offset_metric_with_query_time_filters__plan0.sql +++ b/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Redshift/test_offset_metric_with_query_time_filters__plan0.sql @@ -45,10 +45,10 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.metric_time__day AS metric_time__day + subq_5.country_latest AS listing__country_latest + , subq_2.metric_time__day AS metric_time__day , subq_2.listing AS listing , subq_2.booking__is_instant AS booking__is_instant - , subq_5.country_latest AS listing__country_latest , subq_2.bookings AS bookings FROM ( -- Pass Only Elements: ['bookings', 'booking__is_instant', 'metric_time__day', 'listing'] @@ -438,10 +438,10 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_17.metric_time__day AS metric_time__day + subq_20.country_latest AS listing__country_latest + , subq_17.metric_time__day AS metric_time__day , subq_17.listing AS listing , subq_17.booking__is_instant AS booking__is_instant - , subq_20.country_latest AS listing__country_latest , subq_17.bookings AS bookings FROM ( -- Pass Only Elements: ['bookings', 'booking__is_instant', 'metric_time__day', 'listing'] diff --git a/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Redshift/test_saved_query_with_metric_joins_and_filter__plan0.sql b/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Redshift/test_saved_query_with_metric_joins_and_filter__plan0.sql index f70f5cf344..dbbc66559b 100644 --- a/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Redshift/test_saved_query_with_metric_joins_and_filter__plan0.sql +++ b/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Redshift/test_saved_query_with_metric_joins_and_filter__plan0.sql @@ -36,10 +36,10 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.metric_time__day AS metric_time__day - , subq_2.listing AS listing - , subq_5.is_lux_latest AS listing__is_lux_latest + subq_5.is_lux_latest AS listing__is_lux_latest , subq_5.capacity_latest AS listing__capacity_latest + , subq_2.metric_time__day AS metric_time__day + , subq_2.listing AS listing , subq_2.bookings AS bookings FROM ( -- Pass Only Elements: ['bookings', 'metric_time__day', 'listing'] @@ -425,10 +425,10 @@ FULL OUTER JOIN ( FROM ( -- Join Standard Outputs SELECT - subq_14.metric_time__day AS metric_time__day - , subq_14.listing AS listing - , subq_17.is_lux_latest AS listing__is_lux_latest + subq_17.is_lux_latest AS listing__is_lux_latest , subq_17.capacity_latest AS listing__capacity_latest + , subq_14.metric_time__day AS metric_time__day + , subq_14.listing AS listing , subq_14.views AS views FROM ( -- Pass Only Elements: ['views', 'metric_time__day', 'listing'] @@ -750,10 +750,10 @@ FULL OUTER JOIN ( FROM ( -- Join Standard Outputs SELECT - subq_26.metric_time__day AS metric_time__day - , subq_26.listing AS listing - , subq_29.is_lux_latest AS listing__is_lux_latest + subq_29.is_lux_latest AS listing__is_lux_latest , subq_29.capacity_latest AS listing__capacity_latest + , subq_26.metric_time__day AS metric_time__day + , subq_26.listing AS listing , subq_26.bookings AS bookings FROM ( -- Pass Only Elements: ['bookings', 'metric_time__day', 'listing'] @@ -1139,10 +1139,10 @@ FULL OUTER JOIN ( FROM ( -- Join Standard Outputs SELECT - subq_38.metric_time__day AS metric_time__day - , subq_38.listing AS listing - , subq_41.is_lux_latest AS listing__is_lux_latest + subq_41.is_lux_latest AS listing__is_lux_latest , subq_41.capacity_latest AS listing__capacity_latest + , subq_38.metric_time__day AS metric_time__day + , subq_38.listing AS listing , subq_38.views AS views FROM ( -- Pass Only Elements: ['views', 'metric_time__day', 'listing'] diff --git a/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Redshift/test_single_categorical_dimension_pushdown__plan0.sql b/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Redshift/test_single_categorical_dimension_pushdown__plan0.sql index ca12d47e7b..0419ff0896 100644 --- a/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Redshift/test_single_categorical_dimension_pushdown__plan0.sql +++ b/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Redshift/test_single_categorical_dimension_pushdown__plan0.sql @@ -27,9 +27,9 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.listing AS listing + subq_5.country_latest AS listing__country_latest + , subq_2.listing AS listing , subq_2.booking__is_instant AS booking__is_instant - , subq_5.country_latest AS listing__country_latest , subq_2.bookings AS bookings FROM ( -- Pass Only Elements: ['bookings', 'booking__is_instant', 'listing'] diff --git a/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Redshift/test_skipped_pushdown__plan0.sql b/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Redshift/test_skipped_pushdown__plan0.sql index 2c072d5394..c2b3bf1c19 100644 --- a/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Redshift/test_skipped_pushdown__plan0.sql +++ b/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Redshift/test_skipped_pushdown__plan0.sql @@ -29,10 +29,10 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.listing AS listing - , subq_2.booking__is_instant AS booking__is_instant - , subq_5.country_latest AS listing__country_latest + subq_5.country_latest AS listing__country_latest , subq_5.is_lux_latest AS listing__is_lux_latest + , subq_2.listing AS listing + , subq_2.booking__is_instant AS booking__is_instant , subq_2.bookings AS bookings FROM ( -- Pass Only Elements: ['bookings', 'booking__is_instant', 'listing'] diff --git a/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Snowflake/test_conversion_metric_query_filters__plan0.sql b/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Snowflake/test_conversion_metric_query_filters__plan0.sql index 38283c28fb..2efc5047fe 100644 --- a/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Snowflake/test_conversion_metric_query_filters__plan0.sql +++ b/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Snowflake/test_conversion_metric_query_filters__plan0.sql @@ -39,10 +39,10 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.metric_time__day AS metric_time__day + subq_4.home_state_latest AS user__home_state_latest + , subq_2.metric_time__day AS metric_time__day , subq_2.user AS user , subq_2.visit__referrer_id AS visit__referrer_id - , subq_4.home_state_latest AS user__home_state_latest , subq_2.visits AS visits FROM ( -- Pass Only Elements: ['visits', 'visit__referrer_id', 'metric_time__day', 'user'] @@ -301,7 +301,8 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_11.ds__day AS ds__day + subq_13.home_state_latest AS user__home_state_latest + , subq_11.ds__day AS ds__day , subq_11.ds__week AS ds__week , subq_11.ds__month AS ds__month , subq_11.ds__quarter AS ds__quarter @@ -340,7 +341,6 @@ FROM ( , subq_11.visit__session AS visit__session , subq_11.referrer_id AS referrer_id , subq_11.visit__referrer_id AS visit__referrer_id - , subq_13.home_state_latest AS user__home_state_latest , subq_11.visits AS visits , subq_11.visitors AS visitors FROM ( diff --git a/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Snowflake/test_conversion_metric_query_filters__plan0_optimized.sql b/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Snowflake/test_conversion_metric_query_filters__plan0_optimized.sql index 1993bb93cd..581340ed0f 100644 --- a/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Snowflake/test_conversion_metric_query_filters__plan0_optimized.sql +++ b/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Snowflake/test_conversion_metric_query_filters__plan0_optimized.sql @@ -112,10 +112,10 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_36.metric_time__day AS metric_time__day + users_latest_src_28000.home_state_latest AS user__home_state_latest + , subq_36.metric_time__day AS metric_time__day , subq_36.user AS user , subq_36.visit__referrer_id AS visit__referrer_id - , users_latest_src_28000.home_state_latest AS user__home_state_latest , subq_36.visits AS visits FROM ( -- Read Elements From Semantic Model 'visits_source' diff --git a/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Snowflake/test_cumulative_metric_with_query_time_filters__plan0.sql b/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Snowflake/test_cumulative_metric_with_query_time_filters__plan0.sql index b5836aa84e..3e68c769a7 100644 --- a/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Snowflake/test_cumulative_metric_with_query_time_filters__plan0.sql +++ b/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Snowflake/test_cumulative_metric_with_query_time_filters__plan0.sql @@ -32,10 +32,10 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_5.metric_time__day AS metric_time__day + subq_8.country_latest AS listing__country_latest + , subq_5.metric_time__day AS metric_time__day , subq_5.listing AS listing , subq_5.booking__is_instant AS booking__is_instant - , subq_8.country_latest AS listing__country_latest , subq_5.bookers AS bookers FROM ( -- Pass Only Elements: ['bookers', 'booking__is_instant', 'metric_time__day', 'listing'] diff --git a/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Snowflake/test_fill_nulls_time_spine_metric_predicate_pushdown__plan0.sql b/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Snowflake/test_fill_nulls_time_spine_metric_predicate_pushdown__plan0.sql index 2abe1d63f8..99537dad6a 100644 --- a/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Snowflake/test_fill_nulls_time_spine_metric_predicate_pushdown__plan0.sql +++ b/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Snowflake/test_fill_nulls_time_spine_metric_predicate_pushdown__plan0.sql @@ -57,10 +57,10 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.metric_time__day AS metric_time__day + subq_5.country_latest AS listing__country_latest + , subq_2.metric_time__day AS metric_time__day , subq_2.listing AS listing , subq_2.booking__is_instant AS booking__is_instant - , subq_5.country_latest AS listing__country_latest , subq_2.bookings AS bookings FROM ( -- Pass Only Elements: ['bookings', 'booking__is_instant', 'metric_time__day', 'listing'] @@ -465,10 +465,10 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_20.metric_time__day AS metric_time__day + subq_23.country_latest AS listing__country_latest + , subq_20.metric_time__day AS metric_time__day , subq_20.listing AS listing , subq_20.booking__is_instant AS booking__is_instant - , subq_23.country_latest AS listing__country_latest , subq_20.bookings AS bookings FROM ( -- Pass Only Elements: ['bookings', 'booking__is_instant', 'metric_time__day', 'listing'] diff --git a/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Snowflake/test_metric_time_filter_with_two_targets__plan0.sql b/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Snowflake/test_metric_time_filter_with_two_targets__plan0.sql index 0d615c925d..22832d0a62 100644 --- a/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Snowflake/test_metric_time_filter_with_two_targets__plan0.sql +++ b/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Snowflake/test_metric_time_filter_with_two_targets__plan0.sql @@ -27,9 +27,9 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.metric_time__day AS metric_time__day + subq_5.country_latest AS listing__country_latest + , subq_2.metric_time__day AS metric_time__day , subq_2.listing AS listing - , subq_5.country_latest AS listing__country_latest , subq_2.bookings AS bookings FROM ( -- Pass Only Elements: ['bookings', 'metric_time__day', 'listing'] diff --git a/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Snowflake/test_multiple_categorical_dimension_pushdown__plan0.sql b/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Snowflake/test_multiple_categorical_dimension_pushdown__plan0.sql index abf48552a5..d02ed59ec0 100644 --- a/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Snowflake/test_multiple_categorical_dimension_pushdown__plan0.sql +++ b/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Snowflake/test_multiple_categorical_dimension_pushdown__plan0.sql @@ -29,10 +29,10 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.user AS user + subq_4.home_state_latest AS user__home_state_latest + , subq_2.user AS user , subq_2.listing__is_lux_latest AS listing__is_lux_latest , subq_2.listing__capacity_latest AS listing__capacity_latest - , subq_4.home_state_latest AS user__home_state_latest , subq_2.listings AS listings FROM ( -- Pass Only Elements: ['listings', 'listing__is_lux_latest', 'listing__capacity_latest', 'user'] diff --git a/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Snowflake/test_offset_metric_with_query_time_filters__plan0.sql b/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Snowflake/test_offset_metric_with_query_time_filters__plan0.sql index 50abb77a30..51e8d74fc0 100644 --- a/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Snowflake/test_offset_metric_with_query_time_filters__plan0.sql +++ b/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Snowflake/test_offset_metric_with_query_time_filters__plan0.sql @@ -45,10 +45,10 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.metric_time__day AS metric_time__day + subq_5.country_latest AS listing__country_latest + , subq_2.metric_time__day AS metric_time__day , subq_2.listing AS listing , subq_2.booking__is_instant AS booking__is_instant - , subq_5.country_latest AS listing__country_latest , subq_2.bookings AS bookings FROM ( -- Pass Only Elements: ['bookings', 'booking__is_instant', 'metric_time__day', 'listing'] @@ -438,10 +438,10 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_17.metric_time__day AS metric_time__day + subq_20.country_latest AS listing__country_latest + , subq_17.metric_time__day AS metric_time__day , subq_17.listing AS listing , subq_17.booking__is_instant AS booking__is_instant - , subq_20.country_latest AS listing__country_latest , subq_17.bookings AS bookings FROM ( -- Pass Only Elements: ['bookings', 'booking__is_instant', 'metric_time__day', 'listing'] diff --git a/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Snowflake/test_saved_query_with_metric_joins_and_filter__plan0.sql b/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Snowflake/test_saved_query_with_metric_joins_and_filter__plan0.sql index a259cf2d3e..ad35b4be09 100644 --- a/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Snowflake/test_saved_query_with_metric_joins_and_filter__plan0.sql +++ b/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Snowflake/test_saved_query_with_metric_joins_and_filter__plan0.sql @@ -36,10 +36,10 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.metric_time__day AS metric_time__day - , subq_2.listing AS listing - , subq_5.is_lux_latest AS listing__is_lux_latest + subq_5.is_lux_latest AS listing__is_lux_latest , subq_5.capacity_latest AS listing__capacity_latest + , subq_2.metric_time__day AS metric_time__day + , subq_2.listing AS listing , subq_2.bookings AS bookings FROM ( -- Pass Only Elements: ['bookings', 'metric_time__day', 'listing'] @@ -425,10 +425,10 @@ FULL OUTER JOIN ( FROM ( -- Join Standard Outputs SELECT - subq_14.metric_time__day AS metric_time__day - , subq_14.listing AS listing - , subq_17.is_lux_latest AS listing__is_lux_latest + subq_17.is_lux_latest AS listing__is_lux_latest , subq_17.capacity_latest AS listing__capacity_latest + , subq_14.metric_time__day AS metric_time__day + , subq_14.listing AS listing , subq_14.views AS views FROM ( -- Pass Only Elements: ['views', 'metric_time__day', 'listing'] @@ -750,10 +750,10 @@ FULL OUTER JOIN ( FROM ( -- Join Standard Outputs SELECT - subq_26.metric_time__day AS metric_time__day - , subq_26.listing AS listing - , subq_29.is_lux_latest AS listing__is_lux_latest + subq_29.is_lux_latest AS listing__is_lux_latest , subq_29.capacity_latest AS listing__capacity_latest + , subq_26.metric_time__day AS metric_time__day + , subq_26.listing AS listing , subq_26.bookings AS bookings FROM ( -- Pass Only Elements: ['bookings', 'metric_time__day', 'listing'] @@ -1139,10 +1139,10 @@ FULL OUTER JOIN ( FROM ( -- Join Standard Outputs SELECT - subq_38.metric_time__day AS metric_time__day - , subq_38.listing AS listing - , subq_41.is_lux_latest AS listing__is_lux_latest + subq_41.is_lux_latest AS listing__is_lux_latest , subq_41.capacity_latest AS listing__capacity_latest + , subq_38.metric_time__day AS metric_time__day + , subq_38.listing AS listing , subq_38.views AS views FROM ( -- Pass Only Elements: ['views', 'metric_time__day', 'listing'] diff --git a/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Snowflake/test_single_categorical_dimension_pushdown__plan0.sql b/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Snowflake/test_single_categorical_dimension_pushdown__plan0.sql index 2609a5731f..a42afda3af 100644 --- a/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Snowflake/test_single_categorical_dimension_pushdown__plan0.sql +++ b/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Snowflake/test_single_categorical_dimension_pushdown__plan0.sql @@ -27,9 +27,9 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.listing AS listing + subq_5.country_latest AS listing__country_latest + , subq_2.listing AS listing , subq_2.booking__is_instant AS booking__is_instant - , subq_5.country_latest AS listing__country_latest , subq_2.bookings AS bookings FROM ( -- Pass Only Elements: ['bookings', 'booking__is_instant', 'listing'] diff --git a/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Snowflake/test_skipped_pushdown__plan0.sql b/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Snowflake/test_skipped_pushdown__plan0.sql index 00cabc8f0d..1a5ca51d1b 100644 --- a/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Snowflake/test_skipped_pushdown__plan0.sql +++ b/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Snowflake/test_skipped_pushdown__plan0.sql @@ -29,10 +29,10 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.listing AS listing - , subq_2.booking__is_instant AS booking__is_instant - , subq_5.country_latest AS listing__country_latest + subq_5.country_latest AS listing__country_latest , subq_5.is_lux_latest AS listing__is_lux_latest + , subq_2.listing AS listing + , subq_2.booking__is_instant AS booking__is_instant , subq_2.bookings AS bookings FROM ( -- Pass Only Elements: ['bookings', 'booking__is_instant', 'listing'] diff --git a/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Trino/test_conversion_metric_query_filters__plan0.sql b/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Trino/test_conversion_metric_query_filters__plan0.sql index 421207a198..fc7c8fd64e 100644 --- a/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Trino/test_conversion_metric_query_filters__plan0.sql +++ b/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Trino/test_conversion_metric_query_filters__plan0.sql @@ -39,10 +39,10 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.metric_time__day AS metric_time__day + subq_4.home_state_latest AS user__home_state_latest + , subq_2.metric_time__day AS metric_time__day , subq_2.user AS user , subq_2.visit__referrer_id AS visit__referrer_id - , subq_4.home_state_latest AS user__home_state_latest , subq_2.visits AS visits FROM ( -- Pass Only Elements: ['visits', 'visit__referrer_id', 'metric_time__day', 'user'] @@ -301,7 +301,8 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_11.ds__day AS ds__day + subq_13.home_state_latest AS user__home_state_latest + , subq_11.ds__day AS ds__day , subq_11.ds__week AS ds__week , subq_11.ds__month AS ds__month , subq_11.ds__quarter AS ds__quarter @@ -340,7 +341,6 @@ FROM ( , subq_11.visit__session AS visit__session , subq_11.referrer_id AS referrer_id , subq_11.visit__referrer_id AS visit__referrer_id - , subq_13.home_state_latest AS user__home_state_latest , subq_11.visits AS visits , subq_11.visitors AS visitors FROM ( diff --git a/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Trino/test_conversion_metric_query_filters__plan0_optimized.sql b/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Trino/test_conversion_metric_query_filters__plan0_optimized.sql index aeb9b7ad4f..db457a0d4c 100644 --- a/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Trino/test_conversion_metric_query_filters__plan0_optimized.sql +++ b/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Trino/test_conversion_metric_query_filters__plan0_optimized.sql @@ -112,10 +112,10 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_36.metric_time__day AS metric_time__day + users_latest_src_28000.home_state_latest AS user__home_state_latest + , subq_36.metric_time__day AS metric_time__day , subq_36.user AS user , subq_36.visit__referrer_id AS visit__referrer_id - , users_latest_src_28000.home_state_latest AS user__home_state_latest , subq_36.visits AS visits FROM ( -- Read Elements From Semantic Model 'visits_source' diff --git a/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Trino/test_cumulative_metric_with_query_time_filters__plan0.sql b/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Trino/test_cumulative_metric_with_query_time_filters__plan0.sql index 84395ca4b1..269a96e3b7 100644 --- a/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Trino/test_cumulative_metric_with_query_time_filters__plan0.sql +++ b/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Trino/test_cumulative_metric_with_query_time_filters__plan0.sql @@ -32,10 +32,10 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_5.metric_time__day AS metric_time__day + subq_8.country_latest AS listing__country_latest + , subq_5.metric_time__day AS metric_time__day , subq_5.listing AS listing , subq_5.booking__is_instant AS booking__is_instant - , subq_8.country_latest AS listing__country_latest , subq_5.bookers AS bookers FROM ( -- Pass Only Elements: ['bookers', 'booking__is_instant', 'metric_time__day', 'listing'] diff --git a/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Trino/test_fill_nulls_time_spine_metric_predicate_pushdown__plan0.sql b/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Trino/test_fill_nulls_time_spine_metric_predicate_pushdown__plan0.sql index b5ca222b03..e216213ca8 100644 --- a/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Trino/test_fill_nulls_time_spine_metric_predicate_pushdown__plan0.sql +++ b/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Trino/test_fill_nulls_time_spine_metric_predicate_pushdown__plan0.sql @@ -57,10 +57,10 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.metric_time__day AS metric_time__day + subq_5.country_latest AS listing__country_latest + , subq_2.metric_time__day AS metric_time__day , subq_2.listing AS listing , subq_2.booking__is_instant AS booking__is_instant - , subq_5.country_latest AS listing__country_latest , subq_2.bookings AS bookings FROM ( -- Pass Only Elements: ['bookings', 'booking__is_instant', 'metric_time__day', 'listing'] @@ -465,10 +465,10 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_20.metric_time__day AS metric_time__day + subq_23.country_latest AS listing__country_latest + , subq_20.metric_time__day AS metric_time__day , subq_20.listing AS listing , subq_20.booking__is_instant AS booking__is_instant - , subq_23.country_latest AS listing__country_latest , subq_20.bookings AS bookings FROM ( -- Pass Only Elements: ['bookings', 'booking__is_instant', 'metric_time__day', 'listing'] diff --git a/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Trino/test_metric_time_filter_with_two_targets__plan0.sql b/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Trino/test_metric_time_filter_with_two_targets__plan0.sql index f72b9e5a36..3d2201c867 100644 --- a/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Trino/test_metric_time_filter_with_two_targets__plan0.sql +++ b/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Trino/test_metric_time_filter_with_two_targets__plan0.sql @@ -27,9 +27,9 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.metric_time__day AS metric_time__day + subq_5.country_latest AS listing__country_latest + , subq_2.metric_time__day AS metric_time__day , subq_2.listing AS listing - , subq_5.country_latest AS listing__country_latest , subq_2.bookings AS bookings FROM ( -- Pass Only Elements: ['bookings', 'metric_time__day', 'listing'] diff --git a/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Trino/test_multiple_categorical_dimension_pushdown__plan0.sql b/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Trino/test_multiple_categorical_dimension_pushdown__plan0.sql index dd272e56ef..1a914de5eb 100644 --- a/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Trino/test_multiple_categorical_dimension_pushdown__plan0.sql +++ b/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Trino/test_multiple_categorical_dimension_pushdown__plan0.sql @@ -29,10 +29,10 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.user AS user + subq_4.home_state_latest AS user__home_state_latest + , subq_2.user AS user , subq_2.listing__is_lux_latest AS listing__is_lux_latest , subq_2.listing__capacity_latest AS listing__capacity_latest - , subq_4.home_state_latest AS user__home_state_latest , subq_2.listings AS listings FROM ( -- Pass Only Elements: ['listings', 'listing__is_lux_latest', 'listing__capacity_latest', 'user'] diff --git a/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Trino/test_offset_metric_with_query_time_filters__plan0.sql b/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Trino/test_offset_metric_with_query_time_filters__plan0.sql index 2444e2528e..ce7c5bf491 100644 --- a/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Trino/test_offset_metric_with_query_time_filters__plan0.sql +++ b/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Trino/test_offset_metric_with_query_time_filters__plan0.sql @@ -45,10 +45,10 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.metric_time__day AS metric_time__day + subq_5.country_latest AS listing__country_latest + , subq_2.metric_time__day AS metric_time__day , subq_2.listing AS listing , subq_2.booking__is_instant AS booking__is_instant - , subq_5.country_latest AS listing__country_latest , subq_2.bookings AS bookings FROM ( -- Pass Only Elements: ['bookings', 'booking__is_instant', 'metric_time__day', 'listing'] @@ -438,10 +438,10 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_17.metric_time__day AS metric_time__day + subq_20.country_latest AS listing__country_latest + , subq_17.metric_time__day AS metric_time__day , subq_17.listing AS listing , subq_17.booking__is_instant AS booking__is_instant - , subq_20.country_latest AS listing__country_latest , subq_17.bookings AS bookings FROM ( -- Pass Only Elements: ['bookings', 'booking__is_instant', 'metric_time__day', 'listing'] diff --git a/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Trino/test_saved_query_with_metric_joins_and_filter__plan0.sql b/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Trino/test_saved_query_with_metric_joins_and_filter__plan0.sql index ab47adc254..c7a63c63af 100644 --- a/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Trino/test_saved_query_with_metric_joins_and_filter__plan0.sql +++ b/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Trino/test_saved_query_with_metric_joins_and_filter__plan0.sql @@ -36,10 +36,10 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.metric_time__day AS metric_time__day - , subq_2.listing AS listing - , subq_5.is_lux_latest AS listing__is_lux_latest + subq_5.is_lux_latest AS listing__is_lux_latest , subq_5.capacity_latest AS listing__capacity_latest + , subq_2.metric_time__day AS metric_time__day + , subq_2.listing AS listing , subq_2.bookings AS bookings FROM ( -- Pass Only Elements: ['bookings', 'metric_time__day', 'listing'] @@ -425,10 +425,10 @@ FULL OUTER JOIN ( FROM ( -- Join Standard Outputs SELECT - subq_14.metric_time__day AS metric_time__day - , subq_14.listing AS listing - , subq_17.is_lux_latest AS listing__is_lux_latest + subq_17.is_lux_latest AS listing__is_lux_latest , subq_17.capacity_latest AS listing__capacity_latest + , subq_14.metric_time__day AS metric_time__day + , subq_14.listing AS listing , subq_14.views AS views FROM ( -- Pass Only Elements: ['views', 'metric_time__day', 'listing'] @@ -750,10 +750,10 @@ FULL OUTER JOIN ( FROM ( -- Join Standard Outputs SELECT - subq_26.metric_time__day AS metric_time__day - , subq_26.listing AS listing - , subq_29.is_lux_latest AS listing__is_lux_latest + subq_29.is_lux_latest AS listing__is_lux_latest , subq_29.capacity_latest AS listing__capacity_latest + , subq_26.metric_time__day AS metric_time__day + , subq_26.listing AS listing , subq_26.bookings AS bookings FROM ( -- Pass Only Elements: ['bookings', 'metric_time__day', 'listing'] @@ -1139,10 +1139,10 @@ FULL OUTER JOIN ( FROM ( -- Join Standard Outputs SELECT - subq_38.metric_time__day AS metric_time__day - , subq_38.listing AS listing - , subq_41.is_lux_latest AS listing__is_lux_latest + subq_41.is_lux_latest AS listing__is_lux_latest , subq_41.capacity_latest AS listing__capacity_latest + , subq_38.metric_time__day AS metric_time__day + , subq_38.listing AS listing , subq_38.views AS views FROM ( -- Pass Only Elements: ['views', 'metric_time__day', 'listing'] diff --git a/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Trino/test_single_categorical_dimension_pushdown__plan0.sql b/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Trino/test_single_categorical_dimension_pushdown__plan0.sql index 4f90619754..9910ef6159 100644 --- a/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Trino/test_single_categorical_dimension_pushdown__plan0.sql +++ b/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Trino/test_single_categorical_dimension_pushdown__plan0.sql @@ -27,9 +27,9 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.listing AS listing + subq_5.country_latest AS listing__country_latest + , subq_2.listing AS listing , subq_2.booking__is_instant AS booking__is_instant - , subq_5.country_latest AS listing__country_latest , subq_2.bookings AS bookings FROM ( -- Pass Only Elements: ['bookings', 'booking__is_instant', 'listing'] diff --git a/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Trino/test_skipped_pushdown__plan0.sql b/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Trino/test_skipped_pushdown__plan0.sql index 02b7d9119b..13bfe67ed2 100644 --- a/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Trino/test_skipped_pushdown__plan0.sql +++ b/tests_metricflow/snapshots/test_predicate_pushdown_rendering.py/SqlQueryPlan/Trino/test_skipped_pushdown__plan0.sql @@ -29,10 +29,10 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.listing AS listing - , subq_2.booking__is_instant AS booking__is_instant - , subq_5.country_latest AS listing__country_latest + subq_5.country_latest AS listing__country_latest , subq_5.is_lux_latest AS listing__is_lux_latest + , subq_2.listing AS listing + , subq_2.booking__is_instant AS booking__is_instant , subq_2.bookings AS bookings FROM ( -- Pass Only Elements: ['bookings', 'booking__is_instant', 'listing'] diff --git a/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/BigQuery/test_filter_with_where_constraint_on_join_dim__plan0.sql b/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/BigQuery/test_filter_with_where_constraint_on_join_dim__plan0.sql index f2721226e3..15d05a41a4 100644 --- a/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/BigQuery/test_filter_with_where_constraint_on_join_dim__plan0.sql +++ b/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/BigQuery/test_filter_with_where_constraint_on_join_dim__plan0.sql @@ -27,9 +27,9 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.listing AS listing + subq_5.country_latest AS listing__country_latest + , subq_2.listing AS listing , subq_2.booking__is_instant AS booking__is_instant - , subq_5.country_latest AS listing__country_latest , subq_2.bookings AS bookings FROM ( -- Pass Only Elements: ['bookings', 'booking__is_instant', 'listing'] diff --git a/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/BigQuery/test_join_to_scd_dimension__plan0.sql b/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/BigQuery/test_join_to_scd_dimension__plan0.sql index b2a9819846..fb3160b25d 100644 --- a/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/BigQuery/test_join_to_scd_dimension__plan0.sql +++ b/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/BigQuery/test_join_to_scd_dimension__plan0.sql @@ -27,11 +27,11 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.metric_time__day AS metric_time__day + subq_4.capacity AS listing__capacity , subq_4.window_start__day AS listing__window_start__day , subq_4.window_end__day AS listing__window_end__day + , subq_2.metric_time__day AS metric_time__day , subq_2.listing AS listing - , subq_4.capacity AS listing__capacity , subq_2.bookings AS bookings FROM ( -- Pass Only Elements: ['bookings', 'metric_time__day', 'listing'] diff --git a/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/BigQuery/test_measure_constraint__plan0.sql b/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/BigQuery/test_measure_constraint__plan0.sql index 027af7df06..6fe39bd079 100644 --- a/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/BigQuery/test_measure_constraint__plan0.sql +++ b/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/BigQuery/test_measure_constraint__plan0.sql @@ -39,9 +39,9 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.metric_time__day AS metric_time__day + subq_5.is_lux_latest AS listing__is_lux_latest + , subq_2.metric_time__day AS metric_time__day , subq_2.listing AS listing - , subq_5.is_lux_latest AS listing__is_lux_latest , subq_2.average_booking_value AS average_booking_value FROM ( -- Pass Only Elements: ['average_booking_value', 'metric_time__day', 'listing'] @@ -424,9 +424,9 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_14.metric_time__day AS metric_time__day + subq_17.is_lux_latest AS listing__is_lux_latest + , subq_14.metric_time__day AS metric_time__day , subq_14.listing AS listing - , subq_17.is_lux_latest AS listing__is_lux_latest , subq_14.bookings AS bookings FROM ( -- Pass Only Elements: ['bookings', 'metric_time__day', 'listing'] diff --git a/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/BigQuery/test_multi_hop_through_scd_dimension__plan0.sql b/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/BigQuery/test_multi_hop_through_scd_dimension__plan0.sql index 3560e89b7b..ccb19f439d 100644 --- a/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/BigQuery/test_multi_hop_through_scd_dimension__plan0.sql +++ b/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/BigQuery/test_multi_hop_through_scd_dimension__plan0.sql @@ -18,11 +18,11 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.metric_time__day AS metric_time__day + subq_7.user__home_state_latest AS listing__user__home_state_latest , subq_7.window_start__day AS listing__window_start__day , subq_7.window_end__day AS listing__window_end__day + , subq_2.metric_time__day AS metric_time__day , subq_2.listing AS listing - , subq_7.user__home_state_latest AS listing__user__home_state_latest , subq_2.bookings AS bookings FROM ( -- Pass Only Elements: ['bookings', 'metric_time__day', 'listing'] @@ -224,7 +224,19 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_3.window_start__day AS window_start__day + subq_5.home_state_latest AS user__home_state_latest + , subq_5.ds__day AS user__ds__day + , subq_5.ds__week AS user__ds__week + , subq_5.ds__month AS user__ds__month + , subq_5.ds__quarter AS user__ds__quarter + , subq_5.ds__year AS user__ds__year + , subq_5.ds__extract_year AS user__ds__extract_year + , subq_5.ds__extract_quarter AS user__ds__extract_quarter + , subq_5.ds__extract_month AS user__ds__extract_month + , subq_5.ds__extract_day AS user__ds__extract_day + , subq_5.ds__extract_dow AS user__ds__extract_dow + , subq_5.ds__extract_doy AS user__ds__extract_doy + , subq_3.window_start__day AS window_start__day , subq_3.window_start__week AS window_start__week , subq_3.window_start__month AS window_start__month , subq_3.window_start__quarter AS window_start__quarter @@ -268,17 +280,6 @@ FROM ( , subq_3.listing__window_end__extract_day AS listing__window_end__extract_day , subq_3.listing__window_end__extract_dow AS listing__window_end__extract_dow , subq_3.listing__window_end__extract_doy AS listing__window_end__extract_doy - , subq_5.ds__day AS user__ds__day - , subq_5.ds__week AS user__ds__week - , subq_5.ds__month AS user__ds__month - , subq_5.ds__quarter AS user__ds__quarter - , subq_5.ds__year AS user__ds__year - , subq_5.ds__extract_year AS user__ds__extract_year - , subq_5.ds__extract_quarter AS user__ds__extract_quarter - , subq_5.ds__extract_month AS user__ds__extract_month - , subq_5.ds__extract_day AS user__ds__extract_day - , subq_5.ds__extract_dow AS user__ds__extract_dow - , subq_5.ds__extract_doy AS user__ds__extract_doy , subq_3.listing AS listing , subq_3.user AS user , subq_3.listing__user AS listing__user @@ -288,7 +289,6 @@ FROM ( , subq_3.listing__country AS listing__country , subq_3.listing__is_lux AS listing__is_lux , subq_3.listing__capacity AS listing__capacity - , subq_5.home_state_latest AS user__home_state_latest FROM ( -- Read Elements From Semantic Model 'listings' SELECT diff --git a/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/BigQuery/test_multi_hop_to_scd_dimension__plan0.sql b/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/BigQuery/test_multi_hop_to_scd_dimension__plan0.sql index 344de3883f..c20a671fd7 100644 --- a/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/BigQuery/test_multi_hop_to_scd_dimension__plan0.sql +++ b/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/BigQuery/test_multi_hop_to_scd_dimension__plan0.sql @@ -18,11 +18,11 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.metric_time__day AS metric_time__day + subq_7.lux_listing__is_confirmed_lux AS listing__lux_listing__is_confirmed_lux , subq_7.lux_listing__window_start__day AS listing__lux_listing__window_start__day , subq_7.lux_listing__window_end__day AS listing__lux_listing__window_end__day + , subq_2.metric_time__day AS metric_time__day , subq_2.listing AS listing - , subq_7.lux_listing__is_confirmed_lux AS listing__lux_listing__is_confirmed_lux , subq_2.bookings AS bookings FROM ( -- Pass Only Elements: ['bookings', 'metric_time__day', 'listing'] @@ -224,7 +224,8 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_5.window_start__day AS lux_listing__window_start__day + subq_5.is_confirmed_lux AS lux_listing__is_confirmed_lux + , subq_5.window_start__day AS lux_listing__window_start__day , subq_5.window_start__week AS lux_listing__window_start__week , subq_5.window_start__month AS lux_listing__window_start__month , subq_5.window_start__quarter AS lux_listing__window_start__quarter @@ -249,7 +250,6 @@ FROM ( , subq_3.listing AS listing , subq_3.lux_listing AS lux_listing , subq_3.listing__lux_listing AS listing__lux_listing - , subq_5.is_confirmed_lux AS lux_listing__is_confirmed_lux FROM ( -- Read Elements From Semantic Model 'lux_listing_mapping' SELECT diff --git a/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/BigQuery/test_multihop_node__plan0.sql b/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/BigQuery/test_multihop_node__plan0.sql index 9b356cf368..d4181dd14d 100644 --- a/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/BigQuery/test_multihop_node__plan0.sql +++ b/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/BigQuery/test_multihop_node__plan0.sql @@ -15,10 +15,10 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.ds_partitioned__day AS ds_partitioned__day + subq_9.customer_id__customer_name AS account_id__customer_id__customer_name , subq_9.ds_partitioned__day AS account_id__ds_partitioned__day + , subq_2.ds_partitioned__day AS ds_partitioned__day , subq_2.account_id AS account_id - , subq_9.customer_id__customer_name AS account_id__customer_id__customer_name , subq_2.txn_count AS txn_count FROM ( -- Pass Only Elements: ['txn_count', 'ds_partitioned__day', 'account_id'] @@ -152,7 +152,31 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_4.ds_partitioned__day AS ds_partitioned__day + subq_7.customer_name AS customer_id__customer_name + , subq_7.customer_atomic_weight AS customer_id__customer_atomic_weight + , subq_7.ds_partitioned__day AS customer_id__ds_partitioned__day + , subq_7.ds_partitioned__week AS customer_id__ds_partitioned__week + , subq_7.ds_partitioned__month AS customer_id__ds_partitioned__month + , subq_7.ds_partitioned__quarter AS customer_id__ds_partitioned__quarter + , subq_7.ds_partitioned__year AS customer_id__ds_partitioned__year + , subq_7.ds_partitioned__extract_year AS customer_id__ds_partitioned__extract_year + , subq_7.ds_partitioned__extract_quarter AS customer_id__ds_partitioned__extract_quarter + , subq_7.ds_partitioned__extract_month AS customer_id__ds_partitioned__extract_month + , subq_7.ds_partitioned__extract_day AS customer_id__ds_partitioned__extract_day + , subq_7.ds_partitioned__extract_dow AS customer_id__ds_partitioned__extract_dow + , subq_7.ds_partitioned__extract_doy AS customer_id__ds_partitioned__extract_doy + , subq_7.metric_time__day AS customer_id__metric_time__day + , subq_7.metric_time__week AS customer_id__metric_time__week + , subq_7.metric_time__month AS customer_id__metric_time__month + , subq_7.metric_time__quarter AS customer_id__metric_time__quarter + , subq_7.metric_time__year AS customer_id__metric_time__year + , subq_7.metric_time__extract_year AS customer_id__metric_time__extract_year + , subq_7.metric_time__extract_quarter AS customer_id__metric_time__extract_quarter + , subq_7.metric_time__extract_month AS customer_id__metric_time__extract_month + , subq_7.metric_time__extract_day AS customer_id__metric_time__extract_day + , subq_7.metric_time__extract_dow AS customer_id__metric_time__extract_dow + , subq_7.metric_time__extract_doy AS customer_id__metric_time__extract_doy + , subq_4.ds_partitioned__day AS ds_partitioned__day , subq_4.ds_partitioned__week AS ds_partitioned__week , subq_4.ds_partitioned__month AS ds_partitioned__month , subq_4.ds_partitioned__quarter AS ds_partitioned__quarter @@ -196,28 +220,6 @@ FROM ( , subq_4.metric_time__extract_day AS metric_time__extract_day , subq_4.metric_time__extract_dow AS metric_time__extract_dow , subq_4.metric_time__extract_doy AS metric_time__extract_doy - , subq_7.ds_partitioned__day AS customer_id__ds_partitioned__day - , subq_7.ds_partitioned__week AS customer_id__ds_partitioned__week - , subq_7.ds_partitioned__month AS customer_id__ds_partitioned__month - , subq_7.ds_partitioned__quarter AS customer_id__ds_partitioned__quarter - , subq_7.ds_partitioned__year AS customer_id__ds_partitioned__year - , subq_7.ds_partitioned__extract_year AS customer_id__ds_partitioned__extract_year - , subq_7.ds_partitioned__extract_quarter AS customer_id__ds_partitioned__extract_quarter - , subq_7.ds_partitioned__extract_month AS customer_id__ds_partitioned__extract_month - , subq_7.ds_partitioned__extract_day AS customer_id__ds_partitioned__extract_day - , subq_7.ds_partitioned__extract_dow AS customer_id__ds_partitioned__extract_dow - , subq_7.ds_partitioned__extract_doy AS customer_id__ds_partitioned__extract_doy - , subq_7.metric_time__day AS customer_id__metric_time__day - , subq_7.metric_time__week AS customer_id__metric_time__week - , subq_7.metric_time__month AS customer_id__metric_time__month - , subq_7.metric_time__quarter AS customer_id__metric_time__quarter - , subq_7.metric_time__year AS customer_id__metric_time__year - , subq_7.metric_time__extract_year AS customer_id__metric_time__extract_year - , subq_7.metric_time__extract_quarter AS customer_id__metric_time__extract_quarter - , subq_7.metric_time__extract_month AS customer_id__metric_time__extract_month - , subq_7.metric_time__extract_day AS customer_id__metric_time__extract_day - , subq_7.metric_time__extract_dow AS customer_id__metric_time__extract_dow - , subq_7.metric_time__extract_doy AS customer_id__metric_time__extract_doy , subq_4.account_id AS account_id , subq_4.customer_id AS customer_id , subq_4.account_id__customer_id AS account_id__customer_id @@ -226,8 +228,6 @@ FROM ( , subq_4.extra_dim AS extra_dim , subq_4.account_id__extra_dim AS account_id__extra_dim , subq_4.bridge_account__extra_dim AS bridge_account__extra_dim - , subq_7.customer_name AS customer_id__customer_name - , subq_7.customer_atomic_weight AS customer_id__customer_atomic_weight , subq_4.account_customer_combos AS account_customer_combos FROM ( -- Metric Time Dimension 'ds_partitioned' diff --git a/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/BigQuery/test_partitioned_join__plan0.sql b/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/BigQuery/test_partitioned_join__plan0.sql index c8b50dbfa4..f0b7a55606 100644 --- a/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/BigQuery/test_partitioned_join__plan0.sql +++ b/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/BigQuery/test_partitioned_join__plan0.sql @@ -15,10 +15,10 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.ds_partitioned__day AS ds_partitioned__day + subq_5.home_state AS user__home_state , subq_5.ds_partitioned__day AS user__ds_partitioned__day + , subq_2.ds_partitioned__day AS ds_partitioned__day , subq_2.user AS user - , subq_5.home_state AS user__home_state , subq_2.identity_verifications AS identity_verifications FROM ( -- Pass Only Elements: ['identity_verifications', 'ds_partitioned__day', 'user'] diff --git a/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/Databricks/test_filter_with_where_constraint_on_join_dim__plan0.sql b/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/Databricks/test_filter_with_where_constraint_on_join_dim__plan0.sql index ceabe93cc3..d1d6bf4fc2 100644 --- a/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/Databricks/test_filter_with_where_constraint_on_join_dim__plan0.sql +++ b/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/Databricks/test_filter_with_where_constraint_on_join_dim__plan0.sql @@ -27,9 +27,9 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.listing AS listing + subq_5.country_latest AS listing__country_latest + , subq_2.listing AS listing , subq_2.booking__is_instant AS booking__is_instant - , subq_5.country_latest AS listing__country_latest , subq_2.bookings AS bookings FROM ( -- Pass Only Elements: ['bookings', 'booking__is_instant', 'listing'] diff --git a/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/Databricks/test_join_to_scd_dimension__plan0.sql b/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/Databricks/test_join_to_scd_dimension__plan0.sql index 7f6a50d2d6..477d954daf 100644 --- a/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/Databricks/test_join_to_scd_dimension__plan0.sql +++ b/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/Databricks/test_join_to_scd_dimension__plan0.sql @@ -27,11 +27,11 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.metric_time__day AS metric_time__day + subq_4.capacity AS listing__capacity , subq_4.window_start__day AS listing__window_start__day , subq_4.window_end__day AS listing__window_end__day + , subq_2.metric_time__day AS metric_time__day , subq_2.listing AS listing - , subq_4.capacity AS listing__capacity , subq_2.bookings AS bookings FROM ( -- Pass Only Elements: ['bookings', 'metric_time__day', 'listing'] diff --git a/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/Databricks/test_measure_constraint__plan0.sql b/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/Databricks/test_measure_constraint__plan0.sql index ecab24365e..38d7fcc594 100644 --- a/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/Databricks/test_measure_constraint__plan0.sql +++ b/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/Databricks/test_measure_constraint__plan0.sql @@ -39,9 +39,9 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.metric_time__day AS metric_time__day + subq_5.is_lux_latest AS listing__is_lux_latest + , subq_2.metric_time__day AS metric_time__day , subq_2.listing AS listing - , subq_5.is_lux_latest AS listing__is_lux_latest , subq_2.average_booking_value AS average_booking_value FROM ( -- Pass Only Elements: ['average_booking_value', 'metric_time__day', 'listing'] @@ -424,9 +424,9 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_14.metric_time__day AS metric_time__day + subq_17.is_lux_latest AS listing__is_lux_latest + , subq_14.metric_time__day AS metric_time__day , subq_14.listing AS listing - , subq_17.is_lux_latest AS listing__is_lux_latest , subq_14.bookings AS bookings FROM ( -- Pass Only Elements: ['bookings', 'metric_time__day', 'listing'] diff --git a/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/Databricks/test_multi_hop_through_scd_dimension__plan0.sql b/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/Databricks/test_multi_hop_through_scd_dimension__plan0.sql index c9ccb81485..2f03037fef 100644 --- a/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/Databricks/test_multi_hop_through_scd_dimension__plan0.sql +++ b/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/Databricks/test_multi_hop_through_scd_dimension__plan0.sql @@ -18,11 +18,11 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.metric_time__day AS metric_time__day + subq_7.user__home_state_latest AS listing__user__home_state_latest , subq_7.window_start__day AS listing__window_start__day , subq_7.window_end__day AS listing__window_end__day + , subq_2.metric_time__day AS metric_time__day , subq_2.listing AS listing - , subq_7.user__home_state_latest AS listing__user__home_state_latest , subq_2.bookings AS bookings FROM ( -- Pass Only Elements: ['bookings', 'metric_time__day', 'listing'] @@ -224,7 +224,19 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_3.window_start__day AS window_start__day + subq_5.home_state_latest AS user__home_state_latest + , subq_5.ds__day AS user__ds__day + , subq_5.ds__week AS user__ds__week + , subq_5.ds__month AS user__ds__month + , subq_5.ds__quarter AS user__ds__quarter + , subq_5.ds__year AS user__ds__year + , subq_5.ds__extract_year AS user__ds__extract_year + , subq_5.ds__extract_quarter AS user__ds__extract_quarter + , subq_5.ds__extract_month AS user__ds__extract_month + , subq_5.ds__extract_day AS user__ds__extract_day + , subq_5.ds__extract_dow AS user__ds__extract_dow + , subq_5.ds__extract_doy AS user__ds__extract_doy + , subq_3.window_start__day AS window_start__day , subq_3.window_start__week AS window_start__week , subq_3.window_start__month AS window_start__month , subq_3.window_start__quarter AS window_start__quarter @@ -268,17 +280,6 @@ FROM ( , subq_3.listing__window_end__extract_day AS listing__window_end__extract_day , subq_3.listing__window_end__extract_dow AS listing__window_end__extract_dow , subq_3.listing__window_end__extract_doy AS listing__window_end__extract_doy - , subq_5.ds__day AS user__ds__day - , subq_5.ds__week AS user__ds__week - , subq_5.ds__month AS user__ds__month - , subq_5.ds__quarter AS user__ds__quarter - , subq_5.ds__year AS user__ds__year - , subq_5.ds__extract_year AS user__ds__extract_year - , subq_5.ds__extract_quarter AS user__ds__extract_quarter - , subq_5.ds__extract_month AS user__ds__extract_month - , subq_5.ds__extract_day AS user__ds__extract_day - , subq_5.ds__extract_dow AS user__ds__extract_dow - , subq_5.ds__extract_doy AS user__ds__extract_doy , subq_3.listing AS listing , subq_3.user AS user , subq_3.listing__user AS listing__user @@ -288,7 +289,6 @@ FROM ( , subq_3.listing__country AS listing__country , subq_3.listing__is_lux AS listing__is_lux , subq_3.listing__capacity AS listing__capacity - , subq_5.home_state_latest AS user__home_state_latest FROM ( -- Read Elements From Semantic Model 'listings' SELECT diff --git a/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/Databricks/test_multi_hop_to_scd_dimension__plan0.sql b/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/Databricks/test_multi_hop_to_scd_dimension__plan0.sql index ed2197c59b..709e0720a8 100644 --- a/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/Databricks/test_multi_hop_to_scd_dimension__plan0.sql +++ b/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/Databricks/test_multi_hop_to_scd_dimension__plan0.sql @@ -18,11 +18,11 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.metric_time__day AS metric_time__day + subq_7.lux_listing__is_confirmed_lux AS listing__lux_listing__is_confirmed_lux , subq_7.lux_listing__window_start__day AS listing__lux_listing__window_start__day , subq_7.lux_listing__window_end__day AS listing__lux_listing__window_end__day + , subq_2.metric_time__day AS metric_time__day , subq_2.listing AS listing - , subq_7.lux_listing__is_confirmed_lux AS listing__lux_listing__is_confirmed_lux , subq_2.bookings AS bookings FROM ( -- Pass Only Elements: ['bookings', 'metric_time__day', 'listing'] @@ -224,7 +224,8 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_5.window_start__day AS lux_listing__window_start__day + subq_5.is_confirmed_lux AS lux_listing__is_confirmed_lux + , subq_5.window_start__day AS lux_listing__window_start__day , subq_5.window_start__week AS lux_listing__window_start__week , subq_5.window_start__month AS lux_listing__window_start__month , subq_5.window_start__quarter AS lux_listing__window_start__quarter @@ -249,7 +250,6 @@ FROM ( , subq_3.listing AS listing , subq_3.lux_listing AS lux_listing , subq_3.listing__lux_listing AS listing__lux_listing - , subq_5.is_confirmed_lux AS lux_listing__is_confirmed_lux FROM ( -- Read Elements From Semantic Model 'lux_listing_mapping' SELECT diff --git a/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/Databricks/test_multihop_node__plan0.sql b/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/Databricks/test_multihop_node__plan0.sql index 28a6ba862c..a5a7d0a80c 100644 --- a/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/Databricks/test_multihop_node__plan0.sql +++ b/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/Databricks/test_multihop_node__plan0.sql @@ -15,10 +15,10 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.ds_partitioned__day AS ds_partitioned__day + subq_9.customer_id__customer_name AS account_id__customer_id__customer_name , subq_9.ds_partitioned__day AS account_id__ds_partitioned__day + , subq_2.ds_partitioned__day AS ds_partitioned__day , subq_2.account_id AS account_id - , subq_9.customer_id__customer_name AS account_id__customer_id__customer_name , subq_2.txn_count AS txn_count FROM ( -- Pass Only Elements: ['txn_count', 'ds_partitioned__day', 'account_id'] @@ -152,7 +152,31 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_4.ds_partitioned__day AS ds_partitioned__day + subq_7.customer_name AS customer_id__customer_name + , subq_7.customer_atomic_weight AS customer_id__customer_atomic_weight + , subq_7.ds_partitioned__day AS customer_id__ds_partitioned__day + , subq_7.ds_partitioned__week AS customer_id__ds_partitioned__week + , subq_7.ds_partitioned__month AS customer_id__ds_partitioned__month + , subq_7.ds_partitioned__quarter AS customer_id__ds_partitioned__quarter + , subq_7.ds_partitioned__year AS customer_id__ds_partitioned__year + , subq_7.ds_partitioned__extract_year AS customer_id__ds_partitioned__extract_year + , subq_7.ds_partitioned__extract_quarter AS customer_id__ds_partitioned__extract_quarter + , subq_7.ds_partitioned__extract_month AS customer_id__ds_partitioned__extract_month + , subq_7.ds_partitioned__extract_day AS customer_id__ds_partitioned__extract_day + , subq_7.ds_partitioned__extract_dow AS customer_id__ds_partitioned__extract_dow + , subq_7.ds_partitioned__extract_doy AS customer_id__ds_partitioned__extract_doy + , subq_7.metric_time__day AS customer_id__metric_time__day + , subq_7.metric_time__week AS customer_id__metric_time__week + , subq_7.metric_time__month AS customer_id__metric_time__month + , subq_7.metric_time__quarter AS customer_id__metric_time__quarter + , subq_7.metric_time__year AS customer_id__metric_time__year + , subq_7.metric_time__extract_year AS customer_id__metric_time__extract_year + , subq_7.metric_time__extract_quarter AS customer_id__metric_time__extract_quarter + , subq_7.metric_time__extract_month AS customer_id__metric_time__extract_month + , subq_7.metric_time__extract_day AS customer_id__metric_time__extract_day + , subq_7.metric_time__extract_dow AS customer_id__metric_time__extract_dow + , subq_7.metric_time__extract_doy AS customer_id__metric_time__extract_doy + , subq_4.ds_partitioned__day AS ds_partitioned__day , subq_4.ds_partitioned__week AS ds_partitioned__week , subq_4.ds_partitioned__month AS ds_partitioned__month , subq_4.ds_partitioned__quarter AS ds_partitioned__quarter @@ -196,28 +220,6 @@ FROM ( , subq_4.metric_time__extract_day AS metric_time__extract_day , subq_4.metric_time__extract_dow AS metric_time__extract_dow , subq_4.metric_time__extract_doy AS metric_time__extract_doy - , subq_7.ds_partitioned__day AS customer_id__ds_partitioned__day - , subq_7.ds_partitioned__week AS customer_id__ds_partitioned__week - , subq_7.ds_partitioned__month AS customer_id__ds_partitioned__month - , subq_7.ds_partitioned__quarter AS customer_id__ds_partitioned__quarter - , subq_7.ds_partitioned__year AS customer_id__ds_partitioned__year - , subq_7.ds_partitioned__extract_year AS customer_id__ds_partitioned__extract_year - , subq_7.ds_partitioned__extract_quarter AS customer_id__ds_partitioned__extract_quarter - , subq_7.ds_partitioned__extract_month AS customer_id__ds_partitioned__extract_month - , subq_7.ds_partitioned__extract_day AS customer_id__ds_partitioned__extract_day - , subq_7.ds_partitioned__extract_dow AS customer_id__ds_partitioned__extract_dow - , subq_7.ds_partitioned__extract_doy AS customer_id__ds_partitioned__extract_doy - , subq_7.metric_time__day AS customer_id__metric_time__day - , subq_7.metric_time__week AS customer_id__metric_time__week - , subq_7.metric_time__month AS customer_id__metric_time__month - , subq_7.metric_time__quarter AS customer_id__metric_time__quarter - , subq_7.metric_time__year AS customer_id__metric_time__year - , subq_7.metric_time__extract_year AS customer_id__metric_time__extract_year - , subq_7.metric_time__extract_quarter AS customer_id__metric_time__extract_quarter - , subq_7.metric_time__extract_month AS customer_id__metric_time__extract_month - , subq_7.metric_time__extract_day AS customer_id__metric_time__extract_day - , subq_7.metric_time__extract_dow AS customer_id__metric_time__extract_dow - , subq_7.metric_time__extract_doy AS customer_id__metric_time__extract_doy , subq_4.account_id AS account_id , subq_4.customer_id AS customer_id , subq_4.account_id__customer_id AS account_id__customer_id @@ -226,8 +228,6 @@ FROM ( , subq_4.extra_dim AS extra_dim , subq_4.account_id__extra_dim AS account_id__extra_dim , subq_4.bridge_account__extra_dim AS bridge_account__extra_dim - , subq_7.customer_name AS customer_id__customer_name - , subq_7.customer_atomic_weight AS customer_id__customer_atomic_weight , subq_4.account_customer_combos AS account_customer_combos FROM ( -- Metric Time Dimension 'ds_partitioned' diff --git a/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/Databricks/test_partitioned_join__plan0.sql b/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/Databricks/test_partitioned_join__plan0.sql index c98923aec3..f4e2ab7a3d 100644 --- a/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/Databricks/test_partitioned_join__plan0.sql +++ b/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/Databricks/test_partitioned_join__plan0.sql @@ -15,10 +15,10 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.ds_partitioned__day AS ds_partitioned__day + subq_5.home_state AS user__home_state , subq_5.ds_partitioned__day AS user__ds_partitioned__day + , subq_2.ds_partitioned__day AS ds_partitioned__day , subq_2.user AS user - , subq_5.home_state AS user__home_state , subq_2.identity_verifications AS identity_verifications FROM ( -- Pass Only Elements: ['identity_verifications', 'ds_partitioned__day', 'user'] diff --git a/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/DuckDB/test_filter_with_where_constraint_on_join_dim__plan0.sql b/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/DuckDB/test_filter_with_where_constraint_on_join_dim__plan0.sql index b8f95359e6..1305901493 100644 --- a/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/DuckDB/test_filter_with_where_constraint_on_join_dim__plan0.sql +++ b/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/DuckDB/test_filter_with_where_constraint_on_join_dim__plan0.sql @@ -27,9 +27,9 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.listing AS listing + subq_5.country_latest AS listing__country_latest + , subq_2.listing AS listing , subq_2.booking__is_instant AS booking__is_instant - , subq_5.country_latest AS listing__country_latest , subq_2.bookings AS bookings FROM ( -- Pass Only Elements: ['bookings', 'booking__is_instant', 'listing'] diff --git a/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/DuckDB/test_join_to_scd_dimension__plan0.sql b/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/DuckDB/test_join_to_scd_dimension__plan0.sql index ee768657d4..388adce3a5 100644 --- a/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/DuckDB/test_join_to_scd_dimension__plan0.sql +++ b/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/DuckDB/test_join_to_scd_dimension__plan0.sql @@ -27,11 +27,11 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.metric_time__day AS metric_time__day + subq_4.capacity AS listing__capacity , subq_4.window_start__day AS listing__window_start__day , subq_4.window_end__day AS listing__window_end__day + , subq_2.metric_time__day AS metric_time__day , subq_2.listing AS listing - , subq_4.capacity AS listing__capacity , subq_2.bookings AS bookings FROM ( -- Pass Only Elements: ['bookings', 'metric_time__day', 'listing'] diff --git a/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/DuckDB/test_measure_constraint__plan0.sql b/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/DuckDB/test_measure_constraint__plan0.sql index 0c1c45d416..d37eadd7ca 100644 --- a/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/DuckDB/test_measure_constraint__plan0.sql +++ b/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/DuckDB/test_measure_constraint__plan0.sql @@ -39,9 +39,9 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.metric_time__day AS metric_time__day + subq_5.is_lux_latest AS listing__is_lux_latest + , subq_2.metric_time__day AS metric_time__day , subq_2.listing AS listing - , subq_5.is_lux_latest AS listing__is_lux_latest , subq_2.average_booking_value AS average_booking_value FROM ( -- Pass Only Elements: ['average_booking_value', 'metric_time__day', 'listing'] @@ -424,9 +424,9 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_14.metric_time__day AS metric_time__day + subq_17.is_lux_latest AS listing__is_lux_latest + , subq_14.metric_time__day AS metric_time__day , subq_14.listing AS listing - , subq_17.is_lux_latest AS listing__is_lux_latest , subq_14.bookings AS bookings FROM ( -- Pass Only Elements: ['bookings', 'metric_time__day', 'listing'] diff --git a/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/DuckDB/test_multi_hop_through_scd_dimension__plan0.sql b/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/DuckDB/test_multi_hop_through_scd_dimension__plan0.sql index 2db2db0066..23e1270946 100644 --- a/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/DuckDB/test_multi_hop_through_scd_dimension__plan0.sql +++ b/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/DuckDB/test_multi_hop_through_scd_dimension__plan0.sql @@ -18,11 +18,11 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.metric_time__day AS metric_time__day + subq_7.user__home_state_latest AS listing__user__home_state_latest , subq_7.window_start__day AS listing__window_start__day , subq_7.window_end__day AS listing__window_end__day + , subq_2.metric_time__day AS metric_time__day , subq_2.listing AS listing - , subq_7.user__home_state_latest AS listing__user__home_state_latest , subq_2.bookings AS bookings FROM ( -- Pass Only Elements: ['bookings', 'metric_time__day', 'listing'] @@ -224,7 +224,19 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_3.window_start__day AS window_start__day + subq_5.home_state_latest AS user__home_state_latest + , subq_5.ds__day AS user__ds__day + , subq_5.ds__week AS user__ds__week + , subq_5.ds__month AS user__ds__month + , subq_5.ds__quarter AS user__ds__quarter + , subq_5.ds__year AS user__ds__year + , subq_5.ds__extract_year AS user__ds__extract_year + , subq_5.ds__extract_quarter AS user__ds__extract_quarter + , subq_5.ds__extract_month AS user__ds__extract_month + , subq_5.ds__extract_day AS user__ds__extract_day + , subq_5.ds__extract_dow AS user__ds__extract_dow + , subq_5.ds__extract_doy AS user__ds__extract_doy + , subq_3.window_start__day AS window_start__day , subq_3.window_start__week AS window_start__week , subq_3.window_start__month AS window_start__month , subq_3.window_start__quarter AS window_start__quarter @@ -268,17 +280,6 @@ FROM ( , subq_3.listing__window_end__extract_day AS listing__window_end__extract_day , subq_3.listing__window_end__extract_dow AS listing__window_end__extract_dow , subq_3.listing__window_end__extract_doy AS listing__window_end__extract_doy - , subq_5.ds__day AS user__ds__day - , subq_5.ds__week AS user__ds__week - , subq_5.ds__month AS user__ds__month - , subq_5.ds__quarter AS user__ds__quarter - , subq_5.ds__year AS user__ds__year - , subq_5.ds__extract_year AS user__ds__extract_year - , subq_5.ds__extract_quarter AS user__ds__extract_quarter - , subq_5.ds__extract_month AS user__ds__extract_month - , subq_5.ds__extract_day AS user__ds__extract_day - , subq_5.ds__extract_dow AS user__ds__extract_dow - , subq_5.ds__extract_doy AS user__ds__extract_doy , subq_3.listing AS listing , subq_3.user AS user , subq_3.listing__user AS listing__user @@ -288,7 +289,6 @@ FROM ( , subq_3.listing__country AS listing__country , subq_3.listing__is_lux AS listing__is_lux , subq_3.listing__capacity AS listing__capacity - , subq_5.home_state_latest AS user__home_state_latest FROM ( -- Read Elements From Semantic Model 'listings' SELECT diff --git a/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/DuckDB/test_multi_hop_to_scd_dimension__plan0.sql b/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/DuckDB/test_multi_hop_to_scd_dimension__plan0.sql index b2dfef8dbc..fd5244631f 100644 --- a/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/DuckDB/test_multi_hop_to_scd_dimension__plan0.sql +++ b/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/DuckDB/test_multi_hop_to_scd_dimension__plan0.sql @@ -18,11 +18,11 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.metric_time__day AS metric_time__day + subq_7.lux_listing__is_confirmed_lux AS listing__lux_listing__is_confirmed_lux , subq_7.lux_listing__window_start__day AS listing__lux_listing__window_start__day , subq_7.lux_listing__window_end__day AS listing__lux_listing__window_end__day + , subq_2.metric_time__day AS metric_time__day , subq_2.listing AS listing - , subq_7.lux_listing__is_confirmed_lux AS listing__lux_listing__is_confirmed_lux , subq_2.bookings AS bookings FROM ( -- Pass Only Elements: ['bookings', 'metric_time__day', 'listing'] @@ -224,7 +224,8 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_5.window_start__day AS lux_listing__window_start__day + subq_5.is_confirmed_lux AS lux_listing__is_confirmed_lux + , subq_5.window_start__day AS lux_listing__window_start__day , subq_5.window_start__week AS lux_listing__window_start__week , subq_5.window_start__month AS lux_listing__window_start__month , subq_5.window_start__quarter AS lux_listing__window_start__quarter @@ -249,7 +250,6 @@ FROM ( , subq_3.listing AS listing , subq_3.lux_listing AS lux_listing , subq_3.listing__lux_listing AS listing__lux_listing - , subq_5.is_confirmed_lux AS lux_listing__is_confirmed_lux FROM ( -- Read Elements From Semantic Model 'lux_listing_mapping' SELECT diff --git a/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/DuckDB/test_multihop_node__plan0.sql b/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/DuckDB/test_multihop_node__plan0.sql index a2cfac78b0..457fb2d259 100644 --- a/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/DuckDB/test_multihop_node__plan0.sql +++ b/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/DuckDB/test_multihop_node__plan0.sql @@ -15,10 +15,10 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.ds_partitioned__day AS ds_partitioned__day + subq_9.customer_id__customer_name AS account_id__customer_id__customer_name , subq_9.ds_partitioned__day AS account_id__ds_partitioned__day + , subq_2.ds_partitioned__day AS ds_partitioned__day , subq_2.account_id AS account_id - , subq_9.customer_id__customer_name AS account_id__customer_id__customer_name , subq_2.txn_count AS txn_count FROM ( -- Pass Only Elements: ['txn_count', 'ds_partitioned__day', 'account_id'] @@ -152,7 +152,31 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_4.ds_partitioned__day AS ds_partitioned__day + subq_7.customer_name AS customer_id__customer_name + , subq_7.customer_atomic_weight AS customer_id__customer_atomic_weight + , subq_7.ds_partitioned__day AS customer_id__ds_partitioned__day + , subq_7.ds_partitioned__week AS customer_id__ds_partitioned__week + , subq_7.ds_partitioned__month AS customer_id__ds_partitioned__month + , subq_7.ds_partitioned__quarter AS customer_id__ds_partitioned__quarter + , subq_7.ds_partitioned__year AS customer_id__ds_partitioned__year + , subq_7.ds_partitioned__extract_year AS customer_id__ds_partitioned__extract_year + , subq_7.ds_partitioned__extract_quarter AS customer_id__ds_partitioned__extract_quarter + , subq_7.ds_partitioned__extract_month AS customer_id__ds_partitioned__extract_month + , subq_7.ds_partitioned__extract_day AS customer_id__ds_partitioned__extract_day + , subq_7.ds_partitioned__extract_dow AS customer_id__ds_partitioned__extract_dow + , subq_7.ds_partitioned__extract_doy AS customer_id__ds_partitioned__extract_doy + , subq_7.metric_time__day AS customer_id__metric_time__day + , subq_7.metric_time__week AS customer_id__metric_time__week + , subq_7.metric_time__month AS customer_id__metric_time__month + , subq_7.metric_time__quarter AS customer_id__metric_time__quarter + , subq_7.metric_time__year AS customer_id__metric_time__year + , subq_7.metric_time__extract_year AS customer_id__metric_time__extract_year + , subq_7.metric_time__extract_quarter AS customer_id__metric_time__extract_quarter + , subq_7.metric_time__extract_month AS customer_id__metric_time__extract_month + , subq_7.metric_time__extract_day AS customer_id__metric_time__extract_day + , subq_7.metric_time__extract_dow AS customer_id__metric_time__extract_dow + , subq_7.metric_time__extract_doy AS customer_id__metric_time__extract_doy + , subq_4.ds_partitioned__day AS ds_partitioned__day , subq_4.ds_partitioned__week AS ds_partitioned__week , subq_4.ds_partitioned__month AS ds_partitioned__month , subq_4.ds_partitioned__quarter AS ds_partitioned__quarter @@ -196,28 +220,6 @@ FROM ( , subq_4.metric_time__extract_day AS metric_time__extract_day , subq_4.metric_time__extract_dow AS metric_time__extract_dow , subq_4.metric_time__extract_doy AS metric_time__extract_doy - , subq_7.ds_partitioned__day AS customer_id__ds_partitioned__day - , subq_7.ds_partitioned__week AS customer_id__ds_partitioned__week - , subq_7.ds_partitioned__month AS customer_id__ds_partitioned__month - , subq_7.ds_partitioned__quarter AS customer_id__ds_partitioned__quarter - , subq_7.ds_partitioned__year AS customer_id__ds_partitioned__year - , subq_7.ds_partitioned__extract_year AS customer_id__ds_partitioned__extract_year - , subq_7.ds_partitioned__extract_quarter AS customer_id__ds_partitioned__extract_quarter - , subq_7.ds_partitioned__extract_month AS customer_id__ds_partitioned__extract_month - , subq_7.ds_partitioned__extract_day AS customer_id__ds_partitioned__extract_day - , subq_7.ds_partitioned__extract_dow AS customer_id__ds_partitioned__extract_dow - , subq_7.ds_partitioned__extract_doy AS customer_id__ds_partitioned__extract_doy - , subq_7.metric_time__day AS customer_id__metric_time__day - , subq_7.metric_time__week AS customer_id__metric_time__week - , subq_7.metric_time__month AS customer_id__metric_time__month - , subq_7.metric_time__quarter AS customer_id__metric_time__quarter - , subq_7.metric_time__year AS customer_id__metric_time__year - , subq_7.metric_time__extract_year AS customer_id__metric_time__extract_year - , subq_7.metric_time__extract_quarter AS customer_id__metric_time__extract_quarter - , subq_7.metric_time__extract_month AS customer_id__metric_time__extract_month - , subq_7.metric_time__extract_day AS customer_id__metric_time__extract_day - , subq_7.metric_time__extract_dow AS customer_id__metric_time__extract_dow - , subq_7.metric_time__extract_doy AS customer_id__metric_time__extract_doy , subq_4.account_id AS account_id , subq_4.customer_id AS customer_id , subq_4.account_id__customer_id AS account_id__customer_id @@ -226,8 +228,6 @@ FROM ( , subq_4.extra_dim AS extra_dim , subq_4.account_id__extra_dim AS account_id__extra_dim , subq_4.bridge_account__extra_dim AS bridge_account__extra_dim - , subq_7.customer_name AS customer_id__customer_name - , subq_7.customer_atomic_weight AS customer_id__customer_atomic_weight , subq_4.account_customer_combos AS account_customer_combos FROM ( -- Metric Time Dimension 'ds_partitioned' diff --git a/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/DuckDB/test_partitioned_join__plan0.sql b/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/DuckDB/test_partitioned_join__plan0.sql index 9331570241..c1a2b4b8f9 100644 --- a/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/DuckDB/test_partitioned_join__plan0.sql +++ b/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/DuckDB/test_partitioned_join__plan0.sql @@ -15,10 +15,10 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.ds_partitioned__day AS ds_partitioned__day + subq_5.home_state AS user__home_state , subq_5.ds_partitioned__day AS user__ds_partitioned__day + , subq_2.ds_partitioned__day AS ds_partitioned__day , subq_2.user AS user - , subq_5.home_state AS user__home_state , subq_2.identity_verifications AS identity_verifications FROM ( -- Pass Only Elements: ['identity_verifications', 'ds_partitioned__day', 'user'] diff --git a/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/Postgres/test_filter_with_where_constraint_on_join_dim__plan0.sql b/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/Postgres/test_filter_with_where_constraint_on_join_dim__plan0.sql index b8f95359e6..1305901493 100644 --- a/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/Postgres/test_filter_with_where_constraint_on_join_dim__plan0.sql +++ b/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/Postgres/test_filter_with_where_constraint_on_join_dim__plan0.sql @@ -27,9 +27,9 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.listing AS listing + subq_5.country_latest AS listing__country_latest + , subq_2.listing AS listing , subq_2.booking__is_instant AS booking__is_instant - , subq_5.country_latest AS listing__country_latest , subq_2.bookings AS bookings FROM ( -- Pass Only Elements: ['bookings', 'booking__is_instant', 'listing'] diff --git a/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/Postgres/test_join_to_scd_dimension__plan0.sql b/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/Postgres/test_join_to_scd_dimension__plan0.sql index ee768657d4..388adce3a5 100644 --- a/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/Postgres/test_join_to_scd_dimension__plan0.sql +++ b/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/Postgres/test_join_to_scd_dimension__plan0.sql @@ -27,11 +27,11 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.metric_time__day AS metric_time__day + subq_4.capacity AS listing__capacity , subq_4.window_start__day AS listing__window_start__day , subq_4.window_end__day AS listing__window_end__day + , subq_2.metric_time__day AS metric_time__day , subq_2.listing AS listing - , subq_4.capacity AS listing__capacity , subq_2.bookings AS bookings FROM ( -- Pass Only Elements: ['bookings', 'metric_time__day', 'listing'] diff --git a/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/Postgres/test_measure_constraint__plan0.sql b/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/Postgres/test_measure_constraint__plan0.sql index 0c1c45d416..d37eadd7ca 100644 --- a/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/Postgres/test_measure_constraint__plan0.sql +++ b/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/Postgres/test_measure_constraint__plan0.sql @@ -39,9 +39,9 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.metric_time__day AS metric_time__day + subq_5.is_lux_latest AS listing__is_lux_latest + , subq_2.metric_time__day AS metric_time__day , subq_2.listing AS listing - , subq_5.is_lux_latest AS listing__is_lux_latest , subq_2.average_booking_value AS average_booking_value FROM ( -- Pass Only Elements: ['average_booking_value', 'metric_time__day', 'listing'] @@ -424,9 +424,9 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_14.metric_time__day AS metric_time__day + subq_17.is_lux_latest AS listing__is_lux_latest + , subq_14.metric_time__day AS metric_time__day , subq_14.listing AS listing - , subq_17.is_lux_latest AS listing__is_lux_latest , subq_14.bookings AS bookings FROM ( -- Pass Only Elements: ['bookings', 'metric_time__day', 'listing'] diff --git a/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/Postgres/test_multi_hop_through_scd_dimension__plan0.sql b/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/Postgres/test_multi_hop_through_scd_dimension__plan0.sql index 2db2db0066..23e1270946 100644 --- a/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/Postgres/test_multi_hop_through_scd_dimension__plan0.sql +++ b/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/Postgres/test_multi_hop_through_scd_dimension__plan0.sql @@ -18,11 +18,11 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.metric_time__day AS metric_time__day + subq_7.user__home_state_latest AS listing__user__home_state_latest , subq_7.window_start__day AS listing__window_start__day , subq_7.window_end__day AS listing__window_end__day + , subq_2.metric_time__day AS metric_time__day , subq_2.listing AS listing - , subq_7.user__home_state_latest AS listing__user__home_state_latest , subq_2.bookings AS bookings FROM ( -- Pass Only Elements: ['bookings', 'metric_time__day', 'listing'] @@ -224,7 +224,19 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_3.window_start__day AS window_start__day + subq_5.home_state_latest AS user__home_state_latest + , subq_5.ds__day AS user__ds__day + , subq_5.ds__week AS user__ds__week + , subq_5.ds__month AS user__ds__month + , subq_5.ds__quarter AS user__ds__quarter + , subq_5.ds__year AS user__ds__year + , subq_5.ds__extract_year AS user__ds__extract_year + , subq_5.ds__extract_quarter AS user__ds__extract_quarter + , subq_5.ds__extract_month AS user__ds__extract_month + , subq_5.ds__extract_day AS user__ds__extract_day + , subq_5.ds__extract_dow AS user__ds__extract_dow + , subq_5.ds__extract_doy AS user__ds__extract_doy + , subq_3.window_start__day AS window_start__day , subq_3.window_start__week AS window_start__week , subq_3.window_start__month AS window_start__month , subq_3.window_start__quarter AS window_start__quarter @@ -268,17 +280,6 @@ FROM ( , subq_3.listing__window_end__extract_day AS listing__window_end__extract_day , subq_3.listing__window_end__extract_dow AS listing__window_end__extract_dow , subq_3.listing__window_end__extract_doy AS listing__window_end__extract_doy - , subq_5.ds__day AS user__ds__day - , subq_5.ds__week AS user__ds__week - , subq_5.ds__month AS user__ds__month - , subq_5.ds__quarter AS user__ds__quarter - , subq_5.ds__year AS user__ds__year - , subq_5.ds__extract_year AS user__ds__extract_year - , subq_5.ds__extract_quarter AS user__ds__extract_quarter - , subq_5.ds__extract_month AS user__ds__extract_month - , subq_5.ds__extract_day AS user__ds__extract_day - , subq_5.ds__extract_dow AS user__ds__extract_dow - , subq_5.ds__extract_doy AS user__ds__extract_doy , subq_3.listing AS listing , subq_3.user AS user , subq_3.listing__user AS listing__user @@ -288,7 +289,6 @@ FROM ( , subq_3.listing__country AS listing__country , subq_3.listing__is_lux AS listing__is_lux , subq_3.listing__capacity AS listing__capacity - , subq_5.home_state_latest AS user__home_state_latest FROM ( -- Read Elements From Semantic Model 'listings' SELECT diff --git a/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/Postgres/test_multi_hop_to_scd_dimension__plan0.sql b/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/Postgres/test_multi_hop_to_scd_dimension__plan0.sql index b2dfef8dbc..fd5244631f 100644 --- a/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/Postgres/test_multi_hop_to_scd_dimension__plan0.sql +++ b/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/Postgres/test_multi_hop_to_scd_dimension__plan0.sql @@ -18,11 +18,11 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.metric_time__day AS metric_time__day + subq_7.lux_listing__is_confirmed_lux AS listing__lux_listing__is_confirmed_lux , subq_7.lux_listing__window_start__day AS listing__lux_listing__window_start__day , subq_7.lux_listing__window_end__day AS listing__lux_listing__window_end__day + , subq_2.metric_time__day AS metric_time__day , subq_2.listing AS listing - , subq_7.lux_listing__is_confirmed_lux AS listing__lux_listing__is_confirmed_lux , subq_2.bookings AS bookings FROM ( -- Pass Only Elements: ['bookings', 'metric_time__day', 'listing'] @@ -224,7 +224,8 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_5.window_start__day AS lux_listing__window_start__day + subq_5.is_confirmed_lux AS lux_listing__is_confirmed_lux + , subq_5.window_start__day AS lux_listing__window_start__day , subq_5.window_start__week AS lux_listing__window_start__week , subq_5.window_start__month AS lux_listing__window_start__month , subq_5.window_start__quarter AS lux_listing__window_start__quarter @@ -249,7 +250,6 @@ FROM ( , subq_3.listing AS listing , subq_3.lux_listing AS lux_listing , subq_3.listing__lux_listing AS listing__lux_listing - , subq_5.is_confirmed_lux AS lux_listing__is_confirmed_lux FROM ( -- Read Elements From Semantic Model 'lux_listing_mapping' SELECT diff --git a/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/Postgres/test_multihop_node__plan0.sql b/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/Postgres/test_multihop_node__plan0.sql index a2cfac78b0..457fb2d259 100644 --- a/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/Postgres/test_multihop_node__plan0.sql +++ b/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/Postgres/test_multihop_node__plan0.sql @@ -15,10 +15,10 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.ds_partitioned__day AS ds_partitioned__day + subq_9.customer_id__customer_name AS account_id__customer_id__customer_name , subq_9.ds_partitioned__day AS account_id__ds_partitioned__day + , subq_2.ds_partitioned__day AS ds_partitioned__day , subq_2.account_id AS account_id - , subq_9.customer_id__customer_name AS account_id__customer_id__customer_name , subq_2.txn_count AS txn_count FROM ( -- Pass Only Elements: ['txn_count', 'ds_partitioned__day', 'account_id'] @@ -152,7 +152,31 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_4.ds_partitioned__day AS ds_partitioned__day + subq_7.customer_name AS customer_id__customer_name + , subq_7.customer_atomic_weight AS customer_id__customer_atomic_weight + , subq_7.ds_partitioned__day AS customer_id__ds_partitioned__day + , subq_7.ds_partitioned__week AS customer_id__ds_partitioned__week + , subq_7.ds_partitioned__month AS customer_id__ds_partitioned__month + , subq_7.ds_partitioned__quarter AS customer_id__ds_partitioned__quarter + , subq_7.ds_partitioned__year AS customer_id__ds_partitioned__year + , subq_7.ds_partitioned__extract_year AS customer_id__ds_partitioned__extract_year + , subq_7.ds_partitioned__extract_quarter AS customer_id__ds_partitioned__extract_quarter + , subq_7.ds_partitioned__extract_month AS customer_id__ds_partitioned__extract_month + , subq_7.ds_partitioned__extract_day AS customer_id__ds_partitioned__extract_day + , subq_7.ds_partitioned__extract_dow AS customer_id__ds_partitioned__extract_dow + , subq_7.ds_partitioned__extract_doy AS customer_id__ds_partitioned__extract_doy + , subq_7.metric_time__day AS customer_id__metric_time__day + , subq_7.metric_time__week AS customer_id__metric_time__week + , subq_7.metric_time__month AS customer_id__metric_time__month + , subq_7.metric_time__quarter AS customer_id__metric_time__quarter + , subq_7.metric_time__year AS customer_id__metric_time__year + , subq_7.metric_time__extract_year AS customer_id__metric_time__extract_year + , subq_7.metric_time__extract_quarter AS customer_id__metric_time__extract_quarter + , subq_7.metric_time__extract_month AS customer_id__metric_time__extract_month + , subq_7.metric_time__extract_day AS customer_id__metric_time__extract_day + , subq_7.metric_time__extract_dow AS customer_id__metric_time__extract_dow + , subq_7.metric_time__extract_doy AS customer_id__metric_time__extract_doy + , subq_4.ds_partitioned__day AS ds_partitioned__day , subq_4.ds_partitioned__week AS ds_partitioned__week , subq_4.ds_partitioned__month AS ds_partitioned__month , subq_4.ds_partitioned__quarter AS ds_partitioned__quarter @@ -196,28 +220,6 @@ FROM ( , subq_4.metric_time__extract_day AS metric_time__extract_day , subq_4.metric_time__extract_dow AS metric_time__extract_dow , subq_4.metric_time__extract_doy AS metric_time__extract_doy - , subq_7.ds_partitioned__day AS customer_id__ds_partitioned__day - , subq_7.ds_partitioned__week AS customer_id__ds_partitioned__week - , subq_7.ds_partitioned__month AS customer_id__ds_partitioned__month - , subq_7.ds_partitioned__quarter AS customer_id__ds_partitioned__quarter - , subq_7.ds_partitioned__year AS customer_id__ds_partitioned__year - , subq_7.ds_partitioned__extract_year AS customer_id__ds_partitioned__extract_year - , subq_7.ds_partitioned__extract_quarter AS customer_id__ds_partitioned__extract_quarter - , subq_7.ds_partitioned__extract_month AS customer_id__ds_partitioned__extract_month - , subq_7.ds_partitioned__extract_day AS customer_id__ds_partitioned__extract_day - , subq_7.ds_partitioned__extract_dow AS customer_id__ds_partitioned__extract_dow - , subq_7.ds_partitioned__extract_doy AS customer_id__ds_partitioned__extract_doy - , subq_7.metric_time__day AS customer_id__metric_time__day - , subq_7.metric_time__week AS customer_id__metric_time__week - , subq_7.metric_time__month AS customer_id__metric_time__month - , subq_7.metric_time__quarter AS customer_id__metric_time__quarter - , subq_7.metric_time__year AS customer_id__metric_time__year - , subq_7.metric_time__extract_year AS customer_id__metric_time__extract_year - , subq_7.metric_time__extract_quarter AS customer_id__metric_time__extract_quarter - , subq_7.metric_time__extract_month AS customer_id__metric_time__extract_month - , subq_7.metric_time__extract_day AS customer_id__metric_time__extract_day - , subq_7.metric_time__extract_dow AS customer_id__metric_time__extract_dow - , subq_7.metric_time__extract_doy AS customer_id__metric_time__extract_doy , subq_4.account_id AS account_id , subq_4.customer_id AS customer_id , subq_4.account_id__customer_id AS account_id__customer_id @@ -226,8 +228,6 @@ FROM ( , subq_4.extra_dim AS extra_dim , subq_4.account_id__extra_dim AS account_id__extra_dim , subq_4.bridge_account__extra_dim AS bridge_account__extra_dim - , subq_7.customer_name AS customer_id__customer_name - , subq_7.customer_atomic_weight AS customer_id__customer_atomic_weight , subq_4.account_customer_combos AS account_customer_combos FROM ( -- Metric Time Dimension 'ds_partitioned' diff --git a/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/Postgres/test_partitioned_join__plan0.sql b/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/Postgres/test_partitioned_join__plan0.sql index 9331570241..c1a2b4b8f9 100644 --- a/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/Postgres/test_partitioned_join__plan0.sql +++ b/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/Postgres/test_partitioned_join__plan0.sql @@ -15,10 +15,10 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.ds_partitioned__day AS ds_partitioned__day + subq_5.home_state AS user__home_state , subq_5.ds_partitioned__day AS user__ds_partitioned__day + , subq_2.ds_partitioned__day AS ds_partitioned__day , subq_2.user AS user - , subq_5.home_state AS user__home_state , subq_2.identity_verifications AS identity_verifications FROM ( -- Pass Only Elements: ['identity_verifications', 'ds_partitioned__day', 'user'] diff --git a/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/Redshift/test_filter_with_where_constraint_on_join_dim__plan0.sql b/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/Redshift/test_filter_with_where_constraint_on_join_dim__plan0.sql index 0a2073111e..c4b0570d6b 100644 --- a/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/Redshift/test_filter_with_where_constraint_on_join_dim__plan0.sql +++ b/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/Redshift/test_filter_with_where_constraint_on_join_dim__plan0.sql @@ -27,9 +27,9 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.listing AS listing + subq_5.country_latest AS listing__country_latest + , subq_2.listing AS listing , subq_2.booking__is_instant AS booking__is_instant - , subq_5.country_latest AS listing__country_latest , subq_2.bookings AS bookings FROM ( -- Pass Only Elements: ['bookings', 'booking__is_instant', 'listing'] diff --git a/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/Redshift/test_join_to_scd_dimension__plan0.sql b/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/Redshift/test_join_to_scd_dimension__plan0.sql index 37ddb6903a..e255b7e61a 100644 --- a/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/Redshift/test_join_to_scd_dimension__plan0.sql +++ b/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/Redshift/test_join_to_scd_dimension__plan0.sql @@ -27,11 +27,11 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.metric_time__day AS metric_time__day + subq_4.capacity AS listing__capacity , subq_4.window_start__day AS listing__window_start__day , subq_4.window_end__day AS listing__window_end__day + , subq_2.metric_time__day AS metric_time__day , subq_2.listing AS listing - , subq_4.capacity AS listing__capacity , subq_2.bookings AS bookings FROM ( -- Pass Only Elements: ['bookings', 'metric_time__day', 'listing'] diff --git a/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/Redshift/test_measure_constraint__plan0.sql b/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/Redshift/test_measure_constraint__plan0.sql index 72bed47741..5449922a0f 100644 --- a/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/Redshift/test_measure_constraint__plan0.sql +++ b/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/Redshift/test_measure_constraint__plan0.sql @@ -39,9 +39,9 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.metric_time__day AS metric_time__day + subq_5.is_lux_latest AS listing__is_lux_latest + , subq_2.metric_time__day AS metric_time__day , subq_2.listing AS listing - , subq_5.is_lux_latest AS listing__is_lux_latest , subq_2.average_booking_value AS average_booking_value FROM ( -- Pass Only Elements: ['average_booking_value', 'metric_time__day', 'listing'] @@ -424,9 +424,9 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_14.metric_time__day AS metric_time__day + subq_17.is_lux_latest AS listing__is_lux_latest + , subq_14.metric_time__day AS metric_time__day , subq_14.listing AS listing - , subq_17.is_lux_latest AS listing__is_lux_latest , subq_14.bookings AS bookings FROM ( -- Pass Only Elements: ['bookings', 'metric_time__day', 'listing'] diff --git a/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/Redshift/test_multi_hop_through_scd_dimension__plan0.sql b/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/Redshift/test_multi_hop_through_scd_dimension__plan0.sql index fc0e21ba80..5ee49254f6 100644 --- a/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/Redshift/test_multi_hop_through_scd_dimension__plan0.sql +++ b/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/Redshift/test_multi_hop_through_scd_dimension__plan0.sql @@ -18,11 +18,11 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.metric_time__day AS metric_time__day + subq_7.user__home_state_latest AS listing__user__home_state_latest , subq_7.window_start__day AS listing__window_start__day , subq_7.window_end__day AS listing__window_end__day + , subq_2.metric_time__day AS metric_time__day , subq_2.listing AS listing - , subq_7.user__home_state_latest AS listing__user__home_state_latest , subq_2.bookings AS bookings FROM ( -- Pass Only Elements: ['bookings', 'metric_time__day', 'listing'] @@ -224,7 +224,19 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_3.window_start__day AS window_start__day + subq_5.home_state_latest AS user__home_state_latest + , subq_5.ds__day AS user__ds__day + , subq_5.ds__week AS user__ds__week + , subq_5.ds__month AS user__ds__month + , subq_5.ds__quarter AS user__ds__quarter + , subq_5.ds__year AS user__ds__year + , subq_5.ds__extract_year AS user__ds__extract_year + , subq_5.ds__extract_quarter AS user__ds__extract_quarter + , subq_5.ds__extract_month AS user__ds__extract_month + , subq_5.ds__extract_day AS user__ds__extract_day + , subq_5.ds__extract_dow AS user__ds__extract_dow + , subq_5.ds__extract_doy AS user__ds__extract_doy + , subq_3.window_start__day AS window_start__day , subq_3.window_start__week AS window_start__week , subq_3.window_start__month AS window_start__month , subq_3.window_start__quarter AS window_start__quarter @@ -268,17 +280,6 @@ FROM ( , subq_3.listing__window_end__extract_day AS listing__window_end__extract_day , subq_3.listing__window_end__extract_dow AS listing__window_end__extract_dow , subq_3.listing__window_end__extract_doy AS listing__window_end__extract_doy - , subq_5.ds__day AS user__ds__day - , subq_5.ds__week AS user__ds__week - , subq_5.ds__month AS user__ds__month - , subq_5.ds__quarter AS user__ds__quarter - , subq_5.ds__year AS user__ds__year - , subq_5.ds__extract_year AS user__ds__extract_year - , subq_5.ds__extract_quarter AS user__ds__extract_quarter - , subq_5.ds__extract_month AS user__ds__extract_month - , subq_5.ds__extract_day AS user__ds__extract_day - , subq_5.ds__extract_dow AS user__ds__extract_dow - , subq_5.ds__extract_doy AS user__ds__extract_doy , subq_3.listing AS listing , subq_3.user AS user , subq_3.listing__user AS listing__user @@ -288,7 +289,6 @@ FROM ( , subq_3.listing__country AS listing__country , subq_3.listing__is_lux AS listing__is_lux , subq_3.listing__capacity AS listing__capacity - , subq_5.home_state_latest AS user__home_state_latest FROM ( -- Read Elements From Semantic Model 'listings' SELECT diff --git a/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/Redshift/test_multi_hop_to_scd_dimension__plan0.sql b/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/Redshift/test_multi_hop_to_scd_dimension__plan0.sql index 1b36ee10d2..6b1ccfeece 100644 --- a/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/Redshift/test_multi_hop_to_scd_dimension__plan0.sql +++ b/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/Redshift/test_multi_hop_to_scd_dimension__plan0.sql @@ -18,11 +18,11 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.metric_time__day AS metric_time__day + subq_7.lux_listing__is_confirmed_lux AS listing__lux_listing__is_confirmed_lux , subq_7.lux_listing__window_start__day AS listing__lux_listing__window_start__day , subq_7.lux_listing__window_end__day AS listing__lux_listing__window_end__day + , subq_2.metric_time__day AS metric_time__day , subq_2.listing AS listing - , subq_7.lux_listing__is_confirmed_lux AS listing__lux_listing__is_confirmed_lux , subq_2.bookings AS bookings FROM ( -- Pass Only Elements: ['bookings', 'metric_time__day', 'listing'] @@ -224,7 +224,8 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_5.window_start__day AS lux_listing__window_start__day + subq_5.is_confirmed_lux AS lux_listing__is_confirmed_lux + , subq_5.window_start__day AS lux_listing__window_start__day , subq_5.window_start__week AS lux_listing__window_start__week , subq_5.window_start__month AS lux_listing__window_start__month , subq_5.window_start__quarter AS lux_listing__window_start__quarter @@ -249,7 +250,6 @@ FROM ( , subq_3.listing AS listing , subq_3.lux_listing AS lux_listing , subq_3.listing__lux_listing AS listing__lux_listing - , subq_5.is_confirmed_lux AS lux_listing__is_confirmed_lux FROM ( -- Read Elements From Semantic Model 'lux_listing_mapping' SELECT diff --git a/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/Redshift/test_multihop_node__plan0.sql b/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/Redshift/test_multihop_node__plan0.sql index 7d90a6fa5f..273a4e3fb8 100644 --- a/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/Redshift/test_multihop_node__plan0.sql +++ b/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/Redshift/test_multihop_node__plan0.sql @@ -15,10 +15,10 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.ds_partitioned__day AS ds_partitioned__day + subq_9.customer_id__customer_name AS account_id__customer_id__customer_name , subq_9.ds_partitioned__day AS account_id__ds_partitioned__day + , subq_2.ds_partitioned__day AS ds_partitioned__day , subq_2.account_id AS account_id - , subq_9.customer_id__customer_name AS account_id__customer_id__customer_name , subq_2.txn_count AS txn_count FROM ( -- Pass Only Elements: ['txn_count', 'ds_partitioned__day', 'account_id'] @@ -152,7 +152,31 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_4.ds_partitioned__day AS ds_partitioned__day + subq_7.customer_name AS customer_id__customer_name + , subq_7.customer_atomic_weight AS customer_id__customer_atomic_weight + , subq_7.ds_partitioned__day AS customer_id__ds_partitioned__day + , subq_7.ds_partitioned__week AS customer_id__ds_partitioned__week + , subq_7.ds_partitioned__month AS customer_id__ds_partitioned__month + , subq_7.ds_partitioned__quarter AS customer_id__ds_partitioned__quarter + , subq_7.ds_partitioned__year AS customer_id__ds_partitioned__year + , subq_7.ds_partitioned__extract_year AS customer_id__ds_partitioned__extract_year + , subq_7.ds_partitioned__extract_quarter AS customer_id__ds_partitioned__extract_quarter + , subq_7.ds_partitioned__extract_month AS customer_id__ds_partitioned__extract_month + , subq_7.ds_partitioned__extract_day AS customer_id__ds_partitioned__extract_day + , subq_7.ds_partitioned__extract_dow AS customer_id__ds_partitioned__extract_dow + , subq_7.ds_partitioned__extract_doy AS customer_id__ds_partitioned__extract_doy + , subq_7.metric_time__day AS customer_id__metric_time__day + , subq_7.metric_time__week AS customer_id__metric_time__week + , subq_7.metric_time__month AS customer_id__metric_time__month + , subq_7.metric_time__quarter AS customer_id__metric_time__quarter + , subq_7.metric_time__year AS customer_id__metric_time__year + , subq_7.metric_time__extract_year AS customer_id__metric_time__extract_year + , subq_7.metric_time__extract_quarter AS customer_id__metric_time__extract_quarter + , subq_7.metric_time__extract_month AS customer_id__metric_time__extract_month + , subq_7.metric_time__extract_day AS customer_id__metric_time__extract_day + , subq_7.metric_time__extract_dow AS customer_id__metric_time__extract_dow + , subq_7.metric_time__extract_doy AS customer_id__metric_time__extract_doy + , subq_4.ds_partitioned__day AS ds_partitioned__day , subq_4.ds_partitioned__week AS ds_partitioned__week , subq_4.ds_partitioned__month AS ds_partitioned__month , subq_4.ds_partitioned__quarter AS ds_partitioned__quarter @@ -196,28 +220,6 @@ FROM ( , subq_4.metric_time__extract_day AS metric_time__extract_day , subq_4.metric_time__extract_dow AS metric_time__extract_dow , subq_4.metric_time__extract_doy AS metric_time__extract_doy - , subq_7.ds_partitioned__day AS customer_id__ds_partitioned__day - , subq_7.ds_partitioned__week AS customer_id__ds_partitioned__week - , subq_7.ds_partitioned__month AS customer_id__ds_partitioned__month - , subq_7.ds_partitioned__quarter AS customer_id__ds_partitioned__quarter - , subq_7.ds_partitioned__year AS customer_id__ds_partitioned__year - , subq_7.ds_partitioned__extract_year AS customer_id__ds_partitioned__extract_year - , subq_7.ds_partitioned__extract_quarter AS customer_id__ds_partitioned__extract_quarter - , subq_7.ds_partitioned__extract_month AS customer_id__ds_partitioned__extract_month - , subq_7.ds_partitioned__extract_day AS customer_id__ds_partitioned__extract_day - , subq_7.ds_partitioned__extract_dow AS customer_id__ds_partitioned__extract_dow - , subq_7.ds_partitioned__extract_doy AS customer_id__ds_partitioned__extract_doy - , subq_7.metric_time__day AS customer_id__metric_time__day - , subq_7.metric_time__week AS customer_id__metric_time__week - , subq_7.metric_time__month AS customer_id__metric_time__month - , subq_7.metric_time__quarter AS customer_id__metric_time__quarter - , subq_7.metric_time__year AS customer_id__metric_time__year - , subq_7.metric_time__extract_year AS customer_id__metric_time__extract_year - , subq_7.metric_time__extract_quarter AS customer_id__metric_time__extract_quarter - , subq_7.metric_time__extract_month AS customer_id__metric_time__extract_month - , subq_7.metric_time__extract_day AS customer_id__metric_time__extract_day - , subq_7.metric_time__extract_dow AS customer_id__metric_time__extract_dow - , subq_7.metric_time__extract_doy AS customer_id__metric_time__extract_doy , subq_4.account_id AS account_id , subq_4.customer_id AS customer_id , subq_4.account_id__customer_id AS account_id__customer_id @@ -226,8 +228,6 @@ FROM ( , subq_4.extra_dim AS extra_dim , subq_4.account_id__extra_dim AS account_id__extra_dim , subq_4.bridge_account__extra_dim AS bridge_account__extra_dim - , subq_7.customer_name AS customer_id__customer_name - , subq_7.customer_atomic_weight AS customer_id__customer_atomic_weight , subq_4.account_customer_combos AS account_customer_combos FROM ( -- Metric Time Dimension 'ds_partitioned' diff --git a/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/Redshift/test_partitioned_join__plan0.sql b/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/Redshift/test_partitioned_join__plan0.sql index d6f8a697c9..dd730927f9 100644 --- a/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/Redshift/test_partitioned_join__plan0.sql +++ b/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/Redshift/test_partitioned_join__plan0.sql @@ -15,10 +15,10 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.ds_partitioned__day AS ds_partitioned__day + subq_5.home_state AS user__home_state , subq_5.ds_partitioned__day AS user__ds_partitioned__day + , subq_2.ds_partitioned__day AS ds_partitioned__day , subq_2.user AS user - , subq_5.home_state AS user__home_state , subq_2.identity_verifications AS identity_verifications FROM ( -- Pass Only Elements: ['identity_verifications', 'ds_partitioned__day', 'user'] diff --git a/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/Snowflake/test_filter_with_where_constraint_on_join_dim__plan0.sql b/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/Snowflake/test_filter_with_where_constraint_on_join_dim__plan0.sql index 461a4020b5..2124d8bd4a 100644 --- a/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/Snowflake/test_filter_with_where_constraint_on_join_dim__plan0.sql +++ b/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/Snowflake/test_filter_with_where_constraint_on_join_dim__plan0.sql @@ -27,9 +27,9 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.listing AS listing + subq_5.country_latest AS listing__country_latest + , subq_2.listing AS listing , subq_2.booking__is_instant AS booking__is_instant - , subq_5.country_latest AS listing__country_latest , subq_2.bookings AS bookings FROM ( -- Pass Only Elements: ['bookings', 'booking__is_instant', 'listing'] diff --git a/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/Snowflake/test_join_to_scd_dimension__plan0.sql b/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/Snowflake/test_join_to_scd_dimension__plan0.sql index a0a9112c84..4c40cd5bc6 100644 --- a/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/Snowflake/test_join_to_scd_dimension__plan0.sql +++ b/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/Snowflake/test_join_to_scd_dimension__plan0.sql @@ -27,11 +27,11 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.metric_time__day AS metric_time__day + subq_4.capacity AS listing__capacity , subq_4.window_start__day AS listing__window_start__day , subq_4.window_end__day AS listing__window_end__day + , subq_2.metric_time__day AS metric_time__day , subq_2.listing AS listing - , subq_4.capacity AS listing__capacity , subq_2.bookings AS bookings FROM ( -- Pass Only Elements: ['bookings', 'metric_time__day', 'listing'] diff --git a/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/Snowflake/test_measure_constraint__plan0.sql b/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/Snowflake/test_measure_constraint__plan0.sql index fe9207298e..702c58a9bc 100644 --- a/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/Snowflake/test_measure_constraint__plan0.sql +++ b/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/Snowflake/test_measure_constraint__plan0.sql @@ -39,9 +39,9 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.metric_time__day AS metric_time__day + subq_5.is_lux_latest AS listing__is_lux_latest + , subq_2.metric_time__day AS metric_time__day , subq_2.listing AS listing - , subq_5.is_lux_latest AS listing__is_lux_latest , subq_2.average_booking_value AS average_booking_value FROM ( -- Pass Only Elements: ['average_booking_value', 'metric_time__day', 'listing'] @@ -424,9 +424,9 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_14.metric_time__day AS metric_time__day + subq_17.is_lux_latest AS listing__is_lux_latest + , subq_14.metric_time__day AS metric_time__day , subq_14.listing AS listing - , subq_17.is_lux_latest AS listing__is_lux_latest , subq_14.bookings AS bookings FROM ( -- Pass Only Elements: ['bookings', 'metric_time__day', 'listing'] diff --git a/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/Snowflake/test_multi_hop_through_scd_dimension__plan0.sql b/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/Snowflake/test_multi_hop_through_scd_dimension__plan0.sql index 057513ae3e..b2184830a3 100644 --- a/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/Snowflake/test_multi_hop_through_scd_dimension__plan0.sql +++ b/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/Snowflake/test_multi_hop_through_scd_dimension__plan0.sql @@ -18,11 +18,11 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.metric_time__day AS metric_time__day + subq_7.user__home_state_latest AS listing__user__home_state_latest , subq_7.window_start__day AS listing__window_start__day , subq_7.window_end__day AS listing__window_end__day + , subq_2.metric_time__day AS metric_time__day , subq_2.listing AS listing - , subq_7.user__home_state_latest AS listing__user__home_state_latest , subq_2.bookings AS bookings FROM ( -- Pass Only Elements: ['bookings', 'metric_time__day', 'listing'] @@ -224,7 +224,19 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_3.window_start__day AS window_start__day + subq_5.home_state_latest AS user__home_state_latest + , subq_5.ds__day AS user__ds__day + , subq_5.ds__week AS user__ds__week + , subq_5.ds__month AS user__ds__month + , subq_5.ds__quarter AS user__ds__quarter + , subq_5.ds__year AS user__ds__year + , subq_5.ds__extract_year AS user__ds__extract_year + , subq_5.ds__extract_quarter AS user__ds__extract_quarter + , subq_5.ds__extract_month AS user__ds__extract_month + , subq_5.ds__extract_day AS user__ds__extract_day + , subq_5.ds__extract_dow AS user__ds__extract_dow + , subq_5.ds__extract_doy AS user__ds__extract_doy + , subq_3.window_start__day AS window_start__day , subq_3.window_start__week AS window_start__week , subq_3.window_start__month AS window_start__month , subq_3.window_start__quarter AS window_start__quarter @@ -268,17 +280,6 @@ FROM ( , subq_3.listing__window_end__extract_day AS listing__window_end__extract_day , subq_3.listing__window_end__extract_dow AS listing__window_end__extract_dow , subq_3.listing__window_end__extract_doy AS listing__window_end__extract_doy - , subq_5.ds__day AS user__ds__day - , subq_5.ds__week AS user__ds__week - , subq_5.ds__month AS user__ds__month - , subq_5.ds__quarter AS user__ds__quarter - , subq_5.ds__year AS user__ds__year - , subq_5.ds__extract_year AS user__ds__extract_year - , subq_5.ds__extract_quarter AS user__ds__extract_quarter - , subq_5.ds__extract_month AS user__ds__extract_month - , subq_5.ds__extract_day AS user__ds__extract_day - , subq_5.ds__extract_dow AS user__ds__extract_dow - , subq_5.ds__extract_doy AS user__ds__extract_doy , subq_3.listing AS listing , subq_3.user AS user , subq_3.listing__user AS listing__user @@ -288,7 +289,6 @@ FROM ( , subq_3.listing__country AS listing__country , subq_3.listing__is_lux AS listing__is_lux , subq_3.listing__capacity AS listing__capacity - , subq_5.home_state_latest AS user__home_state_latest FROM ( -- Read Elements From Semantic Model 'listings' SELECT diff --git a/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/Snowflake/test_multi_hop_to_scd_dimension__plan0.sql b/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/Snowflake/test_multi_hop_to_scd_dimension__plan0.sql index f1f5859d67..5203fc30b0 100644 --- a/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/Snowflake/test_multi_hop_to_scd_dimension__plan0.sql +++ b/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/Snowflake/test_multi_hop_to_scd_dimension__plan0.sql @@ -18,11 +18,11 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.metric_time__day AS metric_time__day + subq_7.lux_listing__is_confirmed_lux AS listing__lux_listing__is_confirmed_lux , subq_7.lux_listing__window_start__day AS listing__lux_listing__window_start__day , subq_7.lux_listing__window_end__day AS listing__lux_listing__window_end__day + , subq_2.metric_time__day AS metric_time__day , subq_2.listing AS listing - , subq_7.lux_listing__is_confirmed_lux AS listing__lux_listing__is_confirmed_lux , subq_2.bookings AS bookings FROM ( -- Pass Only Elements: ['bookings', 'metric_time__day', 'listing'] @@ -224,7 +224,8 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_5.window_start__day AS lux_listing__window_start__day + subq_5.is_confirmed_lux AS lux_listing__is_confirmed_lux + , subq_5.window_start__day AS lux_listing__window_start__day , subq_5.window_start__week AS lux_listing__window_start__week , subq_5.window_start__month AS lux_listing__window_start__month , subq_5.window_start__quarter AS lux_listing__window_start__quarter @@ -249,7 +250,6 @@ FROM ( , subq_3.listing AS listing , subq_3.lux_listing AS lux_listing , subq_3.listing__lux_listing AS listing__lux_listing - , subq_5.is_confirmed_lux AS lux_listing__is_confirmed_lux FROM ( -- Read Elements From Semantic Model 'lux_listing_mapping' SELECT diff --git a/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/Snowflake/test_multihop_node__plan0.sql b/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/Snowflake/test_multihop_node__plan0.sql index f1ed37dd36..bae9c0e022 100644 --- a/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/Snowflake/test_multihop_node__plan0.sql +++ b/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/Snowflake/test_multihop_node__plan0.sql @@ -15,10 +15,10 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.ds_partitioned__day AS ds_partitioned__day + subq_9.customer_id__customer_name AS account_id__customer_id__customer_name , subq_9.ds_partitioned__day AS account_id__ds_partitioned__day + , subq_2.ds_partitioned__day AS ds_partitioned__day , subq_2.account_id AS account_id - , subq_9.customer_id__customer_name AS account_id__customer_id__customer_name , subq_2.txn_count AS txn_count FROM ( -- Pass Only Elements: ['txn_count', 'ds_partitioned__day', 'account_id'] @@ -152,7 +152,31 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_4.ds_partitioned__day AS ds_partitioned__day + subq_7.customer_name AS customer_id__customer_name + , subq_7.customer_atomic_weight AS customer_id__customer_atomic_weight + , subq_7.ds_partitioned__day AS customer_id__ds_partitioned__day + , subq_7.ds_partitioned__week AS customer_id__ds_partitioned__week + , subq_7.ds_partitioned__month AS customer_id__ds_partitioned__month + , subq_7.ds_partitioned__quarter AS customer_id__ds_partitioned__quarter + , subq_7.ds_partitioned__year AS customer_id__ds_partitioned__year + , subq_7.ds_partitioned__extract_year AS customer_id__ds_partitioned__extract_year + , subq_7.ds_partitioned__extract_quarter AS customer_id__ds_partitioned__extract_quarter + , subq_7.ds_partitioned__extract_month AS customer_id__ds_partitioned__extract_month + , subq_7.ds_partitioned__extract_day AS customer_id__ds_partitioned__extract_day + , subq_7.ds_partitioned__extract_dow AS customer_id__ds_partitioned__extract_dow + , subq_7.ds_partitioned__extract_doy AS customer_id__ds_partitioned__extract_doy + , subq_7.metric_time__day AS customer_id__metric_time__day + , subq_7.metric_time__week AS customer_id__metric_time__week + , subq_7.metric_time__month AS customer_id__metric_time__month + , subq_7.metric_time__quarter AS customer_id__metric_time__quarter + , subq_7.metric_time__year AS customer_id__metric_time__year + , subq_7.metric_time__extract_year AS customer_id__metric_time__extract_year + , subq_7.metric_time__extract_quarter AS customer_id__metric_time__extract_quarter + , subq_7.metric_time__extract_month AS customer_id__metric_time__extract_month + , subq_7.metric_time__extract_day AS customer_id__metric_time__extract_day + , subq_7.metric_time__extract_dow AS customer_id__metric_time__extract_dow + , subq_7.metric_time__extract_doy AS customer_id__metric_time__extract_doy + , subq_4.ds_partitioned__day AS ds_partitioned__day , subq_4.ds_partitioned__week AS ds_partitioned__week , subq_4.ds_partitioned__month AS ds_partitioned__month , subq_4.ds_partitioned__quarter AS ds_partitioned__quarter @@ -196,28 +220,6 @@ FROM ( , subq_4.metric_time__extract_day AS metric_time__extract_day , subq_4.metric_time__extract_dow AS metric_time__extract_dow , subq_4.metric_time__extract_doy AS metric_time__extract_doy - , subq_7.ds_partitioned__day AS customer_id__ds_partitioned__day - , subq_7.ds_partitioned__week AS customer_id__ds_partitioned__week - , subq_7.ds_partitioned__month AS customer_id__ds_partitioned__month - , subq_7.ds_partitioned__quarter AS customer_id__ds_partitioned__quarter - , subq_7.ds_partitioned__year AS customer_id__ds_partitioned__year - , subq_7.ds_partitioned__extract_year AS customer_id__ds_partitioned__extract_year - , subq_7.ds_partitioned__extract_quarter AS customer_id__ds_partitioned__extract_quarter - , subq_7.ds_partitioned__extract_month AS customer_id__ds_partitioned__extract_month - , subq_7.ds_partitioned__extract_day AS customer_id__ds_partitioned__extract_day - , subq_7.ds_partitioned__extract_dow AS customer_id__ds_partitioned__extract_dow - , subq_7.ds_partitioned__extract_doy AS customer_id__ds_partitioned__extract_doy - , subq_7.metric_time__day AS customer_id__metric_time__day - , subq_7.metric_time__week AS customer_id__metric_time__week - , subq_7.metric_time__month AS customer_id__metric_time__month - , subq_7.metric_time__quarter AS customer_id__metric_time__quarter - , subq_7.metric_time__year AS customer_id__metric_time__year - , subq_7.metric_time__extract_year AS customer_id__metric_time__extract_year - , subq_7.metric_time__extract_quarter AS customer_id__metric_time__extract_quarter - , subq_7.metric_time__extract_month AS customer_id__metric_time__extract_month - , subq_7.metric_time__extract_day AS customer_id__metric_time__extract_day - , subq_7.metric_time__extract_dow AS customer_id__metric_time__extract_dow - , subq_7.metric_time__extract_doy AS customer_id__metric_time__extract_doy , subq_4.account_id AS account_id , subq_4.customer_id AS customer_id , subq_4.account_id__customer_id AS account_id__customer_id @@ -226,8 +228,6 @@ FROM ( , subq_4.extra_dim AS extra_dim , subq_4.account_id__extra_dim AS account_id__extra_dim , subq_4.bridge_account__extra_dim AS bridge_account__extra_dim - , subq_7.customer_name AS customer_id__customer_name - , subq_7.customer_atomic_weight AS customer_id__customer_atomic_weight , subq_4.account_customer_combos AS account_customer_combos FROM ( -- Metric Time Dimension 'ds_partitioned' diff --git a/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/Snowflake/test_partitioned_join__plan0.sql b/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/Snowflake/test_partitioned_join__plan0.sql index 61d75185f0..3e25438f88 100644 --- a/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/Snowflake/test_partitioned_join__plan0.sql +++ b/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/Snowflake/test_partitioned_join__plan0.sql @@ -15,10 +15,10 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.ds_partitioned__day AS ds_partitioned__day + subq_5.home_state AS user__home_state , subq_5.ds_partitioned__day AS user__ds_partitioned__day + , subq_2.ds_partitioned__day AS ds_partitioned__day , subq_2.user AS user - , subq_5.home_state AS user__home_state , subq_2.identity_verifications AS identity_verifications FROM ( -- Pass Only Elements: ['identity_verifications', 'ds_partitioned__day', 'user'] diff --git a/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/Trino/test_filter_with_where_constraint_on_join_dim__plan0.sql b/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/Trino/test_filter_with_where_constraint_on_join_dim__plan0.sql index dfb6893a52..29a44f49a4 100644 --- a/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/Trino/test_filter_with_where_constraint_on_join_dim__plan0.sql +++ b/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/Trino/test_filter_with_where_constraint_on_join_dim__plan0.sql @@ -27,9 +27,9 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.listing AS listing + subq_5.country_latest AS listing__country_latest + , subq_2.listing AS listing , subq_2.booking__is_instant AS booking__is_instant - , subq_5.country_latest AS listing__country_latest , subq_2.bookings AS bookings FROM ( -- Pass Only Elements: ['bookings', 'booking__is_instant', 'listing'] diff --git a/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/Trino/test_join_to_scd_dimension__plan0.sql b/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/Trino/test_join_to_scd_dimension__plan0.sql index ec853b87d4..6e92a52050 100644 --- a/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/Trino/test_join_to_scd_dimension__plan0.sql +++ b/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/Trino/test_join_to_scd_dimension__plan0.sql @@ -27,11 +27,11 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.metric_time__day AS metric_time__day + subq_4.capacity AS listing__capacity , subq_4.window_start__day AS listing__window_start__day , subq_4.window_end__day AS listing__window_end__day + , subq_2.metric_time__day AS metric_time__day , subq_2.listing AS listing - , subq_4.capacity AS listing__capacity , subq_2.bookings AS bookings FROM ( -- Pass Only Elements: ['bookings', 'metric_time__day', 'listing'] diff --git a/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/Trino/test_measure_constraint__plan0.sql b/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/Trino/test_measure_constraint__plan0.sql index 6170bbfafa..5f50cba6f2 100644 --- a/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/Trino/test_measure_constraint__plan0.sql +++ b/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/Trino/test_measure_constraint__plan0.sql @@ -39,9 +39,9 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.metric_time__day AS metric_time__day + subq_5.is_lux_latest AS listing__is_lux_latest + , subq_2.metric_time__day AS metric_time__day , subq_2.listing AS listing - , subq_5.is_lux_latest AS listing__is_lux_latest , subq_2.average_booking_value AS average_booking_value FROM ( -- Pass Only Elements: ['average_booking_value', 'metric_time__day', 'listing'] @@ -424,9 +424,9 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_14.metric_time__day AS metric_time__day + subq_17.is_lux_latest AS listing__is_lux_latest + , subq_14.metric_time__day AS metric_time__day , subq_14.listing AS listing - , subq_17.is_lux_latest AS listing__is_lux_latest , subq_14.bookings AS bookings FROM ( -- Pass Only Elements: ['bookings', 'metric_time__day', 'listing'] diff --git a/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/Trino/test_multi_hop_through_scd_dimension__plan0.sql b/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/Trino/test_multi_hop_through_scd_dimension__plan0.sql index b121adaecf..38184cbcf1 100644 --- a/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/Trino/test_multi_hop_through_scd_dimension__plan0.sql +++ b/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/Trino/test_multi_hop_through_scd_dimension__plan0.sql @@ -18,11 +18,11 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.metric_time__day AS metric_time__day + subq_7.user__home_state_latest AS listing__user__home_state_latest , subq_7.window_start__day AS listing__window_start__day , subq_7.window_end__day AS listing__window_end__day + , subq_2.metric_time__day AS metric_time__day , subq_2.listing AS listing - , subq_7.user__home_state_latest AS listing__user__home_state_latest , subq_2.bookings AS bookings FROM ( -- Pass Only Elements: ['bookings', 'metric_time__day', 'listing'] @@ -224,7 +224,19 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_3.window_start__day AS window_start__day + subq_5.home_state_latest AS user__home_state_latest + , subq_5.ds__day AS user__ds__day + , subq_5.ds__week AS user__ds__week + , subq_5.ds__month AS user__ds__month + , subq_5.ds__quarter AS user__ds__quarter + , subq_5.ds__year AS user__ds__year + , subq_5.ds__extract_year AS user__ds__extract_year + , subq_5.ds__extract_quarter AS user__ds__extract_quarter + , subq_5.ds__extract_month AS user__ds__extract_month + , subq_5.ds__extract_day AS user__ds__extract_day + , subq_5.ds__extract_dow AS user__ds__extract_dow + , subq_5.ds__extract_doy AS user__ds__extract_doy + , subq_3.window_start__day AS window_start__day , subq_3.window_start__week AS window_start__week , subq_3.window_start__month AS window_start__month , subq_3.window_start__quarter AS window_start__quarter @@ -268,17 +280,6 @@ FROM ( , subq_3.listing__window_end__extract_day AS listing__window_end__extract_day , subq_3.listing__window_end__extract_dow AS listing__window_end__extract_dow , subq_3.listing__window_end__extract_doy AS listing__window_end__extract_doy - , subq_5.ds__day AS user__ds__day - , subq_5.ds__week AS user__ds__week - , subq_5.ds__month AS user__ds__month - , subq_5.ds__quarter AS user__ds__quarter - , subq_5.ds__year AS user__ds__year - , subq_5.ds__extract_year AS user__ds__extract_year - , subq_5.ds__extract_quarter AS user__ds__extract_quarter - , subq_5.ds__extract_month AS user__ds__extract_month - , subq_5.ds__extract_day AS user__ds__extract_day - , subq_5.ds__extract_dow AS user__ds__extract_dow - , subq_5.ds__extract_doy AS user__ds__extract_doy , subq_3.listing AS listing , subq_3.user AS user , subq_3.listing__user AS listing__user @@ -288,7 +289,6 @@ FROM ( , subq_3.listing__country AS listing__country , subq_3.listing__is_lux AS listing__is_lux , subq_3.listing__capacity AS listing__capacity - , subq_5.home_state_latest AS user__home_state_latest FROM ( -- Read Elements From Semantic Model 'listings' SELECT diff --git a/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/Trino/test_multi_hop_to_scd_dimension__plan0.sql b/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/Trino/test_multi_hop_to_scd_dimension__plan0.sql index 6b614251bf..ab5787e746 100644 --- a/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/Trino/test_multi_hop_to_scd_dimension__plan0.sql +++ b/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/Trino/test_multi_hop_to_scd_dimension__plan0.sql @@ -18,11 +18,11 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.metric_time__day AS metric_time__day + subq_7.lux_listing__is_confirmed_lux AS listing__lux_listing__is_confirmed_lux , subq_7.lux_listing__window_start__day AS listing__lux_listing__window_start__day , subq_7.lux_listing__window_end__day AS listing__lux_listing__window_end__day + , subq_2.metric_time__day AS metric_time__day , subq_2.listing AS listing - , subq_7.lux_listing__is_confirmed_lux AS listing__lux_listing__is_confirmed_lux , subq_2.bookings AS bookings FROM ( -- Pass Only Elements: ['bookings', 'metric_time__day', 'listing'] @@ -224,7 +224,8 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_5.window_start__day AS lux_listing__window_start__day + subq_5.is_confirmed_lux AS lux_listing__is_confirmed_lux + , subq_5.window_start__day AS lux_listing__window_start__day , subq_5.window_start__week AS lux_listing__window_start__week , subq_5.window_start__month AS lux_listing__window_start__month , subq_5.window_start__quarter AS lux_listing__window_start__quarter @@ -249,7 +250,6 @@ FROM ( , subq_3.listing AS listing , subq_3.lux_listing AS lux_listing , subq_3.listing__lux_listing AS listing__lux_listing - , subq_5.is_confirmed_lux AS lux_listing__is_confirmed_lux FROM ( -- Read Elements From Semantic Model 'lux_listing_mapping' SELECT diff --git a/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/Trino/test_multihop_node__plan0.sql b/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/Trino/test_multihop_node__plan0.sql index e5a563d88a..5b4334b3c5 100644 --- a/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/Trino/test_multihop_node__plan0.sql +++ b/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/Trino/test_multihop_node__plan0.sql @@ -15,10 +15,10 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.ds_partitioned__day AS ds_partitioned__day + subq_9.customer_id__customer_name AS account_id__customer_id__customer_name , subq_9.ds_partitioned__day AS account_id__ds_partitioned__day + , subq_2.ds_partitioned__day AS ds_partitioned__day , subq_2.account_id AS account_id - , subq_9.customer_id__customer_name AS account_id__customer_id__customer_name , subq_2.txn_count AS txn_count FROM ( -- Pass Only Elements: ['txn_count', 'ds_partitioned__day', 'account_id'] @@ -152,7 +152,31 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_4.ds_partitioned__day AS ds_partitioned__day + subq_7.customer_name AS customer_id__customer_name + , subq_7.customer_atomic_weight AS customer_id__customer_atomic_weight + , subq_7.ds_partitioned__day AS customer_id__ds_partitioned__day + , subq_7.ds_partitioned__week AS customer_id__ds_partitioned__week + , subq_7.ds_partitioned__month AS customer_id__ds_partitioned__month + , subq_7.ds_partitioned__quarter AS customer_id__ds_partitioned__quarter + , subq_7.ds_partitioned__year AS customer_id__ds_partitioned__year + , subq_7.ds_partitioned__extract_year AS customer_id__ds_partitioned__extract_year + , subq_7.ds_partitioned__extract_quarter AS customer_id__ds_partitioned__extract_quarter + , subq_7.ds_partitioned__extract_month AS customer_id__ds_partitioned__extract_month + , subq_7.ds_partitioned__extract_day AS customer_id__ds_partitioned__extract_day + , subq_7.ds_partitioned__extract_dow AS customer_id__ds_partitioned__extract_dow + , subq_7.ds_partitioned__extract_doy AS customer_id__ds_partitioned__extract_doy + , subq_7.metric_time__day AS customer_id__metric_time__day + , subq_7.metric_time__week AS customer_id__metric_time__week + , subq_7.metric_time__month AS customer_id__metric_time__month + , subq_7.metric_time__quarter AS customer_id__metric_time__quarter + , subq_7.metric_time__year AS customer_id__metric_time__year + , subq_7.metric_time__extract_year AS customer_id__metric_time__extract_year + , subq_7.metric_time__extract_quarter AS customer_id__metric_time__extract_quarter + , subq_7.metric_time__extract_month AS customer_id__metric_time__extract_month + , subq_7.metric_time__extract_day AS customer_id__metric_time__extract_day + , subq_7.metric_time__extract_dow AS customer_id__metric_time__extract_dow + , subq_7.metric_time__extract_doy AS customer_id__metric_time__extract_doy + , subq_4.ds_partitioned__day AS ds_partitioned__day , subq_4.ds_partitioned__week AS ds_partitioned__week , subq_4.ds_partitioned__month AS ds_partitioned__month , subq_4.ds_partitioned__quarter AS ds_partitioned__quarter @@ -196,28 +220,6 @@ FROM ( , subq_4.metric_time__extract_day AS metric_time__extract_day , subq_4.metric_time__extract_dow AS metric_time__extract_dow , subq_4.metric_time__extract_doy AS metric_time__extract_doy - , subq_7.ds_partitioned__day AS customer_id__ds_partitioned__day - , subq_7.ds_partitioned__week AS customer_id__ds_partitioned__week - , subq_7.ds_partitioned__month AS customer_id__ds_partitioned__month - , subq_7.ds_partitioned__quarter AS customer_id__ds_partitioned__quarter - , subq_7.ds_partitioned__year AS customer_id__ds_partitioned__year - , subq_7.ds_partitioned__extract_year AS customer_id__ds_partitioned__extract_year - , subq_7.ds_partitioned__extract_quarter AS customer_id__ds_partitioned__extract_quarter - , subq_7.ds_partitioned__extract_month AS customer_id__ds_partitioned__extract_month - , subq_7.ds_partitioned__extract_day AS customer_id__ds_partitioned__extract_day - , subq_7.ds_partitioned__extract_dow AS customer_id__ds_partitioned__extract_dow - , subq_7.ds_partitioned__extract_doy AS customer_id__ds_partitioned__extract_doy - , subq_7.metric_time__day AS customer_id__metric_time__day - , subq_7.metric_time__week AS customer_id__metric_time__week - , subq_7.metric_time__month AS customer_id__metric_time__month - , subq_7.metric_time__quarter AS customer_id__metric_time__quarter - , subq_7.metric_time__year AS customer_id__metric_time__year - , subq_7.metric_time__extract_year AS customer_id__metric_time__extract_year - , subq_7.metric_time__extract_quarter AS customer_id__metric_time__extract_quarter - , subq_7.metric_time__extract_month AS customer_id__metric_time__extract_month - , subq_7.metric_time__extract_day AS customer_id__metric_time__extract_day - , subq_7.metric_time__extract_dow AS customer_id__metric_time__extract_dow - , subq_7.metric_time__extract_doy AS customer_id__metric_time__extract_doy , subq_4.account_id AS account_id , subq_4.customer_id AS customer_id , subq_4.account_id__customer_id AS account_id__customer_id @@ -226,8 +228,6 @@ FROM ( , subq_4.extra_dim AS extra_dim , subq_4.account_id__extra_dim AS account_id__extra_dim , subq_4.bridge_account__extra_dim AS bridge_account__extra_dim - , subq_7.customer_name AS customer_id__customer_name - , subq_7.customer_atomic_weight AS customer_id__customer_atomic_weight , subq_4.account_customer_combos AS account_customer_combos FROM ( -- Metric Time Dimension 'ds_partitioned' diff --git a/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/Trino/test_partitioned_join__plan0.sql b/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/Trino/test_partitioned_join__plan0.sql index 7d2a3efd92..20a2c516fb 100644 --- a/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/Trino/test_partitioned_join__plan0.sql +++ b/tests_metricflow/snapshots/test_query_rendering.py/SqlQueryPlan/Trino/test_partitioned_join__plan0.sql @@ -15,10 +15,10 @@ FROM ( FROM ( -- Join Standard Outputs SELECT - subq_2.ds_partitioned__day AS ds_partitioned__day + subq_5.home_state AS user__home_state , subq_5.ds_partitioned__day AS user__ds_partitioned__day + , subq_2.ds_partitioned__day AS ds_partitioned__day , subq_2.user AS user - , subq_5.home_state AS user__home_state , subq_2.identity_verifications AS identity_verifications FROM ( -- Pass Only Elements: ['identity_verifications', 'ds_partitioned__day', 'user']