From daf76959587d454355f513df4302b100297ac45e Mon Sep 17 00:00:00 2001 From: Courtney Holcomb Date: Tue, 10 Oct 2023 17:49:11 -0700 Subject: [PATCH 01/15] Changelog --- .changes/unreleased/Features-20231010-174851.yaml | 6 ++++++ 1 file changed, 6 insertions(+) create mode 100644 .changes/unreleased/Features-20231010-174851.yaml diff --git a/.changes/unreleased/Features-20231010-174851.yaml b/.changes/unreleased/Features-20231010-174851.yaml new file mode 100644 index 0000000000..6febb24e80 --- /dev/null +++ b/.changes/unreleased/Features-20231010-174851.yaml @@ -0,0 +1,6 @@ +kind: Features +body: Support querying dimensions without metrics. +time: 2023-10-10T17:48:51.152712-07:00 +custom: + Author: courtneyholcomb + Issue: "804" From 45b66dd5f590417cc8aec5e42a712eee361e1c2c Mon Sep 17 00:00:00 2001 From: Courtney Holcomb Date: Tue, 10 Oct 2023 18:06:23 -0700 Subject: [PATCH 02/15] Write dataflow plan for distinct values --- metricflow/dataflow/builder/costing.py | 6 +- .../dataflow/builder/dataflow_plan_builder.py | 320 ++++++++---------- metricflow/dataflow/builder/source_node.py | 4 + metricflow/dataflow/dataflow_plan.py | 16 +- metricflow/engine/metricflow_engine.py | 19 +- metricflow/plan_conversion/node_processor.py | 4 +- metricflow/query/query_parser.py | 28 +- 7 files changed, 207 insertions(+), 190 deletions(-) diff --git a/metricflow/dataflow/builder/costing.py b/metricflow/dataflow/builder/costing.py index 981bf406d5..b7c47a0faf 100644 --- a/metricflow/dataflow/builder/costing.py +++ b/metricflow/dataflow/builder/costing.py @@ -132,7 +132,11 @@ def visit_write_to_result_table_node(self, node: WriteToResultTableNode) -> Defa return DefaultCost.sum([x.accept(self) for x in node.parent_nodes]) def visit_pass_elements_filter_node(self, node: FilterElementsNode) -> DefaultCost: # noqa: D - return DefaultCost.sum([x.accept(self) for x in node.parent_nodes]) + parent_costs = [x.accept(self) for x in node.parent_nodes] + + # 1 aggregation if grouping by distinct values + node_cost = DefaultCost(num_aggregations=1 if node.distinct else 0) + return DefaultCost.sum(parent_costs + [node_cost]) def visit_combine_metrics_node(self, node: CombineMetricsNode) -> DefaultCost: # noqa: D return DefaultCost.sum([x.accept(self) for x in node.parent_nodes]) diff --git a/metricflow/dataflow/builder/dataflow_plan_builder.py b/metricflow/dataflow/builder/dataflow_plan_builder.py index 48e3c743a7..5065420725 100644 --- a/metricflow/dataflow/builder/dataflow_plan_builder.py +++ b/metricflow/dataflow/builder/dataflow_plan_builder.py @@ -9,10 +9,11 @@ from dbt_semantic_interfaces.enum_extension import assert_values_exhausted from dbt_semantic_interfaces.pretty_print import pformat_big_objects from dbt_semantic_interfaces.protocols.metric import MetricTimeWindow, MetricType -from dbt_semantic_interfaces.references import TimeDimensionReference +from dbt_semantic_interfaces.references import ( + TimeDimensionReference, +) from dbt_semantic_interfaces.type_enums.time_granularity import TimeGranularity -from metricflow.assert_one_arg import assert_exactly_one_arg_set from metricflow.dag.id_generation import DATAFLOW_PLAN_PREFIX, IdGeneratorRegistry from metricflow.dataflow.builder.costing import DataflowPlanNodeCostFunction, DefaultCostFunction from metricflow.dataflow.builder.measure_additiveness import group_measure_specs_by_additiveness @@ -50,11 +51,9 @@ from metricflow.filters.time_constraint import TimeRangeConstraint from metricflow.model.semantic_manifest_lookup import SemanticManifestLookup from metricflow.plan_conversion.column_resolver import DunderColumnAssociationResolver -from metricflow.plan_conversion.node_processor import PreDimensionJoinNodeProcessor +from metricflow.plan_conversion.node_processor import PreJoinNodeProcessor from metricflow.specs.column_assoc import ColumnAssociationResolver from metricflow.specs.specs import ( - DimensionSpec, - EntitySpec, InstanceSpecSet, LinkableInstanceSpec, LinkableSpecSet, @@ -74,18 +73,60 @@ @dataclass(frozen=True) -class MeasureRecipe: - """Get a recipe for how to build a dataflow plan node that outputs measures and the needed linkable instances. - - The recipe involves filtering the measure node so that it only outputs the measures and the instances associated with - required_local_linkable_specs, then joining the nodes containing the linkable instances according to the recipes - in join_linkable_instances_recipes. - """ +class DataflowRecipe: + """Get a recipe for how to build a dataflow plan node that outputs measures and linkable instances as needed.""" - measure_node: BaseOutput + source_node: BaseOutput required_local_linkable_specs: Tuple[LinkableInstanceSpec, ...] join_linkable_instances_recipes: Tuple[JoinLinkableInstancesRecipe, ...] + @property + def join_targets(self) -> List[JoinDescription]: + """Joins to be made to source node.""" + join_targets = [] + for join_recipe in self.join_linkable_instances_recipes: + # Figure out what elements to filter from the joined node. + + # Sanity check - all linkable specs should have a link, or else why would we be joining them. + assert all([len(x.entity_links) > 0 for x in join_recipe.satisfiable_linkable_specs]) + + # If we're joining something in, then we need the associated entity, partitions, and time dimension + # specs defining the validity window (if necessary) + include_specs: List[LinkableInstanceSpec] = [ + LinklessEntitySpec.from_reference(x.entity_links[0]) for x in join_recipe.satisfiable_linkable_specs + ] + include_specs.extend([x.node_to_join_dimension_spec for x in join_recipe.join_on_partition_dimensions]) + include_specs.extend( + [x.node_to_join_time_dimension_spec for x in join_recipe.join_on_partition_time_dimensions] + ) + if join_recipe.validity_window: + include_specs.extend( + [ + join_recipe.validity_window.window_start_dimension, + join_recipe.validity_window.window_end_dimension, + ] + ) + + # satisfiable_linkable_specs describes what can be satisfied after the join, so remove the entity + # link when filtering before the join. + # e.g. if the node is used to satisfy "user_id__country", then the node must have the entity + # "user_id" and the "country" dimension so that it can be joined to the measure node. + include_specs.extend([x.without_first_entity_link for x in join_recipe.satisfiable_linkable_specs]) + filtered_node_to_join = FilterElementsNode( + parent_node=join_recipe.node_to_join, + include_specs=InstanceSpecSet.create_from_linkable_specs(include_specs), + ) + join_targets.append( + JoinDescription( + join_node=filtered_node_to_join, + join_on_entity=join_recipe.join_on_entity, + join_on_partition_dimensions=join_recipe.join_on_partition_dimensions, + join_on_partition_time_dimensions=join_recipe.join_on_partition_time_dimensions, + validity_window=join_recipe.validity_window, + ) + ) + return join_targets + @dataclass(frozen=True) class MeasureSpecProperties: @@ -103,6 +144,7 @@ class DataflowPlanBuilder: def __init__( # noqa: D self, source_nodes: Sequence[BaseOutput], + read_nodes: Sequence[BaseOutput], semantic_manifest_lookup: SemanticManifestLookup, cost_function: DataflowPlanNodeCostFunction = DefaultCostFunction(), node_output_resolver: Optional[DataflowPlanNodeOutputDataSetResolver] = None, @@ -113,6 +155,7 @@ def __init__( # noqa: D self._metric_time_dimension_reference = DataSet.metric_time_dimension_reference() self._cost_function = cost_function self._source_nodes = source_nodes + self._read_nodes = read_nodes self._column_association_resolver = ( DunderColumnAssociationResolver(semantic_manifest_lookup) if not column_association_resolver @@ -146,8 +189,8 @@ def build_plan( time_range_constraint=query_spec.time_range_constraint, ) - sink_node = DataflowPlanBuilder.build_sink_node_from_metrics_output_node( - computed_metrics_output=metrics_output_node, + sink_node = DataflowPlanBuilder.build_sink_node( + parent_node=metrics_output_node, order_by_specs=query_spec.order_by_specs, output_sql_table=output_sql_table, limit=query_spec.limit, @@ -259,67 +302,50 @@ def _build_metrics_output_node( join_type=combine_metrics_join_type, ) - def build_plan_for_distinct_values( - self, - metric_specs: Sequence[MetricSpec], - dimension_spec: Optional[DimensionSpec] = None, - time_dimension_spec: Optional[TimeDimensionSpec] = None, - entity_spec: Optional[EntitySpec] = None, - time_range_constraint: Optional[TimeRangeConstraint] = None, - limit: Optional[int] = None, - ) -> DataflowPlan: + def build_plan_for_distinct_values(self, query_spec: MetricFlowQuerySpec) -> DataflowPlan: """Generate a plan that would get the distinct values of a linkable instance. e.g. distinct listing__country_latest for bookings by listing__country_latest """ - assert_exactly_one_arg_set( - dimension_spec=dimension_spec, time_dimension_spec=time_dimension_spec, entity_spec=entity_spec - ) + assert not query_spec.metric_specs, "Can't build distinct values plan with metrics." - # Doing this to keep the type checker happy, but assert_exactly_one_arg_set should ensure this. - linkable_spec: Optional[LinkableInstanceSpec] = dimension_spec or time_dimension_spec or entity_spec - assert linkable_spec + dataflow_recipe = self._find_dataflow_recipe(linkable_spec_set=query_spec.linkable_specs) + if not dataflow_recipe: + raise UnableToSatisfyQueryError(f"Recipe not found for linkable specs: {query_spec.linkable_specs}") - query_spec = MetricFlowQuerySpec( - metric_specs=tuple(metric_specs), - dimension_specs=(dimension_spec,) if dimension_spec else (), - time_dimension_specs=(time_dimension_spec,) if time_dimension_spec else (), - entity_specs=(entity_spec,) if entity_spec else (), - time_range_constraint=time_range_constraint, - ) - metrics_output_node = self._build_metrics_output_node( - metric_specs=query_spec.metric_specs, - queried_linkable_specs=query_spec.linkable_specs, - where_constraint=query_spec.where_constraint, - time_range_constraint=query_spec.time_range_constraint, - ) + join_targets = dataflow_recipe.join_targets + if join_targets: + joined_node = JoinToBaseOutputNode(left_node=dataflow_recipe.source_node, join_targets=join_targets) + distinct_values_node = FilterElementsNode( + parent_node=joined_node, include_specs=query_spec.linkable_specs.as_spec_set, distinct=True + ) + else: + distinct_values_node = FilterElementsNode( + parent_node=dataflow_recipe.source_node, + include_specs=query_spec.linkable_specs.as_spec_set, + distinct=True, + ) - distinct_values_node = FilterElementsNode( - parent_node=metrics_output_node, - include_specs=InstanceSpecSet.create_from_linkable_specs((linkable_spec,)), - ) + where_constraint_node: Optional[WhereConstraintNode] = None + if query_spec.where_constraint: + where_constraint_node = WhereConstraintNode( + parent_node=distinct_values_node, + where_constraint=query_spec.where_constraint, + ) - sink_node = self.build_sink_node_from_metrics_output_node( - computed_metrics_output=distinct_values_node, - order_by_specs=( - OrderBySpec( - instance_spec=linkable_spec, - descending=False, - ), - ), - limit=limit, + sink_node = self.build_sink_node( + parent_node=where_constraint_node or distinct_values_node, + order_by_specs=query_spec.order_by_specs, + limit=query_spec.limit, ) plan_id = IdGeneratorRegistry.for_class(DataflowPlanBuilder).create_id(DATAFLOW_PLAN_PREFIX) - return DataflowPlan( - plan_id=plan_id, - sink_output_nodes=[sink_node], - ) + return DataflowPlan(plan_id=plan_id, sink_output_nodes=[sink_node]) @staticmethod - def build_sink_node_from_metrics_output_node( - computed_metrics_output: BaseOutput, + def build_sink_node( + parent_node: BaseOutput, order_by_specs: Sequence[OrderBySpec], output_sql_table: Optional[SqlTable] = None, limit: Optional[int] = None, @@ -330,26 +356,20 @@ def build_sink_node_from_metrics_output_node( if order_by_specs or limit: pre_result_node = OrderByLimitNode( - order_by_specs=list(order_by_specs), - limit=limit, - parent_node=computed_metrics_output, + order_by_specs=list(order_by_specs), limit=limit, parent_node=parent_node ) if output_selection_specs: pre_result_node = FilterElementsNode( - parent_node=pre_result_node or computed_metrics_output, - include_specs=output_selection_specs, + parent_node=pre_result_node or parent_node, include_specs=output_selection_specs ) write_result_node: SinkOutput if not output_sql_table: - write_result_node = WriteToResultDataframeNode( - parent_node=pre_result_node or computed_metrics_output, - ) + write_result_node = WriteToResultDataframeNode(parent_node=pre_result_node or parent_node) else: write_result_node = WriteToResultTableNode( - parent_node=pre_result_node or computed_metrics_output, - output_sql_table=output_sql_table, + parent_node=pre_result_node or parent_node, output_sql_table=output_sql_table ) return write_result_node @@ -397,6 +417,21 @@ def _select_source_nodes_with_measures( nodes.append(source_node) return nodes + def _select_read_nodes_with_linkable_specs( + self, linkable_specs: LinkableSpecSet, read_nodes: Sequence[BaseOutput] + ) -> Dict[BaseOutput, Set[LinkableInstanceSpec]]: + """Find source nodes with requested linkable specs and no measures.""" + nodes_to_linkable_specs: Dict[BaseOutput, Set[LinkableInstanceSpec]] = {} + linkable_specs_set = set(linkable_specs.as_tuple) + for read_node in read_nodes: + output_spec_set = self._node_data_set_resolver.get_output_data_set(read_node).instance_set.spec_set + linkable_specs_in_node = set(output_spec_set.linkable_specs) + requested_linkable_specs_in_node = linkable_specs_set.intersection(linkable_specs_in_node) + if requested_linkable_specs_in_node: + nodes_to_linkable_specs[read_node] = requested_linkable_specs_in_node + + return nodes_to_linkable_specs + def _find_non_additive_dimension_in_linkable_specs( self, agg_time_dimension: TimeDimensionReference, @@ -449,39 +484,38 @@ def _build_measure_spec_properties(self, measure_specs: Sequence[MeasureSpec]) - non_additive_dimension_spec=non_additive_dimension_spec, ) - def _find_measure_recipe( + def _find_dataflow_recipe( self, - measure_spec_properties: MeasureSpecProperties, - linkable_specs: Sequence[LinkableInstanceSpec], + linkable_spec_set: LinkableSpecSet, + measure_spec_properties: Optional[MeasureSpecProperties] = None, time_range_constraint: Optional[TimeRangeConstraint] = None, - ) -> Optional[MeasureRecipe]: - """Find a recipe for getting measure_specs along with the linkable specs. + ) -> Optional[DataflowRecipe]: + linkable_specs = linkable_spec_set.as_tuple + if measure_spec_properties: + source_nodes = self._source_nodes + potential_source_nodes: Sequence[BaseOutput] = self._select_source_nodes_with_measures( + measure_specs=set(measure_spec_properties.measure_specs), source_nodes=source_nodes + ) + else: + # Only read nodes can be source nodes for queries without measures + source_nodes = self._read_nodes + source_nodes_to_linkable_specs = self._select_read_nodes_with_linkable_specs( + linkable_specs=linkable_spec_set, read_nodes=source_nodes + ) + potential_source_nodes = list(source_nodes_to_linkable_specs.keys()) + # issue: getting ds__day from the wrong table + logger.info(f"There are {len(potential_source_nodes)} potential source nodes") - Prior to calling this method we should always be checking that all input measure specs come from - the same base semantic model, otherwise the internal conditions here will be impossible to satisfy - """ - measure_specs = measure_spec_properties.measure_specs - node_processor = PreDimensionJoinNodeProcessor( + logger.info(f"Starting search with {len(source_nodes)} source nodes") + start_time = time.time() + + node_processor = PreJoinNodeProcessor( semantic_model_lookup=self._semantic_model_lookup, node_data_set_resolver=self._node_data_set_resolver, ) - - source_nodes: Sequence[BaseOutput] = self._source_nodes - - # We only care about nodes that have all required measures - potential_measure_nodes: Sequence[BaseOutput] = self._select_source_nodes_with_measures( - measure_specs=set(measure_specs), source_nodes=source_nodes - ) - - logger.info(f"There are {len(potential_measure_nodes)} potential measure source nodes") - - logger.info(f"Starting search with {len(source_nodes)} source nodes") - start_time = time.time() - # Only apply the time constraint to nodes that will be used for measures because some dimensional sources have - # measures in them, and time constraining those would result in incomplete joins. if time_range_constraint: - potential_measure_nodes = node_processor.add_time_range_constraint( - source_nodes=potential_measure_nodes, + potential_source_nodes = node_processor.add_time_range_constraint( + source_nodes=potential_source_nodes, metric_time_dimension_reference=self._metric_time_dimension_reference, time_range_constraint=time_range_constraint, ) @@ -500,7 +534,6 @@ def _find_measure_recipe( f"After adding multi-hop nodes, there are {len(nodes_available_for_joins)} nodes available for joins:\n" f"{pformat_big_objects(nodes_available_for_joins)}" ) - logger.info(f"Processing nodes took: {time.time()-start_time:.2f}s") node_evaluator = NodeEvaluatorForLinkableInstances( @@ -509,36 +542,23 @@ def _find_measure_recipe( node_data_set_resolver=self._node_data_set_resolver, ) - # Dict from the node that contains the measure spec to the evaluation results. + # Dict from the node that contains the source node to the evaluation results. node_to_evaluation: Dict[BaseOutput, LinkableInstanceSatisfiabilityEvaluation] = {} - for node in self._sort_by_suitability(potential_measure_nodes): - data_set = self._node_data_set_resolver.get_output_data_set(node) - - missing_specs = [spec for spec in measure_specs if spec not in data_set.instance_set.spec_set.measure_specs] - if missing_specs: - logger.debug( - f"Skipping evaluation for node since it does not have all of the measure specs {missing_specs}:" - f"\n\n{dataflow_dag_as_text(node)}" - ) - continue - - logger.debug(f"Evaluating measure node:\n{pformat_big_objects(measure_node=dataflow_dag_as_text(node))}") + for node in self._sort_by_suitability(potential_source_nodes): + logger.debug(f"Evaluating source node:\n{pformat_big_objects(source_node=dataflow_dag_as_text(node))}") start_time = time.time() - evaluation = node_evaluator.evaluate_node( - start_node=node, - required_linkable_specs=list(linkable_specs), - ) + evaluation = node_evaluator.evaluate_node(start_node=node, required_linkable_specs=list(linkable_specs)) logger.info(f"Evaluation of {node} took {time.time() - start_time:.2f}s") logger.debug( - f"Evaluation for measure node is:\n" + f"Evaluation for source node is:\n" f"{pformat_big_objects(node=dataflow_dag_as_text(node), evaluation=evaluation)}" ) if len(evaluation.unjoinable_linkable_specs) > 0: - logger.debug( + logger.info( f"Skipping {node.node_id} since it contains un-joinable specs: " f"{evaluation.unjoinable_linkable_specs}" ) @@ -555,7 +575,7 @@ def _find_measure_recipe( logger.info("Not evaluating other nodes since we found one that doesn't require joins") break - logger.info(f"Found {len(node_to_evaluation)} candidate measure nodes.") + logger.info(f"Found {len(node_to_evaluation)} candidate source nodes.") if len(node_to_evaluation) > 0: # All source nodes cost the same. Find evaluation with lowest number of joins. @@ -572,8 +592,8 @@ def _find_measure_recipe( ) ) - # Nodes containing the linkable instances will be joined to the node containing the measure, so these - # entities will need to be present in the measure node. + # Nodes containing the linkable instances will be joined to the source node, so these + # entities will need to be present in the source node. required_local_entity_specs = tuple(x.join_on_entity for x in evaluation.join_recipes) # Same thing with partitions. required_local_dimension_specs = tuple( @@ -585,8 +605,8 @@ def _find_measure_recipe( for y in x.join_on_partition_time_dimensions ) - return MeasureRecipe( - measure_node=node_with_lowest_cost_plan, + return DataflowRecipe( + source_node=node_with_lowest_cost_plan, required_local_linkable_specs=( evaluation.local_linkable_specs + required_local_entity_specs @@ -756,10 +776,10 @@ def _build_aggregated_measures_from_measure_source_node( ) find_recipe_start_time = time.time() - measure_recipe = self._find_measure_recipe( + measure_recipe = self._find_dataflow_recipe( measure_spec_properties=measure_properties, time_range_constraint=cumulative_metric_adjusted_time_constraint or time_range_constraint, - linkable_specs=required_linkable_specs.as_tuple, + linkable_spec_set=required_linkable_specs, ) logger.info( f"With {len(self._source_nodes)} source nodes, finding a recipe took " @@ -779,7 +799,7 @@ def _build_aggregated_measures_from_measure_source_node( time_range_node: Optional[JoinOverTimeRangeNode] = None if cumulative and metric_time_dimension_requested: time_range_node = JoinOverTimeRangeNode( - parent_node=measure_recipe.measure_node, + parent_node=measure_recipe.source_node, window=cumulative_window, grain_to_date=cumulative_grain_to_date, time_range_constraint=time_range_constraint, @@ -790,7 +810,7 @@ def _build_aggregated_measures_from_measure_source_node( if metric_spec.offset_window or metric_spec.offset_to_grain: assert metric_time_dimension_specs, "Joining to time spine requires querying with metric time." join_to_time_spine_node = JoinToTimeSpineNode( - parent_node=time_range_node or measure_recipe.measure_node, + parent_node=time_range_node or measure_recipe.source_node, metric_time_dimension_specs=metric_time_dimension_specs, time_range_constraint=time_range_constraint, offset_window=metric_spec.offset_window, @@ -799,7 +819,7 @@ def _build_aggregated_measures_from_measure_source_node( # Only get the required measure and the local linkable instances so that aggregations work correctly. filtered_measure_source_node = FilterElementsNode( - parent_node=join_to_time_spine_node or time_range_node or measure_recipe.measure_node, + parent_node=join_to_time_spine_node or time_range_node or measure_recipe.source_node, include_specs=InstanceSpecSet.merge( ( InstanceSpecSet(measure_specs=measure_specs), @@ -808,49 +828,7 @@ def _build_aggregated_measures_from_measure_source_node( ), ) - join_targets = [] - for join_recipe in measure_recipe.join_linkable_instances_recipes: - # Figure out what elements to filter from the joined node. - - # Sanity check - all linkable specs should have a link, or else why would we be joining them. - assert all([len(x.entity_links) > 0 for x in join_recipe.satisfiable_linkable_specs]) - - # If we're joining something in, then we need the associated entity, partitions, and time dimension - # specs defining the validity window (if necessary) - include_specs: List[LinkableInstanceSpec] = [ - LinklessEntitySpec.from_reference(x.entity_links[0]) for x in join_recipe.satisfiable_linkable_specs - ] - include_specs.extend([x.node_to_join_dimension_spec for x in join_recipe.join_on_partition_dimensions]) - include_specs.extend( - [x.node_to_join_time_dimension_spec for x in join_recipe.join_on_partition_time_dimensions] - ) - if join_recipe.validity_window: - include_specs.extend( - [ - join_recipe.validity_window.window_start_dimension, - join_recipe.validity_window.window_end_dimension, - ] - ) - - # satisfiable_linkable_specs describes what can be satisfied after the join, so remove the entity - # link when filtering before the join. - # e.g. if the node is used to satisfy "user_id__country", then the node must have the entity - # "user_id" and the "country" dimension so that it can be joined to the measure node. - include_specs.extend([x.without_first_entity_link for x in join_recipe.satisfiable_linkable_specs]) - filtered_node_to_join = FilterElementsNode( - parent_node=join_recipe.node_to_join, - include_specs=InstanceSpecSet.create_from_linkable_specs(include_specs), - ) - join_targets.append( - JoinDescription( - join_node=filtered_node_to_join, - join_on_entity=join_recipe.join_on_entity, - join_on_partition_dimensions=join_recipe.join_on_partition_dimensions, - join_on_partition_time_dimensions=join_recipe.join_on_partition_time_dimensions, - validity_window=join_recipe.validity_window, - ) - ) - + join_targets = measure_recipe.join_targets unaggregated_measure_node: BaseOutput if len(join_targets) > 0: filtered_measures_with_joined_elements = JoinToBaseOutputNode( diff --git a/metricflow/dataflow/builder/source_node.py b/metricflow/dataflow/builder/source_node.py index e2f2da7790..b77c2ac69f 100644 --- a/metricflow/dataflow/builder/source_node.py +++ b/metricflow/dataflow/builder/source_node.py @@ -46,3 +46,7 @@ def create_from_data_sets(self, data_sets: Sequence[SemanticModelDataSet]) -> Se ) ) return source_nodes + + def create_read_nodes_from_data_sets(self, data_sets: Sequence[SemanticModelDataSet]) -> Sequence[BaseOutput]: + """Creates read nodes from SemanticModelDataSets.""" + return [ReadSqlSourceNode(data_set) for data_set in data_sets] diff --git a/metricflow/dataflow/dataflow_plan.py b/metricflow/dataflow/dataflow_plan.py index 93fa93f2a1..a5f1bf9a35 100644 --- a/metricflow/dataflow/dataflow_plan.py +++ b/metricflow/dataflow/dataflow_plan.py @@ -1098,10 +1098,12 @@ def __init__( # noqa: D parent_node: BaseOutput, include_specs: InstanceSpecSet, replace_description: Optional[str] = None, + distinct: bool = False, ) -> None: self._include_specs = include_specs self._replace_description = replace_description self._parent_node = parent_node + self._distinct = distinct super().__init__(node_id=self.create_unique_id(), parent_nodes=[parent_node]) @classmethod @@ -1113,6 +1115,11 @@ def include_specs(self) -> InstanceSpecSet: """Returns the specs for the elements that it should pass.""" return self._include_specs + @property + def distinct(self) -> bool: + """True if you only want the distinct values for the selected specs.""" + return self._distinct + def accept(self, visitor: DataflowPlanNodeVisitor[VisitorOutputT]) -> VisitorOutputT: # noqa: D return visitor.visit_pass_elements_filter_node(self) @@ -1132,7 +1139,7 @@ def displayed_properties(self) -> List[DisplayedProperty]: # noqa: D if not self._replace_description: additional_properties = [ DisplayedProperty("include_spec", include_spec) for include_spec in self._include_specs.all_specs - ] + ] + [DisplayedProperty("distinct", self._distinct)] return super().displayed_properties + additional_properties @property @@ -1140,13 +1147,18 @@ def parent_node(self) -> BaseOutput: # noqa: D return self._parent_node def functionally_identical(self, other_node: DataflowPlanNode) -> bool: # noqa: D - return isinstance(other_node, self.__class__) and other_node.include_specs == self.include_specs + return ( + isinstance(other_node, self.__class__) + and other_node.include_specs == self.include_specs + and other_node.distinct == self.distinct + ) def with_new_parents(self, new_parent_nodes: Sequence[BaseOutput]) -> FilterElementsNode: # noqa: D assert len(new_parent_nodes) == 1 return FilterElementsNode( parent_node=new_parent_nodes[0], include_specs=self.include_specs, + distinct=self.distinct, replace_description=self._replace_description, ) diff --git a/metricflow/engine/metricflow_engine.py b/metricflow/engine/metricflow_engine.py index bf66a994b6..666821cf07 100644 --- a/metricflow/engine/metricflow_engine.py +++ b/metricflow/engine/metricflow_engine.py @@ -350,6 +350,7 @@ def __init__( source_node_builder = SourceNodeBuilder(self._semantic_manifest_lookup) source_nodes = source_node_builder.create_from_data_sets(self._source_data_sets) + read_nodes = source_node_builder.create_read_nodes_from_data_sets(self._source_data_sets) node_output_resolver = DataflowPlanNodeOutputDataSetResolver( column_association_resolver=DunderColumnAssociationResolver(semantic_manifest_lookup), @@ -358,6 +359,7 @@ def __init__( self._dataflow_plan_builder = DataflowPlanBuilder( source_nodes=source_nodes, + read_nodes=read_nodes, semantic_manifest_lookup=self._semantic_manifest_lookup, ) self._to_sql_query_plan_converter = DataflowToSqlQueryPlanConverter( @@ -374,7 +376,7 @@ def __init__( self._query_parser = MetricFlowQueryParser( column_association_resolver=self._column_association_resolver, model=self._semantic_manifest_lookup, - source_nodes=source_nodes, + read_nodes=read_nodes, node_output_resolver=node_output_resolver, ) @@ -483,12 +485,15 @@ def _create_execution_plan(self, mf_query_request: MetricFlowQueryRequest) -> Me time_dimension_specs=query_spec.time_dimension_specs, ) - dataflow_plan = self._dataflow_plan_builder.build_plan( - query_spec=query_spec, - output_sql_table=output_table, - output_selection_specs=output_selection_specs, - optimizers=(SourceScanOptimizer(),), - ) + if query_spec.metric_specs: + dataflow_plan = self._dataflow_plan_builder.build_plan( + query_spec=query_spec, + output_sql_table=output_table, + output_selection_specs=output_selection_specs, + optimizers=(SourceScanOptimizer(),), + ) + else: + dataflow_plan = self._dataflow_plan_builder.build_plan_for_distinct_values(query_spec=query_spec) if len(dataflow_plan.sink_output_nodes) > 1: raise NotImplementedError( diff --git a/metricflow/plan_conversion/node_processor.py b/metricflow/plan_conversion/node_processor.py index febcb3d5ac..1b94e4ceac 100644 --- a/metricflow/plan_conversion/node_processor.py +++ b/metricflow/plan_conversion/node_processor.py @@ -56,8 +56,8 @@ class MultiHopJoinCandidate: lineage: MultiHopJoinCandidateLineage -class PreDimensionJoinNodeProcessor: - """Processes source nodes before measures are joined to dimensions. +class PreJoinNodeProcessor: + """Processes source nodes before other nodes are joined. Generally, the source nodes will be combined with other dataflow plan nodes to produce a new set of nodes to realize a condition of the query. For example, to realize a time range constraint, a ConstrainTimeRangeNode will be added diff --git a/metricflow/query/query_parser.py b/metricflow/query/query_parser.py index 6bb4185da2..e70a782880 100644 --- a/metricflow/query/query_parser.py +++ b/metricflow/query/query_parser.py @@ -118,13 +118,15 @@ def __init__( # noqa: D self, column_association_resolver: ColumnAssociationResolver, model: SemanticManifestLookup, - source_nodes: Sequence[BaseOutput], + read_nodes: Sequence[BaseOutput], node_output_resolver: DataflowPlanNodeOutputDataSetResolver, ) -> None: self._column_association_resolver = column_association_resolver self._model = model self._metric_lookup = model.metric_lookup self._semantic_model_lookup = model.semantic_model_lookup + self._node_output_resolver = node_output_resolver + self._read_nodes = read_nodes # Set up containers for known element names self._known_entity_element_references = self._semantic_model_lookup.get_entity_references() @@ -228,12 +230,15 @@ def _validate_no_time_dimension_query(self, metric_references: Sequence[MetricRe "dimension 'metric_time'." ) - def _validate_linkable_specs( + def _validate_linkable_specs_for_metrics( self, metric_references: Tuple[MetricReference, ...], all_linkable_specs: QueryTimeLinkableSpecSet, time_dimension_specs: Tuple[TimeDimensionSpec, ...], ) -> None: + if not metric_references: + return None + invalid_group_bys = self._get_invalid_linkable_specs( metric_references=metric_references, dimension_specs=all_linkable_specs.dimension_specs, @@ -296,6 +301,9 @@ def _construct_metric_specs_for_query( def _get_metric_names( self, metric_names: Optional[Sequence[str]], metrics: Optional[Sequence[MetricQueryParameter]] ) -> Sequence[str]: + if not (metric_names or metrics): + return [] + assert_exactly_one_arg_set(metric_names=metric_names, metrics=metrics) return metric_names if metric_names else [m.name for m in metrics] if metrics else [] @@ -402,6 +410,8 @@ def _parse_and_validate_query( self._time_granularity_solver.resolve_granularity_for_partial_time_dimension_specs( metric_references=metric_references, partial_time_dimension_specs=requested_linkable_specs.partial_time_dimension_specs, + read_nodes=self._read_nodes, + node_output_resolver=self._node_output_resolver, ) ) @@ -422,7 +432,7 @@ def _parse_and_validate_query( # For each metric, verify that it's possible to retrieve all group by elements, including the ones as required # by the filters. - # TODO: Consider moving this logic into _validate_linkable_specs(). + # TODO: Consider moving this logic into _validate_linkable_specs_for_metrics(). for metric_reference in metric_references: metric = self._metric_lookup.get_metric(metric_reference) if metric.filter is not None: @@ -434,7 +444,7 @@ def _parse_and_validate_query( # Combine the group by elements from the query with the group by elements that are required by the # metric filter to see if that's a valid set that could be queried. - self._validate_linkable_specs( + self._validate_linkable_specs_for_metrics( metric_references=(metric_reference,), all_linkable_specs=QueryTimeLinkableSpecSet.combine( ( @@ -452,7 +462,7 @@ def _parse_and_validate_query( ) # Validate all of them together. - self._validate_linkable_specs( + self._validate_linkable_specs_for_metrics( metric_references=metric_references, all_linkable_specs=requested_linkable_specs_with_requested_filter_specs, time_dimension_specs=time_dimension_specs, @@ -573,6 +583,8 @@ def _adjust_time_range_constraint( self._time_granularity_solver.resolve_granularity_for_partial_time_dimension_specs( metric_references=metric_references, partial_time_dimension_specs=(partial_metric_time_spec,), + read_nodes=self._read_nodes, + node_output_resolver=self._node_output_resolver, ) ) adjust_to_granularity = partial_time_dimension_spec_to_time_dimension_spec[ @@ -670,7 +682,6 @@ def _parse_group_by( group_by: Optional[Tuple[GroupByParameter, ...]] = None, ) -> QueryTimeLinkableSpecSet: """Convert the linkable spec names into the respective specification objects.""" - # TODO: refactor to only support group_by object inputs (removing group_by_names param) assert not ( group_by_names and group_by ), "Both group_by_names and group_by were set, but if a group by is specified you should only use one of these!" @@ -772,7 +783,10 @@ def _verify_resolved_granularity_for_date_part( ensure that the correct value was passed in. """ resolved_granularity = self._time_granularity_solver.find_minimum_granularity_for_partial_time_dimension_spec( - partial_time_dimension_spec=partial_time_dimension_spec, metric_references=metric_references + partial_time_dimension_spec=partial_time_dimension_spec, + metric_references=metric_references, + read_nodes=self._read_nodes, + node_output_resolver=self._node_output_resolver, ) if resolved_granularity != requested_dimension_structured_name.time_granularity: raise RequestTimeGranularityException( From a1afcbb0498ac37b973915f1c13c00d3802f6fef Mon Sep 17 00:00:00 2001 From: Courtney Holcomb Date: Tue, 10 Oct 2023 18:07:43 -0700 Subject: [PATCH 03/15] Find default granularity for time dimension without metrics --- metricflow/time/time_granularity_solver.py | 96 +++++++++++++++++----- 1 file changed, 74 insertions(+), 22 deletions(-) diff --git a/metricflow/time/time_granularity_solver.py b/metricflow/time/time_granularity_solver.py index f23551f612..1a3bbe62d4 100644 --- a/metricflow/time/time_granularity_solver.py +++ b/metricflow/time/time_granularity_solver.py @@ -14,8 +14,11 @@ ) from dbt_semantic_interfaces.type_enums.time_granularity import TimeGranularity +from metricflow.dataflow.builder.node_data_set import DataflowPlanNodeOutputDataSetResolver +from metricflow.dataflow.dataflow_plan import BaseOutput from metricflow.filters.time_constraint import TimeRangeConstraint from metricflow.model.semantic_manifest_lookup import SemanticManifestLookup +from metricflow.naming.linkable_spec_name import StructuredLinkableSpecName from metricflow.specs.specs import ( TimeDimensionSpec, ) @@ -74,6 +77,9 @@ def validate_time_granularity( e.g. throw an error if "ds__week" is specified for a metric with a time granularity of MONTH. """ + if not metric_references: + return None + valid_group_by_elements = self._semantic_manifest_lookup.metric_lookup.linkable_set_for_metrics( metric_references=metric_references, ) @@ -97,15 +103,21 @@ def resolve_granularity_for_partial_time_dimension_specs( self, metric_references: Sequence[MetricReference], partial_time_dimension_specs: Sequence[PartialTimeDimensionSpec], + read_nodes: Sequence[BaseOutput], + node_output_resolver: DataflowPlanNodeOutputDataSetResolver, ) -> Dict[PartialTimeDimensionSpec, TimeDimensionSpec]: """Figure out the lowest granularity possible for the partially specified time dimension specs. Returns a dictionary that maps how the partial time dimension spec should be turned into a time dimension spec. """ result: Dict[PartialTimeDimensionSpec, TimeDimensionSpec] = {} + for partial_time_dimension_spec in partial_time_dimension_specs: minimum_time_granularity = self.find_minimum_granularity_for_partial_time_dimension_spec( - partial_time_dimension_spec=partial_time_dimension_spec, metric_references=metric_references + partial_time_dimension_spec=partial_time_dimension_spec, + metric_references=metric_references, + read_nodes=read_nodes, + node_output_resolver=node_output_resolver, ) result[partial_time_dimension_spec] = TimeDimensionSpec( element_name=partial_time_dimension_spec.element_name, @@ -116,35 +128,75 @@ def resolve_granularity_for_partial_time_dimension_specs( return result def find_minimum_granularity_for_partial_time_dimension_spec( - self, partial_time_dimension_spec: PartialTimeDimensionSpec, metric_references: Sequence[MetricReference] + self, + partial_time_dimension_spec: PartialTimeDimensionSpec, + metric_references: Sequence[MetricReference], + read_nodes: Sequence[BaseOutput], + node_output_resolver: DataflowPlanNodeOutputDataSetResolver, ) -> TimeGranularity: """Find minimum granularity allowed for time dimension when queried with given metrics.""" - valid_group_by_elements = self._semantic_manifest_lookup.metric_lookup.linkable_set_for_metrics( - metric_references=metric_references, - ) - minimum_time_granularity: Optional[TimeGranularity] = None - for path_key in valid_group_by_elements.path_key_to_linkable_dimensions: - if ( - path_key.element_name == partial_time_dimension_spec.element_name - and path_key.entity_links == partial_time_dimension_spec.entity_links - and path_key.time_granularity is not None - ): - minimum_time_granularity = ( - path_key.time_granularity - if minimum_time_granularity is None - else min(minimum_time_granularity, path_key.time_granularity) - ) - if not minimum_time_granularity: - raise RequestTimeGranularityException( - f"Unable to resolve the time dimension spec for {partial_time_dimension_spec}. " - f"Valid group by elements are:\n" - f"{pformat_big_objects([spec.qualified_name for spec in valid_group_by_elements.as_spec_set.as_tuple])}" + if metric_references: + valid_group_by_elements = self._semantic_manifest_lookup.metric_lookup.linkable_set_for_metrics( + metric_references=metric_references, ) + for path_key in valid_group_by_elements.path_key_to_linkable_dimensions: + if ( + path_key.element_name == partial_time_dimension_spec.element_name + and path_key.entity_links == partial_time_dimension_spec.entity_links + and path_key.time_granularity is not None + ): + minimum_time_granularity = ( + path_key.time_granularity + if minimum_time_granularity is None + else min(minimum_time_granularity, path_key.time_granularity) + ) + if not minimum_time_granularity: + raise RequestTimeGranularityException( + f"Unable to resolve the time dimension spec for {partial_time_dimension_spec}. " + f"Valid group by elements are:\n" + f"{pformat_big_objects([spec.qualified_name for spec in valid_group_by_elements.as_spec_set.as_tuple])}" + ) + else: + minimum_time_granularity = self.get_min_granularity_for_partial_time_dimension_without_metrics( + read_nodes=read_nodes, + node_output_resolver=node_output_resolver, + partial_time_dimension_spec=partial_time_dimension_spec, + ) + if not minimum_time_granularity: + raise RequestTimeGranularityException( + f"Unable to resolve the time dimension spec for {partial_time_dimension_spec}. " + ) return minimum_time_granularity + def get_min_granularity_for_partial_time_dimension_without_metrics( + self, + read_nodes: Sequence[BaseOutput], + node_output_resolver: DataflowPlanNodeOutputDataSetResolver, + partial_time_dimension_spec: PartialTimeDimensionSpec, + ) -> Optional[TimeGranularity]: + """Find the minimum.""" + granularity_free_qualified_name = StructuredLinkableSpecName( + entity_link_names=tuple( + [entity_link.element_name for entity_link in partial_time_dimension_spec.entity_links] + ), + element_name=partial_time_dimension_spec.element_name, + ).granularity_free_qualified_name + for read_node in read_nodes: + output_data_set = node_output_resolver.get_output_data_set(read_node) + for time_dimension_instance in output_data_set.instance_set.time_dimension_instances: + if time_dimension_instance.spec.date_part: + continue + time_dim_name_without_granularity = StructuredLinkableSpecName.from_name( + time_dimension_instance.spec.qualified_name + ).granularity_free_qualified_name + if time_dim_name_without_granularity == granularity_free_qualified_name: + return time_dimension_instance.spec.time_granularity + + return None + def adjust_time_range_to_granularity( self, time_range_constraint: TimeRangeConstraint, time_granularity: TimeGranularity ) -> TimeRangeConstraint: From 4f0d5d2552fde763928aee5b932cd9dc772d8667 Mon Sep 17 00:00:00 2001 From: Courtney Holcomb Date: Tue, 10 Oct 2023 18:09:02 -0700 Subject: [PATCH 04/15] Update dataflow to SQL plan for FilterElementsNode --- metricflow/plan_conversion/dataflow_to_sql.py | 14 +++++++++----- 1 file changed, 9 insertions(+), 5 deletions(-) diff --git a/metricflow/plan_conversion/dataflow_to_sql.py b/metricflow/plan_conversion/dataflow_to_sql.py index ff5b0f9359..9b61cdde9d 100644 --- a/metricflow/plan_conversion/dataflow_to_sql.py +++ b/metricflow/plan_conversion/dataflow_to_sql.py @@ -797,18 +797,22 @@ def visit_pass_elements_filter_node(self, node: FilterElementsNode) -> SqlDataSe # Also, the output columns should always follow the resolver format. output_instance_set = output_instance_set.transform(ChangeAssociatedColumns(self._column_association_resolver)) + # This creates select expressions for all columns referenced in the instance set. + select_columns = output_instance_set.transform( + CreateSelectColumnsForInstances(from_data_set_alias, self._column_association_resolver) + ).as_tuple() + + # If distinct values requested, group by all select columns. + group_bys = select_columns if node.distinct else () return SqlDataSet( instance_set=output_instance_set, sql_select_node=SqlSelectStatementNode( description=node.description, - # This creates select expressions for all columns referenced in the instance set. - select_columns=output_instance_set.transform( - CreateSelectColumnsForInstances(from_data_set_alias, self._column_association_resolver) - ).as_tuple(), + select_columns=select_columns, from_source=from_data_set.sql_select_node, from_source_alias=from_data_set_alias, joins_descs=(), - group_bys=(), + group_bys=group_bys, where=None, order_bys=(), ), From c9526e564566111831b158170d21911b2d74dd0a Mon Sep 17 00:00:00 2001 From: Courtney Holcomb Date: Tue, 10 Oct 2023 18:10:13 -0700 Subject: [PATCH 05/15] Write & update tests --- .../test/dataflow/builder/test_cyclic_join.py | 1 + .../builder/test_dataflow_plan_builder.py | 78 ++++++++++++++++-- .../dataflow/builder/test_node_evaluator.py | 4 +- metricflow/test/fixtures/dataflow_fixtures.py | 3 + metricflow/test/fixtures/model_fixtures.py | 15 +++- .../test/integration/configured_test_case.py | 2 +- .../test_cases/itest_dimensions.yaml | 80 +++++++++++++++++++ .../test_dataflow_to_sql_plan.py | 30 +++++-- metricflow/test/query/test_query_parser.py | 3 + .../test/time/test_time_granularity_solver.py | 22 ++++- 10 files changed, 219 insertions(+), 19 deletions(-) diff --git a/metricflow/test/dataflow/builder/test_cyclic_join.py b/metricflow/test/dataflow/builder/test_cyclic_join.py index 8762e45ce3..d35a2b3987 100644 --- a/metricflow/test/dataflow/builder/test_cyclic_join.py +++ b/metricflow/test/dataflow/builder/test_cyclic_join.py @@ -34,6 +34,7 @@ def cyclic_join_manifest_dataflow_plan_builder( # noqa: D return DataflowPlanBuilder( source_nodes=consistent_id_object_repository.cyclic_join_source_nodes, + read_nodes=list(consistent_id_object_repository.cyclic_join_read_nodes.values()), semantic_manifest_lookup=cyclic_join_semantic_manifest_lookup, cost_function=DefaultCostFunction(), ) diff --git a/metricflow/test/dataflow/builder/test_dataflow_plan_builder.py b/metricflow/test/dataflow/builder/test_dataflow_plan_builder.py index cb97cee1e2..586e2e6d21 100644 --- a/metricflow/test/dataflow/builder/test_dataflow_plan_builder.py +++ b/metricflow/test/dataflow/builder/test_dataflow_plan_builder.py @@ -562,15 +562,81 @@ def test_distinct_values_plan( # noqa: D request: FixtureRequest, mf_test_session_state: MetricFlowTestSessionState, dataflow_plan_builder: DataflowPlanBuilder, + column_association_resolver: ColumnAssociationResolver, ) -> None: """Tests a plan to get distinct values of a dimension.""" dataflow_plan = dataflow_plan_builder.build_plan_for_distinct_values( - metric_specs=(MetricSpec(element_name="bookings"),), - dimension_spec=DimensionSpec( - element_name="country_latest", - entity_links=(EntityReference(element_name="listing"),), - ), - limit=100, + query_spec=MetricFlowQuerySpec( + dimension_specs=( + DimensionSpec(element_name="country_latest", entity_links=(EntityReference(element_name="listing"),)), + ), + where_constraint=( + WhereSpecFactory( + column_association_resolver=column_association_resolver, + ).create_from_where_filter( + PydanticWhereFilter( + where_sql_template="{{ Dimension('listing__country_latest') }} = 'us'", + ) + ) + ), + order_by_specs=( + OrderBySpec( + instance_spec=DimensionSpec( + element_name="country_latest", entity_links=(EntityReference(element_name="listing"),) + ), + descending=True, + ), + ), + limit=100, + ) + ) + + assert_plan_snapshot_text_equal( + request=request, + mf_test_session_state=mf_test_session_state, + plan=dataflow_plan, + plan_snapshot_text=dataflow_plan_as_text(dataflow_plan), + ) + + display_graph_if_requested( + request=request, + mf_test_session_state=mf_test_session_state, + dag_graph=dataflow_plan, + ) + + +def test_distinct_values_plan_with_join( # noqa: D + request: FixtureRequest, + mf_test_session_state: MetricFlowTestSessionState, + dataflow_plan_builder: DataflowPlanBuilder, + column_association_resolver: ColumnAssociationResolver, +) -> None: + """Tests a plan to get distinct values of 2 dimensions, where a join is required.""" + dataflow_plan = dataflow_plan_builder.build_plan_for_distinct_values( + query_spec=MetricFlowQuerySpec( + dimension_specs=( + DimensionSpec(element_name="home_state_latest", entity_links=(EntityReference(element_name="user"),)), + DimensionSpec(element_name="is_lux_latest", entity_links=(EntityReference(element_name="listing"),)), + ), + where_constraint=( + WhereSpecFactory( + column_association_resolver=column_association_resolver, + ).create_from_where_filter( + PydanticWhereFilter( + where_sql_template="{{ Dimension('listing__country_latest') }} = 'us'", + ) + ) + ), + order_by_specs=( + OrderBySpec( + instance_spec=DimensionSpec( + element_name="country_latest", entity_links=(EntityReference(element_name="listing"),) + ), + descending=True, + ), + ), + limit=100, + ) ) assert_plan_snapshot_text_equal( diff --git a/metricflow/test/dataflow/builder/test_node_evaluator.py b/metricflow/test/dataflow/builder/test_node_evaluator.py index 523cb416ed..c88fa7319e 100644 --- a/metricflow/test/dataflow/builder/test_node_evaluator.py +++ b/metricflow/test/dataflow/builder/test_node_evaluator.py @@ -18,7 +18,7 @@ from metricflow.dataset.dataset import DataSet from metricflow.model.semantic_manifest_lookup import SemanticManifestLookup from metricflow.plan_conversion.column_resolver import DunderColumnAssociationResolver -from metricflow.plan_conversion.node_processor import PreDimensionJoinNodeProcessor +from metricflow.plan_conversion.node_processor import PreJoinNodeProcessor from metricflow.specs.specs import ( DimensionSpec, EntityReference, @@ -65,7 +65,7 @@ def make_multihop_node_evaluator( semantic_manifest_lookup=semantic_manifest_lookup_with_multihop_links, ) - node_processor = PreDimensionJoinNodeProcessor( + node_processor = PreJoinNodeProcessor( semantic_model_lookup=semantic_manifest_lookup_with_multihop_links.semantic_model_lookup, node_data_set_resolver=node_data_set_resolver, ) diff --git a/metricflow/test/fixtures/dataflow_fixtures.py b/metricflow/test/fixtures/dataflow_fixtures.py index 06616aa7f1..e1af9103d2 100644 --- a/metricflow/test/fixtures/dataflow_fixtures.py +++ b/metricflow/test/fixtures/dataflow_fixtures.py @@ -34,6 +34,7 @@ def dataflow_plan_builder( # noqa: D ) -> DataflowPlanBuilder: return DataflowPlanBuilder( source_nodes=consistent_id_object_repository.simple_model_source_nodes, + read_nodes=list(consistent_id_object_repository.simple_model_read_nodes.values()), semantic_manifest_lookup=simple_semantic_manifest_lookup, cost_function=DefaultCostFunction(), ) @@ -47,6 +48,7 @@ def multihop_dataflow_plan_builder( # noqa: D ) -> DataflowPlanBuilder: return DataflowPlanBuilder( source_nodes=consistent_id_object_repository.multihop_model_source_nodes, + read_nodes=list(consistent_id_object_repository.multihop_model_read_nodes.values()), semantic_manifest_lookup=multi_hop_join_semantic_manifest_lookup, cost_function=DefaultCostFunction(), ) @@ -68,6 +70,7 @@ def scd_dataflow_plan_builder( # noqa: D ) -> DataflowPlanBuilder: return DataflowPlanBuilder( source_nodes=consistent_id_object_repository.scd_model_source_nodes, + read_nodes=list(consistent_id_object_repository.scd_model_read_nodes.values()), semantic_manifest_lookup=scd_semantic_manifest_lookup, cost_function=DefaultCostFunction(), column_association_resolver=scd_column_association_resolver, diff --git a/metricflow/test/fixtures/model_fixtures.py b/metricflow/test/fixtures/model_fixtures.py index c863a1eb00..afcf7bf38e 100644 --- a/metricflow/test/fixtures/model_fixtures.py +++ b/metricflow/test/fixtures/model_fixtures.py @@ -61,11 +61,10 @@ def query_parser_from_yaml(yaml_contents: List[YamlConfigFile]) -> MetricFlowQue ).semantic_manifest ) SemanticManifestValidator[SemanticManifest]().checked_validations(semantic_manifest_lookup.semantic_manifest) - source_nodes = _data_set_to_source_nodes(semantic_manifest_lookup, create_data_sets(semantic_manifest_lookup)) return MetricFlowQueryParser( model=semantic_manifest_lookup, column_association_resolver=DunderColumnAssociationResolver(semantic_manifest_lookup), - source_nodes=source_nodes, + read_nodes=list(_data_set_to_read_nodes(create_data_sets(semantic_manifest_lookup)).values()), node_output_resolver=DataflowPlanNodeOutputDataSetResolver( column_association_resolver=DunderColumnAssociationResolver(semantic_manifest_lookup), semantic_manifest_lookup=semantic_manifest_lookup, @@ -88,6 +87,7 @@ class ConsistentIdObjectRepository: scd_model_read_nodes: OrderedDict[str, ReadSqlSourceNode] scd_model_source_nodes: Sequence[BaseOutput] + cyclic_join_read_nodes: OrderedDict[str, ReadSqlSourceNode] cyclic_join_source_nodes: Sequence[BaseOutput] @@ -122,6 +122,7 @@ def consistent_id_object_repository( scd_model_source_nodes=_data_set_to_source_nodes( semantic_manifest_lookup=scd_semantic_manifest_lookup, data_sets=scd_data_sets ), + cyclic_join_read_nodes=_data_set_to_read_nodes(cyclic_join_data_sets), cyclic_join_source_nodes=_data_set_to_source_nodes( semantic_manifest_lookup=cyclic_join_semantic_manifest_lookup, data_sets=cyclic_join_data_sets ), @@ -239,3 +240,13 @@ def cyclic_join_semantic_manifest_lookup(template_mapping: Dict[str, str]) -> Se """Manifest that contains a potential cycle in the join graph (if not handled properly).""" build_result = load_semantic_manifest("cyclic_join_manifest", template_mapping) return SemanticManifestLookup(build_result.semantic_manifest) + + +@pytest.fixture(scope="session") +def node_output_resolver( # noqa:D + simple_semantic_manifest_lookup: SemanticManifestLookup, +) -> DataflowPlanNodeOutputDataSetResolver: + return DataflowPlanNodeOutputDataSetResolver( + column_association_resolver=DunderColumnAssociationResolver(simple_semantic_manifest_lookup), + semantic_manifest_lookup=simple_semantic_manifest_lookup, + ) diff --git a/metricflow/test/integration/configured_test_case.py b/metricflow/test/integration/configured_test_case.py index 766873b9b0..b0e59d3ab2 100644 --- a/metricflow/test/integration/configured_test_case.py +++ b/metricflow/test/integration/configured_test_case.py @@ -48,10 +48,10 @@ class Config: # noqa: D name: str # Name of the semantic model to use. model: IntegrationTestModel - metrics: Tuple[str, ...] # The SQL query that can be run to obtain the expected results. check_query: str file_path: str + metrics: Tuple[str, ...] = () group_bys: Tuple[str, ...] = () group_by_objs: Tuple[Dict, ...] = () order_bys: Tuple[str, ...] = () diff --git a/metricflow/test/integration/test_cases/itest_dimensions.yaml b/metricflow/test/integration/test_cases/itest_dimensions.yaml index ea5affa47c..c6d5912629 100644 --- a/metricflow/test/integration/test_cases/itest_dimensions.yaml +++ b/metricflow/test/integration/test_cases/itest_dimensions.yaml @@ -102,3 +102,83 @@ integration_test: GROUP BY v.ds , u.home_state +--- +integration_test: + name: query_dimension_only + description: Query dimenension only + model: SIMPLE_MODEL + group_bys: ["user__home_state"] + check_query: | + SELECT + u.home_state AS user__home_state + FROM {{ source_schema }}.dim_users u + GROUP BY + u.home_state +--- +integration_test: + name: query_dimensions_only + description: Query multiple dimensions without metrics + model: SIMPLE_MODEL + group_bys: ["ds__day", "user__home_state"] + check_query: | + SELECT + u.home_state AS user__home_state + , u.ds AS ds__day + FROM {{ source_schema }}.dim_users u + GROUP BY + u.ds + , u.home_state +--- +integration_test: + name: query_dimensions_from_different_tables + description: Query multiple dimensions without metrics, requiring a join + model: SIMPLE_MODEL + group_bys: ["user__home_state_latest", "listing__is_lux_latest"] + check_query: | + SELECT + u.home_state_latest AS user__home_state_latest + , l.is_lux AS listing__is_lux_latest + FROM {{ source_schema }}.dim_listings_latest l + LEFT OUTER JOIN {{ source_schema }}.dim_users_latest u + ON u.user_id = l.user_id + GROUP BY + u.home_state_latest + , l.is_lux +--- +integration_test: + name: query_time_dimension_without_granularity + description: Query just a time dimension, no granularity specified. Should assume default granularity for dimension. + model: SIMPLE_MODEL + group_bys: [ "verification__ds"] + check_query: | + SELECT + v.ds as verification__ds__day + FROM {{ source_schema }}.fct_id_verifications v + GROUP BY + v.ds +--- +integration_test: + name: query_non_default_time_dimension_without_granularity + description: Query just a time dimension, no granularity specified. Should assume default granularity for dimension. + model: EXTENDED_DATE_MODEL + group_bys: [ "monthly_ds"] + check_query: | + SELECT + ds AS monthly_ds__month + FROM {{ source_schema }}.fct_bookings_extended_monthly + GROUP BY + ds +--- +integration_test: + name: query_dimension_only_with_constraint + description: Query dimenension only + model: SIMPLE_MODEL + group_bys: ["user__home_state"] + where_filter: "{{ render_dimension_template('user__home_state') }} = 'CA'" + check_query: | + SELECT + u.home_state AS user__home_state + FROM {{ source_schema }}.dim_users u + WHERE user__home_state = 'CA' + GROUP BY + u.home_state diff --git a/metricflow/test/plan_conversion/test_dataflow_to_sql_plan.py b/metricflow/test/plan_conversion/test_dataflow_to_sql_plan.py index 71d78e45fe..16e3f696ba 100644 --- a/metricflow/test/plan_conversion/test_dataflow_to_sql_plan.py +++ b/metricflow/test/plan_conversion/test_dataflow_to_sql_plan.py @@ -1249,16 +1249,34 @@ def test_distinct_values( # noqa: D mf_test_session_state: MetricFlowTestSessionState, dataflow_plan_builder: DataflowPlanBuilder, dataflow_to_sql_converter: DataflowToSqlQueryPlanConverter, + column_association_resolver: ColumnAssociationResolver, sql_client: SqlClient, ) -> None: """Tests a plan to get distinct values for a dimension.""" dataflow_plan = dataflow_plan_builder.build_plan_for_distinct_values( - metric_specs=(MetricSpec(element_name="bookings"),), - dimension_spec=DimensionSpec( - element_name="country_latest", - entity_links=(EntityReference(element_name="listing"),), - ), - limit=100, + query_spec=MetricFlowQuerySpec( + dimension_specs=( + DimensionSpec(element_name="country_latest", entity_links=(EntityReference(element_name="listing"),)), + ), + where_constraint=( + WhereSpecFactory( + column_association_resolver=column_association_resolver, + ).create_from_where_filter( + PydanticWhereFilter( + where_sql_template="{{ Dimension('listing__country_latest') }} = 'us'", + ) + ) + ), + order_by_specs=( + OrderBySpec( + instance_spec=DimensionSpec( + element_name="country_latest", entity_links=(EntityReference(element_name="listing"),) + ), + descending=True, + ), + ), + limit=100, + ) ) convert_and_check( diff --git a/metricflow/test/query/test_query_parser.py b/metricflow/test/query/test_query_parser.py index 19ece527b4..4c2bc78680 100644 --- a/metricflow/test/query/test_query_parser.py +++ b/metricflow/test/query/test_query_parser.py @@ -249,6 +249,9 @@ def test_query_parser_case_insensitivity(bookings_query_parser: MetricFlowQueryP ), ) + with pytest.raises(UnableToSatisfyQueryError): + bookings_query_parser.parse_and_validate_query(group_by_names=["random_stuff"]) + def test_query_parser_with_object_params(bookings_query_parser: MetricFlowQueryParser) -> None: # noqa: D Metric = namedtuple("Metric", ["name", "descending"]) diff --git a/metricflow/test/time/test_time_granularity_solver.py b/metricflow/test/time/test_time_granularity_solver.py index 025778df10..0fcdf0ce28 100644 --- a/metricflow/test/time/test_time_granularity_solver.py +++ b/metricflow/test/time/test_time_granularity_solver.py @@ -6,9 +6,11 @@ from dbt_semantic_interfaces.references import MetricReference from dbt_semantic_interfaces.type_enums.time_granularity import TimeGranularity +from metricflow.dataflow.builder.node_data_set import DataflowPlanNodeOutputDataSetResolver from metricflow.dataset.dataset import DataSet from metricflow.filters.time_constraint import TimeRangeConstraint from metricflow.model.semantic_manifest_lookup import SemanticManifestLookup +from metricflow.test.fixtures.model_fixtures import ConsistentIdObjectRepository from metricflow.test.time.metric_time_dimension import MTD_SPEC_DAY, MTD_SPEC_MONTH from metricflow.time.time_granularity_solver import ( PartialTimeDimensionSpec, @@ -89,19 +91,31 @@ def test_validate_day_granularity_for_day_and_month_metric( # noqa: D PARTIAL_PTD_SPEC = PartialTimeDimensionSpec(element_name=DataSet.metric_time_dimension_name(), entity_links=()) -def test_granularity_solution_for_day_metric(time_granularity_solver: TimeGranularitySolver) -> None: # noqa: D +def test_granularity_solution_for_day_metric( # noqa: D + time_granularity_solver: TimeGranularitySolver, + node_output_resolver: DataflowPlanNodeOutputDataSetResolver, + consistent_id_object_repository: ConsistentIdObjectRepository, +) -> None: assert time_granularity_solver.resolve_granularity_for_partial_time_dimension_specs( metric_references=[MetricReference(element_name="bookings")], partial_time_dimension_specs=[PARTIAL_PTD_SPEC], + node_output_resolver=node_output_resolver, + read_nodes=list(consistent_id_object_repository.simple_model_read_nodes.values()), ) == { PARTIAL_PTD_SPEC: MTD_SPEC_DAY, } -def test_granularity_solution_for_month_metric(time_granularity_solver: TimeGranularitySolver) -> None: # noqa: D +def test_granularity_solution_for_month_metric( # noqa: D + time_granularity_solver: TimeGranularitySolver, + node_output_resolver: DataflowPlanNodeOutputDataSetResolver, + consistent_id_object_repository: ConsistentIdObjectRepository, +) -> None: assert time_granularity_solver.resolve_granularity_for_partial_time_dimension_specs( metric_references=[MetricReference(element_name="bookings_monthly")], partial_time_dimension_specs=[PARTIAL_PTD_SPEC], + node_output_resolver=node_output_resolver, + read_nodes=list(consistent_id_object_repository.simple_model_read_nodes.values()), ) == { PARTIAL_PTD_SPEC: MTD_SPEC_MONTH, } @@ -109,10 +123,14 @@ def test_granularity_solution_for_month_metric(time_granularity_solver: TimeGran def test_granularity_solution_for_day_and_month_metrics( # noqa: D time_granularity_solver: TimeGranularitySolver, + node_output_resolver: DataflowPlanNodeOutputDataSetResolver, + consistent_id_object_repository: ConsistentIdObjectRepository, ) -> None: assert time_granularity_solver.resolve_granularity_for_partial_time_dimension_specs( metric_references=[MetricReference(element_name="bookings"), MetricReference(element_name="bookings_monthly")], partial_time_dimension_specs=[PARTIAL_PTD_SPEC], + node_output_resolver=node_output_resolver, + read_nodes=list(consistent_id_object_repository.simple_model_read_nodes.values()), ) == {PARTIAL_PTD_SPEC: MTD_SPEC_MONTH} From c7e63cbd9cc1d72d6772154e7bfe9884434929e9 Mon Sep 17 00:00:00 2001 From: Courtney Holcomb Date: Tue, 10 Oct 2023 18:34:14 -0700 Subject: [PATCH 06/15] Update integration test to require entity name --- metricflow/test/integration/test_cases/itest_dimensions.yaml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/metricflow/test/integration/test_cases/itest_dimensions.yaml b/metricflow/test/integration/test_cases/itest_dimensions.yaml index c6d5912629..3fe4405733 100644 --- a/metricflow/test/integration/test_cases/itest_dimensions.yaml +++ b/metricflow/test/integration/test_cases/itest_dimensions.yaml @@ -119,11 +119,11 @@ integration_test: name: query_dimensions_only description: Query multiple dimensions without metrics model: SIMPLE_MODEL - group_bys: ["ds__day", "user__home_state"] + group_bys: ["user__ds__day", "user__home_state"] check_query: | SELECT u.home_state AS user__home_state - , u.ds AS ds__day + , u.ds AS user__ds__day FROM {{ source_schema }}.dim_users u GROUP BY u.ds From 71f57a034c78a21a4485f8192188fc801df11e9c Mon Sep 17 00:00:00 2001 From: Courtney Holcomb Date: Tue, 10 Oct 2023 18:38:02 -0700 Subject: [PATCH 07/15] Delete debug comment --- metricflow/dataflow/builder/dataflow_plan_builder.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/metricflow/dataflow/builder/dataflow_plan_builder.py b/metricflow/dataflow/builder/dataflow_plan_builder.py index 5065420725..b62f54ae0c 100644 --- a/metricflow/dataflow/builder/dataflow_plan_builder.py +++ b/metricflow/dataflow/builder/dataflow_plan_builder.py @@ -503,7 +503,7 @@ def _find_dataflow_recipe( linkable_specs=linkable_spec_set, read_nodes=source_nodes ) potential_source_nodes = list(source_nodes_to_linkable_specs.keys()) - # issue: getting ds__day from the wrong table + logger.info(f"There are {len(potential_source_nodes)} potential source nodes") logger.info(f"Starting search with {len(source_nodes)} source nodes") From 87c566e4c81e85d3d46677ac39fead5c2769e45b Mon Sep 17 00:00:00 2001 From: Courtney Holcomb Date: Tue, 10 Oct 2023 18:38:32 -0700 Subject: [PATCH 08/15] Update snapshots --- .../test_filter_combination__dfpo_0.xml | 1 + .../DataflowPlan/test_cyclic_join__dfp_0.xml | 7 +- .../test_common_semantic_model__dfp_0.xml | 6 + ...indow_or_grain_with_metric_time__dfp_0.xml | 1 + ...ow_or_grain_without_metric_time__dfp_0.xml | 1 + ...t_cumulative_metric_with_window__dfp_0.xml | 1 + ..._derived_metric_offset_to_grain__dfp_0.xml | 2 + ...st_derived_metric_offset_window__dfp_0.xml | 1 + ..._metric_offset_with_granularity__dfp_0.xml | 1 + ...erived_offset_cumulative_metric__dfp_0.xml | 1 + .../test_distinct_values_plan__dfp_0.xml | 131 +- ..._distinct_values_plan_with_join__dfp_0.xml | 87 + .../DataflowPlan/test_joined_plan__dfp_0.xml | 3 + .../test_limit_rows_plan__dfp_0.xml | 1 + .../test_measure_constraint_plan__dfp_0.xml | 9 + ...traint_with_reused_measure_plan__dfp_0.xml | 3 + ...mantic_model_ratio_metrics_plan__dfp_0.xml | 6 + .../test_multihop_join_plan__dfp_0.xml | 4 + .../test_multiple_metrics_plan__dfp_0.xml | 2 + .../test_order_by_plan__dfp_0.xml | 1 + .../test_primary_entity_dimension__dfp_0.xml | 1 + .../DataflowPlan/test_simple_plan__dfp_0.xml | 1 + ...mantic_model_ratio_metrics_plan__dfp_0.xml | 6 + .../test_where_constrained_plan__dfp_0.xml | 4 + ...constrained_plan_time_dimension__dfp_0.xml | 2 + ...ained_with_common_linkable_plan__dfp_0.xml | 3 + ...ompute_metrics_node_simple_expr__plan0.xml | 2 + ...spine_node_with_offset_to_grain__plan0.xml | 1 + ...e_spine_node_with_offset_window__plan0.xml | 1 + ..._time_spine_node_without_offset__plan0.xml | 1 + .../BigQuery/test_distinct_values__plan0.sql | 470 +---- .../test_distinct_values__plan0_optimized.sql | 28 +- .../DuckDB/test_distinct_values__plan0.sql | 470 +---- .../test_distinct_values__plan0_optimized.sql | 30 +- .../Postgres/test_distinct_values__plan0.sql | 470 +---- .../test_distinct_values__plan0_optimized.sql | 30 +- .../Redshift/test_distinct_values__plan0.sql | 470 +---- .../test_distinct_values__plan0_optimized.sql | 30 +- .../Snowflake/test_distinct_values__plan0.sql | 470 +---- .../test_distinct_values__plan0_optimized.sql | 30 +- .../test_distinct_values__plan0.xml | 1754 +++-------------- ...2_metrics_from_1_semantic_model__dfp_0.xml | 6 + ..._metrics_from_1_semantic_model__dfpo_0.xml | 3 + ..._metrics_from_2_semantic_models__dfp_0.xml | 2 + ...metrics_from_2_semantic_models__dfpo_0.xml | 2 + ...o_metrics_from_1_semantic_model__dfp_0.xml | 4 + ..._metrics_from_1_semantic_model__dfpo_0.xml | 1 + ..._metrics_from_2_semantic_models__dfp_0.xml | 3 + ...metrics_from_2_semantic_models__dfpo_0.xml | 2 + ...constrained_metric_not_combined__dfp_0.xml | 3 + ...onstrained_metric_not_combined__dfpo_0.xml | 3 + .../test_derived_metric__dfp_0.xml | 2 + .../test_derived_metric__dfpo_0.xml | 1 + ..._metric_with_non_derived_metric__dfp_0.xml | 3 + ...metric_with_non_derived_metric__dfpo_0.xml | 2 + .../test_nested_derived_metric__dfp_0.xml | 4 + .../test_nested_derived_metric__dfpo_0.xml | 2 + 57 files changed, 958 insertions(+), 3628 deletions(-) create mode 100644 metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_distinct_values_plan_with_join__dfp_0.xml diff --git a/metricflow/test/snapshots/test_cm_branch_combiner.py/DataflowPlan/test_filter_combination__dfpo_0.xml b/metricflow/test/snapshots/test_cm_branch_combiner.py/DataflowPlan/test_filter_combination__dfpo_0.xml index a3e9b689d1..8b44ecbdd0 100644 --- a/metricflow/test/snapshots/test_cm_branch_combiner.py/DataflowPlan/test_filter_combination__dfpo_0.xml +++ b/metricflow/test/snapshots/test_cm_branch_combiner.py/DataflowPlan/test_filter_combination__dfpo_0.xml @@ -15,6 +15,7 @@ + diff --git a/metricflow/test/snapshots/test_cyclic_join.py/DataflowPlan/test_cyclic_join__dfp_0.xml b/metricflow/test/snapshots/test_cyclic_join.py/DataflowPlan/test_cyclic_join__dfp_0.xml index e5b3d39969..2382b41493 100644 --- a/metricflow/test/snapshots/test_cyclic_join.py/DataflowPlan/test_cyclic_join__dfp_0.xml +++ b/metricflow/test/snapshots/test_cyclic_join.py/DataflowPlan/test_cyclic_join__dfp_0.xml @@ -29,6 +29,7 @@ + @@ -54,6 +55,7 @@ + @@ -61,7 +63,7 @@ - + @@ -80,10 +82,11 @@ + - + diff --git a/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_common_semantic_model__dfp_0.xml b/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_common_semantic_model__dfp_0.xml index fea487abac..ff9c8b4707 100644 --- a/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_common_semantic_model__dfp_0.xml +++ b/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_common_semantic_model__dfp_0.xml @@ -40,6 +40,7 @@ + @@ -69,6 +70,7 @@ + @@ -90,6 +92,7 @@ + @@ -140,6 +143,7 @@ + @@ -169,6 +173,7 @@ + @@ -190,6 +195,7 @@ + diff --git a/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_cumulative_metric_no_window_or_grain_with_metric_time__dfp_0.xml b/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_cumulative_metric_no_window_or_grain_with_metric_time__dfp_0.xml index 315dbe3f63..b25be80860 100644 --- a/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_cumulative_metric_no_window_or_grain_with_metric_time__dfp_0.xml +++ b/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_cumulative_metric_no_window_or_grain_with_metric_time__dfp_0.xml @@ -31,6 +31,7 @@ + diff --git a/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_cumulative_metric_no_window_or_grain_without_metric_time__dfp_0.xml b/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_cumulative_metric_no_window_or_grain_without_metric_time__dfp_0.xml index bd50f72a90..3b58bd51b7 100644 --- a/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_cumulative_metric_no_window_or_grain_without_metric_time__dfp_0.xml +++ b/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_cumulative_metric_no_window_or_grain_without_metric_time__dfp_0.xml @@ -24,6 +24,7 @@ + diff --git a/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_cumulative_metric_with_window__dfp_0.xml b/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_cumulative_metric_with_window__dfp_0.xml index 8c87a7ea7c..12efb1df0d 100644 --- a/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_cumulative_metric_with_window__dfp_0.xml +++ b/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_cumulative_metric_with_window__dfp_0.xml @@ -31,6 +31,7 @@ + diff --git a/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_derived_metric_offset_to_grain__dfp_0.xml b/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_derived_metric_offset_to_grain__dfp_0.xml index 0ea5b4678a..7704ab793b 100644 --- a/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_derived_metric_offset_to_grain__dfp_0.xml +++ b/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_derived_metric_offset_to_grain__dfp_0.xml @@ -45,6 +45,7 @@ + @@ -89,6 +90,7 @@ + diff --git a/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_derived_metric_offset_window__dfp_0.xml b/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_derived_metric_offset_window__dfp_0.xml index 97bb2cf97f..bedaa4a5af 100644 --- a/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_derived_metric_offset_window__dfp_0.xml +++ b/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_derived_metric_offset_window__dfp_0.xml @@ -43,6 +43,7 @@ + diff --git a/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_derived_metric_offset_with_granularity__dfp_0.xml b/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_derived_metric_offset_with_granularity__dfp_0.xml index f2f1132769..72074aa37a 100644 --- a/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_derived_metric_offset_with_granularity__dfp_0.xml +++ b/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_derived_metric_offset_with_granularity__dfp_0.xml @@ -43,6 +43,7 @@ + diff --git a/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_derived_offset_cumulative_metric__dfp_0.xml b/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_derived_offset_cumulative_metric__dfp_0.xml index 17be7c1387..1f10a03b3f 100644 --- a/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_derived_offset_cumulative_metric__dfp_0.xml +++ b/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_derived_offset_cumulative_metric__dfp_0.xml @@ -43,6 +43,7 @@ + diff --git a/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_distinct_values_plan__dfp_0.xml b/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_distinct_values_plan__dfp_0.xml index 29f0eaa0ed..793a1a5506 100644 --- a/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_distinct_values_plan__dfp_0.xml +++ b/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_distinct_values_plan__dfp_0.xml @@ -11,104 +11,41 @@ - + - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + diff --git a/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_distinct_values_plan_with_join__dfp_0.xml b/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_distinct_values_plan_with_join__dfp_0.xml new file mode 100644 index 0000000000..620c81421f --- /dev/null +++ b/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_distinct_values_plan_with_join__dfp_0.xml @@ -0,0 +1,87 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + diff --git a/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_joined_plan__dfp_0.xml b/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_joined_plan__dfp_0.xml index 55f0470165..07ef1d04f2 100644 --- a/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_joined_plan__dfp_0.xml +++ b/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_joined_plan__dfp_0.xml @@ -32,6 +32,7 @@ + @@ -58,6 +59,7 @@ + @@ -79,6 +81,7 @@ + diff --git a/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_limit_rows_plan__dfp_0.xml b/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_limit_rows_plan__dfp_0.xml index 8b12c3de7b..b4fd8b7143 100644 --- a/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_limit_rows_plan__dfp_0.xml +++ b/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_limit_rows_plan__dfp_0.xml @@ -35,6 +35,7 @@ + diff --git a/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_measure_constraint_plan__dfp_0.xml b/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_measure_constraint_plan__dfp_0.xml index ba549be3d2..14e3100122 100644 --- a/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_measure_constraint_plan__dfp_0.xml +++ b/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_measure_constraint_plan__dfp_0.xml @@ -55,6 +55,7 @@ + @@ -89,6 +90,7 @@ + @@ -118,6 +120,7 @@ + @@ -138,6 +141,7 @@ + @@ -196,6 +200,7 @@ + @@ -230,6 +235,7 @@ + @@ -259,6 +265,7 @@ + @@ -279,6 +286,7 @@ + @@ -327,6 +335,7 @@ + diff --git a/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_measure_constraint_with_reused_measure_plan__dfp_0.xml b/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_measure_constraint_with_reused_measure_plan__dfp_0.xml index 749bda11de..f25d9f6748 100644 --- a/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_measure_constraint_with_reused_measure_plan__dfp_0.xml +++ b/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_measure_constraint_with_reused_measure_plan__dfp_0.xml @@ -55,6 +55,7 @@ + @@ -89,6 +90,7 @@ + @@ -135,6 +137,7 @@ + diff --git a/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_multi_semantic_model_ratio_metrics_plan__dfp_0.xml b/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_multi_semantic_model_ratio_metrics_plan__dfp_0.xml index dd83dca62a..7de1511904 100644 --- a/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_multi_semantic_model_ratio_metrics_plan__dfp_0.xml +++ b/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_multi_semantic_model_ratio_metrics_plan__dfp_0.xml @@ -49,6 +49,7 @@ + @@ -78,6 +79,7 @@ + @@ -99,6 +101,7 @@ + @@ -149,6 +152,7 @@ + @@ -178,6 +182,7 @@ + @@ -199,6 +204,7 @@ + diff --git a/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_multihop_join_plan__dfp_0.xml b/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_multihop_join_plan__dfp_0.xml index 108556d708..a98cafb501 100644 --- a/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_multihop_join_plan__dfp_0.xml +++ b/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_multihop_join_plan__dfp_0.xml @@ -29,6 +29,7 @@ + @@ -73,6 +74,7 @@ + @@ -106,6 +108,7 @@ + @@ -352,6 +355,7 @@ + diff --git a/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_multiple_metrics_plan__dfp_0.xml b/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_multiple_metrics_plan__dfp_0.xml index b34126863d..7234390739 100644 --- a/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_multiple_metrics_plan__dfp_0.xml +++ b/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_multiple_metrics_plan__dfp_0.xml @@ -40,6 +40,7 @@ + @@ -88,6 +89,7 @@ + diff --git a/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_order_by_plan__dfp_0.xml b/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_order_by_plan__dfp_0.xml index 1298e65b05..86c6ccefde 100644 --- a/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_order_by_plan__dfp_0.xml +++ b/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_order_by_plan__dfp_0.xml @@ -53,6 +53,7 @@ + diff --git a/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_primary_entity_dimension__dfp_0.xml b/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_primary_entity_dimension__dfp_0.xml index 75695a7288..ccb2dd4669 100644 --- a/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_primary_entity_dimension__dfp_0.xml +++ b/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_primary_entity_dimension__dfp_0.xml @@ -28,6 +28,7 @@ + diff --git a/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_simple_plan__dfp_0.xml b/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_simple_plan__dfp_0.xml index 75695a7288..ccb2dd4669 100644 --- a/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_simple_plan__dfp_0.xml +++ b/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_simple_plan__dfp_0.xml @@ -28,6 +28,7 @@ + diff --git a/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_single_semantic_model_ratio_metrics_plan__dfp_0.xml b/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_single_semantic_model_ratio_metrics_plan__dfp_0.xml index 0fc68ed052..22856f8489 100644 --- a/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_single_semantic_model_ratio_metrics_plan__dfp_0.xml +++ b/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_single_semantic_model_ratio_metrics_plan__dfp_0.xml @@ -49,6 +49,7 @@ + @@ -78,6 +79,7 @@ + @@ -99,6 +101,7 @@ + @@ -149,6 +152,7 @@ + @@ -178,6 +182,7 @@ + @@ -199,6 +204,7 @@ + diff --git a/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_where_constrained_plan__dfp_0.xml b/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_where_constrained_plan__dfp_0.xml index 347e702ec1..062689be39 100644 --- a/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_where_constrained_plan__dfp_0.xml +++ b/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_where_constrained_plan__dfp_0.xml @@ -28,6 +28,7 @@ + @@ -59,6 +60,7 @@ + @@ -85,6 +87,7 @@ + @@ -106,6 +109,7 @@ + diff --git a/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_where_constrained_plan_time_dimension__dfp_0.xml b/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_where_constrained_plan_time_dimension__dfp_0.xml index c5493a3239..b8248be3ce 100644 --- a/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_where_constrained_plan_time_dimension__dfp_0.xml +++ b/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_where_constrained_plan_time_dimension__dfp_0.xml @@ -28,6 +28,7 @@ + @@ -64,6 +65,7 @@ + diff --git a/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_where_constrained_with_common_linkable_plan__dfp_0.xml b/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_where_constrained_with_common_linkable_plan__dfp_0.xml index d6b05b3cd9..1ef34a8a04 100644 --- a/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_where_constrained_with_common_linkable_plan__dfp_0.xml +++ b/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_where_constrained_with_common_linkable_plan__dfp_0.xml @@ -42,6 +42,7 @@ + @@ -64,6 +65,7 @@ + @@ -85,6 +87,7 @@ + diff --git a/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/DataflowPlan/test_compute_metrics_node_simple_expr__plan0.xml b/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/DataflowPlan/test_compute_metrics_node_simple_expr__plan0.xml index 1b06389d8f..d1662fcdda 100644 --- a/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/DataflowPlan/test_compute_metrics_node_simple_expr__plan0.xml +++ b/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/DataflowPlan/test_compute_metrics_node_simple_expr__plan0.xml @@ -37,6 +37,7 @@ + @@ -53,6 +54,7 @@ + diff --git a/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/DataflowPlan/test_join_to_time_spine_node_with_offset_to_grain__plan0.xml b/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/DataflowPlan/test_join_to_time_spine_node_with_offset_to_grain__plan0.xml index 9ab80bd341..4a4e2a24f4 100644 --- a/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/DataflowPlan/test_join_to_time_spine_node_with_offset_to_grain__plan0.xml +++ b/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/DataflowPlan/test_join_to_time_spine_node_with_offset_to_grain__plan0.xml @@ -38,6 +38,7 @@ + diff --git a/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/DataflowPlan/test_join_to_time_spine_node_with_offset_window__plan0.xml b/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/DataflowPlan/test_join_to_time_spine_node_with_offset_window__plan0.xml index 4f3968902a..b0c2d678ce 100644 --- a/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/DataflowPlan/test_join_to_time_spine_node_with_offset_window__plan0.xml +++ b/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/DataflowPlan/test_join_to_time_spine_node_with_offset_window__plan0.xml @@ -38,6 +38,7 @@ + diff --git a/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/DataflowPlan/test_join_to_time_spine_node_without_offset__plan0.xml b/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/DataflowPlan/test_join_to_time_spine_node_without_offset__plan0.xml index 5f87539f4f..5a31f12687 100644 --- a/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/DataflowPlan/test_join_to_time_spine_node_without_offset__plan0.xml +++ b/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/DataflowPlan/test_join_to_time_spine_node_without_offset__plan0.xml @@ -38,6 +38,7 @@ + diff --git a/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/BigQuery/test_distinct_values__plan0.sql b/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/BigQuery/test_distinct_values__plan0.sql index 837156f409..aee1a40a63 100644 --- a/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/BigQuery/test_distinct_values__plan0.sql +++ b/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/BigQuery/test_distinct_values__plan0.sql @@ -1,404 +1,84 @@ -- Order By ['listing__country_latest'] Limit 100 SELECT - subq_10.listing__country_latest + subq_2.listing__country_latest FROM ( - -- Pass Only Elements: - -- ['listing__country_latest'] + -- Constrain Output with WHERE SELECT - subq_9.listing__country_latest + subq_1.listing__country_latest FROM ( - -- Compute Metrics via Expressions + -- Pass Only Elements: + -- ['listing__country_latest'] SELECT - subq_8.listing__country_latest - , subq_8.bookings + subq_0.listing__country_latest FROM ( - -- Aggregate Measures + -- Read Elements From Semantic Model 'listings_latest' SELECT - subq_7.listing__country_latest - , SUM(subq_7.bookings) AS bookings - FROM ( - -- Pass Only Elements: - -- ['bookings', 'listing__country_latest'] - SELECT - subq_6.listing__country_latest - , subq_6.bookings - FROM ( - -- Join Standard Outputs - SELECT - subq_2.listing AS listing - , subq_5.country_latest AS listing__country_latest - , subq_2.bookings AS bookings - FROM ( - -- Pass Only Elements: - -- ['bookings', 'listing'] - SELECT - subq_1.listing - , subq_1.bookings - FROM ( - -- Metric Time Dimension 'ds' - SELECT - subq_0.ds__day - , subq_0.ds__week - , subq_0.ds__month - , subq_0.ds__quarter - , subq_0.ds__year - , subq_0.ds__extract_year - , subq_0.ds__extract_quarter - , subq_0.ds__extract_month - , subq_0.ds__extract_week - , subq_0.ds__extract_day - , subq_0.ds__extract_dow - , subq_0.ds__extract_doy - , subq_0.ds_partitioned__day - , subq_0.ds_partitioned__week - , subq_0.ds_partitioned__month - , subq_0.ds_partitioned__quarter - , subq_0.ds_partitioned__year - , subq_0.ds_partitioned__extract_year - , subq_0.ds_partitioned__extract_quarter - , subq_0.ds_partitioned__extract_month - , subq_0.ds_partitioned__extract_week - , subq_0.ds_partitioned__extract_day - , subq_0.ds_partitioned__extract_dow - , subq_0.ds_partitioned__extract_doy - , subq_0.paid_at__day - , subq_0.paid_at__week - , subq_0.paid_at__month - , subq_0.paid_at__quarter - , subq_0.paid_at__year - , subq_0.paid_at__extract_year - , subq_0.paid_at__extract_quarter - , subq_0.paid_at__extract_month - , subq_0.paid_at__extract_week - , subq_0.paid_at__extract_day - , subq_0.paid_at__extract_dow - , subq_0.paid_at__extract_doy - , subq_0.booking__ds__day - , subq_0.booking__ds__week - , subq_0.booking__ds__month - , subq_0.booking__ds__quarter - , subq_0.booking__ds__year - , subq_0.booking__ds__extract_year - , subq_0.booking__ds__extract_quarter - , subq_0.booking__ds__extract_month - , subq_0.booking__ds__extract_week - , subq_0.booking__ds__extract_day - , subq_0.booking__ds__extract_dow - , subq_0.booking__ds__extract_doy - , subq_0.booking__ds_partitioned__day - , subq_0.booking__ds_partitioned__week - , subq_0.booking__ds_partitioned__month - , subq_0.booking__ds_partitioned__quarter - , subq_0.booking__ds_partitioned__year - , subq_0.booking__ds_partitioned__extract_year - , subq_0.booking__ds_partitioned__extract_quarter - , subq_0.booking__ds_partitioned__extract_month - , subq_0.booking__ds_partitioned__extract_week - , subq_0.booking__ds_partitioned__extract_day - , subq_0.booking__ds_partitioned__extract_dow - , subq_0.booking__ds_partitioned__extract_doy - , subq_0.booking__paid_at__day - , subq_0.booking__paid_at__week - , subq_0.booking__paid_at__month - , subq_0.booking__paid_at__quarter - , subq_0.booking__paid_at__year - , subq_0.booking__paid_at__extract_year - , subq_0.booking__paid_at__extract_quarter - , subq_0.booking__paid_at__extract_month - , subq_0.booking__paid_at__extract_week - , subq_0.booking__paid_at__extract_day - , subq_0.booking__paid_at__extract_dow - , subq_0.booking__paid_at__extract_doy - , subq_0.ds__day AS metric_time__day - , subq_0.ds__week AS metric_time__week - , subq_0.ds__month AS metric_time__month - , subq_0.ds__quarter AS metric_time__quarter - , subq_0.ds__year AS metric_time__year - , subq_0.ds__extract_year AS metric_time__extract_year - , subq_0.ds__extract_quarter AS metric_time__extract_quarter - , subq_0.ds__extract_month AS metric_time__extract_month - , subq_0.ds__extract_week AS metric_time__extract_week - , subq_0.ds__extract_day AS metric_time__extract_day - , subq_0.ds__extract_dow AS metric_time__extract_dow - , subq_0.ds__extract_doy AS metric_time__extract_doy - , subq_0.listing - , subq_0.guest - , subq_0.host - , subq_0.booking__listing - , subq_0.booking__guest - , subq_0.booking__host - , subq_0.is_instant - , subq_0.booking__is_instant - , subq_0.bookings - , subq_0.instant_bookings - , subq_0.booking_value - , subq_0.max_booking_value - , subq_0.min_booking_value - , subq_0.bookers - , subq_0.average_booking_value - , subq_0.referred_bookings - , subq_0.median_booking_value - , subq_0.booking_value_p99 - , subq_0.discrete_booking_value_p99 - , subq_0.approximate_continuous_booking_value_p99 - , subq_0.approximate_discrete_booking_value_p99 - FROM ( - -- Read Elements From Semantic Model 'bookings_source' - SELECT - 1 AS bookings - , CASE WHEN is_instant THEN 1 ELSE 0 END AS instant_bookings - , bookings_source_src_10001.booking_value - , bookings_source_src_10001.booking_value AS max_booking_value - , bookings_source_src_10001.booking_value AS min_booking_value - , bookings_source_src_10001.guest_id AS bookers - , bookings_source_src_10001.booking_value AS average_booking_value - , bookings_source_src_10001.booking_value AS booking_payments - , CASE WHEN referrer_id IS NOT NULL THEN 1 ELSE 0 END AS referred_bookings - , bookings_source_src_10001.booking_value AS median_booking_value - , bookings_source_src_10001.booking_value AS booking_value_p99 - , bookings_source_src_10001.booking_value AS discrete_booking_value_p99 - , bookings_source_src_10001.booking_value AS approximate_continuous_booking_value_p99 - , bookings_source_src_10001.booking_value AS approximate_discrete_booking_value_p99 - , bookings_source_src_10001.is_instant - , DATE_TRUNC(bookings_source_src_10001.ds, day) AS ds__day - , DATE_TRUNC(bookings_source_src_10001.ds, isoweek) AS ds__week - , DATE_TRUNC(bookings_source_src_10001.ds, month) AS ds__month - , DATE_TRUNC(bookings_source_src_10001.ds, quarter) AS ds__quarter - , DATE_TRUNC(bookings_source_src_10001.ds, year) AS ds__year - , EXTRACT(year FROM bookings_source_src_10001.ds) AS ds__extract_year - , EXTRACT(quarter FROM bookings_source_src_10001.ds) AS ds__extract_quarter - , EXTRACT(month FROM bookings_source_src_10001.ds) AS ds__extract_month - , EXTRACT(isoweek FROM bookings_source_src_10001.ds) AS ds__extract_week - , EXTRACT(day FROM bookings_source_src_10001.ds) AS ds__extract_day - , EXTRACT(dayofweek FROM bookings_source_src_10001.ds) AS ds__extract_dow - , EXTRACT(dayofyear FROM bookings_source_src_10001.ds) AS ds__extract_doy - , DATE_TRUNC(bookings_source_src_10001.ds_partitioned, day) AS ds_partitioned__day - , DATE_TRUNC(bookings_source_src_10001.ds_partitioned, isoweek) AS ds_partitioned__week - , DATE_TRUNC(bookings_source_src_10001.ds_partitioned, month) AS ds_partitioned__month - , DATE_TRUNC(bookings_source_src_10001.ds_partitioned, quarter) AS ds_partitioned__quarter - , DATE_TRUNC(bookings_source_src_10001.ds_partitioned, year) AS ds_partitioned__year - , EXTRACT(year FROM bookings_source_src_10001.ds_partitioned) AS ds_partitioned__extract_year - , EXTRACT(quarter FROM bookings_source_src_10001.ds_partitioned) AS ds_partitioned__extract_quarter - , EXTRACT(month FROM bookings_source_src_10001.ds_partitioned) AS ds_partitioned__extract_month - , EXTRACT(isoweek FROM bookings_source_src_10001.ds_partitioned) AS ds_partitioned__extract_week - , EXTRACT(day FROM bookings_source_src_10001.ds_partitioned) AS ds_partitioned__extract_day - , EXTRACT(dayofweek FROM bookings_source_src_10001.ds_partitioned) AS ds_partitioned__extract_dow - , EXTRACT(dayofyear FROM bookings_source_src_10001.ds_partitioned) AS ds_partitioned__extract_doy - , DATE_TRUNC(bookings_source_src_10001.paid_at, day) AS paid_at__day - , DATE_TRUNC(bookings_source_src_10001.paid_at, isoweek) AS paid_at__week - , DATE_TRUNC(bookings_source_src_10001.paid_at, month) AS paid_at__month - , DATE_TRUNC(bookings_source_src_10001.paid_at, quarter) AS paid_at__quarter - , DATE_TRUNC(bookings_source_src_10001.paid_at, year) AS paid_at__year - , EXTRACT(year FROM bookings_source_src_10001.paid_at) AS paid_at__extract_year - , EXTRACT(quarter FROM bookings_source_src_10001.paid_at) AS paid_at__extract_quarter - , EXTRACT(month FROM bookings_source_src_10001.paid_at) AS paid_at__extract_month - , EXTRACT(isoweek FROM bookings_source_src_10001.paid_at) AS paid_at__extract_week - , EXTRACT(day FROM bookings_source_src_10001.paid_at) AS paid_at__extract_day - , EXTRACT(dayofweek FROM bookings_source_src_10001.paid_at) AS paid_at__extract_dow - , EXTRACT(dayofyear FROM bookings_source_src_10001.paid_at) AS paid_at__extract_doy - , bookings_source_src_10001.is_instant AS booking__is_instant - , DATE_TRUNC(bookings_source_src_10001.ds, day) AS booking__ds__day - , DATE_TRUNC(bookings_source_src_10001.ds, isoweek) AS booking__ds__week - , DATE_TRUNC(bookings_source_src_10001.ds, month) AS booking__ds__month - , DATE_TRUNC(bookings_source_src_10001.ds, quarter) AS booking__ds__quarter - , DATE_TRUNC(bookings_source_src_10001.ds, year) AS booking__ds__year - , EXTRACT(year FROM bookings_source_src_10001.ds) AS booking__ds__extract_year - , EXTRACT(quarter FROM bookings_source_src_10001.ds) AS booking__ds__extract_quarter - , EXTRACT(month FROM bookings_source_src_10001.ds) AS booking__ds__extract_month - , EXTRACT(isoweek FROM bookings_source_src_10001.ds) AS booking__ds__extract_week - , EXTRACT(day FROM bookings_source_src_10001.ds) AS booking__ds__extract_day - , EXTRACT(dayofweek FROM bookings_source_src_10001.ds) AS booking__ds__extract_dow - , EXTRACT(dayofyear FROM bookings_source_src_10001.ds) AS booking__ds__extract_doy - , DATE_TRUNC(bookings_source_src_10001.ds_partitioned, day) AS booking__ds_partitioned__day - , DATE_TRUNC(bookings_source_src_10001.ds_partitioned, isoweek) AS booking__ds_partitioned__week - , DATE_TRUNC(bookings_source_src_10001.ds_partitioned, month) AS booking__ds_partitioned__month - , DATE_TRUNC(bookings_source_src_10001.ds_partitioned, quarter) AS booking__ds_partitioned__quarter - , DATE_TRUNC(bookings_source_src_10001.ds_partitioned, year) AS booking__ds_partitioned__year - , EXTRACT(year FROM bookings_source_src_10001.ds_partitioned) AS booking__ds_partitioned__extract_year - , EXTRACT(quarter FROM bookings_source_src_10001.ds_partitioned) AS booking__ds_partitioned__extract_quarter - , EXTRACT(month FROM bookings_source_src_10001.ds_partitioned) AS booking__ds_partitioned__extract_month - , EXTRACT(isoweek FROM bookings_source_src_10001.ds_partitioned) AS booking__ds_partitioned__extract_week - , EXTRACT(day FROM bookings_source_src_10001.ds_partitioned) AS booking__ds_partitioned__extract_day - , EXTRACT(dayofweek FROM bookings_source_src_10001.ds_partitioned) AS booking__ds_partitioned__extract_dow - , EXTRACT(dayofyear FROM bookings_source_src_10001.ds_partitioned) AS booking__ds_partitioned__extract_doy - , DATE_TRUNC(bookings_source_src_10001.paid_at, day) AS booking__paid_at__day - , DATE_TRUNC(bookings_source_src_10001.paid_at, isoweek) AS booking__paid_at__week - , DATE_TRUNC(bookings_source_src_10001.paid_at, month) AS booking__paid_at__month - , DATE_TRUNC(bookings_source_src_10001.paid_at, quarter) AS booking__paid_at__quarter - , DATE_TRUNC(bookings_source_src_10001.paid_at, year) AS booking__paid_at__year - , EXTRACT(year FROM bookings_source_src_10001.paid_at) AS booking__paid_at__extract_year - , EXTRACT(quarter FROM bookings_source_src_10001.paid_at) AS booking__paid_at__extract_quarter - , EXTRACT(month FROM bookings_source_src_10001.paid_at) AS booking__paid_at__extract_month - , EXTRACT(isoweek FROM bookings_source_src_10001.paid_at) AS booking__paid_at__extract_week - , EXTRACT(day FROM bookings_source_src_10001.paid_at) AS booking__paid_at__extract_day - , EXTRACT(dayofweek FROM bookings_source_src_10001.paid_at) AS booking__paid_at__extract_dow - , EXTRACT(dayofyear FROM bookings_source_src_10001.paid_at) AS booking__paid_at__extract_doy - , bookings_source_src_10001.listing_id AS listing - , bookings_source_src_10001.guest_id AS guest - , bookings_source_src_10001.host_id AS host - , bookings_source_src_10001.listing_id AS booking__listing - , bookings_source_src_10001.guest_id AS booking__guest - , bookings_source_src_10001.host_id AS booking__host - FROM ***************************.fct_bookings bookings_source_src_10001 - ) subq_0 - ) subq_1 - ) subq_2 - LEFT OUTER JOIN ( - -- Pass Only Elements: - -- ['country_latest', 'listing'] - SELECT - subq_4.listing - , subq_4.country_latest - FROM ( - -- Metric Time Dimension 'ds' - SELECT - subq_3.ds__day - , subq_3.ds__week - , subq_3.ds__month - , subq_3.ds__quarter - , subq_3.ds__year - , subq_3.ds__extract_year - , subq_3.ds__extract_quarter - , subq_3.ds__extract_month - , subq_3.ds__extract_week - , subq_3.ds__extract_day - , subq_3.ds__extract_dow - , subq_3.ds__extract_doy - , subq_3.created_at__day - , subq_3.created_at__week - , subq_3.created_at__month - , subq_3.created_at__quarter - , subq_3.created_at__year - , subq_3.created_at__extract_year - , subq_3.created_at__extract_quarter - , subq_3.created_at__extract_month - , subq_3.created_at__extract_week - , subq_3.created_at__extract_day - , subq_3.created_at__extract_dow - , subq_3.created_at__extract_doy - , subq_3.listing__ds__day - , subq_3.listing__ds__week - , subq_3.listing__ds__month - , subq_3.listing__ds__quarter - , subq_3.listing__ds__year - , subq_3.listing__ds__extract_year - , subq_3.listing__ds__extract_quarter - , subq_3.listing__ds__extract_month - , subq_3.listing__ds__extract_week - , subq_3.listing__ds__extract_day - , subq_3.listing__ds__extract_dow - , subq_3.listing__ds__extract_doy - , subq_3.listing__created_at__day - , subq_3.listing__created_at__week - , subq_3.listing__created_at__month - , subq_3.listing__created_at__quarter - , subq_3.listing__created_at__year - , subq_3.listing__created_at__extract_year - , subq_3.listing__created_at__extract_quarter - , subq_3.listing__created_at__extract_month - , subq_3.listing__created_at__extract_week - , subq_3.listing__created_at__extract_day - , subq_3.listing__created_at__extract_dow - , subq_3.listing__created_at__extract_doy - , subq_3.ds__day AS metric_time__day - , subq_3.ds__week AS metric_time__week - , subq_3.ds__month AS metric_time__month - , subq_3.ds__quarter AS metric_time__quarter - , subq_3.ds__year AS metric_time__year - , subq_3.ds__extract_year AS metric_time__extract_year - , subq_3.ds__extract_quarter AS metric_time__extract_quarter - , subq_3.ds__extract_month AS metric_time__extract_month - , subq_3.ds__extract_week AS metric_time__extract_week - , subq_3.ds__extract_day AS metric_time__extract_day - , subq_3.ds__extract_dow AS metric_time__extract_dow - , subq_3.ds__extract_doy AS metric_time__extract_doy - , subq_3.listing - , subq_3.user - , subq_3.listing__user - , subq_3.country_latest - , subq_3.is_lux_latest - , subq_3.capacity_latest - , subq_3.listing__country_latest - , subq_3.listing__is_lux_latest - , subq_3.listing__capacity_latest - , subq_3.listings - , subq_3.largest_listing - , subq_3.smallest_listing - FROM ( - -- Read Elements From Semantic Model 'listings_latest' - SELECT - 1 AS listings - , listings_latest_src_10004.capacity AS largest_listing - , listings_latest_src_10004.capacity AS smallest_listing - , DATE_TRUNC(listings_latest_src_10004.created_at, day) AS ds__day - , DATE_TRUNC(listings_latest_src_10004.created_at, isoweek) AS ds__week - , DATE_TRUNC(listings_latest_src_10004.created_at, month) AS ds__month - , DATE_TRUNC(listings_latest_src_10004.created_at, quarter) AS ds__quarter - , DATE_TRUNC(listings_latest_src_10004.created_at, year) AS ds__year - , EXTRACT(year FROM listings_latest_src_10004.created_at) AS ds__extract_year - , EXTRACT(quarter FROM listings_latest_src_10004.created_at) AS ds__extract_quarter - , EXTRACT(month FROM listings_latest_src_10004.created_at) AS ds__extract_month - , EXTRACT(isoweek FROM listings_latest_src_10004.created_at) AS ds__extract_week - , EXTRACT(day FROM listings_latest_src_10004.created_at) AS ds__extract_day - , EXTRACT(dayofweek FROM listings_latest_src_10004.created_at) AS ds__extract_dow - , EXTRACT(dayofyear FROM listings_latest_src_10004.created_at) AS ds__extract_doy - , DATE_TRUNC(listings_latest_src_10004.created_at, day) AS created_at__day - , DATE_TRUNC(listings_latest_src_10004.created_at, isoweek) AS created_at__week - , DATE_TRUNC(listings_latest_src_10004.created_at, month) AS created_at__month - , DATE_TRUNC(listings_latest_src_10004.created_at, quarter) AS created_at__quarter - , DATE_TRUNC(listings_latest_src_10004.created_at, year) AS created_at__year - , EXTRACT(year FROM listings_latest_src_10004.created_at) AS created_at__extract_year - , EXTRACT(quarter FROM listings_latest_src_10004.created_at) AS created_at__extract_quarter - , EXTRACT(month FROM listings_latest_src_10004.created_at) AS created_at__extract_month - , EXTRACT(isoweek FROM listings_latest_src_10004.created_at) AS created_at__extract_week - , EXTRACT(day FROM listings_latest_src_10004.created_at) AS created_at__extract_day - , EXTRACT(dayofweek FROM listings_latest_src_10004.created_at) AS created_at__extract_dow - , EXTRACT(dayofyear FROM listings_latest_src_10004.created_at) AS created_at__extract_doy - , listings_latest_src_10004.country AS country_latest - , listings_latest_src_10004.is_lux AS is_lux_latest - , listings_latest_src_10004.capacity AS capacity_latest - , DATE_TRUNC(listings_latest_src_10004.created_at, day) AS listing__ds__day - , DATE_TRUNC(listings_latest_src_10004.created_at, isoweek) AS listing__ds__week - , DATE_TRUNC(listings_latest_src_10004.created_at, month) AS listing__ds__month - , DATE_TRUNC(listings_latest_src_10004.created_at, quarter) AS listing__ds__quarter - , DATE_TRUNC(listings_latest_src_10004.created_at, year) AS listing__ds__year - , EXTRACT(year FROM listings_latest_src_10004.created_at) AS listing__ds__extract_year - , EXTRACT(quarter FROM listings_latest_src_10004.created_at) AS listing__ds__extract_quarter - , EXTRACT(month FROM listings_latest_src_10004.created_at) AS listing__ds__extract_month - , EXTRACT(isoweek FROM listings_latest_src_10004.created_at) AS listing__ds__extract_week - , EXTRACT(day FROM listings_latest_src_10004.created_at) AS listing__ds__extract_day - , EXTRACT(dayofweek FROM listings_latest_src_10004.created_at) AS listing__ds__extract_dow - , EXTRACT(dayofyear FROM listings_latest_src_10004.created_at) AS listing__ds__extract_doy - , DATE_TRUNC(listings_latest_src_10004.created_at, day) AS listing__created_at__day - , DATE_TRUNC(listings_latest_src_10004.created_at, isoweek) AS listing__created_at__week - , DATE_TRUNC(listings_latest_src_10004.created_at, month) AS listing__created_at__month - , DATE_TRUNC(listings_latest_src_10004.created_at, quarter) AS listing__created_at__quarter - , DATE_TRUNC(listings_latest_src_10004.created_at, year) AS listing__created_at__year - , EXTRACT(year FROM listings_latest_src_10004.created_at) AS listing__created_at__extract_year - , EXTRACT(quarter FROM listings_latest_src_10004.created_at) AS listing__created_at__extract_quarter - , EXTRACT(month FROM listings_latest_src_10004.created_at) AS listing__created_at__extract_month - , EXTRACT(isoweek FROM listings_latest_src_10004.created_at) AS listing__created_at__extract_week - , EXTRACT(day FROM listings_latest_src_10004.created_at) AS listing__created_at__extract_day - , EXTRACT(dayofweek FROM listings_latest_src_10004.created_at) AS listing__created_at__extract_dow - , EXTRACT(dayofyear FROM listings_latest_src_10004.created_at) AS listing__created_at__extract_doy - , listings_latest_src_10004.country AS listing__country_latest - , listings_latest_src_10004.is_lux AS listing__is_lux_latest - , listings_latest_src_10004.capacity AS listing__capacity_latest - , listings_latest_src_10004.listing_id AS listing - , listings_latest_src_10004.user_id AS user - , listings_latest_src_10004.user_id AS listing__user - FROM ***************************.dim_listings_latest listings_latest_src_10004 - ) subq_3 - ) subq_4 - ) subq_5 - ON - subq_2.listing = subq_5.listing - ) subq_6 - ) subq_7 - GROUP BY - listing__country_latest - ) subq_8 - ) subq_9 -) subq_10 -ORDER BY subq_10.listing__country_latest + 1 AS listings + , listings_latest_src_10004.capacity AS largest_listing + , listings_latest_src_10004.capacity AS smallest_listing + , DATE_TRUNC(listings_latest_src_10004.created_at, day) AS ds__day + , DATE_TRUNC(listings_latest_src_10004.created_at, isoweek) AS ds__week + , DATE_TRUNC(listings_latest_src_10004.created_at, month) AS ds__month + , DATE_TRUNC(listings_latest_src_10004.created_at, quarter) AS ds__quarter + , DATE_TRUNC(listings_latest_src_10004.created_at, year) AS ds__year + , EXTRACT(year FROM listings_latest_src_10004.created_at) AS ds__extract_year + , EXTRACT(quarter FROM listings_latest_src_10004.created_at) AS ds__extract_quarter + , EXTRACT(month FROM listings_latest_src_10004.created_at) AS ds__extract_month + , EXTRACT(isoweek FROM listings_latest_src_10004.created_at) AS ds__extract_week + , EXTRACT(day FROM listings_latest_src_10004.created_at) AS ds__extract_day + , EXTRACT(dayofweek FROM listings_latest_src_10004.created_at) AS ds__extract_dow + , EXTRACT(dayofyear FROM listings_latest_src_10004.created_at) AS ds__extract_doy + , DATE_TRUNC(listings_latest_src_10004.created_at, day) AS created_at__day + , DATE_TRUNC(listings_latest_src_10004.created_at, isoweek) AS created_at__week + , DATE_TRUNC(listings_latest_src_10004.created_at, month) AS created_at__month + , DATE_TRUNC(listings_latest_src_10004.created_at, quarter) AS created_at__quarter + , DATE_TRUNC(listings_latest_src_10004.created_at, year) AS created_at__year + , EXTRACT(year FROM listings_latest_src_10004.created_at) AS created_at__extract_year + , EXTRACT(quarter FROM listings_latest_src_10004.created_at) AS created_at__extract_quarter + , EXTRACT(month FROM listings_latest_src_10004.created_at) AS created_at__extract_month + , EXTRACT(isoweek FROM listings_latest_src_10004.created_at) AS created_at__extract_week + , EXTRACT(day FROM listings_latest_src_10004.created_at) AS created_at__extract_day + , EXTRACT(dayofweek FROM listings_latest_src_10004.created_at) AS created_at__extract_dow + , EXTRACT(dayofyear FROM listings_latest_src_10004.created_at) AS created_at__extract_doy + , listings_latest_src_10004.country AS country_latest + , listings_latest_src_10004.is_lux AS is_lux_latest + , listings_latest_src_10004.capacity AS capacity_latest + , DATE_TRUNC(listings_latest_src_10004.created_at, day) AS listing__ds__day + , DATE_TRUNC(listings_latest_src_10004.created_at, isoweek) AS listing__ds__week + , DATE_TRUNC(listings_latest_src_10004.created_at, month) AS listing__ds__month + , DATE_TRUNC(listings_latest_src_10004.created_at, quarter) AS listing__ds__quarter + , DATE_TRUNC(listings_latest_src_10004.created_at, year) AS listing__ds__year + , EXTRACT(year FROM listings_latest_src_10004.created_at) AS listing__ds__extract_year + , EXTRACT(quarter FROM listings_latest_src_10004.created_at) AS listing__ds__extract_quarter + , EXTRACT(month FROM listings_latest_src_10004.created_at) AS listing__ds__extract_month + , EXTRACT(isoweek FROM listings_latest_src_10004.created_at) AS listing__ds__extract_week + , EXTRACT(day FROM listings_latest_src_10004.created_at) AS listing__ds__extract_day + , EXTRACT(dayofweek FROM listings_latest_src_10004.created_at) AS listing__ds__extract_dow + , EXTRACT(dayofyear FROM listings_latest_src_10004.created_at) AS listing__ds__extract_doy + , DATE_TRUNC(listings_latest_src_10004.created_at, day) AS listing__created_at__day + , DATE_TRUNC(listings_latest_src_10004.created_at, isoweek) AS listing__created_at__week + , DATE_TRUNC(listings_latest_src_10004.created_at, month) AS listing__created_at__month + , DATE_TRUNC(listings_latest_src_10004.created_at, quarter) AS listing__created_at__quarter + , DATE_TRUNC(listings_latest_src_10004.created_at, year) AS listing__created_at__year + , EXTRACT(year FROM listings_latest_src_10004.created_at) AS listing__created_at__extract_year + , EXTRACT(quarter FROM listings_latest_src_10004.created_at) AS listing__created_at__extract_quarter + , EXTRACT(month FROM listings_latest_src_10004.created_at) AS listing__created_at__extract_month + , EXTRACT(isoweek FROM listings_latest_src_10004.created_at) AS listing__created_at__extract_week + , EXTRACT(day FROM listings_latest_src_10004.created_at) AS listing__created_at__extract_day + , EXTRACT(dayofweek FROM listings_latest_src_10004.created_at) AS listing__created_at__extract_dow + , EXTRACT(dayofyear FROM listings_latest_src_10004.created_at) AS listing__created_at__extract_doy + , listings_latest_src_10004.country AS listing__country_latest + , listings_latest_src_10004.is_lux AS listing__is_lux_latest + , listings_latest_src_10004.capacity AS listing__capacity_latest + , listings_latest_src_10004.listing_id AS listing + , listings_latest_src_10004.user_id AS user + , listings_latest_src_10004.user_id AS listing__user + FROM ***************************.dim_listings_latest listings_latest_src_10004 + ) subq_0 + GROUP BY + listing__country_latest + ) subq_1 + WHERE listing__country_latest = 'us' +) subq_2 +ORDER BY subq_2.listing__country_latest DESC LIMIT 100 diff --git a/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/BigQuery/test_distinct_values__plan0_optimized.sql b/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/BigQuery/test_distinct_values__plan0_optimized.sql index 6fafa37146..53419bb1c0 100644 --- a/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/BigQuery/test_distinct_values__plan0_optimized.sql +++ b/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/BigQuery/test_distinct_values__plan0_optimized.sql @@ -1,19 +1,17 @@ --- Join Standard Outputs --- Pass Only Elements: --- ['bookings', 'listing__country_latest'] --- Aggregate Measures --- Compute Metrics via Expressions --- Pass Only Elements: --- ['listing__country_latest'] +-- Constrain Output with WHERE -- Order By ['listing__country_latest'] Limit 100 SELECT - listings_latest_src_10004.country AS listing__country_latest -FROM ***************************.fct_bookings bookings_source_src_10001 -LEFT OUTER JOIN - ***************************.dim_listings_latest listings_latest_src_10004 -ON - bookings_source_src_10001.listing_id = listings_latest_src_10004.listing_id -GROUP BY listing__country_latest -ORDER BY listing__country_latest +FROM ( + -- Read Elements From Semantic Model 'listings_latest' + -- Pass Only Elements: + -- ['listing__country_latest'] + SELECT + country AS listing__country_latest + FROM ***************************.dim_listings_latest listings_latest_src_10004 + GROUP BY + listing__country_latest +) subq_4 +WHERE listing__country_latest = 'us' +ORDER BY listing__country_latest DESC LIMIT 100 diff --git a/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/DuckDB/test_distinct_values__plan0.sql b/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/DuckDB/test_distinct_values__plan0.sql index 86ba05d2c1..7818858e77 100644 --- a/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/DuckDB/test_distinct_values__plan0.sql +++ b/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/DuckDB/test_distinct_values__plan0.sql @@ -1,404 +1,84 @@ -- Order By ['listing__country_latest'] Limit 100 SELECT - subq_10.listing__country_latest + subq_2.listing__country_latest FROM ( - -- Pass Only Elements: - -- ['listing__country_latest'] + -- Constrain Output with WHERE SELECT - subq_9.listing__country_latest + subq_1.listing__country_latest FROM ( - -- Compute Metrics via Expressions + -- Pass Only Elements: + -- ['listing__country_latest'] SELECT - subq_8.listing__country_latest - , subq_8.bookings + subq_0.listing__country_latest FROM ( - -- Aggregate Measures + -- Read Elements From Semantic Model 'listings_latest' SELECT - subq_7.listing__country_latest - , SUM(subq_7.bookings) AS bookings - FROM ( - -- Pass Only Elements: - -- ['bookings', 'listing__country_latest'] - SELECT - subq_6.listing__country_latest - , subq_6.bookings - FROM ( - -- Join Standard Outputs - SELECT - subq_2.listing AS listing - , subq_5.country_latest AS listing__country_latest - , subq_2.bookings AS bookings - FROM ( - -- Pass Only Elements: - -- ['bookings', 'listing'] - SELECT - subq_1.listing - , subq_1.bookings - FROM ( - -- Metric Time Dimension 'ds' - SELECT - subq_0.ds__day - , subq_0.ds__week - , subq_0.ds__month - , subq_0.ds__quarter - , subq_0.ds__year - , subq_0.ds__extract_year - , subq_0.ds__extract_quarter - , subq_0.ds__extract_month - , subq_0.ds__extract_week - , subq_0.ds__extract_day - , subq_0.ds__extract_dow - , subq_0.ds__extract_doy - , subq_0.ds_partitioned__day - , subq_0.ds_partitioned__week - , subq_0.ds_partitioned__month - , subq_0.ds_partitioned__quarter - , subq_0.ds_partitioned__year - , subq_0.ds_partitioned__extract_year - , subq_0.ds_partitioned__extract_quarter - , subq_0.ds_partitioned__extract_month - , subq_0.ds_partitioned__extract_week - , subq_0.ds_partitioned__extract_day - , subq_0.ds_partitioned__extract_dow - , subq_0.ds_partitioned__extract_doy - , subq_0.paid_at__day - , subq_0.paid_at__week - , subq_0.paid_at__month - , subq_0.paid_at__quarter - , subq_0.paid_at__year - , subq_0.paid_at__extract_year - , subq_0.paid_at__extract_quarter - , subq_0.paid_at__extract_month - , subq_0.paid_at__extract_week - , subq_0.paid_at__extract_day - , subq_0.paid_at__extract_dow - , subq_0.paid_at__extract_doy - , subq_0.booking__ds__day - , subq_0.booking__ds__week - , subq_0.booking__ds__month - , subq_0.booking__ds__quarter - , subq_0.booking__ds__year - , subq_0.booking__ds__extract_year - , subq_0.booking__ds__extract_quarter - , subq_0.booking__ds__extract_month - , subq_0.booking__ds__extract_week - , subq_0.booking__ds__extract_day - , subq_0.booking__ds__extract_dow - , subq_0.booking__ds__extract_doy - , subq_0.booking__ds_partitioned__day - , subq_0.booking__ds_partitioned__week - , subq_0.booking__ds_partitioned__month - , subq_0.booking__ds_partitioned__quarter - , subq_0.booking__ds_partitioned__year - , subq_0.booking__ds_partitioned__extract_year - , subq_0.booking__ds_partitioned__extract_quarter - , subq_0.booking__ds_partitioned__extract_month - , subq_0.booking__ds_partitioned__extract_week - , subq_0.booking__ds_partitioned__extract_day - , subq_0.booking__ds_partitioned__extract_dow - , subq_0.booking__ds_partitioned__extract_doy - , subq_0.booking__paid_at__day - , subq_0.booking__paid_at__week - , subq_0.booking__paid_at__month - , subq_0.booking__paid_at__quarter - , subq_0.booking__paid_at__year - , subq_0.booking__paid_at__extract_year - , subq_0.booking__paid_at__extract_quarter - , subq_0.booking__paid_at__extract_month - , subq_0.booking__paid_at__extract_week - , subq_0.booking__paid_at__extract_day - , subq_0.booking__paid_at__extract_dow - , subq_0.booking__paid_at__extract_doy - , subq_0.ds__day AS metric_time__day - , subq_0.ds__week AS metric_time__week - , subq_0.ds__month AS metric_time__month - , subq_0.ds__quarter AS metric_time__quarter - , subq_0.ds__year AS metric_time__year - , subq_0.ds__extract_year AS metric_time__extract_year - , subq_0.ds__extract_quarter AS metric_time__extract_quarter - , subq_0.ds__extract_month AS metric_time__extract_month - , subq_0.ds__extract_week AS metric_time__extract_week - , subq_0.ds__extract_day AS metric_time__extract_day - , subq_0.ds__extract_dow AS metric_time__extract_dow - , subq_0.ds__extract_doy AS metric_time__extract_doy - , subq_0.listing - , subq_0.guest - , subq_0.host - , subq_0.booking__listing - , subq_0.booking__guest - , subq_0.booking__host - , subq_0.is_instant - , subq_0.booking__is_instant - , subq_0.bookings - , subq_0.instant_bookings - , subq_0.booking_value - , subq_0.max_booking_value - , subq_0.min_booking_value - , subq_0.bookers - , subq_0.average_booking_value - , subq_0.referred_bookings - , subq_0.median_booking_value - , subq_0.booking_value_p99 - , subq_0.discrete_booking_value_p99 - , subq_0.approximate_continuous_booking_value_p99 - , subq_0.approximate_discrete_booking_value_p99 - FROM ( - -- Read Elements From Semantic Model 'bookings_source' - SELECT - 1 AS bookings - , CASE WHEN is_instant THEN 1 ELSE 0 END AS instant_bookings - , bookings_source_src_10001.booking_value - , bookings_source_src_10001.booking_value AS max_booking_value - , bookings_source_src_10001.booking_value AS min_booking_value - , bookings_source_src_10001.guest_id AS bookers - , bookings_source_src_10001.booking_value AS average_booking_value - , bookings_source_src_10001.booking_value AS booking_payments - , CASE WHEN referrer_id IS NOT NULL THEN 1 ELSE 0 END AS referred_bookings - , bookings_source_src_10001.booking_value AS median_booking_value - , bookings_source_src_10001.booking_value AS booking_value_p99 - , bookings_source_src_10001.booking_value AS discrete_booking_value_p99 - , bookings_source_src_10001.booking_value AS approximate_continuous_booking_value_p99 - , bookings_source_src_10001.booking_value AS approximate_discrete_booking_value_p99 - , bookings_source_src_10001.is_instant - , DATE_TRUNC('day', bookings_source_src_10001.ds) AS ds__day - , DATE_TRUNC('week', bookings_source_src_10001.ds) AS ds__week - , DATE_TRUNC('month', bookings_source_src_10001.ds) AS ds__month - , DATE_TRUNC('quarter', bookings_source_src_10001.ds) AS ds__quarter - , DATE_TRUNC('year', bookings_source_src_10001.ds) AS ds__year - , EXTRACT(year FROM bookings_source_src_10001.ds) AS ds__extract_year - , EXTRACT(quarter FROM bookings_source_src_10001.ds) AS ds__extract_quarter - , EXTRACT(month FROM bookings_source_src_10001.ds) AS ds__extract_month - , EXTRACT(week FROM bookings_source_src_10001.ds) AS ds__extract_week - , EXTRACT(day FROM bookings_source_src_10001.ds) AS ds__extract_day - , EXTRACT(dow FROM bookings_source_src_10001.ds) AS ds__extract_dow - , EXTRACT(doy FROM bookings_source_src_10001.ds) AS ds__extract_doy - , DATE_TRUNC('day', bookings_source_src_10001.ds_partitioned) AS ds_partitioned__day - , DATE_TRUNC('week', bookings_source_src_10001.ds_partitioned) AS ds_partitioned__week - , DATE_TRUNC('month', bookings_source_src_10001.ds_partitioned) AS ds_partitioned__month - , DATE_TRUNC('quarter', bookings_source_src_10001.ds_partitioned) AS ds_partitioned__quarter - , DATE_TRUNC('year', bookings_source_src_10001.ds_partitioned) AS ds_partitioned__year - , EXTRACT(year FROM bookings_source_src_10001.ds_partitioned) AS ds_partitioned__extract_year - , EXTRACT(quarter FROM bookings_source_src_10001.ds_partitioned) AS ds_partitioned__extract_quarter - , EXTRACT(month FROM bookings_source_src_10001.ds_partitioned) AS ds_partitioned__extract_month - , EXTRACT(week FROM bookings_source_src_10001.ds_partitioned) AS ds_partitioned__extract_week - , EXTRACT(day FROM bookings_source_src_10001.ds_partitioned) AS ds_partitioned__extract_day - , EXTRACT(dow FROM bookings_source_src_10001.ds_partitioned) AS ds_partitioned__extract_dow - , EXTRACT(doy FROM bookings_source_src_10001.ds_partitioned) AS ds_partitioned__extract_doy - , DATE_TRUNC('day', bookings_source_src_10001.paid_at) AS paid_at__day - , DATE_TRUNC('week', bookings_source_src_10001.paid_at) AS paid_at__week - , DATE_TRUNC('month', bookings_source_src_10001.paid_at) AS paid_at__month - , DATE_TRUNC('quarter', bookings_source_src_10001.paid_at) AS paid_at__quarter - , DATE_TRUNC('year', bookings_source_src_10001.paid_at) AS paid_at__year - , EXTRACT(year FROM bookings_source_src_10001.paid_at) AS paid_at__extract_year - , EXTRACT(quarter FROM bookings_source_src_10001.paid_at) AS paid_at__extract_quarter - , EXTRACT(month FROM bookings_source_src_10001.paid_at) AS paid_at__extract_month - , EXTRACT(week FROM bookings_source_src_10001.paid_at) AS paid_at__extract_week - , EXTRACT(day FROM bookings_source_src_10001.paid_at) AS paid_at__extract_day - , EXTRACT(dow FROM bookings_source_src_10001.paid_at) AS paid_at__extract_dow - , EXTRACT(doy FROM bookings_source_src_10001.paid_at) AS paid_at__extract_doy - , bookings_source_src_10001.is_instant AS booking__is_instant - , DATE_TRUNC('day', bookings_source_src_10001.ds) AS booking__ds__day - , DATE_TRUNC('week', bookings_source_src_10001.ds) AS booking__ds__week - , DATE_TRUNC('month', bookings_source_src_10001.ds) AS booking__ds__month - , DATE_TRUNC('quarter', bookings_source_src_10001.ds) AS booking__ds__quarter - , DATE_TRUNC('year', bookings_source_src_10001.ds) AS booking__ds__year - , EXTRACT(year FROM bookings_source_src_10001.ds) AS booking__ds__extract_year - , EXTRACT(quarter FROM bookings_source_src_10001.ds) AS booking__ds__extract_quarter - , EXTRACT(month FROM bookings_source_src_10001.ds) AS booking__ds__extract_month - , EXTRACT(week FROM bookings_source_src_10001.ds) AS booking__ds__extract_week - , EXTRACT(day FROM bookings_source_src_10001.ds) AS booking__ds__extract_day - , EXTRACT(dow FROM bookings_source_src_10001.ds) AS booking__ds__extract_dow - , EXTRACT(doy FROM bookings_source_src_10001.ds) AS booking__ds__extract_doy - , DATE_TRUNC('day', bookings_source_src_10001.ds_partitioned) AS booking__ds_partitioned__day - , DATE_TRUNC('week', bookings_source_src_10001.ds_partitioned) AS booking__ds_partitioned__week - , DATE_TRUNC('month', bookings_source_src_10001.ds_partitioned) AS booking__ds_partitioned__month - , DATE_TRUNC('quarter', bookings_source_src_10001.ds_partitioned) AS booking__ds_partitioned__quarter - , DATE_TRUNC('year', bookings_source_src_10001.ds_partitioned) AS booking__ds_partitioned__year - , EXTRACT(year FROM bookings_source_src_10001.ds_partitioned) AS booking__ds_partitioned__extract_year - , EXTRACT(quarter FROM bookings_source_src_10001.ds_partitioned) AS booking__ds_partitioned__extract_quarter - , EXTRACT(month FROM bookings_source_src_10001.ds_partitioned) AS booking__ds_partitioned__extract_month - , EXTRACT(week FROM bookings_source_src_10001.ds_partitioned) AS booking__ds_partitioned__extract_week - , EXTRACT(day FROM bookings_source_src_10001.ds_partitioned) AS booking__ds_partitioned__extract_day - , EXTRACT(dow FROM bookings_source_src_10001.ds_partitioned) AS booking__ds_partitioned__extract_dow - , EXTRACT(doy FROM bookings_source_src_10001.ds_partitioned) AS booking__ds_partitioned__extract_doy - , DATE_TRUNC('day', bookings_source_src_10001.paid_at) AS booking__paid_at__day - , DATE_TRUNC('week', bookings_source_src_10001.paid_at) AS booking__paid_at__week - , DATE_TRUNC('month', bookings_source_src_10001.paid_at) AS booking__paid_at__month - , DATE_TRUNC('quarter', bookings_source_src_10001.paid_at) AS booking__paid_at__quarter - , DATE_TRUNC('year', bookings_source_src_10001.paid_at) AS booking__paid_at__year - , EXTRACT(year FROM bookings_source_src_10001.paid_at) AS booking__paid_at__extract_year - , EXTRACT(quarter FROM bookings_source_src_10001.paid_at) AS booking__paid_at__extract_quarter - , EXTRACT(month FROM bookings_source_src_10001.paid_at) AS booking__paid_at__extract_month - , EXTRACT(week FROM bookings_source_src_10001.paid_at) AS booking__paid_at__extract_week - , EXTRACT(day FROM bookings_source_src_10001.paid_at) AS booking__paid_at__extract_day - , EXTRACT(dow FROM bookings_source_src_10001.paid_at) AS booking__paid_at__extract_dow - , EXTRACT(doy FROM bookings_source_src_10001.paid_at) AS booking__paid_at__extract_doy - , bookings_source_src_10001.listing_id AS listing - , bookings_source_src_10001.guest_id AS guest - , bookings_source_src_10001.host_id AS host - , bookings_source_src_10001.listing_id AS booking__listing - , bookings_source_src_10001.guest_id AS booking__guest - , bookings_source_src_10001.host_id AS booking__host - FROM ***************************.fct_bookings bookings_source_src_10001 - ) subq_0 - ) subq_1 - ) subq_2 - LEFT OUTER JOIN ( - -- Pass Only Elements: - -- ['country_latest', 'listing'] - SELECT - subq_4.listing - , subq_4.country_latest - FROM ( - -- Metric Time Dimension 'ds' - SELECT - subq_3.ds__day - , subq_3.ds__week - , subq_3.ds__month - , subq_3.ds__quarter - , subq_3.ds__year - , subq_3.ds__extract_year - , subq_3.ds__extract_quarter - , subq_3.ds__extract_month - , subq_3.ds__extract_week - , subq_3.ds__extract_day - , subq_3.ds__extract_dow - , subq_3.ds__extract_doy - , subq_3.created_at__day - , subq_3.created_at__week - , subq_3.created_at__month - , subq_3.created_at__quarter - , subq_3.created_at__year - , subq_3.created_at__extract_year - , subq_3.created_at__extract_quarter - , subq_3.created_at__extract_month - , subq_3.created_at__extract_week - , subq_3.created_at__extract_day - , subq_3.created_at__extract_dow - , subq_3.created_at__extract_doy - , subq_3.listing__ds__day - , subq_3.listing__ds__week - , subq_3.listing__ds__month - , subq_3.listing__ds__quarter - , subq_3.listing__ds__year - , subq_3.listing__ds__extract_year - , subq_3.listing__ds__extract_quarter - , subq_3.listing__ds__extract_month - , subq_3.listing__ds__extract_week - , subq_3.listing__ds__extract_day - , subq_3.listing__ds__extract_dow - , subq_3.listing__ds__extract_doy - , subq_3.listing__created_at__day - , subq_3.listing__created_at__week - , subq_3.listing__created_at__month - , subq_3.listing__created_at__quarter - , subq_3.listing__created_at__year - , subq_3.listing__created_at__extract_year - , subq_3.listing__created_at__extract_quarter - , subq_3.listing__created_at__extract_month - , subq_3.listing__created_at__extract_week - , subq_3.listing__created_at__extract_day - , subq_3.listing__created_at__extract_dow - , subq_3.listing__created_at__extract_doy - , subq_3.ds__day AS metric_time__day - , subq_3.ds__week AS metric_time__week - , subq_3.ds__month AS metric_time__month - , subq_3.ds__quarter AS metric_time__quarter - , subq_3.ds__year AS metric_time__year - , subq_3.ds__extract_year AS metric_time__extract_year - , subq_3.ds__extract_quarter AS metric_time__extract_quarter - , subq_3.ds__extract_month AS metric_time__extract_month - , subq_3.ds__extract_week AS metric_time__extract_week - , subq_3.ds__extract_day AS metric_time__extract_day - , subq_3.ds__extract_dow AS metric_time__extract_dow - , subq_3.ds__extract_doy AS metric_time__extract_doy - , subq_3.listing - , subq_3.user - , subq_3.listing__user - , subq_3.country_latest - , subq_3.is_lux_latest - , subq_3.capacity_latest - , subq_3.listing__country_latest - , subq_3.listing__is_lux_latest - , subq_3.listing__capacity_latest - , subq_3.listings - , subq_3.largest_listing - , subq_3.smallest_listing - FROM ( - -- Read Elements From Semantic Model 'listings_latest' - SELECT - 1 AS listings - , listings_latest_src_10004.capacity AS largest_listing - , listings_latest_src_10004.capacity AS smallest_listing - , DATE_TRUNC('day', listings_latest_src_10004.created_at) AS ds__day - , DATE_TRUNC('week', listings_latest_src_10004.created_at) AS ds__week - , DATE_TRUNC('month', listings_latest_src_10004.created_at) AS ds__month - , DATE_TRUNC('quarter', listings_latest_src_10004.created_at) AS ds__quarter - , DATE_TRUNC('year', listings_latest_src_10004.created_at) AS ds__year - , EXTRACT(year FROM listings_latest_src_10004.created_at) AS ds__extract_year - , EXTRACT(quarter FROM listings_latest_src_10004.created_at) AS ds__extract_quarter - , EXTRACT(month FROM listings_latest_src_10004.created_at) AS ds__extract_month - , EXTRACT(week FROM listings_latest_src_10004.created_at) AS ds__extract_week - , EXTRACT(day FROM listings_latest_src_10004.created_at) AS ds__extract_day - , EXTRACT(dow FROM listings_latest_src_10004.created_at) AS ds__extract_dow - , EXTRACT(doy FROM listings_latest_src_10004.created_at) AS ds__extract_doy - , DATE_TRUNC('day', listings_latest_src_10004.created_at) AS created_at__day - , DATE_TRUNC('week', listings_latest_src_10004.created_at) AS created_at__week - , DATE_TRUNC('month', listings_latest_src_10004.created_at) AS created_at__month - , DATE_TRUNC('quarter', listings_latest_src_10004.created_at) AS created_at__quarter - , DATE_TRUNC('year', listings_latest_src_10004.created_at) AS created_at__year - , EXTRACT(year FROM listings_latest_src_10004.created_at) AS created_at__extract_year - , EXTRACT(quarter FROM listings_latest_src_10004.created_at) AS created_at__extract_quarter - , EXTRACT(month FROM listings_latest_src_10004.created_at) AS created_at__extract_month - , EXTRACT(week FROM listings_latest_src_10004.created_at) AS created_at__extract_week - , EXTRACT(day FROM listings_latest_src_10004.created_at) AS created_at__extract_day - , EXTRACT(dow FROM listings_latest_src_10004.created_at) AS created_at__extract_dow - , EXTRACT(doy FROM listings_latest_src_10004.created_at) AS created_at__extract_doy - , listings_latest_src_10004.country AS country_latest - , listings_latest_src_10004.is_lux AS is_lux_latest - , listings_latest_src_10004.capacity AS capacity_latest - , DATE_TRUNC('day', listings_latest_src_10004.created_at) AS listing__ds__day - , DATE_TRUNC('week', listings_latest_src_10004.created_at) AS listing__ds__week - , DATE_TRUNC('month', listings_latest_src_10004.created_at) AS listing__ds__month - , DATE_TRUNC('quarter', listings_latest_src_10004.created_at) AS listing__ds__quarter - , DATE_TRUNC('year', listings_latest_src_10004.created_at) AS listing__ds__year - , EXTRACT(year FROM listings_latest_src_10004.created_at) AS listing__ds__extract_year - , EXTRACT(quarter FROM listings_latest_src_10004.created_at) AS listing__ds__extract_quarter - , EXTRACT(month FROM listings_latest_src_10004.created_at) AS listing__ds__extract_month - , EXTRACT(week FROM listings_latest_src_10004.created_at) AS listing__ds__extract_week - , EXTRACT(day FROM listings_latest_src_10004.created_at) AS listing__ds__extract_day - , EXTRACT(dow FROM listings_latest_src_10004.created_at) AS listing__ds__extract_dow - , EXTRACT(doy FROM listings_latest_src_10004.created_at) AS listing__ds__extract_doy - , DATE_TRUNC('day', listings_latest_src_10004.created_at) AS listing__created_at__day - , DATE_TRUNC('week', listings_latest_src_10004.created_at) AS listing__created_at__week - , DATE_TRUNC('month', listings_latest_src_10004.created_at) AS listing__created_at__month - , DATE_TRUNC('quarter', listings_latest_src_10004.created_at) AS listing__created_at__quarter - , DATE_TRUNC('year', listings_latest_src_10004.created_at) AS listing__created_at__year - , EXTRACT(year FROM listings_latest_src_10004.created_at) AS listing__created_at__extract_year - , EXTRACT(quarter FROM listings_latest_src_10004.created_at) AS listing__created_at__extract_quarter - , EXTRACT(month FROM listings_latest_src_10004.created_at) AS listing__created_at__extract_month - , EXTRACT(week FROM listings_latest_src_10004.created_at) AS listing__created_at__extract_week - , EXTRACT(day FROM listings_latest_src_10004.created_at) AS listing__created_at__extract_day - , EXTRACT(dow FROM listings_latest_src_10004.created_at) AS listing__created_at__extract_dow - , EXTRACT(doy FROM listings_latest_src_10004.created_at) AS listing__created_at__extract_doy - , listings_latest_src_10004.country AS listing__country_latest - , listings_latest_src_10004.is_lux AS listing__is_lux_latest - , listings_latest_src_10004.capacity AS listing__capacity_latest - , listings_latest_src_10004.listing_id AS listing - , listings_latest_src_10004.user_id AS user - , listings_latest_src_10004.user_id AS listing__user - FROM ***************************.dim_listings_latest listings_latest_src_10004 - ) subq_3 - ) subq_4 - ) subq_5 - ON - subq_2.listing = subq_5.listing - ) subq_6 - ) subq_7 - GROUP BY - subq_7.listing__country_latest - ) subq_8 - ) subq_9 -) subq_10 -ORDER BY subq_10.listing__country_latest + 1 AS listings + , listings_latest_src_10004.capacity AS largest_listing + , listings_latest_src_10004.capacity AS smallest_listing + , DATE_TRUNC('day', listings_latest_src_10004.created_at) AS ds__day + , DATE_TRUNC('week', listings_latest_src_10004.created_at) AS ds__week + , DATE_TRUNC('month', listings_latest_src_10004.created_at) AS ds__month + , DATE_TRUNC('quarter', listings_latest_src_10004.created_at) AS ds__quarter + , DATE_TRUNC('year', listings_latest_src_10004.created_at) AS ds__year + , EXTRACT(year FROM listings_latest_src_10004.created_at) AS ds__extract_year + , EXTRACT(quarter FROM listings_latest_src_10004.created_at) AS ds__extract_quarter + , EXTRACT(month FROM listings_latest_src_10004.created_at) AS ds__extract_month + , EXTRACT(week FROM listings_latest_src_10004.created_at) AS ds__extract_week + , EXTRACT(day FROM listings_latest_src_10004.created_at) AS ds__extract_day + , EXTRACT(dow FROM listings_latest_src_10004.created_at) AS ds__extract_dow + , EXTRACT(doy FROM listings_latest_src_10004.created_at) AS ds__extract_doy + , DATE_TRUNC('day', listings_latest_src_10004.created_at) AS created_at__day + , DATE_TRUNC('week', listings_latest_src_10004.created_at) AS created_at__week + , DATE_TRUNC('month', listings_latest_src_10004.created_at) AS created_at__month + , DATE_TRUNC('quarter', listings_latest_src_10004.created_at) AS created_at__quarter + , DATE_TRUNC('year', listings_latest_src_10004.created_at) AS created_at__year + , EXTRACT(year FROM listings_latest_src_10004.created_at) AS created_at__extract_year + , EXTRACT(quarter FROM listings_latest_src_10004.created_at) AS created_at__extract_quarter + , EXTRACT(month FROM listings_latest_src_10004.created_at) AS created_at__extract_month + , EXTRACT(week FROM listings_latest_src_10004.created_at) AS created_at__extract_week + , EXTRACT(day FROM listings_latest_src_10004.created_at) AS created_at__extract_day + , EXTRACT(dow FROM listings_latest_src_10004.created_at) AS created_at__extract_dow + , EXTRACT(doy FROM listings_latest_src_10004.created_at) AS created_at__extract_doy + , listings_latest_src_10004.country AS country_latest + , listings_latest_src_10004.is_lux AS is_lux_latest + , listings_latest_src_10004.capacity AS capacity_latest + , DATE_TRUNC('day', listings_latest_src_10004.created_at) AS listing__ds__day + , DATE_TRUNC('week', listings_latest_src_10004.created_at) AS listing__ds__week + , DATE_TRUNC('month', listings_latest_src_10004.created_at) AS listing__ds__month + , DATE_TRUNC('quarter', listings_latest_src_10004.created_at) AS listing__ds__quarter + , DATE_TRUNC('year', listings_latest_src_10004.created_at) AS listing__ds__year + , EXTRACT(year FROM listings_latest_src_10004.created_at) AS listing__ds__extract_year + , EXTRACT(quarter FROM listings_latest_src_10004.created_at) AS listing__ds__extract_quarter + , EXTRACT(month FROM listings_latest_src_10004.created_at) AS listing__ds__extract_month + , EXTRACT(week FROM listings_latest_src_10004.created_at) AS listing__ds__extract_week + , EXTRACT(day FROM listings_latest_src_10004.created_at) AS listing__ds__extract_day + , EXTRACT(dow FROM listings_latest_src_10004.created_at) AS listing__ds__extract_dow + , EXTRACT(doy FROM listings_latest_src_10004.created_at) AS listing__ds__extract_doy + , DATE_TRUNC('day', listings_latest_src_10004.created_at) AS listing__created_at__day + , DATE_TRUNC('week', listings_latest_src_10004.created_at) AS listing__created_at__week + , DATE_TRUNC('month', listings_latest_src_10004.created_at) AS listing__created_at__month + , DATE_TRUNC('quarter', listings_latest_src_10004.created_at) AS listing__created_at__quarter + , DATE_TRUNC('year', listings_latest_src_10004.created_at) AS listing__created_at__year + , EXTRACT(year FROM listings_latest_src_10004.created_at) AS listing__created_at__extract_year + , EXTRACT(quarter FROM listings_latest_src_10004.created_at) AS listing__created_at__extract_quarter + , EXTRACT(month FROM listings_latest_src_10004.created_at) AS listing__created_at__extract_month + , EXTRACT(week FROM listings_latest_src_10004.created_at) AS listing__created_at__extract_week + , EXTRACT(day FROM listings_latest_src_10004.created_at) AS listing__created_at__extract_day + , EXTRACT(dow FROM listings_latest_src_10004.created_at) AS listing__created_at__extract_dow + , EXTRACT(doy FROM listings_latest_src_10004.created_at) AS listing__created_at__extract_doy + , listings_latest_src_10004.country AS listing__country_latest + , listings_latest_src_10004.is_lux AS listing__is_lux_latest + , listings_latest_src_10004.capacity AS listing__capacity_latest + , listings_latest_src_10004.listing_id AS listing + , listings_latest_src_10004.user_id AS user + , listings_latest_src_10004.user_id AS listing__user + FROM ***************************.dim_listings_latest listings_latest_src_10004 + ) subq_0 + GROUP BY + subq_0.listing__country_latest + ) subq_1 + WHERE listing__country_latest = 'us' +) subq_2 +ORDER BY subq_2.listing__country_latest DESC LIMIT 100 diff --git a/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/DuckDB/test_distinct_values__plan0_optimized.sql b/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/DuckDB/test_distinct_values__plan0_optimized.sql index 328d6a2d79..592773c228 100644 --- a/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/DuckDB/test_distinct_values__plan0_optimized.sql +++ b/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/DuckDB/test_distinct_values__plan0_optimized.sql @@ -1,19 +1,17 @@ --- Join Standard Outputs --- Pass Only Elements: --- ['bookings', 'listing__country_latest'] --- Aggregate Measures --- Compute Metrics via Expressions --- Pass Only Elements: --- ['listing__country_latest'] +-- Constrain Output with WHERE -- Order By ['listing__country_latest'] Limit 100 SELECT - listings_latest_src_10004.country AS listing__country_latest -FROM ***************************.fct_bookings bookings_source_src_10001 -LEFT OUTER JOIN - ***************************.dim_listings_latest listings_latest_src_10004 -ON - bookings_source_src_10001.listing_id = listings_latest_src_10004.listing_id -GROUP BY - listings_latest_src_10004.country -ORDER BY listing__country_latest + listing__country_latest +FROM ( + -- Read Elements From Semantic Model 'listings_latest' + -- Pass Only Elements: + -- ['listing__country_latest'] + SELECT + country AS listing__country_latest + FROM ***************************.dim_listings_latest listings_latest_src_10004 + GROUP BY + country +) subq_4 +WHERE listing__country_latest = 'us' +ORDER BY listing__country_latest DESC LIMIT 100 diff --git a/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Postgres/test_distinct_values__plan0.sql b/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Postgres/test_distinct_values__plan0.sql index 86ba05d2c1..7818858e77 100644 --- a/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Postgres/test_distinct_values__plan0.sql +++ b/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Postgres/test_distinct_values__plan0.sql @@ -1,404 +1,84 @@ -- Order By ['listing__country_latest'] Limit 100 SELECT - subq_10.listing__country_latest + subq_2.listing__country_latest FROM ( - -- Pass Only Elements: - -- ['listing__country_latest'] + -- Constrain Output with WHERE SELECT - subq_9.listing__country_latest + subq_1.listing__country_latest FROM ( - -- Compute Metrics via Expressions + -- Pass Only Elements: + -- ['listing__country_latest'] SELECT - subq_8.listing__country_latest - , subq_8.bookings + subq_0.listing__country_latest FROM ( - -- Aggregate Measures + -- Read Elements From Semantic Model 'listings_latest' SELECT - subq_7.listing__country_latest - , SUM(subq_7.bookings) AS bookings - FROM ( - -- Pass Only Elements: - -- ['bookings', 'listing__country_latest'] - SELECT - subq_6.listing__country_latest - , subq_6.bookings - FROM ( - -- Join Standard Outputs - SELECT - subq_2.listing AS listing - , subq_5.country_latest AS listing__country_latest - , subq_2.bookings AS bookings - FROM ( - -- Pass Only Elements: - -- ['bookings', 'listing'] - SELECT - subq_1.listing - , subq_1.bookings - FROM ( - -- Metric Time Dimension 'ds' - SELECT - subq_0.ds__day - , subq_0.ds__week - , subq_0.ds__month - , subq_0.ds__quarter - , subq_0.ds__year - , subq_0.ds__extract_year - , subq_0.ds__extract_quarter - , subq_0.ds__extract_month - , subq_0.ds__extract_week - , subq_0.ds__extract_day - , subq_0.ds__extract_dow - , subq_0.ds__extract_doy - , subq_0.ds_partitioned__day - , subq_0.ds_partitioned__week - , subq_0.ds_partitioned__month - , subq_0.ds_partitioned__quarter - , subq_0.ds_partitioned__year - , subq_0.ds_partitioned__extract_year - , subq_0.ds_partitioned__extract_quarter - , subq_0.ds_partitioned__extract_month - , subq_0.ds_partitioned__extract_week - , subq_0.ds_partitioned__extract_day - , subq_0.ds_partitioned__extract_dow - , subq_0.ds_partitioned__extract_doy - , subq_0.paid_at__day - , subq_0.paid_at__week - , subq_0.paid_at__month - , subq_0.paid_at__quarter - , subq_0.paid_at__year - , subq_0.paid_at__extract_year - , subq_0.paid_at__extract_quarter - , subq_0.paid_at__extract_month - , subq_0.paid_at__extract_week - , subq_0.paid_at__extract_day - , subq_0.paid_at__extract_dow - , subq_0.paid_at__extract_doy - , subq_0.booking__ds__day - , subq_0.booking__ds__week - , subq_0.booking__ds__month - , subq_0.booking__ds__quarter - , subq_0.booking__ds__year - , subq_0.booking__ds__extract_year - , subq_0.booking__ds__extract_quarter - , subq_0.booking__ds__extract_month - , subq_0.booking__ds__extract_week - , subq_0.booking__ds__extract_day - , subq_0.booking__ds__extract_dow - , subq_0.booking__ds__extract_doy - , subq_0.booking__ds_partitioned__day - , subq_0.booking__ds_partitioned__week - , subq_0.booking__ds_partitioned__month - , subq_0.booking__ds_partitioned__quarter - , subq_0.booking__ds_partitioned__year - , subq_0.booking__ds_partitioned__extract_year - , subq_0.booking__ds_partitioned__extract_quarter - , subq_0.booking__ds_partitioned__extract_month - , subq_0.booking__ds_partitioned__extract_week - , subq_0.booking__ds_partitioned__extract_day - , subq_0.booking__ds_partitioned__extract_dow - , subq_0.booking__ds_partitioned__extract_doy - , subq_0.booking__paid_at__day - , subq_0.booking__paid_at__week - , subq_0.booking__paid_at__month - , subq_0.booking__paid_at__quarter - , subq_0.booking__paid_at__year - , subq_0.booking__paid_at__extract_year - , subq_0.booking__paid_at__extract_quarter - , subq_0.booking__paid_at__extract_month - , subq_0.booking__paid_at__extract_week - , subq_0.booking__paid_at__extract_day - , subq_0.booking__paid_at__extract_dow - , subq_0.booking__paid_at__extract_doy - , subq_0.ds__day AS metric_time__day - , subq_0.ds__week AS metric_time__week - , subq_0.ds__month AS metric_time__month - , subq_0.ds__quarter AS metric_time__quarter - , subq_0.ds__year AS metric_time__year - , subq_0.ds__extract_year AS metric_time__extract_year - , subq_0.ds__extract_quarter AS metric_time__extract_quarter - , subq_0.ds__extract_month AS metric_time__extract_month - , subq_0.ds__extract_week AS metric_time__extract_week - , subq_0.ds__extract_day AS metric_time__extract_day - , subq_0.ds__extract_dow AS metric_time__extract_dow - , subq_0.ds__extract_doy AS metric_time__extract_doy - , subq_0.listing - , subq_0.guest - , subq_0.host - , subq_0.booking__listing - , subq_0.booking__guest - , subq_0.booking__host - , subq_0.is_instant - , subq_0.booking__is_instant - , subq_0.bookings - , subq_0.instant_bookings - , subq_0.booking_value - , subq_0.max_booking_value - , subq_0.min_booking_value - , subq_0.bookers - , subq_0.average_booking_value - , subq_0.referred_bookings - , subq_0.median_booking_value - , subq_0.booking_value_p99 - , subq_0.discrete_booking_value_p99 - , subq_0.approximate_continuous_booking_value_p99 - , subq_0.approximate_discrete_booking_value_p99 - FROM ( - -- Read Elements From Semantic Model 'bookings_source' - SELECT - 1 AS bookings - , CASE WHEN is_instant THEN 1 ELSE 0 END AS instant_bookings - , bookings_source_src_10001.booking_value - , bookings_source_src_10001.booking_value AS max_booking_value - , bookings_source_src_10001.booking_value AS min_booking_value - , bookings_source_src_10001.guest_id AS bookers - , bookings_source_src_10001.booking_value AS average_booking_value - , bookings_source_src_10001.booking_value AS booking_payments - , CASE WHEN referrer_id IS NOT NULL THEN 1 ELSE 0 END AS referred_bookings - , bookings_source_src_10001.booking_value AS median_booking_value - , bookings_source_src_10001.booking_value AS booking_value_p99 - , bookings_source_src_10001.booking_value AS discrete_booking_value_p99 - , bookings_source_src_10001.booking_value AS approximate_continuous_booking_value_p99 - , bookings_source_src_10001.booking_value AS approximate_discrete_booking_value_p99 - , bookings_source_src_10001.is_instant - , DATE_TRUNC('day', bookings_source_src_10001.ds) AS ds__day - , DATE_TRUNC('week', bookings_source_src_10001.ds) AS ds__week - , DATE_TRUNC('month', bookings_source_src_10001.ds) AS ds__month - , DATE_TRUNC('quarter', bookings_source_src_10001.ds) AS ds__quarter - , DATE_TRUNC('year', bookings_source_src_10001.ds) AS ds__year - , EXTRACT(year FROM bookings_source_src_10001.ds) AS ds__extract_year - , EXTRACT(quarter FROM bookings_source_src_10001.ds) AS ds__extract_quarter - , EXTRACT(month FROM bookings_source_src_10001.ds) AS ds__extract_month - , EXTRACT(week FROM bookings_source_src_10001.ds) AS ds__extract_week - , EXTRACT(day FROM bookings_source_src_10001.ds) AS ds__extract_day - , EXTRACT(dow FROM bookings_source_src_10001.ds) AS ds__extract_dow - , EXTRACT(doy FROM bookings_source_src_10001.ds) AS ds__extract_doy - , DATE_TRUNC('day', bookings_source_src_10001.ds_partitioned) AS ds_partitioned__day - , DATE_TRUNC('week', bookings_source_src_10001.ds_partitioned) AS ds_partitioned__week - , DATE_TRUNC('month', bookings_source_src_10001.ds_partitioned) AS ds_partitioned__month - , DATE_TRUNC('quarter', bookings_source_src_10001.ds_partitioned) AS ds_partitioned__quarter - , DATE_TRUNC('year', bookings_source_src_10001.ds_partitioned) AS ds_partitioned__year - , EXTRACT(year FROM bookings_source_src_10001.ds_partitioned) AS ds_partitioned__extract_year - , EXTRACT(quarter FROM bookings_source_src_10001.ds_partitioned) AS ds_partitioned__extract_quarter - , EXTRACT(month FROM bookings_source_src_10001.ds_partitioned) AS ds_partitioned__extract_month - , EXTRACT(week FROM bookings_source_src_10001.ds_partitioned) AS ds_partitioned__extract_week - , EXTRACT(day FROM bookings_source_src_10001.ds_partitioned) AS ds_partitioned__extract_day - , EXTRACT(dow FROM bookings_source_src_10001.ds_partitioned) AS ds_partitioned__extract_dow - , EXTRACT(doy FROM bookings_source_src_10001.ds_partitioned) AS ds_partitioned__extract_doy - , DATE_TRUNC('day', bookings_source_src_10001.paid_at) AS paid_at__day - , DATE_TRUNC('week', bookings_source_src_10001.paid_at) AS paid_at__week - , DATE_TRUNC('month', bookings_source_src_10001.paid_at) AS paid_at__month - , DATE_TRUNC('quarter', bookings_source_src_10001.paid_at) AS paid_at__quarter - , DATE_TRUNC('year', bookings_source_src_10001.paid_at) AS paid_at__year - , EXTRACT(year FROM bookings_source_src_10001.paid_at) AS paid_at__extract_year - , EXTRACT(quarter FROM bookings_source_src_10001.paid_at) AS paid_at__extract_quarter - , EXTRACT(month FROM bookings_source_src_10001.paid_at) AS paid_at__extract_month - , EXTRACT(week FROM bookings_source_src_10001.paid_at) AS paid_at__extract_week - , EXTRACT(day FROM bookings_source_src_10001.paid_at) AS paid_at__extract_day - , EXTRACT(dow FROM bookings_source_src_10001.paid_at) AS paid_at__extract_dow - , EXTRACT(doy FROM bookings_source_src_10001.paid_at) AS paid_at__extract_doy - , bookings_source_src_10001.is_instant AS booking__is_instant - , DATE_TRUNC('day', bookings_source_src_10001.ds) AS booking__ds__day - , DATE_TRUNC('week', bookings_source_src_10001.ds) AS booking__ds__week - , DATE_TRUNC('month', bookings_source_src_10001.ds) AS booking__ds__month - , DATE_TRUNC('quarter', bookings_source_src_10001.ds) AS booking__ds__quarter - , DATE_TRUNC('year', bookings_source_src_10001.ds) AS booking__ds__year - , EXTRACT(year FROM bookings_source_src_10001.ds) AS booking__ds__extract_year - , EXTRACT(quarter FROM bookings_source_src_10001.ds) AS booking__ds__extract_quarter - , EXTRACT(month FROM bookings_source_src_10001.ds) AS booking__ds__extract_month - , EXTRACT(week FROM bookings_source_src_10001.ds) AS booking__ds__extract_week - , EXTRACT(day FROM bookings_source_src_10001.ds) AS booking__ds__extract_day - , EXTRACT(dow FROM bookings_source_src_10001.ds) AS booking__ds__extract_dow - , EXTRACT(doy FROM bookings_source_src_10001.ds) AS booking__ds__extract_doy - , DATE_TRUNC('day', bookings_source_src_10001.ds_partitioned) AS booking__ds_partitioned__day - , DATE_TRUNC('week', bookings_source_src_10001.ds_partitioned) AS booking__ds_partitioned__week - , DATE_TRUNC('month', bookings_source_src_10001.ds_partitioned) AS booking__ds_partitioned__month - , DATE_TRUNC('quarter', bookings_source_src_10001.ds_partitioned) AS booking__ds_partitioned__quarter - , DATE_TRUNC('year', bookings_source_src_10001.ds_partitioned) AS booking__ds_partitioned__year - , EXTRACT(year FROM bookings_source_src_10001.ds_partitioned) AS booking__ds_partitioned__extract_year - , EXTRACT(quarter FROM bookings_source_src_10001.ds_partitioned) AS booking__ds_partitioned__extract_quarter - , EXTRACT(month FROM bookings_source_src_10001.ds_partitioned) AS booking__ds_partitioned__extract_month - , EXTRACT(week FROM bookings_source_src_10001.ds_partitioned) AS booking__ds_partitioned__extract_week - , EXTRACT(day FROM bookings_source_src_10001.ds_partitioned) AS booking__ds_partitioned__extract_day - , EXTRACT(dow FROM bookings_source_src_10001.ds_partitioned) AS booking__ds_partitioned__extract_dow - , EXTRACT(doy FROM bookings_source_src_10001.ds_partitioned) AS booking__ds_partitioned__extract_doy - , DATE_TRUNC('day', bookings_source_src_10001.paid_at) AS booking__paid_at__day - , DATE_TRUNC('week', bookings_source_src_10001.paid_at) AS booking__paid_at__week - , DATE_TRUNC('month', bookings_source_src_10001.paid_at) AS booking__paid_at__month - , DATE_TRUNC('quarter', bookings_source_src_10001.paid_at) AS booking__paid_at__quarter - , DATE_TRUNC('year', bookings_source_src_10001.paid_at) AS booking__paid_at__year - , EXTRACT(year FROM bookings_source_src_10001.paid_at) AS booking__paid_at__extract_year - , EXTRACT(quarter FROM bookings_source_src_10001.paid_at) AS booking__paid_at__extract_quarter - , EXTRACT(month FROM bookings_source_src_10001.paid_at) AS booking__paid_at__extract_month - , EXTRACT(week FROM bookings_source_src_10001.paid_at) AS booking__paid_at__extract_week - , EXTRACT(day FROM bookings_source_src_10001.paid_at) AS booking__paid_at__extract_day - , EXTRACT(dow FROM bookings_source_src_10001.paid_at) AS booking__paid_at__extract_dow - , EXTRACT(doy FROM bookings_source_src_10001.paid_at) AS booking__paid_at__extract_doy - , bookings_source_src_10001.listing_id AS listing - , bookings_source_src_10001.guest_id AS guest - , bookings_source_src_10001.host_id AS host - , bookings_source_src_10001.listing_id AS booking__listing - , bookings_source_src_10001.guest_id AS booking__guest - , bookings_source_src_10001.host_id AS booking__host - FROM ***************************.fct_bookings bookings_source_src_10001 - ) subq_0 - ) subq_1 - ) subq_2 - LEFT OUTER JOIN ( - -- Pass Only Elements: - -- ['country_latest', 'listing'] - SELECT - subq_4.listing - , subq_4.country_latest - FROM ( - -- Metric Time Dimension 'ds' - SELECT - subq_3.ds__day - , subq_3.ds__week - , subq_3.ds__month - , subq_3.ds__quarter - , subq_3.ds__year - , subq_3.ds__extract_year - , subq_3.ds__extract_quarter - , subq_3.ds__extract_month - , subq_3.ds__extract_week - , subq_3.ds__extract_day - , subq_3.ds__extract_dow - , subq_3.ds__extract_doy - , subq_3.created_at__day - , subq_3.created_at__week - , subq_3.created_at__month - , subq_3.created_at__quarter - , subq_3.created_at__year - , subq_3.created_at__extract_year - , subq_3.created_at__extract_quarter - , subq_3.created_at__extract_month - , subq_3.created_at__extract_week - , subq_3.created_at__extract_day - , subq_3.created_at__extract_dow - , subq_3.created_at__extract_doy - , subq_3.listing__ds__day - , subq_3.listing__ds__week - , subq_3.listing__ds__month - , subq_3.listing__ds__quarter - , subq_3.listing__ds__year - , subq_3.listing__ds__extract_year - , subq_3.listing__ds__extract_quarter - , subq_3.listing__ds__extract_month - , subq_3.listing__ds__extract_week - , subq_3.listing__ds__extract_day - , subq_3.listing__ds__extract_dow - , subq_3.listing__ds__extract_doy - , subq_3.listing__created_at__day - , subq_3.listing__created_at__week - , subq_3.listing__created_at__month - , subq_3.listing__created_at__quarter - , subq_3.listing__created_at__year - , subq_3.listing__created_at__extract_year - , subq_3.listing__created_at__extract_quarter - , subq_3.listing__created_at__extract_month - , subq_3.listing__created_at__extract_week - , subq_3.listing__created_at__extract_day - , subq_3.listing__created_at__extract_dow - , subq_3.listing__created_at__extract_doy - , subq_3.ds__day AS metric_time__day - , subq_3.ds__week AS metric_time__week - , subq_3.ds__month AS metric_time__month - , subq_3.ds__quarter AS metric_time__quarter - , subq_3.ds__year AS metric_time__year - , subq_3.ds__extract_year AS metric_time__extract_year - , subq_3.ds__extract_quarter AS metric_time__extract_quarter - , subq_3.ds__extract_month AS metric_time__extract_month - , subq_3.ds__extract_week AS metric_time__extract_week - , subq_3.ds__extract_day AS metric_time__extract_day - , subq_3.ds__extract_dow AS metric_time__extract_dow - , subq_3.ds__extract_doy AS metric_time__extract_doy - , subq_3.listing - , subq_3.user - , subq_3.listing__user - , subq_3.country_latest - , subq_3.is_lux_latest - , subq_3.capacity_latest - , subq_3.listing__country_latest - , subq_3.listing__is_lux_latest - , subq_3.listing__capacity_latest - , subq_3.listings - , subq_3.largest_listing - , subq_3.smallest_listing - FROM ( - -- Read Elements From Semantic Model 'listings_latest' - SELECT - 1 AS listings - , listings_latest_src_10004.capacity AS largest_listing - , listings_latest_src_10004.capacity AS smallest_listing - , DATE_TRUNC('day', listings_latest_src_10004.created_at) AS ds__day - , DATE_TRUNC('week', listings_latest_src_10004.created_at) AS ds__week - , DATE_TRUNC('month', listings_latest_src_10004.created_at) AS ds__month - , DATE_TRUNC('quarter', listings_latest_src_10004.created_at) AS ds__quarter - , DATE_TRUNC('year', listings_latest_src_10004.created_at) AS ds__year - , EXTRACT(year FROM listings_latest_src_10004.created_at) AS ds__extract_year - , EXTRACT(quarter FROM listings_latest_src_10004.created_at) AS ds__extract_quarter - , EXTRACT(month FROM listings_latest_src_10004.created_at) AS ds__extract_month - , EXTRACT(week FROM listings_latest_src_10004.created_at) AS ds__extract_week - , EXTRACT(day FROM listings_latest_src_10004.created_at) AS ds__extract_day - , EXTRACT(dow FROM listings_latest_src_10004.created_at) AS ds__extract_dow - , EXTRACT(doy FROM listings_latest_src_10004.created_at) AS ds__extract_doy - , DATE_TRUNC('day', listings_latest_src_10004.created_at) AS created_at__day - , DATE_TRUNC('week', listings_latest_src_10004.created_at) AS created_at__week - , DATE_TRUNC('month', listings_latest_src_10004.created_at) AS created_at__month - , DATE_TRUNC('quarter', listings_latest_src_10004.created_at) AS created_at__quarter - , DATE_TRUNC('year', listings_latest_src_10004.created_at) AS created_at__year - , EXTRACT(year FROM listings_latest_src_10004.created_at) AS created_at__extract_year - , EXTRACT(quarter FROM listings_latest_src_10004.created_at) AS created_at__extract_quarter - , EXTRACT(month FROM listings_latest_src_10004.created_at) AS created_at__extract_month - , EXTRACT(week FROM listings_latest_src_10004.created_at) AS created_at__extract_week - , EXTRACT(day FROM listings_latest_src_10004.created_at) AS created_at__extract_day - , EXTRACT(dow FROM listings_latest_src_10004.created_at) AS created_at__extract_dow - , EXTRACT(doy FROM listings_latest_src_10004.created_at) AS created_at__extract_doy - , listings_latest_src_10004.country AS country_latest - , listings_latest_src_10004.is_lux AS is_lux_latest - , listings_latest_src_10004.capacity AS capacity_latest - , DATE_TRUNC('day', listings_latest_src_10004.created_at) AS listing__ds__day - , DATE_TRUNC('week', listings_latest_src_10004.created_at) AS listing__ds__week - , DATE_TRUNC('month', listings_latest_src_10004.created_at) AS listing__ds__month - , DATE_TRUNC('quarter', listings_latest_src_10004.created_at) AS listing__ds__quarter - , DATE_TRUNC('year', listings_latest_src_10004.created_at) AS listing__ds__year - , EXTRACT(year FROM listings_latest_src_10004.created_at) AS listing__ds__extract_year - , EXTRACT(quarter FROM listings_latest_src_10004.created_at) AS listing__ds__extract_quarter - , EXTRACT(month FROM listings_latest_src_10004.created_at) AS listing__ds__extract_month - , EXTRACT(week FROM listings_latest_src_10004.created_at) AS listing__ds__extract_week - , EXTRACT(day FROM listings_latest_src_10004.created_at) AS listing__ds__extract_day - , EXTRACT(dow FROM listings_latest_src_10004.created_at) AS listing__ds__extract_dow - , EXTRACT(doy FROM listings_latest_src_10004.created_at) AS listing__ds__extract_doy - , DATE_TRUNC('day', listings_latest_src_10004.created_at) AS listing__created_at__day - , DATE_TRUNC('week', listings_latest_src_10004.created_at) AS listing__created_at__week - , DATE_TRUNC('month', listings_latest_src_10004.created_at) AS listing__created_at__month - , DATE_TRUNC('quarter', listings_latest_src_10004.created_at) AS listing__created_at__quarter - , DATE_TRUNC('year', listings_latest_src_10004.created_at) AS listing__created_at__year - , EXTRACT(year FROM listings_latest_src_10004.created_at) AS listing__created_at__extract_year - , EXTRACT(quarter FROM listings_latest_src_10004.created_at) AS listing__created_at__extract_quarter - , EXTRACT(month FROM listings_latest_src_10004.created_at) AS listing__created_at__extract_month - , EXTRACT(week FROM listings_latest_src_10004.created_at) AS listing__created_at__extract_week - , EXTRACT(day FROM listings_latest_src_10004.created_at) AS listing__created_at__extract_day - , EXTRACT(dow FROM listings_latest_src_10004.created_at) AS listing__created_at__extract_dow - , EXTRACT(doy FROM listings_latest_src_10004.created_at) AS listing__created_at__extract_doy - , listings_latest_src_10004.country AS listing__country_latest - , listings_latest_src_10004.is_lux AS listing__is_lux_latest - , listings_latest_src_10004.capacity AS listing__capacity_latest - , listings_latest_src_10004.listing_id AS listing - , listings_latest_src_10004.user_id AS user - , listings_latest_src_10004.user_id AS listing__user - FROM ***************************.dim_listings_latest listings_latest_src_10004 - ) subq_3 - ) subq_4 - ) subq_5 - ON - subq_2.listing = subq_5.listing - ) subq_6 - ) subq_7 - GROUP BY - subq_7.listing__country_latest - ) subq_8 - ) subq_9 -) subq_10 -ORDER BY subq_10.listing__country_latest + 1 AS listings + , listings_latest_src_10004.capacity AS largest_listing + , listings_latest_src_10004.capacity AS smallest_listing + , DATE_TRUNC('day', listings_latest_src_10004.created_at) AS ds__day + , DATE_TRUNC('week', listings_latest_src_10004.created_at) AS ds__week + , DATE_TRUNC('month', listings_latest_src_10004.created_at) AS ds__month + , DATE_TRUNC('quarter', listings_latest_src_10004.created_at) AS ds__quarter + , DATE_TRUNC('year', listings_latest_src_10004.created_at) AS ds__year + , EXTRACT(year FROM listings_latest_src_10004.created_at) AS ds__extract_year + , EXTRACT(quarter FROM listings_latest_src_10004.created_at) AS ds__extract_quarter + , EXTRACT(month FROM listings_latest_src_10004.created_at) AS ds__extract_month + , EXTRACT(week FROM listings_latest_src_10004.created_at) AS ds__extract_week + , EXTRACT(day FROM listings_latest_src_10004.created_at) AS ds__extract_day + , EXTRACT(dow FROM listings_latest_src_10004.created_at) AS ds__extract_dow + , EXTRACT(doy FROM listings_latest_src_10004.created_at) AS ds__extract_doy + , DATE_TRUNC('day', listings_latest_src_10004.created_at) AS created_at__day + , DATE_TRUNC('week', listings_latest_src_10004.created_at) AS created_at__week + , DATE_TRUNC('month', listings_latest_src_10004.created_at) AS created_at__month + , DATE_TRUNC('quarter', listings_latest_src_10004.created_at) AS created_at__quarter + , DATE_TRUNC('year', listings_latest_src_10004.created_at) AS created_at__year + , EXTRACT(year FROM listings_latest_src_10004.created_at) AS created_at__extract_year + , EXTRACT(quarter FROM listings_latest_src_10004.created_at) AS created_at__extract_quarter + , EXTRACT(month FROM listings_latest_src_10004.created_at) AS created_at__extract_month + , EXTRACT(week FROM listings_latest_src_10004.created_at) AS created_at__extract_week + , EXTRACT(day FROM listings_latest_src_10004.created_at) AS created_at__extract_day + , EXTRACT(dow FROM listings_latest_src_10004.created_at) AS created_at__extract_dow + , EXTRACT(doy FROM listings_latest_src_10004.created_at) AS created_at__extract_doy + , listings_latest_src_10004.country AS country_latest + , listings_latest_src_10004.is_lux AS is_lux_latest + , listings_latest_src_10004.capacity AS capacity_latest + , DATE_TRUNC('day', listings_latest_src_10004.created_at) AS listing__ds__day + , DATE_TRUNC('week', listings_latest_src_10004.created_at) AS listing__ds__week + , DATE_TRUNC('month', listings_latest_src_10004.created_at) AS listing__ds__month + , DATE_TRUNC('quarter', listings_latest_src_10004.created_at) AS listing__ds__quarter + , DATE_TRUNC('year', listings_latest_src_10004.created_at) AS listing__ds__year + , EXTRACT(year FROM listings_latest_src_10004.created_at) AS listing__ds__extract_year + , EXTRACT(quarter FROM listings_latest_src_10004.created_at) AS listing__ds__extract_quarter + , EXTRACT(month FROM listings_latest_src_10004.created_at) AS listing__ds__extract_month + , EXTRACT(week FROM listings_latest_src_10004.created_at) AS listing__ds__extract_week + , EXTRACT(day FROM listings_latest_src_10004.created_at) AS listing__ds__extract_day + , EXTRACT(dow FROM listings_latest_src_10004.created_at) AS listing__ds__extract_dow + , EXTRACT(doy FROM listings_latest_src_10004.created_at) AS listing__ds__extract_doy + , DATE_TRUNC('day', listings_latest_src_10004.created_at) AS listing__created_at__day + , DATE_TRUNC('week', listings_latest_src_10004.created_at) AS listing__created_at__week + , DATE_TRUNC('month', listings_latest_src_10004.created_at) AS listing__created_at__month + , DATE_TRUNC('quarter', listings_latest_src_10004.created_at) AS listing__created_at__quarter + , DATE_TRUNC('year', listings_latest_src_10004.created_at) AS listing__created_at__year + , EXTRACT(year FROM listings_latest_src_10004.created_at) AS listing__created_at__extract_year + , EXTRACT(quarter FROM listings_latest_src_10004.created_at) AS listing__created_at__extract_quarter + , EXTRACT(month FROM listings_latest_src_10004.created_at) AS listing__created_at__extract_month + , EXTRACT(week FROM listings_latest_src_10004.created_at) AS listing__created_at__extract_week + , EXTRACT(day FROM listings_latest_src_10004.created_at) AS listing__created_at__extract_day + , EXTRACT(dow FROM listings_latest_src_10004.created_at) AS listing__created_at__extract_dow + , EXTRACT(doy FROM listings_latest_src_10004.created_at) AS listing__created_at__extract_doy + , listings_latest_src_10004.country AS listing__country_latest + , listings_latest_src_10004.is_lux AS listing__is_lux_latest + , listings_latest_src_10004.capacity AS listing__capacity_latest + , listings_latest_src_10004.listing_id AS listing + , listings_latest_src_10004.user_id AS user + , listings_latest_src_10004.user_id AS listing__user + FROM ***************************.dim_listings_latest listings_latest_src_10004 + ) subq_0 + GROUP BY + subq_0.listing__country_latest + ) subq_1 + WHERE listing__country_latest = 'us' +) subq_2 +ORDER BY subq_2.listing__country_latest DESC LIMIT 100 diff --git a/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Postgres/test_distinct_values__plan0_optimized.sql b/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Postgres/test_distinct_values__plan0_optimized.sql index 328d6a2d79..592773c228 100644 --- a/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Postgres/test_distinct_values__plan0_optimized.sql +++ b/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Postgres/test_distinct_values__plan0_optimized.sql @@ -1,19 +1,17 @@ --- Join Standard Outputs --- Pass Only Elements: --- ['bookings', 'listing__country_latest'] --- Aggregate Measures --- Compute Metrics via Expressions --- Pass Only Elements: --- ['listing__country_latest'] +-- Constrain Output with WHERE -- Order By ['listing__country_latest'] Limit 100 SELECT - listings_latest_src_10004.country AS listing__country_latest -FROM ***************************.fct_bookings bookings_source_src_10001 -LEFT OUTER JOIN - ***************************.dim_listings_latest listings_latest_src_10004 -ON - bookings_source_src_10001.listing_id = listings_latest_src_10004.listing_id -GROUP BY - listings_latest_src_10004.country -ORDER BY listing__country_latest + listing__country_latest +FROM ( + -- Read Elements From Semantic Model 'listings_latest' + -- Pass Only Elements: + -- ['listing__country_latest'] + SELECT + country AS listing__country_latest + FROM ***************************.dim_listings_latest listings_latest_src_10004 + GROUP BY + country +) subq_4 +WHERE listing__country_latest = 'us' +ORDER BY listing__country_latest DESC LIMIT 100 diff --git a/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Redshift/test_distinct_values__plan0.sql b/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Redshift/test_distinct_values__plan0.sql index 86ba05d2c1..7818858e77 100644 --- a/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Redshift/test_distinct_values__plan0.sql +++ b/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Redshift/test_distinct_values__plan0.sql @@ -1,404 +1,84 @@ -- Order By ['listing__country_latest'] Limit 100 SELECT - subq_10.listing__country_latest + subq_2.listing__country_latest FROM ( - -- Pass Only Elements: - -- ['listing__country_latest'] + -- Constrain Output with WHERE SELECT - subq_9.listing__country_latest + subq_1.listing__country_latest FROM ( - -- Compute Metrics via Expressions + -- Pass Only Elements: + -- ['listing__country_latest'] SELECT - subq_8.listing__country_latest - , subq_8.bookings + subq_0.listing__country_latest FROM ( - -- Aggregate Measures + -- Read Elements From Semantic Model 'listings_latest' SELECT - subq_7.listing__country_latest - , SUM(subq_7.bookings) AS bookings - FROM ( - -- Pass Only Elements: - -- ['bookings', 'listing__country_latest'] - SELECT - subq_6.listing__country_latest - , subq_6.bookings - FROM ( - -- Join Standard Outputs - SELECT - subq_2.listing AS listing - , subq_5.country_latest AS listing__country_latest - , subq_2.bookings AS bookings - FROM ( - -- Pass Only Elements: - -- ['bookings', 'listing'] - SELECT - subq_1.listing - , subq_1.bookings - FROM ( - -- Metric Time Dimension 'ds' - SELECT - subq_0.ds__day - , subq_0.ds__week - , subq_0.ds__month - , subq_0.ds__quarter - , subq_0.ds__year - , subq_0.ds__extract_year - , subq_0.ds__extract_quarter - , subq_0.ds__extract_month - , subq_0.ds__extract_week - , subq_0.ds__extract_day - , subq_0.ds__extract_dow - , subq_0.ds__extract_doy - , subq_0.ds_partitioned__day - , subq_0.ds_partitioned__week - , subq_0.ds_partitioned__month - , subq_0.ds_partitioned__quarter - , subq_0.ds_partitioned__year - , subq_0.ds_partitioned__extract_year - , subq_0.ds_partitioned__extract_quarter - , subq_0.ds_partitioned__extract_month - , subq_0.ds_partitioned__extract_week - , subq_0.ds_partitioned__extract_day - , subq_0.ds_partitioned__extract_dow - , subq_0.ds_partitioned__extract_doy - , subq_0.paid_at__day - , subq_0.paid_at__week - , subq_0.paid_at__month - , subq_0.paid_at__quarter - , subq_0.paid_at__year - , subq_0.paid_at__extract_year - , subq_0.paid_at__extract_quarter - , subq_0.paid_at__extract_month - , subq_0.paid_at__extract_week - , subq_0.paid_at__extract_day - , subq_0.paid_at__extract_dow - , subq_0.paid_at__extract_doy - , subq_0.booking__ds__day - , subq_0.booking__ds__week - , subq_0.booking__ds__month - , subq_0.booking__ds__quarter - , subq_0.booking__ds__year - , subq_0.booking__ds__extract_year - , subq_0.booking__ds__extract_quarter - , subq_0.booking__ds__extract_month - , subq_0.booking__ds__extract_week - , subq_0.booking__ds__extract_day - , subq_0.booking__ds__extract_dow - , subq_0.booking__ds__extract_doy - , subq_0.booking__ds_partitioned__day - , subq_0.booking__ds_partitioned__week - , subq_0.booking__ds_partitioned__month - , subq_0.booking__ds_partitioned__quarter - , subq_0.booking__ds_partitioned__year - , subq_0.booking__ds_partitioned__extract_year - , subq_0.booking__ds_partitioned__extract_quarter - , subq_0.booking__ds_partitioned__extract_month - , subq_0.booking__ds_partitioned__extract_week - , subq_0.booking__ds_partitioned__extract_day - , subq_0.booking__ds_partitioned__extract_dow - , subq_0.booking__ds_partitioned__extract_doy - , subq_0.booking__paid_at__day - , subq_0.booking__paid_at__week - , subq_0.booking__paid_at__month - , subq_0.booking__paid_at__quarter - , subq_0.booking__paid_at__year - , subq_0.booking__paid_at__extract_year - , subq_0.booking__paid_at__extract_quarter - , subq_0.booking__paid_at__extract_month - , subq_0.booking__paid_at__extract_week - , subq_0.booking__paid_at__extract_day - , subq_0.booking__paid_at__extract_dow - , subq_0.booking__paid_at__extract_doy - , subq_0.ds__day AS metric_time__day - , subq_0.ds__week AS metric_time__week - , subq_0.ds__month AS metric_time__month - , subq_0.ds__quarter AS metric_time__quarter - , subq_0.ds__year AS metric_time__year - , subq_0.ds__extract_year AS metric_time__extract_year - , subq_0.ds__extract_quarter AS metric_time__extract_quarter - , subq_0.ds__extract_month AS metric_time__extract_month - , subq_0.ds__extract_week AS metric_time__extract_week - , subq_0.ds__extract_day AS metric_time__extract_day - , subq_0.ds__extract_dow AS metric_time__extract_dow - , subq_0.ds__extract_doy AS metric_time__extract_doy - , subq_0.listing - , subq_0.guest - , subq_0.host - , subq_0.booking__listing - , subq_0.booking__guest - , subq_0.booking__host - , subq_0.is_instant - , subq_0.booking__is_instant - , subq_0.bookings - , subq_0.instant_bookings - , subq_0.booking_value - , subq_0.max_booking_value - , subq_0.min_booking_value - , subq_0.bookers - , subq_0.average_booking_value - , subq_0.referred_bookings - , subq_0.median_booking_value - , subq_0.booking_value_p99 - , subq_0.discrete_booking_value_p99 - , subq_0.approximate_continuous_booking_value_p99 - , subq_0.approximate_discrete_booking_value_p99 - FROM ( - -- Read Elements From Semantic Model 'bookings_source' - SELECT - 1 AS bookings - , CASE WHEN is_instant THEN 1 ELSE 0 END AS instant_bookings - , bookings_source_src_10001.booking_value - , bookings_source_src_10001.booking_value AS max_booking_value - , bookings_source_src_10001.booking_value AS min_booking_value - , bookings_source_src_10001.guest_id AS bookers - , bookings_source_src_10001.booking_value AS average_booking_value - , bookings_source_src_10001.booking_value AS booking_payments - , CASE WHEN referrer_id IS NOT NULL THEN 1 ELSE 0 END AS referred_bookings - , bookings_source_src_10001.booking_value AS median_booking_value - , bookings_source_src_10001.booking_value AS booking_value_p99 - , bookings_source_src_10001.booking_value AS discrete_booking_value_p99 - , bookings_source_src_10001.booking_value AS approximate_continuous_booking_value_p99 - , bookings_source_src_10001.booking_value AS approximate_discrete_booking_value_p99 - , bookings_source_src_10001.is_instant - , DATE_TRUNC('day', bookings_source_src_10001.ds) AS ds__day - , DATE_TRUNC('week', bookings_source_src_10001.ds) AS ds__week - , DATE_TRUNC('month', bookings_source_src_10001.ds) AS ds__month - , DATE_TRUNC('quarter', bookings_source_src_10001.ds) AS ds__quarter - , DATE_TRUNC('year', bookings_source_src_10001.ds) AS ds__year - , EXTRACT(year FROM bookings_source_src_10001.ds) AS ds__extract_year - , EXTRACT(quarter FROM bookings_source_src_10001.ds) AS ds__extract_quarter - , EXTRACT(month FROM bookings_source_src_10001.ds) AS ds__extract_month - , EXTRACT(week FROM bookings_source_src_10001.ds) AS ds__extract_week - , EXTRACT(day FROM bookings_source_src_10001.ds) AS ds__extract_day - , EXTRACT(dow FROM bookings_source_src_10001.ds) AS ds__extract_dow - , EXTRACT(doy FROM bookings_source_src_10001.ds) AS ds__extract_doy - , DATE_TRUNC('day', bookings_source_src_10001.ds_partitioned) AS ds_partitioned__day - , DATE_TRUNC('week', bookings_source_src_10001.ds_partitioned) AS ds_partitioned__week - , DATE_TRUNC('month', bookings_source_src_10001.ds_partitioned) AS ds_partitioned__month - , DATE_TRUNC('quarter', bookings_source_src_10001.ds_partitioned) AS ds_partitioned__quarter - , DATE_TRUNC('year', bookings_source_src_10001.ds_partitioned) AS ds_partitioned__year - , EXTRACT(year FROM bookings_source_src_10001.ds_partitioned) AS ds_partitioned__extract_year - , EXTRACT(quarter FROM bookings_source_src_10001.ds_partitioned) AS ds_partitioned__extract_quarter - , EXTRACT(month FROM bookings_source_src_10001.ds_partitioned) AS ds_partitioned__extract_month - , EXTRACT(week FROM bookings_source_src_10001.ds_partitioned) AS ds_partitioned__extract_week - , EXTRACT(day FROM bookings_source_src_10001.ds_partitioned) AS ds_partitioned__extract_day - , EXTRACT(dow FROM bookings_source_src_10001.ds_partitioned) AS ds_partitioned__extract_dow - , EXTRACT(doy FROM bookings_source_src_10001.ds_partitioned) AS ds_partitioned__extract_doy - , DATE_TRUNC('day', bookings_source_src_10001.paid_at) AS paid_at__day - , DATE_TRUNC('week', bookings_source_src_10001.paid_at) AS paid_at__week - , DATE_TRUNC('month', bookings_source_src_10001.paid_at) AS paid_at__month - , DATE_TRUNC('quarter', bookings_source_src_10001.paid_at) AS paid_at__quarter - , DATE_TRUNC('year', bookings_source_src_10001.paid_at) AS paid_at__year - , EXTRACT(year FROM bookings_source_src_10001.paid_at) AS paid_at__extract_year - , EXTRACT(quarter FROM bookings_source_src_10001.paid_at) AS paid_at__extract_quarter - , EXTRACT(month FROM bookings_source_src_10001.paid_at) AS paid_at__extract_month - , EXTRACT(week FROM bookings_source_src_10001.paid_at) AS paid_at__extract_week - , EXTRACT(day FROM bookings_source_src_10001.paid_at) AS paid_at__extract_day - , EXTRACT(dow FROM bookings_source_src_10001.paid_at) AS paid_at__extract_dow - , EXTRACT(doy FROM bookings_source_src_10001.paid_at) AS paid_at__extract_doy - , bookings_source_src_10001.is_instant AS booking__is_instant - , DATE_TRUNC('day', bookings_source_src_10001.ds) AS booking__ds__day - , DATE_TRUNC('week', bookings_source_src_10001.ds) AS booking__ds__week - , DATE_TRUNC('month', bookings_source_src_10001.ds) AS booking__ds__month - , DATE_TRUNC('quarter', bookings_source_src_10001.ds) AS booking__ds__quarter - , DATE_TRUNC('year', bookings_source_src_10001.ds) AS booking__ds__year - , EXTRACT(year FROM bookings_source_src_10001.ds) AS booking__ds__extract_year - , EXTRACT(quarter FROM bookings_source_src_10001.ds) AS booking__ds__extract_quarter - , EXTRACT(month FROM bookings_source_src_10001.ds) AS booking__ds__extract_month - , EXTRACT(week FROM bookings_source_src_10001.ds) AS booking__ds__extract_week - , EXTRACT(day FROM bookings_source_src_10001.ds) AS booking__ds__extract_day - , EXTRACT(dow FROM bookings_source_src_10001.ds) AS booking__ds__extract_dow - , EXTRACT(doy FROM bookings_source_src_10001.ds) AS booking__ds__extract_doy - , DATE_TRUNC('day', bookings_source_src_10001.ds_partitioned) AS booking__ds_partitioned__day - , DATE_TRUNC('week', bookings_source_src_10001.ds_partitioned) AS booking__ds_partitioned__week - , DATE_TRUNC('month', bookings_source_src_10001.ds_partitioned) AS booking__ds_partitioned__month - , DATE_TRUNC('quarter', bookings_source_src_10001.ds_partitioned) AS booking__ds_partitioned__quarter - , DATE_TRUNC('year', bookings_source_src_10001.ds_partitioned) AS booking__ds_partitioned__year - , EXTRACT(year FROM bookings_source_src_10001.ds_partitioned) AS booking__ds_partitioned__extract_year - , EXTRACT(quarter FROM bookings_source_src_10001.ds_partitioned) AS booking__ds_partitioned__extract_quarter - , EXTRACT(month FROM bookings_source_src_10001.ds_partitioned) AS booking__ds_partitioned__extract_month - , EXTRACT(week FROM bookings_source_src_10001.ds_partitioned) AS booking__ds_partitioned__extract_week - , EXTRACT(day FROM bookings_source_src_10001.ds_partitioned) AS booking__ds_partitioned__extract_day - , EXTRACT(dow FROM bookings_source_src_10001.ds_partitioned) AS booking__ds_partitioned__extract_dow - , EXTRACT(doy FROM bookings_source_src_10001.ds_partitioned) AS booking__ds_partitioned__extract_doy - , DATE_TRUNC('day', bookings_source_src_10001.paid_at) AS booking__paid_at__day - , DATE_TRUNC('week', bookings_source_src_10001.paid_at) AS booking__paid_at__week - , DATE_TRUNC('month', bookings_source_src_10001.paid_at) AS booking__paid_at__month - , DATE_TRUNC('quarter', bookings_source_src_10001.paid_at) AS booking__paid_at__quarter - , DATE_TRUNC('year', bookings_source_src_10001.paid_at) AS booking__paid_at__year - , EXTRACT(year FROM bookings_source_src_10001.paid_at) AS booking__paid_at__extract_year - , EXTRACT(quarter FROM bookings_source_src_10001.paid_at) AS booking__paid_at__extract_quarter - , EXTRACT(month FROM bookings_source_src_10001.paid_at) AS booking__paid_at__extract_month - , EXTRACT(week FROM bookings_source_src_10001.paid_at) AS booking__paid_at__extract_week - , EXTRACT(day FROM bookings_source_src_10001.paid_at) AS booking__paid_at__extract_day - , EXTRACT(dow FROM bookings_source_src_10001.paid_at) AS booking__paid_at__extract_dow - , EXTRACT(doy FROM bookings_source_src_10001.paid_at) AS booking__paid_at__extract_doy - , bookings_source_src_10001.listing_id AS listing - , bookings_source_src_10001.guest_id AS guest - , bookings_source_src_10001.host_id AS host - , bookings_source_src_10001.listing_id AS booking__listing - , bookings_source_src_10001.guest_id AS booking__guest - , bookings_source_src_10001.host_id AS booking__host - FROM ***************************.fct_bookings bookings_source_src_10001 - ) subq_0 - ) subq_1 - ) subq_2 - LEFT OUTER JOIN ( - -- Pass Only Elements: - -- ['country_latest', 'listing'] - SELECT - subq_4.listing - , subq_4.country_latest - FROM ( - -- Metric Time Dimension 'ds' - SELECT - subq_3.ds__day - , subq_3.ds__week - , subq_3.ds__month - , subq_3.ds__quarter - , subq_3.ds__year - , subq_3.ds__extract_year - , subq_3.ds__extract_quarter - , subq_3.ds__extract_month - , subq_3.ds__extract_week - , subq_3.ds__extract_day - , subq_3.ds__extract_dow - , subq_3.ds__extract_doy - , subq_3.created_at__day - , subq_3.created_at__week - , subq_3.created_at__month - , subq_3.created_at__quarter - , subq_3.created_at__year - , subq_3.created_at__extract_year - , subq_3.created_at__extract_quarter - , subq_3.created_at__extract_month - , subq_3.created_at__extract_week - , subq_3.created_at__extract_day - , subq_3.created_at__extract_dow - , subq_3.created_at__extract_doy - , subq_3.listing__ds__day - , subq_3.listing__ds__week - , subq_3.listing__ds__month - , subq_3.listing__ds__quarter - , subq_3.listing__ds__year - , subq_3.listing__ds__extract_year - , subq_3.listing__ds__extract_quarter - , subq_3.listing__ds__extract_month - , subq_3.listing__ds__extract_week - , subq_3.listing__ds__extract_day - , subq_3.listing__ds__extract_dow - , subq_3.listing__ds__extract_doy - , subq_3.listing__created_at__day - , subq_3.listing__created_at__week - , subq_3.listing__created_at__month - , subq_3.listing__created_at__quarter - , subq_3.listing__created_at__year - , subq_3.listing__created_at__extract_year - , subq_3.listing__created_at__extract_quarter - , subq_3.listing__created_at__extract_month - , subq_3.listing__created_at__extract_week - , subq_3.listing__created_at__extract_day - , subq_3.listing__created_at__extract_dow - , subq_3.listing__created_at__extract_doy - , subq_3.ds__day AS metric_time__day - , subq_3.ds__week AS metric_time__week - , subq_3.ds__month AS metric_time__month - , subq_3.ds__quarter AS metric_time__quarter - , subq_3.ds__year AS metric_time__year - , subq_3.ds__extract_year AS metric_time__extract_year - , subq_3.ds__extract_quarter AS metric_time__extract_quarter - , subq_3.ds__extract_month AS metric_time__extract_month - , subq_3.ds__extract_week AS metric_time__extract_week - , subq_3.ds__extract_day AS metric_time__extract_day - , subq_3.ds__extract_dow AS metric_time__extract_dow - , subq_3.ds__extract_doy AS metric_time__extract_doy - , subq_3.listing - , subq_3.user - , subq_3.listing__user - , subq_3.country_latest - , subq_3.is_lux_latest - , subq_3.capacity_latest - , subq_3.listing__country_latest - , subq_3.listing__is_lux_latest - , subq_3.listing__capacity_latest - , subq_3.listings - , subq_3.largest_listing - , subq_3.smallest_listing - FROM ( - -- Read Elements From Semantic Model 'listings_latest' - SELECT - 1 AS listings - , listings_latest_src_10004.capacity AS largest_listing - , listings_latest_src_10004.capacity AS smallest_listing - , DATE_TRUNC('day', listings_latest_src_10004.created_at) AS ds__day - , DATE_TRUNC('week', listings_latest_src_10004.created_at) AS ds__week - , DATE_TRUNC('month', listings_latest_src_10004.created_at) AS ds__month - , DATE_TRUNC('quarter', listings_latest_src_10004.created_at) AS ds__quarter - , DATE_TRUNC('year', listings_latest_src_10004.created_at) AS ds__year - , EXTRACT(year FROM listings_latest_src_10004.created_at) AS ds__extract_year - , EXTRACT(quarter FROM listings_latest_src_10004.created_at) AS ds__extract_quarter - , EXTRACT(month FROM listings_latest_src_10004.created_at) AS ds__extract_month - , EXTRACT(week FROM listings_latest_src_10004.created_at) AS ds__extract_week - , EXTRACT(day FROM listings_latest_src_10004.created_at) AS ds__extract_day - , EXTRACT(dow FROM listings_latest_src_10004.created_at) AS ds__extract_dow - , EXTRACT(doy FROM listings_latest_src_10004.created_at) AS ds__extract_doy - , DATE_TRUNC('day', listings_latest_src_10004.created_at) AS created_at__day - , DATE_TRUNC('week', listings_latest_src_10004.created_at) AS created_at__week - , DATE_TRUNC('month', listings_latest_src_10004.created_at) AS created_at__month - , DATE_TRUNC('quarter', listings_latest_src_10004.created_at) AS created_at__quarter - , DATE_TRUNC('year', listings_latest_src_10004.created_at) AS created_at__year - , EXTRACT(year FROM listings_latest_src_10004.created_at) AS created_at__extract_year - , EXTRACT(quarter FROM listings_latest_src_10004.created_at) AS created_at__extract_quarter - , EXTRACT(month FROM listings_latest_src_10004.created_at) AS created_at__extract_month - , EXTRACT(week FROM listings_latest_src_10004.created_at) AS created_at__extract_week - , EXTRACT(day FROM listings_latest_src_10004.created_at) AS created_at__extract_day - , EXTRACT(dow FROM listings_latest_src_10004.created_at) AS created_at__extract_dow - , EXTRACT(doy FROM listings_latest_src_10004.created_at) AS created_at__extract_doy - , listings_latest_src_10004.country AS country_latest - , listings_latest_src_10004.is_lux AS is_lux_latest - , listings_latest_src_10004.capacity AS capacity_latest - , DATE_TRUNC('day', listings_latest_src_10004.created_at) AS listing__ds__day - , DATE_TRUNC('week', listings_latest_src_10004.created_at) AS listing__ds__week - , DATE_TRUNC('month', listings_latest_src_10004.created_at) AS listing__ds__month - , DATE_TRUNC('quarter', listings_latest_src_10004.created_at) AS listing__ds__quarter - , DATE_TRUNC('year', listings_latest_src_10004.created_at) AS listing__ds__year - , EXTRACT(year FROM listings_latest_src_10004.created_at) AS listing__ds__extract_year - , EXTRACT(quarter FROM listings_latest_src_10004.created_at) AS listing__ds__extract_quarter - , EXTRACT(month FROM listings_latest_src_10004.created_at) AS listing__ds__extract_month - , EXTRACT(week FROM listings_latest_src_10004.created_at) AS listing__ds__extract_week - , EXTRACT(day FROM listings_latest_src_10004.created_at) AS listing__ds__extract_day - , EXTRACT(dow FROM listings_latest_src_10004.created_at) AS listing__ds__extract_dow - , EXTRACT(doy FROM listings_latest_src_10004.created_at) AS listing__ds__extract_doy - , DATE_TRUNC('day', listings_latest_src_10004.created_at) AS listing__created_at__day - , DATE_TRUNC('week', listings_latest_src_10004.created_at) AS listing__created_at__week - , DATE_TRUNC('month', listings_latest_src_10004.created_at) AS listing__created_at__month - , DATE_TRUNC('quarter', listings_latest_src_10004.created_at) AS listing__created_at__quarter - , DATE_TRUNC('year', listings_latest_src_10004.created_at) AS listing__created_at__year - , EXTRACT(year FROM listings_latest_src_10004.created_at) AS listing__created_at__extract_year - , EXTRACT(quarter FROM listings_latest_src_10004.created_at) AS listing__created_at__extract_quarter - , EXTRACT(month FROM listings_latest_src_10004.created_at) AS listing__created_at__extract_month - , EXTRACT(week FROM listings_latest_src_10004.created_at) AS listing__created_at__extract_week - , EXTRACT(day FROM listings_latest_src_10004.created_at) AS listing__created_at__extract_day - , EXTRACT(dow FROM listings_latest_src_10004.created_at) AS listing__created_at__extract_dow - , EXTRACT(doy FROM listings_latest_src_10004.created_at) AS listing__created_at__extract_doy - , listings_latest_src_10004.country AS listing__country_latest - , listings_latest_src_10004.is_lux AS listing__is_lux_latest - , listings_latest_src_10004.capacity AS listing__capacity_latest - , listings_latest_src_10004.listing_id AS listing - , listings_latest_src_10004.user_id AS user - , listings_latest_src_10004.user_id AS listing__user - FROM ***************************.dim_listings_latest listings_latest_src_10004 - ) subq_3 - ) subq_4 - ) subq_5 - ON - subq_2.listing = subq_5.listing - ) subq_6 - ) subq_7 - GROUP BY - subq_7.listing__country_latest - ) subq_8 - ) subq_9 -) subq_10 -ORDER BY subq_10.listing__country_latest + 1 AS listings + , listings_latest_src_10004.capacity AS largest_listing + , listings_latest_src_10004.capacity AS smallest_listing + , DATE_TRUNC('day', listings_latest_src_10004.created_at) AS ds__day + , DATE_TRUNC('week', listings_latest_src_10004.created_at) AS ds__week + , DATE_TRUNC('month', listings_latest_src_10004.created_at) AS ds__month + , DATE_TRUNC('quarter', listings_latest_src_10004.created_at) AS ds__quarter + , DATE_TRUNC('year', listings_latest_src_10004.created_at) AS ds__year + , EXTRACT(year FROM listings_latest_src_10004.created_at) AS ds__extract_year + , EXTRACT(quarter FROM listings_latest_src_10004.created_at) AS ds__extract_quarter + , EXTRACT(month FROM listings_latest_src_10004.created_at) AS ds__extract_month + , EXTRACT(week FROM listings_latest_src_10004.created_at) AS ds__extract_week + , EXTRACT(day FROM listings_latest_src_10004.created_at) AS ds__extract_day + , EXTRACT(dow FROM listings_latest_src_10004.created_at) AS ds__extract_dow + , EXTRACT(doy FROM listings_latest_src_10004.created_at) AS ds__extract_doy + , DATE_TRUNC('day', listings_latest_src_10004.created_at) AS created_at__day + , DATE_TRUNC('week', listings_latest_src_10004.created_at) AS created_at__week + , DATE_TRUNC('month', listings_latest_src_10004.created_at) AS created_at__month + , DATE_TRUNC('quarter', listings_latest_src_10004.created_at) AS created_at__quarter + , DATE_TRUNC('year', listings_latest_src_10004.created_at) AS created_at__year + , EXTRACT(year FROM listings_latest_src_10004.created_at) AS created_at__extract_year + , EXTRACT(quarter FROM listings_latest_src_10004.created_at) AS created_at__extract_quarter + , EXTRACT(month FROM listings_latest_src_10004.created_at) AS created_at__extract_month + , EXTRACT(week FROM listings_latest_src_10004.created_at) AS created_at__extract_week + , EXTRACT(day FROM listings_latest_src_10004.created_at) AS created_at__extract_day + , EXTRACT(dow FROM listings_latest_src_10004.created_at) AS created_at__extract_dow + , EXTRACT(doy FROM listings_latest_src_10004.created_at) AS created_at__extract_doy + , listings_latest_src_10004.country AS country_latest + , listings_latest_src_10004.is_lux AS is_lux_latest + , listings_latest_src_10004.capacity AS capacity_latest + , DATE_TRUNC('day', listings_latest_src_10004.created_at) AS listing__ds__day + , DATE_TRUNC('week', listings_latest_src_10004.created_at) AS listing__ds__week + , DATE_TRUNC('month', listings_latest_src_10004.created_at) AS listing__ds__month + , DATE_TRUNC('quarter', listings_latest_src_10004.created_at) AS listing__ds__quarter + , DATE_TRUNC('year', listings_latest_src_10004.created_at) AS listing__ds__year + , EXTRACT(year FROM listings_latest_src_10004.created_at) AS listing__ds__extract_year + , EXTRACT(quarter FROM listings_latest_src_10004.created_at) AS listing__ds__extract_quarter + , EXTRACT(month FROM listings_latest_src_10004.created_at) AS listing__ds__extract_month + , EXTRACT(week FROM listings_latest_src_10004.created_at) AS listing__ds__extract_week + , EXTRACT(day FROM listings_latest_src_10004.created_at) AS listing__ds__extract_day + , EXTRACT(dow FROM listings_latest_src_10004.created_at) AS listing__ds__extract_dow + , EXTRACT(doy FROM listings_latest_src_10004.created_at) AS listing__ds__extract_doy + , DATE_TRUNC('day', listings_latest_src_10004.created_at) AS listing__created_at__day + , DATE_TRUNC('week', listings_latest_src_10004.created_at) AS listing__created_at__week + , DATE_TRUNC('month', listings_latest_src_10004.created_at) AS listing__created_at__month + , DATE_TRUNC('quarter', listings_latest_src_10004.created_at) AS listing__created_at__quarter + , DATE_TRUNC('year', listings_latest_src_10004.created_at) AS listing__created_at__year + , EXTRACT(year FROM listings_latest_src_10004.created_at) AS listing__created_at__extract_year + , EXTRACT(quarter FROM listings_latest_src_10004.created_at) AS listing__created_at__extract_quarter + , EXTRACT(month FROM listings_latest_src_10004.created_at) AS listing__created_at__extract_month + , EXTRACT(week FROM listings_latest_src_10004.created_at) AS listing__created_at__extract_week + , EXTRACT(day FROM listings_latest_src_10004.created_at) AS listing__created_at__extract_day + , EXTRACT(dow FROM listings_latest_src_10004.created_at) AS listing__created_at__extract_dow + , EXTRACT(doy FROM listings_latest_src_10004.created_at) AS listing__created_at__extract_doy + , listings_latest_src_10004.country AS listing__country_latest + , listings_latest_src_10004.is_lux AS listing__is_lux_latest + , listings_latest_src_10004.capacity AS listing__capacity_latest + , listings_latest_src_10004.listing_id AS listing + , listings_latest_src_10004.user_id AS user + , listings_latest_src_10004.user_id AS listing__user + FROM ***************************.dim_listings_latest listings_latest_src_10004 + ) subq_0 + GROUP BY + subq_0.listing__country_latest + ) subq_1 + WHERE listing__country_latest = 'us' +) subq_2 +ORDER BY subq_2.listing__country_latest DESC LIMIT 100 diff --git a/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Redshift/test_distinct_values__plan0_optimized.sql b/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Redshift/test_distinct_values__plan0_optimized.sql index 328d6a2d79..592773c228 100644 --- a/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Redshift/test_distinct_values__plan0_optimized.sql +++ b/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Redshift/test_distinct_values__plan0_optimized.sql @@ -1,19 +1,17 @@ --- Join Standard Outputs --- Pass Only Elements: --- ['bookings', 'listing__country_latest'] --- Aggregate Measures --- Compute Metrics via Expressions --- Pass Only Elements: --- ['listing__country_latest'] +-- Constrain Output with WHERE -- Order By ['listing__country_latest'] Limit 100 SELECT - listings_latest_src_10004.country AS listing__country_latest -FROM ***************************.fct_bookings bookings_source_src_10001 -LEFT OUTER JOIN - ***************************.dim_listings_latest listings_latest_src_10004 -ON - bookings_source_src_10001.listing_id = listings_latest_src_10004.listing_id -GROUP BY - listings_latest_src_10004.country -ORDER BY listing__country_latest + listing__country_latest +FROM ( + -- Read Elements From Semantic Model 'listings_latest' + -- Pass Only Elements: + -- ['listing__country_latest'] + SELECT + country AS listing__country_latest + FROM ***************************.dim_listings_latest listings_latest_src_10004 + GROUP BY + country +) subq_4 +WHERE listing__country_latest = 'us' +ORDER BY listing__country_latest DESC LIMIT 100 diff --git a/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Snowflake/test_distinct_values__plan0.sql b/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Snowflake/test_distinct_values__plan0.sql index 86ba05d2c1..7818858e77 100644 --- a/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Snowflake/test_distinct_values__plan0.sql +++ b/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Snowflake/test_distinct_values__plan0.sql @@ -1,404 +1,84 @@ -- Order By ['listing__country_latest'] Limit 100 SELECT - subq_10.listing__country_latest + subq_2.listing__country_latest FROM ( - -- Pass Only Elements: - -- ['listing__country_latest'] + -- Constrain Output with WHERE SELECT - subq_9.listing__country_latest + subq_1.listing__country_latest FROM ( - -- Compute Metrics via Expressions + -- Pass Only Elements: + -- ['listing__country_latest'] SELECT - subq_8.listing__country_latest - , subq_8.bookings + subq_0.listing__country_latest FROM ( - -- Aggregate Measures + -- Read Elements From Semantic Model 'listings_latest' SELECT - subq_7.listing__country_latest - , SUM(subq_7.bookings) AS bookings - FROM ( - -- Pass Only Elements: - -- ['bookings', 'listing__country_latest'] - SELECT - subq_6.listing__country_latest - , subq_6.bookings - FROM ( - -- Join Standard Outputs - SELECT - subq_2.listing AS listing - , subq_5.country_latest AS listing__country_latest - , subq_2.bookings AS bookings - FROM ( - -- Pass Only Elements: - -- ['bookings', 'listing'] - SELECT - subq_1.listing - , subq_1.bookings - FROM ( - -- Metric Time Dimension 'ds' - SELECT - subq_0.ds__day - , subq_0.ds__week - , subq_0.ds__month - , subq_0.ds__quarter - , subq_0.ds__year - , subq_0.ds__extract_year - , subq_0.ds__extract_quarter - , subq_0.ds__extract_month - , subq_0.ds__extract_week - , subq_0.ds__extract_day - , subq_0.ds__extract_dow - , subq_0.ds__extract_doy - , subq_0.ds_partitioned__day - , subq_0.ds_partitioned__week - , subq_0.ds_partitioned__month - , subq_0.ds_partitioned__quarter - , subq_0.ds_partitioned__year - , subq_0.ds_partitioned__extract_year - , subq_0.ds_partitioned__extract_quarter - , subq_0.ds_partitioned__extract_month - , subq_0.ds_partitioned__extract_week - , subq_0.ds_partitioned__extract_day - , subq_0.ds_partitioned__extract_dow - , subq_0.ds_partitioned__extract_doy - , subq_0.paid_at__day - , subq_0.paid_at__week - , subq_0.paid_at__month - , subq_0.paid_at__quarter - , subq_0.paid_at__year - , subq_0.paid_at__extract_year - , subq_0.paid_at__extract_quarter - , subq_0.paid_at__extract_month - , subq_0.paid_at__extract_week - , subq_0.paid_at__extract_day - , subq_0.paid_at__extract_dow - , subq_0.paid_at__extract_doy - , subq_0.booking__ds__day - , subq_0.booking__ds__week - , subq_0.booking__ds__month - , subq_0.booking__ds__quarter - , subq_0.booking__ds__year - , subq_0.booking__ds__extract_year - , subq_0.booking__ds__extract_quarter - , subq_0.booking__ds__extract_month - , subq_0.booking__ds__extract_week - , subq_0.booking__ds__extract_day - , subq_0.booking__ds__extract_dow - , subq_0.booking__ds__extract_doy - , subq_0.booking__ds_partitioned__day - , subq_0.booking__ds_partitioned__week - , subq_0.booking__ds_partitioned__month - , subq_0.booking__ds_partitioned__quarter - , subq_0.booking__ds_partitioned__year - , subq_0.booking__ds_partitioned__extract_year - , subq_0.booking__ds_partitioned__extract_quarter - , subq_0.booking__ds_partitioned__extract_month - , subq_0.booking__ds_partitioned__extract_week - , subq_0.booking__ds_partitioned__extract_day - , subq_0.booking__ds_partitioned__extract_dow - , subq_0.booking__ds_partitioned__extract_doy - , subq_0.booking__paid_at__day - , subq_0.booking__paid_at__week - , subq_0.booking__paid_at__month - , subq_0.booking__paid_at__quarter - , subq_0.booking__paid_at__year - , subq_0.booking__paid_at__extract_year - , subq_0.booking__paid_at__extract_quarter - , subq_0.booking__paid_at__extract_month - , subq_0.booking__paid_at__extract_week - , subq_0.booking__paid_at__extract_day - , subq_0.booking__paid_at__extract_dow - , subq_0.booking__paid_at__extract_doy - , subq_0.ds__day AS metric_time__day - , subq_0.ds__week AS metric_time__week - , subq_0.ds__month AS metric_time__month - , subq_0.ds__quarter AS metric_time__quarter - , subq_0.ds__year AS metric_time__year - , subq_0.ds__extract_year AS metric_time__extract_year - , subq_0.ds__extract_quarter AS metric_time__extract_quarter - , subq_0.ds__extract_month AS metric_time__extract_month - , subq_0.ds__extract_week AS metric_time__extract_week - , subq_0.ds__extract_day AS metric_time__extract_day - , subq_0.ds__extract_dow AS metric_time__extract_dow - , subq_0.ds__extract_doy AS metric_time__extract_doy - , subq_0.listing - , subq_0.guest - , subq_0.host - , subq_0.booking__listing - , subq_0.booking__guest - , subq_0.booking__host - , subq_0.is_instant - , subq_0.booking__is_instant - , subq_0.bookings - , subq_0.instant_bookings - , subq_0.booking_value - , subq_0.max_booking_value - , subq_0.min_booking_value - , subq_0.bookers - , subq_0.average_booking_value - , subq_0.referred_bookings - , subq_0.median_booking_value - , subq_0.booking_value_p99 - , subq_0.discrete_booking_value_p99 - , subq_0.approximate_continuous_booking_value_p99 - , subq_0.approximate_discrete_booking_value_p99 - FROM ( - -- Read Elements From Semantic Model 'bookings_source' - SELECT - 1 AS bookings - , CASE WHEN is_instant THEN 1 ELSE 0 END AS instant_bookings - , bookings_source_src_10001.booking_value - , bookings_source_src_10001.booking_value AS max_booking_value - , bookings_source_src_10001.booking_value AS min_booking_value - , bookings_source_src_10001.guest_id AS bookers - , bookings_source_src_10001.booking_value AS average_booking_value - , bookings_source_src_10001.booking_value AS booking_payments - , CASE WHEN referrer_id IS NOT NULL THEN 1 ELSE 0 END AS referred_bookings - , bookings_source_src_10001.booking_value AS median_booking_value - , bookings_source_src_10001.booking_value AS booking_value_p99 - , bookings_source_src_10001.booking_value AS discrete_booking_value_p99 - , bookings_source_src_10001.booking_value AS approximate_continuous_booking_value_p99 - , bookings_source_src_10001.booking_value AS approximate_discrete_booking_value_p99 - , bookings_source_src_10001.is_instant - , DATE_TRUNC('day', bookings_source_src_10001.ds) AS ds__day - , DATE_TRUNC('week', bookings_source_src_10001.ds) AS ds__week - , DATE_TRUNC('month', bookings_source_src_10001.ds) AS ds__month - , DATE_TRUNC('quarter', bookings_source_src_10001.ds) AS ds__quarter - , DATE_TRUNC('year', bookings_source_src_10001.ds) AS ds__year - , EXTRACT(year FROM bookings_source_src_10001.ds) AS ds__extract_year - , EXTRACT(quarter FROM bookings_source_src_10001.ds) AS ds__extract_quarter - , EXTRACT(month FROM bookings_source_src_10001.ds) AS ds__extract_month - , EXTRACT(week FROM bookings_source_src_10001.ds) AS ds__extract_week - , EXTRACT(day FROM bookings_source_src_10001.ds) AS ds__extract_day - , EXTRACT(dow FROM bookings_source_src_10001.ds) AS ds__extract_dow - , EXTRACT(doy FROM bookings_source_src_10001.ds) AS ds__extract_doy - , DATE_TRUNC('day', bookings_source_src_10001.ds_partitioned) AS ds_partitioned__day - , DATE_TRUNC('week', bookings_source_src_10001.ds_partitioned) AS ds_partitioned__week - , DATE_TRUNC('month', bookings_source_src_10001.ds_partitioned) AS ds_partitioned__month - , DATE_TRUNC('quarter', bookings_source_src_10001.ds_partitioned) AS ds_partitioned__quarter - , DATE_TRUNC('year', bookings_source_src_10001.ds_partitioned) AS ds_partitioned__year - , EXTRACT(year FROM bookings_source_src_10001.ds_partitioned) AS ds_partitioned__extract_year - , EXTRACT(quarter FROM bookings_source_src_10001.ds_partitioned) AS ds_partitioned__extract_quarter - , EXTRACT(month FROM bookings_source_src_10001.ds_partitioned) AS ds_partitioned__extract_month - , EXTRACT(week FROM bookings_source_src_10001.ds_partitioned) AS ds_partitioned__extract_week - , EXTRACT(day FROM bookings_source_src_10001.ds_partitioned) AS ds_partitioned__extract_day - , EXTRACT(dow FROM bookings_source_src_10001.ds_partitioned) AS ds_partitioned__extract_dow - , EXTRACT(doy FROM bookings_source_src_10001.ds_partitioned) AS ds_partitioned__extract_doy - , DATE_TRUNC('day', bookings_source_src_10001.paid_at) AS paid_at__day - , DATE_TRUNC('week', bookings_source_src_10001.paid_at) AS paid_at__week - , DATE_TRUNC('month', bookings_source_src_10001.paid_at) AS paid_at__month - , DATE_TRUNC('quarter', bookings_source_src_10001.paid_at) AS paid_at__quarter - , DATE_TRUNC('year', bookings_source_src_10001.paid_at) AS paid_at__year - , EXTRACT(year FROM bookings_source_src_10001.paid_at) AS paid_at__extract_year - , EXTRACT(quarter FROM bookings_source_src_10001.paid_at) AS paid_at__extract_quarter - , EXTRACT(month FROM bookings_source_src_10001.paid_at) AS paid_at__extract_month - , EXTRACT(week FROM bookings_source_src_10001.paid_at) AS paid_at__extract_week - , EXTRACT(day FROM bookings_source_src_10001.paid_at) AS paid_at__extract_day - , EXTRACT(dow FROM bookings_source_src_10001.paid_at) AS paid_at__extract_dow - , EXTRACT(doy FROM bookings_source_src_10001.paid_at) AS paid_at__extract_doy - , bookings_source_src_10001.is_instant AS booking__is_instant - , DATE_TRUNC('day', bookings_source_src_10001.ds) AS booking__ds__day - , DATE_TRUNC('week', bookings_source_src_10001.ds) AS booking__ds__week - , DATE_TRUNC('month', bookings_source_src_10001.ds) AS booking__ds__month - , DATE_TRUNC('quarter', bookings_source_src_10001.ds) AS booking__ds__quarter - , DATE_TRUNC('year', bookings_source_src_10001.ds) AS booking__ds__year - , EXTRACT(year FROM bookings_source_src_10001.ds) AS booking__ds__extract_year - , EXTRACT(quarter FROM bookings_source_src_10001.ds) AS booking__ds__extract_quarter - , EXTRACT(month FROM bookings_source_src_10001.ds) AS booking__ds__extract_month - , EXTRACT(week FROM bookings_source_src_10001.ds) AS booking__ds__extract_week - , EXTRACT(day FROM bookings_source_src_10001.ds) AS booking__ds__extract_day - , EXTRACT(dow FROM bookings_source_src_10001.ds) AS booking__ds__extract_dow - , EXTRACT(doy FROM bookings_source_src_10001.ds) AS booking__ds__extract_doy - , DATE_TRUNC('day', bookings_source_src_10001.ds_partitioned) AS booking__ds_partitioned__day - , DATE_TRUNC('week', bookings_source_src_10001.ds_partitioned) AS booking__ds_partitioned__week - , DATE_TRUNC('month', bookings_source_src_10001.ds_partitioned) AS booking__ds_partitioned__month - , DATE_TRUNC('quarter', bookings_source_src_10001.ds_partitioned) AS booking__ds_partitioned__quarter - , DATE_TRUNC('year', bookings_source_src_10001.ds_partitioned) AS booking__ds_partitioned__year - , EXTRACT(year FROM bookings_source_src_10001.ds_partitioned) AS booking__ds_partitioned__extract_year - , EXTRACT(quarter FROM bookings_source_src_10001.ds_partitioned) AS booking__ds_partitioned__extract_quarter - , EXTRACT(month FROM bookings_source_src_10001.ds_partitioned) AS booking__ds_partitioned__extract_month - , EXTRACT(week FROM bookings_source_src_10001.ds_partitioned) AS booking__ds_partitioned__extract_week - , EXTRACT(day FROM bookings_source_src_10001.ds_partitioned) AS booking__ds_partitioned__extract_day - , EXTRACT(dow FROM bookings_source_src_10001.ds_partitioned) AS booking__ds_partitioned__extract_dow - , EXTRACT(doy FROM bookings_source_src_10001.ds_partitioned) AS booking__ds_partitioned__extract_doy - , DATE_TRUNC('day', bookings_source_src_10001.paid_at) AS booking__paid_at__day - , DATE_TRUNC('week', bookings_source_src_10001.paid_at) AS booking__paid_at__week - , DATE_TRUNC('month', bookings_source_src_10001.paid_at) AS booking__paid_at__month - , DATE_TRUNC('quarter', bookings_source_src_10001.paid_at) AS booking__paid_at__quarter - , DATE_TRUNC('year', bookings_source_src_10001.paid_at) AS booking__paid_at__year - , EXTRACT(year FROM bookings_source_src_10001.paid_at) AS booking__paid_at__extract_year - , EXTRACT(quarter FROM bookings_source_src_10001.paid_at) AS booking__paid_at__extract_quarter - , EXTRACT(month FROM bookings_source_src_10001.paid_at) AS booking__paid_at__extract_month - , EXTRACT(week FROM bookings_source_src_10001.paid_at) AS booking__paid_at__extract_week - , EXTRACT(day FROM bookings_source_src_10001.paid_at) AS booking__paid_at__extract_day - , EXTRACT(dow FROM bookings_source_src_10001.paid_at) AS booking__paid_at__extract_dow - , EXTRACT(doy FROM bookings_source_src_10001.paid_at) AS booking__paid_at__extract_doy - , bookings_source_src_10001.listing_id AS listing - , bookings_source_src_10001.guest_id AS guest - , bookings_source_src_10001.host_id AS host - , bookings_source_src_10001.listing_id AS booking__listing - , bookings_source_src_10001.guest_id AS booking__guest - , bookings_source_src_10001.host_id AS booking__host - FROM ***************************.fct_bookings bookings_source_src_10001 - ) subq_0 - ) subq_1 - ) subq_2 - LEFT OUTER JOIN ( - -- Pass Only Elements: - -- ['country_latest', 'listing'] - SELECT - subq_4.listing - , subq_4.country_latest - FROM ( - -- Metric Time Dimension 'ds' - SELECT - subq_3.ds__day - , subq_3.ds__week - , subq_3.ds__month - , subq_3.ds__quarter - , subq_3.ds__year - , subq_3.ds__extract_year - , subq_3.ds__extract_quarter - , subq_3.ds__extract_month - , subq_3.ds__extract_week - , subq_3.ds__extract_day - , subq_3.ds__extract_dow - , subq_3.ds__extract_doy - , subq_3.created_at__day - , subq_3.created_at__week - , subq_3.created_at__month - , subq_3.created_at__quarter - , subq_3.created_at__year - , subq_3.created_at__extract_year - , subq_3.created_at__extract_quarter - , subq_3.created_at__extract_month - , subq_3.created_at__extract_week - , subq_3.created_at__extract_day - , subq_3.created_at__extract_dow - , subq_3.created_at__extract_doy - , subq_3.listing__ds__day - , subq_3.listing__ds__week - , subq_3.listing__ds__month - , subq_3.listing__ds__quarter - , subq_3.listing__ds__year - , subq_3.listing__ds__extract_year - , subq_3.listing__ds__extract_quarter - , subq_3.listing__ds__extract_month - , subq_3.listing__ds__extract_week - , subq_3.listing__ds__extract_day - , subq_3.listing__ds__extract_dow - , subq_3.listing__ds__extract_doy - , subq_3.listing__created_at__day - , subq_3.listing__created_at__week - , subq_3.listing__created_at__month - , subq_3.listing__created_at__quarter - , subq_3.listing__created_at__year - , subq_3.listing__created_at__extract_year - , subq_3.listing__created_at__extract_quarter - , subq_3.listing__created_at__extract_month - , subq_3.listing__created_at__extract_week - , subq_3.listing__created_at__extract_day - , subq_3.listing__created_at__extract_dow - , subq_3.listing__created_at__extract_doy - , subq_3.ds__day AS metric_time__day - , subq_3.ds__week AS metric_time__week - , subq_3.ds__month AS metric_time__month - , subq_3.ds__quarter AS metric_time__quarter - , subq_3.ds__year AS metric_time__year - , subq_3.ds__extract_year AS metric_time__extract_year - , subq_3.ds__extract_quarter AS metric_time__extract_quarter - , subq_3.ds__extract_month AS metric_time__extract_month - , subq_3.ds__extract_week AS metric_time__extract_week - , subq_3.ds__extract_day AS metric_time__extract_day - , subq_3.ds__extract_dow AS metric_time__extract_dow - , subq_3.ds__extract_doy AS metric_time__extract_doy - , subq_3.listing - , subq_3.user - , subq_3.listing__user - , subq_3.country_latest - , subq_3.is_lux_latest - , subq_3.capacity_latest - , subq_3.listing__country_latest - , subq_3.listing__is_lux_latest - , subq_3.listing__capacity_latest - , subq_3.listings - , subq_3.largest_listing - , subq_3.smallest_listing - FROM ( - -- Read Elements From Semantic Model 'listings_latest' - SELECT - 1 AS listings - , listings_latest_src_10004.capacity AS largest_listing - , listings_latest_src_10004.capacity AS smallest_listing - , DATE_TRUNC('day', listings_latest_src_10004.created_at) AS ds__day - , DATE_TRUNC('week', listings_latest_src_10004.created_at) AS ds__week - , DATE_TRUNC('month', listings_latest_src_10004.created_at) AS ds__month - , DATE_TRUNC('quarter', listings_latest_src_10004.created_at) AS ds__quarter - , DATE_TRUNC('year', listings_latest_src_10004.created_at) AS ds__year - , EXTRACT(year FROM listings_latest_src_10004.created_at) AS ds__extract_year - , EXTRACT(quarter FROM listings_latest_src_10004.created_at) AS ds__extract_quarter - , EXTRACT(month FROM listings_latest_src_10004.created_at) AS ds__extract_month - , EXTRACT(week FROM listings_latest_src_10004.created_at) AS ds__extract_week - , EXTRACT(day FROM listings_latest_src_10004.created_at) AS ds__extract_day - , EXTRACT(dow FROM listings_latest_src_10004.created_at) AS ds__extract_dow - , EXTRACT(doy FROM listings_latest_src_10004.created_at) AS ds__extract_doy - , DATE_TRUNC('day', listings_latest_src_10004.created_at) AS created_at__day - , DATE_TRUNC('week', listings_latest_src_10004.created_at) AS created_at__week - , DATE_TRUNC('month', listings_latest_src_10004.created_at) AS created_at__month - , DATE_TRUNC('quarter', listings_latest_src_10004.created_at) AS created_at__quarter - , DATE_TRUNC('year', listings_latest_src_10004.created_at) AS created_at__year - , EXTRACT(year FROM listings_latest_src_10004.created_at) AS created_at__extract_year - , EXTRACT(quarter FROM listings_latest_src_10004.created_at) AS created_at__extract_quarter - , EXTRACT(month FROM listings_latest_src_10004.created_at) AS created_at__extract_month - , EXTRACT(week FROM listings_latest_src_10004.created_at) AS created_at__extract_week - , EXTRACT(day FROM listings_latest_src_10004.created_at) AS created_at__extract_day - , EXTRACT(dow FROM listings_latest_src_10004.created_at) AS created_at__extract_dow - , EXTRACT(doy FROM listings_latest_src_10004.created_at) AS created_at__extract_doy - , listings_latest_src_10004.country AS country_latest - , listings_latest_src_10004.is_lux AS is_lux_latest - , listings_latest_src_10004.capacity AS capacity_latest - , DATE_TRUNC('day', listings_latest_src_10004.created_at) AS listing__ds__day - , DATE_TRUNC('week', listings_latest_src_10004.created_at) AS listing__ds__week - , DATE_TRUNC('month', listings_latest_src_10004.created_at) AS listing__ds__month - , DATE_TRUNC('quarter', listings_latest_src_10004.created_at) AS listing__ds__quarter - , DATE_TRUNC('year', listings_latest_src_10004.created_at) AS listing__ds__year - , EXTRACT(year FROM listings_latest_src_10004.created_at) AS listing__ds__extract_year - , EXTRACT(quarter FROM listings_latest_src_10004.created_at) AS listing__ds__extract_quarter - , EXTRACT(month FROM listings_latest_src_10004.created_at) AS listing__ds__extract_month - , EXTRACT(week FROM listings_latest_src_10004.created_at) AS listing__ds__extract_week - , EXTRACT(day FROM listings_latest_src_10004.created_at) AS listing__ds__extract_day - , EXTRACT(dow FROM listings_latest_src_10004.created_at) AS listing__ds__extract_dow - , EXTRACT(doy FROM listings_latest_src_10004.created_at) AS listing__ds__extract_doy - , DATE_TRUNC('day', listings_latest_src_10004.created_at) AS listing__created_at__day - , DATE_TRUNC('week', listings_latest_src_10004.created_at) AS listing__created_at__week - , DATE_TRUNC('month', listings_latest_src_10004.created_at) AS listing__created_at__month - , DATE_TRUNC('quarter', listings_latest_src_10004.created_at) AS listing__created_at__quarter - , DATE_TRUNC('year', listings_latest_src_10004.created_at) AS listing__created_at__year - , EXTRACT(year FROM listings_latest_src_10004.created_at) AS listing__created_at__extract_year - , EXTRACT(quarter FROM listings_latest_src_10004.created_at) AS listing__created_at__extract_quarter - , EXTRACT(month FROM listings_latest_src_10004.created_at) AS listing__created_at__extract_month - , EXTRACT(week FROM listings_latest_src_10004.created_at) AS listing__created_at__extract_week - , EXTRACT(day FROM listings_latest_src_10004.created_at) AS listing__created_at__extract_day - , EXTRACT(dow FROM listings_latest_src_10004.created_at) AS listing__created_at__extract_dow - , EXTRACT(doy FROM listings_latest_src_10004.created_at) AS listing__created_at__extract_doy - , listings_latest_src_10004.country AS listing__country_latest - , listings_latest_src_10004.is_lux AS listing__is_lux_latest - , listings_latest_src_10004.capacity AS listing__capacity_latest - , listings_latest_src_10004.listing_id AS listing - , listings_latest_src_10004.user_id AS user - , listings_latest_src_10004.user_id AS listing__user - FROM ***************************.dim_listings_latest listings_latest_src_10004 - ) subq_3 - ) subq_4 - ) subq_5 - ON - subq_2.listing = subq_5.listing - ) subq_6 - ) subq_7 - GROUP BY - subq_7.listing__country_latest - ) subq_8 - ) subq_9 -) subq_10 -ORDER BY subq_10.listing__country_latest + 1 AS listings + , listings_latest_src_10004.capacity AS largest_listing + , listings_latest_src_10004.capacity AS smallest_listing + , DATE_TRUNC('day', listings_latest_src_10004.created_at) AS ds__day + , DATE_TRUNC('week', listings_latest_src_10004.created_at) AS ds__week + , DATE_TRUNC('month', listings_latest_src_10004.created_at) AS ds__month + , DATE_TRUNC('quarter', listings_latest_src_10004.created_at) AS ds__quarter + , DATE_TRUNC('year', listings_latest_src_10004.created_at) AS ds__year + , EXTRACT(year FROM listings_latest_src_10004.created_at) AS ds__extract_year + , EXTRACT(quarter FROM listings_latest_src_10004.created_at) AS ds__extract_quarter + , EXTRACT(month FROM listings_latest_src_10004.created_at) AS ds__extract_month + , EXTRACT(week FROM listings_latest_src_10004.created_at) AS ds__extract_week + , EXTRACT(day FROM listings_latest_src_10004.created_at) AS ds__extract_day + , EXTRACT(dow FROM listings_latest_src_10004.created_at) AS ds__extract_dow + , EXTRACT(doy FROM listings_latest_src_10004.created_at) AS ds__extract_doy + , DATE_TRUNC('day', listings_latest_src_10004.created_at) AS created_at__day + , DATE_TRUNC('week', listings_latest_src_10004.created_at) AS created_at__week + , DATE_TRUNC('month', listings_latest_src_10004.created_at) AS created_at__month + , DATE_TRUNC('quarter', listings_latest_src_10004.created_at) AS created_at__quarter + , DATE_TRUNC('year', listings_latest_src_10004.created_at) AS created_at__year + , EXTRACT(year FROM listings_latest_src_10004.created_at) AS created_at__extract_year + , EXTRACT(quarter FROM listings_latest_src_10004.created_at) AS created_at__extract_quarter + , EXTRACT(month FROM listings_latest_src_10004.created_at) AS created_at__extract_month + , EXTRACT(week FROM listings_latest_src_10004.created_at) AS created_at__extract_week + , EXTRACT(day FROM listings_latest_src_10004.created_at) AS created_at__extract_day + , EXTRACT(dow FROM listings_latest_src_10004.created_at) AS created_at__extract_dow + , EXTRACT(doy FROM listings_latest_src_10004.created_at) AS created_at__extract_doy + , listings_latest_src_10004.country AS country_latest + , listings_latest_src_10004.is_lux AS is_lux_latest + , listings_latest_src_10004.capacity AS capacity_latest + , DATE_TRUNC('day', listings_latest_src_10004.created_at) AS listing__ds__day + , DATE_TRUNC('week', listings_latest_src_10004.created_at) AS listing__ds__week + , DATE_TRUNC('month', listings_latest_src_10004.created_at) AS listing__ds__month + , DATE_TRUNC('quarter', listings_latest_src_10004.created_at) AS listing__ds__quarter + , DATE_TRUNC('year', listings_latest_src_10004.created_at) AS listing__ds__year + , EXTRACT(year FROM listings_latest_src_10004.created_at) AS listing__ds__extract_year + , EXTRACT(quarter FROM listings_latest_src_10004.created_at) AS listing__ds__extract_quarter + , EXTRACT(month FROM listings_latest_src_10004.created_at) AS listing__ds__extract_month + , EXTRACT(week FROM listings_latest_src_10004.created_at) AS listing__ds__extract_week + , EXTRACT(day FROM listings_latest_src_10004.created_at) AS listing__ds__extract_day + , EXTRACT(dow FROM listings_latest_src_10004.created_at) AS listing__ds__extract_dow + , EXTRACT(doy FROM listings_latest_src_10004.created_at) AS listing__ds__extract_doy + , DATE_TRUNC('day', listings_latest_src_10004.created_at) AS listing__created_at__day + , DATE_TRUNC('week', listings_latest_src_10004.created_at) AS listing__created_at__week + , DATE_TRUNC('month', listings_latest_src_10004.created_at) AS listing__created_at__month + , DATE_TRUNC('quarter', listings_latest_src_10004.created_at) AS listing__created_at__quarter + , DATE_TRUNC('year', listings_latest_src_10004.created_at) AS listing__created_at__year + , EXTRACT(year FROM listings_latest_src_10004.created_at) AS listing__created_at__extract_year + , EXTRACT(quarter FROM listings_latest_src_10004.created_at) AS listing__created_at__extract_quarter + , EXTRACT(month FROM listings_latest_src_10004.created_at) AS listing__created_at__extract_month + , EXTRACT(week FROM listings_latest_src_10004.created_at) AS listing__created_at__extract_week + , EXTRACT(day FROM listings_latest_src_10004.created_at) AS listing__created_at__extract_day + , EXTRACT(dow FROM listings_latest_src_10004.created_at) AS listing__created_at__extract_dow + , EXTRACT(doy FROM listings_latest_src_10004.created_at) AS listing__created_at__extract_doy + , listings_latest_src_10004.country AS listing__country_latest + , listings_latest_src_10004.is_lux AS listing__is_lux_latest + , listings_latest_src_10004.capacity AS listing__capacity_latest + , listings_latest_src_10004.listing_id AS listing + , listings_latest_src_10004.user_id AS user + , listings_latest_src_10004.user_id AS listing__user + FROM ***************************.dim_listings_latest listings_latest_src_10004 + ) subq_0 + GROUP BY + subq_0.listing__country_latest + ) subq_1 + WHERE listing__country_latest = 'us' +) subq_2 +ORDER BY subq_2.listing__country_latest DESC LIMIT 100 diff --git a/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Snowflake/test_distinct_values__plan0_optimized.sql b/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Snowflake/test_distinct_values__plan0_optimized.sql index 328d6a2d79..592773c228 100644 --- a/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Snowflake/test_distinct_values__plan0_optimized.sql +++ b/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Snowflake/test_distinct_values__plan0_optimized.sql @@ -1,19 +1,17 @@ --- Join Standard Outputs --- Pass Only Elements: --- ['bookings', 'listing__country_latest'] --- Aggregate Measures --- Compute Metrics via Expressions --- Pass Only Elements: --- ['listing__country_latest'] +-- Constrain Output with WHERE -- Order By ['listing__country_latest'] Limit 100 SELECT - listings_latest_src_10004.country AS listing__country_latest -FROM ***************************.fct_bookings bookings_source_src_10001 -LEFT OUTER JOIN - ***************************.dim_listings_latest listings_latest_src_10004 -ON - bookings_source_src_10001.listing_id = listings_latest_src_10004.listing_id -GROUP BY - listings_latest_src_10004.country -ORDER BY listing__country_latest + listing__country_latest +FROM ( + -- Read Elements From Semantic Model 'listings_latest' + -- Pass Only Elements: + -- ['listing__country_latest'] + SELECT + country AS listing__country_latest + FROM ***************************.dim_listings_latest listings_latest_src_10004 + GROUP BY + country +) subq_4 +WHERE listing__country_latest = 'us' +ORDER BY listing__country_latest DESC LIMIT 100 diff --git a/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/test_distinct_values__plan0.xml b/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/test_distinct_values__plan0.xml index ac358b0579..4186834688 100644 --- a/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/test_distinct_values__plan0.xml +++ b/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/test_distinct_values__plan0.xml @@ -1,1488 +1,292 @@ - - - - - - + + + + + + - - - - + + + + - - - - - - - - - - + + + + + + + + + - - - - - - - - - - - + + + + + + + + + + + + + - - - + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + - - - - - - - - - - - + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - + + + + + diff --git a/metricflow/test/snapshots/test_source_scan_optimizer.py/DataflowPlan/test_2_metrics_from_1_semantic_model__dfp_0.xml b/metricflow/test/snapshots/test_source_scan_optimizer.py/DataflowPlan/test_2_metrics_from_1_semantic_model__dfp_0.xml index fea487abac..ff9c8b4707 100644 --- a/metricflow/test/snapshots/test_source_scan_optimizer.py/DataflowPlan/test_2_metrics_from_1_semantic_model__dfp_0.xml +++ b/metricflow/test/snapshots/test_source_scan_optimizer.py/DataflowPlan/test_2_metrics_from_1_semantic_model__dfp_0.xml @@ -40,6 +40,7 @@ + @@ -69,6 +70,7 @@ + @@ -90,6 +92,7 @@ + @@ -140,6 +143,7 @@ + @@ -169,6 +173,7 @@ + @@ -190,6 +195,7 @@ + diff --git a/metricflow/test/snapshots/test_source_scan_optimizer.py/DataflowPlan/test_2_metrics_from_1_semantic_model__dfpo_0.xml b/metricflow/test/snapshots/test_source_scan_optimizer.py/DataflowPlan/test_2_metrics_from_1_semantic_model__dfpo_0.xml index 3b32bfb08a..878492035e 100644 --- a/metricflow/test/snapshots/test_source_scan_optimizer.py/DataflowPlan/test_2_metrics_from_1_semantic_model__dfpo_0.xml +++ b/metricflow/test/snapshots/test_source_scan_optimizer.py/DataflowPlan/test_2_metrics_from_1_semantic_model__dfpo_0.xml @@ -46,6 +46,7 @@ + @@ -79,6 +80,7 @@ + @@ -100,6 +102,7 @@ + diff --git a/metricflow/test/snapshots/test_source_scan_optimizer.py/DataflowPlan/test_2_metrics_from_2_semantic_models__dfp_0.xml b/metricflow/test/snapshots/test_source_scan_optimizer.py/DataflowPlan/test_2_metrics_from_2_semantic_models__dfp_0.xml index e77cb19195..cb3913a1e9 100644 --- a/metricflow/test/snapshots/test_source_scan_optimizer.py/DataflowPlan/test_2_metrics_from_2_semantic_models__dfp_0.xml +++ b/metricflow/test/snapshots/test_source_scan_optimizer.py/DataflowPlan/test_2_metrics_from_2_semantic_models__dfp_0.xml @@ -36,6 +36,7 @@ + @@ -80,6 +81,7 @@ + diff --git a/metricflow/test/snapshots/test_source_scan_optimizer.py/DataflowPlan/test_2_metrics_from_2_semantic_models__dfpo_0.xml b/metricflow/test/snapshots/test_source_scan_optimizer.py/DataflowPlan/test_2_metrics_from_2_semantic_models__dfpo_0.xml index d3ee23ea35..29f009e284 100644 --- a/metricflow/test/snapshots/test_source_scan_optimizer.py/DataflowPlan/test_2_metrics_from_2_semantic_models__dfpo_0.xml +++ b/metricflow/test/snapshots/test_source_scan_optimizer.py/DataflowPlan/test_2_metrics_from_2_semantic_models__dfpo_0.xml @@ -36,6 +36,7 @@ + @@ -80,6 +81,7 @@ + diff --git a/metricflow/test/snapshots/test_source_scan_optimizer.py/DataflowPlan/test_2_ratio_metrics_from_1_semantic_model__dfp_0.xml b/metricflow/test/snapshots/test_source_scan_optimizer.py/DataflowPlan/test_2_ratio_metrics_from_1_semantic_model__dfp_0.xml index 01ba8a68c5..59b7d47c75 100644 --- a/metricflow/test/snapshots/test_source_scan_optimizer.py/DataflowPlan/test_2_ratio_metrics_from_1_semantic_model__dfp_0.xml +++ b/metricflow/test/snapshots/test_source_scan_optimizer.py/DataflowPlan/test_2_ratio_metrics_from_1_semantic_model__dfp_0.xml @@ -50,6 +50,7 @@ + @@ -94,6 +95,7 @@ + @@ -154,6 +156,7 @@ + @@ -198,6 +201,7 @@ + diff --git a/metricflow/test/snapshots/test_source_scan_optimizer.py/DataflowPlan/test_2_ratio_metrics_from_1_semantic_model__dfpo_0.xml b/metricflow/test/snapshots/test_source_scan_optimizer.py/DataflowPlan/test_2_ratio_metrics_from_1_semantic_model__dfpo_0.xml index defb580aff..443bab0657 100644 --- a/metricflow/test/snapshots/test_source_scan_optimizer.py/DataflowPlan/test_2_ratio_metrics_from_1_semantic_model__dfpo_0.xml +++ b/metricflow/test/snapshots/test_source_scan_optimizer.py/DataflowPlan/test_2_ratio_metrics_from_1_semantic_model__dfpo_0.xml @@ -77,6 +77,7 @@ + diff --git a/metricflow/test/snapshots/test_source_scan_optimizer.py/DataflowPlan/test_3_metrics_from_2_semantic_models__dfp_0.xml b/metricflow/test/snapshots/test_source_scan_optimizer.py/DataflowPlan/test_3_metrics_from_2_semantic_models__dfp_0.xml index 59589a8c8a..af1adfb56f 100644 --- a/metricflow/test/snapshots/test_source_scan_optimizer.py/DataflowPlan/test_3_metrics_from_2_semantic_models__dfp_0.xml +++ b/metricflow/test/snapshots/test_source_scan_optimizer.py/DataflowPlan/test_3_metrics_from_2_semantic_models__dfp_0.xml @@ -36,6 +36,7 @@ + @@ -80,6 +81,7 @@ + @@ -124,6 +126,7 @@ + diff --git a/metricflow/test/snapshots/test_source_scan_optimizer.py/DataflowPlan/test_3_metrics_from_2_semantic_models__dfpo_0.xml b/metricflow/test/snapshots/test_source_scan_optimizer.py/DataflowPlan/test_3_metrics_from_2_semantic_models__dfpo_0.xml index 7b4e7ec6e8..1536e6c8f9 100644 --- a/metricflow/test/snapshots/test_source_scan_optimizer.py/DataflowPlan/test_3_metrics_from_2_semantic_models__dfpo_0.xml +++ b/metricflow/test/snapshots/test_source_scan_optimizer.py/DataflowPlan/test_3_metrics_from_2_semantic_models__dfpo_0.xml @@ -47,6 +47,7 @@ + @@ -91,6 +92,7 @@ + diff --git a/metricflow/test/snapshots/test_source_scan_optimizer.py/DataflowPlan/test_constrained_metric_not_combined__dfp_0.xml b/metricflow/test/snapshots/test_source_scan_optimizer.py/DataflowPlan/test_constrained_metric_not_combined__dfp_0.xml index 9003e62d30..ce2766cc9f 100644 --- a/metricflow/test/snapshots/test_source_scan_optimizer.py/DataflowPlan/test_constrained_metric_not_combined__dfp_0.xml +++ b/metricflow/test/snapshots/test_source_scan_optimizer.py/DataflowPlan/test_constrained_metric_not_combined__dfp_0.xml @@ -36,6 +36,7 @@ + @@ -90,6 +91,7 @@ + @@ -124,6 +126,7 @@ + diff --git a/metricflow/test/snapshots/test_source_scan_optimizer.py/DataflowPlan/test_constrained_metric_not_combined__dfpo_0.xml b/metricflow/test/snapshots/test_source_scan_optimizer.py/DataflowPlan/test_constrained_metric_not_combined__dfpo_0.xml index eeaa437cc5..aa11bdf2c4 100644 --- a/metricflow/test/snapshots/test_source_scan_optimizer.py/DataflowPlan/test_constrained_metric_not_combined__dfpo_0.xml +++ b/metricflow/test/snapshots/test_source_scan_optimizer.py/DataflowPlan/test_constrained_metric_not_combined__dfpo_0.xml @@ -36,6 +36,7 @@ + @@ -90,6 +91,7 @@ + @@ -124,6 +126,7 @@ + diff --git a/metricflow/test/snapshots/test_source_scan_optimizer.py/DataflowPlan/test_derived_metric__dfp_0.xml b/metricflow/test/snapshots/test_source_scan_optimizer.py/DataflowPlan/test_derived_metric__dfp_0.xml index 8d663c8802..4d72662222 100644 --- a/metricflow/test/snapshots/test_source_scan_optimizer.py/DataflowPlan/test_derived_metric__dfp_0.xml +++ b/metricflow/test/snapshots/test_source_scan_optimizer.py/DataflowPlan/test_derived_metric__dfp_0.xml @@ -45,6 +45,7 @@ + @@ -89,6 +90,7 @@ + diff --git a/metricflow/test/snapshots/test_source_scan_optimizer.py/DataflowPlan/test_derived_metric__dfpo_0.xml b/metricflow/test/snapshots/test_source_scan_optimizer.py/DataflowPlan/test_derived_metric__dfpo_0.xml index 97636fa2d4..aa3dfef91a 100644 --- a/metricflow/test/snapshots/test_source_scan_optimizer.py/DataflowPlan/test_derived_metric__dfpo_0.xml +++ b/metricflow/test/snapshots/test_source_scan_optimizer.py/DataflowPlan/test_derived_metric__dfpo_0.xml @@ -52,6 +52,7 @@ + diff --git a/metricflow/test/snapshots/test_source_scan_optimizer.py/DataflowPlan/test_derived_metric_with_non_derived_metric__dfp_0.xml b/metricflow/test/snapshots/test_source_scan_optimizer.py/DataflowPlan/test_derived_metric_with_non_derived_metric__dfp_0.xml index 2a8cc7308e..a26d3cb22b 100644 --- a/metricflow/test/snapshots/test_source_scan_optimizer.py/DataflowPlan/test_derived_metric_with_non_derived_metric__dfp_0.xml +++ b/metricflow/test/snapshots/test_source_scan_optimizer.py/DataflowPlan/test_derived_metric_with_non_derived_metric__dfp_0.xml @@ -36,6 +36,7 @@ + @@ -94,6 +95,7 @@ + @@ -138,6 +140,7 @@ + diff --git a/metricflow/test/snapshots/test_source_scan_optimizer.py/DataflowPlan/test_derived_metric_with_non_derived_metric__dfpo_0.xml b/metricflow/test/snapshots/test_source_scan_optimizer.py/DataflowPlan/test_derived_metric_with_non_derived_metric__dfpo_0.xml index 8286ae3ddb..b23dd5f90b 100644 --- a/metricflow/test/snapshots/test_source_scan_optimizer.py/DataflowPlan/test_derived_metric_with_non_derived_metric__dfpo_0.xml +++ b/metricflow/test/snapshots/test_source_scan_optimizer.py/DataflowPlan/test_derived_metric_with_non_derived_metric__dfpo_0.xml @@ -36,6 +36,7 @@ + @@ -101,6 +102,7 @@ + diff --git a/metricflow/test/snapshots/test_source_scan_optimizer.py/DataflowPlan/test_nested_derived_metric__dfp_0.xml b/metricflow/test/snapshots/test_source_scan_optimizer.py/DataflowPlan/test_nested_derived_metric__dfp_0.xml index a7b7790b04..5af98e9f24 100644 --- a/metricflow/test/snapshots/test_source_scan_optimizer.py/DataflowPlan/test_nested_derived_metric__dfp_0.xml +++ b/metricflow/test/snapshots/test_source_scan_optimizer.py/DataflowPlan/test_nested_derived_metric__dfp_0.xml @@ -59,6 +59,7 @@ + @@ -103,6 +104,7 @@ + @@ -149,6 +151,7 @@ + @@ -193,6 +196,7 @@ + diff --git a/metricflow/test/snapshots/test_source_scan_optimizer.py/DataflowPlan/test_nested_derived_metric__dfpo_0.xml b/metricflow/test/snapshots/test_source_scan_optimizer.py/DataflowPlan/test_nested_derived_metric__dfpo_0.xml index 74c85eef7a..dcccd29791 100644 --- a/metricflow/test/snapshots/test_source_scan_optimizer.py/DataflowPlan/test_nested_derived_metric__dfpo_0.xml +++ b/metricflow/test/snapshots/test_source_scan_optimizer.py/DataflowPlan/test_nested_derived_metric__dfpo_0.xml @@ -66,6 +66,7 @@ + @@ -122,6 +123,7 @@ + From 20cc790208af38a443b1d6c82087d777cc1f4145 Mon Sep 17 00:00:00 2001 From: Courtney Holcomb Date: Wed, 11 Oct 2023 11:18:02 -0700 Subject: [PATCH 09/15] Test fix: don't use alias in where clause --- metricflow/test/integration/test_cases/itest_dimensions.yaml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/metricflow/test/integration/test_cases/itest_dimensions.yaml b/metricflow/test/integration/test_cases/itest_dimensions.yaml index 3fe4405733..225de32521 100644 --- a/metricflow/test/integration/test_cases/itest_dimensions.yaml +++ b/metricflow/test/integration/test_cases/itest_dimensions.yaml @@ -179,6 +179,6 @@ integration_test: SELECT u.home_state AS user__home_state FROM {{ source_schema }}.dim_users u - WHERE user__home_state = 'CA' + WHERE u.home_state = 'CA' GROUP BY u.home_state From d350fc68f6ec0305f0a18e6dac2b13c1faaabcf7 Mon Sep 17 00:00:00 2001 From: Courtney Holcomb Date: Thu, 12 Oct 2023 12:52:56 -0700 Subject: [PATCH 10/15] Stronger typing for read_nodes --- metricflow/dataflow/builder/dataflow_plan_builder.py | 5 +++-- metricflow/dataflow/builder/source_node.py | 4 +++- metricflow/query/query_parser.py | 4 ++-- metricflow/time/time_granularity_solver.py | 8 ++++---- 4 files changed, 12 insertions(+), 9 deletions(-) diff --git a/metricflow/dataflow/builder/dataflow_plan_builder.py b/metricflow/dataflow/builder/dataflow_plan_builder.py index b62f54ae0c..5551365172 100644 --- a/metricflow/dataflow/builder/dataflow_plan_builder.py +++ b/metricflow/dataflow/builder/dataflow_plan_builder.py @@ -37,6 +37,7 @@ JoinToBaseOutputNode, JoinToTimeSpineNode, OrderByLimitNode, + ReadSqlSourceNode, SemiAdditiveJoinNode, SinkOutput, WhereConstraintNode, @@ -144,7 +145,7 @@ class DataflowPlanBuilder: def __init__( # noqa: D self, source_nodes: Sequence[BaseOutput], - read_nodes: Sequence[BaseOutput], + read_nodes: Sequence[ReadSqlSourceNode], semantic_manifest_lookup: SemanticManifestLookup, cost_function: DataflowPlanNodeCostFunction = DefaultCostFunction(), node_output_resolver: Optional[DataflowPlanNodeOutputDataSetResolver] = None, @@ -418,7 +419,7 @@ def _select_source_nodes_with_measures( return nodes def _select_read_nodes_with_linkable_specs( - self, linkable_specs: LinkableSpecSet, read_nodes: Sequence[BaseOutput] + self, linkable_specs: LinkableSpecSet, read_nodes: Sequence[ReadSqlSourceNode] ) -> Dict[BaseOutput, Set[LinkableInstanceSpec]]: """Find source nodes with requested linkable specs and no measures.""" nodes_to_linkable_specs: Dict[BaseOutput, Set[LinkableInstanceSpec]] = {} diff --git a/metricflow/dataflow/builder/source_node.py b/metricflow/dataflow/builder/source_node.py index b77c2ac69f..225ee2fb9f 100644 --- a/metricflow/dataflow/builder/source_node.py +++ b/metricflow/dataflow/builder/source_node.py @@ -47,6 +47,8 @@ def create_from_data_sets(self, data_sets: Sequence[SemanticModelDataSet]) -> Se ) return source_nodes - def create_read_nodes_from_data_sets(self, data_sets: Sequence[SemanticModelDataSet]) -> Sequence[BaseOutput]: + def create_read_nodes_from_data_sets( + self, data_sets: Sequence[SemanticModelDataSet] + ) -> Sequence[ReadSqlSourceNode]: """Creates read nodes from SemanticModelDataSets.""" return [ReadSqlSourceNode(data_set) for data_set in data_sets] diff --git a/metricflow/query/query_parser.py b/metricflow/query/query_parser.py index e70a782880..0589beaf6b 100644 --- a/metricflow/query/query_parser.py +++ b/metricflow/query/query_parser.py @@ -22,7 +22,7 @@ from metricflow.assert_one_arg import assert_exactly_one_arg_set from metricflow.dataflow.builder.node_data_set import DataflowPlanNodeOutputDataSetResolver -from metricflow.dataflow.dataflow_plan import BaseOutput +from metricflow.dataflow.dataflow_plan import ReadSqlSourceNode from metricflow.dataset.dataset import DataSet from metricflow.errors.errors import UnableToSatisfyQueryError from metricflow.filters.time_constraint import TimeRangeConstraint @@ -118,7 +118,7 @@ def __init__( # noqa: D self, column_association_resolver: ColumnAssociationResolver, model: SemanticManifestLookup, - read_nodes: Sequence[BaseOutput], + read_nodes: Sequence[ReadSqlSourceNode], node_output_resolver: DataflowPlanNodeOutputDataSetResolver, ) -> None: self._column_association_resolver = column_association_resolver diff --git a/metricflow/time/time_granularity_solver.py b/metricflow/time/time_granularity_solver.py index 1a3bbe62d4..2227a78c4f 100644 --- a/metricflow/time/time_granularity_solver.py +++ b/metricflow/time/time_granularity_solver.py @@ -15,7 +15,7 @@ from dbt_semantic_interfaces.type_enums.time_granularity import TimeGranularity from metricflow.dataflow.builder.node_data_set import DataflowPlanNodeOutputDataSetResolver -from metricflow.dataflow.dataflow_plan import BaseOutput +from metricflow.dataflow.dataflow_plan import ReadSqlSourceNode from metricflow.filters.time_constraint import TimeRangeConstraint from metricflow.model.semantic_manifest_lookup import SemanticManifestLookup from metricflow.naming.linkable_spec_name import StructuredLinkableSpecName @@ -103,7 +103,7 @@ def resolve_granularity_for_partial_time_dimension_specs( self, metric_references: Sequence[MetricReference], partial_time_dimension_specs: Sequence[PartialTimeDimensionSpec], - read_nodes: Sequence[BaseOutput], + read_nodes: Sequence[ReadSqlSourceNode], node_output_resolver: DataflowPlanNodeOutputDataSetResolver, ) -> Dict[PartialTimeDimensionSpec, TimeDimensionSpec]: """Figure out the lowest granularity possible for the partially specified time dimension specs. @@ -131,7 +131,7 @@ def find_minimum_granularity_for_partial_time_dimension_spec( self, partial_time_dimension_spec: PartialTimeDimensionSpec, metric_references: Sequence[MetricReference], - read_nodes: Sequence[BaseOutput], + read_nodes: Sequence[ReadSqlSourceNode], node_output_resolver: DataflowPlanNodeOutputDataSetResolver, ) -> TimeGranularity: """Find minimum granularity allowed for time dimension when queried with given metrics.""" @@ -173,7 +173,7 @@ def find_minimum_granularity_for_partial_time_dimension_spec( def get_min_granularity_for_partial_time_dimension_without_metrics( self, - read_nodes: Sequence[BaseOutput], + read_nodes: Sequence[ReadSqlSourceNode], node_output_resolver: DataflowPlanNodeOutputDataSetResolver, partial_time_dimension_spec: PartialTimeDimensionSpec, ) -> Optional[TimeGranularity]: From 9b200a0e034c10fa9cce396cf4566ea9fe1edfcb Mon Sep 17 00:00:00 2001 From: Courtney Holcomb Date: Thu, 12 Oct 2023 12:56:08 -0700 Subject: [PATCH 11/15] Add back check for missing measure specs --- .../dataflow/builder/dataflow_plan_builder.py | 14 ++++++++++++++ 1 file changed, 14 insertions(+) diff --git a/metricflow/dataflow/builder/dataflow_plan_builder.py b/metricflow/dataflow/builder/dataflow_plan_builder.py index 5551365172..b25626106f 100644 --- a/metricflow/dataflow/builder/dataflow_plan_builder.py +++ b/metricflow/dataflow/builder/dataflow_plan_builder.py @@ -547,6 +547,20 @@ def _find_dataflow_recipe( node_to_evaluation: Dict[BaseOutput, LinkableInstanceSatisfiabilityEvaluation] = {} for node in self._sort_by_suitability(potential_source_nodes): + data_set = self._node_data_set_resolver.get_output_data_set(node) + + if measure_spec_properties: + measure_specs = measure_spec_properties.measure_specs + missing_specs = [ + spec for spec in measure_specs if spec not in data_set.instance_set.spec_set.measure_specs + ] + if missing_specs: + logger.debug( + f"Skipping evaluation for node since it does not have all of the measure specs {missing_specs}:" + f"\n\n{dataflow_dag_as_text(node)}" + ) + continue + logger.debug(f"Evaluating source node:\n{pformat_big_objects(source_node=dataflow_dag_as_text(node))}") start_time = time.time() From 56332d1c02852ad4a8490eee0da0511ee65cf473 Mon Sep 17 00:00:00 2001 From: Courtney Holcomb Date: Thu, 12 Oct 2023 13:08:05 -0700 Subject: [PATCH 12/15] Take min of all supported granularities for a time dim --- .../test/integration/test_cases/itest_dimensions.yaml | 4 ++-- metricflow/time/time_granularity_solver.py | 8 +++++--- 2 files changed, 7 insertions(+), 5 deletions(-) diff --git a/metricflow/test/integration/test_cases/itest_dimensions.yaml b/metricflow/test/integration/test_cases/itest_dimensions.yaml index 225de32521..ed050c2233 100644 --- a/metricflow/test/integration/test_cases/itest_dimensions.yaml +++ b/metricflow/test/integration/test_cases/itest_dimensions.yaml @@ -161,10 +161,10 @@ integration_test: name: query_non_default_time_dimension_without_granularity description: Query just a time dimension, no granularity specified. Should assume default granularity for dimension. model: EXTENDED_DATE_MODEL - group_bys: [ "monthly_ds"] + group_bys: [ "booking__monthly_ds"] check_query: | SELECT - ds AS monthly_ds__month + ds AS booking__monthly_ds__month FROM {{ source_schema }}.fct_bookings_extended_monthly GROUP BY ds diff --git a/metricflow/time/time_granularity_solver.py b/metricflow/time/time_granularity_solver.py index 2227a78c4f..994f1dcd68 100644 --- a/metricflow/time/time_granularity_solver.py +++ b/metricflow/time/time_granularity_solver.py @@ -2,7 +2,7 @@ import logging from dataclasses import dataclass -from typing import Dict, Optional, Sequence, Tuple +from typing import Dict, Optional, Sequence, Set, Tuple import pandas as pd from dbt_semantic_interfaces.pretty_print import pformat_big_objects @@ -184,6 +184,8 @@ def get_min_granularity_for_partial_time_dimension_without_metrics( ), element_name=partial_time_dimension_spec.element_name, ).granularity_free_qualified_name + + supported_granularities: Set[TimeGranularity] = set() for read_node in read_nodes: output_data_set = node_output_resolver.get_output_data_set(read_node) for time_dimension_instance in output_data_set.instance_set.time_dimension_instances: @@ -193,9 +195,9 @@ def get_min_granularity_for_partial_time_dimension_without_metrics( time_dimension_instance.spec.qualified_name ).granularity_free_qualified_name if time_dim_name_without_granularity == granularity_free_qualified_name: - return time_dimension_instance.spec.time_granularity + supported_granularities.add(time_dimension_instance.spec.time_granularity) - return None + return min(supported_granularities) if supported_granularities else None def adjust_time_range_to_granularity( self, time_range_constraint: TimeRangeConstraint, time_granularity: TimeGranularity From 61d23c6ea51b332d968d9666495c3c4dd7294706 Mon Sep 17 00:00:00 2001 From: Courtney Holcomb Date: Thu, 12 Oct 2023 13:10:04 -0700 Subject: [PATCH 13/15] Separate test case for invalid group by name --- metricflow/test/query/test_query_parser.py | 2 ++ 1 file changed, 2 insertions(+) diff --git a/metricflow/test/query/test_query_parser.py b/metricflow/test/query/test_query_parser.py index 4c2bc78680..03e5ab9ccb 100644 --- a/metricflow/test/query/test_query_parser.py +++ b/metricflow/test/query/test_query_parser.py @@ -249,6 +249,8 @@ def test_query_parser_case_insensitivity(bookings_query_parser: MetricFlowQueryP ), ) + +def test_query_parser_invalid_group_by(bookings_query_parser: MetricFlowQueryParser) -> None: # noqa: D with pytest.raises(UnableToSatisfyQueryError): bookings_query_parser.parse_and_validate_query(group_by_names=["random_stuff"]) From 4cea5b61ee1ee03b26117f2de6253f3268537288 Mon Sep 17 00:00:00 2001 From: Courtney Holcomb Date: Thu, 12 Oct 2023 13:23:03 -0700 Subject: [PATCH 14/15] Apply where filter before aggregating distinct values --- .../dataflow/builder/dataflow_plan_builder.py | 26 +- .../test_distinct_values_plan__dfp_0.xml | 52 ++-- ..._distinct_values_plan_with_join__dfp_0.xml | 66 ++-- .../BigQuery/test_distinct_values__plan0.sql | 73 ++++- .../test_distinct_values__plan0_optimized.sql | 10 +- .../DuckDB/test_distinct_values__plan0.sql | 73 ++++- .../test_distinct_values__plan0_optimized.sql | 10 +- .../Postgres/test_distinct_values__plan0.sql | 73 ++++- .../test_distinct_values__plan0_optimized.sql | 10 +- .../Redshift/test_distinct_values__plan0.sql | 73 ++++- .../test_distinct_values__plan0_optimized.sql | 10 +- .../Snowflake/test_distinct_values__plan0.sql | 73 ++++- .../test_distinct_values__plan0_optimized.sql | 10 +- .../test_distinct_values__plan0.xml | 282 ++++++++++++++++-- 14 files changed, 685 insertions(+), 156 deletions(-) diff --git a/metricflow/dataflow/builder/dataflow_plan_builder.py b/metricflow/dataflow/builder/dataflow_plan_builder.py index b25626106f..bfaa36a3cc 100644 --- a/metricflow/dataflow/builder/dataflow_plan_builder.py +++ b/metricflow/dataflow/builder/dataflow_plan_builder.py @@ -314,28 +314,26 @@ def build_plan_for_distinct_values(self, query_spec: MetricFlowQuerySpec) -> Dat if not dataflow_recipe: raise UnableToSatisfyQueryError(f"Recipe not found for linkable specs: {query_spec.linkable_specs}") - join_targets = dataflow_recipe.join_targets - if join_targets: - joined_node = JoinToBaseOutputNode(left_node=dataflow_recipe.source_node, join_targets=join_targets) - distinct_values_node = FilterElementsNode( - parent_node=joined_node, include_specs=query_spec.linkable_specs.as_spec_set, distinct=True - ) - else: - distinct_values_node = FilterElementsNode( - parent_node=dataflow_recipe.source_node, - include_specs=query_spec.linkable_specs.as_spec_set, - distinct=True, + joined_node: Optional[JoinToBaseOutputNode] = None + if dataflow_recipe.join_targets: + joined_node = JoinToBaseOutputNode( + left_node=dataflow_recipe.source_node, join_targets=dataflow_recipe.join_targets ) where_constraint_node: Optional[WhereConstraintNode] = None if query_spec.where_constraint: where_constraint_node = WhereConstraintNode( - parent_node=distinct_values_node, - where_constraint=query_spec.where_constraint, + parent_node=joined_node or dataflow_recipe.source_node, where_constraint=query_spec.where_constraint ) + distinct_values_node = FilterElementsNode( + parent_node=where_constraint_node or joined_node or dataflow_recipe.source_node, + include_specs=query_spec.linkable_specs.as_spec_set, + distinct=True, + ) + sink_node = self.build_sink_node( - parent_node=where_constraint_node or distinct_values_node, + parent_node=distinct_values_node, order_by_specs=query_spec.order_by_specs, limit=query_spec.limit, ) diff --git a/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_distinct_values_plan__dfp_0.xml b/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_distinct_values_plan__dfp_0.xml index 793a1a5506..f222418611 100644 --- a/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_distinct_values_plan__dfp_0.xml +++ b/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_distinct_values_plan__dfp_0.xml @@ -13,30 +13,30 @@ - - - - - - - - - - - - - - - - - - - - - - - - + + + + + + + + + + + + + + + + + + + + + + + + @@ -44,8 +44,8 @@ - - + + diff --git a/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_distinct_values_plan_with_join__dfp_0.xml b/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_distinct_values_plan_with_join__dfp_0.xml index 620c81421f..44139d9477 100644 --- a/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_distinct_values_plan_with_join__dfp_0.xml +++ b/metricflow/test/snapshots/test_dataflow_plan_builder.py/DataflowPlan/test_distinct_values_plan_with_join__dfp_0.xml @@ -13,40 +13,40 @@ - - - - - - - - - - - - - - - - - - - - - - - - - - - - + + + + + + + + + + + + + + + + + + + + + + + + + + + + - + - + @@ -64,7 +64,7 @@ - + @@ -80,8 +80,8 @@ - - + + diff --git a/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/BigQuery/test_distinct_values__plan0.sql b/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/BigQuery/test_distinct_values__plan0.sql index aee1a40a63..b9e1ed19a6 100644 --- a/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/BigQuery/test_distinct_values__plan0.sql +++ b/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/BigQuery/test_distinct_values__plan0.sql @@ -2,14 +2,73 @@ SELECT subq_2.listing__country_latest FROM ( - -- Constrain Output with WHERE + -- Pass Only Elements: + -- ['listing__country_latest'] SELECT subq_1.listing__country_latest FROM ( - -- Pass Only Elements: - -- ['listing__country_latest'] + -- Constrain Output with WHERE SELECT - subq_0.listing__country_latest + subq_0.ds__day + , subq_0.ds__week + , subq_0.ds__month + , subq_0.ds__quarter + , subq_0.ds__year + , subq_0.ds__extract_year + , subq_0.ds__extract_quarter + , subq_0.ds__extract_month + , subq_0.ds__extract_week + , subq_0.ds__extract_day + , subq_0.ds__extract_dow + , subq_0.ds__extract_doy + , subq_0.created_at__day + , subq_0.created_at__week + , subq_0.created_at__month + , subq_0.created_at__quarter + , subq_0.created_at__year + , subq_0.created_at__extract_year + , subq_0.created_at__extract_quarter + , subq_0.created_at__extract_month + , subq_0.created_at__extract_week + , subq_0.created_at__extract_day + , subq_0.created_at__extract_dow + , subq_0.created_at__extract_doy + , subq_0.listing__ds__day + , subq_0.listing__ds__week + , subq_0.listing__ds__month + , subq_0.listing__ds__quarter + , subq_0.listing__ds__year + , subq_0.listing__ds__extract_year + , subq_0.listing__ds__extract_quarter + , subq_0.listing__ds__extract_month + , subq_0.listing__ds__extract_week + , subq_0.listing__ds__extract_day + , subq_0.listing__ds__extract_dow + , subq_0.listing__ds__extract_doy + , subq_0.listing__created_at__day + , subq_0.listing__created_at__week + , subq_0.listing__created_at__month + , subq_0.listing__created_at__quarter + , subq_0.listing__created_at__year + , subq_0.listing__created_at__extract_year + , subq_0.listing__created_at__extract_quarter + , subq_0.listing__created_at__extract_month + , subq_0.listing__created_at__extract_week + , subq_0.listing__created_at__extract_day + , subq_0.listing__created_at__extract_dow + , subq_0.listing__created_at__extract_doy + , subq_0.listing + , subq_0.user + , subq_0.listing__user + , subq_0.country_latest + , subq_0.is_lux_latest + , subq_0.capacity_latest + , subq_0.listing__country_latest + , subq_0.listing__is_lux_latest + , subq_0.listing__capacity_latest + , subq_0.listings + , subq_0.largest_listing + , subq_0.smallest_listing FROM ( -- Read Elements From Semantic Model 'listings_latest' SELECT @@ -75,10 +134,10 @@ FROM ( , listings_latest_src_10004.user_id AS listing__user FROM ***************************.dim_listings_latest listings_latest_src_10004 ) subq_0 - GROUP BY - listing__country_latest + WHERE listing__country_latest = 'us' ) subq_1 - WHERE listing__country_latest = 'us' + GROUP BY + listing__country_latest ) subq_2 ORDER BY subq_2.listing__country_latest DESC LIMIT 100 diff --git a/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/BigQuery/test_distinct_values__plan0_optimized.sql b/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/BigQuery/test_distinct_values__plan0_optimized.sql index 53419bb1c0..ec0d94dac2 100644 --- a/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/BigQuery/test_distinct_values__plan0_optimized.sql +++ b/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/BigQuery/test_distinct_values__plan0_optimized.sql @@ -1,17 +1,17 @@ -- Constrain Output with WHERE +-- Pass Only Elements: +-- ['listing__country_latest'] -- Order By ['listing__country_latest'] Limit 100 SELECT listing__country_latest FROM ( -- Read Elements From Semantic Model 'listings_latest' - -- Pass Only Elements: - -- ['listing__country_latest'] SELECT country AS listing__country_latest FROM ***************************.dim_listings_latest listings_latest_src_10004 - GROUP BY - listing__country_latest -) subq_4 +) subq_3 WHERE listing__country_latest = 'us' +GROUP BY + listing__country_latest ORDER BY listing__country_latest DESC LIMIT 100 diff --git a/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/DuckDB/test_distinct_values__plan0.sql b/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/DuckDB/test_distinct_values__plan0.sql index 7818858e77..51c90f927c 100644 --- a/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/DuckDB/test_distinct_values__plan0.sql +++ b/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/DuckDB/test_distinct_values__plan0.sql @@ -2,14 +2,73 @@ SELECT subq_2.listing__country_latest FROM ( - -- Constrain Output with WHERE + -- Pass Only Elements: + -- ['listing__country_latest'] SELECT subq_1.listing__country_latest FROM ( - -- Pass Only Elements: - -- ['listing__country_latest'] + -- Constrain Output with WHERE SELECT - subq_0.listing__country_latest + subq_0.ds__day + , subq_0.ds__week + , subq_0.ds__month + , subq_0.ds__quarter + , subq_0.ds__year + , subq_0.ds__extract_year + , subq_0.ds__extract_quarter + , subq_0.ds__extract_month + , subq_0.ds__extract_week + , subq_0.ds__extract_day + , subq_0.ds__extract_dow + , subq_0.ds__extract_doy + , subq_0.created_at__day + , subq_0.created_at__week + , subq_0.created_at__month + , subq_0.created_at__quarter + , subq_0.created_at__year + , subq_0.created_at__extract_year + , subq_0.created_at__extract_quarter + , subq_0.created_at__extract_month + , subq_0.created_at__extract_week + , subq_0.created_at__extract_day + , subq_0.created_at__extract_dow + , subq_0.created_at__extract_doy + , subq_0.listing__ds__day + , subq_0.listing__ds__week + , subq_0.listing__ds__month + , subq_0.listing__ds__quarter + , subq_0.listing__ds__year + , subq_0.listing__ds__extract_year + , subq_0.listing__ds__extract_quarter + , subq_0.listing__ds__extract_month + , subq_0.listing__ds__extract_week + , subq_0.listing__ds__extract_day + , subq_0.listing__ds__extract_dow + , subq_0.listing__ds__extract_doy + , subq_0.listing__created_at__day + , subq_0.listing__created_at__week + , subq_0.listing__created_at__month + , subq_0.listing__created_at__quarter + , subq_0.listing__created_at__year + , subq_0.listing__created_at__extract_year + , subq_0.listing__created_at__extract_quarter + , subq_0.listing__created_at__extract_month + , subq_0.listing__created_at__extract_week + , subq_0.listing__created_at__extract_day + , subq_0.listing__created_at__extract_dow + , subq_0.listing__created_at__extract_doy + , subq_0.listing + , subq_0.user + , subq_0.listing__user + , subq_0.country_latest + , subq_0.is_lux_latest + , subq_0.capacity_latest + , subq_0.listing__country_latest + , subq_0.listing__is_lux_latest + , subq_0.listing__capacity_latest + , subq_0.listings + , subq_0.largest_listing + , subq_0.smallest_listing FROM ( -- Read Elements From Semantic Model 'listings_latest' SELECT @@ -75,10 +134,10 @@ FROM ( , listings_latest_src_10004.user_id AS listing__user FROM ***************************.dim_listings_latest listings_latest_src_10004 ) subq_0 - GROUP BY - subq_0.listing__country_latest + WHERE listing__country_latest = 'us' ) subq_1 - WHERE listing__country_latest = 'us' + GROUP BY + subq_1.listing__country_latest ) subq_2 ORDER BY subq_2.listing__country_latest DESC LIMIT 100 diff --git a/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/DuckDB/test_distinct_values__plan0_optimized.sql b/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/DuckDB/test_distinct_values__plan0_optimized.sql index 592773c228..ec0d94dac2 100644 --- a/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/DuckDB/test_distinct_values__plan0_optimized.sql +++ b/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/DuckDB/test_distinct_values__plan0_optimized.sql @@ -1,17 +1,17 @@ -- Constrain Output with WHERE +-- Pass Only Elements: +-- ['listing__country_latest'] -- Order By ['listing__country_latest'] Limit 100 SELECT listing__country_latest FROM ( -- Read Elements From Semantic Model 'listings_latest' - -- Pass Only Elements: - -- ['listing__country_latest'] SELECT country AS listing__country_latest FROM ***************************.dim_listings_latest listings_latest_src_10004 - GROUP BY - country -) subq_4 +) subq_3 WHERE listing__country_latest = 'us' +GROUP BY + listing__country_latest ORDER BY listing__country_latest DESC LIMIT 100 diff --git a/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Postgres/test_distinct_values__plan0.sql b/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Postgres/test_distinct_values__plan0.sql index 7818858e77..51c90f927c 100644 --- a/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Postgres/test_distinct_values__plan0.sql +++ b/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Postgres/test_distinct_values__plan0.sql @@ -2,14 +2,73 @@ SELECT subq_2.listing__country_latest FROM ( - -- Constrain Output with WHERE + -- Pass Only Elements: + -- ['listing__country_latest'] SELECT subq_1.listing__country_latest FROM ( - -- Pass Only Elements: - -- ['listing__country_latest'] + -- Constrain Output with WHERE SELECT - subq_0.listing__country_latest + subq_0.ds__day + , subq_0.ds__week + , subq_0.ds__month + , subq_0.ds__quarter + , subq_0.ds__year + , subq_0.ds__extract_year + , subq_0.ds__extract_quarter + , subq_0.ds__extract_month + , subq_0.ds__extract_week + , subq_0.ds__extract_day + , subq_0.ds__extract_dow + , subq_0.ds__extract_doy + , subq_0.created_at__day + , subq_0.created_at__week + , subq_0.created_at__month + , subq_0.created_at__quarter + , subq_0.created_at__year + , subq_0.created_at__extract_year + , subq_0.created_at__extract_quarter + , subq_0.created_at__extract_month + , subq_0.created_at__extract_week + , subq_0.created_at__extract_day + , subq_0.created_at__extract_dow + , subq_0.created_at__extract_doy + , subq_0.listing__ds__day + , subq_0.listing__ds__week + , subq_0.listing__ds__month + , subq_0.listing__ds__quarter + , subq_0.listing__ds__year + , subq_0.listing__ds__extract_year + , subq_0.listing__ds__extract_quarter + , subq_0.listing__ds__extract_month + , subq_0.listing__ds__extract_week + , subq_0.listing__ds__extract_day + , subq_0.listing__ds__extract_dow + , subq_0.listing__ds__extract_doy + , subq_0.listing__created_at__day + , subq_0.listing__created_at__week + , subq_0.listing__created_at__month + , subq_0.listing__created_at__quarter + , subq_0.listing__created_at__year + , subq_0.listing__created_at__extract_year + , subq_0.listing__created_at__extract_quarter + , subq_0.listing__created_at__extract_month + , subq_0.listing__created_at__extract_week + , subq_0.listing__created_at__extract_day + , subq_0.listing__created_at__extract_dow + , subq_0.listing__created_at__extract_doy + , subq_0.listing + , subq_0.user + , subq_0.listing__user + , subq_0.country_latest + , subq_0.is_lux_latest + , subq_0.capacity_latest + , subq_0.listing__country_latest + , subq_0.listing__is_lux_latest + , subq_0.listing__capacity_latest + , subq_0.listings + , subq_0.largest_listing + , subq_0.smallest_listing FROM ( -- Read Elements From Semantic Model 'listings_latest' SELECT @@ -75,10 +134,10 @@ FROM ( , listings_latest_src_10004.user_id AS listing__user FROM ***************************.dim_listings_latest listings_latest_src_10004 ) subq_0 - GROUP BY - subq_0.listing__country_latest + WHERE listing__country_latest = 'us' ) subq_1 - WHERE listing__country_latest = 'us' + GROUP BY + subq_1.listing__country_latest ) subq_2 ORDER BY subq_2.listing__country_latest DESC LIMIT 100 diff --git a/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Postgres/test_distinct_values__plan0_optimized.sql b/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Postgres/test_distinct_values__plan0_optimized.sql index 592773c228..ec0d94dac2 100644 --- a/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Postgres/test_distinct_values__plan0_optimized.sql +++ b/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Postgres/test_distinct_values__plan0_optimized.sql @@ -1,17 +1,17 @@ -- Constrain Output with WHERE +-- Pass Only Elements: +-- ['listing__country_latest'] -- Order By ['listing__country_latest'] Limit 100 SELECT listing__country_latest FROM ( -- Read Elements From Semantic Model 'listings_latest' - -- Pass Only Elements: - -- ['listing__country_latest'] SELECT country AS listing__country_latest FROM ***************************.dim_listings_latest listings_latest_src_10004 - GROUP BY - country -) subq_4 +) subq_3 WHERE listing__country_latest = 'us' +GROUP BY + listing__country_latest ORDER BY listing__country_latest DESC LIMIT 100 diff --git a/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Redshift/test_distinct_values__plan0.sql b/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Redshift/test_distinct_values__plan0.sql index 7818858e77..51c90f927c 100644 --- a/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Redshift/test_distinct_values__plan0.sql +++ b/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Redshift/test_distinct_values__plan0.sql @@ -2,14 +2,73 @@ SELECT subq_2.listing__country_latest FROM ( - -- Constrain Output with WHERE + -- Pass Only Elements: + -- ['listing__country_latest'] SELECT subq_1.listing__country_latest FROM ( - -- Pass Only Elements: - -- ['listing__country_latest'] + -- Constrain Output with WHERE SELECT - subq_0.listing__country_latest + subq_0.ds__day + , subq_0.ds__week + , subq_0.ds__month + , subq_0.ds__quarter + , subq_0.ds__year + , subq_0.ds__extract_year + , subq_0.ds__extract_quarter + , subq_0.ds__extract_month + , subq_0.ds__extract_week + , subq_0.ds__extract_day + , subq_0.ds__extract_dow + , subq_0.ds__extract_doy + , subq_0.created_at__day + , subq_0.created_at__week + , subq_0.created_at__month + , subq_0.created_at__quarter + , subq_0.created_at__year + , subq_0.created_at__extract_year + , subq_0.created_at__extract_quarter + , subq_0.created_at__extract_month + , subq_0.created_at__extract_week + , subq_0.created_at__extract_day + , subq_0.created_at__extract_dow + , subq_0.created_at__extract_doy + , subq_0.listing__ds__day + , subq_0.listing__ds__week + , subq_0.listing__ds__month + , subq_0.listing__ds__quarter + , subq_0.listing__ds__year + , subq_0.listing__ds__extract_year + , subq_0.listing__ds__extract_quarter + , subq_0.listing__ds__extract_month + , subq_0.listing__ds__extract_week + , subq_0.listing__ds__extract_day + , subq_0.listing__ds__extract_dow + , subq_0.listing__ds__extract_doy + , subq_0.listing__created_at__day + , subq_0.listing__created_at__week + , subq_0.listing__created_at__month + , subq_0.listing__created_at__quarter + , subq_0.listing__created_at__year + , subq_0.listing__created_at__extract_year + , subq_0.listing__created_at__extract_quarter + , subq_0.listing__created_at__extract_month + , subq_0.listing__created_at__extract_week + , subq_0.listing__created_at__extract_day + , subq_0.listing__created_at__extract_dow + , subq_0.listing__created_at__extract_doy + , subq_0.listing + , subq_0.user + , subq_0.listing__user + , subq_0.country_latest + , subq_0.is_lux_latest + , subq_0.capacity_latest + , subq_0.listing__country_latest + , subq_0.listing__is_lux_latest + , subq_0.listing__capacity_latest + , subq_0.listings + , subq_0.largest_listing + , subq_0.smallest_listing FROM ( -- Read Elements From Semantic Model 'listings_latest' SELECT @@ -75,10 +134,10 @@ FROM ( , listings_latest_src_10004.user_id AS listing__user FROM ***************************.dim_listings_latest listings_latest_src_10004 ) subq_0 - GROUP BY - subq_0.listing__country_latest + WHERE listing__country_latest = 'us' ) subq_1 - WHERE listing__country_latest = 'us' + GROUP BY + subq_1.listing__country_latest ) subq_2 ORDER BY subq_2.listing__country_latest DESC LIMIT 100 diff --git a/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Redshift/test_distinct_values__plan0_optimized.sql b/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Redshift/test_distinct_values__plan0_optimized.sql index 592773c228..ec0d94dac2 100644 --- a/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Redshift/test_distinct_values__plan0_optimized.sql +++ b/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Redshift/test_distinct_values__plan0_optimized.sql @@ -1,17 +1,17 @@ -- Constrain Output with WHERE +-- Pass Only Elements: +-- ['listing__country_latest'] -- Order By ['listing__country_latest'] Limit 100 SELECT listing__country_latest FROM ( -- Read Elements From Semantic Model 'listings_latest' - -- Pass Only Elements: - -- ['listing__country_latest'] SELECT country AS listing__country_latest FROM ***************************.dim_listings_latest listings_latest_src_10004 - GROUP BY - country -) subq_4 +) subq_3 WHERE listing__country_latest = 'us' +GROUP BY + listing__country_latest ORDER BY listing__country_latest DESC LIMIT 100 diff --git a/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Snowflake/test_distinct_values__plan0.sql b/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Snowflake/test_distinct_values__plan0.sql index 7818858e77..51c90f927c 100644 --- a/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Snowflake/test_distinct_values__plan0.sql +++ b/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Snowflake/test_distinct_values__plan0.sql @@ -2,14 +2,73 @@ SELECT subq_2.listing__country_latest FROM ( - -- Constrain Output with WHERE + -- Pass Only Elements: + -- ['listing__country_latest'] SELECT subq_1.listing__country_latest FROM ( - -- Pass Only Elements: - -- ['listing__country_latest'] + -- Constrain Output with WHERE SELECT - subq_0.listing__country_latest + subq_0.ds__day + , subq_0.ds__week + , subq_0.ds__month + , subq_0.ds__quarter + , subq_0.ds__year + , subq_0.ds__extract_year + , subq_0.ds__extract_quarter + , subq_0.ds__extract_month + , subq_0.ds__extract_week + , subq_0.ds__extract_day + , subq_0.ds__extract_dow + , subq_0.ds__extract_doy + , subq_0.created_at__day + , subq_0.created_at__week + , subq_0.created_at__month + , subq_0.created_at__quarter + , subq_0.created_at__year + , subq_0.created_at__extract_year + , subq_0.created_at__extract_quarter + , subq_0.created_at__extract_month + , subq_0.created_at__extract_week + , subq_0.created_at__extract_day + , subq_0.created_at__extract_dow + , subq_0.created_at__extract_doy + , subq_0.listing__ds__day + , subq_0.listing__ds__week + , subq_0.listing__ds__month + , subq_0.listing__ds__quarter + , subq_0.listing__ds__year + , subq_0.listing__ds__extract_year + , subq_0.listing__ds__extract_quarter + , subq_0.listing__ds__extract_month + , subq_0.listing__ds__extract_week + , subq_0.listing__ds__extract_day + , subq_0.listing__ds__extract_dow + , subq_0.listing__ds__extract_doy + , subq_0.listing__created_at__day + , subq_0.listing__created_at__week + , subq_0.listing__created_at__month + , subq_0.listing__created_at__quarter + , subq_0.listing__created_at__year + , subq_0.listing__created_at__extract_year + , subq_0.listing__created_at__extract_quarter + , subq_0.listing__created_at__extract_month + , subq_0.listing__created_at__extract_week + , subq_0.listing__created_at__extract_day + , subq_0.listing__created_at__extract_dow + , subq_0.listing__created_at__extract_doy + , subq_0.listing + , subq_0.user + , subq_0.listing__user + , subq_0.country_latest + , subq_0.is_lux_latest + , subq_0.capacity_latest + , subq_0.listing__country_latest + , subq_0.listing__is_lux_latest + , subq_0.listing__capacity_latest + , subq_0.listings + , subq_0.largest_listing + , subq_0.smallest_listing FROM ( -- Read Elements From Semantic Model 'listings_latest' SELECT @@ -75,10 +134,10 @@ FROM ( , listings_latest_src_10004.user_id AS listing__user FROM ***************************.dim_listings_latest listings_latest_src_10004 ) subq_0 - GROUP BY - subq_0.listing__country_latest + WHERE listing__country_latest = 'us' ) subq_1 - WHERE listing__country_latest = 'us' + GROUP BY + subq_1.listing__country_latest ) subq_2 ORDER BY subq_2.listing__country_latest DESC LIMIT 100 diff --git a/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Snowflake/test_distinct_values__plan0_optimized.sql b/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Snowflake/test_distinct_values__plan0_optimized.sql index 592773c228..ec0d94dac2 100644 --- a/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Snowflake/test_distinct_values__plan0_optimized.sql +++ b/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/Snowflake/test_distinct_values__plan0_optimized.sql @@ -1,17 +1,17 @@ -- Constrain Output with WHERE +-- Pass Only Elements: +-- ['listing__country_latest'] -- Order By ['listing__country_latest'] Limit 100 SELECT listing__country_latest FROM ( -- Read Elements From Semantic Model 'listings_latest' - -- Pass Only Elements: - -- ['listing__country_latest'] SELECT country AS listing__country_latest FROM ***************************.dim_listings_latest listings_latest_src_10004 - GROUP BY - country -) subq_4 +) subq_3 WHERE listing__country_latest = 'us' +GROUP BY + listing__country_latest ORDER BY listing__country_latest DESC LIMIT 100 diff --git a/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/test_distinct_values__plan0.xml b/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/test_distinct_values__plan0.xml index 4186834688..da366436ac 100644 --- a/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/test_distinct_values__plan0.xml +++ b/metricflow/test/snapshots/test_dataflow_to_sql_plan.py/SqlQueryPlan/test_distinct_values__plan0.xml @@ -2,41 +2,277 @@ - - - - + + + + - - - - + + + + - + + + - - - - + + + + - - + + + + + - - - + - + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + - - + + + + + + + + + - - + + + + + + + + + + + + From 722d7eab6979605b46944a64f2944f5f71c00ed0 Mon Sep 17 00:00:00 2001 From: Courtney Holcomb Date: Thu, 12 Oct 2023 13:51:45 -0700 Subject: [PATCH 15/15] Refactor to simplify time granularity solver --- metricflow/query/query_parser.py | 14 +--- metricflow/test/fixtures/model_fixtures.py | 9 +++ .../test/time/test_time_granularity_solver.py | 27 ++----- metricflow/time/time_granularity_solver.py | 74 ++++++++----------- 4 files changed, 50 insertions(+), 74 deletions(-) diff --git a/metricflow/query/query_parser.py b/metricflow/query/query_parser.py index 0589beaf6b..202115267f 100644 --- a/metricflow/query/query_parser.py +++ b/metricflow/query/query_parser.py @@ -146,6 +146,8 @@ def __init__( # noqa: D self._metric_time_dimension_reference = DataSet.metric_time_dimension_reference() self._time_granularity_solver = TimeGranularitySolver( semantic_manifest_lookup=self._model, + read_nodes=self._read_nodes, + node_output_resolver=self._node_output_resolver, ) @staticmethod @@ -410,8 +412,6 @@ def _parse_and_validate_query( self._time_granularity_solver.resolve_granularity_for_partial_time_dimension_specs( metric_references=metric_references, partial_time_dimension_specs=requested_linkable_specs.partial_time_dimension_specs, - read_nodes=self._read_nodes, - node_output_resolver=self._node_output_resolver, ) ) @@ -581,10 +581,7 @@ def _adjust_time_range_constraint( ) partial_time_dimension_spec_to_time_dimension_spec = ( self._time_granularity_solver.resolve_granularity_for_partial_time_dimension_specs( - metric_references=metric_references, - partial_time_dimension_specs=(partial_metric_time_spec,), - read_nodes=self._read_nodes, - node_output_resolver=self._node_output_resolver, + metric_references=metric_references, partial_time_dimension_specs=(partial_metric_time_spec,) ) ) adjust_to_granularity = partial_time_dimension_spec_to_time_dimension_spec[ @@ -783,10 +780,7 @@ def _verify_resolved_granularity_for_date_part( ensure that the correct value was passed in. """ resolved_granularity = self._time_granularity_solver.find_minimum_granularity_for_partial_time_dimension_spec( - partial_time_dimension_spec=partial_time_dimension_spec, - metric_references=metric_references, - read_nodes=self._read_nodes, - node_output_resolver=self._node_output_resolver, + partial_time_dimension_spec=partial_time_dimension_spec, metric_references=metric_references ) if resolved_granularity != requested_dimension_structured_name.time_granularity: raise RequestTimeGranularityException( diff --git a/metricflow/test/fixtures/model_fixtures.py b/metricflow/test/fixtures/model_fixtures.py index afcf7bf38e..63e39d1b8b 100644 --- a/metricflow/test/fixtures/model_fixtures.py +++ b/metricflow/test/fixtures/model_fixtures.py @@ -90,6 +90,9 @@ class ConsistentIdObjectRepository: cyclic_join_read_nodes: OrderedDict[str, ReadSqlSourceNode] cyclic_join_source_nodes: Sequence[BaseOutput] + extended_date_model_read_nodes: OrderedDict[str, ReadSqlSourceNode] + extended_date_model_source_nodes: Sequence[BaseOutput] + @pytest.fixture(scope="session") def consistent_id_object_repository( @@ -97,6 +100,7 @@ def consistent_id_object_repository( multi_hop_join_semantic_manifest_lookup: SemanticManifestLookup, scd_semantic_manifest_lookup: SemanticManifestLookup, cyclic_join_semantic_manifest_lookup: SemanticManifestLookup, + extended_date_semantic_manifest_lookup: SemanticManifestLookup, ) -> ConsistentIdObjectRepository: # noqa: D """Create objects that have incremental numeric IDs with a consistent value. @@ -108,6 +112,7 @@ def consistent_id_object_repository( multihop_data_sets = create_data_sets(multi_hop_join_semantic_manifest_lookup) scd_data_sets = create_data_sets(scd_semantic_manifest_lookup) cyclic_join_data_sets = create_data_sets(cyclic_join_semantic_manifest_lookup) + extended_date_data_sets = create_data_sets(extended_date_semantic_manifest_lookup) return ConsistentIdObjectRepository( simple_model_data_sets=sm_data_sets, @@ -126,6 +131,10 @@ def consistent_id_object_repository( cyclic_join_source_nodes=_data_set_to_source_nodes( semantic_manifest_lookup=cyclic_join_semantic_manifest_lookup, data_sets=cyclic_join_data_sets ), + extended_date_model_read_nodes=_data_set_to_read_nodes(extended_date_data_sets), + extended_date_model_source_nodes=_data_set_to_source_nodes( + semantic_manifest_lookup=extended_date_semantic_manifest_lookup, data_sets=extended_date_data_sets + ), ) diff --git a/metricflow/test/time/test_time_granularity_solver.py b/metricflow/test/time/test_time_granularity_solver.py index 0fcdf0ce28..4e645e0186 100644 --- a/metricflow/test/time/test_time_granularity_solver.py +++ b/metricflow/test/time/test_time_granularity_solver.py @@ -22,9 +22,13 @@ @pytest.fixture(scope="session") def time_granularity_solver( # noqa: D extended_date_semantic_manifest_lookup: SemanticManifestLookup, + consistent_id_object_repository: ConsistentIdObjectRepository, + node_output_resolver: DataflowPlanNodeOutputDataSetResolver, ) -> TimeGranularitySolver: return TimeGranularitySolver( semantic_manifest_lookup=extended_date_semantic_manifest_lookup, + read_nodes=list(consistent_id_object_repository.extended_date_model_read_nodes.values()), + node_output_resolver=node_output_resolver, ) @@ -91,31 +95,18 @@ def test_validate_day_granularity_for_day_and_month_metric( # noqa: D PARTIAL_PTD_SPEC = PartialTimeDimensionSpec(element_name=DataSet.metric_time_dimension_name(), entity_links=()) -def test_granularity_solution_for_day_metric( # noqa: D - time_granularity_solver: TimeGranularitySolver, - node_output_resolver: DataflowPlanNodeOutputDataSetResolver, - consistent_id_object_repository: ConsistentIdObjectRepository, -) -> None: +def test_granularity_solution_for_day_metric(time_granularity_solver: TimeGranularitySolver) -> None: # noqa: D assert time_granularity_solver.resolve_granularity_for_partial_time_dimension_specs( - metric_references=[MetricReference(element_name="bookings")], - partial_time_dimension_specs=[PARTIAL_PTD_SPEC], - node_output_resolver=node_output_resolver, - read_nodes=list(consistent_id_object_repository.simple_model_read_nodes.values()), + metric_references=[MetricReference(element_name="bookings")], partial_time_dimension_specs=[PARTIAL_PTD_SPEC] ) == { PARTIAL_PTD_SPEC: MTD_SPEC_DAY, } -def test_granularity_solution_for_month_metric( # noqa: D - time_granularity_solver: TimeGranularitySolver, - node_output_resolver: DataflowPlanNodeOutputDataSetResolver, - consistent_id_object_repository: ConsistentIdObjectRepository, -) -> None: +def test_granularity_solution_for_month_metric(time_granularity_solver: TimeGranularitySolver) -> None: # noqa: D assert time_granularity_solver.resolve_granularity_for_partial_time_dimension_specs( metric_references=[MetricReference(element_name="bookings_monthly")], partial_time_dimension_specs=[PARTIAL_PTD_SPEC], - node_output_resolver=node_output_resolver, - read_nodes=list(consistent_id_object_repository.simple_model_read_nodes.values()), ) == { PARTIAL_PTD_SPEC: MTD_SPEC_MONTH, } @@ -123,14 +114,10 @@ def test_granularity_solution_for_month_metric( # noqa: D def test_granularity_solution_for_day_and_month_metrics( # noqa: D time_granularity_solver: TimeGranularitySolver, - node_output_resolver: DataflowPlanNodeOutputDataSetResolver, - consistent_id_object_repository: ConsistentIdObjectRepository, ) -> None: assert time_granularity_solver.resolve_granularity_for_partial_time_dimension_specs( metric_references=[MetricReference(element_name="bookings"), MetricReference(element_name="bookings_monthly")], partial_time_dimension_specs=[PARTIAL_PTD_SPEC], - node_output_resolver=node_output_resolver, - read_nodes=list(consistent_id_object_repository.simple_model_read_nodes.values()), ) == {PARTIAL_PTD_SPEC: MTD_SPEC_MONTH} diff --git a/metricflow/time/time_granularity_solver.py b/metricflow/time/time_granularity_solver.py index 994f1dcd68..802fc7fd45 100644 --- a/metricflow/time/time_granularity_solver.py +++ b/metricflow/time/time_granularity_solver.py @@ -1,6 +1,7 @@ from __future__ import annotations import logging +from collections import defaultdict from dataclasses import dataclass from typing import Dict, Optional, Sequence, Set, Tuple @@ -67,8 +68,22 @@ class TimeGranularitySolver: def __init__( # noqa: D self, semantic_manifest_lookup: SemanticManifestLookup, + node_output_resolver: DataflowPlanNodeOutputDataSetResolver, + read_nodes: Sequence[ReadSqlSourceNode], ) -> None: self._semantic_manifest_lookup = semantic_manifest_lookup + self._time_dimension_names_to_supported_granularities: Dict[str, Set[TimeGranularity]] = defaultdict(set) + for read_node in read_nodes: + output_data_set = node_output_resolver.get_output_data_set(read_node) + for time_dimension_instance in output_data_set.instance_set.time_dimension_instances: + if time_dimension_instance.spec.date_part: + continue + granularity_free_qualified_name = StructuredLinkableSpecName.from_name( + time_dimension_instance.spec.qualified_name + ).granularity_free_qualified_name + self._time_dimension_names_to_supported_granularities[granularity_free_qualified_name].add( + time_dimension_instance.spec.time_granularity + ) def validate_time_granularity( self, metric_references: Sequence[MetricReference], time_dimension_specs: Sequence[TimeDimensionSpec] @@ -103,8 +118,6 @@ def resolve_granularity_for_partial_time_dimension_specs( self, metric_references: Sequence[MetricReference], partial_time_dimension_specs: Sequence[PartialTimeDimensionSpec], - read_nodes: Sequence[ReadSqlSourceNode], - node_output_resolver: DataflowPlanNodeOutputDataSetResolver, ) -> Dict[PartialTimeDimensionSpec, TimeDimensionSpec]: """Figure out the lowest granularity possible for the partially specified time dimension specs. @@ -114,10 +127,7 @@ def resolve_granularity_for_partial_time_dimension_specs( for partial_time_dimension_spec in partial_time_dimension_specs: minimum_time_granularity = self.find_minimum_granularity_for_partial_time_dimension_spec( - partial_time_dimension_spec=partial_time_dimension_spec, - metric_references=metric_references, - read_nodes=read_nodes, - node_output_resolver=node_output_resolver, + partial_time_dimension_spec=partial_time_dimension_spec, metric_references=metric_references ) result[partial_time_dimension_spec] = TimeDimensionSpec( element_name=partial_time_dimension_spec.element_name, @@ -128,11 +138,7 @@ def resolve_granularity_for_partial_time_dimension_specs( return result def find_minimum_granularity_for_partial_time_dimension_spec( - self, - partial_time_dimension_spec: PartialTimeDimensionSpec, - metric_references: Sequence[MetricReference], - read_nodes: Sequence[ReadSqlSourceNode], - node_output_resolver: DataflowPlanNodeOutputDataSetResolver, + self, partial_time_dimension_spec: PartialTimeDimensionSpec, metric_references: Sequence[MetricReference] ) -> TimeGranularity: """Find minimum granularity allowed for time dimension when queried with given metrics.""" minimum_time_granularity: Optional[TimeGranularity] = None @@ -159,46 +165,26 @@ def find_minimum_granularity_for_partial_time_dimension_spec( f"{pformat_big_objects([spec.qualified_name for spec in valid_group_by_elements.as_spec_set.as_tuple])}" ) else: - minimum_time_granularity = self.get_min_granularity_for_partial_time_dimension_without_metrics( - read_nodes=read_nodes, - node_output_resolver=node_output_resolver, - partial_time_dimension_spec=partial_time_dimension_spec, + granularity_free_qualified_name = StructuredLinkableSpecName( + entity_link_names=tuple( + [entity_link.element_name for entity_link in partial_time_dimension_spec.entity_links] + ), + element_name=partial_time_dimension_spec.element_name, + ).granularity_free_qualified_name + + supported_granularities = self._time_dimension_names_to_supported_granularities.get( + granularity_free_qualified_name ) - if not minimum_time_granularity: + if not supported_granularities: raise RequestTimeGranularityException( f"Unable to resolve the time dimension spec for {partial_time_dimension_spec}. " ) + minimum_time_granularity = min( + self._time_dimension_names_to_supported_granularities[granularity_free_qualified_name] + ) return minimum_time_granularity - def get_min_granularity_for_partial_time_dimension_without_metrics( - self, - read_nodes: Sequence[ReadSqlSourceNode], - node_output_resolver: DataflowPlanNodeOutputDataSetResolver, - partial_time_dimension_spec: PartialTimeDimensionSpec, - ) -> Optional[TimeGranularity]: - """Find the minimum.""" - granularity_free_qualified_name = StructuredLinkableSpecName( - entity_link_names=tuple( - [entity_link.element_name for entity_link in partial_time_dimension_spec.entity_links] - ), - element_name=partial_time_dimension_spec.element_name, - ).granularity_free_qualified_name - - supported_granularities: Set[TimeGranularity] = set() - for read_node in read_nodes: - output_data_set = node_output_resolver.get_output_data_set(read_node) - for time_dimension_instance in output_data_set.instance_set.time_dimension_instances: - if time_dimension_instance.spec.date_part: - continue - time_dim_name_without_granularity = StructuredLinkableSpecName.from_name( - time_dimension_instance.spec.qualified_name - ).granularity_free_qualified_name - if time_dim_name_without_granularity == granularity_free_qualified_name: - supported_granularities.add(time_dimension_instance.spec.time_granularity) - - return min(supported_granularities) if supported_granularities else None - def adjust_time_range_to_granularity( self, time_range_constraint: TimeRangeConstraint, time_granularity: TimeGranularity ) -> TimeRangeConstraint: