[jira] [Commented] (AIRFLOW-648) Using the LatestOnlyOperator results in tasks being reenqueued many times over

2017-07-28 Thread Russell Pierce (JIRA)

[ 
https://issues.apache.org/jira/browse/AIRFLOW-648?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16105997#comment-16105997
 ] 

Russell Pierce commented on AIRFLOW-648:


I wonder if it is really the LatestOnly operator that is to blame here.  I've 
noticed similar behavior (although it was in the context of a SubDAG) when... 

> Our scheduler is daemonized by upstart and runs with -n 5

Specifically, at the end of each 5th cycle the Scheduler restarts.  When it 
restarts, it appears to not respect the things that are already waiting on the 
Redis queue, and if things look like they haven't been run yet - it will re-run 
them.  I'd be curious, if you eliminated your scheduler restart (which IIRC is 
no longer recommended practice under 1.8.x), whether your multiple enqueuing 
issue would resolve. 

> Using the LatestOnlyOperator results in tasks being reenqueued many times over
> --
>
> Key: AIRFLOW-648
> URL: https://issues.apache.org/jira/browse/AIRFLOW-648
> Project: Apache Airflow
>  Issue Type: Bug
>  Components: operators, scheduler
>Affects Versions: Airflow 1.8
> Environment: Linux 3.2.0-109-virtual #150-Ubuntu SMP x86_64
> Python Version: 2.7.3
> Airflow Version: 1.7.1.3 with plugin to bring in LatestOnlyOperator
> CeleryExecutor with Redis as a backend
> Airflow Config (subset):
> {code}
> [core]
> executor = CeleryExecutor
> parallelism = 32
> dag_concurrency = 16
> dags_are_paused_at_creation = False
> max_active_runs_per_dag = 16
> [scheduler]
> job_heartbeat_sec = 5
> scheduler_heartbeat_sec = 5
> {code}
>Reporter: Kevin Gao
>
> We ported over a number of our cronjobs to run using Airflow. To achieve the 
> desired behavior, we use the {{LatestOnlyOperator}} that was merged to master 
> by pull request 1752: https://github.com/apache/incubator-airflow/pull/1752.
> When we moved over our cronjobs, we migrated many at a time (using ansible). 
> These DAGs had a start date going back a few days.
> The first thing I noticed is that it seemed to take a long time to process 
> the backfilled DAGs. They were being processed correctly, in the sense that 
> the {{'latest_only'}} operator was being completely successfully, and the 
> downstream {{BashOperator}} was marked as skipped. It also appeared that the 
> DAG run completed successfully in the tree view. However, when I searched the 
> DAG runs for {{state contains running}}, I saw that they were present.
> One thing I noticed was that in the logs for one of the "stuck" DAG runs, it 
> appeared that the {{'latest_only'}} task was processed multiple times.
> {code}
> [2016-11-22 12:26:27,701] {models.py:154} INFO - Filling up the DagBag from 
> /DAGDIR/my_dag.py
> [2016-11-22 12:28:50,335] {models.py:154} INFO - Filling up the DagBag from 
> /DAGDIR/my_dag.py
> [2016-11-22 12:28:53,288] {models.py:154} INFO - Filling up the DagBag from 
> /DAGDIR/my_dag.py
> [2016-11-22 12:28:58,400] {models.py:1196} INFO -
> 
> Starting attempt 1 of 1
> 
> [2016-11-22 12:28:59,334] {models.py:1219} INFO - Executing 
>  on 2016-11-20 04:00:00
> [2016-11-22 12:29:00,671] {airflow_next.py:27} INFO - Checking latest only 
> with left_window: 2016-11-21 04:00:00 right_window: 2016-11-22 04:00:00 now: 
> 2016-11-22 12:29:00.670321
> [2016-11-22 12:29:00,671] {airflow_next.py:29} INFO - Not latest execution, 
> skipping downstream.
> [2016-11-22 12:29:00,672] {airflow_next.py:34} INFO - Skipping task: my_dag
> [2016-11-22 12:29:01,397] {airflow_next.py:41} INFO - Done.
> [2016-11-22 12:31:13,055] {models.py:154} INFO - Filling up the DagBag from 
> /DAGDIR/my_dag.py
> [2016-11-22 12:31:17,899] {models.py:1150} INFO - Task  my_dag.latest_only 2016-11-20 04:00:00 [success]> previously succeeded on 
> 2016-11-22 12:29:01.397870
> [2016-11-22 12:32:31,907] {models.py:154} INFO - Filling up the DagBag from 
> /DAGDIR/my_dag.py
> [2016-11-22 12:34:56,522] {models.py:154} INFO - Filling up the DagBag from 
> /DAGDIR/my_dag.py
> [2016-11-22 12:35:00,975] {models.py:1150} INFO - Task  my_dag.latest_only 2016-11-20 04:00:00 [success]> previously succeeded on 
> 2016-11-22 12:29:01.397870
> [2016-11-22 12:35:36,323] {models.py:154} INFO - Filling up the DagBag from 
> /DAGDIR/my_dag.py
> [2016-11-22 12:38:00,140] {models.py:154} INFO - Filling up the DagBag from 
> /DAGDIR/my_dag.py
> [2016-11-22 12:38:05,057] {models.py:1150} INFO - Task  my_dag.latest_only 2016-11-20 04:00:00 [success]> previously succeeded on 
> 2016-11-22 12:29:01.397870
> [2016-11-22 12:38:50,014] {models.py:154} INFO - Filling up the DagBag from 
> /DAGDIR/my_dag.py
> [2016-11-22 

[jira] [Commented] (AIRFLOW-1349) Max active dagrun check for backfills shouldn't include the backfilled dagrun

2017-07-28 Thread ASF subversion and git services (JIRA)

[ 
https://issues.apache.org/jira/browse/AIRFLOW-1349?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16105815#comment-16105815
 ] 

ASF subversion and git services commented on AIRFLOW-1349:
--

Commit 547f8184be1e0b3f902faf354fa39579d2f08af2 in incubator-airflow's branch 
refs/heads/master from [~erod]
[ https://git-wip-us.apache.org/repos/asf?p=incubator-airflow.git;h=547f818 ]

[AIRFLOW-1349] Refactor BackfillJob _execute

BackfillJob._execute is doing multiple things - it is pretty hard to
follow and maintain.

Changes included are just a re-org of the code, no logic has been
changed.

Refactor includes:
- Break BackfillJob._execute into functions
- Add a Status object to track BackfillJob
internal status while
  executing the job.

Closes #2463 from edgarRd/erod-backfill-refactor


> Max active dagrun check for backfills shouldn't include the backfilled dagrun
> -
>
> Key: AIRFLOW-1349
> URL: https://issues.apache.org/jira/browse/AIRFLOW-1349
> Project: Apache Airflow
>  Issue Type: Bug
>  Components: backfill
>Reporter: Dan Davydov
>Assignee: Edgar Rodriguez
>
> When you backfill a dag with e.g. 1 max active dagrun, if that dagrun is 
> already running then it shouldn't count against the max active dagruns of the 
> backfill and make the backfill fail.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


incubator-airflow git commit: [AIRFLOW-1349] Refactor BackfillJob _execute

2017-07-28 Thread davydov
Repository: incubator-airflow
Updated Branches:
  refs/heads/master 322ec9609 -> 547f8184b


[AIRFLOW-1349] Refactor BackfillJob _execute

BackfillJob._execute is doing multiple things - it is pretty hard to
follow and maintain.

Changes included are just a re-org of the code, no logic has been
changed.

Refactor includes:
- Break BackfillJob._execute into functions
- Add a Status object to track BackfillJob
internal status while
  executing the job.

Closes #2463 from edgarRd/erod-backfill-refactor


Project: http://git-wip-us.apache.org/repos/asf/incubator-airflow/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-airflow/commit/547f8184
Tree: http://git-wip-us.apache.org/repos/asf/incubator-airflow/tree/547f8184
Diff: http://git-wip-us.apache.org/repos/asf/incubator-airflow/diff/547f8184

Branch: refs/heads/master
Commit: 547f8184be1e0b3f902faf354fa39579d2f08af2
Parents: 322ec96
Author: Edgar Rodriguez 
Authored: Fri Jul 28 15:49:50 2017 -0700
Committer: Dan Davydov 
Committed: Fri Jul 28 15:50:24 2017 -0700

--
 airflow/jobs.py   | 515 -
 airflow/models.py |  31 +++
 tests/jobs.py | 189 +-
 3 files changed, 501 insertions(+), 234 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/547f8184/airflow/jobs.py
--
diff --git a/airflow/jobs.py b/airflow/jobs.py
index e2f8c94..668973e 100644
--- a/airflow/jobs.py
+++ b/airflow/jobs.py
@@ -1817,6 +1817,62 @@ class BackfillJob(BaseJob):
 'polymorphic_identity': 'BackfillJob'
 }
 
+class _DagRunTaskStatus(object):
+"""
+Internal status of the backfill job. This class is intended to be 
instantiated
+only within a BackfillJob instance and will track the execution of 
tasks,
+e.g. started, skipped, succeeded, failed, etc. Information about the 
dag runs
+related to the backfill job are also being tracked in this structure,
+.e.g finished runs, etc. Any other status related information related 
to the
+execution of dag runs / tasks can be included in this structure since 
it makes
+it easier to pass it around.
+"""
+# TODO(edgarRd): AIRFLOW-1444: Add consistency check on counts
+def __init__(self,
+ to_run=None,
+ started=None,
+ skipped=None,
+ succeeded=None,
+ failed=None,
+ not_ready=None,
+ deadlocked=None,
+ active_runs=None,
+ finished_runs=0,
+ total_runs=0,
+ ):
+"""
+:param to_run: Tasks to run in the backfill
+:type to_run: dict
+:param started: Maps started task instance key to task instance 
object
+:type started: dict
+:param skipped: Tasks that have been skipped
+:type skipped: set
+:param succeeded: Tasks that have succeeded so far
+:type succeeded: set
+:param failed: Tasks that have failed
+:type failed: set
+:param not_ready: Tasks not ready for execution
+:type not_ready: set
+:param deadlocked: Deadlocked tasks
+:type deadlocked: set
+:param active_runs: Active tasks at a certain point in time
+:type active_runs: list
+:param finished_runs: Number of finished runs so far
+:type finished_runs: int
+:param total_runs: Number of total dag runs able to run
+:type total_runs: int
+"""
+self.to_run = to_run or dict()
+self.started = started or dict()
+self.skipped = skipped or set()
+self.succeeded = succeeded or set()
+self.failed = failed or set()
+self.not_ready = not_ready or set()
+self.deadlocked = deadlocked or set()
+self.active_runs = active_runs or list()
+self.finished_runs = finished_runs
+self.total_runs = total_runs
+
 def __init__(
 self,
 dag,
@@ -1841,41 +1897,38 @@ class BackfillJob(BaseJob):
 self.pool = pool
 super(BackfillJob, self).__init__(*args, **kwargs)
 
-def _update_counters(self, started, succeeded, skipped, failed, 
tasks_to_run):
+def _update_counters(self, ti_status):
 """
 Updates the counters per state of the tasks that were running. Can 
re-add
 to tasks to run in case required.
-:param started:
-:param succeeded:
-:param skipped:
-:param failed:
-

[jira] [Created] (AIRFLOW-1474) Add dag_id regex for 'airflow clear' CLI command

2017-07-28 Thread Joy Gao (JIRA)
Joy Gao created AIRFLOW-1474:


 Summary: Add dag_id regex for 'airflow clear' CLI command
 Key: AIRFLOW-1474
 URL: https://issues.apache.org/jira/browse/AIRFLOW-1474
 Project: Apache Airflow
  Issue Type: Improvement
  Components: cli
Reporter: Joy Gao
Assignee: Joy Gao
Priority: Minor


The 'airflow clear' CLI command is currently limited to clearing a single DAG 
per operation. It would be useful to add the capability to clear multiple DAGs 
per operation using regex, similar to how task_id can be filtered via regex.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Commented] (AIRFLOW-1463) Clear state of queued task when it fails due to DAG import error

2017-07-28 Thread George Leslie-Waksman (JIRA)

[ 
https://issues.apache.org/jira/browse/AIRFLOW-1463?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16105327#comment-16105327
 ] 

George Leslie-Waksman commented on AIRFLOW-1463:


This is a scheduler / executor bug.

The crux of the problem is that the celery executor maintains internal state on 
what it has sent to be scheduled. For certain task instance states, as recorded 
in the metadata db, the scheduler will not attempt to reschedule until the 
state has changed. If the worker that picks up the task instance fails before 
it is able to change the task instance state in the metadata db, the task will 
get stuck in a "QUEUED" state.

As a work around, you can restart the scheduler to clear its internal state. In 
1.8.1, the -r option can be used to automatically restart at a regular interval.

In order to fix this issue, the celery executor will need to be modified to: a) 
regularly clear its internal state; b) synchronize its internal state with the 
state of the message queue; or c) use message queue state directly to determine 
what has been queued.

Your proposed fix may ameliorate the issue in your particular case but it 
introduces a number of race conditions around scheduler restarts that break 
some of the existing protections against running the same task multiple times.

It might clarify if you changed the title of this issue to something like, 
"Scheduler does not reschedule tasks in QUEUED state"

> Clear state of queued task when it fails due to DAG import error
> 
>
> Key: AIRFLOW-1463
> URL: https://issues.apache.org/jira/browse/AIRFLOW-1463
> Project: Apache Airflow
>  Issue Type: Improvement
>  Components: cli
> Environment: Ubuntu 14.04
> Airflow 1.8.0
> SQS backed task queue, AWS RDS backed meta storage
> DAG folder is synced by script on code push: archive is downloaded from s3, 
> unpacked, moved, install script is run. airflow executable is replaced with 
> symlink pointing to the latest version of code, no airflow processes are 
> restarted.
>Reporter: Stanislav Pak
>Assignee: Stanislav Pak
>Priority: Minor
>   Original Estimate: 24h
>  Remaining Estimate: 24h
>
> Our pipelines related code is deployed almost simultaneously on all airflow 
> boxes: scheduler+webserver box, workers boxes. Some common python package is 
> deployed on those boxes on every other code push (3-5 deployments per hour). 
> Due to installation specifics, a DAG that imports module from that package 
> might fail. If DAG import fails when worker runs a task, the task is still 
> removed from the queue but task state is not changed, so in this case the 
> task stays in QUEUED state forever.
> Beside the described case, there is scenario when it happens because of DAG 
> update lag in scheduler. A task can be scheduled with old DAG and worker can 
> run the task with new DAG that fails to be imported.
> There might be other scenarios when it happens.
> Proposal:
> Catch errors when importing DAG on task run and clear task instance state if 
> import fails. This should fix transient issues of this kind.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Commented] (AIRFLOW-1420) Dag fails, but none of tasks has failed

2017-07-28 Thread JIRA

[ 
https://issues.apache.org/jira/browse/AIRFLOW-1420?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16105030#comment-16105030
 ] 

Maciej BryƄski commented on AIRFLOW-1420:
-

No.
If you have such a task inside your DAG (not at the end), it's marked as 
skipped and dag can succeed


> Dag fails, but none of tasks has failed
> ---
>
> Key: AIRFLOW-1420
> URL: https://issues.apache.org/jira/browse/AIRFLOW-1420
> Project: Apache Airflow
>  Issue Type: Bug
>Reporter: Maciej Z
> Attachments: screenshot-1.png, screenshot-2.png
>
>
> Here's the code:
> {noformat}
> # -*- coding: utf-8 -*-
> from airflow.operators.dummy_operator import DummyOperator
> from airflow.models import DAG
> from datetime import date, datetime, timedelta
> day_ago = datetime.combine(datetime.today() - timedelta(1), 
> datetime.min.time())
> args = {'owner': 'airflow',  'start_date': day_ago,}
> dag = DAG(dag_id='not_running_dummy2_test', default_args=args, 
> schedule_interval='00 * * * *',  dagrun_timeout=timedelta(hours=12))
> DummyOperator(task_id='a', dag=dag)  >>  DummyOperator(task_id='b', dag=dag, 
> trigger_rule='one_failed')
> if __name__ == "__main__":
> dag.cli()
> {noformat}



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Commented] (AIRFLOW-1420) Dag fails, but none of tasks has failed

2017-07-28 Thread Daniel Surename (JIRA)

[ 
https://issues.apache.org/jira/browse/AIRFLOW-1420?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16104901#comment-16104901
 ] 

Daniel Surename commented on AIRFLOW-1420:
--

Isn't this the expected result? the upstream task can only be scheduled if 
there is one fail which there isn't, so the dag can't complete and fails.

> Dag fails, but none of tasks has failed
> ---
>
> Key: AIRFLOW-1420
> URL: https://issues.apache.org/jira/browse/AIRFLOW-1420
> Project: Apache Airflow
>  Issue Type: Bug
>Reporter: Maciej Z
> Attachments: screenshot-1.png, screenshot-2.png
>
>
> Here's the code:
> {noformat}
> # -*- coding: utf-8 -*-
> from airflow.operators.dummy_operator import DummyOperator
> from airflow.models import DAG
> from datetime import date, datetime, timedelta
> day_ago = datetime.combine(datetime.today() - timedelta(1), 
> datetime.min.time())
> args = {'owner': 'airflow',  'start_date': day_ago,}
> dag = DAG(dag_id='not_running_dummy2_test', default_args=args, 
> schedule_interval='00 * * * *',  dagrun_timeout=timedelta(hours=12))
> DummyOperator(task_id='a', dag=dag)  >>  DummyOperator(task_id='b', dag=dag, 
> trigger_rule='one_failed')
> if __name__ == "__main__":
> dag.cli()
> {noformat}



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Updated] (AIRFLOW-1473) DAG state set failed, no failed task

2017-07-28 Thread Daniel Surename (JIRA)

 [ 
https://issues.apache.org/jira/browse/AIRFLOW-1473?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Daniel Surename updated AIRFLOW-1473:
-
Description: 
*Next Scheduled task which never gets queued or run:*

Task Instance Details:
Task's trigger rule 'all_success' requires all upstream tasks to have 
succeeded, but *found 1 non-success*(es). upstream_tasks_state={'successes': 
54L, 'failed': 0L, 'upstream_failed': 0L, 'skipped': 0L, 'done': 54L}

Dagrun Running:
Task instance's dagrun was not in the 'running' state but in the state 
'failed'.

  was:
*Next Scheduled task which never gets queued or run:*

Task Instance Details:
Task's trigger rule 'all_success' requires all upstream tasks to have 
succeeded, but *found 1 non-success*(es). upstream_tasks_state=*{'successes': 
54L, 'failed': 0L, 'upstream_failed': 0L, 'skipped': 0L, 'done': 54L}*

Dagrun Running:
Task instance's dagrun was not in the 'running' state but in the state 
'failed'.


> DAG state set failed, no failed task
> 
>
> Key: AIRFLOW-1473
> URL: https://issues.apache.org/jira/browse/AIRFLOW-1473
> Project: Apache Airflow
>  Issue Type: Bug
>  Components: scheduler
>Affects Versions: 1.9.0
> Environment: Deployed on Kubernetes. Using Local executor.
>Reporter: Daniel Surename
>Priority: Critical
> Attachments: 1.png, 2.png, 3.png, 4.png
>
>
> *Next Scheduled task which never gets queued or run:*
> Task Instance Details:
> Task's trigger rule 'all_success' requires all upstream tasks to have 
> succeeded, but *found 1 non-success*(es). upstream_tasks_state={'successes': 
> 54L, 'failed': 0L, 'upstream_failed': 0L, 'skipped': 0L, 'done': 54L}
> Dagrun Running:
>   Task instance's dagrun was not in the 'running' state but in the state 
> 'failed'.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Updated] (AIRFLOW-1473) DAG state set failed, no failed task

2017-07-28 Thread Daniel Surename (JIRA)

 [ 
https://issues.apache.org/jira/browse/AIRFLOW-1473?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Daniel Surename updated AIRFLOW-1473:
-
Description: 
*Next Scheduled task which never gets queued or run:*

Task Instance Details:
Task's trigger rule 'all_success' requires all upstream tasks to have 
succeeded, but *found 1 non-success*(es). upstream_tasks_state={'successes': 
54L, * 'failed': 0L *, 'upstream_failed': 0L, 'skipped': 0L, 'done': 54L}

Dagrun Running:
Task instance's dagrun was not in the 'running' state but in the state 
'failed'.

  was:
*Next Scheduled task which never gets queued or run:*

Task Instance Details:
Task's trigger rule 'all_success' requires all upstream tasks to have 
succeeded, but *found 1 non-success*(es). upstream_tasks_state={'successes': 
54L, *'failed': 0L*, 'upstream_failed': 0L, 'skipped': 0L, 'done': 54L}

Dagrun Running:
Task instance's dagrun was not in the 'running' state but in the state 
'failed'.


> DAG state set failed, no failed task
> 
>
> Key: AIRFLOW-1473
> URL: https://issues.apache.org/jira/browse/AIRFLOW-1473
> Project: Apache Airflow
>  Issue Type: Bug
>  Components: scheduler
>Affects Versions: 1.9.0
> Environment: Deployed on Kubernetes. Using Local executor.
>Reporter: Daniel Surename
>Priority: Critical
> Attachments: 1.png, 2.png, 3.png, 4.png
>
>
> *Next Scheduled task which never gets queued or run:*
> Task Instance Details:
> Task's trigger rule 'all_success' requires all upstream tasks to have 
> succeeded, but *found 1 non-success*(es). upstream_tasks_state={'successes': 
> 54L, * 'failed': 0L *, 'upstream_failed': 0L, 'skipped': 0L, 'done': 54L}
> Dagrun Running:
>   Task instance's dagrun was not in the 'running' state but in the state 
> 'failed'.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Updated] (AIRFLOW-1473) DAG state set failed, no failed task

2017-07-28 Thread Daniel Surename (JIRA)

 [ 
https://issues.apache.org/jira/browse/AIRFLOW-1473?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Daniel Surename updated AIRFLOW-1473:
-
Description: 
*Next Scheduled task which never gets queued or run:*

Task Instance Details:
Task's trigger rule 'all_success' requires all upstream tasks to have 
succeeded, but *found 1 non-success*(es). upstream_tasks_state=*{'successes': 
54L, 'failed': 0L, 'upstream_failed': 0L, 'skipped': 0L, 'done': 54L}*

Dagrun Running:
Task instance's dagrun was not in the 'running' state but in the state 
'failed'.

  was:
*Next Scheduled task which never gets queued or run:*

Task Instance Details:
Task's trigger rule 'all_success' requires all upstream tasks to have 
succeeded, but *found 1 non-success*(es). upstream_tasks_state={'successes': 
54L, * 'failed': 0L *, 'upstream_failed': 0L, 'skipped': 0L, 'done': 54L}

Dagrun Running:
Task instance's dagrun was not in the 'running' state but in the state 
'failed'.


> DAG state set failed, no failed task
> 
>
> Key: AIRFLOW-1473
> URL: https://issues.apache.org/jira/browse/AIRFLOW-1473
> Project: Apache Airflow
>  Issue Type: Bug
>  Components: scheduler
>Affects Versions: 1.9.0
> Environment: Deployed on Kubernetes. Using Local executor.
>Reporter: Daniel Surename
>Priority: Critical
> Attachments: 1.png, 2.png, 3.png, 4.png
>
>
> *Next Scheduled task which never gets queued or run:*
> Task Instance Details:
> Task's trigger rule 'all_success' requires all upstream tasks to have 
> succeeded, but *found 1 non-success*(es). upstream_tasks_state=*{'successes': 
> 54L, 'failed': 0L, 'upstream_failed': 0L, 'skipped': 0L, 'done': 54L}*
> Dagrun Running:
>   Task instance's dagrun was not in the 'running' state but in the state 
> 'failed'.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Updated] (AIRFLOW-1473) DAG state set failed, no failed task

2017-07-28 Thread Daniel Surename (JIRA)

 [ 
https://issues.apache.org/jira/browse/AIRFLOW-1473?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Daniel Surename updated AIRFLOW-1473:
-
Description: 
*Next Scheduled task which never gets queued or run:*

Task Instance Details:
Task's trigger rule 'all_success' requires all upstream tasks to have 
succeeded, but *found 1 non-success*(es). upstream_tasks_state={'successes': 
54L, *'failed': 0*L, 'upstream_failed': 0L, 'skipped': 0L, 'done': 54L}

Dagrun Running:
Task instance's dagrun was not in the 'running' state but in the state 
'failed'.

  was:
Task Instance Details:
Task's trigger rule 'all_success' requires all upstream tasks to have 
succeeded, but found 1 non-success(es). upstream_tasks_state={'successes': 54L, 
'failed': 0L, 'upstream_failed': 0L, 'skipped': 0L, 'done': 54L}

Dagrun Running:
Task instance's dagrun was not in the 'running' state but in the state 
'failed'.


> DAG state set failed, no failed task
> 
>
> Key: AIRFLOW-1473
> URL: https://issues.apache.org/jira/browse/AIRFLOW-1473
> Project: Apache Airflow
>  Issue Type: Bug
>  Components: scheduler
>Affects Versions: 1.9.0
> Environment: Deployed on Kubernetes. Using Local executor.
>Reporter: Daniel Surename
>Priority: Critical
> Attachments: 1.png, 2.png, 3.png, 4.png
>
>
> *Next Scheduled task which never gets queued or run:*
> Task Instance Details:
> Task's trigger rule 'all_success' requires all upstream tasks to have 
> succeeded, but *found 1 non-success*(es). upstream_tasks_state={'successes': 
> 54L, *'failed': 0*L, 'upstream_failed': 0L, 'skipped': 0L, 'done': 54L}
> Dagrun Running:
>   Task instance's dagrun was not in the 'running' state but in the state 
> 'failed'.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Updated] (AIRFLOW-1473) DAG state set failed, no failed task

2017-07-28 Thread Daniel Surename (JIRA)

 [ 
https://issues.apache.org/jira/browse/AIRFLOW-1473?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Daniel Surename updated AIRFLOW-1473:
-
Description: 
*Next Scheduled task which never gets queued or run:*

Task Instance Details:
Task's trigger rule 'all_success' requires all upstream tasks to have 
succeeded, but *found 1 non-success*(es). upstream_tasks_state={'successes': 
54L, *'failed': 0L*, 'upstream_failed': 0L, 'skipped': 0L, 'done': 54L}

Dagrun Running:
Task instance's dagrun was not in the 'running' state but in the state 
'failed'.

  was:
*Next Scheduled task which never gets queued or run:*

Task Instance Details:
Task's trigger rule 'all_success' requires all upstream tasks to have 
succeeded, but *found 1 non-success*(es). upstream_tasks_state={'successes': 
54L, *'failed': 0*L, 'upstream_failed': 0L, 'skipped': 0L, 'done': 54L}

Dagrun Running:
Task instance's dagrun was not in the 'running' state but in the state 
'failed'.


> DAG state set failed, no failed task
> 
>
> Key: AIRFLOW-1473
> URL: https://issues.apache.org/jira/browse/AIRFLOW-1473
> Project: Apache Airflow
>  Issue Type: Bug
>  Components: scheduler
>Affects Versions: 1.9.0
> Environment: Deployed on Kubernetes. Using Local executor.
>Reporter: Daniel Surename
>Priority: Critical
> Attachments: 1.png, 2.png, 3.png, 4.png
>
>
> *Next Scheduled task which never gets queued or run:*
> Task Instance Details:
> Task's trigger rule 'all_success' requires all upstream tasks to have 
> succeeded, but *found 1 non-success*(es). upstream_tasks_state={'successes': 
> 54L, *'failed': 0L*, 'upstream_failed': 0L, 'skipped': 0L, 'done': 54L}
> Dagrun Running:
>   Task instance's dagrun was not in the 'running' state but in the state 
> 'failed'.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Created] (AIRFLOW-1473) DAG state set failed, no failed task

2017-07-28 Thread Daniel Surename (JIRA)
Daniel Surename created AIRFLOW-1473:


 Summary: DAG state set failed, no failed task
 Key: AIRFLOW-1473
 URL: https://issues.apache.org/jira/browse/AIRFLOW-1473
 Project: Apache Airflow
  Issue Type: Bug
  Components: scheduler
Affects Versions: 1.9.0
 Environment: Deployed on Kubernetes. Using Local executor.
Reporter: Daniel Surename
Priority: Critical
 Attachments: 1.png, 2.png, 3.png, 4.png

Task Instance Details:
Task's trigger rule 'all_success' requires all upstream tasks to have 
succeeded, but found 1 non-success(es). upstream_tasks_state={'successes': 54L, 
'failed': 0L, 'upstream_failed': 0L, 'skipped': 0L, 'done': 54L}

Dagrun Running:
Task instance's dagrun was not in the 'running' state but in the state 
'failed'.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Issue Comment Deleted] (AIRFLOW-1420) Dag fails, but none of tasks has failed

2017-07-28 Thread Daniel Surename (JIRA)

 [ 
https://issues.apache.org/jira/browse/AIRFLOW-1420?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Daniel Surename updated AIRFLOW-1420:
-
Comment: was deleted

(was: I am also affected by this issue, running version: 1.9.0.dev0+incubating 
(cloned from master).)

> Dag fails, but none of tasks has failed
> ---
>
> Key: AIRFLOW-1420
> URL: https://issues.apache.org/jira/browse/AIRFLOW-1420
> Project: Apache Airflow
>  Issue Type: Bug
>Reporter: Maciej Z
> Attachments: screenshot-1.png, screenshot-2.png
>
>
> Here's the code:
> {noformat}
> # -*- coding: utf-8 -*-
> from airflow.operators.dummy_operator import DummyOperator
> from airflow.models import DAG
> from datetime import date, datetime, timedelta
> day_ago = datetime.combine(datetime.today() - timedelta(1), 
> datetime.min.time())
> args = {'owner': 'airflow',  'start_date': day_ago,}
> dag = DAG(dag_id='not_running_dummy2_test', default_args=args, 
> schedule_interval='00 * * * *',  dagrun_timeout=timedelta(hours=12))
> DummyOperator(task_id='a', dag=dag)  >>  DummyOperator(task_id='b', dag=dag, 
> trigger_rule='one_failed')
> if __name__ == "__main__":
> dag.cli()
> {noformat}



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Commented] (AIRFLOW-1420) Dag fails, but none of tasks has failed

2017-07-28 Thread Daniel Surename (JIRA)

[ 
https://issues.apache.org/jira/browse/AIRFLOW-1420?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16104713#comment-16104713
 ] 

Daniel Surename commented on AIRFLOW-1420:
--

I am also affected by this issue, running version: 1.9.0.dev0+incubating 
(cloned from master).

> Dag fails, but none of tasks has failed
> ---
>
> Key: AIRFLOW-1420
> URL: https://issues.apache.org/jira/browse/AIRFLOW-1420
> Project: Apache Airflow
>  Issue Type: Bug
>Reporter: Maciej Z
> Attachments: screenshot-1.png, screenshot-2.png
>
>
> Here's the code:
> {noformat}
> # -*- coding: utf-8 -*-
> from airflow.operators.dummy_operator import DummyOperator
> from airflow.models import DAG
> from datetime import date, datetime, timedelta
> day_ago = datetime.combine(datetime.today() - timedelta(1), 
> datetime.min.time())
> args = {'owner': 'airflow',  'start_date': day_ago,}
> dag = DAG(dag_id='not_running_dummy2_test', default_args=args, 
> schedule_interval='00 * * * *',  dagrun_timeout=timedelta(hours=12))
> DummyOperator(task_id='a', dag=dag)  >>  DummyOperator(task_id='b', dag=dag, 
> trigger_rule='one_failed')
> if __name__ == "__main__":
> dag.cli()
> {noformat}



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Created] (AIRFLOW-1472) Skipped tasks shouldn't be counted towards SLA misses

2017-07-28 Thread Angela Zhang (JIRA)
Angela Zhang created AIRFLOW-1472:
-

 Summary: Skipped tasks shouldn't be counted towards SLA misses
 Key: AIRFLOW-1472
 URL: https://issues.apache.org/jira/browse/AIRFLOW-1472
 Project: Apache Airflow
  Issue Type: Bug
Affects Versions: Airflow 2.0
Reporter: Angela Zhang
Priority: Minor


If you have a dag with a branch operator (so you skip some tasks when the 
condition is false), and you have an SLA on this task, you will get an SLA miss 
email every night because the skipped tasks aren't executed. 



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)