Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

on_state_change_callback() implementation #319

Open
wants to merge 7 commits into
base: lyft-stable-2.3.4
Choose a base branch
from
Open
Show file tree
Hide file tree
Changes from 2 commits
Commits
File filter

Filter by extension

Filter by extension


Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
3 changes: 3 additions & 0 deletions airflow/jobs/scheduler_job.py
Original file line number Diff line number Diff line change
Expand Up @@ -518,6 +518,9 @@ def _executable_task_instances_to_queued(self, max_tis: int, session: Session =
)

for ti in executable_tis:
# Handles the following states:
# - QUEUED
ti.call_state_change_callback()
make_transient(ti)
return executable_tis

Expand Down
7 changes: 7 additions & 0 deletions airflow/models/baseoperator.py
Original file line number Diff line number Diff line change
Expand Up @@ -216,6 +216,7 @@ def partial(
on_failure_callback: Optional[TaskStateChangeCallback] = None,
on_success_callback: Optional[TaskStateChangeCallback] = None,
on_retry_callback: Optional[TaskStateChangeCallback] = None,
on_state_change_callback: Optional[TaskStateChangeCallback] = None,
run_as_user: Optional[str] = None,
executor_config: Optional[Dict] = None,
inlets: Optional[Any] = None,
Expand Down Expand Up @@ -276,6 +277,7 @@ def partial(
partial_kwargs.setdefault("on_failure_callback", on_failure_callback)
partial_kwargs.setdefault("on_retry_callback", on_retry_callback)
partial_kwargs.setdefault("on_success_callback", on_success_callback)
partial_kwargs.setdefault("on_state_change_callback", on_state_change_callback)
partial_kwargs.setdefault("run_as_user", run_as_user)
partial_kwargs.setdefault("executor_config", executor_config)
partial_kwargs.setdefault("inlets", inlets)
Expand Down Expand Up @@ -564,6 +566,8 @@ class derived from this one results in the creation of a task object,
that it is executed when retries occur.
:param on_success_callback: much like the ``on_failure_callback`` except
that it is executed when the task succeeds.
:param on_state_change_callback: much like the ``on_failure_callback`` except
that it is executed when the task state is changed.
:param pre_execute: a function to be called immediately before task
execution, receiving a context dictionary; raising an exception will
prevent the task from being executed.
Expand Down Expand Up @@ -667,6 +671,7 @@ class derived from this one results in the creation of a task object,
'on_failure_callback',
'on_success_callback',
'on_retry_callback',
'on_state_change_callback',
'do_xcom_push',
}

Expand Down Expand Up @@ -730,6 +735,7 @@ def __init__(
on_failure_callback: Optional[TaskStateChangeCallback] = None,
on_success_callback: Optional[TaskStateChangeCallback] = None,
on_retry_callback: Optional[TaskStateChangeCallback] = None,
on_state_change_callback: Optional[TaskStateChangeCallback] = None,
pre_execute: Optional[TaskPreExecuteHook] = None,
post_execute: Optional[TaskPostExecuteHook] = None,
trigger_rule: str = DEFAULT_TRIGGER_RULE,
Expand Down Expand Up @@ -793,6 +799,7 @@ def __init__(
self.on_failure_callback = on_failure_callback
self.on_success_callback = on_success_callback
self.on_retry_callback = on_retry_callback
self.on_state_change_callback = on_state_change_callback
self._pre_execute_hook = pre_execute
self._post_execute_hook = post_execute

Expand Down
10 changes: 10 additions & 0 deletions airflow/models/dagrun.py
Original file line number Diff line number Diff line change
Expand Up @@ -954,6 +954,11 @@ def _check_for_removed_or_restored_tasks(
Stats.incr(f"task_restored_to_dag.{dag.dag_id}", 1, 1)
existing_indexes[task].append(ti.map_index)
expected_indexes[task] = range(total_length)

# Handles the following states:
# - REMOVED
# - None
ti.call_state_change_callback()
# Check if we have some missing indexes to create ti for
missing_indexes: Dict["MappedOperator", Sequence[int]] = defaultdict(list)
for k, v in existing_indexes.items():
Expand Down Expand Up @@ -1208,6 +1213,11 @@ def schedule_tis(self, schedulable_tis: Iterable[TI], session: Session = NEW_SES
)
.update({TI.state: State.SCHEDULED}, synchronize_session=False)
)

# Handles the following state
# - SCHEDULED
for ti in schedulable_tis:
ti.call_state_change_callback()

# Tasks using EmptyOperator should not be executed, mark them as success
if dummy_ti_ids:
Expand Down
42 changes: 41 additions & 1 deletion airflow/models/taskinstance.py
Original file line number Diff line number Diff line change
Expand Up @@ -219,6 +219,9 @@ def clear_task_instances(
# the task is terminated and becomes eligible for retry.
ti.state = TaskInstanceState.RESTARTING
job_ids.append(ti.job_id)
# Handles the following state
# - RESTARTING
ti.call_state_change_callback()
else:
task_id = ti.task_id
if dag and dag.has_task(task_id):
Expand Down Expand Up @@ -272,6 +275,7 @@ def clear_task_instances(

delete_qry = TR.__table__.delete().where(conditions)
session.execute(delete_qry)


if job_ids:
from airflow.jobs.base_job import BaseJob
Expand Down Expand Up @@ -987,6 +991,11 @@ def set_state(self, state: Optional[str], session=NEW_SESSION):
self.end_date = self.end_date or current_time
self.duration = (self.end_date - self.start_date).total_seconds()
session.merge(self)
# Handles the following states:
# - UPSTREAM_FAILED
# - SKIPPED
# - FAILED
self.call_state_change_callback()

@property
def is_premature(self):
Expand Down Expand Up @@ -1350,6 +1359,9 @@ def check_and_change_state_before_execution(
task_reschedule: TR = TR.query_for_task_instance(self, session=session).first()
if task_reschedule:
self.start_date = task_reschedule.start_date
# Handles the following states
# - UP_FOR_RESCHEDULE
self.call_state_change_callback()

# Secondly we find non-runnable but requeueable tis. We reset its state.
# This is because we might have hit concurrency limits,
Expand Down Expand Up @@ -1391,6 +1403,9 @@ def check_and_change_state_before_execution(
if not test_mode:
session.merge(self).task = task
session.commit()
# Handles state
# - RUNNING
self.call_state_change_callback()

# Closing all pooled connections to prevent
# "max number of connections reached"
Expand Down Expand Up @@ -1635,6 +1650,9 @@ def _update_ti_state_for_sensing(self, session=NEW_SESSION):
self.start_date = timezone.utcnow()
session.merge(self)
session.commit()
# Handles the following states:
# - SENSING
self.call_state_change_callback()
# Raise exception for sensing state
raise AirflowSmartSensorException("Task successfully registered in smart sensor.")

Expand Down Expand Up @@ -1730,6 +1748,9 @@ def _defer_task(self, session, defer: TaskDeferred):
self.trigger_timeout = min(self.start_date + execution_timeout, self.trigger_timeout)
else:
self.trigger_timeout = self.start_date + execution_timeout
# Handles the following states:
# - DEFERRED
self.call_state_change_callback()

def _run_execute_callback(self, context: Context, task):
"""Functions that need to be run before a Task is executed"""
Expand Down Expand Up @@ -1772,6 +1793,11 @@ def _run_finished_callback(self, error: Optional[Union[str, Exception]] = None)
task.on_retry_callback(context)
except Exception:
self.log.exception("Error when executing on_retry_callback")
# Handles the following states:
# - SUCCESS
# - UP_FOR_RETRY
# - FAILED
self.call_state_change_callback()

@provide_session
def run(
Expand Down Expand Up @@ -2604,7 +2630,21 @@ def ti_selector_condition(cls, vals: Collection[Union[str, Tuple[str, int]]]) ->
if len(filters) == 1:
return filters[0]
return or_(*filters)



@classmethod
def call_state_change_callback(self):
self.log.info("State changed for DAG: %s, Task: %s, to state: %s", self.dag_id, self.task_id, self.state)
task = self.task
if task.on_state_change_callback is not None:
# Ensure the state and timestamps are up-to-date
self.refresh_from_db()
context = self.get_template_context()
try:
task.on_state_change_callback(context)
except Exception:
self.log.exception("Error when executing on_state_change_callback")


# State of the task instance.
# Stores string version of the task state.
Expand Down
2 changes: 1 addition & 1 deletion setup.py
Original file line number Diff line number Diff line change
Expand Up @@ -45,7 +45,7 @@

logger = logging.getLogger(__name__)

version = '2.3.4.post35'
version = '2.3.4.post36'
eschachar marked this conversation as resolved.
Show resolved Hide resolved

AIRFLOW_SOURCES_ROOT = Path(__file__).parent.resolve()
my_dir = dirname(__file__)
Expand Down