Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
6 changes: 3 additions & 3 deletions tests/api/common/test_delete_dag.py
Original file line number Diff line number Diff line change
Expand Up @@ -19,11 +19,11 @@

import pytest

import datetime
from airflow import models
from airflow.api.common.delete_dag import delete_dag
from airflow.exceptions import AirflowException, DagNotFound
from airflow.operators.dummy import DummyOperator
from airflow.utils.dates import days_ago
from airflow.utils.session import create_session
from airflow.utils.state import State
from airflow.utils.types import DagRunType
Expand Down Expand Up @@ -73,11 +73,11 @@ def setup_dag_models(self, for_sub_dag=False):

task = DummyOperator(
task_id='dummy',
dag=models.DAG(dag_id=self.key, default_args={'start_date': days_ago(2)}),
dag=models.DAG(dag_id=self.key, default_args={'start_date': datetime.datetime(2022, 1, 1)}),
owner='airflow',
)

test_date = days_ago(1)
test_date = datetime.datetime(2022, 1, 1)
with create_session() as session:
session.add(DM(dag_id=self.key, fileloc=self.dag_file_path, is_subdag=for_sub_dag))
dr = DR(dag_id=self.key, run_type=DagRunType.MANUAL, run_id="test", execution_date=test_date)
Expand Down
6 changes: 3 additions & 3 deletions tests/api/common/test_mark_tasks.py
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@
from datetime import timedelta

import pytest
import datetime
from sqlalchemy.orm import eagerload

