Skip to content

Commit 949e865

Browse files
authored
Consolidate pendulum usage in prefect.server.services (#17100)
1 parent b87f0e8 commit 949e865

8 files changed

+66
-68
lines changed

src/prefect/server/database/orm_models.py

+38-41
Original file line numberDiff line numberDiff line change
@@ -5,7 +5,6 @@
55
from pathlib import Path
66
from typing import TYPE_CHECKING, Any, ClassVar, Optional, Union
77

8-
import pendulum
98
import sqlalchemy as sa
109
from sqlalchemy import FetchedValue
1110
from sqlalchemy.dialects import postgresql
@@ -46,11 +45,9 @@
4645
Timestamp,
4746
)
4847
from prefect.server.utilities.encryption import decrypt_fernet, encrypt_fernet
48+
from prefect.types._datetime import DateTime, now
4949
from prefect.utilities.names import generate_slug
5050

51-
if TYPE_CHECKING:
52-
DateTime = pendulum.DateTime
53-
5451
# for 'plain JSON' columns, use the postgresql variant (which comes with an
5552
# extra operator) and fall back to the generic JSON variant for SQLite
5653
sa_JSON: postgresql.JSON = postgresql.JSON().with_variant(sa.JSON(), "sqlite")
@@ -89,7 +86,7 @@ class Base(DeclarativeBase):
8986
),
9087
type_annotation_map={
9188
uuid.UUID: UUID,
92-
pendulum.DateTime: Timestamp,
89+
DateTime: Timestamp,
9390
},
9491
)
9592

@@ -121,17 +118,17 @@ def __tablename__(cls) -> str:
121118
default=uuid.uuid4,
122119
)
123120

