Skip to content

Commit

Permalink
/* PR_START p--short-term-perf 03 */ Bulk update logging call sites.
Browse files Browse the repository at this point in the history
  • Loading branch information
plypaul committed Sep 24, 2024
1 parent ea66615 commit cf25768
Show file tree
Hide file tree
Showing 39 changed files with 429 additions and 226 deletions.
3 changes: 2 additions & 1 deletion dbt-metricflow/dbt_metricflow/cli/cli_context.py
Original file line number Diff line number Diff line change
Expand Up @@ -6,6 +6,7 @@
from typing import Dict, Optional

from dbt_semantic_interfaces.protocols.semantic_manifest import SemanticManifest
from metricflow_semantics.mf_logging.lazy_formattable import LazyFormat
from metricflow_semantics.model.semantic_manifest_lookup import SemanticManifestLookup

from dbt_metricflow.cli.dbt_connectors.adapter_backed_client import AdapterBackedSqlClient
Expand Down Expand Up @@ -111,7 +112,7 @@ def run_health_checks(self) -> Dict[str, Dict[str, str]]:
except Exception as e:
status = "FAIL"
err_string = str(e)
logger.error(f"Health Check Item {step}: failed with error {err_string}")
logger.error(LazyFormat(lambda: f"Health Check Item {step}: failed with error {err_string}"))

