Can you check 1.8.1? 1.8.0 was a big release after a year so some thing slipped though. Would be good to know if it persists on 1.8.1.
Bolke Sent from my iPhone > On 4 Aug 2017, at 21:02, Marc Weil <[email protected]> wrote: > > This is on version 1.8.0. I don't recall seeing this behavior when I was > running 1.7.x. >> On Fri, Aug 4, 2017 at 2:58 PM Bolke de Bruin <[email protected]> wrote: >> >> What version of airflow? From the top of my mind 1.8.X >> >> 1) we do use db locking >> 2) we check the state after we get the lock >> 3) I don't think the task sets a state if it finds out it is running >> somewhere else >> >> Maybe we do something at the executor/scheduler level. That I need to >> investigate if this issue is on a recent version. >> >> Bolke >> >> Sent from my iPhone >> >>> On 4 Aug 2017, at 19:24, George Leslie-Waksman >>> <[email protected]> >> wrote: >>> >>> Pretty sure (not 100%) what is happening is: >>> >>> 1. Scheduler bugs result in task getting scheduled twice >>> 2. Worker 1 grabs task >>> 3. Worker 2 grabs task >>> 4. Worker 1 starts task >>> 5. Worker 2 starts task >>> 6. Worker 2 sees that Worker 1 has started and plans to abort >>> 7. Worker 1 finishes and marks task as done >>> 8. Worker 2 finishes aborting and marks task as not done >>> >>> >>>> On Fri, Jul 28, 2017 at 3:50 PM Marc Weil <[email protected]> wrote: >>>> >>>> Hey Max, >>>> >>>> Thanks for the suggestions. I believe it was a retry (I'm using remote >>>> logging so I can only check after the task completes), but the UI never >>>> reported it as such. The latest_heartbeat column is in the jobs table, >> and >>>> interestingly I do see some running jobs that haven't heartbeated for >> ~22 >>>> minutes. They are LocalTaskJob instances with CeleryExecutor properly >>>> listed as the executory class. I can't really correlate these to a >> specific >>>> task instance, however, as there doesn't appear to be any key written to >>>> the jobs table (the dag_id column is all null, and there's no task_id >>>> column or anything). >>>> >>>> Any ideas on what could be making these tasks stop heartbeating >> regularly? >>>> That could explain why eventually (after an overnight period of time) >>>> everything is marked as finished in the Airflow UI: eventually these >> tasks >>>> do heartbeat again, but quite long after they are finished running. >>>> >>>> Thanks again! >>>> ᐧ >>>> >>>> -- >>>> Marc Weil | Lead Engineer | Growth Automation, Marketing, and >> Engagement | >>>> New Relic >>>> >>>> On Fri, Jul 28, 2017 at 3:15 PM, Maxime Beauchemin < >>>> [email protected]> wrote: >>>> >>>>> Are you sure there hasn't been a retry at that point? [One of] the >>>> expected >>>>> behavior is the one I described, where if a task finished without >>>> reporting >>>>> it's success [or failure], it will stay marked as RUNNING, but will >> fail >>>> to >>>>> emit a heartbeat (which is a timestamp updated in the task_instance >> table >>>>> as last_heartbeat or something). The scheduler monitors for RUNNING >>>> tasks >>>>> without heartbeat and eventually will handle the failure (send emails, >>>> call >>>>> on_failure_callback, ...). >>>>> >>>>> Looking for heartbeat in the DB might give you some clues as to what is >>>>> going on. >>>>> >>>>> Also there have been versions where we'd occasionally see double >>>>> triggering, and double firing, which can be confusing. Then you can >> have >>>>> different processes reporting their status and debugging those issues >> can >>>>> be problematic. I think there's good prevention against that now, using >>>>> database transactions as the task instance sets itself as RUNNING. I'm >>>> not >>>>> sure if 1.8.0 is 100% clean from that regard. >>>>> >>>>> Max >>>>> >>>>>> On Fri, Jul 28, 2017 at 10:01 AM, Marc Weil <[email protected]> >> wrote: >>>>>> >>>>>> It happens mostly when the scheduler is catching up. More >> specifically, >>>>>> when I load a brand new DAG with a start date in the past. Usually I >>>> have >>>>>> it set to run 5 DAG runs at the same time, and up to 16 tasks at the >>>> same >>>>>> time. >>>>>> >>>>>> What I've also noticed is that the tasks will sit completed in reality >>>>> but >>>>>> uncompleted in the Airflow DB for many hours, but if I just leave them >>>>> all >>>>>> sitting there over night they all tend to be marked complete the next >>>>>> morning. Perhaps this points to some sort of Celery timeout or >>>> connection >>>>>> retry interval? >>>>>> ᐧ >>>>>> >>>>>> -- >>>>>> Marc Weil | Lead Engineer | Growth Automation, Marketing, and >>>> Engagement >>>>> | >>>>>> New Relic >>>>>> >>>>>> On Fri, Jul 28, 2017 at 9:58 AM, Maxime Beauchemin < >>>>>> [email protected]> wrote: >>>>>> >>>>>>> By the time "INFO - Task exited with return code 0" gets logged, the >>>>> task >>>>>>> should have been marked as successful by the subprocess. I have no >>>>>> specific >>>>>>> intuition as to what the issue may be. >>>>>>> >>>>>>> I'm guessing at that point the job stops emitting heartbeat and >>>>>> eventually >>>>>>> the scheduler will handle it as a failure? >>>>>>> >>>>>>> How often does that happen? >>>>>>> >>>>>>> Max >>>>>>> >>>>>>> On Fri, Jul 28, 2017 at 9:43 AM, Marc Weil <[email protected]> >>>> wrote: >>>>>>> >>>>>>>> From what I can tell, it only affects CeleryExecutor. I've never >>>> seen >>>>>>> this >>>>>>>> behavior with LocalExecutor before. >>>>>>>> >>>>>>>> Max, do you know anything about this type of failure mode? >>>>>>>> ᐧ >>>>>>>> >>>>>>>> -- >>>>>>>> Marc Weil | Lead Engineer | Growth Automation, Marketing, and >>>>>> Engagement >>>>>>> | >>>>>>>> New Relic >>>>>>>> >>>>>>>> On Fri, Jul 28, 2017 at 5:48 AM, Jonas Karlsson < >>>> [email protected]> >>>>>>>> wrote: >>>>>>>> >>>>>>>>> We have the exact same problem. In our case, it's a bash operator >>>>>>>> starting >>>>>>>>> a docker container. The container and process it ran exit, but >>>> the >>>>>>>> 'docker >>>>>>>>> run' command is still showing up in the process table, waiting >>>> for >>>>> an >>>>>>>>> event. >>>>>>>>> I'm trying to switch to LocalExecutor to see if that will help. >>>>>>>>> >>>>>>>>> _jonas >>>>>>>>> >>>>>>>>> >>>>>>>>> On Thu, Jul 27, 2017 at 4:28 PM Marc Weil <[email protected]> >>>>>> wrote: >>>>>>>>> >>>>>>>>>> Hello, >>>>>>>>>> >>>>>>>>>> Has anyone seen the behavior when using CeleryExecutor where >>>>>> workers >>>>>>>> will >>>>>>>>>> finish their tasks ("INFO - Task exited with return code 0" >>>> shows >>>>>> in >>>>>>>> the >>>>>>>>>> logs) but are never marked as complete in the airflow DB or UI? >>>>>>>>> Effectively >>>>>>>>>> this causes tasks to hang even though they are complete, and >>>> the >>>>>> DAG >>>>>>>> will >>>>>>>>>> not continue. >>>>>>>>>> >>>>>>>>>> This is happening on 1.8.0. Anyone else seen this or perhaps >>>>> have a >>>>>>>>>> workaround? >>>>>>>>>> >>>>>>>>>> Thanks! >>>>>>>>>> >>>>>>>>>> -- >>>>>>>>>> Marc Weil | Lead Engineer | Growth Automation, Marketing, and >>>>>>>> Engagement >>>>>>>>> | >>>>>>>>>> New Relic >>>>>>>>>> ᐧ >>>>>>>>>> >>>>>>>>> >>>>>>>> >>>>>>> >>>>>> >>>>> >>>> >> > -- > Marc Weil | Lead Engineer | Growth Automation, Marketing, and Engagement | > New Relic
