forked from apache/airflow
-
Notifications
You must be signed in to change notification settings - Fork 0
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
[AIRFLOW-198] Implement latest_only_operator
Dear Airflow Maintainers, Please accept this PR that addresses the following issues: - https://issues.apache.org/jira/browse/AIRFLOW-198 Testing Done: - Local testing of dag operation with LatestOnlyOperator - Unit test added Closes apache#1752 from gwax/latest_only
- Loading branch information
Showing
10 changed files
with
305 additions
and
2 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,34 @@ | ||
# -*- coding: utf-8 -*- | ||
# | ||
# Licensed under the Apache License, Version 2.0 (the "License"); | ||
# you may not use this file except in compliance with the License. | ||
# You may obtain a copy of the License at | ||
# | ||
# http://www.apache.org/licenses/LICENSE-2.0 | ||
# | ||
# Unless required by applicable law or agreed to in writing, software | ||
# distributed under the License is distributed on an "AS IS" BASIS, | ||
# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||
# See the License for the specific language governing permissions and | ||
# limitations under the License. | ||
""" | ||
Example of the LatestOnlyOperator | ||
""" | ||
import datetime as dt | ||
|
||
from airflow.models import DAG | ||
from airflow.operators.dummy_operator import DummyOperator | ||
from airflow.operators.latest_only_operator import LatestOnlyOperator | ||
from airflow.utils.trigger_rule import TriggerRule | ||
|
||
|
||
dag = DAG( | ||
dag_id='latest_only', | ||
schedule_interval=dt.timedelta(hours=4), | ||
start_date=dt.datetime(2016, 9, 20), | ||
) | ||
|
||
latest_only = LatestOnlyOperator(task_id='latest_only', dag=dag) | ||
|
||
task1 = DummyOperator(task_id='task1', dag=dag) | ||
task1.set_upstream(latest_only) |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,43 @@ | ||
# -*- coding: utf-8 -*- | ||
# | ||
# Licensed under the Apache License, Version 2.0 (the "License"); | ||
# you may not use this file except in compliance with the License. | ||
# You may obtain a copy of the License at | ||
# | ||
# http://www.apache.org/licenses/LICENSE-2.0 | ||
# | ||
# Unless required by applicable law or agreed to in writing, software | ||
# distributed under the License is distributed on an "AS IS" BASIS, | ||
# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||
# See the License for the specific language governing permissions and | ||
# limitations under the License. | ||
""" | ||
Example LatestOnlyOperator and TriggerRule interactions | ||
""" | ||
import datetime as dt | ||
|
||
from airflow.models import DAG | ||
from airflow.operators.dummy_operator import DummyOperator | ||
from airflow.operators.latest_only_operator import LatestOnlyOperator | ||
from airflow.utils.trigger_rule import TriggerRule | ||
|
||
|
||
dag = DAG( | ||
dag_id='latest_only_with_trigger', | ||
schedule_interval=dt.timedelta(hours=4), | ||
start_date=dt.datetime(2016, 9, 20), | ||
) | ||
|
||
latest_only = LatestOnlyOperator(task_id='latest_only', dag=dag) | ||
|
||
task1 = DummyOperator(task_id='task1', dag=dag) | ||
task1.set_upstream(latest_only) | ||
|
||
task2 = DummyOperator(task_id='task2', dag=dag) | ||
|
||
task3 = DummyOperator(task_id='task3', dag=dag) | ||
task3.set_upstream([task1, task2]) | ||
|
||
task4 = DummyOperator(task_id='task4', dag=dag, | ||
trigger_rule=TriggerRule.ALL_DONE) | ||
task4.set_upstream([task1, task2]) |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,57 @@ | ||
# -*- coding: utf-8 -*- | ||
# | ||
# Licensed under the Apache License, Version 2.0 (the "License"); | ||
# you may not use this file except in compliance with the License. | ||
# You may obtain a copy of the License at | ||
# | ||
# http://www.apache.org/licenses/LICENSE-2.0 | ||
# | ||
# Unless required by applicable law or agreed to in writing, software | ||
# distributed under the License is distributed on an "AS IS" BASIS, | ||
# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||
# See the License for the specific language governing permissions and | ||
# limitations under the License. | ||
|
||
import datetime | ||
import logging | ||
|
||
from airflow.models import BaseOperator, TaskInstance | ||
from airflow.utils.state import State | ||
from airflow import settings | ||
|
||
|
||
class LatestOnlyOperator(BaseOperator): | ||
""" | ||
Allows a workflow to skip tasks that are not running during the most | ||
recent schedule interval. | ||
If the task is run outside of the latest schedule interval, all | ||
directly downstream tasks will be skipped. | ||
""" | ||
|
||
ui_color = '#e9ffdb' # nyanza | ||
|
||
def execute(self, context): | ||
now = datetime.datetime.now() | ||
left_window = context['dag'].following_schedule( | ||
context['execution_date']) | ||
right_window = context['dag'].following_schedule(left_window) | ||
logging.info( | ||
'Checking latest only with left_window: %s right_window: %s ' | ||
'now: %s', left_window, right_window, now) | ||
if not left_window < now <= right_window: | ||
logging.info('Not latest execution, skipping downstream.') | ||
session = settings.Session() | ||
for task in context['task'].downstream_list: | ||
ti = TaskInstance( | ||
task, execution_date=context['ti'].execution_date) | ||
logging.info('Skipping task: %s', ti.task_id) | ||
ti.state = State.SKIPPED | ||
ti.start_date = now | ||
ti.end_date = now | ||
session.merge(ti) | ||
session.commit() | ||
session.close() | ||
logging.info('Done.') | ||
else: | ||
logging.info('Latest, allowing execution to proceed.') |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Loading
Sorry, something went wrong. Reload?
Sorry, we cannot display this file.
Sorry, this file is invalid so it cannot be displayed.
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,93 @@ | ||
# -*- coding: utf-8 -*- | ||
# | ||
# Licensed under the Apache License, Version 2.0 (the "License"); | ||
# you may not use this file except in compliance with the License. | ||
# You may obtain a copy of the License at | ||
# | ||
# http://www.apache.org/licenses/LICENSE-2.0 | ||
# | ||
# Unless required by applicable law or agreed to in writing, software | ||
# distributed under the License is distributed on an "AS IS" BASIS, | ||
# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||
# See the License for the specific language governing permissions and | ||
# limitations under the License. | ||
|
||
from __future__ import print_function, unicode_literals | ||
|
||
import datetime | ||
import logging | ||
import unittest | ||
|
||
from airflow import configuration, DAG, settings | ||
from airflow.jobs import BackfillJob | ||
from airflow.models import TaskInstance | ||
from airflow.operators.latest_only_operator import LatestOnlyOperator | ||
from airflow.operators.dummy_operator import DummyOperator | ||
from freezegun import freeze_time | ||
|
||
DEFAULT_DATE = datetime.datetime(2016, 1, 1) | ||
END_DATE = datetime.datetime(2016, 1, 2) | ||
INTERVAL = datetime.timedelta(hours=12) | ||
FROZEN_NOW = datetime.datetime(2016, 1, 2, 12, 1, 1) | ||
|
||
|
||
def get_task_instances(task_id): | ||
session = settings.Session() | ||
return session \ | ||
.query(TaskInstance) \ | ||
.filter(TaskInstance.task_id == task_id) \ | ||
.order_by(TaskInstance.execution_date) \ | ||
.all() | ||
|
||
|
||
class LatestOnlyOperatorTest(unittest.TestCase): | ||
|
||
def setUp(self): | ||
super(LatestOnlyOperatorTest, self).setUp() | ||
configuration.load_test_config() | ||
self.dag = DAG( | ||
'test_dag', | ||
default_args={ | ||
'owner': 'airflow', | ||
'start_date': DEFAULT_DATE}, | ||
schedule_interval=INTERVAL) | ||
self.addCleanup(self.dag.clear) | ||
freezer = freeze_time(FROZEN_NOW) | ||
freezer.start() | ||
self.addCleanup(freezer.stop) | ||
|
||
def test_run(self): | ||
task = LatestOnlyOperator( | ||
task_id='latest', | ||
dag=self.dag) | ||
task.run(start_date=DEFAULT_DATE, end_date=DEFAULT_DATE) | ||
|
||
def test_skipping(self): | ||
latest_task = LatestOnlyOperator( | ||
task_id='latest', | ||
dag=self.dag) | ||
downstream_task = DummyOperator( | ||
task_id='downstream', | ||
dag=self.dag) | ||
downstream_task.set_upstream(latest_task) | ||
|
||
latest_task.run(start_date=DEFAULT_DATE, end_date=END_DATE) | ||
downstream_task.run(start_date=DEFAULT_DATE, end_date=END_DATE) | ||
|
||
latest_instances = get_task_instances('latest') | ||
exec_date_to_latest_state = { | ||
ti.execution_date: ti.state for ti in latest_instances} | ||
assert exec_date_to_latest_state == { | ||
datetime.datetime(2016, 1, 1): 'success', | ||
datetime.datetime(2016, 1, 1, 12): 'success', | ||
datetime.datetime(2016, 1, 2): 'success', | ||
} | ||
|
||
downstream_instances = get_task_instances('downstream') | ||
exec_date_to_downstream_state = { | ||
ti.execution_date: ti.state for ti in downstream_instances} | ||
assert exec_date_to_downstream_state == { | ||
datetime.datetime(2016, 1, 1): 'skipped', | ||
datetime.datetime(2016, 1, 1, 12): 'skipped', | ||
datetime.datetime(2016, 1, 2): 'success', | ||
} |