124-
created: Mapped[pendulum.DateTime] = mapped_column(
125-
server_default=sa.func.now(), default=lambda: pendulum.now("UTC")
121+
created: Mapped[DateTime] = mapped_column(
122+
server_default=sa.func.now(), default=lambda: now("UTC")
126123
)
127124

128125
# onupdate is only called when statements are actually issued
129126
# against the database. until COMMIT is issued, this column
130127
# will not be updated
131-
updated: Mapped[pendulum.DateTime] = mapped_column(
128+
updated: Mapped[DateTime] = mapped_column(
132129
index=True,
133130
server_default=sa.func.now(),
134-
default=lambda: pendulum.now("UTC"),
131+
default=lambda: now("UTC"),
135132
onupdate=sa.func.now(),
136133
server_onupdate=FetchedValue(),
137134
)
@@ -170,8 +167,8 @@ class FlowRunState(Base):
170167
type: Mapped[schemas.states.StateType] = mapped_column(
171168
sa.Enum(schemas.states.StateType, name="state_type"), index=True
172169
)
173-
timestamp: Mapped[pendulum.DateTime] = mapped_column(
174-
server_default=sa.func.now(), default=lambda: pendulum.now("UTC")
170+
timestamp: Mapped[DateTime] = mapped_column(
171+
server_default=sa.func.now(), default=lambda: now("UTC")
175172
)
176173
name: Mapped[str] = mapped_column(index=True)
177174
message: Mapped[Optional[str]]
@@ -235,8 +232,8 @@ class TaskRunState(Base):
235232
type: Mapped[schemas.states.StateType] = mapped_column(
236233
sa.Enum(schemas.states.StateType, name="state_type"), index=True
237234
)
238-
timestamp: Mapped[pendulum.DateTime] = mapped_column(
239-
server_default=sa.func.now(), default=lambda: pendulum.now("UTC")
235+
timestamp: Mapped[DateTime] = mapped_column(
236+
server_default=sa.func.now(), default=lambda: now("UTC")
240237
)
241238
name: Mapped[str] = mapped_column(index=True)
242239
message: Mapped[Optional[str]]
@@ -358,7 +355,7 @@ class TaskRunStateCache(Base):
358355
"""
359356

360357
cache_key: Mapped[str] = mapped_column()
361-
cache_expiration: Mapped[Optional[pendulum.DateTime]]
358+
cache_expiration: Mapped[Optional[DateTime]]
362359
task_run_state_id: Mapped[uuid.UUID]
363360

364361
@declared_attr.directive
@@ -385,12 +382,12 @@ class Run(Base):
385382
sa.Enum(schemas.states.StateType, name="state_type")
386383
)
387384
state_name: Mapped[Optional[str]]
388-
state_timestamp: Mapped[Optional[pendulum.DateTime]]
385+
state_timestamp: Mapped[Optional[DateTime]]
389386
run_count: Mapped[int] = mapped_column(server_default="0", default=0)
390-
expected_start_time: Mapped[Optional[pendulum.DateTime]]
391-
next_scheduled_start_time: Mapped[Optional[pendulum.DateTime]]
392-
start_time: Mapped[Optional[pendulum.DateTime]]
393-
end_time: Mapped[Optional[pendulum.DateTime]]
387+
expected_start_time: Mapped[Optional[DateTime]]
388+
next_scheduled_start_time: Mapped[Optional[DateTime]]
389+
start_time: Mapped[Optional[DateTime]]
390+
end_time: Mapped[Optional[DateTime]]
394391
total_run_time: Mapped[datetime.timedelta] = mapped_column(
395392
server_default="0", default=datetime.timedelta(0)
396393
)
@@ -403,7 +400,7 @@ def estimated_run_time(self) -> datetime.timedelta:
403400
if self.state_type and self.state_type == schemas.states.StateType.RUNNING:
404401
if TYPE_CHECKING:
405402
assert self.state_timestamp is not None
406-
return self.total_run_time + (pendulum.now("UTC") - self.state_timestamp)
403+
return self.total_run_time + (now("UTC") - self.state_timestamp)
407404
else:
408405
return self.total_run_time
409406

@@ -445,10 +442,10 @@ def estimated_start_time_delta(self) -> datetime.timedelta:
445442
elif (
446443
self.start_time is None
447444
and self.expected_start_time
448-
and self.expected_start_time < pendulum.now("UTC")
445+
and self.expected_start_time < now("UTC")
449446
and self.state_type not in schemas.states.TERMINAL_STATES
450447
):
451-
return pendulum.now("UTC") - self.expected_start_time
448+
return now("UTC") - self.expected_start_time
452449
else:
453450
return datetime.timedelta(0)
454451

@@ -660,7 +657,7 @@ class TaskRun(Run):
660657
task_key: Mapped[str] = mapped_column()
661658
dynamic_key: Mapped[str] = mapped_column()
662659
cache_key: Mapped[Optional[str]]
663-
cache_expiration: Mapped[Optional[pendulum.DateTime]]
660+
cache_expiration: Mapped[Optional[DateTime]]
664661
task_version: Mapped[Optional[str]]
665662
flow_run_run_count: Mapped[int] = mapped_column(server_default="0", default=0)
666663
empirical_policy: Mapped[schemas.core.TaskRunPolicy] = mapped_column(
@@ -803,7 +800,7 @@ class Deployment(Base):
803800
path: Mapped[Optional[str]]
804801
entrypoint: Mapped[Optional[str]]
805802

806-
last_polled: Mapped[Optional[pendulum.DateTime]]
803+
last_polled: Mapped[Optional[DateTime]]
807804
status: Mapped[DeploymentStatus] = mapped_column(
808805
sa.Enum(DeploymentStatus, name="deployment_status"),
809806
default=DeploymentStatus.NOT_READY,
@@ -913,7 +910,7 @@ class Log(Base):
913910
message: Mapped[str] = mapped_column(sa.Text)
914911

915912
# The client-side timestamp of this logged statement.
916-
timestamp: Mapped[pendulum.DateTime] = mapped_column(index=True)
913+
timestamp: Mapped[DateTime] = mapped_column(index=True)
917914

918915
__table_args__: Any = (
919916
sa.Index(
@@ -1104,7 +1101,7 @@ class WorkQueue(Base):
11041101
concurrency_limit: Mapped[Optional[int]]
11051102
priority: Mapped[int]
11061103

1107-
last_polled: Mapped[Optional[pendulum.DateTime]]
1104+
last_polled: Mapped[Optional[DateTime]]
11081105
status: Mapped[WorkQueueStatus] = mapped_column(
11091106
sa.Enum(WorkQueueStatus, name="work_queue_status"),
11101107
default=WorkQueueStatus.NOT_READY,
@@ -1150,7 +1147,7 @@ class WorkPool(Base):
11501147
default=WorkPoolStatus.NOT_READY,
11511148
server_default=WorkPoolStatus.NOT_READY,
11521149
)
1153-
last_transitioned_status_at: Mapped[Optional[pendulum.DateTime]]
1150+
last_transitioned_status_at: Mapped[Optional[DateTime]]
11541151
last_status_event_id: Mapped[Optional[uuid.UUID]]
11551152

11561153
__table_args__: Any = (sa.UniqueConstraint("name"),)
@@ -1164,8 +1161,8 @@ class Worker(Base):
11641161
)
11651162

11661163
name: Mapped[str]
1167-
last_heartbeat_time: Mapped[pendulum.DateTime] = mapped_column(
1168-
server_default=sa.func.now(), default=lambda: pendulum.now("UTC")
1164+
last_heartbeat_time: Mapped[DateTime] = mapped_column(
1165+
server_default=sa.func.now(), default=lambda: now("UTC")
11691166
)
11701167
heartbeat_interval_seconds: Mapped[Optional[int]]
11711168

@@ -1194,8 +1191,8 @@ class Agent(Base):
11941191
sa.ForeignKey("work_queue.id"), index=True
11951192
)
11961193

1197-
last_activity_time: Mapped[pendulum.DateTime] = mapped_column(
1198-
server_default=sa.func.now(), default=lambda: pendulum.now("UTC")
1194+
last_activity_time: Mapped[DateTime] = mapped_column(
1195+
server_default=sa.func.now(), default=lambda: now("UTC")
11991196
)
12001197

12011198
__table_args__: Any = (sa.UniqueConstraint("name"),)
@@ -1249,7 +1246,7 @@ class FlowRunInput(Base):
12491246
class CsrfToken(Base):
12501247
token: Mapped[str]
12511248
client: Mapped[str] = mapped_column(unique=True)
1252-
expiration: Mapped[pendulum.DateTime]
1249+
expiration: Mapped[DateTime]
12531250

12541251

12551252
class Automation(Base):
@@ -1314,14 +1311,14 @@ class AutomationBucket(Base):
13141311

13151312
last_event: Mapped[Optional[ReceivedEvent]] = mapped_column(Pydantic(ReceivedEvent))
13161313

1317-
start: Mapped[pendulum.DateTime]
1318-
end: Mapped[pendulum.DateTime]
1314+
start: Mapped[DateTime]
1315+
end: Mapped[DateTime]
13191316

13201317
count: Mapped[int]
13211318

13221319
last_operation: Mapped[Optional[str]]
13231320

1324-
triggered_at: Mapped[Optional[pendulum.DateTime]]
1321+
triggered_at: Mapped[Optional[DateTime]]
13251322

13261323

13271324
class AutomationRelatedResource(Base):
@@ -1367,7 +1364,7 @@ class CompositeTriggerChildFiring(Base):
13671364

13681365
child_trigger_id: Mapped[uuid.UUID]
13691366
child_firing_id: Mapped[uuid.UUID]
1370-
child_fired_at: Mapped[Optional[pendulum.DateTime]]
1367+
child_fired_at: Mapped[Optional[DateTime]]
13711368
child_firing: Mapped[Firing] = mapped_column(Pydantic(Firing))
13721369

13731370

@@ -1383,7 +1380,7 @@ class AutomationEventFollower(Base):
13831380
scope: Mapped[str] = mapped_column(default="", index=True)
13841381
leader_event_id: Mapped[uuid.UUID] = mapped_column(index=True)
13851382
follower_event_id: Mapped[uuid.UUID]
1386-
received: Mapped[pendulum.DateTime] = mapped_column(index=True)
1383+
received: Mapped[DateTime] = mapped_column(index=True)
13871384
follower: Mapped[ReceivedEvent] = mapped_column(Pydantic(ReceivedEvent))
13881385

13891386

@@ -1407,7 +1404,7 @@ def __tablename__(cls) -> str:
14071404
sa.Index("ix_events__event_related_occurred", "event", "related", "occurred"),
14081405
)
14091406

1410-
occurred: Mapped[pendulum.DateTime]
1407+
occurred: Mapped[DateTime]
14111408
event: Mapped[str] = mapped_column(sa.Text())
14121409
resource_id: Mapped[str] = mapped_column(sa.Text())
14131410
resource: Mapped[dict[str, Any]] = mapped_column(JSON())
@@ -1418,8 +1415,8 @@ def __tablename__(cls) -> str:
14181415
JSON(), server_default="[]", default=list
14191416
)
14201417
payload: Mapped[dict[str, Any]] = mapped_column(JSON())
1421-
received: Mapped[pendulum.DateTime]
1422-
recorded: Mapped[pendulum.DateTime]
1418+
received: Mapped[DateTime]
1419+
recorded: Mapped[DateTime]
14231420
follows: Mapped[Optional[uuid.UUID]]
14241421

14251422

@@ -1436,7 +1433,7 @@ def __tablename__(cls) -> str:
14361433
),
14371434
)
14381435

1439-
occurred: Mapped[pendulum.DateTime]
1436+
occurred: Mapped[DateTime]
14401437
resource_id: Mapped[str] = mapped_column(sa.Text())
14411438
resource_role: Mapped[str] = mapped_column(sa.Text())
14421439
resource: Mapped[dict[str, Any]] = mapped_column(sa_JSON)

src/prefect/server/services/cancellation_cleanup.py

+3-2
Original file line numberDiff line numberDiff line change
@@ -3,10 +3,10 @@
33
"""
44

55
import asyncio
6+
import datetime
67
from typing import Any, Optional
78
from uuid import UUID
89

9-
import pendulum
1010
import sqlalchemy as sa
1111
from sqlalchemy.sql.expression import or_
1212

@@ -18,6 +18,7 @@
1818
from prefect.settings import PREFECT_API_SERVICES_CANCELLATION_CLEANUP_LOOP_SECONDS
1919
from prefect.settings.context import get_current_settings
2020
from prefect.settings.models.server.services import ServicesBaseSetting
21+
from prefect.types._datetime import now
2122

2223
NON_TERMINAL_STATES = list(set(states.StateType) - states.TERMINAL_STATES)
2324

@@ -64,7 +65,7 @@ async def clean_up_cancelled_flow_run_task_runs(
6465
.where(
6566
db.FlowRun.state_type == states.StateType.CANCELLED,
6667
db.FlowRun.end_time.is_not(None),
67-
db.FlowRun.end_time >= (pendulum.now("UTC").subtract(days=1)),
68+
db.FlowRun.end_time >= (now("UTC") - datetime.timedelta(days=1)),
6869
)
6970
.limit(self.batch_size)
7071
)

src/prefect/server/services/foreman.py

+3-3
Original file line numberDiff line numberDiff line change
@@ -5,7 +5,6 @@
55
from datetime import timedelta
66
from typing import Any, Optional
77

8-
import pendulum
98
import sqlalchemy as sa
109

1110
from prefect.server import models
@@ -29,6 +28,7 @@
2928
get_current_settings,
3029
)
3130
from prefect.settings.models.server.services import ServicesBaseSetting
31+
from prefect.types._datetime import now
3232

3333

3434
class Foreman(LoopService):
@@ -179,7 +179,7 @@ async def _mark_deployments_as_not_ready(self, db: PrefectDBInterface) -> None:
179179
session (AsyncSession): The session to use for the database operation.
180180
"""
181181
async with db.session_context(begin_transaction=True) as session:
182-
status_timeout_threshold = pendulum.now("UTC") - timedelta(
182+
status_timeout_threshold = now("UTC") - timedelta(
183183
seconds=self._deployment_last_polled_timeout_seconds
184184
)
185185
deployment_id_select_stmt = (
@@ -222,7 +222,7 @@ async def _mark_work_queues_as_not_ready(self, db: PrefectDBInterface):
222222
session (AsyncSession): The session to use for the database operation.
223223
"""
224224
async with db.session_context(begin_transaction=True) as session:
225-
status_timeout_threshold = pendulum.now("UTC") - timedelta(
225+
status_timeout_threshold = now("UTC") - timedelta(
226226
seconds=self._work_queue_last_polled_timeout_seconds
227227
)
228228
id_select_stmt = (

src/prefect/server/services/late_runs.py

+4-4
Original file line numberDiff line numberDiff line change
@@ -9,7 +9,6 @@
99
import datetime
1010
from typing import TYPE_CHECKING, Any
1111

12-
import pendulum
1312
import sqlalchemy as sa
1413
from sqlalchemy.ext.asyncio import AsyncSession
1514

@@ -26,6 +25,7 @@
2625
)
2726
from prefect.settings.context import get_current_settings
2827
from prefect.settings.models.server.services import ServicesBaseSetting
28+
from prefect.types._datetime import DateTime, now
2929

3030
if TYPE_CHECKING:
3131
from uuid import UUID
@@ -67,7 +67,7 @@ async def run_once(self, db: PrefectDBInterface) -> None:
6767
- Querying for flow runs in a scheduled state that are Scheduled to start in the past
6868
- For any runs past the "late" threshold, setting the flow run state to a new `Late` state
6969
"""
70-
scheduled_to_start_before = pendulum.now("UTC").subtract(
70+
scheduled_to_start_before = now("UTC") - datetime.timedelta(
7171
seconds=self.mark_late_after.total_seconds()
7272
)
7373

@@ -93,7 +93,7 @@ async def run_once(self, db: PrefectDBInterface) -> None:
9393
@inject_db
9494
def _get_select_late_flow_runs_query(
9595
self, scheduled_to_start_before: datetime.datetime, db: PrefectDBInterface
96-
) -> sa.Select[tuple["UUID", pendulum.DateTime | None]]:
96+
) -> sa.Select[tuple["UUID", DateTime | None]]:
9797
"""
9898
Returns a sqlalchemy query for late flow runs.
9999
@@ -120,7 +120,7 @@ def _get_select_late_flow_runs_query(
120120
async def _mark_flow_run_as_late(
121121
self,
122122
session: AsyncSession,
123-
flow_run: sa.Row[tuple["UUID", pendulum.DateTime | None]],
123+
flow_run: sa.Row[tuple["UUID", DateTime | None]],
124124
) -> None:
125125
"""
126126
Mark a flow run as late.

src/prefect/server/services/pause_expirations.py

+2-2
Original file line numberDiff line numberDiff line change
@@ -5,7 +5,6 @@
55
import asyncio
66
from typing import Any, Optional
77

8-
import pendulum
98
import sqlalchemy as sa
109
from sqlalchemy.ext.asyncio import AsyncSession
1110

@@ -18,6 +17,7 @@
1817
from prefect.settings import PREFECT_API_SERVICES_PAUSE_EXPIRATIONS_LOOP_SECONDS
1918
from prefect.settings.context import get_current_settings
2019
from prefect.settings.models.server.services import ServicesBaseSetting
20+
from prefect.types._datetime import now
2121

2222

2323
class FailExpiredPauses(LoopService):
@@ -82,7 +82,7 @@ async def _mark_flow_run_as_failed(
8282
if (
8383
flow_run.state is not None
8484
and flow_run.state.state_details.pause_timeout is not None
85-
and flow_run.state.state_details.pause_timeout < pendulum.now("UTC")
85+
and flow_run.state.state_details.pause_timeout < now("UTC")
8686
):
8787
await models.flow_runs.set_flow_run_state(
8888
session=session,

0 commit comments

Comments
 (0)