Skip to content

Commit

Permalink
Specify UTC everywhere
Browse files Browse the repository at this point in the history
  • Loading branch information
treysp committed Dec 14, 2024
1 parent 094b03e commit 0f52d9c
Show file tree
Hide file tree
Showing 9 changed files with 51 additions and 53 deletions.
4 changes: 1 addition & 3 deletions sqlmesh/core/test/definition.py
Original file line number Diff line number Diff line change
Expand Up @@ -649,9 +649,7 @@ def runTest(self) -> None:

def _execute_model(self) -> pd.DataFrame:
"""Executes the python model and returns a DataFrame."""
time_ctx = (
freeze_time(self._execution_time, tick=False) if self._execution_time else nullcontext()
)
time_ctx = freeze_time(self._execution_time) if self._execution_time else nullcontext()
with patch.dict(self._test_adapter_dialect.generator_class.TRANSFORMS, self._transforms):
with t.cast(AbstractContextManager, time_ctx):
variables = self.body.get("vars", {}).copy()
Expand Down
2 changes: 1 addition & 1 deletion tests/cli/test_cli.py
Original file line number Diff line number Diff line change
Expand Up @@ -14,7 +14,7 @@
from sqlmesh.integrations.dlt import generate_dlt_models
from sqlmesh.utils.date import yesterday_ds

FREEZE_TIME = "2023-01-01 00:00:00"
FREEZE_TIME = "2023-01-01 00:00:00 UTC"

pytestmark = pytest.mark.slow

