sunank200 commented on code in PR #43902:
URL: https://github.com/apache/airflow/pull/43902#discussion_r1843033737


##########
providers/src/airflow/providers/trino/hooks/trino.py:
##########
@@ -23,33 +23,39 @@
 from typing import TYPE_CHECKING, Any, Iterable, Mapping, TypeVar
 
 import trino
+from packaging.version import Version
 from trino.exceptions import DatabaseError
 from trino.transaction import IsolationLevel
 
+from airflow import __version__ as airflow_version
 from airflow.configuration import conf
 from airflow.exceptions import AirflowException
 from airflow.providers.common.sql.hooks.sql import DbApiHook
 from airflow.utils.helpers import exactly_one
 from airflow.utils.operator_helpers import AIRFLOW_VAR_NAME_FORMAT_MAPPING, 
DEFAULT_FORMAT_PREFIX
 
+AIRFLOW_VERSION = Version(airflow_version)
+AIRFLOW_V_3_0_PLUS = Version(AIRFLOW_VERSION.base_version) >= Version("3.0.0")
+
 if TYPE_CHECKING:
     from airflow.models import Connection
 
 T = TypeVar("T")
 
 
 def generate_trino_client_info() -> str:
-    """Return json string with dag_id, task_id, execution_date and 
try_number."""
+    """Return json string with dag_id, task_id, logical_date and try_number."""
     context_var = {
         format_map["default"].replace(DEFAULT_FORMAT_PREFIX, ""): 
os.environ.get(
             format_map["env_var_format"], ""
         )
         for format_map in AIRFLOW_VAR_NAME_FORMAT_MAPPING.values()
     }
+    date_key = "logical_date" if AIRFLOW_V_3_0_PLUS else "execution_date"
     task_info = {
         "dag_id": context_var["dag_id"],
         "task_id": context_var["task_id"],
-        "execution_date": context_var["execution_date"],
+        date_key: context_var["logical_date"] if AIRFLOW_V_3_0_PLUS else 
context_var["execution_date"],

Review Comment:
   Changed it



##########
providers/src/airflow/providers/standard/operators/python.py:
##########
@@ -360,35 +359,25 @@ class _BasePythonVirtualenvOperator(PythonOperator, 
metaclass=ABCMeta):
         "ds_nodash",
         "expanded_ti_count",
         "inlets",
-        "next_ds",
-        "next_ds_nodash",
         "outlets",
-        "prev_ds",
-        "prev_ds_nodash",
         "run_id",
         "task_instance_key_str",
         "test_mode",
-        "tomorrow_ds",
-        "tomorrow_ds_nodash",
         "ts",
         "ts_nodash",
         "ts_nodash_with_tz",
-        "yesterday_ds",
-        "yesterday_ds_nodash",
     }
     PENDULUM_SERIALIZABLE_CONTEXT_KEYS = {
         "data_interval_end",
         "data_interval_start",
-        "execution_date",
         "logical_date",
-        "next_execution_date",
         "prev_data_interval_end_success",
         "prev_data_interval_start_success",
-        "prev_execution_date",
-        "prev_execution_date_success",
         "prev_start_date_success",
         "prev_end_date_success",
     }

Review Comment:
   Added it back and added the comment



##########
tests/cli/commands/test_task_command.py:
##########
@@ -183,84 +181,6 @@ def test_test_filters_secrets(self, capsys):
             task_command.task_test(args)
         assert capsys.readouterr().out.endswith(f"{not_password}\n")
 
-    def test_cli_test_different_path(self, session, tmp_path):

Review Comment:
   Changed it.



##########
tests/api_connexion/endpoints/test_task_instance_endpoint.py:
##########
@@ -1748,25 +1735,12 @@ def test_should_assert_call_mocked_api(self, 
mock_set_task_instance_state, sessi
                 {
                     "dag_id": "example_python_operator",
                     "dag_run_id": "TEST_DAG_RUN_ID",
-                    "execution_date": "2020-01-01T00:00:00+00:00",
+                    "logical_date": "2020-01-01T00:00:00+00:00",
                     "task_id": "print_the_context",
                 }
             ]
         }
 
-        mock_set_task_instance_state.assert_called_once_with(
-            commit=False,
-            downstream=True,
-            run_id=None,
-            execution_date=DEFAULT_DATETIME_1,
-            future=True,
-            past=True,
-            state="failed",
-            task_id="print_the_context",
-            upstream=True,
-            session=session,
-        )