results[f"{self.sql_client.sql_engine_type} - {step}"] = {
"status": status,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -9,6 +9,7 @@
from dbt_semantic_interfaces.enum_extension import assert_values_exhausted
from metricflow_semantics.errors.error_classes import SqlBindParametersNotSupportedError
from metricflow_semantics.mf_logging.formatting import indent
from metricflow_semantics.mf_logging.lazy_formattable import LazyFormat
from metricflow_semantics.mf_logging.pretty_print import mf_pformat
from metricflow_semantics.random_id import random_id
from metricflow_semantics.sql.sql_bind_parameters import SqlBindParameters
Expand Down Expand Up @@ -111,7 +112,9 @@ def __init__(self, adapter: BaseAdapter):

self._sql_engine_type = adapter_type.sql_engine_type
self._sql_query_plan_renderer = adapter_type.sql_query_plan_renderer
logger.info(f"Initialized AdapterBackedSqlClient with dbt adapter type `{adapter_type.value}`")
logger.debug(
LazyFormat(lambda: f"Initialized AdapterBackedSqlClient with dbt adapter type `{adapter_type.value}`")
)

@property
def sql_engine_type(self) -> SqlEngine:
Expand Down Expand Up @@ -142,11 +145,13 @@ def query(
f"Invalid execute statement - we do not support queries with bind parameters through dbt adapters! "
f"Bind params: {sql_bind_parameters.param_dict}"
)
logger.info(AdapterBackedSqlClient._format_run_query_log_message(stmt, sql_bind_parameters))
logger.debug(
LazyFormat(lambda: AdapterBackedSqlClient._format_run_query_log_message(stmt, sql_bind_parameters))
)
with self._adapter.connection_named(f"MetricFlow_request_{request_id}"):
# returns a Tuple[AdapterResponse, agate.Table] but the decorator converts it to Any
result = self._adapter.execute(sql=stmt, auto_begin=True, fetch=True)
logger.info(f"Query returned from dbt Adapter with response {result[0]}")
logger.debug(LazyFormat(lambda: f"Query returned from dbt Adapter with response {result[0]}"))

agate_data = result[1]
rows = [row.values() for row in agate_data.rows]
Expand All @@ -155,7 +160,11 @@ def query(
rows=rows,
)
stop = time.time()
logger.info(f"Finished running the query in {stop - start:.2f}s with {data_table.row_count} row(s) returned")
logger.debug(
LazyFormat(
lambda: f"Finished running the query in {stop - start:.2f}s with {data_table.row_count} row(s) returned"
)
)
return data_table

def execute(
Expand All @@ -177,14 +186,16 @@ def execute(
)
start = time.time()
request_id = SqlRequestId(f"mf_rid__{random_id()}")
logger.info(AdapterBackedSqlClient._format_run_query_log_message(stmt, sql_bind_parameters))
logger.debug(
LazyFormat(lambda: AdapterBackedSqlClient._format_run_query_log_message(stmt, sql_bind_parameters))
)
with self._adapter.connection_named(f"MetricFlow_request_{request_id}"):
result = self._adapter.execute(stmt, auto_begin=True, fetch=False)
# Calls to execute often involve some amount of DDL so we commit here
self._adapter.commit_if_has_connection()
logger.info(f"Query executed via dbt Adapter with response {result[0]}")
logger.debug(LazyFormat(lambda: f"Query executed via dbt Adapter with response {result[0]}"))
stop = time.time()
logger.info(f"Finished running the query in {stop - start:.2f}s")
logger.debug(LazyFormat(lambda: f"Finished running the query in {stop - start:.2f}s"))
return None

def dry_run(
Expand All @@ -202,10 +213,16 @@ def dry_run(
concrete values for SQL query parameters.
"""
start = time.time()
logger.info(
f"Running dry_run of:"
f"\n\n{indent(stmt)}\n"
+ (f"\nwith parameters: {dict(sql_bind_parameters.param_dict)}" if sql_bind_parameters.param_dict else "")
logger.debug(
LazyFormat(
lambda: f"Running dry_run of:"
f"\n\n{indent(stmt)}\n"
+ (
f"\nwith parameters: {dict(sql_bind_parameters.param_dict)}"
if sql_bind_parameters.param_dict
else ""
)
)
)
request_id = SqlRequestId(f"mf_rid__{random_id()}")
connection_name = f"MetricFlow_dry_run_request_{request_id}"
Expand Down Expand Up @@ -242,7 +259,7 @@ def dry_run(
raise DbtDatabaseError(f"Encountered error in Databricks dry run. Full output: {plan_output_str}")

stop = time.time()
logger.info(f"Finished running the dry_run in {stop - start:.2f}s")
logger.debug(LazyFormat(lambda: f"Finished running the dry_run in {stop - start:.2f}s"))
return

def close(self) -> None: # noqa: D102
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -13,6 +13,7 @@
from dbt.config.runtime import load_profile, load_project
from dbt_semantic_interfaces.protocols.semantic_manifest import SemanticManifest
from metricflow_semantics.errors.error_classes import ModelCreationException
from metricflow_semantics.mf_logging.lazy_formattable import LazyFormat
from metricflow_semantics.mf_logging.pretty_print import mf_pformat
from metricflow_semantics.model.dbt_manifest_parser import parse_manifest_from_dbt_generated_manifest
from typing_extensions import Self
Expand Down Expand Up @@ -40,12 +41,14 @@ class dbtProjectMetadata:
@classmethod
def load_from_project_path(cls: Type[Self], project_path: Path) -> Self:
"""Loads all dbt artifacts for the project associated with the given project path."""
logger.info(f"Loading dbt project metadata for project located at {project_path}")
logger.debug(LazyFormat(lambda: f"Loading dbt project metadata for project located at {project_path}"))
dbtRunner().invoke(["-q", "debug"], project_dir=str(project_path))
profile = load_profile(str(project_path), {})
project = load_project(str(project_path), version_check=False, profile=profile)
project_path = project_path
logger.info(f"Loaded project {project.project_name} with profile details:\n{mf_pformat(profile)}")
logger.debug(
LazyFormat(lambda: f"Loaded project {project.project_name} with profile details:\n{mf_pformat(profile)}")
)
return cls(profile=profile, project=project, project_path=project_path)

@property
Expand Down
3 changes: 2 additions & 1 deletion dbt-metricflow/dbt_metricflow/cli/main.py
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@
from dbt_semantic_interfaces.validations.validator_helpers import SemanticManifestValidationResults
from halo import Halo
from metricflow_semantics.dag.dag_visualization import display_dag_as_svg
from metricflow_semantics.mf_logging.lazy_formattable import LazyFormat
from update_checker import UpdateChecker

import dbt_metricflow.cli.custom_click_types as click_custom
Expand Down Expand Up @@ -92,7 +93,7 @@ def exit_signal_handler(signal_type: int, frame) -> None: # type: ignore

try:
# Note: we may wish to add support for canceling all queries if zombie queries are a problem
logger.info("Closing client connections")
logger.debug(LazyFormat(lambda: "Closing client connections"))
cfg.sql_client.close()
finally:
sys.exit(-1)
Expand Down
7 changes: 5 additions & 2 deletions dbt-metricflow/dbt_metricflow/cli/utils.py
Original file line number Diff line number Diff line change
Expand Up @@ -9,6 +9,7 @@

import click
from dateutil.parser import parse
from metricflow_semantics.mf_logging.lazy_formattable import LazyFormat

import dbt_metricflow.cli.custom_click_types as click_custom
from dbt_metricflow.cli.cli_context import CLIContext
Expand Down Expand Up @@ -122,8 +123,10 @@ def wrapper(*args: Any, **kwargs: Any) -> Any: # type: ignore[misc]
else:
if not isinstance(args[0], CLIContext):
logger.error(
f"Missing {CLIContext.__name__} as the first argument to the function "
f"{getattr(func, '__name__', repr(func))}"
LazyFormat(
lambda: f"Missing {CLIContext.__name__} as the first argument to the function "
f"{getattr(func, '__name__', repr(func))}"
)
)
click.echo(f"\nERROR: {str(e)}")
if args and hasattr(args[0], "verbose") and args[0].verbose is True:
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -6,6 +6,8 @@

from dbt_semantic_interfaces.dataclass_serialization import SerializableDataclass

from metricflow_semantics.mf_logging.lazy_formattable import LazyFormat

logger = logging.getLogger(__name__)


Expand All @@ -18,10 +20,18 @@ class TimeRangeConstraint(SerializableDataclass):

def __post_init__(self) -> None: # noqa: D105
if self.start_time > self.end_time:
logger.warning(f"start_time must not be > end_time. start_time={self.start_time} end_time={self.end_time}")
logger.warning(
LazyFormat(
lambda: f"start_time must not be > end_time. start_time={self.start_time} end_time={self.end_time}"
)
)

if self.start_time < TimeRangeConstraint.ALL_TIME_BEGIN():
logger.warning(f"start_time={self.start_time} exceeds the limits of {TimeRangeConstraint.ALL_TIME_BEGIN()}")
logger.warning(
LazyFormat(
lambda: f"start_time={self.start_time} exceeds the limits of {TimeRangeConstraint.ALL_TIME_BEGIN()}"
)
)

if self.end_time > TimeRangeConstraint.ALL_TIME_END():
raise RuntimeError(f"end_time={self.end_time} exceeds the limits of {TimeRangeConstraint.ALL_TIME_END()}")
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -16,7 +16,7 @@ class LazyFormat:
incur a performance overhead in production.
Example:
logger.debug(LazyFormat("Found path", start_point=point_0, end_point=point_1))
logger.debug(LazyFormat(lambda: LazyFormat("Found path", start_point=point_0, end_point=point_1)))
->
Expand All @@ -28,7 +28,7 @@ class LazyFormat:
evaluated as well. This style will be deprecated as log statements are updated.
Example:
logger.debug(f"Result is: {expensive_function()}")
logger.debug(LazyFormat(lambda: f"Result is: {expensive_function()}"))
->
Expand Down
14 changes: 8 additions & 6 deletions metricflow-semantics/metricflow_semantics/mf_logging/runtime.py
Original file line number Diff line number Diff line change
Expand Up @@ -8,6 +8,8 @@

from typing_extensions import ParamSpec

from metricflow_semantics.mf_logging.lazy_formattable import LazyFormat

logger = logging.getLogger(__name__)

ReturnType = TypeVar("ReturnType")
Expand All @@ -29,15 +31,15 @@ def _inner(*args: ParametersType.args, **kwargs: ParametersType.kwargs) -> Retur
# __qualname__ includes the path like MyClass.my_function
function_name = f"{wrapped_function.__qualname__}()"
start_time = time.time()
logger.info(f"Starting {function_name}")
logger.debug(LazyFormat(lambda: f"Starting {function_name}"))

try:
result = wrapped_function(*args, **kwargs)
finally:
runtime = time.time() - start_time
logger.info(f"Finished {function_name} in {runtime:.1f}s")
logger.debug(LazyFormat(lambda: f"Finished {function_name} in {runtime:.1f}s"))
if runtime > runtime_warning_threshold:
logger.warning(f"{function_name} is slow with a runtime of {runtime:.1f}s")
logger.warning(LazyFormat(lambda: f"{function_name} is slow with a runtime of {runtime:.1f}s"))

return result

Expand All @@ -51,11 +53,11 @@ def log_block_runtime(code_block_name: str, runtime_warning_threshold: float = 5
"""Logs the runtime of the enclosed code block."""
start_time = time.time()
description = f"code_block_name={repr(code_block_name)}"
logger.info(f"Starting {description}")
logger.debug(LazyFormat(lambda: f"Starting {description}"))

yield

runtime = time.time() - start_time
logger.info(f"Finished {description} in {runtime:.1f}s")
logger.debug(LazyFormat(lambda: f"Finished {description} in {runtime:.1f}s"))
if runtime > runtime_warning_threshold:
logger.warning(f"{description} is slow with a runtime of {runtime:.1f}s")
logger.warning(LazyFormat(lambda: f"{description} is slow with a runtime of {runtime:.1f}s"))
Original file line number Diff line number Diff line change
Expand Up @@ -10,6 +10,7 @@
from dbt_semantic_interfaces.references import SemanticModelReference
from typing_extensions import override

from metricflow_semantics.mf_logging.lazy_formattable import LazyFormat
from metricflow_semantics.model.linkable_element_property import LinkableElementProperty
from metricflow_semantics.model.semantic_model_derivation import SemanticModelDerivation
from metricflow_semantics.model.semantics.linkable_element import (
Expand Down Expand Up @@ -413,5 +414,5 @@ def filter_by_spec_patterns(self, spec_patterns: Sequence[SpecPattern]) -> Linka
path_key_to_linkable_entities=path_key_to_linkable_entities,
path_key_to_linkable_metrics=path_key_to_linkable_metrics,
)
logger.info(f"Filtering valid linkable elements took: {time.time() - start_time:.2f}s")
logger.debug(LazyFormat(lambda: f"Filtering valid linkable elements took: {time.time() - start_time:.2f}s"))
return filtered_elements
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,7 @@
from dbt_semantic_interfaces.validations.unique_valid_name import MetricFlowReservedKeywords

from metricflow_semantics.errors.error_classes import UnknownMetricLinkingError
from metricflow_semantics.mf_logging.lazy_formattable import LazyFormat
from metricflow_semantics.mf_logging.pretty_print import mf_pformat
from metricflow_semantics.model.linkable_element_property import LinkableElementProperty
from metricflow_semantics.model.semantic_model_derivation import SemanticModelDerivation
Expand Down Expand Up @@ -203,7 +204,11 @@ def __init__(
for linkable_entity in linkable_entities:
# TODO: some users encounter a situation in which the entity reference is in the entity links. Debug why.
if linkable_entity.reference in linkable_entity.entity_links:
logger.info(f"Found entity reference in entity links for linkable entity: {linkable_entity}")
logger.debug(
LazyFormat(
lambda: f"Found entity reference in entity links for linkable entity: {linkable_entity}"
)
)
continue
metric_subquery_join_path_element = MetricSubqueryJoinPathElement(
metric_reference=metric_reference,
Expand All @@ -227,7 +232,11 @@ def __init__(
semantic_model, SemanticModelJoinPath(left_semantic_model_reference=semantic_model.reference)
)
)
logger.info(f"Building valid linkable metrics took: {time.time() - linkable_metrics_start_time:.2f}s")
logger.debug(
LazyFormat(
lambda: f"Building valid linkable metrics took: {time.time() - linkable_metrics_start_time:.2f}s"
)
)

for semantic_model in semantic_manifest.semantic_models:
linkable_element_sets_for_no_metrics_queries.append(self._get_elements_in_semantic_model(semantic_model))
Expand All @@ -237,7 +246,7 @@ def __init__(
linkable_element_sets_for_no_metrics_queries + [metric_time_elements_for_no_metrics]
)

logger.info(f"Building valid group-by-item indexes took: {time.time() - start_time:.2f}s")
logger.debug(LazyFormat(lambda: f"Building valid group-by-item indexes took: {time.time() - start_time:.2f}s"))

def _metric_requires_metric_time(self, metric: Metric) -> bool:
"""Checks if the metric can only be queried with metric_time. Also checks input metrics.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -12,6 +12,7 @@
from dbt_semantic_interfaces.type_enums.time_granularity import TimeGranularity

from metricflow_semantics.errors.error_classes import DuplicateMetricError, MetricNotFoundError, NonExistentMeasureError
from metricflow_semantics.mf_logging.lazy_formattable import LazyFormat
from metricflow_semantics.model.linkable_element_property import LinkableElementProperty
from metricflow_semantics.model.semantics.linkable_element_set import LinkableElementSet
from metricflow_semantics.model.semantics.linkable_spec_resolver import (
Expand Down Expand Up @@ -63,8 +64,10 @@ def linkable_elements_for_measure(
with_any_of=frozen_with_any_of,
without_any_of=frozen_without_any_of,
)
logger.info(
f"Getting valid linkable elements for measure '{measure_reference.element_name}' took: {time.time() - start_time:.2f}s"
logger.debug(
LazyFormat(
lambda: f"Getting valid linkable elements for measure '{measure_reference.element_name}' took: {time.time() - start_time:.2f}s"
)
)

return linkable_element_set
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@
from dbt_semantic_interfaces.type_enums import AggregationType, DimensionType, EntityType, TimeGranularity

from metricflow_semantics.errors.error_classes import InvalidSemanticModelError
from metricflow_semantics.mf_logging.lazy_formattable import LazyFormat
from metricflow_semantics.mf_logging.pretty_print import mf_pformat
from metricflow_semantics.model.semantics.element_group import ElementGrouper
from metricflow_semantics.model.spec_converters import MeasureConverter
Expand Down Expand Up @@ -242,8 +243,10 @@ def _add_semantic_model(self, semantic_model: SemanticModel) -> None:
if not StructuredLinkableSpecName.from_name(dim.name).is_element_name:
# TODO: [custom granularity] change this to an assertion once we're sure there aren't exceptions
logger.warning(
f"Dimension name `{dim.name}` contains annotations, but this name should be the plain element name "
"from the original model. This should have been blocked by validation!"
LazyFormat(
lambda: f"Dimension name `{dim.name}` contains annotations, but this name should be the plain element name "
"from the original model. This should have been blocked by validation!"
)
)

# TODO: Construct these specs correctly. All of the time dimension specs have the default granularity
Expand Down
Loading

0 comments on commit cf25768

Please sign in to comment.