from airflow import models
Expand All @@ -32,7 +33,6 @@
)
from airflow.models import DagRun
from airflow.utils import timezone
from airflow.utils.dates import days_ago
from airflow.utils.session import create_session, provide_session
from airflow.utils.state import State
from airflow.utils.types import DagRunType
Expand All @@ -58,7 +58,7 @@ def create_dags(cls, dagbag):
cls.dag1 = dagbag.get_dag('miscellaneous_test_dag')
cls.dag2 = dagbag.get_dag('example_subdag_operator')
cls.dag3 = dagbag.get_dag('example_trigger_target_dag')
cls.execution_dates = [days_ago(2), days_ago(1)]
cls.execution_dates = [datetime.datetime(2022, 1, 1), datetime.datetime(2022, 1, 1)]
start_date3 = cls.dag3.start_date
cls.dag3_execution_dates = [
start_date3,
Expand Down Expand Up @@ -425,7 +425,7 @@ def setup_class(cls):
cls.dag1.sync_to_db()
cls.dag2 = dagbag.dags['example_subdag_operator']
cls.dag2.sync_to_db()
cls.execution_dates = [days_ago(2), days_ago(1), days_ago(0)]
cls.execution_dates = [datetime.datetime(2022, 1, 1), datetime.datetime(2022, 1, 1), datetime.datetime(2022, 1, 1)]

def setup_method(self):
clear_db_runs()
Expand Down
3 changes: 1 addition & 2 deletions tests/api_connexion/endpoints/test_extra_link_endpoint.py
Original file line number Diff line number Diff line change
Expand Up @@ -29,7 +29,6 @@
from airflow.plugins_manager import AirflowPlugin
from airflow.providers.google.cloud.operators.bigquery import BigQueryExecuteQueryOperator
from airflow.security import permissions
from airflow.utils.dates import days_ago
from airflow.utils.timezone import datetime
from airflow.utils.types import DagRunType
from tests.test_utils.api_connexion_utils import create_user, delete_user
Expand Down Expand Up @@ -95,7 +94,7 @@ def _create_dag():
with DAG(
dag_id="TEST_DAG_ID",
default_args=dict(
start_date=days_ago(2),
start_date=datetime.datetime(2022, 1, 1),
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I suspect we should move away from datetime.datetime to pendulum.datetime for start_date values in unit tests too based on the conversation in #21646. @eladkal WDYT?

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Do you mean just for tests or also for example dags?
I have no objection.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

For unit tests as well as example DAGs for consistency in tests, examples, docs, etc. The core example DAGs have been transitioned but the provider ones haven't yet.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I too think pendulum is a better alternative for datetime

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Then yeah lets do that

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Of course! Not a problem at all

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

ok thanks

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

80a6ae5

Another pull request after swapping datetime with pendulum where start dates are involved

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

For tests specifically I actually prefer datetime.datetime since it better ensures we are coercing timezones correctly internally.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

That one is also done in the previous pull request. I guess my task is done so please let me know if there is anything more to update.

),
) as dag:
BigQueryExecuteQueryOperator(task_id="TEST_SINGLE_QUERY", sql="SELECT 1")
Expand Down
4 changes: 2 additions & 2 deletions tests/cli/commands/test_task_command.py
Original file line number Diff line number Diff line change
Expand Up @@ -30,21 +30,21 @@
import pytest
from parameterized import parameterized

import datetime
from airflow import DAG
from airflow.cli import cli_parser
from airflow.cli.commands import task_command
from airflow.configuration import conf
from airflow.exceptions import AirflowException, DagRunNotFound
from airflow.models import DagBag, DagRun, TaskInstance
from airflow.utils import timezone
from airflow.utils.dates import days_ago
from airflow.utils.session import create_session
from airflow.utils.state import State
from airflow.utils.types import DagRunType
from tests.test_utils.config import conf_vars
from tests.test_utils.db import clear_db_runs

DEFAULT_DATE = days_ago(1)
DEFAULT_DATE = datetime.datetime(2022, 1, 1)
ROOT_FOLDER = os.path.realpath(
os.path.join(os.path.dirname(os.path.realpath(__file__)), os.pardir, os.pardir)
)
Expand Down
17 changes: 8 additions & 9 deletions tests/dag_processing/test_processor.py
Original file line number Diff line number Diff line change
Expand Up @@ -34,7 +34,6 @@
from airflow.models.taskinstance import SimpleTaskInstance
from airflow.operators.dummy import DummyOperator
from airflow.utils import timezone
from airflow.utils.dates import days_ago
from airflow.utils.session import create_session
from airflow.utils.state import State
from airflow.utils.types import DagRunType
Expand Down Expand Up @@ -113,7 +112,7 @@ def test_dag_file_processor_sla_miss_callback(self, create_dummy_dag):

# Create dag with a start of 1 day ago, but an sla of 0
# so we'll already have an sla_miss on the books.
test_start_date = days_ago(1)
test_start_date = datetime.datetime(2022, 1, 1)
dag, task = create_dummy_dag(
dag_id='test_sla_miss',
task_id='dummy',
Expand Down Expand Up @@ -142,7 +141,7 @@ def test_dag_file_processor_sla_miss_callback_invalid_sla(self, create_dummy_dag
# Create dag with a start of 1 day ago, but an sla of 0
# so we'll already have an sla_miss on the books.
# Pass anything besides a timedelta object to the sla argument.
test_start_date = days_ago(1)
test_start_date = datetime.datetime(2022, 1, 1)
dag, task = create_dummy_dag(
dag_id='test_sla_miss',
task_id='dummy',
Expand Down Expand Up @@ -170,7 +169,7 @@ def test_dag_file_processor_sla_miss_callback_sent_notification(self, create_dum

# Create dag with a start of 2 days ago, but an sla of 1 day
# ago so we'll already have an sla_miss on the books
test_start_date = days_ago(2)
test_start_date = datetime.datetime(2022, 1, 1)
dag, task = create_dummy_dag(
dag_id='test_sla_miss',
task_id='dummy',
Expand Down Expand Up @@ -206,7 +205,7 @@ def test_dag_file_processor_sla_miss_doesnot_raise_integrity_error(self, dag_mak

# Create dag with a start of 2 days ago, but an sla of 1 day
# ago so we'll already have an sla_miss on the books
test_start_date = days_ago(2)
test_start_date = datetime.datetime(2022, 1, 1)
with dag_maker(
dag_id='test_sla_miss',
default_args={'start_date': test_start_date, 'sla': datetime.timedelta(days=1)},
Expand Down Expand Up @@ -247,7 +246,7 @@ def test_dag_file_processor_sla_miss_callback_exception(self, mock_stats_incr, c

sla_callback = MagicMock(side_effect=RuntimeError('Could not call function'))

test_start_date = days_ago(2)
test_start_date = datetime.datetime(2022, 1, 1)
dag, task = create_dummy_dag(
dag_id='test_sla_miss',
task_id='dummy',
Expand Down Expand Up @@ -277,7 +276,7 @@ def test_dag_file_processor_only_collect_emails_from_sla_missed_tasks(
):
session = settings.Session()

test_start_date = days_ago(2)
test_start_date = datetime.datetime(2022, 1, 1)
email1 = '[email protected]'
dag, task = create_dummy_dag(
dag_id='test_sla_miss',
Expand Down Expand Up @@ -317,7 +316,7 @@ def test_dag_file_processor_sla_miss_email_exception(
# Mock the callback function so we can verify that it was not called
mock_send_email.side_effect = RuntimeError('Could not send an email')

test_start_date = days_ago(2)
test_start_date = datetime.datetime(2022, 1, 1)
dag, task = create_dummy_dag(
dag_id='test_sla_miss',
task_id='dummy',
Expand Down Expand Up @@ -347,7 +346,7 @@ def test_dag_file_processor_sla_miss_deleted_task(self, create_dummy_dag):
"""
session = settings.Session()

test_start_date = days_ago(2)
test_start_date = datetime.datetime(2022, 1, 1)
dag, task = create_dummy_dag(
dag_id='test_sla_miss',
task_id='dummy',
Expand Down
4 changes: 2 additions & 2 deletions tests/dags/test_default_views.py
Original file line number Diff line number Diff line change
Expand Up @@ -16,9 +16,9 @@
# specific language governing permissions and limitations
# under the License.
from airflow.models import DAG
from airflow.utils.dates import days_ago
import datetime

args = {'owner': 'airflow', 'retries': 3, 'start_date': days_ago(2)}
args = {'owner': 'airflow', 'retries': 3, 'start_date': datetime.datetime(2022,1,1)}

tree_dag = DAG(
dag_id='test_tree_view',
Expand Down
5 changes: 2 additions & 3 deletions tests/dags/test_example_bash_operator.py
Original file line number Diff line number Diff line change
Expand Up @@ -16,13 +16,12 @@
# specific language governing permissions and limitations
# under the License.
from datetime import timedelta

import datetime
Comment on lines 18 to +19
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

If from datetime import timedelta already present why not doing
from datetime import datetime, timedelta

and use datetime(2022,1,1) rather than datetime.datetime(2022,1,1)

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

oh! didn't see that there. There are a lot of files and I checked each one manually for maximum efficiency so forgot to use some of the already present libraries. If you want I can change them again quickly

from airflow.models import DAG
from airflow.operators.bash import BashOperator
from airflow.operators.dummy import DummyOperator
from airflow.utils.dates import days_ago

args = {'owner': 'airflow', 'retries': 3, 'start_date': days_ago(2)}
args = {'owner': 'airflow', 'retries': 3, 'start_date': datetime.datetime(2022,1,1)}

dag = DAG(
dag_id='test_example_bash_operator',
Expand Down
5 changes: 2 additions & 3 deletions tests/jobs/test_backfill_job.py
Original file line number Diff line number Diff line change
Expand Up @@ -41,7 +41,6 @@
from airflow.models.taskinstance import TaskInstanceKey
from airflow.operators.dummy import DummyOperator
from airflow.utils import timezone
from airflow.utils.dates import days_ago
from airflow.utils.session import create_session
from airflow.utils.state import State
from airflow.utils.timeout import timeout
Expand Down Expand Up @@ -1532,8 +1531,8 @@ def test_mapped_dag(self, dag_id):

job = BackfillJob(
dag=dag,
start_date=days_ago(1),
end_date=days_ago(1),
start_date= datetime.datetime(2022,1,1),
end_date=datetime.datetime(2022,1,1),
donot_pickle=True,
executor=SequentialExecutor(),
)
Expand Down
4 changes: 2 additions & 2 deletions tests/operators/test_python.py
Original file line number Diff line number Diff line change
Expand Up @@ -42,7 +42,7 @@
)
from airflow.utils import timezone
from airflow.utils.context import AirflowContextDeprecationWarning, Context
from airflow.utils.dates import days_ago
import datetime
from airflow.utils.session import create_session
from airflow.utils.state import State
from airflow.utils.trigger_rule import TriggerRule
Expand Down Expand Up @@ -1134,7 +1134,7 @@ def f():
DEFAULT_ARGS = {
"owner": "test",
"depends_on_past": True,
"start_date": days_ago(1),
"start_date": datetime.datetime(2022,1,1),
"end_date": datetime.today(),
"schedule_interval": "@once",
"retries": 1,
Expand Down
4 changes: 2 additions & 2 deletions tests/test_utils/perf/dags/perf_dag_1.py
Original file line number Diff line number Diff line change
Expand Up @@ -22,11 +22,11 @@

from airflow.models import DAG
from airflow.operators.bash_operator import BashOperator
from airflow.utils.dates import days_ago
import datetime

args = {
'owner': 'airflow',
'start_date': days_ago(3),
'start_date': datetime.datetime(2022,1,1),
}

dag = DAG(
Expand Down
4 changes: 2 additions & 2 deletions tests/test_utils/perf/dags/perf_dag_2.py
Original file line number Diff line number Diff line change
Expand Up @@ -22,11 +22,11 @@

from airflow.models import DAG
from airflow.operators.bash import BashOperator
from airflow.utils.dates import days_ago
import datetime

args = {
'owner': 'airflow',
'start_date': days_ago(3),
'start_date': datetime.datetime(2022,1,1),
}

dag = DAG(
Expand Down
4 changes: 2 additions & 2 deletions tests/utils/test_task_group.py
Original file line number Diff line number Diff line change
Expand Up @@ -26,7 +26,7 @@
from airflow.operators.bash import BashOperator
from airflow.operators.dummy import DummyOperator
from airflow.operators.python import PythonOperator
from airflow.utils.dates import days_ago
import datetime
from airflow.utils.task_group import MappedTaskGroup, TaskGroup
from airflow.www.views import dag_edges, task_group_to_dict
from tests.models import DEFAULT_DATE
Expand Down Expand Up @@ -1027,7 +1027,7 @@ def add_one(i):
def increment(num):
return num + 1

@dag(schedule_interval=None, start_date=days_ago(1), default_args={"owner": "airflow"})
@dag(schedule_interval=None, start_date=datetime.datetime(2022,1,1), default_args={"owner": "airflow"})
def wrap():
total_1 = one()
assert isinstance(total_1, XComArg)
Expand Down