Review Comment:
   Added it back 



##########
tests/api_connexion/endpoints/test_task_instance_endpoint.py:
##########
@@ -2500,56 +2461,6 @@ def test_should_respond_empty_non_scheduled(self, 
session):
         assert response.status_code == 200, response.text
         assert response.json == {"dependencies": []}
 
-    @pytest.mark.parametrize(
-        "state, dependencies",
-        [
-            (
-                State.SCHEDULED,
-                {
-                    "dependencies": [
-                        {
-                            "name": "Execution Date",
-                            "reason": "The execution date is 
2020-01-01T00:00:00+00:00 but this is "
-                            "before the task's start date 
2021-01-01T00:00:00+00:00.",
-                        },
-                        {
-                            "name": "Execution Date",
-                            "reason": "The execution date is 
2020-01-01T00:00:00+00:00 but this is "
-                            "before the task's DAG's start date 
2021-01-01T00:00:00+00:00.",
-                        },
-                    ],
-                },
-            ),
-            (
-                State.NONE,
-                {
-                    "dependencies": [
-                        {
-                            "name": "Execution Date",
-                            "reason": "The execution date is 
2020-01-01T00:00:00+00:00 but this is before the task's start date 
2021-01-01T00:00:00+00:00.",
-                        },
-                        {
-                            "name": "Execution Date",
-                            "reason": "The execution date is 
2020-01-01T00:00:00+00:00 but this is before the task's DAG's start date 
2021-01-01T00:00:00+00:00.",
-                        },
-                        {"name": "Task Instance State", "reason": "Task is in 
the 'None' state."},
-                    ]
-                },
-            ),
-        ],
-    )
-    @provide_session
-    def test_should_respond_dependencies(self, session, state, dependencies):
-        self.create_task_instances(session, task_instances=[{"state": state}], 
update_extras=True)
-
-        response = self.client.get(
-            
"api/v1/dags/example_python_operator/dagRuns/TEST_DAG_RUN_ID/taskInstances/"
-            "print_the_context/dependencies",
-            environ_overrides={"REMOTE_USER": "test"},
-        )
-        assert response.status_code == 200, response.text
-        assert response.json == dependencies

Review Comment:
   Added it back



##########
providers/tests/google/cloud/transfers/test_gcs_to_gcs.py:
##########
@@ -495,10 +495,9 @@ def test_execute_with_empty_destination_bucket(self, 
mock_hook):
 
         with mock.patch.object(operator.log, "warning") as mock_warn:
             operator.execute(None)
-            mock_warn.assert_called_once_with(
+            mock_warn.assert_any_call(
                 "destination_bucket is None. Defaulting it to source_bucket 
(%s)", TEST_BUCKET
             )
-            assert operator.destination_bucket == operator.source_bucket
 

Review Comment:
   Changed it 



##########
providers/tests/celery/cli/test_celery_command.py:
##########
@@ -197,7 +197,7 @@ def test_worker_started_with_required_arguments(self, 
mock_celery_app, mock_pope
                 "--hostname",
                 celery_hostname,
                 "--loglevel",
-                conf.get("logging", "CELERY_LOGGING_LEVEL"),
+                conf.get("logging", "CELERY_LOGGING_LEVEL", fallback="INFO"),

Review Comment:
   I have changed it. I think I was testing something and I forgot to revert it 
back.



##########
tests/api_connexion/schemas/test_dag_run_schema.py:
##########
@@ -129,13 +128,13 @@ def test_deserialize(self, serialized_dagrun, 
expected_result):
         assert result == expected_result
 
     def test_autofill_fields(self):
-        """Dag_run_id and execution_date fields are autogenerated if missing"""
+        """Dag_run_id fields are autogenerated if missing"""

Review Comment:
   Changed it



##########
providers/tests/apache/hive/hooks/test_hive.py:
##########
@@ -881,7 +896,6 @@ def test_default_values(self):
         assert hook.use_beeline
         assert hook.auth is None
         assert hook.sub_process is None
-        assert hook.mapred_queue == "airflow"

Review Comment:
   Added it back



##########
providers/tests/amazon/aws/operators/test_eks.py:
##########
@@ -339,19 +339,6 @@ def 
test_fargate_compute_missing_fargate_pod_execution_role_arn(self):
         ):
             missing_fargate_pod_execution_role_arn.execute({})
 
-    @mock.patch.object(EksHook, "create_cluster")
-    def test_eks_create_cluster_short_circuit_early(self, mock_create_cluster, 
caplog):
-        mock_create_cluster.return_value = None
-        eks_create_cluster_operator = EksCreateClusterOperator(
-            task_id=TASK_ID,
-            **self.create_cluster_params,
-            compute=None,
-            wait_for_completion=False,
-            deferrable=False,
-        )
-        eks_create_cluster_operator.execute({})
-        assert len(caplog.records) == 0
-

Review Comment:
   Added it back. I think this was rebase problem



##########
tests/cli/commands/test_task_command.py:
##########
@@ -589,20 +509,6 @@ def test_mapped_task_render_with_template(self, dag_maker):
         assert 'echo "2022-01-01"' in output
         assert 'echo "2022-01-08"' in output
 
-    @mock.patch("airflow.cli.commands.task_command.select")
-    @mock.patch("sqlalchemy.orm.session.Session.scalars")
-    @mock.patch("airflow.cli.commands.task_command.DagRun")
-    def test_task_render_with_custom_timetable(self, mock_dagrun, 
mock_scalars, mock_select):

Review Comment:
   Added it back



##########
tests/api_fastapi/core_api/routes/public/test_dags.py:
##########
@@ -178,6 +178,7 @@ class TestGetDags(TestDagEndpoint):
     )
     def test_get_dags(self, test_client, query_params, expected_total_entries, 
expected_ids):
         response = test_client.get("/public/dags", params=query_params)
+        print("Response JSON:", response.json())

Review Comment:
   removed it



##########
providers/tests/openlineage/plugins/test_macros.py:
##########
@@ -39,37 +44,19 @@ def test_lineage_job_name():
         dag_id="dag_id",
         task_id="task_id",
         try_number=1,
-        execution_date=datetime(2020, 1, 1, 1, 1, 1, 0, tzinfo=timezone.utc),
+        **{LOGICAL_DATE_KEY: datetime(2020, 1, 1, 1, 1, 1, 0, 
tzinfo=timezone.utc)},
     )
     assert lineage_job_name(task_instance) == "dag_id.task_id"
 
 
