Re: [PR] Rename execution_date to logical_date across codebase [airflow]

2024-11-15 Thread via GitHub


uranusjr merged PR #43902:
URL: https://github.com/apache/airflow/pull/43902


-- 
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



Re: [PR] Rename execution_date to logical_date across codebase [airflow]

2024-11-14 Thread via GitHub


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


##
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:
   Still deletion?



-- 
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



Re: [PR] Rename execution_date to logical_date across codebase [airflow]

2024-11-14 Thread via GitHub


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


##
providers/src/airflow/providers/presto/hooks/presto.py:
##
@@ -22,32 +22,38 @@
 from typing import TYPE_CHECKING, Any, Iterable, Mapping, TypeVar
 
 import prestodb
+from packaging.version import Version
 from prestodb.exceptions import DatabaseError
 from prestodb.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.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_presto_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:
   I had changed this but it was failing.



-- 
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



Re: [PR] Rename execution_date to logical_date across codebase [airflow]

2024-11-14 Thread via GitHub


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


##
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:
   It is there. Please check: 
https://github.com/apache/airflow/pull/43902/files#diff-0463235a55240b8bb3203f40f39a258c969ad69b3551ca1d9872073d3634d00a



-- 
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



Re: [PR] Rename execution_date to logical_date across codebase [airflow]

2024-11-14 Thread via GitHub


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


##
providers/src/airflow/providers/presto/hooks/presto.py:
##
@@ -22,32 +22,38 @@
 from typing import TYPE_CHECKING, Any, Iterable, Mapping, TypeVar
 
 import prestodb
+from packaging.version import Version
 from prestodb.exceptions import DatabaseError
 from prestodb.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.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_presto_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:
   Not changed



-- 
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



Re: [PR] Rename execution_date to logical_date across codebase [airflow]

2024-11-14 Thread via GitHub


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


##
providers/tests/openlineage/plugins/test_listener.py:
##


Review Comment:
   Changed it



-- 
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



Re: [PR] Rename execution_date to logical_date across codebase [airflow]

2024-11-14 Thread via GitHub


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