Expand Down
4 changes: 2 additions & 2 deletions tests/core/test_context.py
Original file line number Diff line number Diff line change
Expand Up @@ -850,15 +850,15 @@ def test_plan_default_end(sushi_context_pre_scheduling: Context):
@pytest.mark.slow
def test_plan_start_ahead_of_end(copy_to_temp_path):
path = copy_to_temp_path("examples/sushi")
with time_machine.travel("2024-01-02 00:00:00"):
with time_machine.travel("2024-01-02 00:00:00 UTC"):
context = Context(paths=path, gateway="duckdb_persistent")
context.plan("prod", no_prompts=True, auto_apply=True)
assert all(
i == to_timestamp("2024-01-02")
for i in context.state_sync.max_interval_end_per_model("prod").values()
)
context.close()
with time_machine.travel("2024-01-03 00:00:00"):
with time_machine.travel("2024-01-03 00:00:00 UTC"):
context = Context(paths=path, gateway="duckdb_persistent")
expression = d.parse(
"""
Expand Down
60 changes: 30 additions & 30 deletions tests/core/test_integration.py
Original file line number Diff line number Diff line change
Expand Up @@ -75,7 +75,7 @@ def plan_choice(plan_builder: PlanBuilder, choice: SnapshotChangeCategory) -> No
plan_builder.set_choice(snapshot, choice)


@time_machine.travel("2023-01-08 15:00:00")
@time_machine.travel("2023-01-08 15:00:00 UTC")
@pytest.mark.parametrize(
"context_fixture",
["sushi_context", "sushi_no_default_catalog"],
Expand Down Expand Up @@ -215,7 +215,7 @@ def test_forward_only_plan_with_effective_date(context_fixture: Context, request
]


@time_machine.travel("2023-01-08 15:00:00")
@time_machine.travel("2023-01-08 15:00:00 UTC")
def test_forward_only_model_regular_plan(init_and_plan_context: t.Callable):
context, plan = init_and_plan_context("examples/sushi")
context.apply(plan)
Expand Down Expand Up @@ -308,7 +308,7 @@ def test_forward_only_model_regular_plan(init_and_plan_context: t.Callable):
assert not prod_df["event_date"].tolist()


@time_machine.travel("2023-01-08 15:00:00")
@time_machine.travel("2023-01-08 15:00:00 UTC")
def test_forward_only_model_regular_plan_preview_enabled(init_and_plan_context: t.Callable):
context, plan = init_and_plan_context("examples/sushi")
context.apply(plan)
Expand Down Expand Up @@ -358,7 +358,7 @@ def test_forward_only_model_regular_plan_preview_enabled(init_and_plan_context:
assert dev_df["event_date"].tolist() == [pd.to_datetime("2023-01-07")]


@time_machine.travel("2023-01-08 15:00:00")
@time_machine.travel("2023-01-08 15:00:00 UTC")
def test_full_history_restatement_model_regular_plan_preview_enabled(
init_and_plan_context: t.Callable,
):
Expand Down Expand Up @@ -431,7 +431,7 @@ def test_full_history_restatement_model_regular_plan_preview_enabled(
context.apply(plan)


@time_machine.travel("2023-01-08 15:00:00")
@time_machine.travel("2023-01-08 15:00:00 UTC")
def test_metadata_changed_regular_plan_preview_enabled(init_and_plan_context: t.Callable):
context, plan = init_and_plan_context("examples/sushi")
context.apply(plan)
Expand Down Expand Up @@ -459,7 +459,7 @@ def test_metadata_changed_regular_plan_preview_enabled(init_and_plan_context: t.
assert not plan.restatements


@time_machine.travel("2023-01-08 15:00:00")
@time_machine.travel("2023-01-08 15:00:00 UTC")
def test_hourly_model_with_lookback_no_backfill_in_dev(init_and_plan_context: t.Callable):
context, plan = init_and_plan_context("examples/sushi")

Expand Down Expand Up @@ -551,7 +551,7 @@ def test_parent_cron_after_child(init_and_plan_context: t.Callable):
]


@time_machine.travel("2023-01-08 00:00:00")
@time_machine.travel("2023-01-08 00:00:00 UTC")
@pytest.mark.parametrize(
"forward_only, expected_intervals",
[
Expand Down Expand Up @@ -606,7 +606,7 @@ def test_cron_not_aligned_with_day_boundary(
"sushi.waiter_revenue_by_day", raise_if_missing=True
)

with time_machine.travel("2023-01-08 00:10:00"): # Past model's cron.
with time_machine.travel("2023-01-08 00:10:00 UTC"): # Past model's cron.
plan = context.plan(
"dev", select_models=[model.name], no_prompts=True, skip_tests=True, enable_preview=True
)
Expand All @@ -618,7 +618,7 @@ def test_cron_not_aligned_with_day_boundary(
]


@time_machine.travel("2023-01-08 15:00:00")
@time_machine.travel("2023-01-08 15:00:00 UTC")
def test_forward_only_parent_created_in_dev_child_created_in_prod(
init_and_plan_context: t.Callable,
):
Expand Down Expand Up @@ -672,7 +672,7 @@ def test_forward_only_parent_created_in_dev_child_created_in_prod(
context.apply(plan)


@time_machine.travel("2023-01-08 15:00:00")
@time_machine.travel("2023-01-08 15:00:00 UTC")
def test_plan_set_choice_is_reflected_in_missing_intervals(init_and_plan_context: t.Callable):
context, plan = init_and_plan_context("examples/sushi")
context.apply(plan)
Expand Down Expand Up @@ -933,7 +933,7 @@ def test_non_breaking_change_after_forward_only_in_dev(
assert prod_df.empty


@time_machine.travel("2023-01-08 15:00:00")
@time_machine.travel("2023-01-08 15:00:00 UTC")
def test_indirect_non_breaking_change_after_forward_only_in_dev(init_and_plan_context: t.Callable):
context, plan = init_and_plan_context("examples/sushi")
context.apply(plan)
Expand Down Expand Up @@ -1055,7 +1055,7 @@ def test_indirect_non_breaking_change_after_forward_only_in_dev(init_and_plan_co
assert not context.plan("prod", no_prompts=True, skip_tests=True).requires_backfill


@time_machine.travel("2023-01-08 15:00:00")
@time_machine.travel("2023-01-08 15:00:00 UTC")
def test_forward_only_precedence_over_indirect_non_breaking(init_and_plan_context: t.Callable):
context, plan = init_and_plan_context("examples/sushi")
context.apply(plan)
Expand Down Expand Up @@ -1128,14 +1128,14 @@ def test_forward_only_precedence_over_indirect_non_breaking(init_and_plan_contex
assert not context.plan("prod", no_prompts=True, skip_tests=True).requires_backfill


@time_machine.travel("2023-01-08 15:00:00")
@time_machine.travel("2023-01-08 15:00:00 UTC")
def test_run_with_select_models(
init_and_plan_context: t.Callable,
):
context, plan = init_and_plan_context("examples/sushi")
context.apply(plan)

with time_machine.travel("2023-01-09 00:00:00"):
with time_machine.travel("2023-01-09 00:00:00 UTC"):
assert context.run(select_models=["*waiter_revenue_by_day"])

snapshots = context.state_sync.state_sync.get_snapshots(context.snapshots.values())
Expand All @@ -1159,7 +1159,7 @@ def test_run_with_select_models(
}


@time_machine.travel("2023-01-08 15:00:00")
@time_machine.travel("2023-01-08 15:00:00 UTC")
def test_run_with_select_models_no_auto_upstream(
init_and_plan_context: t.Callable,
):
Expand Down Expand Up @@ -1195,7 +1195,7 @@ def test_run_with_select_models_no_auto_upstream(
}


@time_machine.travel("2023-01-08 15:00:00")
@time_machine.travel("2023-01-08 15:00:00 UTC")
def test_select_models(init_and_plan_context: t.Callable):
context, plan = init_and_plan_context("examples/sushi")
context.apply(plan)
Expand Down Expand Up @@ -1273,7 +1273,7 @@ def test_select_models(init_and_plan_context: t.Callable):
)


@time_machine.travel("2023-01-08 15:00:00")
@time_machine.travel("2023-01-08 15:00:00 UTC")
def test_select_unchanged_model_for_backfill(init_and_plan_context: t.Callable):
context, plan = init_and_plan_context("examples/sushi")
context.apply(plan)
Expand Down Expand Up @@ -1347,14 +1347,14 @@ def test_select_unchanged_model_for_backfill(init_and_plan_context: t.Callable):
assert {o.name for o in schema_objects} == {"waiter_revenue_by_day", "top_waiters"}


@time_machine.travel("2023-01-08 15:00:00")
@time_machine.travel("2023-01-08 15:00:00 UTC")
def test_max_interval_end_per_model_not_applied_when_end_is_provided(
init_and_plan_context: t.Callable,
):
context, plan = init_and_plan_context("examples/sushi")
context.apply(plan)

with time_machine.travel("2023-01-09 00:00:00"):
with time_machine.travel("2023-01-09 00:00:00 UTC"):
context.run()

plan = context.plan(
Expand All @@ -1363,7 +1363,7 @@ def test_max_interval_end_per_model_not_applied_when_end_is_provided(
context.apply(plan)


@time_machine.travel("2023-01-08 15:00:00")
@time_machine.travel("2023-01-08 15:00:00 UTC")
def test_select_models_for_backfill(init_and_plan_context: t.Callable):
context, _ = init_and_plan_context("examples/sushi")

Expand Down Expand Up @@ -1431,7 +1431,7 @@ def test_select_models_for_backfill(init_and_plan_context: t.Callable):
)


@time_machine.travel("2023-01-08 15:00:00")
@time_machine.travel("2023-01-08 15:00:00 UTC")
def test_dbt_select_star_is_directly_modified(sushi_test_dbt_context: Context):
context = sushi_test_dbt_context

Expand Down Expand Up @@ -1476,7 +1476,7 @@ def test_model_attr(sushi_test_dbt_context: Context, assert_exp_eq):
)


@time_machine.travel("2023-01-08 15:00:00")
@time_machine.travel("2023-01-08 15:00:00 UTC")
def test_incremental_by_partition(init_and_plan_context: t.Callable):
context, plan = init_and_plan_context("examples/sushi")
context.apply(plan)
Expand Down Expand Up @@ -1530,7 +1530,7 @@ def test_incremental_by_partition(init_and_plan_context: t.Callable):
]


@time_machine.travel("2023-01-08 15:00:00")
@time_machine.travel("2023-01-08 15:00:00 UTC")
def test_custom_materialization(init_and_plan_context: t.Callable):
context, _ = init_and_plan_context("examples/sushi")

Expand Down Expand Up @@ -1565,7 +1565,7 @@ def insert(
assert custom_insert_called


@time_machine.travel("2023-01-08 15:00:00")
@time_machine.travel("2023-01-08 15:00:00 UTC")
def test_unaligned_start_snapshot_with_non_deployable_downstream(init_and_plan_context: t.Callable):
context, _ = init_and_plan_context("examples/sushi")

Expand Down Expand Up @@ -1620,7 +1620,7 @@ def test_unaligned_start_snapshot_with_non_deployable_downstream(init_and_plan_c
assert snapshot_interval.intervals[0][0] == to_timestamp("2023-01-07")


@time_machine.travel("2023-01-08 15:00:00")
@time_machine.travel("2023-01-08 15:00:00 UTC")
def test_restatement_plan_ignores_changes(init_and_plan_context: t.Callable):
context, plan = init_and_plan_context("examples/sushi")
context.apply(plan)
Expand Down Expand Up @@ -1651,7 +1651,7 @@ def test_restatement_plan_ignores_changes(init_and_plan_context: t.Callable):
context.apply(plan)


@time_machine.travel("2023-01-08 15:00:00")
@time_machine.travel("2023-01-08 15:00:00 UTC")
def test_plan_against_expired_environment(init_and_plan_context: t.Callable):
context, plan = init_and_plan_context("examples/sushi")
context.apply(plan)
Expand Down Expand Up @@ -1682,7 +1682,7 @@ def test_plan_against_expired_environment(init_and_plan_context: t.Callable):
context.apply(plan)


@time_machine.travel("2023-01-08 15:00:00")
@time_machine.travel("2023-01-08 15:00:00 UTC")
def test_new_forward_only_model_concurrent_versions(init_and_plan_context: t.Callable):
context, plan = init_and_plan_context("examples/sushi")
context.apply(plan)
Expand Down Expand Up @@ -1774,7 +1774,7 @@ def test_new_forward_only_model_concurrent_versions(init_and_plan_context: t.Cal
assert df.to_dict() == {"ds": {0: "2023-01-07"}, "b": {0: None}}


@time_machine.travel("2023-01-08 15:00:00")
@time_machine.travel("2023-01-08 15:00:00 UTC")
def test_new_forward_only_model_same_dev_environment(init_and_plan_context: t.Callable):
context, plan = init_and_plan_context("examples/sushi")
context.apply(plan)
Expand Down Expand Up @@ -1877,7 +1877,7 @@ def test_plan_twice_with_star_macro_yields_no_diff(tmp_path: Path):
assert not new_plan.new_snapshots


@time_machine.travel("2023-01-08 15:00:00")
@time_machine.travel("2023-01-08 15:00:00 UTC")
def test_create_environment_no_changes_with_selector(init_and_plan_context: t.Callable):
context, plan = init_and_plan_context("examples/sushi")
context.apply(plan)
Expand All @@ -1893,7 +1893,7 @@ def test_create_environment_no_changes_with_selector(init_and_plan_context: t.Ca
assert {o.name for o in schema_objects} == {"top_waiters"}


@time_machine.travel("2023-01-08 15:00:00")
@time_machine.travel("2023-01-08 15:00:00 UTC")
def test_empty_bacfkill(init_and_plan_context: t.Callable):
context, _ = init_and_plan_context("examples/sushi")

Expand Down
6 changes: 3 additions & 3 deletions tests/core/test_plan.py
Original file line number Diff line number Diff line change
@@ -1,6 +1,6 @@
import logging
import typing as t
from datetime import datetime, timedelta
from datetime import timedelta
from unittest.mock import patch

import pytest
Expand Down Expand Up @@ -727,7 +727,7 @@ def test_missing_intervals_lookback(make_snapshot, mocker: MockerFixture):


@pytest.mark.slow
@time_machine.travel(datetime.now(), tick=False)
@time_machine.travel(now(), tick=False)
def test_restate_models(sushi_context_pre_scheduling: Context):
plan = sushi_context_pre_scheduling.plan(
restate_models=["sushi.waiter_revenue_by_day", "tag:expensive"], no_prompts=True
Expand Down Expand Up @@ -781,7 +781,7 @@ def test_restate_models(sushi_context_pre_scheduling: Context):


@pytest.mark.slow
@time_machine.travel(datetime.now(), tick=False)
@time_machine.travel(now(), tick=False)
def test_restate_models_with_existing_missing_intervals(sushi_context: Context):
yesterday_ts = to_timestamp(yesterday_ds())

Expand Down
2 changes: 1 addition & 1 deletion tests/dbt/test_integration.py
Original file line number Diff line number Diff line change
Expand Up @@ -289,7 +289,7 @@ def _init_test(
adapter.create_schema("sushi")
if test_type.is_sqlmesh_runtime:
self._replace_source_table(adapter, [])
with time_machine.travel("2019-12-31 00:00:00"):
with time_machine.travel("2019-12-31 00:00:00 UTC"):
context.plan("prod", auto_apply=True, no_prompts=True) # type: ignore
return run, adapter, context

Expand Down
Loading

0 comments on commit 0f52d9c

Please sign in to comment.