Hi Zhou, Totally understood, thank you for that. Streaming logic does cover most cases, tho we still have the worst cases where os.walk doesn't give us consistent file and file/dir additions/renames causing a different result order of list_py_file_paths( e.g. right after we parsed the first dir, it was renamed and will be parsed last in the 2nd DAG loading, or we merged a new file right after the file paths are collected). Maybe there's a way to guarantee the order of parsing but not sure if it worth the effort given that it is less of a problem if the end to end parsing time is small enough. I understand it may be started as a short term improvement but since it should not be too much more complicated, we'd rather start with unified long term pattern.
Cheers, Kevin Y On Mon, Jul 29, 2019 at 3:59 PM Zhou Fang <zhouf...@google.com> wrote: > Hi Kevin, > > Yes. DAG persistence in DB is definitely the way to go. I referred to the > aysnc dag loader because it may alleviate your current problem (since it is > code ready). > > It actually reduces the time to 15min, because DAGs are refreshed by the > background process in a streaming way and you don't need to restart > webserver per 20min. > > > > Thanks, > Zhou > > > On Mon, Jul 29, 2019 at 3:14 PM Kevin Yang <yrql...@gmail.com> wrote: > >> Hi Zhou, >> >> Thank you for the pointer. This solves the issue gunicorn restart rate >> throttles webserver refresh rate but not the long DAG parsing time issue, >> right? Worst case scenario we still wait 30 mins for the change to show up, >> comparing to the previous 35 mins( I was wrong on the number, it should be >> 35 mins instead of 55 mins as the clock starts whenever the webserver >> restarts). I believe in the previous discussion, we firstly proposed this >> local webserver DAG parsing optimization to use the same DAG parsing logic >> in scheduler to speed up the parsing. Then the stateless webserver proposal >> came up and we were brought in that it is a better idea to persist DAGs >> into the DB and read directly from the DB, better DAG def consistency and >> webserver cluster consistency. I'm all supportive on the proposed structure >> in AIP-24 but -1 on just feed webserver with a single subprocess parsing >> the DAGs. I would image there won't be too many additional work to fetch >> from DB instead of a subprocess, would there?( haven't look into the >> serialization format part but assuming they are the same/similar) >> >> Cheers, >> Kevin Y >> >> On Mon, Jul 29, 2019 at 2:18 PM Zhou Fang <zhouf...@google.com> wrote: >> >>> Hi Kevin, >>> >>> The problem that DAG parsing takes a long time can be solved by >>> Asynchronous DAG loading: https://github.com/apache/airflow/pull/5594 >>> >>> The idea is the a background process parses DAG files, and sends DAGs to >>> webserver process every [webserver] dagbag_sync_interval = 10s. >>> >>> We have launched it in Composer, so our users can set webserver worker >>> restart interval to 1 hour (or longer). The background DAG parsing >>> processing refresh all DAGs per [webserver] = collect_dags_interval = 30s >>> . >>> >>> If parsing all DAGs take 15min, you can see DAGs being gradually freshed >>> with this feature. >>> >>> Thanks, >>> Zhou >>> >>> >>> On Sat, Jul 27, 2019 at 2:43 AM Kevin Yang <yrql...@gmail.com> wrote: >>> >>>> Nice job Zhou! >>>> >>>> Really excited, exactly what we wanted for the webserver scaling issue. >>>> Want to add another big drive for Airbnb to start think about this >>>> previously to support the effort: it can not only bring consistency >>>> between >>>> webservers but also bring consistency between webserver and >>>> scheduler/workers. It may be less of a problem if total DAG parsing >>>> time is >>>> small, but for us the total DAG parsing time is 15+ mins and we had to >>>> set >>>> the webserver( gunicorn subprocesses) restart interval to 20 mins, which >>>> leads to a worst case 15+20+15=50 mins delay between scheduler start to >>>> schedule things and users can see their deployed DAGs/changes... >>>> >>>> I'm not so sure about the scheduler performance improvement: currently >>>> we >>>> already feed the main scheduler process with SimpleDag through >>>> DagFileProcessorManager running in a subprocess--in the future we feed >>>> it >>>> with data from DB, which is likely slower( tho the diff should have >>>> negligible impact to the scheduler performance). In fact if we'd keep >>>> the >>>> existing behavior, try schedule only fresh parsed DAGs, then we may >>>> need to >>>> deal with some consistency issue--dag processor and the scheduler race >>>> for >>>> updating the flag indicating if the DAG is newly parsed. No big deal >>>> there >>>> but just some thoughts on the top of my head and hopefully can be >>>> helpful. >>>> >>>> And good idea on pre-rendering the template, believe template rendering >>>> was >>>> the biggest concern in the previous discussion. We've also chose the >>>> pre-rendering+JSON approach in our smart sensor API >>>> < >>>> https://cwiki.apache.org/confluence/display/AIRFLOW/AIP-17+Airflow+sensor+optimization >>>> > >>>> and >>>> seems to be working fine--a supporting case for ur proposal ;) There's >>>> a WIP >>>> PR <https://github.com/apache/airflow/pull/5499> for it just in case >>>> you >>>> are interested--maybe we can even share some logics. >>>> >>>> Thumbs-up again for this and please don't heisitate to reach out if you >>>> want to discuss further with us or need any help from us. >>>> >>>> >>>> Cheers, >>>> Kevin Y >>>> >>>> On Sat, Jul 27, 2019 at 12:54 AM Driesprong, Fokko <fo...@driesprong.frl >>>> > >>>> wrote: >>>> >>>> > Looks great Zhou, >>>> > >>>> > I have one thing that pops in my mind while reading the AIP; should >>>> keep >>>> > the caching on the webserver level. As the famous quote goes: *"There >>>> are >>>> > only two hard things in Computer Science: cache invalidation and >>>> naming >>>> > things." -- Phil Karlton* >>>> > >>>> > Right now, the fundamental change that is being proposed in the AIP is >>>> > fetching the DAGs from the database in a serialized format, and not >>>> parsing >>>> > the Python files all the time. This will give already a great >>>> performance >>>> > improvement on the webserver side because it removes a lot of the >>>> > processing. However, since we're still fetching the DAGs from the >>>> database >>>> > in a regular interval, cache it in the local process, so we still >>>> have the >>>> > two issues that Airflow is suffering from right now: >>>> > >>>> > 1. No snappy UI because it is still polling the database in a >>>> regular >>>> > interval. >>>> > 2. Inconsistency between webservers because they might poll in a >>>> > different interval, I think we've all seen this: >>>> > https://www.youtube.com/watch?v=sNrBruPS3r4 >>>> > >>>> > As I also mentioned in the Slack channel, I strongly feel that we >>>> should be >>>> > able to render most views from the tables in the database, so without >>>> > touching the blob. For specific views, we could just pull the blob >>>> from the >>>> > database. In this case we always have the latest version, and we >>>> tackle the >>>> > second point above. >>>> > >>>> > To tackle the first one, I also have an idea. We should change the DAG >>>> > parser from a loop to something that uses inotify >>>> > https://pypi.org/project/inotify_simple/. This will change it from >>>> polling >>>> > to an event-driven design, which is much more performant and less >>>> resource >>>> > hungry. But this would be an AIP on its own. >>>> > >>>> > Again, great design and a comprehensive AIP, but I would include the >>>> > caching on the webserver to greatly improve the user experience in >>>> the UI. >>>> > Looking forward to the opinion of others on this. >>>> > >>>> > Cheers, Fokko >>>> > >>>> > >>>> > >>>> > >>>> > >>>> > >>>> > >>>> > >>>> > Op za 27 jul. 2019 om 01:44 schreef Zhou Fang >>>> <zhouf...@google.com.invalid >>>> > >: >>>> > >>>> > > Hi Kaxi, >>>> > > >>>> > > Just sent out the AIP: >>>> > > >>>> > > >>>> > >>>> https://cwiki.apache.org/confluence/display/AIRFLOW/AIP-24+DAG+Persistence+in+DB+using+JSON+for+Airflow+Webserver+and+%28optional%29+Scheduler >>>> > > >>>> > > Thanks! >>>> > > Zhou >>>> > > >>>> > > >>>> > > On Fri, Jul 26, 2019 at 1:33 PM Zhou Fang <zhouf...@google.com> >>>> wrote: >>>> > > >>>> > > > Hi Kaxil, >>>> > > > >>>> > > > We are also working on persisting DAGs into DB using JSON for >>>> Airflow >>>> > > > webserver in Google Composer. We target at minimizing the change >>>> to the >>>> > > > current Airflow code. Happy to get synced on this! >>>> > > > >>>> > > > Here is our progress: >>>> > > > (1) Serializing DAGs using Pickle to be used in webserver >>>> > > > It has been launched in Composer. I am working on the PR to >>>> upstream >>>> > it: >>>> > > > https://github.com/apache/airflow/pull/5594 >>>> > > > Currently it does not support non-Airflow operators and we are >>>> working >>>> > on >>>> > > > a fix. >>>> > > > >>>> > > > (2) Caching Pickled DAGs in DB to be used by webserver >>>> > > > We have a proof-of-concept implementation, working on an AIP now. >>>> > > > >>>> > > > (3) Using JSON instead of Pickle in (1) and (2) >>>> > > > Decided to use JSON because Pickle is not secure and human >>>> readable. >>>> > The >>>> > > > serialization approach is very similar to (1). >>>> > > > >>>> > > > I will update the RP (https://github.com/apache/airflow/pull/5594) >>>> to >>>> > > > replace Pickle by JSON, and send our design of (2) as an AIP next >>>> week. >>>> > > > Glad to check together whether our implementation makes sense and >>>> do >>>> > > > improvements on that. >>>> > > > >>>> > > > Thanks! >>>> > > > Zhou >>>> > > > >>>> > > > >>>> > > > On Fri, Jul 26, 2019 at 7:37 AM Kaxil Naik <kaxiln...@gmail.com> >>>> > wrote: >>>> > > > >>>> > > >> Hi all, >>>> > > >> >>>> > > >> We, at Astronomer, are going to spend time working on DAG >>>> > Serialisation. >>>> > > >> There are 2 AIPs that are somewhat related to what we plan to >>>> work on: >>>> > > >> >>>> > > >> - AIP-18 Persist all information from DAG file in DB >>>> > > >> < >>>> > > >> >>>> > > >>>> > >>>> https://cwiki.apache.org/confluence/display/AIRFLOW/AIP-18+Persist+all+information+from+DAG+file+in+DB >>>> > > >> > >>>> > > >> - AIP-19 Making the webserver stateless >>>> > > >> < >>>> > > >> >>>> > > >>>> > >>>> https://cwiki.apache.org/confluence/display/AIRFLOW/AIP-19+Making+the+webserver+stateless >>>> > > >> > >>>> > > >> >>>> > > >> We plan to use JSON as the Serialisation format and store it as >>>> a blob >>>> > > in >>>> > > >> metadata DB. >>>> > > >> >>>> > > >> *Goals:* >>>> > > >> >>>> > > >> - Make Webserver Stateless >>>> > > >> - Use the same version of the DAG across Webserver & Scheduler >>>> > > >> - Keep backward compatibility and have a flag (globally & at >>>> DAG >>>> > > level) >>>> > > >> to turn this feature on/off >>>> > > >> - Enable DAG Versioning (extended Goal) >>>> > > >> >>>> > > >> >>>> > > >> We will be preparing a proposal (AIP) after some research and >>>> some >>>> > > initial >>>> > > >> work and open it for the suggestions of the community. >>>> > > >> >>>> > > >> We already had some good brain-storming sessions with Twitter >>>> folks >>>> > > (DanD >>>> > > >> & >>>> > > >> Sumit), folks from GoDataDriven (Fokko & Bas) & Alex (from Uber) >>>> which >>>> > > >> will >>>> > > >> be a good starting point for us. >>>> > > >> >>>> > > >> If anyone in the community is interested in it or has some >>>> experience >>>> > > >> about >>>> > > >> the same and want to collaborate please let me know and join >>>> > > >> #dag-serialisation channel on Airflow Slack. >>>> > > >> >>>> > > >> Regards, >>>> > > >> Kaxil >>>> > > >> >>>> > > > >>>> > > >>>> > >>>> >>>