##
providers/src/airflow/providers/dbt/cloud/utils/openlineage.py:
##
@@ -142,7 +142,7 @@ async def get_artifacts_for_steps(steps, artifacts):
 parent_run_id = OpenLineageAdapter.build_task_instance_run_id(
 dag_id=task_instance.dag_id,
 task_id=operator.task_id,
-execution_date=task_instance.execution_date,
+logical_date=task_instance.logical_date,

Review Comment:
   The argument is not, but `task_instance.logical_date` does not work on 2.



-- 
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



Re: [PR] Rename execution_date to logical_date across codebase [airflow]

2024-11-14 Thread via GitHub


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


##
providers/tests/amazon/aws/links/test_base_aws.py:
##
@@ -21,14 +21,18 @@
 from unittest import mock
 
 import pytest
+from packaging.version import Version
 
+from airflow import __version__ as airflow_version
 from airflow.models.xcom import XCom
 from airflow.providers.amazon.aws.links.base_aws import BaseAwsLink
 from airflow.serialization.serialized_objects import SerializedDAG
 
-from tests_common.test_utils.compat import AIRFLOW_V_3_0_PLUS
 from tests_common.test_utils.mock_operators import MockOperator
 
+AIRFLOW_VERSION = Version(airflow_version)
+AIRFLOW_V_3_0_PLUS = Version(AIRFLOW_VERSION.base_version) >= Version("3.0.0")

Review Comment:
   Changed it



-- 
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



Re: [PR] Rename execution_date to logical_date across codebase [airflow]

2024-11-14 Thread via GitHub


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


##
tests/api_connexion/endpoints/test_task_instance_endpoint.py:
##
@@ -462,20 +462,6 @@ class TestGetTaskInstances(TestTaskInstanceEndpoint):
 @pytest.mark.parametrize(
 "task_instances, update_extras, url, expected_ti",
 [
-pytest.param(
-[
-{"execution_date": DEFAULT_DATETIME_1},
-{"execution_date": DEFAULT_DATETIME_1 + 
dt.timedelta(days=1)},
-{"execution_date": DEFAULT_DATETIME_1 + 
dt.timedelta(days=2)},
-],
-False,
-(
-"/api/v1/dags/example_python_operator/dagRuns/~/"
-
f"taskInstances?execution_date_lte={QUOTED_DEFAULT_DATETIME_STR_1}"
-),
-1,
-id="test execution date filter",
-),

Review Comment:
   Changed it



##
tests/assets/test_manager.py:
##


Review Comment:
   Changed it



-- 
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



Re: [PR] Rename execution_date to logical_date across codebase [airflow]

2024-11-14 Thread via GitHub


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.",
-   

Re: [PR] Rename execution_date to logical_date across codebase [airflow]

2024-11-14 Thread via GitHub


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


##
newsfragments/43902.significant.rst:
##


Review Comment:
   Changed it



##
providers/src/airflow/providers/google/cloud/operators/gcs.py:
##
@@ -36,13 +36,18 @@
 
 from google.api_core.exceptions import Conflict
 from google.cloud.exceptions import GoogleCloudError
+from packaging.version import Version
 
+from airflow import __version__ as airflow_version
 from airflow.exceptions import AirflowException, 
AirflowProviderDeprecationWarning
 from airflow.providers.google.cloud.hooks.gcs import GCSHook
 from airflow.providers.google.cloud.operators.cloud_base import 
GoogleCloudBaseOperator
 from airflow.providers.google.common.links.storage import FileDetailsLink, 
StorageLink
 from airflow.utils import timezone
 
+AIRFLOW_VERSION = Version(airflow_version)
+AIRFLOW_V_3_0_PLUS = Version(AIRFLOW_VERSION.base_version) >= Version("3.0.0")

Review Comment:
   Removed it



##
providers/src/airflow/providers/presto/hooks/presto.py:
##
@@ -22,32 +22,38 @@
 from typing import TYPE_CHECKING, Any, Iterable, Mapping, TypeVar
 
 import prestodb
+from packaging.version import Version
 from prestodb.exceptions import DatabaseError
 from prestodb.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.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_presto_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:
##
@@ -310,7 +309,7 @@ def get_tasks_to_skip():
 self.skip(
 dag_run=dag_run,
 tasks=to_skip,
-execution_date=cast("DateTime", dag_run.execution_date),  # 
type: ignore[call-arg]
+execution_date=cast("DateTime", dag_run.execution_date),  # 
type: ignore[call-arg, union-attr]

Review Comment:
   Changed it



##
providers/src/airflow/providers/google/cloud/operators/gcs.py:
##
@@ -36,13 +36,18 @@
 
 from google.api_core.exceptions import Conflict
 from google.cloud.exceptions import GoogleCloudError
+from packaging.version import Version
 
+from airflow import __version__ as airflow_version
 from airflow.exceptions import AirflowException, 
AirflowProviderDeprecationWarning
 from airflow.providers.google.cloud.hooks.gcs import GCSHook
 from airflow.providers.google.cloud.operators.cloud_base import 
GoogleCloudBaseOperator
 from airflow.providers.google.common.links.storage import FileDetailsLink, 
StorageLink
 from airflow.utils import timezone
 
+AIRFLOW_VERSION = Version(airflow_version)
+AIRFLOW_V_3_0_PLUS = Version(AIRFLOW_VERSION.base_version) >= Version("3.0.0")

Review Comment:
   Removed it



##
providers/src/airflow/providers/opensearch/log/os_task_handler.py:
##
@@ -306,17 +313,32 @@ def _render_log_id(self, ti: TaskInstance | 
TaskInstanceKey, try_number: int) ->
 data_interval_end = data_interval[1].isoformat()
 else:
 data_interval_end = ""
-execution_date = dag_run.execution_date.isoformat()
-
-return log_id_template.format(
-dag_id=ti.dag_id,
-task_id=ti.task_id,
-run_id=getattr(ti, "run_id", ""),
-data_interval_start=data_interval_start,
-data_interval_end=data_interval_end,
-execution_date=execution_date,
-try_number=try_number,
-map_index=getattr(ti, "map_index", ""),
+logical_date = (
+dag_run.logical_date.isoformat() if AIRFLOW_V_3_0_PLUS else 
dag_run.execution_date.isoformat()
+)

Review Comment:
   Changed it



##
providers/src/airflow/providers/

Re: [PR] Rename execution_date to logical_date across codebase [airflow]

2024-11-13 Thread via GitHub


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


##
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:
   Should not be removed?



-- 
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



Re: [PR] Rename execution_date to logical_date across codebase [airflow]

2024-11-13 Thread via GitHub


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


##
providers/tests/openlineage/plugins/test_listener.py:
##


Review Comment:
   Also many changes that do not belong here



-- 
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



Re: [PR] Rename execution_date to logical_date across codebase [airflow]

2024-11-13 Thread via GitHub


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


##
providers/tests/google/common/hooks/test_base_google.py:
##


Review Comment:
   Changes in this file look out of place



-- 
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



Re: [PR] Rename execution_date to logical_date across codebase [airflow]

2024-11-13 Thread via GitHub


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


##
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:
   This change does not look right?



-- 
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



Re: [PR] Rename execution_date to logical_date across codebase [airflow]

2024-11-13 Thread via GitHub


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


##
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:
   This should include `"logical_date"`



-- 
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



Re: [PR] Rename execution_date to logical_date across codebase [airflow]

2024-11-13 Thread via GitHub


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


##
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:
   Should not be removed



##
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:
   Should not be removed



##
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:
   Should not be removed



-- 
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



Re: [PR] Rename execution_date to logical_date across codebase [airflow]

2024-11-13 Thread via GitHub


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


##
tests/api_connexion/endpoints/test_task_instance_endpoint.py:
##
@@ -462,20 +462,6 @@ class TestGetTaskInstances(TestTaskInstanceEndpoint):
 @pytest.mark.parametrize(
 "task_instances, update_extras, url, expected_ti",
 [
-pytest.param(
-[
-{"execution_date": DEFAULT_DATETIME_1},
-{"execution_date": DEFAULT_DATETIME_1 + 
dt.timedelta(days=1)},
-{"execution_date": DEFAULT_DATETIME_1 + 
dt.timedelta(days=2)},
-],
-False,
-(
-"/api/v1/dags/example_python_operator/dagRuns/~/"
-
f"taskInstances?execution_date_lte={QUOTED_DEFAULT_DATETIME_STR_1}"
-),
-1,
-id="test execution date filter",
-),

Review Comment:
   These should not be removed (but modified to use logical_date)



-- 
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



Re: [PR] Rename execution_date to logical_date across codebase [airflow]

2024-11-13 Thread via GitHub


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


##
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:
   ```suggestion
   """Dag_run_id and logical_date fields are autogenerated if missing"""
   ```



-- 
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



Re: [PR] Rename execution_date to logical_date across codebase [airflow]

2024-11-13 Thread via GitHub


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


##
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:
   Should be removed



-- 
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



Re: [PR] Rename execution_date to logical_date across codebase [airflow]

2024-11-13 Thread via GitHub


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


##
tests/assets/test_manager.py:
##


Review Comment:
   Multiple asserts removed in this file but should not be.



-- 
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



Re: [PR] Rename execution_date to logical_date across codebase [airflow]

2024-11-13 Thread via GitHub


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


##
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:
   This one probably should not be deleted?



##
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:
   This too



-- 
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



Re: [PR] Rename execution_date to logical_date across codebase [airflow]

2024-11-13 Thread via GitHub


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


##
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:
   Is this supposed to be deleted?



-- 
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



Re: [PR] Rename execution_date to logical_date across codebase [airflow]

2024-11-13 Thread via GitHub


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


##
providers/tests/fab/auth_manager/api_endpoints/test_auth.py:
##
@@ -20,13 +20,17 @@
 
 import pytest
 from flask_login import current_user
+from packaging.version import Version
+
+from airflow import __version__ as airflow_version
 
 from tests_common.test_utils.api_connexion_utils import assert_401
-from tests_common.test_utils.compat import AIRFLOW_V_3_0_PLUS
 from tests_common.test_utils.config import conf_vars
 from tests_common.test_utils.db import clear_db_pools
 from tests_common.test_utils.www import client_with_login
 
+AIRFLOW_VERSION = Version(airflow_version)
+AIRFLOW_V_3_0_PLUS = Version(AIRFLOW_VERSION.base_version) >= Version("3.0.0")

Review Comment:
   Not needed



-- 
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



Re: [PR] Rename execution_date to logical_date across codebase [airflow]

2024-11-13 Thread via GitHub


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


##
providers/tests/fab/auth_manager/api_endpoints/test_auth.py:
##
@@ -20,13 +20,17 @@
 
 import pytest
 from flask_login import current_user
+from packaging.version import Version
+
+from airflow import __version__ as airflow_version
 
 from tests_common.test_utils.api_connexion_utils import assert_401
-from tests_common.test_utils.compat import AIRFLOW_V_3_0_PLUS
 from tests_common.test_utils.config import conf_vars
 from tests_common.test_utils.db import clear_db_pools
 from tests_common.test_utils.www import client_with_login
 
+AIRFLOW_VERSION = Version(airflow_version)
+AIRFLOW_V_3_0_PLUS = Version(AIRFLOW_VERSION.base_version) >= Version("3.0.0")

Review Comment:
   Not needed



-- 
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



Re: [PR] Rename execution_date to logical_date across codebase [airflow]

2024-11-13 Thread via GitHub


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


##
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:
   Why does this need to change?



-- 
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



Re: [PR] Rename execution_date to logical_date across codebase [airflow]

2024-11-13 Thread via GitHub


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


##
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:
   Is this supposed to be deleted?



-- 
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



Re: [PR] Rename execution_date to logical_date across codebase [airflow]

2024-11-13 Thread via GitHub


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


##
providers/tests/amazon/aws/links/test_base_aws.py:
##
@@ -21,14 +21,18 @@
 from unittest import mock
 
 import pytest
+from packaging.version import Version
 
+from airflow import __version__ as airflow_version
 from airflow.models.xcom import XCom
 from airflow.providers.amazon.aws.links.base_aws import BaseAwsLink
 from airflow.serialization.serialized_objects import SerializedDAG
 
-from tests_common.test_utils.compat import AIRFLOW_V_3_0_PLUS
 from tests_common.test_utils.mock_operators import MockOperator
 
+AIRFLOW_VERSION = Version(airflow_version)
+AIRFLOW_V_3_0_PLUS = Version(AIRFLOW_VERSION.base_version) >= Version("3.0.0")

Review Comment:
   Not needed?



-- 
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



Re: [PR] Rename execution_date to logical_date across codebase [airflow]

2024-11-13 Thread via GitHub


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


##
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:
   ```suggestion
   date_key: context_var["logical_date"],
   ```



-- 
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



Re: [PR] Rename execution_date to logical_date across codebase [airflow]

2024-11-13 Thread via GitHub


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


##
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:
   These can not be deleted; the operator will not work correctly on Airflow 2. 
We can add comments these should be deleted when we drop Airflow 2 support on 
the provider.



-- 
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



Re: [PR] Rename execution_date to logical_date across codebase [airflow]

2024-11-13 Thread via GitHub


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


##
providers/src/airflow/providers/standard/operators/python.py:
##
@@ -310,7 +309,7 @@ def get_tasks_to_skip():
 self.skip(
 dag_run=dag_run,
 tasks=to_skip,
-execution_date=cast("DateTime", dag_run.execution_date),  # 
type: ignore[call-arg]
+execution_date=cast("DateTime", dag_run.execution_date),  # 
type: ignore[call-arg, union-attr]

Review Comment:
   ```suggestion
   execution_date=cast("DateTime", dag_run.logical_date),  # 
type: ignore[call-arg, union-attr]
   ```



-- 
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



Re: [PR] Rename execution_date to logical_date across codebase [airflow]

2024-11-13 Thread via GitHub


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


##
providers/src/airflow/providers/presto/hooks/presto.py:
##
@@ -22,32 +22,38 @@
 from typing import TYPE_CHECKING, Any, Iterable, Mapping, TypeVar
 
 import prestodb
+from packaging.version import Version
 from prestodb.exceptions import DatabaseError
 from prestodb.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.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_presto_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:
   ```suggestion
   date_key: context_var["logical_date"],
   ```



-- 
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



Re: [PR] Rename execution_date to logical_date across codebase [airflow]

2024-11-13 Thread via GitHub


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


##
providers/src/airflow/providers/opensearch/log/os_task_handler.py:
##
@@ -306,17 +313,32 @@ def _render_log_id(self, ti: TaskInstance | 
TaskInstanceKey, try_number: int) ->
 data_interval_end = data_interval[1].isoformat()
 else:
 data_interval_end = ""
-execution_date = dag_run.execution_date.isoformat()
-
-return log_id_template.format(
-dag_id=ti.dag_id,
-task_id=ti.task_id,
-run_id=getattr(ti, "run_id", ""),
-data_interval_start=data_interval_start,
-data_interval_end=data_interval_end,
-execution_date=execution_date,
-try_number=try_number,
-map_index=getattr(ti, "map_index", ""),
+logical_date = (
+dag_run.logical_date.isoformat() if AIRFLOW_V_3_0_PLUS else 
dag_run.execution_date.isoformat()
+)
+
+return (
+log_id_template.format(
+dag_id=ti.dag_id,
+task_id=ti.task_id,
+run_id=getattr(ti, "run_id", ""),
+data_interval_start=data_interval_start,
+data_interval_end=data_interval_end,
+logical_date=logical_date,
+try_number=try_number,
+map_index=getattr(ti, "map_index", ""),
+)
+if AIRFLOW_V_3_0_PLUS
+else log_id_template.format(
+dag_id=ti.dag_id,
+task_id=ti.task_id,
+run_id=getattr(ti, "run_id", ""),
+data_interval_start=data_interval_start,
+data_interval_end=data_interval_end,
+execution_date=logical_date,
+try_number=try_number,
+map_index=getattr(ti, "map_index", ""),
+)
 )

Review Comment:
   ```suggestion
   return log_id_template.format(
   dag_id=ti.dag_id,
   task_id=ti.task_id,
   run_id=getattr(ti, "run_id", ""),
   data_interval_start=data_interval_start,
   data_interval_end=data_interval_end,
   execution_date=logical_date,  # Airlow 2 Compatibility.
   logical_date=logical_date,
   try_number=try_number,
   map_index=ti.map_index,
   )
   ```



-- 
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



Re: [PR] Rename execution_date to logical_date across codebase [airflow]

2024-11-13 Thread via GitHub


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


##
providers/src/airflow/providers/opensearch/log/os_task_handler.py:
##
@@ -306,17 +313,32 @@ def _render_log_id(self, ti: TaskInstance | 
TaskInstanceKey, try_number: int) ->
 data_interval_end = data_interval[1].isoformat()
 else:
 data_interval_end = ""
-execution_date = dag_run.execution_date.isoformat()
-
-return log_id_template.format(
-dag_id=ti.dag_id,
-task_id=ti.task_id,
-run_id=getattr(ti, "run_id", ""),
-data_interval_start=data_interval_start,
-data_interval_end=data_interval_end,
-execution_date=execution_date,
-try_number=try_number,
-map_index=getattr(ti, "map_index", ""),
+logical_date = (
+dag_run.logical_date.isoformat() if AIRFLOW_V_3_0_PLUS else 
dag_run.execution_date.isoformat()
+)

Review Comment:
   ```suggestion
   logical_date = dag_run.logical_date.isoformat()
   ```



-- 
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



Re: [PR] Rename execution_date to logical_date across codebase [airflow]

2024-11-13 Thread via GitHub


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


##
providers/src/airflow/providers/google/cloud/sensors/gcs.py:
##
@@ -42,6 +44,8 @@
 from google.api_core.retry import Retry
 
 from airflow.utils.context import Context
+AIRFLOW_VERSION = Version(airflow_version)
+AIRFLOW_V_3_0_PLUS = Version(AIRFLOW_VERSION.base_version) >= Version("3.0.0")

Review Comment:
   Not needed?



-- 
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



Re: [PR] Rename execution_date to logical_date across codebase [airflow]

2024-11-13 Thread via GitHub


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


##
providers/src/airflow/providers/google/cloud/operators/gcs.py:
##
@@ -36,13 +36,18 @@
 
 from google.api_core.exceptions import Conflict
 from google.cloud.exceptions import GoogleCloudError
+from packaging.version import Version
 
+from airflow import __version__ as airflow_version
 from airflow.exceptions import AirflowException, 
AirflowProviderDeprecationWarning
 from airflow.providers.google.cloud.hooks.gcs import GCSHook
 from airflow.providers.google.cloud.operators.cloud_base import 
GoogleCloudBaseOperator
 from airflow.providers.google.common.links.storage import FileDetailsLink, 
StorageLink
 from airflow.utils import timezone
 
+AIRFLOW_VERSION = Version(airflow_version)
+AIRFLOW_V_3_0_PLUS = Version(AIRFLOW_VERSION.base_version) >= Version("3.0.0")

Review Comment:
   Not needed anymore?



-- 
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



Re: [PR] Rename execution_date to logical_date across codebase [airflow]

2024-11-13 Thread via GitHub


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


##
providers/src/airflow/providers/elasticsearch/log/es_task_handler.py:
##
@@ -266,15 +271,18 @@ def _render_log_id(self, ti: TaskInstance | 
TaskInstanceKey, try_number: int) ->
 data_interval_end = data_interval[1].isoformat()
 else:
 data_interval_end = ""
-execution_date = dag_run.execution_date.isoformat()
+logical_date = (
+dag_run.logical_date.isoformat() if AIRFLOW_V_3_0_PLUS else 
dag_run.execution_date.isoformat()
+)

Review Comment:
   ```suggestion
   logical_date = dag_run.logical_date.isoformat()
   ```
   
   `dag_run.logical_date` is available on Airflow 2.



-- 
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



Re: [PR] Rename execution_date to logical_date across codebase [airflow]

2024-11-13 Thread via GitHub


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


##
airflow/utils/context.py:
##
@@ -451,7 +426,6 @@ def context_copy_partial(source: Context, keys: 
Container[str]) -> Context:
 :meta private:
 """
 new = Context({k: v for k, v in source._context.items() if k in keys})
-new._deprecation_replacements = source._deprecation_replacements.copy()

Review Comment:
   This line should not be removed.



-- 
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



Re: [PR] Rename execution_date to logical_date across codebase [airflow]

2024-11-13 Thread via GitHub


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


##
newsfragments/43902.significant.rst:
##


Review Comment:
   This also needs to mention that some keys in the context have been removed.



-- 
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



Re: [PR] Rename execution_date to logical_date across codebase [airflow]

2024-11-13 Thread via GitHub


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


##
providers/src/airflow/providers/openlineage/plugins/adapter.py:
##
@@ -128,11 +128,11 @@ def build_task_instance_run_id(
 dag_id: str,
 task_id: str,
 try_number: int,
-execution_date: datetime,
+logical_date: datetime,

Review Comment:
   Noted



-- 
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



Re: [PR] Rename execution_date to logical_date across codebase [airflow]

2024-11-13 Thread via GitHub


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


##
providers/src/airflow/providers/standard/sensors/date_time.py:
##
@@ -69,7 +69,7 @@ class DateTimeSensor(BaseSensorOperator):
 
 DateTimeSensor(
 task_id="wait_for_0100",
-target_time="{{ next_execution_date.tomorrow().replace(hour=1) 
}}",
+target_time="{{ next_logical_date.tomorrow().replace(hour=1) 
}}",

Review Comment:
   Changed it



-- 
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



Re: [PR] Rename execution_date to logical_date across codebase [airflow]

2024-11-13 Thread via GitHub


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


##
airflow/models/taskreschedule.py:
##
@@ -75,7 +75,7 @@ class TaskReschedule(TaskInstanceDependencies):
 ),
 )
 dag_run = relationship("DagRun")
-execution_date = association_proxy("dag_run", "execution_date")

Review Comment:
   Keeping it for now as discussed.



##
airflow/models/xcom.py:
##
@@ -111,7 +111,7 @@ class BaseXCom(TaskInstanceDependencies, LoggingMixin):
 lazy="joined",
 passive_deletes="all",
 )
-execution_date = association_proxy("dag_run", "execution_date")

Review Comment:
   Keeping it for now as discussed.



-- 
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



Re: [PR] Rename execution_date to logical_date across codebase [airflow]

2024-11-13 Thread via GitHub


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


##
providers/src/airflow/providers/dbt/cloud/utils/openlineage.py:
##
@@ -142,7 +142,7 @@ async def get_artifacts_for_steps(steps, artifacts):
 parent_run_id = OpenLineageAdapter.build_task_instance_run_id(
 dag_id=task_instance.dag_id,
 task_id=operator.task_id,
-execution_date=task_instance.execution_date,
+logical_date=task_instance.logical_date,

Review Comment:
   I think this change is not needed for this PR right as per 
https://github.com/apache/airflow/pull/43902#discussion_r1839379068?



-- 
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



Re: [PR] Rename execution_date to logical_date across codebase [airflow]

2024-11-13 Thread via GitHub


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


##
providers/src/airflow/providers/google/cloud/operators/gcs.py:
##
@@ -794,7 +799,10 @@ def execute(self, context: Context) -> list[str]:
 orig_start = context["data_interval_start"]
 orig_end = context["data_interval_end"]
 except KeyError:
-orig_start = pendulum.instance(context["execution_date"])
+if AIRFLOW_V_3_0_PLUS:
+orig_start = pendulum.instance(context["logical_date"])
+else:
+orig_start = pendulum.instance(context["execution_date"])  # 
type: ignore[typeddict-item]

Review Comment:
   Changed it



-- 
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



Re: [PR] Rename execution_date to logical_date across codebase [airflow]

2024-11-13 Thread via GitHub


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


##
providers/src/airflow/providers/elasticsearch/log/es_task_handler.py:
##
@@ -266,17 +275,32 @@ def _render_log_id(self, ti: TaskInstance | 
TaskInstanceKey, try_number: int) ->
 data_interval_end = data_interval[1].isoformat()
 else:
 data_interval_end = ""
-execution_date = dag_run.execution_date.isoformat()
-
-return log_id_template.format(
-dag_id=ti.dag_id,
-task_id=ti.task_id,
-run_id=getattr(ti, "run_id", ""),
-data_interval_start=data_interval_start,
-data_interval_end=data_interval_end,
-execution_date=execution_date,
-try_number=try_number,
-map_index=getattr(ti, "map_index", ""),
+logical_date = (
+dag_run.logical_date.isoformat() if AIRFLOW_V_3_0_PLUS else 
dag_run.execution_date.isoformat()
+)
+
+return (
+log_id_template.format(
+dag_id=ti.dag_id,
+task_id=ti.task_id,
+run_id=getattr(ti, "run_id", ""),
+data_interval_start=data_interval_start,
+data_interval_end=data_interval_end,
+logical_date=logical_date,
+try_number=try_number,
+map_index=getattr(ti, "map_index", ""),
+)
+if AIRFLOW_V_3_0_PLUS
+else log_id_template.format(
+dag_id=ti.dag_id,
+task_id=ti.task_id,
+run_id=getattr(ti, "run_id", ""),
+data_interval_start=data_interval_start,
+data_interval_end=data_interval_end,
+execution_date=logical_date,
+try_number=try_number,
+map_index=getattr(ti, "map_index", ""),
+)
 )

Review Comment:
   Changed it



-- 
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



Re: [PR] Rename execution_date to logical_date across codebase [airflow]

2024-11-13 Thread via GitHub


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


##
providers/src/airflow/providers/elasticsearch/log/es_task_handler.py:
##
@@ -256,7 +261,11 @@ def _render_log_id(self, ti: TaskInstance | 
TaskInstanceKey, try_number: int) ->
 if self.json_format:
 data_interval_start = self._clean_date(data_interval[0])
 data_interval_end = self._clean_date(data_interval[1])
-execution_date = self._clean_date(dag_run.execution_date)
+logical_date = (
+self._clean_date(dag_run.logical_date)
+if AIRFLOW_V_3_0_PLUS
+else self._clean_date(dag_run.execution_date)
+)

Review Comment:
   Changed it.



-- 
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



Re: [PR] Rename execution_date to logical_date across codebase [airflow]

2024-11-13 Thread via GitHub


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


##
newsfragments/43902.significant.rst:
##
@@ -0,0 +1,5 @@
+Renamed ``execution_date`` to ``logical_date`` across the codebase to align 
with Airflow 3.0.
+
+The shift towards ``logical_date`` helps move away from the limitations of 
``execution_date``, particularly with dynamic DAG runs and cases where multiple 
runs occur at the same time. This change impacts database models, templates, 
and functions:

Review Comment:
   Changed it



##
newsfragments/43902.significant.rst:
##
@@ -0,0 +1,5 @@
+Renamed ``execution_date`` to ``logical_date`` across the codebase to align 
with Airflow 3.0.
+
+The shift towards ``logical_date`` helps move away from the limitations of 
``execution_date``, particularly with dynamic DAG runs and cases where multiple 
runs occur at the same time. This change impacts database models, templates, 
and functions:
+- Renamed columns and function references to ``logical_date``.
+- Removed unique constraints on ``execution_date`` to support concurrent DAG 
runs with the same logical date.

Review Comment:
   Removed it.



-- 
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



Re: [PR] Rename execution_date to logical_date across codebase [airflow]

2024-11-13 Thread via GitHub


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


##
airflow/utils/context.pyi:
##
@@ -102,24 +102,19 @@ class Context(TypedDict, total=False):
 ds: str
 ds_nodash: str
 exception: BaseException | str | None
-execution_date: DateTime
 expanded_ti_count: int | None
 inlets: list
 inlet_events: InletEventsAccessors
 logical_date: DateTime
 macros: Any
 map_index_template: str
-next_ds: str | None
-next_ds_nodash: str | None
-next_execution_date: DateTime | None
+next_logical_date: DateTime | None

Review Comment:
   I have removed: prev_logical_date_success, prev_logical_date and 
next_logical_date.
   
   



-- 
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



Re: [PR] Rename execution_date to logical_date across codebase [airflow]

2024-11-13 Thread via GitHub


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


##
airflow/api_connexion/openapi/v1.yaml:
##
@@ -4734,14 +4721,14 @@ components:
   description: The task ID.
   type: string
 
-execution_date:
+logical_date:
   description: The execution date. Either set this or dag_run_id but 
not both.

Review Comment:
   Changed it.



-- 
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



Re: [PR] Rename execution_date to logical_date across codebase [airflow]

2024-11-13 Thread via GitHub


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


##
airflow/api_fastapi/core_api/datamodels/event_logs.py:
##
@@ -33,7 +33,7 @@ class EventLogResponse(BaseModel):
 map_index: int | None
 try_number: int | None
 event: str
-execution_date: datetime | None = Field(alias="logical_date")
+logical_date: datetime | None = Field(alias="logical_date")

Review Comment:
   Changed it.



-- 
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



Re: [PR] Rename execution_date to logical_date across codebase [airflow]

2024-11-13 Thread via GitHub


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


##
airflow/api_connexion/openapi/v1.yaml:
##
@@ -4734,14 +4721,14 @@ components:
   description: The task ID.
   type: string
 
-execution_date:
+logical_date:
   description: The execution date. Either set this or dag_run_id but 
not both.

Review Comment:
   Changed it.



-- 
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



Re: [PR] Rename execution_date to logical_date across codebase [airflow]

2024-11-13 Thread via GitHub


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


##
airflow/models/taskinstance.py:
##
@@ -1124,24 +1085,19 @@ def get_triggering_events() -> dict[str, 
list[AssetEvent | AssetEventPydantic]]:
 "outlet_events": OutletEventAccessors(),
 "ds": ds,
 "ds_nodash": ds_nodash,
-"execution_date": logical_date,
+"logical_date": logical_date,
 "expanded_ti_count": expanded_ti_count,
 "inlets": task.inlets,
 "inlet_events": InletEventsAccessors(task.inlets, session=session),
-"logical_date": logical_date,
 "macros": macros,
 "map_index_template": task.map_index_template,
-"next_ds": get_next_ds(),
-"next_ds_nodash": get_next_ds_nodash(),
-"next_execution_date": get_next_execution_date(),
+"next_logical_date": get_next_logical_date(),

Review Comment:
   I have removed: prev_logical_date_success, prev_logical_date and 
next_logical_date.



-- 
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



Re: [PR] Rename execution_date to logical_date across codebase [airflow]

2024-11-13 Thread via GitHub


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


##
airflow/utils/context.py:
##


Review Comment:
   changed it



-- 
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



Re: [PR] Rename execution_date to logical_date across codebase [airflow]

2024-11-13 Thread via GitHub


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


##
providers/src/airflow/providers/apache/druid/transfers/hive_to_druid.py:
##
@@ -94,7 +94,7 @@ def __init__(
 self.sql = sql
 self.druid_datasource = druid_datasource
 self.ts_dim = ts_dim
-self.intervals = intervals or ["{{ ds }}/{{ tomorrow_ds }}"]
+self.intervals = intervals or ["{{ ds }}"]

Review Comment:
   changed it



-- 
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



Re: [PR] Rename execution_date to logical_date across codebase [airflow]

2024-11-13 Thread via GitHub


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


##
airflow/utils/context.py:
##
@@ -70,26 +66,21 @@
 "data_interval_start",
 "ds",
 "ds_nodash",
-"execution_date",
 "expanded_ti_count",
 "exception",
 "inlets",
 "inlet_events",
 "logical_date",
 "macros",
 "map_index_template",
-"next_ds",
-"next_ds_nodash",
-"next_execution_date",
+"next_logical_date",

Review Comment:
   I have removed: `prev_logical_date_success`, `prev_logical_date` and  
`next_logical_date`.



-- 
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



Re: [PR] Rename execution_date to logical_date across codebase [airflow]

2024-11-13 Thread via GitHub


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


##
airflow/api_connexion/openapi/v1.yaml:
##
@@ -4734,14 +4721,14 @@ components:
   description: The task ID.
   type: string
 
-execution_date:
+logical_date:
   description: The execution date. Either set this or dag_run_id but 
not both.

Review Comment:
   Changed it.



-- 
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



Re: [PR] Rename execution_date to logical_date across codebase [airflow]

2024-11-13 Thread via GitHub


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


##
airflow/api_connexion/openapi/v1.yaml:
##


Review Comment:
   I can do that in another PR.



-- 
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



Re: [PR] Rename execution_date to logical_date across codebase [airflow]

2024-11-12 Thread via GitHub


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


##
providers/src/airflow/providers/openlineage/plugins/adapter.py:
##
@@ -128,11 +128,11 @@ def build_task_instance_run_id(
 dag_id: str,
 task_id: str,
 try_number: int,
-execution_date: datetime,
+logical_date: datetime,

Review Comment:
   I did a quick search and at least 
`providers/src/airflow/providers/dbt/cloud/utils/openlineage.py` calls this 
function, so we need additional compatibility measures for different provider 
version combinations.
   
   It’s probably best to leave this function as-is for now (don’t change the 
`execution_date` argument in this PR) and figure out the considerations later.



-- 
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



Re: [PR] Rename execution_date to logical_date across codebase [airflow]

2024-11-12 Thread via GitHub


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


##
providers/src/airflow/providers/standard/sensors/date_time.py:
##
@@ -69,7 +69,7 @@ class DateTimeSensor(BaseSensorOperator):
 
 DateTimeSensor(
 task_id="wait_for_0100",
-target_time="{{ next_execution_date.tomorrow().replace(hour=1) 
}}",
+target_time="{{ next_logical_date.tomorrow().replace(hour=1) 
}}",

Review Comment:
   We should also remove `next_logical_date`. Since this is just an example, we 
can change it to use `data_interval_end` instead.



-- 
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



Re: [PR] Rename execution_date to logical_date across codebase [airflow]

2024-11-12 Thread via GitHub


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


##
providers/src/airflow/providers/google/cloud/operators/gcs.py:
##
@@ -794,7 +799,10 @@ def execute(self, context: Context) -> list[str]:
 orig_start = context["data_interval_start"]
 orig_end = context["data_interval_end"]
 except KeyError:
-orig_start = pendulum.instance(context["execution_date"])
+if AIRFLOW_V_3_0_PLUS:
+orig_start = pendulum.instance(context["logical_date"])
+else:
+orig_start = pendulum.instance(context["execution_date"])  # 
type: ignore[typeddict-item]

Review Comment:
   `logical_date` is in the context on Airflow 2 too, so we can always just use 
that
   
   ```suggestion
   orig_start = pendulum.instance(context["logical_date"])
   ```



-- 
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



Re: [PR] Rename execution_date to logical_date across codebase [airflow]

2024-11-12 Thread via GitHub


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


##
providers/src/airflow/providers/google/cloud/sensors/gcs.py:
##
@@ -179,16 +183,21 @@ def ts_function(context):
 Act as a default callback for the GoogleCloudStorageObjectUpdatedSensor.
 
 The default behaviour is check for the object being updated after the data 
interval's end,
-or execution_date + interval on Airflow versions prior to 2.2 (before 
AIP-39 implementation).
+or logical_date + interval on Airflow versions prior to 2.2 (before AIP-39 
implementation).
 """
 try:
 return context["data_interval_end"]
 except KeyError:
 from airflow.utils import timezone
 
-data_interval = context["dag"].infer_automated_data_interval(
-timezone.coerce_datetime(context["execution_date"])
-)
+if AIRFLOW_V_3_0_PLUS:
+data_interval = context["dag"].infer_automated_data_interval(
+timezone.coerce_datetime(context["logical_date"])
+)
+else:
+data_interval = context["dag"].infer_automated_data_interval(
+timezone.coerce_datetime(context["execution_date"])
+)

Review Comment:
   Weird… `logical_date` should be available since Airflow 2.3. I’ll take a 
look at this.
   
   
https://github.com/apache/airflow/commit/25a50bb1fbf0e228706c7927cb36570921881adb



-- 
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



Re: [PR] Rename execution_date to logical_date across codebase [airflow]

2024-11-12 Thread via GitHub


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


##
providers/src/airflow/providers/apache/druid/transfers/hive_to_druid.py:
##
@@ -94,7 +94,7 @@ def __init__(
 self.sql = sql
 self.druid_datasource = druid_datasource
 self.ts_dim = ts_dim
-self.intervals = intervals or ["{{ ds }}/{{ tomorrow_ds }}"]
+self.intervals = intervals or ["{{ ds }}"]

Review Comment:
   Hmm… we need to make this work across versions. Maybe something like this:
   
   ```suggestion
   self.intervals = intervals or ["{{ ds }}/{{ logical_date.add_days(1) 
| ds }}"]
   ```



-- 
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



Re: [PR] Rename execution_date to logical_date across codebase [airflow]

2024-11-12 Thread via GitHub


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


##
providers/src/airflow/providers/elasticsearch/log/es_task_handler.py:
##
@@ -266,17 +275,32 @@ def _render_log_id(self, ti: TaskInstance | 
TaskInstanceKey, try_number: int) ->
 data_interval_end = data_interval[1].isoformat()
 else:
 data_interval_end = ""
-execution_date = dag_run.execution_date.isoformat()
-
-return log_id_template.format(
-dag_id=ti.dag_id,
-task_id=ti.task_id,
-run_id=getattr(ti, "run_id", ""),
-data_interval_start=data_interval_start,
-data_interval_end=data_interval_end,
-execution_date=execution_date,
-try_number=try_number,
-map_index=getattr(ti, "map_index", ""),
+logical_date = (
+dag_run.logical_date.isoformat() if AIRFLOW_V_3_0_PLUS else 
dag_run.execution_date.isoformat()
+)
+
+return (
+log_id_template.format(
+dag_id=ti.dag_id,
+task_id=ti.task_id,
+run_id=getattr(ti, "run_id", ""),
+data_interval_start=data_interval_start,
+data_interval_end=data_interval_end,
+logical_date=logical_date,
+try_number=try_number,
+map_index=getattr(ti, "map_index", ""),
+)
+if AIRFLOW_V_3_0_PLUS
+else log_id_template.format(
+dag_id=ti.dag_id,
+task_id=ti.task_id,
+run_id=getattr(ti, "run_id", ""),
+data_interval_start=data_interval_start,
+data_interval_end=data_interval_end,
+execution_date=logical_date,
+try_number=try_number,
+map_index=getattr(ti, "map_index", ""),
+)
 )

Review Comment:
   ```suggestion
   log_id_template.format(
   dag_id=ti.dag_id,
   task_id=ti.task_id,
   run_id=getattr(ti, "run_id", ""),
   data_interval_start=data_interval_start,
   data_interval_end=data_interval_end,
   execution_date=logical_date,
   logical_date=logical_date,
   try_number=try_number,
   map_index=getattr(ti, "map_index", ""),
   )
   ```
   
   `str.format` ignores extra arguments, so we can just add both keys with 
checking version.



-- 
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



Re: [PR] Rename execution_date to logical_date across codebase [airflow]

2024-11-12 Thread via GitHub


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


##
providers/src/airflow/providers/elasticsearch/log/es_task_handler.py:
##
@@ -256,7 +261,11 @@ def _render_log_id(self, ti: TaskInstance | 
TaskInstanceKey, try_number: int) ->
 if self.json_format:
 data_interval_start = self._clean_date(data_interval[0])
 data_interval_end = self._clean_date(data_interval[1])
-execution_date = self._clean_date(dag_run.execution_date)
+logical_date = (
+self._clean_date(dag_run.logical_date)
+if AIRFLOW_V_3_0_PLUS
+else self._clean_date(dag_run.execution_date)
+)

Review Comment:
   But this one actually doesn’t—`dag_run.logical_date` is available on Airflow 
2.



-- 
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



Re: [PR] Rename execution_date to logical_date across codebase [airflow]

2024-11-12 Thread via GitHub


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


##
providers/src/airflow/providers/dbt/cloud/utils/openlineage.py:
##
@@ -142,7 +142,7 @@ async def get_artifacts_for_steps(steps, artifacts):
 parent_run_id = OpenLineageAdapter.build_task_instance_run_id(
 dag_id=task_instance.dag_id,
 task_id=operator.task_id,
-execution_date=task_instance.execution_date,
+logical_date=task_instance.logical_date,

Review Comment:
   I think this needs a version check too?



-- 
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



Re: [PR] Rename execution_date to logical_date across codebase [airflow]

2024-11-12 Thread via GitHub


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


##
newsfragments/43902.significant.rst:
##
@@ -0,0 +1,5 @@
+Renamed ``execution_date`` to ``logical_date`` across the codebase to align 
with Airflow 3.0.
+
+The shift towards ``logical_date`` helps move away from the limitations of 
``execution_date``, particularly with dynamic DAG runs and cases where multiple 
runs occur at the same time. This change impacts database models, templates, 
and functions:

Review Comment:
   ```suggestion
   The shift towards ``logical_date`` helps move away from the limitations of 
``execution_date``, particularly with dynamic DAG runs and cases where multiple 
runs occur at the same time. This change impacts database models, templates, 
and functions:
   
   ```
   
   Needs an extra empty line to fix rendering.



-- 
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



Re: [PR] Rename execution_date to logical_date across codebase [airflow]

2024-11-12 Thread via GitHub


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


##
airflow/utils/context.pyi:
##
@@ -102,24 +102,19 @@ class Context(TypedDict, total=False):
 ds: str
 ds_nodash: str
 exception: BaseException | str | None
-execution_date: DateTime
 expanded_ti_count: int | None
 inlets: list
 inlet_events: InletEventsAccessors
 logical_date: DateTime
 macros: Any
 map_index_template: str
-next_ds: str | None
-next_ds_nodash: str | None
-next_execution_date: DateTime | None
+next_logical_date: DateTime | None

Review Comment:
   Same



-- 
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



Re: [PR] Rename execution_date to logical_date across codebase [airflow]

2024-11-12 Thread via GitHub


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


##
newsfragments/43902.significant.rst:
##
@@ -0,0 +1,5 @@
+Renamed ``execution_date`` to ``logical_date`` across the codebase to align 
with Airflow 3.0.
+
+The shift towards ``logical_date`` helps move away from the limitations of 
``execution_date``, particularly with dynamic DAG runs and cases where multiple 
runs occur at the same time. This change impacts database models, templates, 
and functions:
+- Renamed columns and function references to ``logical_date``.
+- Removed unique constraints on ``execution_date`` to support concurrent DAG 
runs with the same logical date.

Review Comment:
   I don’t think this PR includes this second one? This was done in a previous 
PR and should already has its own fragment.



-- 
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



Re: [PR] Rename execution_date to logical_date across codebase [airflow]

2024-11-12 Thread via GitHub


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


##
airflow/utils/context.py:
##


Review Comment:
   Let’s keep all the deprecation entry mechanism, only make 
`_DEPRECATION_REPLACEMENTS` empty. This way we can more easily add things in 
the future for deprecation.



-- 
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



Re: [PR] Rename execution_date to logical_date across codebase [airflow]

2024-11-12 Thread via GitHub


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


##
airflow/utils/context.py:
##
@@ -70,26 +66,21 @@
 "data_interval_start",
 "ds",
 "ds_nodash",
-"execution_date",
 "expanded_ti_count",
 "exception",
 "inlets",
 "inlet_events",
 "logical_date",
 "macros",
 "map_index_template",
-"next_ds",
-"next_ds_nodash",
-"next_execution_date",
+"next_logical_date",

Review Comment:
   Same



-- 
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



Re: [PR] Rename execution_date to logical_date across codebase [airflow]

2024-11-12 Thread via GitHub


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


##
airflow/models/taskinstance.py:
##
@@ -1124,24 +1085,19 @@ def get_triggering_events() -> dict[str, 
list[AssetEvent | AssetEventPydantic]]:
 "outlet_events": OutletEventAccessors(),
 "ds": ds,
 "ds_nodash": ds_nodash,
-"execution_date": logical_date,
+"logical_date": logical_date,
 "expanded_ti_count": expanded_ti_count,
 "inlets": task.inlets,
 "inlet_events": InletEventsAccessors(task.inlets, session=session),
-"logical_date": logical_date,
 "macros": macros,
 "map_index_template": task.map_index_template,
-"next_ds": get_next_ds(),
-"next_ds_nodash": get_next_ds_nodash(),
-"next_execution_date": get_next_execution_date(),
+"next_logical_date": get_next_logical_date(),

Review Comment:
   This one is also deprecated and should be removed.



-- 
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



Re: [PR] Rename execution_date to logical_date across codebase [airflow]

2024-11-12 Thread via GitHub


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


##
airflow/api_fastapi/core_api/datamodels/event_logs.py:
##
@@ -33,7 +33,7 @@ class EventLogResponse(BaseModel):
 map_index: int | None
 try_number: int | None
 event: str
-execution_date: datetime | None = Field(alias="logical_date")
+logical_date: datetime | None = Field(alias="logical_date")

Review Comment:
   ```suggestion
   logical_date: datetime | None
   ```



-- 
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



Re: [PR] Rename execution_date to logical_date across codebase [airflow]

2024-11-12 Thread via GitHub


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


##
airflow/api_connexion/openapi/v1.yaml:
##
@@ -4734,14 +4721,14 @@ components:
   description: The task ID.
   type: string
 
-execution_date:
+logical_date:
   description: The execution date. Either set this or dag_run_id but 
not both.

Review Comment:
   ```suggestion
 description: The logical date. Either set this or dag_run_id but 
not both.
   ```



-- 
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



Re: [PR] Rename execution_date to logical_date across codebase [airflow]

2024-11-12 Thread via GitHub


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


##
airflow/api_connexion/openapi/v1.yaml:
##


Review Comment:
   Just a note, we’ll need to add `logical_date` to the 2.x branch on all 
entries so users can migrate more easily.



-- 
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



Re: [PR] Rename execution_date to logical_date across codebase [airflow]

2024-11-12 Thread via GitHub


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


##
providers/src/airflow/providers/google/cloud/sensors/gcs.py:
##
@@ -179,16 +183,21 @@ def ts_function(context):
 Act as a default callback for the GoogleCloudStorageObjectUpdatedSensor.
 
 The default behaviour is check for the object being updated after the data 
interval's end,
-or execution_date + interval on Airflow versions prior to 2.2 (before 
AIP-39 implementation).
+or logical_date + interval on Airflow versions prior to 2.2 (before AIP-39 
implementation).
 """
 try:
 return context["data_interval_end"]
 except KeyError:
 from airflow.utils import timezone
 
-data_interval = context["dag"].infer_automated_data_interval(
-timezone.coerce_datetime(context["execution_date"])
-)
+if AIRFLOW_V_3_0_PLUS:
+data_interval = context["dag"].infer_automated_data_interval(
+timezone.coerce_datetime(context["logical_date"])
+)
+else:
+data_interval = context["dag"].infer_automated_data_interval(
+timezone.coerce_datetime(context["execution_date"])
+)

Review Comment:
   This is actually required by compat tests as it fails for 2 with following 
errors:
   ```
   providers/tests/google/cloud/sensors/test_gcs.py:281: in 
test_should_support_cron
   result = ts_function(context)
   
/usr/local/lib/python3.9/site-packages/airflow/providers/google/cloud/sensors/gcs.py:194:
 in ts_function
   timezone.coerce_datetime(context["logical_date"])
   E   KeyError: 'logical_date'
   ```



-- 
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



Re: [PR] Rename execution_date to logical_date across codebase [airflow]

2024-11-12 Thread via GitHub


sunank200 commented on PR #43902:
URL: https://github.com/apache/airflow/pull/43902#issuecomment-2471650423

   > The shift towards `logical_date` helps move away from the limitations of 
`execution_date`, particularly with dynamic DAG runs and cases where multiple 
runs occur at the same time.
   
   Added the newsfragment


-- 
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



Re: [PR] Rename execution_date to logical_date across codebase [airflow]

2024-11-12 Thread via GitHub


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


##
airflow/api_connexion/schemas/dag_run_schema.py:
##
@@ -109,9 +101,8 @@ def autogenerate(self, data, **kwargs):
 
 @post_dump
 def autofill(self, data, **kwargs):
-"""Populate execution_date from logical_date for compatibility."""

Review Comment:
   Changed the documentation as it autofills other fields as well



-- 
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



Re: [PR] Rename execution_date to logical_date across codebase [airflow]

2024-11-12 Thread via GitHub


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


##
airflow/models/log.py:
##
@@ -73,7 +73,7 @@ def __init__(
 if task_instance:
 self.dag_id = task_instance.dag_id
 self.task_id = task_instance.task_id
-if execution_date := getattr(task_instance, "execution_date", 
None):
+if execution_date := getattr(task_instance, "logical_date", None):
 self.execution_date = execution_date

Review Comment:
   I have changed it now. Now if `task_instance.logical_date` exists and is not 
None,it assigns this value to `self.logical_date`.



-- 
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



Re: [PR] Rename execution_date to logical_date across codebase [airflow]

2024-11-12 Thread via GitHub


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


##
airflow/api_connexion/schemas/dag_run_schema.py:
##
@@ -81,22 +81,14 @@ def autogenerate(self, data, **kwargs):
 """
 Auto generate run_id and logical_date if they are not provided.
 
-For compatibility, if `execution_date` is submitted, it is converted
+For compatibility, if `logical_date` is submitted, it is converted
 to `logical_date`.

Review Comment:
   Changed it



-- 
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



Re: [PR] Rename execution_date to logical_date across codebase [airflow]

2024-11-12 Thread via GitHub


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


##
providers/src/airflow/providers/cncf/kubernetes/cli/kubernetes_command.py:
##
@@ -36,15 +36,20 @@
 from airflow.utils.cli import get_dag
 from airflow.utils.providers_configuration_loader import 
providers_configuration_loaded
 
+from tests_common.test_utils.compat import AIRFLOW_V_3_0_PLUS

Review Comment:
   I have changed it to: 
   ```
   AIRFLOW_VERSION = Version(airflow_version)
   AIRFLOW_V_3_0_PLUS = Version(AIRFLOW_VERSION.base_version) >= 
Version("3.0.0")
   ```



##
providers/src/airflow/providers/elasticsearch/log/es_task_handler.py:
##
@@ -44,6 +44,8 @@
 from airflow.utils.module_loading import import_string
 from airflow.utils.session import create_session
 
+from tests_common.test_utils.compat import AIRFLOW_V_3_0_PLUS

Review Comment:
   I have changed it to: 
   ```
   AIRFLOW_VERSION = Version(airflow_version)
   AIRFLOW_V_3_0_PLUS = Version(AIRFLOW_VERSION.base_version) >= 
Version("3.0.0")
   ```



-- 
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



Re: [PR] Rename execution_date to logical_date across codebase [airflow]

2024-11-12 Thread via GitHub


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


##
airflow/utils/cli.py:
##
@@ -75,7 +75,7 @@ def action_logging(f: T) -> T:
 log : airflow.models.log.Log ORM instance
 dag_id : dag id (optional)
 task_id : task_id (optional)
-execution_date : execution date (optional)
+execution_date : logical date (optional)

Review Comment:
   changed it



-- 
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



Re: [PR] Rename execution_date to logical_date across codebase [airflow]

2024-11-12 Thread via GitHub


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


##
providers/src/airflow/providers/google/cloud/log/stackdriver_task_handler.py:
##
@@ -36,6 +36,8 @@
 from airflow.utils.log.trigger_handler import ctx_indiv_trigger
 from airflow.utils.types import NOTSET, ArgNotSet
 
+from tests_common.test_utils.compat import AIRFLOW_V_3_0_PLUS

Review Comment:
   I have changed it to: 
   ```
   AIRFLOW_VERSION = Version(airflow_version)
   AIRFLOW_V_3_0_PLUS = Version(AIRFLOW_VERSION.base_version) >= 
Version("3.0.0")
   ```



##
providers/src/airflow/providers/google/cloud/sensors/gcs.py:
##
@@ -42,6 +44,8 @@
 from google.api_core.retry import Retry
 
 from airflow.utils.context import Context
+AIRFLOW_VERSION = Version(airflow_version)
+AIRFLOW_V_3_0_PLUS = Version(AIRFLOW_VERSION.base_version) >= Version("3.0.0")

Review Comment:
   Changed it



-- 
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



Re: [PR] Rename execution_date to logical_date across codebase [airflow]

2024-11-12 Thread via GitHub


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


##
providers/src/airflow/providers/google/cloud/sensors/gcs.py:
##
@@ -179,16 +183,21 @@ def ts_function(context):
 Act as a default callback for the GoogleCloudStorageObjectUpdatedSensor.
 
 The default behaviour is check for the object being updated after the data 
interval's end,
-or execution_date + interval on Airflow versions prior to 2.2 (before 
AIP-39 implementation).
+or logical_date + interval on Airflow versions prior to 2.2 (before AIP-39 
implementation).
 """
 try:
 return context["data_interval_end"]
 except KeyError:
 from airflow.utils import timezone
 
-data_interval = context["dag"].infer_automated_data_interval(
-timezone.coerce_datetime(context["execution_date"])
-)
+if AIRFLOW_V_3_0_PLUS:
+data_interval = context["dag"].infer_automated_data_interval(
+timezone.coerce_datetime(context["logical_date"])
+)
+else:
+data_interval = context["dag"].infer_automated_data_interval(
+timezone.coerce_datetime(context["execution_date"])
+)

Review Comment:
   Changed it



-- 
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



Re: [PR] Rename execution_date to logical_date across codebase [airflow]

2024-11-12 Thread via GitHub


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


##
docs/apache-airflow/templates-ref.rst:
##
@@ -117,24 +117,14 @@ existing code to use other variables instead.
 =   
==
 Deprecated Variable Description
 =   
==
-``{{ execution_date }}``the execution date (logical date), 
same as ``logical_date``
-``{{ next_execution_date }}``   the logical date of the next scheduled 
run,
-you may be able to use 
``data_interval_end`` instead; for manually
-triggered dagruns that aren't on a 
schedule, ``next_execution_date`` is
-set to ``logical_date``
 ``{{ next_ds }}``   the next execution date as 
``-MM-DD`` if exists, else ``None``
 ``{{ next_ds_nodash }}``the next execution date as 
``MMDD`` if exists, else ``None``
-``{{ prev_execution_date }}``   the logical date of the previous 
scheduled run (if applicable)
 ``{{ prev_ds }}``   the previous execution date as 
``-MM-DD`` if exists, else ``None``
 ``{{ prev_ds_nodash }}``the previous execution date as 
``MMDD`` if exists, else ``None``
 ``{{ yesterday_ds }}``  the day before the execution date as 
``-MM-DD``
 ``{{ yesterday_ds_nodash }}``   the day before the execution date as 
``MMDD``
 ``{{ tomorrow_ds }}``   the day after the execution date as 
``-MM-DD``
 ``{{ tomorrow_ds_nodash }}``the day after the execution date as 
``MMDD``
-``{{ prev_execution_date_success }}``   execution date from prior successful 
DAG run;
-you may be able to use 
``prev_data_interval_start_success`` instead if
-the timetable/schedule you use for the 
DAG defines ``data_interval_start``
-compatible with the legacy 
``execution_date``.
 =   
==

Review Comment:
   Removed them



-- 
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



Re: [PR] Rename execution_date to logical_date across codebase [airflow]

2024-11-12 Thread via GitHub


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


##
docs/apache-airflow/templates-ref.rst:
##
@@ -117,24 +117,14 @@ existing code to use other variables instead.
 =   
==
 Deprecated Variable Description
 =   
==
-``{{ execution_date }}``the execution date (logical date), 
same as ``logical_date``
-``{{ next_execution_date }}``   the logical date of the next scheduled 
run,
-you may be able to use 
``data_interval_end`` instead; for manually
-triggered dagruns that aren't on a 
schedule, ``next_execution_date`` is
-set to ``logical_date``
 ``{{ next_ds }}``   the next execution date as 
``-MM-DD`` if exists, else ``None``
 ``{{ next_ds_nodash }}``the next execution date as 
``MMDD`` if exists, else ``None``
-``{{ prev_execution_date }}``   the logical date of the previous 
scheduled run (if applicable)
 ``{{ prev_ds }}``   the previous execution date as 
``-MM-DD`` if exists, else ``None``
 ``{{ prev_ds_nodash }}``the previous execution date as 
``MMDD`` if exists, else ``None``
 ``{{ yesterday_ds }}``  the day before the execution date as 
``-MM-DD``
 ``{{ yesterday_ds_nodash }}``   the day before the execution date as 
``MMDD``
 ``{{ tomorrow_ds }}``   the day after the execution date as 
``-MM-DD``
 ``{{ tomorrow_ds_nodash }}``the day after the execution date as 
``MMDD``
-``{{ prev_execution_date_success }}``   execution date from prior successful 
DAG run;
-you may be able to use 
``prev_data_interval_start_success`` instead if
-the timetable/schedule you use for the 
DAG defines ``data_interval_start``
-compatible with the legacy 
``execution_date``.
 =   
==

Review Comment:
   removed them



-- 
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



Re: [PR] Rename execution_date to logical_date across codebase [airflow]

2024-11-12 Thread via GitHub


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


##
airflow/utils/context.pyi:
##
@@ -102,24 +102,23 @@ class Context(TypedDict, total=False):
 ds: str
 ds_nodash: str
 exception: BaseException | str | None
-execution_date: DateTime
+logical_date: DateTime

Review Comment:
   changed it.



-- 
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



Re: [PR] Rename execution_date to logical_date across codebase [airflow]

2024-11-12 Thread via GitHub


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


##
airflow/models/taskinstance.py:
##
@@ -1119,24 +1119,23 @@ def get_triggering_events() -> dict[str, 
list[AssetEvent | AssetEventPydantic]]:
 "outlet_events": OutletEventAccessors(),
 "ds": ds,
 "ds_nodash": ds_nodash,
-"execution_date": logical_date,
+"logical_date": logical_date,
 "expanded_ti_count": expanded_ti_count,
 "inlets": task.inlets,
 "inlet_events": InletEventsAccessors(task.inlets, session=session),
-"logical_date": logical_date,
 "macros": macros,
 "map_index_template": task.map_index_template,
 "next_ds": get_next_ds(),

Review Comment:
   removed them



-- 
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



Re: [PR] Rename execution_date to logical_date across codebase [airflow]

2024-11-12 Thread via GitHub


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


##
airflow/utils/context.py:
##
@@ -70,26 +66,25 @@
 "data_interval_start",
 "ds",
 "ds_nodash",
-"execution_date",
+"logical_date",

Review Comment:
   Changed it.



-- 
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



Re: [PR] Rename execution_date to logical_date across codebase [airflow]

2024-11-12 Thread via GitHub


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


##
airflow/api_connexion/openapi/v1.yaml:
##
@@ -3571,31 +3558,6 @@ components:
   items:
 $ref: "#/components/schemas/Provider"
 
-SLAMiss:
-  type: object
-  properties:
-task_id:
-  type: string
-  readOnly: true
-  description: The task ID.
-dag_id:
-  type: string
-  description: The DAG ID.
-execution_date:
-  type: string
-  format: datetime
-email_sent:
-  type: boolean
-timestamp:
-  type: string
-  format: datetime
-description:
-  type: string
-  nullable: true
-notification_sent:
-  type: boolean
-  nullable: true

Review Comment:
   Done



-- 
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



Re: [PR] Rename execution_date to logical_date across codebase [airflow]

2024-11-12 Thread via GitHub


potiuk commented on PR #43902:
URL: https://github.com/apache/airflow/pull/43902#issuecomment-2470118445

   > Providers don’t generally use `execution_date`; in the rare cases they do, 
a compatibility layer is provided so they work on both Airflow 2 and 3. No 
official providers have interface that expose `execution_date` to the user.
   
   Yeah. I think we need to bite the bullet and change it - even if we know 
that **some** things outside of our providers will be broken. Our compat tests 
are passing, which means that the change is "good to go" from our provider's 
point of view, but we should add this change to be listed in 
https://github.com/apache/airflow/issues/41641 so that we remember there is a 2 
to 3 migration rule that will check if exection_date is still not used 
somewhwere in the user code when they are attempting to migrate. 


-- 
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



Re: [PR] Rename execution_date to logical_date across codebase [airflow]

2024-11-12 Thread via GitHub


uranusjr commented on PR #43902:
URL: https://github.com/apache/airflow/pull/43902#issuecomment-2469928527

   Providers don’t generally use `execution_date`; in the rare cases they do, a 
compatibility layer is provided so they work on both Airflow 2 and 3. No 
official providers have interface that expose `execution_date` to the user.


-- 
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



Re: [PR] Rename execution_date to logical_date across codebase [airflow]

2024-11-12 Thread via GitHub


eladkal commented on PR #43902:
URL: https://github.com/apache/airflow/pull/43902#issuecomment-2469907149

   Is this change backward compatible for all providers?


-- 
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



Re: [PR] Rename execution_date to logical_date across codebase [airflow]

2024-11-12 Thread via GitHub


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


##
airflow/utils/cli.py:
##
@@ -75,7 +75,7 @@ def action_logging(f: T) -> T:
 log : airflow.models.log.Log ORM instance
 dag_id : dag id (optional)
 task_id : task_id (optional)
-execution_date : execution date (optional)
+execution_date : logical date (optional)

Review Comment:
   Yes we do (it’s in `_build_metrics`)



-- 
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



Re: [PR] Rename execution_date to logical_date across codebase [airflow]

2024-11-12 Thread via GitHub


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


##
airflow/models/log.py:
##
@@ -73,7 +73,7 @@ def __init__(
 if task_instance:
 self.dag_id = task_instance.dag_id
 self.task_id = task_instance.task_id
-if execution_date := getattr(task_instance, "execution_date", 
None):
+if execution_date := getattr(task_instance, "logical_date", None):
 self.execution_date = execution_date

Review Comment:
   It’s probably better to just remove this. This field was only kept for 
compatibility and is no longer needed anywhere.



-- 
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



Re: [PR] Rename execution_date to logical_date across codebase [airflow]

2024-11-11 Thread via GitHub


Lee-W commented on code in PR #43902:
URL: https://github.com/apache/airflow/pull/43902#discussion_r1837618899


##
airflow/models/log.py:
##
@@ -73,7 +73,7 @@ def __init__(
 if task_instance:
 self.dag_id = task_instance.dag_id
 self.task_id = task_instance.task_id
-if execution_date := getattr(task_instance, "execution_date", 
None):
+if execution_date := getattr(task_instance, "logical_date", None):
 self.execution_date = execution_date

Review Comment:
   ```suggestion
   if logical_date := getattr(task_instance, "logical_date", None):
   self.execution_date = logical_date
   ```
   
   I guess the column itself will be renamed in another PR?



##
airflow/utils/cli.py:
##
@@ -75,7 +75,7 @@ def action_logging(f: T) -> T:
 log : airflow.models.log.Log ORM instance
 dag_id : dag id (optional)
 task_id : task_id (optional)
-execution_date : execution date (optional)
+execution_date : logical date (optional)

Review Comment:
   do we need to rename it as well



-- 
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



Re: [PR] Rename execution_date to logical_date across codebase [airflow]

2024-11-11 Thread via GitHub


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


##
airflow/api/common/mark_tasks.py:
##
@@ -107,11 +107,11 @@ def set_state(
 :param tasks: the iterable of tasks or (task, map_index) tuples from which 
to work.
 ``task.dag`` needs to be set
 :param run_id: the run_id of the dagrun to start looking from
-:param execution_date: the execution date from which to start looking 
(deprecated)
+:param logical_date: the logical date from which to start looking 
(deprecated)

Review Comment:
   There’s a later PR to remove this. I believe this is already done in the 
unified PR, just not included here to simplify review.



-- 
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



Re: [PR] Rename execution_date to logical_date across codebase [airflow]

2024-11-11 Thread via GitHub


Lee-W commented on code in PR #43902:
URL: https://github.com/apache/airflow/pull/43902#discussion_r1837529116


##
airflow/api/common/mark_tasks.py:
##
@@ -107,11 +107,11 @@ def set_state(
 :param tasks: the iterable of tasks or (task, map_index) tuples from which 
to work.
 ``task.dag`` needs to be set
 :param run_id: the run_id of the dagrun to start looking from
-:param execution_date: the execution date from which to start looking 
(deprecated)
+:param logical_date: the logical date from which to start looking 
(deprecated)

Review Comment:
   Should we deprecate it in Airflow 3? Maybe something to do in the future prs



##
airflow/api/common/mark_tasks.py:
##
@@ -279,37 +279,37 @@ def _set_dag_run_state(dag_id: str, run_id: str, state: 
DagRunState, session: SA
 def set_dag_run_state_to_success(
 *,
 dag: DAG,
-execution_date: datetime | None = None,
+logical_date: datetime | None = None,
 run_id: str | None = None,
 commit: bool = False,
 session: SASession = NEW_SESSION,
 ) -> list[TaskInstance]:
 """
 Set the dag run's state to success.
 
-Set for a specific execution date and its task instances to success.
+Set for a specific logical date and its task instances to success.
 
 :param dag: the DAG of which to alter state
-:param execution_date: the execution date from which to start 
looking(deprecated)
+:param logical_date: the logical date from which to start 
looking(deprecated)

Review Comment:
   same here



-- 
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



Re: [PR] Rename execution_date to logical_date across codebase [airflow]

2024-11-11 Thread via GitHub


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


##
providers/src/airflow/providers/cncf/kubernetes/cli/kubernetes_command.py:
##
@@ -36,15 +36,20 @@
 from airflow.utils.cli import get_dag
 from airflow.utils.providers_configuration_loader import 
providers_configuration_loaded
 
+from tests_common.test_utils.compat import AIRFLOW_V_3_0_PLUS

Review Comment:
   It’s probably easiest to use `getattr` with `get_logical_date_key` in 
`backcompat`.



-- 
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



Re: [PR] Rename execution_date to logical_date across codebase [airflow]

2024-11-11 Thread via GitHub


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


##
providers/src/airflow/providers/google/cloud/sensors/gcs.py:
##
@@ -179,16 +183,21 @@ def ts_function(context):
 Act as a default callback for the GoogleCloudStorageObjectUpdatedSensor.
 
 The default behaviour is check for the object being updated after the data 
interval's end,
-or execution_date + interval on Airflow versions prior to 2.2 (before 
AIP-39 implementation).
+or logical_date + interval on Airflow versions prior to 2.2 (before AIP-39 
implementation).
 """
 try:
 return context["data_interval_end"]
 except KeyError:
 from airflow.utils import timezone
 
-data_interval = context["dag"].infer_automated_data_interval(
-timezone.coerce_datetime(context["execution_date"])
-)
+if AIRFLOW_V_3_0_PLUS:
+data_interval = context["dag"].infer_automated_data_interval(
+timezone.coerce_datetime(context["logical_date"])
+)
+else:
+data_interval = context["dag"].infer_automated_data_interval(
+timezone.coerce_datetime(context["execution_date"])
+)

Review Comment:
   But we don’t need it here; `context["logical_date"]` works in both 2 and 3.



##
providers/src/airflow/providers/google/cloud/sensors/gcs.py:
##
@@ -42,6 +44,8 @@
 from google.api_core.retry import Retry
 
 from airflow.utils.context import Context
+AIRFLOW_VERSION = Version(airflow_version)
+AIRFLOW_V_3_0_PLUS = Version(AIRFLOW_VERSION.base_version) >= Version("3.0.0")

Review Comment:
   This is the correct way to detect version.



-- 
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



Re: [PR] Rename execution_date to logical_date across codebase [airflow]

2024-11-11 Thread via GitHub


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


##
providers/src/airflow/providers/google/cloud/log/stackdriver_task_handler.py:
##
@@ -36,6 +36,8 @@
 from airflow.utils.log.trigger_handler import ctx_indiv_trigger
 from airflow.utils.types import NOTSET, ArgNotSet
 
+from tests_common.test_utils.compat import AIRFLOW_V_3_0_PLUS

Review Comment:
   Another



-- 
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



  1   2   >