-def test_lineage_run_id():
-    task_instance = mock.MagicMock(
-        dag_id="dag_id",
-        task_id="task_id",
-        dag_run=mock.MagicMock(run_id="run_id"),
-        execution_date=datetime(2020, 1, 1, 1, 1, 1, 0, tzinfo=timezone.utc),
-        try_number=1,
-    )
-
-    call_result1 = lineage_run_id(task_instance)
-    call_result2 = lineage_run_id(task_instance)
-
-    # random part value does not matter, it just have to be the same for the 
same TaskInstance
-    assert call_result1 == call_result2
-    # execution_date is used as most significant bits of UUID
-    assert call_result1.startswith("016f5e9e-c4c8-")

Review Comment:
   Added it back



##########
tests/api_connexion/endpoints/test_dag_run_endpoint.py:
##########
@@ -448,7 +442,7 @@ def test_should_raises_401_unauthenticated(self):
         "fields",
         [
             ["dag_run_id", "logical_date"],
-            ["dag_run_id", "state", "conf", "execution_date"],
+            ["dag_run_id", "state", "conf"],

Review Comment:
   Changed it



##########
tests/api_connexion/endpoints/test_task_instance_endpoint.py:
##########
@@ -1797,30 +1771,17 @@ def 
test_should_assert_call_mocked_api_when_run_id(self, mock_set_task_instance_
                 {
                     "dag_id": "example_python_operator",
                     "dag_run_id": "TEST_DAG_RUN_ID",
-                    "execution_date": "2020-01-01T00:00:00+00:00",
+                    "logical_date": "2020-01-01T00:00:00+00:00",
                     "task_id": "print_the_context",
                 }
             ]
         }
 
-        mock_set_task_instance_state.assert_called_once_with(
-            commit=False,
-            downstream=True,
-            run_id=run_id,
-            execution_date=None,
-            future=True,
-            past=True,
-            state="failed",
-            task_id="print_the_context",
-            upstream=True,
-            session=session,
-        )

Review Comment:
   Added it back.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscr...@airflow.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org

Reply via email to